Merge branch 'master' of github.com:ClickHouse/ClickHouse into backport_mergetask

This commit is contained in:
Alexander Gololobov 2024-09-12 09:31:54 +02:00
commit 8ea2573a9d
115 changed files with 2187 additions and 827 deletions

View File

@ -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}"

View File

@ -124,6 +124,8 @@ function setup_logs_replication
check_logs_credentials || return 0 check_logs_credentials || return 0
__set_connection_args __set_connection_args
echo "My hostname is ${HOSTNAME}"
echo 'Create all configured system logs' echo 'Create all configured system logs'
clickhouse-client --query "SYSTEM FLUSH LOGS" clickhouse-client --query "SYSTEM FLUSH LOGS"
@ -184,7 +186,17 @@ function setup_logs_replication
/^TTL /d /^TTL /d
') ')
echo -e "Creating remote destination table ${table}_${hash} with statement:\n${statement}" >&2 echo -e "Creating remote destination table ${table}_${hash} with statement:" >&2
echo "::group::${table}"
# there's the only way big "$statement" can be printed without causing EAGAIN error
# cat: write error: Resource temporarily unavailable
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 "$statement" | clickhouse-client --database_replicated_initial_query_timeout_sec=10 \ echo "$statement" | clickhouse-client --database_replicated_initial_query_timeout_sec=10 \
--distributed_ddl_task_timeout=30 --distributed_ddl_output_mode=throw_only_active \ --distributed_ddl_task_timeout=30 --distributed_ddl_output_mode=throw_only_active \

View 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).

View File

@ -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.

View File

@ -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**

View File

@ -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();

View File

@ -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);

View File

@ -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,7 +142,14 @@ 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);
cur_data.insert(columns[0], serialization, row_num, 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);
} }
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override

View File

@ -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()

View File

@ -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);
}
} }
{ {

View File

@ -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);

View File

@ -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

View File

@ -3,53 +3,27 @@
#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).
{ /// You can specify a callback to react on changes. The callback typically reloads the configuration, i.e. Server
virtual ~ICgroupsReader() = default; /// 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.).
virtual uint64_t readMemoryUsage() = 0; /// The goal of this is to provide elasticity when the container is scaled-up/scaled-down. The mechanism (polling
/// cgroups) is quite implicit, unfortunately there is currently no better way to communicate memory threshold changes
virtual std::string dumpAllStats() = 0; /// to the database.
};
/// 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
/// 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.).
/// The goal of this is to provide elasticity when the container is scaled-up/scaled-down. The mechanism (polling
/// cgroups) is quite implicit, unfortunately there is currently no better way to communicate memory threshold changes
/// to the database.
#if defined(OS_LINUX) #if defined(OS_LINUX)
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
{ {

View File

@ -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();

View File

@ -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

View File

@ -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);

View File

@ -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
View 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
View 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
};
}

View File

@ -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

View File

@ -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

View File

@ -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_))
{ {
} }

View File

@ -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;

View File

@ -148,7 +148,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;
} }

View File

@ -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

View File

@ -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

View File

@ -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) \

View File

@ -79,6 +79,7 @@ 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"},
} }
}, },
{"24.8", {"24.8",
@ -99,7 +100,10 @@ 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"},
{"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.7", {"24.7",

View File

@ -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())

View File

@ -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

View File

@ -36,7 +36,7 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax)
/// Case when bucket name represented in domain name of S3 URL. /// Case when bucket name represented in domain name of S3 URL.
/// E.g. (https://bucket-name.s3.region.amazonaws.com/key) /// E.g. (https://bucket-name.s3.region.amazonaws.com/key)
/// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#virtual-hosted-style-access /// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#virtual-hosted-style-access
static const RE2 virtual_hosted_style_pattern(R"((.+)\.(s3express[\-a-z0-9]+|s3|cos|obs|oss|eos)([.\-][a-z0-9\-.:]+))"); static const RE2 virtual_hosted_style_pattern(R"(([^.]+)\.(s3express[\-a-z0-9]+|s3|cos|obs|.*oss[^\/]*|eos)([.\-][a-z0-9\-.:]+))");
/// Case when AWS Private Link Interface is being used /// Case when AWS Private Link Interface is being used
/// E.g. (bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com/bucket-name/key) /// E.g. (bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com/bucket-name/key)

View File

@ -204,6 +204,14 @@ TEST(S3UriTest, validPatterns)
ASSERT_EQ("", uri.version_id); ASSERT_EQ("", uri.version_id);
ASSERT_EQ(true, uri.is_virtual_hosted_style); ASSERT_EQ(true, uri.is_virtual_hosted_style);
} }
{
S3::URI uri("https://bucket-test.cn-beijing-internal.oss-data-acc.aliyuncs.com/cc-2zeh496zqm0g6e09g");
ASSERT_EQ("https://cn-beijing-internal.oss-data-acc.aliyuncs.com", uri.endpoint);
ASSERT_EQ("bucket-test", uri.bucket);
ASSERT_EQ("cc-2zeh496zqm0g6e09g", uri.key);
ASSERT_EQ("", uri.version_id);
ASSERT_EQ(true, uri.is_virtual_hosted_style);
}
} }
TEST(S3UriTest, versionIdChecks) TEST(S3UriTest, versionIdChecks)

View File

@ -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,21 +519,147 @@ 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");
} }
auto external_tables = new_context->getExternalTables(); const auto & shard = new_cluster->getShardsInfo().at(0);
auto read_from_remote = std::make_unique<ReadFromParallelRemoteReplicasStep>( size_t max_replicas_to_use = settings.max_parallel_replicas;
query_ast, if (max_replicas_to_use > shard.getAllNodeCount())
new_cluster, {
storage_id, LOG_INFO(
header, getLogger("ReadFromParallelRemoteReplicasStep"),
processed_stage, "The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). "
new_context, "Will use the latter number to execute the query.",
getThrottler(new_context), settings.max_parallel_replicas,
std::move(scalars), shard.getAllNodeCount());
std::move(external_tables), max_replicas_to_use = shard.getAllNodeCount();
getLogger("ReadFromParallelRemoteReplicasStep"), }
std::move(storage_limits));
query_plan.addStep(std::move(read_from_remote)); auto coordinator = std::make_shared<ParallelReplicasReadingCoordinator>(max_replicas_to_use, settings.parallel_replicas_mark_segment_size);
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>(
query_ast,
new_cluster,
storage_id,
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),
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));
}
} }
void executeQueryWithParallelReplicas( void executeQueryWithParallelReplicas(
@ -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;
}
} }
} }

View File

@ -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,

View File

@ -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>

View File

@ -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>>;

View File

@ -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)

View File

@ -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)

View File

@ -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;
}
} }

View File

@ -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);
}; };
} }

View File

@ -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>;
} }

View File

@ -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;

View File

@ -258,7 +258,7 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q
query_context, query_context,
query_, query_,
client_info, client_info,
priorities.insert(static_cast<int>(settings.priority)), priorities.insert(settings.priority),
std::move(thread_group), std::move(thread_group),
query_kind, query_kind,
settings, settings,

View File

