Tests and fixes

This commit is contained in:
Mikhail Filimonov 2020-12-14 15:21:31 +01:00
parent 87ea1528cb
commit 24abcc970b
No known key found for this signature in database
GPG Key ID: 6E49C2E9AF1220BE
7 changed files with 352 additions and 25 deletions

View File

@ -151,6 +151,24 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
String old_metadata_path = getObjectMetadataPath(table_name);
String new_metadata_path = to_database.getObjectMetadataPath(to_table_name);
if (!exchange)
{
if (auto target_db = dynamic_cast<DatabaseOnDisk *>(&to_database))
{
auto attach_query = parseQueryFromMetadata(log, context, old_metadata_path);
auto & create = attach_query->as<ASTCreateQuery &>();
create.database = to_database.getDatabaseName();
create.table = to_table_name;
// we run checks for rename same way as for create table (attach has more relaxed checks)
create.attach = false;
/// if after that check and before the actual rename the dest table
/// will be created & detached permanently the rename will overwrite it.
target_db->checkTableAttachPossible(context, create);
}
}
auto detach = [](DatabaseAtomic & db, const String & table_name_, bool has_symlink)
{
auto it = db.table_name_to_path.find(table_name_);

View File

@ -151,7 +151,7 @@ void DatabaseOnDisk::createTable(
const auto & create = query->as<ASTCreateQuery &>();
assert(table_name == create.table);
checkTableAttachPossible(context, table_name, create);
checkTableAttachPossible(context, create);
/// Create a file with metadata if necessary - if the query is not ATTACH.
/// Write the query of `ATTACH table` to it.
@ -231,13 +231,27 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora
void DatabaseOnDisk::detachTablePermanently(const String & table_name)
{
detachTable(table_name);
auto table = detachTable(table_name);
String table_metadata_path = getObjectMetadataPath(table_name);
String table_metadata_path_detached = table_metadata_path + detached_suffix;
try
{
if (table)
{
auto table_id = table->getStorageID();
/// usual detach don't remove UUID from the mapping
/// (it's done to prevent accidental reuse of table UUID)
/// but since reattach of permanently detached table
/// happens in a similar way as normal table creation
/// (with sql recreation, and adding uuid to a mapping)
/// we need to have uuid free and avaliable for further attaches.
if (table_id.hasUUID())
DatabaseCatalog::instance().removeUUIDMappingFinally(table_id.uuid);
}
/// it will silently overwrite the file if exists, and it's ok
Poco::File(table_metadata_path).renameTo(table_metadata_path_detached);
}
@ -289,29 +303,29 @@ void DatabaseOnDisk::dropTable(const Context & context, const String & table_nam
}
/// will throw when the table exists (in active / detached / detached permanently form) and we try to do to wrong create
void DatabaseOnDisk::checkTableAttachPossible(const Context & context, const String & table_name, const ASTCreateQuery & create)
void DatabaseOnDisk::checkTableAttachPossible(const Context & context, const ASTCreateQuery & create) const
{
if (isDictionaryExist(table_name))
throw Exception("Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.",
ErrorCodes::DICTIONARY_ALREADY_EXISTS);
String to_table_name = create.table;
if (isTableExist(table_name, global_context))
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
String table_metadata_path = getObjectMetadataPath(to_table_name);
String table_metadata_path = getObjectMetadataPath(table_name);
if (isDictionaryExist(to_table_name))
throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS, "Dictionary {}.{} already exists", backQuote(getDatabaseName()), backQuote(to_table_name));
if (isTableExist(to_table_name, global_context))
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists", backQuote(getDatabaseName()), backQuote(to_table_name));
if (!create.attach && Poco::File(table_metadata_path).exists())
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists (detached).", ErrorCodes::TABLE_ALREADY_EXISTS);
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists (detached)", backQuote(getDatabaseName()), backQuote(to_table_name));
/// if we have a table with a same name detached permanently we only allow
/// attaching it (and uuid should be the same), but not creating
String table_metadata_path_detached = getObjectMetadataPath(table_name) + detached_suffix;
String table_metadata_path_detached = table_metadata_path + detached_suffix;
if (Poco::File(table_metadata_path_detached).exists())
{
if (!create.attach)
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists (detached permanently).", ErrorCodes::TABLE_ALREADY_EXISTS);
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists (detached permanently)", backQuote(getDatabaseName()), backQuote(to_table_name));
if (!create.attach_short_syntax)
{
@ -320,7 +334,7 @@ void DatabaseOnDisk::checkTableAttachPossible(const Context & context, const Str
// either both should be Nil, either values should be equal
if (create.uuid != create_detached.uuid)
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exist (detached permanently). To attach it back you need to use short ATTACH syntax or a full statement with the same UUID.", ErrorCodes::TABLE_ALREADY_EXISTS);
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exist (detached permanently). To attach it back you need to use short ATTACH syntax or a full statement with the same UUID", backQuote(getDatabaseName()), backQuote(to_table_name));
}
}
}
@ -371,7 +385,13 @@ void DatabaseOnDisk::renameTable(
if (from_atomic_to_ordinary)
create.uuid = UUIDHelpers::Nil;
checkTableAttachPossible(context, table_name, create);
if (auto target_db = dynamic_cast<DatabaseOnDisk *>(&to_database))
{
// we run checks for rename same way as for create table (attach has more relaxed checks)
create.attach = false;
target_db->checkTableAttachPossible(context, create);
}
/// Notify the table that it is renamed. It will move data to new path (if it stores data on disk) and update StorageID
table->rename(to_database.getTableDataPath(create), StorageID(create));

View File

@ -69,6 +69,11 @@ public:
static ASTPtr parseQueryFromMetadata(Poco::Logger * log, const Context & context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false);
/// will throw when the table we want to attach already exists (in active / detached / detached permanently form)
/// still allow to overwrite the detached table if the table we want to attach is the old one
/// uses locks.
void checkTableAttachPossible(const Context & context, const ASTCreateQuery & create) const;
protected:
static constexpr const char * create_suffix = ".tmp";
static constexpr const char * drop_suffix = ".tmp_drop";
@ -95,8 +100,6 @@ protected:
const String metadata_path;
const String data_path;
private:
void checkTableAttachPossible(const Context & context, const String & table_name, const ASTCreateQuery & create);
};
}

View File

@ -72,7 +72,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n
auto it = tables.find(table_name);
if (it == tables.end())
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist.",
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist",
backQuote(database_name), backQuote(table_name));
res = it->second;
tables.erase(it);
@ -157,7 +157,7 @@ StoragePtr DatabaseWithOwnTablesBase::getTableUnlocked(const String & table_name
auto it = tables.find(table_name);
if (it != tables.end())
return it->second;
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist.",
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist",
backQuote(database_name), backQuote(table_name));
}

