mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-17 20:02:05 +00:00
add test
This commit is contained in:
parent
033d7f9b32
commit
f0edb65e33
@ -29,7 +29,7 @@ static bool supportsRenameat2Impl()
|
||||
if (uname(&sysinfo))
|
||||
return false;
|
||||
char * point = nullptr;
|
||||
long v_major = strtol(sysinfo.release, &point, 10);
|
||||
auto v_major = strtol(sysinfo.release, &point, 10);
|
||||
|
||||
errno = 0;
|
||||
if (errno || *point != '.' || v_major < 3)
|
||||
@ -38,7 +38,7 @@ static bool supportsRenameat2Impl()
|
||||
return true;
|
||||
|
||||
errno = 0;
|
||||
long v_minor = strtol(point + 1, nullptr, 10);
|
||||
auto v_minor = strtol(point + 1, nullptr, 10);
|
||||
return !errno && 15 <= v_minor;
|
||||
#else
|
||||
return false;
|
||||
|
@ -17,6 +17,7 @@ namespace ErrorCodes
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
extern const int CANNOT_ASSIGN_ALTER;
|
||||
extern const int DATABASE_NOT_EMPTY;
|
||||
}
|
||||
|
||||
class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator
|
||||
@ -117,23 +118,23 @@ 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);
|
||||
|
||||
auto detach = [this](DatabaseAtomic & db, const String & table_name_)
|
||||
auto detach = [](DatabaseAtomic & db, const String & table_name_)
|
||||
{
|
||||
auto table_data_path_saved = db.table_name_to_path.find(table_name_)->second;
|
||||
db.tables.erase(table_name_);
|
||||
db.table_name_to_path.erase(table_name_);
|
||||
tryRemoveSymlink(table_name_);
|
||||
db.tryRemoveSymlink(table_name_);
|
||||
return table_data_path_saved;
|
||||
};
|
||||
|
||||
auto attach = [this](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_)
|
||||
auto attach = [](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_)
|
||||
{
|
||||
db.tables.emplace(table_name_, table_);
|
||||
db.table_name_to_path.emplace(table_name_, table_data_path_);
|
||||
tryCreateSymlink(table_name_, table_data_path_);
|
||||
db.tryCreateSymlink(table_name_, table_data_path_);
|
||||
};
|
||||
|
||||
auto assertCanMoveMatView = [&](const StoragePtr & table_)
|
||||
auto assertCanMoveMatView = [inside_database](const StoragePtr & table_)
|
||||
{
|
||||
if (inside_database)
|
||||
return;
|
||||
@ -256,6 +257,22 @@ DatabaseAtomic::DetachedTables DatabaseAtomic::cleenupDetachedTables()
|
||||
return not_in_use;
|
||||
}
|
||||
|
||||
void DatabaseAtomic::assertCanBeDetached(bool cleenup)
|
||||
{
|
||||
if (cleenup)
|
||||
{
|
||||
DetachedTables not_in_use;
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
not_in_use = cleenupDetachedTables();
|
||||
}
|
||||
}
|
||||
std::lock_guard lock(mutex);
|
||||
if (!detached_tables.empty())
|
||||
throw Exception("Database " + backQuoteIfNeed(database_name) + " cannot be detached, "
|
||||
"because some tables are still in use. Retry later.", ErrorCodes::DATABASE_NOT_EMPTY);
|
||||
}
|
||||
|
||||
DatabaseTablesIteratorPtr DatabaseAtomic::getTablesIterator(const IDatabase::FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
auto base_iter = DatabaseWithOwnTablesBase::getTablesIterator(filter_by_table_name);
|
||||
|
@ -38,6 +38,8 @@ public:
|
||||
|
||||
void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override;
|
||||
|
||||
void assertCanBeDetached(bool cleenup);
|
||||
|
||||
private:
|
||||
void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) override;
|
||||
void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
|
||||
|
@ -262,8 +262,15 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d
|
||||
assertDatabaseExistsUnlocked(database_name);
|
||||
db = databases.find(database_name)->second;
|
||||
|
||||
if (check_empty && !db->empty(*global_context))
|
||||
throw Exception("New table appeared in database being dropped or detached. Try again.", ErrorCodes::DATABASE_NOT_EMPTY);
|
||||
if (check_empty)
|
||||
{
|
||||
if (!db->empty(*global_context))
|
||||
throw Exception("New table appeared in database being dropped or detached. Try again.",
|
||||
ErrorCodes::DATABASE_NOT_EMPTY);
|
||||
auto database_atomic = typeid_cast<DatabaseAtomic *>(db.get());
|
||||
if (!drop && database_atomic)
|
||||
database_atomic->assertCanBeDetached(false);
|
||||
}
|
||||
|
||||
databases.erase(database_name);
|
||||
}
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Databases/DatabaseAtomic.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -234,6 +235,10 @@ BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, AS
|
||||
executeToDictionary(database_name, current_dictionary, kind, false, false, false);
|
||||
}
|
||||
|
||||
auto database_atomic = typeid_cast<DatabaseAtomic *>(database.get());
|
||||
if (!drop && database_atomic)
|
||||
database_atomic->assertCanBeDetached(true);
|
||||
|
||||
/// DETACH or DROP database itself
|
||||
DatabaseCatalog::instance().detachDatabase(database_name, drop);
|
||||
}
|
||||
|
@ -367,7 +367,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context)
|
||||
for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next())
|
||||
{
|
||||
if (dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
||||
replica_names.emplace_back(StorageID{database->getDatabaseName(), iterator->name(), iterator->uuid()});
|
||||
replica_names.emplace_back(StorageID{database->getDatabaseName(), iterator->name()});
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1,2 +1,4 @@
|
||||
OK
|
||||
OK
|
||||
5 10
|
||||
5 10
|
||||
|
@ -7,14 +7,19 @@ $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01107"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01107 ENGINE=Atomic"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01107.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" &
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" &
|
||||
sleep 1
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DETACH TABLE test_01107.mt"
|
||||
$CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" 2>&1 | grep -F "Code: 57" > /dev/null && echo "OK"
|
||||
$CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" 2>&1 | grep -F "Code: 219" > /dev/null && echo "OK"
|
||||
|
||||
sleep 5
|
||||
wait
|
||||
$CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt"
|
||||
$CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107"
|
||||
$CLICKHOUSE_CLIENT -q "ATTACH DATABASE test_01107 ENGINE=Atomic"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(100)" &
|
||||
$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01107"
|
||||
|
@ -3,11 +3,15 @@ CREATE DATABASE test_01109 ENGINE=Atomic;
|
||||
|
||||
USE test_01109;
|
||||
|
||||
CREATE TABLE t1 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks(), * FROM (SELECT toLowCardinality(arrayJoin(['exchange', 'tables'])));
|
||||
CREATE TABLE t2 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks() + (SELECT count() FROM t1), * FROM (SELECT arrayJoin(['hello', 'world']));
|
||||
CREATE TABLE t0 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks(), * FROM (SELECT toLowCardinality(arrayJoin(['exchange', 'tables'])));
|
||||
CREATE TABLE t1 ENGINE=Log() AS SELECT * FROM system.tables AS t JOIN system.databases AS d ON t.database=d.name;
|
||||
CREATE TABLE t2 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks() + (SELECT count() FROM t0), * FROM (SELECT arrayJoin(['hello', 'world']));
|
||||
|
||||
EXCHANGE TABLES t1 AND t3; -- { serverError 60 }
|
||||
EXCHANGE TABLES t4 AND t2; -- { serverError 60 }
|
||||
RENAME TABLE t0 TO t1; -- { serverError 57 }
|
||||
DROP TABLE t1;
|
||||
RENAME TABLE t0 TO t1;
|
||||
SELECT * FROM t1;
|
||||
SELECT * FROM t2;
|
||||
|
||||
|
16
tests/queries/0_stateless/01114_database_atomic.reference
Normal file
16
tests/queries/0_stateless/01114_database_atomic.reference
Normal file
@ -0,0 +1,16 @@
|
||||
CREATE DATABASE test_01114_1\nENGINE = Atomic
|
||||
CREATE DATABASE test_01114_2\nENGINE = Atomic
|
||||
CREATE DATABASE test_01114_3\nENGINE = Ordinary
|
||||
test_01114_1 Atomic store metadata test_01114_1
|
||||
test_01114_2 Atomic store metadata test_01114_2
|
||||
test_01114_3 Ordinary test_01114_3 metadata test_01114_3
|
||||
20
|
||||
100
|
||||
CREATE TABLE test_01114_2.mt\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192
|
||||
mt 00001114-0000-4000-8000-000000000001 CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000001\' (`n` UInt64) ENGINE = MergeTree() PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
20
|
||||
CREATE TABLE test_01114_1.mt\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192
|
||||
CREATE TABLE test_01114_2.mt\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192
|
||||
5
|
||||
20 190
|
||||
30 435
|
63
tests/queries/0_stateless/01114_database_atomic.sh
Executable file
63
tests/queries/0_stateless/01114_database_atomic.sh
Executable file
@ -0,0 +1,63 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_1"
|
||||
$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_2"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Atomic"
|
||||
$CLICKHOUSE_CLIENT --default_database_engine=Atomic --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01114_2"
|
||||
$CLICKHOUSE_CLIENT --default_database_engine=Ordinary -q "CREATE DATABASE test_01114_3"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SHOW CREATE DATABASE test_01114_1"
|
||||
$CLICKHOUSE_CLIENT -q "SHOW CREATE DATABASE test_01114_2"
|
||||
$CLICKHOUSE_CLIENT -q "SHOW CREATE DATABASE test_01114_3"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT name, engine, splitByChar('/', data_path)[-2], splitByChar('/', metadata_path)[-3], splitByChar('/', metadata_path)[-2] FROM system.databases WHERE name LIKE 'test_01114_%'"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_1.mt_tmp (n UInt64) ENGINE=MergeTree() ORDER BY tuple()"
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_1.mt_tmp SELECT * FROM numbers(100)"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_3.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)"
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_3.mt SELECT * FROM numbers(20)"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_1.mt_tmp TO test_01114_3.mt_tmp" # move from Atomic to Ordinary
|
||||
$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_3.mt TO test_01114_1.mt" # move from Ordinary to Atomic
|
||||
$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_3.mt_tmp"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_3"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_2.mt UUID '00001114-0000-4000-8000-000000000001' (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)"
|
||||
$CLICKHOUSE_CLIENT -q "SHOW CREATE TABLE test_01114_2.mt"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='test_01114_2'"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1) AS col FROM test_01114_1.mt)" & # 20s, result: 20, 190
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1) FROM numbers(30)" & # 30s
|
||||
sleep 1 # SELECT and INSERT should start before the following RENAMEs
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_1.mt TO test_01114_1.mt_tmp"
|
||||
$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_1.mt_tmp TO test_01114_2.mt_tmp"
|
||||
$CLICKHOUSE_CLIENT -q "EXCHANGE TABLES test_01114_2.mt AND test_01114_2.mt_tmp"
|
||||
$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_2.mt_tmp TO test_01114_1.mt"
|
||||
$CLICKHOUSE_CLIENT -q "EXCHANGE TABLES test_01114_1.mt AND test_01114_2.mt"
|
||||
|
||||
# Check that nothing changed
|
||||
$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt"
|
||||
$CLICKHOUSE_CLIENT -q "SHOW CREATE TABLE test_01114_1.mt"
|
||||
$CLICKHOUSE_CLIENT -q "SHOW CREATE TABLE test_01114_2.mt"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE test_01114_1.mt"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_1.mt (s String) ENGINE=Log()"
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_1.mt SELECT 's' || toString(number) FROM numbers(5)"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt" # result: 5
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s
|
||||
sleep 1
|
||||
$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_1"
|
||||
|
||||
wait # for INSERT
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01114_2.mt" # result: 30, 435
|
||||
$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_2"
|
Loading…
Reference in New Issue
Block a user