未验证 提交 89df9915 编写于 作者: A alexey-milovidov 提交者: GitHub

Merge pull request #11592 from ClickHouse/replicated-merge-tree-create-drop-race-garbage

Fix race conditions in CREATE/DROP of different replicas of ReplicatedMergeTree
......@@ -122,7 +122,12 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query)
return statement_stream.str();
}
DatabaseOnDisk::DatabaseOnDisk(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context)
DatabaseOnDisk::DatabaseOnDisk(
const String & name,
const String & metadata_path_,
const String & data_path_,
const String & logger,
const Context & context)
: DatabaseWithOwnTablesBase(name, logger, context)
, metadata_path(metadata_path_)
, data_path(data_path_)
......@@ -154,7 +159,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("Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.",
ErrorCodes::DICTIONARY_ALREADY_EXISTS);
......
......@@ -152,7 +152,8 @@ MergeTreeData::MergeTreeData(
if (metadata.sample_by_ast != nullptr)
{
StorageMetadataKeyField candidate_sampling_key = StorageMetadataKeyField::getKeyFromAST(metadata.sample_by_ast, getColumns(), global_context);
StorageMetadataKeyField candidate_sampling_key = StorageMetadataKeyField::getKeyFromAST(
metadata.sample_by_ast, getColumns(), global_context);
const auto & pk_sample_block = getPrimaryKey().sample_block;
if (!pk_sample_block.has(candidate_sampling_key.column_names[0]) && !attach
......@@ -1304,6 +1305,24 @@ void MergeTreeData::dropAllData()
LOG_TRACE(log, "dropAllData: done.");
}
void MergeTreeData::dropIfEmpty()
{
LOG_TRACE(log, "dropIfEmpty");
auto lock = lockParts();
if (!data_parts_by_info.empty())
return;
for (const auto & [path, disk] : getRelativeDataPathsWithDisks())
{
/// Non recursive, exception is thrown if there are more files.
disk->remove(path + "format_version.txt");
disk->remove(path + "detached");
disk->remove(path);
}
}
namespace
{
......
......@@ -485,6 +485,9 @@ public:
/// Deletes the data directory and flushes the uncompressed blocks cache and the marks cache.
void dropAllData();
/// Drop data directories if they are empty. It is safe to call this method if table creation was unsuccessful.
void dropIfEmpty();
/// Moves the entire data directory.
/// Flushes the uncompressed blocks cache and the marks cache.
/// Must be called with locked lockStructureForAlter().
......
......@@ -291,9 +291,10 @@ private:
template <class Func>
void foreachCommittedParts(const Func & func) const;
/** Creates the minimum set of nodes in ZooKeeper.
/** Creates the minimum set of nodes in ZooKeeper and create first replica.
* Returns true if was created, false if exists.
*/
void createTableIfNotExists();
bool createTableIfNotExists();
/** Creates a replica in ZooKeeper and adds to the queue all that it takes to catch up with the rest of the replicas.
*/
......
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
set -e
function thread()
{
while true; do
$CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS test_table_$1;
CREATE TABLE test_table_$1 (a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/alter_table', 'r_$1') ORDER BY tuple();" 2>&1 |
grep -vP '(^$)|(^Received exception from server)|(^\d+\. )|because the last replica of the table was dropped right now|is already started to be removing by another replica right now|is already finished removing by another replica right now|Removing leftovers from table|Another replica was suddenly created|was successfully removed from ZooKeeper|was created by another server at the same moment|was suddenly removed'
done
}
# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout
export -f thread;
TIMEOUT=10
timeout $TIMEOUT bash -c 'thread 1' &
timeout $TIMEOUT bash -c 'thread 2' &
wait
for i in {1,2}; do $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_table_$i"; done
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册