diff --git a/dbms/src/CMakeLists.txt b/dbms/src/CMakeLists.txt
index 058c15d7cc5..9c8fe816ba0 100644
--- a/dbms/src/CMakeLists.txt
+++ b/dbms/src/CMakeLists.txt
@@ -23,7 +23,6 @@ add_subdirectory (Storages)
 add_subdirectory (Parsers)
 add_subdirectory (IO)
 add_subdirectory (Functions)
-add_subdirectory (Interpreters)
 add_subdirectory (AggregateFunctions)
 add_subdirectory (Server)
 add_subdirectory (Client)
diff --git a/dbms/src/Common/CPUAffinityManager.cpp b/dbms/src/Common/CPUAffinityManager.cpp
index 44676903321..ca326e23fd7 100644
--- a/dbms/src/Common/CPUAffinityManager.cpp
+++ b/dbms/src/Common/CPUAffinityManager.cpp
@@ -24,9 +24,9 @@
 
 #include <Common/CPUAffinityManager.h>
 #include <Common/Exception.h>
+#include <Common/Logger.h>
 #include <Common/setThreadName.h>
 #include <Poco/DirectoryIterator.h>
-#include <Poco/Logger.h>
 #include <Poco/Util/LayeredConfiguration.h>
 #include <boost_wrapper/string.h>
 #include <common/logger_useful.h>
@@ -75,7 +75,7 @@ CPUAffinityManager & CPUAffinityManager::getInstance()
 CPUAffinityManager::CPUAffinityManager()
     : query_cpu_percent(0)
     , cpu_cores(0)
-    , log(&Poco::Logger::get("CPUAffinityManager"))
+    , log(Logger::get())
 {}
 
 #ifdef __linux__
