mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge 318e9b4f33
into e0f8b8d351
This commit is contained in:
commit
1139f64205
@ -41,7 +41,7 @@ static struct InitFiu
|
|||||||
REGULAR(use_delayed_remote_source) \
|
REGULAR(use_delayed_remote_source) \
|
||||||
REGULAR(cluster_discovery_faults) \
|
REGULAR(cluster_discovery_faults) \
|
||||||
REGULAR(replicated_sends_failpoint) \
|
REGULAR(replicated_sends_failpoint) \
|
||||||
REGULAR(stripe_log_sink_write_fallpoint)\
|
REGULAR(stripe_log_sink_write_fallpoint) \
|
||||||
ONCE(smt_commit_merge_mutate_zk_fail_after_op) \
|
ONCE(smt_commit_merge_mutate_zk_fail_after_op) \
|
||||||
ONCE(smt_commit_merge_mutate_zk_fail_before_op) \
|
ONCE(smt_commit_merge_mutate_zk_fail_before_op) \
|
||||||
ONCE(smt_commit_write_zk_fail_after_op) \
|
ONCE(smt_commit_write_zk_fail_after_op) \
|
||||||
@ -77,6 +77,8 @@ static struct InitFiu
|
|||||||
REGULAR(replicated_merge_tree_all_replicas_stale) \
|
REGULAR(replicated_merge_tree_all_replicas_stale) \
|
||||||
REGULAR(zero_copy_lock_zk_fail_before_op) \
|
REGULAR(zero_copy_lock_zk_fail_before_op) \
|
||||||
REGULAR(zero_copy_lock_zk_fail_after_op) \
|
REGULAR(zero_copy_lock_zk_fail_after_op) \
|
||||||
|
REGULAR(plain_object_storage_write_fail_on_directory_create) \
|
||||||
|
REGULAR(plain_object_storage_write_fail_on_directory_move) \
|
||||||
|
|
||||||
|
|
||||||
namespace FailPoints
|
namespace FailPoints
|
||||||
|
@ -57,6 +57,12 @@ struct InMemoryDirectoryPathMap
|
|||||||
return it->second;
|
return it->second;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool removePathIfExists(const std::filesystem::path & path)
|
||||||
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
return map.erase(path) != 0;
|
||||||
|
}
|
||||||
|
|
||||||
mutable SharedMutex mutex;
|
mutable SharedMutex mutex;
|
||||||
|
|
||||||
#ifdef OS_LINUX
|
#ifdef OS_LINUX
|
||||||
|
@ -7,6 +7,7 @@
|
|||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
#include <Poco/Timestamp.h>
|
#include <Poco/Timestamp.h>
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
|
#include <Common/FailPoint.h>
|
||||||
#include <Common/SharedLockGuard.h>
|
#include <Common/SharedLockGuard.h>
|
||||||
#include <Common/logger_useful.h>
|
#include <Common/logger_useful.h>
|
||||||
|
|
||||||
@ -18,8 +19,15 @@ namespace ErrorCodes
|
|||||||
extern const int FILE_DOESNT_EXIST;
|
extern const int FILE_DOESNT_EXIST;
|
||||||
extern const int FILE_ALREADY_EXISTS;
|
extern const int FILE_ALREADY_EXISTS;
|
||||||
extern const int INCORRECT_DATA;
|
extern const int INCORRECT_DATA;
|
||||||
|
extern const int FAULT_INJECTED;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
namespace FailPoints
|
||||||
|
{
|
||||||
|
extern const char plain_object_storage_write_fail_on_directory_create[];
|
||||||
|
extern const char plain_object_storage_write_fail_on_directory_move[];
|
||||||
|
}
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -72,8 +80,6 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std:
|
|||||||
/* buf_size */ DBMS_DEFAULT_BUFFER_SIZE,
|
/* buf_size */ DBMS_DEFAULT_BUFFER_SIZE,
|
||||||
/* settings */ {});
|
/* settings */ {});
|
||||||
|
|
||||||
write_created = true;
|
|
||||||
|
|
||||||
{
|
{
|
||||||
std::lock_guard lock(path_map.mutex);
|
std::lock_guard lock(path_map.mutex);
|
||||||
auto & map = path_map.map;
|
auto & map = path_map.map;
|
||||||
@ -85,6 +91,9 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std:
|
|||||||
CurrentMetrics::add(metric, 1);
|
CurrentMetrics::add(metric, 1);
|
||||||
|
|
||||||
writeString(path.string(), *buf);
|
writeString(path.string(), *buf);
|
||||||
|
fiu_do_on(FailPoints::plain_object_storage_write_fail_on_directory_create, {
|
||||||
|
throw Exception(ErrorCodes::FAULT_INJECTED, "Injecting fault when creating '{}' directory", path);
|
||||||
|
});
|
||||||
buf->finalize();
|
buf->finalize();
|
||||||
|
|
||||||
write_finalized = true;
|
write_finalized = true;
|
||||||
@ -99,18 +108,16 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un
|
|||||||
|
|
||||||
if (write_finalized)
|
if (write_finalized)
|
||||||
{
|
{
|
||||||
|
LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageCreateDirectoryOperation"), "Undoing '{}' directory creation", path);
|
||||||
const auto base_path = path.parent_path();
|
const auto base_path = path.parent_path();
|
||||||
|
if (path_map.removePathIfExists(base_path))
|
||||||
{
|
{
|
||||||
std::lock_guard lock(path_map.mutex);
|
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
|
||||||
path_map.map.erase(base_path);
|
CurrentMetrics::sub(metric, 1);
|
||||||
}
|
}
|
||||||
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
|
|
||||||
CurrentMetrics::sub(metric, 1);
|
|
||||||
|
|
||||||
object_storage->removeObjectIfExists(StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME));
|
object_storage->removeObjectIfExists(StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME));
|
||||||
}
|
}
|
||||||
else if (write_created)
|
|
||||||
object_storage->removeObjectIfExists(StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFromPlainObjectStorageMoveDirectoryOperation(
|
MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFromPlainObjectStorageMoveDirectoryOperation(
|
||||||
@ -184,8 +191,10 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u
|
|||||||
getLogger("MetadataStorageFromPlainObjectStorageMoveDirectoryOperation"), "Moving directory '{}' to '{}'", path_from, path_to);
|
getLogger("MetadataStorageFromPlainObjectStorageMoveDirectoryOperation"), "Moving directory '{}' to '{}'", path_from, path_to);
|
||||||
|
|
||||||
auto write_buf = createWriteBuf(path_from, path_to, /* validate_content */ true);
|
auto write_buf = createWriteBuf(path_from, path_to, /* validate_content */ true);
|
||||||
write_created = true;
|
|
||||||
writeString(path_to.string(), *write_buf);
|
writeString(path_to.string(), *write_buf);
|
||||||
|
fiu_do_on(FailPoints::plain_object_storage_write_fail_on_directory_move, {
|
||||||
|
throw Exception(ErrorCodes::FAULT_INJECTED, "Injecting fault when moving from '{}' to '{}'", path_from, path_to);
|
||||||
|
});
|
||||||
write_buf->finalize();
|
write_buf->finalize();
|
||||||
|
|
||||||
/// parent_path() removes the trailing '/'.
|
/// parent_path() removes the trailing '/'.
|
||||||
@ -210,10 +219,7 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq
|
|||||||
std::lock_guard lock(path_map.mutex);
|
std::lock_guard lock(path_map.mutex);
|
||||||
auto & map = path_map.map;
|
auto & map = path_map.map;
|
||||||
map.emplace(path_from.parent_path(), map.extract(path_to.parent_path()).mapped());
|
map.emplace(path_from.parent_path(), map.extract(path_to.parent_path()).mapped());
|
||||||
}
|
|
||||||
|
|
||||||
if (write_created)
|
|
||||||
{
|
|
||||||
auto write_buf = createWriteBuf(path_to, path_from, /* verify_content */ false);
|
auto write_buf = createWriteBuf(path_to, path_from, /* verify_content */ false);
|
||||||
writeString(path_from.string(), *write_buf);
|
writeString(path_from.string(), *write_buf);
|
||||||
write_buf->finalize();
|
write_buf->finalize();
|
||||||
@ -249,19 +255,16 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std:
|
|||||||
auto metadata_object = StoredObject(/*remote_path*/ metadata_object_key.serialize(), /*local_path*/ path / PREFIX_PATH_FILE_NAME);
|
auto metadata_object = StoredObject(/*remote_path*/ metadata_object_key.serialize(), /*local_path*/ path / PREFIX_PATH_FILE_NAME);
|
||||||
object_storage->removeObjectIfExists(metadata_object);
|
object_storage->removeObjectIfExists(metadata_object);
|
||||||
|
|
||||||
|
if (path_map.removePathIfExists(base_path))
|
||||||
{
|
{
|
||||||
std::lock_guard lock(path_map.mutex);
|
removed = true;
|
||||||
auto & map = path_map.map;
|
|
||||||
map.erase(base_path);
|
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
|
||||||
|
CurrentMetrics::sub(metric, 1);
|
||||||
|
|
||||||
|
auto event = object_storage->getMetadataStorageMetrics().directory_removed;
|
||||||
|
ProfileEvents::increment(event);
|
||||||
}
|
}
|
||||||
|
|
||||||
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
|
|
||||||
CurrentMetrics::sub(metric, 1);
|
|
||||||
|
|
||||||
removed = true;
|
|
||||||
|
|
||||||
auto event = object_storage->getMetadataStorageMetrics().directory_removed;
|
|
||||||
ProfileEvents::increment(event);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::unique_lock<SharedMutex> &)
|
void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::unique_lock<SharedMutex> &)
|
||||||
|
@ -19,7 +19,6 @@ private:
|
|||||||
const std::string metadata_key_prefix;
|
const std::string metadata_key_prefix;
|
||||||
const std::string object_key_prefix;
|
const std::string object_key_prefix;
|
||||||
|
|
||||||
bool write_created = false;
|
|
||||||
bool write_finalized = false;
|
bool write_finalized = false;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
@ -43,7 +42,6 @@ private:
|
|||||||
ObjectStoragePtr object_storage;
|
ObjectStoragePtr object_storage;
|
||||||
const std::string metadata_key_prefix;
|
const std::string metadata_key_prefix;
|
||||||
|
|
||||||
bool write_created = false;
|
|
||||||
bool write_finalized = false;
|
bool write_finalized = false;
|
||||||
|
|
||||||
std::unique_ptr<WriteBufferFromFileBase>
|
std::unique_ptr<WriteBufferFromFileBase>
|
||||||
|
@ -0,0 +1,12 @@
|
|||||||
|
1 2
|
||||||
|
2 2
|
||||||
|
3 1
|
||||||
|
4 7
|
||||||
|
5 10
|
||||||
|
6 12
|
||||||
|
1 2
|
||||||
|
2 2
|
||||||
|
3 1
|
||||||
|
4 7
|
||||||
|
5 10
|
||||||
|
6 12
|
61
tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh
Executable file
61
tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh
Executable file
@ -0,0 +1,61 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
# Tags: no-fasttest, no-shared-merge-tree, no-parallel
|
||||||
|
# Tag no-fasttest: requires S3
|
||||||
|
# Tag no-shared-merge-tree: does not support replication
|
||||||
|
# Tag no-parallel: uses failpoints
|
||||||
|
|
||||||
|
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
# shellcheck source=../shell_config.sh
|
||||||
|
. "$CUR_DIR"/../shell_config.sh
|
||||||
|
|
||||||
|
on_exit() {
|
||||||
|
${CLICKHOUSE_CLIENT} -m --query "
|
||||||
|
SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_create;
|
||||||
|
SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_move;
|
||||||
|
"
|
||||||
|
}
|
||||||
|
|
||||||
|
trap on_exit EXIT
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_s3_mt_fault"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "
|
||||||
|
CREATE TABLE test_s3_mt_fault (a Int32, b Int64) engine = MergeTree() ORDER BY tuple(a, b)
|
||||||
|
SETTINGS disk = disk(
|
||||||
|
name = 03008_s3_plain_rewritable_fault,
|
||||||
|
type = s3_plain_rewritable,
|
||||||
|
endpoint = 'http://localhost:11111/test/03008_test_s3_mt_fault/',
|
||||||
|
access_key_id = clickhouse,
|
||||||
|
secret_access_key = clickhouse);
|
||||||
|
"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "
|
||||||
|
INSERT INTO test_s3_mt_fault (*) VALUES (1, 2), (2, 2), (3, 1), (4, 7), (5, 10), (6, 12);
|
||||||
|
OPTIMIZE TABLE test_s3_mt_fault FINAL;
|
||||||
|
"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "
|
||||||
|
SYSTEM ENABLE FAILPOINT plain_object_storage_write_fail_on_directory_create
|
||||||
|
"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "
|
||||||
|
INSERT INTO test_s3_mt_fault (*) select number, number from numbers_mt(100)" 2>&1 | grep -Fq "FAULT_INJECTED"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_s3_mt_fault;"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "
|
||||||
|
SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_create;
|
||||||
|
SYSTEM ENABLE FAILPOINT plain_object_storage_write_fail_on_directory_move;
|
||||||
|
"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "
|
||||||
|
INSERT INTO test_s3_mt_fault (*) select number, number from numbers_mt(100);
|
||||||
|
" 2>&1 | grep -Fq "FAULT_INJECTED"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_s3_mt_fault;"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "
|
||||||
|
SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_move;
|
||||||
|
"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE test_s3_mt_fault"
|
Loading…
Reference in New Issue
Block a user