implement reneme dictionary query

This commit is contained in:
Alexander Tokmakov 2020-07-17 00:41:26 +03:00
parent d10b4c504d
commit ebf98ba006
14 changed files with 160 additions and 51 deletions

View File

@ -22,6 +22,7 @@ namespace ErrorCodes
extern const int DATABASE_NOT_EMPTY;
extern const int NOT_IMPLEMENTED;
extern const int FILE_ALREADY_EXISTS;
extern const int INCORRECT_QUERY;
}
class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator
@ -124,16 +125,20 @@ void DatabaseAtomic::dropTable(const Context &, const String & table_name, bool
}
void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database,
const String & to_table_name, bool exchange)
const String & to_table_name, bool exchange, bool dictionary)
{
if (typeid(*this) != typeid(to_database))
{
if (!typeid_cast<DatabaseOrdinary *>(&to_database))
throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED);
/// Allow moving tables between Atomic and Ordinary (with table lock)
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange);
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary);
return;
}
if (exchange && dictionary)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot exchange dictionaries");
auto & other_db = dynamic_cast<DatabaseAtomic &>(to_database);
bool inside_database = this == &other_db;
@ -142,16 +147,24 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
auto detach = [](DatabaseAtomic & db, const String & table_name_)
{
auto table_data_path_saved = db.table_name_to_path.find(table_name_)->second;
auto it = db.table_name_to_path.find(table_name_);
String table_data_path_saved;
/// Path can be not set for DDL dictionaries, but it does not matter for StorageDictionary.
if (it != db.table_name_to_path.end())
table_data_path_saved = it->second;
assert(!table_data_path_saved.empty() || db.dictionaries.find(table_name_) != db.dictionaries.end());
db.tables.erase(table_name_);
db.table_name_to_path.erase(table_name_);
db.tryRemoveSymlink(table_name_);
if (!table_data_path_saved.empty())
db.tryRemoveSymlink(table_name_);
return table_data_path_saved;
};
auto attach = [](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_)
{
db.tables.emplace(table_name_, table_);
if (table_data_path_.empty())
return;
db.table_name_to_path.emplace(table_name_, table_data_path_);
db.tryCreateSymlink(table_name_, table_data_path_);
};
@ -186,6 +199,17 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
db_lock = std::unique_lock{mutex};
}
bool is_dictionary = dictionaries.find(table_name) != dictionaries.end();
if (exchange && other_db.dictionaries.find(to_table_name) != other_db.dictionaries.end())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot exchange dictionaries");
if (dictionary != is_dictionary)
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Use RENAME DICTIONARY for dictionaries and RENAME TABLE for tables.");
if (is_dictionary && !inside_database)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot move dictionary to other database");
StoragePtr table = getTableUnlocked(table_name, db_lock);
assert_can_move_mat_view(table);
StoragePtr other_table;
@ -206,13 +230,15 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
if (exchange)
other_table_data_path = detach(other_db, to_table_name);
table->renameInMemory({other_db.database_name, to_table_name, table->getStorageID().uuid});
auto old_table_id = table->getStorageID();
table->renameInMemory({other_db.database_name, to_table_name, old_table_id.uuid});
if (exchange)
other_table->renameInMemory({database_name, table_name, other_table->getStorageID().uuid});
if (!inside_database)
{
DatabaseCatalog::instance().updateUUIDMapping(table->getStorageID().uuid, other_db.shared_from_this(), table);
DatabaseCatalog::instance().updateUUIDMapping(old_table_id.uuid, other_db.shared_from_this(), table);
if (exchange)
DatabaseCatalog::instance().updateUUIDMapping(other_table->getStorageID().uuid, shared_from_this(), other_table);
}
@ -220,6 +246,12 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
attach(other_db, to_table_name, table_data_path, table);
if (exchange)
attach(*this, table_name, other_table_data_path, other_table);
if (is_dictionary)
{
auto new_table_id = StorageID(other_db.database_name, to_table_name, old_table_id.uuid);
renameDictionaryInMemoryUnlocked(old_table_id, new_table_id);
}
}
void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
@ -444,6 +476,18 @@ void DatabaseAtomic::renameDictionaryInMemoryUnlocked(const StorageID & old_name
assert(old_name.uuid == new_name.uuid);
it->second.config->setString("dictionary.database", new_name.database_name);
it->second.config->setString("dictionary.name", new_name.table_name);
auto & create = it->second.create_query->as<ASTCreateQuery &>();
create.database = new_name.database_name;
create.table = new_name.table_name;
assert(create.uuid == new_name.uuid);
if (old_name.table_name != new_name.table_name)
{
auto attach_info = std::move(it->second);
dictionaries.erase(it);
dictionaries.emplace(new_name.table_name, std::move(attach_info));
}
auto result = external_loader.getLoadResult(toString(old_name.uuid));
if (!result.object)
return;

View File

@ -33,7 +33,8 @@ public:
const String & table_name,
IDatabase & to_database,
const String & to_table_name,
bool exchange) override;
bool exchange,
bool dictionary) override;
void dropTable(const Context & context, const String & table_name, bool no_delay) override;

