mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Change table name to table UUID, fix test
This commit is contained in:
parent
b88048e6d9
commit
57aa049fef
@ -1442,13 +1442,19 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
|
||||
"ATTACH ... FROM ... query is not supported for {} table engine, "
|
||||
"because such tables do not store any data on disk. Use CREATE instead.", res->getName());
|
||||
|
||||
bool is_replicated_storage = typeid_cast<const StorageReplicatedMergeTree *>(res.get()) != nullptr;
|
||||
if (is_replicated_storage)
|
||||
auto * replicated_storage = typeid_cast<StorageReplicatedMergeTree *>(res.get());
|
||||
if (replicated_storage)
|
||||
{
|
||||
const auto probability = getContext()->getSettingsRef().create_replicated_merge_tree_fault_injection_probability;
|
||||
std::bernoulli_distribution fault(probability);
|
||||
if (fault(thread_local_rng))
|
||||
{
|
||||
/// We emulate the case when the exception was thrown in StorageReplicatedMergeTree constructor
|
||||
if (!create.attach)
|
||||
replicated_storage->dropIfEmpty();
|
||||
|
||||
throw Coordination::Exception(Coordination::Error::ZCONNECTIONLOSS, "Fault injected (during table creation)");
|
||||
}
|
||||
}
|
||||
|
||||
database->createTable(getContext(), create.getTable(), res, query_ptr);
|
||||
|
@ -836,7 +836,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr
|
||||
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/mutation_pointer", "",
|
||||
zkutil::CreateMode::Persistent));
|
||||
|
||||
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/creator_info", toString(getStorageID().getFullTableName()) + "|" + toString(ServerUUID::get()),
|
||||
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/creator_info", toString(getStorageID().uuid) + "|" + toString(ServerUUID::get()),
|
||||
zkutil::CreateMode::Persistent));
|
||||
|
||||
Coordination::Responses responses;
|
||||
@ -869,7 +869,7 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada
|
||||
const String local_metadata = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString();
|
||||
const String local_columns = metadata_snapshot->getColumns().toString();
|
||||
const String local_metadata_version = toString(metadata_snapshot->getMetadataVersion());
|
||||
const String creator_info = toString(getStorageID().getFullTableName()) + "|" + toString(ServerUUID::get());
|
||||
const String creator_info = toString(getStorageID().uuid) + "|" + toString(ServerUUID::get());
|
||||
|
||||
/// It is possible for the replica to fail after creating ZK nodes without saving local metadata.
|
||||
/// Because of that we need to check whether the replica exists and is newly created.
|
||||
|
@ -1,4 +1,4 @@
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
|
@ -1,38 +1,42 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: zookeeper
|
||||
# Tags: zookeeper, no-parallel
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test_exception_replicated SYNC"
|
||||
|
||||
UUID=$(${CLICKHOUSE_CLIENT} --query "SELECT reinterpretAsUUID(currentDatabase())")
|
||||
|
||||
#### 1 - There is only one replica
|
||||
|
||||
${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection_probability=1 \
|
||||
-q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "Fault injected"
|
||||
-q "CREATE TABLE test_exception_replicated UUID '$UUID' (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -cm1 "Fault injected"
|
||||
|
||||
# We will see that the replica is empty and throw the same 'Fault injected' exception as before
|
||||
${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection_probability=1 \
|
||||
-q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "Fault injected"
|
||||
-q "CREATE TABLE test_exception_replicated UUID '$UUID' (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -cm1 "Fault injected"
|
||||
|
||||
# We will succeed
|
||||
${CLICKHOUSE_CLIENT} \
|
||||
-q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date"
|
||||
-q "CREATE TABLE test_exception_replicated UUID '$UUID' (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "DROP TABLE test_exception_replicated SYNC"
|
||||
|
||||
#### 2 - There are two replicas
|
||||
|
||||
${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection_probability=1 \
|
||||
-q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -c "Fault injected"
|
||||
-q "CREATE TABLE test_exception_replicated UUID '$UUID' (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date" 2>&1 | grep -cm1 "Fault injected"
|
||||
${CLICKHOUSE_CLIENT} --create_replicated_merge_tree_fault_injection_probability=1 \
|
||||
-q "CREATE TABLE test_exception_replicated_2 (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r2') ORDER BY date" 2>&1 | grep -c "Fault injected"
|
||||
-q "CREATE TABLE test_exception_replicated_2 (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r2') ORDER BY date" 2>&1 | grep -cm1 "Fault injected"
|
||||
|
||||
# We will succeed
|
||||
${CLICKHOUSE_CLIENT} \
|
||||
-q "CREATE TABLE test_exception_replicated (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date"
|
||||
-q "CREATE TABLE test_exception_replicated UUID '$UUID' (date Date) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/recreate', 'r1') ORDER BY date"
|
||||
|
||||
# The trash from the second replica creation will not prevent us from dropping the table fully, so we delete it separately
|
||||
${CLICKHOUSE_CLIENT} -q "SYSTEM DROP REPLICA 'r2' FROM TABLE test_exception_replicated"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "DROP TABLE test_exception_replicated SYNC"
|
Loading…
Reference in New Issue
Block a user