mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge remote-tracking branch 'origin/master' into tomic
This commit is contained in:
commit
7e99f05981
@ -481,6 +481,11 @@ if (ARCH_S390X)
|
|||||||
else()
|
else()
|
||||||
set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt75l_dat.S" )
|
set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt75l_dat.S" )
|
||||||
endif()
|
endif()
|
||||||
|
# ^^ you might be confused how for different little endian platforms (x86, ARM) the same assembly files can be used.
|
||||||
|
# These files are indeed assembly but they only contain data ('.long' directive), which makes them portable accross CPUs.
|
||||||
|
# Only the endianness and the character set (ASCII, EBCDIC) makes a difference, also see
|
||||||
|
# https://unicode-org.github.io/icu/userguide/icu_data/#sharing-icu-data-between-platforms, 'Sharing ICU Data Between Platforms')
|
||||||
|
# (and as an experiment, try re-generating the data files on x86 vs. ARM, ... you'll get exactly the same files)
|
||||||
|
|
||||||
set(ICUDATA_SOURCES
|
set(ICUDATA_SOURCES
|
||||||
"${ICUDATA_SOURCE_FILE}"
|
"${ICUDATA_SOURCE_FILE}"
|
||||||
|
@ -187,10 +187,15 @@ function setup_logs_replication
|
|||||||
')
|
')
|
||||||
|
|
||||||
echo -e "Creating remote destination table ${table}_${hash} with statement:" >&2
|
echo -e "Creating remote destination table ${table}_${hash} with statement:" >&2
|
||||||
|
|
||||||
echo "::group::${table}"
|
echo "::group::${table}"
|
||||||
# there's the only way big "$statement" can be printed without causing EAGAIN error
|
# there's the only way big "$statement" can be printed without causing EAGAIN error
|
||||||
# cat: write error: Resource temporarily unavailable
|
# cat: write error: Resource temporarily unavailable
|
||||||
echo "$statement" | cat
|
statement_print="${statement}"
|
||||||
|
if [ "${#statement_print}" -gt 4000 ]; then
|
||||||
|
statement_print="${statement::1999}\n…\n${statement:${#statement}-1999}"
|
||||||
|
fi
|
||||||
|
echo -e "$statement_print"
|
||||||
echo "::endgroup::"
|
echo "::endgroup::"
|
||||||
|
|
||||||
echo "$statement" | clickhouse-client --database_replicated_initial_query_timeout_sec=10 \
|
echo "$statement" | clickhouse-client --database_replicated_initial_query_timeout_sec=10 \
|
||||||
|
72
docs/en/engines/table-engines/integrations/azure-queue.md
Normal file
72
docs/en/engines/table-engines/integrations/azure-queue.md
Normal file
@ -0,0 +1,72 @@
|
|||||||
|
---
|
||||||
|
slug: /en/engines/table-engines/integrations/azure-queue
|
||||||
|
sidebar_position: 181
|
||||||
|
sidebar_label: AzureQueue
|
||||||
|
---
|
||||||
|
|
||||||
|
# AzureQueue Table Engine
|
||||||
|
|
||||||
|
This engine provides an integration with [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs) ecosystem, allowing streaming data import.
|
||||||
|
|
||||||
|
## Create Table {#creating-a-table}
|
||||||
|
|
||||||
|
``` sql
|
||||||
|
CREATE TABLE test (name String, value UInt32)
|
||||||
|
ENGINE = AzureQueue(...)
|
||||||
|
[SETTINGS]
|
||||||
|
[mode = '',]
|
||||||
|
[after_processing = 'keep',]
|
||||||
|
[keeper_path = '',]
|
||||||
|
...
|
||||||
|
```
|
||||||
|
|
||||||
|
**Engine parameters**
|
||||||
|
|
||||||
|
`AzureQueue` parameters are the same as `AzureBlobStorage` table engine supports. See parameters section [here](../../../engines/table-engines/integrations/azureBlobStorage.md).
|
||||||
|
|
||||||
|
**Example**
|
||||||
|
|
||||||
|
```sql
|
||||||
|
CREATE TABLE azure_queue_engine_table (name String, value UInt32)
|
||||||
|
ENGINE=AzureQueue('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/data/')
|
||||||
|
SETTINGS
|
||||||
|
mode = 'unordered'
|
||||||
|
```
|
||||||
|
|
||||||
|
## Settings {#settings}
|
||||||
|
|
||||||
|
The set of supported settings is the same as for `S3Queue` table engine, but without `s3queue_` prefix. See [full list of settings settings](../../../engines/table-engines/integrations/s3queue.md#settings).
|
||||||
|
|
||||||
|
## Description {#description}
|
||||||
|
|
||||||
|
`SELECT` is not particularly useful for streaming import (except for debugging), because each file can be imported only once. It is more practical to create real-time threads using [materialized views](../../../sql-reference/statements/create/view.md). To do this:
|
||||||
|
|
||||||
|
1. Use the engine to create a table for consuming from specified path in S3 and consider it a data stream.
|
||||||
|
2. Create a table with the desired structure.
|
||||||
|
3. Create a materialized view that converts data from the engine and puts it into a previously created table.
|
||||||
|
|
||||||
|
When the `MATERIALIZED VIEW` joins the engine, it starts collecting data in the background.
|
||||||
|
|
||||||
|
Example:
|
||||||
|
|
||||||
|
``` sql
|
||||||
|
CREATE TABLE azure_queue_engine_table (name String, value UInt32)
|
||||||
|
ENGINE=AzureQueue('<endpoint>', 'CSV', 'gzip')
|
||||||
|
SETTINGS
|
||||||
|
mode = 'unordered';
|
||||||
|
|
||||||
|
CREATE TABLE stats (name String, value UInt32)
|
||||||
|
ENGINE = MergeTree() ORDER BY name;
|
||||||
|
|
||||||
|
CREATE MATERIALIZED VIEW consumer TO stats
|
||||||
|
AS SELECT name, value FROM azure_queue_engine_table;
|
||||||
|
|
||||||
|
SELECT * FROM stats ORDER BY name;
|
||||||
|
```
|
||||||
|
|
||||||
|
## Virtual columns {#virtual-columns}
|
||||||
|
|
||||||
|
- `_path` — Path to the file.
|
||||||
|
- `_file` — Name of the file.
|
||||||
|
|
||||||
|
For more information about virtual columns see [here](../../../engines/table-engines/index.md#table_engines-virtual_columns).
|
@ -35,7 +35,7 @@ CREATE TABLE s3_engine_table (name String, value UInt32)
|
|||||||
[SETTINGS ...]
|
[SETTINGS ...]
|
||||||
```
|
```
|
||||||
|
|
||||||
### Engine parameters
|
### Engine parameters {#parameters}
|
||||||
|
|
||||||
- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path).
|
- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path).
|
||||||
- `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed.
|
- `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed.
|
||||||
|
@ -5,6 +5,7 @@ sidebar_label: S3Queue
|
|||||||
---
|
---
|
||||||
|
|
||||||
# S3Queue Table Engine
|
# S3Queue Table Engine
|
||||||
|
|
||||||
This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ecosystem and allows streaming import. This engine is similar to the [Kafka](../../../engines/table-engines/integrations/kafka.md), [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) engines, but provides S3-specific features.
|
This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ecosystem and allows streaming import. This engine is similar to the [Kafka](../../../engines/table-engines/integrations/kafka.md), [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) engines, but provides S3-specific features.
|
||||||
|
|
||||||
## Create Table {#creating-a-table}
|
## Create Table {#creating-a-table}
|
||||||
@ -16,27 +17,25 @@ CREATE TABLE s3_queue_engine_table (name String, value UInt32)
|
|||||||
[mode = '',]
|
[mode = '',]
|
||||||
[after_processing = 'keep',]
|
[after_processing = 'keep',]
|
||||||
[keeper_path = '',]
|
[keeper_path = '',]
|
||||||
[s3queue_loading_retries = 0,]
|
[loading_retries = 0,]
|
||||||
[s3queue_processing_threads_num = 1,]
|
[processing_threads_num = 1,]
|
||||||
[s3queue_enable_logging_to_s3queue_log = 0,]
|
[enable_logging_to_s3queue_log = 0,]
|
||||||
[s3queue_polling_min_timeout_ms = 1000,]
|
[polling_min_timeout_ms = 1000,]
|
||||||
[s3queue_polling_max_timeout_ms = 10000,]
|
[polling_max_timeout_ms = 10000,]
|
||||||
[s3queue_polling_backoff_ms = 0,]
|
[polling_backoff_ms = 0,]
|
||||||
[s3queue_tracked_file_ttl_sec = 0,]
|
[tracked_file_ttl_sec = 0,]
|
||||||
[s3queue_tracked_files_limit = 1000,]
|
[tracked_files_limit = 1000,]
|
||||||
[s3queue_cleanup_interval_min_ms = 10000,]
|
[cleanup_interval_min_ms = 10000,]
|
||||||
[s3queue_cleanup_interval_max_ms = 30000,]
|
[cleanup_interval_max_ms = 30000,]
|
||||||
```
|
```
|
||||||
|
|
||||||
Starting with `24.7` settings without `s3queue_` prefix are also supported.
|
:::warning
|
||||||
|
Before `24.7`, it is required to use `s3queue_` prefix for all settings apart from `mode`, `after_processing` and `keeper_path`.
|
||||||
|
:::
|
||||||
|
|
||||||
**Engine parameters**
|
**Engine parameters**
|
||||||
|
|
||||||
- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path).
|
`S3Queue` parameters are the same as `S3` table engine supports. See parameters section [here](../../../engines/table-engines/integrations/s3.md#parameters).
|
||||||
- `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed.
|
|
||||||
- `format` — The [format](../../../interfaces/formats.md#formats) of the file.
|
|
||||||
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3).
|
|
||||||
- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension.
|
|
||||||
|
|
||||||
**Example**
|
**Example**
|
||||||
|
|
||||||
|
@ -6,7 +6,7 @@ import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.m
|
|||||||
|
|
||||||
<SelfManaged />
|
<SelfManaged />
|
||||||
|
|
||||||
[SSL 'strict' option](../server-configuration-parameters/settings.md#server_configuration_parameters-openssl) enables mandatory certificate validation for the incoming connections. In this case, only connections with trusted certificates can be established. Connections with untrusted certificates will be rejected. Thus, certificate validation allows to uniquely authenticate an incoming connection. `Common Name` or `subjectAltName extension` field of the certificate is used to identify the connected user. This allows to associate multiple certificates with the same user. Additionally, reissuing and revoking of the certificates does not affect the ClickHouse configuration.
|
[SSL 'strict' option](../server-configuration-parameters/settings.md#server_configuration_parameters-openssl) enables mandatory certificate validation for the incoming connections. In this case, only connections with trusted certificates can be established. Connections with untrusted certificates will be rejected. Thus, certificate validation allows to uniquely authenticate an incoming connection. `Common Name` or `subjectAltName extension` field of the certificate is used to identify the connected user. `subjectAltName extension` supports the usage of one wildcard '*' in the server configuration. This allows to associate multiple certificates with the same user. Additionally, reissuing and revoking of the certificates does not affect the ClickHouse configuration.
|
||||||
|
|
||||||
To enable SSL certificate authentication, a list of `Common Name`'s or `Subject Alt Name`'s for each ClickHouse user must be specified in the settings file `users.xml `:
|
To enable SSL certificate authentication, a list of `Common Name`'s or `Subject Alt Name`'s for each ClickHouse user must be specified in the settings file `users.xml `:
|
||||||
|
|
||||||
@ -30,6 +30,12 @@ To enable SSL certificate authentication, a list of `Common Name`'s or `Subject
|
|||||||
</ssl_certificates>
|
</ssl_certificates>
|
||||||
<!-- Other settings -->
|
<!-- Other settings -->
|
||||||
</user_name_2>
|
</user_name_2>
|
||||||
|
<user_name_3>
|
||||||
|
<ssl_certificates>
|
||||||
|
<!-- Wildcard support -->
|
||||||
|
<subject_alt_name>URI:spiffe://foo.com/*/bar</subject_alt_name>
|
||||||
|
</ssl_certificates>
|
||||||
|
</user_name_3>
|
||||||
</users>
|
</users>
|
||||||
</clickhouse>
|
</clickhouse>
|
||||||
```
|
```
|
||||||
|
@ -55,7 +55,7 @@ keeper foo bar
|
|||||||
- `touch '<path>'` -- Creates new node with an empty string as value. Doesn't throw an exception if the node already exists
|
- `touch '<path>'` -- Creates new node with an empty string as value. Doesn't throw an exception if the node already exists
|
||||||
- `get '<path>'` -- Returns the node's value
|
- `get '<path>'` -- Returns the node's value
|
||||||
- `rm '<path>' [version]` -- Removes the node only if version matches (default: -1)
|
- `rm '<path>' [version]` -- Removes the node only if version matches (default: -1)
|
||||||
- `rmr '<path>'` -- Recursively deletes path. Confirmation required
|
- `rmr '<path>' [limit]` -- Recursively deletes path if the subtree size is smaller than the limit. Confirmation required (default limit = 100)
|
||||||
- `flwc <command>` -- Executes four-letter-word command
|
- `flwc <command>` -- Executes four-letter-word command
|
||||||
- `help` -- Prints this message
|
- `help` -- Prints this message
|
||||||
- `get_direct_children_number '[path]'` -- Get numbers of direct children nodes under a specific path
|
- `get_direct_children_number '[path]'` -- Get numbers of direct children nodes under a specific path
|
||||||
|
@ -506,14 +506,23 @@ bool RMRCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & nod
|
|||||||
return false;
|
return false;
|
||||||
node->args.push_back(std::move(path));
|
node->args.push_back(std::move(path));
|
||||||
|
|
||||||
|
ASTPtr remove_nodes_limit;
|
||||||
|
if (ParserUnsignedInteger{}.parse(pos, remove_nodes_limit, expected))
|
||||||
|
node->args.push_back(remove_nodes_limit->as<ASTLiteral &>().value);
|
||||||
|
else
|
||||||
|
node->args.push_back(UInt64(100));
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
void RMRCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
|
void RMRCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
|
||||||
{
|
{
|
||||||
String path = client->getAbsolutePath(query->args[0].safeGet<String>());
|
String path = client->getAbsolutePath(query->args[0].safeGet<String>());
|
||||||
|
UInt64 remove_nodes_limit = query->args[1].safeGet<UInt64>();
|
||||||
|
|
||||||
client->askConfirmation(
|
client->askConfirmation(
|
||||||
"You are going to recursively delete path " + path, [client, path] { client->zookeeper->removeRecursive(path); });
|
"You are going to recursively delete path " + path,
|
||||||
|
[client, path, remove_nodes_limit] { client->zookeeper->removeRecursive(path, static_cast<UInt32>(remove_nodes_limit)); });
|
||||||
}
|
}
|
||||||
|
|
||||||
bool ReconfigCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, DB::Expected & expected) const
|
bool ReconfigCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, DB::Expected & expected) const
|
||||||
|
@ -184,7 +184,7 @@ class RMRCommand : public IKeeperClientCommand
|
|||||||
|
|
||||||
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
|
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
|
||||||
|
|
||||||
String getHelpMessage() const override { return "{} <path> -- Recursively deletes path. Confirmation required"; }
|
String getHelpMessage() const override { return "{} <path> [limit] -- Recursively deletes path if the subtree size is smaller than the limit. Confirmation required (default limit = 100)"; }
|
||||||
};
|
};
|
||||||
|
|
||||||
class ReconfigCommand : public IKeeperClientCommand
|
class ReconfigCommand : public IKeeperClientCommand
|
||||||
|
@ -11,6 +11,7 @@
|
|||||||
#include <Core/ServerUUID.h>
|
#include <Core/ServerUUID.h>
|
||||||
#include <Common/logger_useful.h>
|
#include <Common/logger_useful.h>
|
||||||
#include <Common/CgroupsMemoryUsageObserver.h>
|
#include <Common/CgroupsMemoryUsageObserver.h>
|
||||||
|
#include <Common/MemoryWorker.h>
|
||||||
#include <Common/ErrorHandlers.h>
|
#include <Common/ErrorHandlers.h>
|
||||||
#include <Common/assertProcessUserMatchesDataOwner.h>
|
#include <Common/assertProcessUserMatchesDataOwner.h>
|
||||||
#include <Common/makeSocketAddress.h>
|
#include <Common/makeSocketAddress.h>
|
||||||
@ -384,6 +385,9 @@ try
|
|||||||
LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds());
|
LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds());
|
||||||
});
|
});
|
||||||
|
|
||||||
|
MemoryWorker memory_worker(config().getUInt64("memory_worker_period_ms", 0));
|
||||||
|
memory_worker.start();
|
||||||
|
|
||||||
static ServerErrorHandler error_handler;
|
static ServerErrorHandler error_handler;
|
||||||
Poco::ErrorHandler::set(&error_handler);
|
Poco::ErrorHandler::set(&error_handler);
|
||||||
|
|
||||||
@ -425,8 +429,9 @@ try
|
|||||||
for (const auto & server : *servers)
|
for (const auto & server : *servers)
|
||||||
metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads(), server.refusedConnections()});
|
metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads(), server.refusedConnections()});
|
||||||
return metrics;
|
return metrics;
|
||||||
}
|
},
|
||||||
);
|
/*update_jemalloc_epoch_=*/memory_worker.getSource() != MemoryWorker::MemoryUsageSource::Jemalloc,
|
||||||
|
/*update_rss_=*/memory_worker.getSource() == MemoryWorker::MemoryUsageSource::None);
|
||||||
|
|
||||||
std::vector<std::string> listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host");
|
std::vector<std::string> listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host");
|
||||||
|
|
||||||
@ -655,7 +660,6 @@ try
|
|||||||
GWPAsan::initFinished();
|
GWPAsan::initFinished();
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
|
||||||
LOG_INFO(log, "Ready for connections.");
|
LOG_INFO(log, "Ready for connections.");
|
||||||
|
|
||||||
waitForTerminationRequest();
|
waitForTerminationRequest();
|
||||||
|
@ -11,7 +11,6 @@
|
|||||||
#include <Poco/Util/HelpFormatter.h>
|
#include <Poco/Util/HelpFormatter.h>
|
||||||
#include <Poco/Environment.h>
|
#include <Poco/Environment.h>
|
||||||
#include <Poco/Config.h>
|
#include <Poco/Config.h>
|
||||||
#include <Common/Jemalloc.h>
|
|
||||||
#include <Common/scope_guard_safe.h>
|
#include <Common/scope_guard_safe.h>
|
||||||
#include <Common/logger_useful.h>
|
#include <Common/logger_useful.h>
|
||||||
#include <base/phdr_cache.h>
|
#include <base/phdr_cache.h>
|
||||||
@ -25,6 +24,7 @@
|
|||||||
#include <base/Numa.h>
|
#include <base/Numa.h>
|
||||||
#include <Common/PoolId.h>
|
#include <Common/PoolId.h>
|
||||||
#include <Common/MemoryTracker.h>
|
#include <Common/MemoryTracker.h>
|
||||||
|
#include <Common/MemoryWorker.h>
|
||||||
#include <Common/ClickHouseRevision.h>
|
#include <Common/ClickHouseRevision.h>
|
||||||
#include <Common/DNSResolver.h>
|
#include <Common/DNSResolver.h>
|
||||||
#include <Common/CgroupsMemoryUsageObserver.h>
|
#include <Common/CgroupsMemoryUsageObserver.h>
|
||||||
@ -111,6 +111,8 @@
|
|||||||
#include <filesystem>
|
#include <filesystem>
|
||||||
#include <unordered_set>
|
#include <unordered_set>
|
||||||
|
|
||||||
|
#include <Common/Jemalloc.h>
|
||||||
|
|
||||||
#include "config.h"
|
#include "config.h"
|
||||||
#include <Common/config_version.h>
|
#include <Common/config_version.h>
|
||||||
|
|
||||||
@ -449,9 +451,12 @@ void checkForUsersNotInMainConfig(
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
/// Unused in other builds
|
/// Unused in other builds
|
||||||
#if defined(OS_LINUX)
|
#if defined(OS_LINUX)
|
||||||
static String readLine(const String & path)
|
String readLine(const String & path)
|
||||||
{
|
{
|
||||||
ReadBufferFromFile in(path);
|
ReadBufferFromFile in(path);
|
||||||
String contents;
|
String contents;
|
||||||
@ -459,7 +464,7 @@ static String readLine(const String & path)
|
|||||||
return contents;
|
return contents;
|
||||||
}
|
}
|
||||||
|
|
||||||
static int readNumber(const String & path)
|
int readNumber(const String & path)
|
||||||
{
|
{
|
||||||
ReadBufferFromFile in(path);
|
ReadBufferFromFile in(path);
|
||||||
int result;
|
int result;
|
||||||
@ -469,7 +474,7 @@ static int readNumber(const String & path)
|
|||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
static void sanityChecks(Server & server)
|
void sanityChecks(Server & server)
|
||||||
{
|
{
|
||||||
std::string data_path = getCanonicalPath(server.config().getString("path", DBMS_DEFAULT_PATH));
|
std::string data_path = getCanonicalPath(server.config().getString("path", DBMS_DEFAULT_PATH));
|
||||||
std::string logs_path = server.config().getString("logger.log", "");
|
std::string logs_path = server.config().getString("logger.log", "");
|
||||||
@ -590,6 +595,8 @@ static void sanityChecks(Server & server)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
void loadStartupScripts(const Poco::Util::AbstractConfiguration & config, ContextMutablePtr context, Poco::Logger * log)
|
void loadStartupScripts(const Poco::Util::AbstractConfiguration & config, ContextMutablePtr context, Poco::Logger * log)
|
||||||
{
|
{
|
||||||
try
|
try
|
||||||
@ -906,6 +913,8 @@ try
|
|||||||
LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds());
|
LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds());
|
||||||
});
|
});
|
||||||
|
|
||||||
|
MemoryWorker memory_worker(global_context->getServerSettings().memory_worker_period_ms);
|
||||||
|
|
||||||
/// This object will periodically calculate some metrics.
|
/// This object will periodically calculate some metrics.
|
||||||
ServerAsynchronousMetrics async_metrics(
|
ServerAsynchronousMetrics async_metrics(
|
||||||
global_context,
|
global_context,
|
||||||
@ -924,8 +933,9 @@ try
|
|||||||
for (const auto & server : servers)
|
for (const auto & server : servers)
|
||||||
metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads(), server.refusedConnections()});
|
metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads(), server.refusedConnections()});
|
||||||
return metrics;
|
return metrics;
|
||||||
}
|
},
|
||||||
);
|
/*update_jemalloc_epoch_=*/memory_worker.getSource() != MemoryWorker::MemoryUsageSource::Jemalloc,
|
||||||
|
/*update_rss_=*/memory_worker.getSource() == MemoryWorker::MemoryUsageSource::None);
|
||||||
|
|
||||||
/// NOTE: global context should be destroyed *before* GlobalThreadPool::shutdown()
|
/// NOTE: global context should be destroyed *before* GlobalThreadPool::shutdown()
|
||||||
/// Otherwise GlobalThreadPool::shutdown() will hang, since Context holds some threads.
|
/// Otherwise GlobalThreadPool::shutdown() will hang, since Context holds some threads.
|
||||||
@ -1204,6 +1214,8 @@ try
|
|||||||
|
|
||||||
FailPointInjection::enableFromGlobalConfig(config());
|
FailPointInjection::enableFromGlobalConfig(config());
|
||||||
|
|
||||||
|
memory_worker.start();
|
||||||
|
|
||||||
int default_oom_score = 0;
|
int default_oom_score = 0;
|
||||||
|
|
||||||
#if !defined(NDEBUG)
|
#if !defined(NDEBUG)
|
||||||
@ -1547,15 +1559,6 @@ try
|
|||||||
total_memory_tracker.setDescription("(total)");
|
total_memory_tracker.setDescription("(total)");
|
||||||
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
|
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
|
||||||
|
|
||||||
if (cgroups_memory_usage_observer)
|
|
||||||
{
|
|
||||||
double hard_limit_ratio = new_server_settings.cgroup_memory_watcher_hard_limit_ratio;
|
|
||||||
double soft_limit_ratio = new_server_settings.cgroup_memory_watcher_soft_limit_ratio;
|
|
||||||
cgroups_memory_usage_observer->setMemoryUsageLimits(
|
|
||||||
static_cast<uint64_t>(max_server_memory_usage * hard_limit_ratio),
|
|
||||||
static_cast<uint64_t>(max_server_memory_usage * soft_limit_ratio));
|
|
||||||
}
|
|
||||||
|
|
||||||
size_t merges_mutations_memory_usage_soft_limit = new_server_settings.merges_mutations_memory_usage_soft_limit;
|
size_t merges_mutations_memory_usage_soft_limit = new_server_settings.merges_mutations_memory_usage_soft_limit;
|
||||||
|
|
||||||
size_t default_merges_mutations_server_memory_usage = static_cast<size_t>(current_physical_server_memory * new_server_settings.merges_mutations_memory_usage_to_ram_ratio);
|
size_t default_merges_mutations_server_memory_usage = static_cast<size_t>(current_physical_server_memory * new_server_settings.merges_mutations_memory_usage_to_ram_ratio);
|
||||||
@ -1584,8 +1587,6 @@ try
|
|||||||
background_memory_tracker.setDescription("(background)");
|
background_memory_tracker.setDescription("(background)");
|
||||||
background_memory_tracker.setMetric(CurrentMetrics::MergesMutationsMemoryTracking);
|
background_memory_tracker.setMetric(CurrentMetrics::MergesMutationsMemoryTracking);
|
||||||
|
|
||||||
total_memory_tracker.setAllowUseJemallocMemory(new_server_settings.allow_use_jemalloc_memory);
|
|
||||||
|
|
||||||
auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker();
|
auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker();
|
||||||
total_memory_tracker.setOvercommitTracker(global_overcommit_tracker);
|
total_memory_tracker.setOvercommitTracker(global_overcommit_tracker);
|
||||||
|
|
||||||
|
@ -239,15 +239,36 @@ bool Authentication::areCredentialsValid(
|
|||||||
throw Authentication::Require<GSSAcceptorContext>(auth_data.getKerberosRealm());
|
throw Authentication::Require<GSSAcceptorContext>(auth_data.getKerberosRealm());
|
||||||
|
|
||||||
case AuthenticationType::SSL_CERTIFICATE:
|
case AuthenticationType::SSL_CERTIFICATE:
|
||||||
|
{
|
||||||
for (SSLCertificateSubjects::Type type : {SSLCertificateSubjects::Type::CN, SSLCertificateSubjects::Type::SAN})
|
for (SSLCertificateSubjects::Type type : {SSLCertificateSubjects::Type::CN, SSLCertificateSubjects::Type::SAN})
|
||||||
{
|
{
|
||||||
for (const auto & subject : auth_data.getSSLCertificateSubjects().at(type))
|
for (const auto & subject : auth_data.getSSLCertificateSubjects().at(type))
|
||||||
{
|
{
|
||||||
if (ssl_certificate_credentials->getSSLCertificateSubjects().at(type).contains(subject))
|
if (ssl_certificate_credentials->getSSLCertificateSubjects().at(type).contains(subject))
|
||||||
return true;
|
return true;
|
||||||
|
|
||||||
|
// Wildcard support (1 only)
|
||||||
|
if (subject.contains('*'))
|
||||||
|
{
|
||||||
|
auto prefix = std::string_view(subject).substr(0, subject.find('*'));
|
||||||
|
auto suffix = std::string_view(subject).substr(subject.find('*') + 1);
|
||||||
|
auto slashes = std::count(subject.begin(), subject.end(), '/');
|
||||||
|
|
||||||
|
for (const auto & certificate_subject : ssl_certificate_credentials->getSSLCertificateSubjects().at(type))
|
||||||
|
{
|
||||||
|
bool matches_wildcard = certificate_subject.starts_with(prefix) && certificate_subject.ends_with(suffix);
|
||||||
|
|
||||||
|
// '*' must not represent a '/' in URI, so check if the number of '/' are equal
|
||||||
|
bool matches_slashes = slashes == count(certificate_subject.begin(), certificate_subject.end(), '/');
|
||||||
|
|
||||||
|
if (matches_wildcard && matches_slashes)
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return false;
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
case AuthenticationType::SSH_KEY:
|
case AuthenticationType::SSH_KEY:
|
||||||
#if USE_SSH
|
#if USE_SSH
|
||||||
|
@ -116,15 +116,17 @@ class GroupConcatImpl final
|
|||||||
SerializationPtr serialization;
|
SerializationPtr serialization;
|
||||||
UInt64 limit;
|
UInt64 limit;
|
||||||
const String delimiter;
|
const String delimiter;
|
||||||
|
const DataTypePtr type;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
GroupConcatImpl(const DataTypePtr & data_type_, const Array & parameters_, UInt64 limit_, const String & delimiter_)
|
GroupConcatImpl(const DataTypePtr & data_type_, const Array & parameters_, UInt64 limit_, const String & delimiter_)
|
||||||
: IAggregateFunctionDataHelper<GroupConcatData<has_limit>, GroupConcatImpl<has_limit>>(
|
: IAggregateFunctionDataHelper<GroupConcatData<has_limit>, GroupConcatImpl<has_limit>>(
|
||||||
{data_type_}, parameters_, std::make_shared<DataTypeString>())
|
{data_type_}, parameters_, std::make_shared<DataTypeString>())
|
||||||
, serialization(this->argument_types[0]->getDefaultSerialization())
|
|
||||||
, limit(limit_)
|
, limit(limit_)
|
||||||
, delimiter(delimiter_)
|
, delimiter(delimiter_)
|
||||||
|
, type(data_type_)
|
||||||
{
|
{
|
||||||
|
serialization = isFixedString(type) ? std::make_shared<DataTypeString>()->getDefaultSerialization() : this->argument_types[0]->getDefaultSerialization();
|
||||||
}
|
}
|
||||||
|
|
||||||
String getName() const override { return name; }
|
String getName() const override { return name; }
|
||||||
@ -140,6 +142,13 @@ public:
|
|||||||
if (cur_data.data_size != 0)
|
if (cur_data.data_size != 0)
|
||||||
cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena);
|
cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena);
|
||||||
|
|
||||||
|
if (isFixedString(type))
|
||||||
|
{
|
||||||
|
ColumnWithTypeAndName col = {columns[0]->getPtr(), type, "column"};
|
||||||
|
const auto & col_str = castColumn(col, std::make_shared<DataTypeString>());
|
||||||
|
cur_data.insert(col_str.get(), serialization, row_num, arena);
|
||||||
|
}
|
||||||
|
else
|
||||||
cur_data.insert(columns[0], serialization, row_num, arena);
|
cur_data.insert(columns[0], serialization, row_num, arena);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -176,7 +176,7 @@ add_library (clickhouse_new_delete STATIC Common/new_delete.cpp)
|
|||||||
target_link_libraries (clickhouse_new_delete PRIVATE clickhouse_common_io)
|
target_link_libraries (clickhouse_new_delete PRIVATE clickhouse_common_io)
|
||||||
if (TARGET ch_contrib::jemalloc)
|
if (TARGET ch_contrib::jemalloc)
|
||||||
target_link_libraries (clickhouse_new_delete PRIVATE ch_contrib::jemalloc)
|
target_link_libraries (clickhouse_new_delete PRIVATE ch_contrib::jemalloc)
|
||||||
target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::jemalloc)
|
target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::jemalloc)
|
||||||
target_link_libraries (clickhouse_storages_system PRIVATE ch_contrib::jemalloc)
|
target_link_libraries (clickhouse_storages_system PRIVATE ch_contrib::jemalloc)
|
||||||
endif()
|
endif()
|
||||||
|
|
||||||
|
@ -1,5 +1,3 @@
|
|||||||
#include <Common/AsynchronousMetrics.h>
|
|
||||||
|
|
||||||
#include <IO/MMappedFileCache.h>
|
#include <IO/MMappedFileCache.h>
|
||||||
#include <IO/ReadHelpers.h>
|
#include <IO/ReadHelpers.h>
|
||||||
#include <IO/UncompressedCache.h>
|
#include <IO/UncompressedCache.h>
|
||||||
@ -8,8 +6,10 @@
|
|||||||
#include <base/find_symbols.h>
|
#include <base/find_symbols.h>
|
||||||
#include <base/getPageSize.h>
|
#include <base/getPageSize.h>
|
||||||
#include <sys/resource.h>
|
#include <sys/resource.h>
|
||||||
|
#include <Common/AsynchronousMetrics.h>
|
||||||
#include <Common/CurrentMetrics.h>
|
#include <Common/CurrentMetrics.h>
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
|
#include <Common/Jemalloc.h>
|
||||||
#include <Common/filesystemHelpers.h>
|
#include <Common/filesystemHelpers.h>
|
||||||
#include <Common/formatReadable.h>
|
#include <Common/formatReadable.h>
|
||||||
#include <Common/logger_useful.h>
|
#include <Common/logger_useful.h>
|
||||||
@ -69,10 +69,14 @@ static void openCgroupv2MetricFile(const std::string & filename, std::optional<R
|
|||||||
|
|
||||||
AsynchronousMetrics::AsynchronousMetrics(
|
AsynchronousMetrics::AsynchronousMetrics(
|
||||||
unsigned update_period_seconds,
|
unsigned update_period_seconds,
|
||||||
const ProtocolServerMetricsFunc & protocol_server_metrics_func_)
|
const ProtocolServerMetricsFunc & protocol_server_metrics_func_,
|
||||||
|
bool update_jemalloc_epoch_,
|
||||||
|
bool update_rss_)
|
||||||
: update_period(update_period_seconds)
|
: update_period(update_period_seconds)
|
||||||
, log(getLogger("AsynchronousMetrics"))
|
, log(getLogger("AsynchronousMetrics"))
|
||||||
, protocol_server_metrics_func(protocol_server_metrics_func_)
|
, protocol_server_metrics_func(protocol_server_metrics_func_)
|
||||||
|
, update_jemalloc_epoch(update_jemalloc_epoch_)
|
||||||
|
, update_rss(update_rss_)
|
||||||
{
|
{
|
||||||
#if defined(OS_LINUX)
|
#if defined(OS_LINUX)
|
||||||
openFileIfExists("/proc/cpuinfo", cpuinfo);
|
openFileIfExists("/proc/cpuinfo", cpuinfo);
|
||||||
@ -411,9 +415,7 @@ Value saveJemallocMetricImpl(
|
|||||||
const std::string & jemalloc_full_name,
|
const std::string & jemalloc_full_name,
|
||||||
const std::string & clickhouse_full_name)
|
const std::string & clickhouse_full_name)
|
||||||
{
|
{
|
||||||
Value value{};
|
auto value = getJemallocValue<Value>(jemalloc_full_name.c_str());
|
||||||
size_t size = sizeof(value);
|
|
||||||
mallctl(jemalloc_full_name.c_str(), &value, &size, nullptr, 0);
|
|
||||||
values[clickhouse_full_name] = AsynchronousMetricValue(value, "An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html");
|
values[clickhouse_full_name] = AsynchronousMetricValue(value, "An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html");
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
@ -768,8 +770,11 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update)
|
|||||||
// 'epoch' is a special mallctl -- it updates the statistics. Without it, all
|
// 'epoch' is a special mallctl -- it updates the statistics. Without it, all
|
||||||
// the following calls will return stale values. It increments and returns
|
// the following calls will return stale values. It increments and returns
|
||||||
// the current epoch number, which might be useful to log as a sanity check.
|
// the current epoch number, which might be useful to log as a sanity check.
|
||||||
auto epoch = updateJemallocEpoch();
|
auto epoch = update_jemalloc_epoch ? updateJemallocEpoch() : getJemallocValue<uint64_t>("epoch");
|
||||||
new_values["jemalloc.epoch"] = { epoch, "An internal incremental update number of the statistics of jemalloc (Jason Evans' memory allocator), used in all other `jemalloc` metrics." };
|
new_values["jemalloc.epoch"]
|
||||||
|
= {epoch,
|
||||||
|
"An internal incremental update number of the statistics of jemalloc (Jason Evans' memory allocator), used in all other "
|
||||||
|
"`jemalloc` metrics."};
|
||||||
|
|
||||||
// Collect the statistics themselves.
|
// Collect the statistics themselves.
|
||||||
saveJemallocMetric<size_t>(new_values, "allocated");
|
saveJemallocMetric<size_t>(new_values, "allocated");
|
||||||
@ -782,10 +787,10 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update)
|
|||||||
saveJemallocMetric<size_t>(new_values, "background_thread.num_threads");
|
saveJemallocMetric<size_t>(new_values, "background_thread.num_threads");
|
||||||
saveJemallocMetric<uint64_t>(new_values, "background_thread.num_runs");
|
saveJemallocMetric<uint64_t>(new_values, "background_thread.num_runs");
|
||||||
saveJemallocMetric<uint64_t>(new_values, "background_thread.run_intervals");
|
saveJemallocMetric<uint64_t>(new_values, "background_thread.run_intervals");
|
||||||
saveJemallocProf<size_t>(new_values, "active");
|
saveJemallocProf<bool>(new_values, "active");
|
||||||
saveAllArenasMetric<size_t>(new_values, "pactive");
|
saveAllArenasMetric<size_t>(new_values, "pactive");
|
||||||
[[maybe_unused]] size_t je_malloc_pdirty = saveAllArenasMetric<size_t>(new_values, "pdirty");
|
saveAllArenasMetric<size_t>(new_values, "pdirty");
|
||||||
[[maybe_unused]] size_t je_malloc_pmuzzy = saveAllArenasMetric<size_t>(new_values, "pmuzzy");
|
saveAllArenasMetric<size_t>(new_values, "pmuzzy");
|
||||||
saveAllArenasMetric<size_t>(new_values, "dirty_purged");
|
saveAllArenasMetric<size_t>(new_values, "dirty_purged");
|
||||||
saveAllArenasMetric<size_t>(new_values, "muzzy_purged");
|
saveAllArenasMetric<size_t>(new_values, "muzzy_purged");
|
||||||
#endif
|
#endif
|
||||||
@ -814,41 +819,8 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update)
|
|||||||
" It is unspecified whether it includes the per-thread stacks and most of the allocated memory, that is allocated with the 'mmap' system call."
|
" It is unspecified whether it includes the per-thread stacks and most of the allocated memory, that is allocated with the 'mmap' system call."
|
||||||
" This metric exists only for completeness reasons. I recommend to use the `MemoryResident` metric for monitoring."};
|
" This metric exists only for completeness reasons. I recommend to use the `MemoryResident` metric for monitoring."};
|
||||||
|
|
||||||
/// We must update the value of total_memory_tracker periodically.
|
if (update_rss)
|
||||||
/// Otherwise it might be calculated incorrectly - it can include a "drift" of memory amount.
|
MemoryTracker::updateRSS(data.resident);
|
||||||
/// See https://github.com/ClickHouse/ClickHouse/issues/10293
|
|
||||||
{
|
|
||||||
Int64 amount = total_memory_tracker.get();
|
|
||||||
Int64 peak = total_memory_tracker.getPeak();
|
|
||||||
Int64 rss = data.resident;
|
|
||||||
Int64 free_memory_in_allocator_arenas = 0;
|
|
||||||
|
|
||||||
#if USE_JEMALLOC
|
|
||||||
/// According to jemalloc man, pdirty is:
|
|
||||||
///
|
|
||||||
/// Number of pages within unused extents that are potentially
|
|
||||||
/// dirty, and for which madvise() or similar has not been called.
|
|
||||||
///
|
|
||||||
/// So they will be subtracted from RSS to make accounting more
|
|
||||||
/// accurate, since those pages are not really RSS but a memory
|
|
||||||
/// that can be used at anytime via jemalloc.
|
|
||||||
free_memory_in_allocator_arenas = je_malloc_pdirty * getPageSize();
|
|
||||||
#endif
|
|
||||||
|
|
||||||
Int64 difference = rss - amount;
|
|
||||||
|
|
||||||
/// Log only if difference is high. This is for convenience. The threshold is arbitrary.
|
|
||||||
if (difference >= 1048576 || difference <= -1048576)
|
|
||||||
LOG_TRACE(log,
|
|
||||||
"MemoryTracking: was {}, peak {}, free memory in arenas {}, will set to {} (RSS), difference: {}",
|
|
||||||
ReadableSize(amount),
|
|
||||||
ReadableSize(peak),
|
|
||||||
ReadableSize(free_memory_in_allocator_arenas),
|
|
||||||
ReadableSize(rss),
|
|
||||||
ReadableSize(difference));
|
|
||||||
|
|
||||||
MemoryTracker::setRSS(rss, free_memory_in_allocator_arenas);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
{
|
{
|
||||||
|
@ -1,15 +1,14 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <Common/CgroupsMemoryUsageObserver.h>
|
||||||
#include <Common/MemoryStatisticsOS.h>
|
#include <Common/MemoryStatisticsOS.h>
|
||||||
#include <Common/ThreadPool.h>
|
#include <Common/ThreadPool.h>
|
||||||
#include <Common/Stopwatch.h>
|
#include <Common/Stopwatch.h>
|
||||||
#include <IO/ReadBufferFromFile.h>
|
#include <IO/ReadBufferFromFile.h>
|
||||||
|
|
||||||
#include <condition_variable>
|
#include <condition_variable>
|
||||||
#include <map>
|
|
||||||
#include <mutex>
|
#include <mutex>
|
||||||
#include <string>
|
#include <string>
|
||||||
#include <thread>
|
|
||||||
#include <vector>
|
#include <vector>
|
||||||
#include <optional>
|
#include <optional>
|
||||||
#include <unordered_map>
|
#include <unordered_map>
|
||||||
@ -69,7 +68,9 @@ public:
|
|||||||
|
|
||||||
AsynchronousMetrics(
|
AsynchronousMetrics(
|
||||||
unsigned update_period_seconds,
|
unsigned update_period_seconds,
|
||||||
const ProtocolServerMetricsFunc & protocol_server_metrics_func_);
|
const ProtocolServerMetricsFunc & protocol_server_metrics_func_,
|
||||||
|
bool update_jemalloc_epoch_,
|
||||||
|
bool update_rss_);
|
||||||
|
|
||||||
virtual ~AsynchronousMetrics();
|
virtual ~AsynchronousMetrics();
|
||||||
|
|
||||||
@ -112,6 +113,9 @@ private:
|
|||||||
MemoryStatisticsOS memory_stat TSA_GUARDED_BY(data_mutex);
|
MemoryStatisticsOS memory_stat TSA_GUARDED_BY(data_mutex);
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
[[maybe_unused]] const bool update_jemalloc_epoch;
|
||||||
|
[[maybe_unused]] const bool update_rss;
|
||||||
|
|
||||||
#if defined(OS_LINUX)
|
#if defined(OS_LINUX)
|
||||||
std::optional<ReadBufferFromFilePRead> meminfo TSA_GUARDED_BY(data_mutex);
|
std::optional<ReadBufferFromFilePRead> meminfo TSA_GUARDED_BY(data_mutex);
|
||||||
std::optional<ReadBufferFromFilePRead> loadavg TSA_GUARDED_BY(data_mutex);
|
std::optional<ReadBufferFromFilePRead> loadavg TSA_GUARDED_BY(data_mutex);
|
||||||
|
@ -14,239 +14,21 @@
|
|||||||
#include <fmt/ranges.h>
|
#include <fmt/ranges.h>
|
||||||
|
|
||||||
#include <cstdint>
|
#include <cstdint>
|
||||||
#include <filesystem>
|
|
||||||
#include <memory>
|
|
||||||
#include <optional>
|
|
||||||
|
|
||||||
#include "config.h"
|
|
||||||
#if USE_JEMALLOC
|
|
||||||
# include <jemalloc/jemalloc.h>
|
|
||||||
#define STRINGIFY_HELPER(x) #x
|
|
||||||
#define STRINGIFY(x) STRINGIFY_HELPER(x)
|
|
||||||
#endif
|
|
||||||
|
|
||||||
using namespace DB;
|
using namespace DB;
|
||||||
namespace fs = std::filesystem;
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int FILE_DOESNT_EXIST;
|
|
||||||
extern const int INCORRECT_DATA;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
namespace
|
|
||||||
{
|
|
||||||
|
|
||||||
/// Format is
|
|
||||||
/// kernel 5
|
|
||||||
/// rss 15
|
|
||||||
/// [...]
|
|
||||||
using Metrics = std::map<std::string, uint64_t>;
|
|
||||||
|
|
||||||
Metrics readAllMetricsFromStatFile(ReadBufferFromFile & buf)
|
|
||||||
{
|
|
||||||
Metrics metrics;
|
|
||||||
while (!buf.eof())
|
|
||||||
{
|
|
||||||
std::string current_key;
|
|
||||||
readStringUntilWhitespace(current_key, buf);
|
|
||||||
|
|
||||||
assertChar(' ', buf);
|
|
||||||
|
|
||||||
uint64_t value = 0;
|
|
||||||
readIntText(value, buf);
|
|
||||||
assertChar('\n', buf);
|
|
||||||
|
|
||||||
auto [_, inserted] = metrics.emplace(std::move(current_key), value);
|
|
||||||
chassert(inserted, "Duplicate keys in stat file");
|
|
||||||
}
|
|
||||||
return metrics;
|
|
||||||
}
|
|
||||||
|
|
||||||
uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & key)
|
|
||||||
{
|
|
||||||
const auto all_metrics = readAllMetricsFromStatFile(buf);
|
|
||||||
if (const auto it = all_metrics.find(key); it != all_metrics.end())
|
|
||||||
return it->second;
|
|
||||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot find '{}' in '{}'", key, buf.getFileName());
|
|
||||||
}
|
|
||||||
|
|
||||||
struct CgroupsV1Reader : ICgroupsReader
|
|
||||||
{
|
|
||||||
explicit CgroupsV1Reader(const fs::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { }
|
|
||||||
|
|
||||||
uint64_t readMemoryUsage() override
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
buf.rewind();
|
|
||||||
return readMetricFromStatFile(buf, "rss");
|
|
||||||
}
|
|
||||||
|
|
||||||
std::string dumpAllStats() override
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
buf.rewind();
|
|
||||||
return fmt::format("{}", readAllMetricsFromStatFile(buf));
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
|
||||||
std::mutex mutex;
|
|
||||||
ReadBufferFromFile buf TSA_GUARDED_BY(mutex);
|
|
||||||
};
|
|
||||||
|
|
||||||
struct CgroupsV2Reader : ICgroupsReader
|
|
||||||
{
|
|
||||||
explicit CgroupsV2Reader(const fs::path & stat_file_dir)
|
|
||||||
: current_buf(stat_file_dir / "memory.current"), stat_buf(stat_file_dir / "memory.stat")
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
uint64_t readMemoryUsage() override
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
current_buf.rewind();
|
|
||||||
stat_buf.rewind();
|
|
||||||
|
|
||||||
int64_t mem_usage = 0;
|
|
||||||
/// memory.current contains a single number
|
|
||||||
/// the reason why we subtract it described here: https://github.com/ClickHouse/ClickHouse/issues/64652#issuecomment-2149630667
|
|
||||||
readIntText(mem_usage, current_buf);
|
|
||||||
mem_usage -= readMetricFromStatFile(stat_buf, "inactive_file");
|
|
||||||
chassert(mem_usage >= 0, "Negative memory usage");
|
|
||||||
return mem_usage;
|
|
||||||
}
|
|
||||||
|
|
||||||
std::string dumpAllStats() override
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
stat_buf.rewind();
|
|
||||||
return fmt::format("{}", readAllMetricsFromStatFile(stat_buf));
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
|
||||||
std::mutex mutex;
|
|
||||||
ReadBufferFromFile current_buf TSA_GUARDED_BY(mutex);
|
|
||||||
ReadBufferFromFile stat_buf TSA_GUARDED_BY(mutex);
|
|
||||||
};
|
|
||||||
|
|
||||||
/// Caveats:
|
|
||||||
/// - All of the logic in this file assumes that the current process is the only process in the
|
|
||||||
/// containing cgroup (or more precisely: the only process with significant memory consumption).
|
|
||||||
/// If this is not the case, then other processe's memory consumption may affect the internal
|
|
||||||
/// memory tracker ...
|
|
||||||
/// - Cgroups v1 and v2 allow nested cgroup hierarchies. As v1 is deprecated for over half a
|
|
||||||
/// decade and will go away at some point, hierarchical detection is only implemented for v2.
|
|
||||||
/// - I did not test what happens if a host has v1 and v2 simultaneously enabled. I believe such
|
|
||||||
/// systems existed only for a short transition period.
|
|
||||||
|
|
||||||
std::optional<std::string> getCgroupsV1Path()
|
|
||||||
{
|
|
||||||
auto path = default_cgroups_mount / "memory/memory.stat";
|
|
||||||
if (!fs::exists(path))
|
|
||||||
return {};
|
|
||||||
return {default_cgroups_mount / "memory"};
|
|
||||||
}
|
|
||||||
|
|
||||||
std::pair<std::string, CgroupsMemoryUsageObserver::CgroupsVersion> getCgroupsPath()
|
|
||||||
{
|
|
||||||
auto v2_path = getCgroupsV2PathContainingFile("memory.current");
|
|
||||||
if (v2_path.has_value())
|
|
||||||
return {*v2_path, CgroupsMemoryUsageObserver::CgroupsVersion::V2};
|
|
||||||
|
|
||||||
auto v1_path = getCgroupsV1Path();
|
|
||||||
if (v1_path.has_value())
|
|
||||||
return {*v1_path, CgroupsMemoryUsageObserver::CgroupsVersion::V1};
|
|
||||||
|
|
||||||
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot find cgroups v1 or v2 current memory file");
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_)
|
CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_)
|
||||||
: log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_)
|
: log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_)
|
||||||
{
|
{}
|
||||||
const auto [cgroup_path, version] = getCgroupsPath();
|
|
||||||
|
|
||||||
cgroup_reader = createCgroupsReader(version, cgroup_path);
|
|
||||||
|
|
||||||
LOG_INFO(
|
|
||||||
log,
|
|
||||||
"Will read the current memory usage from '{}' (cgroups version: {}), wait time is {} sec",
|
|
||||||
cgroup_path,
|
|
||||||
(version == CgroupsVersion::V1) ? "v1" : "v2",
|
|
||||||
wait_time.count());
|
|
||||||
}
|
|
||||||
|
|
||||||
CgroupsMemoryUsageObserver::~CgroupsMemoryUsageObserver()
|
CgroupsMemoryUsageObserver::~CgroupsMemoryUsageObserver()
|
||||||
{
|
{
|
||||||
stopThread();
|
stopThread();
|
||||||
}
|
}
|
||||||
|
|
||||||
void CgroupsMemoryUsageObserver::setMemoryUsageLimits(uint64_t hard_limit_, uint64_t soft_limit_)
|
|
||||||
{
|
|
||||||
std::lock_guard<std::mutex> limit_lock(limit_mutex);
|
|
||||||
|
|
||||||
if (hard_limit_ == hard_limit && soft_limit_ == soft_limit)
|
|
||||||
return;
|
|
||||||
|
|
||||||
hard_limit = hard_limit_;
|
|
||||||
soft_limit = soft_limit_;
|
|
||||||
|
|
||||||
on_hard_limit = [this, hard_limit_](bool up)
|
|
||||||
{
|
|
||||||
if (up)
|
|
||||||
{
|
|
||||||
LOG_WARNING(log, "Exceeded hard memory limit ({})", ReadableSize(hard_limit_));
|
|
||||||
|
|
||||||
/// Update current usage in memory tracker. Also reset free_memory_in_allocator_arenas to zero though we don't know if they are
|
|
||||||
/// really zero. Trying to avoid OOM ...
|
|
||||||
MemoryTracker::setRSS(hard_limit_, 0);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
LOG_INFO(log, "Dropped below hard memory limit ({})", ReadableSize(hard_limit_));
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
on_soft_limit = [this, soft_limit_](bool up)
|
|
||||||
{
|
|
||||||
if (up)
|
|
||||||
{
|
|
||||||
LOG_WARNING(log, "Exceeded soft memory limit ({})", ReadableSize(soft_limit_));
|
|
||||||
|
|
||||||
# if USE_JEMALLOC
|
|
||||||
LOG_INFO(log, "Purging jemalloc arenas");
|
|
||||||
mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0);
|
|
||||||
# endif
|
|
||||||
/// Reset current usage in memory tracker. Expect zero for free_memory_in_allocator_arenas as we just purged them.
|
|
||||||
uint64_t memory_usage = cgroup_reader->readMemoryUsage();
|
|
||||||
LOG_TRACE(
|
|
||||||
log,
|
|
||||||
"Read current memory usage {} bytes ({}) from cgroups, full available stats: {}",
|
|
||||||
memory_usage,
|
|
||||||
ReadableSize(memory_usage),
|
|
||||||
cgroup_reader->dumpAllStats());
|
|
||||||
MemoryTracker::setRSS(memory_usage, 0);
|
|
||||||
|
|
||||||
LOG_INFO(log, "Purged jemalloc arenas. Current memory usage is {}", ReadableSize(memory_usage));
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
LOG_INFO(log, "Dropped below soft memory limit ({})", ReadableSize(soft_limit_));
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
LOG_INFO(log, "Set new limits, soft limit: {}, hard limit: {}", ReadableSize(soft_limit_), ReadableSize(hard_limit_));
|
|
||||||
}
|
|
||||||
|
|
||||||
void CgroupsMemoryUsageObserver::setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_)
|
void CgroupsMemoryUsageObserver::setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_)
|
||||||
{
|
{
|
||||||
std::lock_guard<std::mutex> memory_amount_available_changed_lock(memory_amount_available_changed_mutex);
|
std::lock_guard<std::mutex> memory_amount_available_changed_lock(memory_amount_available_changed_mutex);
|
||||||
@ -300,35 +82,6 @@ void CgroupsMemoryUsageObserver::runThread()
|
|||||||
std::lock_guard<std::mutex> memory_amount_available_changed_lock(memory_amount_available_changed_mutex);
|
std::lock_guard<std::mutex> memory_amount_available_changed_lock(memory_amount_available_changed_mutex);
|
||||||
on_memory_amount_available_changed();
|
on_memory_amount_available_changed();
|
||||||
}
|
}
|
||||||
|
|
||||||
std::lock_guard<std::mutex> limit_lock(limit_mutex);
|
|
||||||
if (soft_limit > 0 && hard_limit > 0)
|
|
||||||
{
|
|
||||||
uint64_t memory_usage = cgroup_reader->readMemoryUsage();
|
|
||||||
LOG_TRACE(log, "Read current memory usage {} bytes ({}) from cgroups", memory_usage, ReadableSize(memory_usage));
|
|
||||||
if (memory_usage > hard_limit)
|
|
||||||
{
|
|
||||||
if (last_memory_usage <= hard_limit)
|
|
||||||
on_hard_limit(true);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
if (last_memory_usage > hard_limit)
|
|
||||||
on_hard_limit(false);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (memory_usage > soft_limit)
|
|
||||||
{
|
|
||||||
if (last_memory_usage <= soft_limit)
|
|
||||||
on_soft_limit(true);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
if (last_memory_usage > soft_limit)
|
|
||||||
on_soft_limit(false);
|
|
||||||
}
|
|
||||||
last_memory_usage = memory_usage;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
@ -337,13 +90,6 @@ void CgroupsMemoryUsageObserver::runThread()
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
std::unique_ptr<ICgroupsReader> createCgroupsReader(CgroupsMemoryUsageObserver::CgroupsVersion version, const fs::path & cgroup_path)
|
|
||||||
{
|
|
||||||
if (version == CgroupsMemoryUsageObserver::CgroupsVersion::V2)
|
|
||||||
return std::make_unique<CgroupsV2Reader>(cgroup_path);
|
|
||||||
else
|
|
||||||
return std::make_unique<CgroupsV1Reader>(cgroup_path);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
@ -3,30 +3,12 @@
|
|||||||
#include <Common/ThreadPool.h>
|
#include <Common/ThreadPool.h>
|
||||||
|
|
||||||
#include <chrono>
|
#include <chrono>
|
||||||
#include <memory>
|
|
||||||
#include <mutex>
|
#include <mutex>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct ICgroupsReader
|
/// Periodically reads the the maximum memory available to the process (which can change due to cgroups settings).
|
||||||
{
|
|
||||||
virtual ~ICgroupsReader() = default;
|
|
||||||
|
|
||||||
virtual uint64_t readMemoryUsage() = 0;
|
|
||||||
|
|
||||||
virtual std::string dumpAllStats() = 0;
|
|
||||||
};
|
|
||||||
|
|
||||||
/// Does two things:
|
|
||||||
/// 1. Periodically reads the memory usage of the process from Linux cgroups.
|
|
||||||
/// You can specify soft or hard memory limits:
|
|
||||||
/// - When the soft memory limit is hit, drop jemalloc cache.
|
|
||||||
/// - When the hard memory limit is hit, update MemoryTracking metric to throw memory exceptions faster.
|
|
||||||
/// The goal of this is to avoid that the process hits the maximum allowed memory limit at which there is a good
|
|
||||||
/// chance that the Limux OOM killer terminates it. All of this is done is because internal memory tracking in
|
|
||||||
/// ClickHouse can unfortunately under-estimate the actually used memory.
|
|
||||||
/// 2. Periodically reads the the maximum memory available to the process (which can change due to cgroups settings).
|
|
||||||
/// You can specify a callback to react on changes. The callback typically reloads the configuration, i.e. Server
|
/// You can specify a callback to react on changes. The callback typically reloads the configuration, i.e. Server
|
||||||
/// or Keeper configuration file. This reloads settings 'max_server_memory_usage' (Server) and 'max_memory_usage_soft_limit'
|
/// or Keeper configuration file. This reloads settings 'max_server_memory_usage' (Server) and 'max_memory_usage_soft_limit'
|
||||||
/// (Keeper) from which various other internal limits are calculated, including the soft and hard limits for (1.).
|
/// (Keeper) from which various other internal limits are calculated, including the soft and hard limits for (1.).
|
||||||
@ -37,19 +19,11 @@ struct ICgroupsReader
|
|||||||
class CgroupsMemoryUsageObserver
|
class CgroupsMemoryUsageObserver
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
using OnMemoryLimitFn = std::function<void(bool)>;
|
|
||||||
using OnMemoryAmountAvailableChangedFn = std::function<void()>;
|
using OnMemoryAmountAvailableChangedFn = std::function<void()>;
|
||||||
|
|
||||||
enum class CgroupsVersion : uint8_t
|
|
||||||
{
|
|
||||||
V1,
|
|
||||||
V2
|
|
||||||
};
|
|
||||||
|
|
||||||
explicit CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_);
|
explicit CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_);
|
||||||
~CgroupsMemoryUsageObserver();
|
~CgroupsMemoryUsageObserver();
|
||||||
|
|
||||||
void setMemoryUsageLimits(uint64_t hard_limit_, uint64_t soft_limit_);
|
|
||||||
void setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_);
|
void setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_);
|
||||||
|
|
||||||
void startThread();
|
void startThread();
|
||||||
@ -60,32 +34,22 @@ private:
|
|||||||
const std::chrono::seconds wait_time;
|
const std::chrono::seconds wait_time;
|
||||||
|
|
||||||
std::mutex limit_mutex;
|
std::mutex limit_mutex;
|
||||||
size_t hard_limit TSA_GUARDED_BY(limit_mutex) = 0;
|
|
||||||
size_t soft_limit TSA_GUARDED_BY(limit_mutex) = 0;
|
|
||||||
OnMemoryLimitFn on_hard_limit TSA_GUARDED_BY(limit_mutex);
|
|
||||||
OnMemoryLimitFn on_soft_limit TSA_GUARDED_BY(limit_mutex);
|
|
||||||
|
|
||||||
std::mutex memory_amount_available_changed_mutex;
|
std::mutex memory_amount_available_changed_mutex;
|
||||||
OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed TSA_GUARDED_BY(memory_amount_available_changed_mutex);
|
OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed TSA_GUARDED_BY(memory_amount_available_changed_mutex);
|
||||||
|
|
||||||
uint64_t last_memory_usage = 0; /// how much memory does the process use
|
|
||||||
uint64_t last_available_memory_amount; /// how much memory can the process use
|
uint64_t last_available_memory_amount; /// how much memory can the process use
|
||||||
|
|
||||||
void stopThread();
|
void stopThread();
|
||||||
|
|
||||||
void runThread();
|
void runThread();
|
||||||
|
|
||||||
std::unique_ptr<ICgroupsReader> cgroup_reader;
|
|
||||||
|
|
||||||
std::mutex thread_mutex;
|
std::mutex thread_mutex;
|
||||||
std::condition_variable cond;
|
std::condition_variable cond;
|
||||||
ThreadFromGlobalPool thread;
|
ThreadFromGlobalPool thread;
|
||||||
bool quit = false;
|
bool quit = false;
|
||||||
};
|
};
|
||||||
|
|
||||||
std::unique_ptr<ICgroupsReader>
|
|
||||||
createCgroupsReader(CgroupsMemoryUsageObserver::CgroupsVersion version, const std::filesystem::path & cgroup_path);
|
|
||||||
|
|
||||||
#else
|
#else
|
||||||
class CgroupsMemoryUsageObserver
|
class CgroupsMemoryUsageObserver
|
||||||
{
|
{
|
||||||
|
@ -5,7 +5,6 @@
|
|||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
#include <Common/Stopwatch.h>
|
#include <Common/Stopwatch.h>
|
||||||
#include <Common/logger_useful.h>
|
#include <Common/logger_useful.h>
|
||||||
#include <jemalloc/jemalloc.h>
|
|
||||||
|
|
||||||
#define STRINGIFY_HELPER(x) #x
|
#define STRINGIFY_HELPER(x) #x
|
||||||
#define STRINGIFY(x) STRINGIFY_HELPER(x)
|
#define STRINGIFY(x) STRINGIFY_HELPER(x)
|
||||||
@ -26,7 +25,6 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
void purgeJemallocArenas()
|
void purgeJemallocArenas()
|
||||||
{
|
{
|
||||||
LOG_TRACE(getLogger("SystemJemalloc"), "Purging unused memory");
|
|
||||||
Stopwatch watch;
|
Stopwatch watch;
|
||||||
mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0);
|
mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0);
|
||||||
ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge);
|
ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge);
|
||||||
@ -46,20 +44,6 @@ void checkJemallocProfilingEnabled()
|
|||||||
"set: MALLOC_CONF=background_thread:true,prof:true");
|
"set: MALLOC_CONF=background_thread:true,prof:true");
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename T>
|
|
||||||
void setJemallocValue(const char * name, T value)
|
|
||||||
{
|
|
||||||
T old_value;
|
|
||||||
size_t old_value_size = sizeof(T);
|
|
||||||
if (mallctl(name, &old_value, &old_value_size, reinterpret_cast<void*>(&value), sizeof(T)))
|
|
||||||
{
|
|
||||||
LOG_WARNING(getLogger("Jemalloc"), "mallctl for {} failed", name);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
LOG_INFO(getLogger("Jemalloc"), "Value for {} set to {} (from {})", name, value, old_value);
|
|
||||||
}
|
|
||||||
|
|
||||||
void setJemallocProfileActive(bool value)
|
void setJemallocProfileActive(bool value)
|
||||||
{
|
{
|
||||||
checkJemallocProfilingEnabled();
|
checkJemallocProfilingEnabled();
|
||||||
|
@ -5,6 +5,8 @@
|
|||||||
#if USE_JEMALLOC
|
#if USE_JEMALLOC
|
||||||
|
|
||||||
#include <string>
|
#include <string>
|
||||||
|
#include <Common/logger_useful.h>
|
||||||
|
#include <jemalloc/jemalloc.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -21,6 +23,59 @@ void setJemallocBackgroundThreads(bool enabled);
|
|||||||
|
|
||||||
void setJemallocMaxBackgroundThreads(size_t max_threads);
|
void setJemallocMaxBackgroundThreads(size_t max_threads);
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
void setJemallocValue(const char * name, T value)
|
||||||
|
{
|
||||||
|
T old_value;
|
||||||
|
size_t old_value_size = sizeof(T);
|
||||||
|
mallctl(name, &old_value, &old_value_size, reinterpret_cast<void*>(&value), sizeof(T));
|
||||||
|
LOG_INFO(getLogger("Jemalloc"), "Value for {} set to {} (from {})", name, value, old_value);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
T getJemallocValue(const char * name)
|
||||||
|
{
|
||||||
|
T value;
|
||||||
|
size_t value_size = sizeof(T);
|
||||||
|
mallctl(name, &value, &value_size, nullptr, 0);
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Each mallctl call consists of string name lookup which can be expensive.
|
||||||
|
/// This can be avoided by translating name to "Management Information Base" (MIB)
|
||||||
|
/// and using it in mallctlbymib calls
|
||||||
|
template <typename T>
|
||||||
|
struct JemallocMibCache
|
||||||
|
{
|
||||||
|
explicit JemallocMibCache(const char * name)
|
||||||
|
{
|
||||||
|
mallctlnametomib(name, mib, &mib_length);
|
||||||
|
}
|
||||||
|
|
||||||
|
void setValue(T value)
|
||||||
|
{
|
||||||
|
mallctlbymib(mib, mib_length, nullptr, nullptr, reinterpret_cast<void*>(&value), sizeof(T));
|
||||||
|
}
|
||||||
|
|
||||||
|
T getValue()
|
||||||
|
{
|
||||||
|
T value;
|
||||||
|
size_t value_size = sizeof(T);
|
||||||
|
mallctlbymib(mib, mib_length, &value, &value_size, nullptr, 0);
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
|
||||||
|
void run()
|
||||||
|
{
|
||||||
|
mallctlbymib(mib, mib_length, nullptr, nullptr, nullptr, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
static constexpr size_t max_mib_length = 4;
|
||||||
|
size_t mib[max_mib_length];
|
||||||
|
size_t mib_length = max_mib_length;
|
||||||
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
@ -20,13 +20,9 @@
|
|||||||
#if USE_JEMALLOC
|
#if USE_JEMALLOC
|
||||||
# include <jemalloc/jemalloc.h>
|
# include <jemalloc/jemalloc.h>
|
||||||
|
|
||||||
#define STRINGIFY_HELPER(x) #x
|
|
||||||
#define STRINGIFY(x) STRINGIFY_HELPER(x)
|
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#include <atomic>
|
#include <atomic>
|
||||||
#include <cmath>
|
|
||||||
#include <random>
|
#include <random>
|
||||||
#include <cstdlib>
|
#include <cstdlib>
|
||||||
#include <string>
|
#include <string>
|
||||||
@ -115,8 +111,6 @@ void AllocationTrace::onFreeImpl(void * ptr, size_t size) const
|
|||||||
namespace ProfileEvents
|
namespace ProfileEvents
|
||||||
{
|
{
|
||||||
extern const Event QueryMemoryLimitExceeded;
|
extern const Event QueryMemoryLimitExceeded;
|
||||||
extern const Event MemoryAllocatorPurge;
|
|
||||||
extern const Event MemoryAllocatorPurgeTimeMicroseconds;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
using namespace std::chrono_literals;
|
using namespace std::chrono_literals;
|
||||||
@ -126,15 +120,13 @@ static constexpr size_t log_peak_memory_usage_every = 1ULL << 30;
|
|||||||
MemoryTracker total_memory_tracker(nullptr, VariableContext::Global);
|
MemoryTracker total_memory_tracker(nullptr, VariableContext::Global);
|
||||||
MemoryTracker background_memory_tracker(&total_memory_tracker, VariableContext::User, false);
|
MemoryTracker background_memory_tracker(&total_memory_tracker, VariableContext::User, false);
|
||||||
|
|
||||||
std::atomic<Int64> MemoryTracker::free_memory_in_allocator_arenas;
|
|
||||||
|
|
||||||
MemoryTracker::MemoryTracker(VariableContext level_) : parent(&total_memory_tracker), level(level_) {}
|
MemoryTracker::MemoryTracker(VariableContext level_) : parent(&total_memory_tracker), level(level_) {}
|
||||||
MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : parent(parent_), level(level_) {}
|
MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : parent(parent_), level(level_) {}
|
||||||
|
|
||||||
MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_, bool log_peak_memory_usage_in_destructor_)
|
MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_, bool log_peak_memory_usage_in_destructor_)
|
||||||
: parent(parent_)
|
: parent(parent_), log_peak_memory_usage_in_destructor(log_peak_memory_usage_in_destructor_), level(level_)
|
||||||
, log_peak_memory_usage_in_destructor(log_peak_memory_usage_in_destructor_)
|
{
|
||||||
, level(level_)
|
}
|
||||||
{}
|
|
||||||
|
|
||||||
MemoryTracker::~MemoryTracker()
|
MemoryTracker::~MemoryTracker()
|
||||||
{
|
{
|
||||||
@ -204,10 +196,14 @@ void MemoryTracker::debugLogBigAllocationWithoutCheck(Int64 size [[maybe_unused]
|
|||||||
return;
|
return;
|
||||||
|
|
||||||
MemoryTrackerBlockerInThread blocker(VariableContext::Global);
|
MemoryTrackerBlockerInThread blocker(VariableContext::Global);
|
||||||
LOG_TEST(getLogger("MemoryTracker"), "Too big allocation ({} bytes) without checking memory limits, "
|
LOG_TEST(
|
||||||
"it may lead to OOM. Stack trace: {}", size, StackTrace().toString());
|
getLogger("MemoryTracker"),
|
||||||
|
"Too big allocation ({} bytes) without checking memory limits, "
|
||||||
|
"it may lead to OOM. Stack trace: {}",
|
||||||
|
size,
|
||||||
|
StackTrace().toString());
|
||||||
#else
|
#else
|
||||||
return; /// Avoid trash logging in release builds
|
/// Avoid trash logging in release builds
|
||||||
#endif
|
#endif
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -228,6 +224,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed
|
|||||||
{
|
{
|
||||||
/// For global memory tracker always update memory usage.
|
/// For global memory tracker always update memory usage.
|
||||||
amount.fetch_add(size, std::memory_order_relaxed);
|
amount.fetch_add(size, std::memory_order_relaxed);
|
||||||
|
rss.fetch_add(size, std::memory_order_relaxed);
|
||||||
|
|
||||||
auto metric_loaded = metric.load(std::memory_order_relaxed);
|
auto metric_loaded = metric.load(std::memory_order_relaxed);
|
||||||
if (metric_loaded != CurrentMetrics::end())
|
if (metric_loaded != CurrentMetrics::end())
|
||||||
@ -249,6 +246,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed
|
|||||||
* So, we allow over-allocations.
|
* So, we allow over-allocations.
|
||||||
*/
|
*/
|
||||||
Int64 will_be = size ? size + amount.fetch_add(size, std::memory_order_relaxed) : amount.load(std::memory_order_relaxed);
|
Int64 will_be = size ? size + amount.fetch_add(size, std::memory_order_relaxed) : amount.load(std::memory_order_relaxed);
|
||||||
|
Int64 will_be_rss = size ? size + rss.fetch_add(size, std::memory_order_relaxed) : rss.load(std::memory_order_relaxed);
|
||||||
|
|
||||||
auto metric_loaded = metric.load(std::memory_order_relaxed);
|
auto metric_loaded = metric.load(std::memory_order_relaxed);
|
||||||
if (metric_loaded != CurrentMetrics::end() && size)
|
if (metric_loaded != CurrentMetrics::end() && size)
|
||||||
@ -275,6 +273,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed
|
|||||||
{
|
{
|
||||||
/// Revert
|
/// Revert
|
||||||
amount.fetch_sub(size, std::memory_order_relaxed);
|
amount.fetch_sub(size, std::memory_order_relaxed);
|
||||||
|
rss.fetch_sub(size, std::memory_order_relaxed);
|
||||||
|
|
||||||
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
||||||
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
||||||
@ -297,33 +296,8 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
Int64 limit_to_check = current_hard_limit;
|
if (unlikely(
|
||||||
|
current_hard_limit && (will_be > current_hard_limit || (level == VariableContext::Global && will_be_rss > current_hard_limit))))
|
||||||
#if USE_JEMALLOC
|
|
||||||
if (level == VariableContext::Global && allow_use_jemalloc_memory.load(std::memory_order_relaxed))
|
|
||||||
{
|
|
||||||
/// Jemalloc arenas may keep some extra memory.
|
|
||||||
/// This memory was substucted from RSS to decrease memory drift.
|
|
||||||
/// In case memory is close to limit, try to pugre the arenas.
|
|
||||||
/// This is needed to avoid OOM, because some allocations are directly done with mmap.
|
|
||||||
Int64 current_free_memory_in_allocator_arenas = free_memory_in_allocator_arenas.load(std::memory_order_relaxed);
|
|
||||||
|
|
||||||
if (current_free_memory_in_allocator_arenas > 0 && current_hard_limit && current_free_memory_in_allocator_arenas + will_be > current_hard_limit)
|
|
||||||
{
|
|
||||||
if (free_memory_in_allocator_arenas.exchange(-current_free_memory_in_allocator_arenas) > 0)
|
|
||||||
{
|
|
||||||
Stopwatch watch;
|
|
||||||
mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0);
|
|
||||||
ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge);
|
|
||||||
ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, watch.elapsedMicroseconds());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
limit_to_check += abs(current_free_memory_in_allocator_arenas);
|
|
||||||
}
|
|
||||||
#endif
|
|
||||||
|
|
||||||
if (unlikely(current_hard_limit && will_be > limit_to_check))
|
|
||||||
{
|
{
|
||||||
if (memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded)
|
if (memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded)
|
||||||
{
|
{
|
||||||
@ -335,6 +309,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed
|
|||||||
{
|
{
|
||||||
/// Revert
|
/// Revert
|
||||||
amount.fetch_sub(size, std::memory_order_relaxed);
|
amount.fetch_sub(size, std::memory_order_relaxed);
|
||||||
|
rss.fetch_sub(size, std::memory_order_relaxed);
|
||||||
|
|
||||||
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
||||||
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
|
||||||
@ -343,12 +318,13 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed
|
|||||||
throw DB::Exception(
|
throw DB::Exception(
|
||||||
DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED,
|
DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED,
|
||||||
"Memory limit{}{} exceeded: "
|
"Memory limit{}{} exceeded: "
|
||||||
"would use {} (attempt to allocate chunk of {} bytes), maximum: {}."
|
"would use {} (attempt to allocate chunk of {} bytes), current RSS {}, maximum: {}."
|
||||||
"{}{}",
|
"{}{}",
|
||||||
description ? " " : "",
|
description ? " " : "",
|
||||||
description ? description : "",
|
description ? description : "",
|
||||||
formatReadableSizeWithBinarySuffix(will_be),
|
formatReadableSizeWithBinarySuffix(will_be),
|
||||||
size,
|
size,
|
||||||
|
formatReadableSizeWithBinarySuffix(rss.load(std::memory_order_relaxed)),
|
||||||
formatReadableSizeWithBinarySuffix(current_hard_limit),
|
formatReadableSizeWithBinarySuffix(current_hard_limit),
|
||||||
overcommit_result == OvercommitResult::NONE ? "" : " OvercommitTracker decision: ",
|
overcommit_result == OvercommitResult::NONE ? "" : " OvercommitTracker decision: ",
|
||||||
toDescription(overcommit_result));
|
toDescription(overcommit_result));
|
||||||
@ -442,6 +418,7 @@ AllocationTrace MemoryTracker::free(Int64 size, double _sample_probability)
|
|||||||
{
|
{
|
||||||
/// For global memory tracker always update memory usage.
|
/// For global memory tracker always update memory usage.
|
||||||
amount.fetch_sub(size, std::memory_order_relaxed);
|
amount.fetch_sub(size, std::memory_order_relaxed);
|
||||||
|
rss.fetch_sub(size, std::memory_order_relaxed);
|
||||||
auto metric_loaded = metric.load(std::memory_order_relaxed);
|
auto metric_loaded = metric.load(std::memory_order_relaxed);
|
||||||
if (metric_loaded != CurrentMetrics::end())
|
if (metric_loaded != CurrentMetrics::end())
|
||||||
CurrentMetrics::sub(metric_loaded, size);
|
CurrentMetrics::sub(metric_loaded, size);
|
||||||
@ -455,7 +432,12 @@ AllocationTrace MemoryTracker::free(Int64 size, double _sample_probability)
|
|||||||
}
|
}
|
||||||
|
|
||||||
Int64 accounted_size = size;
|
Int64 accounted_size = size;
|
||||||
if (level == VariableContext::Thread || level == VariableContext::Global)
|
if (level == VariableContext::Global)
|
||||||
|
{
|
||||||
|
amount.fetch_sub(accounted_size, std::memory_order_relaxed);
|
||||||
|
rss.fetch_sub(accounted_size, std::memory_order_relaxed);
|
||||||
|
}
|
||||||
|
else if (level == VariableContext::Thread)
|
||||||
{
|
{
|
||||||
/// Could become negative if memory allocated in this thread is freed in another one
|
/// Could become negative if memory allocated in this thread is freed in another one
|
||||||
amount.fetch_sub(accounted_size, std::memory_order_relaxed);
|
amount.fetch_sub(accounted_size, std::memory_order_relaxed);
|
||||||
@ -529,21 +511,29 @@ void MemoryTracker::reset()
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void MemoryTracker::setRSS(Int64 rss_, Int64 free_memory_in_allocator_arenas_)
|
void MemoryTracker::updateRSS(Int64 rss_)
|
||||||
{
|
{
|
||||||
Int64 new_amount = rss_;
|
total_memory_tracker.rss.store(rss_, std::memory_order_relaxed);
|
||||||
|
}
|
||||||
|
|
||||||
|
void MemoryTracker::updateAllocated(Int64 allocated_)
|
||||||
|
{
|
||||||
|
Int64 new_amount = allocated_;
|
||||||
|
LOG_INFO(
|
||||||
|
getLogger("MemoryTracker"),
|
||||||
|
"Correcting the value of global memory tracker from {} to {}",
|
||||||
|
ReadableSize(total_memory_tracker.amount.load(std::memory_order_relaxed)),
|
||||||
|
ReadableSize(allocated_));
|
||||||
total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed);
|
total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed);
|
||||||
free_memory_in_allocator_arenas.store(free_memory_in_allocator_arenas_, std::memory_order_relaxed);
|
|
||||||
|
|
||||||
auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed);
|
auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed);
|
||||||
if (metric_loaded != CurrentMetrics::end())
|
if (metric_loaded != CurrentMetrics::end())
|
||||||
CurrentMetrics::set(metric_loaded, new_amount);
|
CurrentMetrics::set(metric_loaded, new_amount);
|
||||||
|
|
||||||
bool log_memory_usage = true;
|
bool log_memory_usage = true;
|
||||||
total_memory_tracker.updatePeak(rss_, log_memory_usage);
|
total_memory_tracker.updatePeak(new_amount, log_memory_usage);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void MemoryTracker::setSoftLimit(Int64 value)
|
void MemoryTracker::setSoftLimit(Int64 value)
|
||||||
{
|
{
|
||||||
soft_limit.store(value, std::memory_order_relaxed);
|
soft_limit.store(value, std::memory_order_relaxed);
|
||||||
|
@ -2,7 +2,6 @@
|
|||||||
|
|
||||||
#include <atomic>
|
#include <atomic>
|
||||||
#include <chrono>
|
#include <chrono>
|
||||||
#include <optional>
|
|
||||||
#include <base/types.h>
|
#include <base/types.h>
|
||||||
#include <Common/CurrentMetrics.h>
|
#include <Common/CurrentMetrics.h>
|
||||||
#include <Common/VariableContext.h>
|
#include <Common/VariableContext.h>
|
||||||
@ -57,9 +56,8 @@ private:
|
|||||||
std::atomic<Int64> soft_limit {0};
|
std::atomic<Int64> soft_limit {0};
|
||||||
std::atomic<Int64> hard_limit {0};
|
std::atomic<Int64> hard_limit {0};
|
||||||
std::atomic<Int64> profiler_limit {0};
|
std::atomic<Int64> profiler_limit {0};
|
||||||
std::atomic_bool allow_use_jemalloc_memory {true};
|
|
||||||
|
|
||||||
static std::atomic<Int64> free_memory_in_allocator_arenas;
|
std::atomic<Int64> rss{0};
|
||||||
|
|
||||||
Int64 profiler_step = 0;
|
Int64 profiler_step = 0;
|
||||||
|
|
||||||
@ -122,6 +120,11 @@ public:
|
|||||||
return amount.load(std::memory_order_relaxed);
|
return amount.load(std::memory_order_relaxed);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Int64 getRSS() const
|
||||||
|
{
|
||||||
|
return rss.load(std::memory_order_relaxed);
|
||||||
|
}
|
||||||
|
|
||||||
// Merges and mutations may pass memory ownership to other threads thus in the end of execution
|
// Merges and mutations may pass memory ownership to other threads thus in the end of execution
|
||||||
// MemoryTracker for background task may have a non-zero counter.
|
// MemoryTracker for background task may have a non-zero counter.
|
||||||
// This method is intended to fix the counter inside of background_memory_tracker.
|
// This method is intended to fix the counter inside of background_memory_tracker.
|
||||||
@ -154,14 +157,6 @@ public:
|
|||||||
{
|
{
|
||||||
return soft_limit.load(std::memory_order_relaxed);
|
return soft_limit.load(std::memory_order_relaxed);
|
||||||
}
|
}
|
||||||
void setAllowUseJemallocMemory(bool value)
|
|
||||||
{
|
|
||||||
allow_use_jemalloc_memory.store(value, std::memory_order_relaxed);
|
|
||||||
}
|
|
||||||
bool getAllowUseJemallocMmemory() const
|
|
||||||
{
|
|
||||||
return allow_use_jemalloc_memory.load(std::memory_order_relaxed);
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Set limit if it was not set.
|
/** Set limit if it was not set.
|
||||||
* Otherwise, set limit to new value, if new value is greater than previous limit.
|
* Otherwise, set limit to new value, if new value is greater than previous limit.
|
||||||
@ -249,10 +244,9 @@ public:
|
|||||||
/// Reset the accumulated data.
|
/// Reset the accumulated data.
|
||||||
void reset();
|
void reset();
|
||||||
|
|
||||||
/// Reset current counter to an RSS value.
|
/// update values based on external information (e.g. jemalloc's stat)
|
||||||
/// Jemalloc may have pre-allocated arenas, they are accounted in RSS.
|
static void updateRSS(Int64 rss_);
|
||||||
/// We can free this arenas in case of exception to avoid OOM.
|
static void updateAllocated(Int64 allocated_);
|
||||||
static void setRSS(Int64 rss_, Int64 free_memory_in_allocator_arenas_);
|
|
||||||
|
|
||||||
/// Prints info about peak memory consumption into log.
|
/// Prints info about peak memory consumption into log.
|
||||||
void logPeakMemoryUsage();
|
void logPeakMemoryUsage();
|
||||||
|
333
src/Common/MemoryWorker.cpp
Normal file
333
src/Common/MemoryWorker.cpp
Normal file
@ -0,0 +1,333 @@
|
|||||||
|
#include <Common/MemoryWorker.h>
|
||||||
|
|
||||||
|
#include <IO/ReadBufferFromFile.h>
|
||||||
|
#include <IO/ReadBufferFromFileDescriptor.h>
|
||||||
|
#include <IO/ReadHelpers.h>
|
||||||
|
#include <base/cgroupsv2.h>
|
||||||
|
#include <Common/Jemalloc.h>
|
||||||
|
#include <Common/MemoryTracker.h>
|
||||||
|
#include <Common/ProfileEvents.h>
|
||||||
|
#include <Common/formatReadable.h>
|
||||||
|
#include <Common/logger_useful.h>
|
||||||
|
|
||||||
|
#include <fmt/ranges.h>
|
||||||
|
|
||||||
|
#include <filesystem>
|
||||||
|
#include <optional>
|
||||||
|
|
||||||
|
namespace fs = std::filesystem;
|
||||||
|
|
||||||
|
namespace ProfileEvents
|
||||||
|
{
|
||||||
|
extern const Event MemoryAllocatorPurge;
|
||||||
|
extern const Event MemoryAllocatorPurgeTimeMicroseconds;
|
||||||
|
extern const Event MemoryWorkerRun;
|
||||||
|
extern const Event MemoryWorkerRunElapsedMicroseconds;
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int FILE_DOESNT_EXIST;
|
||||||
|
extern const int LOGICAL_ERROR;
|
||||||
|
}
|
||||||
|
|
||||||
|
#if defined(OS_LINUX)
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
using Metrics = std::map<std::string, uint64_t>;
|
||||||
|
|
||||||
|
/// Format is
|
||||||
|
/// kernel 5
|
||||||
|
/// rss 15
|
||||||
|
/// [...]
|
||||||
|
Metrics readAllMetricsFromStatFile(ReadBufferFromFile & buf)
|
||||||
|
{
|
||||||
|
Metrics metrics;
|
||||||
|
while (!buf.eof())
|
||||||
|
{
|
||||||
|
std::string current_key;
|
||||||
|
readStringUntilWhitespace(current_key, buf);
|
||||||
|
|
||||||
|
assertChar(' ', buf);
|
||||||
|
|
||||||
|
uint64_t value = 0;
|
||||||
|
readIntText(value, buf);
|
||||||
|
assertChar('\n', buf);
|
||||||
|
|
||||||
|
auto [_, inserted] = metrics.emplace(std::move(current_key), value);
|
||||||
|
chassert(inserted, "Duplicate keys in stat file");
|
||||||
|
}
|
||||||
|
return metrics;
|
||||||
|
}
|
||||||
|
|
||||||
|
uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, std::string_view key)
|
||||||
|
{
|
||||||
|
while (!buf.eof())
|
||||||
|
{
|
||||||
|
std::string current_key;
|
||||||
|
readStringUntilWhitespace(current_key, buf);
|
||||||
|
if (current_key != key)
|
||||||
|
{
|
||||||
|
std::string dummy;
|
||||||
|
readStringUntilNewlineInto(dummy, buf);
|
||||||
|
buf.ignore();
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
assertChar(' ', buf);
|
||||||
|
uint64_t value = 0;
|
||||||
|
readIntText(value, buf);
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
LOG_ERROR(getLogger("CgroupsReader"), "Cannot find '{}' in '{}'", key, buf.getFileName());
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
struct CgroupsV1Reader : ICgroupsReader
|
||||||
|
{
|
||||||
|
explicit CgroupsV1Reader(const fs::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { }
|
||||||
|
|
||||||
|
uint64_t readMemoryUsage() override
|
||||||
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
buf.rewind();
|
||||||
|
return readMetricFromStatFile(buf, "rss");
|
||||||
|
}
|
||||||
|
|
||||||
|
std::string dumpAllStats() override
|
||||||
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
buf.rewind();
|
||||||
|
return fmt::format("{}", readAllMetricsFromStatFile(buf));
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
std::mutex mutex;
|
||||||
|
ReadBufferFromFile buf TSA_GUARDED_BY(mutex);
|
||||||
|
};
|
||||||
|
|
||||||
|
struct CgroupsV2Reader : ICgroupsReader
|
||||||
|
{
|
||||||
|
explicit CgroupsV2Reader(const fs::path & stat_file_dir) : stat_buf(stat_file_dir / "memory.stat") { }
|
||||||
|
|
||||||
|
uint64_t readMemoryUsage() override
|
||||||
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
stat_buf.rewind();
|
||||||
|
return readMetricFromStatFile(stat_buf, "anon");
|
||||||
|
}
|
||||||
|
|
||||||
|
std::string dumpAllStats() override
|
||||||
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
stat_buf.rewind();
|
||||||
|
return fmt::format("{}", readAllMetricsFromStatFile(stat_buf));
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
std::mutex mutex;
|
||||||
|
ReadBufferFromFile stat_buf TSA_GUARDED_BY(mutex);
|
||||||
|
};
|
||||||
|
|
||||||
|
/// Caveats:
|
||||||
|
/// - All of the logic in this file assumes that the current process is the only process in the
|
||||||
|
/// containing cgroup (or more precisely: the only process with significant memory consumption).
|
||||||
|
/// If this is not the case, then other processe's memory consumption may affect the internal
|
||||||
|
/// memory tracker ...
|
||||||
|
/// - Cgroups v1 and v2 allow nested cgroup hierarchies. As v1 is deprecated for over half a
|
||||||
|
/// decade and will go away at some point, hierarchical detection is only implemented for v2.
|
||||||
|
/// - I did not test what happens if a host has v1 and v2 simultaneously enabled. I believe such
|
||||||
|
/// systems existed only for a short transition period.
|
||||||
|
|
||||||
|
std::optional<std::string> getCgroupsV1Path()
|
||||||
|
{
|
||||||
|
auto path = default_cgroups_mount / "memory/memory.stat";
|
||||||
|
if (!fs::exists(path))
|
||||||
|
return {};
|
||||||
|
return {default_cgroups_mount / "memory"};
|
||||||
|
}
|
||||||
|
|
||||||
|
std::pair<std::string, ICgroupsReader::CgroupsVersion> getCgroupsPath()
|
||||||
|
{
|
||||||
|
auto v2_path = getCgroupsV2PathContainingFile("memory.current");
|
||||||
|
if (v2_path.has_value())
|
||||||
|
return {*v2_path, ICgroupsReader::CgroupsVersion::V2};
|
||||||
|
|
||||||
|
auto v1_path = getCgroupsV1Path();
|
||||||
|
if (v1_path.has_value())
|
||||||
|
return {*v1_path, ICgroupsReader::CgroupsVersion::V1};
|
||||||
|
|
||||||
|
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot find cgroups v1 or v2 current memory file");
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
std::shared_ptr<ICgroupsReader> ICgroupsReader::createCgroupsReader(ICgroupsReader::CgroupsVersion version, const std::filesystem::path & cgroup_path)
|
||||||
|
{
|
||||||
|
if (version == CgroupsVersion::V2)
|
||||||
|
return std::make_shared<CgroupsV2Reader>(cgroup_path);
|
||||||
|
else
|
||||||
|
{
|
||||||
|
chassert(version == CgroupsVersion::V1);
|
||||||
|
return std::make_shared<CgroupsV1Reader>(cgroup_path);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
#endif
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
std::string_view sourceToString(MemoryWorker::MemoryUsageSource source)
|
||||||
|
{
|
||||||
|
switch (source)
|
||||||
|
{
|
||||||
|
case MemoryWorker::MemoryUsageSource::Cgroups: return "Cgroups";
|
||||||
|
case MemoryWorker::MemoryUsageSource::Jemalloc: return "Jemalloc";
|
||||||
|
case MemoryWorker::MemoryUsageSource::None: return "None";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/// We try to pick the best possible supported source for reading memory usage.
|
||||||
|
/// Supported sources in order of priority
|
||||||
|
/// - reading from cgroups' pseudo-files (fastest and most accurate)
|
||||||
|
/// - reading jemalloc's resident stat (doesn't take into account allocations that didn't use jemalloc)
|
||||||
|
/// Also, different tick rates are used because not all options are equally fast
|
||||||
|
MemoryWorker::MemoryWorker(uint64_t period_ms_)
|
||||||
|
: log(getLogger("MemoryWorker"))
|
||||||
|
, period_ms(period_ms_)
|
||||||
|
{
|
||||||
|
#if defined(OS_LINUX)
|
||||||
|
try
|
||||||
|
{
|
||||||
|
static constexpr uint64_t cgroups_memory_usage_tick_ms{50};
|
||||||
|
|
||||||
|
const auto [cgroup_path, version] = getCgroupsPath();
|
||||||
|
LOG_INFO(
|
||||||
|
getLogger("CgroupsReader"),
|
||||||
|
"Will create cgroup reader from '{}' (cgroups version: {})",
|
||||||
|
cgroup_path,
|
||||||
|
(version == ICgroupsReader::CgroupsVersion::V1) ? "v1" : "v2");
|
||||||
|
|
||||||
|
cgroups_reader = ICgroupsReader::createCgroupsReader(version, cgroup_path);
|
||||||
|
source = MemoryUsageSource::Cgroups;
|
||||||
|
if (period_ms == 0)
|
||||||
|
period_ms = cgroups_memory_usage_tick_ms;
|
||||||
|
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
tryLogCurrentException(log, "Cannot use cgroups reader");
|
||||||
|
}
|
||||||
|
#endif
|
||||||
|
|
||||||
|
#if USE_JEMALLOC
|
||||||
|
static constexpr uint64_t jemalloc_memory_usage_tick_ms{100};
|
||||||
|
|
||||||
|
source = MemoryUsageSource::Jemalloc;
|
||||||
|
if (period_ms == 0)
|
||||||
|
period_ms = jemalloc_memory_usage_tick_ms;
|
||||||
|
#endif
|
||||||
|
}
|
||||||
|
|
||||||
|
MemoryWorker::MemoryUsageSource MemoryWorker::getSource()
|
||||||
|
{
|
||||||
|
return source;
|
||||||
|
}
|
||||||
|
|
||||||
|
void MemoryWorker::start()
|
||||||
|
{
|
||||||
|
if (source == MemoryUsageSource::None)
|
||||||
|
return;
|
||||||
|
|
||||||
|
LOG_INFO(
|
||||||
|
getLogger("MemoryWorker"),
|
||||||
|
"Starting background memory thread with period of {}ms, using {} as source",
|
||||||
|
period_ms,
|
||||||
|
sourceToString(source));
|
||||||
|
background_thread = ThreadFromGlobalPool([this] { backgroundThread(); });
|
||||||
|
}
|
||||||
|
|
||||||
|
MemoryWorker::~MemoryWorker()
|
||||||
|
{
|
||||||
|
{
|
||||||
|
std::unique_lock lock(mutex);
|
||||||
|
shutdown = true;
|
||||||
|
}
|
||||||
|
cv.notify_all();
|
||||||
|
|
||||||
|
if (background_thread.joinable())
|
||||||
|
background_thread.join();
|
||||||
|
}
|
||||||
|
|
||||||
|
uint64_t MemoryWorker::getMemoryUsage()
|
||||||
|
{
|
||||||
|
switch (source)
|
||||||
|
{
|
||||||
|
case MemoryUsageSource::Cgroups:
|
||||||
|
return cgroups_reader != nullptr ? cgroups_reader->readMemoryUsage() : 0;
|
||||||
|
case MemoryUsageSource::Jemalloc:
|
||||||
|
#if USE_JEMALLOC
|
||||||
|
return resident_mib.getValue();
|
||||||
|
#else
|
||||||
|
return 0;
|
||||||
|
#endif
|
||||||
|
case MemoryUsageSource::None:
|
||||||
|
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Trying to fetch memory usage while no memory source can be used");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void MemoryWorker::backgroundThread()
|
||||||
|
{
|
||||||
|
std::chrono::milliseconds chrono_period_ms{period_ms};
|
||||||
|
[[maybe_unused]] bool first_run = true;
|
||||||
|
std::unique_lock lock(mutex);
|
||||||
|
while (true)
|
||||||
|
{
|
||||||
|
cv.wait_for(lock, chrono_period_ms, [this] { return shutdown; });
|
||||||
|
if (shutdown)
|
||||||
|
return;
|
||||||
|
|
||||||
|
Stopwatch total_watch;
|
||||||
|
|
||||||
|
#if USE_JEMALLOC
|
||||||
|
if (source == MemoryUsageSource::Jemalloc)
|
||||||
|
epoch_mib.setValue(0);
|
||||||
|
#endif
|
||||||
|
|
||||||
|
Int64 resident = getMemoryUsage();
|
||||||
|
MemoryTracker::updateRSS(resident);
|
||||||
|
|
||||||
|
#if USE_JEMALLOC
|
||||||
|
if (resident > total_memory_tracker.getHardLimit())
|
||||||
|
{
|
||||||
|
Stopwatch purge_watch;
|
||||||
|
purge_mib.run();
|
||||||
|
ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge);
|
||||||
|
ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, purge_watch.elapsedMicroseconds());
|
||||||
|
}
|
||||||
|
#endif
|
||||||
|
|
||||||
|
#if USE_JEMALLOC
|
||||||
|
if (unlikely(first_run || total_memory_tracker.get() < 0))
|
||||||
|
{
|
||||||
|
if (source != MemoryUsageSource::Jemalloc)
|
||||||
|
epoch_mib.setValue(0);
|
||||||
|
|
||||||
|
MemoryTracker::updateAllocated(allocated_mib.getValue());
|
||||||
|
}
|
||||||
|
#endif
|
||||||
|
|
||||||
|
ProfileEvents::increment(ProfileEvents::MemoryWorkerRun);
|
||||||
|
ProfileEvents::increment(ProfileEvents::MemoryWorkerRunElapsedMicroseconds, total_watch.elapsedMicroseconds());
|
||||||
|
first_run = false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
84
src/Common/MemoryWorker.h
Normal file
84
src/Common/MemoryWorker.h
Normal file
@ -0,0 +1,84 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Common/CgroupsMemoryUsageObserver.h>
|
||||||
|
#include <Common/ThreadPool.h>
|
||||||
|
#include <Common/Jemalloc.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct ICgroupsReader
|
||||||
|
{
|
||||||
|
enum class CgroupsVersion : uint8_t
|
||||||
|
{
|
||||||
|
V1,
|
||||||
|
V2
|
||||||
|
};
|
||||||
|
|
||||||
|
#if defined(OS_LINUX)
|
||||||
|
static std::shared_ptr<ICgroupsReader>
|
||||||
|
createCgroupsReader(ICgroupsReader::CgroupsVersion version, const std::filesystem::path & cgroup_path);
|
||||||
|
#endif
|
||||||
|
|
||||||
|
virtual ~ICgroupsReader() = default;
|
||||||
|
|
||||||
|
virtual uint64_t readMemoryUsage() = 0;
|
||||||
|
|
||||||
|
virtual std::string dumpAllStats() = 0;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
/// Correct MemoryTracker based on external information (e.g. Cgroups or stats.resident from jemalloc)
|
||||||
|
/// The worker spawns a background thread which periodically reads current resident memory from the source,
|
||||||
|
/// whose value is sent to global MemoryTracker.
|
||||||
|
/// It can do additional things like purging jemalloc dirty pages if the current memory usage is higher than global hard limit.
|
||||||
|
class MemoryWorker
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit MemoryWorker(uint64_t period_ms_);
|
||||||
|
|
||||||
|
enum class MemoryUsageSource : uint8_t
|
||||||
|
{
|
||||||
|
None,
|
||||||
|
Cgroups,
|
||||||
|
Jemalloc
|
||||||
|
};
|
||||||
|
|
||||||
|
MemoryUsageSource getSource();
|
||||||
|
|
||||||
|
void start();
|
||||||
|
|
||||||
|
~MemoryWorker();
|
||||||
|
private:
|
||||||
|
uint64_t getMemoryUsage();
|
||||||
|
|
||||||
|
void backgroundThread();
|
||||||
|
|
||||||
|
ThreadFromGlobalPool background_thread;
|
||||||
|
|
||||||
|
std::mutex mutex;
|
||||||
|
std::condition_variable cv;
|
||||||
|
bool shutdown = false;
|
||||||
|
|
||||||
|
LoggerPtr log;
|
||||||
|
|
||||||
|
uint64_t period_ms;
|
||||||
|
|
||||||
|
MemoryUsageSource source{MemoryUsageSource::None};
|
||||||
|
|
||||||
|
std::shared_ptr<ICgroupsReader> cgroups_reader;
|
||||||
|
|
||||||
|
#if USE_JEMALLOC
|
||||||
|
JemallocMibCache<uint64_t> epoch_mib{"epoch"};
|
||||||
|
JemallocMibCache<size_t> resident_mib{"stats.resident"};
|
||||||
|
JemallocMibCache<size_t> allocated_mib{"stats.allocated"};
|
||||||
|
|
||||||
|
#define STRINGIFY_HELPER(x) #x
|
||||||
|
#define STRINGIFY(x) STRINGIFY_HELPER(x)
|
||||||
|
JemallocMibCache<size_t> purge_mib{"arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge"};
|
||||||
|
#undef STRINGIFY
|
||||||
|
#undef STRINGIFY_HELPER
|
||||||
|
#endif
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -827,6 +827,9 @@ The server successfully detected this situation and will download merged part fr
|
|||||||
M(GWPAsanAllocateSuccess, "Number of successful allocations done by GWPAsan") \
|
M(GWPAsanAllocateSuccess, "Number of successful allocations done by GWPAsan") \
|
||||||
M(GWPAsanAllocateFailed, "Number of failed allocations done by GWPAsan (i.e. filled pool)") \
|
M(GWPAsanAllocateFailed, "Number of failed allocations done by GWPAsan (i.e. filled pool)") \
|
||||||
M(GWPAsanFree, "Number of free operations done by GWPAsan") \
|
M(GWPAsanFree, "Number of free operations done by GWPAsan") \
|
||||||
|
\
|
||||||
|
M(MemoryWorkerRun, "Number of runs done by MemoryWorker in background") \
|
||||||
|
M(MemoryWorkerRunElapsedMicroseconds, "Total time spent by MemoryWorker for background work") \
|
||||||
|
|
||||||
|
|
||||||
#ifdef APPLY_FOR_EXTERNAL_EVENTS
|
#ifdef APPLY_FOR_EXTERNAL_EVENTS
|
||||||
|
@ -171,6 +171,7 @@ bool isUserError(Error zk_return_code)
|
|||||||
|
|
||||||
void CreateRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
void CreateRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
||||||
void RemoveRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
void RemoveRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
||||||
|
void RemoveRecursiveRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
||||||
void ExistsRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
void ExistsRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
||||||
void GetRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
void GetRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
||||||
void SetRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
void SetRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
|
||||||
|
@ -248,6 +248,23 @@ struct RemoveResponse : virtual Response
|
|||||||
{
|
{
|
||||||
};
|
};
|
||||||
|
|
||||||
|
struct RemoveRecursiveRequest : virtual Request
|
||||||
|
{
|
||||||
|
String path;
|
||||||
|
|
||||||
|
/// strict limit for number of deleted nodes
|
||||||
|
uint32_t remove_nodes_limit = 1;
|
||||||
|
|
||||||
|
void addRootPath(const String & root_path) override;
|
||||||
|
String getPath() const override { return path; }
|
||||||
|
|
||||||
|
size_t bytesSize() const override { return path.size() + sizeof(remove_nodes_limit); }
|
||||||
|
};
|
||||||
|
|
||||||
|
struct RemoveRecursiveResponse : virtual Response
|
||||||
|
{
|
||||||
|
};
|
||||||
|
|
||||||
struct ExistsRequest : virtual Request
|
struct ExistsRequest : virtual Request
|
||||||
{
|
{
|
||||||
String path;
|
String path;
|
||||||
@ -430,6 +447,7 @@ struct ErrorResponse : virtual Response
|
|||||||
|
|
||||||
using CreateCallback = std::function<void(const CreateResponse &)>;
|
using CreateCallback = std::function<void(const CreateResponse &)>;
|
||||||
using RemoveCallback = std::function<void(const RemoveResponse &)>;
|
using RemoveCallback = std::function<void(const RemoveResponse &)>;
|
||||||
|
using RemoveRecursiveCallback = std::function<void(const RemoveRecursiveResponse &)>;
|
||||||
using ExistsCallback = std::function<void(const ExistsResponse &)>;
|
using ExistsCallback = std::function<void(const ExistsResponse &)>;
|
||||||
using GetCallback = std::function<void(const GetResponse &)>;
|
using GetCallback = std::function<void(const GetResponse &)>;
|
||||||
using SetCallback = std::function<void(const SetResponse &)>;
|
using SetCallback = std::function<void(const SetResponse &)>;
|
||||||
@ -587,6 +605,11 @@ public:
|
|||||||
int32_t version,
|
int32_t version,
|
||||||
RemoveCallback callback) = 0;
|
RemoveCallback callback) = 0;
|
||||||
|
|
||||||
|
virtual void removeRecursive(
|
||||||
|
const String & path,
|
||||||
|
uint32_t remove_nodes_limit,
|
||||||
|
RemoveRecursiveCallback callback) = 0;
|
||||||
|
|
||||||
virtual void exists(
|
virtual void exists(
|
||||||
const String & path,
|
const String & path,
|
||||||
ExistsCallback callback,
|
ExistsCallback callback,
|
||||||
|
@ -90,6 +90,36 @@ struct TestKeeperRemoveRequest final : RemoveRequest, TestKeeperRequest
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
struct TestKeeperRemoveRecursiveRequest final : RemoveRecursiveRequest, TestKeeperRequest
|
||||||
|
{
|
||||||
|
TestKeeperRemoveRecursiveRequest() = default;
|
||||||
|
explicit TestKeeperRemoveRecursiveRequest(const RemoveRecursiveRequest & base) : RemoveRecursiveRequest(base) {}
|
||||||
|
ResponsePtr createResponse() const override;
|
||||||
|
std::pair<ResponsePtr, Undo> process(TestKeeper::Container & container, int64_t zxid) const override;
|
||||||
|
|
||||||
|
void processWatches(TestKeeper::Watches & node_watches, TestKeeper::Watches & list_watches) const override
|
||||||
|
{
|
||||||
|
std::vector<std::pair<String, size_t>> deleted;
|
||||||
|
|
||||||
|
auto add_deleted_watches = [&](TestKeeper::Watches & w)
|
||||||
|
{
|
||||||
|
for (const auto & [watch_path, _] : w)
|
||||||
|
if (watch_path.starts_with(path))
|
||||||
|
deleted.emplace_back(watch_path, std::count(watch_path.begin(), watch_path.end(), '/'));
|
||||||
|
};
|
||||||
|
|
||||||
|
add_deleted_watches(node_watches);
|
||||||
|
add_deleted_watches(list_watches);
|
||||||
|
std::sort(deleted.begin(), deleted.end(), [](const auto & lhs, const auto & rhs)
|
||||||
|
{
|
||||||
|
return lhs.second < rhs.second;
|
||||||
|
});
|
||||||
|
|
||||||
|
for (const auto & [watch_path, _] : deleted)
|
||||||
|
processWatchesImpl(watch_path, node_watches, list_watches);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
struct TestKeeperExistsRequest final : ExistsRequest, TestKeeperRequest
|
struct TestKeeperExistsRequest final : ExistsRequest, TestKeeperRequest
|
||||||
{
|
{
|
||||||
ResponsePtr createResponse() const override;
|
ResponsePtr createResponse() const override;
|
||||||
@ -175,6 +205,10 @@ struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest
|
|||||||
{
|
{
|
||||||
requests.push_back(std::make_shared<TestKeeperRemoveRequest>(*concrete_request_remove));
|
requests.push_back(std::make_shared<TestKeeperRemoveRequest>(*concrete_request_remove));
|
||||||
}
|
}
|
||||||
|
else if (const auto * concrete_request_remove_recursive = dynamic_cast<const RemoveRecursiveRequest *>(generic_request.get()))
|
||||||
|
{
|
||||||
|
requests.push_back(std::make_shared<TestKeeperRemoveRecursiveRequest>(*concrete_request_remove_recursive));
|
||||||
|
}
|
||||||
else if (const auto * concrete_request_set = dynamic_cast<const SetRequest *>(generic_request.get()))
|
else if (const auto * concrete_request_set = dynamic_cast<const SetRequest *>(generic_request.get()))
|
||||||
{
|
{
|
||||||
requests.push_back(std::make_shared<TestKeeperSetRequest>(*concrete_request_set));
|
requests.push_back(std::make_shared<TestKeeperSetRequest>(*concrete_request_set));
|
||||||
@ -313,6 +347,62 @@ std::pair<ResponsePtr, Undo> TestKeeperRemoveRequest::process(TestKeeper::Contai
|
|||||||
return { std::make_shared<RemoveResponse>(response), undo };
|
return { std::make_shared<RemoveResponse>(response), undo };
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<ResponsePtr, Undo> TestKeeperRemoveRecursiveRequest::process(TestKeeper::Container & container, int64_t zxid) const
|
||||||
|
{
|
||||||
|
RemoveRecursiveResponse response;
|
||||||
|
response.zxid = zxid;
|
||||||
|
Undo undo;
|
||||||
|
|
||||||
|
auto root_it = container.find(path);
|
||||||
|
if (root_it == container.end())
|
||||||
|
{
|
||||||
|
response.error = Error::ZNONODE;
|
||||||
|
return { std::make_shared<RemoveRecursiveResponse>(response), undo };
|
||||||
|
}
|
||||||
|
|
||||||
|
std::vector<std::pair<std::string, Coordination::TestKeeper::Node>> children;
|
||||||
|
|
||||||
|
for (auto it = std::next(root_it); it != container.end(); ++it)
|
||||||
|
{
|
||||||
|
const auto & [child_path, child_node] = *it;
|
||||||
|
|
||||||
|
if (child_path.starts_with(path))
|
||||||
|
children.emplace_back(child_path, child_node);
|
||||||
|
else
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (children.size() > remove_nodes_limit)
|
||||||
|
{
|
||||||
|
response.error = Error::ZNOTEMPTY;
|
||||||
|
return { std::make_shared<RemoveRecursiveResponse>(response), undo };
|
||||||
|
}
|
||||||
|
|
||||||
|
auto & parent = container.at(parentPath(path));
|
||||||
|
--parent.stat.numChildren;
|
||||||
|
++parent.stat.cversion;
|
||||||
|
|
||||||
|
for (const auto & [child_path, child_node] : children)
|
||||||
|
{
|
||||||
|
auto child_it = container.find(child_path);
|
||||||
|
chassert(child_it != container.end());
|
||||||
|
container.erase(child_it);
|
||||||
|
}
|
||||||
|
|
||||||
|
response.error = Error::ZOK;
|
||||||
|
undo = [&container, dead = std::move(children), root_path = path]()
|
||||||
|
{
|
||||||
|
for (auto && [child_path, child_node] : dead)
|
||||||
|
container.emplace(child_path, child_node);
|
||||||
|
|
||||||
|
auto & undo_parent = container.at(parentPath(root_path));
|
||||||
|
++undo_parent.stat.numChildren;
|
||||||
|
--undo_parent.stat.cversion;
|
||||||
|
};
|
||||||
|
|
||||||
|
return { std::make_shared<RemoveRecursiveResponse>(response), undo };
|
||||||
|
}
|
||||||
|
|
||||||
std::pair<ResponsePtr, Undo> TestKeeperExistsRequest::process(TestKeeper::Container & container, int64_t zxid) const
|
std::pair<ResponsePtr, Undo> TestKeeperExistsRequest::process(TestKeeper::Container & container, int64_t zxid) const
|
||||||
{
|
{
|
||||||
ExistsResponse response;
|
ExistsResponse response;
|
||||||
@ -530,6 +620,7 @@ std::pair<ResponsePtr, Undo> TestKeeperMultiRequest::process(TestKeeper::Contain
|
|||||||
|
|
||||||
ResponsePtr TestKeeperCreateRequest::createResponse() const { return std::make_shared<CreateResponse>(); }
|
ResponsePtr TestKeeperCreateRequest::createResponse() const { return std::make_shared<CreateResponse>(); }
|
||||||
ResponsePtr TestKeeperRemoveRequest::createResponse() const { return std::make_shared<RemoveResponse>(); }
|
ResponsePtr TestKeeperRemoveRequest::createResponse() const { return std::make_shared<RemoveResponse>(); }
|
||||||
|
ResponsePtr TestKeeperRemoveRecursiveRequest::createResponse() const { return std::make_shared<RemoveRecursiveResponse>(); }
|
||||||
ResponsePtr TestKeeperExistsRequest::createResponse() const { return std::make_shared<ExistsResponse>(); }
|
ResponsePtr TestKeeperExistsRequest::createResponse() const { return std::make_shared<ExistsResponse>(); }
|
||||||
ResponsePtr TestKeeperGetRequest::createResponse() const { return std::make_shared<GetResponse>(); }
|
ResponsePtr TestKeeperGetRequest::createResponse() const { return std::make_shared<GetResponse>(); }
|
||||||
ResponsePtr TestKeeperSetRequest::createResponse() const { return std::make_shared<SetResponse>(); }
|
ResponsePtr TestKeeperSetRequest::createResponse() const { return std::make_shared<SetResponse>(); }
|
||||||
@ -771,6 +862,21 @@ void TestKeeper::remove(
|
|||||||
pushRequest(std::move(request_info));
|
pushRequest(std::move(request_info));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void TestKeeper::removeRecursive(
|
||||||
|
const String & path,
|
||||||
|
uint32_t remove_nodes_limit,
|
||||||
|
RemoveRecursiveCallback callback)
|
||||||
|
{
|
||||||
|
TestKeeperRemoveRecursiveRequest request;
|
||||||
|
request.path = path;
|
||||||
|
request.remove_nodes_limit = remove_nodes_limit;
|
||||||
|
|
||||||
|
RequestInfo request_info;
|
||||||
|
request_info.request = std::make_shared<TestKeeperRemoveRecursiveRequest>(std::move(request));
|
||||||
|
request_info.callback = [callback](const Response & response) { callback(dynamic_cast<const RemoveRecursiveResponse &>(response)); };
|
||||||
|
pushRequest(std::move(request_info));
|
||||||
|
}
|
||||||
|
|
||||||
void TestKeeper::exists(
|
void TestKeeper::exists(
|
||||||
const String & path,
|
const String & path,
|
||||||
ExistsCallback callback,
|
ExistsCallback callback,
|
||||||
|
@ -58,6 +58,11 @@ public:
|
|||||||
int32_t version,
|
int32_t version,
|
||||||
RemoveCallback callback) override;
|
RemoveCallback callback) override;
|
||||||
|
|
||||||
|
void removeRecursive(
|
||||||
|
const String & path,
|
||||||
|
uint32_t remove_nodes_limit,
|
||||||
|
RemoveRecursiveCallback callback) override;
|
||||||
|
|
||||||
void exists(
|
void exists(
|
||||||
const String & path,
|
const String & path,
|
||||||
ExistsCallback callback,
|
ExistsCallback callback,
|
||||||
|
@ -31,6 +31,7 @@ using AsyncResponses = std::vector<std::pair<std::string, std::future<R>>>;
|
|||||||
|
|
||||||
Coordination::RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode, bool ignore_if_exists = false);
|
Coordination::RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode, bool ignore_if_exists = false);
|
||||||
Coordination::RequestPtr makeRemoveRequest(const std::string & path, int version);
|
Coordination::RequestPtr makeRemoveRequest(const std::string & path, int version);
|
||||||
|
Coordination::RequestPtr makeRemoveRecursiveRequest(const std::string & path, uint32_t remove_nodes_limit);
|
||||||
Coordination::RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version);
|
Coordination::RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version);
|
||||||
Coordination::RequestPtr makeCheckRequest(const std::string & path, int version);
|
Coordination::RequestPtr makeCheckRequest(const std::string & path, int version);
|
||||||
|
|
||||||
|
@ -979,18 +979,47 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab
|
|||||||
return removed_as_expected;
|
return removed_as_expected;
|
||||||
}
|
}
|
||||||
|
|
||||||
void ZooKeeper::removeRecursive(const std::string & path)
|
void ZooKeeper::removeRecursive(const std::string & path, uint32_t remove_nodes_limit)
|
||||||
|
{
|
||||||
|
if (!isFeatureEnabled(DB::KeeperFeatureFlag::REMOVE_RECURSIVE))
|
||||||
{
|
{
|
||||||
removeChildrenRecursive(path);
|
removeChildrenRecursive(path);
|
||||||
remove(path);
|
remove(path);
|
||||||
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
void ZooKeeper::tryRemoveRecursive(const std::string & path)
|
check(tryRemoveRecursive(path, remove_nodes_limit), path);
|
||||||
|
}
|
||||||
|
|
||||||
|
Coordination::Error ZooKeeper::tryRemoveRecursive(const std::string & path, uint32_t remove_nodes_limit)
|
||||||
|
{
|
||||||
|
if (!isFeatureEnabled(DB::KeeperFeatureFlag::REMOVE_RECURSIVE))
|
||||||
{
|
{
|
||||||
tryRemoveChildrenRecursive(path);
|
tryRemoveChildrenRecursive(path);
|
||||||
tryRemove(path);
|
return tryRemove(path);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
auto promise = std::make_shared<std::promise<Coordination::RemoveRecursiveResponse>>();
|
||||||
|
auto future = promise->get_future();
|
||||||
|
|
||||||
|
auto callback = [promise](const Coordination::RemoveRecursiveResponse & response) mutable
|
||||||
|
{
|
||||||
|
promise->set_value(response);
|
||||||
|
};
|
||||||
|
|
||||||
|
impl->removeRecursive(path, remove_nodes_limit, std::move(callback));
|
||||||
|
|
||||||
|
if (future.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready)
|
||||||
|
{
|
||||||
|
impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::RemoveRecursive, path));
|
||||||
|
return Coordination::Error::ZOPERATIONTIMEOUT;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
auto response = future.get();
|
||||||
|
return response.error;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
@ -1619,6 +1648,14 @@ Coordination::RequestPtr makeRemoveRequest(const std::string & path, int version
|
|||||||
return request;
|
return request;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Coordination::RequestPtr makeRemoveRecursiveRequest(const std::string & path, uint32_t remove_nodes_limit)
|
||||||
|
{
|
||||||
|
auto request = std::make_shared<Coordination::RemoveRecursiveRequest>();
|
||||||
|
request->path = path;
|
||||||
|
request->remove_nodes_limit = remove_nodes_limit;
|
||||||
|
return request;
|
||||||
|
}
|
||||||
|
|
||||||
Coordination::RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version)
|
Coordination::RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version)
|
||||||
{
|
{
|
||||||
auto request = std::make_shared<Coordination::SetRequest>();
|
auto request = std::make_shared<Coordination::SetRequest>();
|
||||||
|
@ -479,15 +479,16 @@ public:
|
|||||||
|
|
||||||
Int64 getClientID();
|
Int64 getClientID();
|
||||||
|
|
||||||
/// Remove the node with the subtree. If someone concurrently adds or removes a node
|
/// Remove the node with the subtree.
|
||||||
/// in the subtree, the result is undefined.
|
/// If Keeper supports RemoveRecursive operation then it will be performed atomically.
|
||||||
void removeRecursive(const std::string & path);
|
/// Otherwise if someone concurrently adds or removes a node in the subtree, the result is undefined.
|
||||||
|
void removeRecursive(const std::string & path, uint32_t remove_nodes_limit = 100);
|
||||||
|
|
||||||
/// Remove the node with the subtree. If someone concurrently removes a node in the subtree,
|
/// Same as removeRecursive but in case if Keeper does not supports RemoveRecursive and
|
||||||
/// this will not cause errors.
|
/// if someone concurrently removes a node in the subtree, this will not cause errors.
|
||||||
/// For instance, you can call this method twice concurrently for the same node and the end
|
/// For instance, you can call this method twice concurrently for the same node and the end
|
||||||
/// result would be the same as for the single call.
|
/// result would be the same as for the single call.
|
||||||
void tryRemoveRecursive(const std::string & path);
|
Coordination::Error tryRemoveRecursive(const std::string & path, uint32_t remove_nodes_limit = 100);
|
||||||
|
|
||||||
/// Similar to removeRecursive(...) and tryRemoveRecursive(...), but does not remove path itself.
|
/// Similar to removeRecursive(...) and tryRemoveRecursive(...), but does not remove path itself.
|
||||||
/// Node defined as RemoveException will not be deleted.
|
/// Node defined as RemoveException will not be deleted.
|
||||||
|
@ -1,5 +1,5 @@
|
|||||||
#include "Common/ZooKeeper/IKeeper.h"
|
#include <Common/ZooKeeper/IKeeper.h>
|
||||||
#include "Common/ZooKeeper/ZooKeeperConstants.h"
|
#include <Common/ZooKeeper/ZooKeeperConstants.h>
|
||||||
#include <Common/ZooKeeper/ZooKeeperCommon.h>
|
#include <Common/ZooKeeper/ZooKeeperCommon.h>
|
||||||
#include <Common/ZooKeeper/ZooKeeperIO.h>
|
#include <Common/ZooKeeper/ZooKeeperIO.h>
|
||||||
#include <Common/Stopwatch.h>
|
#include <Common/Stopwatch.h>
|
||||||
@ -232,6 +232,27 @@ void ZooKeeperRemoveRequest::readImpl(ReadBuffer & in)
|
|||||||
Coordination::read(version, in);
|
Coordination::read(version, in);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void ZooKeeperRemoveRecursiveRequest::writeImpl(WriteBuffer & out) const
|
||||||
|
{
|
||||||
|
Coordination::write(path, out);
|
||||||
|
Coordination::write(remove_nodes_limit, out);
|
||||||
|
}
|
||||||
|
|
||||||
|
void ZooKeeperRemoveRecursiveRequest::readImpl(ReadBuffer & in)
|
||||||
|
{
|
||||||
|
Coordination::read(path, in);
|
||||||
|
Coordination::read(remove_nodes_limit, in);
|
||||||
|
}
|
||||||
|
|
||||||
|
std::string ZooKeeperRemoveRecursiveRequest::toStringImpl(bool /*short_format*/) const
|
||||||
|
{
|
||||||
|
return fmt::format(
|
||||||
|
"path = {}\n"
|
||||||
|
"remove_nodes_limit = {}",
|
||||||
|
path,
|
||||||
|
remove_nodes_limit);
|
||||||
|
}
|
||||||
|
|
||||||
void ZooKeeperExistsRequest::writeImpl(WriteBuffer & out) const
|
void ZooKeeperExistsRequest::writeImpl(WriteBuffer & out) const
|
||||||
{
|
{
|
||||||
Coordination::write(path, out);
|
Coordination::write(path, out);
|
||||||
@ -510,6 +531,11 @@ ZooKeeperMultiRequest::ZooKeeperMultiRequest(std::span<const Coordination::Reque
|
|||||||
checkOperationType(Write);
|
checkOperationType(Write);
|
||||||
requests.push_back(std::make_shared<ZooKeeperRemoveRequest>(*concrete_request_remove));
|
requests.push_back(std::make_shared<ZooKeeperRemoveRequest>(*concrete_request_remove));
|
||||||
}
|
}
|
||||||
|
else if (const auto * concrete_request_remove_recursive = dynamic_cast<const RemoveRecursiveRequest *>(generic_request.get()))
|
||||||
|
{
|
||||||
|
checkOperationType(Write);
|
||||||
|
requests.push_back(std::make_shared<ZooKeeperRemoveRecursiveRequest>(*concrete_request_remove_recursive));
|
||||||
|
}
|
||||||
else if (const auto * concrete_request_set = dynamic_cast<const SetRequest *>(generic_request.get()))
|
else if (const auto * concrete_request_set = dynamic_cast<const SetRequest *>(generic_request.get()))
|
||||||
{
|
{
|
||||||
checkOperationType(Write);
|
checkOperationType(Write);
|
||||||
@ -707,6 +733,7 @@ ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return se
|
|||||||
ZooKeeperResponsePtr ZooKeeperSyncRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperSyncResponse>()); }
|
ZooKeeperResponsePtr ZooKeeperSyncRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperSyncResponse>()); }
|
||||||
ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperAuthResponse>()); }
|
ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperAuthResponse>()); }
|
||||||
ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperRemoveResponse>()); }
|
ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperRemoveResponse>()); }
|
||||||
|
ZooKeeperResponsePtr ZooKeeperRemoveRecursiveRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperRemoveRecursiveResponse>()); }
|
||||||
ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperExistsResponse>()); }
|
ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperExistsResponse>()); }
|
||||||
ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperGetResponse>()); }
|
ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperGetResponse>()); }
|
||||||
ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperSetResponse>()); }
|
ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperSetResponse>()); }
|
||||||
@ -1024,6 +1051,7 @@ ZooKeeperRequestFactory::ZooKeeperRequestFactory()
|
|||||||
registerZooKeeperRequest<OpNum::SetACL, ZooKeeperSetACLRequest>(*this);
|
registerZooKeeperRequest<OpNum::SetACL, ZooKeeperSetACLRequest>(*this);
|
||||||
registerZooKeeperRequest<OpNum::FilteredList, ZooKeeperFilteredListRequest>(*this);
|
registerZooKeeperRequest<OpNum::FilteredList, ZooKeeperFilteredListRequest>(*this);
|
||||||
registerZooKeeperRequest<OpNum::CheckNotExists, ZooKeeperCheckRequest>(*this);
|
registerZooKeeperRequest<OpNum::CheckNotExists, ZooKeeperCheckRequest>(*this);
|
||||||
|
registerZooKeeperRequest<OpNum::RemoveRecursive, ZooKeeperRemoveRecursiveRequest>(*this);
|
||||||
}
|
}
|
||||||
|
|
||||||
PathMatchResult matchPath(std::string_view path, std::string_view match_to)
|
PathMatchResult matchPath(std::string_view path, std::string_view match_to)
|
||||||
|
@ -285,6 +285,31 @@ struct ZooKeeperRemoveResponse final : RemoveResponse, ZooKeeperResponse
|
|||||||
size_t bytesSize() const override { return RemoveResponse::bytesSize() + sizeof(xid) + sizeof(zxid); }
|
size_t bytesSize() const override { return RemoveResponse::bytesSize() + sizeof(xid) + sizeof(zxid); }
|
||||||
};
|
};
|
||||||
|
|
||||||
|
struct ZooKeeperRemoveRecursiveRequest final : RemoveRecursiveRequest, ZooKeeperRequest
|
||||||
|
{
|
||||||
|
ZooKeeperRemoveRecursiveRequest() = default;
|
||||||
|
explicit ZooKeeperRemoveRecursiveRequest(const RemoveRecursiveRequest & base) : RemoveRecursiveRequest(base) {}
|
||||||
|
|
||||||
|
OpNum getOpNum() const override { return OpNum::RemoveRecursive; }
|
||||||
|
void writeImpl(WriteBuffer & out) const override;
|
||||||
|
void readImpl(ReadBuffer & in) override;
|
||||||
|
std::string toStringImpl(bool short_format) const override;
|
||||||
|
|
||||||
|
ZooKeeperResponsePtr makeResponse() const override;
|
||||||
|
bool isReadRequest() const override { return false; }
|
||||||
|
|
||||||
|
size_t bytesSize() const override { return RemoveRecursiveRequest::bytesSize() + sizeof(xid); }
|
||||||
|
};
|
||||||
|
|
||||||
|
struct ZooKeeperRemoveRecursiveResponse : RemoveRecursiveResponse, ZooKeeperResponse
|
||||||
|
{
|
||||||
|
void readImpl(ReadBuffer &) override {}
|
||||||
|
void writeImpl(WriteBuffer &) const override {}
|
||||||
|
OpNum getOpNum() const override { return OpNum::RemoveRecursive; }
|
||||||
|
|
||||||
|
size_t bytesSize() const override { return RemoveRecursiveResponse::bytesSize() + sizeof(xid) + sizeof(zxid); }
|
||||||
|
};
|
||||||
|
|
||||||
struct ZooKeeperExistsRequest final : ExistsRequest, ZooKeeperRequest
|
struct ZooKeeperExistsRequest final : ExistsRequest, ZooKeeperRequest
|
||||||
{
|
{
|
||||||
ZooKeeperExistsRequest() = default;
|
ZooKeeperExistsRequest() = default;
|
||||||
|
@ -29,6 +29,7 @@ static const std::unordered_set<int32_t> VALID_OPERATIONS =
|
|||||||
static_cast<int32_t>(OpNum::GetACL),
|
static_cast<int32_t>(OpNum::GetACL),
|
||||||
static_cast<int32_t>(OpNum::FilteredList),
|
static_cast<int32_t>(OpNum::FilteredList),
|
||||||
static_cast<int32_t>(OpNum::CheckNotExists),
|
static_cast<int32_t>(OpNum::CheckNotExists),
|
||||||
|
static_cast<int32_t>(OpNum::RemoveRecursive),
|
||||||
};
|
};
|
||||||
|
|
||||||
OpNum getOpNum(int32_t raw_op_num)
|
OpNum getOpNum(int32_t raw_op_num)
|
||||||
|
@ -40,6 +40,7 @@ enum class OpNum : int32_t
|
|||||||
FilteredList = 500,
|
FilteredList = 500,
|
||||||
CheckNotExists = 501,
|
CheckNotExists = 501,
|
||||||
CreateIfNotExists = 502,
|
CreateIfNotExists = 502,
|
||||||
|
RemoveRecursive = 503,
|
||||||
|
|
||||||
SessionID = 997, /// Special internal request
|
SessionID = 997, /// Special internal request
|
||||||
};
|
};
|
||||||
|
@ -1347,6 +1347,25 @@ void ZooKeeper::remove(
|
|||||||
ProfileEvents::increment(ProfileEvents::ZooKeeperRemove);
|
ProfileEvents::increment(ProfileEvents::ZooKeeperRemove);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void ZooKeeper::removeRecursive(
|
||||||
|
const String &path,
|
||||||
|
uint32_t remove_nodes_limit,
|
||||||
|
RemoveRecursiveCallback callback)
|
||||||
|
{
|
||||||
|
if (!isFeatureEnabled(KeeperFeatureFlag::REMOVE_RECURSIVE))
|
||||||
|
throw Exception::fromMessage(Error::ZBADARGUMENTS, "RemoveRecursive request type cannot be used because it's not supported by the server");
|
||||||
|
|
||||||
|
ZooKeeperRemoveRecursiveRequest request;
|
||||||
|
request.path = path;
|
||||||
|
request.remove_nodes_limit = remove_nodes_limit;
|
||||||
|
|
||||||
|
RequestInfo request_info;
|
||||||
|
request_info.request = std::make_shared<ZooKeeperRemoveRecursiveRequest>(std::move(request));
|
||||||
|
request_info.callback = [callback](const Response & response) { callback(dynamic_cast<const RemoveRecursiveResponse &>(response)); };
|
||||||
|
|
||||||
|
pushRequest(std::move(request_info));
|
||||||
|
ProfileEvents::increment(ProfileEvents::ZooKeeperRemove);
|
||||||
|
}
|
||||||
|
|
||||||
void ZooKeeper::exists(
|
void ZooKeeper::exists(
|
||||||
const String & path,
|
const String & path,
|
||||||
|
@ -146,6 +146,11 @@ public:
|
|||||||
int32_t version,
|
int32_t version,
|
||||||
RemoveCallback callback) override;
|
RemoveCallback callback) override;
|
||||||
|
|
||||||
|
void removeRecursive(
|
||||||
|
const String &path,
|
||||||
|
uint32_t remove_nodes_limit,
|
||||||
|
RemoveRecursiveCallback callback) override;
|
||||||
|
|
||||||
void exists(
|
void exists(
|
||||||
const String & path,
|
const String & path,
|
||||||
ExistsCallback callback,
|
ExistsCallback callback,
|
||||||
|
@ -6,7 +6,7 @@
|
|||||||
#include <filesystem>
|
#include <filesystem>
|
||||||
|
|
||||||
#include <IO/WriteBufferFromFile.h>
|
#include <IO/WriteBufferFromFile.h>
|
||||||
#include <Common/CgroupsMemoryUsageObserver.h>
|
#include <Common/MemoryWorker.h>
|
||||||
#include <Common/filesystemHelpers.h>
|
#include <Common/filesystemHelpers.h>
|
||||||
|
|
||||||
using namespace DB;
|
using namespace DB;
|
||||||
@ -126,7 +126,7 @@ const std::string EXPECTED[2]
|
|||||||
"\"workingset_restore_anon\": 0, \"workingset_restore_file\": 0, \"zswap\": 0, \"zswapped\": 0, \"zswpin\": 0, \"zswpout\": 0}"};
|
"\"workingset_restore_anon\": 0, \"workingset_restore_file\": 0, \"zswap\": 0, \"zswapped\": 0, \"zswpin\": 0, \"zswpout\": 0}"};
|
||||||
|
|
||||||
|
|
||||||
class CgroupsMemoryUsageObserverFixture : public ::testing::TestWithParam<CgroupsMemoryUsageObserver::CgroupsVersion>
|
class CgroupsMemoryUsageObserverFixture : public ::testing::TestWithParam<ICgroupsReader::CgroupsVersion>
|
||||||
{
|
{
|
||||||
void SetUp() override
|
void SetUp() override
|
||||||
{
|
{
|
||||||
@ -138,7 +138,7 @@ class CgroupsMemoryUsageObserverFixture : public ::testing::TestWithParam<Cgroup
|
|||||||
stat_file.write(SAMPLE_FILE[version].data(), SAMPLE_FILE[version].size());
|
stat_file.write(SAMPLE_FILE[version].data(), SAMPLE_FILE[version].size());
|
||||||
stat_file.sync();
|
stat_file.sync();
|
||||||
|
|
||||||
if (GetParam() == CgroupsMemoryUsageObserver::CgroupsVersion::V2)
|
if (GetParam() == ICgroupsReader::CgroupsVersion::V2)
|
||||||
{
|
{
|
||||||
auto current_file = WriteBufferFromFile(tmp_dir + "/memory.current");
|
auto current_file = WriteBufferFromFile(tmp_dir + "/memory.current");
|
||||||
current_file.write("29645422592", 11);
|
current_file.write("29645422592", 11);
|
||||||
@ -154,18 +154,18 @@ protected:
|
|||||||
TEST_P(CgroupsMemoryUsageObserverFixture, ReadMemoryUsageTest)
|
TEST_P(CgroupsMemoryUsageObserverFixture, ReadMemoryUsageTest)
|
||||||
{
|
{
|
||||||
const auto version = GetParam();
|
const auto version = GetParam();
|
||||||
auto reader = createCgroupsReader(version, tmp_dir);
|
auto reader = ICgroupsReader::createCgroupsReader(version, tmp_dir);
|
||||||
ASSERT_EQ(
|
ASSERT_EQ(
|
||||||
reader->readMemoryUsage(),
|
reader->readMemoryUsage(),
|
||||||
version == CgroupsMemoryUsageObserver::CgroupsVersion::V1 ? /* rss from memory.stat */ 2232029184
|
version == ICgroupsReader::CgroupsVersion::V1 ? /* rss from memory.stat */ 2232029184
|
||||||
: /* value from memory.current - inactive_file */ 20952338432);
|
: /* anon from memory.stat */ 10429399040);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
TEST_P(CgroupsMemoryUsageObserverFixture, DumpAllStatsTest)
|
TEST_P(CgroupsMemoryUsageObserverFixture, DumpAllStatsTest)
|
||||||
{
|
{
|
||||||
const auto version = GetParam();
|
const auto version = GetParam();
|
||||||
auto reader = createCgroupsReader(version, tmp_dir);
|
auto reader = ICgroupsReader::createCgroupsReader(version, tmp_dir);
|
||||||
ASSERT_EQ(reader->dumpAllStats(), EXPECTED[static_cast<uint8_t>(version)]);
|
ASSERT_EQ(reader->dumpAllStats(), EXPECTED[static_cast<uint8_t>(version)]);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -173,6 +173,6 @@ TEST_P(CgroupsMemoryUsageObserverFixture, DumpAllStatsTest)
|
|||||||
INSTANTIATE_TEST_SUITE_P(
|
INSTANTIATE_TEST_SUITE_P(
|
||||||
CgroupsMemoryUsageObserverTests,
|
CgroupsMemoryUsageObserverTests,
|
||||||
CgroupsMemoryUsageObserverFixture,
|
CgroupsMemoryUsageObserverFixture,
|
||||||
::testing::Values(CgroupsMemoryUsageObserver::CgroupsVersion::V1, CgroupsMemoryUsageObserver::CgroupsVersion::V2));
|
::testing::Values(ICgroupsReader::CgroupsVersion::V1, ICgroupsReader::CgroupsVersion::V2));
|
||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
@ -39,7 +39,7 @@ using Checksum = CityHash_v1_0_2::uint128;
|
|||||||
|
|
||||||
|
|
||||||
/// Validate checksum of data, and if it mismatches, find out possible reason and throw exception.
|
/// Validate checksum of data, and if it mismatches, find out possible reason and throw exception.
|
||||||
static void validateChecksum(char * data, size_t size, const Checksum expected_checksum)
|
static void validateChecksum(char * data, size_t size, const Checksum expected_checksum, bool external_data)
|
||||||
{
|
{
|
||||||
auto calculated_checksum = CityHash_v1_0_2::CityHash128(data, size);
|
auto calculated_checksum = CityHash_v1_0_2::CityHash128(data, size);
|
||||||
if (expected_checksum == calculated_checksum)
|
if (expected_checksum == calculated_checksum)
|
||||||
@ -64,6 +64,8 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c
|
|||||||
"this can be caused by disk bit rot. This exception protects ClickHouse "
|
"this can be caused by disk bit rot. This exception protects ClickHouse "
|
||||||
"from data corruption due to hardware failures.";
|
"from data corruption due to hardware failures.";
|
||||||
|
|
||||||
|
int error_code = external_data ? ErrorCodes::CANNOT_DECOMPRESS : ErrorCodes::CHECKSUM_DOESNT_MATCH;
|
||||||
|
|
||||||
auto flip_bit = [](char * buf, size_t pos)
|
auto flip_bit = [](char * buf, size_t pos)
|
||||||
{
|
{
|
||||||
buf[pos / 8] ^= 1 << pos % 8;
|
buf[pos / 8] ^= 1 << pos % 8;
|
||||||
@ -87,7 +89,7 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c
|
|||||||
{
|
{
|
||||||
message << ". The mismatch is caused by single bit flip in data block at byte " << (bit_pos / 8) << ", bit " << (bit_pos % 8) << ". "
|
message << ". The mismatch is caused by single bit flip in data block at byte " << (bit_pos / 8) << ", bit " << (bit_pos % 8) << ". "
|
||||||
<< message_hardware_failure;
|
<< message_hardware_failure;
|
||||||
throw Exception::createDeprecated(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH);
|
throw Exception::createDeprecated(message.str(), error_code);
|
||||||
}
|
}
|
||||||
|
|
||||||
flip_bit(tmp_data, bit_pos); /// Restore
|
flip_bit(tmp_data, bit_pos); /// Restore
|
||||||
@ -102,10 +104,10 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c
|
|||||||
{
|
{
|
||||||
message << ". The mismatch is caused by single bit flip in checksum. "
|
message << ". The mismatch is caused by single bit flip in checksum. "
|
||||||
<< message_hardware_failure;
|
<< message_hardware_failure;
|
||||||
throw Exception::createDeprecated(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH);
|
throw Exception::createDeprecated(message.str(), error_code);
|
||||||
}
|
}
|
||||||
|
|
||||||
throw Exception::createDeprecated(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH);
|
throw Exception::createDeprecated(message.str(), error_code);
|
||||||
}
|
}
|
||||||
|
|
||||||
static void readHeaderAndGetCodecAndSize(
|
static void readHeaderAndGetCodecAndSize(
|
||||||
@ -151,7 +153,7 @@ static void readHeaderAndGetCodecAndSize(
|
|||||||
"Most likely corrupted data.", size_compressed_without_checksum);
|
"Most likely corrupted data.", size_compressed_without_checksum);
|
||||||
|
|
||||||
if (size_compressed_without_checksum < header_size)
|
if (size_compressed_without_checksum < header_size)
|
||||||
throw Exception(ErrorCodes::CORRUPTED_DATA, "Can't decompress data: "
|
throw Exception(external_data ? ErrorCodes::CANNOT_DECOMPRESS : ErrorCodes::CORRUPTED_DATA, "Can't decompress data: "
|
||||||
"the compressed data size ({}, this should include header size) is less than the header size ({})",
|
"the compressed data size ({}, this should include header size) is less than the header size ({})",
|
||||||
size_compressed_without_checksum, static_cast<size_t>(header_size));
|
size_compressed_without_checksum, static_cast<size_t>(header_size));
|
||||||
}
|
}
|
||||||
@ -202,7 +204,7 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed,
|
|||||||
readBinaryLittleEndian(checksum.low64, checksum_in);
|
readBinaryLittleEndian(checksum.low64, checksum_in);
|
||||||
readBinaryLittleEndian(checksum.high64, checksum_in);
|
readBinaryLittleEndian(checksum.high64, checksum_in);
|
||||||
|
|
||||||
validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum);
|
validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum, external_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum));
|
ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum));
|
||||||
@ -247,7 +249,7 @@ size_t CompressedReadBufferBase::readCompressedDataBlockForAsynchronous(size_t &
|
|||||||
readBinaryLittleEndian(checksum.low64, checksum_in);
|
readBinaryLittleEndian(checksum.low64, checksum_in);
|
||||||
readBinaryLittleEndian(checksum.high64, checksum_in);
|
readBinaryLittleEndian(checksum.high64, checksum_in);
|
||||||
|
|
||||||
validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum);
|
validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum, external_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum));
|
ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum));
|
||||||
@ -307,7 +309,7 @@ void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_d
|
|||||||
|
|
||||||
UInt8 header_size = ICompressionCodec::getHeaderSize();
|
UInt8 header_size = ICompressionCodec::getHeaderSize();
|
||||||
if (size_compressed_without_checksum < header_size)
|
if (size_compressed_without_checksum < header_size)
|
||||||
throw Exception(ErrorCodes::CORRUPTED_DATA,
|
throw Exception(external_data ? ErrorCodes::CANNOT_DECOMPRESS : ErrorCodes::CORRUPTED_DATA,
|
||||||
"Can't decompress data: the compressed data size ({}, this should include header size) is less than the header size ({})",
|
"Can't decompress data: the compressed data size ({}, this should include header size) is less than the header size ({})",
|
||||||
size_compressed_without_checksum, static_cast<size_t>(header_size));
|
size_compressed_without_checksum, static_cast<size_t>(header_size));
|
||||||
|
|
||||||
|
@ -114,8 +114,13 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM
|
|||||||
}
|
}
|
||||||
|
|
||||||
KeeperAsynchronousMetrics::KeeperAsynchronousMetrics(
|
KeeperAsynchronousMetrics::KeeperAsynchronousMetrics(
|
||||||
ContextPtr context_, unsigned update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_)
|
ContextPtr context_,
|
||||||
: AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), context(std::move(context_))
|
unsigned update_period_seconds,
|
||||||
|
const ProtocolServerMetricsFunc & protocol_server_metrics_func_,
|
||||||
|
bool update_jemalloc_epoch_,
|
||||||
|
bool update_rss_)
|
||||||
|
: AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, update_jemalloc_epoch_, update_rss_)
|
||||||
|
, context(std::move(context_))
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -13,9 +13,13 @@ class KeeperAsynchronousMetrics : public AsynchronousMetrics
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
KeeperAsynchronousMetrics(
|
KeeperAsynchronousMetrics(
|
||||||
ContextPtr context_, unsigned update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_);
|
ContextPtr context_,
|
||||||
~KeeperAsynchronousMetrics() override;
|
unsigned update_period_seconds,
|
||||||
|
const ProtocolServerMetricsFunc & protocol_server_metrics_func_,
|
||||||
|
bool update_jemalloc_epoch_,
|
||||||
|
bool update_rss_);
|
||||||
|
|
||||||
|
~KeeperAsynchronousMetrics() override;
|
||||||
private:
|
private:
|
||||||
ContextPtr context;
|
ContextPtr context;
|
||||||
|
|
||||||
|
@ -11,6 +11,7 @@ enum class KeeperApiVersion : uint8_t
|
|||||||
WITH_FILTERED_LIST,
|
WITH_FILTERED_LIST,
|
||||||
WITH_MULTI_READ,
|
WITH_MULTI_READ,
|
||||||
WITH_CHECK_NOT_EXISTS,
|
WITH_CHECK_NOT_EXISTS,
|
||||||
|
WITH_REMOVE_RECURSIVE,
|
||||||
};
|
};
|
||||||
|
|
||||||
const String keeper_system_path = "/keeper";
|
const String keeper_system_path = "/keeper";
|
||||||
|
@ -91,6 +91,12 @@ bool checkIfRequestIncreaseMem(const Coordination::ZooKeeperRequestPtr & request
|
|||||||
memory_delta -= remove_req.bytesSize();
|
memory_delta -= remove_req.bytesSize();
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
case Coordination::OpNum::RemoveRecursive:
|
||||||
|
{
|
||||||
|
Coordination::ZooKeeperRemoveRecursiveRequest & remove_req = dynamic_cast<Coordination::ZooKeeperRemoveRecursiveRequest &>(*sub_zk_request);
|
||||||
|
memory_delta -= remove_req.bytesSize();
|
||||||
|
break;
|
||||||
|
}
|
||||||
default:
|
default:
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
@ -148,7 +154,14 @@ void KeeperDispatcher::requestThread()
|
|||||||
Int64 mem_soft_limit = keeper_context->getKeeperMemorySoftLimit();
|
Int64 mem_soft_limit = keeper_context->getKeeperMemorySoftLimit();
|
||||||
if (configuration_and_settings->standalone_keeper && isExceedingMemorySoftLimit() && checkIfRequestIncreaseMem(request.request))
|
if (configuration_and_settings->standalone_keeper && isExceedingMemorySoftLimit() && checkIfRequestIncreaseMem(request.request))
|
||||||
{
|
{
|
||||||
LOG_WARNING(log, "Processing requests refused because of max_memory_usage_soft_limit {}, the total used memory is {}, request type is {}", ReadableSize(mem_soft_limit), ReadableSize(total_memory_tracker.get()), request.request->getOpNum());
|
LOG_WARNING(
|
||||||
|
log,
|
||||||
|
"Processing requests refused because of max_memory_usage_soft_limit {}, the total allocated memory is {}, RSS is {}, request type "
|
||||||
|
"is {}",
|
||||||
|
ReadableSize(mem_soft_limit),
|
||||||
|
ReadableSize(total_memory_tracker.get()),
|
||||||
|
ReadableSize(total_memory_tracker.getRSS()),
|
||||||
|
request.request->getOpNum());
|
||||||
addErrorResponses({request}, Coordination::Error::ZCONNECTIONLOSS);
|
addErrorResponses({request}, Coordination::Error::ZCONNECTIONLOSS);
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
@ -12,6 +12,7 @@ enum class KeeperFeatureFlag : size_t
|
|||||||
MULTI_READ,
|
MULTI_READ,
|
||||||
CHECK_NOT_EXISTS,
|
CHECK_NOT_EXISTS,
|
||||||
CREATE_IF_NOT_EXISTS,
|
CREATE_IF_NOT_EXISTS,
|
||||||
|
REMOVE_RECURSIVE,
|
||||||
};
|
};
|
||||||
|
|
||||||
class KeeperFeatureFlags
|
class KeeperFeatureFlags
|
||||||
|
@ -602,7 +602,7 @@ bool KeeperServer::isLeaderAlive() const
|
|||||||
bool KeeperServer::isExceedingMemorySoftLimit() const
|
bool KeeperServer::isExceedingMemorySoftLimit() const
|
||||||
{
|
{
|
||||||
Int64 mem_soft_limit = keeper_context->getKeeperMemorySoftLimit();
|
Int64 mem_soft_limit = keeper_context->getKeeperMemorySoftLimit();
|
||||||
return mem_soft_limit > 0 && total_memory_tracker.get() >= mem_soft_limit;
|
return mem_soft_limit > 0 && std::max(total_memory_tracker.get(), total_memory_tracker.getRSS()) >= mem_soft_limit;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// TODO test whether taking failed peer in count
|
/// TODO test whether taking failed peer in count
|
||||||
|
@ -832,6 +832,15 @@ std::shared_ptr<typename Container::Node> KeeperStorage<Container>::UncommittedS
|
|||||||
return tryGetNodeFromStorage(path);
|
return tryGetNodeFromStorage(path);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
template<typename Container>
|
||||||
|
const typename Container::Node * KeeperStorage<Container>::UncommittedState::getActualNodeView(StringRef path, const Node & storage_node) const
|
||||||
|
{
|
||||||
|
if (auto node_it = nodes.find(path.toView()); node_it != nodes.end())
|
||||||
|
return node_it->second.node.get();
|
||||||
|
|
||||||
|
return &storage_node;
|
||||||
|
}
|
||||||
|
|
||||||
template<typename Container>
|
template<typename Container>
|
||||||
Coordination::ACLs KeeperStorage<Container>::UncommittedState::getACLs(StringRef path) const
|
Coordination::ACLs KeeperStorage<Container>::UncommittedState::getACLs(StringRef path) const
|
||||||
{
|
{
|
||||||
@ -1124,7 +1133,7 @@ struct KeeperStorageRequestProcessor
|
|||||||
}
|
}
|
||||||
|
|
||||||
virtual KeeperStorageBase::ResponsesForSessions
|
virtual KeeperStorageBase::ResponsesForSessions
|
||||||
processWatches(KeeperStorageBase::Watches & /*watches*/, KeeperStorageBase::Watches & /*list_watches*/) const
|
processWatches(const Storage & /*storage*/, int64_t /*zxid*/, KeeperStorageBase::Watches & /*watches*/, KeeperStorageBase::Watches & /*list_watches*/) const
|
||||||
{
|
{
|
||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
@ -1241,7 +1250,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
|
|||||||
using KeeperStorageRequestProcessor<Storage>::KeeperStorageRequestProcessor;
|
using KeeperStorageRequestProcessor<Storage>::KeeperStorageRequestProcessor;
|
||||||
|
|
||||||
KeeperStorageBase::ResponsesForSessions
|
KeeperStorageBase::ResponsesForSessions
|
||||||
processWatches(KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override
|
processWatches(const Storage & /*storage*/, int64_t /*zxid*/, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override
|
||||||
{
|
{
|
||||||
return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED);
|
return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED);
|
||||||
}
|
}
|
||||||
@ -1462,16 +1471,41 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
template <typename Storage>
|
||||||
|
void addUpdateParentPzxidDelta(Storage & storage, std::vector<typename Storage::Delta> & deltas, int64_t zxid, StringRef path)
|
||||||
|
{
|
||||||
|
auto parent_path = parentNodePath(path);
|
||||||
|
if (!storage.uncommitted_state.getNode(parent_path))
|
||||||
|
return;
|
||||||
|
|
||||||
|
deltas.emplace_back(
|
||||||
|
std::string{parent_path},
|
||||||
|
zxid,
|
||||||
|
typename Storage::UpdateNodeDelta
|
||||||
|
{
|
||||||
|
[zxid](Storage::Node & parent)
|
||||||
|
{
|
||||||
|
parent.pzxid = std::max(parent.pzxid, zxid);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
template<typename Storage>
|
template<typename Storage>
|
||||||
struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor<Storage>
|
struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor<Storage>
|
||||||
{
|
{
|
||||||
|
using KeeperStorageRequestProcessor<Storage>::KeeperStorageRequestProcessor;
|
||||||
|
|
||||||
bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override
|
bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override
|
||||||
{
|
{
|
||||||
return storage.checkACL(parentNodePath(this->zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local);
|
return storage.checkACL(parentNodePath(this->zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local);
|
||||||
}
|
}
|
||||||
|
|
||||||
using KeeperStorageRequestProcessor<Storage>::KeeperStorageRequestProcessor;
|
|
||||||
|
|
||||||
std::vector<typename Storage::Delta>
|
std::vector<typename Storage::Delta>
|
||||||
preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override
|
preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override
|
||||||
{
|
{
|
||||||
@ -1488,31 +1522,12 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
|
|||||||
return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}};
|
return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}};
|
||||||
}
|
}
|
||||||
|
|
||||||
const auto update_parent_pzxid = [&]()
|
|
||||||
{
|
|
||||||
auto parent_path = parentNodePath(request.path);
|
|
||||||
if (!storage.uncommitted_state.getNode(parent_path))
|
|
||||||
return;
|
|
||||||
|
|
||||||
new_deltas.emplace_back(
|
|
||||||
std::string{parent_path},
|
|
||||||
zxid,
|
|
||||||
typename Storage::UpdateNodeDelta
|
|
||||||
{
|
|
||||||
[zxid](Storage::Node & parent)
|
|
||||||
{
|
|
||||||
parent.pzxid = std::max(parent.pzxid, zxid);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
};
|
|
||||||
|
|
||||||
auto node = storage.uncommitted_state.getNode(request.path);
|
auto node = storage.uncommitted_state.getNode(request.path);
|
||||||
|
|
||||||
if (!node)
|
if (!node)
|
||||||
{
|
{
|
||||||
if (request.restored_from_zookeeper_log)
|
if (request.restored_from_zookeeper_log)
|
||||||
update_parent_pzxid();
|
addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path);
|
||||||
return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}};
|
return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}};
|
||||||
}
|
}
|
||||||
else if (request.version != -1 && request.version != node->version)
|
else if (request.version != -1 && request.version != node->version)
|
||||||
@ -1521,7 +1536,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
|
|||||||
return {typename Storage::Delta{zxid, Coordination::Error::ZNOTEMPTY}};
|
return {typename Storage::Delta{zxid, Coordination::Error::ZNOTEMPTY}};
|
||||||
|
|
||||||
if (request.restored_from_zookeeper_log)
|
if (request.restored_from_zookeeper_log)
|
||||||
update_parent_pzxid();
|
addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path);
|
||||||
|
|
||||||
new_deltas.emplace_back(
|
new_deltas.emplace_back(
|
||||||
std::string{parentNodePath(request.path)},
|
std::string{parentNodePath(request.path)},
|
||||||
@ -1552,12 +1567,318 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
|
|||||||
}
|
}
|
||||||
|
|
||||||
KeeperStorageBase::ResponsesForSessions
|
KeeperStorageBase::ResponsesForSessions
|
||||||
processWatches(KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override
|
processWatches(const Storage & /*storage*/, int64_t /*zxid*/, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override
|
||||||
{
|
{
|
||||||
return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED);
|
return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
template<typename Storage>
|
||||||
|
struct KeeperStorageRemoveRecursiveRequestProcessor final : public KeeperStorageRequestProcessor<Storage>
|
||||||
|
{
|
||||||
|
using KeeperStorageRequestProcessor<Storage>::KeeperStorageRequestProcessor;
|
||||||
|
|
||||||
|
bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override
|
||||||
|
{
|
||||||
|
return storage.checkACL(parentNodePath(this->zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local);
|
||||||
|
}
|
||||||
|
|
||||||
|
std::vector<typename Storage::Delta>
|
||||||
|
preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override
|
||||||
|
{
|
||||||
|
ProfileEvents::increment(ProfileEvents::KeeperRemoveRequest);
|
||||||
|
Coordination::ZooKeeperRemoveRecursiveRequest & request = dynamic_cast<Coordination::ZooKeeperRemoveRecursiveRequest &>(*this->zk_request);
|
||||||
|
|
||||||
|
std::vector<typename Storage::Delta> new_deltas;
|
||||||
|
|
||||||
|
if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH)
|
||||||
|
{
|
||||||
|
auto error_msg = fmt::format("Trying to delete an internal Keeper path ({}) which is not allowed", request.path);
|
||||||
|
|
||||||
|
handleSystemNodeModification(keeper_context, error_msg);
|
||||||
|
return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}};
|
||||||
|
}
|
||||||
|
|
||||||
|
auto node = storage.uncommitted_state.getNode(request.path);
|
||||||
|
|
||||||
|
if (!node)
|
||||||
|
{
|
||||||
|
if (request.restored_from_zookeeper_log)
|
||||||
|
addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path);
|
||||||
|
|
||||||
|
return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}};
|
||||||
|
}
|
||||||
|
|
||||||
|
ToDeleteTreeCollector collector(storage, zxid, session_id, request.remove_nodes_limit);
|
||||||
|
auto collect_status = collector.collect(request.path, *node);
|
||||||
|
|
||||||
|
if (collect_status == ToDeleteTreeCollector::CollectStatus::NoAuth)
|
||||||
|
return {typename Storage::Delta{zxid, Coordination::Error::ZNOAUTH}};
|
||||||
|
|
||||||
|
if (collect_status == ToDeleteTreeCollector::CollectStatus::LimitExceeded)
|
||||||
|
return {typename Storage::Delta{zxid, Coordination::Error::ZNOTEMPTY}};
|
||||||
|
|
||||||
|
if (request.restored_from_zookeeper_log)
|
||||||
|
addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path);
|
||||||
|
|
||||||
|
auto delete_deltas = collector.extractDeltas();
|
||||||
|
|
||||||
|
for (const auto & delta : delete_deltas)
|
||||||
|
{
|
||||||
|
const auto * remove_delta = std::get_if<typename Storage::RemoveNodeDelta>(&delta.operation);
|
||||||
|
if (remove_delta && remove_delta->ephemeral_owner)
|
||||||
|
storage.unregisterEphemeralPath(remove_delta->ephemeral_owner, delta.path);
|
||||||
|
}
|
||||||
|
|
||||||
|
new_deltas.insert(new_deltas.end(), std::make_move_iterator(delete_deltas.begin()), std::make_move_iterator(delete_deltas.end()));
|
||||||
|
|
||||||
|
digest = storage.calculateNodesDigest(digest, new_deltas);
|
||||||
|
|
||||||
|
return new_deltas;
|
||||||
|
}
|
||||||
|
|
||||||
|
Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override
|
||||||
|
{
|
||||||
|
Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse();
|
||||||
|
Coordination::ZooKeeperRemoveRecursiveResponse & response = dynamic_cast<Coordination::ZooKeeperRemoveRecursiveResponse &>(*response_ptr);
|
||||||
|
|
||||||
|
response.error = storage.commit(zxid);
|
||||||
|
return response_ptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
KeeperStorageBase::ResponsesForSessions
|
||||||
|
processWatches(const Storage & storage, int64_t zxid, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override
|
||||||
|
{
|
||||||
|
/// need to iterate over zxid deltas and update watches for deleted tree.
|
||||||
|
const auto & deltas = storage.uncommitted_state.deltas;
|
||||||
|
|
||||||
|
KeeperStorageBase::ResponsesForSessions responses;
|
||||||
|
for (auto it = deltas.rbegin(); it != deltas.rend() && it->zxid == zxid; ++it)
|
||||||
|
{
|
||||||
|
const auto * remove_delta = std::get_if<typename Storage::RemoveNodeDelta>(&it->operation);
|
||||||
|
if (remove_delta)
|
||||||
|
{
|
||||||
|
auto new_responses = processWatchesImpl(it->path, watches, list_watches, Coordination::Event::DELETED);
|
||||||
|
responses.insert(responses.end(), std::make_move_iterator(new_responses.begin()), std::make_move_iterator(new_responses.end()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return responses;
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
using SNode = typename Storage::Node;
|
||||||
|
|
||||||
|
class ToDeleteTreeCollector
|
||||||
|
{
|
||||||
|
Storage & storage;
|
||||||
|
int64_t zxid;
|
||||||
|
int64_t session_id;
|
||||||
|
uint32_t limit;
|
||||||
|
|
||||||
|
uint32_t max_level = 0;
|
||||||
|
uint32_t nodes_observed = 1; /// root node
|
||||||
|
std::unordered_map<uint32_t, std::vector<typename Storage::Delta>> by_level_deltas;
|
||||||
|
|
||||||
|
struct Step
|
||||||
|
{
|
||||||
|
String path;
|
||||||
|
std::variant<SNode, const SNode *> node;
|
||||||
|
uint32_t level;
|
||||||
|
};
|
||||||
|
|
||||||
|
enum class CollectStatus
|
||||||
|
{
|
||||||
|
Ok,
|
||||||
|
NoAuth,
|
||||||
|
LimitExceeded,
|
||||||
|
};
|
||||||
|
|
||||||
|
friend struct KeeperStorageRemoveRecursiveRequestProcessor;
|
||||||
|
|
||||||
|
public:
|
||||||
|
ToDeleteTreeCollector(Storage & storage_, int64_t zxid_, int64_t session_id_, uint32_t limit_)
|
||||||
|
: storage(storage_)
|
||||||
|
, zxid(zxid_)
|
||||||
|
, session_id(session_id_)
|
||||||
|
, limit(limit_)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
CollectStatus collect(StringRef root_path, const SNode & root_node)
|
||||||
|
{
|
||||||
|
std::deque<Step> steps;
|
||||||
|
|
||||||
|
if (checkLimits(&root_node))
|
||||||
|
return CollectStatus::LimitExceeded;
|
||||||
|
|
||||||
|
steps.push_back(Step{root_path.toString(), &root_node, 0});
|
||||||
|
|
||||||
|
while (!steps.empty())
|
||||||
|
{
|
||||||
|
Step step = std::move(steps.front());
|
||||||
|
steps.pop_front();
|
||||||
|
|
||||||
|
StringRef path = step.path;
|
||||||
|
uint32_t level = step.level;
|
||||||
|
const SNode * node_ptr = nullptr;
|
||||||
|
|
||||||
|
if (auto * rdb = std::get_if<SNode>(&step.node))
|
||||||
|
node_ptr = rdb;
|
||||||
|
else
|
||||||
|
node_ptr = std::get<const SNode *>(step.node);
|
||||||
|
|
||||||
|
chassert(!path.empty());
|
||||||
|
chassert(node_ptr != nullptr);
|
||||||
|
|
||||||
|
const auto & node = *node_ptr;
|
||||||
|
auto actual_node_ptr = storage.uncommitted_state.getActualNodeView(path, node);
|
||||||
|
chassert(actual_node_ptr != nullptr); /// explicitly check that node is not deleted
|
||||||
|
|
||||||
|
if (actual_node_ptr->numChildren() > 0 && !storage.checkACL(path, Coordination::ACL::Delete, session_id, /*is_local=*/false))
|
||||||
|
return CollectStatus::NoAuth;
|
||||||
|
|
||||||
|
if (auto status = visitRocksDBNode(steps, path, level); status != CollectStatus::Ok)
|
||||||
|
return status;
|
||||||
|
|
||||||
|
if (auto status = visitMemNode(steps, path, level); status != CollectStatus::Ok)
|
||||||
|
return status;
|
||||||
|
|
||||||
|
if (auto status = visitRootAndUncommitted(steps, path, node, level); status != CollectStatus::Ok)
|
||||||
|
return status;
|
||||||
|
}
|
||||||
|
|
||||||
|
return CollectStatus::Ok;
|
||||||
|
}
|
||||||
|
|
||||||
|
std::vector<typename Storage::Delta> extractDeltas()
|
||||||
|
{
|
||||||
|
std::vector<typename Storage::Delta> deltas;
|
||||||
|
|
||||||
|
for (ssize_t level = max_level; level >= 0; --level)
|
||||||
|
{
|
||||||
|
auto & level_deltas = by_level_deltas[static_cast<uint32_t>(level)];
|
||||||
|
deltas.insert(deltas.end(), std::make_move_iterator(level_deltas.begin()), std::make_move_iterator(level_deltas.end()));
|
||||||
|
}
|
||||||
|
|
||||||
|
return std::move(deltas);
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
CollectStatus visitRocksDBNode(std::deque<Step> & steps, StringRef root_path, uint32_t level)
|
||||||
|
{
|
||||||
|
if constexpr (Storage::use_rocksdb)
|
||||||
|
{
|
||||||
|
std::filesystem::path root_fs_path(root_path.toString());
|
||||||
|
auto children = storage.container.getChildren(root_path.toString());
|
||||||
|
|
||||||
|
for (auto && [child_name, child_node] : children)
|
||||||
|
{
|
||||||
|
auto child_path = (root_fs_path / child_name).generic_string();
|
||||||
|
const auto actual_child_node_ptr = storage.uncommitted_state.getActualNodeView(child_path, child_node);
|
||||||
|
|
||||||
|
if (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction
|
||||||
|
continue;
|
||||||
|
|
||||||
|
if (checkLimits(actual_child_node_ptr))
|
||||||
|
return CollectStatus::LimitExceeded;
|
||||||
|
|
||||||
|
steps.push_back(Step{std::move(child_path), std::move(child_node), level + 1});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return CollectStatus::Ok;
|
||||||
|
}
|
||||||
|
|
||||||
|
CollectStatus visitMemNode(std::deque<Step> & steps, StringRef root_path, uint32_t level)
|
||||||
|
{
|
||||||
|
if constexpr (!Storage::use_rocksdb)
|
||||||
|
{
|
||||||
|
auto node_it = storage.container.find(root_path);
|
||||||
|
if (node_it == storage.container.end())
|
||||||
|
return CollectStatus::Ok;
|
||||||
|
|
||||||
|
std::filesystem::path root_fs_path(root_path.toString());
|
||||||
|
const auto & children = node_it->value.getChildren();
|
||||||
|
|
||||||
|
for (const auto & child_name : children)
|
||||||
|
{
|
||||||
|
auto child_path = (root_fs_path / child_name.toView()).generic_string();
|
||||||
|
|
||||||
|
auto child_it = storage.container.find(child_path);
|
||||||
|
chassert(child_it != storage.container.end());
|
||||||
|
const auto & child_node = child_it->value;
|
||||||
|
|
||||||
|
const auto actual_child_node_ptr = storage.uncommitted_state.getActualNodeView(child_path, child_node);
|
||||||
|
|
||||||
|
if (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction
|
||||||
|
continue;
|
||||||
|
|
||||||
|
if (checkLimits(actual_child_node_ptr))
|
||||||
|
return CollectStatus::LimitExceeded;
|
||||||
|
|
||||||
|
steps.push_back(Step{std::move(child_path), &child_node, level + 1});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return CollectStatus::Ok;
|
||||||
|
}
|
||||||
|
|
||||||
|
CollectStatus visitRootAndUncommitted(std::deque<Step> & steps, StringRef root_path, const SNode & root_node, uint32_t level)
|
||||||
|
{
|
||||||
|
const auto & nodes = storage.uncommitted_state.nodes;
|
||||||
|
|
||||||
|
/// nodes are sorted by paths with level locality
|
||||||
|
auto it = nodes.upper_bound(root_path.toString() + "/");
|
||||||
|
|
||||||
|
for (; it != nodes.end() && parentNodePath(it->first) == root_path; ++it)
|
||||||
|
{
|
||||||
|
const auto actual_child_node_ptr = it->second.node.get();
|
||||||
|
|
||||||
|
if (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction
|
||||||
|
continue;
|
||||||
|
|
||||||
|
if (checkLimits(actual_child_node_ptr))
|
||||||
|
return CollectStatus::LimitExceeded;
|
||||||
|
|
||||||
|
const String & child_path = it->first;
|
||||||
|
const SNode & child_node = *it->second.node;
|
||||||
|
|
||||||
|
steps.push_back(Step{child_path, &child_node, level + 1});
|
||||||
|
}
|
||||||
|
|
||||||
|
addDelta(root_path, root_node, level);
|
||||||
|
|
||||||
|
return CollectStatus::Ok;
|
||||||
|
}
|
||||||
|
|
||||||
|
void addDelta(StringRef root_path, const SNode & root_node, uint32_t level)
|
||||||
|
{
|
||||||
|
max_level = std::max(max_level, level);
|
||||||
|
|
||||||
|
by_level_deltas[level].emplace_back(
|
||||||
|
parentNodePath(root_path).toString(),
|
||||||
|
zxid,
|
||||||
|
typename Storage::UpdateNodeDelta{
|
||||||
|
[](SNode & parent)
|
||||||
|
{
|
||||||
|
++parent.cversion;
|
||||||
|
parent.decreaseNumChildren();
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
by_level_deltas[level].emplace_back(root_path.toString(), zxid, typename Storage::RemoveNodeDelta{root_node.version, root_node.ephemeralOwner()});
|
||||||
|
}
|
||||||
|
|
||||||
|
bool checkLimits(const SNode * node)
|
||||||
|
{
|
||||||
|
chassert(node != nullptr);
|
||||||
|
nodes_observed += node->numChildren();
|
||||||
|
return nodes_observed > limit;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
};
|
||||||
|
|
||||||
template<typename Storage>
|
template<typename Storage>
|
||||||
struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestProcessor<Storage>
|
struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestProcessor<Storage>
|
||||||
{
|
{
|
||||||
@ -1709,7 +2030,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce
|
|||||||
}
|
}
|
||||||
|
|
||||||
KeeperStorageBase::ResponsesForSessions
|
KeeperStorageBase::ResponsesForSessions
|
||||||
processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override
|
processWatches(const Storage & /*storage*/, int64_t /*zxid*/, typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override
|
||||||
{
|
{
|
||||||
return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED);
|
return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED);
|
||||||
}
|
}
|
||||||
@ -2131,6 +2452,10 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro
|
|||||||
check_operation_type(OperationType::Write);
|
check_operation_type(OperationType::Write);
|
||||||
concrete_requests.push_back(std::make_shared<KeeperStorageRemoveRequestProcessor<Storage>>(sub_zk_request));
|
concrete_requests.push_back(std::make_shared<KeeperStorageRemoveRequestProcessor<Storage>>(sub_zk_request));
|
||||||
break;
|
break;
|
||||||
|
case Coordination::OpNum::RemoveRecursive:
|
||||||
|
check_operation_type(OperationType::Write);
|
||||||
|
concrete_requests.push_back(std::make_shared<KeeperStorageRemoveRecursiveRequestProcessor<Storage>>(sub_zk_request));
|
||||||
|
break;
|
||||||
case Coordination::OpNum::Set:
|
case Coordination::OpNum::Set:
|
||||||
check_operation_type(OperationType::Write);
|
check_operation_type(OperationType::Write);
|
||||||
concrete_requests.push_back(std::make_shared<KeeperStorageSetRequestProcessor<Storage>>(sub_zk_request));
|
concrete_requests.push_back(std::make_shared<KeeperStorageSetRequestProcessor<Storage>>(sub_zk_request));
|
||||||
@ -2250,12 +2575,12 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro
|
|||||||
}
|
}
|
||||||
|
|
||||||
KeeperStorageBase::ResponsesForSessions
|
KeeperStorageBase::ResponsesForSessions
|
||||||
processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override
|
processWatches(const Storage & storage, int64_t zxid, typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override
|
||||||
{
|
{
|
||||||
typename Storage::ResponsesForSessions result;
|
typename Storage::ResponsesForSessions result;
|
||||||
for (const auto & generic_request : concrete_requests)
|
for (const auto & generic_request : concrete_requests)
|
||||||
{
|
{
|
||||||
auto responses = generic_request->processWatches(watches, list_watches);
|
auto responses = generic_request->processWatches(storage, zxid, watches, list_watches);
|
||||||
result.insert(result.end(), responses.begin(), responses.end());
|
result.insert(result.end(), responses.begin(), responses.end());
|
||||||
}
|
}
|
||||||
return result;
|
return result;
|
||||||
@ -2400,6 +2725,7 @@ KeeperStorageRequestProcessorsFactory<Storage>::KeeperStorageRequestProcessorsFa
|
|||||||
registerKeeperRequestProcessor<Coordination::OpNum::SetACL, KeeperStorageSetACLRequestProcessor<Storage>>(*this);
|
registerKeeperRequestProcessor<Coordination::OpNum::SetACL, KeeperStorageSetACLRequestProcessor<Storage>>(*this);
|
||||||
registerKeeperRequestProcessor<Coordination::OpNum::GetACL, KeeperStorageGetACLRequestProcessor<Storage>>(*this);
|
registerKeeperRequestProcessor<Coordination::OpNum::GetACL, KeeperStorageGetACLRequestProcessor<Storage>>(*this);
|
||||||
registerKeeperRequestProcessor<Coordination::OpNum::CheckNotExists, KeeperStorageCheckRequestProcessor<Storage>>(*this);
|
registerKeeperRequestProcessor<Coordination::OpNum::CheckNotExists, KeeperStorageCheckRequestProcessor<Storage>>(*this);
|
||||||
|
registerKeeperRequestProcessor<Coordination::OpNum::RemoveRecursive, KeeperStorageRemoveRecursiveRequestProcessor<Storage>>(*this);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -2718,7 +3044,7 @@ KeeperStorage<Container>::ResponsesForSessions KeeperStorage<Container>::process
|
|||||||
/// If this requests processed successfully we need to check watches
|
/// If this requests processed successfully we need to check watches
|
||||||
if (response->error == Coordination::Error::ZOK)
|
if (response->error == Coordination::Error::ZOK)
|
||||||
{
|
{
|
||||||
auto watch_responses = request_processor->processWatches(watches, list_watches);
|
auto watch_responses = request_processor->processWatches(*this, zxid, watches, list_watches);
|
||||||
results.insert(results.end(), watch_responses.begin(), watch_responses.end());
|
results.insert(results.end(), watch_responses.begin(), watch_responses.end());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -566,6 +566,7 @@ public:
|
|||||||
void rollback(int64_t rollback_zxid);
|
void rollback(int64_t rollback_zxid);
|
||||||
|
|
||||||
std::shared_ptr<Node> getNode(StringRef path) const;
|
std::shared_ptr<Node> getNode(StringRef path) const;
|
||||||
|
const Node * getActualNodeView(StringRef path, const Node & storage_node) const;
|
||||||
Coordination::ACLs getACLs(StringRef path) const;
|
Coordination::ACLs getACLs(StringRef path) const;
|
||||||
|
|
||||||
void applyDelta(const Delta & delta);
|
void applyDelta(const Delta & delta);
|
||||||
@ -609,7 +610,18 @@ public:
|
|||||||
using is_transparent = void; // required to make find() work with different type than key_type
|
using is_transparent = void; // required to make find() work with different type than key_type
|
||||||
};
|
};
|
||||||
|
|
||||||
mutable std::unordered_map<std::string, UncommittedNode, Hash, Equal> nodes;
|
struct PathCmp
|
||||||
|
{
|
||||||
|
using is_transparent = std::true_type;
|
||||||
|
|
||||||
|
auto operator()(const std::string_view a,
|
||||||
|
const std::string_view b) const
|
||||||
|
{
|
||||||
|
return a.size() < b.size() || (a.size() == b.size() && a < b);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
mutable std::map<std::string, UncommittedNode, PathCmp> nodes;
|
||||||
std::unordered_map<std::string, std::list<const Delta *>, Hash, Equal> deltas_for_path;
|
std::unordered_map<std::string, std::list<const Delta *>, Hash, Equal> deltas_for_path;
|
||||||
|
|
||||||
std::list<Delta> deltas;
|
std::list<Delta> deltas;
|
||||||
|
@ -3113,6 +3113,8 @@ TYPED_TEST(CoordinationTest, TestFeatureFlags)
|
|||||||
ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::FILTERED_LIST));
|
ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::FILTERED_LIST));
|
||||||
ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::MULTI_READ));
|
ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::MULTI_READ));
|
||||||
ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::CHECK_NOT_EXISTS));
|
ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::CHECK_NOT_EXISTS));
|
||||||
|
ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::CREATE_IF_NOT_EXISTS));
|
||||||
|
ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::REMOVE_RECURSIVE));
|
||||||
}
|
}
|
||||||
|
|
||||||
TYPED_TEST(CoordinationTest, TestSystemNodeModify)
|
TYPED_TEST(CoordinationTest, TestSystemNodeModify)
|
||||||
@ -3374,6 +3376,474 @@ TYPED_TEST(CoordinationTest, TestReapplyingDeltas)
|
|||||||
ASSERT_TRUE(children1_set == children2_set);
|
ASSERT_TRUE(children1_set == children2_set);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
TYPED_TEST(CoordinationTest, TestRemoveRecursiveRequest)
|
||||||
|
{
|
||||||
|
using namespace DB;
|
||||||
|
using namespace Coordination;
|
||||||
|
|
||||||
|
using Storage = typename TestFixture::Storage;
|
||||||
|
|
||||||
|
ChangelogDirTest rocks("./rocksdb");
|
||||||
|
this->setRocksDBDirectory("./rocksdb");
|
||||||
|
|
||||||
|
Storage storage{500, "", this->keeper_context};
|
||||||
|
|
||||||
|
int32_t zxid = 0;
|
||||||
|
|
||||||
|
const auto create = [&](const String & path, int create_mode)
|
||||||
|
{
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
|
||||||
|
const auto create_request = std::make_shared<ZooKeeperCreateRequest>();
|
||||||
|
create_request->path = path;
|
||||||
|
create_request->is_ephemeral = create_mode == zkutil::CreateMode::Ephemeral || create_mode == zkutil::CreateMode::EphemeralSequential;
|
||||||
|
create_request->is_sequential = create_mode == zkutil::CreateMode::PersistentSequential || create_mode == zkutil::CreateMode::EphemeralSequential;
|
||||||
|
|
||||||
|
storage.preprocessRequest(create_request, 1, 0, new_zxid);
|
||||||
|
auto responses = storage.processRequest(create_request, 1, new_zxid);
|
||||||
|
|
||||||
|
EXPECT_EQ(responses.size(), 1);
|
||||||
|
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to create " << path;
|
||||||
|
};
|
||||||
|
|
||||||
|
const auto remove = [&](const String & path, int32_t version = -1)
|
||||||
|
{
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
|
||||||
|
auto remove_request = std::make_shared<ZooKeeperRemoveRequest>();
|
||||||
|
remove_request->path = path;
|
||||||
|
remove_request->version = version;
|
||||||
|
|
||||||
|
storage.preprocessRequest(remove_request, 1, 0, new_zxid);
|
||||||
|
return storage.processRequest(remove_request, 1, new_zxid);
|
||||||
|
};
|
||||||
|
|
||||||
|
const auto remove_recursive = [&](const String & path, uint32_t remove_nodes_limit = 1)
|
||||||
|
{
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
|
||||||
|
auto remove_request = std::make_shared<ZooKeeperRemoveRecursiveRequest>();
|
||||||
|
remove_request->path = path;
|
||||||
|
remove_request->remove_nodes_limit = remove_nodes_limit;
|
||||||
|
|
||||||
|
storage.preprocessRequest(remove_request, 1, 0, new_zxid);
|
||||||
|
return storage.processRequest(remove_request, 1, new_zxid);
|
||||||
|
};
|
||||||
|
|
||||||
|
const auto exists = [&](const String & path)
|
||||||
|
{
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
|
||||||
|
const auto exists_request = std::make_shared<ZooKeeperExistsRequest>();
|
||||||
|
exists_request->path = path;
|
||||||
|
|
||||||
|
storage.preprocessRequest(exists_request, 1, 0, new_zxid);
|
||||||
|
auto responses = storage.processRequest(exists_request, 1, new_zxid);
|
||||||
|
|
||||||
|
EXPECT_EQ(responses.size(), 1);
|
||||||
|
return responses[0].response->error == Coordination::Error::ZOK;
|
||||||
|
};
|
||||||
|
|
||||||
|
{
|
||||||
|
SCOPED_TRACE("Single Remove Single Node");
|
||||||
|
create("/T1", zkutil::CreateMode::Persistent);
|
||||||
|
|
||||||
|
auto responses = remove("/T1");
|
||||||
|
ASSERT_EQ(responses.size(), 1);
|
||||||
|
ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK);
|
||||||
|
ASSERT_FALSE(exists("/T1"));
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
SCOPED_TRACE("Single Remove Tree");
|
||||||
|
create("/T2", zkutil::CreateMode::Persistent);
|
||||||
|
create("/T2/A", zkutil::CreateMode::Persistent);
|
||||||
|
|
||||||
|
auto responses = remove("/T2");
|
||||||
|
ASSERT_EQ(responses.size(), 1);
|
||||||
|
ASSERT_EQ(responses[0].response->error, Coordination::Error::ZNOTEMPTY);
|
||||||
|
ASSERT_TRUE(exists("/T2"));
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
SCOPED_TRACE("Recursive Remove Single Node");
|
||||||
|
create("/T3", zkutil::CreateMode::Persistent);
|
||||||
|
|
||||||
|
auto responses = remove_recursive("/T3", 100);
|
||||||
|
ASSERT_EQ(responses.size(), 1);
|
||||||
|
ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK);
|
||||||
|
ASSERT_FALSE(exists("/T3"));
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
SCOPED_TRACE("Recursive Remove Tree Small Limit");
|
||||||
|
create("/T5", zkutil::CreateMode::Persistent);
|
||||||
|
create("/T5/A", zkutil::CreateMode::Persistent);
|
||||||
|
create("/T5/B", zkutil::CreateMode::Persistent);
|
||||||
|
create("/T5/A/C", zkutil::CreateMode::Persistent);
|
||||||
|
|
||||||
|
auto responses = remove_recursive("/T5", 2);
|
||||||
|
ASSERT_EQ(responses.size(), 1);
|
||||||
|
ASSERT_EQ(responses[0].response->error, Coordination::Error::ZNOTEMPTY);
|
||||||
|
ASSERT_TRUE(exists("/T5"));
|
||||||
|
ASSERT_TRUE(exists("/T5/A"));
|
||||||
|
ASSERT_TRUE(exists("/T5/B"));
|
||||||
|
ASSERT_TRUE(exists("/T5/A/C"));
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
SCOPED_TRACE("Recursive Remove Tree Big Limit");
|
||||||
|
create("/T6", zkutil::CreateMode::Persistent);
|
||||||
|
create("/T6/A", zkutil::CreateMode::Persistent);
|
||||||
|
create("/T6/B", zkutil::CreateMode::Persistent);
|
||||||
|
create("/T6/A/C", zkutil::CreateMode::Persistent);
|
||||||
|
|
||||||
|
auto responses = remove_recursive("/T6", 4);
|
||||||
|
ASSERT_EQ(responses.size(), 1);
|
||||||
|
ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK);
|
||||||
|
ASSERT_FALSE(exists("/T6"));
|
||||||
|
ASSERT_FALSE(exists("/T6/A"));
|
||||||
|
ASSERT_FALSE(exists("/T6/B"));
|
||||||
|
ASSERT_FALSE(exists("/T6/A/C"));
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
SCOPED_TRACE("Recursive Remove Ephemeral");
|
||||||
|
create("/T7", zkutil::CreateMode::Ephemeral);
|
||||||
|
ASSERT_EQ(storage.ephemerals.size(), 1);
|
||||||
|
|
||||||
|
auto responses = remove_recursive("/T7", 100);
|
||||||
|
ASSERT_EQ(responses.size(), 1);
|
||||||
|
ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK);
|
||||||
|
ASSERT_EQ(storage.ephemerals.size(), 0);
|
||||||
|
ASSERT_FALSE(exists("/T7"));
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
SCOPED_TRACE("Recursive Remove Tree With Ephemeral");
|
||||||
|
create("/T8", zkutil::CreateMode::Persistent);
|
||||||
|
create("/T8/A", zkutil::CreateMode::Persistent);
|
||||||
|
create("/T8/B", zkutil::CreateMode::Ephemeral);
|
||||||
|
create("/T8/A/C", zkutil::CreateMode::Ephemeral);
|
||||||
|
ASSERT_EQ(storage.ephemerals.size(), 1);
|
||||||
|
|
||||||
|
auto responses = remove_recursive("/T8", 4);
|
||||||
|
ASSERT_EQ(responses.size(), 1);
|
||||||
|
ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK);
|
||||||
|
ASSERT_EQ(storage.ephemerals.size(), 0);
|
||||||
|
ASSERT_FALSE(exists("/T8"));
|
||||||
|
ASSERT_FALSE(exists("/T8/A"));
|
||||||
|
ASSERT_FALSE(exists("/T8/B"));
|
||||||
|
ASSERT_FALSE(exists("/T8/A/C"));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
TYPED_TEST(CoordinationTest, TestRemoveRecursiveInMultiRequest)
|
||||||
|
{
|
||||||
|
using namespace DB;
|
||||||
|
using namespace Coordination;
|
||||||
|
|
||||||
|
using Storage = typename TestFixture::Storage;
|
||||||
|
|
||||||
|
ChangelogDirTest rocks("./rocksdb");
|
||||||
|
this->setRocksDBDirectory("./rocksdb");
|
||||||
|
|
||||||
|
Storage storage{500, "", this->keeper_context};
|
||||||
|
int zxid = 0;
|
||||||
|
|
||||||
|
auto prepare_create_tree = []()
|
||||||
|
{
|
||||||
|
return Coordination::Requests{
|
||||||
|
zkutil::makeCreateRequest("/A", "A", zkutil::CreateMode::Persistent),
|
||||||
|
zkutil::makeCreateRequest("/A/B", "B", zkutil::CreateMode::Persistent),
|
||||||
|
zkutil::makeCreateRequest("/A/C", "C", zkutil::CreateMode::Ephemeral),
|
||||||
|
zkutil::makeCreateRequest("/A/B/D", "D", zkutil::CreateMode::Ephemeral),
|
||||||
|
};
|
||||||
|
};
|
||||||
|
|
||||||
|
const auto exists = [&](const String & path)
|
||||||
|
{
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
|
||||||
|
const auto exists_request = std::make_shared<ZooKeeperExistsRequest>();
|
||||||
|
exists_request->path = path;
|
||||||
|
|
||||||
|
storage.preprocessRequest(exists_request, 1, 0, new_zxid);
|
||||||
|
auto responses = storage.processRequest(exists_request, 1, new_zxid);
|
||||||
|
|
||||||
|
EXPECT_EQ(responses.size(), 1);
|
||||||
|
return responses[0].response->error == Coordination::Error::ZOK;
|
||||||
|
};
|
||||||
|
|
||||||
|
const auto is_multi_ok = [&](Coordination::ZooKeeperResponsePtr response)
|
||||||
|
{
|
||||||
|
const auto & multi_response = dynamic_cast<Coordination::ZooKeeperMultiResponse &>(*response);
|
||||||
|
|
||||||
|
for (const auto & op_response : multi_response.responses)
|
||||||
|
if (op_response->error != Coordination::Error::ZOK)
|
||||||
|
return false;
|
||||||
|
|
||||||
|
return true;
|
||||||
|
};
|
||||||
|
|
||||||
|
{
|
||||||
|
SCOPED_TRACE("Remove In Multi Tx");
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
auto ops = prepare_create_tree();
|
||||||
|
|
||||||
|
ops.push_back(zkutil::makeRemoveRequest("/A", -1));
|
||||||
|
const auto request = std::make_shared<ZooKeeperMultiRequest>(ops, ACLs{});
|
||||||
|
|
||||||
|
storage.preprocessRequest(request, 1, 0, new_zxid);
|
||||||
|
auto responses = storage.processRequest(request, 1, new_zxid);
|
||||||
|
ops.pop_back();
|
||||||
|
|
||||||
|
ASSERT_EQ(responses.size(), 1);
|
||||||
|
ASSERT_FALSE(is_multi_ok(responses[0].response));
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
SCOPED_TRACE("Recursive Remove In Multi Tx");
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
auto ops = prepare_create_tree();
|
||||||
|
|
||||||
|
ops.push_back(zkutil::makeRemoveRecursiveRequest("/A", 4));
|
||||||
|
const auto request = std::make_shared<ZooKeeperMultiRequest>(ops, ACLs{});
|
||||||
|
|
||||||
|
storage.preprocessRequest(request, 1, 0, new_zxid);
|
||||||
|
auto responses = storage.processRequest(request, 1, new_zxid);
|
||||||
|
ops.pop_back();
|
||||||
|
|
||||||
|
ASSERT_EQ(responses.size(), 1);
|
||||||
|
ASSERT_TRUE(is_multi_ok(responses[0].response));
|
||||||
|
ASSERT_FALSE(exists("/A"));
|
||||||
|
ASSERT_FALSE(exists("/A/C"));
|
||||||
|
ASSERT_FALSE(exists("/A/B"));
|
||||||
|
ASSERT_FALSE(exists("/A/B/D"));
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
SCOPED_TRACE("Recursive Remove With Regular In Multi Tx");
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
auto ops = prepare_create_tree();
|
||||||
|
|
||||||
|
ops.push_back(zkutil::makeRemoveRequest("/A/C", -1));
|
||||||
|
ops.push_back(zkutil::makeRemoveRecursiveRequest("/A", 3));
|
||||||
|
const auto request = std::make_shared<ZooKeeperMultiRequest>(ops, ACLs{});
|
||||||
|
|
||||||
|
storage.preprocessRequest(request, 1, 0, new_zxid);
|
||||||
|
auto responses = storage.processRequest(request, 1, new_zxid);
|
||||||
|
ops.pop_back();
|
||||||
|
ops.pop_back();
|
||||||
|
|
||||||
|
ASSERT_EQ(responses.size(), 1);
|
||||||
|
ASSERT_TRUE(is_multi_ok(responses[0].response));
|
||||||
|
ASSERT_FALSE(exists("/A"));
|
||||||
|
ASSERT_FALSE(exists("/A/C"));
|
||||||
|
ASSERT_FALSE(exists("/A/B"));
|
||||||
|
ASSERT_FALSE(exists("/A/B/D"));
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
SCOPED_TRACE("Recursive Remove From Committed and Uncommitted states");
|
||||||
|
int create_zxid = ++zxid;
|
||||||
|
auto ops = prepare_create_tree();
|
||||||
|
|
||||||
|
/// First create nodes
|
||||||
|
const auto create_request = std::make_shared<ZooKeeperMultiRequest>(ops, ACLs{});
|
||||||
|
storage.preprocessRequest(create_request, 1, 0, create_zxid);
|
||||||
|
auto create_responses = storage.processRequest(create_request, 1, create_zxid);
|
||||||
|
ASSERT_EQ(create_responses.size(), 1);
|
||||||
|
ASSERT_TRUE(is_multi_ok(create_responses[0].response));
|
||||||
|
ASSERT_TRUE(exists("/A"));
|
||||||
|
ASSERT_TRUE(exists("/A/C"));
|
||||||
|
ASSERT_TRUE(exists("/A/B"));
|
||||||
|
ASSERT_TRUE(exists("/A/B/D"));
|
||||||
|
|
||||||
|
/// Remove node A/C as a single remove request.
|
||||||
|
/// Remove all other as remove recursive request.
|
||||||
|
/// In this case we should list storage to understand the tree topology
|
||||||
|
/// but ignore already deleted nodes in uncommitted state.
|
||||||
|
|
||||||
|
int remove_zxid = ++zxid;
|
||||||
|
ops = {
|
||||||
|
zkutil::makeRemoveRequest("/A/C", -1),
|
||||||
|
zkutil::makeRemoveRecursiveRequest("/A", 3),
|
||||||
|
};
|
||||||
|
const auto remove_request = std::make_shared<ZooKeeperMultiRequest>(ops, ACLs{});
|
||||||
|
|
||||||
|
storage.preprocessRequest(remove_request, 1, 0, remove_zxid);
|
||||||
|
auto remove_responses = storage.processRequest(remove_request, 1, remove_zxid);
|
||||||
|
|
||||||
|
ASSERT_EQ(remove_responses.size(), 1);
|
||||||
|
ASSERT_TRUE(is_multi_ok(remove_responses[0].response));
|
||||||
|
ASSERT_FALSE(exists("/A"));
|
||||||
|
ASSERT_FALSE(exists("/A/C"));
|
||||||
|
ASSERT_FALSE(exists("/A/B"));
|
||||||
|
ASSERT_FALSE(exists("/A/B/D"));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
TYPED_TEST(CoordinationTest, TestRemoveRecursiveWatches)
|
||||||
|
{
|
||||||
|
using namespace DB;
|
||||||
|
using namespace Coordination;
|
||||||
|
|
||||||
|
using Storage = typename TestFixture::Storage;
|
||||||
|
|
||||||
|
ChangelogDirTest rocks("./rocksdb");
|
||||||
|
this->setRocksDBDirectory("./rocksdb");
|
||||||
|
|
||||||
|
Storage storage{500, "", this->keeper_context};
|
||||||
|
int zxid = 0;
|
||||||
|
|
||||||
|
const auto create = [&](const String & path, int create_mode)
|
||||||
|
{
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
|
||||||
|
const auto create_request = std::make_shared<ZooKeeperCreateRequest>();
|
||||||
|
create_request->path = path;
|
||||||
|
create_request->is_ephemeral = create_mode == zkutil::CreateMode::Ephemeral || create_mode == zkutil::CreateMode::EphemeralSequential;
|
||||||
|
create_request->is_sequential = create_mode == zkutil::CreateMode::PersistentSequential || create_mode == zkutil::CreateMode::EphemeralSequential;
|
||||||
|
|
||||||
|
storage.preprocessRequest(create_request, 1, 0, new_zxid);
|
||||||
|
auto responses = storage.processRequest(create_request, 1, new_zxid);
|
||||||
|
|
||||||
|
EXPECT_EQ(responses.size(), 1);
|
||||||
|
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to create " << path;
|
||||||
|
};
|
||||||
|
|
||||||
|
const auto add_watch = [&](const String & path)
|
||||||
|
{
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
|
||||||
|
const auto exists_request = std::make_shared<ZooKeeperExistsRequest>();
|
||||||
|
exists_request->path = path;
|
||||||
|
exists_request->has_watch = true;
|
||||||
|
|
||||||
|
storage.preprocessRequest(exists_request, 1, 0, new_zxid);
|
||||||
|
auto responses = storage.processRequest(exists_request, 1, new_zxid);
|
||||||
|
|
||||||
|
EXPECT_EQ(responses.size(), 1);
|
||||||
|
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK);
|
||||||
|
};
|
||||||
|
|
||||||
|
const auto add_list_watch = [&](const String & path)
|
||||||
|
{
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
|
||||||
|
const auto list_request = std::make_shared<ZooKeeperListRequest>();
|
||||||
|
list_request->path = path;
|
||||||
|
list_request->has_watch = true;
|
||||||
|
|
||||||
|
storage.preprocessRequest(list_request, 1, 0, new_zxid);
|
||||||
|
auto responses = storage.processRequest(list_request, 1, new_zxid);
|
||||||
|
|
||||||
|
EXPECT_EQ(responses.size(), 1);
|
||||||
|
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK);
|
||||||
|
};
|
||||||
|
|
||||||
|
create("/A", zkutil::CreateMode::Persistent);
|
||||||
|
create("/A/B", zkutil::CreateMode::Persistent);
|
||||||
|
create("/A/C", zkutil::CreateMode::Ephemeral);
|
||||||
|
create("/A/B/D", zkutil::CreateMode::Ephemeral);
|
||||||
|
|
||||||
|
add_watch("/A");
|
||||||
|
add_watch("/A/B");
|
||||||
|
add_watch("/A/C");
|
||||||
|
add_watch("/A/B/D");
|
||||||
|
add_list_watch("/A");
|
||||||
|
add_list_watch("/A/B");
|
||||||
|
ASSERT_EQ(storage.watches.size(), 4);
|
||||||
|
ASSERT_EQ(storage.list_watches.size(), 2);
|
||||||
|
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
|
||||||
|
auto remove_request = std::make_shared<ZooKeeperRemoveRecursiveRequest>();
|
||||||
|
remove_request->path = "/A";
|
||||||
|
remove_request->remove_nodes_limit = 4;
|
||||||
|
|
||||||
|
storage.preprocessRequest(remove_request, 1, 0, new_zxid);
|
||||||
|
auto responses = storage.processRequest(remove_request, 1, new_zxid);
|
||||||
|
|
||||||
|
ASSERT_EQ(responses.size(), 7);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < 7; ++i)
|
||||||
|
{
|
||||||
|
ASSERT_EQ(responses[i].response->error, Coordination::Error::ZOK);
|
||||||
|
|
||||||
|
if (const auto * watch_response = dynamic_cast<Coordination::ZooKeeperWatchResponse *>(responses[i].response.get()))
|
||||||
|
ASSERT_EQ(watch_response->type, Coordination::Event::DELETED);
|
||||||
|
}
|
||||||
|
|
||||||
|
ASSERT_EQ(storage.watches.size(), 0);
|
||||||
|
ASSERT_EQ(storage.list_watches.size(), 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
TYPED_TEST(CoordinationTest, TestRemoveRecursiveAcls)
|
||||||
|
{
|
||||||
|
using namespace DB;
|
||||||
|
using namespace Coordination;
|
||||||
|
|
||||||
|
using Storage = typename TestFixture::Storage;
|
||||||
|
|
||||||
|
ChangelogDirTest rocks("./rocksdb");
|
||||||
|
this->setRocksDBDirectory("./rocksdb");
|
||||||
|
|
||||||
|
Storage storage{500, "", this->keeper_context};
|
||||||
|
int zxid = 0;
|
||||||
|
|
||||||
|
{
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
String user_auth_data = "test_user:test_password";
|
||||||
|
|
||||||
|
const auto auth_request = std::make_shared<ZooKeeperAuthRequest>();
|
||||||
|
auth_request->scheme = "digest";
|
||||||
|
auth_request->data = user_auth_data;
|
||||||
|
|
||||||
|
storage.preprocessRequest(auth_request, 1, 0, new_zxid);
|
||||||
|
auto responses = storage.processRequest(auth_request, 1, new_zxid);
|
||||||
|
|
||||||
|
EXPECT_EQ(responses.size(), 1);
|
||||||
|
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to add auth to session";
|
||||||
|
}
|
||||||
|
|
||||||
|
const auto create = [&](const String & path)
|
||||||
|
{
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
|
||||||
|
const auto create_request = std::make_shared<ZooKeeperCreateRequest>();
|
||||||
|
create_request->path = path;
|
||||||
|
create_request->acls = {{.permissions = ACL::Create, .scheme = "auth", .id = ""}};
|
||||||
|
|
||||||
|
storage.preprocessRequest(create_request, 1, 0, new_zxid);
|
||||||
|
auto responses = storage.processRequest(create_request, 1, new_zxid);
|
||||||
|
|
||||||
|
EXPECT_EQ(responses.size(), 1);
|
||||||
|
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to create " << path;
|
||||||
|
};
|
||||||
|
|
||||||
|
/// Add nodes with only Create ACL
|
||||||
|
create("/A");
|
||||||
|
create("/A/B");
|
||||||
|
create("/A/C");
|
||||||
|
create("/A/B/D");
|
||||||
|
|
||||||
|
{
|
||||||
|
int new_zxid = ++zxid;
|
||||||
|
|
||||||
|
auto remove_request = std::make_shared<ZooKeeperRemoveRecursiveRequest>();
|
||||||
|
remove_request->path = "/A";
|
||||||
|
remove_request->remove_nodes_limit = 4;
|
||||||
|
|
||||||
|
storage.preprocessRequest(remove_request, 1, 0, new_zxid);
|
||||||
|
auto responses = storage.processRequest(remove_request, 1, new_zxid);
|
||||||
|
|
||||||
|
EXPECT_EQ(responses.size(), 1);
|
||||||
|
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZNOAUTH);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/// INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite,
|
/// INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite,
|
||||||
/// CoordinationTest,
|
/// CoordinationTest,
|
||||||
/// ::testing::ValuesIn(std::initializer_list<CompressionParam>{CompressionParam{true, ".zstd"}, CompressionParam{false, ""}}));
|
/// ::testing::ValuesIn(std::initializer_list<CompressionParam>{CompressionParam{true, ".zstd"}, CompressionParam{false, ""}}));
|
||||||
|
@ -148,6 +148,7 @@ namespace DB
|
|||||||
M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \
|
M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \
|
||||||
M(UInt64, max_materialized_views_count_for_table, 0, "A limit on the number of materialized views attached to a table.", 0) \
|
M(UInt64, max_materialized_views_count_for_table, 0, "A limit on the number of materialized views attached to a table.", 0) \
|
||||||
M(UInt32, max_database_replicated_create_table_thread_pool_size, 1, "The number of threads to create tables during replica recovery in DatabaseReplicated. Zero means number of threads equal number of cores.", 0) \
|
M(UInt32, max_database_replicated_create_table_thread_pool_size, 1, "The number of threads to create tables during replica recovery in DatabaseReplicated. Zero means number of threads equal number of cores.", 0) \
|
||||||
|
M(Bool, database_replicated_allow_detach_permanently, true, "Allow detaching tables permanently in Replicated databases", 0) \
|
||||||
M(Bool, format_alter_operations_with_parentheses, false, "If enabled, each operation in alter queries will be surrounded with parentheses in formatted queries to make them less ambiguous.", 0) \
|
M(Bool, format_alter_operations_with_parentheses, false, "If enabled, each operation in alter queries will be surrounded with parentheses in formatted queries to make them less ambiguous.", 0) \
|
||||||
M(String, default_replica_path, "/clickhouse/tables/{uuid}/{shard}", "The path to the table in ZooKeeper", 0) \
|
M(String, default_replica_path, "/clickhouse/tables/{uuid}/{shard}", "The path to the table in ZooKeeper", 0) \
|
||||||
M(String, default_replica_name, "{replica}", "The replica name in ZooKeeper", 0) \
|
M(String, default_replica_name, "{replica}", "The replica name in ZooKeeper", 0) \
|
||||||
@ -169,6 +170,7 @@ namespace DB
|
|||||||
M(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \
|
M(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \
|
||||||
M(Double, gwp_asan_force_sample_probability, 0.0003, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \
|
M(Double, gwp_asan_force_sample_probability, 0.0003, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \
|
||||||
M(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \
|
M(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \
|
||||||
|
M(UInt64, memory_worker_period_ms, 0, "Tick period of background memory worker which corrects memory tracker memory usages and cleans up unused pages during higher memory usage. If set to 0, default value will be used depending on the memory usage source", 0) \
|
||||||
M(Bool, disable_insertion_and_mutation, false, "Disable all insert/alter/delete queries. This setting will be enabled if someone needs read-only nodes to prevent insertion and mutation affect reading performance.", 0)
|
M(Bool, disable_insertion_and_mutation, false, "Disable all insert/alter/delete queries. This setting will be enabled if someone needs read-only nodes to prevent insertion and mutation affect reading performance.", 0)
|
||||||
|
|
||||||
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp
|
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp
|
||||||
|
@ -923,6 +923,9 @@ class IColumn;
|
|||||||
M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \
|
M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \
|
||||||
M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \
|
M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \
|
||||||
M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \
|
M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \
|
||||||
|
M(Int32, join_to_sort_minimum_perkey_rows, 40, "The lower limit of per-key average rows in the right table to determine whether to rerange the right table by key in left or inner join. This setting ensures that the optimization is not applied for sparse table keys", 0) \
|
||||||
|
M(Int32, join_to_sort_maximum_table_rows, 10000, "The maximum number of rows in the right table to determine whether to rerange the right table by key in left or inner join.", 0) \
|
||||||
|
M(Bool, allow_experimental_join_right_table_sorting, false, "If it is set to true, and the conditions of `join_to_sort_minimum_perkey_rows` and `join_to_sort_maximum_table_rows` are met, rerange the right table by key to improve the performance in left or inner hash join.", 0) \
|
||||||
M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \
|
M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \
|
||||||
M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\
|
M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\
|
||||||
\
|
\
|
||||||
@ -945,6 +948,7 @@ class IColumn;
|
|||||||
M(Bool, parallel_replicas_prefer_local_join, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN.", 0) \
|
M(Bool, parallel_replicas_prefer_local_join, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN.", 0) \
|
||||||
M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \
|
M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \
|
||||||
M(Bool, allow_archive_path_syntax, true, "File/S3 engines/table function will parse paths with '::' as '<archive> :: <file>' if archive has correct extension", 0) \
|
M(Bool, allow_archive_path_syntax, true, "File/S3 engines/table function will parse paths with '::' as '<archive> :: <file>' if archive has correct extension", 0) \
|
||||||
|
M(Bool, parallel_replicas_local_plan, false, "Build local plan for local replica", 0) \
|
||||||
\
|
\
|
||||||
M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \
|
M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \
|
||||||
M(Bool, allow_experimental_full_text_index, false, "If it is set to true, allow to use experimental full-text index.", 0) \
|
M(Bool, allow_experimental_full_text_index, false, "If it is set to true, allow to use experimental full-text index.", 0) \
|
||||||
|
@ -79,6 +79,10 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
|
|||||||
{"output_format_identifier_quoting_style", "Backticks", "Backticks", "New setting."},
|
{"output_format_identifier_quoting_style", "Backticks", "Backticks", "New setting."},
|
||||||
{"database_replicated_allow_replicated_engine_arguments", 1, 0, "Don't allow explicit arguments by default"},
|
{"database_replicated_allow_replicated_engine_arguments", 1, 0, "Don't allow explicit arguments by default"},
|
||||||
{"database_replicated_allow_explicit_uuid", 0, 0, "Added a new setting to disallow explicitly specifying table UUID"},
|
{"database_replicated_allow_explicit_uuid", 0, 0, "Added a new setting to disallow explicitly specifying table UUID"},
|
||||||
|
{"parallel_replicas_local_plan", false, false, "Use local plan for local replica in a query with parallel replicas"},
|
||||||
|
{"join_to_sort_minimum_perkey_rows", 0, 40, "The lower limit of per-key average rows in the right table to determine whether to rerange the right table by key in left or inner join. This setting ensures that the optimization is not applied for sparse table keys"},
|
||||||
|
{"join_to_sort_maximum_table_rows", 0, 10000, "The maximum number of rows in the right table to determine whether to rerange the right table by key in left or inner join"},
|
||||||
|
{"allow_experimental_join_right_table_sorting", false, false, "If it is set to true, and the conditions of `join_to_sort_minimum_perkey_rows` and `join_to_sort_maximum_table_rows` are met, rerange the right table by key to improve the performance in left or inner hash join"}
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
{"24.8",
|
{"24.8",
|
||||||
@ -99,7 +103,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
|
|||||||
{"use_json_alias_for_old_object_type", true, false, "Use JSON type alias to create new JSON type"},
|
{"use_json_alias_for_old_object_type", true, false, "Use JSON type alias to create new JSON type"},
|
||||||
{"type_json_skip_duplicated_paths", false, false, "Allow to skip duplicated paths during JSON parsing"},
|
{"type_json_skip_duplicated_paths", false, false, "Allow to skip duplicated paths during JSON parsing"},
|
||||||
{"allow_experimental_vector_similarity_index", false, false, "Added new setting to allow experimental vector similarity indexes"},
|
{"allow_experimental_vector_similarity_index", false, false, "Added new setting to allow experimental vector similarity indexes"},
|
||||||
{"input_format_try_infer_datetimes_only_datetime64", true, false, "Allow to infer DateTime instead of DateTime64 in data formats"}
|
{"input_format_try_infer_datetimes_only_datetime64", true, false, "Allow to infer DateTime instead of DateTime64 in data formats"},
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
{"24.7",
|
{"24.7",
|
||||||
|
@ -63,6 +63,7 @@ namespace ErrorCodes
|
|||||||
extern const int NO_ACTIVE_REPLICAS;
|
extern const int NO_ACTIVE_REPLICAS;
|
||||||
extern const int CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT;
|
extern const int CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT;
|
||||||
extern const int CANNOT_RESTORE_TABLE;
|
extern const int CANNOT_RESTORE_TABLE;
|
||||||
|
extern const int SUPPORT_IS_DISABLED;
|
||||||
}
|
}
|
||||||
|
|
||||||
static constexpr const char * REPLICATED_DATABASE_MARK = "DatabaseReplicated";
|
static constexpr const char * REPLICATED_DATABASE_MARK = "DatabaseReplicated";
|
||||||
@ -1741,6 +1742,9 @@ void DatabaseReplicated::detachTablePermanently(ContextPtr local_context, const
|
|||||||
{
|
{
|
||||||
waitDatabaseStarted();
|
waitDatabaseStarted();
|
||||||
|
|
||||||
|
if (!local_context->getServerSettings().database_replicated_allow_detach_permanently)
|
||||||
|
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Support for DETACH TABLE PERMANENTLY is disabled");
|
||||||
|
|
||||||
auto txn = local_context->getZooKeeperMetadataTransaction();
|
auto txn = local_context->getZooKeeperMetadataTransaction();
|
||||||
assert(!ddl_worker->isCurrentlyActive() || txn);
|
assert(!ddl_worker->isCurrentlyActive() || txn);
|
||||||
if (txn && txn->isInitialQuery())
|
if (txn && txn->isInitialQuery())
|
||||||
|
@ -124,7 +124,7 @@ public:
|
|||||||
std::string_view sqid = col_non_const->getDataAt(i).toView();
|
std::string_view sqid = col_non_const->getDataAt(i).toView();
|
||||||
std::vector<UInt64> integers = sqids.decode(String(sqid));
|
std::vector<UInt64> integers = sqids.decode(String(sqid));
|
||||||
res_nested_data.insert(integers.begin(), integers.end());
|
res_nested_data.insert(integers.begin(), integers.end());
|
||||||
res_offsets_data.push_back(integers.size());
|
res_offsets_data.push_back(res_offsets_data.back() + integers.size());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
|
@ -28,6 +28,7 @@
|
|||||||
#include <Storages/StorageReplicatedMergeTree.h>
|
#include <Storages/StorageReplicatedMergeTree.h>
|
||||||
#include <Storages/StorageSnapshot.h>
|
#include <Storages/StorageSnapshot.h>
|
||||||
#include <Storages/buildQueryTreeForShard.h>
|
#include <Storages/buildQueryTreeForShard.h>
|
||||||
|
#include <Processors/QueryPlan/ParallelReplicasLocalPlan.h>
|
||||||
#include <Storages/getStructureOfRemoteTable.h>
|
#include <Storages/getStructureOfRemoteTable.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -38,6 +39,7 @@ namespace ErrorCodes
|
|||||||
extern const int TOO_LARGE_DISTRIBUTED_DEPTH;
|
extern const int TOO_LARGE_DISTRIBUTED_DEPTH;
|
||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
extern const int UNEXPECTED_CLUSTER;
|
extern const int UNEXPECTED_CLUSTER;
|
||||||
|
extern const int INCONSISTENT_CLUSTER_DEFINITION;
|
||||||
}
|
}
|
||||||
|
|
||||||
namespace ClusterProxy
|
namespace ClusterProxy
|
||||||
@ -439,7 +441,8 @@ void executeQueryWithParallelReplicas(
|
|||||||
QueryProcessingStage::Enum processed_stage,
|
QueryProcessingStage::Enum processed_stage,
|
||||||
const ASTPtr & query_ast,
|
const ASTPtr & query_ast,
|
||||||
ContextPtr context,
|
ContextPtr context,
|
||||||
std::shared_ptr<const StorageLimitsList> storage_limits)
|
std::shared_ptr<const StorageLimitsList> storage_limits,
|
||||||
|
QueryPlanStepPtr analyzed_read_from_merge_tree)
|
||||||
{
|
{
|
||||||
auto logger = getLogger("executeQueryWithParallelReplicas");
|
auto logger = getLogger("executeQueryWithParallelReplicas");
|
||||||
LOG_DEBUG(logger, "Executing read from {}, header {}, query ({}), stage {} with parallel replicas",
|
LOG_DEBUG(logger, "Executing read from {}, header {}, query ({}), stage {} with parallel replicas",
|
||||||
@ -485,12 +488,12 @@ void executeQueryWithParallelReplicas(
|
|||||||
shard_num = column->getUInt(0);
|
shard_num = column->getUInt(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
const auto shard_count = not_optimized_cluster->getShardCount();
|
|
||||||
ClusterPtr new_cluster = not_optimized_cluster;
|
ClusterPtr new_cluster = not_optimized_cluster;
|
||||||
/// if got valid shard_num from query initiator, then parallel replicas scope is the specified shard
|
/// if got valid shard_num from query initiator, then parallel replicas scope is the specified shard
|
||||||
/// shards are numbered in order of appearance in the cluster config
|
/// shards are numbered in order of appearance in the cluster config
|
||||||
if (shard_num > 0)
|
if (shard_num > 0)
|
||||||
{
|
{
|
||||||
|
const auto shard_count = not_optimized_cluster->getShardCount();
|
||||||
if (shard_num > shard_count)
|
if (shard_num > shard_count)
|
||||||
throw Exception(
|
throw Exception(
|
||||||
ErrorCodes::LOGICAL_ERROR,
|
ErrorCodes::LOGICAL_ERROR,
|
||||||
@ -516,11 +519,99 @@ void executeQueryWithParallelReplicas(
|
|||||||
"`cluster_for_parallel_replicas` setting refers to cluster with several shards. Expected a cluster with one shard");
|
"`cluster_for_parallel_replicas` setting refers to cluster with several shards. Expected a cluster with one shard");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
const auto & shard = new_cluster->getShardsInfo().at(0);
|
||||||
|
size_t max_replicas_to_use = settings.max_parallel_replicas;
|
||||||
|
if (max_replicas_to_use > shard.getAllNodeCount())
|
||||||
|
{
|
||||||
|
LOG_INFO(
|
||||||
|
getLogger("ReadFromParallelRemoteReplicasStep"),
|
||||||
|
"The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). "
|
||||||
|
"Will use the latter number to execute the query.",
|
||||||
|
settings.max_parallel_replicas,
|
||||||
|
shard.getAllNodeCount());
|
||||||
|
max_replicas_to_use = shard.getAllNodeCount();
|
||||||
|
}
|
||||||
|
|
||||||
|
auto coordinator = std::make_shared<ParallelReplicasReadingCoordinator>(max_replicas_to_use, settings.parallel_replicas_mark_segment_size);
|
||||||
|
|
||||||
auto external_tables = new_context->getExternalTables();
|
auto external_tables = new_context->getExternalTables();
|
||||||
|
|
||||||
|
std::vector<ConnectionPoolWithFailover::Base::ShuffledPool> shuffled_pool;
|
||||||
|
if (max_replicas_to_use < shard.getAllNodeCount())
|
||||||
|
{
|
||||||
|
// will be shuffled according to `load_balancing` setting
|
||||||
|
shuffled_pool = shard.pool->getShuffledPools(settings);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
/// If all replicas in cluster are used for query execution,
|
||||||
|
/// try to preserve replicas order as in cluster definition.
|
||||||
|
/// It's important for data locality during query execution
|
||||||
|
/// independently of the query initiator
|
||||||
|
auto priority_func = [](size_t i) { return Priority{static_cast<Int64>(i)}; };
|
||||||
|
shuffled_pool = shard.pool->getShuffledPools(settings, priority_func);
|
||||||
|
}
|
||||||
|
|
||||||
|
std::vector<ConnectionPoolPtr> pools_to_use;
|
||||||
|
pools_to_use.reserve(shuffled_pool.size());
|
||||||
|
for (auto & pool : shuffled_pool)
|
||||||
|
pools_to_use.emplace_back(std::move(pool.pool));
|
||||||
|
|
||||||
|
/// do not build local plan for distributed queries for now (address it later)
|
||||||
|
if (settings.allow_experimental_analyzer && settings.parallel_replicas_local_plan && !shard_num)
|
||||||
|
{
|
||||||
|
/// find local replica index in pool
|
||||||
|
std::optional<size_t> local_replica_index;
|
||||||
|
for (size_t i = 0, s = pools_to_use.size(); i < s; ++i)
|
||||||
|
{
|
||||||
|
const auto & hostname = pools_to_use[i]->getHost();
|
||||||
|
const auto found = std::find_if(
|
||||||
|
begin(shard.local_addresses),
|
||||||
|
end(shard.local_addresses),
|
||||||
|
[&hostname](const Cluster::Address & local_addr) { return hostname == local_addr.host_name; });
|
||||||
|
if (found != shard.local_addresses.end())
|
||||||
|
{
|
||||||
|
local_replica_index = i;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (!local_replica_index)
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION,
|
||||||
|
"Local replica is not found in '{}' cluster definition, see 'cluster_for_parallel_replicas' setting",
|
||||||
|
new_cluster->getName());
|
||||||
|
|
||||||
|
// resize the pool but keep local replicas in it (and update its index)
|
||||||
|
chassert(max_replicas_to_use <= pools_to_use.size());
|
||||||
|
if (local_replica_index >= max_replicas_to_use)
|
||||||
|
{
|
||||||
|
std::swap(pools_to_use[max_replicas_to_use - 1], pools_to_use[local_replica_index.value()]);
|
||||||
|
local_replica_index = max_replicas_to_use - 1;
|
||||||
|
}
|
||||||
|
pools_to_use.resize(max_replicas_to_use);
|
||||||
|
|
||||||
|
auto [local_plan, with_parallel_replicas] = createLocalPlanForParallelReplicas(
|
||||||
|
query_ast,
|
||||||
|
header,
|
||||||
|
new_context,
|
||||||
|
processed_stage,
|
||||||
|
coordinator,
|
||||||
|
std::move(analyzed_read_from_merge_tree),
|
||||||
|
local_replica_index.value());
|
||||||
|
|
||||||
|
if (!with_parallel_replicas)
|
||||||
|
{
|
||||||
|
query_plan = std::move(*local_plan);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
LOG_DEBUG(logger, "Local replica got replica number {}", local_replica_index.value());
|
||||||
|
|
||||||
auto read_from_remote = std::make_unique<ReadFromParallelRemoteReplicasStep>(
|
auto read_from_remote = std::make_unique<ReadFromParallelRemoteReplicasStep>(
|
||||||
query_ast,
|
query_ast,
|
||||||
new_cluster,
|
new_cluster,
|
||||||
storage_id,
|
storage_id,
|
||||||
|
coordinator,
|
||||||
header,
|
header,
|
||||||
processed_stage,
|
processed_stage,
|
||||||
new_context,
|
new_context,
|
||||||
@ -528,10 +619,48 @@ void executeQueryWithParallelReplicas(
|
|||||||
std::move(scalars),
|
std::move(scalars),
|
||||||
std::move(external_tables),
|
std::move(external_tables),
|
||||||
getLogger("ReadFromParallelRemoteReplicasStep"),
|
getLogger("ReadFromParallelRemoteReplicasStep"),
|
||||||
std::move(storage_limits));
|
std::move(storage_limits),
|
||||||
|
std::move(pools_to_use),
|
||||||
|
local_replica_index);
|
||||||
|
|
||||||
|
auto remote_plan = std::make_unique<QueryPlan>();
|
||||||
|
remote_plan->addStep(std::move(read_from_remote));
|
||||||
|
|
||||||
|
DataStreams input_streams;
|
||||||
|
input_streams.reserve(2);
|
||||||
|
input_streams.emplace_back(local_plan->getCurrentDataStream());
|
||||||
|
input_streams.emplace_back(remote_plan->getCurrentDataStream());
|
||||||
|
|
||||||
|
std::vector<QueryPlanPtr> plans;
|
||||||
|
plans.emplace_back(std::move(local_plan));
|
||||||
|
plans.emplace_back(std::move(remote_plan));
|
||||||
|
|
||||||
|
auto union_step = std::make_unique<UnionStep>(std::move(input_streams));
|
||||||
|
query_plan.unitePlans(std::move(union_step), std::move(plans));
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
chassert(max_replicas_to_use <= pools_to_use.size());
|
||||||
|
pools_to_use.resize(max_replicas_to_use);
|
||||||
|
|
||||||
|
auto read_from_remote = std::make_unique<ReadFromParallelRemoteReplicasStep>(
|
||||||
|
query_ast,
|
||||||
|
new_cluster,
|
||||||
|
storage_id,
|
||||||
|
std::move(coordinator),
|
||||||
|
header,
|
||||||
|
processed_stage,
|
||||||
|
new_context,
|
||||||
|
getThrottler(new_context),
|
||||||
|
std::move(scalars),
|
||||||
|
std::move(external_tables),
|
||||||
|
getLogger("ReadFromParallelRemoteReplicasStep"),
|
||||||
|
std::move(storage_limits),
|
||||||
|
std::move(pools_to_use));
|
||||||
|
|
||||||
query_plan.addStep(std::move(read_from_remote));
|
query_plan.addStep(std::move(read_from_remote));
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
void executeQueryWithParallelReplicas(
|
void executeQueryWithParallelReplicas(
|
||||||
QueryPlan & query_plan,
|
QueryPlan & query_plan,
|
||||||
@ -540,7 +669,8 @@ void executeQueryWithParallelReplicas(
|
|||||||
const QueryTreeNodePtr & query_tree,
|
const QueryTreeNodePtr & query_tree,
|
||||||
const PlannerContextPtr & planner_context,
|
const PlannerContextPtr & planner_context,
|
||||||
ContextPtr context,
|
ContextPtr context,
|
||||||
std::shared_ptr<const StorageLimitsList> storage_limits)
|
std::shared_ptr<const StorageLimitsList> storage_limits,
|
||||||
|
QueryPlanStepPtr analyzed_read_from_merge_tree)
|
||||||
{
|
{
|
||||||
QueryTreeNodePtr modified_query_tree = query_tree->clone();
|
QueryTreeNodePtr modified_query_tree = query_tree->clone();
|
||||||
rewriteJoinToGlobalJoin(modified_query_tree, context);
|
rewriteJoinToGlobalJoin(modified_query_tree, context);
|
||||||
@ -550,7 +680,8 @@ void executeQueryWithParallelReplicas(
|
|||||||
= InterpreterSelectQueryAnalyzer::getSampleBlock(modified_query_tree, context, SelectQueryOptions(processed_stage).analyze());
|
= InterpreterSelectQueryAnalyzer::getSampleBlock(modified_query_tree, context, SelectQueryOptions(processed_stage).analyze());
|
||||||
auto modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree);
|
auto modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree);
|
||||||
|
|
||||||
executeQueryWithParallelReplicas(query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits);
|
executeQueryWithParallelReplicas(
|
||||||
|
query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits, std::move(analyzed_read_from_merge_tree));
|
||||||
}
|
}
|
||||||
|
|
||||||
void executeQueryWithParallelReplicas(
|
void executeQueryWithParallelReplicas(
|
||||||
@ -646,6 +777,49 @@ void executeQueryWithParallelReplicasCustomKey(
|
|||||||
context, query_info.query, storage_id.getDatabaseName(), storage_id.getTableName(), /*table_function_ptr=*/nullptr);
|
context, query_info.query, storage_id.getDatabaseName(), storage_id.getTableName(), /*table_function_ptr=*/nullptr);
|
||||||
executeQueryWithParallelReplicasCustomKey(query_plan, storage_id, query_info, columns, snapshot, processed_stage, header, context);
|
executeQueryWithParallelReplicasCustomKey(query_plan, storage_id, query_info, columns, snapshot, processed_stage, header, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool canUseParallelReplicasOnInitiator(const ContextPtr & context)
|
||||||
|
{
|
||||||
|
if (!context->canUseParallelReplicasOnInitiator())
|
||||||
|
return false;
|
||||||
|
|
||||||
|
auto cluster = context->getClusterForParallelReplicas();
|
||||||
|
if (cluster->getShardCount() == 1)
|
||||||
|
return cluster->getShardsInfo()[0].getAllNodeCount() > 1;
|
||||||
|
|
||||||
|
/// parallel replicas with distributed table
|
||||||
|
auto scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{};
|
||||||
|
UInt64 shard_num = 0; /// shard_num is 1-based, so 0 - no shard specified
|
||||||
|
const auto it = scalars.find("_shard_num");
|
||||||
|
if (it != scalars.end())
|
||||||
|
{
|
||||||
|
const Block & block = it->second;
|
||||||
|
const auto & column = block.safeGetByPosition(0).column;
|
||||||
|
shard_num = column->getUInt(0);
|
||||||
|
}
|
||||||
|
if (shard_num > 0)
|
||||||
|
{
|
||||||
|
const auto shard_count = cluster->getShardCount();
|
||||||
|
if (shard_num > shard_count)
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::LOGICAL_ERROR,
|
||||||
|
"Shard number is greater than shard count: shard_num={} shard_count={} cluster={}",
|
||||||
|
shard_num,
|
||||||
|
shard_count,
|
||||||
|
cluster->getName());
|
||||||
|
|
||||||
|
return cluster->getShardsInfo().at(shard_num - 1).getAllNodeCount() > 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (cluster->getShardCount() > 1)
|
||||||
|
throw DB::Exception(
|
||||||
|
ErrorCodes::UNEXPECTED_CLUSTER,
|
||||||
|
"`cluster_for_parallel_replicas` setting refers to cluster with {} shards. Expected a cluster with one shard",
|
||||||
|
cluster->getShardCount());
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -35,6 +35,9 @@ using QueryTreeNodePtr = std::shared_ptr<IQueryTreeNode>;
|
|||||||
class PlannerContext;
|
class PlannerContext;
|
||||||
using PlannerContextPtr = std::shared_ptr<PlannerContext>;
|
using PlannerContextPtr = std::shared_ptr<PlannerContext>;
|
||||||
|
|
||||||
|
class IQueryPlanStep;
|
||||||
|
using QueryPlanStepPtr = std::unique_ptr<IQueryPlanStep>;
|
||||||
|
|
||||||
namespace ClusterProxy
|
namespace ClusterProxy
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -55,6 +58,8 @@ using AdditionalShardFilterGenerator = std::function<ASTPtr(uint64_t)>;
|
|||||||
AdditionalShardFilterGenerator
|
AdditionalShardFilterGenerator
|
||||||
getShardFilterGeneratorForCustomKey(const Cluster & cluster, ContextPtr context, const ColumnsDescription & columns);
|
getShardFilterGeneratorForCustomKey(const Cluster & cluster, ContextPtr context, const ColumnsDescription & columns);
|
||||||
|
|
||||||
|
bool canUseParallelReplicasOnInitiator(const ContextPtr & context);
|
||||||
|
|
||||||
/// Execute a distributed query, creating a query plan, from which the query pipeline can be built.
|
/// Execute a distributed query, creating a query plan, from which the query pipeline can be built.
|
||||||
/// `stream_factory` object encapsulates the logic of creating plans for a different type of query
|
/// `stream_factory` object encapsulates the logic of creating plans for a different type of query
|
||||||
/// (currently SELECT, DESCRIBE).
|
/// (currently SELECT, DESCRIBE).
|
||||||
@ -81,7 +86,8 @@ void executeQueryWithParallelReplicas(
|
|||||||
QueryProcessingStage::Enum processed_stage,
|
QueryProcessingStage::Enum processed_stage,
|
||||||
const ASTPtr & query_ast,
|
const ASTPtr & query_ast,
|
||||||
ContextPtr context,
|
ContextPtr context,
|
||||||
std::shared_ptr<const StorageLimitsList> storage_limits);
|
std::shared_ptr<const StorageLimitsList> storage_limits,
|
||||||
|
QueryPlanStepPtr read_from_merge_tree = nullptr);
|
||||||
|
|
||||||
void executeQueryWithParallelReplicas(
|
void executeQueryWithParallelReplicas(
|
||||||
QueryPlan & query_plan,
|
QueryPlan & query_plan,
|
||||||
@ -98,7 +104,8 @@ void executeQueryWithParallelReplicas(
|
|||||||
const QueryTreeNodePtr & query_tree,
|
const QueryTreeNodePtr & query_tree,
|
||||||
const PlannerContextPtr & planner_context,
|
const PlannerContextPtr & planner_context,
|
||||||
ContextPtr context,
|
ContextPtr context,
|
||||||
std::shared_ptr<const StorageLimitsList> storage_limits);
|
std::shared_ptr<const StorageLimitsList> storage_limits,
|
||||||
|
QueryPlanStepPtr read_from_merge_tree);
|
||||||
|
|
||||||
void executeQueryWithParallelReplicasCustomKey(
|
void executeQueryWithParallelReplicasCustomKey(
|
||||||
QueryPlan & query_plan,
|
QueryPlan & query_plan,
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
#include <Poco/UUID.h>
|
#include <Poco/UUID.h>
|
||||||
#include <Poco/Util/Application.h>
|
#include <Poco/Util/Application.h>
|
||||||
#include <Common/AsyncLoader.h>
|
#include <Common/AsyncLoader.h>
|
||||||
|
#include <Common/CgroupsMemoryUsageObserver.h>
|
||||||
#include <Common/PoolId.h>
|
#include <Common/PoolId.h>
|
||||||
#include <Common/SensitiveDataMasker.h>
|
#include <Common/SensitiveDataMasker.h>
|
||||||
#include <Common/Macros.h>
|
#include <Common/Macros.h>
|
||||||
|
@ -152,6 +152,7 @@ class ServerType;
|
|||||||
template <class Queue>
|
template <class Queue>
|
||||||
class MergeTreeBackgroundExecutor;
|
class MergeTreeBackgroundExecutor;
|
||||||
class AsyncLoader;
|
class AsyncLoader;
|
||||||
|
struct ICgroupsReader;
|
||||||
|
|
||||||
struct TemporaryTableHolder;
|
struct TemporaryTableHolder;
|
||||||
using TemporaryTablesMapping = std::map<String, std::shared_ptr<TemporaryTableHolder>>;
|
using TemporaryTablesMapping = std::map<String, std::shared_ptr<TemporaryTableHolder>>;
|
||||||
|
@ -34,6 +34,23 @@ void AddedColumns<true>::buildOutput()
|
|||||||
{
|
{
|
||||||
if (join_data_avg_perkey_rows < output_by_row_list_threshold)
|
if (join_data_avg_perkey_rows < output_by_row_list_threshold)
|
||||||
buildOutputFromBlocks<true>();
|
buildOutputFromBlocks<true>();
|
||||||
|
else if (join_data_sorted)
|
||||||
|
{
|
||||||
|
for (size_t i = 0; i < this->size(); ++i)
|
||||||
|
{
|
||||||
|
auto & col = columns[i];
|
||||||
|
for (auto row_ref_i : lazy_output.row_refs)
|
||||||
|
{
|
||||||
|
if (row_ref_i)
|
||||||
|
{
|
||||||
|
const RowRefList * row_ref_list = reinterpret_cast<const RowRefList *>(row_ref_i);
|
||||||
|
col->insertRangeFrom(*row_ref_list->block->getByPosition(right_indexes[i]).column, row_ref_list->row_num, row_ref_list->rows);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
type_name[i].type->insertDefaultInto(*col);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
for (size_t i = 0; i < this->size(); ++i)
|
for (size_t i = 0; i < this->size(); ++i)
|
||||||
|
@ -66,6 +66,9 @@ public:
|
|||||||
, join_on_keys(join_on_keys_)
|
, join_on_keys(join_on_keys_)
|
||||||
, additional_filter_expression(additional_filter_expression_)
|
, additional_filter_expression(additional_filter_expression_)
|
||||||
, rows_to_add(left_block.rows())
|
, rows_to_add(left_block.rows())
|
||||||
|
, join_data_avg_perkey_rows(join.getJoinedData()->avgPerKeyRows())
|
||||||
|
, output_by_row_list_threshold(join.getTableJoin().outputByRowListPerkeyRowsThreshold())
|
||||||
|
, join_data_sorted(join.getJoinedData()->sorted)
|
||||||
, is_join_get(is_join_get_)
|
, is_join_get(is_join_get_)
|
||||||
{
|
{
|
||||||
size_t num_columns_to_add = block_with_columns_to_add.columns();
|
size_t num_columns_to_add = block_with_columns_to_add.columns();
|
||||||
@ -113,8 +116,6 @@ public:
|
|||||||
if (columns[j]->isNullable() && !saved_column->isNullable())
|
if (columns[j]->isNullable() && !saved_column->isNullable())
|
||||||
nullable_column_ptrs[j] = typeid_cast<ColumnNullable *>(columns[j].get());
|
nullable_column_ptrs[j] = typeid_cast<ColumnNullable *>(columns[j].get());
|
||||||
}
|
}
|
||||||
join_data_avg_perkey_rows = join.getJoinedData()->avgPerKeyRows();
|
|
||||||
output_by_row_list_threshold = join.getTableJoin().outputByRowListPerkeyRowsThreshold();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
size_t size() const { return columns.size(); }
|
size_t size() const { return columns.size(); }
|
||||||
@ -149,6 +150,7 @@ public:
|
|||||||
bool output_by_row_list = false;
|
bool output_by_row_list = false;
|
||||||
size_t join_data_avg_perkey_rows = 0;
|
size_t join_data_avg_perkey_rows = 0;
|
||||||
size_t output_by_row_list_threshold = 0;
|
size_t output_by_row_list_threshold = 0;
|
||||||
|
bool join_data_sorted = false;
|
||||||
IColumn::Filter filter;
|
IColumn::Filter filter;
|
||||||
|
|
||||||
void reserve(bool need_replicate)
|
void reserve(bool need_replicate)
|
||||||
|
@ -649,7 +649,6 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
|
|||||||
}
|
}
|
||||||
data->keys_to_join = total_rows;
|
data->keys_to_join = total_rows;
|
||||||
shrinkStoredBlocksToFit(total_bytes);
|
shrinkStoredBlocksToFit(total_bytes);
|
||||||
|
|
||||||
return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
|
return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1361,4 +1360,96 @@ bool HashJoin::needUsedFlagsForPerRightTableRow(std::shared_ptr<TableJoin> table
|
|||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
template <JoinKind KIND, typename Map, JoinStrictness STRICTNESS>
|
||||||
|
void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]])
|
||||||
|
{
|
||||||
|
constexpr JoinFeatures<KIND, STRICTNESS, Map> join_features;
|
||||||
|
if constexpr (!join_features.is_all_join || (!join_features.left && !join_features.inner))
|
||||||
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Only left or inner join table can be reranged.");
|
||||||
|
else
|
||||||
|
{
|
||||||
|
auto merge_rows_into_one_block = [&](BlocksList & blocks, RowRefList & rows_ref)
|
||||||
|
{
|
||||||
|
auto it = rows_ref.begin();
|
||||||
|
if (it.ok())
|
||||||
|
{
|
||||||
|
if (blocks.empty() || blocks.back().rows() >= DEFAULT_BLOCK_SIZE)
|
||||||
|
blocks.emplace_back(it->block->cloneEmpty());
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
auto & block = blocks.back();
|
||||||
|
size_t start_row = block.rows();
|
||||||
|
for (; it.ok(); ++it)
|
||||||
|
{
|
||||||
|
for (size_t i = 0; i < block.columns(); ++i)
|
||||||
|
{
|
||||||
|
auto & col = block.getByPosition(i).column->assumeMutableRef();
|
||||||
|
col.insertFrom(*it->block->getByPosition(i).column, it->row_num);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (block.rows() > start_row)
|
||||||
|
{
|
||||||
|
RowRefList new_rows_ref(&block, start_row, block.rows() - start_row);
|
||||||
|
rows_ref = std::move(new_rows_ref);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
auto visit_rows_map = [&](BlocksList & blocks, MapsAll & rows_map)
|
||||||
|
{
|
||||||
|
switch (data->type)
|
||||||
|
{
|
||||||
|
#define M(TYPE) \
|
||||||
|
case Type::TYPE: \
|
||||||
|
{\
|
||||||
|
rows_map.TYPE->forEachMapped([&](RowRefList & rows_ref) { merge_rows_into_one_block(blocks, rows_ref); }); \
|
||||||
|
break; \
|
||||||
|
}
|
||||||
|
APPLY_FOR_JOIN_VARIANTS(M)
|
||||||
|
#undef M
|
||||||
|
default:
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
BlocksList sorted_blocks;
|
||||||
|
visit_rows_map(sorted_blocks, map);
|
||||||
|
data->blocks.swap(sorted_blocks);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void HashJoin::tryRerangeRightTableData()
|
||||||
|
{
|
||||||
|
if (!table_join->allowJoinSorting() || table_join->getMixedJoinExpression() || !isInnerOrLeft(kind) || strictness != JoinStrictness::All)
|
||||||
|
return;
|
||||||
|
|
||||||
|
/// We should not rerange the right table on such conditions:
|
||||||
|
/// 1. the right table is already reranged by key or it is empty.
|
||||||
|
/// 2. the join clauses size is greater than 1, like `...join on a.key1=b.key1 or a.key2=b.key2`, we can not rerange the right table on different set of keys.
|
||||||
|
/// 3. the number of right table rows exceed the threshold, which may result in a significant cost for reranging and lead to performance degradation.
|
||||||
|
/// 4. the keys of right table is very sparse, which may result in insignificant performance improvement after reranging by key.
|
||||||
|
if (!data || data->sorted || data->blocks.empty() || data->maps.size() > 1 || data->rows_to_join > table_join->sortRightMaximumTableRows() || data->avgPerKeyRows() < table_join->sortRightMinimumPerkeyRows())
|
||||||
|
return;
|
||||||
|
|
||||||
|
if (data->keys_to_join == 0)
|
||||||
|
data->keys_to_join = getTotalRowCount();
|
||||||
|
|
||||||
|
/// If the there is no columns to add, means no columns to output, then the rerange would not improve performance by using column's `insertRangeFrom`
|
||||||
|
/// to replace column's `insertFrom` to make the output.
|
||||||
|
if (sample_block_with_columns_to_add.columns() == 0)
|
||||||
|
{
|
||||||
|
LOG_DEBUG(log, "The joined right table total rows :{}, total keys :{}", data->rows_to_join, data->keys_to_join);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
[[maybe_unused]] bool result = joinDispatch(
|
||||||
|
kind,
|
||||||
|
strictness,
|
||||||
|
data->maps.front(),
|
||||||
|
/*prefer_use_maps_all*/ false,
|
||||||
|
[&](auto kind_, auto strictness_, auto & map_) { tryRerangeRightTableDataImpl<kind_, decltype(map_), strictness_>(map_); });
|
||||||
|
chassert(result);
|
||||||
|
data->sorted = true;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -345,11 +345,12 @@ public:
|
|||||||
|
|
||||||
size_t blocks_allocated_size = 0;
|
size_t blocks_allocated_size = 0;
|
||||||
size_t blocks_nullmaps_allocated_size = 0;
|
size_t blocks_nullmaps_allocated_size = 0;
|
||||||
|
|
||||||
/// Number of rows of right table to join
|
/// Number of rows of right table to join
|
||||||
size_t rows_to_join = 0;
|
size_t rows_to_join = 0;
|
||||||
/// Number of keys of right table to join
|
/// Number of keys of right table to join
|
||||||
size_t keys_to_join = 0;
|
size_t keys_to_join = 0;
|
||||||
|
/// Whether the right table reranged by key
|
||||||
|
bool sorted = false;
|
||||||
|
|
||||||
size_t avgPerKeyRows() const
|
size_t avgPerKeyRows() const
|
||||||
{
|
{
|
||||||
@ -465,6 +466,10 @@ private:
|
|||||||
|
|
||||||
void validateAdditionalFilterExpression(std::shared_ptr<ExpressionActions> additional_filter_expression);
|
void validateAdditionalFilterExpression(std::shared_ptr<ExpressionActions> additional_filter_expression);
|
||||||
bool needUsedFlagsForPerRightTableRow(std::shared_ptr<TableJoin> table_join_) const;
|
bool needUsedFlagsForPerRightTableRow(std::shared_ptr<TableJoin> table_join_) const;
|
||||||
|
|
||||||
|
void tryRerangeRightTableData() override;
|
||||||
|
template <JoinKind KIND, typename Map, JoinStrictness STRICTNESS>
|
||||||
|
void tryRerangeRightTableDataImpl(Map & map);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -83,7 +83,6 @@ public:
|
|||||||
const Block & block_with_columns_to_add,
|
const Block & block_with_columns_to_add,
|
||||||
const MapsTemplateVector & maps_,
|
const MapsTemplateVector & maps_,
|
||||||
bool is_join_get = false);
|
bool is_join_get = false);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
template <typename KeyGetter, bool is_asof_join>
|
template <typename KeyGetter, bool is_asof_join>
|
||||||
static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes);
|
static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes);
|
||||||
@ -199,4 +198,3 @@ extern template class HashJoinMethods<JoinKind::Full, JoinStrictness::Semi, Hash
|
|||||||
extern template class HashJoinMethods<JoinKind::Full, JoinStrictness::Anti, HashJoin::MapsOne>;
|
extern template class HashJoinMethods<JoinKind::Full, JoinStrictness::Anti, HashJoin::MapsOne>;
|
||||||
extern template class HashJoinMethods<JoinKind::Full, JoinStrictness::Asof, HashJoin::MapsAsof>;
|
extern template class HashJoinMethods<JoinKind::Full, JoinStrictness::Asof, HashJoin::MapsAsof>;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -115,6 +115,7 @@ public:
|
|||||||
/// Peek next stream of delayed joined blocks.
|
/// Peek next stream of delayed joined blocks.
|
||||||
virtual IBlocksStreamPtr getDelayedBlocks() { return nullptr; }
|
virtual IBlocksStreamPtr getDelayedBlocks() { return nullptr; }
|
||||||
virtual bool hasDelayedBlocks() const { return false; }
|
virtual bool hasDelayedBlocks() const { return false; }
|
||||||
|
virtual void tryRerangeRightTableData() {}
|
||||||
|
|
||||||
virtual IBlocksStreamPtr
|
virtual IBlocksStreamPtr
|
||||||
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const = 0;
|
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const = 0;
|
||||||
|
@ -1065,6 +1065,11 @@ namespace
|
|||||||
|
|
||||||
void setNullTableEngine(ASTStorage & storage)
|
void setNullTableEngine(ASTStorage & storage)
|
||||||
{
|
{
|
||||||
|
storage.forEachPointerToChild([](void ** ptr) mutable
|
||||||
|
{
|
||||||
|
*ptr = nullptr;
|
||||||
|
});
|
||||||
|
|
||||||
auto engine_ast = std::make_shared<ASTFunction>();
|
auto engine_ast = std::make_shared<ASTFunction>();
|
||||||
engine_ast->name = "Null";
|
engine_ast->name = "Null";
|
||||||
engine_ast->no_empty_args = true;
|
engine_ast->no_empty_args = true;
|
||||||
@ -1151,8 +1156,10 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
|
|||||||
else if (getContext()->getSettingsRef().restore_replace_external_engines_to_null)
|
else if (getContext()->getSettingsRef().restore_replace_external_engines_to_null)
|
||||||
{
|
{
|
||||||
if (StorageFactory::instance().getStorageFeatures(create.storage->engine->name).source_access_type != AccessType::NONE)
|
if (StorageFactory::instance().getStorageFeatures(create.storage->engine->name).source_access_type != AccessType::NONE)
|
||||||
|
{
|
||||||
setNullTableEngine(*create.storage);
|
setNullTableEngine(*create.storage);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1224,9 +1224,18 @@ void MutationsInterpreter::Source::read(
|
|||||||
|
|
||||||
createReadFromPartStep(
|
createReadFromPartStep(
|
||||||
MergeTreeSequentialSourceType::Mutation,
|
MergeTreeSequentialSourceType::Mutation,
|
||||||
plan, *data, storage_snapshot,
|
plan,
|
||||||
part, alter_conversions, required_columns,
|
*data,
|
||||||
apply_deleted_mask_, std::move(filter), context_,
|
storage_snapshot,
|
||||||
|
part,
|
||||||
|
alter_conversions,
|
||||||
|
required_columns,
|
||||||
|
nullptr,
|
||||||
|
apply_deleted_mask_,
|
||||||
|
std::move(filter),
|
||||||
|
false,
|
||||||
|
false,
|
||||||
|
context_,
|
||||||
getLogger("MutationsInterpreter"));
|
getLogger("MutationsInterpreter"));
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
|
@ -123,6 +123,7 @@ struct RowRefList : RowRef
|
|||||||
|
|
||||||
RowRefList() {} /// NOLINT
|
RowRefList() {} /// NOLINT
|
||||||
RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_), rows(1) {}
|
RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_), rows(1) {}
|
||||||
|
RowRefList(const Block * block_, size_t row_start_, size_t rows_) : RowRef(block_, row_start_), rows(static_cast<SizeT>(rows_)) {}
|
||||||
|
|
||||||
ForwardIterator begin() const { return ForwardIterator(this); }
|
ForwardIterator begin() const { return ForwardIterator(this); }
|
||||||
|
|
||||||
|
@ -55,9 +55,11 @@ ServerAsynchronousMetrics::ServerAsynchronousMetrics(
|
|||||||
ContextPtr global_context_,
|
ContextPtr global_context_,
|
||||||
unsigned update_period_seconds,
|
unsigned update_period_seconds,
|
||||||
unsigned heavy_metrics_update_period_seconds,
|
unsigned heavy_metrics_update_period_seconds,
|
||||||
const ProtocolServerMetricsFunc & protocol_server_metrics_func_)
|
const ProtocolServerMetricsFunc & protocol_server_metrics_func_,
|
||||||
|
bool update_jemalloc_epoch_,
|
||||||
|
bool update_rss_)
|
||||||
: WithContext(global_context_)
|
: WithContext(global_context_)
|
||||||
, AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_)
|
, AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, update_jemalloc_epoch_, update_rss_)
|
||||||
, heavy_metric_update_period(heavy_metrics_update_period_seconds)
|
, heavy_metric_update_period(heavy_metrics_update_period_seconds)
|
||||||
{
|
{
|
||||||
/// sanity check
|
/// sanity check
|
||||||
|
@ -14,7 +14,10 @@ public:
|
|||||||
ContextPtr global_context_,
|
ContextPtr global_context_,
|
||||||
unsigned update_period_seconds,
|
unsigned update_period_seconds,
|
||||||
unsigned heavy_metrics_update_period_seconds,
|
unsigned heavy_metrics_update_period_seconds,
|
||||||
const ProtocolServerMetricsFunc & protocol_server_metrics_func_);
|
const ProtocolServerMetricsFunc & protocol_server_metrics_func_,
|
||||||
|
bool update_jemalloc_epoch_,
|
||||||
|
bool update_rss_);
|
||||||
|
|
||||||
~ServerAsynchronousMetrics() override;
|
~ServerAsynchronousMetrics() override;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
@ -45,7 +45,7 @@ Chunk Squashing::squash(Chunk && input_chunk)
|
|||||||
|
|
||||||
Chunk Squashing::add(Chunk && input_chunk)
|
Chunk Squashing::add(Chunk && input_chunk)
|
||||||
{
|
{
|
||||||
if (!input_chunk)
|
if (!input_chunk || input_chunk.getNumRows() == 0)
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
/// Just read block is already enough.
|
/// Just read block is already enough.
|
||||||
|
@ -116,6 +116,9 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary
|
|||||||
, max_files_to_merge(settings.join_on_disk_max_files_to_merge)
|
, max_files_to_merge(settings.join_on_disk_max_files_to_merge)
|
||||||
, temporary_files_codec(settings.temporary_files_codec)
|
, temporary_files_codec(settings.temporary_files_codec)
|
||||||
, output_by_rowlist_perkey_rows_threshold(settings.join_output_by_rowlist_perkey_rows_threshold)
|
, output_by_rowlist_perkey_rows_threshold(settings.join_output_by_rowlist_perkey_rows_threshold)
|
||||||
|
, sort_right_minimum_perkey_rows(settings.join_to_sort_minimum_perkey_rows)
|
||||||
|
, sort_right_maximum_table_rows(settings.join_to_sort_maximum_table_rows)
|
||||||
|
, allow_join_sorting(settings.allow_experimental_join_right_table_sorting)
|
||||||
, max_memory_usage(settings.max_memory_usage)
|
, max_memory_usage(settings.max_memory_usage)
|
||||||
, tmp_volume(tmp_volume_)
|
, tmp_volume(tmp_volume_)
|
||||||
, tmp_data(tmp_data_)
|
, tmp_data(tmp_data_)
|
||||||
|
@ -149,6 +149,9 @@ private:
|
|||||||
const size_t max_files_to_merge = 0;
|
const size_t max_files_to_merge = 0;
|
||||||
const String temporary_files_codec = "LZ4";
|
const String temporary_files_codec = "LZ4";
|
||||||
const size_t output_by_rowlist_perkey_rows_threshold = 0;
|
const size_t output_by_rowlist_perkey_rows_threshold = 0;
|
||||||
|
const size_t sort_right_minimum_perkey_rows = 0;
|
||||||
|
const size_t sort_right_maximum_table_rows = 0;
|
||||||
|
const bool allow_join_sorting = false;
|
||||||
|
|
||||||
/// Value if setting max_memory_usage for query, can be used when max_bytes_in_join is not specified.
|
/// Value if setting max_memory_usage for query, can be used when max_bytes_in_join is not specified.
|
||||||
size_t max_memory_usage = 0;
|
size_t max_memory_usage = 0;
|
||||||
@ -297,6 +300,9 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
size_t outputByRowListPerkeyRowsThreshold() const { return output_by_rowlist_perkey_rows_threshold; }
|
size_t outputByRowListPerkeyRowsThreshold() const { return output_by_rowlist_perkey_rows_threshold; }
|
||||||
|
size_t sortRightMinimumPerkeyRows() const { return sort_right_minimum_perkey_rows; }
|
||||||
|
size_t sortRightMaximumTableRows() const { return sort_right_maximum_table_rows; }
|
||||||
|
bool allowJoinSorting() const { return allow_join_sorting; }
|
||||||
size_t defaultMaxBytes() const { return default_max_bytes; }
|
size_t defaultMaxBytes() const { return default_max_bytes; }
|
||||||
size_t maxJoinedBlockRows() const { return max_joined_block_rows; }
|
size_t maxJoinedBlockRows() const { return max_joined_block_rows; }
|
||||||
size_t maxRowsInRightBlock() const { return partial_merge_join_rows_in_right_blocks; }
|
size_t maxRowsInRightBlock() const { return partial_merge_join_rows_in_right_blocks; }
|
||||||
|
@ -93,6 +93,7 @@ ColumnsDescription ZooKeeperLogElement::getColumnsDescription()
|
|||||||
{"FilteredList", static_cast<Int16>(Coordination::OpNum::FilteredList)},
|
{"FilteredList", static_cast<Int16>(Coordination::OpNum::FilteredList)},
|
||||||
{"CheckNotExists", static_cast<Int16>(Coordination::OpNum::CheckNotExists)},
|
{"CheckNotExists", static_cast<Int16>(Coordination::OpNum::CheckNotExists)},
|
||||||
{"CreateIfNotExists", static_cast<Int16>(Coordination::OpNum::CreateIfNotExists)},
|
{"CreateIfNotExists", static_cast<Int16>(Coordination::OpNum::CreateIfNotExists)},
|
||||||
|
{"RemoveRecursive", static_cast<Int16>(Coordination::OpNum::RemoveRecursive)},
|
||||||
});
|
});
|
||||||
|
|
||||||
auto error_enum = getCoordinationErrorCodesEnumType();
|
auto error_enum = getCoordinationErrorCodesEnumType();
|
||||||
|
@ -1118,6 +1118,19 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
&& settings.use_query_cache
|
&& settings.use_query_cache
|
||||||
&& !internal
|
&& !internal
|
||||||
&& client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY
|
&& client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY
|
||||||
|
/// Bug 67476: Avoid that the query cache stores truncated results if the query ran with a non-THROW overflow mode and hit a limit.
|
||||||
|
/// This is more workaround than a fix ... unfortunately it is hard to detect from the perspective of the query cache that the
|
||||||
|
/// query result is truncated.
|
||||||
|
&& (settings.read_overflow_mode == OverflowMode::THROW
|
||||||
|
&& settings.read_overflow_mode_leaf == OverflowMode::THROW
|
||||||
|
&& settings.group_by_overflow_mode == OverflowMode::THROW
|
||||||
|
&& settings.sort_overflow_mode == OverflowMode::THROW
|
||||||
|
&& settings.result_overflow_mode == OverflowMode::THROW
|
||||||
|
&& settings.timeout_overflow_mode == OverflowMode::THROW
|
||||||
|
&& settings.set_overflow_mode == OverflowMode::THROW
|
||||||
|
&& settings.join_overflow_mode == OverflowMode::THROW
|
||||||
|
&& settings.transfer_overflow_mode == OverflowMode::THROW
|
||||||
|
&& settings.distinct_overflow_mode == OverflowMode::THROW)
|
||||||
&& (ast->as<ASTSelectQuery>() || ast->as<ASTSelectWithUnionQuery>());
|
&& (ast->as<ASTSelectQuery>() || ast->as<ASTSelectWithUnionQuery>());
|
||||||
QueryCache::Usage query_cache_usage = QueryCache::Usage::None;
|
QueryCache::Usage query_cache_usage = QueryCache::Usage::None;
|
||||||
|
|
||||||
|
@ -892,6 +892,30 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
|||||||
|
|
||||||
/// query_plan can be empty if there is nothing to read
|
/// query_plan can be empty if there is nothing to read
|
||||||
if (query_plan.isInitialized() && parallel_replicas_enabled_for_storage(storage, settings))
|
if (query_plan.isInitialized() && parallel_replicas_enabled_for_storage(storage, settings))
|
||||||
|
{
|
||||||
|
if (query_context->canUseParallelReplicasCustomKey() && query_context->getClientInfo().distributed_depth == 0)
|
||||||
|
{
|
||||||
|
if (auto cluster = query_context->getClusterForParallelReplicas();
|
||||||
|
query_context->canUseParallelReplicasCustomKeyForCluster(*cluster))
|
||||||
|
{
|
||||||
|
planner_context->getMutableQueryContext()->setSetting("prefer_localhost_replica", Field{0});
|
||||||
|
auto modified_query_info = select_query_info;
|
||||||
|
modified_query_info.cluster = std::move(cluster);
|
||||||
|
from_stage = QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit;
|
||||||
|
QueryPlan query_plan_parallel_replicas;
|
||||||
|
ClusterProxy::executeQueryWithParallelReplicasCustomKey(
|
||||||
|
query_plan_parallel_replicas,
|
||||||
|
storage->getStorageID(),
|
||||||
|
modified_query_info,
|
||||||
|
storage->getInMemoryMetadataPtr()->getColumns(),
|
||||||
|
storage_snapshot,
|
||||||
|
from_stage,
|
||||||
|
table_expression_query_info.query_tree,
|
||||||
|
query_context);
|
||||||
|
query_plan = std::move(query_plan_parallel_replicas);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else if (ClusterProxy::canUseParallelReplicasOnInitiator(query_context))
|
||||||
{
|
{
|
||||||
// (1) find read step
|
// (1) find read step
|
||||||
QueryPlan::Node * node = query_plan.getRootNode();
|
QueryPlan::Node * node = query_plan.getRootNode();
|
||||||
@ -918,43 +942,23 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
|||||||
}
|
}
|
||||||
|
|
||||||
chassert(reading);
|
chassert(reading);
|
||||||
if (query_context->canUseParallelReplicasCustomKey() && query_context->getClientInfo().distributed_depth == 0)
|
|
||||||
{
|
|
||||||
if (auto cluster = query_context->getClusterForParallelReplicas();
|
|
||||||
query_context->canUseParallelReplicasCustomKeyForCluster(*cluster))
|
|
||||||
{
|
|
||||||
planner_context->getMutableQueryContext()->setSetting("prefer_localhost_replica", Field{0});
|
|
||||||
auto modified_query_info = select_query_info;
|
|
||||||
modified_query_info.cluster = std::move(cluster);
|
|
||||||
from_stage = QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit;
|
|
||||||
QueryPlan query_plan_parallel_replicas;
|
|
||||||
ClusterProxy::executeQueryWithParallelReplicasCustomKey(
|
|
||||||
query_plan_parallel_replicas,
|
|
||||||
storage->getStorageID(),
|
|
||||||
modified_query_info,
|
|
||||||
storage->getInMemoryMetadataPtr()->getColumns(),
|
|
||||||
storage_snapshot,
|
|
||||||
from_stage,
|
|
||||||
table_expression_query_info.query_tree,
|
|
||||||
query_context);
|
|
||||||
query_plan = std::move(query_plan_parallel_replicas);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
else if (query_context->canUseParallelReplicasOnInitiator())
|
|
||||||
{
|
|
||||||
// (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read
|
// (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read
|
||||||
if (settings.parallel_replicas_min_number_of_rows_per_replica > 0)
|
if (settings.parallel_replicas_min_number_of_rows_per_replica > 0)
|
||||||
{
|
{
|
||||||
auto result_ptr = reading->selectRangesToRead();
|
auto result_ptr = reading->selectRangesToRead();
|
||||||
|
|
||||||
UInt64 rows_to_read = result_ptr->selected_rows;
|
UInt64 rows_to_read = result_ptr->selected_rows;
|
||||||
|
|
||||||
|
reading->setAnalyzedResult(std::move(result_ptr));
|
||||||
|
|
||||||
if (table_expression_query_info.trivial_limit > 0 && table_expression_query_info.trivial_limit < rows_to_read)
|
if (table_expression_query_info.trivial_limit > 0 && table_expression_query_info.trivial_limit < rows_to_read)
|
||||||
rows_to_read = table_expression_query_info.trivial_limit;
|
rows_to_read = table_expression_query_info.trivial_limit;
|
||||||
|
|
||||||
if (max_block_size_limited && (max_block_size_limited < rows_to_read))
|
if (max_block_size_limited && (max_block_size_limited < rows_to_read))
|
||||||
rows_to_read = max_block_size_limited;
|
rows_to_read = max_block_size_limited;
|
||||||
|
|
||||||
const size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica;
|
const size_t number_of_replicas_to_use
|
||||||
|
= rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica;
|
||||||
LOG_TRACE(
|
LOG_TRACE(
|
||||||
getLogger("Planner"),
|
getLogger("Planner"),
|
||||||
"Estimated {} rows to read. It is enough work for {} parallel replicas",
|
"Estimated {} rows to read. It is enough work for {} parallel replicas",
|
||||||
@ -980,6 +984,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
|||||||
{
|
{
|
||||||
from_stage = QueryProcessingStage::WithMergeableState;
|
from_stage = QueryProcessingStage::WithMergeableState;
|
||||||
QueryPlan query_plan_parallel_replicas;
|
QueryPlan query_plan_parallel_replicas;
|
||||||
|
QueryPlanStepPtr reading_step = std::move(node->step);
|
||||||
ClusterProxy::executeQueryWithParallelReplicas(
|
ClusterProxy::executeQueryWithParallelReplicas(
|
||||||
query_plan_parallel_replicas,
|
query_plan_parallel_replicas,
|
||||||
storage->getStorageID(),
|
storage->getStorageID(),
|
||||||
@ -987,9 +992,24 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
|||||||
table_expression_query_info.query_tree,
|
table_expression_query_info.query_tree,
|
||||||
table_expression_query_info.planner_context,
|
table_expression_query_info.planner_context,
|
||||||
query_context,
|
query_context,
|
||||||
table_expression_query_info.storage_limits);
|
table_expression_query_info.storage_limits,
|
||||||
|
std::move(reading_step));
|
||||||
query_plan = std::move(query_plan_parallel_replicas);
|
query_plan = std::move(query_plan_parallel_replicas);
|
||||||
}
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
QueryPlan query_plan_no_parallel_replicas;
|
||||||
|
storage->read(
|
||||||
|
query_plan_no_parallel_replicas,
|
||||||
|
columns_names,
|
||||||
|
storage_snapshot,
|
||||||
|
table_expression_query_info,
|
||||||
|
query_context,
|
||||||
|
from_stage,
|
||||||
|
max_block_size,
|
||||||
|
max_streams);
|
||||||
|
query_plan = std::move(query_plan_no_parallel_replicas);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -52,7 +52,13 @@ std::stack<const QueryNode *> getSupportingParallelReplicasQuery(const IQueryTre
|
|||||||
const auto & storage = table_node.getStorage();
|
const auto & storage = table_node.getStorage();
|
||||||
/// Here we check StorageDummy as well, to support a query tree with replaced storages.
|
/// Here we check StorageDummy as well, to support a query tree with replaced storages.
|
||||||
if (std::dynamic_pointer_cast<MergeTreeData>(storage) || typeid_cast<const StorageDummy *>(storage.get()))
|
if (std::dynamic_pointer_cast<MergeTreeData>(storage) || typeid_cast<const StorageDummy *>(storage.get()))
|
||||||
|
{
|
||||||
|
/// parallel replicas is not supported with FINAL
|
||||||
|
if (table_node.getTableExpressionModifiers() && table_node.getTableExpressionModifiers()->hasFinal())
|
||||||
|
return {};
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
|
@ -12,12 +12,15 @@ namespace DB
|
|||||||
struct Settings;
|
struct Settings;
|
||||||
class QueryStatus;
|
class QueryStatus;
|
||||||
using QueryStatusPtr = std::shared_ptr<QueryStatus>;
|
using QueryStatusPtr = std::shared_ptr<QueryStatus>;
|
||||||
|
struct ITemporaryFileLookup;
|
||||||
|
using TemporaryFileLookupPtr = std::shared_ptr<ITemporaryFileLookup>;
|
||||||
|
|
||||||
struct BuildQueryPipelineSettings
|
struct BuildQueryPipelineSettings
|
||||||
{
|
{
|
||||||
ExpressionActionsSettings actions_settings;
|
ExpressionActionsSettings actions_settings;
|
||||||
QueryStatusPtr process_list_element;
|
QueryStatusPtr process_list_element;
|
||||||
ProgressCallback progress_callback = nullptr;
|
ProgressCallback progress_callback = nullptr;
|
||||||
|
TemporaryFileLookupPtr temporary_file_lookup;
|
||||||
|
|
||||||
const ExpressionActionsSettings & getActionsSettings() const { return actions_settings; }
|
const ExpressionActionsSettings & getActionsSettings() const { return actions_settings; }
|
||||||
static BuildQueryPipelineSettings fromContext(ContextPtr from);
|
static BuildQueryPipelineSettings fromContext(ContextPtr from);
|
||||||
|
32
src/Processors/QueryPlan/ConvertingActions.cpp
Normal file
32
src/Processors/QueryPlan/ConvertingActions.cpp
Normal file
@ -0,0 +1,32 @@
|
|||||||
|
#include <Processors/QueryPlan/QueryPlan.h>
|
||||||
|
#include <Interpreters/ActionsDAG.h>
|
||||||
|
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects)
|
||||||
|
{
|
||||||
|
if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header))
|
||||||
|
return;
|
||||||
|
|
||||||
|
auto mode = has_missing_objects ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name;
|
||||||
|
|
||||||
|
auto get_converting_dag = [mode](const Block & block_, const Block & header_)
|
||||||
|
{
|
||||||
|
/// Convert header structure to expected.
|
||||||
|
/// Also we ignore constants from result and replace it with constants from header.
|
||||||
|
/// It is needed for functions like `now64()` or `randConstant()` because their values may be different.
|
||||||
|
return ActionsDAG::makeConvertingActions(
|
||||||
|
block_.getColumnsWithTypeAndName(),
|
||||||
|
header_.getColumnsWithTypeAndName(),
|
||||||
|
mode,
|
||||||
|
true);
|
||||||
|
};
|
||||||
|
|
||||||
|
auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header);
|
||||||
|
auto converting = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(convert_actions_dag));
|
||||||
|
plan.addStep(std::move(converting));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
9
src/Processors/QueryPlan/ConvertingActions.h
Normal file
9
src/Processors/QueryPlan/ConvertingActions.h
Normal file
@ -0,0 +1,9 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
class QueryPlan;
|
||||||
|
class Block;
|
||||||
|
|
||||||
|
void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects);
|
||||||
|
}
|
@ -2,43 +2,13 @@
|
|||||||
|
|
||||||
#include <Common/checkStackSize.h>
|
#include <Common/checkStackSize.h>
|
||||||
#include <Core/Settings.h>
|
#include <Core/Settings.h>
|
||||||
#include <Interpreters/ActionsDAG.h>
|
|
||||||
#include <Interpreters/InterpreterSelectQuery.h>
|
#include <Interpreters/InterpreterSelectQuery.h>
|
||||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||||
#include <Processors/QueryPlan/ExpressionStep.h>
|
#include <Processors/QueryPlan/ConvertingActions.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
namespace
|
|
||||||
{
|
|
||||||
|
|
||||||
void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects)
|
|
||||||
{
|
|
||||||
if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header))
|
|
||||||
return;
|
|
||||||
|
|
||||||
auto mode = has_missing_objects ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name;
|
|
||||||
|
|
||||||
auto get_converting_dag = [mode](const Block & block_, const Block & header_)
|
|
||||||
{
|
|
||||||
/// Convert header structure to expected.
|
|
||||||
/// Also we ignore constants from result and replace it with constants from header.
|
|
||||||
/// It is needed for functions like `now64()` or `randConstant()` because their values may be different.
|
|
||||||
return ActionsDAG::makeConvertingActions(
|
|
||||||
block_.getColumnsWithTypeAndName(),
|
|
||||||
header_.getColumnsWithTypeAndName(),
|
|
||||||
mode,
|
|
||||||
true);
|
|
||||||
};
|
|
||||||
|
|
||||||
auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header);
|
|
||||||
auto converting = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(convert_actions_dag));
|
|
||||||
plan.addStep(std::move(converting));
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
std::unique_ptr<QueryPlan> createLocalPlan(
|
std::unique_ptr<QueryPlan> createLocalPlan(
|
||||||
const ASTPtr & query_ast,
|
const ASTPtr & query_ast,
|
||||||
const Block & header,
|
const Block & header,
|
||||||
|
@ -18,6 +18,7 @@
|
|||||||
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
|
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
|
||||||
#include <Processors/QueryPlan/Optimizations/actionsDAGUtils.h>
|
#include <Processors/QueryPlan/Optimizations/actionsDAGUtils.h>
|
||||||
#include <Processors/QueryPlan/ReadFromMergeTree.h>
|
#include <Processors/QueryPlan/ReadFromMergeTree.h>
|
||||||
|
#include <Processors/QueryPlan/ReadFromRemote.h>
|
||||||
#include <Processors/QueryPlan/SortingStep.h>
|
#include <Processors/QueryPlan/SortingStep.h>
|
||||||
#include <Processors/QueryPlan/TotalsHavingStep.h>
|
#include <Processors/QueryPlan/TotalsHavingStep.h>
|
||||||
#include <Processors/QueryPlan/UnionStep.h>
|
#include <Processors/QueryPlan/UnionStep.h>
|
||||||
@ -899,6 +900,18 @@ AggregationInputOrder buildInputOrderInfo(AggregatingStep & aggregating, QueryPl
|
|||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static bool readingFromParallelReplicas(const QueryPlan::Node * node)
|
||||||
|
{
|
||||||
|
IQueryPlanStep * step = node->step.get();
|
||||||
|
while (!node->children.empty())
|
||||||
|
{
|
||||||
|
step = node->children.front()->step.get();
|
||||||
|
node = node->children.front();
|
||||||
|
}
|
||||||
|
|
||||||
|
return typeid_cast<const ReadFromParallelRemoteReplicasStep *>(step);
|
||||||
|
}
|
||||||
|
|
||||||
void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes)
|
void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes)
|
||||||
{
|
{
|
||||||
if (node.children.size() != 1)
|
if (node.children.size() != 1)
|
||||||
@ -924,6 +937,16 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes)
|
|||||||
std::vector<InputOrderInfoPtr> infos;
|
std::vector<InputOrderInfoPtr> infos;
|
||||||
infos.reserve(node.children.size());
|
infos.reserve(node.children.size());
|
||||||
|
|
||||||
|
for (const auto * child : union_node->children)
|
||||||
|
{
|
||||||
|
/// in case of parallel replicas
|
||||||
|
/// avoid applying read-in-order optimization for local replica
|
||||||
|
/// since it will lead to different parallel replicas modes
|
||||||
|
/// between local and remote nodes
|
||||||
|
if (readingFromParallelReplicas(child))
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
for (auto * child : union_node->children)
|
for (auto * child : union_node->children)
|
||||||
{
|
{
|
||||||
infos.push_back(buildInputOrderInfo(*sorting, *child, steps_to_update));
|
infos.push_back(buildInputOrderInfo(*sorting, *child, steps_to_update));
|
||||||
|
97
src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp
Normal file
97
src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp
Normal file
@ -0,0 +1,97 @@
|
|||||||
|
#include <Processors/QueryPlan/ParallelReplicasLocalPlan.h>
|
||||||
|
|
||||||
|
#include <Common/checkStackSize.h>
|
||||||
|
#include <Interpreters/ActionsDAG.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||||
|
#include <Interpreters/StorageID.h>
|
||||||
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Processors/QueryPlan/ConvertingActions.h>
|
||||||
|
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||||
|
#include <Processors/QueryPlan/ISourceStep.h>
|
||||||
|
#include <Processors/QueryPlan/ReadFromMergeTree.h>
|
||||||
|
#include <Processors/Sources/NullSource.h>
|
||||||
|
#include <Processors/Transforms/ExpressionTransform.h>
|
||||||
|
#include <Processors/Transforms/FilterTransform.h>
|
||||||
|
#include <QueryPipeline/Pipe.h>
|
||||||
|
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||||
|
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
||||||
|
#include <Storages/MergeTree/RequestResponse.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
std::pair<std::unique_ptr<QueryPlan>, bool> createLocalPlanForParallelReplicas(
|
||||||
|
const ASTPtr & query_ast,
|
||||||
|
const Block & header,
|
||||||
|
ContextPtr context,
|
||||||
|
QueryProcessingStage::Enum processed_stage,
|
||||||
|
ParallelReplicasReadingCoordinatorPtr coordinator,
|
||||||
|
QueryPlanStepPtr analyzed_read_from_merge_tree,
|
||||||
|
size_t replica_number)
|
||||||
|
{
|
||||||
|
checkStackSize();
|
||||||
|
|
||||||
|
auto query_plan = std::make_unique<QueryPlan>();
|
||||||
|
auto new_context = Context::createCopy(context);
|
||||||
|
|
||||||
|
/// Do not push down limit to local plan, as it will break `rows_before_limit_at_least` counter.
|
||||||
|
if (processed_stage == QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit)
|
||||||
|
processed_stage = QueryProcessingStage::WithMergeableStateAfterAggregation;
|
||||||
|
|
||||||
|
/// Do not apply AST optimizations, because query
|
||||||
|
/// is already optimized and some optimizations
|
||||||
|
/// can be applied only for non-distributed tables
|
||||||
|
/// and we can produce query, inconsistent with remote plans.
|
||||||
|
auto select_query_options = SelectQueryOptions(processed_stage).ignoreASTOptimizations();
|
||||||
|
|
||||||
|
/// For Analyzer, identifier in GROUP BY/ORDER BY/LIMIT BY lists has been resolved to
|
||||||
|
/// ConstantNode in QueryTree if it is an alias of a constant, so we should not replace
|
||||||
|
/// ConstantNode with ProjectionNode again(https://github.com/ClickHouse/ClickHouse/issues/62289).
|
||||||
|
new_context->setSetting("enable_positional_arguments", Field(false));
|
||||||
|
new_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0));
|
||||||
|
auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, new_context, select_query_options);
|
||||||
|
query_plan = std::make_unique<QueryPlan>(std::move(interpreter).extractQueryPlan());
|
||||||
|
|
||||||
|
QueryPlan::Node * node = query_plan->getRootNode();
|
||||||
|
ReadFromMergeTree * reading = nullptr;
|
||||||
|
while (node)
|
||||||
|
{
|
||||||
|
reading = typeid_cast<ReadFromMergeTree *>(node->step.get());
|
||||||
|
if (reading)
|
||||||
|
break;
|
||||||
|
|
||||||
|
if (!node->children.empty())
|
||||||
|
node = node->children.at(0);
|
||||||
|
else
|
||||||
|
node = nullptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!reading)
|
||||||
|
/// it can happened if merge tree table is empty, - it'll be replaced with ReadFromPreparedSource
|
||||||
|
return {std::move(query_plan), false};
|
||||||
|
|
||||||
|
ReadFromMergeTree::AnalysisResultPtr analyzed_result_ptr;
|
||||||
|
if (analyzed_read_from_merge_tree.get())
|
||||||
|
{
|
||||||
|
auto * analyzed_merge_tree = typeid_cast<ReadFromMergeTree *>(analyzed_read_from_merge_tree.get());
|
||||||
|
if (analyzed_merge_tree)
|
||||||
|
analyzed_result_ptr = analyzed_merge_tree->getAnalyzedResult();
|
||||||
|
}
|
||||||
|
|
||||||
|
MergeTreeAllRangesCallback all_ranges_cb = [coordinator](InitialAllRangesAnnouncement announcement)
|
||||||
|
{ coordinator->handleInitialAllRangesAnnouncement(std::move(announcement)); };
|
||||||
|
|
||||||
|
MergeTreeReadTaskCallback read_task_cb = [coordinator](ParallelReadRequest req) -> std::optional<ParallelReadResponse>
|
||||||
|
{ return coordinator->handleRequest(std::move(req)); };
|
||||||
|
|
||||||
|
auto read_from_merge_tree_parallel_replicas = reading->createLocalParallelReplicasReadingStep(
|
||||||
|
analyzed_result_ptr, std::move(all_ranges_cb), std::move(read_task_cb), replica_number);
|
||||||
|
node->step = std::move(read_from_merge_tree_parallel_replicas);
|
||||||
|
|
||||||
|
addConvertingActions(*query_plan, header, /*has_missing_objects=*/false);
|
||||||
|
|
||||||
|
return {std::move(query_plan), true};
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
19
src/Processors/QueryPlan/ParallelReplicasLocalPlan.h
Normal file
19
src/Processors/QueryPlan/ParallelReplicasLocalPlan.h
Normal file
@ -0,0 +1,19 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Core/QueryProcessingStage.h>
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
#include <Processors/QueryPlan/QueryPlan.h>
|
||||||
|
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
std::pair<std::unique_ptr<QueryPlan>, bool> createLocalPlanForParallelReplicas(
|
||||||
|
const ASTPtr & query_ast,
|
||||||
|
const Block & header,
|
||||||
|
ContextPtr context,
|
||||||
|
QueryProcessingStage::Enum processed_stage,
|
||||||
|
ParallelReplicasReadingCoordinatorPtr coordinator,
|
||||||
|
QueryPlanStepPtr read_from_merge_tree,
|
||||||
|
size_t replica_number);
|
||||||
|
}
|
@ -277,7 +277,10 @@ ReadFromMergeTree::ReadFromMergeTree(
|
|||||||
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read_,
|
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read_,
|
||||||
LoggerPtr log_,
|
LoggerPtr log_,
|
||||||
AnalysisResultPtr analyzed_result_ptr_,
|
AnalysisResultPtr analyzed_result_ptr_,
|
||||||
bool enable_parallel_reading)
|
bool enable_parallel_reading_,
|
||||||
|
std::optional<MergeTreeAllRangesCallback> all_ranges_callback_,
|
||||||
|
std::optional<MergeTreeReadTaskCallback> read_task_callback_,
|
||||||
|
std::optional<size_t> number_of_current_replica_)
|
||||||
: SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader(
|
: SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader(
|
||||||
storage_snapshot_->getSampleBlockForColumns(all_column_names_),
|
storage_snapshot_->getSampleBlockForColumns(all_column_names_),
|
||||||
query_info_.prewhere_info)}, all_column_names_, query_info_, storage_snapshot_, context_)
|
query_info_.prewhere_info)}, all_column_names_, query_info_, storage_snapshot_, context_)
|
||||||
@ -295,12 +298,20 @@ ReadFromMergeTree::ReadFromMergeTree(
|
|||||||
, max_block_numbers_to_read(std::move(max_block_numbers_to_read_))
|
, max_block_numbers_to_read(std::move(max_block_numbers_to_read_))
|
||||||
, log(std::move(log_))
|
, log(std::move(log_))
|
||||||
, analyzed_result_ptr(analyzed_result_ptr_)
|
, analyzed_result_ptr(analyzed_result_ptr_)
|
||||||
, is_parallel_reading_from_replicas(enable_parallel_reading)
|
, is_parallel_reading_from_replicas(enable_parallel_reading_)
|
||||||
, enable_remove_parts_from_snapshot_optimization(query_info_.merge_tree_enable_remove_parts_from_snapshot_optimization)
|
, enable_remove_parts_from_snapshot_optimization(query_info_.merge_tree_enable_remove_parts_from_snapshot_optimization)
|
||||||
|
, number_of_current_replica(number_of_current_replica_)
|
||||||
{
|
{
|
||||||
if (is_parallel_reading_from_replicas)
|
if (is_parallel_reading_from_replicas)
|
||||||
{
|
{
|
||||||
|
if (all_ranges_callback_.has_value())
|
||||||
|
all_ranges_callback = all_ranges_callback_.value();
|
||||||
|
else
|
||||||
all_ranges_callback = context->getMergeTreeAllRangesCallback();
|
all_ranges_callback = context->getMergeTreeAllRangesCallback();
|
||||||
|
|
||||||
|
if (read_task_callback_.has_value())
|
||||||
|
read_task_callback = read_task_callback_.value();
|
||||||
|
else
|
||||||
read_task_callback = context->getMergeTreeReadTaskCallback();
|
read_task_callback = context->getMergeTreeReadTaskCallback();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -335,11 +346,33 @@ ReadFromMergeTree::ReadFromMergeTree(
|
|||||||
enable_vertical_final);
|
enable_vertical_final);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::unique_ptr<ReadFromMergeTree> ReadFromMergeTree::createLocalParallelReplicasReadingStep(
|
||||||
|
AnalysisResultPtr analyzed_result_ptr_,
|
||||||
|
MergeTreeAllRangesCallback all_ranges_callback_,
|
||||||
|
MergeTreeReadTaskCallback read_task_callback_,
|
||||||
|
size_t replica_number)
|
||||||
|
{
|
||||||
|
const bool enable_parallel_reading = true;
|
||||||
|
return std::make_unique<ReadFromMergeTree>(
|
||||||
|
prepared_parts,
|
||||||
|
mutations_snapshot,
|
||||||
|
all_column_names,
|
||||||
|
data,
|
||||||
|
getQueryInfo(),
|
||||||
|
getStorageSnapshot(),
|
||||||
|
getContext(),
|
||||||
|
block_size.max_block_size_rows,
|
||||||
|
requested_num_streams,
|
||||||
|
max_block_numbers_to_read,
|
||||||
|
log,
|
||||||
|
std::move(analyzed_result_ptr_),
|
||||||
|
enable_parallel_reading,
|
||||||
|
all_ranges_callback_,
|
||||||
|
read_task_callback_,
|
||||||
|
replica_number);
|
||||||
|
}
|
||||||
|
|
||||||
Pipe ReadFromMergeTree::readFromPoolParallelReplicas(
|
Pipe ReadFromMergeTree::readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings)
|
||||||
RangesInDataParts parts_with_range,
|
|
||||||
Names required_columns,
|
|
||||||
PoolSettings pool_settings)
|
|
||||||
{
|
{
|
||||||
const auto & client_info = context->getClientInfo();
|
const auto & client_info = context->getClientInfo();
|
||||||
|
|
||||||
@ -347,7 +380,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas(
|
|||||||
{
|
{
|
||||||
.all_callback = all_ranges_callback.value(),
|
.all_callback = all_ranges_callback.value(),
|
||||||
.callback = read_task_callback.value(),
|
.callback = read_task_callback.value(),
|
||||||
.number_of_current_replica = client_info.number_of_current_replica,
|
.number_of_current_replica = number_of_current_replica.value_or(client_info.number_of_current_replica),
|
||||||
};
|
};
|
||||||
|
|
||||||
/// We have a special logic for local replica. It has to read less data, because in some cases it should
|
/// We have a special logic for local replica. It has to read less data, because in some cases it should
|
||||||
@ -529,7 +562,7 @@ Pipe ReadFromMergeTree::readInOrder(
|
|||||||
{
|
{
|
||||||
.all_callback = all_ranges_callback.value(),
|
.all_callback = all_ranges_callback.value(),
|
||||||
.callback = read_task_callback.value(),
|
.callback = read_task_callback.value(),
|
||||||
.number_of_current_replica = client_info.number_of_current_replica,
|
.number_of_current_replica = number_of_current_replica.value_or(client_info.number_of_current_replica),
|
||||||
};
|
};
|
||||||
|
|
||||||
auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier;
|
auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier;
|
||||||
@ -584,11 +617,12 @@ Pipe ReadFromMergeTree::readInOrder(
|
|||||||
context);
|
context);
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Actually it means that parallel reading from replicas enabled
|
/// If parallel replicas enabled, set total rows in progress here only on initiator with local plan
|
||||||
/// and we have to collaborate with initiator.
|
/// Otherwise rows will counted multiple times
|
||||||
/// In this case we won't set approximate rows, because it will be accounted multiple times.
|
const UInt64 in_order_limit = query_info.input_order_info ? query_info.input_order_info->limit : 0;
|
||||||
const auto in_order_limit = query_info.input_order_info ? query_info.input_order_info->limit : 0;
|
const bool parallel_replicas_local_plan_for_initiator = is_parallel_reading_from_replicas
|
||||||
const bool set_total_rows_approx = !is_parallel_reading_from_replicas;
|
&& context->getSettingsRef().parallel_replicas_local_plan && context->canUseParallelReplicasOnInitiator();
|
||||||
|
const bool set_total_rows_approx = !is_parallel_reading_from_replicas || parallel_replicas_local_plan_for_initiator;
|
||||||
|
|
||||||
Pipes pipes;
|
Pipes pipes;
|
||||||
for (size_t i = 0; i < parts_with_ranges.size(); ++i)
|
for (size_t i = 0; i < parts_with_ranges.size(); ++i)
|
||||||
@ -1422,11 +1456,8 @@ static void buildIndexes(
|
|||||||
|
|
||||||
const auto & settings = context->getSettingsRef();
|
const auto & settings = context->getSettingsRef();
|
||||||
|
|
||||||
indexes.emplace(ReadFromMergeTree::Indexes{{
|
indexes.emplace(
|
||||||
filter_actions_dag,
|
ReadFromMergeTree::Indexes{KeyCondition{filter_actions_dag, context, primary_key_column_names, primary_key.expression}});
|
||||||
context,
|
|
||||||
primary_key_column_names,
|
|
||||||
primary_key.expression}, {}, {}, {}, {}, false, {}});
|
|
||||||
|
|
||||||
if (metadata_snapshot->hasPartitionKey())
|
if (metadata_snapshot->hasPartitionKey())
|
||||||
{
|
{
|
||||||
@ -1978,6 +2009,33 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
|
|||||||
{
|
{
|
||||||
auto result = getAnalysisResult();
|
auto result = getAnalysisResult();
|
||||||
|
|
||||||
|
if (is_parallel_reading_from_replicas && context->canUseParallelReplicasOnInitiator()
|
||||||
|
&& context->getSettingsRef().parallel_replicas_local_plan)
|
||||||
|
{
|
||||||
|
CoordinationMode mode = CoordinationMode::Default;
|
||||||
|
switch (result.read_type)
|
||||||
|
{
|
||||||
|
case ReadFromMergeTree::ReadType::Default:
|
||||||
|
mode = CoordinationMode::Default;
|
||||||
|
break;
|
||||||
|
case ReadFromMergeTree::ReadType::InOrder:
|
||||||
|
mode = CoordinationMode::WithOrder;
|
||||||
|
break;
|
||||||
|
case ReadFromMergeTree::ReadType::InReverseOrder:
|
||||||
|
mode = CoordinationMode::ReverseOrder;
|
||||||
|
break;
|
||||||
|
case ReadFromMergeTree::ReadType::ParallelReplicas:
|
||||||
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Read type can't be ParallelReplicas on initiator");
|
||||||
|
}
|
||||||
|
|
||||||
|
chassert(number_of_current_replica.has_value());
|
||||||
|
chassert(all_ranges_callback.has_value());
|
||||||
|
|
||||||
|
/// initialize working set from local replica
|
||||||
|
all_ranges_callback.value()(
|
||||||
|
InitialAllRangesAnnouncement(mode, result.parts_with_ranges.getDescriptions(), number_of_current_replica.value()));
|
||||||
|
}
|
||||||
|
|
||||||
if (enable_remove_parts_from_snapshot_optimization)
|
if (enable_remove_parts_from_snapshot_optimization)
|
||||||
{
|
{
|
||||||
/// Do not keep data parts in snapshot.
|
/// Do not keep data parts in snapshot.
|
||||||
|
@ -121,7 +121,16 @@ public:
|
|||||||
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read_,
|
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read_,
|
||||||
LoggerPtr log_,
|
LoggerPtr log_,
|
||||||
AnalysisResultPtr analyzed_result_ptr_,
|
AnalysisResultPtr analyzed_result_ptr_,
|
||||||
bool enable_parallel_reading);
|
bool enable_parallel_reading_,
|
||||||
|
std::optional<MergeTreeAllRangesCallback> all_ranges_callback_ = std::nullopt,
|
||||||
|
std::optional<MergeTreeReadTaskCallback> read_task_callback_ = std::nullopt,
|
||||||
|
std::optional<size_t> number_of_current_replica_ = std::nullopt);
|
||||||
|
|
||||||
|
std::unique_ptr<ReadFromMergeTree> createLocalParallelReplicasReadingStep(
|
||||||
|
AnalysisResultPtr analyzed_result_ptr_,
|
||||||
|
MergeTreeAllRangesCallback all_ranges_callback_,
|
||||||
|
MergeTreeReadTaskCallback read_task_callback_,
|
||||||
|
size_t replica_number);
|
||||||
|
|
||||||
static constexpr auto name = "ReadFromMergeTree";
|
static constexpr auto name = "ReadFromMergeTree";
|
||||||
String getName() const override { return name; }
|
String getName() const override { return name; }
|
||||||
@ -143,6 +152,11 @@ public:
|
|||||||
|
|
||||||
struct Indexes
|
struct Indexes
|
||||||
{
|
{
|
||||||
|
explicit Indexes(KeyCondition key_condition_)
|
||||||
|
: key_condition(std::move(key_condition_))
|
||||||
|
, use_skip_indexes(false)
|
||||||
|
{}
|
||||||
|
|
||||||
KeyCondition key_condition;
|
KeyCondition key_condition;
|
||||||
std::optional<PartitionPruner> partition_pruner;
|
std::optional<PartitionPruner> partition_pruner;
|
||||||
std::optional<KeyCondition> minmax_idx_condition;
|
std::optional<KeyCondition> minmax_idx_condition;
|
||||||
@ -268,6 +282,7 @@ private:
|
|||||||
std::optional<MergeTreeReadTaskCallback> read_task_callback;
|
std::optional<MergeTreeReadTaskCallback> read_task_callback;
|
||||||
bool enable_vertical_final = false;
|
bool enable_vertical_final = false;
|
||||||
bool enable_remove_parts_from_snapshot_optimization = true;
|
bool enable_remove_parts_from_snapshot_optimization = true;
|
||||||
|
std::optional<size_t> number_of_current_replica;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -21,8 +21,8 @@
|
|||||||
#include <Client/ConnectionPoolWithFailover.h>
|
#include <Client/ConnectionPoolWithFailover.h>
|
||||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
|
|
||||||
#include <boost/algorithm/string/join.hpp>
|
#include <fmt/format.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -362,6 +362,7 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
|
|||||||
ASTPtr query_ast_,
|
ASTPtr query_ast_,
|
||||||
ClusterPtr cluster_,
|
ClusterPtr cluster_,
|
||||||
const StorageID & storage_id_,
|
const StorageID & storage_id_,
|
||||||
|
ParallelReplicasReadingCoordinatorPtr coordinator_,
|
||||||
Block header_,
|
Block header_,
|
||||||
QueryProcessingStage::Enum stage_,
|
QueryProcessingStage::Enum stage_,
|
||||||
ContextMutablePtr context_,
|
ContextMutablePtr context_,
|
||||||
@ -369,11 +370,14 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
|
|||||||
Scalars scalars_,
|
Scalars scalars_,
|
||||||
Tables external_tables_,
|
Tables external_tables_,
|
||||||
LoggerPtr log_,
|
LoggerPtr log_,
|
||||||
std::shared_ptr<const StorageLimitsList> storage_limits_)
|
std::shared_ptr<const StorageLimitsList> storage_limits_,
|
||||||
|
std::vector<ConnectionPoolPtr> pools_to_use_,
|
||||||
|
std::optional<size_t> exclude_pool_index_)
|
||||||
: ISourceStep(DataStream{.header = std::move(header_)})
|
: ISourceStep(DataStream{.header = std::move(header_)})
|
||||||
, cluster(cluster_)
|
, cluster(cluster_)
|
||||||
, query_ast(query_ast_)
|
, query_ast(query_ast_)
|
||||||
, storage_id(storage_id_)
|
, storage_id(storage_id_)
|
||||||
|
, coordinator(std::move(coordinator_))
|
||||||
, stage(std::move(stage_))
|
, stage(std::move(stage_))
|
||||||
, context(context_)
|
, context(context_)
|
||||||
, throttler(throttler_)
|
, throttler(throttler_)
|
||||||
@ -381,16 +385,24 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
|
|||||||
, external_tables{external_tables_}
|
, external_tables{external_tables_}
|
||||||
, storage_limits(std::move(storage_limits_))
|
, storage_limits(std::move(storage_limits_))
|
||||||
, log(log_)
|
, log(log_)
|
||||||
|
, pools_to_use(std::move(pools_to_use_))
|
||||||
|
, exclude_pool_index(exclude_pool_index_)
|
||||||
{
|
{
|
||||||
chassert(cluster->getShardCount() == 1);
|
chassert(cluster->getShardCount() == 1);
|
||||||
|
|
||||||
std::vector<String> description;
|
std::vector<String> replicas;
|
||||||
description.push_back(fmt::format("query: {}", formattedAST(query_ast)));
|
replicas.reserve(pools_to_use.size());
|
||||||
|
|
||||||
for (const auto & pool : cluster->getShardsInfo().front().per_replica_pools)
|
for (size_t i = 0, l = pools_to_use.size(); i < l; ++i)
|
||||||
description.push_back(fmt::format("Replica: {}", pool->getHost()));
|
{
|
||||||
|
if (exclude_pool_index.has_value() && i == exclude_pool_index)
|
||||||
|
continue;
|
||||||
|
|
||||||
setStepDescription(boost::algorithm::join(description, ", "));
|
replicas.push_back(pools_to_use[i]->getAddress());
|
||||||
|
}
|
||||||
|
|
||||||
|
auto description = fmt::format("Query: {} Replicas: {}", formattedAST(query_ast), fmt::join(replicas, ", "));
|
||||||
|
setStepDescription(std::move(description));
|
||||||
}
|
}
|
||||||
|
|
||||||
void ReadFromParallelRemoteReplicasStep::enforceSorting(SortDescription output_sort_description)
|
void ReadFromParallelRemoteReplicasStep::enforceSorting(SortDescription output_sort_description)
|
||||||
@ -406,48 +418,29 @@ void ReadFromParallelRemoteReplicasStep::enforceAggregationInOrder()
|
|||||||
void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||||
{
|
{
|
||||||
Pipes pipes;
|
Pipes pipes;
|
||||||
const Settings & current_settings = context->getSettingsRef();
|
|
||||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
|
|
||||||
|
|
||||||
const auto & shard = cluster->getShardsInfo().at(0);
|
std::vector<std::string_view> addresses;
|
||||||
size_t max_replicas_to_use = current_settings.max_parallel_replicas;
|
addresses.reserve(pools_to_use.size());
|
||||||
if (max_replicas_to_use > shard.getAllNodeCount())
|
for (size_t i = 0, l = pools_to_use.size(); i < l; ++i)
|
||||||
{
|
{
|
||||||
LOG_INFO(
|
if (exclude_pool_index.has_value() && i == exclude_pool_index)
|
||||||
getLogger("ReadFromParallelRemoteReplicasStep"),
|
continue;
|
||||||
"The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). "
|
|
||||||
"Will use the latter number to execute the query.",
|
addresses.emplace_back(pools_to_use[i]->getAddress());
|
||||||
current_settings.max_parallel_replicas,
|
|
||||||
shard.getAllNodeCount());
|
|
||||||
max_replicas_to_use = shard.getAllNodeCount();
|
|
||||||
}
|
}
|
||||||
|
LOG_DEBUG(getLogger("ReadFromParallelRemoteReplicasStep"), "Addresses to use: {}", fmt::join(addresses, ", "));
|
||||||
|
|
||||||
std::vector<ConnectionPoolWithFailover::Base::ShuffledPool> shuffled_pool;
|
for (size_t i = 0, l = pools_to_use.size(); i < l; ++i)
|
||||||
if (max_replicas_to_use < shard.getAllNodeCount())
|
|
||||||
{
|
{
|
||||||
shuffled_pool = shard.pool->getShuffledPools(current_settings);
|
if (exclude_pool_index.has_value() && i == exclude_pool_index)
|
||||||
shuffled_pool.resize(max_replicas_to_use);
|
continue;
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
/// try to preserve replicas order if all replicas in cluster are used for query execution
|
|
||||||
/// it's important for data locality during query execution
|
|
||||||
auto priority_func = [](size_t i) { return Priority{static_cast<Int64>(i)}; };
|
|
||||||
shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func);
|
|
||||||
}
|
|
||||||
|
|
||||||
coordinator
|
IConnections::ReplicaInfo replica_info{
|
||||||
= std::make_shared<ParallelReplicasReadingCoordinator>(max_replicas_to_use, current_settings.parallel_replicas_mark_segment_size);
|
|
||||||
|
|
||||||
for (size_t i=0; i < max_replicas_to_use; ++i)
|
|
||||||
{
|
|
||||||
IConnections::ReplicaInfo replica_info
|
|
||||||
{
|
|
||||||
/// we should use this number specifically because efficiency of data distribution by consistent hash depends on it.
|
/// we should use this number specifically because efficiency of data distribution by consistent hash depends on it.
|
||||||
.number_of_current_replica = i,
|
.number_of_current_replica = i,
|
||||||
};
|
};
|
||||||
|
|
||||||
addPipeForSingeReplica(pipes, shuffled_pool[i].pool, replica_info);
|
addPipeForSingeReplica(pipes, pools_to_use[i], replica_info);
|
||||||
}
|
}
|
||||||
|
|
||||||
auto pipe = Pipe::unitePipes(std::move(pipes));
|
auto pipe = Pipe::unitePipes(std::move(pipes));
|
||||||
|
@ -70,6 +70,7 @@ public:
|
|||||||
ASTPtr query_ast_,
|
ASTPtr query_ast_,
|
||||||
ClusterPtr cluster_,
|
ClusterPtr cluster_,
|
||||||
const StorageID & storage_id_,
|
const StorageID & storage_id_,
|
||||||
|
ParallelReplicasReadingCoordinatorPtr coordinator_,
|
||||||
Block header_,
|
Block header_,
|
||||||
QueryProcessingStage::Enum stage_,
|
QueryProcessingStage::Enum stage_,
|
||||||
ContextMutablePtr context_,
|
ContextMutablePtr context_,
|
||||||
@ -77,7 +78,9 @@ public:
|
|||||||
Scalars scalars_,
|
Scalars scalars_,
|
||||||
Tables external_tables_,
|
Tables external_tables_,
|
||||||
LoggerPtr log_,
|
LoggerPtr log_,
|
||||||
std::shared_ptr<const StorageLimitsList> storage_limits_);
|
std::shared_ptr<const StorageLimitsList> storage_limits_,
|
||||||
|
std::vector<ConnectionPoolPtr> pools_to_use,
|
||||||
|
std::optional<size_t> exclude_pool_index_ = std::nullopt);
|
||||||
|
|
||||||
String getName() const override { return "ReadFromRemoteParallelReplicas"; }
|
String getName() const override { return "ReadFromRemoteParallelReplicas"; }
|
||||||
|
|
||||||
@ -100,6 +103,8 @@ private:
|
|||||||
Tables external_tables;
|
Tables external_tables;
|
||||||
std::shared_ptr<const StorageLimitsList> storage_limits;
|
std::shared_ptr<const StorageLimitsList> storage_limits;
|
||||||
LoggerPtr log;
|
LoggerPtr log;
|
||||||
|
std::vector<ConnectionPoolPtr> pools_to_use;
|
||||||
|
std::optional<size_t> exclude_pool_index;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
31
src/Processors/QueryPlan/TemporaryFiles.h
Normal file
31
src/Processors/QueryPlan/TemporaryFiles.h
Normal file
@ -0,0 +1,31 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <base/types.h>
|
||||||
|
#include <boost/noncopyable.hpp>
|
||||||
|
#include <vector>
|
||||||
|
#include <memory>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class WriteBuffer;
|
||||||
|
class ReadBuffer;
|
||||||
|
|
||||||
|
/// Interface for accessing temporary files by some logical name (or id).
|
||||||
|
/// While building query pipeline processors can lookup temporary files by some id and use them for writing and/or reading temporary data
|
||||||
|
/// without knowing what exactly is behind the name: local file, memory buffer, object in cloud storage, etc.
|
||||||
|
struct ITemporaryFileLookup : boost::noncopyable
|
||||||
|
{
|
||||||
|
virtual ~ITemporaryFileLookup() = default;
|
||||||
|
|
||||||
|
/// Give the caller a temporary write buffer, but don't give away the ownership.
|
||||||
|
virtual WriteBuffer & getTemporaryFileForWriting(const String & file_id) = 0;
|
||||||
|
|
||||||
|
/// Give the caller a temporary read buffer, it exclusively belongs to the caller.
|
||||||
|
/// Other callers can get their own read buffer for the same temporary file.
|
||||||
|
virtual std::unique_ptr<ReadBuffer> getTemporaryFileForReading(const String & file_id) = 0;
|
||||||
|
};
|
||||||
|
|
||||||
|
using TemporaryFileLookupPtr = std::shared_ptr<ITemporaryFileLookup>;
|
||||||
|
|
||||||
|
}
|
@ -183,13 +183,14 @@ void ColumnGathererStream::consume(Input & input, size_t source_num)
|
|||||||
ColumnGathererTransform::ColumnGathererTransform(
|
ColumnGathererTransform::ColumnGathererTransform(
|
||||||
const Block & header,
|
const Block & header,
|
||||||
size_t num_inputs,
|
size_t num_inputs,
|
||||||
ReadBuffer & row_sources_buf_,
|
std::unique_ptr<ReadBuffer> row_sources_buf_,
|
||||||
size_t block_preferred_size_rows_,
|
size_t block_preferred_size_rows_,
|
||||||
size_t block_preferred_size_bytes_,
|
size_t block_preferred_size_bytes_,
|
||||||
bool is_result_sparse_)
|
bool is_result_sparse_)
|
||||||
: IMergingTransform<ColumnGathererStream>(
|
: IMergingTransform<ColumnGathererStream>(
|
||||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false,
|
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false,
|
||||||
num_inputs, row_sources_buf_, block_preferred_size_rows_, block_preferred_size_bytes_, is_result_sparse_)
|
num_inputs, *row_sources_buf_, block_preferred_size_rows_, block_preferred_size_bytes_, is_result_sparse_)
|
||||||
|
, row_sources_buf_holder(std::move(row_sources_buf_))
|
||||||
, log(getLogger("ColumnGathererStream"))
|
, log(getLogger("ColumnGathererStream"))
|
||||||
{
|
{
|
||||||
if (header.columns() != 1)
|
if (header.columns() != 1)
|
||||||
|
@ -115,7 +115,7 @@ public:
|
|||||||
ColumnGathererTransform(
|
ColumnGathererTransform(
|
||||||
const Block & header,
|
const Block & header,
|
||||||
size_t num_inputs,
|
size_t num_inputs,
|
||||||
ReadBuffer & row_sources_buf_,
|
std::unique_ptr<ReadBuffer> row_sources_buf_,
|
||||||
size_t block_preferred_size_rows_,
|
size_t block_preferred_size_rows_,
|
||||||
size_t block_preferred_size_bytes_,
|
size_t block_preferred_size_bytes_,
|
||||||
bool is_result_sparse_);
|
bool is_result_sparse_);
|
||||||
@ -124,6 +124,8 @@ public:
|
|||||||
|
|
||||||
protected:
|
protected:
|
||||||
void onFinish() override;
|
void onFinish() override;
|
||||||
|
|
||||||
|
std::unique_ptr<ReadBuffer> row_sources_buf_holder; /// Keep ownership of row_sources_buf while it's in use by ColumnGathererStream.
|
||||||
LoggerPtr log;
|
LoggerPtr log;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -299,13 +299,17 @@ IProcessor::Status FillingRightJoinSideTransform::prepare()
|
|||||||
|
|
||||||
void FillingRightJoinSideTransform::work()
|
void FillingRightJoinSideTransform::work()
|
||||||
{
|
{
|
||||||
auto block = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns());
|
auto & input = inputs.front();
|
||||||
|
auto block = input.getHeader().cloneWithColumns(chunk.detachColumns());
|
||||||
|
|
||||||
if (for_totals)
|
if (for_totals)
|
||||||
join->setTotals(block);
|
join->setTotals(block);
|
||||||
else
|
else
|
||||||
stop_reading = !join->addBlockToJoin(block);
|
stop_reading = !join->addBlockToJoin(block);
|
||||||
|
|
||||||
|
if (input.isFinished())
|
||||||
|
join->tryRerangeRightTableData();
|
||||||
|
|
||||||
set_totals = for_totals;
|
set_totals = for_totals;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2133,7 +2133,7 @@ bool TCPHandler::receiveUnexpectedData(bool throw_exception)
|
|||||||
|
|
||||||
std::shared_ptr<ReadBuffer> maybe_compressed_in;
|
std::shared_ptr<ReadBuffer> maybe_compressed_in;
|
||||||
if (last_block_in.compression == Protocol::Compression::Enable)
|
if (last_block_in.compression == Protocol::Compression::Enable)
|
||||||
maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in, /* allow_different_codecs */ true);
|
maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in, /* allow_different_codecs */ true, /* external_data */ query_kind != ClientInfo::QueryKind::SECONDARY_QUERY);
|
||||||
else
|
else
|
||||||
maybe_compressed_in = in;
|
maybe_compressed_in = in;
|
||||||
|
|
||||||
@ -2157,7 +2157,7 @@ void TCPHandler::initBlockInput()
|
|||||||
/// with another codec that the rest of the data. Example: data sent by Distributed tables.
|
/// with another codec that the rest of the data. Example: data sent by Distributed tables.
|
||||||
|
|
||||||
if (state.compression == Protocol::Compression::Enable)
|
if (state.compression == Protocol::Compression::Enable)
|
||||||
state.maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in, /* allow_different_codecs */ true);
|
state.maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in, /* allow_different_codecs */ true, /* external_data */ query_kind != ClientInfo::QueryKind::SECONDARY_QUERY);
|
||||||
else
|
else
|
||||||
state.maybe_compressed_in = in;
|
state.maybe_compressed_in = in;
|
||||||
|
|
||||||
|
@ -14,7 +14,7 @@
|
|||||||
#include <Compression/CompressedWriteBuffer.h>
|
#include <Compression/CompressedWriteBuffer.h>
|
||||||
#include <DataTypes/ObjectUtils.h>
|
#include <DataTypes/ObjectUtils.h>
|
||||||
#include <DataTypes/Serializations/SerializationInfo.h>
|
#include <DataTypes/Serializations/SerializationInfo.h>
|
||||||
#include <IO/IReadableWriteBuffer.h>
|
#include <IO/ReadBufferFromEmptyFile.h>
|
||||||
#include <Storages/MergeTree/MergeTreeData.h>
|
#include <Storages/MergeTree/MergeTreeData.h>
|
||||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||||
#include <Storages/MergeTree/MergeTreeSequentialSource.h>
|
#include <Storages/MergeTree/MergeTreeSequentialSource.h>
|
||||||
@ -38,6 +38,12 @@
|
|||||||
#include <Processors/Transforms/DistinctSortedTransform.h>
|
#include <Processors/Transforms/DistinctSortedTransform.h>
|
||||||
#include <Processors/Transforms/DistinctTransform.h>
|
#include <Processors/Transforms/DistinctTransform.h>
|
||||||
#include <Processors/QueryPlan/CreatingSetsStep.h>
|
#include <Processors/QueryPlan/CreatingSetsStep.h>
|
||||||
|
#include <Processors/QueryPlan/DistinctStep.h>
|
||||||
|
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||||
|
#include <Processors/QueryPlan/QueryPlan.h>
|
||||||
|
#include <Processors/QueryPlan/UnionStep.h>
|
||||||
|
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||||
|
#include <Processors/QueryPlan/TemporaryFiles.h>
|
||||||
#include <Interpreters/PreparedSets.h>
|
#include <Interpreters/PreparedSets.h>
|
||||||
#include <Interpreters/MergeTreeTransaction.h>
|
#include <Interpreters/MergeTreeTransaction.h>
|
||||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||||
@ -85,6 +91,68 @@ static ColumnsStatistics getStatisticsForColumns(
|
|||||||
return all_statistics;
|
return all_statistics;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Manages the "rows_sources" temporary file that is used during vertical merge.
|
||||||
|
class RowsSourcesTemporaryFile : public ITemporaryFileLookup
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
/// A logical name of the temporary file under which it will be known to the plan steps that use it.
|
||||||
|
static constexpr auto FILE_ID = "rows_sources";
|
||||||
|
|
||||||
|
explicit RowsSourcesTemporaryFile(TemporaryDataOnDiskScopePtr temporary_data_on_disk_)
|
||||||
|
: tmp_disk(std::make_unique<TemporaryDataOnDisk>(temporary_data_on_disk_))
|
||||||
|
, uncompressed_write_buffer(tmp_disk->createRawStream())
|
||||||
|
, tmp_file_name_on_disk(uncompressed_write_buffer->getFileName())
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
WriteBuffer & getTemporaryFileForWriting(const String & name) override
|
||||||
|
{
|
||||||
|
if (name != FILE_ID)
|
||||||
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected temporary file name requested: {}", name);
|
||||||
|
|
||||||
|
if (write_buffer)
|
||||||
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file was already requested for writing, there musto be only one writer");
|
||||||
|
|
||||||
|
write_buffer = (std::make_unique<CompressedWriteBuffer>(*uncompressed_write_buffer));
|
||||||
|
return *write_buffer;
|
||||||
|
}
|
||||||
|
|
||||||
|
std::unique_ptr<ReadBuffer> getTemporaryFileForReading(const String & name) override
|
||||||
|
{
|
||||||
|
if (name != FILE_ID)
|
||||||
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected temporary file name requested: {}", name);
|
||||||
|
|
||||||
|
if (!finalized)
|
||||||
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file is not finalized yet");
|
||||||
|
|
||||||
|
/// tmp_disk might not create real file if no data was written to it.
|
||||||
|
if (final_size == 0)
|
||||||
|
return std::make_unique<ReadBufferFromEmptyFile>();
|
||||||
|
|
||||||
|
/// Reopen the file for each read so that multiple reads can be performed in parallel and there is no need to seek to the beginning.
|
||||||
|
auto raw_file_read_buffer = std::make_unique<ReadBufferFromFile>(tmp_file_name_on_disk);
|
||||||
|
return std::make_unique<CompressedReadBufferFromFile>(std::move(raw_file_read_buffer));
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Returns written data size in bytes
|
||||||
|
size_t finalizeWriting()
|
||||||
|
{
|
||||||
|
write_buffer->finalize();
|
||||||
|
uncompressed_write_buffer->finalize();
|
||||||
|
finalized = true;
|
||||||
|
final_size = write_buffer->count();
|
||||||
|
return final_size;
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
std::unique_ptr<TemporaryDataOnDisk> tmp_disk;
|
||||||
|
std::unique_ptr<WriteBufferFromFileBase> uncompressed_write_buffer;
|
||||||
|
std::unique_ptr<WriteBuffer> write_buffer;
|
||||||
|
const String tmp_file_name_on_disk;
|
||||||
|
bool finalized = false;
|
||||||
|
size_t final_size = 0;
|
||||||
|
};
|
||||||
|
|
||||||
static void addMissedColumnsToSerializationInfos(
|
static void addMissedColumnsToSerializationInfos(
|
||||||
size_t num_rows_in_parts,
|
size_t num_rows_in_parts,
|
||||||
const Names & part_columns,
|
const Names & part_columns,
|
||||||
@ -120,19 +188,19 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu
|
|||||||
std::set<String> key_columns(sort_key_columns_vec.cbegin(), sort_key_columns_vec.cend());
|
std::set<String> key_columns(sort_key_columns_vec.cbegin(), sort_key_columns_vec.cend());
|
||||||
|
|
||||||
/// Force sign column for Collapsing mode
|
/// Force sign column for Collapsing mode
|
||||||
if (ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing)
|
if (global_ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing)
|
||||||
key_columns.emplace(ctx->merging_params.sign_column);
|
key_columns.emplace(global_ctx->merging_params.sign_column);
|
||||||
|
|
||||||
/// Force version column for Replacing mode
|
/// Force version column for Replacing mode
|
||||||
if (ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing)
|
if (global_ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing)
|
||||||
{
|
{
|
||||||
key_columns.emplace(ctx->merging_params.is_deleted_column);
|
key_columns.emplace(global_ctx->merging_params.is_deleted_column);
|
||||||
key_columns.emplace(ctx->merging_params.version_column);
|
key_columns.emplace(global_ctx->merging_params.version_column);
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Force sign column for VersionedCollapsing mode. Version is already in primary key.
|
/// Force sign column for VersionedCollapsing mode. Version is already in primary key.
|
||||||
if (ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing)
|
if (global_ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing)
|
||||||
key_columns.emplace(ctx->merging_params.sign_column);
|
key_columns.emplace(global_ctx->merging_params.sign_column);
|
||||||
|
|
||||||
/// Force to merge at least one column in case of empty key
|
/// Force to merge at least one column in case of empty key
|
||||||
if (key_columns.empty())
|
if (key_columns.empty())
|
||||||
@ -190,7 +258,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
|
bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const
|
||||||
{
|
{
|
||||||
ProfileEvents::increment(ProfileEvents::Merge);
|
ProfileEvents::increment(ProfileEvents::Merge);
|
||||||
|
|
||||||
@ -201,7 +269,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
|
|||||||
// E.g. `proj_a.proj` for a normal projection merge and `proj_a.tmp_proj` for a projection materialization merge.
|
// E.g. `proj_a.proj` for a normal projection merge and `proj_a.tmp_proj` for a projection materialization merge.
|
||||||
local_tmp_prefix = global_ctx->parent_part ? "" : "tmp_merge_";
|
local_tmp_prefix = global_ctx->parent_part ? "" : "tmp_merge_";
|
||||||
}
|
}
|
||||||
const String local_tmp_suffix = global_ctx->parent_part ? ctx->suffix : "";
|
|
||||||
|
const String local_tmp_suffix = global_ctx->parent_part ? global_ctx->suffix : "";
|
||||||
|
|
||||||
if (global_ctx->merges_blocker->isCancelled() || global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed))
|
if (global_ctx->merges_blocker->isCancelled() || global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed))
|
||||||
throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts");
|
throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts");
|
||||||
@ -226,7 +295,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
|
|||||||
LOG_DEBUG(ctx->log, "DEDUPLICATE BY ('{}')", fmt::join(global_ctx->deduplicate_by_columns, "', '"));
|
LOG_DEBUG(ctx->log, "DEDUPLICATE BY ('{}')", fmt::join(global_ctx->deduplicate_by_columns, "', '"));
|
||||||
}
|
}
|
||||||
|
|
||||||
ctx->disk = global_ctx->space_reservation->getDisk();
|
global_ctx->disk = global_ctx->space_reservation->getDisk();
|
||||||
auto local_tmp_part_basename = local_tmp_prefix + global_ctx->future_part->name + local_tmp_suffix;
|
auto local_tmp_part_basename = local_tmp_prefix + global_ctx->future_part->name + local_tmp_suffix;
|
||||||
|
|
||||||
std::optional<MergeTreeDataPartBuilder> builder;
|
std::optional<MergeTreeDataPartBuilder> builder;
|
||||||
@ -238,7 +307,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
auto local_single_disk_volume = std::make_shared<SingleDiskVolume>("volume_" + global_ctx->future_part->name, ctx->disk, 0);
|
auto local_single_disk_volume = std::make_shared<SingleDiskVolume>("volume_" + global_ctx->future_part->name, global_ctx->disk, 0);
|
||||||
builder.emplace(global_ctx->data->getDataPartBuilder(global_ctx->future_part->name, local_single_disk_volume, local_tmp_part_basename));
|
builder.emplace(global_ctx->data->getDataPartBuilder(global_ctx->future_part->name, local_single_disk_volume, local_tmp_part_basename));
|
||||||
builder->withPartStorageType(global_ctx->future_part->part_format.storage_type);
|
builder->withPartStorageType(global_ctx->future_part->part_format.storage_type);
|
||||||
}
|
}
|
||||||
@ -359,8 +428,6 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
|
|||||||
ctx->compression_codec = global_ctx->data->getCompressionCodecForPart(
|
ctx->compression_codec = global_ctx->data->getCompressionCodecForPart(
|
||||||
global_ctx->merge_list_element_ptr->total_size_bytes_compressed, global_ctx->new_data_part->ttl_infos, global_ctx->time_of_merge);
|
global_ctx->merge_list_element_ptr->total_size_bytes_compressed, global_ctx->new_data_part->ttl_infos, global_ctx->time_of_merge);
|
||||||
|
|
||||||
ctx->tmp_disk = std::make_unique<TemporaryDataOnDisk>(global_ctx->context->getTempDataOnDisk());
|
|
||||||
|
|
||||||
switch (global_ctx->chosen_merge_algorithm)
|
switch (global_ctx->chosen_merge_algorithm)
|
||||||
{
|
{
|
||||||
case MergeAlgorithm::Horizontal:
|
case MergeAlgorithm::Horizontal:
|
||||||
@ -373,8 +440,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
|
|||||||
}
|
}
|
||||||
case MergeAlgorithm::Vertical:
|
case MergeAlgorithm::Vertical:
|
||||||
{
|
{
|
||||||
ctx->rows_sources_uncompressed_write_buf = ctx->tmp_disk->createRawStream();
|
ctx->rows_sources_temporary_file = std::make_shared<RowsSourcesTemporaryFile>(global_ctx->context->getTempDataOnDisk());
|
||||||
ctx->rows_sources_write_buf = std::make_unique<CompressedWriteBuffer>(*ctx->rows_sources_uncompressed_write_buf);
|
|
||||||
|
|
||||||
std::map<String, UInt64> local_merged_column_to_size;
|
std::map<String, UInt64> local_merged_column_to_size;
|
||||||
for (const auto & part : global_ctx->future_part->parts)
|
for (const auto & part : global_ctx->future_part->parts)
|
||||||
@ -494,11 +560,9 @@ MergeTask::StageRuntimeContextPtr MergeTask::ExecuteAndFinalizeHorizontalPart::g
|
|||||||
|
|
||||||
auto new_ctx = std::make_shared<VerticalMergeRuntimeContext>();
|
auto new_ctx = std::make_shared<VerticalMergeRuntimeContext>();
|
||||||
|
|
||||||
new_ctx->rows_sources_write_buf = std::move(ctx->rows_sources_write_buf);
|
new_ctx->rows_sources_temporary_file = std::move(ctx->rows_sources_temporary_file);
|
||||||
new_ctx->rows_sources_uncompressed_write_buf = std::move(ctx->rows_sources_uncompressed_write_buf);
|
|
||||||
new_ctx->column_sizes = std::move(ctx->column_sizes);
|
new_ctx->column_sizes = std::move(ctx->column_sizes);
|
||||||
new_ctx->compression_codec = std::move(ctx->compression_codec);
|
new_ctx->compression_codec = std::move(ctx->compression_codec);
|
||||||
new_ctx->tmp_disk = std::move(ctx->tmp_disk);
|
|
||||||
new_ctx->it_name_and_type = std::move(ctx->it_name_and_type);
|
new_ctx->it_name_and_type = std::move(ctx->it_name_and_type);
|
||||||
new_ctx->read_with_direct_io = std::move(ctx->read_with_direct_io);
|
new_ctx->read_with_direct_io = std::move(ctx->read_with_direct_io);
|
||||||
new_ctx->need_sync = std::move(ctx->need_sync);
|
new_ctx->need_sync = std::move(ctx->need_sync);
|
||||||
@ -554,9 +618,9 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRe
|
|||||||
const bool merge_may_reduce_rows =
|
const bool merge_may_reduce_rows =
|
||||||
global_ctx->cleanup ||
|
global_ctx->cleanup ||
|
||||||
global_ctx->deduplicate ||
|
global_ctx->deduplicate ||
|
||||||
ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing ||
|
global_ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing ||
|
||||||
ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing ||
|
global_ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing ||
|
||||||
ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing;
|
global_ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing;
|
||||||
|
|
||||||
const auto & projections = global_ctx->metadata_snapshot->getProjections();
|
const auto & projections = global_ctx->metadata_snapshot->getProjections();
|
||||||
|
|
||||||
@ -664,7 +728,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::constructTaskForProjectionPart
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() // NOLINT
|
bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() const
|
||||||
{
|
{
|
||||||
/// In case if there are no projections we didn't construct a task
|
/// In case if there are no projections we didn't construct a task
|
||||||
if (!ctx->merge_projection_parts_task_ptr)
|
if (!ctx->merge_projection_parts_task_ptr)
|
||||||
@ -683,7 +747,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() // N
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl()
|
bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() const
|
||||||
{
|
{
|
||||||
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
|
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
|
||||||
UInt64 step_time_ms = global_ctx->data->getSettings()->background_task_preferred_step_execution_time_ms.totalMilliseconds();
|
UInt64 step_time_ms = global_ctx->data->getSettings()->background_task_preferred_step_execution_time_ms.totalMilliseconds();
|
||||||
@ -755,11 +819,7 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const
|
|||||||
global_ctx->merge_list_element_ptr->progress.store(ctx->column_sizes->keyColumnsWeight(), std::memory_order_relaxed);
|
global_ctx->merge_list_element_ptr->progress.store(ctx->column_sizes->keyColumnsWeight(), std::memory_order_relaxed);
|
||||||
|
|
||||||
/// Ensure data has written to disk.
|
/// Ensure data has written to disk.
|
||||||
ctx->rows_sources_write_buf->finalize();
|
size_t rows_sources_count = ctx->rows_sources_temporary_file->finalizeWriting();
|
||||||
ctx->rows_sources_uncompressed_write_buf->finalize();
|
|
||||||
ctx->rows_sources_uncompressed_write_buf->finalize();
|
|
||||||
|
|
||||||
size_t rows_sources_count = ctx->rows_sources_write_buf->count();
|
|
||||||
/// In special case, when there is only one source part, and no rows were skipped, we may have
|
/// In special case, when there is only one source part, and no rows were skipped, we may have
|
||||||
/// skipped writing rows_sources file. Otherwise rows_sources_count must be equal to the total
|
/// skipped writing rows_sources file. Otherwise rows_sources_count must be equal to the total
|
||||||
/// number of input rows.
|
/// number of input rows.
|
||||||
@ -770,29 +830,6 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const
|
|||||||
"of bytes written to rows_sources file ({}). It is a bug.",
|
"of bytes written to rows_sources file ({}). It is a bug.",
|
||||||
sum_input_rows_exact, input_rows_filtered, rows_sources_count);
|
sum_input_rows_exact, input_rows_filtered, rows_sources_count);
|
||||||
|
|
||||||
/// TemporaryDataOnDisk::createRawStream returns WriteBufferFromFile implementing IReadableWriteBuffer
|
|
||||||
/// and we expect to get ReadBufferFromFile here.
|
|
||||||
/// So, it's relatively safe to use dynamic_cast here and downcast to ReadBufferFromFile.
|
|
||||||
auto * wbuf_readable = dynamic_cast<IReadableWriteBuffer *>(ctx->rows_sources_uncompressed_write_buf.get());
|
|
||||||
std::unique_ptr<ReadBuffer> reread_buf = wbuf_readable ? wbuf_readable->tryGetReadBuffer() : nullptr;
|
|
||||||
if (!reread_buf)
|
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot read temporary file {}", ctx->rows_sources_uncompressed_write_buf->getFileName());
|
|
||||||
|
|
||||||
auto * reread_buffer_raw = dynamic_cast<ReadBufferFromFileBase *>(reread_buf.get());
|
|
||||||
if (!reread_buffer_raw)
|
|
||||||
{
|
|
||||||
const auto & reread_buf_ref = *reread_buf;
|
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected ReadBufferFromFileBase, but got {}", demangle(typeid(reread_buf_ref).name()));
|
|
||||||
}
|
|
||||||
/// Move ownership from std::unique_ptr<ReadBuffer> to std::unique_ptr<ReadBufferFromFile> for CompressedReadBufferFromFile.
|
|
||||||
/// First, release ownership from unique_ptr to base type.
|
|
||||||
reread_buf.release(); /// NOLINT(bugprone-unused-return-value,hicpp-ignored-remove-result): we already have the pointer value in `reread_buffer_raw`
|
|
||||||
|
|
||||||
/// Then, move ownership to unique_ptr to concrete type.
|
|
||||||
std::unique_ptr<ReadBufferFromFileBase> reread_buffer_from_file(reread_buffer_raw);
|
|
||||||
|
|
||||||
/// CompressedReadBufferFromFile expects std::unique_ptr<ReadBufferFromFile> as argument.
|
|
||||||
ctx->rows_sources_read_buf = std::make_unique<CompressedReadBufferFromFile>(std::move(reread_buffer_from_file));
|
|
||||||
ctx->it_name_and_type = global_ctx->gathering_columns.cbegin();
|
ctx->it_name_and_type = global_ctx->gathering_columns.cbegin();
|
||||||
|
|
||||||
const auto & settings = global_ctx->context->getSettingsRef();
|
const auto & settings = global_ctx->context->getSettingsRef();
|
||||||
@ -812,33 +849,158 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const
|
|||||||
ctx->use_prefetch = all_parts_on_remote_disks && global_ctx->data->getSettings()->vertical_merge_remote_filesystem_prefetch;
|
ctx->use_prefetch = all_parts_on_remote_disks && global_ctx->data->getSettings()->vertical_merge_remote_filesystem_prefetch;
|
||||||
|
|
||||||
if (ctx->use_prefetch && ctx->it_name_and_type != global_ctx->gathering_columns.end())
|
if (ctx->use_prefetch && ctx->it_name_and_type != global_ctx->gathering_columns.end())
|
||||||
ctx->prepared_pipe = createPipeForReadingOneColumn(ctx->it_name_and_type->name);
|
ctx->prepared_pipeline = createPipelineForReadingOneColumn(ctx->it_name_and_type->name);
|
||||||
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
Pipe MergeTask::VerticalMergeStage::createPipeForReadingOneColumn(const String & column_name) const
|
/// Gathers values from all parts for one column using rows sources temporary file
|
||||||
|
class ColumnGathererStep : public ITransformingStep
|
||||||
{
|
{
|
||||||
Pipes pipes;
|
public:
|
||||||
|
ColumnGathererStep(
|
||||||
|
const DataStream & input_stream_,
|
||||||
|
const String & rows_sources_temporary_file_name_,
|
||||||
|
UInt64 merge_block_size_rows_,
|
||||||
|
UInt64 merge_block_size_bytes_,
|
||||||
|
bool is_result_sparse_)
|
||||||
|
: ITransformingStep(input_stream_, input_stream_.header, getTraits())
|
||||||
|
, rows_sources_temporary_file_name(rows_sources_temporary_file_name_)
|
||||||
|
, merge_block_size_rows(merge_block_size_rows_)
|
||||||
|
, merge_block_size_bytes(merge_block_size_bytes_)
|
||||||
|
, is_result_sparse(is_result_sparse_)
|
||||||
|
{}
|
||||||
|
|
||||||
|
String getName() const override { return "ColumnGatherer"; }
|
||||||
|
|
||||||
|
void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & pipeline_settings) override
|
||||||
|
{
|
||||||
|
const auto &header = pipeline.getHeader();
|
||||||
|
const auto input_streams_count = pipeline.getNumStreams();
|
||||||
|
|
||||||
|
if (!pipeline_settings.temporary_file_lookup)
|
||||||
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file lookup is not set in pipeline settings for vertical merge");
|
||||||
|
|
||||||
|
auto rows_sources_read_buf = pipeline_settings.temporary_file_lookup->getTemporaryFileForReading(rows_sources_temporary_file_name);
|
||||||
|
|
||||||
|
auto transform = std::make_unique<ColumnGathererTransform>(
|
||||||
|
header,
|
||||||
|
input_streams_count,
|
||||||
|
std::move(rows_sources_read_buf),
|
||||||
|
merge_block_size_rows,
|
||||||
|
merge_block_size_bytes,
|
||||||
|
is_result_sparse);
|
||||||
|
|
||||||
|
pipeline.addTransform(std::move(transform));
|
||||||
|
}
|
||||||
|
|
||||||
|
void updateOutputStream() override
|
||||||
|
{
|
||||||
|
output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits());
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
static Traits getTraits()
|
||||||
|
{
|
||||||
|
return ITransformingStep::Traits
|
||||||
|
{
|
||||||
|
{
|
||||||
|
.returns_single_stream = true,
|
||||||
|
.preserves_number_of_streams = true,
|
||||||
|
.preserves_sorting = true,
|
||||||
|
},
|
||||||
|
{
|
||||||
|
.preserves_number_of_rows = false,
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
MergeTreeData::MergingParams merging_params{};
|
||||||
|
const String rows_sources_temporary_file_name;
|
||||||
|
const UInt64 merge_block_size_rows;
|
||||||
|
const UInt64 merge_block_size_bytes;
|
||||||
|
const bool is_result_sparse;
|
||||||
|
};
|
||||||
|
|
||||||
|
MergeTask::VerticalMergeRuntimeContext::PreparedColumnPipeline MergeTask::VerticalMergeStage::createPipelineForReadingOneColumn(const String & column_name) const
|
||||||
|
{
|
||||||
|
/// Read from all parts
|
||||||
|
std::vector<QueryPlanPtr> plans;
|
||||||
for (size_t part_num = 0; part_num < global_ctx->future_part->parts.size(); ++part_num)
|
for (size_t part_num = 0; part_num < global_ctx->future_part->parts.size(); ++part_num)
|
||||||
{
|
{
|
||||||
Pipe pipe = createMergeTreeSequentialSource(
|
auto plan_for_part = std::make_unique<QueryPlan>();
|
||||||
|
createReadFromPartStep(
|
||||||
MergeTreeSequentialSourceType::Merge,
|
MergeTreeSequentialSourceType::Merge,
|
||||||
|
*plan_for_part,
|
||||||
*global_ctx->data,
|
*global_ctx->data,
|
||||||
global_ctx->storage_snapshot,
|
global_ctx->storage_snapshot,
|
||||||
global_ctx->future_part->parts[part_num],
|
global_ctx->future_part->parts[part_num],
|
||||||
global_ctx->alter_conversions[part_num],
|
global_ctx->alter_conversions[part_num],
|
||||||
Names{column_name},
|
Names{column_name},
|
||||||
/*mark_ranges=*/ {},
|
|
||||||
global_ctx->input_rows_filtered,
|
global_ctx->input_rows_filtered,
|
||||||
/*apply_deleted_mask=*/ true,
|
/*apply_deleted_mask=*/ true,
|
||||||
|
std::nullopt,
|
||||||
ctx->read_with_direct_io,
|
ctx->read_with_direct_io,
|
||||||
ctx->use_prefetch);
|
ctx->use_prefetch,
|
||||||
|
global_ctx->context,
|
||||||
|
getLogger("VerticalMergeStage"));
|
||||||
|
|
||||||
pipes.emplace_back(std::move(pipe));
|
plans.emplace_back(std::move(plan_for_part));
|
||||||
}
|
}
|
||||||
|
|
||||||
return Pipe::unitePipes(std::move(pipes));
|
QueryPlan merge_column_query_plan;
|
||||||
|
|
||||||
|
/// Union of all parts streams
|
||||||
|
{
|
||||||
|
DataStreams input_streams;
|
||||||
|
input_streams.reserve(plans.size());
|
||||||
|
for (auto & plan : plans)
|
||||||
|
input_streams.emplace_back(plan->getCurrentDataStream());
|
||||||
|
|
||||||
|
auto union_step = std::make_unique<UnionStep>(std::move(input_streams));
|
||||||
|
merge_column_query_plan.unitePlans(std::move(union_step), std::move(plans));
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Add column gatherer step
|
||||||
|
{
|
||||||
|
bool is_result_sparse = global_ctx->new_data_part->getSerialization(column_name)->getKind() == ISerialization::Kind::SPARSE;
|
||||||
|
const auto data_settings = global_ctx->data->getSettings();
|
||||||
|
auto merge_step = std::make_unique<ColumnGathererStep>(
|
||||||
|
merge_column_query_plan.getCurrentDataStream(),
|
||||||
|
RowsSourcesTemporaryFile::FILE_ID,
|
||||||
|
data_settings->merge_max_block_size,
|
||||||
|
data_settings->merge_max_block_size_bytes,
|
||||||
|
is_result_sparse);
|
||||||
|
merge_step->setStepDescription("Gather column");
|
||||||
|
merge_column_query_plan.addStep(std::move(merge_step));
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Add expression step for indexes
|
||||||
|
MergeTreeIndices indexes_to_recalc;
|
||||||
|
IndicesDescription indexes_to_recalc_description;
|
||||||
|
{
|
||||||
|
auto indexes_it = global_ctx->skip_indexes_by_column.find(column_name);
|
||||||
|
|
||||||
|
if (indexes_it != global_ctx->skip_indexes_by_column.end())
|
||||||
|
{
|
||||||
|
indexes_to_recalc_description = indexes_it->second;
|
||||||
|
indexes_to_recalc = MergeTreeIndexFactory::instance().getMany(indexes_it->second);
|
||||||
|
|
||||||
|
auto indices_expression_dag = indexes_it->second.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext())->getActionsDAG().clone();
|
||||||
|
indices_expression_dag.addMaterializingOutputActions(); /// Const columns cannot be written without materialization.
|
||||||
|
auto calculate_indices_expression_step = std::make_unique<ExpressionStep>(
|
||||||
|
merge_column_query_plan.getCurrentDataStream(),
|
||||||
|
std::move(indices_expression_dag));
|
||||||
|
merge_column_query_plan.addStep(std::move(calculate_indices_expression_step));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
auto pipeline_settings = BuildQueryPipelineSettings::fromContext(global_ctx->context);
|
||||||
|
pipeline_settings.temporary_file_lookup = ctx->rows_sources_temporary_file;
|
||||||
|
auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context);
|
||||||
|
auto builder = merge_column_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings);
|
||||||
|
|
||||||
|
return {QueryPipelineBuilder::getPipeline(std::move(*builder)), std::move(indexes_to_recalc)};
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const
|
void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const
|
||||||
@ -848,50 +1010,22 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const
|
|||||||
ctx->progress_before = global_ctx->merge_list_element_ptr->progress.load(std::memory_order_relaxed);
|
ctx->progress_before = global_ctx->merge_list_element_ptr->progress.load(std::memory_order_relaxed);
|
||||||
global_ctx->column_progress = std::make_unique<MergeStageProgress>(ctx->progress_before, ctx->column_sizes->columnWeight(column_name));
|
global_ctx->column_progress = std::make_unique<MergeStageProgress>(ctx->progress_before, ctx->column_sizes->columnWeight(column_name));
|
||||||
|
|
||||||
Pipe pipe;
|
VerticalMergeRuntimeContext::PreparedColumnPipeline column_pipepline;
|
||||||
if (ctx->prepared_pipe)
|
if (ctx->prepared_pipeline)
|
||||||
{
|
{
|
||||||
pipe = std::move(*ctx->prepared_pipe);
|
column_pipepline = std::move(*ctx->prepared_pipeline);
|
||||||
|
|
||||||
|
/// Prepare next column pipeline to initiate prefetching
|
||||||
auto next_column_it = std::next(ctx->it_name_and_type);
|
auto next_column_it = std::next(ctx->it_name_and_type);
|
||||||
if (next_column_it != global_ctx->gathering_columns.end())
|
if (next_column_it != global_ctx->gathering_columns.end())
|
||||||
ctx->prepared_pipe = createPipeForReadingOneColumn(next_column_it->name);
|
ctx->prepared_pipeline = createPipelineForReadingOneColumn(next_column_it->name);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
pipe = createPipeForReadingOneColumn(column_name);
|
column_pipepline = createPipelineForReadingOneColumn(column_name);
|
||||||
}
|
}
|
||||||
|
|
||||||
ctx->rows_sources_read_buf->seek(0, 0);
|
ctx->column_parts_pipeline = std::move(column_pipepline.pipeline);
|
||||||
bool is_result_sparse = global_ctx->new_data_part->getSerialization(column_name)->getKind() == ISerialization::Kind::SPARSE;
|
|
||||||
|
|
||||||
const auto data_settings = global_ctx->data->getSettings();
|
|
||||||
auto transform = std::make_unique<ColumnGathererTransform>(
|
|
||||||
pipe.getHeader(),
|
|
||||||
pipe.numOutputPorts(),
|
|
||||||
*ctx->rows_sources_read_buf,
|
|
||||||
data_settings->merge_max_block_size,
|
|
||||||
data_settings->merge_max_block_size_bytes,
|
|
||||||
is_result_sparse);
|
|
||||||
|
|
||||||
pipe.addTransform(std::move(transform));
|
|
||||||
|
|
||||||
MergeTreeIndices indexes_to_recalc;
|
|
||||||
auto indexes_it = global_ctx->skip_indexes_by_column.find(column_name);
|
|
||||||
|
|
||||||
if (indexes_it != global_ctx->skip_indexes_by_column.end())
|
|
||||||
{
|
|
||||||
indexes_to_recalc = MergeTreeIndexFactory::instance().getMany(indexes_it->second);
|
|
||||||
|
|
||||||
pipe.addTransform(std::make_shared<ExpressionTransform>(
|
|
||||||
pipe.getHeader(),
|
|
||||||
indexes_it->second.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(),
|
|
||||||
global_ctx->data->getContext())));
|
|
||||||
|
|
||||||
pipe.addTransform(std::make_shared<MaterializingTransform>(pipe.getHeader()));
|
|
||||||
}
|
|
||||||
|
|
||||||
ctx->column_parts_pipeline = QueryPipeline(std::move(pipe));
|
|
||||||
|
|
||||||
/// Dereference unique_ptr
|
/// Dereference unique_ptr
|
||||||
ctx->column_parts_pipeline.setProgressCallback(MergeProgressCallback(
|
ctx->column_parts_pipeline.setProgressCallback(MergeProgressCallback(
|
||||||
@ -909,7 +1043,7 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const
|
|||||||
global_ctx->metadata_snapshot,
|
global_ctx->metadata_snapshot,
|
||||||
columns_list,
|
columns_list,
|
||||||
ctx->compression_codec,
|
ctx->compression_codec,
|
||||||
indexes_to_recalc,
|
column_pipepline.indexes_to_recalc,
|
||||||
getStatisticsForColumns(columns_list, global_ctx->metadata_snapshot),
|
getStatisticsForColumns(columns_list, global_ctx->metadata_snapshot),
|
||||||
&global_ctx->written_offset_columns,
|
&global_ctx->written_offset_columns,
|
||||||
global_ctx->to->getIndexGranularity());
|
global_ctx->to->getIndexGranularity());
|
||||||
@ -1219,12 +1353,208 @@ bool MergeTask::execute()
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
|
/// Apply merge strategy (Ordinary, Colapsing, Aggregating, etc) to the stream
|
||||||
|
class MergePartsStep : public ITransformingStep
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
MergePartsStep(
|
||||||
|
const DataStream & input_stream_,
|
||||||
|
const SortDescription & sort_description_,
|
||||||
|
const Names partition_key_columns_,
|
||||||
|
const MergeTreeData::MergingParams & merging_params_,
|
||||||
|
const String & rows_sources_temporary_file_name_,
|
||||||
|
UInt64 merge_block_size_rows_,
|
||||||
|
UInt64 merge_block_size_bytes_,
|
||||||
|
bool blocks_are_granules_size_,
|
||||||
|
bool cleanup_,
|
||||||
|
time_t time_of_merge_)
|
||||||
|
: ITransformingStep(input_stream_, input_stream_.header, getTraits())
|
||||||
|
, sort_description(sort_description_)
|
||||||
|
, partition_key_columns(partition_key_columns_)
|
||||||
|
, merging_params(merging_params_)
|
||||||
|
, rows_sources_temporary_file_name(rows_sources_temporary_file_name_)
|
||||||
|
, merge_block_size_rows(merge_block_size_rows_)
|
||||||
|
, merge_block_size_bytes(merge_block_size_bytes_)
|
||||||
|
, blocks_are_granules_size(blocks_are_granules_size_)
|
||||||
|
, cleanup(cleanup_)
|
||||||
|
, time_of_merge(time_of_merge_)
|
||||||
|
{}
|
||||||
|
|
||||||
|
String getName() const override { return "MergeParts"; }
|
||||||
|
|
||||||
|
void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & pipeline_settings) override
|
||||||
|
{
|
||||||
|
/// The order of the streams is important: when the key is matched, the elements go in the order of the source stream number.
|
||||||
|
/// In the merged part, the lines with the same key must be in the ascending order of the identifier of original part,
|
||||||
|
/// that is going in insertion order.
|
||||||
|
ProcessorPtr merged_transform;
|
||||||
|
|
||||||
|
const auto &header = pipeline.getHeader();
|
||||||
|
const auto input_streams_count = pipeline.getNumStreams();
|
||||||
|
|
||||||
|
WriteBuffer * rows_sources_write_buf = nullptr;
|
||||||
|
if (!rows_sources_temporary_file_name.empty())
|
||||||
|
{
|
||||||
|
if (!pipeline_settings.temporary_file_lookup)
|
||||||
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file lookup is not set in pipeline settings for vertical merge");
|
||||||
|
rows_sources_write_buf = &pipeline_settings.temporary_file_lookup->getTemporaryFileForWriting(rows_sources_temporary_file_name);
|
||||||
|
}
|
||||||
|
|
||||||
|
switch (merging_params.mode)
|
||||||
|
{
|
||||||
|
case MergeTreeData::MergingParams::Ordinary:
|
||||||
|
merged_transform = std::make_shared<MergingSortedTransform>(
|
||||||
|
header,
|
||||||
|
input_streams_count,
|
||||||
|
sort_description,
|
||||||
|
merge_block_size_rows,
|
||||||
|
merge_block_size_bytes,
|
||||||
|
SortingQueueStrategy::Default,
|
||||||
|
/* limit_= */0,
|
||||||
|
/* always_read_till_end_= */false,
|
||||||
|
rows_sources_write_buf,
|
||||||
|
blocks_are_granules_size);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case MergeTreeData::MergingParams::Collapsing:
|
||||||
|
merged_transform = std::make_shared<CollapsingSortedTransform>(
|
||||||
|
header, input_streams_count, sort_description, merging_params.sign_column, false,
|
||||||
|
merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case MergeTreeData::MergingParams::Summing:
|
||||||
|
merged_transform = std::make_shared<SummingSortedTransform>(
|
||||||
|
header, input_streams_count, sort_description, merging_params.columns_to_sum, partition_key_columns, merge_block_size_rows, merge_block_size_bytes);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case MergeTreeData::MergingParams::Aggregating:
|
||||||
|
merged_transform = std::make_shared<AggregatingSortedTransform>(header, input_streams_count, sort_description, merge_block_size_rows, merge_block_size_bytes);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case MergeTreeData::MergingParams::Replacing:
|
||||||
|
merged_transform = std::make_shared<ReplacingSortedTransform>(
|
||||||
|
header, input_streams_count, sort_description, merging_params.is_deleted_column, merging_params.version_column,
|
||||||
|
merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size,
|
||||||
|
cleanup);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case MergeTreeData::MergingParams::Graphite:
|
||||||
|
merged_transform = std::make_shared<GraphiteRollupSortedTransform>(
|
||||||
|
header, input_streams_count, sort_description, merge_block_size_rows, merge_block_size_bytes,
|
||||||
|
merging_params.graphite_params, time_of_merge);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case MergeTreeData::MergingParams::VersionedCollapsing:
|
||||||
|
merged_transform = std::make_shared<VersionedCollapsingTransform>(
|
||||||
|
header, input_streams_count, sort_description, merging_params.sign_column,
|
||||||
|
merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
pipeline.addTransform(std::move(merged_transform));
|
||||||
|
|
||||||
|
#ifndef NDEBUG
|
||||||
|
if (!sort_description.empty())
|
||||||
|
{
|
||||||
|
pipeline.addSimpleTransform([&](const Block & header_)
|
||||||
|
{
|
||||||
|
auto transform = std::make_shared<CheckSortedTransform>(header_, sort_description);
|
||||||
|
return transform;
|
||||||
|
});
|
||||||
|
}
|
||||||
|
#endif
|
||||||
|
}
|
||||||
|
|
||||||
|
void updateOutputStream() override
|
||||||
|
{
|
||||||
|
output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits());
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
static Traits getTraits()
|
||||||
|
{
|
||||||
|
return ITransformingStep::Traits
|
||||||
|
{
|
||||||
|
{
|
||||||
|
.returns_single_stream = true,
|
||||||
|
.preserves_number_of_streams = true,
|
||||||
|
.preserves_sorting = true,
|
||||||
|
},
|
||||||
|
{
|
||||||
|
.preserves_number_of_rows = false,
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
const SortDescription sort_description;
|
||||||
|
const Names partition_key_columns;
|
||||||
|
const MergeTreeData::MergingParams merging_params{};
|
||||||
|
const String rows_sources_temporary_file_name;
|
||||||
|
const UInt64 merge_block_size_rows;
|
||||||
|
const UInt64 merge_block_size_bytes;
|
||||||
|
const bool blocks_are_granules_size;
|
||||||
|
const bool cleanup{false};
|
||||||
|
const time_t time_of_merge{0};
|
||||||
|
};
|
||||||
|
|
||||||
|
class TTLStep : public ITransformingStep
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
TTLStep(
|
||||||
|
const DataStream & input_stream_,
|
||||||
|
const ContextPtr & context_,
|
||||||
|
const MergeTreeData & storage_,
|
||||||
|
const StorageMetadataPtr & metadata_snapshot_,
|
||||||
|
const MergeTreeData::MutableDataPartPtr & data_part_,
|
||||||
|
time_t current_time,
|
||||||
|
bool force_)
|
||||||
|
: ITransformingStep(input_stream_, input_stream_.header, getTraits())
|
||||||
|
{
|
||||||
|
transform = std::make_shared<TTLTransform>(context_, input_stream_.header, storage_, metadata_snapshot_, data_part_, current_time, force_);
|
||||||
|
subqueries_for_sets = transform->getSubqueries();
|
||||||
|
}
|
||||||
|
|
||||||
|
String getName() const override { return "TTL"; }
|
||||||
|
|
||||||
|
PreparedSets::Subqueries getSubqueries() { return std::move(subqueries_for_sets); }
|
||||||
|
|
||||||
|
void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override
|
||||||
|
{
|
||||||
|
pipeline.addTransform(transform);
|
||||||
|
}
|
||||||
|
|
||||||
|
void updateOutputStream() override
|
||||||
|
{
|
||||||
|
output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits());
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
static Traits getTraits()
|
||||||
|
{
|
||||||
|
return ITransformingStep::Traits
|
||||||
|
{
|
||||||
|
{
|
||||||
|
.returns_single_stream = true,
|
||||||
|
.preserves_number_of_streams = true,
|
||||||
|
.preserves_sorting = true,
|
||||||
|
},
|
||||||
|
{
|
||||||
|
.preserves_number_of_rows = false,
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
std::shared_ptr<TTLTransform> transform;
|
||||||
|
PreparedSets::Subqueries subqueries_for_sets;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const
|
||||||
{
|
{
|
||||||
/** Read from all parts, merge and write into a new one.
|
/** Read from all parts, merge and write into a new one.
|
||||||
* In passing, we calculate expression for sorting.
|
* In passing, we calculate expression for sorting.
|
||||||
*/
|
*/
|
||||||
Pipes pipes;
|
|
||||||
global_ctx->watch_prev_elapsed = 0;
|
global_ctx->watch_prev_elapsed = 0;
|
||||||
|
|
||||||
/// We count total amount of bytes in parts
|
/// We count total amount of bytes in parts
|
||||||
@ -1251,33 +1581,58 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
|
|||||||
global_ctx->horizontal_stage_progress = std::make_unique<MergeStageProgress>(
|
global_ctx->horizontal_stage_progress = std::make_unique<MergeStageProgress>(
|
||||||
ctx->column_sizes ? ctx->column_sizes->keyColumnsWeight() : 1.0);
|
ctx->column_sizes ? ctx->column_sizes->keyColumnsWeight() : 1.0);
|
||||||
|
|
||||||
|
/// Read from all parts
|
||||||
|
std::vector<QueryPlanPtr> plans;
|
||||||
for (size_t i = 0; i < global_ctx->future_part->parts.size(); ++i)
|
for (size_t i = 0; i < global_ctx->future_part->parts.size(); ++i)
|
||||||
{
|
{
|
||||||
Pipe pipe = createMergeTreeSequentialSource(
|
if (global_ctx->future_part->parts[i]->getMarksCount() == 0)
|
||||||
|
LOG_TRACE(ctx->log, "Part {} is empty", global_ctx->future_part->parts[i]->name);
|
||||||
|
|
||||||
|
auto plan_for_part = std::make_unique<QueryPlan>();
|
||||||
|
createReadFromPartStep(
|
||||||
MergeTreeSequentialSourceType::Merge,
|
MergeTreeSequentialSourceType::Merge,
|
||||||
|
*plan_for_part,
|
||||||
*global_ctx->data,
|
*global_ctx->data,
|
||||||
global_ctx->storage_snapshot,
|
global_ctx->storage_snapshot,
|
||||||
global_ctx->future_part->parts[i],
|
global_ctx->future_part->parts[i],
|
||||||
global_ctx->alter_conversions[i],
|
global_ctx->alter_conversions[i],
|
||||||
global_ctx->merging_columns.getNames(),
|
global_ctx->merging_columns.getNames(),
|
||||||
/*mark_ranges=*/ {},
|
|
||||||
global_ctx->input_rows_filtered,
|
global_ctx->input_rows_filtered,
|
||||||
/*apply_deleted_mask=*/ true,
|
/*apply_deleted_mask=*/ true,
|
||||||
|
/*filter=*/ std::nullopt,
|
||||||
ctx->read_with_direct_io,
|
ctx->read_with_direct_io,
|
||||||
/*prefetch=*/ false);
|
/*prefetch=*/ false,
|
||||||
|
global_ctx->context,
|
||||||
|
ctx->log);
|
||||||
|
|
||||||
|
plans.emplace_back(std::move(plan_for_part));
|
||||||
|
}
|
||||||
|
|
||||||
|
QueryPlan merge_parts_query_plan;
|
||||||
|
|
||||||
|
/// Union of all parts streams
|
||||||
|
{
|
||||||
|
DataStreams input_streams;
|
||||||
|
input_streams.reserve(plans.size());
|
||||||
|
for (auto & plan : plans)
|
||||||
|
input_streams.emplace_back(plan->getCurrentDataStream());
|
||||||
|
|
||||||
|
auto union_step = std::make_unique<UnionStep>(std::move(input_streams));
|
||||||
|
merge_parts_query_plan.unitePlans(std::move(union_step), std::move(plans));
|
||||||
|
}
|
||||||
|
|
||||||
if (global_ctx->metadata_snapshot->hasSortingKey())
|
if (global_ctx->metadata_snapshot->hasSortingKey())
|
||||||
{
|
{
|
||||||
pipe.addSimpleTransform([this](const Block & header)
|
/// Calculate sorting key expressions so that they are available for merge sorting.
|
||||||
|
auto sorting_key_expression_dag = global_ctx->metadata_snapshot->getSortingKey().expression->getActionsDAG().clone();
|
||||||
|
auto calculate_sorting_key_expression_step = std::make_unique<ExpressionStep>(
|
||||||
|
merge_parts_query_plan.getCurrentDataStream(),
|
||||||
|
std::move(sorting_key_expression_dag));
|
||||||
|
merge_parts_query_plan.addStep(std::move(calculate_sorting_key_expression_step));
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Merge
|
||||||
{
|
{
|
||||||
return std::make_shared<ExpressionTransform>(header, global_ctx->metadata_snapshot->getSortingKey().expression);
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
pipes.emplace_back(std::move(pipe));
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
Names sort_columns = global_ctx->metadata_snapshot->getSortingKeyColumns();
|
Names sort_columns = global_ctx->metadata_snapshot->getSortingKeyColumns();
|
||||||
SortDescription sort_description;
|
SortDescription sort_description;
|
||||||
sort_description.compile_sort_description = global_ctx->data->getContext()->getSettingsRef().compile_sort_description;
|
sort_description.compile_sort_description = global_ctx->data->getContext()->getSettingsRef().compile_sort_description;
|
||||||
@ -1288,107 +1643,31 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
|
|||||||
|
|
||||||
Names partition_key_columns = global_ctx->metadata_snapshot->getPartitionKey().column_names;
|
Names partition_key_columns = global_ctx->metadata_snapshot->getPartitionKey().column_names;
|
||||||
|
|
||||||
Block header = pipes.at(0).getHeader();
|
|
||||||
for (size_t i = 0; i < sort_columns_size; ++i)
|
for (size_t i = 0; i < sort_columns_size; ++i)
|
||||||
sort_description.emplace_back(sort_columns[i], 1, 1);
|
sort_description.emplace_back(sort_columns[i], 1, 1);
|
||||||
|
|
||||||
#ifndef NDEBUG
|
const bool is_vertical_merge = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical);
|
||||||
if (!sort_description.empty())
|
|
||||||
{
|
|
||||||
for (size_t i = 0; i < pipes.size(); ++i)
|
|
||||||
{
|
|
||||||
auto & pipe = pipes[i];
|
|
||||||
pipe.addSimpleTransform([&](const Block & header_)
|
|
||||||
{
|
|
||||||
auto transform = std::make_shared<CheckSortedTransform>(header_, sort_description);
|
|
||||||
transform->setDescription(global_ctx->future_part->parts[i]->name);
|
|
||||||
return transform;
|
|
||||||
});
|
|
||||||
}
|
|
||||||
}
|
|
||||||
#endif
|
|
||||||
|
|
||||||
/// The order of the streams is important: when the key is matched, the elements go in the order of the source stream number.
|
|
||||||
/// In the merged part, the lines with the same key must be in the ascending order of the identifier of original part,
|
|
||||||
/// that is going in insertion order.
|
|
||||||
ProcessorPtr merged_transform;
|
|
||||||
|
|
||||||
/// If merge is vertical we cannot calculate it
|
/// If merge is vertical we cannot calculate it
|
||||||
ctx->blocks_are_granules_size = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical);
|
ctx->blocks_are_granules_size = is_vertical_merge;
|
||||||
|
|
||||||
/// There is no sense to have the block size bigger than one granule for merge operations.
|
|
||||||
const UInt64 merge_block_size_rows = data_settings->merge_max_block_size;
|
|
||||||
const UInt64 merge_block_size_bytes = data_settings->merge_max_block_size_bytes;
|
|
||||||
|
|
||||||
switch (ctx->merging_params.mode)
|
|
||||||
{
|
|
||||||
case MergeTreeData::MergingParams::Ordinary:
|
|
||||||
merged_transform = std::make_shared<MergingSortedTransform>(
|
|
||||||
header,
|
|
||||||
pipes.size(),
|
|
||||||
sort_description,
|
|
||||||
merge_block_size_rows,
|
|
||||||
merge_block_size_bytes,
|
|
||||||
SortingQueueStrategy::Default,
|
|
||||||
/* limit_= */0,
|
|
||||||
/* always_read_till_end_= */false,
|
|
||||||
ctx->rows_sources_write_buf.get(),
|
|
||||||
ctx->blocks_are_granules_size);
|
|
||||||
break;
|
|
||||||
|
|
||||||
case MergeTreeData::MergingParams::Collapsing:
|
|
||||||
merged_transform = std::make_shared<CollapsingSortedTransform>(
|
|
||||||
header, pipes.size(), sort_description, ctx->merging_params.sign_column, false,
|
|
||||||
merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size);
|
|
||||||
break;
|
|
||||||
|
|
||||||
case MergeTreeData::MergingParams::Summing:
|
|
||||||
merged_transform = std::make_shared<SummingSortedTransform>(
|
|
||||||
header, pipes.size(), sort_description, ctx->merging_params.columns_to_sum, partition_key_columns, merge_block_size_rows, merge_block_size_bytes);
|
|
||||||
break;
|
|
||||||
|
|
||||||
case MergeTreeData::MergingParams::Aggregating:
|
|
||||||
merged_transform = std::make_shared<AggregatingSortedTransform>(header, pipes.size(), sort_description, merge_block_size_rows, merge_block_size_bytes);
|
|
||||||
break;
|
|
||||||
|
|
||||||
case MergeTreeData::MergingParams::Replacing:
|
|
||||||
if (global_ctx->cleanup && !data_settings->allow_experimental_replacing_merge_with_cleanup)
|
if (global_ctx->cleanup && !data_settings->allow_experimental_replacing_merge_with_cleanup)
|
||||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental merges with CLEANUP are not allowed");
|
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental merges with CLEANUP are not allowed");
|
||||||
|
|
||||||
merged_transform = std::make_shared<ReplacingSortedTransform>(
|
auto merge_step = std::make_unique<MergePartsStep>(
|
||||||
header, pipes.size(), sort_description, ctx->merging_params.is_deleted_column, ctx->merging_params.version_column,
|
merge_parts_query_plan.getCurrentDataStream(),
|
||||||
merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size,
|
sort_description,
|
||||||
global_ctx->cleanup);
|
partition_key_columns,
|
||||||
break;
|
global_ctx->merging_params,
|
||||||
|
(is_vertical_merge ? RowsSourcesTemporaryFile::FILE_ID : ""), /// rows_sources temporaty file is used only for vertical merge
|
||||||
case MergeTreeData::MergingParams::Graphite:
|
data_settings->merge_max_block_size,
|
||||||
merged_transform = std::make_shared<GraphiteRollupSortedTransform>(
|
data_settings->merge_max_block_size_bytes,
|
||||||
header, pipes.size(), sort_description, merge_block_size_rows, merge_block_size_bytes,
|
ctx->blocks_are_granules_size,
|
||||||
ctx->merging_params.graphite_params, global_ctx->time_of_merge);
|
global_ctx->cleanup,
|
||||||
break;
|
global_ctx->time_of_merge);
|
||||||
|
merge_step->setStepDescription("Merge sorted parts");
|
||||||
case MergeTreeData::MergingParams::VersionedCollapsing:
|
merge_parts_query_plan.addStep(std::move(merge_step));
|
||||||
merged_transform = std::make_shared<VersionedCollapsingTransform>(
|
|
||||||
header, pipes.size(), sort_description, ctx->merging_params.sign_column,
|
|
||||||
merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size);
|
|
||||||
break;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
auto builder = std::make_unique<QueryPipelineBuilder>();
|
|
||||||
builder->init(Pipe::unitePipes(std::move(pipes)));
|
|
||||||
builder->addTransform(std::move(merged_transform));
|
|
||||||
|
|
||||||
#ifndef NDEBUG
|
|
||||||
if (!sort_description.empty())
|
|
||||||
{
|
|
||||||
builder->addSimpleTransform([&](const Block & header_)
|
|
||||||
{
|
|
||||||
auto transform = std::make_shared<CheckSortedTransform>(header_, sort_description);
|
|
||||||
return transform;
|
|
||||||
});
|
|
||||||
}
|
|
||||||
#endif
|
|
||||||
|
|
||||||
if (global_ctx->deduplicate)
|
if (global_ctx->deduplicate)
|
||||||
{
|
{
|
||||||
const auto & virtuals = *global_ctx->data->getVirtualsPtr();
|
const auto & virtuals = *global_ctx->data->getVirtualsPtr();
|
||||||
@ -1406,37 +1685,51 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (DistinctSortedTransform::isApplicable(header, sort_description, global_ctx->deduplicate_by_columns))
|
auto deduplication_step = std::make_unique<DistinctStep>(
|
||||||
builder->addTransform(std::make_shared<DistinctSortedTransform>(
|
merge_parts_query_plan.getCurrentDataStream(),
|
||||||
builder->getHeader(), sort_description, SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns));
|
SizeLimits(), 0 /*limit_hint*/,
|
||||||
else
|
global_ctx->deduplicate_by_columns,
|
||||||
builder->addTransform(std::make_shared<DistinctTransform>(
|
false /*pre_distinct*/,
|
||||||
builder->getHeader(), SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns));
|
true /*optimize_distinct_in_order TODO: looks like it should be enabled*/);
|
||||||
|
deduplication_step->setStepDescription("Deduplication step");
|
||||||
|
merge_parts_query_plan.addStep(std::move(deduplication_step));
|
||||||
}
|
}
|
||||||
|
|
||||||
PreparedSets::Subqueries subqueries;
|
PreparedSets::Subqueries subqueries;
|
||||||
|
|
||||||
|
/// TTL step
|
||||||
if (ctx->need_remove_expired_values)
|
if (ctx->need_remove_expired_values)
|
||||||
{
|
{
|
||||||
auto transform = std::make_shared<TTLTransform>(global_ctx->context, builder->getHeader(), *global_ctx->data, global_ctx->metadata_snapshot, global_ctx->new_data_part, global_ctx->time_of_merge, ctx->force_ttl);
|
auto ttl_step = std::make_unique<TTLStep>(
|
||||||
subqueries = transform->getSubqueries();
|
merge_parts_query_plan.getCurrentDataStream(), global_ctx->context, *global_ctx->data, global_ctx->metadata_snapshot, global_ctx->new_data_part, global_ctx->time_of_merge, ctx->force_ttl);
|
||||||
builder->addTransform(std::move(transform));
|
subqueries = ttl_step->getSubqueries();
|
||||||
|
ttl_step->setStepDescription("TTL step");
|
||||||
|
merge_parts_query_plan.addStep(std::move(ttl_step));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Secondary indices expressions
|
||||||
if (!global_ctx->merging_skip_indexes.empty())
|
if (!global_ctx->merging_skip_indexes.empty())
|
||||||
{
|
{
|
||||||
builder->addTransform(std::make_shared<ExpressionTransform>(
|
auto indices_expression_dag = global_ctx->merging_skip_indexes.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext())->getActionsDAG().clone();
|
||||||
builder->getHeader(),
|
indices_expression_dag.addMaterializingOutputActions(); /// Const columns cannot be written without materialization.
|
||||||
global_ctx->merging_skip_indexes.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(),
|
auto calculate_indices_expression_step = std::make_unique<ExpressionStep>(
|
||||||
global_ctx->data->getContext())));
|
merge_parts_query_plan.getCurrentDataStream(),
|
||||||
|
std::move(indices_expression_dag));
|
||||||
builder->addTransform(std::make_shared<MaterializingTransform>(builder->getHeader()));
|
merge_parts_query_plan.addStep(std::move(calculate_indices_expression_step));
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!subqueries.empty())
|
if (!subqueries.empty())
|
||||||
builder = addCreatingSetsTransform(std::move(builder), std::move(subqueries), global_ctx->context);
|
addCreatingSetsStep(merge_parts_query_plan, std::move(subqueries), global_ctx->context);
|
||||||
|
|
||||||
|
{
|
||||||
|
auto pipeline_settings = BuildQueryPipelineSettings::fromContext(global_ctx->context);
|
||||||
|
pipeline_settings.temporary_file_lookup = ctx->rows_sources_temporary_file;
|
||||||
|
auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context);
|
||||||
|
auto builder = merge_parts_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings);
|
||||||
|
|
||||||
global_ctx->merged_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
|
global_ctx->merged_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
|
||||||
|
}
|
||||||
|
|
||||||
/// Dereference unique_ptr and pass horizontal_stage_progress by reference
|
/// Dereference unique_ptr and pass horizontal_stage_progress by reference
|
||||||
global_ctx->merged_pipeline.setProgressCallback(MergeProgressCallback(global_ctx->merge_list_element_ptr, global_ctx->watch_prev_elapsed, *global_ctx->horizontal_stage_progress));
|
global_ctx->merged_pipeline.setProgressCallback(MergeProgressCallback(global_ctx->merge_list_element_ptr, global_ctx->watch_prev_elapsed, *global_ctx->horizontal_stage_progress));
|
||||||
/// Is calculated inside MergeProgressCallback.
|
/// Is calculated inside MergeProgressCallback.
|
||||||
@ -1475,10 +1768,10 @@ MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm
|
|||||||
}
|
}
|
||||||
|
|
||||||
bool is_supported_storage =
|
bool is_supported_storage =
|
||||||
ctx->merging_params.mode == MergeTreeData::MergingParams::Ordinary ||
|
global_ctx->merging_params.mode == MergeTreeData::MergingParams::Ordinary ||
|
||||||
ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing ||
|
global_ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing ||
|
||||||
ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing ||
|
global_ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing ||
|
||||||
ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing;
|
global_ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing;
|
||||||
|
|
||||||
bool enough_ordinary_cols = global_ctx->gathering_columns.size() >= data_settings->vertical_merge_algorithm_min_columns_to_activate;
|
bool enough_ordinary_cols = global_ctx->gathering_columns.size() >= data_settings->vertical_merge_algorithm_min_columns_to_activate;
|
||||||
|
|
||||||
|
@ -40,6 +40,7 @@ namespace DB
|
|||||||
|
|
||||||
class MergeTask;
|
class MergeTask;
|
||||||
using MergeTaskPtr = std::shared_ptr<MergeTask>;
|
using MergeTaskPtr = std::shared_ptr<MergeTask>;
|
||||||
|
class RowsSourcesTemporaryFile;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Overview of the merge algorithm
|
* Overview of the merge algorithm
|
||||||
@ -100,6 +101,7 @@ public:
|
|||||||
global_ctx->context = std::move(context_);
|
global_ctx->context = std::move(context_);
|
||||||
global_ctx->holder = &holder;
|
global_ctx->holder = &holder;
|
||||||
global_ctx->space_reservation = std::move(space_reservation_);
|
global_ctx->space_reservation = std::move(space_reservation_);
|
||||||
|
global_ctx->disk = global_ctx->space_reservation->getDisk();
|
||||||
global_ctx->deduplicate = std::move(deduplicate_);
|
global_ctx->deduplicate = std::move(deduplicate_);
|
||||||
global_ctx->deduplicate_by_columns = std::move(deduplicate_by_columns_);
|
global_ctx->deduplicate_by_columns = std::move(deduplicate_by_columns_);
|
||||||
global_ctx->cleanup = std::move(cleanup_);
|
global_ctx->cleanup = std::move(cleanup_);
|
||||||
@ -110,12 +112,10 @@ public:
|
|||||||
global_ctx->ttl_merges_blocker = std::move(ttl_merges_blocker_);
|
global_ctx->ttl_merges_blocker = std::move(ttl_merges_blocker_);
|
||||||
global_ctx->txn = std::move(txn);
|
global_ctx->txn = std::move(txn);
|
||||||
global_ctx->need_prefix = need_prefix;
|
global_ctx->need_prefix = need_prefix;
|
||||||
|
global_ctx->suffix = std::move(suffix_);
|
||||||
|
global_ctx->merging_params = std::move(merging_params_);
|
||||||
|
|
||||||
auto prepare_stage_ctx = std::make_shared<ExecuteAndFinalizeHorizontalPartRuntimeContext>();
|
auto prepare_stage_ctx = std::make_shared<ExecuteAndFinalizeHorizontalPartRuntimeContext>();
|
||||||
|
|
||||||
prepare_stage_ctx->suffix = std::move(suffix_);
|
|
||||||
prepare_stage_ctx->merging_params = std::move(merging_params_);
|
|
||||||
|
|
||||||
(*stages.begin())->setRuntimeContext(std::move(prepare_stage_ctx), global_ctx);
|
(*stages.begin())->setRuntimeContext(std::move(prepare_stage_ctx), global_ctx);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -172,6 +172,7 @@ private:
|
|||||||
ContextPtr context{nullptr};
|
ContextPtr context{nullptr};
|
||||||
time_t time_of_merge{0};
|
time_t time_of_merge{0};
|
||||||
ReservationSharedPtr space_reservation{nullptr};
|
ReservationSharedPtr space_reservation{nullptr};
|
||||||
|
DiskPtr disk{nullptr};
|
||||||
bool deduplicate{false};
|
bool deduplicate{false};
|
||||||
Names deduplicate_by_columns{};
|
Names deduplicate_by_columns{};
|
||||||
bool cleanup{false};
|
bool cleanup{false};
|
||||||
@ -210,6 +211,8 @@ private:
|
|||||||
|
|
||||||
MergeTreeTransactionPtr txn;
|
MergeTreeTransactionPtr txn;
|
||||||
bool need_prefix;
|
bool need_prefix;
|
||||||
|
String suffix;
|
||||||
|
MergeTreeData::MergingParams merging_params{};
|
||||||
|
|
||||||
scope_guard temporary_directory_lock;
|
scope_guard temporary_directory_lock;
|
||||||
UInt64 prev_elapsed_ms{0};
|
UInt64 prev_elapsed_ms{0};
|
||||||
@ -222,19 +225,11 @@ private:
|
|||||||
/// Proper initialization is responsibility of the author
|
/// Proper initialization is responsibility of the author
|
||||||
struct ExecuteAndFinalizeHorizontalPartRuntimeContext : public IStageRuntimeContext
|
struct ExecuteAndFinalizeHorizontalPartRuntimeContext : public IStageRuntimeContext
|
||||||
{
|
{
|
||||||
/// Dependencies
|
|
||||||
String suffix;
|
|
||||||
bool need_prefix;
|
|
||||||
MergeTreeData::MergingParams merging_params{};
|
|
||||||
|
|
||||||
TemporaryDataOnDiskPtr tmp_disk{nullptr};
|
|
||||||
DiskPtr disk{nullptr};
|
|
||||||
bool need_remove_expired_values{false};
|
bool need_remove_expired_values{false};
|
||||||
bool force_ttl{false};
|
bool force_ttl{false};
|
||||||
CompressionCodecPtr compression_codec{nullptr};
|
CompressionCodecPtr compression_codec{nullptr};
|
||||||
size_t sum_input_rows_upper_bound{0};
|
size_t sum_input_rows_upper_bound{0};
|
||||||
std::unique_ptr<WriteBufferFromFileBase> rows_sources_uncompressed_write_buf{nullptr};
|
std::shared_ptr<RowsSourcesTemporaryFile> rows_sources_temporary_file;
|
||||||
std::unique_ptr<WriteBuffer> rows_sources_write_buf{nullptr};
|
|
||||||
std::optional<ColumnSizeEstimator> column_sizes{};
|
std::optional<ColumnSizeEstimator> column_sizes{};
|
||||||
|
|
||||||
/// For projections to rebuild
|
/// For projections to rebuild
|
||||||
@ -264,17 +259,16 @@ private:
|
|||||||
|
|
||||||
using ExecuteAndFinalizeHorizontalPartRuntimeContextPtr = std::shared_ptr<ExecuteAndFinalizeHorizontalPartRuntimeContext>;
|
using ExecuteAndFinalizeHorizontalPartRuntimeContextPtr = std::shared_ptr<ExecuteAndFinalizeHorizontalPartRuntimeContext>;
|
||||||
|
|
||||||
|
|
||||||
struct ExecuteAndFinalizeHorizontalPart : public IStage
|
struct ExecuteAndFinalizeHorizontalPart : public IStage
|
||||||
{
|
{
|
||||||
bool execute() override;
|
bool execute() override;
|
||||||
|
|
||||||
bool prepare();
|
bool prepare() const;
|
||||||
bool executeImpl();
|
bool executeImpl() const;
|
||||||
void finalize() const;
|
void finalize() const;
|
||||||
|
|
||||||
/// NOTE: Using pointer-to-member instead of std::function and lambda makes stacktraces much more concise and readable
|
/// NOTE: Using pointer-to-member instead of std::function and lambda makes stacktraces much more concise and readable
|
||||||
using ExecuteAndFinalizeHorizontalPartSubtasks = std::array<bool(ExecuteAndFinalizeHorizontalPart::*)(), 3>;
|
using ExecuteAndFinalizeHorizontalPartSubtasks = std::array<bool(ExecuteAndFinalizeHorizontalPart::*)()const, 3>;
|
||||||
|
|
||||||
const ExecuteAndFinalizeHorizontalPartSubtasks subtasks
|
const ExecuteAndFinalizeHorizontalPartSubtasks subtasks
|
||||||
{
|
{
|
||||||
@ -289,10 +283,10 @@ private:
|
|||||||
void calculateProjections(const Block & block) const;
|
void calculateProjections(const Block & block) const;
|
||||||
void finalizeProjections() const;
|
void finalizeProjections() const;
|
||||||
void constructTaskForProjectionPartsMerge() const;
|
void constructTaskForProjectionPartsMerge() const;
|
||||||
bool executeMergeProjections();
|
bool executeMergeProjections() const;
|
||||||
|
|
||||||
MergeAlgorithm chooseMergeAlgorithm() const;
|
MergeAlgorithm chooseMergeAlgorithm() const;
|
||||||
void createMergedStream();
|
void createMergedStream() const;
|
||||||
void extractMergingAndGatheringColumns() const;
|
void extractMergingAndGatheringColumns() const;
|
||||||
|
|
||||||
void setRuntimeContext(StageRuntimeContextPtr local, StageRuntimeContextPtr global) override
|
void setRuntimeContext(StageRuntimeContextPtr local, StageRuntimeContextPtr global) override
|
||||||
@ -314,11 +308,9 @@ private:
|
|||||||
struct VerticalMergeRuntimeContext : public IStageRuntimeContext
|
struct VerticalMergeRuntimeContext : public IStageRuntimeContext
|
||||||
{
|
{
|
||||||
/// Begin dependencies from previous stage
|
/// Begin dependencies from previous stage
|
||||||
std::unique_ptr<WriteBufferFromFileBase> rows_sources_uncompressed_write_buf{nullptr};
|
std::shared_ptr<RowsSourcesTemporaryFile> rows_sources_temporary_file;
|
||||||
std::unique_ptr<WriteBuffer> rows_sources_write_buf{nullptr};
|
|
||||||
std::optional<ColumnSizeEstimator> column_sizes;
|
std::optional<ColumnSizeEstimator> column_sizes;
|
||||||
CompressionCodecPtr compression_codec;
|
CompressionCodecPtr compression_codec;
|
||||||
TemporaryDataOnDiskPtr tmp_disk{nullptr};
|
|
||||||
std::list<DB::NameAndTypePair>::const_iterator it_name_and_type;
|
std::list<DB::NameAndTypePair>::const_iterator it_name_and_type;
|
||||||
bool read_with_direct_io{false};
|
bool read_with_direct_io{false};
|
||||||
bool need_sync{false};
|
bool need_sync{false};
|
||||||
@ -334,20 +326,27 @@ private:
|
|||||||
|
|
||||||
Float64 progress_before = 0;
|
Float64 progress_before = 0;
|
||||||
std::unique_ptr<MergedColumnOnlyOutputStream> column_to{nullptr};
|
std::unique_ptr<MergedColumnOnlyOutputStream> column_to{nullptr};
|
||||||
std::optional<Pipe> prepared_pipe;
|
|
||||||
|
/// Used for prefetching. Right before starting merge of a column we create a pipeline for the next column
|
||||||
|
/// and it initiates prefetching of the first range of that column.
|
||||||
|
struct PreparedColumnPipeline
|
||||||
|
{
|
||||||
|
QueryPipeline pipeline;
|
||||||
|
MergeTreeIndices indexes_to_recalc;
|
||||||
|
};
|
||||||
|
|
||||||
|
std::optional<PreparedColumnPipeline> prepared_pipeline;
|
||||||
size_t max_delayed_streams = 0;
|
size_t max_delayed_streams = 0;
|
||||||
bool use_prefetch = false;
|
bool use_prefetch = false;
|
||||||
std::list<std::unique_ptr<MergedColumnOnlyOutputStream>> delayed_streams;
|
std::list<std::unique_ptr<MergedColumnOnlyOutputStream>> delayed_streams;
|
||||||
size_t column_elems_written{0};
|
size_t column_elems_written{0};
|
||||||
QueryPipeline column_parts_pipeline;
|
QueryPipeline column_parts_pipeline;
|
||||||
std::unique_ptr<PullingPipelineExecutor> executor;
|
std::unique_ptr<PullingPipelineExecutor> executor;
|
||||||
std::unique_ptr<CompressedReadBufferFromFile> rows_sources_read_buf{nullptr};
|
|
||||||
UInt64 elapsed_execute_ns{0};
|
UInt64 elapsed_execute_ns{0};
|
||||||
};
|
};
|
||||||
|
|
||||||
using VerticalMergeRuntimeContextPtr = std::shared_ptr<VerticalMergeRuntimeContext>;
|
using VerticalMergeRuntimeContextPtr = std::shared_ptr<VerticalMergeRuntimeContext>;
|
||||||
|
|
||||||
|
|
||||||
struct VerticalMergeStage : public IStage
|
struct VerticalMergeStage : public IStage
|
||||||
{
|
{
|
||||||
bool execute() override;
|
bool execute() override;
|
||||||
@ -379,7 +378,7 @@ private:
|
|||||||
bool executeVerticalMergeForOneColumn() const;
|
bool executeVerticalMergeForOneColumn() const;
|
||||||
void finalizeVerticalMergeForOneColumn() const;
|
void finalizeVerticalMergeForOneColumn() const;
|
||||||
|
|
||||||
Pipe createPipeForReadingOneColumn(const String & column_name) const;
|
VerticalMergeRuntimeContext::PreparedColumnPipeline createPipelineForReadingOneColumn(const String & column_name) const;
|
||||||
|
|
||||||
VerticalMergeRuntimeContextPtr ctx;
|
VerticalMergeRuntimeContextPtr ctx;
|
||||||
GlobalRuntimeContextPtr global_ctx;
|
GlobalRuntimeContextPtr global_ctx;
|
||||||
|
@ -354,8 +354,11 @@ public:
|
|||||||
MergeTreeData::DataPartPtr data_part_,
|
MergeTreeData::DataPartPtr data_part_,
|
||||||
AlterConversionsPtr alter_conversions_,
|
AlterConversionsPtr alter_conversions_,
|
||||||
Names columns_to_read_,
|
Names columns_to_read_,
|
||||||
|
std::shared_ptr<std::atomic<size_t>> filtered_rows_count_,
|
||||||
bool apply_deleted_mask_,
|
bool apply_deleted_mask_,
|
||||||
std::optional<ActionsDAG> filter_,
|
std::optional<ActionsDAG> filter_,
|
||||||
|
bool read_with_direct_io_,
|
||||||
|
bool prefetch_,
|
||||||
ContextPtr context_,
|
ContextPtr context_,
|
||||||
LoggerPtr log_)
|
LoggerPtr log_)
|
||||||
: ISourceStep(DataStream{.header = storage_snapshot_->getSampleBlockForColumns(columns_to_read_)})
|
: ISourceStep(DataStream{.header = storage_snapshot_->getSampleBlockForColumns(columns_to_read_)})
|
||||||
@ -365,8 +368,11 @@ public:
|
|||||||
, data_part(std::move(data_part_))
|
, data_part(std::move(data_part_))
|
||||||
, alter_conversions(std::move(alter_conversions_))
|
, alter_conversions(std::move(alter_conversions_))
|
||||||
, columns_to_read(std::move(columns_to_read_))
|
, columns_to_read(std::move(columns_to_read_))
|
||||||
|
, filtered_rows_count(std::move(filtered_rows_count_))
|
||||||
, apply_deleted_mask(apply_deleted_mask_)
|
, apply_deleted_mask(apply_deleted_mask_)
|
||||||
, filter(std::move(filter_))
|
, filter(std::move(filter_))
|
||||||
|
, read_with_direct_io(read_with_direct_io_)
|
||||||
|
, prefetch(prefetch_)
|
||||||
, context(std::move(context_))
|
, context(std::move(context_))
|
||||||
, log(log_)
|
, log(log_)
|
||||||
{
|
{
|
||||||
@ -410,25 +416,28 @@ public:
|
|||||||
alter_conversions,
|
alter_conversions,
|
||||||
columns_to_read,
|
columns_to_read,
|
||||||
std::move(mark_ranges),
|
std::move(mark_ranges),
|
||||||
/*filtered_rows_count=*/ nullptr,
|
filtered_rows_count,
|
||||||
apply_deleted_mask,
|
apply_deleted_mask,
|
||||||
/*read_with_direct_io=*/ false,
|
read_with_direct_io,
|
||||||
/*prefetch=*/ false);
|
prefetch);
|
||||||
|
|
||||||
pipeline.init(Pipe(std::move(source)));
|
pipeline.init(Pipe(std::move(source)));
|
||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
MergeTreeSequentialSourceType type;
|
const MergeTreeSequentialSourceType type;
|
||||||
const MergeTreeData & storage;
|
const MergeTreeData & storage;
|
||||||
StorageSnapshotPtr storage_snapshot;
|
const StorageSnapshotPtr storage_snapshot;
|
||||||
MergeTreeData::DataPartPtr data_part;
|
const MergeTreeData::DataPartPtr data_part;
|
||||||
AlterConversionsPtr alter_conversions;
|
const AlterConversionsPtr alter_conversions;
|
||||||
Names columns_to_read;
|
const Names columns_to_read;
|
||||||
bool apply_deleted_mask;
|
const std::shared_ptr<std::atomic<size_t>> filtered_rows_count;
|
||||||
std::optional<ActionsDAG> filter;
|
const bool apply_deleted_mask;
|
||||||
ContextPtr context;
|
const std::optional<ActionsDAG> filter;
|
||||||
LoggerPtr log;
|
const bool read_with_direct_io;
|
||||||
|
const bool prefetch;
|
||||||
|
const ContextPtr context;
|
||||||
|
const LoggerPtr log;
|
||||||
};
|
};
|
||||||
|
|
||||||
void createReadFromPartStep(
|
void createReadFromPartStep(
|
||||||
@ -439,16 +448,28 @@ void createReadFromPartStep(
|
|||||||
MergeTreeData::DataPartPtr data_part,
|
MergeTreeData::DataPartPtr data_part,
|
||||||
AlterConversionsPtr alter_conversions,
|
AlterConversionsPtr alter_conversions,
|
||||||
Names columns_to_read,
|
Names columns_to_read,
|
||||||
|
std::shared_ptr<std::atomic<size_t>> filtered_rows_count,
|
||||||
bool apply_deleted_mask,
|
bool apply_deleted_mask,
|
||||||
std::optional<ActionsDAG> filter,
|
std::optional<ActionsDAG> filter,
|
||||||
|
bool read_with_direct_io,
|
||||||
|
bool prefetch,
|
||||||
ContextPtr context,
|
ContextPtr context,
|
||||||
LoggerPtr log)
|
LoggerPtr log)
|
||||||
{
|
{
|
||||||
auto reading = std::make_unique<ReadFromPart>(type,
|
auto reading = std::make_unique<ReadFromPart>(
|
||||||
storage, storage_snapshot,
|
type,
|
||||||
std::move(data_part), std::move(alter_conversions),
|
storage,
|
||||||
std::move(columns_to_read), apply_deleted_mask,
|
storage_snapshot,
|
||||||
std::move(filter), std::move(context), log);
|
std::move(data_part),
|
||||||
|
std::move(alter_conversions),
|
||||||
|
std::move(columns_to_read),
|
||||||
|
filtered_rows_count,
|
||||||
|
apply_deleted_mask,
|
||||||
|
std::move(filter),
|
||||||
|
read_with_direct_io,
|
||||||
|
prefetch,
|
||||||
|
std::move(context),
|
||||||
|
log);
|
||||||
|
|
||||||
plan.addStep(std::move(reading));
|
plan.addStep(std::move(reading));
|
||||||
}
|
}
|
||||||
|
@ -39,8 +39,11 @@ void createReadFromPartStep(
|
|||||||
MergeTreeData::DataPartPtr data_part,
|
MergeTreeData::DataPartPtr data_part,
|
||||||
AlterConversionsPtr alter_conversions,
|
AlterConversionsPtr alter_conversions,
|
||||||
Names columns_to_read,
|
Names columns_to_read,
|
||||||
|
std::shared_ptr<std::atomic<size_t>> filtered_rows_count,
|
||||||
bool apply_deleted_mask,
|
bool apply_deleted_mask,
|
||||||
std::optional<ActionsDAG> filter,
|
std::optional<ActionsDAG> filter,
|
||||||
|
bool read_with_direct_io,
|
||||||
|
bool prefetch,
|
||||||
ContextPtr context,
|
ContextPtr context,
|
||||||
LoggerPtr log);
|
LoggerPtr log);
|
||||||
|
|
||||||
|
@ -185,7 +185,8 @@ public:
|
|||||||
void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement)
|
void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement)
|
||||||
{
|
{
|
||||||
if (++sent_initial_requests > replicas_count)
|
if (++sent_initial_requests > replicas_count)
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Initiator received more initial requests than there are replicas");
|
throw Exception(
|
||||||
|
ErrorCodes::LOGICAL_ERROR, "Initiator received more initial requests than there are replicas: replica_num={}", announcement.replica_num);
|
||||||
|
|
||||||
doHandleInitialAllRangesAnnouncement(std::move(announcement));
|
doHandleInitialAllRangesAnnouncement(std::move(announcement));
|
||||||
}
|
}
|
||||||
@ -433,9 +434,9 @@ void DefaultCoordinator::setProgressCallback()
|
|||||||
|
|
||||||
void DefaultCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement)
|
void DefaultCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement)
|
||||||
{
|
{
|
||||||
const auto replica_num = announcement.replica_num;
|
LOG_DEBUG(log, "Initial request: {}", announcement.describe());
|
||||||
|
|
||||||
LOG_DEBUG(log, "Initial request from replica {}: {}", announcement.replica_num, announcement.describe());
|
const auto replica_num = announcement.replica_num;
|
||||||
|
|
||||||
initializeReadingState(std::move(announcement));
|
initializeReadingState(std::move(announcement));
|
||||||
|
|
||||||
@ -839,6 +840,7 @@ public:
|
|||||||
|
|
||||||
Parts all_parts_to_read;
|
Parts all_parts_to_read;
|
||||||
size_t total_rows_to_read = 0;
|
size_t total_rows_to_read = 0;
|
||||||
|
bool state_initialized{false};
|
||||||
|
|
||||||
LoggerPtr log = getLogger(fmt::format("{}{}", magic_enum::enum_name(mode), "Coordinator"));
|
LoggerPtr log = getLogger(fmt::format("{}{}", magic_enum::enum_name(mode), "Coordinator"));
|
||||||
};
|
};
|
||||||
@ -858,7 +860,9 @@ void InOrderCoordinator<mode>::markReplicaAsUnavailable(size_t replica_number)
|
|||||||
template <CoordinationMode mode>
|
template <CoordinationMode mode>
|
||||||
void InOrderCoordinator<mode>::doHandleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement)
|
void InOrderCoordinator<mode>::doHandleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement)
|
||||||
{
|
{
|
||||||
LOG_TRACE(log, "Received an announcement {}", announcement.describe());
|
LOG_TRACE(log, "Received an announcement : {}", announcement.describe());
|
||||||
|
|
||||||
|
++stats[announcement.replica_num].number_of_requests;
|
||||||
|
|
||||||
size_t new_rows_to_read = 0;
|
size_t new_rows_to_read = 0;
|
||||||
|
|
||||||
@ -868,13 +872,16 @@ void InOrderCoordinator<mode>::doHandleInitialAllRangesAnnouncement(InitialAllRa
|
|||||||
auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(),
|
auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(),
|
||||||
[&part] (const Part & other) { return other.description.info == part.info; });
|
[&part] (const Part & other) { return other.description.info == part.info; });
|
||||||
|
|
||||||
/// We have the same part - add the info about presence on current replica to it
|
/// We have the same part - add the info about presence on the corresponding replica to it
|
||||||
if (the_same_it != all_parts_to_read.end())
|
if (the_same_it != all_parts_to_read.end())
|
||||||
{
|
{
|
||||||
the_same_it->replicas.insert(announcement.replica_num);
|
the_same_it->replicas.insert(announcement.replica_num);
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (state_initialized)
|
||||||
|
continue;
|
||||||
|
|
||||||
auto covering_or_the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(),
|
auto covering_or_the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(),
|
||||||
[&part] (const Part & other) { return other.description.info.contains(part.info) || part.info.contains(other.description.info); });
|
[&part] (const Part & other) { return other.description.info.contains(part.info) || part.info.contains(other.description.info); });
|
||||||
|
|
||||||
@ -889,9 +896,10 @@ void InOrderCoordinator<mode>::doHandleInitialAllRangesAnnouncement(InitialAllRa
|
|||||||
std::sort(ranges.begin(), ranges.end());
|
std::sort(ranges.begin(), ranges.end());
|
||||||
}
|
}
|
||||||
|
|
||||||
++stats[announcement.replica_num].number_of_requests;
|
state_initialized = true;
|
||||||
|
|
||||||
if (new_rows_to_read > 0)
|
// progress_callback is not set when local plan is used for initiator
|
||||||
|
if (progress_callback && new_rows_to_read > 0)
|
||||||
{
|
{
|
||||||
Progress progress;
|
Progress progress;
|
||||||
progress.total_rows_to_read = new_rows_to_read;
|
progress.total_rows_to_read = new_rows_to_read;
|
||||||
@ -911,7 +919,7 @@ ParallelReadResponse InOrderCoordinator<mode>::handleRequest(ParallelReadRequest
|
|||||||
"Replica {} decided to read in {} mode, not in {}. This is a bug",
|
"Replica {} decided to read in {} mode, not in {}. This is a bug",
|
||||||
request.replica_num, magic_enum::enum_name(request.mode), magic_enum::enum_name(mode));
|
request.replica_num, magic_enum::enum_name(request.mode), magic_enum::enum_name(mode));
|
||||||
|
|
||||||
LOG_TRACE(log, "Got request from replica {}, data {}", request.replica_num, request.describe());
|
LOG_TRACE(log, "Got read request: {}", request.describe());
|
||||||
|
|
||||||
ParallelReadResponse response;
|
ParallelReadResponse response;
|
||||||
response.description = request.description;
|
response.description = request.description;
|
||||||
@ -925,8 +933,15 @@ ParallelReadResponse InOrderCoordinator<mode>::handleRequest(ParallelReadRequest
|
|||||||
if (global_part_it == all_parts_to_read.end())
|
if (global_part_it == all_parts_to_read.end())
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
|
if (global_part_it->replicas.empty())
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::LOGICAL_ERROR,
|
||||||
|
"Part {} requested by replica {} is not registered in working set",
|
||||||
|
part.info.getPartNameV1(),
|
||||||
|
request.replica_num);
|
||||||
|
|
||||||
if (!global_part_it->replicas.contains(request.replica_num))
|
if (!global_part_it->replicas.contains(request.replica_num))
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} doesn't exist on replica {} according to the global state", part.info.getPartNameV1(), request.replica_num);
|
continue;
|
||||||
|
|
||||||
size_t current_mark_size = 0;
|
size_t current_mark_size = 0;
|
||||||
|
|
||||||
@ -1057,6 +1072,7 @@ void ParallelReplicasReadingCoordinator::initialize(CoordinationMode mode)
|
|||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// progress_callback is not set when local plan is used for initiator
|
||||||
if (progress_callback)
|
if (progress_callback)
|
||||||
pimpl->setProgressCallback(std::move(progress_callback));
|
pimpl->setProgressCallback(std::move(progress_callback));
|
||||||
|
|
||||||
|
@ -44,9 +44,7 @@ void ParallelReadRequest::serialize(WriteBuffer & out) const
|
|||||||
|
|
||||||
String ParallelReadRequest::describe() const
|
String ParallelReadRequest::describe() const
|
||||||
{
|
{
|
||||||
String result;
|
String result = fmt::format("replica_num {}, min_num_of_marks {}, ", replica_num, min_number_of_marks);
|
||||||
result += fmt::format("replica_num: {} \n", replica_num);
|
|
||||||
result += fmt::format("min_num_of_marks: {} \n", min_number_of_marks);
|
|
||||||
result += description.describe();
|
result += description.describe();
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
@ -131,10 +129,7 @@ void InitialAllRangesAnnouncement::serialize(WriteBuffer & out) const
|
|||||||
|
|
||||||
String InitialAllRangesAnnouncement::describe()
|
String InitialAllRangesAnnouncement::describe()
|
||||||
{
|
{
|
||||||
String result;
|
return fmt::format("replica {}, mode {}, {}", replica_num, mode, description.describe());
|
||||||
result += description.describe();
|
|
||||||
result += fmt::format("----------\nReceived from {} replica\n", replica_num);
|
|
||||||
return result;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffer & in)
|
InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffer & in)
|
||||||
|
@ -24,6 +24,7 @@ namespace ErrorCodes
|
|||||||
{
|
{
|
||||||
extern const int BAD_ARGUMENTS;
|
extern const int BAD_ARGUMENTS;
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
|
extern const int LOGICAL_ERROR;
|
||||||
}
|
}
|
||||||
|
|
||||||
const std::unordered_set<std::string_view> required_configuration_keys = {
|
const std::unordered_set<std::string_view> required_configuration_keys = {
|
||||||
@ -146,14 +147,13 @@ void StorageAzureConfiguration::fromNamedCollection(const NamedCollection & coll
|
|||||||
|
|
||||||
void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, bool with_structure)
|
void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, bool with_structure)
|
||||||
{
|
{
|
||||||
if (engine_args.size() < 3 || engine_args.size() > (with_structure ? 8 : 7))
|
if (engine_args.size() < 3 || engine_args.size() > getMaxNumberOfArguments(with_structure))
|
||||||
{
|
{
|
||||||
throw Exception(
|
throw Exception(
|
||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||||
"Storage AzureBlobStorage requires 3 to {} arguments: "
|
"Storage AzureBlobStorage requires 1 to {} arguments. All supported signatures:\n{}",
|
||||||
"AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, "
|
getMaxNumberOfArguments(with_structure),
|
||||||
"[account_name, account_key, format, compression, structure)])",
|
getSignatures(with_structure));
|
||||||
(with_structure ? 8 : 7));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
for (auto & engine_arg : engine_args)
|
for (auto & engine_arg : engine_args)
|
||||||
@ -272,26 +272,30 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context,
|
|||||||
connection_params = getConnectionParams(connection_url, container_name, account_name, account_key, context);
|
connection_params = getConnectionParams(connection_url, container_name, account_name, account_key, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
void StorageAzureConfiguration::addStructureAndFormatToArgs(
|
void StorageAzureConfiguration::addStructureAndFormatToArgsIfNeeded(
|
||||||
ASTs & args, const String & structure_, const String & format_, ContextPtr context)
|
ASTs & args, const String & structure_, const String & format_, ContextPtr context)
|
||||||
{
|
{
|
||||||
if (tryGetNamedCollectionWithOverrides(args, context))
|
if (auto collection = tryGetNamedCollectionWithOverrides(args, context))
|
||||||
{
|
{
|
||||||
/// In case of named collection, just add key-value pair "structure='...'"
|
/// In case of named collection, just add key-value pairs "format='...', structure='...'"
|
||||||
/// at the end of arguments to override existed structure.
|
/// at the end of arguments to override existed format and structure with "auto" values.
|
||||||
ASTs equal_func_args = {std::make_shared<ASTIdentifier>("structure"), std::make_shared<ASTLiteral>(structure_)};
|
if (collection->getOrDefault<String>("format", "auto") == "auto")
|
||||||
auto equal_func = makeASTFunction("equals", std::move(equal_func_args));
|
{
|
||||||
args.push_back(equal_func);
|
ASTs format_equal_func_args = {std::make_shared<ASTIdentifier>("format"), std::make_shared<ASTLiteral>(format_)};
|
||||||
|
auto format_equal_func = makeASTFunction("equals", std::move(format_equal_func_args));
|
||||||
|
args.push_back(format_equal_func);
|
||||||
|
}
|
||||||
|
if (collection->getOrDefault<String>("structure", "auto") == "auto")
|
||||||
|
{
|
||||||
|
ASTs structure_equal_func_args = {std::make_shared<ASTIdentifier>("structure"), std::make_shared<ASTLiteral>(structure_)};
|
||||||
|
auto structure_equal_func = makeASTFunction("equals", std::move(structure_equal_func_args));
|
||||||
|
args.push_back(structure_equal_func);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
if (args.size() < 3 || args.size() > 8)
|
if (args.size() < 3 || args.size() > getMaxNumberOfArguments())
|
||||||
{
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 3 to {} arguments in table function azureBlobStorage, got {}", getMaxNumberOfArguments(), args.size());
|
||||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
|
||||||
"Storage Azure requires 3 to 7 arguments: "
|
|
||||||
"StorageObjectStorage(connection_string|storage_account_url, container_name, "
|
|
||||||
"blobpath, [account_name, account_key, format, compression, structure])");
|
|
||||||
}
|
|
||||||
|
|
||||||
for (auto & arg : args)
|
for (auto & arg : args)
|
||||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
||||||
|
@ -22,6 +22,29 @@ public:
|
|||||||
|
|
||||||
static constexpr auto type_name = "azure";
|
static constexpr auto type_name = "azure";
|
||||||
static constexpr auto engine_name = "Azure";
|
static constexpr auto engine_name = "Azure";
|
||||||
|
/// All possible signatures for Azure engine with structure argument (for example for azureBlobStorage table function).
|
||||||
|
static constexpr auto max_number_of_arguments_with_structure = 8;
|
||||||
|
static constexpr auto signatures_with_structure =
|
||||||
|
" - connection_string, container_name, blobpath\n"
|
||||||
|
" - connection_string, container_name, blobpath, structure \n"
|
||||||
|
" - connection_string, container_name, blobpath, format \n"
|
||||||
|
" - connection_string, container_name, blobpath, format, compression \n"
|
||||||
|
" - connection_string, container_name, blobpath, format, compression, structure \n"
|
||||||
|
" - storage_account_url, container_name, blobpath, account_name, account_key\n"
|
||||||
|
" - storage_account_url, container_name, blobpath, account_name, account_key, structure\n"
|
||||||
|
" - storage_account_url, container_name, blobpath, account_name, account_key, format\n"
|
||||||
|
" - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n"
|
||||||
|
" - storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure\n";
|
||||||
|
|
||||||
|
/// All possible signatures for Azure engine without structure argument (for example for AzureBlobStorage table engine).
|
||||||
|
static constexpr auto max_number_of_arguments_without_structure = 7;
|
||||||
|
static constexpr auto signatures_without_structure =
|
||||||
|
" - connection_string, container_name, blobpath\n"
|
||||||
|
" - connection_string, container_name, blobpath, format \n"
|
||||||
|
" - connection_string, container_name, blobpath, format, compression \n"
|
||||||
|
" - storage_account_url, container_name, blobpath, account_name, account_key\n"
|
||||||
|
" - storage_account_url, container_name, blobpath, account_name, account_key, format\n"
|
||||||
|
" - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n";
|
||||||
|
|
||||||
StorageAzureConfiguration() = default;
|
StorageAzureConfiguration() = default;
|
||||||
StorageAzureConfiguration(const StorageAzureConfiguration & other);
|
StorageAzureConfiguration(const StorageAzureConfiguration & other);
|
||||||
@ -29,6 +52,9 @@ public:
|
|||||||
std::string getTypeName() const override { return type_name; }
|
std::string getTypeName() const override { return type_name; }
|
||||||
std::string getEngineName() const override { return engine_name; }
|
std::string getEngineName() const override { return engine_name; }
|
||||||
|
|
||||||
|
std::string getSignatures(bool with_structure = true) const { return with_structure ? signatures_with_structure : signatures_without_structure; }
|
||||||
|
size_t getMaxNumberOfArguments(bool with_structure = true) const { return with_structure ? max_number_of_arguments_with_structure : max_number_of_arguments_without_structure; }
|
||||||
|
|
||||||
Path getPath() const override { return blob_path; }
|
Path getPath() const override { return blob_path; }
|
||||||
void setPath(const Path & path) override { blob_path = path; }
|
void setPath(const Path & path) override { blob_path = path; }
|
||||||
|
|
||||||
@ -44,7 +70,7 @@ public:
|
|||||||
|
|
||||||
ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override;
|
ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override;
|
||||||
|
|
||||||
void addStructureAndFormatToArgs(
|
void addStructureAndFormatToArgsIfNeeded(
|
||||||
ASTs & args,
|
ASTs & args,
|
||||||
const String & structure_,
|
const String & structure_,
|
||||||
const String & format_,
|
const String & format_,
|
||||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user