View File

@ -78,10 +78,11 @@ void DatabaseLazy::renameTable(
const String & table_name,
IDatabase & to_database,
const String & to_table_name,
bool exchange)
bool exchange,
bool dictionary)
{
SCOPE_EXIT({ clearExpiredTables(); });
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange);
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary);
}

View File

@ -42,7 +42,8 @@ public:
const String & table_name,
IDatabase & to_database,
const String & to_table_name,
bool exchange) override;
bool exchange,
bool dictionary) override;
void alterTable(
const Context & context,

View File

@ -217,7 +217,8 @@ void DatabaseOnDisk::dropTable(const Context & context, const String & table_nam
String table_metadata_path = getObjectMetadataPath(table_name);
String table_metadata_path_drop = table_metadata_path + drop_suffix;
String table_data_path_relative = getTableDataPath(table_name);
assert(!table_data_path_relative.empty());
if (table_data_path_relative.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Path is empty");
StoragePtr table = detachTable(table_name);
bool renamed = false;
@ -249,10 +250,13 @@ void DatabaseOnDisk::renameTable(
const String & table_name,
IDatabase & to_database,
const String & to_table_name,
bool exchange)
bool exchange,
bool dictionary)
{
if (exchange)
throw Exception("Tables can be exchanged only in Atomic databases", ErrorCodes::NOT_IMPLEMENTED);
if (dictionary)
throw Exception("Dictionaries can be renamed only in Atomic databases", ErrorCodes::NOT_IMPLEMENTED);
bool from_ordinary_to_atomic = false;
bool from_atomic_to_ordinary = false;

View File

@ -49,7 +49,8 @@ public:
const String & table_name,
IDatabase & to_database,
const String & to_table_name,
bool exchange) override;
bool exchange,
bool dictionary) override;
ASTPtr getCreateDatabaseQuery() const override;

View File

@ -79,7 +79,6 @@ void DatabaseWithDictionaries::detachDictionary(const String & dictionary_name)
void DatabaseWithDictionaries::detachDictionaryImpl(const String & dictionary_name, DictionaryAttachInfo & attach_info)
{
auto dict_id = StorageID::createEmpty();
String internal_name;

View File

@ -247,7 +247,8 @@ public:
const String & /*name*/,
IDatabase & /*to_database*/,
const String & /*to_name*/,
bool /*exchange*/)
bool /*exchange*/,
bool /*dictionary*/)
{
throw Exception(getEngineName() + ": renameTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
}

View File

@ -77,7 +77,8 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c
elem.from_table_name,
*database_catalog.getDatabase(elem.to_database_name),
elem.to_table_name,
rename.exchange);
rename.exchange,
rename.dictionary);
}
return {};

View File

