mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
better rename
This commit is contained in:
parent
a2101c9b1c
commit
4c48b7dd80
@ -121,10 +121,10 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
|
|||||||
|
|
||||||
auto detach = [](DatabaseAtomic & db, const String & table_name_)
|
auto detach = [](DatabaseAtomic & db, const String & table_name_)
|
||||||
{
|
{
|
||||||
auto table_data_path_ = db.table_name_to_path.find(table_name_)->second;
|
auto table_data_path_saved = db.table_name_to_path.find(table_name_)->second;
|
||||||
db.tables.erase(table_name_);
|
db.tables.erase(table_name_);
|
||||||
db.table_name_to_path.erase(table_name_);
|
db.table_name_to_path.erase(table_name_);
|
||||||
return table_data_path_;
|
return table_data_path_saved;
|
||||||
};
|
};
|
||||||
|
|
||||||
auto attach = [](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_)
|
auto attach = [](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_)
|
||||||
@ -164,9 +164,9 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
|
|||||||
if (exchange)
|
if (exchange)
|
||||||
other_table_data_path = detach(other_db, to_table_name);
|
other_table_data_path = detach(other_db, to_table_name);
|
||||||
|
|
||||||
table->renameInMemory(other_db.getDatabaseName(), to_table_name);
|
table->renameInMemory({other_db.getDatabaseName(), to_table_name, table->getStorageID().uuid});
|
||||||
if (exchange)
|
if (exchange)
|
||||||
other_table->renameInMemory(getDatabaseName(), table_name);
|
other_table->renameInMemory({getDatabaseName(), table_name, other_table->getStorageID().uuid});
|
||||||
|
|
||||||
if (!inside_database)
|
if (!inside_database)
|
||||||
{
|
{
|
||||||
|
@ -57,7 +57,7 @@ std::pair<String, StoragePtr> createTableFromAST(
|
|||||||
const auto & table_function = ast_create_query.as_table_function->as<ASTFunction &>();
|
const auto & table_function = ast_create_query.as_table_function->as<ASTFunction &>();
|
||||||
const auto & factory = TableFunctionFactory::instance();
|
const auto & factory = TableFunctionFactory::instance();
|
||||||
StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table);
|
StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table);
|
||||||
storage->resetStorageID({ast_create_query.database, ast_create_query.table, ast_create_query.uuid});
|
storage->renameInMemory(ast_create_query);
|
||||||
return {ast_create_query.table, storage};
|
return {ast_create_query.table, storage};
|
||||||
}
|
}
|
||||||
/// We do not directly use `InterpreterCreateQuery::execute`, because
|
/// We do not directly use `InterpreterCreateQuery::execute`, because
|
||||||
@ -285,8 +285,7 @@ void DatabaseOnDisk::renameTable(
|
|||||||
create.uuid = UUIDHelpers::Nil;
|
create.uuid = UUIDHelpers::Nil;
|
||||||
|
|
||||||
/// Notify the table that it is renamed. It will move data to new path (if it stores data on disk) and update StorageID
|
/// Notify the table that it is renamed. It will move data to new path (if it stores data on disk) and update StorageID
|
||||||
table->rename(to_database.getTableDataPath(create), to_database.getDatabaseName(), to_table_name, table_lock);
|
table->rename(to_database.getTableDataPath(create), StorageID(create));
|
||||||
table->resetStorageID({create.database, create.table, create.uuid}); /// reset UUID
|
|
||||||
}
|
}
|
||||||
catch (const Exception &)
|
catch (const Exception &)
|
||||||
{
|
{
|
||||||
|
@ -638,7 +638,7 @@ void DDLWorker::processTask(DDLTask & task, const ZooKeeperPtr & zookeeper)
|
|||||||
if (!query_with_table->table.empty())
|
if (!query_with_table->table.empty())
|
||||||
{
|
{
|
||||||
/// It's not CREATE DATABASE
|
/// It's not CREATE DATABASE
|
||||||
auto table_id = context.resolveStorageID(*query_with_table, Context::ResolveOrdinary);
|
auto table_id = context.tryResolveStorageID(*query_with_table, Context::ResolveOrdinary);
|
||||||
storage = DatabaseCatalog::instance().tryGetTable(table_id);
|
storage = DatabaseCatalog::instance().tryGetTable(table_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -647,7 +647,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
|
|||||||
const auto & table_function = create.as_table_function->as<ASTFunction &>();
|
const auto & table_function = create.as_table_function->as<ASTFunction &>();
|
||||||
const auto & factory = TableFunctionFactory::instance();
|
const auto & factory = TableFunctionFactory::instance();
|
||||||
res = factory.get(table_function.name, context)->execute(create.as_table_function, context, create.table);
|
res = factory.get(table_function.name, context)->execute(create.as_table_function, context, create.table);
|
||||||
res->resetStorageID({create.database, create.table, create.uuid});
|
res->renameInMemory({create.database, create.table, create.uuid});
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
|
@ -14,6 +14,7 @@
|
|||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Interpreters/InterpreterShowCreateQuery.h>
|
#include <Interpreters/InterpreterShowCreateQuery.h>
|
||||||
|
|
||||||
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -72,6 +73,10 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
|
|||||||
if (!create_query && show_query && show_query->temporary)
|
if (!create_query && show_query && show_query->temporary)
|
||||||
throw Exception("Unable to show the create query of " + show_query->table + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY);
|
throw Exception("Unable to show the create query of " + show_query->table + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY);
|
||||||
|
|
||||||
|
//FIXME temporary print create query without UUID for tests (remove it)
|
||||||
|
auto & create = create_query->as<ASTCreateQuery &>();
|
||||||
|
create.uuid = UUIDHelpers::Nil;
|
||||||
|
|
||||||
std::stringstream stream;
|
std::stringstream stream;
|
||||||
formatAST(*create_query, stream, false, true);
|
formatAST(*create_query, stream, false, true);
|
||||||
String res = stream.str();
|
String res = stream.str();
|
||||||
|
@ -33,9 +33,7 @@ StorageID::StorageID(const ASTIdentifier & table_identifier_node)
|
|||||||
|
|
||||||
StorageID::StorageID(const ASTPtr & node)
|
StorageID::StorageID(const ASTPtr & node)
|
||||||
{
|
{
|
||||||
if (!node)
|
if (auto identifier = dynamic_cast<const ASTIdentifier *>(node.get()))
|
||||||
*this = StorageID();
|
|
||||||
else if (auto identifier = dynamic_cast<const ASTIdentifier *>(node.get()))
|
|
||||||
*this = StorageID(*identifier);
|
*this = StorageID(*identifier);
|
||||||
else if (auto simple_query = dynamic_cast<const ASTQueryWithTableAndOutput *>(node.get()))
|
else if (auto simple_query = dynamic_cast<const ASTQueryWithTableAndOutput *>(node.get()))
|
||||||
*this = StorageID(*simple_query);
|
*this = StorageID(*simple_query);
|
||||||
|
@ -13,6 +13,7 @@ namespace DB
|
|||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int UNEXPECTED_AST_STRUCTURE;
|
extern const int UNEXPECTED_AST_STRUCTURE;
|
||||||
|
extern const int SYNTAX_ERROR;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -163,4 +164,17 @@ void setIdentifierSpecial(ASTPtr & ast)
|
|||||||
id->semantic->special = true;
|
id->semantic->special = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
StorageID getTableIdentifier(const ASTPtr & ast)
|
||||||
|
{
|
||||||
|
if (!ast)
|
||||||
|
throw Exception("AST node is nullptr", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||||
|
const auto & identifier = dynamic_cast<const ASTIdentifier &>(*ast);
|
||||||
|
if (identifier.name_parts.size() > 2)
|
||||||
|
throw Exception("Logical error: more than two components in table expression", ErrorCodes::SYNTAX_ERROR);
|
||||||
|
|
||||||
|
if (identifier.name_parts.size() == 2)
|
||||||
|
return { identifier.name_parts[0], identifier.name_parts[1], identifier.uuid };
|
||||||
|
return { "", identifier.name, identifier.uuid };
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -66,6 +66,7 @@ private:
|
|||||||
friend struct IdentifierSemantic;
|
friend struct IdentifierSemantic;
|
||||||
friend ASTPtr createTableIdentifier(const StorageID & table_id);
|
friend ASTPtr createTableIdentifier(const StorageID & table_id);
|
||||||
friend void setIdentifierSpecial(ASTPtr & ast);
|
friend void setIdentifierSpecial(ASTPtr & ast);
|
||||||
|
friend StorageID getTableIdentifier(const ASTPtr & ast);
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
@ -78,6 +79,7 @@ void setIdentifierSpecial(ASTPtr & ast);
|
|||||||
String getIdentifierName(const IAST * ast);
|
String getIdentifierName(const IAST * ast);
|
||||||
std::optional<String> tryGetIdentifierName(const IAST * ast);
|
std::optional<String> tryGetIdentifierName(const IAST * ast);
|
||||||
bool tryGetIdentifierNameInto(const IAST * ast, String & name);
|
bool tryGetIdentifierNameInto(const IAST * ast, String & name);
|
||||||
|
StorageID getTableIdentifier(const ASTPtr & ast);
|
||||||
|
|
||||||
inline String getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); }
|
inline String getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); }
|
||||||
inline std::optional<String> tryGetIdentifierName(const ASTPtr & ast) { return tryGetIdentifierName(ast.get()); }
|
inline std::optional<String> tryGetIdentifierName(const ASTPtr & ast) { return tryGetIdentifierName(ast.get()); }
|
||||||
|
@ -194,7 +194,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
|
|||||||
ParserKeyword s_uuid("UUID");
|
ParserKeyword s_uuid("UUID");
|
||||||
UUID uuid = UUIDHelpers::Nil;
|
UUID uuid = UUIDHelpers::Nil;
|
||||||
|
|
||||||
if (allow_uuid && s_uuid.ignore(pos, expected))
|
if (table_name_with_optional_uuid && parts.size() <= 2 && s_uuid.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
ParserStringLiteral uuid_p;
|
ParserStringLiteral uuid_p;
|
||||||
ASTPtr ast_uuid;
|
ASTPtr ast_uuid;
|
||||||
|
@ -61,11 +61,12 @@ protected:
|
|||||||
class ParserCompoundIdentifier : public IParserBase
|
class ParserCompoundIdentifier : public IParserBase
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
ParserCompoundIdentifier(bool allow_uuid_ = false) : allow_uuid(allow_uuid_) {}
|
ParserCompoundIdentifier(bool table_name_with_optional_uuid_ = false)
|
||||||
|
: table_name_with_optional_uuid(table_name_with_optional_uuid_) {}
|
||||||
protected:
|
protected:
|
||||||
const char * getName() const override { return "compound identifier"; }
|
const char * getName() const override { return "compound identifier"; }
|
||||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
bool allow_uuid;
|
bool table_name_with_optional_uuid;
|
||||||
};
|
};
|
||||||
|
|
||||||
/// Just *
|
/// Just *
|
||||||
|
@ -391,7 +391,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
|||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
StorageID table_id{table};
|
StorageID table_id = getTableIdentifier(table);
|
||||||
|
|
||||||
// Shortcut for ATTACH a previously detached table
|
// Shortcut for ATTACH a previously detached table
|
||||||
if (attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon))
|
if (attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon))
|
||||||
@ -571,13 +571,14 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
|
|||||||
query->if_not_exists = if_not_exists;
|
query->if_not_exists = if_not_exists;
|
||||||
query->is_live_view = true;
|
query->is_live_view = true;
|
||||||
|
|
||||||
StorageID table_id{table};
|
StorageID table_id = getTableIdentifier(table);
|
||||||
query->database = table_id.database_name;
|
query->database = table_id.database_name;
|
||||||
query->table = table_id.table_name;
|
query->table = table_id.table_name;
|
||||||
query->uuid = table_id.uuid;
|
query->uuid = table_id.uuid;
|
||||||
query->cluster = cluster_str;
|
query->cluster = cluster_str;
|
||||||
|
|
||||||
query->to_table_id = StorageID(to_table);
|
if (to_table)
|
||||||
|
query->to_table_id = getTableIdentifier(to_table);
|
||||||
|
|
||||||
query->set(query->columns_list, columns_list);
|
query->set(query->columns_list, columns_list);
|
||||||
|
|
||||||
@ -764,13 +765,14 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
|||||||
query->is_populate = is_populate;
|
query->is_populate = is_populate;
|
||||||
query->replace_view = replace_view;
|
query->replace_view = replace_view;
|
||||||
|
|
||||||
StorageID table_id{table};
|
StorageID table_id = getTableIdentifier(table);
|
||||||
query->database = table_id.database_name;
|
query->database = table_id.database_name;
|
||||||
query->table = table_id.table_name;
|
query->table = table_id.table_name;
|
||||||
query->uuid = table_id.uuid;
|
query->uuid = table_id.uuid;
|
||||||
query->cluster = cluster_str;
|
query->cluster = cluster_str;
|
||||||
|
|
||||||
query->to_table_id = StorageID(to_table);
|
if (to_table)
|
||||||
|
query->to_table_id = getTableIdentifier(to_table);
|
||||||
|
|
||||||
query->set(query->columns_list, columns_list);
|
query->set(query->columns_list, columns_list);
|
||||||
query->set(query->storage, storage);
|
query->set(query->storage, storage);
|
||||||
|
@ -416,17 +416,10 @@ StorageID IStorage::getStorageID() const
|
|||||||
return storage_id;
|
return storage_id;
|
||||||
}
|
}
|
||||||
|
|
||||||
void IStorage::renameInMemory(const String & new_database_name, const String & new_table_name)
|
void IStorage::renameInMemory(const StorageID & new_table_id)
|
||||||
{
|
{
|
||||||
std::lock_guard lock(id_mutex);
|
std::lock_guard lock(id_mutex);
|
||||||
storage_id.database_name = new_database_name;
|
storage_id = new_table_id;
|
||||||
storage_id.table_name = new_table_name;
|
|
||||||
}
|
|
||||||
|
|
||||||
void IStorage::resetStorageID(const StorageID & actual_table_id)
|
|
||||||
{
|
|
||||||
std::lock_guard lock(id_mutex);
|
|
||||||
storage_id = actual_table_id;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -95,11 +95,6 @@ public:
|
|||||||
/// The name of the table.
|
/// The name of the table.
|
||||||
StorageID getStorageID() const;
|
StorageID getStorageID() const;
|
||||||
|
|
||||||
/// This method is required to set actual storage id for storage created by table function.
|
|
||||||
/// Also it's required for moving table between Ordinary and Atomic databases. Do not use it for other purposes.
|
|
||||||
/// TODO refactor table names in table functions
|
|
||||||
void resetStorageID(const StorageID & actual_table_id);
|
|
||||||
|
|
||||||
/// Returns true if the storage receives data from a remote server or servers.
|
/// Returns true if the storage receives data from a remote server or servers.
|
||||||
virtual bool isRemote() const { return false; }
|
virtual bool isRemote() const { return false; }
|
||||||
|
|
||||||
@ -328,18 +323,18 @@ public:
|
|||||||
* Renaming a name in a file with metadata, the name in the list of tables in the RAM, is done separately.
|
* Renaming a name in a file with metadata, the name in the list of tables in the RAM, is done separately.
|
||||||
* In this function, you need to rename the directory with the data, if any.
|
* In this function, you need to rename the directory with the data, if any.
|
||||||
* Called when the table structure is locked for write.
|
* Called when the table structure is locked for write.
|
||||||
|
* Table UUID must remain unchanged, unless table moved between Ordinary and Atomic databases.
|
||||||
*/
|
*/
|
||||||
virtual void rename(const String & /*new_path_to_table_data*/, const String & new_database_name, const String & new_table_name,
|
virtual void rename(const String & /*new_path_to_table_data*/, const StorageID & new_table_id)
|
||||||
TableStructureWriteLockHolder &)
|
|
||||||
{
|
{
|
||||||
renameInMemory(new_database_name, new_table_name);
|
renameInMemory(new_table_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Just updates names of database and table without moving any data on disk
|
* Just updates names of database and table without moving any data on disk
|
||||||
* Can be called directly only from DatabaseAtomic.
|
* Can be called directly only from DatabaseAtomic.
|
||||||
*/
|
*/
|
||||||
virtual void renameInMemory(const String & new_database_name, const String & new_table_name);
|
virtual void renameInMemory(const StorageID & new_table_id);
|
||||||
|
|
||||||
/** ALTER tables in the form of column changes that do not affect the change to Storage or its parameters.
|
/** ALTER tables in the form of column changes that do not affect the change to Storage or its parameters.
|
||||||
* This method must fully execute the ALTER query, taking care of the locks itself.
|
* This method must fully execute the ALTER query, taking care of the locks itself.
|
||||||
|
@ -1298,9 +1298,7 @@ void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_re
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergeTreeData::rename(
|
void MergeTreeData::rename(const String & new_table_path, const StorageID & new_table_id)
|
||||||
const String & new_table_path, const String & new_database_name,
|
|
||||||
const String & new_table_name, TableStructureWriteLockHolder &)
|
|
||||||
{
|
{
|
||||||
auto disks = getStoragePolicy()->getDisks();
|
auto disks = getStoragePolicy()->getDisks();
|
||||||
|
|
||||||
@ -1320,7 +1318,7 @@ void MergeTreeData::rename(
|
|||||||
global_context.dropCaches();
|
global_context.dropCaches();
|
||||||
|
|
||||||
relative_data_path = new_table_path;
|
relative_data_path = new_table_path;
|
||||||
renameInMemory(new_database_name, new_table_name);
|
renameInMemory(new_table_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergeTreeData::dropAllData()
|
void MergeTreeData::dropAllData()
|
||||||
|
@ -511,8 +511,7 @@ public:
|
|||||||
/// Moves the entire data directory.
|
/// Moves the entire data directory.
|
||||||
/// Flushes the uncompressed blocks cache and the marks cache.
|
/// Flushes the uncompressed blocks cache and the marks cache.
|
||||||
/// Must be called with locked lockStructureForAlter().
|
/// Must be called with locked lockStructureForAlter().
|
||||||
void rename(const String & new_table_path, const String & new_database_name,
|
void rename(const String & new_table_path, const StorageID & new_table_id) override;
|
||||||
const String & new_table_name, TableStructureWriteLockHolder &) override;
|
|
||||||
|
|
||||||
/// Check if the ALTER can be performed:
|
/// Check if the ALTER can be performed:
|
||||||
/// - all needed columns are present.
|
/// - all needed columns are present.
|
||||||
|
@ -717,12 +717,11 @@ void StorageDistributed::flushClusterNodesAllData()
|
|||||||
node.second.flushAllData();
|
node.second.flushAllData();
|
||||||
}
|
}
|
||||||
|
|
||||||
void StorageDistributed::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name,
|
void StorageDistributed::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||||
TableStructureWriteLockHolder &)
|
|
||||||
{
|
{
|
||||||
if (!relative_data_path.empty())
|
if (!relative_data_path.empty())
|
||||||
renameOnDisk(new_path_to_table_data);
|
renameOnDisk(new_path_to_table_data);
|
||||||
renameInMemory(new_database_name, new_table_name);
|
renameInMemory(new_table_id);
|
||||||
}
|
}
|
||||||
void StorageDistributed::renameOnDisk(const String & new_path_to_table_data)
|
void StorageDistributed::renameOnDisk(const String & new_path_to_table_data)
|
||||||
{
|
{
|
||||||
|
@ -82,7 +82,7 @@ public:
|
|||||||
/// Removes temporary data in local filesystem.
|
/// Removes temporary data in local filesystem.
|
||||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||||
|
|
||||||
void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||||
void renameOnDisk(const String & new_path_to_table_data);
|
void renameOnDisk(const String & new_path_to_table_data);
|
||||||
|
|
||||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) override;
|
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) override;
|
||||||
|
@ -497,7 +497,7 @@ Strings StorageFile::getDataPaths() const
|
|||||||
return paths;
|
return paths;
|
||||||
}
|
}
|
||||||
|
|
||||||
void StorageFile::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
void StorageFile::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||||
{
|
{
|
||||||
if (!is_db_table)
|
if (!is_db_table)
|
||||||
throw Exception("Can't rename table " + getStorageID().getNameForLogs() + " binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED);
|
throw Exception("Can't rename table " + getStorageID().getNameForLogs() + " binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED);
|
||||||
@ -512,7 +512,7 @@ void StorageFile::rename(const String & new_path_to_table_data, const String & n
|
|||||||
Poco::File(paths[0]).renameTo(path_new);
|
Poco::File(paths[0]).renameTo(path_new);
|
||||||
|
|
||||||
paths[0] = std::move(path_new);
|
paths[0] = std::move(path_new);
|
||||||
renameInMemory(new_database_name, new_table_name);
|
renameInMemory(new_table_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
void StorageFile::truncate(const ASTPtr & /*query*/, const Context & /* context */, TableStructureWriteLockHolder &)
|
void StorageFile::truncate(const ASTPtr & /*query*/, const Context & /* context */, TableStructureWriteLockHolder &)
|
||||||
|
@ -38,7 +38,7 @@ public:
|
|||||||
|
|
||||||
void truncate(const ASTPtr & /*query*/, const Context & /* context */, TableStructureWriteLockHolder &) override;
|
void truncate(const ASTPtr & /*query*/, const Context & /* context */, TableStructureWriteLockHolder &) override;
|
||||||
|
|
||||||
void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||||
|
|
||||||
Strings getDataPaths() const override;
|
Strings getDataPaths() const override;
|
||||||
|
|
||||||
|
@ -514,7 +514,7 @@ void StorageLog::loadMarks()
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
void StorageLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||||
{
|
{
|
||||||
std::unique_lock<std::shared_mutex> lock(rwlock);
|
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||||
|
|
||||||
@ -527,7 +527,7 @@ void StorageLog::rename(const String & new_path_to_table_data, const String & ne
|
|||||||
file.second.data_file_path = table_path + fileName(file.second.data_file_path);
|
file.second.data_file_path = table_path + fileName(file.second.data_file_path);
|
||||||
|
|
||||||
marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME;
|
marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME;
|
||||||
renameInMemory(new_database_name, new_table_name);
|
renameInMemory(new_table_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &)
|
void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &)
|
||||||
|
@ -34,11 +34,7 @@ public:
|
|||||||
|
|
||||||
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
||||||
|
|
||||||
void rename(
|
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||||
const String & new_path_to_table_data,
|
|
||||||
const String & new_database_name,
|
|
||||||
const String & new_table_name,
|
|
||||||
TableStructureWriteLockHolder &) override;
|
|
||||||
|
|
||||||
CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override;
|
CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override;
|
||||||
|
|
||||||
|
@ -327,13 +327,16 @@ void StorageMaterializedView::mutate(const MutationCommands & commands, const Co
|
|||||||
getTargetTable()->mutate(commands, context);
|
getTargetTable()->mutate(commands, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
void StorageMaterializedView::renameInMemory(const String & new_database_name, const String & new_table_name)
|
void StorageMaterializedView::renameInMemory(const StorageID & new_table_id)
|
||||||
{
|
{
|
||||||
auto old_table_id = getStorageID();
|
auto old_table_id = getStorageID();
|
||||||
///FIXME case when moving between DBs
|
auto old_target_table_name = generateInnerTableName(old_table_id);
|
||||||
if (has_inner_table && tryGetTargetTable() && !old_table_id.hasUUID())
|
auto new_target_table_name = generateInnerTableName(new_table_id);
|
||||||
|
bool inner_same_name = old_table_id.database_name == new_table_id.database_name &&
|
||||||
|
old_target_table_name == new_target_table_name;
|
||||||
|
|
||||||
|
if (has_inner_table && tryGetTargetTable() && !inner_same_name)
|
||||||
{
|
{
|
||||||
auto new_target_table_name = generateInnerTableName({new_database_name, new_table_name});
|
|
||||||
auto rename = std::make_shared<ASTRenameQuery>();
|
auto rename = std::make_shared<ASTRenameQuery>();
|
||||||
|
|
||||||
ASTRenameQuery::Table from;
|
ASTRenameQuery::Table from;
|
||||||
@ -353,7 +356,7 @@ void StorageMaterializedView::renameInMemory(const String & new_database_name, c
|
|||||||
target_table_id.table_name = new_target_table_name;
|
target_table_id.table_name = new_target_table_name;
|
||||||
}
|
}
|
||||||
|
|
||||||
IStorage::renameInMemory(new_database_name, new_table_name);
|
IStorage::renameInMemory(new_table_id);
|
||||||
// TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated
|
// TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated
|
||||||
DatabaseCatalog::instance().updateDependency(select_table_id, old_table_id, select_table_id, getStorageID());
|
DatabaseCatalog::instance().updateDependency(select_table_id, old_table_id, select_table_id, getStorageID());
|
||||||
}
|
}
|
||||||
|
@ -52,7 +52,7 @@ public:
|
|||||||
|
|
||||||
void mutate(const MutationCommands & commands, const Context & context) override;
|
void mutate(const MutationCommands & commands, const Context & context) override;
|
||||||
|
|
||||||
void renameInMemory(const String & new_database_name, const String & new_table_name) override;
|
void renameInMemory(const StorageID & new_table_id) override;
|
||||||
|
|
||||||
void shutdown() override;
|
void shutdown() override;
|
||||||
|
|
||||||
|
@ -3624,11 +3624,9 @@ void StorageReplicatedMergeTree::drop()
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageReplicatedMergeTree::rename(
|
void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||||
const String & new_path_to_table_data, const String & new_database_name,
|
|
||||||
const String & new_table_name, TableStructureWriteLockHolder & lock)
|
|
||||||
{
|
{
|
||||||
MergeTreeData::rename(new_path_to_table_data, new_database_name, new_table_name, lock);
|
MergeTreeData::rename(new_path_to_table_data, new_table_id);
|
||||||
|
|
||||||
/// Update table name in zookeeper
|
/// Update table name in zookeeper
|
||||||
auto zookeeper = getZooKeeper();
|
auto zookeeper = getZooKeeper();
|
||||||
|
@ -117,7 +117,7 @@ public:
|
|||||||
|
|
||||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||||
|
|
||||||
void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||||
|
|
||||||
bool supportsIndexForIn() const override { return true; }
|
bool supportsIndexForIn() const override { return true; }
|
||||||
|
|
||||||
|
@ -197,15 +197,14 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageSetOrJoinBase::rename(
|
void StorageSetOrJoinBase::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||||
const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
|
||||||
{
|
{
|
||||||
/// Rename directory with data.
|
/// Rename directory with data.
|
||||||
String new_path = base_path + new_path_to_table_data;
|
String new_path = base_path + new_path_to_table_data;
|
||||||
Poco::File(path).renameTo(new_path);
|
Poco::File(path).renameTo(new_path);
|
||||||
|
|
||||||
path = new_path;
|
path = new_path;
|
||||||
renameInMemory(new_database_name, new_table_name);
|
renameInMemory(new_table_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -19,7 +19,7 @@ class StorageSetOrJoinBase : public IStorage
|
|||||||
friend class SetOrJoinBlockOutputStream;
|
friend class SetOrJoinBlockOutputStream;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||||
|
|
||||||
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
||||||
|
|
||||||
|
@ -237,7 +237,7 @@ StorageStripeLog::StorageStripeLog(
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageStripeLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
void StorageStripeLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||||
{
|
{
|
||||||
std::unique_lock<std::shared_mutex> lock(rwlock);
|
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||||
|
|
||||||
@ -245,7 +245,7 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Strin
|
|||||||
|
|
||||||
table_path = new_path_to_table_data;
|
table_path = new_path_to_table_data;
|
||||||
file_checker.setPath(table_path + "sizes.json");
|
file_checker.setPath(table_path + "sizes.json");
|
||||||
renameInMemory(new_database_name, new_table_name);
|
renameInMemory(new_table_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -35,11 +35,7 @@ public:
|
|||||||
|
|
||||||
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
||||||
|
|
||||||
void rename(
|
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||||
const String & new_path_to_table_data,
|
|
||||||
const String & new_database_name,
|
|
||||||
const String & new_table_name,
|
|
||||||
TableStructureWriteLockHolder &) override;
|
|
||||||
|
|
||||||
CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override;
|
CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override;
|
||||||
|
|
||||||
|
@ -375,7 +375,7 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageTinyLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
void StorageTinyLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||||
{
|
{
|
||||||
std::unique_lock<std::shared_mutex> lock(rwlock);
|
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||||
|
|
||||||
@ -386,7 +386,7 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const String
|
|||||||
|
|
||||||
for (auto & file : files)
|
for (auto & file : files)
|
||||||
file.second.data_file_path = table_path + fileName(file.second.data_file_path);
|
file.second.data_file_path = table_path + fileName(file.second.data_file_path);
|
||||||
renameInMemory(new_database_name, new_table_name);
|
renameInMemory(new_table_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -34,11 +34,7 @@ public:
|
|||||||
|
|
||||||
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
||||||
|
|
||||||
void rename(
|
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||||
const String & new_path_to_table_data,
|
|
||||||
const String & new_database_name,
|
|
||||||
const String & new_table_name,
|
|
||||||
TableStructureWriteLockHolder &) override;
|
|
||||||
|
|
||||||
CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override;
|
CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override;
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user