@ -31,7 +31,7 @@ namespace DB
class QueryPriorities class QueryPriorities
{ {
public: public:
using Priority = int; using Priority = size_t;
private: private:
friend struct Handle; friend struct Handle;

View File

@ -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); }

View File

@ -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

View File

@ -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:

View File

@ -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.

View File

@ -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_)

View File

@ -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; }

View File

@ -893,31 +893,6 @@ 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))
{ {
// (1) find read step
QueryPlan::Node * node = query_plan.getRootNode();
ReadFromMergeTree * reading = nullptr;
while (node)
{
reading = typeid_cast<ReadFromMergeTree *>(node->step.get());
if (reading)
break;
QueryPlan::Node * prev_node = node;
if (!node->children.empty())
{
chassert(node->children.size() == 1);
node = node->children.at(0);
}
else
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Step is expected to be ReadFromMergeTree but it's {}",
prev_node->step->getName());
}
}
chassert(reading);
if (query_context->canUseParallelReplicasCustomKey() && query_context->getClientInfo().distributed_depth == 0) if (query_context->canUseParallelReplicasCustomKey() && query_context->getClientInfo().distributed_depth == 0)
{ {
if (auto cluster = query_context->getClusterForParallelReplicas(); if (auto cluster = query_context->getClusterForParallelReplicas();
@ -940,21 +915,50 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
query_plan = std::move(query_plan_parallel_replicas); query_plan = std::move(query_plan_parallel_replicas);
} }
} }
else if (query_context->canUseParallelReplicasOnInitiator()) else if (ClusterProxy::canUseParallelReplicasOnInitiator(query_context))
{ {
// (1) find read step
QueryPlan::Node * node = query_plan.getRootNode();
ReadFromMergeTree * reading = nullptr;
while (node)
{
reading = typeid_cast<ReadFromMergeTree *>(node->step.get());
if (reading)
break;
QueryPlan::Node * prev_node = node;
if (!node->children.empty())
{
chassert(node->children.size() == 1);
node = node->children.at(0);
}
else
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Step is expected to be ReadFromMergeTree but it's {}",
prev_node->step->getName());
}
}
chassert(reading);
// (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);
}
} }
} }

View File

@ -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 {};
} }

View 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));
}
}

View File

@ -0,0 +1,9 @@
#pragma once
namespace DB
{
class QueryPlan;
class Block;
void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects);
}

View File

@ -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,

View File

@ -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));

View 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};
}
}

View 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);
}

View File

@ -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,13 +298,21 @@ 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)
{ {
all_ranges_callback = context->getMergeTreeAllRangesCallback(); if (all_ranges_callback_.has_value())
read_task_callback = context->getMergeTreeReadTaskCallback(); all_ranges_callback = all_ranges_callback_.value();
else
all_ranges_callback = context->getMergeTreeAllRangesCallback();
if (read_task_callback_.has_value())
read_task_callback = read_task_callback_.value();
else
read_task_callback = context->getMergeTreeReadTaskCallback();
} }
const auto & settings = context->getSettingsRef(); const auto & settings = context->getSettingsRef();
@ -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.

View File

@ -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;
}; };
} }

View File

@ -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));

View File

@ -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;
}; };
} }

View File

@ -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;
} }

View File

@ -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));

View File

@ -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)

View File

@ -64,9 +64,7 @@ namespace
void checkAndAdjustSettings( void checkAndAdjustSettings(
ObjectStorageQueueSettings & queue_settings, ObjectStorageQueueSettings & queue_settings,
ASTStorage * engine_args, bool is_attach)
bool is_attach,
const LoggerPtr & log)
{ {
if (!is_attach && !queue_settings.mode.changed) if (!is_attach && !queue_settings.mode.changed)
{ {
@ -85,16 +83,6 @@ namespace
"Setting `cleanup_interval_min_ms` ({}) must be less or equal to `cleanup_interval_max_ms` ({})", "Setting `cleanup_interval_min_ms` ({}) must be less or equal to `cleanup_interval_max_ms` ({})",
queue_settings.cleanup_interval_min_ms, queue_settings.cleanup_interval_max_ms); queue_settings.cleanup_interval_min_ms, queue_settings.cleanup_interval_max_ms);
} }
if (!is_attach && !queue_settings.processing_threads_num.changed)
{
queue_settings.processing_threads_num = std::max<uint32_t>(getNumberOfPhysicalCPUCores(), 16);
engine_args->settings->as<ASTSetQuery>()->changes.insertSetting(
"processing_threads_num",
queue_settings.processing_threads_num.value);
LOG_TRACE(log, "Set `processing_threads_num` to {}", queue_settings.processing_threads_num);
}
} }
std::shared_ptr<ObjectStorageQueueLog> getQueueLog(const ObjectStoragePtr & storage, const ContextPtr & context, const ObjectStorageQueueSettings & table_settings) std::shared_ptr<ObjectStorageQueueLog> getQueueLog(const ObjectStoragePtr & storage, const ContextPtr & context, const ObjectStorageQueueSettings & table_settings)
@ -130,7 +118,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue(
const String & comment, const String & comment,
ContextPtr context_, ContextPtr context_,
std::optional<FormatSettings> format_settings_, std::optional<FormatSettings> format_settings_,
ASTStorage * engine_args, ASTStorage * /* engine_args */,
LoadingStrictnessLevel mode) LoadingStrictnessLevel mode)
: IStorage(table_id_) : IStorage(table_id_)
, WithContext(context_) , WithContext(context_)
@ -154,7 +142,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue(
throw Exception(ErrorCodes::BAD_QUERY_PARAMETER, "ObjectStorageQueue url must either end with '/' or contain globs"); throw Exception(ErrorCodes::BAD_QUERY_PARAMETER, "ObjectStorageQueue url must either end with '/' or contain globs");
} }
checkAndAdjustSettings(*queue_settings, engine_args, mode > LoadingStrictnessLevel::CREATE, log); checkAndAdjustSettings(*queue_settings, mode > LoadingStrictnessLevel::CREATE);
object_storage = configuration->createObjectStorage(context_, /* is_readonly */true); object_storage = configuration->createObjectStorage(context_, /* is_readonly */true);
FormatFactory::instance().checkFormatName(configuration->format); FormatFactory::instance().checkFormatName(configuration->format);

View File

@ -63,7 +63,6 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context
/// current setting values, one needs to ask the components directly. /// current setting values, one needs to ask the components directly.
std::unordered_map<String, std::pair<String, ChangeableWithoutRestart>> changeable_settings = { std::unordered_map<String, std::pair<String, ChangeableWithoutRestart>> changeable_settings = {
{"max_server_memory_usage", {std::to_string(total_memory_tracker.getHardLimit()), ChangeableWithoutRestart::Yes}}, {"max_server_memory_usage", {std::to_string(total_memory_tracker.getHardLimit()), ChangeableWithoutRestart::Yes}},
{"allow_use_jemalloc_memory", {std::to_string(total_memory_tracker.getAllowUseJemallocMmemory()), ChangeableWithoutRestart::Yes}},
{"max_table_size_to_drop", {std::to_string(context->getMaxTableSizeToDrop()), ChangeableWithoutRestart::Yes}}, {"max_table_size_to_drop", {std::to_string(context->getMaxTableSizeToDrop()), ChangeableWithoutRestart::Yes}},
{"max_partition_size_to_drop", {std::to_string(context->getMaxPartitionSizeToDrop()), ChangeableWithoutRestart::Yes}}, {"max_partition_size_to_drop", {std::to_string(context->getMaxPartitionSizeToDrop()), ChangeableWithoutRestart::Yes}},

View File

@ -473,7 +473,7 @@ class CI:
JobNames.STATELESS_TEST_FLAKY_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( JobNames.STATELESS_TEST_FLAKY_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_ASAN], required_builds=[BuildNames.PACKAGE_ASAN],
pr_only=True, pr_only=True,
timeout=3600, timeout=3 * 3600,
# TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf
# reference_job_name=JobNames.STATELESS_TEST_RELEASE, # reference_job_name=JobNames.STATELESS_TEST_RELEASE,
), ),