@ -31,6 +31,7 @@ public:
bool exchange{false}; /// For EXCHANGE TABLES
bool database{false}; /// For RENAME DATABASE
bool dictionary{false}; /// For RENAME DICTIONARY
/** Get the text that identifies this element. */
String getID(char) const override { return "Rename"; }
@ -72,8 +73,14 @@ protected:
return;
}
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< (exchange ? "EXCHANGE TABLES " : "RENAME TABLE ") << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "");
if (exchange)
settings.ostr << "EXCHANGE TABLES ";
else if (dictionary)
settings.ostr << "RENAME DICTIONARY ";
else
settings.ostr << "RENAME TABLE ";
settings.ostr << (settings.hilite ? hilite_none : "");
for (auto it = elements.cbegin(); it != elements.cend(); ++it)
{

View File

@ -41,48 +41,51 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_rename_table("RENAME TABLE");
ParserKeyword s_exchange_tables("EXCHANGE TABLES");
ParserKeyword s_rename_dictionary("RENAME DICTIONARY");
ParserKeyword s_rename_database("RENAME DATABASE");
ParserKeyword s_to("TO");
ParserKeyword s_and("AND");
ParserToken s_comma(TokenType::Comma);
bool exchange = false;
bool dictionary = false;
if (!s_rename_table.ignore(pos, expected))
if (s_rename_table.ignore(pos, expected))
;
else if (s_exchange_tables.ignore(pos, expected))
exchange = true;
else if (s_rename_dictionary.ignore(pos, expected))
dictionary = true;
else if (s_rename_database.ignore(pos, expected))
{
if (s_exchange_tables.ignore(pos, expected))
exchange = true;
else if (s_rename_database.ignore(pos, expected))
{
ASTPtr from_db;
ASTPtr to_db;
ParserIdentifier db_name_p;
if (!db_name_p.parse(pos, from_db, expected))
return false;
if (!s_to.ignore(pos, expected))
return false;
if (!db_name_p.parse(pos, to_db, expected))
return false;
String cluster_str;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
auto query = std::make_shared<ASTRenameQuery>();
query->database = true;
query->elements.emplace({});
tryGetIdentifierNameInto(from_db, query->elements.front().from.database);
tryGetIdentifierNameInto(to_db, query->elements.front().to.database);
query->cluster = cluster_str;
node = query;
return true;
}
else
ASTPtr from_db;
ASTPtr to_db;
ParserIdentifier db_name_p;
if (!db_name_p.parse(pos, from_db, expected))
return false;
if (!s_to.ignore(pos, expected))
return false;
if (!db_name_p.parse(pos, to_db, expected))
return false;
String cluster_str;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
auto query = std::make_shared<ASTRenameQuery>();
query->database = true;
query->elements.emplace({});
tryGetIdentifierNameInto(from_db, query->elements.front().from.database);
tryGetIdentifierNameInto(to_db, query->elements.front().to.database);
query->cluster = cluster_str;
node = query;
return true;
}
else
return false;
ASTRenameQuery::Elements elements;
@ -117,6 +120,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
query->elements = elements;
query->exchange = exchange;
query->dictionary = dictionary;
return true;
}

View File

@ -0,0 +1,11 @@
dict NOT_LOADED
dict Dictionary
table Memory
dict1 NOT_LOADED
dict1 Dictionary
table Memory
test
dict2 LOADED
dict2 Dictionary
table Memory
test

View File

@ -0,0 +1,34 @@
DROP DATABASE IF EXISTS test_01191;
CREATE DATABASE test_01191 ENGINE=Atomic;
CREATE TABLE test_01191.table (n UInt64, s String) ENGINE = Memory();
CREATE DICTIONARY test_01191.dict (n UInt64, s String)
PRIMARY KEY n
LAYOUT(DIRECT())
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table' DB 'test_01191'));
INSERT INTO test_01191.table VALUES (42, 'test');
SELECT name, status FROM system.dictionaries WHERE database='test_01191';
SELECT name, engine FROM system.tables WHERE database='test_01191' ORDER BY name;
RENAME DICTIONARY test_01191.table TO test_01191.table1; -- {serverError 80}
EXCHANGE TABLES test_01191.table AND test_01191.dict; -- {serverError 48}
EXCHANGE TABLES test_01191.dict AND test_01191.table; -- {serverError 80}
RENAME TABLE test_01191.dict TO test_01191.dict1; -- {serverError 80}
RENAME DICTIONARY test_01191.dict TO default.dict1; -- {serverError 48}
RENAME DICTIONARY test_01191.dict TO test_01191.dict1;
SELECT name, status FROM system.dictionaries WHERE database='test_01191';
SELECT name, engine FROM system.tables WHERE database='test_01191' ORDER BY name;
SELECT dictGet(test_01191.dict1, 's', toUInt64(42));
RENAME DICTIONARY test_01191.dict1 TO test_01191.dict2;
SELECT name, status FROM system.dictionaries WHERE database='test_01191';
SELECT name, engine FROM system.tables WHERE database='test_01191' ORDER BY name;
SELECT dictGet(test_01191.dict2, 's', toUInt64(42));
DROP DATABASE test_01191;

View File

@ -14,7 +14,7 @@ renamed
10 45
10 45
ok
CREATE DICTIONARY test_01192_atomic.dict UUID \'00001192-0000-4000-8000-000000000002\'\n(\n `n` UInt64,\n `_part` String DEFAULT \'no\'\n)\nPRIMARY KEY n\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'mt\' DB \'test_01192\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(HASHED)
CREATE DICTIONARY test_01192_atomic.dict UUID \'00001192-0000-4000-8000-000000000002\'\n(\n `n` UInt64,\n `_part` String DEFAULT \'no\'\n)\nPRIMARY KEY n\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'mt\' DB \'test_01192\'))\nLAYOUT(DIRECT())
test_01192_atomic dict NOT_LOADED 00001192-0000-4000-8000-000000000002
no
ok