Fixed tests

This commit is contained in:
Maksim Kita 2021-04-22 17:09:30 +03:00 committed by Maksim Kita
parent fa39c30b4b
commit 561b045991
10 changed files with 86 additions and 33 deletions

View File

@ -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;

View File

@ -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));

View File

@ -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();

View File

@ -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;

View File

@ -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

View File

@ -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;

View File

@ -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;

View File

@ -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);

View File

@ -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;

View File

@ -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;