mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 13:02:00 +00:00
add failpoint and test
This commit is contained in:
parent
0ce8088382
commit
4ac4098d2b
@ -41,7 +41,7 @@ static struct InitFiu
|
||||
REGULAR(use_delayed_remote_source) \
|
||||
REGULAR(cluster_discovery_faults) \
|
||||
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_before_op) \
|
||||
ONCE(smt_commit_write_zk_fail_after_op) \
|
||||
@ -77,6 +77,7 @@ static struct InitFiu
|
||||
REGULAR(replicated_merge_tree_all_replicas_stale) \
|
||||
REGULAR(zero_copy_lock_zk_fail_before_op) \
|
||||
REGULAR(zero_copy_lock_zk_fail_after_op) \
|
||||
REGULAR(plain_object_storage_write_fail_on_directory_create) \
|
||||
|
||||
|
||||
namespace FailPoints
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Poco/Timestamp.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/FailPoint.h>
|
||||
#include <Common/SharedLockGuard.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
@ -18,8 +19,14 @@ namespace ErrorCodes
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
extern const int FILE_ALREADY_EXISTS;
|
||||
extern const int INCORRECT_DATA;
|
||||
extern const int FAULT_INJECTED;
|
||||
};
|
||||
|
||||
namespace FailPoints
|
||||
{
|
||||
extern const char plain_object_storage_write_fail_on_directory_create[];
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
@ -72,8 +79,6 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std:
|
||||
/* buf_size */ DBMS_DEFAULT_BUFFER_SIZE,
|
||||
/* settings */ {});
|
||||
|
||||
write_created = true;
|
||||
|
||||
{
|
||||
std::lock_guard lock(path_map.mutex);
|
||||
auto & map = path_map.map;
|
||||
@ -85,6 +90,9 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std:
|
||||
CurrentMetrics::add(metric, 1);
|
||||
|
||||
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();
|
||||
|
||||
write_finalized = true;
|
||||
@ -97,8 +105,9 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un
|
||||
{
|
||||
auto metadata_object_key = createMetadataObjectKey(object_key_prefix, metadata_key_prefix);
|
||||
|
||||
if (write_finalized || write_created)
|
||||
if (write_finalized)
|
||||
{
|
||||
LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageCreateDirectoryOperation"), "Undoing '{}' directory creation", path);
|
||||
const auto base_path = path.parent_path();
|
||||
if (path_map.removePathIfExists(base_path))
|
||||
{
|
||||
|
@ -19,7 +19,6 @@ private:
|
||||
const std::string metadata_key_prefix;
|
||||
const std::string object_key_prefix;
|
||||
|
||||
bool write_created = false;
|
||||
bool write_finalized = false;
|
||||
|
||||
public:
|
||||
|
@ -0,0 +1 @@
|
||||
0
|
43
tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh
Executable file
43
tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh
Executable file
@ -0,0 +1,43 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, no-shared-merge-tree
|
||||
# Tag no-fasttest: requires S3
|
||||
# Tag no-shared-merge-tree: does not support replication
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
on_exit() {
|
||||
${CLICKHOUSE_CLIENT} --query "
|
||||
SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_create
|
||||
"
|
||||
}
|
||||
|
||||
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 "
|
||||
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 count(*) from test_s3_mt_fault;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE test_s3_mt_fault;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "
|
||||
SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_create
|
||||
"
|
Loading…
Reference in New Issue
Block a user