mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Fixed tests
This commit is contained in:
parent
fa39c30b4b
commit
561b045991
@ -24,6 +24,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
|
||||
@ -228,6 +229,11 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_
|
||||
other_db.checkMetadataFilenameAvailabilityUnlocked(to_table_name, inside_database ? db_lock : other_db_lock);
|
||||
|
||||
StoragePtr table = getTableUnlocked(table_name, db_lock);
|
||||
|
||||
if (dictionary != table->isDictionary())
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY,
|
||||
"Use RENAME DICTIONARY for dictionaries and RENAME TABLE for tables.");
|
||||
|
||||
table->checkTableCanBeRenamed();
|
||||
assert_can_move_mat_view(table);
|
||||
StoragePtr other_table;
|
||||
|
@ -227,10 +227,6 @@ void DatabaseOnDisk::createTable(
|
||||
/// A race condition would be possible if a table with the same name is simultaneously created using CREATE and using ATTACH.
|
||||
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
|
||||
|
||||
if (isDictionaryExist(table_name))
|
||||
throw Exception(
|
||||
ErrorCodes::DICTIONARY_ALREADY_EXISTS, "Dictionary {}.{} already exists", backQuote(getDatabaseName()), backQuote(table_name));
|
||||
|
||||
if (isTableExist(table_name, getContext()))
|
||||
throw Exception(
|
||||
ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists", backQuote(getDatabaseName()), backQuote(table_name));
|
||||
|
@ -154,24 +154,52 @@ void DatabaseOrdinary::loadStoredObjects(ContextPtr local_context, bool has_forc
|
||||
/// without having any tables attached. It is so because attaching of a dictionary means
|
||||
/// loading of its config only, it doesn't involve loading the dictionary itself.
|
||||
|
||||
/// Attach dictionaries.
|
||||
for (const auto & name_with_query : file_names)
|
||||
{
|
||||
const auto & create_query = name_with_query.second->as<const ASTCreateQuery &>();
|
||||
|
||||
if (create_query.is_dictionary)
|
||||
{
|
||||
pool.scheduleOrThrowOnError([&]()
|
||||
{
|
||||
tryAttachTable(
|
||||
local_context,
|
||||
create_query,
|
||||
*this,
|
||||
database_name,
|
||||
getMetadataPath() + name_with_query.first,
|
||||
has_force_restore_data_flag);
|
||||
|
||||
/// Messages, so that it's not boring to wait for the server to load for a long time.
|
||||
logAboutProgress(log, ++tables_processed, total_tables, watch);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
pool.wait();
|
||||
|
||||
/// Attach tables.
|
||||
for (const auto & name_with_query : file_names)
|
||||
{
|
||||
|
||||
const auto & create_query = name_with_query.second->as<const ASTCreateQuery &>();
|
||||
pool.scheduleOrThrowOnError([&]()
|
||||
{
|
||||
tryAttachTable(
|
||||
local_context,
|
||||
create_query,
|
||||
*this,
|
||||
database_name,
|
||||
getMetadataPath() + name_with_query.first,
|
||||
has_force_restore_data_flag);
|
||||
|
||||
/// Messages, so that it's not boring to wait for the server to load for a long time.
|
||||
logAboutProgress(log, ++tables_processed, total_tables, watch);
|
||||
});
|
||||
if (!create_query.is_dictionary)
|
||||
{
|
||||
pool.scheduleOrThrowOnError([&]()
|
||||
{
|
||||
tryAttachTable(
|
||||
local_context,
|
||||
create_query,
|
||||
*this,
|
||||
database_name,
|
||||
getMetadataPath() + name_with_query.first,
|
||||
has_force_restore_data_flag);
|
||||
|
||||
/// Messages, so that it's not boring to wait for the server to load for a long time.
|
||||
logAboutProgress(log, ++tables_processed, total_tables, watch);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
pool.wait();
|
||||
|
@ -126,12 +126,6 @@ public:
|
||||
/// Check the existence of the table.
|
||||
virtual bool isTableExist(const String & name, ContextPtr context) const = 0;
|
||||
|
||||
/// Check the existence of the dictionary
|
||||
virtual bool isDictionaryExist(const String & /*name*/) const
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Get the table for work. Return nullptr if there is no table.
|
||||
virtual StoragePtr tryGetTable(const String & name, ContextPtr context) const = 0;
|
||||
|
||||
|
@ -631,7 +631,13 @@ std::unique_lock<std::shared_mutex> DatabaseCatalog::getExclusiveDDLGuardForData
|
||||
bool DatabaseCatalog::isDictionaryExist(const StorageID & table_id) const
|
||||
{
|
||||
auto db = tryGetDatabase(table_id.getDatabaseName());
|
||||
return db && db->isDictionaryExist(table_id.getTableName());
|
||||
if (!db)
|
||||
return false;
|
||||
|
||||
auto storage = db->tryGetTable(table_id.getTableName(), getContext());
|
||||
bool storage_is_dictionary = storage && storage->isDictionary();
|
||||
|
||||
return storage_is_dictionary;
|
||||
}
|
||||
|
||||
StoragePtr DatabaseCatalog::getTable(const StorageID & table_id, ContextPtr local_context) const
|
||||
|
@ -841,9 +841,19 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
|
||||
// Table SQL definition is available even if the table is detached (even permanently)
|
||||
auto query = database->getCreateTableQuery(create.table, getContext());
|
||||
create = query->as<ASTCreateQuery &>(); // Copy the saved create query, but use ATTACH instead of CREATE
|
||||
auto create_query = query->as<ASTCreateQuery &>();
|
||||
|
||||
/// TODO: Check if dictionary
|
||||
if (!create.is_dictionary && create_query.is_dictionary)
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY,
|
||||
"Cannot ATTACH TABLE {}.{}, it is a Dictionary",
|
||||
backQuoteIfNeed(database_name), backQuoteIfNeed(create.table));
|
||||
|
||||
if (create.is_dictionary && !create_query.is_dictionary)
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY,
|
||||
"Cannot ATTACH DICTIONARY {}.{}, it is a Table",
|
||||
backQuoteIfNeed(database_name), backQuoteIfNeed(create.table));
|
||||
|
||||
create = create_query; // Copy the saved create query, but use ATTACH instead of CREATE
|
||||
|
||||
create.attach = true;
|
||||
create.attach_short_syntax = true;
|
||||
|
@ -70,7 +70,6 @@ private:
|
||||
|
||||
BlockIO createDatabase(ASTCreateQuery & create);
|
||||
BlockIO createTable(ASTCreateQuery & create);
|
||||
BlockIO createDictionary(ASTCreateQuery & create);
|
||||
|
||||
/// Calculate list of columns, constraints, indices, etc... of table. Rewrite query in canonical way.
|
||||
TableProperties setProperties(ASTCreateQuery & create) const;
|
||||
|
@ -32,6 +32,7 @@ namespace ErrorCodes
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int INCORRECT_QUERY;
|
||||
}
|
||||
|
||||
|
||||
@ -113,8 +114,17 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ASTDropQuery & query, DatabaseP
|
||||
|
||||
if (database && table)
|
||||
{
|
||||
if (query.as<ASTDropQuery &>().is_view && !table->isView())
|
||||
throw Exception("Table " + table_id.getNameForLogs() + " is not a View", ErrorCodes::LOGICAL_ERROR);
|
||||
auto & ast_drop_query = query.as<ASTDropQuery &>();
|
||||
|
||||
if (ast_drop_query.is_view && !table->isView())
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY,
|
||||
"Table {} is not a View",
|
||||
table_id.getNameForLogs());
|
||||
|
||||
if (ast_drop_query.is_dictionary && !table->isDictionary())
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY,
|
||||
"Table {} is not a Dictionary",
|
||||
table_id.getNameForLogs());
|
||||
|
||||
/// Now get UUID, so we can wait for table data to be finally dropped
|
||||
table_id.uuid = database->tryGetTableUUID(table_id.table_name);
|
||||
|
@ -18,7 +18,7 @@ SHOW CREATE DICTIONARY test_01190.dict;
|
||||
CREATE TABLE log ENGINE = Log AS SELECT 'test' AS s;
|
||||
SHOW CREATE log;
|
||||
DETACH TABLE log;
|
||||
ATTACH DICTIONARY log; -- { serverError 487 }
|
||||
ATTACH DICTIONARY log; -- { serverError 80 }
|
||||
ATTACH TABLE log (s String) ENGINE = Log();
|
||||
SHOW CREATE log;
|
||||
SELECT * FROM log;
|
||||
|
@ -13,11 +13,15 @@ INSERT INTO test_01191._ 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}
|
||||
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}
|
||||
RENAME DICTIONARY test_01191.dict TO default.dict1; -- {serverError 48}
|
||||
|
||||
CREATE DATABASE dummy_db ENGINE=Atomic;
|
||||
RENAME DICTIONARY test_01191.dict TO dummy_db.dict1;
|
||||
RENAME DICTIONARY dummy_db.dict1 TO test_01191.dict;
|
||||
DROP DATABASE dummy_db;
|
||||
|
||||
RENAME DICTIONARY test_01191.dict TO test_01191.dict1;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user