Merge pull request #31638 from ClickHouse/fix_rename_dictionary

Fix rename dictionary
This commit is contained in:
tavplubix 2021-11-24 20:25:35 +03:00 committed by GitHub
commit bebee01b34
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 102 additions and 40 deletions

View File

@ -229,15 +229,8 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_
StoragePtr table = getTableUnlocked(table_name, db_lock);
if (table->isDictionary() && !dictionary)
{
if (exchange)
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Use EXCHANGE DICTIONARIES for dictionaries and EXCHANGE TABLES for tables.");
else
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Use RENAME DICTIONARY for dictionaries and RENAME TABLE for tables.");
}
if (dictionary && !table->isDictionary())
throw Exception(ErrorCodes::INCORRECT_QUERY, "Use RENAME/EXCHANGE TABLE (instead of RENAME/EXCHANGE DICTIONARY) for tables");
table->checkTableCanBeRenamed();
assert_can_move_mat_view(table);
@ -245,6 +238,8 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_
if (exchange)
{
other_table = other_db.getTableUnlocked(to_table_name, other_db_lock);
if (dictionary && !other_table->isDictionary())
throw Exception(ErrorCodes::INCORRECT_QUERY, "Use RENAME/EXCHANGE TABLE (instead of RENAME/EXCHANGE DICTIONARY) for tables");
other_table->checkTableCanBeRenamed();
assert_can_move_mat_view(other_table);
}

View File