View File

@ -9,6 +9,7 @@ from urllib.parse import quote
from unidiff import PatchSet # type: ignore from unidiff import PatchSet # type: ignore
from build_download_helper import get_gh_api from build_download_helper import get_gh_api
from ci_config import Labels
from env_helper import ( from env_helper import (
GITHUB_EVENT_PATH, GITHUB_EVENT_PATH,
GITHUB_REPOSITORY, GITHUB_REPOSITORY,
@ -16,7 +17,6 @@ from env_helper import (
GITHUB_SERVER_URL, GITHUB_SERVER_URL,
GITHUB_UPSTREAM_REPOSITORY, GITHUB_UPSTREAM_REPOSITORY,
) )
from ci_config import Labels
from get_robot_token import get_best_robot_token from get_robot_token import get_best_robot_token
from github_helper import GitHub from github_helper import GitHub
@ -459,16 +459,18 @@ class PRInfo:
sync_repo = gh.get_repo(GITHUB_REPOSITORY) sync_repo = gh.get_repo(GITHUB_REPOSITORY)
sync_pr = sync_repo.get_pull(self.number) sync_pr = sync_repo.get_pull(self.number)
# Find the commit that is in both repos, upstream and cloud # Find the commit that is in both repos, upstream and cloud
sync_commits = sync_pr.get_commits().reversed # Do not ever use `reversed` here, otherwise the list of commits is not full
upstream_commits = upstream_pr.get_commits().reversed sync_commits = list(sync_pr.get_commits())
upstream_commits = list(upstream_pr.get_commits())
# Github objects are compared by _url attribute. We can't compare them directly and # Github objects are compared by _url attribute. We can't compare them directly and
# should compare commits by SHA1 # should compare commits by SHA1
upstream_shas = [c.sha for c in upstream_commits] upstream_shas = [c.sha for c in upstream_commits]
logging.info("Commits in upstream PR:\n %s", ", ".join(upstream_shas)) logging.info("Commits in upstream PR:\n %s", ", ".join(upstream_shas))
sync_shas = [c.sha for c in sync_commits] sync_shas = [c.sha for c in sync_commits]
logging.info("Commits in sync PR:\n %s", ", ".join(reversed(sync_shas))) logging.info("Commits in sync PR:\n %s", ", ".join(sync_shas))
# find latest synced commit # find latest synced commit, search from the latest
upstream_commits.reverse()
last_synced_upstream_commit = None last_synced_upstream_commit = None
for commit in upstream_commits: for commit in upstream_commits:
if commit.sha in sync_shas: if commit.sha in sync_shas:

View File

@ -916,6 +916,7 @@ class SettingsRandomizer:
"min_external_table_block_size_bytes": lambda: random.choice([0, 1, 100000000]), "min_external_table_block_size_bytes": lambda: random.choice([0, 1, 100000000]),
"max_parsing_threads": lambda: random.choice([0, 1, 10]), "max_parsing_threads": lambda: random.choice([0, 1, 10]),
"optimize_functions_to_subcolumns": lambda: random.randint(0, 1), "optimize_functions_to_subcolumns": lambda: random.randint(0, 1),
"parallel_replicas_local_plan": lambda: random.randint(0, 1),
} }
@staticmethod @staticmethod

View File

@ -115,10 +115,6 @@
<parallel_replicas> <parallel_replicas>
<shard> <shard>
<internal_replication>false</internal_replication> <internal_replication>false</internal_replication>
<replica>
<host>127.0.0.1</host>
<port>9000</port>
</replica>
<replica> <replica>
<host>127.0.0.2</host> <host>127.0.0.2</host>
<port>9000</port> <port>9000</port>
@ -147,6 +143,10 @@
<host>127.0.0.8</host> <host>127.0.0.8</host>
<port>9000</port> <port>9000</port>
</replica> </replica>
<replica>
<host>127.0.0.1</host>
<port>9000</port>
</replica>
<replica> <replica>
<host>127.0.0.9</host> <host>127.0.0.9</host>
<port>9000</port> <port>9000</port>

View File

@ -13,8 +13,20 @@ def started_cluster():
main_configs=["config.xml"], main_configs=["config.xml"],
with_minio=True, with_minio=True,
) )
cluster.start() cluster.start()
# local disk requires its `path` directory to exist.
# the two paths below belong to `test1` and `test2` disks
node = cluster.instances["disks_app_test"]
for path in ["path1", "path2"]:
node.exec_in_container(
[
"bash",
"-c",
f"mkdir -p /var/lib/clickhouse/{path}",
]
)
yield cluster yield cluster
finally: finally:

View File

@ -16,7 +16,7 @@
<value>az-zoo2</value> <value>az-zoo2</value>
<enable_auto_detection_on_cloud>1</enable_auto_detection_on_cloud> <enable_auto_detection_on_cloud>1</enable_auto_detection_on_cloud>
</availability_zone> </availability_zone>
<max_memory_usage_soft_limit>20000000</max_memory_usage_soft_limit> <max_memory_usage_soft_limit>200000000</max_memory_usage_soft_limit>
<coordination_settings> <coordination_settings>
<operation_timeout_ms>10000</operation_timeout_ms> <operation_timeout_ms>10000</operation_timeout_ms>

View File

@ -13,7 +13,7 @@
<tcp_port>2181</tcp_port> <tcp_port>2181</tcp_port>
<server_id>3</server_id> <server_id>3</server_id>
<max_memory_usage_soft_limit>20000000</max_memory_usage_soft_limit> <max_memory_usage_soft_limit>200000000</max_memory_usage_soft_limit>
<coordination_settings> <coordination_settings>
<operation_timeout_ms>10000</operation_timeout_ms> <operation_timeout_ms>10000</operation_timeout_ms>

View File

@ -13,7 +13,6 @@ node = cluster.add_instance(
"configs/async_metrics_no.xml", "configs/async_metrics_no.xml",
], ],
mem_limit="4g", mem_limit="4g",
env_variables={"MALLOC_CONF": "dirty_decay_ms:0"},
) )

View File

@ -49,5 +49,6 @@ def test_skip_all_replicas(
"max_parallel_replicas": max_parallel_replicas, "max_parallel_replicas": max_parallel_replicas,
"cluster_for_parallel_replicas": cluster_name, "cluster_for_parallel_replicas": cluster_name,
"skip_unavailable_shards": skip_unavailable_shards, "skip_unavailable_shards": skip_unavailable_shards,
"parallel_replicas_local_plan": 0,
}, },
) )

