mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #31638 from ClickHouse/fix_rename_dictionary
Fix rename dictionary
This commit is contained in:
commit
bebee01b34
@ -229,15 +229,8 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_
|
|||||||
|
|
||||||
StoragePtr table = getTableUnlocked(table_name, db_lock);
|
StoragePtr table = getTableUnlocked(table_name, db_lock);
|
||||||
|
|
||||||
if (table->isDictionary() && !dictionary)
|
if (dictionary && !table->isDictionary())
|
||||||
{
|
throw Exception(ErrorCodes::INCORRECT_QUERY, "Use RENAME/EXCHANGE TABLE (instead of RENAME/EXCHANGE DICTIONARY) for tables");
|
||||||
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.");
|
|
||||||
}
|
|
||||||
|
|
||||||
table->checkTableCanBeRenamed();
|
table->checkTableCanBeRenamed();
|
||||||
assert_can_move_mat_view(table);
|
assert_can_move_mat_view(table);
|
||||||
@ -245,6 +238,8 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_
|
|||||||
if (exchange)
|
if (exchange)
|
||||||
{
|
{
|
||||||
other_table = other_db.getTableUnlocked(to_table_name, other_db_lock);
|
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();
|
other_table->checkTableCanBeRenamed();
|
||||||
assert_can_move_mat_view(other_table);
|
assert_can_move_mat_view(other_table);
|
||||||
}
|
}
|
||||||
|
@ -40,6 +40,7 @@ namespace ErrorCodes
|
|||||||
extern const int TABLE_ALREADY_EXISTS;
|
extern const int TABLE_ALREADY_EXISTS;
|
||||||
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
||||||
extern const int DATABASE_NOT_EMPTY;
|
extern const int DATABASE_NOT_EMPTY;
|
||||||
|
extern const int INCORRECT_QUERY;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -349,8 +350,6 @@ void DatabaseOnDisk::renameTable(
|
|||||||
{
|
{
|
||||||
if (exchange)
|
if (exchange)
|
||||||
throw Exception("Tables can be exchanged only in Atomic databases", ErrorCodes::NOT_IMPLEMENTED);
|
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_ordinary_to_atomic = false;
|
||||||
bool from_atomic_to_ordinary = false;
|
bool from_atomic_to_ordinary = false;
|
||||||
@ -372,7 +371,11 @@ void DatabaseOnDisk::renameTable(
|
|||||||
ASTPtr attach_query;
|
ASTPtr attach_query;
|
||||||
/// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case.
|
/// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case.
|
||||||
StoragePtr table = tryGetTable(table_name, local_context);
|
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);
|
detachTable(local_context, table_name);
|
||||||
|
|
||||||
UUID prev_uuid = UUIDHelpers::Nil;
|
UUID prev_uuid = UUIDHelpers::Nil;
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
|
@ -212,19 +212,39 @@ void StorageDictionary::renameInMemory(const StorageID & new_table_id)
|
|||||||
auto old_table_id = getStorageID();
|
auto old_table_id = getStorageID();
|
||||||
IStorage::renameInMemory(new_table_id);
|
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);
|
std::lock_guard<std::mutex> lock(dictionary_config_mutex);
|
||||||
|
|
||||||
if (configuration)
|
configuration->setString("dictionary.database", new_table_id.database_name);
|
||||||
{
|
configuration->setString("dictionary.name", new_table_id.table_name);
|
||||||
has_configuration = true;
|
if (move_to_atomic)
|
||||||
configuration->setString("dictionary.database", new_table_id.database_name);
|
configuration->setString("dictionary.uuid", toString(new_table_id.uuid));
|
||||||
configuration->setString("dictionary.name", new_table_id.table_name);
|
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();
|
const auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader();
|
||||||
auto result = external_dictionaries_loader.getLoadResult(old_table_id.getInternalDictionaryName());
|
auto result = external_dictionaries_loader.getLoadResult(old_table_id.getInternalDictionaryName());
|
||||||
|
@ -4,8 +4,13 @@
|
|||||||
1 mv1 before moving tablesmv1
|
1 mv1 before moving tablesmv1
|
||||||
1 mv2 before moving tablesmv2
|
1 mv2 before moving tablesmv2
|
||||||
1 src before moving tables
|
1 src before moving tables
|
||||||
|
asdf
|
||||||
|
asdf
|
||||||
|
test_01155_ordinary dict1 00000000-0000-0000-0000-000000000000
|
||||||
|
asdf
|
||||||
ordinary:
|
ordinary:
|
||||||
.inner.mv1
|
.inner.mv1
|
||||||
|
dict
|
||||||
dist
|
dist
|
||||||
dst
|
dst
|
||||||
mv1
|
mv1
|
||||||
@ -14,6 +19,7 @@ src
|
|||||||
ordinary after rename:
|
ordinary after rename:
|
||||||
atomic after rename:
|
atomic after rename:
|
||||||
.inner_id.
|
.inner_id.
|
||||||
|
dict
|
||||||
dist
|
dist
|
||||||
dst
|
dst
|
||||||
mv1
|
mv1
|
||||||
@ -33,12 +39,14 @@ src
|
|||||||
3 src after moving tables
|
3 src after moving tables
|
||||||
3 src after renaming database
|
3 src after renaming database
|
||||||
3 src before moving tables
|
3 src before moving tables
|
||||||
.inner_id.
|
asdf
|
||||||
dist
|
test_01155_ordinary .inner_id.
|
||||||
dst
|
test_01155_ordinary dict
|
||||||
mv1
|
test_01155_ordinary dist
|
||||||
mv2
|
test_01155_ordinary dst
|
||||||
src
|
test_01155_ordinary mv1
|
||||||
|
test_01155_ordinary mv2
|
||||||
|
test_01155_ordinary src
|
||||||
CREATE DATABASE test_01155_atomic\nENGINE = Atomic
|
CREATE DATABASE test_01155_atomic\nENGINE = Atomic
|
||||||
4 .inner.mv1 after renaming databasemv1
|
4 .inner.mv1 after renaming databasemv1
|
||||||
4 .inner.mv1 after renaming tablesmv1
|
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 database
|
||||||
4 src after renaming tables
|
4 src after renaming tables
|
||||||
4 src before moving tables
|
4 src before moving tables
|
||||||
|
asdf
|
||||||
|
test_01155_ordinary dict 00000000-0000-0000-0000-000000000000
|
||||||
test_01155_ordinary:
|
test_01155_ordinary:
|
||||||
.inner.mv1
|
.inner.mv1
|
||||||
|
dict
|
||||||
dist
|
dist
|
||||||
dst
|
dst
|
||||||
mv1
|
mv1
|
||||||
|
@ -7,15 +7,27 @@ CREATE DATABASE test_01155_ordinary ENGINE=Ordinary;
|
|||||||
CREATE DATABASE test_01155_atomic ENGINE=Atomic;
|
CREATE DATABASE test_01155_atomic ENGINE=Atomic;
|
||||||
|
|
||||||
USE test_01155_ordinary;
|
USE test_01155_ordinary;
|
||||||
CREATE TABLE src (s String) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY s;
|
CREATE TABLE src (s String, x String DEFAULT 'a') 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 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) ENGINE=MergeTree() PARTITION BY tuple() ORDER BY s;
|
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) AS SELECT (*,).1 || 'mv2' as s FROM src;
|
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) Engine=Distributed(test_shard_localhost, test_01155_ordinary, src);
|
CREATE TABLE dist (s String, x String DEFAULT 'asdf') ENGINE=Distributed(test_shard_localhost, test_01155_ordinary, src);
|
||||||
INSERT INTO dist VALUES ('before moving tables');
|
INSERT INTO dist(s) VALUES ('before moving tables');
|
||||||
SYSTEM FLUSH DISTRIBUTED dist;
|
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
|
-- 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 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
|
-- Move tables with materialized views from Ordinary to Atomic
|
||||||
SELECT 'ordinary:';
|
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.mv2 TO test_01155_atomic.mv2;
|
||||||
RENAME TABLE test_01155_ordinary.dst TO test_01155_atomic.dst;
|
RENAME TABLE test_01155_ordinary.dst TO test_01155_atomic.dst;
|
||||||
RENAME TABLE test_01155_ordinary.src TO test_01155_atomic.src;
|
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;
|
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 'ordinary after rename:';
|
||||||
SELECT substr(name, 1, 10) FROM system.tables WHERE database='test_01155_ordinary';
|
SELECT substr(name, 1, 10) FROM system.tables WHERE database='test_01155_ordinary';
|
||||||
SELECT 'atomic after rename:';
|
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;
|
DROP DATABASE test_01155_ordinary;
|
||||||
USE default;
|
USE default;
|
||||||
|
|
||||||
INSERT INTO test_01155_atomic.src VALUES ('after moving tables');
|
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 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;
|
RENAME DATABASE test_01155_atomic TO test_01155_ordinary;
|
||||||
USE 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;
|
SYSTEM FLUSH DISTRIBUTED dist;
|
||||||
SELECT materialize(3), substr(_table, 1, 10), s FROM merge('test_01155_ordinary', '') ORDER BY _table, s;
|
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
|
-- Move tables back
|
||||||
RENAME DATABASE test_01155_ordinary TO test_01155_atomic;
|
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.dst TO test_01155_ordinary.dst;
|
||||||
RENAME TABLE test_01155_atomic.src TO test_01155_ordinary.src;
|
RENAME TABLE test_01155_atomic.src TO test_01155_ordinary.src;
|
||||||
RENAME TABLE test_01155_atomic.dist TO test_01155_ordinary.dist;
|
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;
|
SYSTEM FLUSH DISTRIBUTED dist;
|
||||||
SELECT materialize(4), substr(_table, 1, 10), s FROM merge('test_01155_ordinary', '') ORDER BY _table, s;
|
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:';
|
SELECT 'test_01155_ordinary:';
|
||||||
SHOW TABLES FROM test_01155_ordinary;
|
SHOW TABLES FROM test_01155_ordinary;
|
||||||
SELECT 'test_01155_atomic:';
|
SELECT 'test_01155_atomic:';
|
||||||
|
@ -1,7 +1,13 @@
|
|||||||
dict NOT_LOADED
|
dict NOT_LOADED
|
||||||
_ Memory
|
_ Memory
|
||||||
dict Dictionary
|
dict Dictionary
|
||||||
dict1 NOT_LOADED
|
t Memory
|
||||||
|
t NOT_LOADED
|
||||||
|
_ Memory
|
||||||
|
dict Memory
|
||||||
|
t Dictionary
|
||||||
|
test
|
||||||
|
dict1 LOADED
|
||||||
_ Memory
|
_ Memory
|
||||||
dict1 Dictionary
|
dict1 Dictionary
|
||||||
test
|
test
|
||||||
|
@ -4,6 +4,7 @@ DROP DATABASE IF EXISTS test_01191;
|
|||||||
CREATE DATABASE test_01191 ENGINE=Atomic;
|
CREATE DATABASE test_01191 ENGINE=Atomic;
|
||||||
|
|
||||||
CREATE TABLE test_01191._ (n UInt64, s String) ENGINE = Memory();
|
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)
|
CREATE DICTIONARY test_01191.dict (n UInt64, s String)
|
||||||
PRIMARY KEY n
|
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;
|
SELECT name, engine FROM system.tables WHERE database='test_01191' ORDER BY name;
|
||||||
|
|
||||||
RENAME DICTIONARY test_01191.table TO test_01191.table1; -- {serverError 60}
|
RENAME DICTIONARY test_01191.table TO test_01191.table1; -- {serverError 60}
|
||||||
EXCHANGE TABLES test_01191.table AND test_01191.dict; -- {serverError 60}
|
EXCHANGE DICTIONARIES test_01191._ AND test_01191.dict; -- {serverError 80}
|
||||||
EXCHANGE TABLES test_01191.dict AND test_01191.table; -- {serverError 80}
|
EXCHANGE TABLES test_01191.t AND test_01191.dict;
|
||||||
RENAME TABLE test_01191.dict TO test_01191.dict1; -- {serverError 80}
|
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;
|
CREATE DATABASE dummy_db ENGINE=Atomic;
|
||||||
RENAME DICTIONARY test_01191.dict TO dummy_db.dict1;
|
RENAME DICTIONARY test_01191.dict TO dummy_db.dict1;
|
||||||
|
Loading…
Reference in New Issue
Block a user