diff --git a/README.md b/README.md
index abaf27abf11..3270cd19671 100644
--- a/README.md
+++ b/README.md
@@ -42,7 +42,6 @@ Keep an eye out for upcoming meetups and events around the world. Somewhere else
Upcoming meetups
-* [Barcelona Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096876/) - November 12
* [Ghent Meetup](https://www.meetup.com/clickhouse-belgium-user-group/events/303049405/) - November 19
* [Dubai Meetup](https://www.meetup.com/clickhouse-dubai-meetup-group/events/303096989/) - November 21
* [Paris Meetup](https://www.meetup.com/clickhouse-france-user-group/events/303096434) - November 26
@@ -53,6 +52,7 @@ Upcoming meetups
Recently completed meetups
+* [Barcelona Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096876/) - November 12
* [Madrid Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096564/) - October 22
* [Singapore Meetup](https://www.meetup.com/clickhouse-singapore-meetup-group/events/303212064/) - October 3
* [Jakarta Meetup](https://www.meetup.com/clickhouse-indonesia-user-group/events/303191359/) - October 1
diff --git a/base/base/defines.h b/base/base/defines.h
index 5685a6d9833..a0c3c0d1de5 100644
--- a/base/base/defines.h
+++ b/base/base/defines.h
@@ -145,6 +145,7 @@
#define TSA_TRY_ACQUIRE_SHARED(...) __attribute__((try_acquire_shared_capability(__VA_ARGS__))) /// function tries to acquire a shared capability and returns a boolean value indicating success or failure
#define TSA_RELEASE_SHARED(...) __attribute__((release_shared_capability(__VA_ARGS__))) /// function releases the given shared capability
#define TSA_SCOPED_LOCKABLE __attribute__((scoped_lockable)) /// object of a class has scoped lockable capability
+#define TSA_RETURN_CAPABILITY(...) __attribute__((lock_returned(__VA_ARGS__))) /// to return capabilities in functions
/// Macros for suppressing TSA warnings for specific reads/writes (instead of suppressing it for the whole function)
/// They use a lambda function to apply function attribute to a single statement. This enable us to suppress warnings locally instead of
diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu
index 0fe9a409ee4..e6bde845c4e 100644
--- a/docker/server/Dockerfile.ubuntu
+++ b/docker/server/Dockerfile.ubuntu
@@ -113,7 +113,9 @@ RUN clickhouse local -q 'SELECT 1' >/dev/null 2>&1 && exit 0 || : \
/var/lib/apt/lists/* \
/var/cache/debconf \
/tmp/* \
- && apt-get autoremove --purge -yq dirmngr gnupg2
+ && apt-get autoremove --purge -yq dirmngr gnupg2 \
+ && chmod ugo+Xrw -R /etc/clickhouse-server /etc/clickhouse-client
+# The last chmod is here to make the next one is No-op in docker official library Dockerfile
# post install
# we need to allow "others" access to clickhouse folder, because docker container
diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md
index c5f92ccdf68..ca4938b1a47 100644
--- a/docs/en/operations/server-configuration-parameters/settings.md
+++ b/docs/en/operations/server-configuration-parameters/settings.md
@@ -597,6 +597,30 @@ If number of tables is greater than this value, server will throw an exception.
400
```
+## max\_replicated\_table\_num\_to\_throw {#max-replicated-table-num-to-throw}
+If number of replicated tables is greater than this value, server will throw an exception. 0 means no limitation. Only count table in Atomic/Ordinary/Replicated/Lazy database engine.
+
+**Example**
+```xml
+400
+```
+
+## max\_dictionary\_num\_to\_throw {#max-dictionary-num-to-throw}
+If number of dictionaries is greater than this value, server will throw an exception. 0 means no limitation. Only count table in Atomic/Ordinary/Replicated/Lazy database engine.
+
+**Example**
+```xml
+400
+```
+
+## max\_view\_num\_to\_throw {#max-view-num-to-throw}
+If number of views is greater than this value, server will throw an exception. 0 means no limitation. Only count table in Atomic/Ordinary/Replicated/Lazy database engine.
+
+**Example**
+```xml
+400
+```
+
## max\_database\_num\_to\_throw {#max-table-num-to-throw}
If number of _database is greater than this value, server will throw an exception. 0 means no limitation.
Default value: 0
diff --git a/programs/compressor/Compressor.cpp b/programs/compressor/Compressor.cpp
index bf67db8ff2e..e73f61dde83 100644
--- a/programs/compressor/Compressor.cpp
+++ b/programs/compressor/Compressor.cpp
@@ -12,9 +12,12 @@
#include
#include
#include
+#include
+#include
#include
#include
#include
+#include
#include
#include
#include
@@ -43,29 +46,24 @@ namespace CurrentMetrics
namespace
{
-/// Outputs sizes of uncompressed and compressed blocks for compressed file.
+/// Outputs method, sizes of uncompressed and compressed blocks for compressed file.
void checkAndWriteHeader(DB::ReadBuffer & in, DB::WriteBuffer & out)
{
while (!in.eof())
{
- in.ignore(16); /// checksum
-
- char header[COMPRESSED_BLOCK_HEADER_SIZE];
- in.readStrict(header, COMPRESSED_BLOCK_HEADER_SIZE);
-
- UInt32 size_compressed = unalignedLoad(&header[1]);
+ UInt32 size_compressed;
+ UInt32 size_decompressed;
+ auto codec = DB::getCompressionCodecForFile(in, size_compressed, size_decompressed, true /* skip_to_next_block */);
if (size_compressed > DBMS_MAX_COMPRESSED_SIZE)
throw DB::Exception(DB::ErrorCodes::TOO_LARGE_SIZE_COMPRESSED, "Too large size_compressed. Most likely corrupted data.");
- UInt32 size_decompressed = unalignedLoad(&header[5]);
-
+ DB::writeText(queryToString(codec->getFullCodecDesc()), out);
+ DB::writeChar('\t', out);
DB::writeText(size_decompressed, out);
DB::writeChar('\t', out);
DB::writeText(size_compressed, out);
DB::writeChar('\n', out);
-
- in.ignore(size_compressed - COMPRESSED_BLOCK_HEADER_SIZE);
}
}
diff --git a/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.cpp b/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.cpp
index 5819c533fd9..ad1fecac784 100644
--- a/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.cpp
+++ b/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.cpp
@@ -22,6 +22,13 @@ namespace ErrorCodes
namespace
{
+/** Due to a lack of proper code review, this code was contributed with a multiplication of template instantiations
+ * over all pairs of data types, and we deeply regret that.
+ *
+ * We cannot remove all combinations, because the binary representation of serialized data has to remain the same,
+ * but we can partially heal the wound by treating unsigned and signed data types in the same way.
+ */
+
template
struct AggregationFunctionDeltaSumTimestampData
{
@@ -37,23 +44,22 @@ template
class AggregationFunctionDeltaSumTimestamp final
: public IAggregateFunctionDataHelper<
AggregationFunctionDeltaSumTimestampData,
- AggregationFunctionDeltaSumTimestamp
- >
+ AggregationFunctionDeltaSumTimestamp>
{
public:
AggregationFunctionDeltaSumTimestamp(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<
AggregationFunctionDeltaSumTimestampData,
- AggregationFunctionDeltaSumTimestamp
- >{arguments, params, createResultType()}
- {}
+ AggregationFunctionDeltaSumTimestamp>{arguments, params, createResultType()}
+ {
+ }
AggregationFunctionDeltaSumTimestamp()
: IAggregateFunctionDataHelper<
AggregationFunctionDeltaSumTimestampData,
- AggregationFunctionDeltaSumTimestamp
- >{}
- {}
+ AggregationFunctionDeltaSumTimestamp>{}
+ {
+ }
bool allocatesMemoryInArena() const override { return false; }
@@ -63,8 +69,8 @@ public:
void NO_SANITIZE_UNDEFINED ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
- auto value = assert_cast &>(*columns[0]).getData()[row_num];
- auto ts = assert_cast &>(*columns[1]).getData()[row_num];
+ auto value = unalignedLoad(columns[0]->getRawData().data() + row_num * sizeof(ValueType));
+ auto ts = unalignedLoad(columns[1]->getRawData().data() + row_num * sizeof(TimestampType));
auto & data = this->data(place);
@@ -172,10 +178,48 @@ public:
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
- assert_cast &>(to).getData().push_back(this->data(place).sum);
+ static_cast(to).template insertRawData(
+ reinterpret_cast(&this->data(place).sum));
}
};
+
+template class AggregateFunctionTemplate, typename... TArgs>
+IAggregateFunction * createWithTwoTypesSecond(const IDataType & second_type, TArgs && ... args)
+{
+ WhichDataType which(second_type);
+
+ if (which.idx == TypeIndex::UInt32) return new AggregateFunctionTemplate(args...);
+ if (which.idx == TypeIndex::UInt64) return new AggregateFunctionTemplate(args...);
+ if (which.idx == TypeIndex::Int32) return new AggregateFunctionTemplate(args...);
+ if (which.idx == TypeIndex::Int64) return new AggregateFunctionTemplate(args...);
+ if (which.idx == TypeIndex::Float32) return new AggregateFunctionTemplate(args...);
+ if (which.idx == TypeIndex::Float64) return new AggregateFunctionTemplate(args...);
+ if (which.idx == TypeIndex::Date) return new AggregateFunctionTemplate(args...);
+ if (which.idx == TypeIndex::DateTime) return new AggregateFunctionTemplate(args...);
+
+ return nullptr;
+}
+
+template class AggregateFunctionTemplate, typename... TArgs>
+IAggregateFunction * createWithTwoTypes(const IDataType & first_type, const IDataType & second_type, TArgs && ... args)
+{
+ WhichDataType which(first_type);
+
+ if (which.idx == TypeIndex::UInt8) return createWithTwoTypesSecond(second_type, args...);
+ if (which.idx == TypeIndex::UInt16) return createWithTwoTypesSecond(second_type, args...);
+ if (which.idx == TypeIndex::UInt32) return createWithTwoTypesSecond(second_type, args...);
+ if (which.idx == TypeIndex::UInt64) return createWithTwoTypesSecond(second_type, args...);
+ if (which.idx == TypeIndex::Int8) return createWithTwoTypesSecond(second_type, args...);
+ if (which.idx == TypeIndex::Int16) return createWithTwoTypesSecond(second_type, args...);
+ if (which.idx == TypeIndex::Int32) return createWithTwoTypesSecond(second_type, args...);
+ if (which.idx == TypeIndex::Int64) return createWithTwoTypesSecond(second_type, args...);
+ if (which.idx == TypeIndex::Float32) return createWithTwoTypesSecond(second_type, args...);
+ if (which.idx == TypeIndex::Float64) return createWithTwoTypesSecond(second_type, args...);
+
+ return nullptr;
+}
+
AggregateFunctionPtr createAggregateFunctionDeltaSumTimestamp(
const String & name,
const DataTypes & arguments,
@@ -193,7 +237,7 @@ AggregateFunctionPtr createAggregateFunctionDeltaSumTimestamp(
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}, "
"must be Int, Float, Date, DateTime", arguments[1]->getName(), name);
- return AggregateFunctionPtr(createWithTwoNumericOrDateTypes(
+ return AggregateFunctionPtr(createWithTwoTypes(
*arguments[0], *arguments[1], arguments, params));
}
}
diff --git a/src/AggregateFunctions/Helpers.h b/src/AggregateFunctions/Helpers.h
index 2b8459b6684..24842b19f1b 100644
--- a/src/AggregateFunctions/Helpers.h
+++ b/src/AggregateFunctions/Helpers.h
@@ -184,36 +184,8 @@ static IAggregateFunction * createWithDecimalType(const IDataType & argument_typ
}
/** For template with two arguments.
+ * This is an extremely dangerous for code bloat - do not use.
*/
-template class AggregateFunctionTemplate, typename... TArgs>
-static IAggregateFunction * createWithTwoNumericTypesSecond(const IDataType & second_type, TArgs && ... args)
-{
- WhichDataType which(second_type);
-#define DISPATCH(TYPE) \
- if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate(args...);
- FOR_NUMERIC_TYPES(DISPATCH)
-#undef DISPATCH
- if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate(args...);
- if (which.idx == TypeIndex::Enum16) return new AggregateFunctionTemplate(args...);
- return nullptr;
-}
-
-template class AggregateFunctionTemplate, typename... TArgs>
-static IAggregateFunction * createWithTwoNumericTypes(const IDataType & first_type, const IDataType & second_type, TArgs && ... args)
-{
- WhichDataType which(first_type);
-#define DISPATCH(TYPE) \
- if (which.idx == TypeIndex::TYPE) \
- return createWithTwoNumericTypesSecond(second_type, args...);
- FOR_NUMERIC_TYPES(DISPATCH)
-#undef DISPATCH
- if (which.idx == TypeIndex::Enum8)
- return createWithTwoNumericTypesSecond(second_type, args...);
- if (which.idx == TypeIndex::Enum16)
- return createWithTwoNumericTypesSecond(second_type, args...);
- return nullptr;
-}
-
template class AggregateFunctionTemplate, typename... TArgs>
static IAggregateFunction * createWithTwoBasicNumericTypesSecond(const IDataType & second_type, TArgs && ... args)
{
@@ -237,46 +209,6 @@ static IAggregateFunction * createWithTwoBasicNumericTypes(const IDataType & fir
return nullptr;
}
-template class AggregateFunctionTemplate, typename... TArgs>
-static IAggregateFunction * createWithTwoNumericOrDateTypesSecond(const IDataType & second_type, TArgs && ... args)
-{
- WhichDataType which(second_type);
-#define DISPATCH(TYPE) \
- if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate(args...);
- FOR_NUMERIC_TYPES(DISPATCH)
-#undef DISPATCH
- if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate(args...);
- if (which.idx == TypeIndex::Enum16) return new AggregateFunctionTemplate(args...);
-
- /// expects that DataTypeDate based on UInt16, DataTypeDateTime based on UInt32
- if (which.idx == TypeIndex::Date) return new AggregateFunctionTemplate(args...);
- if (which.idx == TypeIndex::DateTime) return new AggregateFunctionTemplate(args...);
-
- return nullptr;
-}
-
-template class AggregateFunctionTemplate, typename... TArgs>
-static IAggregateFunction * createWithTwoNumericOrDateTypes(const IDataType & first_type, const IDataType & second_type, TArgs && ... args)
-{
- WhichDataType which(first_type);
-#define DISPATCH(TYPE) \
- if (which.idx == TypeIndex::TYPE) \
- return createWithTwoNumericOrDateTypesSecond(second_type, args...);
- FOR_NUMERIC_TYPES(DISPATCH)
-#undef DISPATCH
- if (which.idx == TypeIndex::Enum8)
- return createWithTwoNumericOrDateTypesSecond(second_type, args...);
- if (which.idx == TypeIndex::Enum16)
- return createWithTwoNumericOrDateTypesSecond(second_type, args...);
-
- /// expects that DataTypeDate based on UInt16, DataTypeDateTime based on UInt32
- if (which.idx == TypeIndex::Date)
- return createWithTwoNumericOrDateTypesSecond(second_type, args...);
- if (which.idx == TypeIndex::DateTime)
- return createWithTwoNumericOrDateTypesSecond(second_type, args...);
- return nullptr;
-}
-
template class AggregateFunctionTemplate, typename... TArgs>
static IAggregateFunction * createWithStringType(const IDataType & argument_type, TArgs && ... args)
{
diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp
index 1ad7e7996c7..27b46a112e2 100644
--- a/src/Analyzer/FunctionNode.cpp
+++ b/src/Analyzer/FunctionNode.cpp
@@ -88,6 +88,7 @@ void FunctionNode::resolveAsFunction(FunctionBasePtr function_value)
function_name = function_value->getName();
function = std::move(function_value);
kind = FunctionKind::ORDINARY;
+ nulls_action = NullsAction::EMPTY;
}
void FunctionNode::resolveAsAggregateFunction(AggregateFunctionPtr aggregate_function_value)
@@ -95,6 +96,12 @@ void FunctionNode::resolveAsAggregateFunction(AggregateFunctionPtr aggregate_fun
function_name = aggregate_function_value->getName();
function = std::move(aggregate_function_value);
kind = FunctionKind::AGGREGATE;
+ /** When the function is resolved, we do not need the nulls action anymore.
+ * The only thing that the nulls action does is map from one function to another.
+ * Thus, the nulls action is encoded in the function name and does not make sense anymore.
+ * Keeping the nulls action may lead to incorrect comparison of functions, e.g., count() and count() IGNORE NULLS are the same function.
+ */
+ nulls_action = NullsAction::EMPTY;
}
void FunctionNode::resolveAsWindowFunction(AggregateFunctionPtr window_function_value)
diff --git a/src/Analyzer/Passes/FuseFunctionsPass.cpp b/src/Analyzer/Passes/FuseFunctionsPass.cpp
index 17a765a068b..cd2577c3d76 100644
--- a/src/Analyzer/Passes/FuseFunctionsPass.cpp
+++ b/src/Analyzer/Passes/FuseFunctionsPass.cpp
@@ -85,10 +85,9 @@ QueryTreeNodePtr createResolvedFunction(const ContextPtr & context, const String
}
FunctionNodePtr createResolvedAggregateFunction(
- const String & name, const QueryTreeNodePtr & argument, const Array & parameters = {}, NullsAction action = NullsAction::EMPTY)
+ const String & name, const QueryTreeNodePtr & argument, const Array & parameters = {})
{
auto function_node = std::make_shared(name);
- function_node->setNullsAction(action);
if (!parameters.empty())
{
@@ -100,7 +99,7 @@ FunctionNodePtr createResolvedAggregateFunction(
function_node->getArguments().getNodes() = { argument };
AggregateFunctionProperties properties;
- auto aggregate_function = AggregateFunctionFactory::instance().get(name, action, {argument->getResultType()}, parameters, properties);
+ auto aggregate_function = AggregateFunctionFactory::instance().get(name, NullsAction::EMPTY, {argument->getResultType()}, parameters, properties);
function_node->resolveAsAggregateFunction(std::move(aggregate_function));
return function_node;
diff --git a/src/Backups/BackupConcurrencyCheck.cpp b/src/Backups/BackupConcurrencyCheck.cpp
index 8b29ae41b53..a67d241845d 100644
--- a/src/Backups/BackupConcurrencyCheck.cpp
+++ b/src/Backups/BackupConcurrencyCheck.cpp
@@ -14,12 +14,12 @@ namespace ErrorCodes
BackupConcurrencyCheck::BackupConcurrencyCheck(
- const UUID & backup_or_restore_uuid_,
bool is_restore_,
bool on_cluster_,
+ const String & zookeeper_path_,
bool allow_concurrency_,
BackupConcurrencyCounters & counters_)
- : is_restore(is_restore_), backup_or_restore_uuid(backup_or_restore_uuid_), on_cluster(on_cluster_), counters(counters_)
+ : is_restore(is_restore_), on_cluster(on_cluster_), zookeeper_path(zookeeper_path_), counters(counters_)
{
std::lock_guard lock{counters.mutex};
@@ -32,7 +32,7 @@ BackupConcurrencyCheck::BackupConcurrencyCheck(
size_t num_on_cluster_restores = counters.on_cluster_restores.size();
if (on_cluster)
{
- if (!counters.on_cluster_restores.contains(backup_or_restore_uuid))
+ if (!counters.on_cluster_restores.contains(zookeeper_path))
++num_on_cluster_restores;
}
else
@@ -47,7 +47,7 @@ BackupConcurrencyCheck::BackupConcurrencyCheck(
size_t num_on_cluster_backups = counters.on_cluster_backups.size();
if (on_cluster)
{
- if (!counters.on_cluster_backups.contains(backup_or_restore_uuid))
+ if (!counters.on_cluster_backups.contains(zookeeper_path))
++num_on_cluster_backups;
}
else
@@ -64,9 +64,9 @@ BackupConcurrencyCheck::BackupConcurrencyCheck(
if (on_cluster)
{
if (is_restore)
- ++counters.on_cluster_restores[backup_or_restore_uuid];
+ ++counters.on_cluster_restores[zookeeper_path];
else
- ++counters.on_cluster_backups[backup_or_restore_uuid];
+ ++counters.on_cluster_backups[zookeeper_path];
}
else
{
@@ -86,7 +86,7 @@ BackupConcurrencyCheck::~BackupConcurrencyCheck()
{
if (is_restore)
{
- auto it = counters.on_cluster_restores.find(backup_or_restore_uuid);
+ auto it = counters.on_cluster_restores.find(zookeeper_path);
if (it != counters.on_cluster_restores.end())
{
if (!--it->second)
@@ -95,7 +95,7 @@ BackupConcurrencyCheck::~BackupConcurrencyCheck()
}
else
{
- auto it = counters.on_cluster_backups.find(backup_or_restore_uuid);
+ auto it = counters.on_cluster_backups.find(zookeeper_path);
if (it != counters.on_cluster_backups.end())
{
if (!--it->second)
diff --git a/src/Backups/BackupConcurrencyCheck.h b/src/Backups/BackupConcurrencyCheck.h
index 048a23a716a..a1baeff5464 100644
--- a/src/Backups/BackupConcurrencyCheck.h
+++ b/src/Backups/BackupConcurrencyCheck.h
@@ -1,7 +1,8 @@
#pragma once
-#include
+#include
#include
+#include
#include
#include
@@ -19,9 +20,9 @@ public:
/// Checks concurrency of a BACKUP operation or a RESTORE operation.
/// Keep a constructed instance of BackupConcurrencyCheck until the operation is done.
BackupConcurrencyCheck(
- const UUID & backup_or_restore_uuid_,
bool is_restore_,
bool on_cluster_,
+ const String & zookeeper_path_,
bool allow_concurrency_,
BackupConcurrencyCounters & counters_);
@@ -31,8 +32,8 @@ public:
private:
const bool is_restore;
- const UUID backup_or_restore_uuid;
const bool on_cluster;
+ const String zookeeper_path;
BackupConcurrencyCounters & counters;
};
@@ -47,8 +48,8 @@ private:
friend class BackupConcurrencyCheck;
size_t local_backups TSA_GUARDED_BY(mutex) = 0;
size_t local_restores TSA_GUARDED_BY(mutex) = 0;
- std::unordered_map on_cluster_backups TSA_GUARDED_BY(mutex);
- std::unordered_map on_cluster_restores TSA_GUARDED_BY(mutex);
+ std::unordered_map on_cluster_backups TSA_GUARDED_BY(mutex);
+ std::unordered_map on_cluster_restores TSA_GUARDED_BY(mutex);
std::mutex mutex;
};
diff --git a/src/Backups/BackupCoordinationCleaner.cpp b/src/Backups/BackupCoordinationCleaner.cpp
index 1f5068a94de..47095f27eb3 100644
--- a/src/Backups/BackupCoordinationCleaner.cpp
+++ b/src/Backups/BackupCoordinationCleaner.cpp
@@ -4,31 +4,29 @@
namespace DB
{
-BackupCoordinationCleaner::BackupCoordinationCleaner(const String & zookeeper_path_, const WithRetries & with_retries_, LoggerPtr log_)
- : zookeeper_path(zookeeper_path_), with_retries(with_retries_), log(log_)
+BackupCoordinationCleaner::BackupCoordinationCleaner(bool is_restore_, const String & zookeeper_path_, const WithRetries & with_retries_, LoggerPtr log_)
+ : is_restore(is_restore_), zookeeper_path(zookeeper_path_), with_retries(with_retries_), log(log_)
{
}
-void BackupCoordinationCleaner::cleanup()
+bool BackupCoordinationCleaner::cleanup(bool throw_if_error)
{
- tryRemoveAllNodes(/* throw_if_error = */ true, /* retries_kind = */ WithRetries::kNormal);
+ WithRetries::Kind retries_kind = throw_if_error ? WithRetries::kNormal : WithRetries::kErrorHandling;
+ return cleanupImpl(throw_if_error, retries_kind);
}
-bool BackupCoordinationCleaner::tryCleanupAfterError() noexcept
-{
- return tryRemoveAllNodes(/* throw_if_error = */ false, /* retries_kind = */ WithRetries::kNormal);
-}
-
-bool BackupCoordinationCleaner::tryRemoveAllNodes(bool throw_if_error, WithRetries::Kind retries_kind)
+bool BackupCoordinationCleaner::cleanupImpl(bool throw_if_error, WithRetries::Kind retries_kind)
{
{
std::lock_guard lock{mutex};
- if (cleanup_result.succeeded)
- return true;
- if (cleanup_result.exception)
+ if (succeeded)
{
- if (throw_if_error)
- std::rethrow_exception(cleanup_result.exception);
+ LOG_TRACE(log, "Nodes from ZooKeeper are already removed");
+ return true;
+ }
+ if (tried)
+ {
+ LOG_INFO(log, "Skipped removing nodes from ZooKeeper because because earlier we failed to do that");
return false;
}
}
@@ -44,16 +42,18 @@ bool BackupCoordinationCleaner::tryRemoveAllNodes(bool throw_if_error, WithRetri
});
std::lock_guard lock{mutex};
- cleanup_result.succeeded = true;
+ tried = true;
+ succeeded = true;
return true;
}
catch (...)
{
- LOG_TRACE(log, "Caught exception while removing nodes from ZooKeeper for this restore: {}",
+ LOG_TRACE(log, "Caught exception while removing nodes from ZooKeeper for this {}: {}",
+ is_restore ? "restore" : "backup",
getCurrentExceptionMessage(/* with_stacktrace= */ false, /* check_embedded_stacktrace= */ true));
std::lock_guard lock{mutex};
- cleanup_result.exception = std::current_exception();
+ tried = true;
if (throw_if_error)
throw;
diff --git a/src/Backups/BackupCoordinationCleaner.h b/src/Backups/BackupCoordinationCleaner.h
index 43e095d9f33..c760a3611f9 100644
--- a/src/Backups/BackupCoordinationCleaner.h
+++ b/src/Backups/BackupCoordinationCleaner.h
@@ -12,14 +12,14 @@ namespace DB
class BackupCoordinationCleaner
{
public:
- BackupCoordinationCleaner(const String & zookeeper_path_, const WithRetries & with_retries_, LoggerPtr log_);
+ BackupCoordinationCleaner(bool is_restore_, const String & zookeeper_path_, const WithRetries & with_retries_, LoggerPtr log_);
- void cleanup();
- bool tryCleanupAfterError() noexcept;
+ bool cleanup(bool throw_if_error);
private:
- bool tryRemoveAllNodes(bool throw_if_error, WithRetries::Kind retries_kind);
+ bool cleanupImpl(bool throw_if_error, WithRetries::Kind retries_kind);
+ const bool is_restore;
const String zookeeper_path;
/// A reference to a field of the parent object which is either BackupCoordinationOnCluster or RestoreCoordinationOnCluster.
@@ -27,13 +27,8 @@ private:
const LoggerPtr log;
- struct CleanupResult
- {
- bool succeeded = false;
- std::exception_ptr exception;
- };
- CleanupResult cleanup_result TSA_GUARDED_BY(mutex);
-
+ bool tried TSA_GUARDED_BY(mutex) = false;
+ bool succeeded TSA_GUARDED_BY(mutex) = false;
std::mutex mutex;
};
diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp
index 8bd6b4d327d..402e789eacb 100644
--- a/src/Backups/BackupCoordinationLocal.cpp
+++ b/src/Backups/BackupCoordinationLocal.cpp
@@ -11,12 +11,11 @@ namespace DB
{
BackupCoordinationLocal::BackupCoordinationLocal(
- const UUID & backup_uuid_,
bool is_plain_backup_,
bool allow_concurrent_backup_,
BackupConcurrencyCounters & concurrency_counters_)
: log(getLogger("BackupCoordinationLocal"))
- , concurrency_check(backup_uuid_, /* is_restore = */ false, /* on_cluster = */ false, allow_concurrent_backup_, concurrency_counters_)
+ , concurrency_check(/* is_restore = */ false, /* on_cluster = */ false, /* zookeeper_path = */ "", allow_concurrent_backup_, concurrency_counters_)
, file_infos(is_plain_backup_)
{
}
diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h
index 09991c0d301..e63fcde981a 100644
--- a/src/Backups/BackupCoordinationLocal.h
+++ b/src/Backups/BackupCoordinationLocal.h
@@ -23,20 +23,19 @@ class BackupCoordinationLocal : public IBackupCoordination
{
public:
explicit BackupCoordinationLocal(
- const UUID & backup_uuid_,
bool is_plain_backup_,
bool allow_concurrent_backup_,
BackupConcurrencyCounters & concurrency_counters_);
~BackupCoordinationLocal() override;
+ void setBackupQueryIsSentToOtherHosts() override {}
+ bool isBackupQuerySentToOtherHosts() const override { return false; }
Strings setStage(const String &, const String &, bool) override { return {}; }
- void setBackupQueryWasSentToOtherHosts() override {}
- bool trySetError(std::exception_ptr) override { return true; }
- void finish() override {}
- bool tryFinishAfterError() noexcept override { return true; }
- void waitForOtherHostsToFinish() override {}
- bool tryWaitForOtherHostsToFinishAfterError() noexcept override { return true; }
+ bool setError(std::exception_ptr, bool) override { return true; }
+ bool waitOtherHostsFinish(bool) const override { return true; }
+ bool finish(bool) override { return true; }
+ bool cleanup(bool) override { return true; }
void addReplicatedPartNames(const String & table_zk_path, const String & table_name_for_logs, const String & replica_name,
const std::vector & part_names_and_checksums) override;
diff --git a/src/Backups/BackupCoordinationOnCluster.cpp b/src/Backups/BackupCoordinationOnCluster.cpp
index dc34939f805..1b14f226eff 100644
--- a/src/Backups/BackupCoordinationOnCluster.cpp
+++ b/src/Backups/BackupCoordinationOnCluster.cpp
@@ -184,17 +184,21 @@ BackupCoordinationOnCluster::BackupCoordinationOnCluster(
, plain_backup(is_plain_backup_)
, log(getLogger("BackupCoordinationOnCluster"))
, with_retries(log, get_zookeeper_, keeper_settings, process_list_element_, [root_zookeeper_path_](Coordination::ZooKeeperWithFaultInjection::Ptr zk) { zk->sync(root_zookeeper_path_); })
- , concurrency_check(backup_uuid_, /* is_restore = */ false, /* on_cluster = */ true, allow_concurrent_backup_, concurrency_counters_)
- , stage_sync(/* is_restore = */ false, fs::path{zookeeper_path} / "stage", current_host, all_hosts, allow_concurrent_backup_, with_retries, schedule_, process_list_element_, log)
- , cleaner(zookeeper_path, with_retries, log)
+ , cleaner(/* is_restore = */ false, zookeeper_path, with_retries, log)
+ , stage_sync(/* is_restore = */ false, fs::path{zookeeper_path} / "stage", current_host, all_hosts, allow_concurrent_backup_, concurrency_counters_, with_retries, schedule_, process_list_element_, log)
{
- createRootNodes();
+ try
+ {
+ createRootNodes();
+ }
+ catch (...)
+ {
+ stage_sync.setError(std::current_exception(), /* throw_if_error = */ false);
+ throw;
+ }
}
-BackupCoordinationOnCluster::~BackupCoordinationOnCluster()
-{
- tryFinishImpl();
-}
+BackupCoordinationOnCluster::~BackupCoordinationOnCluster() = default;
void BackupCoordinationOnCluster::createRootNodes()
{
@@ -217,69 +221,52 @@ void BackupCoordinationOnCluster::createRootNodes()
});
}
+void BackupCoordinationOnCluster::setBackupQueryIsSentToOtherHosts()
+{
+ stage_sync.setQueryIsSentToOtherHosts();
+}
+
+bool BackupCoordinationOnCluster::isBackupQuerySentToOtherHosts() const
+{
+ return stage_sync.isQuerySentToOtherHosts();
+}
+
Strings BackupCoordinationOnCluster::setStage(const String & new_stage, const String & message, bool sync)
{
stage_sync.setStage(new_stage, message);
-
- if (!sync)
- return {};
-
- return stage_sync.waitForHostsToReachStage(new_stage, all_hosts_without_initiator);
+ if (sync)
+ return stage_sync.waitHostsReachStage(all_hosts_without_initiator, new_stage);
+ return {};
}
-void BackupCoordinationOnCluster::setBackupQueryWasSentToOtherHosts()
+bool BackupCoordinationOnCluster::setError(std::exception_ptr exception, bool throw_if_error)
{
- backup_query_was_sent_to_other_hosts = true;
+ return stage_sync.setError(exception, throw_if_error);
}
-bool BackupCoordinationOnCluster::trySetError(std::exception_ptr exception)
+bool BackupCoordinationOnCluster::waitOtherHostsFinish(bool throw_if_error) const
{
- return stage_sync.trySetError(exception);
+ return stage_sync.waitOtherHostsFinish(throw_if_error);
}
-void BackupCoordinationOnCluster::finish()
+bool BackupCoordinationOnCluster::finish(bool throw_if_error)
{
- bool other_hosts_also_finished = false;
- stage_sync.finish(other_hosts_also_finished);
-
- if ((current_host == kInitiator) && (other_hosts_also_finished || !backup_query_was_sent_to_other_hosts))
- cleaner.cleanup();
+ return stage_sync.finish(throw_if_error);
}
-bool BackupCoordinationOnCluster::tryFinishAfterError() noexcept
+bool BackupCoordinationOnCluster::cleanup(bool throw_if_error)
{
- return tryFinishImpl();
-}
-
-bool BackupCoordinationOnCluster::tryFinishImpl() noexcept
-{
- bool other_hosts_also_finished = false;
- if (!stage_sync.tryFinishAfterError(other_hosts_also_finished))
- return false;
-
- if ((current_host == kInitiator) && (other_hosts_also_finished || !backup_query_was_sent_to_other_hosts))
+ /// All the hosts must finish before we remove the coordination nodes.
+ bool expect_other_hosts_finished = stage_sync.isQuerySentToOtherHosts() || !stage_sync.isErrorSet();
+ bool all_hosts_finished = stage_sync.finished() && (stage_sync.otherHostsFinished() || !expect_other_hosts_finished);
+ if (!all_hosts_finished)
{
- if (!cleaner.tryCleanupAfterError())
- return false;
- }
-
- return true;
-}
-
-void BackupCoordinationOnCluster::waitForOtherHostsToFinish()
-{
- if ((current_host != kInitiator) || !backup_query_was_sent_to_other_hosts)
- return;
- stage_sync.waitForOtherHostsToFinish();
-}
-
-bool BackupCoordinationOnCluster::tryWaitForOtherHostsToFinishAfterError() noexcept
-{
- if (current_host != kInitiator)
+ auto unfinished_hosts = expect_other_hosts_finished ? stage_sync.getUnfinishedHosts() : Strings{current_host};
+ LOG_INFO(log, "Skipping removing nodes from ZooKeeper because hosts {} didn't finish",
+ BackupCoordinationStageSync::getHostsDesc(unfinished_hosts));
return false;
- if (!backup_query_was_sent_to_other_hosts)
- return true;
- return stage_sync.tryWaitForOtherHostsToFinishAfterError();
+ }
+ return cleaner.cleanup(throw_if_error);
}
ZooKeeperRetriesInfo BackupCoordinationOnCluster::getOnClusterInitializationKeeperRetriesInfo() const
diff --git a/src/Backups/BackupCoordinationOnCluster.h b/src/Backups/BackupCoordinationOnCluster.h
index 7369c2cc746..b439ab619d8 100644
--- a/src/Backups/BackupCoordinationOnCluster.h
+++ b/src/Backups/BackupCoordinationOnCluster.h
@@ -1,7 +1,6 @@
#pragma once
#include
-#include
#include
#include
#include
@@ -20,7 +19,7 @@ class BackupCoordinationOnCluster : public IBackupCoordination
{
public:
/// Empty string as the current host is used to mark the initiator of a BACKUP ON CLUSTER query.
- static const constexpr std::string_view kInitiator;
+ static const constexpr std::string_view kInitiator = BackupCoordinationStageSync::kInitiator;
BackupCoordinationOnCluster(
const UUID & backup_uuid_,
@@ -37,13 +36,13 @@ public:
~BackupCoordinationOnCluster() override;
+ void setBackupQueryIsSentToOtherHosts() override;
+ bool isBackupQuerySentToOtherHosts() const override;
Strings setStage(const String & new_stage, const String & message, bool sync) override;
- void setBackupQueryWasSentToOtherHosts() override;
- bool trySetError(std::exception_ptr exception) override;
- void finish() override;
- bool tryFinishAfterError() noexcept override;
- void waitForOtherHostsToFinish() override;
- bool tryWaitForOtherHostsToFinishAfterError() noexcept override;
+ bool setError(std::exception_ptr exception, bool throw_if_error) override;
+ bool waitOtherHostsFinish(bool throw_if_error) const override;
+ bool finish(bool throw_if_error) override;
+ bool cleanup(bool throw_if_error) override;
void addReplicatedPartNames(
const String & table_zk_path,
@@ -110,11 +109,10 @@ private:
const bool plain_backup;
LoggerPtr const log;
+ /// The order is important: `stage_sync` must be initialized after `with_retries` and `cleaner`.
const WithRetries with_retries;
- BackupConcurrencyCheck concurrency_check;
- BackupCoordinationStageSync stage_sync;
BackupCoordinationCleaner cleaner;
- std::atomic backup_query_was_sent_to_other_hosts = false;
+ BackupCoordinationStageSync stage_sync;
mutable std::optional replicated_tables TSA_GUARDED_BY(replicated_tables_mutex);
mutable std::optional replicated_access TSA_GUARDED_BY(replicated_access_mutex);
diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp
index 9a05f9490c2..fcf09d7c315 100644
--- a/src/Backups/BackupCoordinationStageSync.cpp
+++ b/src/Backups/BackupCoordinationStageSync.cpp
@@ -42,9 +42,6 @@ namespace
kCurrentVersion = 2,
};
-
- /// Empty string as the current host is used to mark the initiator of a BACKUP ON CLUSTER or RESTORE ON CLUSTER query.
- const constexpr std::string_view kInitiator;
}
bool BackupCoordinationStageSync::HostInfo::operator ==(const HostInfo & other) const
@@ -63,12 +60,32 @@ bool BackupCoordinationStageSync::State::operator ==(const State & other) const
bool BackupCoordinationStageSync::State::operator !=(const State & other) const = default;
+void BackupCoordinationStageSync::State::merge(const State & other)
+{
+ if (other.host_with_error && !host_with_error)
+ {
+ const String & host = *other.host_with_error;
+ host_with_error = host;
+ hosts.at(host).exception = other.hosts.at(host).exception;
+ }
+
+ for (const auto & [host, other_host_info] : other.hosts)
+ {
+ auto & host_info = hosts.at(host);
+ host_info.stages.insert(other_host_info.stages.begin(), other_host_info.stages.end());
+ if (other_host_info.finished)
+ host_info.finished = true;
+ }
+}
+
+
BackupCoordinationStageSync::BackupCoordinationStageSync(
bool is_restore_,
const String & zookeeper_path_,
const String & current_host_,
const Strings & all_hosts_,
bool allow_concurrency_,
+ BackupConcurrencyCounters & concurrency_counters_,
const WithRetries & with_retries_,
ThreadPoolCallbackRunnerUnsafe schedule_,
QueryStatusPtr process_list_element_,
@@ -89,35 +106,29 @@ BackupCoordinationStageSync::BackupCoordinationStageSync(
, max_attempts_after_bad_version(with_retries.getKeeperSettings().max_attempts_after_bad_version)
, zookeeper_path(zookeeper_path_)
, root_zookeeper_path(zookeeper_path.parent_path().parent_path())
- , operation_node_path(zookeeper_path.parent_path())
+ , operation_zookeeper_path(zookeeper_path.parent_path())
, operation_node_name(zookeeper_path.parent_path().filename())
- , stage_node_path(zookeeper_path)
, start_node_path(zookeeper_path / ("started|" + current_host))
, finish_node_path(zookeeper_path / ("finished|" + current_host))
, num_hosts_node_path(zookeeper_path / "num_hosts")
+ , error_node_path(zookeeper_path / "error")
, alive_node_path(zookeeper_path / ("alive|" + current_host))
, alive_tracker_node_path(fs::path{root_zookeeper_path} / "alive_tracker")
- , error_node_path(zookeeper_path / "error")
, zk_nodes_changed(std::make_shared())
{
- if ((zookeeper_path.filename() != "stage") || !operation_node_name.starts_with(is_restore ? "restore-" : "backup-")
- || (root_zookeeper_path == operation_node_path))
- {
- throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected path in ZooKeeper specified: {}", zookeeper_path);
- }
-
initializeState();
createRootNodes();
try
{
+ concurrency_check.emplace(is_restore, /* on_cluster = */ true, zookeeper_path, allow_concurrency, concurrency_counters_);
createStartAndAliveNodes();
startWatchingThread();
}
catch (...)
{
- trySetError(std::current_exception());
- tryFinishImpl();
+ if (setError(std::current_exception(), /* throw_if_error = */ false))
+ finish(/* throw_if_error = */ false);
throw;
}
}
@@ -125,7 +136,26 @@ BackupCoordinationStageSync::BackupCoordinationStageSync(
BackupCoordinationStageSync::~BackupCoordinationStageSync()
{
- tryFinishImpl();
+ /// Normally either finish() or setError() must be called.
+ if (!tried_to_finish)
+ {
+ if (state.host_with_error)
+ {
+ /// setError() was called and succeeded.
+ finish(/* throw_if_error = */ false);
+ }
+ else if (!tried_to_set_error)
+ {
+ /// Neither finish() nor setError() were called, it's a bug.
+ chassert(false, "~BackupCoordinationStageSync() is called without finish() or setError()");
+ LOG_ERROR(log, "~BackupCoordinationStageSync() is called without finish() or setError()");
+ }
+ }
+
+ /// Normally the watching thread should be stopped already because the finish() function stops it.
+ /// However if an error happened then the watching thread can be still running,
+ /// so here in the destructor we have to ensure that it's stopped.
+ stopWatchingThread();
}
@@ -137,6 +167,12 @@ void BackupCoordinationStageSync::initializeState()
for (const String & host : all_hosts)
state.hosts.emplace(host, HostInfo{.host = host, .last_connection_time = now, .last_connection_time_monotonic = monotonic_now});
+
+ if (!state.hosts.contains(current_host))
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "List of hosts must contain the current host");
+
+ if (!state.hosts.contains(String{kInitiator}))
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "List of hosts must contain the initiator");
}
@@ -179,6 +215,12 @@ String BackupCoordinationStageSync::getHostsDesc(const Strings & hosts)
void BackupCoordinationStageSync::createRootNodes()
{
+ if ((zookeeper_path.filename() != "stage") || !operation_node_name.starts_with(is_restore ? "restore-" : "backup-")
+ || (root_zookeeper_path == operation_zookeeper_path))
+ {
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected path in ZooKeeper specified: {}", zookeeper_path);
+ }
+
auto holder = with_retries.createRetriesControlHolder("BackupStageSync::createRootNodes", WithRetries::kInitialization);
holder.retries_ctl.retryLoop(
[&, &zookeeper = holder.faulty_zookeeper]()
@@ -252,27 +294,27 @@ void BackupCoordinationStageSync::createStartAndAliveNodes(Coordination::ZooKeep
Coordination::Requests requests;
requests.reserve(6);
- size_t operation_node_path_pos = static_cast(-1);
- if (!zookeeper->exists(operation_node_path))
+ size_t operation_node_pos = static_cast(-1);
+ if (!zookeeper->exists(operation_zookeeper_path))
{
- operation_node_path_pos = requests.size();
- requests.emplace_back(zkutil::makeCreateRequest(operation_node_path, "", zkutil::CreateMode::Persistent));
+ operation_node_pos = requests.size();
+ requests.emplace_back(zkutil::makeCreateRequest(operation_zookeeper_path, "", zkutil::CreateMode::Persistent));
}
- size_t stage_node_path_pos = static_cast(-1);
- if (!zookeeper->exists(stage_node_path))
+ size_t zookeeper_path_pos = static_cast(-1);
+ if (!zookeeper->exists(zookeeper_path))
{
- stage_node_path_pos = requests.size();
- requests.emplace_back(zkutil::makeCreateRequest(stage_node_path, "", zkutil::CreateMode::Persistent));
+ zookeeper_path_pos = requests.size();
+ requests.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent));
}
- size_t num_hosts_node_path_pos = requests.size();
+ size_t num_hosts_node_pos = requests.size();
if (num_hosts)
requests.emplace_back(zkutil::makeSetRequest(num_hosts_node_path, toString(*num_hosts + 1), num_hosts_version));
else
requests.emplace_back(zkutil::makeCreateRequest(num_hosts_node_path, "1", zkutil::CreateMode::Persistent));
- size_t alive_tracker_node_path_pos = requests.size();
+ size_t alive_tracker_node_pos = requests.size();
requests.emplace_back(zkutil::makeSetRequest(alive_tracker_node_path, "", alive_tracker_version));
requests.emplace_back(zkutil::makeCreateRequest(start_node_path, std::to_string(kCurrentVersion), zkutil::CreateMode::Persistent));
@@ -284,7 +326,7 @@ void BackupCoordinationStageSync::createStartAndAliveNodes(Coordination::ZooKeep
if (code == Coordination::Error::ZOK)
{
LOG_INFO(log, "Created start node #{} in ZooKeeper for {} (coordination version: {})",
- num_hosts.value_or(0) + 1, current_host_desc, kCurrentVersion);
+ num_hosts.value_or(0) + 1, current_host_desc, static_cast(kCurrentVersion));
return;
}
@@ -294,40 +336,34 @@ void BackupCoordinationStageSync::createStartAndAliveNodes(Coordination::ZooKeep
LOG_TRACE(log, "{} (attempt #{}){}", message, attempt_no, will_try_again ? ", will try again" : "");
};
- if ((responses.size() > operation_node_path_pos) &&
- (responses[operation_node_path_pos]->error == Coordination::Error::ZNODEEXISTS))
+ if ((operation_node_pos < responses.size()) &&
+ (responses[operation_node_pos]->error == Coordination::Error::ZNODEEXISTS))
{
- show_error_before_next_attempt(fmt::format("Node {} in ZooKeeper already exists", operation_node_path));
+ show_error_before_next_attempt(fmt::format("Node {} already exists", operation_zookeeper_path));
/// needs another attempt
}
- else if ((responses.size() > stage_node_path_pos) &&
- (responses[stage_node_path_pos]->error == Coordination::Error::ZNODEEXISTS))
+ else if ((zookeeper_path_pos < responses.size()) &&
+ (responses[zookeeper_path_pos]->error == Coordination::Error::ZNODEEXISTS))
{
- show_error_before_next_attempt(fmt::format("Node {} in ZooKeeper already exists", stage_node_path));
+ show_error_before_next_attempt(fmt::format("Node {} already exists", zookeeper_path));
/// needs another attempt
}
- else if ((responses.size() > num_hosts_node_path_pos) && num_hosts &&
- (responses[num_hosts_node_path_pos]->error == Coordination::Error::ZBADVERSION))
+ else if ((num_hosts_node_pos < responses.size()) && !num_hosts &&
+ (responses[num_hosts_node_pos]->error == Coordination::Error::ZNODEEXISTS))
{
- show_error_before_next_attempt("Other host changed the 'num_hosts' node in ZooKeeper");
+ show_error_before_next_attempt(fmt::format("Node {} already exists", num_hosts_node_path));
+ /// needs another attempt
+ }
+ else if ((num_hosts_node_pos < responses.size()) && num_hosts &&
+ (responses[num_hosts_node_pos]->error == Coordination::Error::ZBADVERSION))
+ {
+ show_error_before_next_attempt(fmt::format("The version of node {} changed", num_hosts_node_path));
num_hosts.reset(); /// needs to reread 'num_hosts' again
}
- else if ((responses.size() > num_hosts_node_path_pos) && num_hosts &&
- (responses[num_hosts_node_path_pos]->error == Coordination::Error::ZNONODE))
+ else if ((alive_tracker_node_pos < responses.size()) &&
+ (responses[alive_tracker_node_pos]->error == Coordination::Error::ZBADVERSION))
{
- show_error_before_next_attempt("Other host removed the 'num_hosts' node in ZooKeeper");
- num_hosts.reset(); /// needs to reread 'num_hosts' again
- }
- else if ((responses.size() > num_hosts_node_path_pos) && !num_hosts &&
- (responses[num_hosts_node_path_pos]->error == Coordination::Error::ZNODEEXISTS))
- {
- show_error_before_next_attempt("Other host created the 'num_hosts' node in ZooKeeper");
- /// needs another attempt
- }
- else if ((responses.size() > alive_tracker_node_path_pos) &&
- (responses[alive_tracker_node_path_pos]->error == Coordination::Error::ZBADVERSION))
- {
- show_error_before_next_attempt("Concurrent backup or restore changed some 'alive' nodes in ZooKeeper");
+ show_error_before_next_attempt(fmt::format("The version of node {} changed", alive_tracker_node_path));
check_concurrency = true; /// needs to recheck for concurrency again
}
else
@@ -337,8 +373,7 @@ void BackupCoordinationStageSync::createStartAndAliveNodes(Coordination::ZooKeep
}
throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE,
- "Couldn't create the 'start' node in ZooKeeper for {} after {} attempts",
- current_host_desc, max_attempts_after_bad_version);
+ "Couldn't create node {} in ZooKeeper after {} attempts", start_node_path, max_attempts_after_bad_version);
}
@@ -387,36 +422,53 @@ void BackupCoordinationStageSync::startWatchingThread()
void BackupCoordinationStageSync::stopWatchingThread()
{
- should_stop_watching_thread = true;
+ {
+ std::lock_guard lock{mutex};
+ if (should_stop_watching_thread)
+ return;
+ should_stop_watching_thread = true;
- /// Wake up waiting threads.
- if (zk_nodes_changed)
- zk_nodes_changed->set();
- state_changed.notify_all();
+ /// Wake up waiting threads.
+ if (zk_nodes_changed)
+ zk_nodes_changed->set();
+ state_changed.notify_all();
+ }
if (watching_thread_future.valid())
watching_thread_future.wait();
+
+ LOG_TRACE(log, "Stopped the watching thread");
}
void BackupCoordinationStageSync::watchingThread()
{
- while (!should_stop_watching_thread)
+ auto should_stop = [&]
+ {
+ std::lock_guard lock{mutex};
+ return should_stop_watching_thread;
+ };
+
+ while (!should_stop())
{
try
{
/// Check if the current BACKUP or RESTORE command is already cancelled.
checkIfQueryCancelled();
+ }
+ catch (...)
+ {
+ tryLogCurrentException(log, "Caugth exception while watching");
+ }
- /// Reset the `connected` flag for each host, we'll set them to true again after we find the 'alive' nodes.
- resetConnectedFlag();
-
+ try
+ {
/// Recreate the 'alive' node if necessary and read a new state from ZooKeeper.
auto holder = with_retries.createRetriesControlHolder("BackupStageSync::watchingThread");
auto & zookeeper = holder.faulty_zookeeper;
with_retries.renewZooKeeper(zookeeper);
- if (should_stop_watching_thread)
+ if (should_stop())
return;
/// Recreate the 'alive' node if it was removed.
@@ -427,7 +479,10 @@ void BackupCoordinationStageSync::watchingThread()
}
catch (...)
{
- tryLogCurrentException(log, "Caugth exception while watching");
+ tryLogCurrentException(log, "Caught exception while watching");
+
+ /// Reset the `connected` flag for each host, we'll set them to true again after we find the 'alive' nodes.
+ resetConnectedFlag();
}
try
@@ -438,7 +493,7 @@ void BackupCoordinationStageSync::watchingThread()
}
catch (...)
{
- tryLogCurrentException(log, "Caugth exception while checking if the query should be cancelled");
+ tryLogCurrentException(log, "Caught exception while watching");
}
zk_nodes_changed->tryWait(sync_period_ms.count());
@@ -473,7 +528,7 @@ void BackupCoordinationStageSync::readCurrentState(Coordination::ZooKeeperWithFa
zk_nodes_changed->reset();
/// Get zk nodes and subscribe on their changes.
- Strings new_zk_nodes = zookeeper->getChildren(stage_node_path, nullptr, zk_nodes_changed);
+ Strings new_zk_nodes = zookeeper->getChildren(zookeeper_path, nullptr, zk_nodes_changed);
std::sort(new_zk_nodes.begin(), new_zk_nodes.end()); /// Sorting is necessary because we compare the list of zk nodes with its previous versions.
State new_state;
@@ -492,6 +547,8 @@ void BackupCoordinationStageSync::readCurrentState(Coordination::ZooKeeperWithFa
zk_nodes = new_zk_nodes;
new_state = state;
+ for (auto & [_, host_info] : new_state.hosts)
+ host_info.connected = false;
}
auto get_host_info = [&](const String & host) -> HostInfo *
@@ -514,7 +571,8 @@ void BackupCoordinationStageSync::readCurrentState(Coordination::ZooKeeperWithFa
{
String serialized_error = zookeeper->get(error_node_path);
auto [exception, host] = parseErrorNode(serialized_error);
- if (auto * host_info = get_host_info(host))
+ auto * host_info = get_host_info(host);
+ if (exception && host_info)
{
host_info->exception = exception;
new_state.host_with_error = host;
@@ -576,6 +634,9 @@ void BackupCoordinationStageSync::readCurrentState(Coordination::ZooKeeperWithFa
{
std::lock_guard lock{mutex};
+ /// We were reading `new_state` from ZooKeeper with `mutex` unlocked, so `state` could get more information during that reading,
+ /// we don't want to lose that information, that's why we use merge() here.
+ new_state.merge(state);
was_state_changed = (new_state != state);
state = std::move(new_state);
}
@@ -604,26 +665,10 @@ int BackupCoordinationStageSync::parseStartNode(const String & start_node_conten
}
-std::pair BackupCoordinationStageSync::parseErrorNode(const String & error_node_contents)
-{
- ReadBufferFromOwnString buf{error_node_contents};
- String host;
- readStringBinary(host, buf);
- auto exception = std::make_exception_ptr(readException(buf, fmt::format("Got error from {}", getHostDesc(host))));
- return {exception, host};
-}
-
-
void BackupCoordinationStageSync::checkIfQueryCancelled()
{
if (process_list_element->checkTimeLimitSoft())
return; /// Not cancelled.
-
- std::lock_guard lock{mutex};
- if (state.cancelled)
- return; /// Already marked as cancelled.
-
- state.cancelled = true;
state_changed.notify_all();
}
@@ -634,13 +679,13 @@ void BackupCoordinationStageSync::cancelQueryIfError()
{
std::lock_guard lock{mutex};
- if (state.cancelled || !state.host_with_error)
+ if (!state.host_with_error)
return;
- state.cancelled = true;
exception = state.hosts.at(*state.host_with_error).exception;
}
+ chassert(exception);
process_list_element->cancelQuery(false, exception);
state_changed.notify_all();
}
@@ -652,7 +697,7 @@ void BackupCoordinationStageSync::cancelQueryIfDisconnectedTooLong()
{
std::lock_guard lock{mutex};
- if (state.cancelled || state.host_with_error || ((failure_after_host_disconnected_for_seconds.count() == 0)))
+ if (state.host_with_error || ((failure_after_host_disconnected_for_seconds.count() == 0)))
return;
auto monotonic_now = std::chrono::steady_clock::now();
@@ -685,27 +730,92 @@ void BackupCoordinationStageSync::cancelQueryIfDisconnectedTooLong()
}
}
}
-
- if (!exception)
- return;
-
- state.cancelled = true;
}
+ if (!exception)
+ return;
+
process_list_element->cancelQuery(false, exception);
state_changed.notify_all();
}
+void BackupCoordinationStageSync::setQueryIsSentToOtherHosts()
+{
+ std::lock_guard lock{mutex};
+ query_is_sent_to_other_hosts = true;
+}
+
+bool BackupCoordinationStageSync::isQuerySentToOtherHosts() const
+{
+ std::lock_guard lock{mutex};
+ return query_is_sent_to_other_hosts;
+}
+
+
void BackupCoordinationStageSync::setStage(const String & stage, const String & stage_result)
{
LOG_INFO(log, "{} reached stage {}", current_host_desc, stage);
+
+ {
+ std::lock_guard lock{mutex};
+ if (state.hosts.at(current_host).stages.contains(stage))
+ return; /// Already set.
+ }
+
+ if ((getInitiatorVersion() == kVersionWithoutFinishNode) && (stage == BackupCoordinationStage::COMPLETED))
+ {
+ LOG_TRACE(log, "Stopping the watching thread because the initiator uses outdated version {}", getInitiatorVersion());
+ stopWatchingThread();
+ }
+
auto holder = with_retries.createRetriesControlHolder("BackupStageSync::setStage");
holder.retries_ctl.retryLoop([&, &zookeeper = holder.faulty_zookeeper]()
{
with_retries.renewZooKeeper(zookeeper);
- zookeeper->createIfNotExists(getStageNodePath(stage), stage_result);
+ createStageNode(stage, stage_result, zookeeper);
});
+
+ /// If the initiator of the query has that old version then it doesn't expect us to create the 'finish' node and moreover
+ /// the initiator can start removing all the nodes immediately after all hosts report about reaching the "completed" status.
+ /// So to avoid weird errors in the logs we won't create the 'finish' node if the initiator of the query has that old version.
+ if ((getInitiatorVersion() == kVersionWithoutFinishNode) && (stage == BackupCoordinationStage::COMPLETED))
+ {
+ LOG_INFO(log, "Skipped creating the 'finish' node because the initiator uses outdated version {}", getInitiatorVersion());
+ std::lock_guard lock{mutex};
+ tried_to_finish = true;
+ state.hosts.at(current_host).finished = true;
+ }
+}
+
+
+void BackupCoordinationStageSync::createStageNode(const String & stage, const String & stage_result, Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper)
+{
+ String serialized_error;
+ if (zookeeper->tryGet(error_node_path, serialized_error))
+ {
+ auto [exception, host] = parseErrorNode(serialized_error);
+ if (exception)
+ std::rethrow_exception(exception);
+ }
+
+ auto code = zookeeper->tryCreate(getStageNodePath(stage), stage_result, zkutil::CreateMode::Persistent);
+ if (code == Coordination::Error::ZOK)
+ {
+ std::lock_guard lock{mutex};
+ state.hosts.at(current_host).stages[stage] = stage_result;
+ return;
+ }
+
+ if (code == Coordination::Error::ZNODEEXISTS)
+ {
+ String another_result = zookeeper->get(getStageNodePath(stage));
+ std::lock_guard lock{mutex};
+ state.hosts.at(current_host).stages[stage] = another_result;
+ return;
+ }
+
+ throw zkutil::KeeperException::fromPath(code, getStageNodePath(stage));
}
@@ -715,71 +825,7 @@ String BackupCoordinationStageSync::getStageNodePath(const String & stage) const
}
-bool BackupCoordinationStageSync::trySetError(std::exception_ptr exception) noexcept
-{
- try
- {
- std::rethrow_exception(exception);
- }
- catch (const Exception & e)
- {
- return trySetError(e);
- }
- catch (...)
- {
- return trySetError(Exception(getCurrentExceptionMessageAndPattern(true, true), getCurrentExceptionCode()));
- }
-}
-
-
-bool BackupCoordinationStageSync::trySetError(const Exception & exception)
-{
- try
- {
- setError(exception);
- return true;
- }
- catch (...)
- {
- return false;
- }
-}
-
-
-void BackupCoordinationStageSync::setError(const Exception & exception)
-{
- /// Most likely this exception has been already logged so here we're logging it without stacktrace.
- String exception_message = getExceptionMessage(exception, /* with_stacktrace= */ false, /* check_embedded_stacktrace= */ true);
- LOG_INFO(log, "Sending exception from {} to other hosts: {}", current_host_desc, exception_message);
-
- auto holder = with_retries.createRetriesControlHolder("BackupStageSync::setError", WithRetries::kErrorHandling);
-
- holder.retries_ctl.retryLoop([&, &zookeeper = holder.faulty_zookeeper]()
- {
- with_retries.renewZooKeeper(zookeeper);
-
- WriteBufferFromOwnString buf;
- writeStringBinary(current_host, buf);
- writeException(exception, buf, true);
- auto code = zookeeper->tryCreate(error_node_path, buf.str(), zkutil::CreateMode::Persistent);
-
- if (code == Coordination::Error::ZOK)
- {
- LOG_TRACE(log, "Sent exception from {} to other hosts", current_host_desc);
- }
- else if (code == Coordination::Error::ZNODEEXISTS)
- {
- LOG_INFO(log, "An error has been already assigned for this {}", operation_name);
- }
- else
- {
- throw zkutil::KeeperException::fromPath(code, error_node_path);
- }
- });
-}
-
-
-Strings BackupCoordinationStageSync::waitForHostsToReachStage(const String & stage_to_wait, const Strings & hosts, std::optional timeout) const
+Strings BackupCoordinationStageSync::waitHostsReachStage(const Strings & hosts, const String & stage_to_wait) const
{
Strings results;
results.resize(hosts.size());
@@ -787,44 +833,28 @@ Strings BackupCoordinationStageSync::waitForHostsToReachStage(const String & sta
std::unique_lock lock{mutex};
/// TSA_NO_THREAD_SAFETY_ANALYSIS is here because Clang Thread Safety Analysis doesn't understand std::unique_lock.
- auto check_if_hosts_ready = [&](bool time_is_out) TSA_NO_THREAD_SAFETY_ANALYSIS
+ auto check_if_hosts_reach_stage = [&]() TSA_NO_THREAD_SAFETY_ANALYSIS
{
- return checkIfHostsReachStage(hosts, stage_to_wait, time_is_out, timeout, results);
+ return checkIfHostsReachStage(hosts, stage_to_wait, results);
};
- if (timeout)
- {
- if (!state_changed.wait_for(lock, *timeout, [&] { return check_if_hosts_ready(/* time_is_out = */ false); }))
- check_if_hosts_ready(/* time_is_out = */ true);
- }
- else
- {
- state_changed.wait(lock, [&] { return check_if_hosts_ready(/* time_is_out = */ false); });
- }
+ state_changed.wait(lock, check_if_hosts_reach_stage);
return results;
}
-bool BackupCoordinationStageSync::checkIfHostsReachStage(
- const Strings & hosts,
- const String & stage_to_wait,
- bool time_is_out,
- std::optional timeout,
- Strings & results) const
+bool BackupCoordinationStageSync::checkIfHostsReachStage(const Strings & hosts, const String & stage_to_wait, Strings & results) const
{
- if (should_stop_watching_thread)
- throw Exception(ErrorCodes::LOGICAL_ERROR, "finish() was called while waiting for a stage");
-
process_list_element->checkTimeLimit();
for (size_t i = 0; i != hosts.size(); ++i)
{
const String & host = hosts[i];
auto it = state.hosts.find(host);
-
if (it == state.hosts.end())
- throw Exception(ErrorCodes::LOGICAL_ERROR, "waitForHostsToReachStage() was called for unexpected {}, all hosts are {}", getHostDesc(host), getHostsDesc(all_hosts));
+ throw Exception(ErrorCodes::LOGICAL_ERROR,
+ "waitHostsReachStage() was called for unexpected {}, all hosts are {}", getHostDesc(host), getHostsDesc(all_hosts));
const HostInfo & host_info = it->second;
auto stage_it = host_info.stages.find(stage_to_wait);
@@ -835,10 +865,11 @@ bool BackupCoordinationStageSync::checkIfHostsReachStage(
}
if (host_info.finished)
- {
throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE,
"{} finished without coming to stage {}", getHostDesc(host), stage_to_wait);
- }
+
+ if (should_stop_watching_thread)
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "waitHostsReachStage() can't wait for stage {} after the watching thread stopped", stage_to_wait);
String host_status;
if (!host_info.started)
@@ -846,85 +877,73 @@ bool BackupCoordinationStageSync::checkIfHostsReachStage(
else if (!host_info.connected)
host_status = fmt::format(": the host is currently disconnected, last connection was at {}", host_info.last_connection_time);
- if (!time_is_out)
- {
- LOG_TRACE(log, "Waiting for {} to reach stage {}{}", getHostDesc(host), stage_to_wait, host_status);
- return false;
- }
- else
- {
- throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE,
- "Waited longer than timeout {} for {} to reach stage {}{}",
- *timeout, getHostDesc(host), stage_to_wait, host_status);
- }
+ LOG_TRACE(log, "Waiting for {} to reach stage {}{}", getHostDesc(host), stage_to_wait, host_status);
+ return false; /// wait for next change of `state_changed`
}
LOG_INFO(log, "Hosts {} reached stage {}", getHostsDesc(hosts), stage_to_wait);
- return true;
+ return true; /// stop waiting
}
-void BackupCoordinationStageSync::finish(bool & other_hosts_also_finished)
+bool BackupCoordinationStageSync::finish(bool throw_if_error)
{
- tryFinishImpl(other_hosts_also_finished, /* throw_if_error = */ true, /* retries_kind = */ WithRetries::kNormal);
+ WithRetries::Kind retries_kind = WithRetries::kNormal;
+ if (throw_if_error)
+ retries_kind = WithRetries::kErrorHandling;
+
+ return finishImpl(throw_if_error, retries_kind);
}
-bool BackupCoordinationStageSync::tryFinishAfterError(bool & other_hosts_also_finished) noexcept
+bool BackupCoordinationStageSync::finishImpl(bool throw_if_error, WithRetries::Kind retries_kind)
{
- return tryFinishImpl(other_hosts_also_finished, /* throw_if_error = */ false, /* retries_kind = */ WithRetries::kErrorHandling);
-}
-
-
-bool BackupCoordinationStageSync::tryFinishImpl()
-{
- bool other_hosts_also_finished;
- return tryFinishAfterError(other_hosts_also_finished);
-}
-
-
-bool BackupCoordinationStageSync::tryFinishImpl(bool & other_hosts_also_finished, bool throw_if_error, WithRetries::Kind retries_kind)
-{
- auto get_value_other_hosts_also_finished = [&] TSA_REQUIRES(mutex)
- {
- other_hosts_also_finished = true;
- for (const auto & [host, host_info] : state.hosts)
- {
- if ((host != current_host) && !host_info.finished)
- other_hosts_also_finished = false;
- }
- };
-
{
std::lock_guard lock{mutex};
- if (finish_result.succeeded)
+
+ if (finishedNoLock())
{
- get_value_other_hosts_also_finished();
+ LOG_INFO(log, "The finish node for {} already exists", current_host_desc);
return true;
}
- if (finish_result.exception)
+
+ if (tried_to_finish)
{
- if (throw_if_error)
- std::rethrow_exception(finish_result.exception);
+ /// We don't repeat creating the finish node, no matter if it was successful or not.
+ LOG_INFO(log, "Skipped creating the finish node for {} because earlier we failed to do that", current_host_desc);
return false;
}
+
+ bool failed_to_set_error = tried_to_set_error && !state.host_with_error;
+ if (failed_to_set_error)
+ {
+ /// Tried to create the 'error' node, but failed.
+ /// Then it's better not to create the 'finish' node in this case because otherwise other hosts might think we've succeeded.
+ LOG_INFO(log, "Skipping creating the finish node for {} because there was an error which we were unable to send to other hosts", current_host_desc);
+ return false;
+ }
+
+ if (current_host == kInitiator)
+ {
+ /// Normally the initiator should wait for other hosts to finish before creating its own finish node.
+ /// We show warning if some of the other hosts didn't finish.
+ bool expect_other_hosts_finished = query_is_sent_to_other_hosts || !state.host_with_error;
+ bool other_hosts_finished = otherHostsFinishedNoLock() || !expect_other_hosts_finished;
+ if (!other_hosts_finished)
+ LOG_WARNING(log, "Hosts {} didn't finish before the initiator", getHostsDesc(getUnfinishedOtherHostsNoLock()));
+ }
}
+ stopWatchingThread();
+
try
{
- stopWatchingThread();
-
auto holder = with_retries.createRetriesControlHolder("BackupStageSync::finish", retries_kind);
holder.retries_ctl.retryLoop([&, &zookeeper = holder.faulty_zookeeper]()
{
with_retries.renewZooKeeper(zookeeper);
- createFinishNodeAndRemoveAliveNode(zookeeper);
+ createFinishNodeAndRemoveAliveNode(zookeeper, throw_if_error);
});
-
- std::lock_guard lock{mutex};
- finish_result.succeeded = true;
- get_value_other_hosts_also_finished();
- return true;
}
catch (...)
{
@@ -933,63 +952,87 @@ bool BackupCoordinationStageSync::tryFinishImpl(bool & other_hosts_also_finished
getCurrentExceptionMessage(/* with_stacktrace= */ false, /* check_embedded_stacktrace= */ true));
std::lock_guard lock{mutex};
- finish_result.exception = std::current_exception();
+ tried_to_finish = true;
+
if (throw_if_error)
throw;
return false;
}
+
+ {
+ std::lock_guard lock{mutex};
+ tried_to_finish = true;
+ state.hosts.at(current_host).finished = true;
+ }
+
+ return true;
}
-void BackupCoordinationStageSync::createFinishNodeAndRemoveAliveNode(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper)
+void BackupCoordinationStageSync::createFinishNodeAndRemoveAliveNode(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper, bool throw_if_error)
{
- if (zookeeper->exists(finish_node_path))
- return;
-
- /// If the initiator of the query has that old version then it doesn't expect us to create the 'finish' node and moreover
- /// the initiator can start removing all the nodes immediately after all hosts report about reaching the "completed" status.
- /// So to avoid weird errors in the logs we won't create the 'finish' node if the initiator of the query has that old version.
- if ((getInitiatorVersion() == kVersionWithoutFinishNode) && (current_host != kInitiator))
- {
- LOG_INFO(log, "Skipped creating the 'finish' node because the initiator uses outdated version {}", getInitiatorVersion());
- return;
- }
-
std::optional num_hosts;
int num_hosts_version = -1;
for (size_t attempt_no = 1; attempt_no <= max_attempts_after_bad_version; ++attempt_no)
{
+ /// The 'num_hosts' node may not exist if createStartAndAliveNodes() failed in the constructor.
if (!num_hosts)
{
+ String num_hosts_str;
Coordination::Stat stat;
- num_hosts = parseFromString(zookeeper->get(num_hosts_node_path, &stat));
- num_hosts_version = stat.version;
+ if (zookeeper->tryGet(num_hosts_node_path, num_hosts_str, &stat))
+ {
+ num_hosts = parseFromString(num_hosts_str);
+ num_hosts_version = stat.version;
+ }
}
+ String serialized_error;
+ if (throw_if_error && zookeeper->tryGet(error_node_path, serialized_error))
+ {
+ auto [exception, host] = parseErrorNode(serialized_error);
+ if (exception)
+ std::rethrow_exception(exception);
+ }
+
+ if (zookeeper->exists(finish_node_path))
+ return;
+
+ bool start_node_exists = zookeeper->exists(start_node_path);
+
Coordination::Requests requests;
requests.reserve(3);
requests.emplace_back(zkutil::makeCreateRequest(finish_node_path, "", zkutil::CreateMode::Persistent));
- size_t num_hosts_node_path_pos = requests.size();
- requests.emplace_back(zkutil::makeSetRequest(num_hosts_node_path, toString(*num_hosts - 1), num_hosts_version));
-
- size_t alive_node_path_pos = static_cast(-1);
+ size_t alive_node_pos = static_cast(-1);
if (zookeeper->exists(alive_node_path))
{
- alive_node_path_pos = requests.size();
+ alive_node_pos = requests.size();
requests.emplace_back(zkutil::makeRemoveRequest(alive_node_path, -1));
}
+ size_t num_hosts_node_pos = static_cast(-1);
+ if (num_hosts)
+ {
+ num_hosts_node_pos = requests.size();
+ requests.emplace_back(zkutil::makeSetRequest(num_hosts_node_path, toString(start_node_exists ? (*num_hosts - 1) : *num_hosts), num_hosts_version));
+ }
+
Coordination::Responses responses;
auto code = zookeeper->tryMulti(requests, responses);
if (code == Coordination::Error::ZOK)
{
- --*num_hosts;
- String hosts_left_desc = ((*num_hosts == 0) ? "no hosts left" : fmt::format("{} hosts left", *num_hosts));
- LOG_INFO(log, "Created the 'finish' node in ZooKeeper for {}, {}", current_host_desc, hosts_left_desc);
+ String hosts_left_desc;
+ if (num_hosts)
+ {
+ if (start_node_exists)
+ --*num_hosts;
+ hosts_left_desc = (*num_hosts == 0) ? ", no hosts left" : fmt::format(", {} hosts left", *num_hosts);
+ }
+ LOG_INFO(log, "Created the 'finish' node in ZooKeeper for {}{}", current_host_desc, hosts_left_desc);
return;
}
@@ -999,18 +1042,18 @@ void BackupCoordinationStageSync::createFinishNodeAndRemoveAliveNode(Coordinatio
LOG_TRACE(log, "{} (attempt #{}){}", message, attempt_no, will_try_again ? ", will try again" : "");
};
- if ((responses.size() > num_hosts_node_path_pos) &&
- (responses[num_hosts_node_path_pos]->error == Coordination::Error::ZBADVERSION))
+ if ((alive_node_pos < responses.size()) &&
+ (responses[alive_node_pos]->error == Coordination::Error::ZNONODE))
{
- show_error_before_next_attempt("Other host changed the 'num_hosts' node in ZooKeeper");
- num_hosts.reset(); /// needs to reread 'num_hosts' again
- }
- else if ((responses.size() > alive_node_path_pos) &&
- (responses[alive_node_path_pos]->error == Coordination::Error::ZNONODE))
- {
- show_error_before_next_attempt(fmt::format("Node {} in ZooKeeper doesn't exist", alive_node_path_pos));
+ show_error_before_next_attempt(fmt::format("Node {} doesn't exist", alive_node_path));
/// needs another attempt
}
+ else if ((num_hosts_node_pos < responses.size()) &&
+ (responses[num_hosts_node_pos]->error == Coordination::Error::ZBADVERSION))
+ {
+ show_error_before_next_attempt(fmt::format("The version of node {} changed", num_hosts_node_path));
+ num_hosts.reset(); /// needs to reread 'num_hosts' again
+ }
else
{
zkutil::KeeperMultiException::check(code, requests, responses);
@@ -1026,60 +1069,73 @@ void BackupCoordinationStageSync::createFinishNodeAndRemoveAliveNode(Coordinatio
int BackupCoordinationStageSync::getInitiatorVersion() const
{
std::lock_guard lock{mutex};
- auto it = state.hosts.find(String{kInitiator});
- if (it == state.hosts.end())
- throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no initiator of this {} query, it's a bug", operation_name);
- const HostInfo & host_info = it->second;
- return host_info.version;
+ return state.hosts.at(String{kInitiator}).version;
}
-void BackupCoordinationStageSync::waitForOtherHostsToFinish() const
-{
- tryWaitForOtherHostsToFinishImpl(/* reason = */ "", /* throw_if_error = */ true, /* timeout = */ {});
-}
-
-
-bool BackupCoordinationStageSync::tryWaitForOtherHostsToFinishAfterError() const noexcept
+bool BackupCoordinationStageSync::waitOtherHostsFinish(bool throw_if_error) const
{
std::optional timeout;
- if (finish_timeout_after_error.count() != 0)
- timeout = finish_timeout_after_error;
+ String reason;
- String reason = fmt::format("{} needs other hosts to finish before cleanup", current_host_desc);
- return tryWaitForOtherHostsToFinishImpl(reason, /* throw_if_error = */ false, timeout);
+ if (!throw_if_error)
+ {
+ if (finish_timeout_after_error.count() != 0)
+ timeout = finish_timeout_after_error;
+ reason = "after error before cleanup";
+ }
+
+ return waitOtherHostsFinishImpl(reason, timeout, throw_if_error);
}
-bool BackupCoordinationStageSync::tryWaitForOtherHostsToFinishImpl(const String & reason, bool throw_if_error, std::optional timeout) const
+bool BackupCoordinationStageSync::waitOtherHostsFinishImpl(const String & reason, std::optional timeout, bool throw_if_error) const
{
std::unique_lock lock{mutex};
/// TSA_NO_THREAD_SAFETY_ANALYSIS is here because Clang Thread Safety Analysis doesn't understand std::unique_lock.
- auto check_if_other_hosts_finish = [&](bool time_is_out) TSA_NO_THREAD_SAFETY_ANALYSIS
+ auto other_hosts_finished = [&]() TSA_NO_THREAD_SAFETY_ANALYSIS { return otherHostsFinishedNoLock(); };
+
+ if (other_hosts_finished())
{
- return checkIfOtherHostsFinish(reason, throw_if_error, time_is_out, timeout);
+ LOG_TRACE(log, "Other hosts have already finished");
+ return true;
+ }
+
+ bool failed_to_set_error = TSA_SUPPRESS_WARNING_FOR_READ(tried_to_set_error) && !TSA_SUPPRESS_WARNING_FOR_READ(state).host_with_error;
+ if (failed_to_set_error)
+ {
+ /// Tried to create the 'error' node, but failed.
+ /// Then it's better not to wait for other hosts to finish in this case because other hosts don't know they should finish.
+ LOG_INFO(log, "Skipping waiting for other hosts to finish because there was an error which we were unable to send to other hosts");
+ return false;
+ }
+
+ bool result = false;
+
+ /// TSA_NO_THREAD_SAFETY_ANALYSIS is here because Clang Thread Safety Analysis doesn't understand std::unique_lock.
+ auto check_if_hosts_finish = [&](bool time_is_out) TSA_NO_THREAD_SAFETY_ANALYSIS
+ {
+ return checkIfOtherHostsFinish(reason, timeout, time_is_out, result, throw_if_error);
};
if (timeout)
{
- if (state_changed.wait_for(lock, *timeout, [&] { return check_if_other_hosts_finish(/* time_is_out = */ false); }))
- return true;
- return check_if_other_hosts_finish(/* time_is_out = */ true);
+ if (!state_changed.wait_for(lock, *timeout, [&] { return check_if_hosts_finish(/* time_is_out = */ false); }))
+ check_if_hosts_finish(/* time_is_out = */ true);
}
else
{
- state_changed.wait(lock, [&] { return check_if_other_hosts_finish(/* time_is_out = */ false); });
- return true;
+ state_changed.wait(lock, [&] { return check_if_hosts_finish(/* time_is_out = */ false); });
}
+
+ return result;
}
-bool BackupCoordinationStageSync::checkIfOtherHostsFinish(const String & reason, bool throw_if_error, bool time_is_out, std::optional timeout) const
+bool BackupCoordinationStageSync::checkIfOtherHostsFinish(
+ const String & reason, std::optional timeout, bool time_is_out, bool & result, bool throw_if_error) const
{
- if (should_stop_watching_thread)
- throw Exception(ErrorCodes::LOGICAL_ERROR, "finish() was called while waiting for other hosts to finish");
-
if (throw_if_error)
process_list_element->checkTimeLimit();
@@ -1088,38 +1144,261 @@ bool BackupCoordinationStageSync::checkIfOtherHostsFinish(const String & reason,
if ((host == current_host) || host_info.finished)
continue;
+ String reason_text = reason.empty() ? "" : (" " + reason);
+
String host_status;
if (!host_info.started)
host_status = fmt::format(": the host hasn't started working on this {} yet", operation_name);
else if (!host_info.connected)
host_status = fmt::format(": the host is currently disconnected, last connection was at {}", host_info.last_connection_time);
- if (!time_is_out)
+ if (time_is_out)
{
- String reason_text = reason.empty() ? "" : (" because " + reason);
- LOG_TRACE(log, "Waiting for {} to finish{}{}", getHostDesc(host), reason_text, host_status);
- return false;
- }
- else
- {
- String reason_text = reason.empty() ? "" : fmt::format(" (reason of waiting: {})", reason);
- if (!throw_if_error)
- {
- LOG_INFO(log, "Waited longer than timeout {} for {} to finish{}{}",
- *timeout, getHostDesc(host), host_status, reason_text);
- return false;
- }
- else
+ if (throw_if_error)
{
throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE,
"Waited longer than timeout {} for {} to finish{}{}",
- *timeout, getHostDesc(host), host_status, reason_text);
+ *timeout, getHostDesc(host), reason_text, host_status);
}
+ LOG_INFO(log, "Waited longer than timeout {} for {} to finish{}{}",
+ *timeout, getHostDesc(host), reason_text, host_status);
+ result = false;
+ return true; /// stop waiting
}
+
+ if (should_stop_watching_thread)
+ {
+ LOG_ERROR(log, "waitOtherHostFinish({}) can't wait for other hosts to finish after the watching thread stopped", throw_if_error);
+ chassert(false, "waitOtherHostFinish() can't wait for other hosts to finish after the watching thread stopped");
+ if (throw_if_error)
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "waitOtherHostsFinish() can't wait for other hosts to finish after the watching thread stopped");
+ result = false;
+ return true; /// stop waiting
+ }
+
+ LOG_TRACE(log, "Waiting for {} to finish{}{}", getHostDesc(host), reason_text, host_status);
+ return false; /// wait for next change of `state_changed`
}
LOG_TRACE(log, "Other hosts finished working on this {}", operation_name);
+ result = true;
+ return true; /// stop waiting
+}
+
+
+bool BackupCoordinationStageSync::finished() const
+{
+ std::lock_guard lock{mutex};
+ return finishedNoLock();
+}
+
+
+bool BackupCoordinationStageSync::finishedNoLock() const
+{
+ return state.hosts.at(current_host).finished;
+}
+
+
+bool BackupCoordinationStageSync::otherHostsFinished() const
+{
+ std::lock_guard lock{mutex};
+ return otherHostsFinishedNoLock();
+}
+
+
+bool BackupCoordinationStageSync::otherHostsFinishedNoLock() const
+{
+ for (const auto & [host, host_info] : state.hosts)
+ {
+ if (!host_info.finished && (host != current_host))
+ return false;
+ }
return true;
}
+
+bool BackupCoordinationStageSync::allHostsFinishedNoLock() const
+{
+ return finishedNoLock() && otherHostsFinishedNoLock();
+}
+
+
+Strings BackupCoordinationStageSync::getUnfinishedHosts() const
+{
+ std::lock_guard lock{mutex};
+ return getUnfinishedHostsNoLock();
+}
+
+
+Strings BackupCoordinationStageSync::getUnfinishedHostsNoLock() const
+{
+ if (allHostsFinishedNoLock())
+ return {};
+
+ Strings res;
+ res.reserve(all_hosts.size());
+ for (const auto & [host, host_info] : state.hosts)
+ {
+ if (!host_info.finished)
+ res.emplace_back(host);
+ }
+ return res;
+}
+
+
+Strings BackupCoordinationStageSync::getUnfinishedOtherHosts() const
+{
+ std::lock_guard lock{mutex};
+ return getUnfinishedOtherHostsNoLock();
+}
+
+
+Strings BackupCoordinationStageSync::getUnfinishedOtherHostsNoLock() const
+{
+ if (otherHostsFinishedNoLock())
+ return {};
+
+ Strings res;
+ res.reserve(all_hosts.size() - 1);
+ for (const auto & [host, host_info] : state.hosts)
+ {
+ if (!host_info.finished && (host != current_host))
+ res.emplace_back(host);
+ }
+ return res;
+}
+
+
+bool BackupCoordinationStageSync::setError(std::exception_ptr exception, bool throw_if_error)
+{
+ try
+ {
+ std::rethrow_exception(exception);
+ }
+ catch (const Exception & e)
+ {
+ return setError(e, throw_if_error);
+ }
+ catch (...)
+ {
+ return setError(Exception{getCurrentExceptionMessageAndPattern(true, true), getCurrentExceptionCode()}, throw_if_error);
+ }
+}
+
+
+bool BackupCoordinationStageSync::setError(const Exception & exception, bool throw_if_error)
+{
+ try
+ {
+ /// Most likely this exception has been already logged so here we're logging it without stacktrace.
+ String exception_message = getExceptionMessage(exception, /* with_stacktrace= */ false, /* check_embedded_stacktrace= */ true);
+ LOG_INFO(log, "Sending exception from {} to other hosts: {}", current_host_desc, exception_message);
+
+ {
+ std::lock_guard lock{mutex};
+ if (state.host_with_error)
+ {
+ LOG_INFO(log, "The error node already exists");
+ return true;
+ }
+
+ if (tried_to_set_error)
+ {
+ LOG_INFO(log, "Skipped creating the error node because earlier we failed to do that");
+ return false;
+ }
+ }
+
+ auto holder = with_retries.createRetriesControlHolder("BackupStageSync::setError", WithRetries::kErrorHandling);
+ holder.retries_ctl.retryLoop([&, &zookeeper = holder.faulty_zookeeper]()
+ {
+ with_retries.renewZooKeeper(zookeeper);
+ createErrorNode(exception, zookeeper);
+ });
+
+ {
+ std::lock_guard lock{mutex};
+ tried_to_set_error = true;
+ return true;
+ }
+ }
+ catch (...)
+ {
+ LOG_TRACE(log, "Caught exception while removing nodes from ZooKeeper for this {}: {}",
+ is_restore ? "restore" : "backup",
+ getCurrentExceptionMessage(/* with_stacktrace= */ false, /* check_embedded_stacktrace= */ true));
+
+ std::lock_guard lock{mutex};
+ tried_to_set_error = true;
+
+ if (throw_if_error)
+ throw;
+ return false;
+ }
+}
+
+
+void BackupCoordinationStageSync::createErrorNode(const Exception & exception, Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper)
+{
+ String serialized_error;
+ {
+ WriteBufferFromOwnString buf;
+ writeStringBinary(current_host, buf);
+ writeException(exception, buf, true);
+ serialized_error = buf.str();
+ }
+
+ auto code = zookeeper->tryCreate(error_node_path, serialized_error, zkutil::CreateMode::Persistent);
+
+ if (code == Coordination::Error::ZOK)
+ {
+ std::lock_guard lock{mutex};
+ if (!state.host_with_error)
+ {
+ state.host_with_error = current_host;
+ state.hosts.at(current_host).exception = parseErrorNode(serialized_error).first;
+ }
+ LOG_TRACE(log, "Sent exception from {} to other hosts", current_host_desc);
+ return;
+ }
+
+ if (code == Coordination::Error::ZNODEEXISTS)
+ {
+ String another_error = zookeeper->get(error_node_path);
+ auto [another_exception, host] = parseErrorNode(another_error);
+ if (another_exception)
+ {
+ std::lock_guard lock{mutex};
+ if (!state.host_with_error)
+ {
+ state.host_with_error = host;
+ state.hosts.at(host).exception = another_exception;
+ }
+ LOG_INFO(log, "Another error is already assigned for this {}", operation_name);
+ return;
+ }
+ }
+
+ throw zkutil::KeeperException::fromPath(code, error_node_path);
+}
+
+
+std::pair BackupCoordinationStageSync::parseErrorNode(const String & error_node_contents) const
+{
+ ReadBufferFromOwnString buf{error_node_contents};
+ String host;
+ readStringBinary(host, buf);
+ if (std::find(all_hosts.begin(), all_hosts.end(), host) == all_hosts.end())
+ return {};
+ auto exception = std::make_exception_ptr(readException(buf, fmt::format("Got error from {}", getHostDesc(host))));
+ return {exception, host};
+}
+
+
+bool BackupCoordinationStageSync::isErrorSet() const
+{
+ std::lock_guard lock{mutex};
+ return state.host_with_error.has_value();
+}
+
}
diff --git a/src/Backups/BackupCoordinationStageSync.h b/src/Backups/BackupCoordinationStageSync.h
index dc0d3c3c83d..11d3d1cf6f4 100644
--- a/src/Backups/BackupCoordinationStageSync.h
+++ b/src/Backups/BackupCoordinationStageSync.h
@@ -1,7 +1,9 @@
#pragma once
+#include
#include
+
namespace DB
{
@@ -9,12 +11,16 @@ namespace DB
class BackupCoordinationStageSync
{
public:
+ /// Empty string as the current host is used to mark the initiator of a BACKUP ON CLUSTER or RESTORE ON CLUSTER query.
+ static const constexpr std::string_view kInitiator;
+
BackupCoordinationStageSync(
bool is_restore_, /// true if this is a RESTORE ON CLUSTER command, false if this is a BACKUP ON CLUSTER command
const String & zookeeper_path_, /// path to the "stage" folder in ZooKeeper
const String & current_host_, /// the current host, or an empty string if it's the initiator of the BACKUP/RESTORE ON CLUSTER command
const Strings & all_hosts_, /// all the hosts (including the initiator and the current host) performing the BACKUP/RESTORE ON CLUSTER command
bool allow_concurrency_, /// whether it's allowed to have concurrent backups or restores.
+ BackupConcurrencyCounters & concurrency_counters_,
const WithRetries & with_retries_,
ThreadPoolCallbackRunnerUnsafe schedule_,
QueryStatusPtr process_list_element_,
@@ -22,30 +28,37 @@ public:
~BackupCoordinationStageSync();
+ /// Sets that the BACKUP or RESTORE query was sent to other hosts.
+ void setQueryIsSentToOtherHosts();
+ bool isQuerySentToOtherHosts() const;
+
/// Sets the stage of the current host and signal other hosts if there were other hosts waiting for that.
void setStage(const String & stage, const String & stage_result = {});
- /// Waits until all the specified hosts come to the specified stage.
- /// The function returns the results which specified hosts set when they came to the required stage.
- /// If it doesn't happen before the timeout then the function will stop waiting and throw an exception.
- Strings waitForHostsToReachStage(const String & stage_to_wait, const Strings & hosts, std::optional timeout = {}) const;
-
- /// Waits until all the other hosts finish their work.
- /// Stops waiting and throws an exception if another host encounters an error or if some host gets cancelled.
- void waitForOtherHostsToFinish() const;
-
- /// Lets other host know that the current host has finished its work.
- void finish(bool & other_hosts_also_finished);
+ /// Waits until specified hosts come to the specified stage.
+ /// The function returns the results which the specified hosts set when they came to the required stage.
+ Strings waitHostsReachStage(const Strings & hosts, const String & stage_to_wait) const;
/// Lets other hosts know that the current host has encountered an error.
- bool trySetError(std::exception_ptr exception) noexcept;
+ /// The function returns true if it successfully created the error node or if the error node was found already exist.
+ bool setError(std::exception_ptr exception, bool throw_if_error);
+ bool isErrorSet() const;
- /// Waits until all the other hosts finish their work (as a part of error-handling process).
- /// Doesn't stops waiting if some host encounters an error or gets cancelled.
- bool tryWaitForOtherHostsToFinishAfterError() const noexcept;
+ /// Waits until the hosts other than the current host finish their work. Must be called before finish().
+ /// Stops waiting and throws an exception if another host encounters an error or if some host gets cancelled.
+ bool waitOtherHostsFinish(bool throw_if_error) const;
+ bool otherHostsFinished() const;
- /// Lets other host know that the current host has finished its work (as a part of error-handling process).
- bool tryFinishAfterError(bool & other_hosts_also_finished) noexcept;
+ /// Lets other hosts know that the current host has finished its work.
+ bool finish(bool throw_if_error);
+ bool finished() const;
+
+ /// Returns true if all the hosts have finished.
+ bool allHostsFinished() const { return finished() && otherHostsFinished(); }
+
+ /// Returns a list of the hosts which haven't finished yet.
+ Strings getUnfinishedHosts() const;
+ Strings getUnfinishedOtherHosts() const;
/// Returns a printable name of a specific host. For empty host the function returns "initiator".
static String getHostDesc(const String & host);
@@ -78,14 +91,17 @@ private:
/// Reads the current state from ZooKeeper without throwing exceptions.
void readCurrentState(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper);
+
+ /// Creates a stage node to let other hosts know we've reached the specified stage.
+ void createStageNode(const String & stage, const String & stage_result, Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper);
String getStageNodePath(const String & stage) const;
/// Lets other hosts know that the current host has encountered an error.
- bool trySetError(const Exception & exception);
- void setError(const Exception & exception);
+ bool setError(const Exception & exception, bool throw_if_error);
+ void createErrorNode(const Exception & exception, Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper);
/// Deserializes an error stored in the error node.
- static std::pair parseErrorNode(const String & error_node_contents);
+ std::pair parseErrorNode(const String & error_node_contents) const;
/// Reset the `connected` flag for each host.
void resetConnectedFlag();
@@ -102,19 +118,27 @@ private:
void cancelQueryIfDisconnectedTooLong();
/// Used by waitForHostsToReachStage() to check if everything is ready to return.
- bool checkIfHostsReachStage(const Strings & hosts, const String & stage_to_wait, bool time_is_out, std::optional timeout, Strings & results) const TSA_REQUIRES(mutex);
+ bool checkIfHostsReachStage(const Strings & hosts, const String & stage_to_wait, Strings & results) const TSA_REQUIRES(mutex);
/// Creates the 'finish' node.
- bool tryFinishImpl();
- bool tryFinishImpl(bool & other_hosts_also_finished, bool throw_if_error, WithRetries::Kind retries_kind);
- void createFinishNodeAndRemoveAliveNode(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper);
+ bool finishImpl(bool throw_if_error, WithRetries::Kind retries_kind);
+ void createFinishNodeAndRemoveAliveNode(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper, bool throw_if_error);
/// Returns the version used by the initiator.
int getInitiatorVersion() const;
/// Waits until all the other hosts finish their work.
- bool tryWaitForOtherHostsToFinishImpl(const String & reason, bool throw_if_error, std::optional timeout) const;
- bool checkIfOtherHostsFinish(const String & reason, bool throw_if_error, bool time_is_out, std::optional timeout) const TSA_REQUIRES(mutex);
+ bool waitOtherHostsFinishImpl(const String & reason, std::optional timeout, bool throw_if_error) const;
+ bool checkIfOtherHostsFinish(const String & reason, std::optional timeout, bool time_is_out, bool & result, bool throw_if_error) const TSA_REQUIRES(mutex);
+
+ /// Returns true if all the hosts have finished.
+ bool allHostsFinishedNoLock() const TSA_REQUIRES(mutex);
+ bool finishedNoLock() const TSA_REQUIRES(mutex);
+ bool otherHostsFinishedNoLock() const TSA_REQUIRES(mutex);
+
+ /// Returns a list of the hosts which haven't finished yet.
+ Strings getUnfinishedHostsNoLock() const TSA_REQUIRES(mutex);
+ Strings getUnfinishedOtherHostsNoLock() const TSA_REQUIRES(mutex);
const bool is_restore;
const String operation_name;
@@ -138,15 +162,16 @@ private:
/// Paths in ZooKeeper.
const std::filesystem::path zookeeper_path;
const String root_zookeeper_path;
- const String operation_node_path;
+ const String operation_zookeeper_path;
const String operation_node_name;
- const String stage_node_path;
const String start_node_path;
const String finish_node_path;
const String num_hosts_node_path;
+ const String error_node_path;
const String alive_node_path;
const String alive_tracker_node_path;
- const String error_node_path;
+
+ std::optional concurrency_check;
std::shared_ptr zk_nodes_changed;
@@ -176,25 +201,21 @@ private:
{
std::map hosts; /// std::map because we need to compare states
std::optional host_with_error;
- bool cancelled = false;
bool operator ==(const State & other) const;
bool operator !=(const State & other) const;
+ void merge(const State & other);
};
State state TSA_GUARDED_BY(mutex);
mutable std::condition_variable state_changed;
std::future watching_thread_future;
- std::atomic should_stop_watching_thread = false;
+ bool should_stop_watching_thread TSA_GUARDED_BY(mutex) = false;
- struct FinishResult
- {
- bool succeeded = false;
- std::exception_ptr exception;
- bool other_hosts_also_finished = false;
- };
- FinishResult finish_result TSA_GUARDED_BY(mutex);
+ bool query_is_sent_to_other_hosts TSA_GUARDED_BY(mutex) = false;
+ bool tried_to_finish TSA_GUARDED_BY(mutex) = false;
+ bool tried_to_set_error TSA_GUARDED_BY(mutex) = false;
mutable std::mutex mutex;
};
diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp
index 8480dc5d64d..88ebf8eef32 100644
--- a/src/Backups/BackupsWorker.cpp
+++ b/src/Backups/BackupsWorker.cpp
@@ -329,6 +329,7 @@ std::pair BackupsWorker::start(const ASTPtr & backup_
struct BackupsWorker::BackupStarter
{
BackupsWorker & backups_worker;
+ LoggerPtr log;
std::shared_ptr backup_query;
ContextPtr query_context; /// We have to keep `query_context` until the end of the operation because a pointer to it is stored inside the ThreadGroup we're using.
ContextMutablePtr backup_context;
@@ -345,6 +346,7 @@ struct BackupsWorker::BackupStarter
BackupStarter(BackupsWorker & backups_worker_, const ASTPtr & query_, const ContextPtr & context_)
: backups_worker(backups_worker_)
+ , log(backups_worker.log)
, backup_query(std::static_pointer_cast(query_->clone()))
, query_context(context_)
, backup_context(Context::createCopy(query_context))
@@ -399,9 +401,20 @@ struct BackupsWorker::BackupStarter
chassert(!backup);
backup = backups_worker.openBackupForWriting(backup_info, backup_settings, backup_coordination, backup_context);
- backups_worker.doBackup(
- backup, backup_query, backup_id, backup_name_for_logging, backup_settings, backup_coordination, backup_context,
- on_cluster, cluster);
+ backups_worker.doBackup(backup, backup_query, backup_id, backup_settings, backup_coordination, backup_context,
+ on_cluster, cluster);
+
+ backup_coordination->finish(/* throw_if_error = */ true);
+ backup.reset();
+
+ /// The backup coordination is not needed anymore.
+ if (!is_internal_backup)
+ backup_coordination->cleanup(/* throw_if_error = */ true);
+ backup_coordination.reset();
+
+ /// NOTE: setStatus is called after setNumFilesAndSize in order to have actual information in a backup log record
+ LOG_INFO(log, "{} {} was created successfully", (is_internal_backup ? "Internal backup" : "Backup"), backup_name_for_logging);
+ backups_worker.setStatus(backup_id, BackupStatus::BACKUP_CREATED);
}
void onException()
@@ -416,16 +429,29 @@ struct BackupsWorker::BackupStarter
if (backup && !backup->setIsCorrupted())
should_remove_files_in_backup = false;
- if (backup_coordination && backup_coordination->trySetError(std::current_exception()))
+ bool all_hosts_finished = false;
+
+ if (backup_coordination && backup_coordination->setError(std::current_exception(), /* throw_if_error = */ false))
{
- bool other_hosts_finished = backup_coordination->tryWaitForOtherHostsToFinishAfterError();
+ bool other_hosts_finished = !is_internal_backup
+ && (!backup_coordination->isBackupQuerySentToOtherHosts() || backup_coordination->waitOtherHostsFinish(/* throw_if_error = */ false));
- if (should_remove_files_in_backup && other_hosts_finished)
- backup->tryRemoveAllFiles();
-
- backup_coordination->tryFinishAfterError();
+ all_hosts_finished = backup_coordination->finish(/* throw_if_error = */ false) && other_hosts_finished;
}
+ if (!all_hosts_finished)
+ should_remove_files_in_backup = false;
+
+ if (backup && should_remove_files_in_backup)
+ backup->tryRemoveAllFiles();
+
+ backup.reset();
+
+ if (backup_coordination && all_hosts_finished)
+ backup_coordination->cleanup(/* throw_if_error = */ false);
+
+ backup_coordination.reset();
+
backups_worker.setStatusSafe(backup_id, getBackupStatusFromCurrentException());
}
};
@@ -497,7 +523,6 @@ void BackupsWorker::doBackup(
BackupMutablePtr backup,
const std::shared_ptr & backup_query,
const OperationID & backup_id,
- const String & backup_name_for_logging,
const BackupSettings & backup_settings,
std::shared_ptr backup_coordination,
ContextMutablePtr context,
@@ -521,10 +546,10 @@ void BackupsWorker::doBackup(
backup_settings.copySettingsToQuery(*backup_query);
sendQueryToOtherHosts(*backup_query, cluster, backup_settings.shard_num, backup_settings.replica_num,
context, required_access, backup_coordination->getOnClusterInitializationKeeperRetriesInfo());
- backup_coordination->setBackupQueryWasSentToOtherHosts();
+ backup_coordination->setBackupQueryIsSentToOtherHosts();
/// Wait until all the hosts have written their backup entries.
- backup_coordination->waitForOtherHostsToFinish();
+ backup_coordination->waitOtherHostsFinish(/* throw_if_error = */ true);
}
else
{
@@ -569,18 +594,8 @@ void BackupsWorker::doBackup(
compressed_size = backup->getCompressedSize();
}
- /// Close the backup.
- backup.reset();
-
- /// The backup coordination is not needed anymore.
- backup_coordination->finish();
-
/// NOTE: we need to update metadata again after backup->finalizeWriting(), because backup metadata is written there.
setNumFilesAndSize(backup_id, num_files, total_size, num_entries, uncompressed_size, compressed_size, 0, 0);
-
- /// NOTE: setStatus is called after setNumFilesAndSize in order to have actual information in a backup log record
- LOG_INFO(log, "{} {} was created successfully", (is_internal_backup ? "Internal backup" : "Backup"), backup_name_for_logging);
- setStatus(backup_id, BackupStatus::BACKUP_CREATED);
}
@@ -687,6 +702,7 @@ void BackupsWorker::writeBackupEntries(
struct BackupsWorker::RestoreStarter
{
BackupsWorker & backups_worker;
+ LoggerPtr log;
std::shared_ptr restore_query;
ContextPtr query_context; /// We have to keep `query_context` until the end of the operation because a pointer to it is stored inside the ThreadGroup we're using.
ContextMutablePtr restore_context;
@@ -702,6 +718,7 @@ struct BackupsWorker::RestoreStarter
RestoreStarter(BackupsWorker & backups_worker_, const ASTPtr & query_, const ContextPtr & context_)
: backups_worker(backups_worker_)
+ , log(backups_worker.log)
, restore_query(std::static_pointer_cast(query_->clone()))
, query_context(context_)
, restore_context(Context::createCopy(query_context))
@@ -753,16 +770,17 @@ struct BackupsWorker::RestoreStarter
}
restore_coordination = backups_worker.makeRestoreCoordination(on_cluster, restore_settings, restore_context);
- backups_worker.doRestore(
- restore_query,
- restore_id,
- backup_name_for_logging,
- backup_info,
- restore_settings,
- restore_coordination,
- restore_context,
- on_cluster,
- cluster);
+ backups_worker.doRestore(restore_query, restore_id, backup_info, restore_settings, restore_coordination, restore_context,
+ on_cluster, cluster);
+
+ /// The restore coordination is not needed anymore.
+ restore_coordination->finish(/* throw_if_error = */ true);
+ if (!is_internal_restore)
+ restore_coordination->cleanup(/* throw_if_error = */ true);
+ restore_coordination.reset();
+
+ LOG_INFO(log, "Restored from {} {} successfully", (is_internal_restore ? "internal backup" : "backup"), backup_name_for_logging);
+ backups_worker.setStatus(restore_id, BackupStatus::RESTORED);
}
void onException()
@@ -770,12 +788,16 @@ struct BackupsWorker::RestoreStarter
/// Something bad happened, some data were not restored.
tryLogCurrentException(backups_worker.log, fmt::format("Failed to restore from {} {}", (is_internal_restore ? "internal backup" : "backup"), backup_name_for_logging));
- if (restore_coordination && restore_coordination->trySetError(std::current_exception()))
+ if (restore_coordination && restore_coordination->setError(std::current_exception(), /* throw_if_error = */ false))
{
- restore_coordination->tryWaitForOtherHostsToFinishAfterError();
- restore_coordination->tryFinishAfterError();
+ bool other_hosts_finished = !is_internal_restore
+ && (!restore_coordination->isRestoreQuerySentToOtherHosts() || restore_coordination->waitOtherHostsFinish(/* throw_if_error = */ false));
+ if (restore_coordination->finish(/* throw_if_error = */ false) && other_hosts_finished)
+ restore_coordination->cleanup(/* throw_if_error = */ false);
}
+ restore_coordination.reset();
+
backups_worker.setStatusSafe(restore_id, getRestoreStatusFromCurrentException());
}
};
@@ -838,7 +860,6 @@ BackupPtr BackupsWorker::openBackupForReading(const BackupInfo & backup_info, co
void BackupsWorker::doRestore(
const std::shared_ptr & restore_query,
const OperationID & restore_id,
- const String & backup_name_for_logging,
const BackupInfo & backup_info,
RestoreSettings restore_settings,
std::shared_ptr restore_coordination,
@@ -882,10 +903,10 @@ void BackupsWorker::doRestore(
restore_settings.copySettingsToQuery(*restore_query);
sendQueryToOtherHosts(*restore_query, cluster, restore_settings.shard_num, restore_settings.replica_num,
context, {}, restore_coordination->getOnClusterInitializationKeeperRetriesInfo());
- restore_coordination->setRestoreQueryWasSentToOtherHosts();
+ restore_coordination->setRestoreQueryIsSentToOtherHosts();
/// Wait until all the hosts have done with their restoring work.
- restore_coordination->waitForOtherHostsToFinish();
+ restore_coordination->waitOtherHostsFinish(/* throw_if_error = */ true);
}
else
{
@@ -905,12 +926,6 @@ void BackupsWorker::doRestore(
backup, context, getThreadPool(ThreadPoolId::RESTORE), after_task_callback};
restorer.run(RestorerFromBackup::RESTORE);
}
-
- /// The restore coordination is not needed anymore.
- restore_coordination->finish();
-
- LOG_INFO(log, "Restored from {} {} successfully", (is_internal_restore ? "internal backup" : "backup"), backup_name_for_logging);
- setStatus(restore_id, BackupStatus::RESTORED);
}
@@ -943,7 +958,7 @@ BackupsWorker::makeBackupCoordination(bool on_cluster, const BackupSettings & ba
if (!on_cluster)
{
return std::make_shared(
- *backup_settings.backup_uuid, !backup_settings.deduplicate_files, allow_concurrent_backups, *concurrency_counters);
+ !backup_settings.deduplicate_files, allow_concurrent_backups, *concurrency_counters);
}
bool is_internal_backup = backup_settings.internal;
@@ -981,8 +996,7 @@ BackupsWorker::makeRestoreCoordination(bool on_cluster, const RestoreSettings &
{
if (!on_cluster)
{
- return std::make_shared(
- *restore_settings.restore_uuid, allow_concurrent_restores, *concurrency_counters);
+ return std::make_shared(allow_concurrent_restores, *concurrency_counters);
}
bool is_internal_restore = restore_settings.internal;
diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h
index 37f91e269a9..2e5ca84f3f6 100644
--- a/src/Backups/BackupsWorker.h
+++ b/src/Backups/BackupsWorker.h
@@ -81,7 +81,6 @@ private:
BackupMutablePtr backup,
const std::shared_ptr & backup_query,
const BackupOperationID & backup_id,
- const String & backup_name_for_logging,
const BackupSettings & backup_settings,
std::shared_ptr backup_coordination,
ContextMutablePtr context,
@@ -102,7 +101,6 @@ private:
void doRestore(
const std::shared_ptr & restore_query,
const BackupOperationID & restore_id,
- const String & backup_name_for_logging,
const BackupInfo & backup_info,
RestoreSettings restore_settings,
std::shared_ptr restore_coordination,
diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h
index c0eb90de89b..8bd874b9d0d 100644
--- a/src/Backups/IBackupCoordination.h
+++ b/src/Backups/IBackupCoordination.h
@@ -20,29 +20,27 @@ class IBackupCoordination
public:
virtual ~IBackupCoordination() = default;
+ /// Sets that the backup query was sent to other hosts.
+ /// Function waitOtherHostsFinish() will check that to find out if it should really wait or not.
+ virtual void setBackupQueryIsSentToOtherHosts() = 0;
+ virtual bool isBackupQuerySentToOtherHosts() const = 0;
+
/// Sets the current stage and waits for other hosts to come to this stage too.
virtual Strings setStage(const String & new_stage, const String & message, bool sync) = 0;
- /// Sets that the backup query was sent to other hosts.
- /// Function waitForOtherHostsToFinish() will check that to find out if it should really wait or not.
- virtual void setBackupQueryWasSentToOtherHosts() = 0;
-
/// Lets other hosts know that the current host has encountered an error.
- virtual bool trySetError(std::exception_ptr exception) = 0;
-
- /// Lets other hosts know that the current host has finished its work.
- virtual void finish() = 0;
-
- /// Lets other hosts know that the current host has finished its work (as a part of error-handling process).
- virtual bool tryFinishAfterError() noexcept = 0;
+ /// Returns true if the information is successfully passed so other hosts can read it.
+ virtual bool setError(std::exception_ptr exception, bool throw_if_error) = 0;
/// Waits until all the other hosts finish their work.
/// Stops waiting and throws an exception if another host encounters an error or if some host gets cancelled.
- virtual void waitForOtherHostsToFinish() = 0;
+ virtual bool waitOtherHostsFinish(bool throw_if_error) const = 0;
- /// Waits until all the other hosts finish their work (as a part of error-handling process).
- /// Doesn't stops waiting if some host encounters an error or gets cancelled.
- virtual bool tryWaitForOtherHostsToFinishAfterError() noexcept = 0;
+ /// Lets other hosts know that the current host has finished its work.
+ virtual bool finish(bool throw_if_error) = 0;
+
+ /// Removes temporary nodes in ZooKeeper.
+ virtual bool cleanup(bool throw_if_error) = 0;
struct PartNameAndChecksum
{
diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h
index daabf1745f3..cc7bfd24202 100644
--- a/src/Backups/IRestoreCoordination.h
+++ b/src/Backups/IRestoreCoordination.h
@@ -18,29 +18,27 @@ class IRestoreCoordination
public:
virtual ~IRestoreCoordination() = default;
+ /// Sets that the restore query was sent to other hosts.
+ /// Function waitOtherHostsFinish() will check that to find out if it should really wait or not.
+ virtual void setRestoreQueryIsSentToOtherHosts() = 0;
+ virtual bool isRestoreQuerySentToOtherHosts() const = 0;
+
/// Sets the current stage and waits for other hosts to come to this stage too.
virtual Strings setStage(const String & new_stage, const String & message, bool sync) = 0;
- /// Sets that the restore query was sent to other hosts.
- /// Function waitForOtherHostsToFinish() will check that to find out if it should really wait or not.
- virtual void setRestoreQueryWasSentToOtherHosts() = 0;
-
/// Lets other hosts know that the current host has encountered an error.
- virtual bool trySetError(std::exception_ptr exception) = 0;
-
- /// Lets other hosts know that the current host has finished its work.
- virtual void finish() = 0;
-
- /// Lets other hosts know that the current host has finished its work (as a part of error-handling process).
- virtual bool tryFinishAfterError() noexcept = 0;
+ /// Returns true if the information is successfully passed so other hosts can read it.
+ virtual bool setError(std::exception_ptr exception, bool throw_if_error) = 0;
/// Waits until all the other hosts finish their work.
/// Stops waiting and throws an exception if another host encounters an error or if some host gets cancelled.
- virtual void waitForOtherHostsToFinish() = 0;
+ virtual bool waitOtherHostsFinish(bool throw_if_error) const = 0;
- /// Waits until all the other hosts finish their work (as a part of error-handling process).
- /// Doesn't stops waiting if some host encounters an error or gets cancelled.
- virtual bool tryWaitForOtherHostsToFinishAfterError() noexcept = 0;
+ /// Lets other hosts know that the current host has finished its work.
+ virtual bool finish(bool throw_if_error) = 0;
+
+ /// Removes temporary nodes in ZooKeeper.
+ virtual bool cleanup(bool throw_if_error) = 0;
/// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table.
virtual bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) = 0;
diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp
index 569f58f1909..a9eee1fb159 100644
--- a/src/Backups/RestoreCoordinationLocal.cpp
+++ b/src/Backups/RestoreCoordinationLocal.cpp
@@ -10,9 +10,9 @@ namespace DB
{
RestoreCoordinationLocal::RestoreCoordinationLocal(
- const UUID & restore_uuid, bool allow_concurrent_restore_, BackupConcurrencyCounters & concurrency_counters_)
+ bool allow_concurrent_restore_, BackupConcurrencyCounters & concurrency_counters_)
: log(getLogger("RestoreCoordinationLocal"))
- , concurrency_check(restore_uuid, /* is_restore = */ true, /* on_cluster = */ false, allow_concurrent_restore_, concurrency_counters_)
+ , concurrency_check(/* is_restore = */ true, /* on_cluster = */ false, /* zookeeper_path = */ "", allow_concurrent_restore_, concurrency_counters_)
{
}
diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h
index 6be357c4b7e..6e3262a8a2e 100644
--- a/src/Backups/RestoreCoordinationLocal.h
+++ b/src/Backups/RestoreCoordinationLocal.h
@@ -17,16 +17,16 @@ class ASTCreateQuery;
class RestoreCoordinationLocal : public IRestoreCoordination
{
public:
- RestoreCoordinationLocal(const UUID & restore_uuid_, bool allow_concurrent_restore_, BackupConcurrencyCounters & concurrency_counters_);
+ RestoreCoordinationLocal(bool allow_concurrent_restore_, BackupConcurrencyCounters & concurrency_counters_);
~RestoreCoordinationLocal() override;
+ void setRestoreQueryIsSentToOtherHosts() override {}
+ bool isRestoreQuerySentToOtherHosts() const override { return false; }
Strings setStage(const String &, const String &, bool) override { return {}; }
- void setRestoreQueryWasSentToOtherHosts() override {}
- bool trySetError(std::exception_ptr) override { return true; }
- void finish() override {}
- bool tryFinishAfterError() noexcept override { return true; }
- void waitForOtherHostsToFinish() override {}
- bool tryWaitForOtherHostsToFinishAfterError() noexcept override { return true; }
+ bool setError(std::exception_ptr, bool) override { return true; }
+ bool waitOtherHostsFinish(bool) const override { return true; }
+ bool finish(bool) override { return true; }
+ bool cleanup(bool) override { return true; }
/// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table.
bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) override;
diff --git a/src/Backups/RestoreCoordinationOnCluster.cpp b/src/Backups/RestoreCoordinationOnCluster.cpp
index 2029ad8b072..fad7341c044 100644
--- a/src/Backups/RestoreCoordinationOnCluster.cpp
+++ b/src/Backups/RestoreCoordinationOnCluster.cpp
@@ -35,17 +35,21 @@ RestoreCoordinationOnCluster::RestoreCoordinationOnCluster(
, current_host_index(BackupCoordinationOnCluster::findCurrentHostIndex(current_host, all_hosts))
, log(getLogger("RestoreCoordinationOnCluster"))
, with_retries(log, get_zookeeper_, keeper_settings, process_list_element_, [root_zookeeper_path_](Coordination::ZooKeeperWithFaultInjection::Ptr zk) { zk->sync(root_zookeeper_path_); })
- , concurrency_check(restore_uuid_, /* is_restore = */ true, /* on_cluster = */ true, allow_concurrent_restore_, concurrency_counters_)
- , stage_sync(/* is_restore = */ true, fs::path{zookeeper_path} / "stage", current_host, all_hosts, allow_concurrent_restore_, with_retries, schedule_, process_list_element_, log)
- , cleaner(zookeeper_path, with_retries, log)
+ , cleaner(/* is_restore = */ true, zookeeper_path, with_retries, log)
+ , stage_sync(/* is_restore = */ true, fs::path{zookeeper_path} / "stage", current_host, all_hosts, allow_concurrent_restore_, concurrency_counters_, with_retries, schedule_, process_list_element_, log)
{
- createRootNodes();
+ try
+ {
+ createRootNodes();
+ }
+ catch (...)
+ {
+ stage_sync.setError(std::current_exception(), /* throw_if_error = */ false);
+ throw;
+ }
}
-RestoreCoordinationOnCluster::~RestoreCoordinationOnCluster()
-{
- tryFinishImpl();
-}
+RestoreCoordinationOnCluster::~RestoreCoordinationOnCluster() = default;
void RestoreCoordinationOnCluster::createRootNodes()
{
@@ -66,69 +70,52 @@ void RestoreCoordinationOnCluster::createRootNodes()
});
}
+void RestoreCoordinationOnCluster::setRestoreQueryIsSentToOtherHosts()
+{
+ stage_sync.setQueryIsSentToOtherHosts();
+}
+
+bool RestoreCoordinationOnCluster::isRestoreQuerySentToOtherHosts() const
+{
+ return stage_sync.isQuerySentToOtherHosts();
+}
+
Strings RestoreCoordinationOnCluster::setStage(const String & new_stage, const String & message, bool sync)
{
stage_sync.setStage(new_stage, message);
-
- if (!sync)
- return {};
-
- return stage_sync.waitForHostsToReachStage(new_stage, all_hosts_without_initiator);
+ if (sync)
+ return stage_sync.waitHostsReachStage(all_hosts_without_initiator, new_stage);
+ return {};
}
-void RestoreCoordinationOnCluster::setRestoreQueryWasSentToOtherHosts()
+bool RestoreCoordinationOnCluster::setError(std::exception_ptr exception, bool throw_if_error)
{
- restore_query_was_sent_to_other_hosts = true;
+ return stage_sync.setError(exception, throw_if_error);
}
-bool RestoreCoordinationOnCluster::trySetError(std::exception_ptr exception)
+bool RestoreCoordinationOnCluster::waitOtherHostsFinish(bool throw_if_error) const
{
- return stage_sync.trySetError(exception);
+ return stage_sync.waitOtherHostsFinish(throw_if_error);
}
-void RestoreCoordinationOnCluster::finish()
+bool RestoreCoordinationOnCluster::finish(bool throw_if_error)
{
- bool other_hosts_also_finished = false;
- stage_sync.finish(other_hosts_also_finished);
-
- if ((current_host == kInitiator) && (other_hosts_also_finished || !restore_query_was_sent_to_other_hosts))
- cleaner.cleanup();
+ return stage_sync.finish(throw_if_error);
}
-bool RestoreCoordinationOnCluster::tryFinishAfterError() noexcept
+bool RestoreCoordinationOnCluster::cleanup(bool throw_if_error)
{
- return tryFinishImpl();
-}
-
-bool RestoreCoordinationOnCluster::tryFinishImpl() noexcept
-{
- bool other_hosts_also_finished = false;
- if (!stage_sync.tryFinishAfterError(other_hosts_also_finished))
- return false;
-
- if ((current_host == kInitiator) && (other_hosts_also_finished || !restore_query_was_sent_to_other_hosts))
+ /// All the hosts must finish before we remove the coordination nodes.
+ bool expect_other_hosts_finished = stage_sync.isQuerySentToOtherHosts() || !stage_sync.isErrorSet();
+ bool all_hosts_finished = stage_sync.finished() && (stage_sync.otherHostsFinished() || !expect_other_hosts_finished);
+ if (!all_hosts_finished)
{
- if (!cleaner.tryCleanupAfterError())
- return false;
- }
-
- return true;
-}
-
-void RestoreCoordinationOnCluster::waitForOtherHostsToFinish()
-{
- if ((current_host != kInitiator) || !restore_query_was_sent_to_other_hosts)
- return;
- stage_sync.waitForOtherHostsToFinish();
-}
-
-bool RestoreCoordinationOnCluster::tryWaitForOtherHostsToFinishAfterError() noexcept
-{
- if (current_host != kInitiator)
+ auto unfinished_hosts = expect_other_hosts_finished ? stage_sync.getUnfinishedHosts() : Strings{current_host};
+ LOG_INFO(log, "Skipping removing nodes from ZooKeeper because hosts {} didn't finish",
+ BackupCoordinationStageSync::getHostsDesc(unfinished_hosts));
return false;
- if (!restore_query_was_sent_to_other_hosts)
- return true;
- return stage_sync.tryWaitForOtherHostsToFinishAfterError();
+ }
+ return cleaner.cleanup(throw_if_error);
}
ZooKeeperRetriesInfo RestoreCoordinationOnCluster::getOnClusterInitializationKeeperRetriesInfo() const
diff --git a/src/Backups/RestoreCoordinationOnCluster.h b/src/Backups/RestoreCoordinationOnCluster.h
index 87a8dd3ce83..99929cbdac3 100644
--- a/src/Backups/RestoreCoordinationOnCluster.h
+++ b/src/Backups/RestoreCoordinationOnCluster.h
@@ -1,7 +1,6 @@
#pragma once
#include
-#include
#include
#include
#include
@@ -15,7 +14,7 @@ class RestoreCoordinationOnCluster : public IRestoreCoordination
{
public:
/// Empty string as the current host is used to mark the initiator of a RESTORE ON CLUSTER query.
- static const constexpr std::string_view kInitiator;
+ static const constexpr std::string_view kInitiator = BackupCoordinationStageSync::kInitiator;
RestoreCoordinationOnCluster(
const UUID & restore_uuid_,
@@ -31,13 +30,13 @@ public:
~RestoreCoordinationOnCluster() override;
+ void setRestoreQueryIsSentToOtherHosts() override;
+ bool isRestoreQuerySentToOtherHosts() const override;
Strings setStage(const String & new_stage, const String & message, bool sync) override;
- void setRestoreQueryWasSentToOtherHosts() override;
- bool trySetError(std::exception_ptr exception) override;
- void finish() override;
- bool tryFinishAfterError() noexcept override;
- void waitForOtherHostsToFinish() override;
- bool tryWaitForOtherHostsToFinishAfterError() noexcept override;
+ bool setError(std::exception_ptr exception, bool throw_if_error) override;
+ bool waitOtherHostsFinish(bool throw_if_error) const override;
+ bool finish(bool throw_if_error) override;
+ bool cleanup(bool throw_if_error) override;
/// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table.
bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) override;
@@ -78,11 +77,10 @@ private:
const size_t current_host_index;
LoggerPtr const log;
+ /// The order is important: `stage_sync` must be initialized after `with_retries` and `cleaner`.
const WithRetries with_retries;
- BackupConcurrencyCheck concurrency_check;
- BackupCoordinationStageSync stage_sync;
BackupCoordinationCleaner cleaner;
- std::atomic restore_query_was_sent_to_other_hosts = false;
+ BackupCoordinationStageSync stage_sync;
};
}
diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp
index f419a23b209..262f3ca3a7a 100644
--- a/src/Common/AsynchronousMetrics.cpp
+++ b/src/Common/AsynchronousMetrics.cpp
@@ -331,7 +331,7 @@ AsynchronousMetrics::~AsynchronousMetrics()
AsynchronousMetricValues AsynchronousMetrics::getValues() const
{
- std::lock_guard lock(data_mutex);
+ SharedLockGuard lock(values_mutex);
return values;
}
@@ -1807,7 +1807,10 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update)
first_run = false;
// Finally, update the current metrics.
- values = new_values;
+ {
+ std::lock_guard values_lock(values_mutex);
+ values.swap(new_values);
+ }
}
}
diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h
index 215dc6e1337..e8d4b7980f9 100644
--- a/src/Common/AsynchronousMetrics.h
+++ b/src/Common/AsynchronousMetrics.h
@@ -4,6 +4,7 @@
#include
#include
#include
+#include
#include
#include
@@ -100,6 +101,7 @@ private:
std::condition_variable wait_cond;
bool quit TSA_GUARDED_BY(thread_mutex) = false;
+ /// Protects all raw data and serializes multiple updates.
mutable std::mutex data_mutex;
/// Some values are incremental and we have to calculate the difference.
@@ -107,7 +109,15 @@ private:
bool first_run TSA_GUARDED_BY(data_mutex) = true;
TimePoint previous_update_time TSA_GUARDED_BY(data_mutex);
- AsynchronousMetricValues values TSA_GUARDED_BY(data_mutex);
+ /// Protects saved values.
+ mutable SharedMutex values_mutex;
+ /// Values store the result of the last update prepared for reading.
+#ifdef OS_LINUX
+ AsynchronousMetricValues values TSA_GUARDED_BY(values_mutex);
+#else
+ /// When SharedMutex == std::shared_mutex it may not be annotated with the 'capability'.
+ AsynchronousMetricValues values;
+#endif
#if defined(OS_LINUX) || defined(OS_FREEBSD)
MemoryStatisticsOS memory_stat TSA_GUARDED_BY(data_mutex);
diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp
index 0526596509d..0a987cf36af 100644
--- a/src/Common/CurrentMetrics.cpp
+++ b/src/Common/CurrentMetrics.cpp
@@ -255,6 +255,7 @@
M(PartsActive, "Active data part, used by current and upcoming SELECTs.") \
M(AttachedDatabase, "Active databases.") \
M(AttachedTable, "Active tables.") \
+ M(AttachedReplicatedTable, "Active replicated tables.") \
M(AttachedView, "Active views.") \
M(AttachedDictionary, "Active dictionaries.") \
M(PartsOutdated, "Not active data part, but could be used by only current SELECTs, could be deleted after SELECTs finishes.") \
diff --git a/src/Common/LockGuard.h b/src/Common/LockGuard.h
index 8a98c5f553a..03c8a3e7617 100644
--- a/src/Common/LockGuard.h
+++ b/src/Common/LockGuard.h
@@ -1,23 +1,47 @@
#pragma once
-#include
#include
+#include
+#include
namespace DB
{
+namespace ErrorCodes
+{
+ extern const int LOGICAL_ERROR;
+};
+
/** LockGuard provides RAII-style locking mechanism for a mutex.
- ** It's intended to be used like std::unique_ptr but with TSA annotations
+ ** It's intended to be used like std::unique_lock but with TSA annotations
*/
template
class TSA_SCOPED_LOCKABLE LockGuard
{
public:
- explicit LockGuard(Mutex & mutex_) TSA_ACQUIRE(mutex_) : mutex(mutex_) { mutex.lock(); }
- ~LockGuard() TSA_RELEASE() { mutex.unlock(); }
+ explicit LockGuard(Mutex & mutex_) TSA_ACQUIRE(mutex_) : mutex(mutex_) { lock(); }
+ ~LockGuard() TSA_RELEASE() { if (locked) unlock(); }
+
+ void lock() TSA_ACQUIRE()
+ {
+ /// Don't allow recursive_mutex for now.
+ if (locked)
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't lock twice the same mutex");
+ mutex.lock();
+ locked = true;
+ }
+
+ void unlock() TSA_RELEASE()
+ {
+ if (!locked)
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't unlock the mutex without locking it first");
+ mutex.unlock();
+ locked = false;
+ }
private:
Mutex & mutex;
+ bool locked = false;
};
template typename TLockGuard, typename Mutex>
diff --git a/src/Compression/getCompressionCodecForFile.cpp b/src/Compression/getCompressionCodecForFile.cpp
index 027ee0ac57a..b04e4b6371a 100644
--- a/src/Compression/getCompressionCodecForFile.cpp
+++ b/src/Compression/getCompressionCodecForFile.cpp
@@ -10,33 +10,50 @@
namespace DB
{
-
using Checksum = CityHash_v1_0_2::uint128;
-CompressionCodecPtr getCompressionCodecForFile(const IDataPartStorage & data_part_storage, const String & relative_path)
+CompressionCodecPtr
+getCompressionCodecForFile(ReadBuffer & read_buffer, UInt32 & size_compressed, UInt32 & size_decompressed, bool skip_to_next_block)
{
- auto read_buffer = data_part_storage.readFile(relative_path, {}, std::nullopt, std::nullopt);
- read_buffer->ignore(sizeof(Checksum));
+ read_buffer.ignore(sizeof(Checksum));
UInt8 header_size = ICompressionCodec::getHeaderSize();
+ size_t starting_bytes = read_buffer.count();
PODArray compressed_buffer;
compressed_buffer.resize(header_size);
- read_buffer->readStrict(compressed_buffer.data(), header_size);
+ read_buffer.readStrict(compressed_buffer.data(), header_size);
uint8_t method = ICompressionCodec::readMethod(compressed_buffer.data());
+ size_compressed = unalignedLoad(&compressed_buffer[1]);
+ size_decompressed = unalignedLoad(&compressed_buffer[5]);
if (method == static_cast(CompressionMethodByte::Multiple))
{
compressed_buffer.resize(1);
- read_buffer->readStrict(compressed_buffer.data(), 1);
+ read_buffer.readStrict(compressed_buffer.data(), 1);
compressed_buffer.resize(1 + compressed_buffer[0]);
- read_buffer->readStrict(compressed_buffer.data() + 1, compressed_buffer[0]);
+ read_buffer.readStrict(compressed_buffer.data() + 1, compressed_buffer[0]);
auto codecs_bytes = CompressionCodecMultiple::getCodecsBytesFromData(compressed_buffer.data());
Codecs codecs;
for (auto byte : codecs_bytes)
codecs.push_back(CompressionCodecFactory::instance().get(byte));
+ if (skip_to_next_block)
+ read_buffer.ignore(size_compressed - (read_buffer.count() - starting_bytes));
+
return std::make_shared(codecs);
}
+
+ if (skip_to_next_block)
+ read_buffer.ignore(size_compressed - (read_buffer.count() - starting_bytes));
+
return CompressionCodecFactory::instance().get(method);
}
+CompressionCodecPtr getCompressionCodecForFile(const IDataPartStorage & data_part_storage, const String & relative_path)
+{
+ auto read_buffer = data_part_storage.readFile(relative_path, {}, std::nullopt, std::nullopt);
+ UInt32 size_compressed;
+ UInt32 size_decompressed;
+ return getCompressionCodecForFile(*read_buffer, size_compressed, size_decompressed, false);
+}
+
}
diff --git a/src/Compression/getCompressionCodecForFile.h b/src/Compression/getCompressionCodecForFile.h
index b6f22750e4d..535befa37e1 100644
--- a/src/Compression/getCompressionCodecForFile.h
+++ b/src/Compression/getCompressionCodecForFile.h
@@ -13,4 +13,8 @@ namespace DB
/// from metadata.
CompressionCodecPtr getCompressionCodecForFile(const IDataPartStorage & data_part_storage, const String & relative_path);
+/// Same as above which is used by clickhouse-compressor to print compression statistics of each data block.
+CompressionCodecPtr
+getCompressionCodecForFile(ReadBuffer & read_buffer, UInt32 & size_compressed, UInt32 & size_decompressed, bool skip_to_next_block);
+
}
diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp
index d573377fc8b..2f8e7b6843a 100644
--- a/src/Core/ServerSettings.cpp
+++ b/src/Core/ServerSettings.cpp
@@ -131,6 +131,9 @@ namespace DB
DECLARE(UInt64, max_database_num_to_warn, 1000lu, "If the number of databases is greater than this value, the server will create a warning that will displayed to user.", 0) \
DECLARE(UInt64, max_part_num_to_warn, 100000lu, "If the number of parts is greater than this value, the server will create a warning that will displayed to user.", 0) \
DECLARE(UInt64, max_table_num_to_throw, 0lu, "If number of tables is greater than this value, server will throw an exception. 0 means no limitation. View, remote tables, dictionary, system tables are not counted. Only count table in Atomic/Ordinary/Replicated/Lazy database engine.", 0) \
+ DECLARE(UInt64, max_replicated_table_num_to_throw, 0lu, "If number of replicated tables is greater than this value, server will throw an exception. 0 means no limitation. Only count table in Atomic/Ordinary/Replicated/Lazy database engine.", 0) \
+ DECLARE(UInt64, max_dictionary_num_to_throw, 0lu, "If number of dictionaries is greater than this value, server will throw an exception. 0 means no limitation. Only count table in Atomic/Ordinary/Replicated/Lazy database engine.", 0) \
+ DECLARE(UInt64, max_view_num_to_throw, 0lu, "If number of views is greater than this value, server will throw an exception. 0 means no limitation. Only count table in Atomic/Ordinary/Replicated/Lazy database engine.", 0) \
DECLARE(UInt64, max_database_num_to_throw, 0lu, "If number of databases is greater than this value, server will throw an exception. 0 means no limitation.", 0) \
DECLARE(UInt64, max_authentication_methods_per_user, 100, "The maximum number of authentication methods a user can be created with or altered. Changing this setting does not affect existing users. Zero means unlimited", 0) \
DECLARE(UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0) \
diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp
index e30663ae51d..3871997dcdd 100644
--- a/src/Core/Settings.cpp
+++ b/src/Core/Settings.cpp
@@ -3672,6 +3672,11 @@ Given that, for example, dictionaries, can be out of sync across nodes, mutation
```
+)", 0) \
+ DECLARE(Bool, validate_mutation_query, true, R"(
+Validate mutation queries before accepting them. Mutations are executed in the background, and running an invalid query will cause mutations to get stuck, requiring manual intervention.
+
+Only change this setting if you encounter a backward-incompatible bug.
)", 0) \
DECLARE(Seconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, R"(
Defines how many seconds a locking request waits before failing.
@@ -4563,7 +4568,7 @@ Possible values:
- 0 - Disable
- 1 - Enable
)", 0) \
- DECLARE(Bool, query_plan_merge_filters, false, R"(
+ DECLARE(Bool, query_plan_merge_filters, true, R"(
Allow to merge filters in the query plan
)", 0) \
DECLARE(Bool, query_plan_filter_push_down, true, R"(
@@ -4864,9 +4869,9 @@ Allows to record the filesystem caching log for each query
DECLARE(Bool, read_from_filesystem_cache_if_exists_otherwise_bypass_cache, false, R"(
Allow to use the filesystem cache in passive mode - benefit from the existing cache entries, but don't put more entries into the cache. If you set this setting for heavy ad-hoc queries and leave it disabled for short real-time queries, this will allows to avoid cache threshing by too heavy queries and to improve the overall system efficiency.
)", 0) \
- DECLARE(Bool, skip_download_if_exceeds_query_cache, true, R"(
+ DECLARE(Bool, filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit, true, R"(
Skip download from remote filesystem if exceeds query cache size
-)", 0) \
+)", 0) ALIAS(skip_download_if_exceeds_query_cache) \
DECLARE(UInt64, filesystem_cache_max_download_size, (128UL * 1024 * 1024 * 1024), R"(
Max remote filesystem cache size that can be downloaded by a single query
)", 0) \
diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp
index b1b686dfa52..6e3329e703b 100644
--- a/src/Core/SettingsChangesHistory.cpp
+++ b/src/Core/SettingsChangesHistory.cpp
@@ -64,6 +64,7 @@ static std::initializer_listisSystemStorage() && !DatabaseCatalog::isPredefinedDatabase(database_name))
{
LOG_TEST(log, "Counting detached table {} to database {}", table_name, database_name);
- CurrentMetrics::sub(getAttachedCounterForStorage(table_storage));
+ for (auto metric : getAttachedCountersForStorage(table_storage))
+ CurrentMetrics::sub(metric);
}
auto table_id = table_storage->getStorageID();
@@ -430,7 +431,8 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c
if (!table->isSystemStorage() && !DatabaseCatalog::isPredefinedDatabase(database_name))
{
LOG_TEST(log, "Counting attached table {} to database {}", table_name, database_name);
- CurrentMetrics::add(getAttachedCounterForStorage(table));
+ for (auto metric : getAttachedCountersForStorage(table))
+ CurrentMetrics::add(metric);
}
}
diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp
index 673c82806bd..b8386bcf967 100644
--- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp
+++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp
@@ -277,19 +277,6 @@ void AzureObjectStorage::removeObjectImpl(const StoredObject & object, const Sha
}
}
-/// Remove file. Throws exception if file doesn't exists or it's a directory.
-void AzureObjectStorage::removeObject(const StoredObject & object)
-{
- removeObjectImpl(object, client.get(), false);
-}
-
-void AzureObjectStorage::removeObjects(const StoredObjects & objects)
-{
- auto client_ptr = client.get();
- for (const auto & object : objects)
- removeObjectImpl(object, client_ptr, false);
-}
-
void AzureObjectStorage::removeObjectIfExists(const StoredObject & object)
{
removeObjectImpl(object, client.get(), true);
diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h
index 58225eccd90..401493be367 100644
--- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h
+++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h
@@ -59,11 +59,6 @@ public:
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
const WriteSettings & write_settings = {}) override;
- /// Remove file. Throws exception if file doesn't exists or it's a directory.
- void removeObject(const StoredObject & object) override;
-
- void removeObjects(const StoredObjects & objects) override;
-
void removeObjectIfExists(const StoredObject & object) override;
void removeObjectsIfExist(const StoredObjects & objects) override;
diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp
index 163ff3a9c68..779b8830fab 100644
--- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp
+++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp
@@ -148,20 +148,6 @@ void CachedObjectStorage::removeCacheIfExists(const std::string & path_key_for_c
cache->removeKeyIfExists(getCacheKey(path_key_for_cache), FileCache::getCommonUser().user_id);
}
-void CachedObjectStorage::removeObject(const StoredObject & object)
-{
- removeCacheIfExists(object.remote_path);
- object_storage->removeObject(object);
-}
-
-void CachedObjectStorage::removeObjects(const StoredObjects & objects)
-{
- for (const auto & object : objects)
- removeCacheIfExists(object.remote_path);
-
- object_storage->removeObjects(objects);
-}
-
void CachedObjectStorage::removeObjectIfExists(const StoredObject & object)
{
removeCacheIfExists(object.remote_path);
diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h
index b77baf21e40..77aa635b89b 100644
--- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h
+++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h
@@ -45,10 +45,6 @@ public:
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
const WriteSettings & write_settings = {}) override;
- void removeObject(const StoredObject & object) override;
-
- void removeObjects(const StoredObjects & objects) override;
-
void removeObjectIfExists(const StoredObject & object) override;
void removeObjectsIfExist(const StoredObjects & objects) override;
diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp
index 64323fb6f3c..19de2bb78af 100644
--- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp
+++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp
@@ -480,8 +480,7 @@ struct WriteFileObjectStorageOperation final : public IDiskObjectStorageOperatio
void undo() override
{
- if (object_storage.exists(object))
- object_storage.removeObject(object);
+ object_storage.removeObjectIfExists(object);
}
void finalize() override
@@ -543,8 +542,7 @@ struct CopyFileObjectStorageOperation final : public IDiskObjectStorageOperation
void undo() override
{
- for (const auto & object : created_objects)
- destination_object_storage.removeObject(object);
+ destination_object_storage.removeObjectsIfExist(created_objects);
}
void finalize() override
diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h
index b53161beb76..7d6c914c398 100644
--- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h
+++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h
@@ -77,11 +77,6 @@ public:
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
const WriteSettings & write_settings = {}) override;
- /// Remove file. Throws exception if file doesn't exists or it's a directory.
- void removeObject(const StoredObject & object) override;
-
- void removeObjects(const StoredObjects & objects) override;
-
void removeObjectIfExists(const StoredObject & object) override;
void removeObjectsIfExist(const StoredObjects & objects) override;
@@ -117,6 +112,11 @@ private:
void initializeHDFSFS() const;
std::string extractObjectKeyFromURL(const StoredObject & object) const;
+ /// Remove file. Throws exception if file doesn't exists or it's a directory.
+ void removeObject(const StoredObject & object);
+
+ void removeObjects(const StoredObjects & objects);
+
const Poco::Util::AbstractConfiguration & config;
mutable HDFSBuilderWrapper hdfs_builder;
diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h
index 8dde96b8b16..adb36762539 100644
--- a/src/Disks/ObjectStorages/IObjectStorage.h
+++ b/src/Disks/ObjectStorages/IObjectStorage.h
@@ -161,11 +161,11 @@ public:
virtual bool isRemote() const = 0;
/// Remove object. Throws exception if object doesn't exists.
- virtual void removeObject(const StoredObject & object) = 0;
+ // virtual void removeObject(const StoredObject & object) = 0;
/// Remove multiple objects. Some object storages can do batch remove in a more
/// optimal way.
- virtual void removeObjects(const StoredObjects & objects) = 0;
+ // virtual void removeObjects(const StoredObjects & objects) = 0;
/// Remove object on path if exists
virtual void removeObjectIfExists(const StoredObject & object) = 0;
diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp
index 5f1b6aedc72..f24501dc60e 100644
--- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp
+++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp
@@ -81,7 +81,7 @@ std::unique_ptr LocalObjectStorage::writeObject( /// NO
return std::make_unique(object.remote_path, buf_size);
}
-void LocalObjectStorage::removeObject(const StoredObject & object)
+void LocalObjectStorage::removeObject(const StoredObject & object) const
{
/// For local object storage files are actually removed when "metadata" is removed.
if (!exists(object))
@@ -91,7 +91,7 @@ void LocalObjectStorage::removeObject(const StoredObject & object)
ErrnoException::throwFromPath(ErrorCodes::CANNOT_UNLINK, object.remote_path, "Cannot unlink file {}", object.remote_path);
}
-void LocalObjectStorage::removeObjects(const StoredObjects & objects)
+void LocalObjectStorage::removeObjects(const StoredObjects & objects) const
{
for (const auto & object : objects)
removeObject(object);
diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h
index f1a0391a984..5b3c3951364 100644
--- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h
+++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h
@@ -42,10 +42,6 @@ public:
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
const WriteSettings & write_settings = {}) override;
- void removeObject(const StoredObject & object) override;
-
- void removeObjects(const StoredObjects & objects) override;
-
void removeObjectIfExists(const StoredObject & object) override;
void removeObjectsIfExist(const StoredObjects & objects) override;
@@ -82,6 +78,10 @@ public:
ReadSettings patchSettings(const ReadSettings & read_settings) const override;
private:
+ void removeObject(const StoredObject & object) const;
+
+ void removeObjects(const StoredObjects & objects) const;
+
String key_prefix;
LoggerPtr log;
std::string description;
diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp
index d56c5d9143c..27aa9304de7 100644
--- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp
+++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp
@@ -203,7 +203,7 @@ void MetadataStorageFromPlainObjectStorageTransaction::unlinkFile(const std::str
{
auto object_key = metadata_storage.object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */);
auto object = StoredObject(object_key.serialize());
- metadata_storage.object_storage->removeObject(object);
+ metadata_storage.object_storage->removeObjectIfExists(object);
}
void MetadataStorageFromPlainObjectStorageTransaction::removeDirectory(const std::string & path)
@@ -211,7 +211,7 @@ void MetadataStorageFromPlainObjectStorageTransaction::removeDirectory(const std
if (metadata_storage.object_storage->isWriteOnce())
{
for (auto it = metadata_storage.iterateDirectory(path); it->isValid(); it->next())
- metadata_storage.object_storage->removeObject(StoredObject(it->path()));
+ metadata_storage.object_storage->removeObjectIfExists(StoredObject(it->path()));
}
else
{
diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp
index ea57d691908..62015631aa5 100644
--- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp
+++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp
@@ -107,7 +107,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
CurrentMetrics::sub(metric, 1);
- object_storage->removeObject(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));
@@ -247,7 +247,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std:
auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix);
auto metadata_object = StoredObject(/*remote_path*/ metadata_object_key.serialize(), /*local_path*/ path / PREFIX_PATH_FILE_NAME);
- object_storage->removeObject(metadata_object);
+ object_storage->removeObjectIfExists(metadata_object);
{
std::lock_guard lock(path_map.mutex);
diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp
index 47ef97401f2..9fca3cad688 100644
--- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp
+++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp
@@ -326,21 +326,11 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e
ProfileEvents::DiskS3DeleteObjects);
}
-void S3ObjectStorage::removeObject(const StoredObject & object)
-{
- removeObjectImpl(object, false);
-}
-
void S3ObjectStorage::removeObjectIfExists(const StoredObject & object)
{
removeObjectImpl(object, true);
}
-void S3ObjectStorage::removeObjects(const StoredObjects & objects)
-{
- removeObjectsImpl(objects, false);
-}
-
void S3ObjectStorage::removeObjectsIfExist(const StoredObjects & objects)
{
removeObjectsImpl(objects, true);
diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h
index d6e84cf57ef..4b9c968ede9 100644
--- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h
+++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h
@@ -101,13 +101,6 @@ public:
ObjectStorageIteratorPtr iterate(const std::string & path_prefix, size_t max_keys) const override;
- /// Uses `DeleteObjectRequest`.
- void removeObject(const StoredObject & object) override;
-
- /// Uses `DeleteObjectsRequest` if it is allowed by `s3_capabilities`, otherwise `DeleteObjectRequest`.
- /// `DeleteObjectsRequest` is not supported on GCS, see https://issuetracker.google.com/issues/162653700 .
- void removeObjects(const StoredObjects & objects) override;
-
/// Uses `DeleteObjectRequest`.
void removeObjectIfExists(const StoredObject & object) override;
diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp
index 871d3b506f6..35abc0ed0df 100644
--- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp
+++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp
@@ -254,16 +254,6 @@ std::unique_ptr WebObjectStorage::writeObject( /// NOLI
throwNotAllowed();
}
-void WebObjectStorage::removeObject(const StoredObject &)
-{
- throwNotAllowed();
-}
-
-void WebObjectStorage::removeObjects(const StoredObjects &)
-{
- throwNotAllowed();
-}
-
void WebObjectStorage::removeObjectIfExists(const StoredObject &)
{
throwNotAllowed();
diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.h b/src/Disks/ObjectStorages/Web/WebObjectStorage.h
index 573221b7e21..1e612bd359c 100644
--- a/src/Disks/ObjectStorages/Web/WebObjectStorage.h
+++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.h
@@ -47,10 +47,6 @@ public:
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
const WriteSettings & write_settings = {}) override;
- void removeObject(const StoredObject & object) override;
-
- void removeObjects(const StoredObjects & objects) override;
-
void removeObjectIfExists(const StoredObject & object) override;
void removeObjectsIfExist(const StoredObjects & objects) override;
diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h
index be0875581a5..9b2328065fc 100644
--- a/src/Functions/FunctionsComparison.h
+++ b/src/Functions/FunctionsComparison.h
@@ -1033,6 +1033,9 @@ private:
size_t tuple_size,
size_t input_rows_count) const
{
+ if (0 == tuple_size)
+ throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Comparison of zero-sized tuples is not implemented");
+
ColumnsWithTypeAndName less_columns(tuple_size);
ColumnsWithTypeAndName equal_columns(tuple_size - 1);
ColumnsWithTypeAndName tmp_columns(2);
diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp
index e03b27b3c39..5e1e7067e86 100644
--- a/src/Functions/if.cpp
+++ b/src/Functions/if.cpp
@@ -668,6 +668,9 @@ private:
temporary_columns[0] = arguments[0];
size_t tuple_size = type1.getElements().size();
+ if (tuple_size == 0)
+ return ColumnTuple::create(input_rows_count);
+
Columns tuple_columns(tuple_size);
for (size_t i = 0; i < tuple_size; ++i)
diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h
index c1747314c76..b66b9867e39 100644
--- a/src/IO/ReadSettings.h
+++ b/src/IO/ReadSettings.h
@@ -69,7 +69,7 @@ struct ReadSettings
std::shared_ptr page_cache;
size_t filesystem_cache_max_download_size = (128UL * 1024 * 1024 * 1024);
- bool skip_download_if_exceeds_query_cache = true;
+ bool filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit = true;
size_t remote_read_min_bytes_for_seek = DBMS_DEFAULT_BUFFER_SIZE;
diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp
index 7c6a21941eb..aefe3ff338c 100644
--- a/src/IO/S3/URI.cpp
+++ b/src/IO/S3/URI.cpp
@@ -37,7 +37,7 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax)
/// Case when bucket name represented in domain name of S3 URL.
/// E.g. (https://bucket-name.s3.region.amazonaws.com/key)
/// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#virtual-hosted-style-access
- static const RE2 virtual_hosted_style_pattern(R"((.+)\.(s3express[\-a-z0-9]+|s3|cos|obs|oss|eos)([.\-][a-z0-9\-.:]+))");
+ static const RE2 virtual_hosted_style_pattern(R"((.+)\.(s3express[\-a-z0-9]+|s3|cos|obs|oss-data-acc|oss|eos)([.\-][a-z0-9\-.:]+))");
/// Case when AWS Private Link Interface is being used
/// E.g. (bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com/bucket-name/key)
@@ -115,7 +115,15 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax)
&& re2::RE2::FullMatch(uri.getAuthority(), virtual_hosted_style_pattern, &bucket, &name, &endpoint_authority_from_uri))
{
is_virtual_hosted_style = true;
- endpoint = uri.getScheme() + "://" + name + endpoint_authority_from_uri;
+ if (name == "oss-data-acc")
+ {
+ bucket = bucket.substr(0, bucket.find('.'));
+ endpoint = uri.getScheme() + "://" + uri.getHost().substr(bucket.length() + 1);
+ }
+ else
+ {
+ endpoint = uri.getScheme() + "://" + name + endpoint_authority_from_uri;
+ }
validateBucket(bucket, uri);
if (!uri.getPath().empty())
diff --git a/src/IO/tests/gtest_s3_uri.cpp b/src/IO/tests/gtest_s3_uri.cpp
index 8696fab0616..6167313b634 100644
--- a/src/IO/tests/gtest_s3_uri.cpp
+++ b/src/IO/tests/gtest_s3_uri.cpp
@@ -212,6 +212,22 @@ TEST(S3UriTest, validPatterns)
ASSERT_EQ("", uri.version_id);
ASSERT_EQ(true, uri.is_virtual_hosted_style);
}
+ {
+ S3::URI uri("https://bucket-test1.oss-cn-beijing-internal.aliyuncs.com/ab-test");
+ ASSERT_EQ("https://oss-cn-beijing-internal.aliyuncs.com", uri.endpoint);
+ ASSERT_EQ("bucket-test1", uri.bucket);
+ ASSERT_EQ("ab-test", uri.key);
+ ASSERT_EQ("", uri.version_id);
+ ASSERT_EQ(true, uri.is_virtual_hosted_style);
+ }
+ {
+ S3::URI uri("https://bucket-test.cn-beijing-internal.oss-data-acc.aliyuncs.com/ab-test");
+ ASSERT_EQ("https://cn-beijing-internal.oss-data-acc.aliyuncs.com", uri.endpoint);
+ ASSERT_EQ("bucket-test", uri.bucket);
+ ASSERT_EQ("ab-test", uri.key);
+ ASSERT_EQ("", uri.version_id);
+ ASSERT_EQ(true, uri.is_virtual_hosted_style);
+ }
}
TEST(S3UriTest, versionIdChecks)
diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp
index 7de3f7af78d..8887165a75d 100644
--- a/src/Interpreters/Cache/FileCache.cpp
+++ b/src/Interpreters/Cache/FileCache.cpp
@@ -8,6 +8,7 @@
#include
#include
#include
+#include
#include
#include
#include
@@ -53,16 +54,6 @@ namespace ErrorCodes
namespace
{
- size_t roundDownToMultiple(size_t num, size_t multiple)
- {
- return (num / multiple) * multiple;
- }
-
- size_t roundUpToMultiple(size_t num, size_t multiple)
- {
- return roundDownToMultiple(num + multiple - 1, multiple);
- }
-
std::string getCommonUserID()
{
auto user_from_context = DB::Context::getGlobalContextInstance()->getFilesystemCacheUser();
@@ -96,6 +87,7 @@ FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & s
: max_file_segment_size(settings.max_file_segment_size)
, bypass_cache_threshold(settings.enable_bypass_cache_with_threshold ? settings.bypass_cache_threshold : 0)
, boundary_alignment(settings.boundary_alignment)
+ , background_download_max_file_segment_size(settings.background_download_max_file_segment_size)
, load_metadata_threads(settings.load_metadata_threads)
, load_metadata_asynchronously(settings.load_metadata_asynchronously)
, write_cache_per_user_directory(settings.write_cache_per_user_id_directory)
@@ -103,7 +95,10 @@ FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & s
, keep_current_elements_to_max_ratio(1 - settings.keep_free_space_elements_ratio)
, keep_up_free_space_remove_batch(settings.keep_free_space_remove_batch)
, log(getLogger("FileCache(" + cache_name + ")"))
- , metadata(settings.base_path, settings.background_download_queue_size_limit, settings.background_download_threads, write_cache_per_user_directory)
+ , metadata(settings.base_path,
+ settings.background_download_queue_size_limit,
+ settings.background_download_threads,
+ write_cache_per_user_directory)
{
if (settings.cache_policy == "LRU")
{
@@ -601,8 +596,8 @@ FileCache::getOrSet(
/// 2. max_file_segments_limit
FileSegment::Range result_range = initial_range;
- const auto aligned_offset = roundDownToMultiple(initial_range.left, boundary_alignment);
- auto aligned_end_offset = std::min(roundUpToMultiple(initial_range.right + 1, boundary_alignment), file_size) - 1;
+ const auto aligned_offset = FileCacheUtils::roundDownToMultiple(initial_range.left, boundary_alignment);
+ auto aligned_end_offset = std::min(FileCacheUtils::roundUpToMultiple(initial_range.right + 1, boundary_alignment), file_size) - 1;
chassert(aligned_offset <= initial_range.left);
chassert(aligned_end_offset >= initial_range.right);
@@ -1600,6 +1595,17 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings,
}
}
+ if (new_settings.background_download_max_file_segment_size != actual_settings.background_download_max_file_segment_size)
+ {
+ background_download_max_file_segment_size = new_settings.background_download_max_file_segment_size;
+
+ LOG_INFO(log, "Changed background_download_max_file_segment_size from {} to {}",
+ actual_settings.background_download_max_file_segment_size,
+ new_settings.background_download_max_file_segment_size);
+
+ actual_settings.background_download_max_file_segment_size = new_settings.background_download_max_file_segment_size;
+ }
+
if (new_settings.max_size != actual_settings.max_size
|| new_settings.max_elements != actual_settings.max_elements)
{
diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h
index 810ed481300..bbe8502fec9 100644
--- a/src/Interpreters/Cache/FileCache.h
+++ b/src/Interpreters/Cache/FileCache.h
@@ -161,6 +161,10 @@ public:
size_t getMaxFileSegmentSize() const { return max_file_segment_size; }
+ size_t getBackgroundDownloadMaxFileSegmentSize() const { return background_download_max_file_segment_size.load(); }
+
+ size_t getBoundaryAlignment() const { return boundary_alignment; }
+
bool tryReserve(
FileSegment & file_segment,
size_t size,
@@ -199,6 +203,7 @@ private:
std::atomic max_file_segment_size;
const size_t bypass_cache_threshold;
const size_t boundary_alignment;
+ std::atomic background_download_max_file_segment_size;
size_t load_metadata_threads;
const bool load_metadata_asynchronously;
std::atomic stop_loading_metadata = false;
diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp
index e162d6b7551..8f0c5206211 100644
--- a/src/Interpreters/Cache/FileCacheSettings.cpp
+++ b/src/Interpreters/Cache/FileCacheSettings.cpp
@@ -62,6 +62,9 @@ void FileCacheSettings::loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetStrin
if (has("background_download_queue_size_limit"))
background_download_queue_size_limit = get_uint("background_download_queue_size_limit");
+ if (has("background_download_max_file_segment_size"))
+ background_download_max_file_segment_size = get_uint("background_download_max_file_segment_size");
+
if (has("load_metadata_threads"))
load_metadata_threads = get_uint("load_metadata_threads");
diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h
index 72a2b6c3369..9cf72a2bdff 100644
--- a/src/Interpreters/Cache/FileCacheSettings.h
+++ b/src/Interpreters/Cache/FileCacheSettings.h
@@ -43,6 +43,8 @@ struct FileCacheSettings
double keep_free_space_elements_ratio = FILECACHE_DEFAULT_FREE_SPACE_ELEMENTS_RATIO;
size_t keep_free_space_remove_batch = FILECACHE_DEFAULT_FREE_SPACE_REMOVE_BATCH;
+ size_t background_download_max_file_segment_size = FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE_WITH_BACKGROUND_DOWLOAD;
+
void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
void loadFromCollection(const NamedCollection & collection);
diff --git a/src/Interpreters/Cache/FileCacheUtils.h b/src/Interpreters/Cache/FileCacheUtils.h
new file mode 100644
index 00000000000..b35ce867a79
--- /dev/null
+++ b/src/Interpreters/Cache/FileCacheUtils.h
@@ -0,0 +1,17 @@
+#pragma once
+#include
+
+namespace FileCacheUtils
+{
+
+static size_t roundDownToMultiple(size_t num, size_t multiple)
+{
+ return (num / multiple) * multiple;
+}
+
+static size_t roundUpToMultiple(size_t num, size_t multiple)
+{
+ return roundDownToMultiple(num + multiple - 1, multiple);
+}
+
+}
diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h
index da75f30f0e8..3d461abd065 100644
--- a/src/Interpreters/Cache/FileCache_fwd.h
+++ b/src/Interpreters/Cache/FileCache_fwd.h
@@ -6,6 +6,7 @@ namespace DB
static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 32 * 1024 * 1024; /// 32Mi
static constexpr int FILECACHE_DEFAULT_FILE_SEGMENT_ALIGNMENT = 4 * 1024 * 1024; /// 4Mi
+static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE_WITH_BACKGROUND_DOWLOAD = 4 * 1024 * 1024; /// 4Mi
static constexpr int FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS = 5;
static constexpr int FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_QUEUE_SIZE_LIMIT = 5000;
static constexpr int FILECACHE_DEFAULT_LOAD_METADATA_THREADS = 16;
diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp
index 541f0f5607a..74148aa5461 100644
--- a/src/Interpreters/Cache/FileSegment.cpp
+++ b/src/Interpreters/Cache/FileSegment.cpp
@@ -4,6 +4,7 @@
#include
#include
#include
+#include
#include
#include
#include
@@ -360,11 +361,14 @@ void FileSegment::write(char * from, size_t size, size_t offset_in_file)
"Expected DOWNLOADING state, got {}", stateToString(download_state));
const size_t first_non_downloaded_offset = getCurrentWriteOffset();
+
if (offset_in_file != first_non_downloaded_offset)
+ {
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Attempt to write {} bytes to offset: {}, but current write offset is {}",
size, offset_in_file, first_non_downloaded_offset);
+ }
const size_t current_downloaded_size = getDownloadedSize();
chassert(reserved_size >= current_downloaded_size);
@@ -375,8 +379,19 @@ void FileSegment::write(char * from, size_t size, size_t offset_in_file)
ErrorCodes::LOGICAL_ERROR,
"Not enough space is reserved. Available: {}, expected: {}", free_reserved_size, size);
- if (!is_unbound && current_downloaded_size == range().size())
- throw Exception(ErrorCodes::LOGICAL_ERROR, "File segment is already fully downloaded");
+ if (!is_unbound)
+ {
+ if (current_downloaded_size == range().size())
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "File segment is already fully downloaded");
+
+ if (current_downloaded_size + size > range().size())
+ {
+ throw Exception(
+ ErrorCodes::LOGICAL_ERROR,
+ "Cannot download beyond file segment boundaries: {}. Write offset: {}, size: {}, downloaded size: {}",
+ range().size(), first_non_downloaded_offset, size, current_downloaded_size);
+ }
+ }
if (!cache_writer && current_downloaded_size > 0)
throw Exception(
@@ -629,6 +644,36 @@ void FileSegment::completePartAndResetDownloader()
LOG_TEST(log, "Complete batch. ({})", getInfoForLogUnlocked(lk));
}
+size_t FileSegment::getSizeForBackgroundDownload() const
+{
+ auto lk = lock();
+ return getSizeForBackgroundDownloadUnlocked(lk);
+}
+
+size_t FileSegment::getSizeForBackgroundDownloadUnlocked(const FileSegmentGuard::Lock &) const
+{
+ if (!background_download_enabled
+ || !downloaded_size
+ || !remote_file_reader)
+ {
+ return 0;
+ }
+
+ chassert(downloaded_size <= range().size());
+
+ const size_t background_download_max_file_segment_size = cache->getBackgroundDownloadMaxFileSegmentSize();
+ size_t desired_size;
+ if (downloaded_size >= background_download_max_file_segment_size)
+ desired_size = FileCacheUtils::roundUpToMultiple(downloaded_size, cache->getBoundaryAlignment());
+ else
+ desired_size = FileCacheUtils::roundUpToMultiple(background_download_max_file_segment_size, cache->getBoundaryAlignment());
+
+ desired_size = std::min(desired_size, range().size());
+ chassert(desired_size >= downloaded_size);
+
+ return desired_size - downloaded_size;
+}
+
void FileSegment::complete(bool allow_background_download)
{
ProfileEventTimeIncrement watch(ProfileEvents::FileSegmentCompleteMicroseconds);
@@ -708,7 +753,8 @@ void FileSegment::complete(bool allow_background_download)
if (is_last_holder)
{
bool added_to_download_queue = false;
- if (allow_background_download && background_download_enabled && remote_file_reader)
+ size_t background_download_size = allow_background_download ? getSizeForBackgroundDownloadUnlocked(segment_lock) : 0;
+ if (background_download_size)
{
ProfileEvents::increment(ProfileEvents::FilesystemCacheBackgroundDownloadQueuePush);
added_to_download_queue = locked_key->addToDownloadQueue(offset(), segment_lock); /// Finish download in background.
@@ -862,7 +908,12 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock & lock)
chassert(downloaded_size == reserved_size);
chassert(downloaded_size == range().size());
chassert(downloaded_size > 0);
- chassert(fs::file_size(getPath()) > 0);
+
+ auto file_size = fs::file_size(getPath());
+ UNUSED(file_size);
+
+ chassert(file_size == range().size());
+ chassert(downloaded_size == range().size());
chassert(queue_iterator || on_delayed_removal);
check_iterator(queue_iterator);
@@ -884,7 +935,13 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock & lock)
chassert(reserved_size >= downloaded_size);
chassert(downloaded_size > 0);
- chassert(fs::file_size(getPath()) > 0);
+
+ auto file_size = fs::file_size(getPath());
+ UNUSED(file_size);
+
+ chassert(file_size > 0);
+ chassert(file_size <= range().size());
+ chassert(downloaded_size <= range().size());
chassert(queue_iterator);
check_iterator(queue_iterator);
diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h
index 21d5f9dab5f..a6bfb203cec 100644
--- a/src/Interpreters/Cache/FileSegment.h
+++ b/src/Interpreters/Cache/FileSegment.h
@@ -185,6 +185,8 @@ public:
bool assertCorrectness() const;
+ size_t getSizeForBackgroundDownload() const;
+
/**
* ========== Methods that must do cv.notify() ==================
*/
@@ -230,6 +232,7 @@ private:
String getDownloaderUnlocked(const FileSegmentGuard::Lock &) const;
bool isDownloaderUnlocked(const FileSegmentGuard::Lock & segment_lock) const;
void resetDownloaderUnlocked(const FileSegmentGuard::Lock &);
+ size_t getSizeForBackgroundDownloadUnlocked(const FileSegmentGuard::Lock &) const;
void setDownloadState(State state, const FileSegmentGuard::Lock &);
void resetDownloadingStateUnlocked(const FileSegmentGuard::Lock &);
diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp
index 231545212cd..4d3033191dc 100644
--- a/src/Interpreters/Cache/Metadata.cpp
+++ b/src/Interpreters/Cache/Metadata.cpp
@@ -676,13 +676,17 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optionalinternalBuffer().empty())
{
if (!memory)
- memory.emplace(DBMS_DEFAULT_BUFFER_SIZE);
+ memory.emplace(std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), size_to_download));
reader->set(memory->data(), memory->size());
}
@@ -701,9 +705,13 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optional(reader->getPosition()))
reader->seek(offset, SEEK_SET);
- while (!reader->eof())
+ while (size_to_download && !reader->eof())
{
- auto size = reader->available();
+ const auto available = reader->available();
+ chassert(available);
+
+ const auto size = std::min(available, size_to_download);
+ size_to_download -= size;
std::string failure_reason;
if (!file_segment.reserve(size, reserve_space_lock_wait_timeout_milliseconds, failure_reason))
@@ -713,7 +721,7 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optional LockedKey::sync()
actual_size, expected_size, file_segment->getInfoForLog());
broken.push_back(FileSegment::getInfo(file_segment));
- it = removeFileSegment(file_segment->offset(), file_segment->lock(), /* can_be_broken */false);
+ it = removeFileSegment(file_segment->offset(), file_segment->lock(), /* can_be_broken */true);
}
return broken;
}
diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h
index 0e85ead3265..24683b2de71 100644
--- a/src/Interpreters/Cache/Metadata.h
+++ b/src/Interpreters/Cache/Metadata.h
@@ -210,6 +210,7 @@ public:
bool setBackgroundDownloadThreads(size_t threads_num);
size_t getBackgroundDownloadThreads() const { return download_threads.size(); }
+
bool setBackgroundDownloadQueueSizeLimit(size_t size);
bool isBackgroundDownloadEnabled();
diff --git a/src/Interpreters/Cache/QueryLimit.cpp b/src/Interpreters/Cache/QueryLimit.cpp
index b18d23a5b7f..a7c964022a5 100644
--- a/src/Interpreters/Cache/QueryLimit.cpp
+++ b/src/Interpreters/Cache/QueryLimit.cpp
@@ -53,7 +53,7 @@ FileCacheQueryLimit::QueryContextPtr FileCacheQueryLimit::getOrSetQueryContext(
{
it->second = std::make_shared(
settings.filesystem_cache_max_download_size,
- !settings.skip_download_if_exceeds_query_cache);
+ !settings.filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit);
}
return it->second;
diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp
index d42002bf98d..d2aad0a52d8 100644
--- a/src/Interpreters/Context.cpp
+++ b/src/Interpreters/Context.cpp
@@ -237,7 +237,7 @@ namespace Setting
extern const SettingsUInt64 remote_fs_read_backoff_max_tries;
extern const SettingsUInt64 remote_read_min_bytes_for_seek;
extern const SettingsBool throw_on_error_from_cache_on_write_operations;
- extern const SettingsBool skip_download_if_exceeds_query_cache;
+ extern const SettingsBool filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit;
extern const SettingsBool s3_allow_parallel_part_upload;
extern const SettingsBool use_page_cache_for_disks_without_file_cache;
extern const SettingsUInt64 use_structure_from_insertion_table_in_table_functions;
@@ -5755,7 +5755,7 @@ ReadSettings Context::getReadSettings() const
res.filesystem_cache_prefer_bigger_buffer_size = settings_ref[Setting::filesystem_cache_prefer_bigger_buffer_size];
res.filesystem_cache_max_download_size = settings_ref[Setting::filesystem_cache_max_download_size];
- res.skip_download_if_exceeds_query_cache = settings_ref[Setting::skip_download_if_exceeds_query_cache];
+ res.filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit = settings_ref[Setting::filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit];
res.page_cache = getPageCache();
res.use_page_cache_for_disks_without_file_cache = settings_ref[Setting::use_page_cache_for_disks_without_file_cache];
diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp
index f6586f8bfc2..b027f6aad61 100644
--- a/src/Interpreters/InterpreterCreateQuery.cpp
+++ b/src/Interpreters/InterpreterCreateQuery.cpp
@@ -98,6 +98,9 @@
namespace CurrentMetrics
{
extern const Metric AttachedTable;
+ extern const Metric AttachedReplicatedTable;
+ extern const Metric AttachedDictionary;
+ extern const Metric AttachedView;
}
namespace DB
@@ -145,7 +148,10 @@ namespace ServerSetting
{
extern const ServerSettingsBool ignore_empty_sql_security_in_create_view_query;
extern const ServerSettingsUInt64 max_database_num_to_throw;
+ extern const ServerSettingsUInt64 max_dictionary_num_to_throw;
extern const ServerSettingsUInt64 max_table_num_to_throw;
+ extern const ServerSettingsUInt64 max_replicated_table_num_to_throw;
+ extern const ServerSettingsUInt64 max_view_num_to_throw;
}
namespace ErrorCodes
@@ -1912,16 +1918,8 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
}
}
- UInt64 table_num_limit = getContext()->getGlobalContext()->getServerSettings()[ServerSetting::max_table_num_to_throw];
- if (table_num_limit > 0 && !internal)
- {
- UInt64 table_count = CurrentMetrics::get(CurrentMetrics::AttachedTable);
- if (table_count >= table_num_limit)
- throw Exception(ErrorCodes::TOO_MANY_TABLES,
- "Too many tables. "
- "The limit (server configuration parameter `max_table_num_to_throw`) is set to {}, the current number of tables is {}",
- table_num_limit, table_count);
- }
+ if (!internal)
+ throwIfTooManyEntities(create, res);
database->createTable(getContext(), create.getTable(), res, query_ptr);
@@ -1948,6 +1946,30 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
}
+void InterpreterCreateQuery::throwIfTooManyEntities(ASTCreateQuery & create, StoragePtr storage) const
+{
+ auto check_and_throw = [&](auto setting, CurrentMetrics::Metric metric, String setting_name, String entity_name)
+ {
+ UInt64 num_limit = getContext()->getGlobalContext()->getServerSettings()[setting];
+ UInt64 attached_count = CurrentMetrics::get(metric);
+ if (num_limit > 0 && attached_count >= num_limit)
+ throw Exception(ErrorCodes::TOO_MANY_TABLES,
+ "Too many {}. "
+ "The limit (server configuration parameter `{}`) is set to {}, the current number is {}",
+ entity_name, setting_name, num_limit, attached_count);
+ };
+
+ if (auto * replicated_storage = typeid_cast(storage.get()))
+ check_and_throw(ServerSetting::max_replicated_table_num_to_throw, CurrentMetrics::AttachedReplicatedTable, "max_replicated_table_num_to_throw", "replicated tables");
+ else if (create.is_dictionary)
+ check_and_throw(ServerSetting::max_dictionary_num_to_throw, CurrentMetrics::AttachedDictionary, "max_dictionary_num_to_throw", "dictionaries");
+ else if (create.isView())
+ check_and_throw(ServerSetting::max_view_num_to_throw, CurrentMetrics::AttachedView, "max_view_num_to_throw", "views");
+ else
+ check_and_throw(ServerSetting::max_table_num_to_throw, CurrentMetrics::AttachedTable, "max_table_num_to_throw", "tables");
+}
+
+
BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create,
const InterpreterCreateQuery::TableProperties & properties, LoadingStrictnessLevel mode)
{
diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h
index cb7af25383e..24cf308951c 100644
--- a/src/Interpreters/InterpreterCreateQuery.h
+++ b/src/Interpreters/InterpreterCreateQuery.h
@@ -122,6 +122,8 @@ private:
BlockIO executeQueryOnCluster(ASTCreateQuery & create);
+ void throwIfTooManyEntities(ASTCreateQuery & create, StoragePtr storage) const;
+
ASTPtr query_ptr;
/// Skip safety threshold when loading tables.
diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp
index 0f25d5ac21c..a35353a6b2a 100644
--- a/src/Interpreters/MutationsInterpreter.cpp
+++ b/src/Interpreters/MutationsInterpreter.cpp
@@ -53,6 +53,7 @@ namespace Setting
extern const SettingsBool allow_nondeterministic_mutations;
extern const SettingsUInt64 max_block_size;
extern const SettingsBool use_concurrency_control;
+ extern const SettingsBool validate_mutation_query;
}
namespace MergeTreeSetting
@@ -1386,6 +1387,18 @@ void MutationsInterpreter::validate()
}
}
+ // Make sure the mutation query is valid
+ if (context->getSettingsRef()[Setting::validate_mutation_query])
+ {
+ if (context->getSettingsRef()[Setting::allow_experimental_analyzer])
+ prepareQueryAffectedQueryTree(commands, source.getStorage(), context);
+ else
+ {
+ ASTPtr select_query = prepareQueryAffectedAST(commands, source.getStorage(), context);
+ InterpreterSelectQuery(select_query, context, source.getStorage(), metadata_snapshot);
+ }
+ }
+
QueryPlan plan;
initQueryPlan(stages.front(), plan);
diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp
index 5ab3fe590e0..52d773b7d1b 100644
--- a/src/Interpreters/QueryMetricLog.cpp
+++ b/src/Interpreters/QueryMetricLog.cpp
@@ -1,6 +1,7 @@
#include
#include
#include
+#include
#include
#include
#include
@@ -16,7 +17,6 @@
#include
#include
-#include
namespace DB
@@ -24,6 +24,15 @@ namespace DB
static auto logger = getLogger("QueryMetricLog");
+String timePointToString(QueryMetricLog::TimePoint time)
+{
+ /// fmtlib supports subsecond formatting in 10.0.0. We're in 9.1.0, so we need to add the milliseconds ourselves.
+ auto seconds = std::chrono::time_point_cast(time);
+ auto microseconds = std::chrono::duration_cast(time - seconds).count();
+
+ return fmt::format("{:%Y.%m.%d %H:%M:%S}.{:06}", seconds, microseconds);
+}
+
ColumnsDescription QueryMetricLogElement::getColumnsDescription()
{
ColumnsDescription result;
@@ -87,36 +96,73 @@ void QueryMetricLog::shutdown()
Base::shutdown();
}
-void QueryMetricLog::startQuery(const String & query_id, TimePoint start_time, UInt64 interval_milliseconds)
+void QueryMetricLog::collectMetric(const ProcessList & process_list, String query_id)
{
- QueryMetricLogStatus status;
- status.interval_milliseconds = interval_milliseconds;
- status.next_collect_time = start_time + std::chrono::milliseconds(interval_milliseconds);
+ auto current_time = std::chrono::system_clock::now();
+ const auto query_info = process_list.getQueryInfo(query_id, false, true, false);
+ if (!query_info)
+ {
+ /// TODO: remove trace before 24.11 release after checking everything is fine on the CI
+ LOG_TRACE(logger, "Query {} is not running anymore, so we couldn't get its QueryStatusInfo", query_id);
+ return;
+ }
+
+ LockGuard global_lock(queries_mutex);
+ auto it = queries.find(query_id);
+
+ /// The query might have finished while the scheduled task is running.
+ if (it == queries.end())
+ {
+ global_lock.unlock();
+ /// TODO: remove trace before 24.11 release after checking everything is fine on the CI
+ LOG_TRACE(logger, "Query {} not found in the list. Finished while this collecting task was running", query_id);
+ return;
+ }
+
+ auto & query_status = it->second;
+ if (!query_status.mutex)
+ {
+ global_lock.unlock();
+ /// TODO: remove trace before 24.11 release after checking everything is fine on the CI
+ LOG_TRACE(logger, "Query {} finished while this collecting task was running", query_id);
+ return;
+ }
+
+ LockGuard query_lock(query_status.getMutex());
+ global_lock.unlock();
+
+ auto elem = query_status.createLogMetricElement(query_id, *query_info, current_time);
+ if (elem)
+ add(std::move(elem.value()));
+}
+
+/// We use TSA_NO_THREAD_SAFETY_ANALYSIS to prevent TSA complaining that we're modifying the query_status fields
+/// without locking the mutex. Since we're building it from scratch, there's no harm in not holding it.
+/// If we locked it to make TSA happy, TSAN build would falsely complain about
+/// lock-order-inversion (potential deadlock)
+/// which is not a real issue since QueryMetricLogStatus's mutex cannot be locked by anything else
+/// until we add it to the queries map.
+void QueryMetricLog::startQuery(const String & query_id, TimePoint start_time, UInt64 interval_milliseconds) TSA_NO_THREAD_SAFETY_ANALYSIS
+{
+ QueryMetricLogStatus query_status;
+ QueryMetricLogStatusInfo & info = query_status.info;
+ info.interval_milliseconds = interval_milliseconds;
+ info.next_collect_time = start_time;
auto context = getContext();
const auto & process_list = context->getProcessList();
- status.task = context->getSchedulePool().createTask("QueryMetricLog", [this, &process_list, query_id] {
- auto current_time = std::chrono::system_clock::now();
- const auto query_info = process_list.getQueryInfo(query_id, false, true, false);
- if (!query_info)
- {
- LOG_TRACE(logger, "Query {} is not running anymore, so we couldn't get its QueryStatusInfo", query_id);
- return;
- }
-
- auto elem = createLogMetricElement(query_id, *query_info, current_time);
- if (elem)
- add(std::move(elem.value()));
+ info.task = context->getSchedulePool().createTask("QueryMetricLog", [this, &process_list, query_id] {
+ collectMetric(process_list, query_id);
});
- std::lock_guard lock(queries_mutex);
- status.task->scheduleAfter(interval_milliseconds);
- queries.emplace(query_id, std::move(status));
+ LockGuard global_lock(queries_mutex);
+ query_status.scheduleNext(query_id);
+ queries.emplace(query_id, std::move(query_status));
}
void QueryMetricLog::finishQuery(const String & query_id, TimePoint finish_time, QueryStatusInfoPtr query_info)
{
- std::unique_lock lock(queries_mutex);
+ LockGuard global_lock(queries_mutex);
auto it = queries.find(query_id);
/// finishQuery may be called from logExceptionBeforeStart when the query has not even started
@@ -124,9 +170,19 @@ void QueryMetricLog::finishQuery(const String & query_id, TimePoint finish_time,
if (it == queries.end())
return;
+ auto & query_status = it->second;
+ decltype(query_status.mutex) query_mutex;
+ LockGuard query_lock(query_status.getMutex());
+
+ /// Move the query mutex here so that we hold it until the end, after removing the query from queries.
+ query_mutex = std::move(query_status.mutex);
+ query_status.mutex = {};
+
+ global_lock.unlock();
+
if (query_info)
{
- auto elem = createLogMetricElement(query_id, *query_info, finish_time, false);
+ auto elem = query_status.createLogMetricElement(query_id, *query_info, finish_time, false);
if (elem)
add(std::move(elem.value()));
}
@@ -139,51 +195,58 @@ void QueryMetricLog::finishQuery(const String & query_id, TimePoint finish_time,
/// that order.
{
/// Take ownership of the task so that we can destroy it in this scope after unlocking `queries_mutex`.
- auto task = std::move(it->second.task);
+ auto task = std::move(query_status.info.task);
/// Build an empty task for the old task to make sure it does not lock any mutex on its destruction.
- it->second.task = {};
+ query_status.info.task = {};
+ query_lock.unlock();
+ global_lock.lock();
queries.erase(query_id);
/// Ensure `queries_mutex` is unlocked before calling task's destructor at the end of this
/// scope which will lock `exec_mutex`.
- lock.unlock();
+ global_lock.unlock();
}
}
-std::optional QueryMetricLog::createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint query_info_time, bool schedule_next)
+void QueryMetricLogStatus::scheduleNext(String query_id)
{
- /// fmtlib supports subsecond formatting in 10.0.0. We're in 9.1.0, so we need to add the milliseconds ourselves.
- auto seconds = std::chrono::time_point_cast(query_info_time);
- auto microseconds = std::chrono::duration_cast(query_info_time - seconds).count();
- LOG_DEBUG(logger, "Collecting query_metric_log for query {} with QueryStatusInfo from {:%Y.%m.%d %H:%M:%S}.{:06}. Schedule next: {}", query_id, seconds, microseconds, schedule_next);
-
- std::unique_lock lock(queries_mutex);
- auto query_status_it = queries.find(query_id);
-
- /// The query might have finished while the scheduled task is running.
- if (query_status_it == queries.end())
+ info.next_collect_time += std::chrono::milliseconds(info.interval_milliseconds);
+ const auto now = std::chrono::system_clock::now();
+ if (info.next_collect_time > now)
{
- lock.unlock();
- LOG_TRACE(logger, "Query {} finished already while this collecting task was running", query_id);
- return {};
+ const auto wait_time = std::chrono::duration_cast(info.next_collect_time - now).count();
+ info.task->scheduleAfter(wait_time);
}
-
- auto & query_status = query_status_it->second;
- if (query_info_time <= query_status.last_collect_time)
+ else
{
- lock.unlock();
+ LOG_TRACE(logger, "The next collecting task for query {} should have already run at {}. Scheduling it right now",
+ query_id, timePointToString(info.next_collect_time));
+ info.task->schedule();
+ }
+}
+
+std::optional QueryMetricLogStatus::createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint query_info_time, bool schedule_next)
+{
+ /// TODO: remove trace before 24.11 release after checking everything is fine on the CI
+ LOG_TRACE(logger, "Collecting query_metric_log for query {} and interval {} ms with QueryStatusInfo from {}. Next collection time: {}",
+ query_id, info.interval_milliseconds, timePointToString(query_info_time),
+ schedule_next ? timePointToString(info.next_collect_time + std::chrono::milliseconds(info.interval_milliseconds)) : "finished");
+
+ if (query_info_time <= info.last_collect_time)
+ {
+ /// TODO: remove trace before 24.11 release after checking everything is fine on the CI
LOG_TRACE(logger, "Query {} has a more recent metrics collected. Skipping this one", query_id);
return {};
}
- query_status.last_collect_time = query_info_time;
+ info.last_collect_time = query_info_time;
QueryMetricLogElement elem;
elem.event_time = timeInSeconds(query_info_time);
elem.event_time_microseconds = timeInMicroseconds(query_info_time);
- elem.query_id = query_status_it->first;
+ elem.query_id = query_id;
elem.memory_usage = query_info.memory_usage > 0 ? query_info.memory_usage : 0;
elem.peak_memory_usage = query_info.peak_memory_usage > 0 ? query_info.peak_memory_usage : 0;
@@ -192,7 +255,7 @@ std::optional QueryMetricLog::createLogMetricElement(cons
for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i)
{
const auto & new_value = (*(query_info.profile_counters))[i];
- auto & old_value = query_status.last_profile_events[i];
+ auto & old_value = info.last_profile_events[i];
/// Profile event counters are supposed to be monotonic. However, at least the `NetworkReceiveBytes` can be inaccurate.
/// So, since in the future the counter should always have a bigger value than in the past, we skip this event.
@@ -208,16 +271,13 @@ std::optional QueryMetricLog::createLogMetricElement(cons
}
else
{
- LOG_TRACE(logger, "Query {} has no profile counters", query_id);
+ /// TODO: remove trace before 24.11 release after checking everything is fine on the CI
+ LOG_DEBUG(logger, "Query {} has no profile counters", query_id);
elem.profile_events = std::vector(ProfileEvents::end());
}
if (schedule_next)
- {
- query_status.next_collect_time += std::chrono::milliseconds(query_status.interval_milliseconds);
- const auto wait_time = std::chrono::duration_cast(query_status.next_collect_time - std::chrono::system_clock::now()).count();
- query_status.task->scheduleAfter(wait_time);
- }
+ scheduleNext(query_id);
return elem;
}
diff --git a/src/Interpreters/QueryMetricLog.h b/src/Interpreters/QueryMetricLog.h
index 802cee7bf26..9371dfbb6b5 100644
--- a/src/Interpreters/QueryMetricLog.h
+++ b/src/Interpreters/QueryMetricLog.h
@@ -1,5 +1,6 @@
#pragma once
+#include
#include
#include
#include
@@ -11,11 +12,17 @@
#include
#include
+#include
namespace DB
{
+namespace ErrorCodes
+{
+ extern const int LOGICAL_ERROR;
+};
+
/** QueryMetricLogElement is a log of query metric values measured at regular time interval.
*/
@@ -34,7 +41,7 @@ struct QueryMetricLogElement
void appendToBlock(MutableColumns & columns) const;
};
-struct QueryMetricLogStatus
+struct QueryMetricLogStatusInfo
{
UInt64 interval_milliseconds;
std::chrono::system_clock::time_point last_collect_time;
@@ -43,24 +50,47 @@ struct QueryMetricLogStatus
BackgroundSchedulePool::TaskHolder task;
};
+struct QueryMetricLogStatus
+{
+ using TimePoint = std::chrono::system_clock::time_point;
+ using Mutex = std::mutex;
+
+ QueryMetricLogStatusInfo info TSA_GUARDED_BY(getMutex());
+
+ /// We need to be able to move it for the hash map, so we need to add an indirection here.
+ std::unique_ptr mutex = std::make_unique();
+
+ /// Return a reference to the mutex, used for Thread Sanitizer annotations.
+ Mutex & getMutex() const TSA_RETURN_CAPABILITY(mutex)
+ {
+ if (!mutex)
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutex cannot be NULL");
+ return *mutex;
+ }
+
+ void scheduleNext(String query_id) TSA_REQUIRES(getMutex());
+ std::optional createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint query_info_time, bool schedule_next = true) TSA_REQUIRES(getMutex());
+};
+
class QueryMetricLog : public SystemLog
{
using SystemLog::SystemLog;
- using TimePoint = std::chrono::system_clock::time_point;
using Base = SystemLog;
public:
+ using TimePoint = std::chrono::system_clock::time_point;
+
void shutdown() final;
- // Both startQuery and finishQuery are called from the thread that executes the query
+ /// Both startQuery and finishQuery are called from the thread that executes the query.
void startQuery(const String & query_id, TimePoint start_time, UInt64 interval_milliseconds);
void finishQuery(const String & query_id, TimePoint finish_time, QueryStatusInfoPtr query_info = nullptr);
private:
- std::optional createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint query_info_time, bool schedule_next = true);
+ void collectMetric(const ProcessList & process_list, String query_id);
- std::recursive_mutex queries_mutex;
- std::unordered_map queries;
+ std::mutex queries_mutex;
+ std::unordered_map queries TSA_GUARDED_BY(queries_mutex);
};
}
diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp
index fa28fa04ab1..0bc1d4956a1 100644
--- a/src/Interpreters/executeQuery.cpp
+++ b/src/Interpreters/executeQuery.cpp
@@ -505,6 +505,7 @@ void logQueryFinish(
auto time_now = std::chrono::system_clock::now();
QueryStatusInfo info = process_list_elem->getInfo(true, settings[Setting::log_profile_events]);
+ logQueryMetricLogFinish(context, internal, elem.client_info.current_query_id, time_now, std::make_shared(info));
elem.type = QueryLogElementType::QUERY_FINISH;
addStatusInfoToQueryLogElement(elem, info, query_ast, context);
@@ -623,6 +624,7 @@ void logQueryException(
{
elem.query_duration_ms = start_watch.elapsedMilliseconds();
}
+ logQueryMetricLogFinish(context, internal, elem.client_info.current_query_id, time_now, info);
elem.query_cache_usage = QueryCache::Usage::None;
@@ -652,8 +654,6 @@ void logQueryException(
query_span->addAttribute("clickhouse.exception_code", elem.exception_code);
query_span->finish();
}
-
- logQueryMetricLogFinish(context, internal, elem.client_info.current_query_id, time_now, info);
}
void logExceptionBeforeStart(
@@ -707,6 +707,8 @@ void logExceptionBeforeStart(
elem.client_info = context->getClientInfo();
+ logQueryMetricLogFinish(context, false, elem.client_info.current_query_id, std::chrono::system_clock::now(), nullptr);
+
elem.log_comment = settings[Setting::log_comment];
if (elem.log_comment.size() > settings[Setting::max_query_size])
elem.log_comment.resize(settings[Setting::max_query_size]);
@@ -751,8 +753,6 @@ void logExceptionBeforeStart(
ProfileEvents::increment(ProfileEvents::FailedInsertQuery);
}
}
-
- logQueryMetricLogFinish(context, false, elem.client_info.current_query_id, std::chrono::system_clock::now(), nullptr);
}
void validateAnalyzerSettings(ASTPtr ast, bool context_value)
diff --git a/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp b/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp
index fb3ed7f80fc..1832cc2ad42 100644
--- a/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp
+++ b/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp
@@ -6,12 +6,23 @@
namespace DB
{
+namespace Setting
+{
+ extern const SettingsBool query_plan_merge_filters;
+}
+
BuildQueryPipelineSettings BuildQueryPipelineSettings::fromContext(ContextPtr from)
{
+ const auto & query_settings = from->getSettingsRef();
BuildQueryPipelineSettings settings;
- settings.actions_settings = ExpressionActionsSettings::fromSettings(from->getSettingsRef(), CompileExpressions::yes);
+ settings.actions_settings = ExpressionActionsSettings::fromSettings(query_settings, CompileExpressions::yes);
settings.process_list_element = from->getProcessListElement();
settings.progress_callback = from->getProgressCallback();
+
+ /// Setting query_plan_merge_filters is enabled by default.
+ /// But it can brake short-circuit without splitting filter step into smaller steps.
+ /// So, enable and disable this optimizations together.
+ settings.enable_multiple_filters_transforms_for_and_chain = query_settings[Setting::query_plan_merge_filters];
return settings;
}
diff --git a/src/Processors/QueryPlan/BuildQueryPipelineSettings.h b/src/Processors/QueryPlan/BuildQueryPipelineSettings.h
index d99f9a7d1f1..6219e37db58 100644
--- a/src/Processors/QueryPlan/BuildQueryPipelineSettings.h
+++ b/src/Processors/QueryPlan/BuildQueryPipelineSettings.h
@@ -17,6 +17,8 @@ using TemporaryFileLookupPtr = std::shared_ptr;
struct BuildQueryPipelineSettings
{
+ bool enable_multiple_filters_transforms_for_and_chain = true;
+
ExpressionActionsSettings actions_settings;
QueryStatusPtr process_list_element;
ProgressCallback progress_callback = nullptr;
diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp
index 862e03d74f2..af9e3f0c515 100644
--- a/src/Processors/QueryPlan/FilterStep.cpp
+++ b/src/Processors/QueryPlan/FilterStep.cpp
@@ -5,6 +5,11 @@
#include
#include
#include
+#include
+#include
+#include
+#include
+#include
namespace DB
{
@@ -24,6 +29,92 @@ static ITransformingStep::Traits getTraits()
};
}
+static bool isTrivialSubtree(const ActionsDAG::Node * node)
+{
+ while (node->type == ActionsDAG::ActionType::ALIAS)
+ node = node->children.at(0);
+
+ return node->type != ActionsDAG::ActionType::FUNCTION && node->type != ActionsDAG::ActionType::ARRAY_JOIN;
+}
+
+struct ActionsAndName
+{
+ ActionsDAG dag;
+ std::string name;
+};
+
+static ActionsAndName splitSingleAndFilter(ActionsDAG & dag, const ActionsDAG::Node * filter_node)
+{
+ auto split_result = dag.split({filter_node}, true);
+ dag = std::move(split_result.second);
+
+ const auto * split_filter_node = split_result.split_nodes_mapping[filter_node];
+ auto filter_type = removeLowCardinality(split_filter_node->result_type);
+ if (!filter_type->onlyNull() && !isUInt8(removeNullable(filter_type)))
+ {
+ DataTypePtr cast_type = std::make_shared();
+ if (filter_type->isNullable())
+ cast_type = std::make_shared(std::move(cast_type));
+
+ split_filter_node = &split_result.first.addCast(*split_filter_node, cast_type, {});
+ }
+
+ split_result.first.getOutputs().emplace(split_result.first.getOutputs().begin(), split_filter_node);
+ auto name = split_filter_node->result_name;
+ return ActionsAndName{std::move(split_result.first), std::move(name)};
+}
+
+/// Try to split the left most AND atom to a separate DAG.
+static std::optional trySplitSingleAndFilter(ActionsDAG & dag, const std::string & filter_name)
+{
+ const auto * filter = &dag.findInOutputs(filter_name);
+ while (filter->type == ActionsDAG::ActionType::ALIAS)
+ filter = filter->children.at(0);
+
+ if (filter->type != ActionsDAG::ActionType::FUNCTION || filter->function_base->getName() != "and")
+ return {};
+
+ const ActionsDAG::Node * condition_to_split = nullptr;
+ std::stack nodes;
+ nodes.push(filter);
+ while (!nodes.empty())
+ {
+ const auto * node = nodes.top();
+ nodes.pop();
+
+ if (node->type == ActionsDAG::ActionType::FUNCTION && node->function_base->getName() == "and")
+ {
+ /// The order is important. We should take the left-most atom, so put conditions on stack in reverse order.
+ for (const auto * child : node->children | std::ranges::views::reverse)
+ nodes.push(child);
+
+ continue;
+ }
+
+ if (isTrivialSubtree(node))
+ continue;
+
+ /// Do not split subtree if it's the last non-trivial one.
+ /// So, split the first found condition only when there is a another one found.
+ if (condition_to_split)
+ return splitSingleAndFilter(dag, condition_to_split);
+
+ condition_to_split = node;
+ }
+
+ return {};
+}
+
+std::vector splitAndChainIntoMultipleFilters(ActionsDAG & dag, const std::string & filter_name)
+{
+ std::vector res;
+
+ while (auto condition = trySplitSingleAndFilter(dag, filter_name))
+ res.push_back(std::move(*condition));
+
+ return res;
+}
+
FilterStep::FilterStep(
const Header & input_header_,
ActionsDAG actions_dag_,
@@ -50,6 +141,23 @@ FilterStep::FilterStep(
void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)
{
+ std::vector and_atoms;
+
+ /// Splitting AND filter condition to steps under the setting, which is enabled with merge_filters optimization.
+ /// This is needed to support short-circuit properly.
+ if (settings.enable_multiple_filters_transforms_for_and_chain && !actions_dag.hasStatefulFunctions())
+ and_atoms = splitAndChainIntoMultipleFilters(actions_dag, filter_column_name);
+
+ for (auto & and_atom : and_atoms)
+ {
+ auto expression = std::make_shared(std::move(and_atom.dag), settings.getActionsSettings());
+ pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type)
+ {
+ bool on_totals = stream_type == QueryPipelineBuilder::StreamType::Totals;
+ return std::make_shared(header, expression, and_atom.name, true, on_totals);
+ });
+ }
+
auto expression = std::make_shared(std::move(actions_dag), settings.getActionsSettings());
pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type)
@@ -76,18 +184,45 @@ void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQ
void FilterStep::describeActions(FormatSettings & settings) const
{
String prefix(settings.offset, settings.indent_char);
+
+ auto cloned_dag = actions_dag.clone();
+
+ std::vector and_atoms;
+ if (!actions_dag.hasStatefulFunctions())
+ and_atoms = splitAndChainIntoMultipleFilters(cloned_dag, filter_column_name);
+
+ for (auto & and_atom : and_atoms)
+ {
+ auto expression = std::make_shared(std::move(and_atom.dag));
+ settings.out << prefix << "AND column: " << and_atom.name << '\n';
+ expression->describeActions(settings.out, prefix);
+ }
+
settings.out << prefix << "Filter column: " << filter_column_name;
if (remove_filter_column)
settings.out << " (removed)";
settings.out << '\n';
- auto expression = std::make_shared(actions_dag.clone());
+ auto expression = std::make_shared(std::move(cloned_dag));
expression->describeActions(settings.out, prefix);
}
void FilterStep::describeActions(JSONBuilder::JSONMap & map) const
{
+ auto cloned_dag = actions_dag.clone();
+
+ std::vector and_atoms;
+ if (!actions_dag.hasStatefulFunctions())
+ and_atoms = splitAndChainIntoMultipleFilters(cloned_dag, filter_column_name);
+
+ for (auto & and_atom : and_atoms)
+ {
+ auto expression = std::make_shared(std::move(and_atom.dag));
+ map.add("AND column", and_atom.name);
+ map.add("Expression", expression->toTree());
+ }
+
map.add("Filter Column", filter_column_name);
map.add("Removes Filter", remove_filter_column);
diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h
index 6232fc7f54f..55a9d18f063 100644
--- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h
+++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h
@@ -32,7 +32,7 @@ struct QueryPlanOptimizationSettings
bool merge_expressions = true;
/// If merge-filters optimization is enabled.
- bool merge_filters = false;
+ bool merge_filters = true;
/// If filter push down optimization is enabled.
bool filter_push_down = true;
diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp
index 26936a19a20..626e43898e4 100644
--- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp
+++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp
@@ -176,6 +176,7 @@ namespace Setting
extern const SettingsBool use_skip_indexes;
extern const SettingsBool use_skip_indexes_if_final;
extern const SettingsBool use_uncompressed_cache;
+ extern const SettingsBool query_plan_merge_filters;
extern const SettingsUInt64 merge_tree_min_read_task_size;
extern const SettingsBool read_in_order_use_virtual_row;
}
@@ -208,6 +209,7 @@ static MergeTreeReaderSettings getMergeTreeReaderSettings(
.use_asynchronous_read_from_pool = settings[Setting::allow_asynchronous_read_from_io_pool_for_merge_tree]
&& (settings[Setting::max_streams_to_max_threads_ratio] > 1 || settings[Setting::max_streams_for_merge_tree_reading] > 1),
.enable_multiple_prewhere_read_steps = settings[Setting::enable_multiple_prewhere_read_steps],
+ .force_short_circuit_execution = settings[Setting::query_plan_merge_filters]
};
}
diff --git a/src/Processors/Transforms/AddingDefaultsTransform.cpp b/src/Processors/Transforms/AddingDefaultsTransform.cpp
index a15ea9d67cb..122691e4ea8 100644
--- a/src/Processors/Transforms/AddingDefaultsTransform.cpp
+++ b/src/Processors/Transforms/AddingDefaultsTransform.cpp
@@ -9,6 +9,7 @@
#include