View File

@ -100,7 +100,7 @@ select a, count() from dist_t_different_dbs group by a, b order by a limit 5 off
504 2000 504 2000
1000000 1000000
-- { echoOn } -- -- { echoOn } --
explain pipeline select a from pr_t group by a order by a limit 5 offset 500; explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings parallel_replicas_local_plan=0;
(Expression) (Expression)
ExpressionTransform ExpressionTransform
(Limit) (Limit)
@ -119,6 +119,31 @@ ExpressionTransform
Resize 1 → 4 Resize 1 → 4
GroupingAggregatedTransform 3 → 1 GroupingAggregatedTransform 3 → 1
(ReadFromRemoteParallelReplicas) (ReadFromRemoteParallelReplicas)
explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=1, parallel_replicas_local_plan=1;
(Expression)
ExpressionTransform
(Limit)
Limit
(Sorting)
MergingSortedTransform 4 → 1
MergeSortingTransform × 4
LimitsCheckingTransform × 4
PartialSortingTransform × 4
(Expression)
ExpressionTransform × 4
(MergingAggregated)
MergingAggregatedBucketTransform × 4
Resize 1 → 4
FinishAggregatingInOrderTransform 3 → 1
(Union)
(Aggregating)
FinalizeAggregatedTransform
AggregatingInOrderTransform
(Expression)
ExpressionTransform
(ReadFromMergeTree)
MergeTreeSelect(pool: ReadPoolParallelReplicasInOrder, algorithm: InOrder) 0 → 1
(ReadFromRemoteParallelReplicas)
select a, count() from pr_t group by a order by a limit 5 offset 500; select a, count() from pr_t group by a order by a limit 5 offset 500;
500 1000 500 1000
501 1000 501 1000

View File

@ -66,14 +66,14 @@ insert into pr_t select number % 1000, number % 1000 from numbers_mt(1e6);
set allow_experimental_parallel_reading_from_replicas = 1; set allow_experimental_parallel_reading_from_replicas = 1;
set parallel_replicas_for_non_replicated_merge_tree = 1; set parallel_replicas_for_non_replicated_merge_tree = 1;
set max_parallel_replicas = 3; set max_parallel_replicas = 3;
set use_hedged_requests = 0;
set cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; set cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost';
set distributed_aggregation_memory_efficient=1; set distributed_aggregation_memory_efficient=1;
select count() from pr_t; select count() from pr_t;
-- { echoOn } -- -- { echoOn } --
explain pipeline select a from pr_t group by a order by a limit 5 offset 500; explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings parallel_replicas_local_plan=0;
explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=1, parallel_replicas_local_plan=1;
select a, count() from pr_t group by a order by a limit 5 offset 500; select a, count() from pr_t group by a order by a limit 5 offset 500;
select a, count() from pr_t group by a, b order by a limit 5 offset 500; select a, count() from pr_t group by a, b order by a limit 5 offset 500;

View File

@ -31,7 +31,7 @@
29 2j&S)ba?XG QuQj 17163829389637435056 29 2j&S)ba?XG QuQj 17163829389637435056
3 UlI+1 14144472852965836438 3 UlI+1 14144472852965836438
=============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE =============== =============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE ===============
0 3 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\') 0 2 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\')
0 3 SELECT `key`, `value1`, `value2`, toUInt64(min(`time`)) AS `start_ts` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` ORDER BY `key` ASC, `value1` ASC, `value2` ASC LIMIT 10 0 3 SELECT `key`, `value1`, `value2`, toUInt64(min(`time`)) AS `start_ts` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` ORDER BY `key` ASC, `value1` ASC, `value2` ASC LIMIT 10
1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0; 1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0;
1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=1; 1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=1;
@ -58,7 +58,7 @@ U c 10
UlI+1 10 UlI+1 10
bX?}ix [ Ny]2 G 10 bX?}ix [ Ny]2 G 10
t<iT X48q:Z]t0 10 t<iT X48q:Z]t0 10
0 3 SELECT `__table2`.`value1` AS `value1`, `__table2`.`value2` AS `value2`, count() AS `count` FROM `default`.`join_outer_table` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`key` AS `key`, `__table3`.`value1` AS `value1`, `__table3`.`value2` AS `value2` FROM `default`.`join_inner_table` AS `__table3` PREWHERE (`__table3`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table3`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table3`.`key`, `__table3`.`value1`, `__table3`.`value2`) AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2` 0 2 SELECT `__table2`.`value1` AS `value1`, `__table2`.`value2` AS `value2`, count() AS `count` FROM `default`.`join_outer_table` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`key` AS `key`, `__table3`.`value1` AS `value1`, `__table3`.`value2` AS `value2` FROM `default`.`join_inner_table` AS `__table3` PREWHERE (`__table3`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table3`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table3`.`key`, `__table3`.`value1`, `__table3`.`value2`) AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2`
0 3 SELECT `key`, `value1`, `value2` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` 0 3 SELECT `key`, `value1`, `value2` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2`
0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_` USING (`key`) GROUP BY `key`, `value1`, `value2` 0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_` USING (`key`) GROUP BY `key`, `value1`, `value2`
1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0; 1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0;

View File

@ -22,8 +22,8 @@ SELECT
LIMIT 100; LIMIT 100;
SET max_parallel_replicas = 3; SET max_parallel_replicas = 3;
SET prefer_localhost_replica = 1;
SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost';
SET parallel_replicas_local_plan = 1;
SET joined_subquery_requires_alias = 0; SET joined_subquery_requires_alias = 0;
SELECT '=============== INNER QUERY (NO PARALLEL) ==============='; SELECT '=============== INNER QUERY (NO PARALLEL) ===============';

View File

@ -4,7 +4,9 @@ INSERT INTO test_parallel_replicas_unavailable_shards SELECT * FROM numbers(10);
SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=11, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1; SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=11, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1;
SET send_logs_level='error'; SET send_logs_level='error';
SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79'; -- with local plan for initiator, the query can be executed fast on initator, we can simply not come to the point where unavailable replica can be detected
-- therefore disable local plan for now
SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79', parallel_replicas_local_plan=0;
SYSTEM FLUSH LOGS; SYSTEM FLUSH LOGS;

View File

@ -9,4 +9,4 @@
7885388429666205427 7885388429666205427
8124171311239967992 8124171311239967992
1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n enable_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n allow_experimental_parallel_reading_from_replicas = 1; 1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n enable_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n allow_experimental_parallel_reading_from_replicas = 1;
0 2 SELECT `__table1`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` AS `__table1` 0 1 SELECT `__table1`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` AS `__table1`

View File

