mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge branch 'master' into enable_http_compression_default
This commit is contained in:
commit
f1b4e7ea59
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -597,6 +597,30 @@ If number of tables is greater than this value, server will throw an exception.
|
||||
<max_table_num_to_throw>400</max_table_num_to_throw>
|
||||
```
|
||||
|
||||
## 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
|
||||
<max_replicated_table_num_to_throw>400</max_replicated_table_num_to_throw>
|
||||
```
|
||||
|
||||
## 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
|
||||
<max_dictionary_num_to_throw>400</max_dictionary_num_to_throw>
|
||||
```
|
||||
|
||||
## 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
|
||||
<max_view_num_to_throw>400</max_view_num_to_throw>
|
||||
```
|
||||
|
||||
## 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
|
||||
|
@ -4773,7 +4773,7 @@ Result:
|
||||
|
||||
## toUTCTimestamp
|
||||
|
||||
Convert DateTime/DateTime64 type value from other time zone to UTC timezone timestamp
|
||||
Convert DateTime/DateTime64 type value from other time zone to UTC timezone timestamp. This function is mainly included for compatibility with Apache Spark and similar frameworks.
|
||||
|
||||
**Syntax**
|
||||
|
||||
@ -4799,14 +4799,14 @@ SELECT toUTCTimestamp(toDateTime('2023-03-16'), 'Asia/Shanghai');
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─toUTCTimestamp(toDateTime('2023-03-16'),'Asia/Shanghai')┐
|
||||
┌─toUTCTimestamp(toDateTime('2023-03-16'), 'Asia/Shanghai')┐
|
||||
│ 2023-03-15 16:00:00 │
|
||||
└─────────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## fromUTCTimestamp
|
||||
|
||||
Convert DateTime/DateTime64 type value from UTC timezone to other time zone timestamp
|
||||
Convert DateTime/DateTime64 type value from UTC timezone to other time zone timestamp. This function is mainly included for compatibility with Apache Spark and similar frameworks.
|
||||
|
||||
**Syntax**
|
||||
|
||||
@ -4832,7 +4832,7 @@ SELECT fromUTCTimestamp(toDateTime64('2023-03-16 10:00:00', 3), 'Asia/Shanghai')
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─fromUTCTimestamp(toDateTime64('2023-03-16 10:00:00',3),'Asia/Shanghai')─┐
|
||||
┌─fromUTCTimestamp(toDateTime64('2023-03-16 10:00:00',3), 'Asia/Shanghai')─┐
|
||||
│ 2023-03-16 18:00:00.000 │
|
||||
└─────────────────────────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
@ -12,9 +12,12 @@
|
||||
#include <Compression/ParallelCompressedWriteBuffer.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <Compression/CompressedReadBufferFromFile.h>
|
||||
#include <Compression/getCompressionCodecForFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/copyData.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Compression/CompressionFactory.h>
|
||||
#include <Common/TerminalSize.h>
|
||||
@ -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<UInt32>(&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<UInt32>(&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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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)
|
||||
|
@ -48,9 +48,15 @@ ASTPtr JoinNode::toASTTableJoin() const
|
||||
auto join_expression_ast = children[join_expression_child_index]->toAST();
|
||||
|
||||
if (is_using_join_expression)
|
||||
join_ast->using_expression_list = std::move(join_expression_ast);
|
||||
{
|
||||
join_ast->using_expression_list = join_expression_ast;
|
||||
join_ast->children.push_back(join_ast->using_expression_list);
|
||||
}
|
||||
else
|
||||
join_ast->on_expression = std::move(join_expression_ast);
|
||||
{
|
||||
join_ast->on_expression = join_expression_ast;
|
||||
join_ast->children.push_back(join_ast->on_expression);
|
||||
}
|
||||
}
|
||||
|
||||
return join_ast;
|
||||
|
@ -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<FunctionNode>(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;
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <memory>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include "Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.h"
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
@ -16,39 +15,39 @@
|
||||
#include <Analyzer/ColumnNode.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Analyzer/Passes/AggregateFunctionOfGroupByKeysPass.h>
|
||||
#include <Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.h>
|
||||
#include <Analyzer/Passes/ArrayExistsToHasPass.h>
|
||||
#include <Analyzer/Passes/AutoFinalOnQueryPass.h>
|
||||
#include <Analyzer/Passes/ComparisonTupleEliminationPass.h>
|
||||
#include <Analyzer/Passes/ConvertOrLikeChainPass.h>
|
||||
#include <Analyzer/Passes/ConvertQueryToCNFPass.h>
|
||||
#include <Analyzer/Passes/CountDistinctPass.h>
|
||||
#include <Analyzer/Passes/CrossToInnerJoinPass.h>
|
||||
#include <Analyzer/Passes/FunctionToSubcolumnsPass.h>
|
||||
#include <Analyzer/Passes/FuseFunctionsPass.h>
|
||||
#include <Analyzer/Passes/GroupingFunctionsResolvePass.h>
|
||||
#include <Analyzer/Passes/IfChainToMultiIfPass.h>
|
||||
#include <Analyzer/Passes/IfConstantConditionPass.h>
|
||||
#include <Analyzer/Passes/IfTransformStringsToEnumPass.h>
|
||||
#include <Analyzer/Passes/LogicalExpressionOptimizerPass.h>
|
||||
#include <Analyzer/Passes/MultiIfToIfPass.h>
|
||||
#include <Analyzer/Passes/NormalizeCountVariantsPass.h>
|
||||
#include <Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.h>
|
||||
#include <Analyzer/Passes/OptimizeGroupByFunctionKeysPass.h>
|
||||
#include <Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.h>
|
||||
#include <Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.h>
|
||||
#include <Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h>
|
||||
#include <Analyzer/Passes/OrderByTupleEliminationPass.h>
|
||||
#include <Analyzer/Passes/QueryAnalysisPass.h>
|
||||
#include <Analyzer/Passes/RemoveUnusedProjectionColumnsPass.h>
|
||||
#include <Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h>
|
||||
#include <Analyzer/Passes/CountDistinctPass.h>
|
||||
#include <Analyzer/Passes/UniqToCountPass.h>
|
||||
#include <Analyzer/Passes/FunctionToSubcolumnsPass.h>
|
||||
#include <Analyzer/Passes/RewriteAggregateFunctionWithIfPass.h>
|
||||
#include <Analyzer/Passes/SumIfToCountIfPass.h>
|
||||
#include <Analyzer/Passes/MultiIfToIfPass.h>
|
||||
#include <Analyzer/Passes/IfConstantConditionPass.h>
|
||||
#include <Analyzer/Passes/IfChainToMultiIfPass.h>
|
||||
#include <Analyzer/Passes/OrderByTupleEliminationPass.h>
|
||||
#include <Analyzer/Passes/NormalizeCountVariantsPass.h>
|
||||
#include <Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.h>
|
||||
#include <Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.h>
|
||||
#include <Analyzer/Passes/OrderByLimitByDuplicateEliminationPass.h>
|
||||
#include <Analyzer/Passes/FuseFunctionsPass.h>
|
||||
#include <Analyzer/Passes/OptimizeGroupByFunctionKeysPass.h>
|
||||
#include <Analyzer/Passes/IfTransformStringsToEnumPass.h>
|
||||
#include <Analyzer/Passes/ConvertOrLikeChainPass.h>
|
||||
#include <Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.h>
|
||||
#include <Analyzer/Passes/GroupingFunctionsResolvePass.h>
|
||||
#include <Analyzer/Passes/AutoFinalOnQueryPass.h>
|
||||
#include <Analyzer/Passes/ArrayExistsToHasPass.h>
|
||||
#include <Analyzer/Passes/ComparisonTupleEliminationPass.h>
|
||||
#include <Analyzer/Passes/LogicalExpressionOptimizerPass.h>
|
||||
#include <Analyzer/Passes/CrossToInnerJoinPass.h>
|
||||
#include <Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h>
|
||||
#include <Analyzer/Passes/ShardNumColumnToFunctionPass.h>
|
||||
#include <Analyzer/Passes/ConvertQueryToCNFPass.h>
|
||||
#include <Analyzer/Passes/AggregateFunctionOfGroupByKeysPass.h>
|
||||
#include <Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.h>
|
||||
|
||||
#include <Analyzer/Passes/SumIfToCountIfPass.h>
|
||||
#include <Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.h>
|
||||
#include <Analyzer/Passes/UniqToCountPass.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Interpreters/ProcessorsProfileLog.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
@ -676,6 +677,8 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden
|
||||
"tuple"});
|
||||
}
|
||||
}
|
||||
|
||||
logProcessorProfile(context, io.pipeline.getProcessors());
|
||||
}
|
||||
|
||||
scalars_cache.emplace(node_with_hash, scalar_block);
|
||||
|
@ -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)
|
||||
|
@ -1,7 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/UUID.h>
|
||||
#include <base/defines.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <base/types.h>
|
||||
#include <mutex>
|
||||
#include <unordered_map>
|
||||
|
||||
@ -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<UUID /* backup_uuid */, size_t /* num_refs */> on_cluster_backups TSA_GUARDED_BY(mutex);
|
||||
std::unordered_map<UUID /* restore_uuid */, size_t /* num_refs */> on_cluster_restores TSA_GUARDED_BY(mutex);
|
||||
std::unordered_map<String /* zookeeper_path */, size_t /* num_refs */> on_cluster_backups TSA_GUARDED_BY(mutex);
|
||||
std::unordered_map<String /* zookeeper_path */, size_t /* num_refs */> on_cluster_restores TSA_GUARDED_BY(mutex);
|
||||
std::mutex mutex;
|
||||
};
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
@ -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_)
|
||||
{
|
||||
}
|
||||
|
@ -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<PartNameAndChecksum> & part_names_and_checksums) override;
|
||||
|
@ -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
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Backups/IBackupCoordination.h>
|
||||
#include <Backups/BackupConcurrencyCheck.h>
|
||||
#include <Backups/BackupCoordinationCleaner.h>
|
||||
#include <Backups/BackupCoordinationFileInfos.h>
|
||||
#include <Backups/BackupCoordinationReplicatedAccess.h>
|
||||
@ -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<bool> backup_query_was_sent_to_other_hosts = false;
|
||||
BackupCoordinationStageSync stage_sync;
|
||||
|
||||
mutable std::optional<BackupCoordinationReplicatedTables> replicated_tables TSA_GUARDED_BY(replicated_tables_mutex);
|
||||
mutable std::optional<BackupCoordinationReplicatedAccess> replicated_access TSA_GUARDED_BY(replicated_access_mutex);
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -1,7 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <Backups/BackupConcurrencyCheck.h>
|
||||
#include <Backups/WithRetries.h>
|
||||
|
||||
|
||||
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<void> 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<std::chrono::milliseconds> 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<std::exception_ptr, String> parseErrorNode(const String & error_node_contents);
|
||||
std::pair<std::exception_ptr, String> 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<std::chrono::milliseconds> 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<std::chrono::seconds> timeout) const;
|
||||
bool checkIfOtherHostsFinish(const String & reason, bool throw_if_error, bool time_is_out, std::optional<std::chrono::milliseconds> timeout) const TSA_REQUIRES(mutex);
|
||||
bool waitOtherHostsFinishImpl(const String & reason, std::optional<std::chrono::seconds> timeout, bool throw_if_error) const;
|
||||
bool checkIfOtherHostsFinish(const String & reason, std::optional<std::chrono::milliseconds> 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<BackupConcurrencyCheck> concurrency_check;
|
||||
|
||||
std::shared_ptr<Poco::Event> zk_nodes_changed;
|
||||
|
||||
@ -176,25 +201,21 @@ private:
|
||||
{
|
||||
std::map<String /* host */, HostInfo> hosts; /// std::map because we need to compare states
|
||||
std::optional<String> 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<void> watching_thread_future;
|
||||
std::atomic<bool> 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;
|
||||
};
|
||||
|
@ -329,6 +329,7 @@ std::pair<OperationID, BackupStatus> BackupsWorker::start(const ASTPtr & backup_
|
||||
struct BackupsWorker::BackupStarter
|
||||
{
|
||||
BackupsWorker & backups_worker;
|
||||
LoggerPtr log;
|
||||
std::shared_ptr<ASTBackupQuery> 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<ASTBackupQuery>(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<ASTBackupQuery> & backup_query,
|
||||
const OperationID & backup_id,
|
||||
const String & backup_name_for_logging,
|
||||
const BackupSettings & backup_settings,
|
||||
std::shared_ptr<IBackupCoordination> 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<ASTBackupQuery> 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<ASTBackupQuery>(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<ASTBackupQuery> & restore_query,
|
||||
const OperationID & restore_id,
|
||||
const String & backup_name_for_logging,
|
||||
const BackupInfo & backup_info,
|
||||
RestoreSettings restore_settings,
|
||||
std::shared_ptr<IRestoreCoordination> 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<BackupCoordinationLocal>(
|
||||
*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<RestoreCoordinationLocal>(
|
||||
*restore_settings.restore_uuid, allow_concurrent_restores, *concurrency_counters);
|
||||
return std::make_shared<RestoreCoordinationLocal>(allow_concurrent_restores, *concurrency_counters);
|
||||
}
|
||||
|
||||
bool is_internal_restore = restore_settings.internal;
|
||||
|
@ -81,7 +81,6 @@ private:
|
||||
BackupMutablePtr backup,
|
||||
const std::shared_ptr<ASTBackupQuery> & backup_query,
|
||||
const BackupOperationID & backup_id,
|
||||
const String & backup_name_for_logging,
|
||||
const BackupSettings & backup_settings,
|
||||
std::shared_ptr<IBackupCoordination> backup_coordination,
|
||||
ContextMutablePtr context,
|
||||
@ -102,7 +101,6 @@ private:
|
||||
void doRestore(
|
||||
const std::shared_ptr<ASTBackupQuery> & restore_query,
|
||||
const BackupOperationID & restore_id,
|
||||
const String & backup_name_for_logging,
|
||||
const BackupInfo & backup_info,
|
||||
RestoreSettings restore_settings,
|
||||
std::shared_ptr<IRestoreCoordination> restore_coordination,
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Backups/IRestoreCoordination.h>
|
||||
#include <Backups/BackupConcurrencyCheck.h>
|
||||
#include <Backups/BackupCoordinationCleaner.h>
|
||||
#include <Backups/BackupCoordinationStageSync.h>
|
||||
#include <Backups/WithRetries.h>
|
||||
@ -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<bool> restore_query_was_sent_to_other_hosts = false;
|
||||
BackupCoordinationStageSync stage_sync;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -140,8 +140,6 @@ void highlight(const String & query, std::vector<replxx::Replxx::Color> & colors
|
||||
/// We don't do highlighting for foreign dialects, such as PRQL and Kusto.
|
||||
/// Only normal ClickHouse SQL queries are highlighted.
|
||||
|
||||
/// Currently we highlight only the first query in the multi-query mode.
|
||||
|
||||
ParserQuery parser(end, false, context.getSettingsRef()[Setting::implicit_select]);
|
||||
ASTPtr ast;
|
||||
bool parse_res = false;
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Common/MemoryStatisticsOS.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
|
||||
#include <condition_variable>
|
||||
@ -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);
|
||||
|
@ -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.") \
|
||||
|
@ -87,6 +87,7 @@ APPLY_FOR_FAILPOINTS(M, M, M, M)
|
||||
|
||||
std::unordered_map<String, std::shared_ptr<FailPointChannel>> FailPointInjection::fail_point_wait_channels;
|
||||
std::mutex FailPointInjection::mu;
|
||||
|
||||
class FailPointChannel : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
|
@ -15,6 +15,7 @@
|
||||
|
||||
#include <unordered_map>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -27,6 +28,7 @@ namespace DB
|
||||
/// 3. in test file, we can use system failpoint enable/disable 'failpoint_name'
|
||||
|
||||
class FailPointChannel;
|
||||
|
||||
class FailPointInjection
|
||||
{
|
||||
public:
|
||||
|
@ -9,6 +9,7 @@
|
||||
|
||||
#include <mutex>
|
||||
#include <algorithm>
|
||||
#include <Poco/Timespan.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
@ -49,16 +50,18 @@ HostResolver::WeakPtr HostResolver::getWeakFromThis()
|
||||
}
|
||||
|
||||
HostResolver::HostResolver(String host_, Poco::Timespan history_)
|
||||
: host(std::move(host_))
|
||||
, history(history_)
|
||||
, resolve_function([](const String & host_to_resolve) { return DNSResolver::instance().resolveHostAllInOriginOrder(host_to_resolve); })
|
||||
{
|
||||
update();
|
||||
}
|
||||
: HostResolver(
|
||||
[](const String & host_to_resolve) { return DNSResolver::instance().resolveHostAllInOriginOrder(host_to_resolve); },
|
||||
host_,
|
||||
history_)
|
||||
{}
|
||||
|
||||
HostResolver::HostResolver(
|
||||
ResolveFunction && resolve_function_, String host_, Poco::Timespan history_)
|
||||
: host(std::move(host_)), history(history_), resolve_function(std::move(resolve_function_))
|
||||
: host(std::move(host_))
|
||||
, history(history_)
|
||||
, resolve_interval(history_.totalMicroseconds() / 3)
|
||||
, resolve_function(std::move(resolve_function_))
|
||||
{
|
||||
update();
|
||||
}
|
||||
@ -203,7 +206,7 @@ bool HostResolver::isUpdateNeeded()
|
||||
Poco::Timestamp now;
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
return last_resolve_time + history < now || records.empty();
|
||||
return last_resolve_time + resolve_interval < now || records.empty();
|
||||
}
|
||||
|
||||
void HostResolver::updateImpl(Poco::Timestamp now, std::vector<Poco::Net::IPAddress> & next_gen)
|
||||
|
@ -26,7 +26,7 @@
|
||||
// a) it still occurs in resolve set after `history_` time or b) all other addresses are pessimized as well.
|
||||
// - resolve schedule
|
||||
// Addresses are resolved through `DB::DNSResolver::instance()`.
|
||||
// Usually it does not happen more often than once in `history_` time.
|
||||
// Usually it does not happen more often than 3 times in `history_` period.
|
||||
// But also new resolve performed each `setFail()` call.
|
||||
|
||||
namespace DB
|
||||
@ -212,6 +212,7 @@ protected:
|
||||
|
||||
const String host;
|
||||
const Poco::Timespan history;
|
||||
const Poco::Timespan resolve_interval;
|
||||
const HostResolverMetrics metrics = getMetrics();
|
||||
|
||||
// for tests purpose
|
||||
@ -245,4 +246,3 @@ private:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -1,23 +1,47 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/OvercommitTracker.h>
|
||||
#include <base/defines.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/OvercommitTracker.h>
|
||||
|
||||
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 <typename Mutex>
|
||||
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 <template<typename> typename TLockGuard, typename Mutex>
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <Compression/ICompressionCodec.h>
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
#include <Parsers/StringRange.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -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<char> 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<UInt32>(&compressed_buffer[1]);
|
||||
size_decompressed = unalignedLoad<UInt32>(&compressed_buffer[5]);
|
||||
if (method == static_cast<uint8_t>(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<CompressionCodecMultiple>(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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -7,7 +7,6 @@
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/IParser.h>
|
||||
#include <Parsers/TokenIterator.h>
|
||||
#include <base/types.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
|
||||
|
@ -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) \
|
||||
|
@ -3669,6 +3669,11 @@ Given that, for example, dictionaries, can be out of sync across nodes, mutation
|
||||
|
||||
</profiles>
|
||||
```
|
||||
)", 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.
|
||||
@ -4861,9 +4866,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) \
|
||||
@ -4875,6 +4880,9 @@ Limit on size of a single batch of file segments that a read buffer can request
|
||||
)", 0) \
|
||||
DECLARE(UInt64, filesystem_cache_reserve_space_wait_lock_timeout_milliseconds, 1000, R"(
|
||||
Wait time to lock cache for space reservation in filesystem cache
|
||||
)", 0) \
|
||||
DECLARE(Bool, filesystem_cache_prefer_bigger_buffer_size, true, R"(
|
||||
Prefer bigger buffer size if filesystem cache is enabled to avoid writing small file segments which deteriorate cache performance. On the other hand, enabling this setting might increase memory usage.
|
||||
)", 0) \
|
||||
DECLARE(UInt64, temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds, (10 * 60 * 1000), R"(
|
||||
Wait time to lock cache for space reservation for temporary data in filesystem cache
|
||||
|
@ -65,6 +65,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
|
||||
{"24.11",
|
||||
{
|
||||
{"enable_http_compression", false, true, "Improvement for read-only clients since they can't change settings"},
|
||||
{"validate_mutation_query", false, true, "New setting to validate mutation queries by default."},
|
||||
{"enable_job_stack_trace", false, true, "Enable by default collecting stack traces from job's scheduling."},
|
||||
{"allow_suspicious_types_in_group_by", true, false, "Don't allow Variant/Dynamic types in GROUP BY by default"},
|
||||
{"allow_suspicious_types_in_order_by", true, false, "Don't allow Variant/Dynamic types in ORDER BY by default"},
|
||||
@ -78,6 +79,8 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
|
||||
{"backup_restore_keeper_max_retries_while_handling_error", 0, 20, "New setting."},
|
||||
{"backup_restore_finish_timeout_after_error_sec", 0, 180, "New setting."},
|
||||
{"parallel_replicas_local_plan", false, true, "Use local plan for local replica in a query with parallel replicas"},
|
||||
{"filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit", 1, 1, "Rename of setting skip_download_if_exceeds_query_cache_limit"},
|
||||
{"filesystem_cache_prefer_bigger_buffer_size", true, true, "New setting"},
|
||||
{"read_in_order_use_virtual_row", false, false, "Use virtual row while reading in order of primary key or its monotonic function fashion. It is useful when searching over multiple parts as only relevant ones are touched."},
|
||||
}
|
||||
},
|
||||
@ -124,7 +127,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
|
||||
{"allow_experimental_refreshable_materialized_view", false, true, "Not experimental anymore"},
|
||||
{"max_parts_to_move", 0, 1000, "New setting"},
|
||||
{"hnsw_candidate_list_size_for_search", 64, 256, "New setting. Previously, the value was optionally specified in CREATE INDEX and 64 by default."},
|
||||
{"allow_reorder_prewhere_conditions", false, true, "New setting"},
|
||||
{"allow_reorder_prewhere_conditions", true, true, "New setting"},
|
||||
{"input_format_parquet_bloom_filter_push_down", false, true, "When reading Parquet files, skip whole row groups based on the WHERE/PREWHERE expressions and bloom filter in the Parquet metadata."},
|
||||
{"date_time_64_output_format_cut_trailing_zeros_align_to_groups_of_thousands", false, false, "Dynamically trim the trailing zeros of datetime64 values to adjust the output scale to (0, 3, 6), corresponding to 'seconds', 'milliseconds', and 'microseconds'."},
|
||||
}
|
||||
|
@ -382,7 +382,8 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n
|
||||
if (!table_storage->isSystemStorage() && !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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -642,7 +642,10 @@ std::unique_ptr<ReadBufferFromFileBase> DiskObjectStorage::readFile(
|
||||
};
|
||||
|
||||
/// Avoid cache fragmentation by choosing bigger buffer size.
|
||||
bool prefer_bigger_buffer_size = object_storage->supportsCache() && read_settings.enable_filesystem_cache;
|
||||
bool prefer_bigger_buffer_size = read_settings.filesystem_cache_prefer_bigger_buffer_size
|
||||
&& object_storage->supportsCache()
|
||||
&& read_settings.enable_filesystem_cache;
|
||||
|
||||
size_t buffer_size = prefer_bigger_buffer_size
|
||||
? std::max<size_t>(settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE)
|
||||
: settings.remote_fs_buffer_size;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -81,7 +81,7 @@ std::unique_ptr<WriteBufferFromFileBase> LocalObjectStorage::writeObject( /// NO
|
||||
return std::make_unique<WriteBufferFromFile>(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);
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
||||
|
@ -254,16 +254,6 @@ std::unique_ptr<WriteBufferFromFileBase> WebObjectStorage::writeObject( /// NOLI
|
||||
throwNotAllowed();
|
||||
}
|
||||
|
||||
void WebObjectStorage::removeObject(const StoredObject &)
|
||||
{
|
||||
throwNotAllowed();
|
||||
}
|
||||
|
||||
void WebObjectStorage::removeObjects(const StoredObjects &)
|
||||
{
|
||||
throwNotAllowed();
|
||||
}
|
||||
|
||||
void WebObjectStorage::removeObjectIfExists(const StoredObject &)
|
||||
{
|
||||
throwNotAllowed();
|
||||
|
@ -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;
|
||||
|
@ -10,7 +10,6 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/parseDateTimeBestEffort.h>
|
||||
#include <Parsers/TokenIterator.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
|
@ -61,6 +61,7 @@ struct ReadSettings
|
||||
bool filesystem_cache_allow_background_download = true;
|
||||
bool filesystem_cache_allow_background_download_for_metadata_files_in_packed_storage = true;
|
||||
bool filesystem_cache_allow_background_download_during_fetch = true;
|
||||
bool filesystem_cache_prefer_bigger_buffer_size = true;
|
||||
|
||||
bool use_page_cache_for_disks_without_file_cache = false;
|
||||
bool read_from_page_cache_if_exists_otherwise_bypass_cache = false;
|
||||
@ -68,7 +69,7 @@ struct ReadSettings
|
||||
std::shared_ptr<PageCache> 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;
|
||||
|
||||
|
@ -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())
|
||||
|
@ -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)
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Interpreters/Cache/FileCacheSettings.h>
|
||||
#include <Interpreters/Cache/LRUFileCachePriority.h>
|
||||
#include <Interpreters/Cache/SLRUFileCachePriority.h>
|
||||
#include <Interpreters/Cache/FileCacheUtils.h>
|
||||
#include <Interpreters/Cache/EvictionCandidates.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <base/hex.h>
|
||||
@ -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)
|
||||
{
|
||||
|
@ -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<size_t> max_file_segment_size;
|
||||
const size_t bypass_cache_threshold;
|
||||
const size_t boundary_alignment;
|
||||
std::atomic<size_t> background_download_max_file_segment_size;
|
||||
size_t load_metadata_threads;
|
||||
const bool load_metadata_asynchronously;
|
||||
std::atomic<bool> stop_loading_metadata = false;
|
||||
|
@ -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");
|
||||
|
||||
|
@ -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);
|
||||
|
||||
|
17
src/Interpreters/Cache/FileCacheUtils.h
Normal file
17
src/Interpreters/Cache/FileCacheUtils.h
Normal file
@ -0,0 +1,17 @@
|
||||
#pragma once
|
||||
#include <Core/Types.h>
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
@ -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;
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Interpreters/Cache/FileCache.h>
|
||||
#include <Interpreters/Cache/FileCacheUtils.h>
|
||||
#include <base/getThreadId.h>
|
||||
#include <base/hex.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
@ -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<Microseconds> 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);
|
||||
|
@ -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 &);
|
||||
|
@ -676,13 +676,17 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optional<Memor
|
||||
log, "Downloading {} bytes for file segment {}",
|
||||
file_segment.range().size() - file_segment.getDownloadedSize(), file_segment.getInfoForLog());
|
||||
|
||||
size_t size_to_download = file_segment.getSizeForBackgroundDownload();
|
||||
if (!size_to_download)
|
||||
return;
|
||||
|
||||
auto reader = file_segment.getRemoteFileReader();
|
||||
if (!reader)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR, "No reader. "
|
||||
"File segment should not have been submitted for background download ({})",
|
||||
file_segment.getInfoForLog());
|
||||
LOG_TEST(log, "No reader in {}:{} (state: {}, range: {}, downloaded size: {})",
|
||||
file_segment.key(), file_segment.offset(), file_segment.state(),
|
||||
file_segment.range().toString(), file_segment.getDownloadedSize());
|
||||
return;
|
||||
}
|
||||
|
||||
/// If remote_fs_read_method == 'threadpool',
|
||||
@ -690,7 +694,7 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optional<Memor
|
||||
if (reader->internalBuffer().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<Memor
|
||||
if (offset != static_cast<size_t>(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<Memor
|
||||
"for {}:{} (downloaded size: {}/{})",
|
||||
file_segment.key(), file_segment.offset(),
|
||||
file_segment.getDownloadedSize(), file_segment.range().size());
|
||||
return;
|
||||
break;
|
||||
}
|
||||
|
||||
try
|
||||
@ -728,12 +736,14 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optional<Memor
|
||||
if (code == /* No space left on device */28 || code == /* Quota exceeded */122)
|
||||
{
|
||||
LOG_INFO(log, "Insert into cache is skipped due to insufficient disk space. ({})", e.displayText());
|
||||
return;
|
||||
break;
|
||||
}
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
file_segment.resetRemoteFileReader();
|
||||
|
||||
LOG_TEST(log, "Downloaded file segment: {}", file_segment.getInfoForLog());
|
||||
}
|
||||
|
||||
@ -1155,7 +1165,7 @@ std::vector<FileSegment::Info> 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;
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -53,7 +53,7 @@ FileCacheQueryLimit::QueryContextPtr FileCacheQueryLimit::getOrSetQueryContext(
|
||||
{
|
||||
it->second = std::make_shared<QueryContext>(
|
||||
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;
|
||||
|
@ -196,6 +196,7 @@ namespace Setting
|
||||
extern const SettingsUInt64 filesystem_cache_segments_batch_size;
|
||||
extern const SettingsBool filesystem_cache_enable_background_download_for_metadata_files_in_packed_storage;
|
||||
extern const SettingsBool filesystem_cache_enable_background_download_during_fetch;
|
||||
extern const SettingsBool filesystem_cache_prefer_bigger_buffer_size;
|
||||
extern const SettingsBool http_make_head_request;
|
||||
extern const SettingsUInt64 http_max_fields;
|
||||
extern const SettingsUInt64 http_max_field_name_size;
|
||||
@ -236,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;
|
||||
@ -5751,9 +5752,10 @@ ReadSettings Context::getReadSettings() const
|
||||
res.filesystem_cache_allow_background_download_for_metadata_files_in_packed_storage
|
||||
= settings_ref[Setting::filesystem_cache_enable_background_download_for_metadata_files_in_packed_storage];
|
||||
res.filesystem_cache_allow_background_download_during_fetch = settings_ref[Setting::filesystem_cache_enable_background_download_during_fetch];
|
||||
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];
|
||||
|
@ -5,9 +5,11 @@
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/ProcessorsProfileLog.h>
|
||||
#include <Interpreters/addTypeConversionToAST.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
@ -19,9 +21,8 @@
|
||||
#include <Parsers/ASTWithElement.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
@ -67,6 +68,18 @@ bool ExecuteScalarSubqueriesMatcher::needChildVisit(ASTPtr & node, const ASTPtr
|
||||
return false;
|
||||
}
|
||||
|
||||
if (auto * tables = node->as<ASTTablesInSelectQueryElement>())
|
||||
{
|
||||
/// Contrary to what's said in the code block above, ARRAY JOIN needs to resolve the subquery if possible
|
||||
/// and assign an alias for 02367_optimize_trivial_count_with_array_join to pass. Otherwise it will fail in
|
||||
/// ArrayJoinedColumnsVisitor (`No alias for non-trivial value in ARRAY JOIN: _a`)
|
||||
/// This looks 100% as a incomplete code working on top of a bug, but this code has already been made obsolete
|
||||
/// by the new analyzer, so it's an inconvenience we can live with until we deprecate it.
|
||||
if (child == tables->array_join)
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -246,6 +259,8 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr
|
||||
|
||||
if (tmp_block.rows() != 0)
|
||||
throw Exception(ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY, "Scalar subquery returned more than one row");
|
||||
|
||||
logProcessorProfile(data.getContext(), io.pipeline.getProcessors());
|
||||
}
|
||||
|
||||
block = materializeBlock(block);
|
||||
|
@ -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<StorageReplicatedMergeTree *>(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)
|
||||
{
|
||||
|
@ -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.
|
||||
|
@ -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);
|
||||
|
@ -1,21 +1,22 @@
|
||||
#include <chrono>
|
||||
#include <variant>
|
||||
#include <Interpreters/PreparedSets.h>
|
||||
#include <Interpreters/Set.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Processors/QueryPlan/CreatingSetsStep.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/PreparedSets.h>
|
||||
#include <Interpreters/ProcessorsProfileLog.h>
|
||||
#include <Interpreters/Set.h>
|
||||
#include <Processors/Executors/CompletedPipelineExecutor.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
#include <Processors/QueryPlan/CreatingSetsStep.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/Sinks/EmptySink.h>
|
||||
#include <Processors/Sinks/NullSink.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Core/Block.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <QueryPipeline/SizeLimits.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -239,6 +240,8 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context)
|
||||
if (!set_and_key->set->isCreated())
|
||||
return nullptr;
|
||||
|
||||
logProcessorProfile(context, pipeline.getProcessors());
|
||||
|
||||
return set_and_key->set;
|
||||
}
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Interpreters/ProcessorsProfileLog.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
@ -8,16 +9,19 @@
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
#include <array>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace Setting
|
||||
{
|
||||
extern const SettingsBool log_processors_profiles;
|
||||
}
|
||||
|
||||
ColumnsDescription ProcessorProfileLogElement::getColumnsDescription()
|
||||
{
|
||||
return ColumnsDescription
|
||||
@ -81,5 +85,57 @@ void ProcessorProfileLogElement::appendToBlock(MutableColumns & columns) const
|
||||
columns[i++]->insert(output_bytes);
|
||||
}
|
||||
|
||||
void logProcessorProfile(ContextPtr context, const Processors & processors)
|
||||
{
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
if (settings[Setting::log_processors_profiles])
|
||||
{
|
||||
if (auto processors_profile_log = context->getProcessorsProfileLog())
|
||||
{
|
||||
ProcessorProfileLogElement processor_elem;
|
||||
|
||||
const auto time_now = std::chrono::system_clock::now();
|
||||
processor_elem.event_time = timeInSeconds(time_now);
|
||||
processor_elem.event_time_microseconds = timeInMicroseconds(time_now);
|
||||
processor_elem.initial_query_id = context->getInitialQueryId();
|
||||
processor_elem.query_id = context->getCurrentQueryId();
|
||||
|
||||
auto get_proc_id = [](const IProcessor & proc) -> UInt64 { return reinterpret_cast<std::uintptr_t>(&proc); };
|
||||
|
||||
for (const auto & processor : processors)
|
||||
{
|
||||
std::vector<UInt64> parents;
|
||||
for (const auto & port : processor->getOutputs())
|
||||
{
|
||||
if (!port.isConnected())
|
||||
continue;
|
||||
const IProcessor & next = port.getInputPort().getProcessor();
|
||||
parents.push_back(get_proc_id(next));
|
||||
}
|
||||
|
||||
processor_elem.id = get_proc_id(*processor);
|
||||
processor_elem.parent_ids = std::move(parents);
|
||||
|
||||
processor_elem.plan_step = reinterpret_cast<std::uintptr_t>(processor->getQueryPlanStep());
|
||||
processor_elem.plan_step_name = processor->getPlanStepName();
|
||||
processor_elem.plan_step_description = processor->getPlanStepDescription();
|
||||
processor_elem.plan_group = processor->getQueryPlanStepGroup();
|
||||
|
||||
processor_elem.processor_name = processor->getName();
|
||||
|
||||
processor_elem.elapsed_us = static_cast<UInt64>(processor->getElapsedNs() / 1000U);
|
||||
processor_elem.input_wait_elapsed_us = static_cast<UInt64>(processor->getInputWaitElapsedNs() / 1000U);
|
||||
processor_elem.output_wait_elapsed_us = static_cast<UInt64>(processor->getOutputWaitElapsedNs() / 1000U);
|
||||
|
||||
auto stats = processor->getProcessorDataStats();
|
||||
processor_elem.input_rows = stats.input_rows;
|
||||
processor_elem.input_bytes = stats.input_bytes;
|
||||
processor_elem.output_rows = stats.output_rows;
|
||||
processor_elem.output_bytes = stats.output_bytes;
|
||||
|
||||
processors_profile_log->add(processor_elem);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -50,4 +50,5 @@ public:
|
||||
using SystemLog<ProcessorProfileLogElement>::SystemLog;
|
||||
};
|
||||
|
||||
void logProcessorProfile(ContextPtr context, const Processors & processors);
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
#include <Common/LockGuard.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
@ -16,7 +17,6 @@
|
||||
|
||||
#include <chrono>
|
||||
#include <fmt/chrono.h>
|
||||
#include <mutex>
|
||||
|
||||
|
||||
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<std::chrono::seconds>(time);
|
||||
auto microseconds = std::chrono::duration_cast<std::chrono::microseconds>(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<QueryMetricLogElement> 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<std::chrono::seconds>(query_info_time);
|
||||
auto microseconds = std::chrono::duration_cast<std::chrono::microseconds>(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<std::chrono::milliseconds>(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<QueryMetricLogElement> 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<QueryMetricLogElement> 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<QueryMetricLogElement> 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::Count>(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<std::chrono::milliseconds>(query_status.next_collect_time - std::chrono::system_clock::now()).count();
|
||||
query_status.task->scheduleAfter(wait_time);
|
||||
}
|
||||
scheduleNext(query_id);
|
||||
|
||||
return elem;
|
||||
}
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/defines.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
@ -11,11 +12,17 @@
|
||||
|
||||
#include <chrono>
|
||||
#include <ctime>
|
||||
#include <mutex>
|
||||
|
||||
|
||||
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> mutex = std::make_unique<Mutex>();
|
||||
|
||||
/// 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<QueryMetricLogElement> createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint query_info_time, bool schedule_next = true) TSA_REQUIRES(getMutex());
|
||||
};
|
||||
|
||||
class QueryMetricLog : public SystemLog<QueryMetricLogElement>
|
||||
{
|
||||
using SystemLog<QueryMetricLogElement>::SystemLog;
|
||||
using TimePoint = std::chrono::system_clock::time_point;
|
||||
using Base = SystemLog<QueryMetricLogElement>;
|
||||
|
||||
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<QueryMetricLogElement> 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<String, QueryMetricLogStatus> queries;
|
||||
std::mutex queries_mutex;
|
||||
std::unordered_map<String, QueryMetricLogStatus> queries TSA_GUARDED_BY(queries_mutex);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -161,7 +161,13 @@ void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr &
|
||||
{
|
||||
auto & join = node.table_join->as<ASTTableJoin &>();
|
||||
if (join.on_expression)
|
||||
{
|
||||
ASTPtr original_on_expression = join.on_expression;
|
||||
visit(join.on_expression, data);
|
||||
if (join.on_expression != original_on_expression)
|
||||
join.children = { join.on_expression };
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -6,6 +6,12 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCode
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
void RewriteArrayExistsFunctionMatcher::visit(ASTPtr & ast, Data & data)
|
||||
{
|
||||
if (auto * func = ast->as<ASTFunction>())
|
||||
@ -20,21 +26,21 @@ void RewriteArrayExistsFunctionMatcher::visit(ASTPtr & ast, Data & data)
|
||||
if (join->using_expression_list)
|
||||
{
|
||||
auto * it = std::find(join->children.begin(), join->children.end(), join->using_expression_list);
|
||||
if (it == join->children.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not find join->using_expression_list in '{}'", join->formatForLogging());
|
||||
|
||||
visit(join->using_expression_list, data);
|
||||
|
||||
if (it && *it != join->using_expression_list)
|
||||
*it = join->using_expression_list;
|
||||
*it = join->using_expression_list;
|
||||
}
|
||||
|
||||
if (join->on_expression)
|
||||
{
|
||||
auto * it = std::find(join->children.begin(), join->children.end(), join->on_expression);
|
||||
if (it == join->children.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not find join->on_expression in '{}'", join->formatForLogging());
|
||||
|
||||
visit(join->on_expression, data);
|
||||
|
||||
if (it && *it != join->on_expression)
|
||||
*it = join->on_expression;
|
||||
*it = join->on_expression;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -117,7 +117,6 @@ namespace Setting
|
||||
extern const SettingsOverflowMode join_overflow_mode;
|
||||
extern const SettingsString log_comment;
|
||||
extern const SettingsBool log_formatted_queries;
|
||||
extern const SettingsBool log_processors_profiles;
|
||||
extern const SettingsBool log_profile_events;
|
||||
extern const SettingsUInt64 log_queries_cut_to_length;
|
||||
extern const SettingsBool log_queries;
|
||||
@ -506,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<QueryStatusInfo>(info));
|
||||
elem.type = QueryLogElementType::QUERY_FINISH;
|
||||
|
||||
addStatusInfoToQueryLogElement(elem, info, query_ast, context);
|
||||
@ -551,53 +551,8 @@ void logQueryFinish(
|
||||
if (auto query_log = context->getQueryLog())
|
||||
query_log->add(elem);
|
||||
}
|
||||
if (settings[Setting::log_processors_profiles])
|
||||
{
|
||||
if (auto processors_profile_log = context->getProcessorsProfileLog())
|
||||
{
|
||||
ProcessorProfileLogElement processor_elem;
|
||||
processor_elem.event_time = elem.event_time;
|
||||
processor_elem.event_time_microseconds = elem.event_time_microseconds;
|
||||
processor_elem.initial_query_id = elem.client_info.initial_query_id;
|
||||
processor_elem.query_id = elem.client_info.current_query_id;
|
||||
|
||||
auto get_proc_id = [](const IProcessor & proc) -> UInt64 { return reinterpret_cast<std::uintptr_t>(&proc); };
|
||||
|
||||
for (const auto & processor : query_pipeline.getProcessors())
|
||||
{
|
||||
std::vector<UInt64> parents;
|
||||
for (const auto & port : processor->getOutputs())
|
||||
{
|
||||
if (!port.isConnected())
|
||||
continue;
|
||||
const IProcessor & next = port.getInputPort().getProcessor();
|
||||
parents.push_back(get_proc_id(next));
|
||||
}
|
||||
|
||||
processor_elem.id = get_proc_id(*processor);
|
||||
processor_elem.parent_ids = std::move(parents);
|
||||
|
||||
processor_elem.plan_step = reinterpret_cast<std::uintptr_t>(processor->getQueryPlanStep());
|
||||
processor_elem.plan_step_name = processor->getPlanStepName();
|
||||
processor_elem.plan_step_description = processor->getPlanStepDescription();
|
||||
processor_elem.plan_group = processor->getQueryPlanStepGroup();
|
||||
|
||||
processor_elem.processor_name = processor->getName();
|
||||
|
||||
processor_elem.elapsed_us = static_cast<UInt64>(processor->getElapsedNs() / 1000U);
|
||||
processor_elem.input_wait_elapsed_us = static_cast<UInt64>(processor->getInputWaitElapsedNs() / 1000U);
|
||||
processor_elem.output_wait_elapsed_us = static_cast<UInt64>(processor->getOutputWaitElapsedNs() / 1000U);
|
||||
|
||||
auto stats = processor->getProcessorDataStats();
|
||||
processor_elem.input_rows = stats.input_rows;
|
||||
processor_elem.input_bytes = stats.input_bytes;
|
||||
processor_elem.output_rows = stats.output_rows;
|
||||
processor_elem.output_bytes = stats.output_bytes;
|
||||
|
||||
processors_profile_log->add(processor_elem);
|
||||
}
|
||||
}
|
||||
}
|
||||
logProcessorProfile(context, query_pipeline.getProcessors());
|
||||
|
||||
logQueryMetricLogFinish(context, internal, elem.client_info.current_query_id, time_now, std::make_shared<QueryStatusInfo>(info));
|
||||
}
|
||||
@ -669,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;
|
||||
|
||||
@ -698,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(
|
||||
@ -753,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]);
|
||||
@ -797,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)
|
||||
|
@ -5,10 +5,10 @@
|
||||
#include <Parsers/ASTPartition.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/ASTQueryParameter.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -61,7 +61,7 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
else
|
||||
fields_count = 0;
|
||||
}
|
||||
else if (const auto* literal_ast = value->as<ASTLiteral>(); literal_ast)
|
||||
else if (const auto * literal_ast = value->as<ASTLiteral>(); literal_ast)
|
||||
{
|
||||
if (literal_ast->value.getType() == Field::Types::Tuple)
|
||||
{
|
||||
|
@ -1,71 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
#include <Parsers/TokenIterator.h>
|
||||
#include <map>
|
||||
#include <memory>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct StringRange
|
||||
{
|
||||
const char * first = nullptr;
|
||||
const char * second = nullptr;
|
||||
|
||||
StringRange() = default;
|
||||
StringRange(const char * begin, const char * end) : first(begin), second(end) {}
|
||||
explicit StringRange(TokenIterator token) : first(token->begin), second(token->end) {}
|
||||
|
||||
StringRange(TokenIterator token_begin, TokenIterator token_end)
|
||||
{
|
||||
/// Empty range.
|
||||
if (token_begin == token_end)
|
||||
{
|
||||
first = token_begin->begin;
|
||||
second = token_begin->begin;
|
||||
return;
|
||||
}
|
||||
|
||||
TokenIterator token_last = token_end;
|
||||
--token_last;
|
||||
|
||||
first = token_begin->begin;
|
||||
second = token_last->end;
|
||||
}
|
||||
};
|
||||
|
||||
using StringPtr = std::shared_ptr<String>;
|
||||
|
||||
|
||||
inline String toString(const StringRange & range)
|
||||
{
|
||||
return range.first ? String(range.first, range.second) : String();
|
||||
}
|
||||
|
||||
/// Hashes only the values of pointers in StringRange. Is used with StringRangePointersEqualTo comparator.
|
||||
struct StringRangePointersHash
|
||||
{
|
||||
UInt64 operator()(const StringRange & range) const
|
||||
{
|
||||
SipHash hash;
|
||||
hash.update(range.first);
|
||||
hash.update(range.second);
|
||||
return hash.get64();
|
||||
}
|
||||
};
|
||||
|
||||
/// Ranges are equal only when they point to the same memory region.
|
||||
/// It may be used when it's enough to compare substrings by their position in the same string.
|
||||
struct StringRangePointersEqualTo
|
||||
{
|
||||
constexpr bool operator()(const StringRange &lhs, const StringRange &rhs) const
|
||||
{
|
||||
return std::tie(lhs.first, lhs.second) == std::tie(rhs.first, rhs.second);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1534,15 +1534,23 @@ static ColumnWithTypeAndName readColumnWithDateData(
|
||||
|
||||
for (size_t i = 0; i < orc_int_column->numElements; ++i)
|
||||
{
|
||||
Int32 days_num = static_cast<Int32>(orc_int_column->data[i]);
|
||||
if (check_date_range && (days_num > DATE_LUT_MAX_EXTEND_DAY_NUM || days_num < -DAYNUM_OFFSET_EPOCH))
|
||||
throw Exception(
|
||||
ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE,
|
||||
"Input value {} of a column \"{}\" exceeds the range of type Date32",
|
||||
days_num,
|
||||
column_name);
|
||||
if (!orc_int_column->hasNulls || orc_int_column->notNull[i])
|
||||
{
|
||||
Int32 days_num = static_cast<Int32>(orc_int_column->data[i]);
|
||||
if (check_date_range && (days_num > DATE_LUT_MAX_EXTEND_DAY_NUM || days_num < -DAYNUM_OFFSET_EPOCH))
|
||||
throw Exception(
|
||||
ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE,
|
||||
"Input value {} of a column \"{}\" exceeds the range of type Date32",
|
||||
days_num,
|
||||
column_name);
|
||||
|
||||
column_data.push_back(days_num);
|
||||
column_data.push_back(days_num);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// ORC library doesn't guarantee that orc_int_column->data[i] is initialized to zero when orc_int_column->notNull[i] is false since https://github.com/ClickHouse/ClickHouse/pull/69473
|
||||
column_data.push_back(0);
|
||||
}
|
||||
}
|
||||
|
||||
return {std::move(internal_column), internal_type, column_name};
|
||||
|
@ -317,8 +317,9 @@ void LimitTransform::splitChunk(PortsData & data)
|
||||
length = offset + limit - (rows_read - num_rows) - start;
|
||||
}
|
||||
|
||||
/// check if other rows in current block equals to last one in limit
|
||||
if (with_ties && length)
|
||||
/// Check if other rows in current block equals to last one in limit
|
||||
/// when rows read >= offset + limit.
|
||||
if (with_ties && offset + limit <= rows_read && length)
|
||||
{
|
||||
UInt64 current_row_num = start + length;
|
||||
previous_row_chunk = makeChunkWithPreviousRow(data.current_chunk, current_row_num - 1);
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -17,6 +17,8 @@ using TemporaryFileLookupPtr = std::shared_ptr<ITemporaryFileLookup>;
|
||||
|
||||
struct BuildQueryPipelineSettings
|
||||
{
|
||||
bool enable_multiple_filters_transforms_for_and_chain = true;
|
||||
|
||||
ExpressionActionsSettings actions_settings;
|
||||
QueryStatusPtr process_list_element;
|
||||
ProgressCallback progress_callback = nullptr;
|
||||
|
@ -5,6 +5,11 @@
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/JSONBuilder.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <stack>
|
||||
#include <ranges>
|
||||
|
||||
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<DataTypeUInt8>();
|
||||
if (filter_type->isNullable())
|
||||
cast_type = std::make_shared<DataTypeNullable>(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<ActionsAndName> 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<const ActionsDAG::Node *> 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<ActionsAndName> splitAndChainIntoMultipleFilters(ActionsDAG & dag, const std::string & filter_name)
|
||||
{
|
||||
std::vector<ActionsAndName> 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<ActionsAndName> 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<ExpressionActions>(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<FilterTransform>(header, expression, and_atom.name, true, on_totals);
|
||||
});
|
||||
}
|
||||
|
||||
auto expression = std::make_shared<ExpressionActions>(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<ActionsAndName> 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<ExpressionActions>(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<ExpressionActions>(actions_dag.clone());
|
||||
auto expression = std::make_shared<ExpressionActions>(std::move(cloned_dag));
|
||||
expression->describeActions(settings.out, prefix);
|
||||
}
|
||||
|
||||
void FilterStep::describeActions(JSONBuilder::JSONMap & map) const
|
||||
{
|
||||
auto cloned_dag = actions_dag.clone();
|
||||
|
||||
std::vector<ActionsAndName> 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<ExpressionActions>(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);
|
||||
|
||||
|
@ -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;
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user