diff --git a/dbms/src/Common/CPUAffinityManager.h b/dbms/src/Common/CPUAffinityManager.h
index 239604f6213..8cac799a90c 100644
--- a/dbms/src/Common/CPUAffinityManager.h
+++ b/dbms/src/Common/CPUAffinityManager.h
@@ -24,7 +24,6 @@
 
 namespace Poco
 {
-class Logger;
 namespace Util
 {
 class LayeredConfiguration;
@@ -33,10 +32,13 @@ class LayeredConfiguration;
 
 namespace DB
 {
+class Logger;
+using LoggerPtr = std::shared_ptr<Logger>;
 namespace tests
 {
 class CPUAffinityManagerTest_CPUAffinityManager_Test;
-}
+} // namespace tests
+
 struct CPUAffinityConfig
 {
     CPUAffinityConfig()
@@ -132,7 +134,7 @@ class CPUAffinityManager
     MAYBE_UNUSED_MEMBER int cpu_cores;
 
     std::vector<std::string> query_threads;
-    Poco::Logger * log;
+    LoggerPtr log;
 
     CPUAffinityManager();
     // Disable copy and move
diff --git a/dbms/src/Common/Config/ConfigProcessor.h b/dbms/src/Common/Config/ConfigProcessor.h
index db5e4fca9bb..7618f58e915 100644
--- a/dbms/src/Common/Config/ConfigProcessor.h
+++ b/dbms/src/Common/Config/ConfigProcessor.h
@@ -18,6 +18,7 @@
 #include <Poco/ConsoleChannel.h>
 #include <Poco/DirectoryIterator.h>
 #include <Poco/File.h>
+#include <Poco/Logger.h>
 #include <Poco/Path.h>
 #include <Poco/Util/AbstractConfiguration.h>
 #include <common/logger_useful.h>
diff --git a/dbms/src/Common/FileChecker.h b/dbms/src/Common/FileChecker.h
index c0f606a3a54..15c0ccccb25 100644
--- a/dbms/src/Common/FileChecker.h
+++ b/dbms/src/Common/FileChecker.h
@@ -15,8 +15,10 @@
 #pragma once
 
 #include <Poco/File.h>
+#include <Poco/Logger.h>
 #include <common/logger_useful.h>
 
+#include <map>
 #include <string>
 
 
diff --git a/dbms/src/Common/LRUCache.h b/dbms/src/Common/LRUCache.h
index d2a817d2f51..cea64994c24 100644
--- a/dbms/src/Common/LRUCache.h
+++ b/dbms/src/Common/LRUCache.h
@@ -315,7 +315,7 @@ class LRUCache
             {
                 // If queue.insert() throws exception, cells and queue will be in inconsistent.
                 cells.erase(it);
-                tryLogCurrentException(Logger::get("LRUCache"), "queue.insert throw exception");
+                tryLogCurrentException(Logger::get(), "queue.insert throw exception");
                 throw;
             }
         }
diff --git a/dbms/src/Common/MyTime.cpp b/dbms/src/Common/MyTime.cpp
index be3bcc19c4a..e01d27b6581 100644
--- a/dbms/src/Common/MyTime.cpp
+++ b/dbms/src/Common/MyTime.cpp
@@ -2647,7 +2647,7 @@ std::optional<UInt64> MyDateTimeParser::parseAsPackedUInt(const StringRef & str_
         if (!f(ctx, my_time))
         {
 #ifndef NDEBUG
-            LOG_TRACE(&Poco::Logger::get("MyDateTimeParser"),
+            LOG_TRACE(Logger::get(),
                       "parse error, [str={}] [format={}] [parse_pos={}]",
                       ctx.view.toString(),
                       format,
diff --git a/dbms/src/Common/getNumberOfCPUCores.cpp b/dbms/src/Common/getNumberOfCPUCores.cpp
index fd615d6a471..2d955368c63 100644
--- a/dbms/src/Common/getNumberOfCPUCores.cpp
+++ b/dbms/src/Common/getNumberOfCPUCores.cpp
@@ -62,9 +62,8 @@ void computeAndSetNumberOfPhysicalCPUCores(UInt16 number_of_logical_cpu_cores_,
     auto hardware_logical_cpu_cores = std::thread::hardware_concurrency();
     UInt16 physical_cpu_cores = number_of_logical_cpu_cores_ / (hardware_logical_cpu_cores / number_of_hardware_physical_cores);
     CPUCores::number_of_physical_cpu_cores = physical_cpu_cores > 0 ? physical_cpu_cores : 1;
-    auto log = DB::Logger::get("CPUCores");
     LOG_INFO(
-        log,
+        DB::Logger::get(),
         "logical cpu cores: {}, hardware logical cpu cores: {}, hardware physical cpu cores: {}, physical cpu cores: {}, number_of_physical_cpu_cores: {}",
         number_of_logical_cpu_cores_,
         hardware_logical_cpu_cores,
diff --git a/dbms/src/Flash/Mpp/MPPHandler.h b/dbms/src/Flash/Mpp/MPPHandler.h
index 8041d02d444..79e548546f6 100644
--- a/dbms/src/Flash/Mpp/MPPHandler.h
+++ b/dbms/src/Flash/Mpp/MPPHandler.h
@@ -24,12 +24,12 @@ class MPPHandler
 {
     const mpp::DispatchTaskRequest & task_request;
 
-    Poco::Logger * log;
+    LoggerPtr log;
 
 public:
-    MPPHandler(const mpp::DispatchTaskRequest & task_request_)
+    explicit MPPHandler(const mpp::DispatchTaskRequest & task_request_)
         : task_request(task_request_)
-        , log(&Poco::Logger::get("MPPHandler"))
+        , log(Logger::get())
     {}
     grpc::Status execute(const ContextPtr & context, mpp::DispatchTaskResponse * response);
     void handleError(const MPPTaskPtr & task, String error);
diff --git a/dbms/src/Interpreters/CMakeLists.txt b/dbms/src/Interpreters/CMakeLists.txt
deleted file mode 100644
index f6f7fb6c660..00000000000
--- a/dbms/src/Interpreters/CMakeLists.txt
+++ /dev/null
@@ -1,25 +0,0 @@
-# Copyright 2022 PingCAP, Ltd.
-#
-# Licensed 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.
-
-
-list(GET Poco_INCLUDE_DIRS 0 Poco_Foundation_INCLUDE_DIR)
-list(GET Poco_INCLUDE_DIRS 1 Poco_Util_INCLUDE_DIR)
-
-if (NOT DOUBLE_CONVERSION_INCLUDE_DIR)
-    get_target_property(DOUBLE_CONVERSION_INCLUDE_DIR ${DOUBLE_CONVERSION_LIBRARIES} INTERFACE_INCLUDE_DIRECTORIES)
-endif ()
-
-if (ENABLE_TESTS)
-    add_subdirectory (tests EXCLUDE_FROM_ALL)
-endif ()
diff --git a/dbms/src/Interpreters/IDAsPathUpgrader.cpp b/dbms/src/Interpreters/IDAsPathUpgrader.cpp
deleted file mode 100644
index e253dfdb66d..00000000000
--- a/dbms/src/Interpreters/IDAsPathUpgrader.cpp
+++ /dev/null
@@ -1,929 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#include <Common/StringUtils/StringUtils.h>
-#include <Common/escapeForFileName.h>
-#include <Common/typeid_cast.h>
-#include <Databases/DatabaseOrdinary.h>
-#include <Databases/DatabasesCommon.h>
-#include <Debug/MockSchemaNameMapper.h>
-#include <Encryption/ReadBufferFromFileProvider.h>
-#include <Encryption/WriteBufferFromFileProvider.h>
-#include <Interpreters/Context.h>
-#include <Interpreters/IDAsPathUpgrader.h>
-#include <Parsers/ASTCreateQuery.h>
-#include <Parsers/ASTExpressionList.h>
-#include <Parsers/ASTFunction.h>
-#include <Parsers/ASTLiteral.h>
-#include <Parsers/ParserCreateQuery.h>
-#include <Parsers/parseQuery.h>
-#include <Poco/DirectoryIterator.h>
-#include <Poco/File.h>
-#include <Poco/FileStream.h>
-#include <Storages/MutableSupport.h>
-#include <Storages/PathPool.h>
-#include <Storages/Transaction/TMTContext.h>
-#include <Storages/Transaction/TiDB.h>
-#include <TiDB/Schema/SchemaNameMapper.h>
-#include <TiDB/Schema/TiDBSchemaSyncer.h>
-#include <common/logger_useful.h>
-#include <fmt/core.h>
-
-namespace DB
-{
-namespace ErrorCodes
-{
-extern const int BAD_ARGUMENTS;
-extern const int LOGICAL_ERROR;
-extern const int FILE_DOESNT_EXIST;
-extern const int SYNTAX_ERROR;
-} // namespace ErrorCodes
-
-static constexpr auto SYSTEM_DATABASE = "system";
-
-namespace
-{
-std::shared_ptr<ASTFunction> getDatabaseEngine(const FileProviderPtr & file_provider, const String & filename)
-{
-    String query;
-    if (Poco::File(filename).exists())
-    {
-        ReadBufferFromFileProvider in(file_provider, filename, EncryptionPath(filename, ""), 1024);
-        readStringUntilEOF(query, in);
-    }
-    else
-    {
-        // only directory exists, "default" database, return "Ordinary" engine by default.
-        return std::static_pointer_cast<ASTFunction>(makeASTFunction("Ordinary"));
-    }
-
-    ParserCreateQuery parser;
-    ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "in file " + filename, 0);
-    ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
-    auto * storage = ast_create_query.storage;
-    if (storage == nullptr || storage->engine == nullptr || storage->engine->name.empty())
-    {
-        throw Exception("Can not get database engine for file: " + filename, ErrorCodes::LOGICAL_ERROR);
-    }
-
-    return std::static_pointer_cast<ASTFunction>(storage->engine->clone());
-}
-
-// Get <TableName, TableInfo> from `table_metadata_file`
-std::pair<String, TiDB::TableInfo> getTableInfo(const FileProviderPtr & file_provider, const String & table_metadata_file)
-{
-    String definition;
-    if (Poco::File(table_metadata_file).exists())
-    {
-        ReadBufferFromFileProvider in(file_provider, table_metadata_file, EncryptionPath(table_metadata_file, ""), 1024);
-        readStringUntilEOF(definition, in);
-    }
-    else
-    {
-        throw Exception("Can not open table schema file: " + table_metadata_file, ErrorCodes::LOGICAL_ERROR);
-    }
-
-    ParserCreateQuery parser;
-    ASTPtr ast = parseQuery(parser, definition.data(), definition.data() + definition.size(), "in file " + table_metadata_file, 0);
-    ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
-    auto * storage = ast_create_query.storage;
-    if (storage == nullptr || storage->engine == nullptr || storage->engine->name.empty())
-    {
-        throw Exception("Can not get table engine for file: " + table_metadata_file, ErrorCodes::LOGICAL_ERROR);
-    }
-
-    TiDB::TableInfo info;
-    ASTFunction * engine = storage->engine;
-    const auto * args = typeid_cast<const ASTExpressionList *>(engine->arguments.get());
-    if (args == nullptr)
-        throw Exception("Can not cast table engine arguments", ErrorCodes::BAD_ARGUMENTS);
-
-    const ASTLiteral * table_info_ast = nullptr;
-    if (engine->name == MutableSupport::delta_tree_storage_name)
-    {
-        if (args->children.size() >= 2)
-        {
-            table_info_ast = typeid_cast<const ASTLiteral *>(args->children[1].get());
-        }
-    }
-    else if (engine->name == MutableSupport::txn_storage_name)
-    {
-        if (args->children.size() >= 3)
-        {
-            table_info_ast = typeid_cast<const ASTLiteral *>(args->children[2].get());
-        }
-    }
-    else
-    {
-        throw Exception("Unknown storage engine: " + engine->name, ErrorCodes::LOGICAL_ERROR);
-    }
-
-    if (table_info_ast && table_info_ast->value.getType() == Field::Types::String)
-    {
-        const auto table_info_json = safeGet<String>(table_info_ast->value);
-        if (!table_info_json.empty())
-        {
-            info.deserialize(table_info_json);
-            return {ast_create_query.table, info};
-        }
-    }
-
-    throw Exception("Can not get TableInfo for file: " + table_metadata_file, ErrorCodes::BAD_ARGUMENTS);
-}
-
-void renamePath(const String & old_path, const String & new_path, Poco::Logger * log, bool must_success)
-{
-    if (auto file = Poco::File{old_path}; file.exists())
-    {
-        file.renameTo(new_path);
-    }
-    else
-    {
-        std::string err_msg = fmt::format(R"(Path "{}" is missing.)", old_path);
-        if (must_success)
-            throw Exception(err_msg);
-        else
-            LOG_WARNING(log, err_msg);
-    }
-}
-
-void writeTableDefinitionToFile(
-    const FileProviderPtr & file_provider,
-    const String & table_meta_path,
-    const ASTPtr & query,
-    bool fsync_metadata)
-{
-    String table_meta_tmp_path = table_meta_path + ".tmp";
-    {
-        String statement = getTableDefinitionFromCreateQuery(query);
-
-        /// Exclusive flags guarantees, that table is not created right now in another thread. Otherwise, exception will be thrown.
-        WriteBufferFromFileProvider out(file_provider, table_meta_tmp_path, EncryptionPath(table_meta_tmp_path, ""), true, nullptr, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
-        writeString(statement, out);
-        out.next();
-        if (fsync_metadata)
-            out.sync();
-        out.close();
-    }
-    file_provider->renameFile(
-        table_meta_tmp_path,
-        EncryptionPath(table_meta_tmp_path, ""),
-        table_meta_path,
-        EncryptionPath(table_meta_path, ""),
-        true);
-}
-
-void writeDatabaseDefinitionToFile(
-    const FileProviderPtr & file_provider,
-    const String & database_meta_path,
-    const ASTPtr & query,
-    bool fsync_metadata)
-{
-    String db_meta_tmp_path = database_meta_path + ".tmp";
-    {
-        String statement = getDatabaseDefinitionFromCreateQuery(query);
-
-        /// Exclusive flags guarantees, that table is not created right now in another thread. Otherwise, exception will be thrown.
-        WriteBufferFromFileProvider out(file_provider, db_meta_tmp_path, EncryptionPath(db_meta_tmp_path, ""), true, nullptr, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
-        writeString(statement, out);
-        out.next();
-        if (fsync_metadata)
-            out.sync();
-        out.close();
-    }
-    file_provider->renameFile(
-        db_meta_tmp_path,
-        EncryptionPath(db_meta_tmp_path, ""),
-        database_meta_path,
-        EncryptionPath(database_meta_path, ""),
-        true);
-}
-
-ASTPtr parseCreateDatabaseAST(const String & statement)
-{
-    ParserCreateQuery parser;
-    const char * pos = statement.data();
-    std::string error_msg;
-    auto ast = tryParseQuery(parser,
-                             pos,
-                             pos + statement.size(),
-                             error_msg,
-                             /*hilite=*/false,
-                             String("in ") + __PRETTY_FUNCTION__,
-                             /*allow_multi_statements=*/false,
-                             0);
-    if (!ast)
-        throw Exception(error_msg, ErrorCodes::SYNTAX_ERROR);
-    return ast;
-}
-
-// By default, only remove directory if it is empy
-void tryRemoveDirectory(const String & directory, Poco::Logger * log, bool recursive = false)
-{
-    if (auto dir = Poco::File(directory); dir.exists() && dir.isDirectory())
-    {
-        try
-        {
-            dir.remove(/*recursive=*/recursive);
-        }
-        catch (Poco::DirectoryNotEmptyException &)
-        {
-            // just ignore and keep that directory if it is not empty
-            LOG_WARNING(log, "Can not remove directory: {}, it is not empty", directory);
-        }
-    }
-}
-
-// This function will tidy up path and compare if them are the same one.
-// For example "/tmp/data/a.sql" is equal to "/tmp//data//a.sql"
-inline bool isSamePath(const String & lhs, const String & rhs)
-{
-    return Poco::Path{lhs}.absolute().toString() == Poco::Path{rhs}.absolute().toString();
-}
-
-} // namespace
-
-
-// ================================================
-//   TableDiskInfo
-// ================================================
-
-String IDAsPathUpgrader::TableDiskInfo::name() const
-{
-    // Name in table_info may not be updated, use the name in `ATTACH TABLE <name> ...`.
-    // The name in table_info will be updated in later schema sync.
-    return old_name;
-}
-String IDAsPathUpgrader::TableDiskInfo::newName() const
-{
-    return mapper->mapTableName(*tidb_table_info);
-}
-const TiDB::TableInfo & IDAsPathUpgrader::TableDiskInfo::getInfo() const
-{
-    return *tidb_table_info;
-}
-
-// "metadata/${db_name}/${tbl_name}.sql"
-String IDAsPathUpgrader::TableDiskInfo::getMetaFilePath(const String & root_path, const DatabaseDiskInfo & db) const
-{
-    return db.getMetaDirectory(root_path) + escapeForFileName(name()) + ".sql";
-}
-// "data/${db_name}/${tbl_name}/"
-String IDAsPathUpgrader::TableDiskInfo::getDataDirectory(
-    const String & root_path,
-    const DatabaseDiskInfo & db,
-    bool escape_db,
-    bool escape_tbl) const
-{
-    String res = db.getDataDirectory(root_path, escape_db);
-    if (escape_tbl)
-        res += escapeForFileName(name());
-    else
-        res += name();
-    return res + "/";
-}
-// "extra_data/${db_name}/${tbl_name}/"
-String IDAsPathUpgrader::TableDiskInfo::getExtraDirectory(
-    const String & root_path,
-    const DatabaseDiskInfo & db,
-    bool escape_db,
-    bool escape_tbl) const
-{
-    String res = db.getExtraDirectory(root_path, escape_db);
-    if (escape_tbl)
-        res += escapeForFileName(name());
-    else
-        res += name();
-    return res + "/";
-}
-
-// "metadata/db_${db_id}/t_${id}.sql"
-String IDAsPathUpgrader::TableDiskInfo::getNewMetaFilePath(const String & root_path, const DatabaseDiskInfo & db) const
-{
-    return db.getNewMetaDirectory(root_path) + escapeForFileName(newName()) + ".sql";
-}
-// "data/t_${id}/"
-String IDAsPathUpgrader::TableDiskInfo::getNewDataDirectory(const String & root_path, const DatabaseDiskInfo & db) const
-{
-    return db.getNewDataDirectory(root_path) + escapeForFileName(newName()) + "/";
-}
-// "extra_data/t_${id}"
-String IDAsPathUpgrader::TableDiskInfo::getNewExtraDirectory(const String & root_path, const DatabaseDiskInfo & db) const
-{
-    return db.getNewExtraDirectory(root_path) + escapeForFileName(newName()) + "/";
-}
-
-// ================================================
-//   DatabaseDiskInfo
-// ================================================
-
-void IDAsPathUpgrader::DatabaseDiskInfo::setDBInfo(TiDB::DBInfoPtr info_)
-{
-    tidb_db_info = info_;
-}
-
-const TiDB::DBInfo & IDAsPathUpgrader::DatabaseDiskInfo::getInfo() const
-{
-    if (!hasValidTiDBInfo())
-        throw Exception("Try to get database info of not inited database: " + name);
-    return *tidb_db_info;
-}
-
-String IDAsPathUpgrader::DatabaseDiskInfo::newName() const
-{
-    return mapper->mapDatabaseName(getInfo());
-}
-
-String IDAsPathUpgrader::DatabaseDiskInfo::getTiDBSerializeInfo() const
-{
-    if (!hasValidTiDBInfo())
-        throw Exception("Try to serialize database info of not inited database: " + name);
-    return tidb_db_info->serialize();
-}
-
-// "metadata/${db_name}.sql"
-String IDAsPathUpgrader::DatabaseDiskInfo::doGetMetaFilePath(const String & root_path, bool tmp) const
-{
-    String meta_dir = doGetMetaDirectory(root_path, tmp);
-    return (endsWith(meta_dir, "/") ? meta_dir.substr(0, meta_dir.size() - 1) : meta_dir) + ".sql";
-}
-// "metadata/${db_name}/"
-String IDAsPathUpgrader::DatabaseDiskInfo::doGetMetaDirectory(const String & root_path, bool tmp) const
-{
-    return root_path + (endsWith(root_path, "/") ? "" : "/") + "metadata/" + escapeForFileName(name + (tmp ? TMP_SUFFIX : "")) + "/";
-}
-// "data/${db_name}/"
-String IDAsPathUpgrader::DatabaseDiskInfo::doGetDataDirectory(const String & root_path, bool escape, bool tmp) const
-{
-    // Old data path don't do escape for path
-    if (escape)
-        return root_path + (endsWith(root_path, "/") ? "" : "/") + "data/" + escapeForFileName(name + (tmp ? TMP_SUFFIX : "")) + "/";
-    else
-    {
-        // Old extra data path (in PathPool) don't escape for path.
-        return root_path + (endsWith(root_path, "/") ? "" : "/") + "data/" + name + (tmp ? TMP_SUFFIX : "") + "/";
-    }
-}
-// "extra_data/${db_name}/"
-String IDAsPathUpgrader::DatabaseDiskInfo::doGetExtraDirectory(const String & extra_root, bool escape, bool tmp) const
-{
-    if (escape)
-        return extra_root + (endsWith(extra_root, "/") ? "" : "/") + escapeForFileName(name + (tmp ? TMP_SUFFIX : "")) + "/";
-    else
-    {
-        // Old extra data path (in PathPool) don't escape for path.
-        return extra_root + (endsWith(extra_root, "/") ? "" : "/") + name + (tmp ? TMP_SUFFIX : "") + "/";
-    }
-}
-
-// "metadata/db_${id}.sql"
-String IDAsPathUpgrader::DatabaseDiskInfo::getNewMetaFilePath(const String & root_path) const
-{
-    String meta_dir = getNewMetaDirectory(root_path);
-    return (endsWith(meta_dir, "/") ? meta_dir.substr(0, meta_dir.size() - 1) : meta_dir) + ".sql";
-}
-// "metadata/db_${id}/"
-String IDAsPathUpgrader::DatabaseDiskInfo::getNewMetaDirectory(const String & root_path) const
-{
-    return root_path + (endsWith(root_path, "/") ? "" : "/") + "/metadata/" + escapeForFileName(newName()) + "/";
-}
-// "data/"
-String IDAsPathUpgrader::DatabaseDiskInfo::getNewDataDirectory(const String & root_path)
-{
-    return root_path + "/data/";
-}
-// "extra_data/"
-String IDAsPathUpgrader::DatabaseDiskInfo::getNewExtraDirectory(const String & extra_root)
-{
-    return extra_root + "/";
-}
-
-
-void IDAsPathUpgrader::DatabaseDiskInfo::renameToTmpDirectories(const Context & ctx, Poco::Logger * log)
-{
-    if (moved_to_tmp)
-        return;
-
-    auto root_path = ctx.getPath();
-    // Rename database meta file if exist
-    renamePath(doGetMetaFilePath(root_path, false), doGetMetaFilePath(root_path, true), log, false);
-    // Rename database meta dir
-    renamePath(doGetMetaDirectory(root_path, false), doGetMetaDirectory(root_path, true), log, true);
-
-    // Rename database data dir
-    renamePath( //
-        doGetDataDirectory(root_path, /*escape*/ true, /*tmp*/ false),
-        doGetDataDirectory(root_path, /*escape*/ true, /*tmp*/ true),
-        log,
-        true);
-
-    // Rename database data dir for multi-paths
-    auto root_pool = ctx.getPathPool();
-    for (const auto & extra_path : root_pool.listPaths())
-        renamePath( //
-            doGetExtraDirectory(extra_path, /*escape*/ true, /*tmp*/ false), //
-            doGetExtraDirectory(extra_path, /*escape*/ true, /*tmp*/ true),
-            log,
-            false);
-
-    moved_to_tmp = true;
-}
-
-
-// ================================================
-//   IDAsPathUpgrader
-// ================================================
-
-IDAsPathUpgrader::IDAsPathUpgrader(Context & global_ctx_, bool is_mock_, std::unordered_set<std::string> reserved_databases_)
-    : global_context(global_ctx_)
-    , root_path{global_context.getPath()}
-    , is_mock(is_mock_)
-    , mapper(is_mock ? std::make_shared<MockSchemaNameMapper>() //
-                     : std::make_shared<SchemaNameMapper>())
-    , reserved_databases{std::move(reserved_databases_)}
-    , log{&Poco::Logger::get("IDAsPathUpgrader")}
-{}
-
-bool IDAsPathUpgrader::needUpgrade()
-{
-    const auto metadata_path = global_context.getPath() + "/metadata";
-
-    // For old version, we have database directories and its `.sql` file
-    Poco::DirectoryIterator dir_end;
-    for (Poco::DirectoryIterator it(metadata_path); it != dir_end; ++it)
-    {
-        if (!it->isDirectory())
-            continue;
-
-        /// For '.svn', '.gitignore' directory and similar.
-        if (it.name().at(0) == '.')
-            continue;
-
-        if (it.name() == SYSTEM_DATABASE)
-            continue;
-
-        String db_name = unescapeForFileName(it.name());
-        databases.emplace(db_name, DatabaseDiskInfo{db_name, mapper});
-    }
-
-    bool has_old_db_engine = false;
-    for (auto && [db_name, db_info] : databases)
-    {
-        (void)db_name;
-        const String database_metadata_file = db_info.getMetaFilePath(root_path);
-        auto engine = getDatabaseEngine(global_context.getFileProvider(), database_metadata_file);
-        db_info.engine = engine->name;
-        if (db_info.engine != "TiFlash")
-        {
-            has_old_db_engine = true;
-            LOG_INFO(log, "Find old style of database engine, doing upgrade [path={}] [engine={}]", database_metadata_file, db_info.engine);
-        }
-    }
-
-    return has_old_db_engine;
-}
-
-std::vector<TiDB::DBInfoPtr> IDAsPathUpgrader::fetchInfosFromTiDB() const
-{
-    // Fetch DBs info from TiDB/TiKV
-    // Note: Not get table info from TiDB, just rename according to TableID in persisted TableInfo
-    for (size_t i = 0; i < 60; i++) // retry for 3 mins
-    {
-        try
-        {
-            auto schema_syncer = global_context.getTMTContext().getSchemaSyncer();
-            return schema_syncer->fetchAllDBs();
-        }
-        catch (Poco::Exception & e)
-        {
-            const int wait_seconds = 3;
-            LOG_ERROR(
-                log,
-                "Upgrade failed because fetch schema error: {}\nWe will sleep for {} seconds and try again.",
-                e.displayText(),
-                wait_seconds);
-            ::sleep(wait_seconds);
-        }
-    }
-    throw Exception("Upgrade failed because fetch schema error.");
-}
-
-static void dropAbsentDatabase(
-    Context & context,
-    const String & db_name,
-    const IDAsPathUpgrader::DatabaseDiskInfo & db_info,
-    Poco::Logger * log)
-{
-    if (db_info.hasValidTiDBInfo())
-        throw Exception("Invalid call for dropAbsentDatabase for database " + db_name + " with info: " + db_info.getTiDBSerializeInfo());
-
-    /// tryRemoveDirectory with recursive=true to clean up
-
-    const auto root_path = context.getPath();
-    // Remove old metadata dir
-    const String old_meta_dir = db_info.getMetaDirectory(root_path);
-    tryRemoveDirectory(old_meta_dir, log, true);
-    // Remove old metadata file
-    const String old_meta_file = db_info.getMetaFilePath(root_path);
-    if (auto file = Poco::File(old_meta_file); file.exists())
-        file.remove();
-    else
-        LOG_WARNING(log, "Can not remove database meta file: {}", old_meta_file);
-    // Remove old data dir
-    const String old_data_dir = db_info.getDataDirectory(root_path);
-    tryRemoveDirectory(old_data_dir, log, true);
-    // not escaped dir created by old PathPool
-    const String old_data_dir_not_escaped = db_info.getDataDirectory(root_path, false);
-    tryRemoveDirectory(old_data_dir_not_escaped, log, true);
-
-    const auto & data_extra_paths = context.getPathPool();
-    for (const auto & extra_root_path : data_extra_paths.listPaths())
-    {
-        tryRemoveDirectory(db_info.getExtraDirectory(extra_root_path), log, true);
-        tryRemoveDirectory(db_info.getExtraDirectory(extra_root_path, false), log, true);
-    }
-}
-
-void IDAsPathUpgrader::linkDatabaseTableInfos(const std::vector<TiDB::DBInfoPtr> & all_databases)
-{
-    for (const auto & db : all_databases)
-    {
-        if (auto iter = databases.find(db->name); iter != databases.end())
-        {
-            iter->second.setDBInfo(db);
-        }
-    }
-
-    // list all table in old style.
-    for (auto iter = databases.begin(); iter != databases.end(); /*empty*/)
-    {
-        const auto & db_name = iter->first;
-        auto & db_info = iter->second;
-        if (!db_info.hasValidTiDBInfo())
-        {
-            // If we can't find it in TiDB, maybe it already dropped.
-            if (reserved_databases.count(db_name) > 0)
-            {
-                // For mock test or develop environment, we may reserve some database
-                // for convenience. Keep them as what they are. Print warnings and
-                // ignore it in later upgrade.
-                LOG_WARNING(log, "Database {} is reserved, ignored in upgrade.", db_name);
-            }
-            else
-            {
-                // If we keep them as "Ordinary", when user actually create database with
-                // same name, next time TiFlash restart and will try to do "upgrade" on
-                // those legacy data, and it will mess everything up.
-                // Drop them.
-                dropAbsentDatabase(global_context, db_name, db_info, log);
-            }
-            iter = databases.erase(iter);
-            continue;
-        }
-
-        if (db_info.engine == "TiFlash")
-        {
-            ++iter;
-            continue;
-        }
-
-        const String db_meta_dir = db_info.getMetaDirectory(root_path);
-        std::vector<std::string> file_names = DatabaseLoading::listSQLFilenames(db_meta_dir, log);
-        for (const auto & table_filename : file_names)
-        {
-            String table_meta_file = db_meta_dir + "/" + table_filename;
-            // Name in table_info may not be updated, use the name in `ATTACH TABLE <name> ...`.
-            auto [old_name, table_info] = getTableInfo(global_context.getFileProvider(), table_meta_file);
-            db_info.tables.emplace_back( //
-                TableDiskInfo{old_name, std::make_shared<TiDB::TableInfo>(table_info), mapper});
-        }
-        ++iter;
-    }
-}
-
-void IDAsPathUpgrader::fixNotEscapedDirectories()
-{
-    for (const auto & [db_name, db_info] : databases)
-    {
-        const auto db_name_escaped = escapeForFileName(db_name);
-
-        // database's meta file, meta dir (created by old DatabaseOrdinary) is escaped.
-        // only need to create data path
-        if (db_name != db_name_escaped)
-        {
-            LOG_INFO(log, "database `{}` fixing name escape to `{}`", db_name, db_name_escaped);
-            // Create directory for escaped database
-            auto escaped_db_data_dir = db_info.getDataDirectory(root_path, /*escape=*/true);
-            if (Poco::File dir(escaped_db_data_dir); !dir.exists())
-                dir.createDirectory();
-
-            const auto & data_extra_paths = global_context.getPathPool();
-            for (const auto & extra_root_path : data_extra_paths.listPaths())
-            {
-                auto escaped_extra_dir = db_info.getExtraDirectory(extra_root_path, /*escape=*/true);
-                if (Poco::File dir(escaped_extra_dir); !dir.exists())
-                    dir.createDirectory();
-            }
-        }
-
-        /// Fix not escaped name for table
-        for (const auto & table : db_info.tables)
-        {
-            const auto table_name_escaped = escapeForFileName(table.name());
-            if (db_name_escaped == db_name && table_name_escaped == table.name())
-                continue;
-
-            LOG_INFO(
-                log,
-                "table `{}`.`{}` fixing name escape to `{}`.`{}`",
-                db_name,
-                table.name(),
-                db_name_escaped,
-                table_name_escaped);
-            // Table's metadata don't need to fix.
-
-            // Fix data path. It was create by DatabaseOrdinary and StorageDeltaMerge,
-            // database name is escaped but table name not.
-            auto not_escaped_path = table.getDataDirectory(root_path, db_info, /*escape_db*/ true, /*escape_tbl*/ false);
-            auto escaped_path = table.getDataDirectory(root_path, db_info, /*escape_db*/ true, /*escape_tbl*/ true);
-            if (auto file = Poco::File{not_escaped_path}; file.exists())
-            {
-                if (auto escaped_dir = Poco::File{escaped_path}; !escaped_dir.exists())
-                    escaped_dir.createDirectory();
-                renamePath(not_escaped_path + "/meta", escaped_path + "/meta", log, true);
-                renamePath(not_escaped_path + "/data", escaped_path + "/data", log, true);
-                renamePath(not_escaped_path + "/log", escaped_path + "/log", log, true);
-                // For the cases that database's name did not need to be escaped but table's name did.
-                renamePath(not_escaped_path + "/stable", escaped_path + "/stable", log, false);
-                tryRemoveDirectory(not_escaped_path, log);
-            }
-            auto db_tbl_not_escaped_path = not_escaped_path;
-            if (db_name != db_name_escaped)
-            {
-                // For the cases that database's name need to be escaped.
-                // Stable dir was created by old PathPool, database name and table name were not escaped.
-                db_tbl_not_escaped_path = table.getDataDirectory(root_path, db_info, false, false);
-                auto not_escaped_stable = db_tbl_not_escaped_path + "/stable";
-                auto escaped_stable = table.getDataDirectory(root_path, db_info, true, true) + "/stable";
-                if (auto file = Poco::File{not_escaped_stable}; file.exists())
-                    renamePath(not_escaped_stable, escaped_stable, log, true);
-            }
-
-            // Fix extra path.
-            const auto & data_extra_paths = global_context.getPathPool();
-            for (const auto & extra_root_path : data_extra_paths.listPaths())
-            {
-                // It was created by old PathPool, both database name and table name are not escaped.
-                auto not_escaped_extra_path = table.getExtraDirectory(extra_root_path, db_info, /*escape_db*/ false, /*escape_tbl*/ false);
-                if (isSamePath(not_escaped_extra_path, db_tbl_not_escaped_path))
-                    continue;
-                auto escaped_extra_path = table.getExtraDirectory(extra_root_path, db_info, /*escape_db*/ true, /*escape_tbl*/ true);
-                renamePath(not_escaped_extra_path, escaped_extra_path, log, false);
-            }
-            LOG_INFO(
-                log,
-                "table `{}`.`{}` fixing name escape to `{}`.`{}` done.",
-                db_name,
-                table.name(),
-                db_name_escaped,
-                table_name_escaped);
-        }
-
-        if (db_name != db_name_escaped)
-        {
-            // clean not escaped database dir created by old PathPool
-            const String not_escaped_data_dir = db_info.getDataDirectory(root_path, /*escape*/ false);
-            tryRemoveDirectory(not_escaped_data_dir, log, true);
-            const auto & data_extra_paths = global_context.getPathPool();
-            for (const auto & extra_root_path : data_extra_paths.listPaths())
-            {
-                auto not_escaped_extra_data_dir = db_info.getExtraDirectory(extra_root_path, /*escape*/ false);
-                if (isSamePath(not_escaped_data_dir, not_escaped_extra_data_dir))
-                    continue;
-                tryRemoveDirectory(not_escaped_extra_data_dir, log);
-            }
-        }
-        LOG_INFO(log, "database `{}` fixing name escape to `{}` done.", db_name, db_name_escaped);
-    }
-}
-
-void IDAsPathUpgrader::resolveConflictDirectories()
-{
-    std::unordered_set<String> conflict_databases;
-    for (const auto & [db_name, db_info] : databases)
-    {
-        // In theory, user can create database naming "t_xx" and there is cyclic renaming between table and database.
-        // First detect if there is any database may have cyclic rename with table.
-        for (const auto & table : db_info.tables)
-        {
-            const auto new_tbl_name = table.newName();
-            if (auto iter = databases.find(new_tbl_name); iter != databases.end())
-            {
-                conflict_databases.insert(iter->first);
-                LOG_INFO(
-                    log,
-                    "Detect cyclic renaming between table `{}`.`{}`(new name:{}) and database `{}`",
-                    db_name,
-                    table.name(),
-                    new_tbl_name,
-                    iter->first);
-            }
-        }
-
-        // In theory, user can create two database naming "db_xx" and there is cyclic renaming.
-        // We need to break that cyclic.
-        const auto new_database_name = db_info.newName();
-        if (auto iter = databases.find(new_database_name); iter != databases.end())
-        {
-            conflict_databases.insert(iter->first);
-            LOG_INFO(
-                log,
-                "Detect cyclic renaming between database `{}`(new name:{}) and database `{}`",
-                db_name,
-                new_database_name,
-                iter->first);
-        }
-    }
-    LOG_INFO(log, "Detect {} cyclic renaming", conflict_databases.size());
-    for (const auto & db_name : conflict_databases)
-    {
-        auto iter = databases.find(db_name);
-        auto & db_info = iter->second;
-        LOG_INFO(log, "Move {} to tmp directories..", db_name);
-        db_info.renameToTmpDirectories(global_context, log);
-    }
-}
-
-void IDAsPathUpgrader::doRename()
-{
-    for (const auto & [db_name, db_info] : databases)
-    {
-        renameDatabase(db_name, db_info);
-    }
-}
-
-void IDAsPathUpgrader::renameDatabase(const String & db_name, const DatabaseDiskInfo & db_info)
-{
-    const auto mapped_db_name = db_info.newName();
-
-    {
-        // Create directory for target database
-        auto new_db_meta_dir = db_info.getNewMetaDirectory(root_path);
-        Poco::File(new_db_meta_dir).createDirectory();
-    }
-
-    // Rename all tables of this database
-    for (const auto & table : db_info.tables)
-    {
-        renameTable(db_name, db_info, mapped_db_name, table);
-    }
-
-    // Then rename database
-    LOG_INFO(log, "database `{}` to `{}` renaming", db_name, mapped_db_name);
-    {
-        // Recreate metadata file for database
-        const String new_meta_file = db_info.getNewMetaFilePath(root_path);
-        const String statement = "ATTACH DATABASE `" + mapped_db_name + "` ENGINE=TiFlash('" + db_info.getTiDBSerializeInfo() + "', 1)\n";
-        auto ast = parseCreateDatabaseAST(statement);
-        const auto & settings = global_context.getSettingsRef();
-        writeDatabaseDefinitionToFile(global_context.getFileProvider(), new_meta_file, ast, settings.fsync_metadata);
-    }
-
-    {
-        // Remove old metadata dir
-        const String old_meta_dir = db_info.getMetaDirectory(root_path);
-        tryRemoveDirectory(old_meta_dir, log);
-        // Remove old metadata file
-        const String old_meta_file = db_info.getMetaFilePath(root_path);
-        if (auto file = Poco::File(old_meta_file); file.exists())
-            file.remove();
-        else
-            LOG_WARNING(log, "Can not remove database meta file: {}", old_meta_file);
-        // Remove old data dir
-        const String old_data_dir = db_info.getDataDirectory(root_path);
-        tryRemoveDirectory(old_data_dir, log);
-        const auto & data_extra_paths = global_context.getPathPool();
-        for (const auto & extra_root_path : data_extra_paths.listPaths())
-        {
-            tryRemoveDirectory(db_info.getExtraDirectory(extra_root_path), log);
-        }
-    }
-    LOG_INFO(log, "database `{}` to `{}` rename done.", db_name, mapped_db_name);
-}
-
-void IDAsPathUpgrader::renameTable(
-    const String & db_name,
-    const DatabaseDiskInfo & db_info,
-    const String & mapped_db_name,
-    const TableDiskInfo & table)
-{
-    const auto mapped_table_name = table.newName();
-    LOG_INFO(
-        log,
-        "table `{}`.`{}` to `{}`.`{}` renaming",
-        db_name,
-        table.name(),
-        mapped_db_name,
-        mapped_table_name);
-
-    String old_tbl_data_path;
-    {
-        // Former data path use ${path}/data/${database}/${table}/ as data path.
-        // Rename it to ${path}/data/${mapped_table_name}.
-        old_tbl_data_path = table.getDataDirectory(root_path, db_info);
-        renamePath(old_tbl_data_path, table.getNewDataDirectory(root_path, db_info), log, true);
-    }
-
-    {
-        // Rename data path for multi disk
-        auto data_extra_paths = global_context.getPathPool();
-        for (const auto & extra_root_path : data_extra_paths.listPaths())
-        {
-            auto old_tbl_extra_data_path = table.getExtraDirectory(extra_root_path, db_info);
-            if (isSamePath(old_tbl_extra_data_path, old_tbl_data_path))
-                continue;
-            renamePath(old_tbl_extra_data_path, table.getNewExtraDirectory(extra_root_path, db_info), log, false);
-        }
-    }
-
-    // Recreate metadata file
-    {
-        auto old_tbl_meta_file = table.getMetaFilePath(root_path, db_info);
-        auto ast = DatabaseLoading::getQueryFromMetadata(global_context, old_tbl_meta_file, /*throw_on_error=*/true);
-        if (!ast)
-            throw Exception("There is no metadata file for table " + table.name() + ", expected file: " + old_tbl_meta_file,
-                            ErrorCodes::FILE_DOESNT_EXIST);
-
-        ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
-        ast_create_query.table = mapped_table_name;
-        ASTStorage * storage_ast = ast_create_query.storage;
-        TiDB::TableInfo table_info = table.getInfo(); // get a copy
-        if (table_info.is_partition_table)
-        {
-            LOG_INFO(
-                log,
-                "partition table `{}`.`{}` to `{}`.`{}` update table info",
-                db_name,
-                table.name(),
-                mapped_db_name,
-                mapped_table_name);
-            // Old partition name is "${table_name}_${physical_id}" while new name is "t_${physical_id}"
-            // If it is a partition table, we need to update TiDB::TableInfo::name
-            do
-            {
-                if (!storage_ast || !storage_ast->engine)
-                    break;
-                auto * args = typeid_cast<ASTExpressionList *>(storage_ast->engine->arguments.get());
-                if (!args)
-                    break;
-
-                table_info.name = mapper->mapPartitionName(table_info);
-                std::shared_ptr<ASTLiteral> literal = std::make_shared<ASTLiteral>(Field(table_info.serialize()));
-                if (args->children.size() == 1)
-                    args->children.emplace_back(literal);
-                else if (args->children.size() >= 2)
-                    args->children.at(1) = literal;
-            } while (false);
-        }
-
-        const String new_tbl_meta_file = table.getNewMetaFilePath(root_path, db_info);
-        const auto & settings = global_context.getSettingsRef();
-        writeTableDefinitionToFile(global_context.getFileProvider(), new_tbl_meta_file, ast, settings.fsync_metadata);
-
-        // Remove old metadata file
-        if (auto file = Poco::File(old_tbl_meta_file); file.exists())
-            file.remove();
-    }
-
-    LOG_INFO(
-        log,
-        "table `{}`.`{}` to `{}`.`{}` rename done.",
-        db_name,
-        table.name(),
-        mapped_db_name,
-        mapped_table_name);
-}
-
-void IDAsPathUpgrader::doUpgrade()
-{
-    auto all_databases = fetchInfosFromTiDB();
-    linkDatabaseTableInfos(all_databases);
-    fixNotEscapedDirectories();
-    // Check if destination db / tbl file exists and resolve conflict
-    resolveConflictDirectories();
-    // Rename
-    doRename();
-}
-
-} // namespace DB
diff --git a/dbms/src/Interpreters/IDAsPathUpgrader.h b/dbms/src/Interpreters/IDAsPathUpgrader.h
deleted file mode 100644
index 38dc37536aa..00000000000
--- a/dbms/src/Interpreters/IDAsPathUpgrader.h
+++ /dev/null
@@ -1,202 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#pragma once
-
-#include <Core/Types.h>
-#include <Storages/Transaction/Types.h>
-
-#include <map>
-#include <memory>
-#include <unordered_set>
-
-namespace Poco
-{
-class Logger;
-}
-
-namespace TiDB
-{
-struct TableInfo;
-using TableInfoPtr = std::shared_ptr<TableInfo>;
-
-struct DBInfo;
-using DBInfoPtr = std::shared_ptr<DBInfo>;
-} // namespace TiDB
-
-namespace DB
-{
-class Context;
-class PathPool;
-struct SchemaNameMapper;
-
-class IDAsPathUpgrader
-{
-public:
-    struct DatabaseDiskInfo;
-
-    struct TableDiskInfo
-    {
-    public:
-        String name() const;
-        String newName() const;
-
-        TableDiskInfo(String old_name_, TiDB::TableInfoPtr info_, std::shared_ptr<SchemaNameMapper> mapper_)
-            : old_name(old_name_)
-            , tidb_table_info(std::move(info_))
-            , mapper(std::move(mapper_))
-        {}
-
-    private:
-        String old_name;
-        TiDB::TableInfoPtr tidb_table_info;
-        std::shared_ptr<SchemaNameMapper> mapper;
-
-    public:
-        // "metadata/${db_name}/${tbl_name}.sql"
-        String getMetaFilePath(const String & root_path, const DatabaseDiskInfo & db) const;
-        // "data/${db_name}/${tbl_name}/"
-        String getDataDirectory(const String & root_path, const DatabaseDiskInfo & db, bool escape_db = true, bool escape_tbl = true) const;
-        // "extra_data/${db_name}/${tbl_name}/"
-        String getExtraDirectory(
-            const String & root_path,
-            const DatabaseDiskInfo & db,
-            bool escape_db = true,
-            bool escape_tbl = true) const;
-
-        // "metadata/db_${db_id}/t_${id}.sql"
-        String getNewMetaFilePath(const String & root_path, const DatabaseDiskInfo & db) const;
-        // "data/t_${id}/"
-        String getNewDataDirectory(const String & root_path, const DatabaseDiskInfo & db) const;
-        // "extra_data/t_${id}"
-        String getNewExtraDirectory(const String & root_path, const DatabaseDiskInfo & db) const;
-
-        const TiDB::TableInfo & getInfo() const;
-    };
-
-    struct DatabaseDiskInfo
-    {
-    public:
-        static constexpr auto TMP_SUFFIX = "_flash_upgrade";
-
-        String engine;
-        std::vector<TableDiskInfo> tables;
-
-    private:
-        String name;
-        std::shared_ptr<SchemaNameMapper> mapper;
-        bool moved_to_tmp = false;
-        TiDB::DBInfoPtr tidb_db_info = nullptr;
-
-        const TiDB::DBInfo & getInfo() const;
-
-    public:
-        DatabaseDiskInfo(String name_, std::shared_ptr<SchemaNameMapper> mapper_)
-            : name(std::move(name_))
-            , mapper(std::move(mapper_))
-        {}
-
-        void setDBInfo(TiDB::DBInfoPtr info_);
-
-        bool hasValidTiDBInfo() const { return tidb_db_info != nullptr; }
-
-        String newName() const;
-
-        String getTiDBSerializeInfo() const;
-
-        // "metadata/${db_name}.sql"
-        String getMetaFilePath(const String & root_path) const { return doGetMetaFilePath(root_path, moved_to_tmp); }
-        // "metadata/${db_name}/"
-        String getMetaDirectory(const String & root_path) const { return doGetMetaDirectory(root_path, moved_to_tmp); }
-        // "data/${db_name}/"
-        String getDataDirectory(const String & root_path, bool escape = true) const
-        {
-            return doGetDataDirectory(root_path, escape, moved_to_tmp);
-        }
-        // "extra_data/${db_name}/". db_name is not escaped.
-        String getExtraDirectory(const String & extra_root, bool escape = true) const
-        {
-            return doGetExtraDirectory(extra_root, escape, moved_to_tmp);
-        }
-
-        void renameToTmpDirectories(const Context & ctx, Poco::Logger * log);
-
-        // "metadata/db_${id}.sql"
-        String getNewMetaFilePath(const String & root_path) const;
-        // "metadata/db_${id}/"
-        String getNewMetaDirectory(const String & root_path) const;
-        // "data/"
-        static String getNewDataDirectory(const String & root_path);
-        // "extra_data/"
-        static String getNewExtraDirectory(const String & extra_root);
-
-    private:
-        // "metadata/${db_name}.sql"
-        String doGetMetaFilePath(const String & root_path, bool tmp) const;
-        // "metadata/${db_name}/"
-        String doGetMetaDirectory(const String & root_path, bool tmp) const;
-        // "data/${db_name}/"
-        String doGetDataDirectory(const String & root_path, bool escape, bool tmp) const;
-        // "extra_data/${db_name}/"
-        String doGetExtraDirectory(const String & extra_root, bool escape, bool tmp) const;
-    };
-
-public:
-    /// Upgrader
-    // If some database can not find in TiDB, they will be dropped
-    // if theirs name is not in reserved_databases
-    IDAsPathUpgrader(Context & global_ctx_, bool is_mock_, std::unordered_set<std::string> reserved_databases_);
-
-    bool needUpgrade();
-
-    void doUpgrade();
-
-private:
-    std::vector<TiDB::DBInfoPtr> fetchInfosFromTiDB() const;
-
-    void linkDatabaseTableInfos(const std::vector<TiDB::DBInfoPtr> & all_databases);
-
-    // Some path created by old PathPool, its database / table name is not escaped,
-    // normalized those names first.
-    void fixNotEscapedDirectories();
-
-    void resolveConflictDirectories();
-
-    void doRename();
-
-    void renameDatabase(const String & db_name, const DatabaseDiskInfo & db_info);
-
-    void renameTable(
-        const String & db_name,
-        const DatabaseDiskInfo & db_info,
-        const String & mapped_db_name,
-        const TableDiskInfo & table_info);
-
-private:
-    Context & global_context;
-
-    const String root_path;
-
-    std::map<String, DatabaseDiskInfo> databases;
-
-    const bool is_mock = false;
-
-    std::shared_ptr<SchemaNameMapper> mapper;
-
-    const std::unordered_set<std::string> reserved_databases;
-
-    Poco::Logger * log;
-};
-
-} // namespace DB
diff --git a/dbms/src/Interpreters/sortBlock.h b/dbms/src/Interpreters/sortBlock.h
index 80098066c72..a768920dab0 100644
--- a/dbms/src/Interpreters/sortBlock.h
+++ b/dbms/src/Interpreters/sortBlock.h
@@ -24,7 +24,7 @@ namespace DB
 void sortBlock(Block & block, const SortDescription & description, size_t limit = 0);
 
 
-/** Used only in StorageMergeTree to sort the data with INSERT.
+/** Used only in StorageDeltaMerge to sort the data with INSERT.
   * Sorting is stable. This is important for keeping the order of rows in the CollapsingMergeTree engine
   *  - because based on the order of rows it is determined whether to delete or leave groups of rows when collapsing.
   * Collations are not supported. Partial sorting is not supported.
diff --git a/dbms/src/Interpreters/tests/CMakeLists.txt b/dbms/src/Interpreters/tests/CMakeLists.txt
deleted file mode 100644
index f310837b3f0..00000000000
--- a/dbms/src/Interpreters/tests/CMakeLists.txt
+++ /dev/null
@@ -1,55 +0,0 @@
-# Copyright 2022 PingCAP, Ltd.
-#
-# Licensed 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.
-
-add_executable (create_query create_query.cpp)
-target_link_libraries (create_query dbms)
-
-add_executable (select_query select_query.cpp)
-target_link_libraries (select_query clickhouse_storages_system dbms)
-
-add_executable (aggregate aggregate.cpp)
-target_link_libraries (aggregate dbms)
-
-add_executable (hash_map hash_map.cpp)
-target_include_directories (hash_map BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR})
-target_link_libraries (hash_map dbms)
-
-add_executable (hash_map3 hash_map3.cpp)
-target_link_libraries (hash_map3 dbms ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES})
-
-add_executable (hash_map_string hash_map_string.cpp)
-target_include_directories (hash_map_string BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR})
-target_link_libraries (hash_map_string dbms)
-
-add_executable (hash_map_string_2 hash_map_string_2.cpp)
-target_link_libraries (hash_map_string_2 dbms)
-
-add_executable (hash_map_string_3 hash_map_string_3.cpp)
-target_link_libraries (hash_map_string_3 dbms ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES})
-target_include_directories (hash_map_string_3 BEFORE PRIVATE ${TiFlash_SOURCE_DIR}/contrib/libfarmhash)
-target_include_directories (hash_map_string_3 BEFORE PRIVATE ${TiFlash_SOURCE_DIR}/contrib/libmetrohash/src)
-
-add_executable (hash_map_string_small hash_map_string_small.cpp)
-target_include_directories (hash_map_string_small BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR})
-target_link_libraries (hash_map_string_small dbms)
-
-add_executable (two_level_hash_map two_level_hash_map.cpp)
-target_include_directories (two_level_hash_map BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR})
-target_link_libraries (two_level_hash_map dbms)
-
-add_executable (logical_expressions_optimizer logical_expressions_optimizer.cpp)
-target_link_libraries (logical_expressions_optimizer dbms)
-
-add_executable (users users.cpp)
-target_link_libraries (users dbms ${Boost_FILESYSTEM_LIBRARY} dbms)
diff --git a/dbms/src/Interpreters/tests/aggregate.cpp b/dbms/src/Interpreters/tests/aggregate.cpp
deleted file mode 100644
index 27491f4afbb..00000000000
--- a/dbms/src/Interpreters/tests/aggregate.cpp
+++ /dev/null
@@ -1,117 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#include <AggregateFunctions/AggregateFunctionFactory.h>
-#include <Columns/ColumnString.h>
-#include <Columns/ColumnsNumber.h>
-#include <DataStreams/OneBlockInputStream.h>
-#include <DataTypes/DataTypeString.h>
-#include <DataTypes/DataTypesNumber.h>
-#include <Encryption/FileProvider.h>
-#include <Encryption/MockKeyManager.h>
-#include <Interpreters/Aggregator.h>
-
-#include <iomanip>
-#include <iostream>
-
-
-int main(int argc, char ** argv)
-{
-    using namespace DB;
-
-    try
-    {
-        size_t n = argc == 2 ? atoi(argv[1]) : 10;
-
-        Block block;
-
-        {
-            ColumnWithTypeAndName column;
-            column.name = "x";
-            column.type = std::make_shared<DataTypeInt16>();
-            auto col = ColumnInt16::create();
-            auto & vec_x = col->getData();
-
-            vec_x.resize(n);
-            for (size_t i = 0; i < n; ++i)
-                vec_x[i] = i % 9;
-
-            column.column = std::move(col);
-            block.insert(column);
-        }
-
-        const char * strings[] = {"abc", "def", "abcd", "defg", "ac"};
-
-        {
-            ColumnWithTypeAndName column;
-            column.name = "s1";
-            column.type = std::make_shared<DataTypeString>();
-            auto col = ColumnString::create();
-
-            for (size_t i = 0; i < n; ++i)
-                col->insert(std::string(strings[i % 5]));
-
-            column.column = std::move(col);
-            block.insert(column);
-        }
-
-        {
-            ColumnWithTypeAndName column;
-            column.name = "s2";
-            column.type = std::make_shared<DataTypeString>();
-            auto col = ColumnString::create();
-
-            for (size_t i = 0; i < n; ++i)
-                col->insert(std::string(strings[i % 3]));
-
-            column.column = std::move(col);
-            block.insert(column);
-        }
-
-        BlockInputStreamPtr stream = std::make_shared<OneBlockInputStream>(block);
-        AggregatedDataVariants aggregated_data_variants;
-
-        AggregateFunctionFactory factory;
-
-        AggregateDescriptions aggregate_descriptions(1);
-
-        DataTypes empty_list_of_types;
-        aggregate_descriptions[0].function = factory.get("count", empty_list_of_types);
-
-        Aggregator::Params params(stream->getHeader(), {0, 1}, aggregate_descriptions, false);
-
-        Aggregator aggregator(params);
-
-        {
-            Stopwatch stopwatch;
-            stopwatch.start();
-
-            KeyManagerPtr key_manager = std::make_shared<MockKeyManager>(false);
-            FileProviderPtr file_provider = std::make_shared<FileProvider>(key_manager, false);
-            aggregator.execute(stream, aggregated_data_variants, file_provider);
-
-            stopwatch.stop();
-            std::cout << std::fixed << std::setprecision(2)
-                      << "Elapsed " << stopwatch.elapsedSeconds() << " sec."
-                      << ", " << n / stopwatch.elapsedSeconds() << " rows/sec."
-                      << std::endl;
-        }
-    }
-    catch (const Exception & e)
-    {
-        std::cerr << e.displayText() << std::endl;
-    }
-
-    return 0;
-}
\ No newline at end of file
diff --git a/dbms/src/Interpreters/tests/create_query.cpp b/dbms/src/Interpreters/tests/create_query.cpp
deleted file mode 100644
index 03f787e1547..00000000000
--- a/dbms/src/Interpreters/tests/create_query.cpp
+++ /dev/null
@@ -1,115 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#include <iostream>
-#include <iomanip>
-
-#include <Parsers/ParserCreateQuery.h>
-#include <Parsers/formatAST.h>
-#include <Parsers/parseQuery.h>
-
-#include <Databases/DatabaseOrdinary.h>
-
-#include <Interpreters/Context.h>
-#include <Interpreters/InterpreterCreateQuery.h>
-
-
-using namespace DB;
-
-int main(int, char **)
-try
-{
-    std::string input = "CREATE TABLE IF NOT EXISTS hits (\n"
-        "WatchID                UInt64,\n"
-        "JavaEnable             UInt8,\n"
-        "Title                  String,\n"
-        "EventTime              DateTime,\n"
-        "CounterID              UInt32,\n"
-        "ClientIP               UInt32,\n"
-        "RegionID               UInt32,\n"
-        "UniqID                 UInt64,\n"
-        "CounterClass           UInt8,\n"
-        "OS                     UInt8,\n"
-        "UserAgent              UInt8,\n"
-        "URL                    String,\n"
-        "Referer                String,\n"
-        "ResolutionWidth        UInt16,\n"
-        "ResolutionHeight       UInt16,\n"
-        "ResolutionDepth        UInt8,\n"
-        "FlashMajor             UInt8,\n"
-        "FlashMinor             UInt8,\n"
-        "FlashMinor2            String,\n"
-        "NetMajor               UInt8,\n"
-        "NetMinor               UInt8,\n"
-        "UserAgentMajor         UInt16,\n"
-        "UserAgentMinor         FixedString(2),\n"
-        "CookieEnable           UInt8,\n"
-        "JavascriptEnable       UInt8,\n"
-        "IsMobile               UInt8,\n"
-        "MobilePhone            UInt8,\n"
-        "MobilePhoneModel       String,\n"
-        "Params                 String,\n"
-        "IPNetworkID            UInt32,\n"
-        "TraficSourceID         Int8,\n"
-        "SearchEngineID         UInt16,\n"
-        "SearchPhrase           String,\n"
-        "AdvEngineID            UInt8,\n"
-        "IsArtifical            UInt8,\n"
-        "WindowClientWidth      UInt16,\n"
-        "WindowClientHeight     UInt16,\n"
-        "ClientTimeZone         Int16,\n"
-        "ClientEventTime        DateTime,\n"
-        "SilverlightVersion1    UInt8,\n"
-        "SilverlightVersion2    UInt8,\n"
-        "SilverlightVersion3    UInt32,\n"
-        "SilverlightVersion4    UInt16,\n"
-        "PageCharset            String,\n"
-        "CodeVersion            UInt32,\n"
-        "IsLink                 UInt8,\n"
-        "IsDownload             UInt8,\n"
-        "IsNotBounce            UInt8,\n"
-        "FUniqID                UInt64,\n"
-        "OriginalURL            String,\n"
-        "HID                    UInt32,\n"
-        "IsOldCounter           UInt8,\n"
-        "IsEvent                UInt8,\n"
-        "IsParameter            UInt8,\n"
-        "DontCountHits          UInt8,\n"
-        "WithHash               UInt8\n"
-        ") ENGINE = Log";
-
-    ParserCreateQuery parser;
-    ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
-
-    Context context = Context::createGlobal();
-
-    context.setPath("./");
-    auto database = std::make_shared<DatabaseOrdinary>("test", "./metadata/test/", context);
-    context.addDatabase("test", database);
-    database->loadTables(context, nullptr, false);
-    context.setCurrentDatabase("test");
-
-    InterpreterCreateQuery interpreter(ast, context);
-    interpreter.execute();
-
-    return 0;
-}
-catch (const Exception & e)
-{
-    std::cerr << e.what() << ", " << e.displayText() << std::endl
-        << std::endl
-        << "Stack trace:" << std::endl
-        << e.getStackTrace().toString();
-    return 1;
-}
diff --git a/dbms/src/Interpreters/tests/gtest_id_as_path_upgrader.cpp b/dbms/src/Interpreters/tests/gtest_id_as_path_upgrader.cpp
deleted file mode 100644
index 1ff1d8febb7..00000000000
--- a/dbms/src/Interpreters/tests/gtest_id_as_path_upgrader.cpp
+++ /dev/null
@@ -1,210 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#include <Databases/IDatabase.h>
-#include <Debug/MockTiDB.h>
-#include <Interpreters/IDAsPathUpgrader.h>
-#include <Interpreters/loadMetadata.h>
-#include <Poco/Environment.h>
-#include <Storages/Transaction/TMTContext.h>
-#include <Storages/registerStorages.h>
-#include <TestUtils/TiFlashTestBasic.h>
-
-namespace DB::tests
-{
-
-class IDAsPathUpgrader_test : public ::testing::Test
-{
-public:
-    void SetUp() override
-    {
-        try
-        {
-            registerStorages();
-        }
-        catch (DB::Exception &)
-        {
-            // Maybe another test has already registed, ignore exception here.
-        }
-    }
-
-    void TearDown() override
-    {
-        auto ctx = TiFlashTestEnv::getContext();
-        auto databases = ctx.getDatabases();
-        for (auto database : databases)
-        {
-            database.second->shutdown();
-            ctx.detachDatabase(database.first);
-        }
-    }
-};
-
-TEST_F(IDAsPathUpgrader_test, ONCALL_1651)
-try
-{
-    CHECK_TESTS_WITH_DATA_ENABLED;
-
-    // prepare a "test" database for upgrader
-    MockTiDB::instance().newDataBase("test"); // id == 2
-
-    // Generated by running these SQL on cluster version v3.1.0
-    // > create table test.aaa(pk int);
-    // > rename table test.aaa TO test.abc;
-    // > CREATE TABLE test.employees(id INT NOT NULL AUTO_INCREMENT PRIMARY KEY,
-    //     fname VARCHAR(25) NOT NULL,
-    //     lname VARCHAR(25) NOT NULL,
-    //     store_id INT NOT NULL,
-    //     department_id INT NOT NULL)
-    //     PARTITION BY RANGE(id)(PARTITION p0 VALUES LESS THAN(5),
-    //         PARTITION p1 VALUES LESS THAN(10),
-    //         PARTITION p2 VALUES LESS THAN(15),
-    //         PARTITION p3 VALUES LESS THAN MAXVALUE);
-    // > RENAME TABLE test.employees TO test.emp;
-    // > RENAME TABLE test.emp TO test.emp_bak;
-    const auto test_path = TiFlashTestEnv::findTestDataPath("oncall-1651");
-    auto ctx = TiFlashTestEnv::getContext(DB::Settings(), test_path);
-
-    IDAsPathUpgrader upgrader(ctx, false, {});
-    ASSERT_TRUE(upgrader.needUpgrade());
-    upgrader.doUpgrade();
-
-    {
-        // After upgrade, next time we don't need it.
-        IDAsPathUpgrader checker_after_upgrade(ctx, false, {});
-        ASSERT_FALSE(checker_after_upgrade.needUpgrade());
-    }
-
-    // load metadata should not throw any exception
-    loadMetadata(ctx);
-
-    ASSERT_TRUE(ctx.isDatabaseExist("db_2")); // "test"
-    auto & storages = ctx.getTMTContext().getStorages();
-    ASSERT_NE(storages.get(45), nullptr); // `test`.`abc`
-    ASSERT_NE(storages.get(48), nullptr); // `test`.`emp_bak`
-    ASSERT_NE(storages.get(49), nullptr);
-    ASSERT_NE(storages.get(50), nullptr);
-    ASSERT_NE(storages.get(51), nullptr);
-    ASSERT_NE(storages.get(52), nullptr);
-}
-CATCH
-
-TEST_F(IDAsPathUpgrader_test, FLASH_1136_4_0_0_rc)
-try
-{
-    CHECK_TESTS_WITH_DATA_ENABLED;
-
-    // prepare a "test" database for upgrader
-    MockTiDB::instance().newDataBase("test");    // id == 2
-    MockTiDB::instance().newDataBase("test-db"); // id == 3
-
-    // Generated by running these SQL on cluster version v4.0.0-rc
-    // > create database `test-db`;
-    // > create table `test-tbl` (pk int);
-    // > cerate table `test`.`test-tbl` (pk int);
-    const auto test_path = TiFlashTestEnv::findTestDataPath("flash-1136");
-    auto ctx = TiFlashTestEnv::getContext(DB::Settings(), test_path);
-
-    IDAsPathUpgrader upgrader(ctx, false, {});
-    ASSERT_TRUE(upgrader.needUpgrade());
-    upgrader.doUpgrade();
-
-    {
-        // After upgrade, next time we don't need it.
-        IDAsPathUpgrader checker_after_upgrade(ctx, false, {});
-        ASSERT_FALSE(checker_after_upgrade.needUpgrade());
-    }
-
-    // load metadata should not throw any exception
-    loadMetadata(ctx);
-
-    ASSERT_TRUE(ctx.isDatabaseExist("db_2"));
-    ASSERT_TRUE(ctx.isDatabaseExist("db_3"));
-    auto & storages = ctx.getTMTContext().getStorages();
-    ASSERT_NE(storages.get(66), nullptr);  // `test-db`.`test-tbl`
-    ASSERT_NE(storages.get(666), nullptr); // `test`.`test-tbl`
-}
-CATCH
-
-TEST_F(IDAsPathUpgrader_test, FLASH_1136_3_1_0)
-try
-{
-    CHECK_TESTS_WITH_DATA_ENABLED;
-
-    // prepare a "test" database for upgrader
-    MockTiDB::instance().newDataBase("test");    // id == 2
-    MockTiDB::instance().newDataBase("test-db"); // id == 3
-
-    // Generated by running these SQL on cluster version v4.0.0-rc
-    // > create database `test-db`;
-    // > create table `test-tbl` (pk int);
-    // > cerate table `test`.`test-tbl` (pk int);
-    const auto test_path = TiFlashTestEnv::findTestDataPath("flash-1136-v3.1.0");
-    auto ctx = TiFlashTestEnv::getContext(DB::Settings(), test_path);
-
-    IDAsPathUpgrader upgrader(ctx, false, {});
-    ASSERT_TRUE(upgrader.needUpgrade());
-    upgrader.doUpgrade();
-
-    {
-        // After upgrade, next time we don't need it.
-        IDAsPathUpgrader checker_after_upgrade(ctx, false, {});
-        ASSERT_FALSE(checker_after_upgrade.needUpgrade());
-    }
-
-    // load metadata should not throw any exception
-    loadMetadata(ctx);
-
-    ASSERT_TRUE(ctx.isDatabaseExist("db_2"));
-    ASSERT_TRUE(ctx.isDatabaseExist("db_3"));
-    auto & storages = ctx.getTMTContext().getStorages();
-    ASSERT_NE(storages.get(66), nullptr);  // `test-db`.`test-tbl`
-    ASSERT_NE(storages.get(666), nullptr); // `test`.`test-tbl`
-}
-CATCH
-
-TEST_F(IDAsPathUpgrader_test, ISSUE_941)
-try
-{
-    CHECK_TESTS_WITH_DATA_ENABLED;
-
-    // prepare a "test" database for upgrader
-    MockTiDB::instance().newDataBase("test"); // id == 2
-
-    // Generated by running these SQL on cluster version v4.0.0-rc
-    // > create database `test`;
-    // > create table `#hello-world` (pk int);
-    const auto test_path = TiFlashTestEnv::findTestDataPath("issue-941");
-    auto ctx = TiFlashTestEnv::getContext(DB::Settings(), test_path);
-
-    IDAsPathUpgrader upgrader(ctx, false, {});
-    ASSERT_TRUE(upgrader.needUpgrade());
-    upgrader.doUpgrade();
-
-    {
-        // After upgrade, next time we don't need it.
-        IDAsPathUpgrader checker_after_upgrade(ctx, false, {});
-        ASSERT_FALSE(checker_after_upgrade.needUpgrade());
-    }
-
-    // load metadata should not throw any exception
-    loadMetadata(ctx);
-
-    ASSERT_TRUE(ctx.isDatabaseExist("db_2"));
-    auto & storages = ctx.getTMTContext().getStorages();
-    ASSERT_NE(storages.get(45), nullptr); // `test`.`#hello-world`
-}
-CATCH
-
-} // namespace DB::tests
diff --git a/dbms/src/Interpreters/tests/hash_map.cpp b/dbms/src/Interpreters/tests/hash_map.cpp
deleted file mode 100644
index d75f578367f..00000000000
--- a/dbms/src/Interpreters/tests/hash_map.cpp
+++ /dev/null
@@ -1,319 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#include <iostream>
-#include <iomanip>
-#include <vector>
-
-#include <unordered_map>
-
-#include <sparsehash/dense_hash_map>
-#include <sparsehash/sparse_hash_map>
-
-#include <Common/Stopwatch.h>
-/*
-#define DBMS_HASH_MAP_COUNT_COLLISIONS
-*/
-#include <Core/Types.h>
-#include <Core/Row.h>
-#include <IO/ReadBufferFromFile.h>
-#include <IO/CompressedReadBuffer.h>
-#include <Common/HashTable/HashMap.h>
-#include <AggregateFunctions/IAggregateFunction.h>
-#include <AggregateFunctions/AggregateFunctionFactory.h>
-#include <DataTypes/DataTypesNumber.h>
-
-
-/** The test checks the speed of hash tables, simulating their use for aggregation.
-  * The first argument specifies the number of elements to be inserted.
-  * The second argument can be a number from 1 to 4 - the number of the data structure being tested.
-  * This is important, because if you run all the tests one by one, the results will be incorrect.
-  * (Due to the peculiarities of the work of the allocator, the first test takes advantage.)
-  *
-  * Depending on USE_AUTO_ARRAY, one of the structures is selected as the value.
-  * USE_AUTO_ARRAY = 0 - uses std::vector (hard-copy structure, sizeof = 24 bytes).
-  * USE_AUTO_ARRAY = 1 - uses AutoArray (a structure specially designed for such cases, sizeof = 8 bytes).
-  *
-  * That is, the test also allows you to compare AutoArray and std::vector.
-  *
-  * If USE_AUTO_ARRAY = 0, then HashMap confidently overtakes all.
-  * If USE_AUTO_ARRAY = 1, then HashMap is slightly less serious (20%) ahead of google::dense_hash_map.
-  *
-  * When using HashMap, AutoArray has a rather serious (40%) advantage over std::vector.
-  * And when using other hash tables, AutoArray even more seriously overtakes std::vector
-  *  (up to three and a half times in the case of std::unordered_map and google::sparse_hash_map).
-  *
-  * HashMap, unlike google::dense_hash_map, much more depends on the quality of the hash function.
-  *
-  * PS. Measure everything yourself, otherwise I'm almost confused.
-  *
-  * PPS. Now the aggregation does not use an array of aggregate functions as values.
-  * States of aggregate functions were separated from the interface to manipulate them, and put in the pool.
-  * But in this test, there was something similar to the old scenario of using hash tables in the aggregation.
-  */
-
-#define USE_AUTO_ARRAY    0
-
-
-struct AlternativeHash
-{
-    size_t operator() (UInt64 x) const
-    {
-        x ^= x >> 23;
-        x *= 0x2127599bf4325c37ULL;
-        x ^= x >> 47;
-
-        return x;
-    }
-};
-
-
-#if defined(__x86_64__)
-
-struct CRC32Hash_
-{
-    size_t operator() (UInt64 x) const
-    {
-        UInt64 crc = -1ULL;
-        asm("crc32q %[x], %[crc]\n" : [crc] "+r" (crc) : [x] "rm" (x));
-        return crc;
-    }
-};
-
-#endif
-
-
-int main(int argc, char ** argv)
-{
-    using namespace DB;
-
-    using Key = UInt64;
-
-#if USE_AUTO_ARRAY
-    using Value = AutoArray<IAggregateFunction*>;
-#else
-    using Value = std::vector<IAggregateFunction*>;
-#endif
-
-    size_t n = argc < 2 ? 10000000 : atoi(argv[1]);
-    //size_t m = atoi(argv[2]);
-
-    AggregateFunctionFactory factory;
-    DataTypes data_types_empty;
-    DataTypes data_types_uint64;
-    data_types_uint64.push_back(std::make_shared<DataTypeUInt64>());
-
-    std::vector<Key> data(n);
-    Value value;
-
-    AggregateFunctionPtr func_count = factory.get("count", data_types_empty);
-    AggregateFunctionPtr func_avg = factory.get("avg", data_types_uint64);
-    AggregateFunctionPtr func_uniq = factory.get("uniq", data_types_uint64);
-
-    #define INIT                \
-    {                            \
-        value.resize(3);        \
-                                \
-        value[0] = func_count.get();\
-        value[1] = func_avg.get();    \
-        value[2] = func_uniq.get();    \
-    }
-
-    INIT;
-
-#ifndef USE_AUTO_ARRAY
-    #undef INIT
-    #define INIT
-#endif
-
-    Row row(1);
-    row[0] = UInt64(0);
-
-    std::cerr << "sizeof(Key) = " << sizeof(Key) << ", sizeof(Value) = " << sizeof(Value) << std::endl;
-
-    {
-        Stopwatch watch;
-    /*    for (size_t i = 0; i < n; ++i)
-            data[i] = rand() % m;
-
-        for (size_t i = 0; i < n; i += 10)
-            data[i] = 0;*/
-
-        ReadBufferFromFile in1("UniqID.bin");
-        CompressedReadBuffer in2(in1);
-
-        in2.readStrict(reinterpret_cast<char*>(&data[0]), sizeof(data[0]) * n);
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "Vector. Size: " << n
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-            << std::endl;
-    }
-
-    if (argc < 3 || atoi(argv[2]) == 1)
-    {
-        Stopwatch watch;
-
-        HashMap<Key, Value> map;
-        HashMap<Key, Value>::iterator it;
-        bool inserted;
-
-        for (size_t i = 0; i < n; ++i)
-        {
-            map.emplace(data[i], it, inserted);
-            if (inserted)
-            {
-                new(&it->second) Value(std::move(value));
-                INIT;
-            }
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "HashMap. Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
-            << ", collisions: " << map.getCollisions()
-#endif
-            << std::endl;
-    }
-
-    if (argc < 3 || atoi(argv[2]) == 2)
-    {
-        Stopwatch watch;
-
-        using Map = HashMap<Key, Value, AlternativeHash>;
-        Map map;
-        Map::iterator it;
-        bool inserted;
-
-        for (size_t i = 0; i < n; ++i)
-        {
-            map.emplace(data[i], it, inserted);
-            if (inserted)
-            {
-                new(&it->second) Value(std::move(value));
-                INIT;
-            }
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "HashMap, AlternativeHash. Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
-            << ", collisions: " << map.getCollisions()
-#endif
-            << std::endl;
-    }
-
-#if defined(__x86_64__)
-    if (argc < 3 || atoi(argv[2]) == 3)
-    {
-        Stopwatch watch;
-
-        using Map = HashMap<Key, Value, CRC32Hash_>;
-        Map map;
-        Map::iterator it;
-        bool inserted;
-
-        for (size_t i = 0; i < n; ++i)
-        {
-            map.emplace(data[i], it, inserted);
-            if (inserted)
-            {
-                new(&it->second) Value(std::move(value));
-                INIT;
-            }
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "HashMap, CRC32Hash. Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
-            << ", collisions: " << map.getCollisions()
-#endif
-            << std::endl;
-    }
-#endif
-
-    if (argc < 3 || atoi(argv[2]) == 4)
-    {
-        Stopwatch watch;
-
-        std::unordered_map<Key, Value, DefaultHash<Key>> map;
-        std::unordered_map<Key, Value, DefaultHash<Key>>::iterator it;
-        for (size_t i = 0; i < n; ++i)
-        {
-            it = map.insert(std::make_pair(data[i], std::move(value))).first;
-            INIT;
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "std::unordered_map. Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-            << std::endl;
-    }
-
-    if (argc < 3 || atoi(argv[2]) == 5)
-    {
-        Stopwatch watch;
-
-        google::dense_hash_map<Key, Value, DefaultHash<Key>> map;
-        google::dense_hash_map<Key, Value, DefaultHash<Key>>::iterator it;
-        map.set_empty_key(-1ULL);
-        for (size_t i = 0; i < n; ++i)
-        {
-            it = map.insert(std::make_pair(data[i], std::move(value))).first;
-            INIT;
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "google::dense_hash_map. Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-            << std::endl;
-    }
-
-    if (argc < 3 || atoi(argv[2]) == 6)
-    {
-        Stopwatch watch;
-
-        google::sparse_hash_map<Key, Value, DefaultHash<Key>> map;
-        google::sparse_hash_map<Key, Value, DefaultHash<Key>>::iterator it;
-        for (size_t i = 0; i < n; ++i)
-        {
-            map.insert(std::make_pair(data[i], std::move(value)));
-            INIT;
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "google::sparse_hash_map. Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-            << std::endl;
-    }
-
-    return 0;
-}
diff --git a/dbms/src/Interpreters/tests/hash_map3.cpp b/dbms/src/Interpreters/tests/hash_map3.cpp
deleted file mode 100644
index daadb8301f8..00000000000
--- a/dbms/src/Interpreters/tests/hash_map3.cpp
+++ /dev/null
@@ -1,105 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#include <iostream>
-
-#define DBMS_HASH_MAP_DEBUG_RESIZES
-#define DBMS_HASH_MAP_COUNT_COLLISIONS
-
-
-#include <string.h>
-
-#include <cstdlib>
-
-#include <utility>
-
-#include <Core/Types.h>
-#include <Common/Exception.h>
-
-#include <IO/ReadHelpers.h>
-
-#include <common/StringRef.h>
-
-#include <Common/HashTable/HashMap.h>
-
-
-template
-<
-    typename Key,
-    typename Mapped,
-    typename Hash = DefaultHash<Key>,
-    typename Grower = HashTableGrower<>,
-    typename Allocator = HashTableAllocator
->
-class HashMapWithDump : public HashMap<Key, Mapped, Hash, Grower, Allocator>
-{
-public:
-    void dump() const
-    {
-        for (size_t i = 0; i < this->grower.bufSize(); ++i)
-        {
-            if (this->buf[i].isZero(*this))
-                std::cerr << "[    ]";
-            else
-                std::cerr << '[' << this->buf[i].getValue().first.data << ", " << this->buf[i].getValue().second << ']';
-        }
-        std::cerr << std::endl;
-    }
-};
-
-
-struct SimpleHash
-{
-    size_t operator() (UInt64 x) const { return x; }
-    size_t operator() (StringRef x) const { return DB::parse<UInt64>(x.data); }
-};
-
-struct Grower : public HashTableGrower<2>
-{
-    void increaseSize()
-    {
-        ++size_degree;
-    }
-};
-
-int main(int, char **)
-{
-    using Map = HashMapWithDump<
-        StringRef,
-        UInt64,
-        SimpleHash,
-        Grower,
-        HashTableAllocatorWithStackMemory<4 * 24>>;
-
-    Map map;
-
-    map.dump();
-    std::cerr << "size: " << map.size() << std::endl;
-    map[StringRef("1", 1)] = 1;
-    map.dump();
-    std::cerr << "size: " << map.size() << std::endl;
-    map[StringRef("9", 1)] = 1;
-    map.dump();
-    std::cerr << "size: " << map.size() << std::endl;
-    std::cerr << "Collisions: " << map.getCollisions() << std::endl;
-    map[StringRef("3", 1)] = 2;
-    map.dump();
-    std::cerr << "size: " << map.size() << std::endl;
-    std::cerr << "Collisions: " << map.getCollisions() << std::endl;
-
-    for (auto x : map)
-        std::cerr << x.first.toString() << " -> " << x.second << std::endl;
-
-    return 0;
-}
diff --git a/dbms/src/Interpreters/tests/hash_map_string.cpp b/dbms/src/Interpreters/tests/hash_map_string.cpp
deleted file mode 100644
index af7717112a6..00000000000
--- a/dbms/src/Interpreters/tests/hash_map_string.cpp
+++ /dev/null
@@ -1,505 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#include <iostream>
-#include <iomanip>
-#include <vector>
-
-#include <unordered_map>
-
-#include <sparsehash/dense_hash_map>
-#include <sparsehash/sparse_hash_map>
-
-#include <Common/Stopwatch.h>
-
-//#define DBMS_HASH_MAP_COUNT_COLLISIONS
-#define DBMS_HASH_MAP_DEBUG_RESIZES
-
-#include <Core/Types.h>
-#include <IO/ReadBufferFromFile.h>
-#include <IO/ReadHelpers.h>
-#include <IO/CompressedReadBuffer.h>
-#include <common/StringRef.h>
-#include <Common/HashTable/HashMap.h>
-#include <Interpreters/AggregationCommon.h>
-
-
-struct CompactStringRef
-{
-    union
-    {
-        const char * data_mixed = nullptr;
-        struct
-        {
-            char dummy[6];
-            UInt16 size;
-        };
-    };
-
-    CompactStringRef(const char * data_, size_t size_)
-    {
-        data_mixed = data_;
-        size = size_;
-    }
-
-    CompactStringRef(const unsigned char * data_, size_t size_) : CompactStringRef(reinterpret_cast<const char *>(data_), size_) {}
-    explicit CompactStringRef(const std::string & s) : CompactStringRef(s.data(), s.size()) {}
-    CompactStringRef() {}
-
-    const char * data() const { return reinterpret_cast<const char *>(reinterpret_cast<intptr_t>(data_mixed) & 0x0000FFFFFFFFFFFFULL); }
-
-    std::string toString() const { return std::string(data(), size); }
-};
-
-inline bool operator==(CompactStringRef lhs, CompactStringRef rhs)
-{
-    if (lhs.size != rhs.size)
-        return false;
-
-    const char * lhs_data = lhs.data();
-    const char * rhs_data = rhs.data();
-    for (size_t pos = lhs.size - 1; pos < lhs.size; --pos)
-        if (lhs_data[pos] != rhs_data[pos])
-            return false;
-
-    return true;
-}
-
-namespace ZeroTraits
-{
-    template <>
-    inline bool check<CompactStringRef>(CompactStringRef x) { return nullptr == x.data_mixed; }
-
-    template <>
-    inline void set<CompactStringRef>(CompactStringRef & x) { x.data_mixed = nullptr; }
-};
-
-template <>
-struct DefaultHash<CompactStringRef>
-{
-    size_t operator() (CompactStringRef x) const
-    {
-        return CityHash_v1_0_2::CityHash64(x.data(), x.size);
-    }
-};
-
-
-#define mix(h) ({                   \
-    (h) ^= (h) >> 23;               \
-    (h) *= 0x2127599bf4325c37ULL;   \
-    (h) ^= (h) >> 47; })
-
-struct FastHash64
-{
-    size_t operator() (CompactStringRef x) const
-    {
-        const char * buf = x.data();
-        size_t len = x.size;
-
-        const UInt64    m = 0x880355f21e6d1965ULL;
-        const UInt64 *pos = reinterpret_cast<const UInt64 *>(buf);
-        const UInt64 *end = pos + (len / 8);
-        const unsigned char *pos2;
-        UInt64 h = len * m;
-        UInt64 v;
-
-        while (pos != end)
-        {
-            v = *pos++;
-            h ^= mix(v);
-            h *= m;
-        }
-
-        pos2 = reinterpret_cast<const unsigned char*>(pos);
-        v = 0;
-
-        switch (len & 7)
-        {
-            case 7: v ^= static_cast<UInt64>(pos2[6]) << 48; [[fallthrough]];
-            case 6: v ^= static_cast<UInt64>(pos2[5]) << 40; [[fallthrough]];
-            case 5: v ^= static_cast<UInt64>(pos2[4]) << 32; [[fallthrough]];
-            case 4: v ^= static_cast<UInt64>(pos2[3]) << 24; [[fallthrough]];
-            case 3: v ^= static_cast<UInt64>(pos2[2]) << 16; [[fallthrough]];
-            case 2: v ^= static_cast<UInt64>(pos2[1]) << 8; [[fallthrough]];
-            case 1: v ^= static_cast<UInt64>(pos2[0]);
-                h ^= mix(v);
-                h *= m;
-        }
-
-        return mix(h);
-    }
-};
-
-
-struct CrapWow
-{
-    size_t operator() (CompactStringRef x) const
-    {
-#if __x86_64__
-        const char * key = x.data();
-        size_t len = x.size;
-        size_t seed = 0;
-
-        const UInt64 m = 0x95b47aa3355ba1a1, n = 0x8a970be7488fda55;
-
-        UInt64 hash;
-        // 3 = m, 4 = n
-        // r12 = h, r13 = k, ecx = seed, r12 = key
-        asm(
-            "leaq (%%rcx,%4), %%r13\n"
-            "movq %%rdx, %%r14\n"
-            "movq %%rcx, %%r15\n"
-            "movq %%rcx, %%r12\n"
-            "addq %%rax, %%r13\n"
-            "andq $0xfffffffffffffff0, %%rcx\n"
-            "jz QW%=\n"
-            "addq %%rcx, %%r14\n\n"
-            "negq %%rcx\n"
-        "XW%=:\n"
-            "movq %4, %%rax\n"
-            "mulq (%%r14,%%rcx)\n"
-            "xorq %%rax, %%r12\n"
-            "xorq %%rdx, %%r13\n"
-            "movq %3, %%rax\n"
-            "mulq 8(%%r14,%%rcx)\n"
-            "xorq %%rdx, %%r12\n"
-            "xorq %%rax, %%r13\n"
-            "addq $16, %%rcx\n"
-            "jnz XW%=\n"
-        "QW%=:\n"
-            "movq %%r15, %%rcx\n"
-            "andq $8, %%r15\n"
-            "jz B%=\n"
-            "movq %4, %%rax\n"
-            "mulq (%%r14)\n"
-            "addq $8, %%r14\n"
-            "xorq %%rax, %%r12\n"
-            "xorq %%rdx, %%r13\n"
-        "B%=:\n"
-            "andq $7, %%rcx\n"
-            "jz F%=\n"
-            "movq $1, %%rdx\n"
-            "shlq $3, %%rcx\n"
-            "movq %3, %%rax\n"
-            "shlq %%cl, %%rdx\n"
-            "addq $-1, %%rdx\n"
-            "andq (%%r14), %%rdx\n"
-            "mulq %%rdx\n"
-            "xorq %%rdx, %%r12\n"
-            "xorq %%rax, %%r13\n"
-        "F%=:\n"
-            "leaq (%%r13,%4), %%rax\n"
-            "xorq %%r12, %%rax\n"
-            "mulq %4\n"
-            "xorq %%rdx, %%rax\n"
-            "xorq %%r12, %%rax\n"
-            "xorq %%r13, %%rax\n"
-            : "=a"(hash), "=c"(key), "=d"(key)
-            : "r"(m), "r"(n), "a"(seed), "c"(len), "d"(key)
-            : "%r12", "%r13", "%r14", "%r15", "cc"
-        );
-        return hash;
-#else
-        return 0;
-#endif
-    }
-};
-
-
-struct SimpleHash
-{
-    size_t operator() (CompactStringRef x) const
-    {
-        const char * pos = x.data();
-        size_t size = x.size;
-
-        const char * end = pos + size;
-
-        size_t res = 0;
-
-        if (size == 0)
-            return 0;
-
-        if (size < 8)
-        {
-            memcpy(reinterpret_cast<char *>(&res), pos, size);
-            return intHash64(res);
-        }
-
-        while (pos + 8 < end)
-        {
-            UInt64 word = *reinterpret_cast<const UInt64 *>(pos);
-            res = intHash64(word ^ res);
-
-            pos += 8;
-        }
-
-        UInt64 word = *reinterpret_cast<const UInt64 *>(end - 8);
-        res = intHash64(word ^ res);
-
-        return res;
-    }
-};
-
-
-using Key = CompactStringRef;
-using Value = UInt64;
-
-
-struct Grower : public HashTableGrower<>
-{
-    /// The state of this structure is enough to get the buffer size of the hash table.
-
-    /// Defines the initial size of the hash table.
-    static const size_t initial_size_degree = 16;
-    Grower() { size_degree = initial_size_degree; }
-
-    size_t max_fill = (1ULL << initial_size_degree) * 0.9;
-
-    /// The size of the hash table in the cells.
-    size_t bufSize() const               { return 1ULL << size_degree; }
-
-    size_t maxFill() const               { return max_fill /*1 << (size_degree - 1)*/; }
-    size_t mask() const                  { return bufSize() - 1; }
-
-    /// From the hash value, get the cell number in the hash table.
-    size_t place(size_t x) const         { return x & mask(); }
-
-    /// The next cell in the collision resolution chain.
-    size_t next(size_t pos) const        { ++pos; return pos & mask(); }
-
-    /// Whether the hash table is sufficiently full. You need to increase the size of the hash table, or remove something unnecessary from it.
-    bool overflow(size_t elems) const    { return elems > maxFill(); }
-
-    /// Increase the size of the hash table.
-    void increaseSize()
-    {
-        size_degree += size_degree >= 23 ? 1 : 2;
-        max_fill = (1ULL << size_degree) * 0.9;
-    }
-
-    /// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table.
-    void set(size_t /*num_elems*/)
-    {
-        throw Poco::Exception(__PRETTY_FUNCTION__);
-    }
-};
-
-
-int main(int argc, char ** argv)
-{
-    if (argc < 3)
-    {
-        std::cerr << "Usage: program n m\n";
-        return 1;
-    }
-
-    size_t n = atoi(argv[1]);
-    size_t m = atoi(argv[2]);
-
-    DB::Arena pool;
-    std::vector<Key> data(n);
-
-    std::cerr << "sizeof(Key) = " << sizeof(Key) << ", sizeof(Value) = " << sizeof(Value) << std::endl;
-
-    {
-        Stopwatch watch;
-        DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO);
-        DB::CompressedReadBuffer in2(in1);
-
-        std::string tmp;
-        for (size_t i = 0; i < n && !in2.eof(); ++i)
-        {
-            DB::readStringBinary(tmp, in2);
-            data[i] = Key(pool.insert(tmp.data(), tmp.size()), tmp.size());
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "Vector. Size: " << n
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-            << std::endl;
-    }
-
-    if (!m || m == 1)
-    {
-        Stopwatch watch;
-
-        //using Map = HashMap<Key, Value>;
-
-        /// Saving the hash accelerates the resize by about 2 times, and the overall performance by 6-8%.
-        using Map = HashMapWithSavedHash<Key, Value, DefaultHash<Key>, Grower>;
-
-        Map map;
-        Map::iterator it;
-        bool inserted;
-
-        for (size_t i = 0; i < n; ++i)
-        {
-            map.emplace(data[i], it, inserted);
-            if (inserted)
-                it->second = 0;
-            ++it->second;
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "HashMap (CityHash64). Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
-            << ", collisions: " << map.getCollisions()
-#endif
-            << std::endl;
-    }
-
-    if (!m || m == 2)
-    {
-        Stopwatch watch;
-
-        using Map = HashMapWithSavedHash<Key, Value, FastHash64, Grower>;
-
-        Map map;
-        Map::iterator it;
-        bool inserted;
-
-        for (size_t i = 0; i < n; ++i)
-        {
-            map.emplace(data[i], it, inserted);
-            if (inserted)
-                it->second = 0;
-            ++it->second;
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "HashMap (FastHash64). Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
-            << ", collisions: " << map.getCollisions()
-#endif
-            << std::endl;
-    }
-
-    if (!m || m == 3)
-    {
-        Stopwatch watch;
-
-        using Map = HashMapWithSavedHash<Key, Value, CrapWow, Grower>;
-
-        Map map;
-        Map::iterator it;
-        bool inserted;
-
-        for (size_t i = 0; i < n; ++i)
-        {
-            map.emplace(data[i], it, inserted);
-            if (inserted)
-                it->second = 0;
-            ++it->second;
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "HashMap (CrapWow). Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
-            << ", collisions: " << map.getCollisions()
-#endif
-            << std::endl;
-    }
-
-    if (!m || m == 4)
-    {
-        Stopwatch watch;
-
-        using Map = HashMapWithSavedHash<Key, Value, SimpleHash, Grower>;
-
-        Map map;
-        Map::iterator it;
-        bool inserted;
-
-        for (size_t i = 0; i < n; ++i)
-        {
-            map.emplace(data[i], it, inserted);
-            if (inserted)
-                it->second = 0;
-            ++it->second;
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "HashMap (SimpleHash). Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
-            << ", collisions: " << map.getCollisions()
-#endif
-            << std::endl;
-    }
-
-    if (!m || m == 5)
-    {
-        Stopwatch watch;
-
-        std::unordered_map<Key, Value, DefaultHash<Key>> map;
-        for (size_t i = 0; i < n; ++i)
-            ++map[data[i]];
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "std::unordered_map. Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-            << std::endl;
-    }
-
-    if (!m || m == 6)
-    {
-        Stopwatch watch;
-
-        google::dense_hash_map<Key, Value, DefaultHash<Key>> map;
-        map.set_empty_key(Key("\0", 1));
-        for (size_t i = 0; i < n; ++i)
-              ++map[data[i]];
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "google::dense_hash_map. Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-            << std::endl;
-    }
-
-    if (!m || m == 7)
-    {
-        Stopwatch watch;
-
-        google::sparse_hash_map<Key, Value, DefaultHash<Key>> map;
-        for (size_t i = 0; i < n; ++i)
-            ++map[data[i]];
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "google::sparse_hash_map. Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-            << std::endl;
-    }
-
-    return 0;
-}
diff --git a/dbms/src/Interpreters/tests/hash_map_string_2.cpp b/dbms/src/Interpreters/tests/hash_map_string_2.cpp
deleted file mode 100644
index e073084aa8c..00000000000
--- a/dbms/src/Interpreters/tests/hash_map_string_2.cpp
+++ /dev/null
@@ -1,686 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#include <iostream>
-#include <iomanip>
-#include <vector>
-
-#include <Common/Stopwatch.h>
-
-//#define DBMS_HASH_MAP_COUNT_COLLISIONS
-#define DBMS_HASH_MAP_DEBUG_RESIZES
-
-#include <Core/Types.h>
-#include <IO/ReadBufferFromFile.h>
-#include <IO/ReadHelpers.h>
-#include <IO/CompressedReadBuffer.h>
-#include <common/StringRef.h>
-#include <Common/HashTable/HashMap.h>
-#include <Interpreters/AggregationCommon.h>
-
-#if __SSE4_1__
-    #include <smmintrin.h>
-#endif
-
-
-/** Do this:
-for file in MobilePhoneModel PageCharset Params URLDomain UTMSource Referer URL Title; do
- for size in 30000 100000 300000 1000000 5000000; do
-  echo
-  BEST_METHOD=0
-  BEST_RESULT=0
-  for method in {1..12}; do
-   echo -ne $file $size $method '';
-   TOTAL_ELEMS=0
-   for i in {0..1000}; do
-    TOTAL_ELEMS=$(( $TOTAL_ELEMS + $size ))
-    if [[ $TOTAL_ELEMS -gt 25000000 ]]; then break; fi
-    ./hash_map_string_2 $size $method < ${file}.bin 2>&1 |
-     grep HashMap | grep -oE '[0-9\.]+ elem';
-   done | awk -W interactive '{ if ($1 > x) { x = $1 }; printf(".") } END { print x }' | tee /tmp/hash_map_string_2_res;
-   CUR_RESULT=$(cat /tmp/hash_map_string_2_res | tr -d '.')
-   if [[ $CUR_RESULT -gt $BEST_RESULT ]]; then
-    BEST_METHOD=$method
-    BEST_RESULT=$CUR_RESULT
-   fi;
-  done;
-  echo Best: $BEST_METHOD - $BEST_RESULT
- done;
-done
-*/
-
-
-#define DefineStringRef(STRUCT) \
-\
-struct STRUCT : public StringRef {}; \
-\
-namespace ZeroTraits \
-{ \
-    template <> \
-    inline bool check<STRUCT>(STRUCT x) { return 0 == x.size; } \
- \
-    template <> \
-    inline void set<STRUCT>(STRUCT & x) { x.size = 0; } \
-}; \
- \
-template <> \
-struct DefaultHash<STRUCT> \
-{ \
-    size_t operator() (STRUCT x) const \
-    { \
-        return CityHash_v1_0_2::CityHash64(x.data, x.size);  \
-    } \
-};
-
-DefineStringRef(StringRef_Compare1_Ptrs)
-DefineStringRef(StringRef_Compare1_Index)
-DefineStringRef(StringRef_CompareMemcmp)
-DefineStringRef(StringRef_Compare8_1_byUInt64)
-DefineStringRef(StringRef_Compare16_1_byMemcmp)
-DefineStringRef(StringRef_Compare16_1_byUInt64_logicAnd)
-DefineStringRef(StringRef_Compare16_1_byUInt64_bitAnd)
-
-#if __SSE4_1__
-DefineStringRef(StringRef_Compare16_1_byIntSSE)
-DefineStringRef(StringRef_Compare16_1_byFloatSSE)
-DefineStringRef(StringRef_Compare16_1_bySSE4)
-DefineStringRef(StringRef_Compare16_1_bySSE4_wide)
-DefineStringRef(StringRef_Compare16_1_bySSE_wide)
-#endif
-
-DefineStringRef(StringRef_CompareAlwaysTrue)
-DefineStringRef(StringRef_CompareAlmostAlwaysTrue)
-
-
-inline bool operator==(StringRef_Compare1_Ptrs lhs, StringRef_Compare1_Ptrs rhs)
-{
-    if (lhs.size != rhs.size)
-        return false;
-
-    if (lhs.size == 0)
-        return true;
-
-    const char * pos1 = lhs.data;
-    const char * pos2 = rhs.data;
-
-    const char * end1 = pos1 + lhs.size;
-
-    while (pos1 < end1)
-    {
-        if (*pos1 != *pos2)
-            return false;
-
-        ++pos1;
-        ++pos2;
-    }
-
-    return true;
-}
-
-inline bool operator==(StringRef_Compare1_Index lhs, StringRef_Compare1_Index rhs)
-{
-    if (lhs.size != rhs.size)
-        return false;
-
-    if (lhs.size == 0)
-        return true;
-
-    for (size_t i = 0; i < lhs.size; ++i)
-        if (lhs.data[i] != rhs.data[i])
-            return false;
-
-    return true;
-}
-
-inline bool operator==(StringRef_CompareMemcmp lhs, StringRef_CompareMemcmp rhs)
-{
-    if (lhs.size != rhs.size)
-        return false;
-
-    if (lhs.size == 0)
-        return true;
-
-    return 0 == memcmp(lhs.data, rhs.data, lhs.size);
-}
-
-
-inline bool operator==(StringRef_Compare8_1_byUInt64 lhs, StringRef_Compare8_1_byUInt64 rhs)
-{
-    if (lhs.size != rhs.size)
-        return false;
-
-    if (lhs.size == 0)
-        return true;
-
-    const char * p1 = lhs.data;
-    const char * p2 = rhs.data;
-    size_t size = lhs.size;
-
-    const char * p1_end = p1 + size;
-    const char * p1_end_8 = p1 + size / 8 * 8;
-
-    while (p1 < p1_end_8)
-    {
-        if (reinterpret_cast<const UInt64 *>(p1)[0] != reinterpret_cast<const UInt64 *>(p2)[0])
-            return false;
-
-        p1 += 8;
-        p2 += 8;
-    }
-
-    while (p1 < p1_end)
-    {
-        if (*p1 != *p2)
-            return false;
-
-        ++p1;
-        ++p2;
-    }
-
-    return true;
-}
-
-
-
-inline bool compare_byMemcmp(const char * p1, const char * p2)
-{
-    return 0 == memcmp(p1, p2, 16);
-}
-
-inline bool compare_byUInt64_logicAnd(const char * p1, const char * p2)
-{
-    return reinterpret_cast<const UInt64 *>(p1)[0] == reinterpret_cast<const UInt64 *>(p2)[0]
-        && reinterpret_cast<const UInt64 *>(p1)[1] == reinterpret_cast<const UInt64 *>(p2)[1];
-}
-
-inline bool compare_byUInt64_bitAnd(const char * p1, const char * p2)
-{
-    return (reinterpret_cast<const UInt64 *>(p1)[0] == reinterpret_cast<const UInt64 *>(p2)[0])
-         & (reinterpret_cast<const UInt64 *>(p1)[1] == reinterpret_cast<const UInt64 *>(p2)[1]);
-}
-
-#if __SSE4_1__
-
-inline bool compare_byIntSSE(const char * p1, const char * p2)
-{
-    return 0xFFFF == _mm_movemask_epi8(_mm_cmpeq_epi8(
-        _mm_loadu_si128(reinterpret_cast<const __m128i *>(p1)),
-        _mm_loadu_si128(reinterpret_cast<const __m128i *>(p2))));
-}
-
-inline bool compare_byFloatSSE(const char * p1, const char * p2)
-{
-    return !_mm_movemask_ps(_mm_cmpneq_ps(                    /// Looks like incorrect while comparing subnormal floats.
-        _mm_loadu_ps(reinterpret_cast<const float *>(p1)),
-        _mm_loadu_ps(reinterpret_cast<const float *>(p2))));
-}
-
-#endif
-
-
-template <bool compare(const char *, const char *)>
-inline bool memequal(const char * p1, const char * p2, size_t size)
-{
-//    const char * p1_end = p1 + size;
-    const char * p1_end_16 = p1 + size / 16 * 16;
-
-    while (p1 < p1_end_16)
-    {
-        if (!compare(p1, p2))
-            return false;
-
-        p1 += 16;
-        p2 += 16;
-    }
-
-/*    while (p1 < p1_end)
-    {
-        if (*p1 != *p2)
-            return false;
-
-        ++p1;
-        ++p2;
-    }*/
-
-    switch (size % 16)
-    {
-        case 15: if (p1[14] != p2[14]) return false; [[fallthrough]];
-        case 14: if (p1[13] != p2[13]) return false; [[fallthrough]];
-        case 13: if (p1[12] != p2[12]) return false; [[fallthrough]];
-        case 12: if (reinterpret_cast<const UInt32 *>(p1)[2] == reinterpret_cast<const UInt32 *>(p2)[2]) goto l8; else return false;
-        case 11: if (p1[10] != p2[10]) return false; [[fallthrough]];
-        case 10: if (p1[9] != p2[9]) return false; [[fallthrough]];
-        case 9:  if (p1[8] != p2[8]) return false;
-    l8: [[fallthrough]];
-        case 8:  return reinterpret_cast<const UInt64 *>(p1)[0] == reinterpret_cast<const UInt64 *>(p2)[0];
-        case 7:  if (p1[6] != p2[6]) return false; [[fallthrough]];
-        case 6:  if (p1[5] != p2[5]) return false; [[fallthrough]];
-        case 5:  if (p1[4] != p2[4]) return false; [[fallthrough]];
-        case 4:  return reinterpret_cast<const UInt32 *>(p1)[0] == reinterpret_cast<const UInt32 *>(p2)[0];
-        case 3:  if (p1[2] != p2[2]) return false; [[fallthrough]];
-        case 2:  return reinterpret_cast<const UInt16 *>(p1)[0] == reinterpret_cast<const UInt16 *>(p2)[0];
-        case 1:  if (p1[0] != p2[0]) return false; [[fallthrough]];
-        case 0:  break;
-    }
-
-    return true;
-}
-
-
-#if __SSE4_1__
-
-inline bool memequal_sse41(const char * p1, const char * p2, size_t size)
-{
-//    const char * p1_end = p1 + size;
-    const char * p1_end_16 = p1 + size / 16 * 16;
-
-    __m128i zero16 = _mm_setzero_si128();
-
-    while (p1 < p1_end_16)
-    {
-        if (!_mm_testc_si128(
-            zero16,
-            _mm_xor_si128(
-                _mm_loadu_si128(reinterpret_cast<const __m128i *>(p1)),
-                _mm_loadu_si128(reinterpret_cast<const __m128i *>(p2)))))
-            return false;
-
-        p1 += 16;
-        p2 += 16;
-    }
-
-/*    while (p1 < p1_end)
-    {
-        if (*p1 != *p2)
-            return false;
-
-        ++p1;
-        ++p2;
-    }*/
-
-    switch (size % 16)
-    {
-        case 15: if (p1[14] != p2[14]) return false; [[fallthrough]];
-        case 14: if (p1[13] != p2[13]) return false; [[fallthrough]];
-        case 13: if (p1[12] != p2[12]) return false; [[fallthrough]];
-        case 12: if (reinterpret_cast<const UInt32 *>(p1)[2] == reinterpret_cast<const UInt32 *>(p2)[2]) goto l8; else return false;
-        case 11: if (p1[10] != p2[10]) return false; [[fallthrough]];
-        case 10: if (p1[9] != p2[9]) return false; [[fallthrough]];
-        case 9:  if (p1[8] != p2[8]) return false;
-    l8: [[fallthrough]];
-        case 8:  return reinterpret_cast<const UInt64 *>(p1)[0] == reinterpret_cast<const UInt64 *>(p2)[0];
-        case 7:  if (p1[6] != p2[6]) return false; [[fallthrough]];
-        case 6:  if (p1[5] != p2[5]) return false; [[fallthrough]];
-        case 5:  if (p1[4] != p2[4]) return false; [[fallthrough]];
-        case 4:  return reinterpret_cast<const UInt32 *>(p1)[0] == reinterpret_cast<const UInt32 *>(p2)[0];
-        case 3:  if (p1[2] != p2[2]) return false; [[fallthrough]];
-        case 2:  return reinterpret_cast<const UInt16 *>(p1)[0] == reinterpret_cast<const UInt16 *>(p2)[0];
-        case 1:  if (p1[0] != p2[0]) return false; [[fallthrough]];
-        case 0:  break;
-    }
-
-    return true;
-}
-
-
-inline bool memequal_sse41_wide(const char * p1, const char * p2, size_t size)
-{
-    __m128i zero16 = _mm_setzero_si128();
-//    const char * p1_end = p1 + size;
-
-    while (size >= 64)
-    {
-        if (_mm_testc_si128(
-                zero16,
-                _mm_xor_si128(
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p1)[0]),
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p2)[0])))
-            && _mm_testc_si128(
-                zero16,
-                _mm_xor_si128(
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p1)[1]),
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p2)[1])))
-            && _mm_testc_si128(
-                zero16,
-                _mm_xor_si128(
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p1)[2]),
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p2)[2])))
-            && _mm_testc_si128(
-                zero16,
-                _mm_xor_si128(
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p1)[3]),
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p2)[3]))))
-        {
-            p1 += 64;
-            p2 += 64;
-            size -= 64;
-        }
-        else
-            return false;
-    }
-
-    switch ((size % 64) / 16)
-    {
-        case 3:
-            if (!_mm_testc_si128(
-                zero16,
-                _mm_xor_si128(
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p1)[2]),
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p2)[2]))))
-                return false;
-            [[fallthrough]];
-        case 2:
-            if (!_mm_testc_si128(
-                zero16,
-                _mm_xor_si128(
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p1)[1]),
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p2)[1]))))
-                return false;
-            [[fallthrough]];
-        case 1:
-            if (!_mm_testc_si128(
-                zero16,
-                _mm_xor_si128(
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p1)[0]),
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p2)[0]))))
-                return false;
-    }
-
-    p1 += (size % 64) / 16 * 16;
-    p2 += (size % 64) / 16 * 16;
-
-/*
-
-    if (size >= 32)
-    {
-        if (_mm_testc_si128(
-                zero16,
-                _mm_xor_si128(
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p1)[0]),
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p2)[0])))
-            & _mm_testc_si128(
-                zero16,
-                _mm_xor_si128(
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p1)[1]),
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p2)[1]))))
-        {
-            p1 += 32;
-            p2 += 32;
-            size -= 32;
-        }
-        else
-            return false;
-    }
-
-    if (size >= 16)
-    {
-        if (_mm_testc_si128(
-                zero16,
-                _mm_xor_si128(
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p1)[0]),
-                    _mm_loadu_si128(&reinterpret_cast<const __m128i *>(p2)[0]))))
-        {
-            p1 += 16;
-            p2 += 16;
-            size -= 16;
-        }
-        else
-            return false;
-    }*/
-
-    switch (size % 16)
-    {
-        case 15: if (p1[14] != p2[14]) return false; [[fallthrough]];
-        case 14: if (p1[13] != p2[13]) return false; [[fallthrough]];
-        case 13: if (p1[12] != p2[12]) return false; [[fallthrough]];
-        case 12: if (reinterpret_cast<const UInt32 *>(p1)[2] == reinterpret_cast<const UInt32 *>(p2)[2]) goto l8; else return false;
-        case 11: if (p1[10] != p2[10]) return false; [[fallthrough]];
-        case 10: if (p1[9] != p2[9]) return false; [[fallthrough]];
-        case 9:  if (p1[8] != p2[8]) return false;
-    l8: [[fallthrough]];
-        case 8:  return reinterpret_cast<const UInt64 *>(p1)[0] == reinterpret_cast<const UInt64 *>(p2)[0];
-        case 7:  if (p1[6] != p2[6]) return false; [[fallthrough]];
-        case 6:  if (p1[5] != p2[5]) return false; [[fallthrough]];
-        case 5:  if (p1[4] != p2[4]) return false; [[fallthrough]];
-        case 4:  return reinterpret_cast<const UInt32 *>(p1)[0] == reinterpret_cast<const UInt32 *>(p2)[0];
-        case 3:  if (p1[2] != p2[2]) return false; [[fallthrough]];
-        case 2:  return reinterpret_cast<const UInt16 *>(p1)[0] == reinterpret_cast<const UInt16 *>(p2)[0];
-        case 1:  if (p1[0] != p2[0]) return false; [[fallthrough]];
-        case 0:  break;
-    }
-
-    return true;
-}
-
-
-inline bool memequal_sse_wide(const char * p1, const char * p2, size_t size)
-{
-    while (size >= 64)
-    {
-        if (   compare_byIntSSE(p1,      p2)
-            && compare_byIntSSE(p1 + 16, p2 + 16)
-            && compare_byIntSSE(p1 + 32, p2 + 32)
-            && compare_byIntSSE(p1 + 48, p2 + 48))
-        {
-            p1 += 64;
-            p2 += 64;
-            size -= 64;
-        }
-        else
-            return false;
-    }
-
-    switch ((size % 64) / 16)
-    {
-        case 3: if (!compare_byIntSSE(p1 + 32, p2 + 32)) return false; [[fallthrough]];
-        case 2: if (!compare_byIntSSE(p1 + 16, p2 + 16)) return false; [[fallthrough]];
-        case 1: if (!compare_byIntSSE(p1     , p2     )) return false;
-    }
-
-    p1 += (size % 64) / 16 * 16;
-    p2 += (size % 64) / 16 * 16;
-
-    switch (size % 16)
-    {
-        case 15: if (p1[14] != p2[14]) return false; [[fallthrough]];
-        case 14: if (p1[13] != p2[13]) return false; [[fallthrough]];
-        case 13: if (p1[12] != p2[12]) return false; [[fallthrough]];
-        case 12: if (reinterpret_cast<const UInt32 *>(p1)[2] == reinterpret_cast<const UInt32 *>(p2)[2]) goto l8; else return false;
-        case 11: if (p1[10] != p2[10]) return false; [[fallthrough]];
-        case 10: if (p1[9] != p2[9]) return false; [[fallthrough]];
-        case 9:  if (p1[8] != p2[8]) return false;
-    l8: [[fallthrough]];
-        case 8:  return reinterpret_cast<const UInt64 *>(p1)[0] == reinterpret_cast<const UInt64 *>(p2)[0];
-        case 7:  if (p1[6] != p2[6]) return false; [[fallthrough]];
-        case 6:  if (p1[5] != p2[5]) return false; [[fallthrough]];
-        case 5:  if (p1[4] != p2[4]) return false; [[fallthrough]];
-        case 4:  return reinterpret_cast<const UInt32 *>(p1)[0] == reinterpret_cast<const UInt32 *>(p2)[0];
-        case 3:  if (p1[2] != p2[2]) return false; [[fallthrough]];
-        case 2:  return reinterpret_cast<const UInt16 *>(p1)[0] == reinterpret_cast<const UInt16 *>(p2)[0];
-        case 1:  if (p1[0] != p2[0]) return false; [[fallthrough]];
-        case 0:  break;
-    }
-
-    return true;
-}
-
-#endif
-
-
-#define Op(METHOD) \
-inline bool operator==(StringRef_Compare16_1_ ## METHOD lhs, StringRef_Compare16_1_ ## METHOD rhs) \
-{ \
-    if (lhs.size != rhs.size) \
-        return false; \
-\
-    if (lhs.size == 0) \
-        return true; \
-\
-    return memequal<compare_  ## METHOD>(lhs.data, rhs.data, lhs.size); \
-}
-
-Op(byMemcmp)
-Op(byUInt64_logicAnd)
-Op(byUInt64_bitAnd)
-
-#if __SSE4_1__
-
-Op(byIntSSE)
-Op(byFloatSSE)
-
-
-inline bool operator==(StringRef_Compare16_1_bySSE4 lhs, StringRef_Compare16_1_bySSE4 rhs)
-{
-    if (lhs.size != rhs.size)
-        return false;
-
-    if (lhs.size == 0)
-        return true;
-
-    return memequal_sse41(lhs.data, rhs.data, lhs.size);
-}
-
-inline bool operator==(StringRef_Compare16_1_bySSE4_wide lhs, StringRef_Compare16_1_bySSE4_wide rhs)
-{
-    if (lhs.size != rhs.size)
-        return false;
-
-    if (lhs.size == 0)
-        return true;
-
-    return memequal_sse41_wide(lhs.data, rhs.data, lhs.size);
-}
-
-inline bool operator==(StringRef_Compare16_1_bySSE_wide lhs, StringRef_Compare16_1_bySSE_wide rhs)
-{
-    if (lhs.size != rhs.size)
-        return false;
-
-    if (lhs.size == 0)
-        return true;
-
-    return memequal_sse_wide(lhs.data, rhs.data, lhs.size);
-}
-
-#endif
-
-
-inline bool operator==(StringRef_CompareAlwaysTrue, StringRef_CompareAlwaysTrue)
-{
-    return true;
-}
-
-inline bool operator==(StringRef_CompareAlmostAlwaysTrue lhs, StringRef_CompareAlmostAlwaysTrue rhs)
-{
-    if (lhs.size != rhs.size)
-        return false;
-
-    return true;
-}
-
-
-
-using Value = UInt64;
-
-
-template <typename Key>
-void NO_INLINE bench(const std::vector<StringRef> & data, const char * name)
-{
-    Stopwatch watch;
-
-    using Map = HashMapWithSavedHash<Key, Value, DefaultHash<Key>>;
-
-    Map map;
-    typename Map::iterator it;
-    bool inserted;
-
-    for (size_t i = 0, size = data.size(); i < size; ++i)
-    {
-        map.emplace(static_cast<const Key &>(data[i]), it, inserted);
-        if (inserted)
-            it->second = 0;
-        ++it->second;
-    }
-
-    watch.stop();
-    std::cerr << std::fixed << std::setprecision(2)
-        << "HashMap (" << name << "). Size: " << map.size()
-        << ", elapsed: " << watch.elapsedSeconds()
-        << " (" << data.size() / watch.elapsedSeconds() << " elem/sec.)"
-#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
-        << ", collisions: " << map.getCollisions()
-#endif
-        << std::endl;
-}
-
-
-int main(int argc, char ** argv)
-{
-    if (argc < 3)
-    {
-        std::cerr << "Usage: program n m\n";
-        return 1;
-    }
-
-    size_t n = atoi(argv[1]);
-    size_t m = atoi(argv[2]);
-
-    DB::Arena pool;
-    std::vector<StringRef> data(n);
-
-    std::cerr << "sizeof(Key) = " << sizeof(StringRef) << ", sizeof(Value) = " << sizeof(Value) << std::endl;
-
-    {
-        Stopwatch watch;
-        DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO);
-        DB::CompressedReadBuffer in2(in1);
-
-        std::string tmp;
-        for (size_t i = 0; i < n && !in2.eof(); ++i)
-        {
-            DB::readStringBinary(tmp, in2);
-            data[i] = StringRef(pool.insert(tmp.data(), tmp.size()), tmp.size());
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "Vector. Size: " << n
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-            << std::endl;
-    }
-
-    if (!m || m == 1) bench<StringRef_Compare1_Ptrs>                (data, "StringRef_Compare1_Ptrs");
-    if (!m || m == 2) bench<StringRef_Compare1_Index>               (data, "StringRef_Compare1_Index");
-    if (!m || m == 3) bench<StringRef_CompareMemcmp>                (data, "StringRef_CompareMemcmp");
-    if (!m || m == 4) bench<StringRef_Compare8_1_byUInt64>          (data, "StringRef_Compare8_1_byUInt64");
-    if (!m || m == 5) bench<StringRef_Compare16_1_byMemcmp>         (data, "StringRef_Compare16_1_byMemcmp");
-    if (!m || m == 6) bench<StringRef_Compare16_1_byUInt64_logicAnd>(data, "StringRef_Compare16_1_byUInt64_logicAnd");
-    if (!m || m == 7) bench<StringRef_Compare16_1_byUInt64_bitAnd>  (data, "StringRef_Compare16_1_byUInt64_bitAnd");
-#if __SSE4_1__
-    if (!m || m == 8) bench<StringRef_Compare16_1_byIntSSE>         (data, "StringRef_Compare16_1_byIntSSE");
-    if (!m || m == 9) bench<StringRef_Compare16_1_byFloatSSE>       (data, "StringRef_Compare16_1_byFloatSSE");
-    if (!m || m == 10) bench<StringRef_Compare16_1_bySSE4>          (data, "StringRef_Compare16_1_bySSE4");
-    if (!m || m == 11) bench<StringRef_Compare16_1_bySSE4_wide>     (data, "StringRef_Compare16_1_bySSE4_wide");
-    if (!m || m == 12) bench<StringRef_Compare16_1_bySSE_wide>      (data, "StringRef_Compare16_1_bySSE_wide");
-#endif
-    if (!m || m == 100) bench<StringRef_CompareAlwaysTrue>          (data, "StringRef_CompareAlwaysTrue");
-    if (!m || m == 101) bench<StringRef_CompareAlmostAlwaysTrue>    (data, "StringRef_CompareAlmostAlwaysTrue");
-
-    /// 10 > 8, 9
-    /// 1, 2, 5 - bad
-
-
-    return 0;
-}
diff --git a/dbms/src/Interpreters/tests/hash_map_string_3.cpp b/dbms/src/Interpreters/tests/hash_map_string_3.cpp
deleted file mode 100644
index 1d6be35a5b1..00000000000
--- a/dbms/src/Interpreters/tests/hash_map_string_3.cpp
+++ /dev/null
@@ -1,525 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#include <iostream>
-#include <iomanip>
-#include <vector>
-
-#include <Common/Stopwatch.h>
-
-#include <farmhash.h>
-#include <metrohash.h>
-
-#define DBMS_HASH_MAP_COUNT_COLLISIONS
-#define DBMS_HASH_MAP_DEBUG_RESIZES
-
-#include <Core/Types.h>
-#include <IO/ReadBufferFromFile.h>
-#include <IO/ReadHelpers.h>
-#include <IO/CompressedReadBuffer.h>
-#include <common/StringRef.h>
-#include <Common/HashTable/HashMap.h>
-#include <Interpreters/AggregationCommon.h>
-
-#if __SSE4_1__
-    #include <smmintrin.h>
-#endif
-
-
-/** Do this:
-for file in MobilePhoneModel PageCharset Params URLDomain UTMSource Referer URL Title; do
- for size in 30000 100000 300000 1000000 5000000; do
-  echo
-  BEST_METHOD=0
-  BEST_RESULT=0
-  for method in {1..11}; do
-   echo -ne $file $size $method '';
-   TOTAL_ELEMS=0
-   for i in {0..1000}; do
-    TOTAL_ELEMS=$(( $TOTAL_ELEMS + $size ))
-    if [[ $TOTAL_ELEMS -gt 25000000 ]]; then break; fi
-    ./hash_map_string_3 $size $method < ${file}.bin 2>&1 |
-     grep HashMap | grep -oE '[0-9\.]+ elem';
-   done | awk -W interactive '{ if ($1 > x) { x = $1 }; printf(".") } END { print x }' | tee /tmp/hash_map_string_3_res;
-   CUR_RESULT=$(cat /tmp/hash_map_string_3_res | tr -d '.')
-   if [[ $CUR_RESULT -gt $BEST_RESULT ]]; then
-    BEST_METHOD=$method
-    BEST_RESULT=$CUR_RESULT
-   fi;
-  done;
-  echo Best: $BEST_METHOD - $BEST_RESULT
- done;
-done
-*/
-
-
-#define DefineStringRef(STRUCT) \
-\
-struct STRUCT : public StringRef {}; \
-\
-namespace ZeroTraits \
-{ \
-    template <> \
-    inline bool check<STRUCT>(STRUCT x) { return nullptr == x.data; } \
- \
-    template <> \
-    inline void set<STRUCT>(STRUCT & x) { x.data = nullptr; } \
-}; \
- \
-template <> \
-struct DefaultHash<STRUCT> \
-{ \
-    size_t operator() (STRUCT x) const \
-    { \
-        return CityHash_v1_0_2::CityHash64(x.data, x.size);  \
-    } \
-};
-
-
-DefineStringRef(StringRef_CompareMemcmp)
-DefineStringRef(StringRef_CompareAlwaysTrue)
-
-
-inline bool operator==(StringRef_CompareMemcmp lhs, StringRef_CompareMemcmp rhs)
-{
-    if (lhs.size != rhs.size)
-        return false;
-
-    if (lhs.size == 0)
-        return true;
-
-    return 0 == memcmp(lhs.data, rhs.data, lhs.size);
-}
-
-inline bool operator==(StringRef_CompareAlwaysTrue, StringRef_CompareAlwaysTrue)
-{
-    return true;
-}
-
-
-#define mix(h) ({                   \
-    (h) ^= (h) >> 23;               \
-    (h) *= 0x2127599bf4325c37ULL;   \
-    (h) ^= (h) >> 47; })
-
-struct FastHash64
-{
-    size_t operator() (StringRef x) const
-    {
-        const char * buf = x.data;
-        size_t len = x.size;
-
-        const uint64_t    m = 0x880355f21e6d1965ULL;
-        const uint64_t *pos = reinterpret_cast<const uint64_t *>(buf);
-        const uint64_t *end = pos + (len / 8);
-        const unsigned char *pos2;
-        uint64_t h = len * m;
-        uint64_t v;
-
-        while (pos != end)
-        {
-            v = *pos++;
-            h ^= mix(v);
-            h *= m;
-        }
-
-        pos2 = reinterpret_cast<const unsigned char*>(pos);
-        v = 0;
-
-        switch (len & 7)
-        {
-            case 7: v ^= static_cast<uint64_t>(pos2[6]) << 48; [[fallthrough]];
-            case 6: v ^= static_cast<uint64_t>(pos2[5]) << 40; [[fallthrough]];
-            case 5: v ^= static_cast<uint64_t>(pos2[4]) << 32; [[fallthrough]];
-            case 4: v ^= static_cast<uint64_t>(pos2[3]) << 24; [[fallthrough]];
-            case 3: v ^= static_cast<uint64_t>(pos2[2]) << 16; [[fallthrough]];
-            case 2: v ^= static_cast<uint64_t>(pos2[1]) << 8; [[fallthrough]];
-            case 1: v ^= static_cast<uint64_t>(pos2[0]);
-                h ^= mix(v);
-                h *= m;
-        }
-
-        return mix(h);
-    }
-};
-
-
-struct FNV1a
-{
-    size_t operator() (StringRef x) const
-    {
-        size_t res = 0xcbf29ce484222325ULL;
-
-        const char * pos = x.data;
-        const char * end = x.data + x.size;
-
-        for (; pos < end; ++pos)
-        {
-            res *= 1099511628211ULL;
-            res ^= *pos;
-        }
-
-        return res;
-    }
-};
-
-
-#if __SSE4_1__
-
-struct CrapWow
-{
-    size_t operator() (StringRef x) const
-    {
-        const char * key = x.data;
-        size_t len = x.size;
-        size_t seed = 0;
-
-        const UInt64 m = 0x95b47aa3355ba1a1, n = 0x8a970be7488fda55;
-        UInt64 hash;
-        // 3 = m, 4 = n
-        // r12 = h, r13 = k, ecx = seed, r12 = key
-        asm(
-            "leaq (%%rcx,%4), %%r13\n"
-            "movq %%rdx, %%r14\n"
-            "movq %%rcx, %%r15\n"
-            "movq %%rcx, %%r12\n"
-            "addq %%rax, %%r13\n"
-            "andq $0xfffffffffffffff0, %%rcx\n"
-            "jz QW%=\n"
-            "addq %%rcx, %%r14\n\n"
-            "negq %%rcx\n"
-        "XW%=:\n"
-            "movq %4, %%rax\n"
-            "mulq (%%r14,%%rcx)\n"
-            "xorq %%rax, %%r12\n"
-            "xorq %%rdx, %%r13\n"
-            "movq %3, %%rax\n"
-            "mulq 8(%%r14,%%rcx)\n"
-            "xorq %%rdx, %%r12\n"
-            "xorq %%rax, %%r13\n"
-            "addq $16, %%rcx\n"
-            "jnz XW%=\n"
-        "QW%=:\n"
-            "movq %%r15, %%rcx\n"
-            "andq $8, %%r15\n"
-            "jz B%=\n"
-            "movq %4, %%rax\n"
-            "mulq (%%r14)\n"
-            "addq $8, %%r14\n"
-            "xorq %%rax, %%r12\n"
-            "xorq %%rdx, %%r13\n"
-        "B%=:\n"
-            "andq $7, %%rcx\n"
-            "jz F%=\n"
-            "movq $1, %%rdx\n"
-            "shlq $3, %%rcx\n"
-            "movq %3, %%rax\n"
-            "shlq %%cl, %%rdx\n"
-            "addq $-1, %%rdx\n"
-            "andq (%%r14), %%rdx\n"
-            "mulq %%rdx\n"
-            "xorq %%rdx, %%r12\n"
-            "xorq %%rax, %%r13\n"
-        "F%=:\n"
-            "leaq (%%r13,%4), %%rax\n"
-            "xorq %%r12, %%rax\n"
-            "mulq %4\n"
-            "xorq %%rdx, %%rax\n"
-            "xorq %%r12, %%rax\n"
-            "xorq %%r13, %%rax\n"
-            : "=a"(hash), "=c"(key), "=d"(key)
-            : "r"(m), "r"(n), "a"(seed), "c"(len), "d"(key)
-            : "%r12", "%r13", "%r14", "%r15", "cc"
-        );
-        return hash;
-    }
-};
-
-#endif
-
-
-struct SimpleHash
-{
-    size_t operator() (StringRef x) const
-    {
-        const char * pos = x.data;
-        size_t size = x.size;
-
-        const char * end = pos + size;
-
-        size_t res = 0;
-
-        if (size == 0)
-            return 0;
-
-        if (size < 8)
-        {
-#if __SSE4_1__
-            return hashLessThan8(x.data, x.size);
-#endif
-        }
-
-        while (pos + 8 < end)
-        {
-            uint64_t word = *reinterpret_cast<const uint64_t *>(pos);
-            res = intHash64(word ^ res);
-
-            pos += 8;
-        }
-
-        uint64_t word = *reinterpret_cast<const uint64_t *>(end - 8);
-        res = intHash64(word ^ res);
-
-        return res;
-    }
-};
-
-
-struct VerySimpleHash
-{
-    size_t operator() (StringRef x) const
-    {
-        const char * pos = x.data;
-        size_t size = x.size;
-
-        const char * end = pos + size;
-
-        size_t res = 0;
-
-        if (size == 0)
-            return 0;
-
-        if (size < 8)
-        {
-#if __SSE4_1__
-            return hashLessThan8(x.data, x.size);
-#endif
-        }
-
-        while (pos + 8 < end)
-        {
-            res ^= reinterpret_cast<const uint64_t *>(pos)[0];
-            res ^= res >> 33;
-            res *= 0xff51afd7ed558ccdULL;
-
-            pos += 8;
-        }
-
-        res ^= *reinterpret_cast<const uint64_t *>(end - 8);
-        res ^= res >> 33;
-        res *= 0xc4ceb9fe1a85ec53ULL;
-        res ^= res >> 33;
-
-        return res;
-    }
-};
-
-
-struct FarmHash64
-{
-    size_t operator() (StringRef x) const
-    {
-        return farmhash::Hash64(x.data, x.size);
-    }
-};
-
-
-template <void metrohash64(const uint8_t * key, uint64_t len, uint32_t seed, uint8_t * out)>
-struct MetroHash64
-{
-    size_t operator() (StringRef x) const
-    {
-        union
-        {
-            uint64_t u64;
-            std::uint8_t u8[sizeof(u64)];
-        };
-
-        metrohash64(reinterpret_cast<const std::uint8_t *>(x.data), x.size, 0, u8);
-
-        return u64;
-    }
-};
-
-
-#if __SSE4_1__
-
-/*struct CRC32Hash
-{
-    size_t operator() (StringRef x) const
-    {
-        const char * pos = x.data;
-        size_t size = x.size;
-
-        if (size == 0)
-            return 0;
-
-        if (size < 8)
-        {
-            return hashLessThan8(x.data, x.size);
-        }
-
-        const char * end = pos + size;
-        size_t res = -1ULL;
-
-        do
-        {
-            uint64_t word = *reinterpret_cast<const uint64_t *>(pos);
-            res = _mm_crc32_u64(res, word);
-
-            pos += 8;
-        } while (pos + 8 < end);
-
-        uint64_t word = *reinterpret_cast<const uint64_t *>(end - 8);
-        res = _mm_crc32_u64(res, word);
-
-        return res;
-    }
-};*/
-
-
-struct CRC32ILPHash
-{
-    size_t operator() (StringRef x) const
-    {
-        const char * pos = x.data;
-        size_t size = x.size;
-
-        if (size == 0)
-            return 0;
-
-        if (size < 16)
-        {
-            return hashLessThan16(x.data, x.size);
-        }
-
-        const char * end = pos + size;
-        const char * end_16 = pos + size / 16 * 16;
-        size_t res0 = -1ULL;
-        size_t res1 = -1ULL;
-
-        do
-        {
-            uint64_t word0 = reinterpret_cast<const uint64_t *>(pos)[0];
-            uint64_t word1 = reinterpret_cast<const uint64_t *>(pos)[1];
-            res0 = _mm_crc32_u64(res0, word0);
-            res1 = _mm_crc32_u64(res1, word1);
-
-            pos += 16;
-        } while (pos < end_16);
-
-        uint64_t word0 = *reinterpret_cast<const uint64_t *>(end - 8);
-        uint64_t word1 = *reinterpret_cast<const uint64_t *>(end - 16);
-
-    /*    return HashLen16(Rotate(word0 - word1, 43) + Rotate(res0, 30) + res1,
-            word0 + Rotate(word1 ^ k3, 20) - res0 + size);*/
-
-        res0 = _mm_crc32_u64(res0, word0);
-        res1 = _mm_crc32_u64(res1, word1);
-
-        return hashLen16(res0, res1);
-    }
-};
-
-#endif
-
-
-using Value = uint64_t;
-
-
-template <typename Key, typename Hash>
-void NO_INLINE bench(const std::vector<StringRef> & data, const char * name)
-{
-    Stopwatch watch;
-
-    using Map = HashMapWithSavedHash<Key, Value, Hash>;
-
-    Map map;
-    typename Map::iterator it;
-    bool inserted;
-
-    for (size_t i = 0, size = data.size(); i < size; ++i)
-    {
-        map.emplace(static_cast<const Key &>(data[i]), it, inserted);
-        if (inserted)
-            it->second = 0;
-        ++it->second;
-    }
-
-    watch.stop();
-    std::cerr << std::fixed << std::setprecision(2)
-        << "HashMap (" << name << "). Size: " << map.size()
-        << ", elapsed: " << watch.elapsedSeconds()
-        << " (" << data.size() / watch.elapsedSeconds() << " elem/sec.)"
-#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
-        << ", collisions: " << map.getCollisions()
-#endif
-        << std::endl;
-}
-
-
-int main(int argc, char ** argv)
-{
-    if (argc < 3)
-    {
-        std::cerr << "Usage: program n m\n";
-        return 1;
-    }
-
-    size_t n = atoi(argv[1]);
-    size_t m = atoi(argv[2]);
-
-    DB::Arena pool;
-    std::vector<StringRef> data(n);
-
-    std::cerr << "sizeof(Key) = " << sizeof(StringRef) << ", sizeof(Value) = " << sizeof(Value) << std::endl;
-
-    {
-        Stopwatch watch;
-        DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO);
-        DB::CompressedReadBuffer in2(in1);
-
-        std::string tmp;
-        for (size_t i = 0; i < n && !in2.eof(); ++i)
-        {
-            DB::readStringBinary(tmp, in2);
-            data[i] = StringRef(pool.insert(tmp.data(), tmp.size()), tmp.size());
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "Vector. Size: " << n
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-            << std::endl;
-    }
-
-    if (!m || m == 1) bench<StringRef, StringRefHash64>(data, "StringRef_CityHash64");
-    if (!m || m == 2) bench<StringRef, FastHash64>     (data, "StringRef_FastHash64");
-    if (!m || m == 3) bench<StringRef, SimpleHash>     (data, "StringRef_SimpleHash");
-    if (!m || m == 4) bench<StringRef, FNV1a>          (data, "StringRef_FNV1a");
-
-#if __SSE4_1__
-    if (!m || m == 5) bench<StringRef, CrapWow>        (data, "StringRef_CrapWow");
-    if (!m || m == 6) bench<StringRef, CRC32Hash>      (data, "StringRef_CRC32Hash");
-    if (!m || m == 7) bench<StringRef, CRC32ILPHash>   (data, "StringRef_CRC32ILPHash");
-#endif
-
-    if (!m || m == 8) bench<StringRef, VerySimpleHash> (data, "StringRef_VerySimpleHash");
-    if (!m || m == 9) bench<StringRef, FarmHash64>     (data, "StringRef_FarmHash64");
-    if (!m || m == 10) bench<StringRef, MetroHash64<metrohash64_1>>(data, "StringRef_MetroHash64_1");
-    if (!m || m == 11) bench<StringRef, MetroHash64<metrohash64_2>>(data, "StringRef_MetroHash64_2");
-
-    return 0;
-}
diff --git a/dbms/src/Interpreters/tests/hash_map_string_small.cpp b/dbms/src/Interpreters/tests/hash_map_string_small.cpp
deleted file mode 100644
index 7c7059d6381..00000000000
--- a/dbms/src/Interpreters/tests/hash_map_string_small.cpp
+++ /dev/null
@@ -1,206 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#include <iostream>
-#include <iomanip>
-#include <vector>
-
-#include <unordered_map>
-
-#include <sparsehash/dense_hash_map>
-#include <sparsehash/sparse_hash_map>
-
-#include <Common/Stopwatch.h>
-
-//#define DBMS_HASH_MAP_COUNT_COLLISIONS
-#define DBMS_HASH_MAP_DEBUG_RESIZES
-
-#include <Core/Types.h>
-#include <IO/ReadBufferFromFile.h>
-#include <IO/ReadHelpers.h>
-#include <IO/CompressedReadBuffer.h>
-#include <common/StringRef.h>
-#include <Common/HashTable/HashMap.h>
-#include <Interpreters/AggregationCommon.h>
-
-
-struct SmallStringRef
-{
-    UInt32 size;
-
-    union
-    {
-        const char * data_big;
-        char data_small[12];
-    };
-
-    bool isSmall() const { return size <= 12; }
-
-    const char * data() const
-    {
-        return isSmall() ? data_small : data_big;
-    }
-
-    SmallStringRef(const char * data_, size_t size_)
-    {
-        size = size_;
-
-        if (isSmall())
-            memcpy(data_small, data_, size_);
-        else
-            data_big = data_;
-    }
-
-    SmallStringRef(const unsigned char * data_, size_t size_) : SmallStringRef(reinterpret_cast<const char *>(data_), size_) {}
-    explicit SmallStringRef(const std::string & s) : SmallStringRef(s.data(), s.size()) {}
-    SmallStringRef() {}
-
-    std::string toString() const { return std::string(data(), size); }
-};
-
-
-inline bool operator==(SmallStringRef lhs, SmallStringRef rhs)
-{
-    if (lhs.size != rhs.size)
-        return false;
-
-    if (lhs.size == 0)
-        return true;
-
-#if __SSE2__
-    return memequalSSE2Wide(lhs.data(), rhs.data(), lhs.size);
-#else
-    return false;
-#endif
-}
-
-
-namespace ZeroTraits
-{
-    template <>
-    inline bool check<SmallStringRef>(SmallStringRef x) { return x.size == 0; }
-
-    template <>
-    inline void set<SmallStringRef>(SmallStringRef & x) { x.size = 0; }
-};
-
-template <>
-struct DefaultHash<SmallStringRef>
-{
-    size_t operator() (SmallStringRef x) const
-    {
-        return DefaultHash<StringRef>()(StringRef(x.data(), x.size));
-    }
-};
-
-
-using Value = UInt64;
-
-
-int main(int argc, char ** argv)
-{
-    if (argc < 3)
-    {
-        std::cerr << "Usage: program n m\n";
-        return 1;
-    }
-
-    size_t n = atoi(argv[1]);
-    size_t m = atoi(argv[2]);
-
-    DB::Arena pool;
-    std::vector<StringRef> data(n);
-
-    std::cerr << "sizeof(Key) = " << sizeof(SmallStringRef) << ", sizeof(Value) = " << sizeof(Value) << std::endl;
-
-    {
-        Stopwatch watch;
-        DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO);
-        DB::CompressedReadBuffer in2(in1);
-
-        std::string tmp;
-        for (size_t i = 0; i < n && !in2.eof(); ++i)
-        {
-            DB::readStringBinary(tmp, in2);
-            data[i] = StringRef(pool.insert(tmp.data(), tmp.size()), tmp.size());
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "Vector. Size: " << n
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-            << std::endl;
-    }
-
-    if (!m || m == 1)
-    {
-        Stopwatch watch;
-
-        using Map = HashMapWithSavedHash<StringRef, Value>;
-
-        Map map;
-        Map::iterator it;
-        bool inserted;
-
-        for (size_t i = 0; i < n; ++i)
-        {
-            map.emplace(data[i], it, inserted);
-            if (inserted)
-                it->second = 0;
-            ++it->second;
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "HashMap (StringRef). Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
-            << ", collisions: " << map.getCollisions()
-#endif
-            << std::endl;
-    }
-
-    if (!m || m == 2)
-    {
-        Stopwatch watch;
-
-        using Map = HashMapWithSavedHash<SmallStringRef, Value>;
-
-        Map map;
-        Map::iterator it;
-        bool inserted;
-
-        for (size_t i = 0; i < n; ++i)
-        {
-            map.emplace(SmallStringRef(data[i].data, data[i].size), it, inserted);
-            if (inserted)
-                it->second = 0;
-            ++it->second;
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "HashMap (SmallStringRef). Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
-            << ", collisions: " << map.getCollisions()
-#endif
-            << std::endl;
-    }
-
-    return 0;
-}
diff --git a/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp b/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp
deleted file mode 100644
index 6da69abade0..00000000000
--- a/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp
+++ /dev/null
@@ -1,310 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#include <Parsers/ASTSelectQuery.h>
-#include <Parsers/ParserSelectQuery.h>
-#include <Parsers/parseQuery.h>
-#include <Parsers/queryToString.h>
-#include <Interpreters/LogicalExpressionsOptimizer.h>
-#include <Interpreters/Settings.h>
-#include <Common/typeid_cast.h>
-
-#include <iostream>
-#include <vector>
-#include <utility>
-#include <string>
-
-namespace
-{
-
-struct TestEntry
-{
-    std::string input;
-    std::string expected_output;
-    UInt64 limit;
-};
-
-using TestEntries = std::vector<TestEntry>;
-using TestResult = std::pair<bool, std::string>;
-
-void run();
-void performTests(const TestEntries & entries);
-TestResult check(const TestEntry & entry);
-bool parse(DB::ASTPtr & ast, const std::string & query);
-bool equals(const DB::ASTPtr & lhs, const DB::ASTPtr & rhs);
-void reorder(DB::IAST * ast);
-
-
-void run()
-{
-    /// NOTE: Queries are not always realistic, but we are only interested in the syntax.
-    TestEntries entries =
-    {
-        {
-            "SELECT 1",
-            "SELECT 1",
-            3
-        },
-
-        // WHERE
-
-        {
-            "SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
-            "SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
-            4
-        },
-
-        {
-            "SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
-            "SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol')",
-            3
-        },
-
-        {
-            "SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
-            "SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol')",
-            2
-        },
-
-        {
-            "SELECT name, value FROM report WHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol')",
-            "SELECT name, value FROM report WHERE (value = 1000) OR name IN ('Alice', 'Bob', 'Carol')",
-            2
-        },
-
-        {
-            "SELECT name, value FROM report WHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol') OR (value = 2000)",
-            "SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol') OR value IN (1000, 2000)",
-            2
-        },
-
-        {
-            "SELECT value FROM report WHERE ((value + 1) = 1000) OR ((2 * value) = 2000) OR ((2 * value) = 4000) OR ((value + 1) = 3000)",
-            "SELECT value FROM report WHERE ((value + 1) IN (1000, 3000)) OR ((2 * value) IN (2000, 4000))",
-            2
-        },
-
-        {
-            "SELECT name, value FROM report WHERE ((name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')) AND ((value = 1000) OR (value = 2000))",
-            "SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol') AND ((value = 1000) OR (value = 2000))",
-            3
-        },
-
-        // PREWHERE
-
-        {
-            "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
-            "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
-            4
-        },
-
-        {
-            "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
-            "SELECT name, value FROM report PREWHERE name IN ('Alice', 'Bob', 'Carol')",
-            3
-        },
-
-        {
-            "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')",
-            "SELECT name, value FROM report PREWHERE name IN ('Alice', 'Bob', 'Carol')",
-            2
-        },
-
-        {
-            "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol')",
-            "SELECT name, value FROM report PREWHERE (value = 1000) OR name IN ('Alice', 'Bob', 'Carol')",
-            2
-        },
-
-        {
-            "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol') OR (value = 2000)",
-            "SELECT name, value FROM report PREWHERE name IN ('Alice', 'Bob', 'Carol') OR value IN (1000, 2000)",
-            2
-        },
-
-        {
-            "SELECT value FROM report PREWHERE ((value + 1) = 1000) OR ((2 * value) = 2000) OR ((2 * value) = 4000) OR ((value + 1) = 3000)",
-            "SELECT value FROM report PREWHERE (value + 1) IN (1000, 3000) OR (2 * value) IN (2000, 4000)",
-            2
-        },
-
-        // HAVING
-
-        {
-            "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING number = 1",
-            "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING number = 1",
-            2
-        },
-
-        {
-            "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING (number = 1) OR (number = 2)",
-            "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING number IN (1, 2)",
-            2
-        },
-
-        {
-            "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING (number = 1) OR (number = 2)",
-            "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING (number = 1) OR (number = 2)",
-            3
-        },
-
-        {
-            "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING ((number + 1) = 1) OR ((number + 1) = 2) OR ((number + 3) = 7)",
-            "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING ((number + 3) = 7) OR (number + 1) IN (1, 2)",
-            2
-        },
-
-        // PREWHERE + WHERE + HAVING
-
-        {
-            "SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE (U = 1) OR (U = 2) "
-            "WHERE (T = 1) OR (T = 2) GROUP BY number HAVING (number = 1) OR (number = 2)",
-            "SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE U IN (1, 2) "
-            "WHERE T IN (1, 2) GROUP BY number HAVING number IN (1, 2)",
-            2
-        },
-
-        {
-            "SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE (U = 1) OR (U = 2) OR (U = 3) "
-            "WHERE (T = 1) OR (T = 2) GROUP BY number HAVING (number = 1) OR (number = 2)",
-            "SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE U IN (1, 2, 3) "
-            "WHERE (T = 1) OR (T = 2) GROUP BY number HAVING (number = 1) OR (number = 2)",
-            3
-        },
-
-        {
-            "SELECT x = 1 OR x=2 OR (x = 3 AS x3) AS y, 4 AS x",
-            "SELECT x IN (1, 2, 3) AS y, 4 AS x",
-            2
-        }
-    };
-
-    performTests(entries);
-}
-
-void performTests(const TestEntries & entries)
-{
-    unsigned int count = 0;
-    unsigned int i = 1;
-
-    for (const auto & entry : entries)
-    {
-        auto res = check(entry);
-        if (res.first)
-        {
-            ++count;
-            std::cout << "Test " << i << " passed.\n";
-        }
-        else
-            std::cout << "Test " << i << " failed. Expected: " << entry.expected_output << ". Received: " << res.second << "\n";
-
-        ++i;
-    }
-    std::cout << count << " out of " << entries.size() << " test(s) passed.\n";
-}
-
-TestResult check(const TestEntry & entry)
-{
-    try
-    {
-        /// Parse and optimize the incoming query.
-        DB::ASTPtr ast_input;
-        if (!parse(ast_input, entry.input))
-            return TestResult(false, "parse error");
-
-        auto select_query = typeid_cast<DB::ASTSelectQuery *>(&*ast_input);
-
-        DB::Settings settings;
-        settings.optimize_min_equality_disjunction_chain_length = entry.limit;
-
-        DB::LogicalExpressionsOptimizer optimizer(select_query, settings);
-        optimizer.perform();
-
-        /// Parse the expected result.
-        DB::ASTPtr ast_expected;
-        if (!parse(ast_expected, entry.expected_output))
-            return TestResult(false, "parse error");
-
-        /// Compare the optimized query and the expected result.
-        bool res = equals(ast_input, ast_expected);
-        std::string output = DB::queryToString(ast_input);
-
-        return TestResult(res, output);
-    }
-    catch (DB::Exception & e)
-    {
-        return TestResult(false, e.displayText());
-    }
-}
-
-bool parse(DB::ASTPtr & ast, const std::string & query)
-{
-    DB::ParserSelectQuery parser;
-    std::string message;
-    auto begin = query.data();
-    auto end = begin + query.size();
-    ast = DB::tryParseQuery(parser, begin, end, message, false, "", false, 0);
-    return ast != nullptr;
-}
-
-bool equals(const DB::ASTPtr & lhs, const DB::ASTPtr & rhs)
-{
-    DB::ASTPtr lhs_reordered = lhs->clone();
-    reorder(&*lhs_reordered);
-
-    DB::ASTPtr rhs_reordered = rhs->clone();
-    reorder(&*rhs_reordered);
-
-    return lhs_reordered->getTreeHash() == rhs_reordered->getTreeHash();
-}
-
-void reorderImpl(DB::IAST * ast)
-{
-    if (ast == nullptr)
-        return;
-
-    auto & children = ast->children;
-    if (children.empty())
-        return;
-
-    for (auto & child : children)
-        reorderImpl(&*child);
-
-    std::sort(children.begin(), children.end(), [](const DB::ASTPtr & lhs, const DB::ASTPtr & rhs)
-    {
-        return lhs->getTreeHash() < rhs->getTreeHash();
-    });
-}
-
-void reorder(DB::IAST * ast)
-{
-    if (ast == nullptr)
-        return;
-
-    auto select_query = typeid_cast<DB::ASTSelectQuery *>(ast);
-    if (select_query == nullptr)
-        return;
-
-    reorderImpl(select_query->where_expression.get());
-    reorderImpl(select_query->prewhere_expression.get());
-    reorderImpl(select_query->having_expression.get());
-}
-
-}
-
-int main()
-{
-    run();
-    return 0;
-}
diff --git a/dbms/src/Interpreters/tests/select_query.cpp b/dbms/src/Interpreters/tests/select_query.cpp
deleted file mode 100644
index 2d57d3bee89..00000000000
--- a/dbms/src/Interpreters/tests/select_query.cpp
+++ /dev/null
@@ -1,69 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#include <Databases/DatabaseOrdinary.h>
-#include <Databases/IDatabase.h>
-#include <IO/ReadBufferFromFileDescriptor.h>
-#include <IO/WriteBufferFromFileDescriptor.h>
-#include <Interpreters/Context.h>
-#include <Interpreters/executeQuery.h>
-#include <Interpreters/loadMetadata.h>
-#include <Poco/ConsoleChannel.h>
-#include <Storages/StorageLog.h>
-#include <Storages/System/attachSystemTables.h>
-#include <common/DateLUT.h>
-
-#include <iomanip>
-#include <iostream>
-
-
-using namespace DB;
-
-int main(int, char **)
-try
-{
-    Poco::AutoPtr<Poco::ConsoleChannel> channel = new Poco::ConsoleChannel(std::cerr);
-    Poco::Logger::root().setChannel(channel);
-    Poco::Logger::root().setLevel("trace");
-
-    /// Pre-initialize the `DateLUT` so that the first initialization does not affect the measured execution speed.
-    DateLUT::instance();
-
-    Context context = Context::createGlobal();
-
-    context.setPath("./");
-
-    loadMetadata(context);
-
-    DatabasePtr system = std::make_shared<DatabaseOrdinary>("system", "./metadata/system/", context);
-    context.addDatabase("system", system);
-    system->loadTables(context, nullptr, false);
-    attachSystemTablesLocal(*context.getDatabase("system"));
-    context.setCurrentDatabase("default");
-
-    ReadBufferFromFileDescriptor in(STDIN_FILENO);
-    WriteBufferFromFileDescriptor out(STDOUT_FILENO);
-
-    executeQuery(in, out, /* allow_into_outfile = */ false, context, {});
-
-    return 0;
-}
-catch (const Exception & e)
-{
-    std::cerr << e.what() << ", " << e.displayText() << std::endl
-              << std::endl
-              << "Stack trace:" << std::endl
-              << e.getStackTrace().toString();
-    return 1;
-}
diff --git a/dbms/src/Interpreters/tests/two_level_hash_map.cpp b/dbms/src/Interpreters/tests/two_level_hash_map.cpp
deleted file mode 100644
index 5d7d67d71c2..00000000000
--- a/dbms/src/Interpreters/tests/two_level_hash_map.cpp
+++ /dev/null
@@ -1,149 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#include <iostream>
-#include <iomanip>
-#include <vector>
-
-#include <unordered_map>
-
-#include <sparsehash/dense_hash_map>
-#include <sparsehash/sparse_hash_map>
-
-//#define DBMS_HASH_MAP_DEBUG_RESIZES
-
-#include <Common/Stopwatch.h>
-#include <AggregateFunctions/UniquesHashSet.h>
-
-#include <Core/Types.h>
-#include <IO/ReadBufferFromFile.h>
-#include <IO/CompressedReadBuffer.h>
-#include <Common/HashTable/TwoLevelHashTable.h>
-#include <Common/HashTable/HashMap.h>
-
-
-using Key = UInt64;
-using Value = UInt64;
-
-
-int main(int argc, char ** argv)
-{
-    if (argc < 2)
-    {
-        std::cerr << "Usage: program n\n";
-        return 1;
-    }
-
-    size_t n = atoi(argv[1]);
-
-    std::vector<Key> data(n);
-
-    std::cerr << "sizeof(Key) = " << sizeof(Key) << ", sizeof(Value) = " << sizeof(Value) << std::endl;
-
-    {
-        Stopwatch watch;
-        DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO);
-        DB::CompressedReadBuffer in2(in1);
-
-        in2.readStrict(reinterpret_cast<char*>(&data[0]), sizeof(data[0]) * n);
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "Vector. Size: " << n
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-            << std::endl;
-    }
-
-    {
-        Stopwatch watch;
-
-        std::cerr << sizeof(HashMapCell<Key, Value, DefaultHash<Key>>) << std::endl;
-
-        using Map = TwoLevelHashTable<Key, HashMapCell<Key, Value, DefaultHash<Key>>, DefaultHash<Key>, HashTableGrower<8>, HashTableAllocator>;
-
-        Map map;
-        Map::iterator it;
-        bool inserted;
-
-        for (size_t i = 0; i < n; ++i)
-        {
-            map.emplace(data[i], it, inserted);
-            if (inserted)
-                it->second = 0;
-            ++it->second;
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "HashMap. Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-            << std::endl;
-
-        size_t sum_counts = 0;
-        size_t elems = 0;
-        for (const auto & kv : map)
-        {
-            sum_counts += kv.second;
-            ++elems;
-        }
-
-        std::cerr << "sum_counts: " << sum_counts << ", elems: " << elems << std::endl;
-    }
-
-    {
-        Stopwatch watch;
-
-        using Map = TwoLevelHashTable<Key, HashMapCell<Key, Value, DefaultHash<Key>>, DefaultHash<Key>, HashTableGrower<8>, HashTableAllocator>;
-        //using Map = HashMap<Key, Value, UniquesHashSetDefaultHash>;
-
-        Map map;
-        Map::iterator it;
-        bool inserted;
-
-        for (size_t i = 0; i < n; ++i)
-        {
-            map.emplace(i, it, inserted);
-            if (inserted)
-                it->second = 0;
-            ++it->second;
-        }
-
-        watch.stop();
-        std::cerr << std::fixed << std::setprecision(2)
-            << "HashMap. Size: " << map.size()
-            << ", elapsed: " << watch.elapsedSeconds()
-            << " (" << n / watch.elapsedSeconds() << " elem/sec.)"
-            << std::endl;
-
-        size_t sum_counts = 0;
-        size_t elems = 0;
-        for (const auto & kv : map)
-        {
-            sum_counts += kv.second;
-            ++elems;
-
-            if (kv.first > n)
-                std::cerr << kv.first << std::endl;
-        }
-
-        std::cerr << "sum_counts: " << sum_counts << ", elems: " << elems << std::endl;
-
-        if (sum_counts != n)
-            std::cerr << "!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!" << std::endl;
-    }
-
-    return 0;
-}
diff --git a/dbms/src/Interpreters/tests/users.cpp b/dbms/src/Interpreters/tests/users.cpp
deleted file mode 100644
index 60df6af4d71..00000000000
--- a/dbms/src/Interpreters/tests/users.cpp
+++ /dev/null
@@ -1,299 +0,0 @@
-// Copyright 2022 PingCAP, Ltd.
-//
-// Licensed 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.
-
-#include <Common/Config/ConfigProcessor.h>
-#include <Interpreters/SecurityManager.h>
-#include <boost/filesystem.hpp>
-#include <vector>
-#include <string>
-#include <tuple>
-#include <iostream>
-#include <fstream>
-#include <sstream>
-#include <stdexcept>
-#include <cstdlib>
-#include <port/unistd.h>
-
-namespace
-{
-
-namespace fs = boost::filesystem;
-
-struct TestEntry
-{
-    std::string user_name;
-    std::string database_name;
-    bool is_allowed;
-};
-
-using TestEntries = std::vector<TestEntry>;
-
-struct TestDescriptor
-{
-    const char * config_content;
-    TestEntries entries;
-};
-
-using TestSet = std::vector<TestDescriptor>;
-
-/// Tests description.
-
-TestSet test_set =
-{
-    {
-        "<?xml version=\"1.0\"?><yandex>"
-        "    <profiles><default></default></profiles>"
-        "    <users>"
-        "        <default>"
-        "            <password></password><profile>default</profile><quota>default</quota>"
-        "            <allow_databases>"
-        "              <database>default</database>"
-        "              <database>test</database>"
-        "            </allow_databases>"
-        "        </default>"
-        "        <web>"
-        "            <password></password><profile>default</profile><quota>default</quota>"
-        "        </web>"
-        "    </users>"
-        "    <quotas><default></default></quotas>"
-        "</yandex>",
-
-        {
-            { "default", "default", true },
-            { "default", "test", true },
-            { "default", "stats", false },
-            { "web", "default", true },
-            { "web", "test", true },
-            { "web", "stats", true },
-            { "analytics", "default", false },
-            { "analytics", "test", false },
-            { "analytics", "stats", false }
-        }
-    },
-
-    {
-        "<?xml version=\"1.0\"?><yandex>"
-        "    <profiles><default></default></profiles>"
-        "    <users>"
-        "        <default>"
-        "            <password></password><profile>default</profile><quota>default</quota>"
-        "            <allow_databases>"
-        "              <database>default</database>"
-        "            </allow_databases>"
-        "        </default>"
-        "        <web>"
-        "            <password></password><profile>default</profile><quota>default</quota>"
-        "        </web>"
-        "    </users>"
-        "    <quotas><default></default></quotas>"
-        "</yandex>",
-
-        {
-            { "default", "default", true },
-            { "default", "test", false },
-            { "default", "stats", false },
-            { "web", "default", true },
-            { "web", "test", true },
-            { "web", "stats", true },
-            { "analytics", "default", false },
-            { "analytics", "test", false },
-            { "analytics", "stats", false }
-        }
-    },
-
-    {
-        "<?xml version=\"1.0\"?><yandex>"
-        "    <profiles><default></default></profiles>"
-        "    <users>"
-        "        <default>"
-        "            <password></password><profile>default</profile><quota>default</quota>"
-        "            <allow_databases>"
-        "            </allow_databases>"
-        "        </default>"
-        "        <web>"
-        "            <password></password><profile>default</profile><quota>default</quota>"
-        "        </web>"
-        "    </users>"
-        "    <quotas><default></default></quotas>"
-        "</yandex>",
-
-        {
-            { "default", "default", true },
-            { "default", "test", true },
-            { "default", "stats", true },
-            { "web", "default", true },
-            { "web", "test", true },
-            { "web", "stats", true },
-            { "analytics", "default", false },
-            { "analytics", "test", false },
-            { "analytics", "stats", false }
-        }
-    },
-
-    {
-        "<?xml version=\"1.0\"?><yandex>"
-        "    <profiles><default></default></profiles>"
-        "    <users>"
-        "        <default>"
-        "            <password></password><profile>default</profile><quota>default</quota>"
-        "            <allow_databases>"
-        "              <database>default</database>"
-        "            </allow_databases>"
-        "        </default>"
-        "        <web>"
-        "            <password></password><profile>default</profile><quota>default</quota>"
-        "            <allow_databases>"
-        "              <database>test</database>"
-        "            </allow_databases>"
-        "        </web>"
-        "    </users>"
-        "    <quotas><default></default></quotas>"
-        "</yandex>",
-
-        {
-            { "default", "default", true },
-            { "default", "test", false },
-            { "default", "stats", false },
-            { "web", "default", false },
-            { "web", "test", true },
-            { "web", "stats", false },
-            { "analytics", "default", false },
-            { "analytics", "test", false },
-            { "analytics", "stats", false }
-        }
-    }
-};
-
-std::string createTmpPath(const std::string & filename);
-void createFile(const std::string & filename, const char * data);
-void runOneTest(const TestDescriptor & test_descriptor);
-auto runTestSet(const TestSet & test_set);
-
-std::string createTmpPath(const std::string & filename)
-{
-    char pattern[] = "/tmp/fileXXXXXX";
-    char * dir = mkdtemp(pattern);
-    if (dir == nullptr)
-        throw std::runtime_error("Could not create directory");
-
-    return std::string(dir) + "/" + filename;
-}
-
-void createFile(const std::string & filename, const char * data)
-{
-    std::ofstream ofs(filename.c_str());
-    if (!ofs.is_open())
-        throw std::runtime_error("Could not open file " + filename);
-    ofs << data;
-}
-
-void runOneTest(const TestDescriptor & test_descriptor)
-{
-    const auto path_name = createTmpPath("users.xml");
-    createFile(path_name, test_descriptor.config_content);
-
-    ConfigurationPtr config;
-
-    try
-    {
-        config = ConfigProcessor(path_name).loadConfig().configuration;
-    }
-    catch (const Poco::Exception & ex)
-    {
-        std::ostringstream os;
-        os << "Error: " << ex.what() << ": " << ex.displayText();
-        throw std::runtime_error(os.str());
-    }
-
-    DB::SecurityManager security_manager;
-
-    try
-    {
-        security_manager.loadFromConfig(*config);
-    }
-    catch (const Poco::Exception & ex)
-    {
-        std::ostringstream os;
-        os << "Error: " << ex.what() << ": " << ex.displayText();
-        throw std::runtime_error(os.str());
-    }
-
-    for (const auto & entry : test_descriptor.entries)
-    {
-        bool res;
-
-        try
-        {
-            res = security_manager.hasAccessToDatabase(entry.user_name, entry.database_name);
-        }
-        catch (const Poco::Exception &)
-        {
-            res = false;
-        }
-
-        if (res != entry.is_allowed)
-        {
-            auto to_string = [](bool access){ return (access ? "'granted'" : "'denied'"); };
-            std::ostringstream os;
-            os << "(user=" << entry.user_name << ", database=" << entry.database_name << "): ";
-            os << "Expected " << to_string(entry.is_allowed) << " but got " << to_string(res);
-            throw std::runtime_error(os.str());
-        }
-    }
-
-    fs::remove_all(fs::path(path_name).parent_path().string());
-}
-
-auto runTestSet(const TestSet & test_set)
-{
-    size_t test_num = 1;
-    size_t failure_count = 0;
-
-    for (const auto & test_descriptor : test_set)
-    {
-        try
-        {
-            runOneTest(test_descriptor);
-            std::cout << "Test " << test_num << " passed\n";
-        }
-        catch (const std::runtime_error & ex)
-        {
-            std::cerr << "Test " << test_num << " failed with reason: " << ex.what() << "\n";
-            ++failure_count;
-        }
-        catch (...)
-        {
-            std::cerr << "Test " << test_num << " failed with unknown reason\n";
-            ++failure_count;
-        }
-
-        ++test_num;
-    }
-
-    return std::make_tuple(test_set.size(), failure_count);
-}
-
-}
-
-int main()
-{
-    size_t test_count;
-    size_t failure_count;
-
-    std::tie(test_count, failure_count) = runTestSet(test_set);
-
-    std::cout << (test_count - failure_count) << " test(s) passed out of " << test_count << "\n";
-
-    return (failure_count == 0) ? 0 : EXIT_FAILURE;
-}
diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp
index 87f1b55d29f..5e71ee79684 100644
--- a/dbms/src/Server/Server.cpp
+++ b/dbms/src/Server/Server.cpp
@@ -49,7 +49,6 @@
 #include <IO/ReadHelpers.h>
 #include <IO/createReadBufferFromFileBase.h>
 #include <Interpreters/AsynchronousMetrics.h>
-#include <Interpreters/IDAsPathUpgrader.h>
 #include <Interpreters/ProcessList.h>
 #include <Interpreters/loadMetadata.h>
 #include <Poco/DirectoryIterator.h>
@@ -976,8 +975,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
     global_context->initializePageStorageMode(global_context->getPathPool(), STORAGE_FORMAT_CURRENT.page);
 
     // Use pd address to define which default_database we use by default.
-    // For mock test, we use "default". For deployed with pd/tidb/tikv use "system", which is always exist in TiFlash.
-    std::string default_database = config().getString("default_database", raft_config.pd_addrs.empty() ? "default" : "system");
+    // For deployed with pd/tidb/tikv use "system", which is always exist in TiFlash.
+    std::string default_database = config().getString("default_database", "system");
     Strings all_normal_path = storage_config.getAllNormalPaths();
     const std::string path = all_normal_path[0];
     global_context->setPath(path);
@@ -1206,27 +1205,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
 
     global_context->initializeSharedBlockSchemas();
 
-    {
-        // Note that this must do before initialize schema sync service.
-        do
-        {
-            // Check whether we need to upgrade directories hierarchy
-            // If some database can not find in TiDB, they will be dropped
-            // if theirs name is not in reserved_databases.
-            // Besides, database engine in reserved_databases just keep as
-            // what they are.
-            IDAsPathUpgrader upgrader(
-                *global_context,
-                /*is_mock=*/raft_config.pd_addrs.empty(),
-                /*reserved_databases=*/raft_config.ignore_databases);
-            if (!upgrader.needUpgrade())
-                break;
-            upgrader.doUpgrade();
-        } while (false);
-
-        /// Then, load remaining databases
-        loadMetadata(*global_context);
-    }
+    // Load remaining databases
+    loadMetadata(*global_context);
     LOG_DEBUG(log, "Load metadata done.");
 
     if (!global_context->isDisaggregatedComputeMode())
diff --git a/dbms/src/Storages/DeltaMerge/DeltaIndexManager.h b/dbms/src/Storages/DeltaMerge/DeltaIndexManager.h
index 48c8424ea13..60740424562 100644
--- a/dbms/src/Storages/DeltaMerge/DeltaIndexManager.h
+++ b/dbms/src/Storages/DeltaMerge/DeltaIndexManager.h
@@ -50,7 +50,7 @@ class DeltaIndexManager
     size_t current_size = 0;
     const size_t max_size;
 
-    Poco::Logger * log;
+    LoggerPtr log;
 
     std::mutex mutex;
 
@@ -60,7 +60,7 @@ class DeltaIndexManager
 public:
     explicit DeltaIndexManager(size_t max_size_)
         : max_size(max_size_)
-        , log(&Poco::Logger::get("DeltaIndexManager"))
+        , log(Logger::get())
     {}
 
     /// Note that if isLimit() is false, than this method always return 0.
diff --git a/dbms/src/Storages/DeltaMerge/File/DMFile.cpp b/dbms/src/Storages/DeltaMerge/File/DMFile.cpp
index c3dee70e21a..b27e5908cf6 100644
--- a/dbms/src/Storages/DeltaMerge/File/DMFile.cpp
+++ b/dbms/src/Storages/DeltaMerge/File/DMFile.cpp
@@ -25,6 +25,7 @@
 #include <Storages/DeltaMerge/File/DMFile.h>
 #include <Storages/Page/PageUtil.h>
 #include <boost_wrapper/string_split.h>
+#include <common/logger_useful.h>
 #include <fmt/format.h>
 
 #include <boost/algorithm/string/classification.hpp>
@@ -102,13 +103,11 @@ String DMFile::ngcPath() const
 
 DMFilePtr DMFile::create(UInt64 file_id, const String & parent_path, DMConfigurationOpt configuration)
 {
-    Poco::Logger * log = &Poco::Logger::get("DMFile");
     // On create, ref_id is the same as file_id.
     DMFilePtr new_dmfile(new DMFile(file_id,
                                     file_id,
                                     parent_path,
                                     Status::WRITABLE,
-                                    log,
                                     std::move(configuration)));
 
     auto path = new_dmfile->path();
@@ -116,7 +115,7 @@ DMFilePtr DMFile::create(UInt64 file_id, const String & parent_path, DMConfigura
     if (file.exists())
     {
         file.remove(true);
-        LOG_WARNING(log, "Existing dmfile, removed: {}", path);
+        LOG_WARNING(Logger::get(), "Existing dmfile, removed: {}", path);
     }
 
     file.createDirectories();
@@ -146,8 +145,7 @@ DMFilePtr DMFile::restore(
         file_id,
         page_id,
         parent_path,
-        Status::READABLE,
-        &Poco::Logger::get("DMFile")));
+        Status::READABLE));
     if (!read_meta_mode.isNone())
     {
         dmfile->readConfiguration(file_provider);
@@ -395,7 +393,7 @@ void DMFile::readColumnStat(const FileProviderPtr & file_provider, const MetaPac
         }
         else
         {
-            log->warning(fmt::format("checksum for {} not found", name));
+            LOG_WARNING(log, "checksum for {} not found", name);
         }
     }
 
@@ -486,7 +484,7 @@ void DMFile::readPackProperty(const FileProviderPtr & file_provider, const MetaP
         }
         else
         {
-            log->warning(fmt::format("checksum for {} not found", name));
+            LOG_WARNING(log, "checksum for {} not found", name);
         }
     }
 }