View File

@ -211,7 +211,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
if (!table_id)
{
if (exception)
exception->emplace("Cannot find table: StorageID is empty", ErrorCodes::UNKNOWN_TABLE);
exception->emplace(ErrorCodes::UNKNOWN_TABLE, "Cannot find table: StorageID is empty");
return {};
}
@ -223,7 +223,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
{
assert(!db_and_table.first && !db_and_table.second);
if (exception)
exception->emplace("Table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
exception->emplace(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs());
return {};
}
@ -244,7 +244,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
/// If table_id has no UUID, then the name of database was specified by user and table_id was not resolved through context.
/// Do not allow access to TEMPORARY_DATABASE because it contains all temporary tables of all contexts and users.
if (exception)
exception->emplace("Direct access to `" + String(TEMPORARY_DATABASE) + "` database is not allowed.", ErrorCodes::DATABASE_ACCESS_DENIED);
exception->emplace(ErrorCodes::DATABASE_ACCESS_DENIED, "Direct access to `{}` database is not allowed", String(TEMPORARY_DATABASE));
return {};
}
@ -255,8 +255,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
if (databases.end() == it)
{
if (exception)
exception->emplace("Database " + backQuoteIfNeed(table_id.getDatabaseName()) + " doesn't exist",
ErrorCodes::UNKNOWN_DATABASE);
exception->emplace(ErrorCodes::UNKNOWN_DATABASE, "Database {} doesn't exist", backQuoteIfNeed(table_id.getDatabaseName()));
return {};
}
database = it->second;
@ -264,7 +263,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
auto table = database->tryGetTable(table_id.table_name, context);
if (!table && exception)
exception->emplace("Table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
exception->emplace(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs());
if (!table)
database = nullptr;

View File

@ -0,0 +1,77 @@
database atomic tests
can not create table with same name as detached permanently
can not detach twice
can not drop detached
can not replace with the other table
can still show the create statement
Row 1:
──────
statement: CREATE TABLE test1601_detach_permanently_atomic.test_name_reuse
(
`number` UInt64
)
ENGINE = MergeTree
ORDER BY tuple()
SETTINGS index_granularity = 8192
can not attach with bad uuid
can attach with short syntax
can not detach permanently the table which is already detached (temporary)
After database reattachement the table is back (it was detached temporary)
And we can detach it permanently
After database reattachement the table is still absent (it was detached permamently)
And we can not detach it permanently
But we can attach it back
And detach permanently again to check how database drop will behave
DROP database
-----------------------
database ordinary tests
can not create table with same name as detached permanently
can not detach twice
can not drop detached
can not replace with the other table
can still show the create statement
Row 1:
──────
statement: CREATE TABLE test1601_detach_permanently_ordinary.test_name_reuse
(
`number` UInt64
)
ENGINE = MergeTree
ORDER BY tuple()
SETTINGS index_granularity = 8192
can attach with full syntax
can attach with short syntax
can not detach permanently the table which is already detached (temporary)
After database reattachement the table is back (it was detached temporary)
And we can detach it permanently
After database reattachement the table is still absent (it was detached permamently)
And we can not detach it permanently
But we can attach it back
And detach permanently again to check how database drop will behave
DROP database - Directory not empty error, but database deteched
DROP database - now success
-----------------------
database lazy tests
can not create table with same name as detached permanently
can not detach twice
can not drop detached
can not replace with the other table
can still show the create statement
Row 1:
──────
statement: CREATE TABLE test1601_detach_permanently_lazy.test_name_reuse
(
`number` UInt64
)
ENGINE = Log
can attach with full syntax
can attach with short syntax
can not detach permanently the table which is already detached (temporary)
After database reattachement the table is back (it was detached temporary)
And we can detach it permanently
After database reattachement the table is still absent (it was detached permamently)
And we can not detach it permanently
But we can attach it back
And detach permanently again to check how database drop will behave
DROP database - Directory not empty error, but database deteched
DROP database - now success

View File

@ -0,0 +1,210 @@
SELECT 'database atomic tests';
DROP DATABASE IF EXISTS test1601_detach_permanently_atomic;
CREATE DATABASE test1601_detach_permanently_atomic Engine=Atomic;
create table test1601_detach_permanently_atomic.test_name_reuse (number UInt64) engine=MergeTree order by tuple();
INSERT INTO test1601_detach_permanently_atomic.test_name_reuse SELECT * FROM numbers(100);
DETACH table test1601_detach_permanently_atomic.test_name_reuse PERMANENTLY;
SELECT 'can not create table with same name as detached permanently';
create table test1601_detach_permanently_atomic.test_name_reuse (number UInt64) engine=MergeTree order by tuple(); -- { serverError 57 }
SELECT 'can not detach twice';
DETACH table test1601_detach_permanently_atomic.test_name_reuse PERMANENTLY; -- { serverError 60 }
DETACH table test1601_detach_permanently_atomic.test_name_reuse; -- { serverError 60 }
SELECT 'can not drop detached';
drop table test1601_detach_permanently_atomic.test_name_reuse; -- { serverError 60 }
create table test1601_detach_permanently_atomic.test_name_rename_attempt (number UInt64) engine=MergeTree order by tuple();
SELECT 'can not replace with the other table';
RENAME TABLE test1601_detach_permanently_atomic.test_name_rename_attempt TO test1601_detach_permanently_atomic.test_name_reuse; -- { serverError 57 }
EXCHANGE TABLES test1601_detach_permanently_atomic.test_name_rename_attempt AND test1601_detach_permanently_atomic.test_name_reuse; -- { serverError 60 }
SELECT 'can still show the create statement';
SHOW CREATE TABLE test1601_detach_permanently_atomic.test_name_reuse FORMAT Vertical;
SELECT 'can not attach with bad uuid';
ATTACH TABLE test1601_detach_permanently_atomic.test_name_reuse UUID '00000000-0000-0000-0000-000000000001' (`number` UInt64 ) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192 ; -- { serverError 57 }
SELECT 'can attach with short syntax';
ATTACH TABLE test1601_detach_permanently_atomic.test_name_reuse;
DETACH table test1601_detach_permanently_atomic.test_name_reuse;
SELECT 'can not detach permanently the table which is already detached (temporary)';
DETACH table test1601_detach_permanently_atomic.test_name_reuse PERMANENTLY; -- { serverError 60 }
DETACH DATABASE test1601_detach_permanently_atomic;
ATTACH DATABASE test1601_detach_permanently_atomic;
SELECT 'After database reattachement the table is back (it was detached temporary)';
SELECT 'And we can detach it permanently';
DETACH table test1601_detach_permanently_atomic.test_name_reuse PERMANENTLY;
DETACH DATABASE test1601_detach_permanently_atomic;
ATTACH DATABASE test1601_detach_permanently_atomic;
SELECT 'After database reattachement the table is still absent (it was detached permamently)';
SELECT 'And we can not detach it permanently';
DETACH table test1601_detach_permanently_atomic.test_name_reuse PERMANENTLY; -- { serverError 60 }
SELECT 'But we can attach it back';
ATTACH TABLE test1601_detach_permanently_atomic.test_name_reuse;
SELECT 'And detach permanently again to check how database drop will behave';
DETACH table test1601_detach_permanently_atomic.test_name_reuse PERMANENTLY;
SELECT 'DROP database';
DROP DATABASE test1601_detach_permanently_atomic SYNC;
SELECT '-----------------------';
SELECT 'database ordinary tests';
DROP DATABASE IF EXISTS test1601_detach_permanently_ordinary;
CREATE DATABASE test1601_detach_permanently_ordinary Engine=Ordinary;
create table test1601_detach_permanently_ordinary.test_name_reuse (number UInt64) engine=MergeTree order by tuple();
INSERT INTO test1601_detach_permanently_ordinary.test_name_reuse SELECT * FROM numbers(100);
DETACH table test1601_detach_permanently_ordinary.test_name_reuse PERMANENTLY;
SELECT 'can not create table with same name as detached permanently';
create table test1601_detach_permanently_ordinary.test_name_reuse (number UInt64) engine=MergeTree order by tuple(); -- { serverError 57 }
SELECT 'can not detach twice';
DETACH table test1601_detach_permanently_ordinary.test_name_reuse PERMANENTLY; -- { serverError 60 }
DETACH table test1601_detach_permanently_ordinary.test_name_reuse; -- { serverError 60 }
SELECT 'can not drop detached';
drop table test1601_detach_permanently_ordinary.test_name_reuse; -- { serverError 60 }
create table test1601_detach_permanently_ordinary.test_name_rename_attempt (number UInt64) engine=MergeTree order by tuple();
SELECT 'can not replace with the other table';
RENAME TABLE test1601_detach_permanently_ordinary.test_name_rename_attempt TO test1601_detach_permanently_ordinary.test_name_reuse; -- { serverError 57 }
SELECT 'can still show the create statement';
SHOW CREATE TABLE test1601_detach_permanently_ordinary.test_name_reuse FORMAT Vertical;
SELECT 'can attach with full syntax';
ATTACH TABLE test1601_detach_permanently_ordinary.test_name_reuse (`number` UInt64 ) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192;
DETACH table test1601_detach_permanently_ordinary.test_name_reuse PERMANENTLY;
SELECT 'can attach with short syntax';
ATTACH TABLE test1601_detach_permanently_ordinary.test_name_reuse;
DETACH table test1601_detach_permanently_ordinary.test_name_reuse;
SELECT 'can not detach permanently the table which is already detached (temporary)';
DETACH table test1601_detach_permanently_ordinary.test_name_reuse PERMANENTLY; -- { serverError 60 }
DETACH DATABASE test1601_detach_permanently_ordinary;
ATTACH DATABASE test1601_detach_permanently_ordinary;
SELECT 'After database reattachement the table is back (it was detached temporary)';
SELECT 'And we can detach it permanently';
DETACH table test1601_detach_permanently_ordinary.test_name_reuse PERMANENTLY;
DETACH DATABASE test1601_detach_permanently_ordinary;
ATTACH DATABASE test1601_detach_permanently_ordinary;
SELECT 'After database reattachement the table is still absent (it was detached permamently)';
SELECT 'And we can not detach it permanently';
DETACH table test1601_detach_permanently_ordinary.test_name_reuse PERMANENTLY; -- { serverError 60 }
SELECT 'But we can attach it back';
ATTACH TABLE test1601_detach_permanently_ordinary.test_name_reuse;
SELECT 'And detach permanently again to check how database drop will behave';
DETACH table test1601_detach_permanently_ordinary.test_name_reuse PERMANENTLY;
SELECT 'DROP database - Directory not empty error, but database deteched';
DROP DATABASE test1601_detach_permanently_ordinary; -- { serverError 1000 }
ATTACH DATABASE test1601_detach_permanently_ordinary;
ATTACH TABLE test1601_detach_permanently_ordinary.test_name_reuse;
DROP TABLE test1601_detach_permanently_ordinary.test_name_reuse;
SELECT 'DROP database - now success';
DROP DATABASE test1601_detach_permanently_ordinary;
SELECT '-----------------------';
SELECT 'database lazy tests';
DROP DATABASE IF EXISTS test1601_detach_permanently_lazy;
CREATE DATABASE test1601_detach_permanently_lazy Engine=Lazy(10);
create table test1601_detach_permanently_lazy.test_name_reuse (number UInt64) engine=Log;
INSERT INTO test1601_detach_permanently_lazy.test_name_reuse SELECT * FROM numbers(100);
DETACH table test1601_detach_permanently_lazy.test_name_reuse PERMANENTLY;
SELECT 'can not create table with same name as detached permanently';
create table test1601_detach_permanently_lazy.test_name_reuse (number UInt64) engine=Log; -- { serverError 57 }
SELECT 'can not detach twice';
DETACH table test1601_detach_permanently_lazy.test_name_reuse PERMANENTLY; -- { serverError 60 }
DETACH table test1601_detach_permanently_lazy.test_name_reuse; -- { serverError 60 }
SELECT 'can not drop detached';
drop table test1601_detach_permanently_lazy.test_name_reuse; -- { serverError 60 }
create table test1601_detach_permanently_lazy.test_name_rename_attempt (number UInt64) engine=Log;
SELECT 'can not replace with the other table';
RENAME TABLE test1601_detach_permanently_lazy.test_name_rename_attempt TO test1601_detach_permanently_lazy.test_name_reuse; -- { serverError 57 }
SELECT 'can still show the create statement';
SHOW CREATE TABLE test1601_detach_permanently_lazy.test_name_reuse FORMAT Vertical;
SELECT 'can attach with full syntax';
ATTACH TABLE test1601_detach_permanently_lazy.test_name_reuse (`number` UInt64 ) ENGINE = Log;
DETACH table test1601_detach_permanently_lazy.test_name_reuse PERMANENTLY;
SELECT 'can attach with short syntax';
ATTACH TABLE test1601_detach_permanently_lazy.test_name_reuse;
DETACH table test1601_detach_permanently_lazy.test_name_reuse;
SELECT 'can not detach permanently the table which is already detached (temporary)';
DETACH table test1601_detach_permanently_lazy.test_name_reuse PERMANENTLY; -- { serverError 60 }
DETACH DATABASE test1601_detach_permanently_lazy;
ATTACH DATABASE test1601_detach_permanently_lazy;
SELECT 'After database reattachement the table is back (it was detached temporary)';
SELECT 'And we can detach it permanently';
DETACH table test1601_detach_permanently_lazy.test_name_reuse PERMANENTLY;
DETACH DATABASE test1601_detach_permanently_lazy;
ATTACH DATABASE test1601_detach_permanently_lazy;
SELECT 'After database reattachement the table is still absent (it was detached permamently)';
SELECT 'And we can not detach it permanently';
DETACH table test1601_detach_permanently_lazy.test_name_reuse PERMANENTLY; -- { serverError 60 }
SELECT 'But we can attach it back';
ATTACH TABLE test1601_detach_permanently_lazy.test_name_reuse;
SELECT 'And detach permanently again to check how database drop will behave';
DETACH table test1601_detach_permanently_lazy.test_name_reuse PERMANENTLY;
SELECT 'DROP database - Directory not empty error, but database deteched';
DROP DATABASE test1601_detach_permanently_lazy; -- { serverError 1000 }
ATTACH DATABASE test1601_detach_permanently_lazy;
ATTACH TABLE test1601_detach_permanently_lazy.test_name_reuse;
DROP TABLE test1601_detach_permanently_lazy.test_name_reuse;
SELECT 'DROP database - now success';
DROP DATABASE test1601_detach_permanently_lazy;