@ -40,6 +40,7 @@ namespace ErrorCodes
extern const int TABLE_ALREADY_EXISTS;
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
extern const int DATABASE_NOT_EMPTY;
extern const int INCORRECT_QUERY;
}
@ -349,8 +350,6 @@ void DatabaseOnDisk::renameTable(
{
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;
@ -372,7 +371,11 @@ void DatabaseOnDisk::renameTable(
ASTPtr attach_query;
/// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case.
StoragePtr table = tryGetTable(table_name, local_context);
if (dictionary && table && !table->isDictionary())
throw Exception("Use RENAME/EXCHANGE TABLE (instead of RENAME/EXCHANGE DICTIONARY) for tables", ErrorCodes::INCORRECT_QUERY);
detachTable(local_context, table_name);
UUID prev_uuid = UUIDHelpers::Nil;
try
{

View File

@ -212,19 +212,39 @@ void StorageDictionary::renameInMemory(const StorageID & new_table_id)
auto old_table_id = getStorageID();
IStorage::renameInMemory(new_table_id);
bool has_configuration = false;
assert((location == Location::SameDatabaseAndNameAsDictionary) == (getConfiguration().get() != nullptr));
if (location != Location::SameDatabaseAndNameAsDictionary)
return;
/// It's DDL dictionary, need to update configuration and reload
bool move_to_atomic = old_table_id.uuid == UUIDHelpers::Nil && new_table_id.uuid != UUIDHelpers::Nil;
bool move_to_ordinary = old_table_id.uuid != UUIDHelpers::Nil && new_table_id.uuid == UUIDHelpers::Nil;
assert(old_table_id.uuid == new_table_id.uuid || move_to_atomic || move_to_ordinary);
{
std::lock_guard<std::mutex> lock(dictionary_config_mutex);
if (configuration)
{
has_configuration = true;
configuration->setString("dictionary.database", new_table_id.database_name);
configuration->setString("dictionary.name", new_table_id.table_name);
}
configuration->setString("dictionary.database", new_table_id.database_name);
configuration->setString("dictionary.name", new_table_id.table_name);
if (move_to_atomic)
configuration->setString("dictionary.uuid", toString(new_table_id.uuid));
else if (move_to_ordinary)
configuration->remove("dictionary.uuid");
}
if (has_configuration)
/// Dictionary is moving between databases of different engines or is renaming inside Ordinary database
bool recreate_dictionary = old_table_id.uuid == UUIDHelpers::Nil || new_table_id.uuid == UUIDHelpers::Nil;
if (recreate_dictionary)
{
/// It's too hard to update both name and uuid, better to reload dictionary with new name
removeDictionaryConfigurationFromRepository();
auto repository = std::make_unique<ExternalLoaderDictionaryStorageConfigRepository>(*this);
remove_repository_callback = getContext()->getExternalDictionariesLoader().addConfigRepository(std::move(repository));
/// Dictionary will be reloaded lazily to avoid exceptions in the middle of renaming
}
else
{
const auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader();
auto result = external_dictionaries_loader.getLoadResult(old_table_id.getInternalDictionaryName());

View File

@ -4,8 +4,13 @@
1 mv1 before moving tablesmv1
1 mv2 before moving tablesmv2
1 src before moving tables
asdf
asdf
test_01155_ordinary dict1 00000000-0000-0000-0000-000000000000
asdf
ordinary:
.inner.mv1
dict
dist
dst
mv1
@ -14,6 +19,7 @@ src
ordinary after rename:
atomic after rename:
.inner_id.
dict
dist
dst
mv1
@ -33,12 +39,14 @@ src
3 src after moving tables
3 src after renaming database
3 src before moving tables
.inner_id.
dist
dst
mv1
mv2
src
asdf
test_01155_ordinary .inner_id.
test_01155_ordinary dict
test_01155_ordinary dist
test_01155_ordinary dst
test_01155_ordinary mv1
test_01155_ordinary mv2
test_01155_ordinary src
CREATE DATABASE test_01155_atomic\nENGINE = Atomic
4 .inner.mv1 after renaming databasemv1
4 .inner.mv1 after renaming tablesmv1
@ -60,8 +68,11 @@ CREATE DATABASE test_01155_atomic\nENGINE = Atomic
4 src after renaming database
4 src after renaming tables
4 src before moving tables
asdf
test_01155_ordinary dict 00000000-0000-0000-0000-000000000000
test_01155_ordinary:
.inner.mv1
dict
dist
dst
mv1

View File

@ -7,15 +7,27 @@ CREATE DATABASE test_01155_ordinary ENGINE=Ordinary;
CREATE DATABASE test_01155_atomic ENGINE=Atomic;
USE test_01155_ordinary;
CREATE TABLE src (s String) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY s;
CREATE MATERIALIZED VIEW mv1 (s String) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY s AS SELECT (*,).1 || 'mv1' as s FROM src;
CREATE TABLE dst (s String) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY s;
CREATE MATERIALIZED VIEW mv2 TO dst (s String) AS SELECT (*,).1 || 'mv2' as s FROM src;
CREATE TABLE dist (s String) Engine=Distributed(test_shard_localhost, test_01155_ordinary, src);
INSERT INTO dist VALUES ('before moving tables');
CREATE TABLE src (s String, x String DEFAULT 'a') ENGINE=MergeTree() PARTITION BY tuple() ORDER BY s;
CREATE MATERIALIZED VIEW mv1 (s String, x String DEFAULT 'b') ENGINE=MergeTree() PARTITION BY tuple() ORDER BY s AS SELECT (*,).1 || 'mv1' as s FROM src;
CREATE TABLE dst (s String, x String DEFAULT 'c') ENGINE=MergeTree() PARTITION BY tuple() ORDER BY s;
CREATE MATERIALIZED VIEW mv2 TO dst (s String, x String DEFAULT 'd') AS SELECT (*,).1 || 'mv2' as s FROM src;
CREATE TABLE dist (s String, x String DEFAULT 'asdf') ENGINE=Distributed(test_shard_localhost, test_01155_ordinary, src);
INSERT INTO dist(s) VALUES ('before moving tables');
SYSTEM FLUSH DISTRIBUTED dist;
CREATE DICTIONARY dict (s String, x String DEFAULT 'qwerty') PRIMARY KEY s
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dist' DB 'test_01155_ordinary'))
LIFETIME(MIN 0 MAX 2) LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 123));
-- FIXME Cannot convert column `1` because it is non constant in source stream but must be constant in result
SELECT materialize(1), substr(_table, 1, 10), s FROM merge('test_01155_ordinary', '') ORDER BY _table, s;
SELECT dictGet('test_01155_ordinary.dict', 'x', 'before moving tables');
RENAME DICTIONARY test_01155_ordinary.dict TO test_01155_ordinary.dict1;
SELECT dictGet('test_01155_ordinary.dict1', 'x', 'before moving tables');
SELECT database, name, uuid FROM system.dictionaries WHERE database='test_01155_ordinary';
RENAME TABLE test_01155_ordinary.dict1 TO test_01155_ordinary.dict;
SELECT dictGet('test_01155_ordinary.dict', 'x', 'before moving tables');
-- Move tables with materialized views from Ordinary to Atomic
SELECT 'ordinary:';
@ -24,7 +36,10 @@ RENAME TABLE test_01155_ordinary.mv1 TO test_01155_atomic.mv1;
RENAME TABLE test_01155_ordinary.mv2 TO test_01155_atomic.mv2;
RENAME TABLE test_01155_ordinary.dst TO test_01155_atomic.dst;
RENAME TABLE test_01155_ordinary.src TO test_01155_atomic.src;
SET check_table_dependencies=0;
RENAME TABLE test_01155_ordinary.dist TO test_01155_atomic.dist;
SET check_table_dependencies=1;
RENAME DICTIONARY test_01155_ordinary.dict TO test_01155_atomic.dict;
SELECT 'ordinary after rename:';
SELECT substr(name, 1, 10) FROM system.tables WHERE database='test_01155_ordinary';
SELECT 'atomic after rename:';
@ -32,17 +47,19 @@ SELECT substr(name, 1, 10) FROM system.tables WHERE database='test_01155_atomic'
DROP DATABASE test_01155_ordinary;
USE default;
INSERT INTO test_01155_atomic.src VALUES ('after moving tables');
SELECT materialize(2), substr(_table, 1, 10), s FROM merge('test_01155_atomic', '') ORDER BY _table, s; -- { serverError 81 }
INSERT INTO test_01155_atomic.src(s) VALUES ('after moving tables');
--SELECT materialize(2), substr(_table, 1, 10), s FROM merge('test_01155_atomic', '') ORDER BY _table, s; -- { serverError 81 }
--SELECT dictGet('test_01155_ordinary.dict', 'x', 'after moving tables'); -- { serverError 36 }
RENAME DATABASE test_01155_atomic TO test_01155_ordinary;
USE test_01155_ordinary;
INSERT INTO dist VALUES ('after renaming database');
INSERT INTO dist(s) VALUES ('after renaming database');
SYSTEM FLUSH DISTRIBUTED dist;
SELECT materialize(3), substr(_table, 1, 10), s FROM merge('test_01155_ordinary', '') ORDER BY _table, s;
SELECT dictGet('test_01155_ordinary.dict', 'x', 'after renaming database');
SELECT substr(name, 1, 10) FROM system.tables WHERE database='test_01155_ordinary';
SELECT database, substr(name, 1, 10) FROM system.tables WHERE database like 'test_01155_%';
-- Move tables back
RENAME DATABASE test_01155_ordinary TO test_01155_atomic;
@ -55,10 +72,13 @@ RENAME TABLE test_01155_atomic.mv2 TO test_01155_ordinary.mv2;
RENAME TABLE test_01155_atomic.dst TO test_01155_ordinary.dst;
RENAME TABLE test_01155_atomic.src TO test_01155_ordinary.src;
RENAME TABLE test_01155_atomic.dist TO test_01155_ordinary.dist;
RENAME DICTIONARY test_01155_atomic.dict TO test_01155_ordinary.dict;
INSERT INTO dist VALUES ('after renaming tables');
INSERT INTO dist(s) VALUES ('after renaming tables');
SYSTEM FLUSH DISTRIBUTED dist;
SELECT materialize(4), substr(_table, 1, 10), s FROM merge('test_01155_ordinary', '') ORDER BY _table, s;
SELECT dictGet('test_01155_ordinary.dict', 'x', 'after renaming tables');
SELECT database, name, uuid FROM system.dictionaries WHERE database='test_01155_ordinary';
SELECT 'test_01155_ordinary:';
SHOW TABLES FROM test_01155_ordinary;
SELECT 'test_01155_atomic:';

View File

@ -1,7 +1,13 @@
dict NOT_LOADED
_ Memory
dict Dictionary
dict1 NOT_LOADED
t Memory
t NOT_LOADED
_ Memory
dict Memory
t Dictionary
test
dict1 LOADED
_ Memory
dict1 Dictionary
test

View File

@ -4,6 +4,7 @@ DROP DATABASE IF EXISTS test_01191;
CREATE DATABASE test_01191 ENGINE=Atomic;
CREATE TABLE test_01191._ (n UInt64, s String) ENGINE = Memory();
CREATE TABLE test_01191.t (n UInt64, s String) ENGINE = Memory();
CREATE DICTIONARY test_01191.dict (n UInt64, s String)
PRIMARY KEY n
@ -16,9 +17,15 @@ 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 60}
EXCHANGE TABLES test_01191.table AND test_01191.dict; -- {serverError 60}
EXCHANGE TABLES test_01191.dict AND test_01191.table; -- {serverError 80}
RENAME TABLE test_01191.dict TO test_01191.dict1; -- {serverError 80}
EXCHANGE DICTIONARIES test_01191._ AND test_01191.dict; -- {serverError 80}
EXCHANGE TABLES test_01191.t AND test_01191.dict;
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.t, 's', toUInt64(42));
EXCHANGE TABLES test_01191.dict AND test_01191.t;
RENAME DICTIONARY test_01191.t TO test_01191.dict1; -- {serverError 80}
DROP DICTIONARY test_01191.t; -- {serverError 80}
DROP TABLE test_01191.t;
CREATE DATABASE dummy_db ENGINE=Atomic;
RENAME DICTIONARY test_01191.dict TO dummy_db.dict1;