@@ -533,7 +531,7 @@ void DMFile::finalizeForFolderMode(const FileProviderPtr & file_provider, const
 {
     if (STORAGE_FORMAT_CURRENT.dm_file >= DMFileFormat::V2 && !configuration)
     {
-        log->warning("checksum disabled due to lack of configuration");
+        LOG_WARNING(log, "checksum disabled due to lack of configuration");
     }
     writeMetadata(file_provider, write_limiter);
     if (unlikely(status != Status::WRITING))
@@ -569,7 +567,6 @@ std::set<UInt64> DMFile::listAllInPath(
     std::vector<std::string> file_names;
     folder.list(file_names);
     std::set<UInt64> file_ids;
-    Poco::Logger * log = &Poco::Logger::get("DMFile");
 
     auto try_parse_file_id = [](const String & name) -> std::optional<UInt64> {
         std::vector<std::string> ss;
@@ -597,7 +594,7 @@ std::set<UInt64> DMFile::listAllInPath(
                 auto res = try_parse_file_id(name);
                 if (!res)
                 {
-                    LOG_INFO(log, "Unrecognized temporary or dropped dmfile, ignored: {}", name);
+                    LOG_INFO(Logger::get(), "Unrecognized temporary or dropped dmfile, ignored: {}", name);
                     continue;
                 }
                 UInt64 file_id = *res;
@@ -606,7 +603,7 @@ std::set<UInt64> DMFile::listAllInPath(
                 const auto full_path = parent_path + "/" + name;
                 if (Poco::File file(full_path); file.exists())
                     file.remove(true);
-                LOG_WARNING(log, "Existing temporary or dropped dmfile, removed: {}", full_path);
+                LOG_WARNING(Logger::get(), "Existing temporary or dropped dmfile, removed: {}", full_path);
                 continue;
             }
         }
@@ -619,7 +616,7 @@ std::set<UInt64> DMFile::listAllInPath(
         auto res = try_parse_file_id(name);
         if (!res)
         {
-            LOG_INFO(log, "Unrecognized DM file, ignored: {}", name);
+            LOG_INFO(Logger::get(), "Unrecognized DM file, ignored: {}", name);
             continue;
         }
         UInt64 file_id = *res;
diff --git a/dbms/src/Storages/DeltaMerge/File/DMFile.h b/dbms/src/Storages/DeltaMerge/File/DMFile.h
index 06cf1f9502f..3a45597921a 100644
--- a/dbms/src/Storages/DeltaMerge/File/DMFile.h
+++ b/dbms/src/Storages/DeltaMerge/File/DMFile.h
@@ -287,14 +287,13 @@ class DMFile : private boost::noncopyable
            UInt64 page_id_,
            String parent_path_,
            Status status_,
-           Poco::Logger * log_,
            DMConfigurationOpt configuration_ = std::nullopt)
         : file_id(file_id_)
         , page_id(page_id_)
         , parent_path(std::move(parent_path_))
         , status(status_)
         , configuration(std::move(configuration_))
-        , log(log_)
+        , log(Logger::get())
     {
     }
 
@@ -389,7 +388,7 @@ class DMFile : private boost::noncopyable
 
     SubFileStats sub_file_stats;
 
-    Poco::Logger * log;
+    LoggerPtr log;
 
     friend class DMFileWriter;
     friend class DMFileReader;
diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/CPU.cpp b/dbms/src/Storages/DeltaMerge/ReadThread/CPU.cpp
index a946768795c..5f8684893e3 100644
--- a/dbms/src/Storages/DeltaMerge/ReadThread/CPU.cpp
+++ b/dbms/src/Storages/DeltaMerge/ReadThread/CPU.cpp
@@ -11,10 +11,11 @@
 // 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.
+
 #include <Common/Exception.h>
+#include <Common/Logger.h>
 #include <Poco/DirectoryIterator.h>
 #include <Poco/File.h>
-#include <Poco/Logger.h>
 #include <Storages/DeltaMerge/ReadThread/CPU.h>
 #include <common/logger_useful.h>
 
@@ -93,7 +94,7 @@ std::vector<std::vector<int>> getLinuxNumaNodes()
     return numa_nodes;
 }
 
-std::vector<std::vector<int>> getNumaNodes(Poco::Logger * log)
+std::vector<std::vector<int>> getNumaNodes(const LoggerPtr & log)
 {
 #ifndef __APPLE__ // Apple macbooks does not support NUMA
     try
diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/CPU.h b/dbms/src/Storages/DeltaMerge/ReadThread/CPU.h
index 2d1f564a43a..27439770ec2 100644
--- a/dbms/src/Storages/DeltaMerge/ReadThread/CPU.h
+++ b/dbms/src/Storages/DeltaMerge/ReadThread/CPU.h
@@ -13,12 +13,16 @@
 // limitations under the License.
 #pragma once
 
-#include <Poco/Logger.h>
-
 #include <vector>
 
+namespace DB
+{
+class Logger;
+using LoggerPtr = std::shared_ptr<Logger>;
+} // namespace DB
+
 namespace DB::DM
 {
 // `getNumaNodes` returns cpus of each Numa node.
-std::vector<std::vector<int>> getNumaNodes(Poco::Logger * log);
-} // namespace DB::DM
\ No newline at end of file
+std::vector<std::vector<int>> getNumaNodes(const LoggerPtr & log);
+} // namespace DB::DM
diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/MergedTask.h b/dbms/src/Storages/DeltaMerge/ReadThread/MergedTask.h
index 42e88f01f1e..128ef537cf5 100644
--- a/dbms/src/Storages/DeltaMerge/ReadThread/MergedTask.h
+++ b/dbms/src/Storages/DeltaMerge/ReadThread/MergedTask.h
@@ -47,7 +47,7 @@ class MergedTask
         , inited(false)
         , cur_idx(-1)
         , finished_count(0)
-        , log(&Poco::Logger::get("MergedTask"))
+        , log(Logger::get())
     {
         passive_merged_segments.fetch_add(units.size() - 1, std::memory_order_relaxed);
         GET_METRIC(tiflash_storage_read_thread_gauge, type_merged_task).Increment();
@@ -142,7 +142,7 @@ class MergedTask
     bool inited;
     int cur_idx;
     size_t finished_count;
-    Poco::Logger * log;
+    LoggerPtr log;
     Stopwatch sw;
     inline static std::atomic<int64_t> passive_merged_segments{0};
 };
@@ -156,7 +156,7 @@ class MergedTaskPool
 {
 public:
     MergedTaskPool()
-        : log(&Poco::Logger::get("MergedTaskPool"))
+        : log(Logger::get())
     {}
 
     MergedTaskPtr pop(uint64_t pool_id);
@@ -166,6 +166,6 @@ class MergedTaskPool
 private:
     std::mutex mtx;
     std::list<MergedTaskPtr> merged_task_pool;
-    Poco::Logger * log;
+    LoggerPtr log;
 };
-} // namespace DB::DM
\ No newline at end of file
+} // namespace DB::DM
diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.cpp b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.cpp
index 99dbbc8caab..11d5e7da6a2 100644
--- a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.cpp
+++ b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.cpp
@@ -19,7 +19,7 @@ namespace DB::DM
 {
 SegmentReadTaskScheduler::SegmentReadTaskScheduler()
     : stop(false)
-    , log(&Poco::Logger::get("SegmentReadTaskScheduler"))
+    , log(Logger::get())
 {
     sched_thread = std::thread(&SegmentReadTaskScheduler::schedLoop, this);
 }
diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.h b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.h
index c319275fd0f..5373e226609 100644
--- a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.h
+++ b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.h
@@ -78,6 +78,6 @@ class SegmentReadTaskScheduler
     std::atomic<bool> stop;
     std::thread sched_thread;
 
-    Poco::Logger * log;
+    LoggerPtr log;
 };
 } // namespace DB::DM
diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp
index 2b090e7913d..5981ec05eb9 100644
--- a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp
+++ b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp
@@ -11,6 +11,8 @@
 // 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.
+
+#include <Common/Logger.h>
 #include <Common/setThreadName.h>
 #include <Storages/DeltaMerge/ReadThread/CPU.h>
 #include <Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.h>
@@ -29,7 +31,7 @@ class SegmentReader
     SegmentReader(WorkQueue<MergedTaskPtr> & task_queue_, const std::vector<int> & cpus_)
         : task_queue(task_queue_)
         , stop(false)
-        , log(&Poco::Logger::get(name))
+        , log(Logger::get())
         , cpus(cpus_)
     {
         t = std::thread(&SegmentReader::run, this);
@@ -155,7 +157,7 @@ class SegmentReader
 
     WorkQueue<MergedTaskPtr> & task_queue;
     std::atomic<bool> stop;
-    Poco::Logger * log;
+    LoggerPtr log;
     std::thread t;
     std::vector<int> cpus;
 };
@@ -171,7 +173,7 @@ void SegmentReaderPool::addTask(MergedTaskPtr && task)
 }
 
 SegmentReaderPool::SegmentReaderPool(int thread_count, const std::vector<int> & cpus)
-    : log(&Poco::Logger::get("SegmentReaderPool"))
+    : log(Logger::get())
 {
     LOG_INFO(log, "Create start, thread_count={} cpus={}", thread_count, cpus);
     for (int i = 0; i < thread_count; i++)
@@ -203,7 +205,7 @@ std::vector<std::thread::id> SegmentReaderPool::getReaderIds() const
 // ===== SegmentReaderPoolManager ===== //
 
 SegmentReaderPoolManager::SegmentReaderPoolManager()
-    : log(&Poco::Logger::get("SegmentReaderPoolManager"))
+    : log(Logger::get())
 {}
 
 SegmentReaderPoolManager::~SegmentReaderPoolManager() = default;
diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.h b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.h
index 65d24865d3d..f57095fa623 100644
--- a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.h
+++ b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.h
@@ -17,7 +17,12 @@
 #include <Server/ServerInfo.h>
 #include <Storages/DeltaMerge/ReadThread/WorkQueue.h>
 #include <Storages/DeltaMerge/SegmentReadTaskPool.h>
-#include <common/logger_useful.h>
+
+namespace DB
+{
+class Logger;
+using LoggerPtr = std::shared_ptr<Logger>;
+} // namespace DB
 
 namespace DB::DM
 {
@@ -43,7 +48,7 @@ class SegmentReaderPool
 
     WorkQueue<MergedTaskPtr> task_queue;
     std::vector<SegmentReaderUPtr> readers;
-    Poco::Logger * log;
+    LoggerPtr log;
 };
 
 // SegmentReaderPoolManager is a NUMA-aware singleton that manages several SegmentReaderPool objects.
@@ -74,7 +79,7 @@ class SegmentReaderPoolManager
     SegmentReaderPoolManager();
     std::vector<std::unique_ptr<SegmentReaderPool>> reader_pools;
     std::unordered_set<std::thread::id> reader_ids;
-    Poco::Logger * log;
+    LoggerPtr log;
 };
 
 } // namespace DB::DM