@ -1,5 +1,5 @@
-- Tags: zookeeper -- Tags: zookeeper
DROP TABLE IF EXISTS join_inner_table__fuzz_146_replicated;
CREATE TABLE join_inner_table__fuzz_146_replicated CREATE TABLE join_inner_table__fuzz_146_replicated
( (
`id` UUID, `id` UUID,
@ -17,6 +17,8 @@ INSERT INTO join_inner_table__fuzz_146_replicated
SELECT CAST('833c9e22-c245-4eb5-8745-117a9a1f26b1', 'UUID') AS id, CAST(rowNumberInAllBlocks(), 'String') AS key, * SELECT CAST('833c9e22-c245-4eb5-8745-117a9a1f26b1', 'UUID') AS id, CAST(rowNumberInAllBlocks(), 'String') AS key, *
FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) LIMIT 10; FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) LIMIT 10;
SET parallel_replicas_local_plan = 1;
-- Simple query with analyzer and pure parallel replicas -- Simple query with analyzer and pure parallel replicas
SELECT number SELECT number
FROM join_inner_table__fuzz_146_replicated FROM join_inner_table__fuzz_146_replicated
@ -49,3 +51,5 @@ WHERE
) )
GROUP BY is_initial_query, query GROUP BY is_initial_query, query
ORDER BY is_initial_query DESC, c, query; ORDER BY is_initial_query DESC, c, query;
DROP TABLE join_inner_table__fuzz_146_replicated;

View File

@ -6,16 +6,17 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
CLICKHOUSE_CLIENT_TRACE=${CLICKHOUSE_CLIENT/"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"/"--send_logs_level=trace"} CLICKHOUSE_CLIENT_TRACE=${CLICKHOUSE_CLIENT/"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"/"--send_logs_level=trace"}
function were_parallel_replicas_used () { function were_parallel_replicas_used () {
# Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it
$CLICKHOUSE_CLIENT --query " $CLICKHOUSE_CLIENT --query "
SELECT SELECT
initial_query_id, initial_query_id,
concat('Used parallel replicas: ', (countIf(initial_query_id != query_id) != 0)::bool::String) as used concat('Used parallel replicas: ', (ProfileEvents['ParallelReplicasUsedCount'] > 0)::bool::String) as used
FROM system.query_log FROM system.query_log
WHERE event_date >= yesterday() WHERE event_date >= yesterday()
AND initial_query_id LIKE '$1%' AND initial_query_id LIKE '$1%'
GROUP BY initial_query_id AND query_id = initial_query_id
ORDER BY min(event_time_microseconds) ASC AND type = 'QueryFinish'
AND current_database = '$CLICKHOUSE_DATABASE'
ORDER BY event_time_microseconds ASC
FORMAT TSV" FORMAT TSV"
} }
@ -48,11 +49,11 @@ function run_query_with_pure_parallel_replicas () {
--query "$3" \ --query "$3" \
--query_id "${1}_pure" \ --query_id "${1}_pure" \
--max_parallel_replicas 3 \ --max_parallel_replicas 3 \
--prefer_localhost_replica 1 \
--cluster_for_parallel_replicas "parallel_replicas" \ --cluster_for_parallel_replicas "parallel_replicas" \
--allow_experimental_parallel_reading_from_replicas 1 \ --allow_experimental_parallel_reading_from_replicas 1 \
--parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \
--parallel_replicas_min_number_of_rows_per_replica "$2" \ --parallel_replicas_min_number_of_rows_per_replica "$2" \
--max_threads 5 \
|& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }'
} }

View File

@ -4,8 +4,8 @@
02784_automatic_parallel_replicas_join-default_simple_join_1M_pure 10 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_1M_pure 10 estimated parallel replicas
02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 3 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 3 estimated parallel replicas
02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 33 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 33 estimated parallel replicas
02784_automatic_parallel_replicas_join-default_simple_join_0_pure Distinct parallel subqueries: 2 Used parallel replicas: true 02784_automatic_parallel_replicas_join-default_simple_join_0_pure Used parallel replicas: true
02784_automatic_parallel_replicas_join-default_simple_join_10M_pure Distinct parallel subqueries: 0 Used parallel replicas: false 02784_automatic_parallel_replicas_join-default_simple_join_10M_pure Used parallel replicas: false
02784_automatic_parallel_replicas_join-default_simple_join_5M_pure Distinct parallel subqueries: 1 Used parallel replicas: true 02784_automatic_parallel_replicas_join-default_simple_join_5M_pure Used parallel replicas: true
02784_automatic_parallel_replicas_join-default_simple_join_1M_pure Distinct parallel subqueries: 1 Used parallel replicas: true 02784_automatic_parallel_replicas_join-default_simple_join_1M_pure Used parallel replicas: true
02784_automatic_parallel_replicas_join-default_simple_join_300k_pure Distinct parallel subqueries: 2 Used parallel replicas: true 02784_automatic_parallel_replicas_join-default_simple_join_300k_pure Used parallel replicas: true

View File

@ -9,17 +9,17 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
CLICKHOUSE_CLIENT_TRACE=${CLICKHOUSE_CLIENT/"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"/"--send_logs_level=trace"} CLICKHOUSE_CLIENT_TRACE=${CLICKHOUSE_CLIENT/"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"/"--send_logs_level=trace"}
function were_parallel_replicas_used () { function were_parallel_replicas_used () {
# Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it
$CLICKHOUSE_CLIENT --query " $CLICKHOUSE_CLIENT --query "
SELECT SELECT
initial_query_id, initial_query_id,
concat('Distinct parallel subqueries: ' , countDistinctIf(query, initial_query_id != query_id)::String) as subqueries_parallelized, concat('Used parallel replicas: ', (ProfileEvents['ParallelReplicasUsedCount'] > 0)::bool::String) as used
concat('Used parallel replicas: ', (countIf(initial_query_id != query_id) != 0)::bool::String) as used
FROM system.query_log FROM system.query_log
WHERE event_date >= yesterday() WHERE event_date >= yesterday()
AND initial_query_id LIKE '$1%' AND initial_query_id LIKE '$1%'
GROUP BY initial_query_id AND query_id = initial_query_id
ORDER BY min(event_time_microseconds) ASC AND type = 'QueryFinish'
AND current_database = '$CLICKHOUSE_DATABASE'
ORDER BY event_time_microseconds ASC
FORMAT TSV" FORMAT TSV"
} }

View File

@ -27,6 +27,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE replicas_summary (n Int64) ENGINE = Mer
query_id_base="02841_summary_$CLICKHOUSE_DATABASE" query_id_base="02841_summary_$CLICKHOUSE_DATABASE"
# TODO: rethink the test, for now temporary disable parallel_replicas_local_plan
echo " echo "
SELECT * SELECT *
FROM replicas_summary FROM replicas_summary
@ -36,7 +37,8 @@ echo "
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost',
allow_experimental_parallel_reading_from_replicas = 2, allow_experimental_parallel_reading_from_replicas = 2,
parallel_replicas_for_non_replicated_merge_tree = 1, parallel_replicas_for_non_replicated_merge_tree = 1,
interactive_delay=0 interactive_delay=0,
parallel_replicas_local_plan=0
"\ "\
| ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_0" --data-binary @- -vvv 2>&1 \ | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_0" --data-binary @- -vvv 2>&1 \
| grep "Summary" | grep -cv '"read_rows":"0"' | grep "Summary" | grep -cv '"read_rows":"0"'
@ -50,7 +52,8 @@ echo "
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost',
allow_experimental_parallel_reading_from_replicas = 2, allow_experimental_parallel_reading_from_replicas = 2,
parallel_replicas_for_non_replicated_merge_tree = 1, parallel_replicas_for_non_replicated_merge_tree = 1,
interactive_delay=99999999999 interactive_delay=99999999999,
parallel_replicas_local_plan=0
"\ "\
| ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_high" --data-binary @- -vvv 2>&1 \ | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_high" --data-binary @- -vvv 2>&1 \
| grep "Summary" | grep -cv '"read_rows":"0"' | grep "Summary" | grep -cv '"read_rows":"0"'