diff --git a/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp b/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp
index dd26d1b0637..52dbfe25fee 100644
--- a/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp
+++ b/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp
@@ -221,8 +221,6 @@ void applyAlter(ColumnDefines & table_columns,
     /// Caller should ensure the command is legal.
     /// eg. The column to modify/drop/rename must exist, the column to add must not exist, the new column name of rename must not exists.
 
-    Poco::Logger * log = &Poco::Logger::get("SchemaUpdate");
-
     if (command.type == AlterCommand::MODIFY_COLUMN)
     {
         // find column define and then apply modify
@@ -248,7 +246,7 @@ void applyAlter(ColumnDefines & table_columns,
         {
             // Fall back to find column by name, this path should only call by tests.
             LOG_WARNING(
-                log,
+                Logger::get(),
                 "Try to apply alter to column: {}, id: {},"
                 " but not found by id, fall back locating col by name.",
                 command.column_name,
@@ -311,7 +309,7 @@ void applyAlter(ColumnDefines & table_columns,
     }
     else
     {
-        LOG_WARNING(log, "receive unknown alter command, type: {}", static_cast<Int32>(command.type));
+        LOG_WARNING(Logger::get(), "receive unknown alter command, type: {}", static_cast<Int32>(command.type));
     }
 }
 
diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h
index 1d3016e2853..87271b0e889 100644
--- a/dbms/src/Storages/IStorage.h
+++ b/dbms/src/Storages/IStorage.h
@@ -61,7 +61,7 @@ class IStorage : public std::enable_shared_from_this<IStorage>
     , public ITableDeclaration
 {
 public:
-    /// The main name of the table type (for example, StorageMergeTree).
+    /// The main name of the table type (for example, StorageDeltaMerge).
     virtual std::string getName() const = 0;
 
     /** The name of the table.
diff --git a/dbms/src/Storages/MarkCache.h b/dbms/src/Storages/MarkCache.h
index fe96636a046..c3167486181 100644
--- a/dbms/src/Storages/MarkCache.h
+++ b/dbms/src/Storages/MarkCache.h
@@ -42,7 +42,7 @@ struct MarksWeightFunction
 };
 
 
-/** Cache of 'marks' for StorageMergeTree.
+/** Cache of 'marks' for StorageDeltaMerge.
   * Marks is an index structure that addresses ranges in column file, corresponding to ranges of primary key.
   */
 class MarkCache : public LRUCache<String, MarksInCompressedFile, std::hash<String>, MarksWeightFunction>
diff --git a/dbms/src/Storages/Page/V2/PageFile.cpp b/dbms/src/Storages/Page/V2/PageFile.cpp
index e99f96d7cc0..b6dfd57fde8 100644
--- a/dbms/src/Storages/Page/V2/PageFile.cpp
+++ b/dbms/src/Storages/Page/V2/PageFile.cpp
@@ -94,7 +94,7 @@ std::pair<ByteBuffer, ByteBuffer> genWriteData( //
 
     meta_write_bytes += sizeof(WBSize) + sizeof(PageFormat::Version) + sizeof(WriteBatch::SequenceID);
 
-    for (auto & write : wb.getWrites())
+    for (auto & write : wb.getMutWrites())
     {
         meta_write_bytes += sizeof(IsPut);
         // We don't serialize `PUT_EXTERNAL` for V2, just convert it to `PUT`
@@ -138,7 +138,7 @@ std::pair<ByteBuffer, ByteBuffer> genWriteData( //
     PageUtil::put(meta_pos, wb.getSequence());
 
     PageOffset page_data_file_off = page_file.getDataFileAppendPos();
-    for (auto & write : wb.getWrites())
+    for (auto & write : wb.getMutWrites())
     {
         // We don't serialize `PUT_EXTERNAL` for V2, just convert it to `PUT`
         if (write.type == WriteBatchWriteType::PUT_EXTERNAL)
diff --git a/dbms/src/Storages/Page/V3/BlobStore.cpp b/dbms/src/Storages/Page/V3/BlobStore.cpp
index 8da5bcafd5b..4aa683d6bb1 100644
--- a/dbms/src/Storages/Page/V3/BlobStore.cpp
+++ b/dbms/src/Storages/Page/V3/BlobStore.cpp
@@ -168,7 +168,7 @@ typename BlobStore<Trait>::PageEntriesEdit
 BlobStore<Trait>::handleLargeWrite(typename Trait::WriteBatch & wb, const WriteLimiterPtr & write_limiter)
 {
     PageEntriesEdit edit;
-    for (auto & write : wb.getWrites())
+    for (auto & write : wb.getMutWrites())
     {
         switch (write.type)
         {
@@ -315,7 +315,7 @@ BlobStore<Trait>::write(typename Trait::WriteBatch & wb, const WriteLimiterPtr &
 
     size_t offset_in_allocated = 0;
 
-    for (auto & write : wb.getWrites())
+    for (auto & write : wb.getMutWrites())
     {
         switch (write.type)
         {
diff --git a/dbms/src/Storages/Page/V3/PageDirectory.cpp b/dbms/src/Storages/Page/V3/PageDirectory.cpp
index dfb9f1c740c..1bb6d5e0dce 100644
--- a/dbms/src/Storages/Page/V3/PageDirectory.cpp
+++ b/dbms/src/Storages/Page/V3/PageDirectory.cpp
@@ -31,6 +31,7 @@
 #include <Storages/Page/WriteBatch.h>
 #include <common/logger_useful.h>
 
+#include <magic_enum.hpp>
 #include <memory>
 #include <mutex>
 #include <optional>
@@ -442,7 +443,7 @@ VersionedPageEntries<Trait>::resolveToPageId(UInt64 seq, bool ignore_delete, Pag
     }
     else
     {
-        LOG_WARNING(&Poco::Logger::get("VersionedPageEntries"), "Can't resolve the EditRecordType {}", static_cast<Int32>(type));
+        LOG_WARNING(Logger::get(), "Can't resolve the EditRecordType {}", magic_enum::enum_name(type));
     }
 
     return {ResolveResult::FAIL, Trait::PageIdTrait::getInvalidID(), PageVersion(0)};
diff --git a/dbms/src/Storages/Page/V3/Universal/UniversalWriteBatch.h b/dbms/src/Storages/Page/V3/Universal/UniversalWriteBatch.h
index bfb09307e40..14fe6b891dc 100644
--- a/dbms/src/Storages/Page/V3/Universal/UniversalWriteBatch.h
+++ b/dbms/src/Storages/Page/V3/Universal/UniversalWriteBatch.h
@@ -61,17 +61,14 @@ class UniversalWriteBatch : private boost::noncopyable
             offsets.emplace_back(off, 0);
             off += data_sz;
         }
-        if (unlikely(!data_sizes.empty() && off != size))
-        {
-            throw Exception(fmt::format(
-                                "Try to put Page with fields, but page size and fields total size not match "
-                                "[page_id={}] [num_fields={}] [page_size={}] [all_fields_size={}]",
-                                page_id,
-                                data_sizes.size(),
-                                size,
-                                off),
-                            ErrorCodes::LOGICAL_ERROR);
-        }
+
+        RUNTIME_CHECK_MSG(data_sizes.empty() || off == size,
+                          "Try to put Page with fields, but page size and fields total size not match "
+                          "[page_id={}] [num_fields={}] [page_size={}] [all_fields_size={}]",
+                          page_id,
+                          data_sizes.size(),
+                          size,
+                          off);
 
         Write w{WriteBatchWriteType::PUT, page_id, tag, read_buffer, size, "", std::move(offsets)};
         total_data_size += size;
@@ -113,7 +110,7 @@ class UniversalWriteBatch : private boost::noncopyable
     {
         return writes;
     }
-    Writes & getWrites()
+    Writes & getMutWrites()
     {
         return writes;
     }
diff --git a/dbms/src/Storages/Page/WriteBatch.h b/dbms/src/Storages/Page/WriteBatch.h
index cb6be591be3..209322ef68b 100644
--- a/dbms/src/Storages/Page/WriteBatch.h
+++ b/dbms/src/Storages/Page/WriteBatch.h
@@ -173,7 +173,7 @@ class WriteBatch : private boost::noncopyable
     {
         return writes;
     }
-    Writes & getWrites()
+    Writes & getMutWrites()
     {
         return writes;
     }
diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp
index 0e572b13573..bfe7cbc125d 100644
--- a/libs/libdaemon/src/BaseDaemon.cpp
+++ b/libs/libdaemon/src/BaseDaemon.cpp
@@ -1156,11 +1156,12 @@ void BaseDaemon::initialize(Application & self)
 
 void BaseDaemon::logRevision() const
 {
-    Logger::root().information("Welcome to TiFlash");
-    Logger::root().information("Starting daemon with revision " + Poco::NumberFormatter::format(ClickHouseRevision::get()));
+    auto * log = &Logger::root();
+    LOG_INFO(log, "Welcome to TiFlash");
+    LOG_INFO(log, "Starting daemon with revision " + Poco::NumberFormatter::format(ClickHouseRevision::get()));
     std::stringstream ss;
     TiFlashBuildInfo::outputDetail(ss);
-    Logger::root().information("TiFlash build info: " + ss.str());
+    LOG_INFO(log, "TiFlash build info: {}", ss.str());
 }
 
 /// Used for exitOnTaskError()
diff --git a/tests/_env.sh b/tests/_env.sh
index 7803181c4a1..09e20cb5671 100644
--- a/tests/_env.sh
+++ b/tests/_env.sh
@@ -36,7 +36,7 @@ export storage_server="127.0.0.1"
 export storage_port="9000"
 
 # Default database for scripts
-export storage_db="default"
+export storage_db="system"
 
 # TiDB address
 export tidb_server="127.0.0.1"
diff --git a/tests/delta-merge-test/query/misc/timestamp_rough_set_filter.test b/tests/delta-merge-test/query/misc/timestamp_rough_set_filter.test
index b7a70d25fa6..a956f9a2775 100644
--- a/tests/delta-merge-test/query/misc/timestamp_rough_set_filter.test
+++ b/tests/delta-merge-test/query/misc/timestamp_rough_set_filter.test
@@ -18,11 +18,14 @@
 => DBGInvoke __enable_schema_sync_service('true')
 
 => DBGInvoke __drop_tidb_table(default, test)
+=> DBGInvoke __drop_tidb_db(default)
 => drop table if exists default.test
+=> drop database if exists default
 
 => DBGInvoke __set_flush_threshold(1000000, 1000000)
 
 # Data.
+=> DBGInvoke __mock_tidb_db(default)
 => DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64, col_2 default \'asTiDBType|timestamp(5)\'')
 => DBGInvoke __refresh_schemas()
 => DBGInvoke __put_region(4, 0, 100, default, test)
diff --git a/tests/delta-merge-test/run.sh b/tests/delta-merge-test/run.sh
index 53e49d2d418..cd841ebd949 100755
--- a/tests/delta-merge-test/run.sh
+++ b/tests/delta-merge-test/run.sh
@@ -22,11 +22,7 @@ set -xe
 
 check_env
 
-# We need to separate mock-test for dt and tmt, since this behavior
-# is different in some tests
-# * "tmt" engine ONLY support disable_bg_flush = false.
-# * "dt"  engine ONLY support disable_bg_flush = true.
-# (only tics0 up) (for engine DetlaTree)
+# (only tics0 up)
 docker-compose -f mock-test-dt.yaml down
 clean_data_log
 
diff --git a/tests/docker/_env.sh b/tests/docker/_env.sh
index 1b92af55bb9..abe39c3c699 100755
--- a/tests/docker/_env.sh
+++ b/tests/docker/_env.sh
@@ -26,7 +26,7 @@ export storage_server="127.0.0.1"
 export storage_port="9000"
 
 # Default database for scripts
-export storage_db="default"
+export storage_db="system"
 
 # TiDB address
 export tidb_server="tidb0"
diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test%2Dtbl/stable/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test%2Dtbl/stable/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/data/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/data/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/log/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/log/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/meta/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/meta/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test/test%2Dtbl/stable/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test/test%2Dtbl/stable/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/data/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/data/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/log/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/log/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/meta/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/meta/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test%2Ddb.sql b/tests/testdata/flash-1136-v3.1.0/data0/metadata/test%2Ddb.sql
deleted file mode 100644
index c97c56faf67..00000000000
--- a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test%2Ddb.sql
+++ /dev/null
@@ -1,16 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH DATABASE `test-db`
-ENGINE = Ordinary
diff --git a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test%2Ddb/test%2Dtbl.sql b/tests/testdata/flash-1136-v3.1.0/data0/metadata/test%2Ddb/test%2Dtbl.sql
deleted file mode 100644
index 18be51be751..00000000000
--- a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test%2Ddb/test%2Dtbl.sql
+++ /dev/null
@@ -1,20 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH TABLE `test-tbl`
-(
-    pk Nullable(Int32),
-    _tidb_rowid Int64
-)
-ENGINE = DeltaMerge(_tidb_rowid, '{"cols":[{"comment":"","default":null,"id":1,"name":{"L":"pk","O":"pk"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":66,"name":{"L":"test-tbl","O":"test-tbl"},"partition":null,"pk_is_handle":false,"schema_version":36,"state":5,"update_timestamp":417293448885567496}')
diff --git a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test.sql b/tests/testdata/flash-1136-v3.1.0/data0/metadata/test.sql
deleted file mode 100644
index 897b444f60b..00000000000
--- a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test.sql
+++ /dev/null
@@ -1,16 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH DATABASE `test`
-ENGINE = Ordinary
diff --git a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test/test%2Dtbl.sql b/tests/testdata/flash-1136-v3.1.0/data0/metadata/test/test%2Dtbl.sql
deleted file mode 100644
index f6e5c321a91..00000000000
--- a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test/test%2Dtbl.sql
+++ /dev/null
@@ -1,20 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH TABLE `test-tbl`
-(
-    pk Nullable(Int32),
-    _tidb_rowid Int64
-)
-ENGINE = DeltaMerge(_tidb_rowid, '{"cols":[{"comment":"","default":null,"id":1,"name":{"L":"pk","O":"pk"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":666,"name":{"L":"test-tbl","O":"test-tbl"},"partition":null,"pk_is_handle":false,"schema_version":36,"state":5,"update_timestamp":417293448885567496}')
diff --git a/tests/testdata/flash-1136-v3.1.0/data1/data/test%2Ddb/test%2Dtbl/stable/.gitignore b/tests/testdata/flash-1136-v3.1.0/data1/data/test%2Ddb/test%2Dtbl/stable/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/data/.gitignore b/tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/data/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/log/.gitignore b/tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/log/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/meta/.gitignore b/tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/meta/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136/data0/data/test-db/test-tbl/stable/.gitignore b/tests/testdata/flash-1136/data0/data/test-db/test-tbl/stable/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136/data0/data/test/test-tbl/data/.gitignore b/tests/testdata/flash-1136/data0/data/test/test-tbl/data/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136/data0/data/test/test-tbl/log/.gitignore b/tests/testdata/flash-1136/data0/data/test/test-tbl/log/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136/data0/data/test/test-tbl/meta/.gitignore b/tests/testdata/flash-1136/data0/data/test/test-tbl/meta/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136/data0/data/test/test-tbl/stable/.gitignore b/tests/testdata/flash-1136/data0/data/test/test-tbl/stable/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/flash-1136/data0/metadata/test%2Ddb.sql b/tests/testdata/flash-1136/data0/metadata/test%2Ddb.sql
deleted file mode 100644
index c97c56faf67..00000000000
--- a/tests/testdata/flash-1136/data0/metadata/test%2Ddb.sql
+++ /dev/null
@@ -1,16 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH DATABASE `test-db`
-ENGINE = Ordinary
diff --git a/tests/testdata/flash-1136/data0/metadata/test%2Ddb/test%2Dtbl.sql b/tests/testdata/flash-1136/data0/metadata/test%2Ddb/test%2Dtbl.sql
deleted file mode 100644
index 18be51be751..00000000000
--- a/tests/testdata/flash-1136/data0/metadata/test%2Ddb/test%2Dtbl.sql
+++ /dev/null
@@ -1,20 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH TABLE `test-tbl`
-(
-    pk Nullable(Int32),
-    _tidb_rowid Int64
-)
-ENGINE = DeltaMerge(_tidb_rowid, '{"cols":[{"comment":"","default":null,"id":1,"name":{"L":"pk","O":"pk"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":66,"name":{"L":"test-tbl","O":"test-tbl"},"partition":null,"pk_is_handle":false,"schema_version":36,"state":5,"update_timestamp":417293448885567496}')
diff --git a/tests/testdata/flash-1136/data0/metadata/test.sql b/tests/testdata/flash-1136/data0/metadata/test.sql
deleted file mode 100644
index 897b444f60b..00000000000
--- a/tests/testdata/flash-1136/data0/metadata/test.sql
+++ /dev/null
@@ -1,16 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH DATABASE `test`
-ENGINE = Ordinary
diff --git a/tests/testdata/flash-1136/data0/metadata/test/test%2Dtbl.sql b/tests/testdata/flash-1136/data0/metadata/test/test%2Dtbl.sql
deleted file mode 100644
index f6e5c321a91..00000000000
--- a/tests/testdata/flash-1136/data0/metadata/test/test%2Dtbl.sql
+++ /dev/null
@@ -1,20 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH TABLE `test-tbl`
-(
-    pk Nullable(Int32),
-    _tidb_rowid Int64
-)
-ENGINE = DeltaMerge(_tidb_rowid, '{"cols":[{"comment":"","default":null,"id":1,"name":{"L":"pk","O":"pk"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":666,"name":{"L":"test-tbl","O":"test-tbl"},"partition":null,"pk_is_handle":false,"schema_version":36,"state":5,"update_timestamp":417293448885567496}')
diff --git a/tests/testdata/flash-1136/data1/data/test-db/test-tbl/stable/.gitignore b/tests/testdata/flash-1136/data1/data/test-db/test-tbl/stable/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/issue-941/data0/data/test/#hello-world/data/.gitignore b/tests/testdata/issue-941/data0/data/test/#hello-world/data/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/issue-941/data0/data/test/#hello-world/log/.gitignore b/tests/testdata/issue-941/data0/data/test/#hello-world/log/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/issue-941/data0/data/test/#hello-world/meta/.gitignore b/tests/testdata/issue-941/data0/data/test/#hello-world/meta/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/issue-941/data0/data/test/#hello-world/stable/.gitignore b/tests/testdata/issue-941/data0/data/test/#hello-world/stable/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/issue-941/data0/metadata/test.sql b/tests/testdata/issue-941/data0/metadata/test.sql
deleted file mode 100644
index 897b444f60b..00000000000
--- a/tests/testdata/issue-941/data0/metadata/test.sql
+++ /dev/null
@@ -1,16 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH DATABASE `test`
-ENGINE = Ordinary
diff --git a/tests/testdata/issue-941/data0/metadata/test/%23hello%2Dworld.sql b/tests/testdata/issue-941/data0/metadata/test/%23hello%2Dworld.sql
deleted file mode 100644
index f93ab0a91b4..00000000000
--- a/tests/testdata/issue-941/data0/metadata/test/%23hello%2Dworld.sql
+++ /dev/null
@@ -1,20 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH TABLE `#hello-world`
-(
-    t Nullable(Int32),
-    _tidb_rowid Int64
-)
-ENGINE = DeltaMerge(_tidb_rowid, '{"cols":[{"comment":"","default":null,"id":1,"name":{"L":"t","O":"t"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":45,"name":{"L":"#hello-world","O":"#hello-world"},"partition":null,"pk_is_handle":false,"schema_version":23,"state":5,"update_timestamp":418360557052624904}')
diff --git a/tests/testdata/issue-941/data1/data/test/#hello-world/stable/.gitignore b/tests/testdata/issue-941/data1/data/test/#hello-world/stable/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/abc/data/.gitignore b/tests/testdata/oncall-1651/db/data/test/abc/data/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/abc/log/.gitignore b/tests/testdata/oncall-1651/db/data/test/abc/log/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/abc/meta/.gitignore b/tests/testdata/oncall-1651/db/data/test/abc/meta/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/abc/stable/.gitignore b/tests/testdata/oncall-1651/db/data/test/abc/stable/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak/data/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak/data/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak/log/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak/log/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak/meta/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak/meta/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak/stable/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak/stable/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_49/data/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_49/data/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_49/log/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_49/log/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_49/meta/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_49/meta/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_49/stable/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_49/stable/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_50/data/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_50/data/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_50/log/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_50/log/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_50/meta/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_50/meta/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_50/stable/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_50/stable/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_51/data/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_51/data/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_51/log/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_51/log/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_51/meta/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_51/meta/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_51/stable/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_51/stable/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_52/data/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_52/data/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_52/log/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_52/log/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_52/meta/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_52/meta/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_52/stable/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_52/stable/.gitignore
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/tests/testdata/oncall-1651/db/metadata/test.sql b/tests/testdata/oncall-1651/db/metadata/test.sql
deleted file mode 100644
index a92a7db72c5..00000000000
--- a/tests/testdata/oncall-1651/db/metadata/test.sql
+++ /dev/null
@@ -1,16 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH DATABASE test
-ENGINE = Ordinary
diff --git a/tests/testdata/oncall-1651/db/metadata/test/abc.sql b/tests/testdata/oncall-1651/db/metadata/test/abc.sql
deleted file mode 100644
index 9ea2b02b274..00000000000
--- a/tests/testdata/oncall-1651/db/metadata/test/abc.sql
+++ /dev/null
@@ -1,20 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH TABLE abc
-(
-    pk Nullable(Int32), 
-    _tidb_rowid Int64
-)
-ENGINE = DeltaMerge(_tidb_rowid, '{"cols":[{"comment":"","default":null,"id":1,"name":{"L":"pk","O":"pk"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":45,"name":{"L":"aaa","O":"aaa"},"partition":null,"pk_is_handle":false,"schema_version":23,"state":5,"update_timestamp":417160204721061891}')
diff --git a/tests/testdata/oncall-1651/db/metadata/test/emp_bak.sql b/tests/testdata/oncall-1651/db/metadata/test/emp_bak.sql
deleted file mode 100644
index 4be86529f02..00000000000
--- a/tests/testdata/oncall-1651/db/metadata/test/emp_bak.sql
+++ /dev/null
@@ -1,23 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH TABLE emp_bak
-(
-    id Int32, 
-    fname String, 
-    lname String, 
-    store_id Int32, 
-    department_id Int32
-)
-ENGINE = DeltaMerge(id, '{"belonging_table_id":-1,"cols":[{"comment":"","default":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":515,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":2,"name":{"L":"fname","O":"fname"},"offset":1,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":3,"name":{"L":"lname","O":"lname"},"offset":2,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":4,"name":{"L":"store_id","O":"store_id"},"offset":3,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":5,"name":{"L":"department_id","O":"department_id"},"offset":4,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":48,"name":{"L":"emp","O":"emp"},"partition":{"definitions":[{"comment":"","id":49,"name":{"L":"p0","O":"p0"}},{"comment":"","id":50,"name":{"L":"p1","O":"p1"}},{"comment":"","id":51,"name":{"L":"p2","O":"p2"}},{"comment":"","id":52,"name":{"L":"p3","O":"p3"}}],"enable":true,"expr":"`id`","num":4,"type":1},"pk_is_handle":true,"schema_version":31,"state":5,"update_timestamp":417160573256204297}')
diff --git a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_49.sql b/tests/testdata/oncall-1651/db/metadata/test/emp_bak_49.sql
deleted file mode 100644
index 315b7f24731..00000000000
--- a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_49.sql
+++ /dev/null
@@ -1,23 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH TABLE emp_bak_49
-(
-    id Int32, 
-    fname String, 
-    lname String, 
-    store_id Int32, 
-    department_id Int32
-)
-ENGINE = DeltaMerge(id, '{"belonging_table_id":48,"cols":[{"comment":"","default":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":515,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":2,"name":{"L":"fname","O":"fname"},"offset":1,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":3,"name":{"L":"lname","O":"lname"},"offset":2,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":4,"name":{"L":"store_id","O":"store_id"},"offset":3,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":5,"name":{"L":"department_id","O":"department_id"},"offset":4,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":49,"is_partition_sub_table":true,"name":{"L":"employees_49","O":"employees_49"},"partition":{"definitions":[{"comment":"","id":49,"name":{"L":"p0","O":"p0"}},{"comment":"","id":50,"name":{"L":"p1","O":"p1"}},{"comment":"","id":51,"name":{"L":"p2","O":"p2"}},{"comment":"","id":52,"name":{"L":"p3","O":"p3"}}],"enable":true,"expr":"`id`","num":4,"type":1},"pk_is_handle":true,"schema_version":25,"state":5,"update_timestamp":417160265315647498}')
diff --git a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_50.sql b/tests/testdata/oncall-1651/db/metadata/test/emp_bak_50.sql
deleted file mode 100644
index 055be9e4d91..00000000000
--- a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_50.sql
+++ /dev/null
@@ -1,23 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH TABLE emp_bak_50
-(
-    id Int32, 
-    fname String, 
-    lname String, 
-    store_id Int32, 
-    department_id Int32
-)
-ENGINE = DeltaMerge(id, '{"belonging_table_id":48,"cols":[{"comment":"","default":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":515,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":2,"name":{"L":"fname","O":"fname"},"offset":1,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":3,"name":{"L":"lname","O":"lname"},"offset":2,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":4,"name":{"L":"store_id","O":"store_id"},"offset":3,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":5,"name":{"L":"department_id","O":"department_id"},"offset":4,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":50,"is_partition_sub_table":true,"name":{"L":"employees_50","O":"employees_50"},"partition":{"definitions":[{"comment":"","id":49,"name":{"L":"p0","O":"p0"}},{"comment":"","id":50,"name":{"L":"p1","O":"p1"}},{"comment":"","id":51,"name":{"L":"p2","O":"p2"}},{"comment":"","id":52,"name":{"L":"p3","O":"p3"}}],"enable":true,"expr":"`id`","num":4,"type":1},"pk_is_handle":true,"schema_version":25,"state":5,"update_timestamp":417160265315647498}')
diff --git a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_51.sql b/tests/testdata/oncall-1651/db/metadata/test/emp_bak_51.sql
deleted file mode 100644
index f96af626e07..00000000000
--- a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_51.sql
+++ /dev/null
@@ -1,23 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH TABLE emp_bak_51
-(
-    id Int32, 
-    fname String, 
-    lname String, 
-    store_id Int32, 
-    department_id Int32
-)
-ENGINE = DeltaMerge(id, '{"belonging_table_id":48,"cols":[{"comment":"","default":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":515,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":2,"name":{"L":"fname","O":"fname"},"offset":1,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":3,"name":{"L":"lname","O":"lname"},"offset":2,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":4,"name":{"L":"store_id","O":"store_id"},"offset":3,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":5,"name":{"L":"department_id","O":"department_id"},"offset":4,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":51,"is_partition_sub_table":true,"name":{"L":"employees_51","O":"employees_51"},"partition":{"definitions":[{"comment":"","id":49,"name":{"L":"p0","O":"p0"}},{"comment":"","id":50,"name":{"L":"p1","O":"p1"}},{"comment":"","id":51,"name":{"L":"p2","O":"p2"}},{"comment":"","id":52,"name":{"L":"p3","O":"p3"}}],"enable":true,"expr":"`id`","num":4,"type":1},"pk_is_handle":true,"schema_version":25,"state":5,"update_timestamp":417160265315647498}')
diff --git a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_52.sql b/tests/testdata/oncall-1651/db/metadata/test/emp_bak_52.sql
deleted file mode 100644
index 6e1bb90b0fb..00000000000
--- a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_52.sql
+++ /dev/null
@@ -1,23 +0,0 @@
--- Copyright 2022 PingCAP, Ltd.
---
--- Licensed 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.
-
-ATTACH TABLE emp_bak_52
-(
-    id Int32, 
-    fname String, 
-    lname String, 
-    store_id Int32, 
-    department_id Int32
-)
-ENGINE = DeltaMerge(id, '{"belonging_table_id":48,"cols":[{"comment":"","default":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":515,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":2,"name":{"L":"fname","O":"fname"},"offset":1,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":3,"name":{"L":"lname","O":"lname"},"offset":2,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":4,"name":{"L":"store_id","O":"store_id"},"offset":3,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":5,"name":{"L":"department_id","O":"department_id"},"offset":4,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":52,"is_partition_sub_table":true,"name":{"L":"employees_52","O":"employees_52"},"partition":{"definitions":[{"comment":"","id":49,"name":{"L":"p0","O":"p0"}},{"comment":"","id":50,"name":{"L":"p1","O":"p1"}},{"comment":"","id":51,"name":{"L":"p2","O":"p2"}},{"comment":"","id":52,"name":{"L":"p3","O":"p3"}}],"enable":true,"expr":"`id`","num":4,"type":1},"pk_is_handle":true,"schema_version":25,"state":5,"update_timestamp":417160265315647498}')