View File

@ -26,7 +26,8 @@ WHERE query_id in (select query_id from system.query_log where current_database
AND message LIKE '%Total rows to read: 3000%' SETTINGS allow_experimental_parallel_reading_from_replicas=0; AND message LIKE '%Total rows to read: 3000%' SETTINGS allow_experimental_parallel_reading_from_replicas=0;
-- reading in order coordinator -- reading in order coordinator
SELECT k, sipHash64(v) FROM t1 order by k limit 5 offset 998 SETTINGS optimize_read_in_order=1, log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b'; -- disable parallel_replicas_local_plan since the test relay on traces which only present in case of no local plan
SELECT k, sipHash64(v) FROM t1 order by k limit 5 offset 998 SETTINGS optimize_read_in_order=1, parallel_replicas_local_plan=0, log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b';
SYSTEM FLUSH LOGS; SYSTEM FLUSH LOGS;
SELECT count() > 0 FROM system.text_log SELECT count() > 0 FROM system.text_log

View File

@ -13,5 +13,6 @@ Td1EnWQo [1,2,3,4]
XMbT XMbT
-- invalid sqid -- invalid sqid
[] []
-- bug 69450
-- alias -- alias
XMbT XMbT

View File

@ -25,5 +25,12 @@ SELECT sqidEncode(toNullable(materialize(1)), toLowCardinality(materialize(2)));
SELECT '-- invalid sqid'; SELECT '-- invalid sqid';
SELECT sqidDecode('invalid sqid'); SELECT sqidDecode('invalid sqid');
SELECT '-- bug 69450';
DROP TABLE IF EXISTS tab;
CREATE TABLE tab (id String) ENGINE = MergeTree ORDER BY id;
INSERT INTO tab SELECT * FROM generateRandom() LIMIT 1000000;
SELECT sqidDecode(id) FROM tab FORMAT Null;
DROP TABLE tab;
SELECT '-- alias'; SELECT '-- alias';
SELECT sqid(1, 2); SELECT sqid(1, 2);

View File

@ -1,4 +1,4 @@
100 4950 10000 49995000
1 1
89 89
90 90
@ -6,3 +6,9 @@
92 92
93 93
1 1
93
92
91
90
89
1

View File

@ -2,11 +2,12 @@ DROP TABLE IF EXISTS test;
CREATE TABLE test (k UInt64, v String) CREATE TABLE test (k UInt64, v String)
ENGINE = MergeTree ENGINE = MergeTree
ORDER BY k; ORDER BY k
SETTINGS index_granularity=1;
INSERT INTO test SELECT number, toString(number) FROM numbers(100); INSERT INTO test SELECT number, toString(number) FROM numbers(10_000);
SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost';
-- default coordinator -- default coordinator
SELECT count(), sum(k) SELECT count(), sum(k)
@ -14,12 +15,18 @@ FROM test
SETTINGS log_comment = '02950_parallel_replicas_used_replicas_count'; SETTINGS log_comment = '02950_parallel_replicas_used_replicas_count';
SYSTEM FLUSH LOGS; SYSTEM FLUSH LOGS;
SELECT ProfileEvents['ParallelReplicasUsedCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0;
-- In order coordinator -- In order coordinator
SELECT k FROM test order by k limit 5 offset 89 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_2'; SELECT k FROM test order by k limit 5 offset 89 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_2', merge_tree_min_rows_for_concurrent_read=1, max_threads=1;
SYSTEM FLUSH LOGS; SYSTEM FLUSH LOGS;
SELECT ProfileEvents['ParallelReplicasUsedCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_2' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_2' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0;
-- In reverse order coordinator
SELECT k FROM test order by k desc limit 5 offset 9906 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_3', merge_tree_min_rows_for_concurrent_read=1, max_threads=1;
SYSTEM FLUSH LOGS;
SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_3' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0;
DROP TABLE test; DROP TABLE test;

View File

@ -11,10 +11,8 @@ nested join with analyzer and parallel replicas, both global
420336 420336 420336 -140112 420336 420336 420336 -140112
420378 420378 420378 -140126 420378 420378 420378 -140126
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
<Debug> DefaultCoordinator: Coordination done <Debug> DefaultCoordinator: Coordination done
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
<Debug> DefaultCoordinator: Coordination done <Debug> DefaultCoordinator: Coordination done
nested join with analyzer and parallel replicas, global + local nested join with analyzer and parallel replicas, global + local
@ -29,10 +27,8 @@ nested join with analyzer and parallel replicas, global + local
420336 420336 420336 -140112 420336 420336 420336 -140112
420378 420378 420378 -140126 420378 420378 420378 -140126
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table3`) AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table3`) AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table3`) AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
<Debug> DefaultCoordinator: Coordination done <Debug> DefaultCoordinator: Coordination done
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
<Debug> DefaultCoordinator: Coordination done <Debug> DefaultCoordinator: Coordination done
nested join with analyzer and parallel replicas, both local, both full sorting merge join nested join with analyzer and parallel replicas, both local, both full sorting merge join
@ -47,10 +43,8 @@ nested join with analyzer and parallel replicas, both local, both full sorting m
420336 420336 420336 -140112 420336 420336 420336 -140112
420378 420378 420378 -140126 420378 420378 420378 -140126
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
<Debug> WithOrderCoordinator: Coordination done <Debug> WithOrderCoordinator: Coordination done
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
<Debug> WithOrderCoordinator: Coordination done <Debug> WithOrderCoordinator: Coordination done
nested join with analyzer and parallel replicas, both local, both full sorting and hash join nested join with analyzer and parallel replicas, both local, both full sorting and hash join
@ -65,10 +59,8 @@ nested join with analyzer and parallel replicas, both local, both full sorting a
420336 420336 420336 -140112 420336 420336 420336 -140112
420378 420378 420378 -140126 420378 420378 420378 -140126
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
<Debug> DefaultCoordinator: Coordination done <Debug> DefaultCoordinator: Coordination done
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
<Debug> WithOrderCoordinator: Coordination done <Debug> WithOrderCoordinator: Coordination done
nested join with analyzer and parallel replicas, both local, both full sorting and hash join nested join with analyzer and parallel replicas, both local, both full sorting and hash join
@ -83,8 +75,6 @@ nested join with analyzer and parallel replicas, both local, both full sorting a
420336 420336 420336 -140112 420336 420336 420336 -140112
420378 420378 420378 -140126 420378 420378 420378 -140126
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
<Debug> WithOrderCoordinator: Coordination done <Debug> WithOrderCoordinator: Coordination done
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
<Debug> DefaultCoordinator: Coordination done <Debug> DefaultCoordinator: Coordination done

View File

@ -17,6 +17,8 @@ insert into num_1 select number * 2, toString(number * 2) from numbers(1e7);
insert into num_2 select number * 3, -number from numbers(1.5e6); insert into num_2 select number * 3, -number from numbers(1.5e6);
" "
PARALLEL_REPLICAS_SETTINGS="allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_local_plan=1"
############## ##############
echo echo
echo "nested join with analyzer and parallel replicas, both global" echo "nested join with analyzer and parallel replicas, both global"
@ -26,18 +28,14 @@ select * from (select key, value from num_1) l
inner join (select key, value from num_2 inner join inner join (select key, value from num_2 inner join
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r
on l.key = r.key order by l.key limit 10 offset 10000 on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS enable_analyzer=1, SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0"
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0"
$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q "
select * from (select key, value from num_1) l select * from (select key, value from num_1) l
inner join (select key, value from num_2 inner join inner join (select key, value from num_2 inner join
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r
on l.key = r.key order by l.key limit 10 offset 10000 on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS enable_analyzer=1, send_logs_level='trace', SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, send_logs_level='trace', parallel_replicas_prefer_local_join=0" 2>&1 |
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 |
grep "executeQuery\|<Debug>.*Coordinator: Coordination done" | grep "executeQuery\|<Debug>.*Coordinator: Coordination done" |
grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
@ -51,18 +49,14 @@ select * from (select key, value from num_1) l
inner join (select key, value from num_2 inner join inner join (select key, value from num_2 inner join
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r
on l.key = r.key order by l.key limit 10 offset 10000 on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS enable_analyzer=1, SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0"
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0"
$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q "
select * from (select key, value from num_1) l select * from (select key, value from num_1) l
inner join (select key, value from num_2 inner join inner join (select key, value from num_2 inner join
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r
on l.key = r.key order by l.key limit 10 offset 10000 on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS enable_analyzer=1, send_logs_level='trace', SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, send_logs_level='trace', parallel_replicas_prefer_local_join=0" 2>&1 |
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 |
grep "executeQuery\|<Debug>.*Coordinator: Coordination done" | grep "executeQuery\|<Debug>.*Coordinator: Coordination done" |
grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
@ -77,18 +71,14 @@ select * from (select key, value from num_1) l
inner join (select key, value from num_2 inner join inner join (select key, value from num_2 inner join
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r
on l.key = r.key order by l.key limit 10 offset 10000 on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'"
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'"
$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q "
select * from (select key, value from num_1) l select * from (select key, value from num_1) l
inner join (select key, value from num_2 inner join inner join (select key, value from num_2 inner join
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r
on l.key = r.key order by l.key limit 10 offset 10000 on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='full_sorting_merge'" 2>&1 |
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 |
grep "executeQuery\|<Debug>.*Coordinator: Coordination done" | grep "executeQuery\|<Debug>.*Coordinator: Coordination done" |
grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
@ -102,18 +92,14 @@ select * from (select key, value from num_1) l
inner join (select key, value from num_2 inner join inner join (select key, value from num_2 inner join
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r
on l.key = r.key order by l.key limit 10 offset 10000 on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'"
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'"
$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q "
select * from (select key, value from num_1) l select * from (select key, value from num_1) l
inner join (select key, value from num_2 inner join inner join (select key, value from num_2 inner join
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r
on l.key = r.key order by l.key limit 10 offset 10000 on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='full_sorting_merge'" 2>&1 |
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 |
grep "executeQuery\|<Debug>.*Coordinator: Coordination done" | grep "executeQuery\|<Debug>.*Coordinator: Coordination done" |
grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
@ -127,18 +113,14 @@ select * from (select key, value from num_1) l
inner join (select key, value from num_2 inner join inner join (select key, value from num_2 inner join
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r
on l.key = r.key order by l.key limit 10 offset 10000 on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='hash'"
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'"
$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q "
select * from (select key, value from num_1) l select * from (select key, value from num_1) l
inner join (select key, value from num_2 inner join inner join (select key, value from num_2 inner join
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r
on l.key = r.key order by l.key limit 10 offset 10000 on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='hash'" 2>&1 |
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" 2>&1 |
grep "executeQuery\|<Debug>.*Coordinator: Coordination done" | grep "executeQuery\|<Debug>.*Coordinator: Coordination done" |
grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'

View File

@ -19,8 +19,20 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x
14 14 14 14 0 0 14 14 14 14 0 0
15 15 0 0 0 0 15 15 0 0 0 0
explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Union
ReadFromRemoteParallelReplicas Expression
Join
Expression
Join
Expression
Expression
ReadFromMergeTree
Expression
ReadFromMemoryStorage
Expression
ReadFromMemoryStorage
Expression
ReadFromRemoteParallelReplicas
-- --
-- The same query with cte; -- The same query with cte;
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -55,8 +67,22 @@ select * from sub5 order by x
SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Expression
Sorting Sorting
Expression Union
ReadFromRemoteParallelReplicas Expression
Sorting
Expression
Join
Expression
Join
Expression
Expression
ReadFromMergeTree
Expression
ReadFromMemoryStorage
Expression
ReadFromMemoryStorage
Expression
ReadFromRemoteParallelReplicas
-- --
-- GROUP BY should work up to WithMergableStage -- GROUP BY should work up to WithMergableStage
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -80,8 +106,22 @@ Expression
Sorting Sorting
Expression Expression
MergingAggregated MergingAggregated
Expression Union
ReadFromRemoteParallelReplicas Expression
Aggregating
Expression
Join
Expression
Join
Expression
Expression
ReadFromMergeTree
Expression
ReadFromMemoryStorage
Expression
ReadFromMemoryStorage
Expression
ReadFromRemoteParallelReplicas
-- --
-- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage -- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -118,10 +158,22 @@ Expression
Sorting Sorting
Expression Expression
Join Join
Expression Union
ReadFromRemoteParallelReplicas Expression
Expression Join
ReadFromRemoteParallelReplicas Expression
Expression
ReadFromMergeTree
Expression
ReadFromMemoryStorage
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
-- --
-- ORDER BY in sub1 : sub1 -> WithMergableStage -- ORDER BY in sub1 : sub1 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2 order by y), with sub1 as (select x, y from tab1 where x != 2 order by y),
@ -160,12 +212,24 @@ Expression
Join Join
Expression Expression
Join Join
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression Expression
ReadFromRemoteParallelReplicas ReadFromMergeTree
Expression Expression
ReadFromRemoteParallelReplicas ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
-- --
-- RIGHT JOIN in sub3: sub3 -> WithMergableStage -- RIGHT JOIN in sub3: sub3 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -202,12 +266,24 @@ Expression
Join Join
Expression Expression
Join Join
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression Expression
ReadFromRemoteParallelReplicas ReadFromMergeTree
Expression Expression
ReadFromRemoteParallelReplicas ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
-- --
-- RIGHT JOIN in sub5: sub5 -> WithMergableStage -- RIGHT JOIN in sub5: sub5 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -242,14 +318,26 @@ Expression
Sorting Sorting
Expression Expression
Join Join
Expression Union
ReadFromRemoteParallelReplicas Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Expression Expression
Join Join
Expression Union
ReadFromRemoteParallelReplicas Expression
Expression Expression
ReadFromRemoteParallelReplicas ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
-- --
-- Subqueries for IN allowed -- Subqueries for IN allowed
with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)),
@ -284,8 +372,28 @@ select * from sub5 order by x
SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Expression
Sorting Sorting
Expression Union
ReadFromRemoteParallelReplicas Expression
Sorting
Expression
Join
Expression
Join
Expression
CreatingSets
Expression
Expression
ReadFromMergeTree
CreatingSet
Expression
Filter
ReadFromSystemNumbers
Expression
ReadFromMemoryStorage
Expression
ReadFromMemoryStorage
Expression
ReadFromRemoteParallelReplicas
-- --
-- Subqueries for IN are not allowed -- Subqueries for IN are not allowed
with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)),
@ -333,10 +441,18 @@ Expression
Expression Expression
Filter Filter
ReadFromSystemNumbers ReadFromSystemNumbers
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression Expression
ReadFromRemoteParallelReplicas ReadFromMergeTree
Expression Expression
ReadFromRemoteParallelReplicas ReadFromRemoteParallelReplicas
set parallel_replicas_prefer_local_join = 1; set parallel_replicas_prefer_local_join = 1;
-- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode. -- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode.
select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
@ -356,8 +472,22 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x
14 14 14 14 0 0 14 14 14 14 0 0
15 15 0 0 0 0 15 15 0 0 0 0
explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Union
ReadFromRemoteParallelReplicas Expression
Join
Expression
Join
Expression
Expression
ReadFromMergeTree
Expression
Expression
ReadFromMergeTree
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
-- --
-- The same query with cte; -- The same query with cte;
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -392,8 +522,24 @@ select * from sub5 order by x
SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Expression
Sorting Sorting
Expression Union
ReadFromRemoteParallelReplicas Expression
Sorting
Expression
Join
Expression
Join
Expression
Expression
ReadFromMergeTree
Expression
Expression
ReadFromMergeTree
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
-- --
-- GROUP BY should work up to WithMergableStage -- GROUP BY should work up to WithMergableStage
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -417,8 +563,24 @@ Expression
Sorting Sorting
Expression Expression
MergingAggregated MergingAggregated
Expression Union
ReadFromRemoteParallelReplicas Expression
Aggregating
Expression
Join
Expression
Join
Expression
Expression
ReadFromMergeTree
Expression
Expression
ReadFromMergeTree
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
-- --
-- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage -- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -455,10 +617,23 @@ Expression
Sorting Sorting
Expression Expression
Join Join
Expression Union
ReadFromRemoteParallelReplicas Expression
Expression Join
ReadFromRemoteParallelReplicas Expression
Expression
ReadFromMergeTree
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
-- --
-- ORDER BY in sub1 : sub1 -> WithMergableStage -- ORDER BY in sub1 : sub1 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2 order by y), with sub1 as (select x, y from tab1 where x != 2 order by y),
@ -497,12 +672,24 @@ Expression
Join Join
Expression Expression
Join Join
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression Expression
ReadFromRemoteParallelReplicas ReadFromMergeTree
Expression Expression
ReadFromRemoteParallelReplicas ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
-- --
-- RIGHT JOIN in sub3: sub3 -> WithMergableStage -- RIGHT JOIN in sub3: sub3 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -539,12 +726,24 @@ Expression
Join Join
Expression Expression
Join Join
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression Expression
ReadFromRemoteParallelReplicas ReadFromMergeTree
Expression Expression
ReadFromRemoteParallelReplicas ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
-- --
-- RIGHT JOIN in sub5: sub5 -> WithMergableStage -- RIGHT JOIN in sub5: sub5 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -579,14 +778,26 @@ Expression
Sorting Sorting
Expression Expression
Join Join
Expression Union
ReadFromRemoteParallelReplicas Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Expression Expression
Join Join
Expression Union
ReadFromRemoteParallelReplicas Expression
Expression Expression
ReadFromRemoteParallelReplicas ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
-- --
-- Subqueries for IN allowed -- Subqueries for IN allowed
with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)),
@ -621,8 +832,30 @@ select * from sub5 order by x
SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Expression
Sorting Sorting
Expression Union
ReadFromRemoteParallelReplicas Expression
Sorting
Expression
Join
Expression
Join
Expression
CreatingSets
Expression
Expression
ReadFromMergeTree
CreatingSet
Expression
Filter
ReadFromSystemNumbers
Expression
Expression
ReadFromMergeTree
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
-- --
-- Subqueries for IN are not allowed -- Subqueries for IN are not allowed
with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)),
@ -670,7 +903,15 @@ Expression
Expression Expression
Filter Filter
ReadFromSystemNumbers ReadFromSystemNumbers
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression Expression
ReadFromRemoteParallelReplicas ReadFromMergeTree
Expression Expression
ReadFromRemoteParallelReplicas ReadFromRemoteParallelReplicas

View File

@ -10,6 +10,8 @@ insert into tab1 select number, number, number from numbers(16);
insert into tab2 select number * 2, number * 2 from numbers(8); insert into tab2 select number * 2, number * 2 from numbers(8);
insert into tab3 select number * 4, number * 4 from numbers(4); insert into tab3 select number * 4, number * 4 from numbers(4);
set parallel_replicas_local_plan=1;
{% for use_global_in in [0, 1] -%} {% for use_global_in in [0, 1] -%}
-- { echoOn } -- { echoOn }

View File

@ -0,0 +1 @@
default url_na_log 1 130000 130

View File

@ -21,11 +21,21 @@ SELECT
FROM numbers(130000) FROM numbers(130000)
SETTINGS max_insert_block_size = 200000; SETTINGS max_insert_block_size = 200000;
SET max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=10000;
EXPLAIN ESTIMATE EXPLAIN ESTIMATE
SELECT count() SELECT count()
FROM url_na_log FROM url_na_log
PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20') PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20')
SETTINGS max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=10000; SETTINGS parallel_replicas_local_plan=0;
-- here parallel replicas uses local snapshot as working set
-- so, the estimation can be done
EXPLAIN ESTIMATE
SELECT count()
FROM url_na_log
PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20')
SETTINGS allow_experimental_analyzer=1, parallel_replicas_local_plan=1;
DROP POLICY url_na_log_policy0 ON url_na_log; DROP POLICY url_na_log_policy0 ON url_na_log;
DROP TABLE url_na_log; DROP TABLE url_na_log;

View File

@ -0,0 +1,21 @@
DROP TABLE IF EXISTS test_00808;
CREATE TABLE test_00808
(
`date` Date,
`id` Int8,
`name` String,
`value` Int64,
`sign` Int8
)
ENGINE = CollapsingMergeTree(sign)
ORDER BY (id, date);
INSERT INTO test_00808 VALUES('2000-01-01', 1, 'test string 1', 1, 1);
INSERT INTO test_00808 VALUES('2000-01-01', 2, 'test string 2', 2, 1);
SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost';
SELECT * FROM (SELECT * FROM test_00808 FINAL) WHERE id = 1; -- { serverError SUPPORT_IS_DISABLED }
DROP TABLE test_00808;

Some files were not shown because too many files have changed in this diff Show More