Merge branch 'master' into analyzer-fix-test_wrong_db_or_table_name

This commit is contained in:
Alexey Milovidov 2024-02-15 07:56:56 +01:00 committed by GitHub
commit 10eb948ed6
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
92 changed files with 2480 additions and 447 deletions

View File

@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \
# lts / testing / prestable / etc
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
ARG VERSION="24.1.4.20"
ARG VERSION="24.1.5.6"
ARG PACKAGES="clickhouse-keeper"
ARG DIRECT_DOWNLOAD_URLS=""

View File

@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \
# lts / testing / prestable / etc
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
ARG VERSION="24.1.4.20"
ARG VERSION="24.1.5.6"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
ARG DIRECT_DOWNLOAD_URLS=""

View File

@ -27,7 +27,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main"
ARG VERSION="24.1.4.20"
ARG VERSION="24.1.5.6"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
# set non-empty deb_location_url url to create a docker image

View File

@ -130,7 +130,7 @@ if [[ -n "${CLICKHOUSE_ALWAYS_RUN_INITDB_SCRIPTS}" || -z "${DATABASE_ALREADY_EXI
RUN_INITDB_SCRIPTS='true'
fi
if [ -z "${RUN_INITDB_SCRIPTS}" ]; then
if [ -n "${RUN_INITDB_SCRIPTS}" ]; then
if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then
# port is needed to check if clickhouse-server is ready for connections
HTTP_PORT="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=http_port --try)"

View File

@ -62,7 +62,6 @@ RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - \
# kazoo 2.10.0 is broken
# https://s3.amazonaws.com/clickhouse-test-reports/59337/524625a1d2f4cc608a3f1059e3df2c30f353a649/integration_tests__asan__analyzer__[5_6].html
RUN python3 -m pip install --no-cache-dir \
aerospike==11.1.0 \
PyMySQL==1.1.0 \
asyncio==3.4.3 \
avro==1.10.2 \

View File

@ -0,0 +1,17 @@
---
sidebar_position: 1
sidebar_label: 2024
---
# 2024 Changelog
### ClickHouse release v24.1.5.6-stable (7f67181ff31) FIXME as compared to v24.1.4.20-stable (f59d842b3fa)
#### Bug Fix (user-visible misbehavior in an official stable release)
* UniqExactSet read crash fix [#59928](https://github.com/ClickHouse/ClickHouse/pull/59928) ([Maksim Kita](https://github.com/kitaisreal)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* CI: do not reuse builds on release branches [#59798](https://github.com/ClickHouse/ClickHouse/pull/59798) ([Max K.](https://github.com/maxknv)).

View File

@ -91,7 +91,7 @@ If you use Arch or Gentoo, you probably know it yourself how to install CMake.
## C++ Compiler {#c-compiler}
Compilers Clang starting from version 15 is supported for building ClickHouse.
Compilers Clang starting from version 16 is supported for building ClickHouse.
Clang should be used instead of gcc. Though, our continuous integration (CI) platform runs checks for about a dozen of build combinations.

View File

@ -249,7 +249,7 @@ void LocalServer::tryInitPath()
default_path = parent_folder / fmt::format("clickhouse-local-{}-{}-{}", getpid(), time(nullptr), randomSeed());
if (exists(default_path))
throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Unsuccessful attempt to create working directory: {} exist!", default_path.string());
throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Unsuccessful attempt to create working directory: {} already exists.", default_path.string());
create_directory(default_path);
temporary_directory_to_delete = default_path;

View File

@ -521,7 +521,7 @@ const errorMessages = [
/// Query to fill `queries` list for the dashboard
let search_query = `SELECT title, query FROM system.dashboards WHERE dashboard = 'overview'`;
let search_query = `SELECT title, query FROM system.dashboards WHERE dashboard = 'Overview'`;
let customized = false;
let queries = [];

View File

@ -1380,6 +1380,8 @@ private:
ProjectionNames resolveSortNodeList(QueryTreeNodePtr & sort_node_list, IdentifierResolveScope & scope);
void resolveGroupByNode(QueryNode & query_node_typed, IdentifierResolveScope & scope);
void resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpolate_node_list, IdentifierResolveScope & scope);
void resolveWindowNodeList(QueryTreeNodePtr & window_node_list, IdentifierResolveScope & scope);
@ -5667,7 +5669,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
/// Do not constant fold get scalar functions
bool disable_constant_folding = function_name == "__getScalar" || function_name == "shardNum" ||
function_name == "shardCount" || function_name == "hostName";
function_name == "shardCount" || function_name == "hostName" || function_name == "tcpPort";
/** If function is suitable for constant folding try to convert it to constant.
* Example: SELECT plus(1, 1);
@ -6263,6 +6265,77 @@ ProjectionNames QueryAnalyzer::resolveSortNodeList(QueryTreeNodePtr & sort_node_
return result_projection_names;
}
namespace
{
void expandTuplesInList(QueryTreeNodes & key_list)
{
QueryTreeNodes expanded_keys;
expanded_keys.reserve(key_list.size());
for (auto const & key : key_list)
{
if (auto * function = key->as<FunctionNode>(); function != nullptr && function->getFunctionName() == "tuple")
{
std::copy(function->getArguments().begin(), function->getArguments().end(), std::back_inserter(expanded_keys));
}
else
expanded_keys.push_back(key);
}
key_list = std::move(expanded_keys);
}
}
/** Resolve GROUP BY clause.
*/
void QueryAnalyzer::resolveGroupByNode(QueryNode & query_node_typed, IdentifierResolveScope & scope)
{
const auto & settings = scope.context->getSettingsRef();
if (query_node_typed.isGroupByWithGroupingSets())
{
for (auto & grouping_sets_keys_list_node : query_node_typed.getGroupBy().getNodes())
{
if (settings.enable_positional_arguments)
replaceNodesWithPositionalArguments(grouping_sets_keys_list_node, query_node_typed.getProjection().getNodes(), scope);
resolveExpressionNodeList(grouping_sets_keys_list_node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/);
// Remove redundant calls to `tuple` function. It simplifies checking if expression is an aggregation key.
// It's required to support queries like: SELECT number FROM numbers(3) GROUP BY (number, number % 2)
auto & group_by_list = grouping_sets_keys_list_node->as<ListNode &>().getNodes();
expandTuplesInList(group_by_list);
}
if (scope.group_by_use_nulls)
{
for (const auto & grouping_set : query_node_typed.getGroupBy().getNodes())
{
for (const auto & group_by_elem : grouping_set->as<ListNode>()->getNodes())
scope.nullable_group_by_keys.insert(group_by_elem);
}
}
}
else
{
if (settings.enable_positional_arguments)
replaceNodesWithPositionalArguments(query_node_typed.getGroupByNode(), query_node_typed.getProjection().getNodes(), scope);
resolveExpressionNodeList(query_node_typed.getGroupByNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/);
// Remove redundant calls to `tuple` function. It simplifies checking if expression is an aggregation key.
// It's required to support queries like: SELECT number FROM numbers(3) GROUP BY (number, number % 2)
auto & group_by_list = query_node_typed.getGroupBy().getNodes();
expandTuplesInList(group_by_list);
if (scope.group_by_use_nulls)
{
for (const auto & group_by_elem : query_node_typed.getGroupBy().getNodes())
scope.nullable_group_by_keys.insert(group_by_elem);
}
}
}
/** Resolve interpolate columns nodes list.
*/
void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpolate_node_list, IdentifierResolveScope & scope)
@ -7453,40 +7526,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
resolveExpressionNode(query_node_typed.getWhere(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/);
if (query_node_typed.hasGroupBy())
{
if (query_node_typed.isGroupByWithGroupingSets())
{
for (auto & grouping_sets_keys_list_node : query_node_typed.getGroupBy().getNodes())
{
if (settings.enable_positional_arguments)
replaceNodesWithPositionalArguments(grouping_sets_keys_list_node, query_node_typed.getProjection().getNodes(), scope);
resolveExpressionNodeList(grouping_sets_keys_list_node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/);
}
if (scope.group_by_use_nulls)
{
for (const auto & grouping_set : query_node_typed.getGroupBy().getNodes())
{
for (const auto & group_by_elem : grouping_set->as<ListNode>()->getNodes())
scope.nullable_group_by_keys.insert(group_by_elem);
}
}
}
else
{
if (settings.enable_positional_arguments)
replaceNodesWithPositionalArguments(query_node_typed.getGroupByNode(), query_node_typed.getProjection().getNodes(), scope);
resolveExpressionNodeList(query_node_typed.getGroupByNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/);
if (scope.group_by_use_nulls)
{
for (const auto & group_by_elem : query_node_typed.getGroupBy().getNodes())
scope.nullable_group_by_keys.insert(group_by_elem);
}
}
}
resolveGroupByNode(query_node_typed, scope);
if (query_node_typed.hasHaving())
resolveExpressionNode(query_node_typed.getHaving(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/);

View File

@ -15,6 +15,7 @@
#include <base/bit_cast.h>
#include <base/extended_types.h>
#include <base/sort.h>
#include <Core/Defines.h>
@ -114,6 +115,11 @@ struct RadixSortFloatTraits
{
return x < y;
}
static bool greater(Key x, Key y)
{
return x > y;
}
};
@ -148,6 +154,11 @@ struct RadixSortUIntTraits
{
return x < y;
}
static bool greater(Key x, Key y)
{
return x > y;
}
};
@ -182,6 +193,11 @@ struct RadixSortIntTraits
{
return x < y;
}
static bool greater(Key x, Key y)
{
return x > y;
}
};
@ -214,6 +230,22 @@ private:
static KeyBits keyToBits(Key x) { return bit_cast<KeyBits>(x); }
static Key bitsToKey(KeyBits x) { return bit_cast<Key>(x); }
struct LessComparator
{
ALWAYS_INLINE bool operator()(Element & lhs, Element & rhs)
{
return Traits::less(Traits::extractKey(lhs), Traits::extractKey(rhs));
}
};
struct GreaterComparator
{
ALWAYS_INLINE bool operator()(Element & lhs, Element & rhs)
{
return Traits::greater(Traits::extractKey(lhs), Traits::extractKey(rhs));
}
};
static ALWAYS_INLINE KeyBits getPart(size_t N, KeyBits x)
{
if (Traits::Transform::transform_is_simple)
@ -504,6 +536,24 @@ private:
radixSortMSDInternal<PASS>(arr, size, limit);
}
template <bool DIRECT_WRITE_TO_DESTINATION, typename Comparator>
static void executeLSDWithTrySortInternal(Element * arr, size_t size, bool reverse, Comparator comparator, Result * destination)
{
bool try_sort = ::trySort(arr, arr + size, comparator);
if (try_sort)
{
if constexpr (DIRECT_WRITE_TO_DESTINATION)
{
for (size_t i = 0; i < size; ++i)
destination[i] = Traits::extractResult(arr[i]);
}
return;
}
radixSortLSDInternal<DIRECT_WRITE_TO_DESTINATION>(arr, size, reverse, destination);
}
public:
/** Least significant digit radix sort (stable).
* This function will sort inplace (modify 'arr')
@ -529,6 +579,38 @@ public:
radixSortLSDInternal<true>(arr, size, reverse, destination);
}
/** Tries to fast sort elements for common sorting patterns (unstable).
* If fast sort cannot be performed, execute least significant digit radix sort.
*/
static void executeLSDWithTrySort(Element * arr, size_t size)
{
return executeLSDWithTrySort(arr, size, false);
}
static void executeLSDWithTrySort(Element * arr, size_t size, bool reverse)
{
return executeLSDWithTrySort(arr, size, reverse, nullptr);
}
static void executeLSDWithTrySort(Element * arr, size_t size, bool reverse, Result * destination)
{
if (reverse)
{
if (destination)
return executeLSDWithTrySortInternal<true>(arr, size, reverse, GreaterComparator(), destination);
else
return executeLSDWithTrySortInternal<false>(arr, size, reverse, GreaterComparator(), destination);
}
else
{
if (destination)
return executeLSDWithTrySortInternal<true>(arr, size, reverse, LessComparator(), destination);
else
return executeLSDWithTrySortInternal<false>(arr, size, reverse, LessComparator(), destination);
}
}
/* Most significant digit radix sort
* Is not stable, but allows partial sorting.
* And it's more cache-friendly and usually faster than LSD variant.

View File

@ -40,6 +40,8 @@ class IColumn;
M(UInt64, min_insert_block_size_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough.", 0) \
M(UInt64, min_insert_block_size_rows_for_materialized_views, 0, "Like min_insert_block_size_rows, but applied only during pushing to MATERIALIZED VIEW (default: min_insert_block_size_rows)", 0) \
M(UInt64, min_insert_block_size_bytes_for_materialized_views, 0, "Like min_insert_block_size_bytes, but applied only during pushing to MATERIALIZED VIEW (default: min_insert_block_size_bytes)", 0) \
M(UInt64, min_external_table_block_size_rows, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough.", 0) \
M(UInt64, min_external_table_block_size_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough.", 0) \
M(UInt64, max_joined_block_size_rows, DEFAULT_BLOCK_SIZE, "Maximum block size for JOIN result (if join algorithm supports it). 0 means unlimited.", 0) \
M(UInt64, max_insert_threads, 0, "The maximum number of threads to execute the INSERT SELECT query. Values 0 or 1 means that INSERT SELECT is not run in parallel. Higher values will lead to higher memory usage. Parallel INSERT SELECT has effect only if the SELECT part is run on parallel, see 'max_threads' setting.", 0) \
M(UInt64, max_insert_delayed_streams_for_parallel_write, 0, "The maximum number of streams (columns) to delay final part flush. Default - auto (1000 in case of underlying storage supports parallel write, for example S3 and disabled otherwise)", 0) \
@ -186,6 +188,7 @@ class IColumn;
M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \
M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \
M(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, "Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas'", 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(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards. Shard is marked as unavailable when: 1) The shard cannot be reached due to a connection failure. 2) Shard is unresolvable through DNS. 3) Table does not exist on the shard.", 0) \
@ -831,7 +834,7 @@ class IColumn;
M(UInt64, insert_keeper_retry_max_backoff_ms, 10000, "Max backoff timeout for keeper operations during insert", 0) \
M(Float, insert_keeper_fault_injection_probability, 0.0f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \
M(UInt64, insert_keeper_fault_injection_seed, 0, "0 - random seed, otherwise the setting value", 0) \
M(Bool, force_aggregation_in_order, false, "Force use of aggregation in order on remote nodes during distributed aggregation. PLEASE, NEVER CHANGE THIS SETTING VALUE MANUALLY!", IMPORTANT) \
M(Bool, force_aggregation_in_order, false, "The setting is used by the server itself to support distributed queries. Do not change it manually, because it will break normal operations. (Forces use of aggregation in order on remote nodes during distributed aggregation).", IMPORTANT) \
M(UInt64, http_max_request_param_data_size, 10_MiB, "Limit on size of request data used as a query parameter in predefined HTTP requests.", 0) \
M(Bool, function_json_value_return_type_allow_nullable, false, "Allow function JSON_VALUE to return nullable type.", 0) \
M(Bool, function_json_value_return_type_allow_complex, false, "Allow function JSON_VALUE to return complex type, such as: struct, array, map.", 0) \

View File

@ -84,7 +84,8 @@ namespace SettingsChangesHistory
/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972)
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
{
{"24.2", {{"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."},
{"24.2", {
{"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."},
{"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"},
{"async_insert_use_adaptive_busy_timeout", true, true, "Use adaptive asynchronous insert timeout"},
{"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"},
@ -96,7 +97,11 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"},
{"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"},
{"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."},
{"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}}},
{"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"},
{"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."},
{"parallel_replicas_prefer_local_join", true, 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."},
{"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."},
}},
{"24.1", {{"print_pretty_type_names", false, true, "Better user experience."},
{"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"},
{"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"},

View File

@ -407,7 +407,7 @@ public:
virtual void stopReplication()
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database engine {} does not run a replication thread!", getEngineName());
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database engine {} does not run a replication thread", getEngineName());
}
virtual bool shouldReplicateQuery(const ContextPtr & /*query_context*/, const ASTPtr & /*query_ptr*/) const { return false; }

View File

@ -78,22 +78,22 @@ public:
double getLoadFactor() const override;
size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
size_t getQueryCount() const override { return query_count.load(); }
double getFoundRate() const override
{
size_t queries = query_count.load(std::memory_order_relaxed);
size_t queries = query_count.load();
if (!queries)
return 0;
return static_cast<double>(found_count.load(std::memory_order_relaxed)) / queries;
return std::min(1.0, static_cast<double>(found_count.load()) / queries);
}
double getHitRate() const override
{
size_t queries = query_count.load(std::memory_order_relaxed);
size_t queries = query_count.load();
if (!queries)
return 0;
return static_cast<double>(hit_count.load(std::memory_order_acquire)) / queries;
return static_cast<double>(hit_count.load()) / queries;
}
bool supportUpdates() const override { return false; }

View File

@ -34,14 +34,14 @@ public:
size_t getBytesAllocated() const override { return 0; }
size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
size_t getQueryCount() const override { return query_count.load(); }
double getFoundRate() const override
{
size_t queries = query_count.load(std::memory_order_relaxed);
size_t queries = query_count.load();
if (!queries)
return 0;
return static_cast<double>(found_count.load(std::memory_order_relaxed)) / queries;
return std::min(1.0, static_cast<double>(found_count.load()) / queries);
}
double getHitRate() const override { return 1.0; }

View File

@ -41,14 +41,14 @@ public:
size_t getBytesAllocated() const override { return bytes_allocated; }
size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
size_t getQueryCount() const override { return query_count.load(); }
double getFoundRate() const override
{
size_t queries = query_count.load(std::memory_order_relaxed);
size_t queries = query_count.load();
if (!queries)
return 0;
return static_cast<double>(found_count.load(std::memory_order_relaxed)) / queries;
return std::min(1.0, static_cast<double>(found_count.load()) / queries);
}
double getHitRate() const override { return 1.0; }

View File

@ -57,14 +57,14 @@ public:
size_t getBytesAllocated() const override { return bytes_allocated; }
size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
size_t getQueryCount() const override { return query_count.load(); }
double getFoundRate() const override
{
size_t queries = query_count.load(std::memory_order_relaxed);
size_t queries = query_count.load();
if (!queries)
return 0;
return static_cast<double>(found_count.load(std::memory_order_relaxed)) / queries;
return std::min(1.0, static_cast<double>(found_count.load()) / queries);
}
double getHitRate() const override { return 1.0; }

View File

@ -99,14 +99,14 @@ public:
size_t getBytesAllocated() const override { return bytes_allocated; }
size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
size_t getQueryCount() const override { return query_count.load(); }
double getFoundRate() const override
{
size_t queries = query_count.load(std::memory_order_relaxed);
size_t queries = query_count.load();
if (!queries)
return 0;
return static_cast<double>(found_count.load(std::memory_order_relaxed)) / queries;
return std::min(1.0, static_cast<double>(found_count.load()) / queries);
}
double getHitRate() const override { return 1.0; }

View File

@ -109,6 +109,9 @@ public:
virtual size_t getQueryCount() const = 0;
/// The percentage of time a lookup successfully found an entry.
/// When there were no lookups, it returns zero (instead of NaN).
/// The value is calculated non atomically and can be slightly off in the presence of concurrent lookups.
virtual double getFoundRate() const = 0;
virtual double getHitRate() const = 0;

View File

@ -41,14 +41,14 @@ public:
size_t getBytesAllocated() const override { return bytes_allocated; }
size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
size_t getQueryCount() const override { return query_count.load(); }
double getFoundRate() const override
{
size_t queries = query_count.load(std::memory_order_relaxed);
size_t queries = query_count.load();
if (!queries)
return 0;
return static_cast<double>(found_count.load(std::memory_order_relaxed)) / queries;
return std::min(1.0, static_cast<double>(found_count.load()) / queries);
}
double getHitRate() const override { return 1.0; }

View File

@ -71,14 +71,14 @@ public:
size_t getBytesAllocated() const override { return bytes_allocated; }
size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
size_t getQueryCount() const override { return query_count.load(); }
double getFoundRate() const override
{
size_t queries = query_count.load(std::memory_order_relaxed);
size_t queries = query_count.load();
if (!queries)
return 0;
return static_cast<double>(found_count.load(std::memory_order_relaxed)) / queries;
return std::min(1.0, static_cast<double>(found_count.load()) / queries);
}
double getHitRate() const override { return 1.0; }

View File

@ -85,14 +85,14 @@ public:
size_t getBytesAllocated() const override { return bytes_allocated; }
size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
size_t getQueryCount() const override { return query_count.load(); }
double getFoundRate() const override
{
size_t queries = query_count.load(std::memory_order_relaxed);
size_t queries = query_count.load();
if (!queries)
return 0;
return static_cast<double>(found_count.load(std::memory_order_relaxed)) / queries;
return std::min(1.0, static_cast<double>(found_count.load()) / queries);
}
double getHitRate() const override { return 1.0; }

View File

@ -58,14 +58,14 @@ public:
size_t getBytesAllocated() const override { return bytes_allocated; }
size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
size_t getQueryCount() const override { return query_count.load(); }
double getFoundRate() const override
{
const auto queries = query_count.load(std::memory_order_relaxed);
const auto queries = query_count.load();
if (!queries)
return 0;
return static_cast<double>(found_count.load(std::memory_order_relaxed)) / queries;
return std::min(1.0, static_cast<double>(found_count.load()) / queries);
}
double getHitRate() const override { return 1.0; }

View File

@ -85,7 +85,7 @@ VolumeJBOD::VolumeJBOD(const VolumeJBOD & volume_jbod,
DiskSelectorPtr disk_selector)
: VolumeJBOD(volume_jbod.name, config, config_prefix, disk_selector)
{
are_merges_avoided_user_override = volume_jbod.are_merges_avoided_user_override.load(std::memory_order_relaxed);
are_merges_avoided_user_override = volume_jbod.are_merges_avoided_user_override.load();
last_used = volume_jbod.last_used.load(std::memory_order_relaxed);
}

View File

@ -643,13 +643,12 @@ class FunctionComparison : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionComparison>(context); }
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionComparison>(decimalCheckComparisonOverflow(context)); }
explicit FunctionComparison(ContextPtr context_)
: context(context_), check_decimal_overflow(decimalCheckComparisonOverflow(context)) {}
explicit FunctionComparison(bool check_decimal_overflow_)
: check_decimal_overflow(check_decimal_overflow_) {}
private:
ContextPtr context;
bool check_decimal_overflow = true;
template <typename T0, typename T1>
@ -1190,7 +1189,7 @@ public:
if (left_tuple && right_tuple)
{
auto func = FunctionToOverloadResolverAdaptor(FunctionComparison<Op, Name>::create(context));
auto func = FunctionToOverloadResolverAdaptor(std::make_shared<FunctionComparison<Op, Name>>(check_decimal_overflow));
bool has_nullable = false;
bool has_null = false;

View File

@ -1,5 +1,6 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsComparison.h>
#include <Functions/FunctionsLogical.h>
namespace DB
@ -16,9 +17,16 @@ template <>
ColumnPtr FunctionComparison<EqualsOp, NameEquals>::executeTupleImpl(
const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size, size_t input_rows_count) const
{
FunctionOverloadResolverPtr func_builder_equals
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionEquals>(check_decimal_overflow));
FunctionOverloadResolverPtr func_builder_and
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
return executeTupleEqualityImpl(
FunctionFactory::instance().get("equals", context),
FunctionFactory::instance().get("and", context),
func_builder_equals,
func_builder_and,
x, y, tuple_size, input_rows_count);
}

View File

@ -1,11 +1,12 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsComparison.h>
#include <Functions/FunctionsLogical.h>
namespace DB
{
using FunctionGreater = FunctionComparison<GreaterOp, NameGreater>;
using FunctionEquals = FunctionComparison<EqualsOp, NameEquals>;
REGISTER_FUNCTION(Greater)
{
@ -16,14 +17,24 @@ template <>
ColumnPtr FunctionComparison<GreaterOp, NameGreater>::executeTupleImpl(
const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size, size_t input_rows_count) const
{
auto greater = FunctionFactory::instance().get("greater", context);
FunctionOverloadResolverPtr greater
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionGreater>(check_decimal_overflow));
FunctionOverloadResolverPtr func_builder_or
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionOr>());
FunctionOverloadResolverPtr func_builder_and
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
FunctionOverloadResolverPtr func_builder_equals
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionEquals>(check_decimal_overflow));
return executeTupleLessGreaterImpl(
greater,
greater,
FunctionFactory::instance().get("and", context),
FunctionFactory::instance().get("or", context),
FunctionFactory::instance().get("equals", context),
func_builder_and,
func_builder_or,
func_builder_equals,
x, y, tuple_size, input_rows_count);
}

View File

@ -1,11 +1,14 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsComparison.h>
#include <Functions/FunctionsLogical.h>
namespace DB
{
using FunctionGreaterOrEquals = FunctionComparison<GreaterOrEqualsOp, NameGreaterOrEquals>;
using FunctionGreater = FunctionComparison<GreaterOp, NameGreater>;
using FunctionEquals = FunctionComparison<EqualsOp, NameEquals>;
REGISTER_FUNCTION(GreaterOrEquals)
{
@ -16,12 +19,28 @@ template <>
ColumnPtr FunctionComparison<GreaterOrEqualsOp, NameGreaterOrEquals>::executeTupleImpl(
const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size, size_t input_rows_count) const
{
FunctionOverloadResolverPtr greater
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionGreater>(check_decimal_overflow));
FunctionOverloadResolverPtr greater_or_equals
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionGreaterOrEquals>(check_decimal_overflow));
FunctionOverloadResolverPtr func_builder_or
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionOr>());
FunctionOverloadResolverPtr func_builder_and
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
FunctionOverloadResolverPtr func_builder_equals
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionEquals>(check_decimal_overflow));
return executeTupleLessGreaterImpl(
FunctionFactory::instance().get("greater", context),
FunctionFactory::instance().get("greaterOrEquals", context),
FunctionFactory::instance().get("and", context),
FunctionFactory::instance().get("or", context),
FunctionFactory::instance().get("equals", context),
greater,
greater_or_equals,
func_builder_and,
func_builder_or,
func_builder_equals,
x, y, tuple_size, input_rows_count);
}

View File

@ -1,11 +1,13 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsComparison.h>
#include <Functions/FunctionsLogical.h>
namespace DB
{
using FunctionLess = FunctionComparison<LessOp, NameLess>;
using FunctionEquals = FunctionComparison<EqualsOp, NameEquals>;
REGISTER_FUNCTION(Less)
{
@ -16,14 +18,24 @@ template <>
ColumnPtr FunctionComparison<LessOp, NameLess>::executeTupleImpl(
const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size, size_t input_rows_count) const
{
auto less = FunctionFactory::instance().get("less", context);
FunctionOverloadResolverPtr less
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionLess>(check_decimal_overflow));
FunctionOverloadResolverPtr func_builder_or
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionOr>());
FunctionOverloadResolverPtr func_builder_and
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
FunctionOverloadResolverPtr func_builder_equals
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionEquals>(check_decimal_overflow));
return executeTupleLessGreaterImpl(
less,
less,
FunctionFactory::instance().get("and", context),
FunctionFactory::instance().get("or", context),
FunctionFactory::instance().get("equals", context),
func_builder_and,
func_builder_or,
func_builder_equals,
x, y, tuple_size, input_rows_count);
}

View File

@ -1,11 +1,14 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsComparison.h>
#include <Functions/FunctionsLogical.h>
namespace DB
{
using FunctionLessOrEquals = FunctionComparison<LessOrEqualsOp, NameLessOrEquals>;
using FunctionLess = FunctionComparison<LessOp, NameLess>;
using FunctionEquals = FunctionComparison<EqualsOp, NameEquals>;
REGISTER_FUNCTION(LessOrEquals)
{
@ -16,12 +19,27 @@ template <>
ColumnPtr FunctionComparison<LessOrEqualsOp, NameLessOrEquals>::executeTupleImpl(
const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size, size_t input_rows_count) const
{
FunctionOverloadResolverPtr less_or_equals
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionLessOrEquals>(check_decimal_overflow));
FunctionOverloadResolverPtr less
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionLess>(check_decimal_overflow));
FunctionOverloadResolverPtr func_builder_or
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionOr>());
FunctionOverloadResolverPtr func_builder_and
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
FunctionOverloadResolverPtr func_builder_equals
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionEquals>(check_decimal_overflow));
return executeTupleLessGreaterImpl(
FunctionFactory::instance().get("less", context),
FunctionFactory::instance().get("lessOrEquals", context),
FunctionFactory::instance().get("and", context),
FunctionFactory::instance().get("or", context),
FunctionFactory::instance().get("equals", context),
less,
less_or_equals,
func_builder_and,
func_builder_or,
func_builder_equals,
x, y, tuple_size, input_rows_count);
}

View File

@ -1,6 +1,6 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsComparison.h>
#include <Functions/FunctionsLogical.h>
namespace DB
{
@ -16,9 +16,15 @@ template <>
ColumnPtr FunctionComparison<NotEqualsOp, NameNotEquals>::executeTupleImpl(
const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size, size_t input_rows_count) const
{
FunctionOverloadResolverPtr func_builder_not_equals
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionNotEquals>(check_decimal_overflow));
FunctionOverloadResolverPtr func_builder_or
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionOr>());
return executeTupleEqualityImpl(
FunctionFactory::instance().get("notEquals", context),
FunctionFactory::instance().get("or", context),
func_builder_not_equals,
func_builder_or,
x, y, tuple_size, input_rows_count);
}

View File

@ -44,7 +44,7 @@ class FunctionWidthBucket : public IFunction
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Logical error in function {}: argument {} has unexpected type or size!",
"Logical error in function {}: argument {} has unexpected type or size.",
getName(),
argument_index);
}
@ -157,7 +157,7 @@ class FunctionWidthBucket : public IFunction
if (are_all_const_cols)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR, "Logical error in function {}: unexpected combination of argument types!", getName());
ErrorCodes::LOGICAL_ERROR, "Logical error in function {}: unexpected combination of argument types.", getName());
}
auto result_column = ColumnVector<ResultType>::create();

View File

@ -552,7 +552,7 @@ bool ReadWriteBufferFromHTTPBase<UpdatableSessionPtr>::nextImpl()
if (!can_retry_request)
throw;
LOG_ERROR(
LOG_WARNING(
log,
"HTTP request to `{}` failed at try {}/{} with bytes read: {}/{}. "
"Error: {}. (Current backoff wait is {}/{} ms)",

View File

@ -319,7 +319,7 @@ bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info)
if (cluster_info.current_cluster_is_invisible)
{
LOG_DEBUG(log, "cluster '{}' is invisible!", cluster_info.name);
LOG_DEBUG(log, "Cluster '{}' is invisible.", cluster_info.name);
return true;
}

View File

@ -907,7 +907,7 @@ Strings Context::getWarnings() const
if (CurrentMetrics::get(CurrentMetrics::AttachedTable) > static_cast<Int64>(shared->max_table_num_to_warn))
common_warnings.emplace_back(fmt::format("The number of attached tables is more than {}", shared->max_table_num_to_warn));
if (CurrentMetrics::get(CurrentMetrics::AttachedDatabase) > static_cast<Int64>(shared->max_database_num_to_warn))
common_warnings.emplace_back(fmt::format("The number of attached databases is more than {}", shared->max_table_num_to_warn));
common_warnings.emplace_back(fmt::format("The number of attached databases is more than {}", shared->max_database_num_to_warn));
if (CurrentMetrics::get(CurrentMetrics::PartsActive) > static_cast<Int64>(shared->max_part_num_to_warn))
common_warnings.emplace_back(fmt::format("The number of active parts is more than {}", shared->max_part_num_to_warn));
}
@ -4154,12 +4154,12 @@ void Context::setMaxTableSizeToDrop(size_t max_size)
size_t Context::getMaxTableSizeToDrop() const
{
return shared->max_table_size_to_drop.load(std::memory_order_relaxed);
return shared->max_table_size_to_drop.load();
}
void Context::checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const
{
size_t max_table_size_to_drop = shared->max_table_size_to_drop.load(std::memory_order_relaxed);
size_t max_table_size_to_drop = shared->max_table_size_to_drop.load();
checkCanBeDropped(database, table, table_size, max_table_size_to_drop);
}
@ -4177,12 +4177,12 @@ void Context::setMaxPartitionSizeToDrop(size_t max_size)
size_t Context::getMaxPartitionSizeToDrop() const
{
return shared->max_partition_size_to_drop.load(std::memory_order_relaxed);
return shared->max_partition_size_to_drop.load();
}
void Context::checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const
{
size_t max_partition_size_to_drop = shared->max_partition_size_to_drop.load(std::memory_order_relaxed);
size_t max_partition_size_to_drop = shared->max_partition_size_to_drop.load();
checkCanBeDropped(database, table, partition_size, max_partition_size_to_drop);
}

View File

@ -274,7 +274,7 @@ Chain InterpreterInsertQuery::buildChain(
auto sample = getSampleBlock(columns, table, metadata_snapshot);
Chain sink = buildSink(table, metadata_snapshot, thread_status_holder, running_group, elapsed_counter_ms);
Chain chain = buildPreSinkChain(sink.getInputHeader(), table, metadata_snapshot, sample, thread_status_holder);
Chain chain = buildPreSinkChain(sink.getInputHeader(), table, metadata_snapshot, sample);
chain.appendChain(std::move(sink));
return chain;
@ -317,25 +317,31 @@ Chain InterpreterInsertQuery::buildSink(
return out;
}
bool InterpreterInsertQuery::shouldAddSquashingFroStorage(const StoragePtr & table) const
{
auto context_ptr = getContext();
const Settings & settings = context_ptr->getSettingsRef();
const ASTInsertQuery * query = nullptr;
if (query_ptr)
query = query_ptr->as<ASTInsertQuery>();
/// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side.
/// Client-side bufferization might cause excessive timeouts (especially in case of big blocks).
return !(settings.distributed_foreground_insert && table->isRemote()) && !async_insert && !no_squash && !(query && query->watch);
}
Chain InterpreterInsertQuery::buildPreSinkChain(
const Block & subsequent_header,
const StoragePtr & table,
const StorageMetadataPtr & metadata_snapshot,
const Block & query_sample_block,
ThreadStatusesHolderPtr thread_status_holder)
const Block & query_sample_block)
{
ThreadStatus * thread_status = current_thread;
if (!thread_status_holder)
thread_status = nullptr;
auto context_ptr = getContext();
const ASTInsertQuery * query = nullptr;
if (query_ptr)
query = query_ptr->as<ASTInsertQuery>();
const Settings & settings = context_ptr->getSettingsRef();
bool null_as_default = query && query->select && context_ptr->getSettingsRef().insert_null_as_default;
/// We create a pipeline of several streams, into which we will write data.
@ -366,26 +372,6 @@ Chain InterpreterInsertQuery::buildPreSinkChain(
/// because some clients break insertion protocol (columns != header)
out.addSource(std::make_shared<ConvertingTransform>(query_sample_block, adding_missing_defaults_actions));
/// It's important to squash blocks as early as possible (before other transforms),
/// because other transforms may work inefficient if block size is small.
/// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side.
/// Client-side bufferization might cause excessive timeouts (especially in case of big blocks).
if (!(settings.distributed_foreground_insert && table->isRemote()) && !async_insert && !no_squash && !(query && query->watch))
{
bool table_prefers_large_blocks = table->prefersLargeBlocks();
out.addSource(std::make_shared<SquashingChunksTransform>(
input_header(),
table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size,
table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL));
}
auto counting = std::make_shared<CountingTransform>(input_header(), thread_status, getContext()->getQuota());
counting->setProcessListElement(context_ptr->getProcessListElement());
counting->setProgressCallback(context_ptr->getProgressCallback());
out.addSource(std::move(counting));
return out;
}
@ -558,8 +544,7 @@ BlockIO InterpreterInsertQuery::execute()
}
for (size_t i = 0; i < pre_streams_size; ++i)
{
auto out = buildPreSinkChain(sink_chains[0].getInputHeader(), table, metadata_snapshot,
query_sample_block, /* thread_status_holder= */ nullptr);
auto out = buildPreSinkChain(sink_chains[0].getInputHeader(), table, metadata_snapshot, query_sample_block);
presink_chains.emplace_back(std::move(out));
}
}
@ -592,6 +577,29 @@ BlockIO InterpreterInsertQuery::execute()
return std::make_shared<MaterializingTransform>(in_header);
});
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
{
auto context_ptr = getContext();
auto counting = std::make_shared<CountingTransform>(in_header, nullptr, context_ptr->getQuota());
counting->setProcessListElement(context_ptr->getProcessListElement());
counting->setProgressCallback(context_ptr->getProgressCallback());
return counting;
});
if (shouldAddSquashingFroStorage(table))
{
bool table_prefers_large_blocks = table->prefersLargeBlocks();
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
{
return std::make_shared<SimpleSquashingChunksTransform>(
in_header,
table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size,
table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL);
});
}
size_t num_select_threads = pipeline.getNumThreads();
for (auto & chain : presink_chains)
@ -634,7 +642,27 @@ BlockIO InterpreterInsertQuery::execute()
}
else
{
presink_chains.at(0).appendChain(std::move(sink_chains.at(0)));
auto & chain = presink_chains.at(0);
chain.appendChain(std::move(sink_chains.at(0)));
if (shouldAddSquashingFroStorage(table))
{
bool table_prefers_large_blocks = table->prefersLargeBlocks();
auto squashing = std::make_shared<SimpleSquashingChunksTransform>(
chain.getInputHeader(),
table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size,
table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL);
chain.addSource(std::move(squashing));
}
auto context_ptr = getContext();
auto counting = std::make_shared<CountingTransform>(chain.getInputHeader(), nullptr, context_ptr->getQuota());
counting->setProcessListElement(context_ptr->getProcessListElement());
counting->setProgressCallback(context_ptr->getProgressCallback());
chain.addSource(std::move(counting));
res.pipeline = QueryPipeline(std::move(presink_chains[0]));
res.pipeline.setNumThreads(std::min<size_t>(res.pipeline.getNumThreads(), settings.max_threads));
res.pipeline.setConcurrencyControl(settings.use_concurrency_control);

View File

@ -59,6 +59,8 @@ public:
void addBuffer(std::unique_ptr<ReadBuffer> buffer) { owned_buffers.push_back(std::move(buffer)); }
bool shouldAddSquashingFroStorage(const StoragePtr & table) const;
private:
Block getSampleBlock(const Names & names, const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const;
@ -81,8 +83,7 @@ private:
const Block & subsequent_header,
const StoragePtr & table,
const StorageMetadataPtr & metadata_snapshot,
const Block & query_sample_block,
ThreadStatusesHolderPtr thread_status_holder);
const Block & query_sample_block);
};

View File

@ -1083,7 +1083,7 @@ void InterpreterSystemQuery::syncReplica(ASTSystemQuery & query)
auto sync_timeout = getContext()->getSettingsRef().receive_timeout.totalMilliseconds();
if (!storage_replicated->waitForProcessingQueue(sync_timeout, query.sync_replica_mode, query.src_replicas))
{
LOG_ERROR(log, "SYNC REPLICA {}: Timed out!", table_id.getNameForLogs());
LOG_ERROR(log, "SYNC REPLICA {}: Timed out.", table_id.getNameForLogs());
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "SYNC REPLICA {}: command timed out. " \
"See the 'receive_timeout' setting", table_id.getNameForLogs());
}

View File

@ -175,45 +175,42 @@ private:
// the array becomes immutable
void sort()
{
if (!sorted.load(std::memory_order_acquire))
if (sorted.load(std::memory_order_acquire))
return;
std::lock_guard<std::mutex> l(lock);
if (sorted.load(std::memory_order_relaxed))
return;
if constexpr (std::is_arithmetic_v<TKey> && !std::is_floating_point_v<TKey>)
{
std::lock_guard<std::mutex> l(lock);
if (!sorted.load(std::memory_order_relaxed))
if (likely(entries.size() > 256))
{
if constexpr (std::is_arithmetic_v<TKey> && !std::is_floating_point_v<TKey>)
struct RadixSortTraits : RadixSortNumTraits<TKey>
{
if (likely(entries.size() > 256))
{
struct RadixSortTraits : RadixSortNumTraits<TKey>
{
using Element = Entry;
using Result = Element;
using Element = Entry;
using Result = Element;
static TKey & extractKey(Element & elem) { return elem.value; }
static Result extractResult(Element & elem) { return elem; }
};
if constexpr (is_descending)
RadixSort<RadixSortTraits>::executeLSD(entries.data(), entries.size(), true);
else
RadixSort<RadixSortTraits>::executeLSD(entries.data(), entries.size(), false);
sorted.store(true, std::memory_order_release);
return;
}
}
if constexpr (is_descending)
::sort(entries.begin(), entries.end(), GreaterEntryOperator());
else
::sort(entries.begin(), entries.end(), LessEntryOperator());
static TKey & extractKey(Element & elem) { return elem.value; }
static Result extractResult(Element & elem) { return elem; }
};
RadixSort<RadixSortTraits>::executeLSDWithTrySort(entries.data(), entries.size(), is_descending /*reverse*/);
sorted.store(true, std::memory_order_release);
return;
}
}
if constexpr (is_descending)
::sort(entries.begin(), entries.end(), GreaterEntryOperator());
else
::sort(entries.begin(), entries.end(), LessEntryOperator());
sorted.store(true, std::memory_order_release);
}
};
}
AsofRowRefs createAsofRowRef(TypeIndex type, ASOFJoinInequality inequality)

View File

@ -927,8 +927,6 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
reason = "asynchronous insert queue is not configured";
else if (insert_query->select)
reason = "insert query has select";
else if (settings.deduplicate_blocks_in_dependent_materialized_views)
reason = "dependent materialized views block deduplication is enabled";
else if (insert_query->hasInlinedData())
async_insert = true;

View File

@ -64,6 +64,7 @@
#include <Analyzer/AggregationUtils.h>
#include <Analyzer/WindowFunctionsUtils.h>
#include <Planner/findQueryForParallelReplicas.h>
#include <Planner/Utils.h>
#include <Planner/PlannerContext.h>
#include <Planner/PlannerActionsVisitor.h>
@ -1057,7 +1058,7 @@ void addBuildSubqueriesForSetsStepIfNeeded(
Planner subquery_planner(
query_tree,
subquery_options,
std::make_shared<GlobalPlannerContext>()); //planner_context->getGlobalPlannerContext());
std::make_shared<GlobalPlannerContext>(nullptr, nullptr));
subquery_planner.buildQueryPlanIfNeeded();
subquery->setQueryPlan(std::make_unique<QueryPlan>(std::move(subquery_planner).extractQueryPlan()));
@ -1160,7 +1161,10 @@ Planner::Planner(const QueryTreeNodePtr & query_tree_,
SelectQueryOptions & select_query_options_)
: query_tree(query_tree_)
, select_query_options(select_query_options_)
, planner_context(buildPlannerContext(query_tree, select_query_options, std::make_shared<GlobalPlannerContext>()))
, planner_context(buildPlannerContext(query_tree, select_query_options,
std::make_shared<GlobalPlannerContext>(
findQueryForParallelReplicas(query_tree, select_query_options),
findTableForParallelReplicas(query_tree, select_query_options))))
{
}
@ -1223,6 +1227,8 @@ void Planner::buildPlanForUnionNode()
query_planner.buildQueryPlanIfNeeded();
for (const auto & row_policy : query_planner.getUsedRowPolicies())
used_row_policies.insert(row_policy);
const auto & mapping = query_planner.getQueryNodeToPlanStepMapping();
query_node_to_plan_step_mapping.insert(mapping.begin(), mapping.end());
auto query_node_plan = std::make_unique<QueryPlan>(std::move(query_planner).extractQueryPlan());
query_plans_headers.push_back(query_node_plan->getCurrentDataStream().header);
query_plans.push_back(std::move(query_node_plan));
@ -1402,16 +1408,27 @@ void Planner::buildPlanForQueryNode()
}
}
auto top_level_identifiers = collectTopLevelColumnIdentifiers(query_tree, planner_context);
auto join_tree_query_plan = buildJoinTreeQueryPlan(query_tree,
select_query_info,
select_query_options,
top_level_identifiers,
planner_context);
JoinTreeQueryPlan join_tree_query_plan;
if (planner_context->getMutableQueryContext()->canUseTaskBasedParallelReplicas()
&& planner_context->getGlobalPlannerContext()->parallel_replicas_node == &query_node)
{
join_tree_query_plan = buildQueryPlanForParallelReplicas(query_node, planner_context, select_query_info.storage_limits);
}
else
{
auto top_level_identifiers = collectTopLevelColumnIdentifiers(query_tree, planner_context);
join_tree_query_plan = buildJoinTreeQueryPlan(query_tree,
select_query_info,
select_query_options,
top_level_identifiers,
planner_context);
}
auto from_stage = join_tree_query_plan.from_stage;
query_plan = std::move(join_tree_query_plan.query_plan);
used_row_policies = std::move(join_tree_query_plan.used_row_policies);
auto & mapping = join_tree_query_plan.query_node_to_plan_step_mapping;
query_node_to_plan_step_mapping.insert(mapping.begin(), mapping.end());
LOG_TRACE(getLogger("Planner"), "Query {} from stage {} to stage {}{}",
query_tree->formatConvertedASTForErrorMessage(),
@ -1681,6 +1698,8 @@ void Planner::buildPlanForQueryNode()
if (!select_query_options.only_analyze)
addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute);
query_node_to_plan_step_mapping[&query_node] = query_plan.getRootNode();
}
SelectQueryInfo Planner::buildSelectQueryInfo() const

View File

@ -65,6 +65,11 @@ public:
return planner_context;
}
/// We support mapping QueryNode -> QueryPlanStep (the last step added to plan from this query)
/// It is useful for parallel replicas analysis.
using QueryNodeToPlanStepMapping = std::unordered_map<const QueryNode *, const QueryPlan::Node *>;
const QueryNodeToPlanStepMapping & getQueryNodeToPlanStepMapping() const { return query_node_to_plan_step_mapping; }
private:
void buildPlanForUnionNode();
@ -76,6 +81,7 @@ private:
QueryPlan query_plan;
StorageLimitsList storage_limits;
std::set<std::string> used_row_policies;
QueryNodeToPlanStepMapping query_node_to_plan_step_mapping;
};
}

View File

@ -18,10 +18,18 @@ namespace DB
*
* 1. Column identifiers.
*/
class QueryNode;
class TableNode;
class GlobalPlannerContext
{
public:
GlobalPlannerContext() = default;
explicit GlobalPlannerContext(const QueryNode * parallel_replicas_node_, const TableNode * parallel_replicas_table_)
: parallel_replicas_node(parallel_replicas_node_)
, parallel_replicas_table(parallel_replicas_table_)
{
}
/** Create column identifier for column node.
*
@ -38,6 +46,13 @@ public:
/// Check if context has column identifier
bool hasColumnIdentifier(const ColumnIdentifier & column_identifier);
/// The query which will be executed with parallel replicas.
/// In case if only the most inner subquery can be executed with parallel replicas, node is nullptr.
const QueryNode * const parallel_replicas_node = nullptr;
/// Table which is used with parallel replicas reading. Now, only one table is supported by the protocol.
/// It is the left-most table of the query (in JOINs, UNIONs and subqueries).
const TableNode * const parallel_replicas_table = nullptr;
private:
std::unordered_set<ColumnIdentifier> column_identifiers;
};

View File

@ -613,6 +613,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
auto * union_node = table_expression->as<UnionNode>();
QueryPlan query_plan;
std::unordered_map<const QueryNode *, const QueryPlan::Node *> query_node_to_plan_step_mapping;
std::set<std::string> used_row_policies;
if (table_node || table_function_node)
@ -623,6 +624,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
auto table_expression_query_info = select_query_info;
table_expression_query_info.table_expression = table_expression;
table_expression_query_info.filter_actions_dag = table_expression_data.getFilterActions();
table_expression_query_info.analyzer_can_use_parallel_replicas_on_follower = table_node == planner_context->getGlobalPlannerContext()->parallel_replicas_table;
size_t max_streams = settings.max_threads;
size_t max_threads_execute_query = settings.max_threads;
@ -915,6 +917,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
/// Propagate storage limits to subquery
subquery_planner.addStorageLimits(*select_query_info.storage_limits);
subquery_planner.buildQueryPlanIfNeeded();
const auto & mapping = subquery_planner.getQueryNodeToPlanStepMapping();
query_node_to_plan_step_mapping.insert(mapping.begin(), mapping.end());
query_plan = std::move(subquery_planner).extractQueryPlan();
}
}
@ -974,6 +978,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
.query_plan = std::move(query_plan),
.from_stage = from_stage,
.used_row_policies = std::move(used_row_policies),
.query_node_to_plan_step_mapping = std::move(query_node_to_plan_step_mapping),
};
}
@ -1520,11 +1525,16 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
if (join_clauses_and_actions.right_join_expressions_actions)
left_join_tree_query_plan.actions_dags.emplace_back(std::move(join_clauses_and_actions.right_join_expressions_actions));
auto mapping = std::move(left_join_tree_query_plan.query_node_to_plan_step_mapping);
auto & r_mapping = right_join_tree_query_plan.query_node_to_plan_step_mapping;
mapping.insert(r_mapping.begin(), r_mapping.end());
return JoinTreeQueryPlan{
.query_plan = std::move(result_plan),
.from_stage = QueryProcessingStage::FetchColumns,
.used_row_policies = std::move(left_join_tree_query_plan.used_row_policies),
.actions_dags = std::move(left_join_tree_query_plan.actions_dags),
.query_node_to_plan_step_mapping = std::move(mapping),
};
}
@ -1611,6 +1621,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
.from_stage = QueryProcessingStage::FetchColumns,
.used_row_policies = std::move(join_tree_query_plan.used_row_policies),
.actions_dags = std::move(join_tree_query_plan.actions_dags),
.query_node_to_plan_step_mapping = std::move(join_tree_query_plan.query_node_to_plan_step_mapping),
};
}

View File

@ -17,6 +17,7 @@ struct JoinTreeQueryPlan
QueryProcessingStage::Enum from_stage;
std::set<std::string> used_row_policies;
std::vector<ActionsDAGPtr> actions_dags;
std::unordered_map<const QueryNode *, const QueryPlan::Node *> query_node_to_plan_step_mapping;
};
/// Build JOIN TREE query plan for query node

View File

@ -37,6 +37,8 @@
#include <Planner/CollectTableExpressionData.h>
#include <Planner/CollectSets.h>
#include <stack>
namespace DB
{
@ -130,6 +132,34 @@ ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node)
return result_ast;
}
static void removeCTEs(ASTPtr & ast)
{
std::stack<IAST *> stack;
stack.push(ast.get());
while (!stack.empty())
{
auto * node = stack.top();
stack.pop();
if (auto * subquery = typeid_cast<ASTSubquery *>(node))
subquery->cte_name = {};
for (const auto & child : node->children)
stack.push(child.get());
}
}
ASTPtr queryNodeToDistributedSelectQuery(const QueryTreeNodePtr & query_node)
{
auto ast = queryNodeToSelectQuery(query_node);
/// Remove CTEs information from distributed queries.
/// Now, if cte_name is set for subquery node, AST -> String serialization will only print cte name.
/// But CTE is defined only for top-level query part, so may not be sent.
/// Removing cte_name forces subquery to be always printed.
removeCTEs(ast);
return ast;
}
/** There are no limits on the maximum size of the result for the subquery.
* Since the result of the query is not the result of the entire query.
*/

View File

@ -34,6 +34,9 @@ Block buildCommonHeaderForUnion(const Blocks & queries_headers, SelectUnionMode
/// Convert query node to ASTSelectQuery
ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node);
/// Convert query node to ASTSelectQuery for distributed processing
ASTPtr queryNodeToDistributedSelectQuery(const QueryTreeNodePtr & query_node);
/// Build context for subquery execution
ContextPtr buildSubqueryContext(const ContextPtr & context);

View File

@ -0,0 +1,438 @@
#include <Planner/findQueryForParallelReplicas.h>
#include <Interpreters/ClusterProxy/SelectStreamFactory.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Processors/QueryPlan/JoinStep.h>
#include <Storages/buildQueryTreeForShard.h>
#include <Interpreters/ClusterProxy/executeQuery.h>
#include <Planner/PlannerJoinTree.h>
#include <Planner/Utils.h>
#include <Analyzer/ArrayJoinNode.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/JoinNode.h>
#include <Analyzer/QueryNode.h>
#include <Analyzer/TableNode.h>
#include <Analyzer/UnionNode.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/queryToString.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/FilterStep.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/StorageDummy.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int UNSUPPORTED_METHOD;
}
/// Returns a list of (sub)queries (candidates) which may support parallel replicas.
/// The rule is :
/// subquery has only LEFT or ALL INNER JOIN (or none), and left part is MergeTree table or subquery candidate as well.
///
/// Additional checks are required, so we return many candidates. The innermost subquery is on top.
std::stack<const QueryNode *> getSupportingParallelReplicasQuery(const IQueryTreeNode * query_tree_node)
{
std::stack<const QueryNode *> res;
while (query_tree_node)
{
auto join_tree_node_type = query_tree_node->getNodeType();
switch (join_tree_node_type)
{
case QueryTreeNodeType::TABLE:
{
const auto & table_node = query_tree_node->as<TableNode &>();
const auto & storage = table_node.getStorage();
/// 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()))
return res;
return {};
}
case QueryTreeNodeType::TABLE_FUNCTION:
{
return {};
}
case QueryTreeNodeType::QUERY:
{
const auto & query_node_to_process = query_tree_node->as<QueryNode &>();
query_tree_node = query_node_to_process.getJoinTree().get();
res.push(&query_node_to_process);
break;
}
case QueryTreeNodeType::UNION:
{
const auto & union_node = query_tree_node->as<UnionNode &>();
const auto & union_queries = union_node.getQueries().getNodes();
if (union_queries.empty())
return {};
query_tree_node = union_queries.front().get();
break;
}
case QueryTreeNodeType::ARRAY_JOIN:
{
const auto & array_join_node = query_tree_node->as<ArrayJoinNode &>();
query_tree_node = array_join_node.getTableExpression().get();
break;
}
case QueryTreeNodeType::JOIN:
{
const auto & join_node = query_tree_node->as<JoinNode &>();
auto join_kind = join_node.getKind();
auto join_strictness = join_node.getStrictness();
bool can_parallelize_join =
join_kind == JoinKind::Left
|| (join_kind == JoinKind::Inner && join_strictness == JoinStrictness::All);
if (!can_parallelize_join)
return {};
query_tree_node = join_node.getLeftTableExpression().get();
break;
}
default:
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Unexpected node type for table expression. "
"Expected table, table function, query, union, join or array join. Actual {}",
query_tree_node->getNodeTypeName());
}
}
}
return res;
}
class ReplaceTableNodeToDummyVisitor : public InDepthQueryTreeVisitor<ReplaceTableNodeToDummyVisitor, true>
{
public:
using Base = InDepthQueryTreeVisitor<ReplaceTableNodeToDummyVisitor, true>;
using Base::Base;
void visitImpl(const QueryTreeNodePtr & node)
{
auto * table_node = node->as<TableNode>();
auto * table_function_node = node->as<TableFunctionNode>();
if (table_node || table_function_node)
{
const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot();
auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals();
auto storage_dummy
= std::make_shared<StorageDummy>(storage_snapshot->storage.getStorageID(), ColumnsDescription(storage_snapshot->getColumns(get_column_options)));
auto dummy_table_node = std::make_shared<TableNode>(std::move(storage_dummy), context);
dummy_table_node->setAlias(node->getAlias());
replacement_map.emplace(node.get(), std::move(dummy_table_node));
}
}
ContextPtr context;
std::unordered_map<const IQueryTreeNode *, QueryTreeNodePtr> replacement_map;
};
QueryTreeNodePtr replaceTablesWithDummyTables(const QueryTreeNodePtr & query, const ContextPtr & context)
{
ReplaceTableNodeToDummyVisitor visitor;
visitor.context = context;
visitor.visit(query);
return query->cloneAndReplace(visitor.replacement_map);
}
/// Find the best candidate for parallel replicas execution by verifying query plan.
/// If query plan has only Expression, Filter of Join steps, we can execute it fully remotely and check the next query.
/// Otherwise we can execute current query up to WithMergableStage only.
const QueryNode * findQueryForParallelReplicas(
std::stack<const QueryNode *> stack,
const std::unordered_map<const QueryNode *, const QueryPlan::Node *> & mapping)
{
const QueryPlan::Node * prev_checked_node = nullptr;
const QueryNode * res = nullptr;
while (!stack.empty())
{
const QueryNode * subquery_node = stack.top();
stack.pop();
auto it = mapping.find(subquery_node);
/// This should not happen ideally.
if (it == mapping.end())
break;
const QueryPlan::Node * curr_node = it->second;
const QueryPlan::Node * next_node_to_check = curr_node;
bool can_distribute_full_node = true;
while (next_node_to_check && next_node_to_check != prev_checked_node)
{
const auto & children = next_node_to_check->children;
auto * step = next_node_to_check->step.get();
if (children.empty())
{
/// Found a source step. This should be possible only in the first iteration.
if (prev_checked_node)
return nullptr;
next_node_to_check = nullptr;
}
else if (children.size() == 1)
{
const auto * expression = typeid_cast<ExpressionStep *>(step);
const auto * filter = typeid_cast<FilterStep *>(step);
if (!expression && !filter)
can_distribute_full_node = false;
next_node_to_check = children.front();
}
else
{
const auto * join = typeid_cast<JoinStep *>(step);
/// We've checked that JOIN is INNER/LEFT in query tree.
/// Don't distribute UNION node.
if (!join)
return res;
next_node_to_check = children.front();
}
}
/// Current node contains steps like GROUP BY / DISTINCT
/// Will try to execute query up to WithMergableStage
if (!can_distribute_full_node)
{
/// Current query node does not contain subqueries.
/// We can execute parallel replicas over storage::read.
if (!res)
return nullptr;
return subquery_node;
}
/// Query is simple enough to be fully distributed.
res = subquery_node;
prev_checked_node = curr_node;
}
return res;
}
const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tree_node, SelectQueryOptions & select_query_options)
{
if (select_query_options.only_analyze)
return nullptr;
auto * query_node = query_tree_node->as<QueryNode>();
auto * union_node = query_tree_node->as<UnionNode>();
if (!query_node && !union_node)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Expected QUERY or UNION node. Actual {}",
query_tree_node->formatASTForErrorMessage());
auto context = query_node ? query_node->getContext() : union_node->getContext();
if (!context->canUseParallelReplicasOnInitiator())
return nullptr;
auto stack = getSupportingParallelReplicasQuery(query_tree_node.get());
/// Empty stack means that storage does not support parallel replicas.
if (stack.empty())
return nullptr;
/// We don't have any subquery and storage can process parallel replicas by itself.
if (stack.top() == query_tree_node.get())
return nullptr;
/// This is needed to avoid infinite recursion.
auto mutable_context = Context::createCopy(context);
mutable_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0));
/// Here we replace tables to dummy, in order to build a temporary query plan for parallel replicas analysis.
ResultReplacementMap replacement_map;
auto updated_query_tree = replaceTablesWithDummyTables(query_tree_node, mutable_context);
SelectQueryOptions options;
Planner planner(updated_query_tree, options, std::make_shared<GlobalPlannerContext>(nullptr, nullptr));
planner.buildQueryPlanIfNeeded();
/// This part is a bit clumsy.
/// We updated a query_tree with dummy storages, and mapping is using updated_query_tree now.
/// But QueryNode result should be taken from initial query tree.
/// So that we build a list of candidates again, and call findQueryForParallelReplicas for it.
auto new_stack = getSupportingParallelReplicasQuery(updated_query_tree.get());
const auto & mapping = planner.getQueryNodeToPlanStepMapping();
const auto * res = findQueryForParallelReplicas(new_stack, mapping);
/// Now, return a query from initial stack.
if (res)
{
while (!new_stack.empty())
{
if (res == new_stack.top())
return stack.top();
stack.pop();
new_stack.pop();
}
}
return res;
}
static const TableNode * findTableForParallelReplicas(const IQueryTreeNode * query_tree_node)
{
std::stack<const IQueryTreeNode *> right_join_nodes;
while (query_tree_node || !right_join_nodes.empty())
{
if (!query_tree_node)
{
query_tree_node = right_join_nodes.top();
right_join_nodes.pop();
}
auto join_tree_node_type = query_tree_node->getNodeType();
switch (join_tree_node_type)
{
case QueryTreeNodeType::TABLE:
{
const auto & table_node = query_tree_node->as<TableNode &>();
const auto & storage = table_node.getStorage();
if (std::dynamic_pointer_cast<MergeTreeData>(storage) || typeid_cast<const StorageDummy *>(storage.get()))
return &table_node;
query_tree_node = nullptr;
break;
}
case QueryTreeNodeType::TABLE_FUNCTION:
{
query_tree_node = nullptr;
break;
}
case QueryTreeNodeType::QUERY:
{
const auto & query_node_to_process = query_tree_node->as<QueryNode &>();
query_tree_node = query_node_to_process.getJoinTree().get();
break;
}
case QueryTreeNodeType::UNION:
{
const auto & union_node = query_tree_node->as<UnionNode &>();
const auto & union_queries = union_node.getQueries().getNodes();
query_tree_node = nullptr;
if (!union_queries.empty())
query_tree_node = union_queries.front().get();
break;
}
case QueryTreeNodeType::ARRAY_JOIN:
{
const auto & array_join_node = query_tree_node->as<ArrayJoinNode &>();
query_tree_node = array_join_node.getTableExpression().get();
break;
}
case QueryTreeNodeType::JOIN:
{
const auto & join_node = query_tree_node->as<JoinNode &>();
query_tree_node = join_node.getLeftTableExpression().get();
right_join_nodes.push(join_node.getRightTableExpression().get());
break;
}
default:
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Unexpected node type for table expression. "
"Expected table, table function, query, union, join or array join. Actual {}",
query_tree_node->getNodeTypeName());
}
}
}
return nullptr;
}
const TableNode * findTableForParallelReplicas(const QueryTreeNodePtr & query_tree_node, SelectQueryOptions & select_query_options)
{
if (select_query_options.only_analyze)
return nullptr;
auto * query_node = query_tree_node->as<QueryNode>();
auto * union_node = query_tree_node->as<UnionNode>();
if (!query_node && !union_node)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Expected QUERY or UNION node. Actual {}",
query_tree_node->formatASTForErrorMessage());
auto context = query_node ? query_node->getContext() : union_node->getContext();
if (!context->canUseParallelReplicasOnFollower())
return nullptr;
return findTableForParallelReplicas(query_tree_node.get());
}
JoinTreeQueryPlan buildQueryPlanForParallelReplicas(
const QueryNode & query_node,
const PlannerContextPtr & planner_context,
std::shared_ptr<const StorageLimitsList> storage_limits)
{
auto processed_stage = QueryProcessingStage::WithMergeableState;
auto context = planner_context->getQueryContext();
QueryTreeNodePtr modified_query_tree = query_node.clone();
Block initial_header = InterpreterSelectQueryAnalyzer::getSampleBlock(
modified_query_tree, context, SelectQueryOptions(processed_stage).analyze());
rewriteJoinToGlobalJoin(modified_query_tree, context);
modified_query_tree = buildQueryTreeForShard(planner_context, modified_query_tree);
ASTPtr modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree);
Block header = InterpreterSelectQueryAnalyzer::getSampleBlock(
modified_query_tree, context, SelectQueryOptions(processed_stage).analyze());
ClusterProxy::SelectStreamFactory select_stream_factory =
ClusterProxy::SelectStreamFactory(
header,
{},
{},
processed_stage);
QueryPlan query_plan;
ClusterProxy::executeQueryWithParallelReplicas(
query_plan,
select_stream_factory,
modified_query_ast,
context,
storage_limits);
auto converting = ActionsDAG::makeConvertingActions(
header.getColumnsWithTypeAndName(),
initial_header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Position);
/// initial_header is a header expected by initial query.
/// header is a header which is returned by the follower.
/// They are different because tables will have different aliases (e.g. _table1 or _table5).
/// Here we just rename columns by position, with the hope the types would match.
auto step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(converting));
step->setStepDescription("Convert distributed names");
query_plan.addStep(std::move(step));
return {std::move(query_plan), std::move(processed_stage), {}, {}, {}};
}
}

View File

@ -0,0 +1,38 @@
#pragma once
#include <list>
#include <memory>
namespace DB
{
class QueryNode;
class TableNode;
class IQueryTreeNode;
using QueryTreeNodePtr = std::shared_ptr<IQueryTreeNode>;
struct SelectQueryOptions;
/// Find a qury which can be executed with parallel replicas up to WithMergableStage.
/// Returned query will always contain some (>1) subqueries, possibly with joins.
const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tree_node, SelectQueryOptions & select_query_options);
/// Find a table from which we should read on follower replica. It's the left-most table within all JOINs and UNIONs.
const TableNode * findTableForParallelReplicas(const QueryTreeNodePtr & query_tree_node, SelectQueryOptions & select_query_options);
struct JoinTreeQueryPlan;
class PlannerContext;
using PlannerContextPtr = std::shared_ptr<PlannerContext>;
struct StorageLimits;
using StorageLimitsList = std::list<StorageLimits>;
/// Execute QueryNode with parallel replicas up to WithMergableStage and return a plan.
/// This method does not check that QueryNode is valid. Ideally it should be a result of findParallelReplicasQuery.
JoinTreeQueryPlan buildQueryPlanForParallelReplicas(
const QueryNode & query_node,
const PlannerContextPtr & planner_context,
std::shared_ptr<const StorageLimitsList> storage_limits);
}

View File

@ -5,6 +5,7 @@
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Parsers/ASTInsertQuery.h>
#include <Processors/Transforms/CountingTransform.h>
#include <Processors/Transforms/SquashingChunksTransform.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
@ -412,6 +413,23 @@ Chain buildPushingToViewsChain(
InterpreterInsertQuery interpreter(nullptr, view_insert_context, false, false, false);
out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, thread_status_holder, view_counter_ms);
if (interpreter.shouldAddSquashingFroStorage(inner_table))
{
bool table_prefers_large_blocks = inner_table->prefersLargeBlocks();
const auto & settings = view_insert_context->getSettingsRef();
out.addSource(std::make_shared<SquashingChunksTransform>(
out.getInputHeader(),
table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size,
table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL));
}
auto counting = std::make_shared<CountingTransform>(out.getInputHeader(), current_thread, view_insert_context->getQuota());
counting->setProcessListElement(view_insert_context->getProcessListElement());
counting->setProgressCallback(view_insert_context->getProgressCallback());
out.addSource(std::move(counting));
out.addStorageHolder(view);
out.addStorageHolder(inner_table);
}

View File

@ -933,7 +933,7 @@ void TCPHandler::processInsertQuery()
if (auto table = DatabaseCatalog::instance().tryGetTable(insert_query.table_id, query_context))
async_insert_enabled |= table->areAsynchronousInsertsEnabled();
if (insert_queue && async_insert_enabled && !insert_query.select && !settings.deduplicate_blocks_in_dependent_materialized_views)
if (insert_queue && async_insert_enabled && !insert_query.select)
{
auto result = processAsyncInsertQuery(*insert_queue);
if (result.status == AsynchronousInsertQueue::PushResult::OK)

View File

@ -436,7 +436,7 @@ MergeTask::StageRuntimeContextPtr MergeTask::VerticalMergeStage::getContextForNe
bool MergeTask::ExecuteAndFinalizeHorizontalPart::execute()
{
assert(subtasks_iterator != subtasks.end());
if ((*subtasks_iterator)())
if ((this->**subtasks_iterator)())
return true;
/// Move to the next subtask in an array of subtasks
@ -827,7 +827,7 @@ bool MergeTask::MergeProjectionsStage::finalizeProjectionsAndWholeMerge() const
bool MergeTask::VerticalMergeStage::execute()
{
assert(subtasks_iterator != subtasks.end());
if ((*subtasks_iterator)())
if ((this->**subtasks_iterator)())
return true;
/// Move to the next subtask in an array of subtasks
@ -838,7 +838,7 @@ bool MergeTask::VerticalMergeStage::execute()
bool MergeTask::MergeProjectionsStage::execute()
{
assert(subtasks_iterator != subtasks.end());
if ((*subtasks_iterator)())
if ((this->**subtasks_iterator)())
return true;
/// Move to the next subtask in an array of subtasks

View File

@ -246,15 +246,16 @@ private:
bool prepare();
bool executeImpl();
using ExecuteAndFinalizeHorizontalPartSubtasks = std::array<std::function<bool()>, 2>;
/// NOTE: Using pointer-to-member instead of std::function and lambda makes stacktraces much more concise and readable
using ExecuteAndFinalizeHorizontalPartSubtasks = std::array<bool(ExecuteAndFinalizeHorizontalPart::*)(), 2>;
ExecuteAndFinalizeHorizontalPartSubtasks subtasks
const ExecuteAndFinalizeHorizontalPartSubtasks subtasks
{
[this] () { return prepare(); },
[this] () { return executeImpl(); }
&ExecuteAndFinalizeHorizontalPart::prepare,
&ExecuteAndFinalizeHorizontalPart::executeImpl
};
ExecuteAndFinalizeHorizontalPartSubtasks::iterator subtasks_iterator = subtasks.begin();
ExecuteAndFinalizeHorizontalPartSubtasks::const_iterator subtasks_iterator = subtasks.begin();
MergeAlgorithm chooseMergeAlgorithm() const;
@ -323,16 +324,17 @@ private:
bool executeVerticalMergeForAllColumns() const;
bool finalizeVerticalMergeForAllColumns() const;
using VerticalMergeStageSubtasks = std::array<std::function<bool()>, 3>;
/// NOTE: Using pointer-to-member instead of std::function and lambda makes stacktraces much more concise and readable
using VerticalMergeStageSubtasks = std::array<bool(VerticalMergeStage::*)()const, 3>;
VerticalMergeStageSubtasks subtasks
const VerticalMergeStageSubtasks subtasks
{
[this] () { return prepareVerticalMergeForAllColumns(); },
[this] () { return executeVerticalMergeForAllColumns(); },
[this] () { return finalizeVerticalMergeForAllColumns(); }
&VerticalMergeStage::prepareVerticalMergeForAllColumns,
&VerticalMergeStage::executeVerticalMergeForAllColumns,
&VerticalMergeStage::finalizeVerticalMergeForAllColumns
};
VerticalMergeStageSubtasks::iterator subtasks_iterator = subtasks.begin();
VerticalMergeStageSubtasks::const_iterator subtasks_iterator = subtasks.begin();
void prepareVerticalMergeForOneColumn() const;
bool executeVerticalMergeForOneColumn() const;
@ -373,16 +375,17 @@ private:
bool executeProjections() const;
bool finalizeProjectionsAndWholeMerge() const;
using MergeProjectionsStageSubtasks = std::array<std::function<bool()>, 3>;
/// NOTE: Using pointer-to-member instead of std::function and lambda makes stacktraces much more concise and readable
using MergeProjectionsStageSubtasks = std::array<bool(MergeProjectionsStage::*)()const, 3>;
MergeProjectionsStageSubtasks subtasks
const MergeProjectionsStageSubtasks subtasks
{
[this] () { return mergeMinMaxIndexAndPrepareProjections(); },
[this] () { return executeProjections(); },
[this] () { return finalizeProjectionsAndWholeMerge(); }
&MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections,
&MergeProjectionsStage::executeProjections,
&MergeProjectionsStage::finalizeProjectionsAndWholeMerge
};
MergeProjectionsStageSubtasks::iterator subtasks_iterator = subtasks.begin();
MergeProjectionsStageSubtasks::const_iterator subtasks_iterator = subtasks.begin();
MergeProjectionsRuntimeContextPtr ctx;
GlobalRuntimeContextPtr global_ctx;
@ -392,14 +395,14 @@ private:
using Stages = std::array<StagePtr, 3>;
Stages stages
const Stages stages
{
std::make_shared<ExecuteAndFinalizeHorizontalPart>(),
std::make_shared<VerticalMergeStage>(),
std::make_shared<MergeProjectionsStage>()
};
Stages::iterator stages_iterator = stages.begin();
Stages::const_iterator stages_iterator = stages.begin();
/// Check for persisting block number column
static bool supportsBlockNumberColumn(GlobalRuntimeContextPtr global_ctx)

View File

@ -161,6 +161,8 @@ struct SelectQueryInfo
/// It's guaranteed to be present in JOIN TREE of `query_tree`
QueryTreeNodePtr table_expression;
bool analyzer_can_use_parallel_replicas_on_follower = false;
/// Table expression modifiers for storage
std::optional<TableExpressionModifiers> table_expression_modifiers;

View File

@ -797,7 +797,7 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info,
auto query_tree_to_modify = query_info.query_tree->cloneAndReplace(query_info.table_expression, std::move(replacement_table_expression));
return buildQueryTreeForShard(query_info, query_tree_to_modify);
return buildQueryTreeForShard(query_info.planner_context, query_tree_to_modify);
}
}
@ -831,7 +831,7 @@ void StorageDistributed::read(
*/
for (auto & column : header)
column.column = column.column->convertToFullColumnIfConst();
query_ast = queryNodeToSelectQuery(query_tree_distributed);
query_ast = queryNodeToDistributedSelectQuery(query_tree_distributed);
}
else
{
@ -976,8 +976,10 @@ std::optional<QueryPipeline> StorageDistributed::distributedWriteBetweenDistribu
new_query->select = select_with_union_query;
}
const Cluster::AddressesWithFailover & src_addresses = src_distributed.getCluster()->getShardsAddresses();
const Cluster::AddressesWithFailover & dst_addresses = getCluster()->getShardsAddresses();
const auto src_cluster = src_distributed.getCluster();
const auto dst_cluster = getCluster();
const Cluster::AddressesWithFailover & src_addresses = src_cluster->getShardsAddresses();
const Cluster::AddressesWithFailover & dst_addresses = dst_cluster->getShardsAddresses();
/// Compare addresses instead of cluster name, to handle remote()/cluster().
/// (since for remote()/cluster() the getClusterName() is empty string)
if (src_addresses != dst_addresses)
@ -1006,8 +1008,7 @@ std::optional<QueryPipeline> StorageDistributed::distributedWriteBetweenDistribu
new_query->table_function.reset();
}
const auto & cluster = getCluster();
const auto & shards_info = cluster->getShardsInfo();
const auto & shards_info = dst_cluster->getShardsInfo();
String new_query_str;
{
@ -1138,7 +1139,8 @@ std::optional<QueryPipeline> StorageDistributed::distributedWriteFromClusterStor
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
/// Here we take addresses from destination cluster and assume source table exists on these nodes
for (const auto & replicas : getCluster()->getShardsInfo())
const auto cluster = getCluster();
for (const auto & replicas : cluster->getShardsInfo())
{
/// Skip unavailable hosts if necessary
auto try_results = replicas.pool->getMany(timeouts, current_settings, PoolMode::GET_MANY, /*async_callback*/ {}, /*skip_unavailable_endpoints*/ true);

View File

@ -221,11 +221,11 @@ void StorageMergeTree::read(
if (local_context->getSettingsRef().allow_experimental_analyzer)
{
QueryTreeNodePtr modified_query_tree = query_info.query_tree->clone();
rewriteJoinToGlobalJoin(modified_query_tree);
modified_query_tree = buildQueryTreeForShard(query_info, modified_query_tree);
rewriteJoinToGlobalJoin(modified_query_tree, local_context);
modified_query_tree = buildQueryTreeForShard(query_info.planner_context, modified_query_tree);
header = InterpreterSelectQueryAnalyzer::getSampleBlock(
modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze());
modified_query_ast = queryNodeToSelectQuery(modified_query_tree);
modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree);
}
else
{
@ -252,7 +252,9 @@ void StorageMergeTree::read(
}
else
{
const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree;
const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower()
&& local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree
&& (!local_context->getSettingsRef().allow_experimental_analyzer || query_info.analyzer_can_use_parallel_replicas_on_follower);
if (auto plan = reader.read(
column_names,

View File

@ -513,8 +513,15 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
if (same_structure)
{
Coordination::Stat metadata_stat;
current_zookeeper->get(zookeeper_path + "/metadata", &metadata_stat);
current_zookeeper->get(fs::path(zookeeper_path) / "metadata", &metadata_stat);
/** We change metadata_snapshot so that `createReplica` method will create `metadata_version` node in ZooKeeper
* with version of table '/metadata' node in Zookeeper.
*
* Otherwise `metadata_version` for not first replica will be initialized with 0 by default.
*/
setInMemoryMetadata(metadata_snapshot->withMetadataVersion(metadata_stat.version));
metadata_snapshot = getInMemoryMetadataPtr();
}
}
catch (Coordination::Exception & e)
@ -5368,12 +5375,12 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl(
if (local_context->getSettingsRef().allow_experimental_analyzer)
{
QueryTreeNodePtr modified_query_tree = query_info.query_tree->clone();
rewriteJoinToGlobalJoin(modified_query_tree);
modified_query_tree = buildQueryTreeForShard(query_info, modified_query_tree);
rewriteJoinToGlobalJoin(modified_query_tree, local_context);
modified_query_tree = buildQueryTreeForShard(query_info.planner_context, modified_query_tree);
header = InterpreterSelectQueryAnalyzer::getSampleBlock(
modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze());
modified_query_ast = queryNodeToSelectQuery(modified_query_tree);
modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree);
}
else
{
@ -5407,11 +5414,14 @@ void StorageReplicatedMergeTree::readLocalImpl(
const size_t max_block_size,
const size_t num_streams)
{
const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower()
&& (!local_context->getSettingsRef().allow_experimental_analyzer || query_info.analyzer_can_use_parallel_replicas_on_follower);
auto plan = reader.read(
column_names, storage_snapshot, query_info,
local_context, max_block_size, num_streams,
/* max_block_numbers_to_read= */ nullptr,
/* enable_parallel_reading= */ local_context->canUseParallelReplicasOnFollower());
enable_parallel_reading);
if (plan)
query_plan = std::move(*plan);
@ -5814,6 +5824,7 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer
Coordination::Requests requests;
requests.emplace_back(zkutil::makeSetRequest(fs::path(replica_path) / "columns", entry.columns_str, -1));
requests.emplace_back(zkutil::makeSetRequest(fs::path(replica_path) / "metadata", entry.metadata_str, -1));
requests.emplace_back(zkutil::makeSetRequest(fs::path(replica_path) / "metadata_version", std::to_string(entry.alter_version), -1));
auto table_id = getStorageID();
auto alter_context = getContext();
@ -5860,10 +5871,6 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer
resetObjectColumnsFromActiveParts(parts_lock);
}
/// This transaction may not happen, but it's OK, because on the next retry we will eventually create/update this node
/// TODO Maybe do in in one transaction for Replicated database?
zookeeper->createOrUpdate(fs::path(replica_path) / "metadata_version", std::to_string(current_metadata->getMetadataVersion()), zkutil::CreateMode::Persistent);
return true;
}

View File

@ -26,192 +26,329 @@ void StorageSystemDashboards::fillData(MutableColumns & res_columns, ContextPtr,
{
static const std::vector<std::map<String, String>> dashboards
{
/// Default dashboard for self-managed ClickHouse
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "Queries/second" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_Query)
FROM system.metric_log
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "CPU Usage (cores)" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSCPUVirtualTimeMicroseconds) / 1000000
FROM system.metric_log
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "Queries Running" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(CurrentMetric_Query)
FROM system.metric_log
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "Merges Running" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(CurrentMetric_Merge)
FROM system.metric_log
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "Selected Bytes/second" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_SelectedBytes)
FROM system.metric_log
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "IO Wait" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSIOWaitMicroseconds) / 1000000
FROM system.metric_log
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "CPU Wait" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSCPUWaitMicroseconds) / 1000000
FROM system.metric_log
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "OS CPU Usage (Userspace)" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value)
FROM system.asynchronous_metric_log
FROM merge('system', '^asynchronous_metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'OSUserTimeNormalized'
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "OS CPU Usage (Kernel)" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value)
FROM system.asynchronous_metric_log
FROM merge('system', '^asynchronous_metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'OSSystemTimeNormalized'
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "Read From Disk" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSReadBytes)
FROM system.metric_log
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "Read From Filesystem" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_OSReadChars)
FROM system.metric_log
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "Memory (tracked)" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(CurrentMetric_MemoryTracking)
FROM system.metric_log
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "Load Average (15 minutes)" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value)
FROM system.asynchronous_metric_log
FROM merge('system', '^asynchronous_metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'LoadAverage15'
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "Selected Rows/second" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_SelectedRows)
FROM system.metric_log
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "Inserted Rows/second" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(ProfileEvent_InsertedRows)
FROM system.metric_log
FROM merge('system', '^metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "Total MergeTree Parts" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value)
FROM system.asynchronous_metric_log
FROM merge('system', '^asynchronous_metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'TotalPartsOfMergeTreeTables'
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
{
{ "dashboard", "overview" },
{ "dashboard", "Overview" },
{ "title", "Max Parts For Partition" },
{ "query", trim(R"EOQ(
SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, max(value)
FROM system.asynchronous_metric_log
FROM merge('system', '^asynchronous_metric_log')
WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'MaxPartCountForPartition'
GROUP BY t
ORDER BY t WITH FILL STEP {rounding:UInt32}
)EOQ") }
},
/// Default dashboard for ClickHouse Cloud
{
{ "dashboard", "Cloud overview" },
{ "title", "Queries/second" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_Query) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "CPU Usage (cores)" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) / 1000000\nFROM (\n SELECT event_time, sum(ProfileEvent_OSCPUVirtualTimeMicroseconds) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32} GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Queries Running" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(CurrentMetric_Query) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Merges Running" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(CurrentMetric_Merge) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Selected Bytes/second" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_SelectedBytes) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "IO Wait (local fs)" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_OSIOWaitMicroseconds) / 1000000 AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "S3 read wait" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_ReadBufferFromS3Microseconds) / 1000000 AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "S3 read errors/sec" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_ReadBufferFromS3RequestsErrors) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "CPU Wait" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_OSCPUWaitMicroseconds) / 1000000 AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "OS CPU Usage (Userspace, normalized)" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'OSUserTimeNormalized'\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "OS CPU Usage (Kernel, normalized)" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'OSSystemTimeNormalized'\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Read From Disk (bytes/sec)" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_OSReadBytes) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Read From Filesystem (bytes/sec)" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_OSReadChars) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Memory (tracked, bytes)" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(CurrentMetric_MemoryTracking) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Load Average (15 minutes)" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value)\nFROM (\n SELECT event_time, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric = 'LoadAverage15'\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Selected Rows/sec" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_SelectedRows) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Inserted Rows/sec" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_InsertedRows) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Total MergeTree Parts" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, max(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'TotalPartsOfMergeTreeTables'\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Max Parts For Partition" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, max(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'MaxPartCountForPartition'\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Read From S3 (bytes/sec)" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_ReadBufferFromS3Bytes) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Filesystem Cache Size" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(CurrentMetric_FilesystemCacheSize) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Disk S3 write req/sec" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_DiskS3PutObject + ProfileEvent_DiskS3UploadPart + ProfileEvent_DiskS3CreateMultipartUpload + ProfileEvent_DiskS3CompleteMultipartUpload) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Disk S3 read req/sec" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_DiskS3GetObject + ProfileEvent_DiskS3HeadObject + ProfileEvent_DiskS3ListObjects) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "FS cache hit rate" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_CachedReadBufferReadFromCacheBytes) / (sum(ProfileEvent_CachedReadBufferReadFromCacheBytes) + sum(ProfileEvent_CachedReadBufferReadFromSourceBytes)) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Page cache hit rate" },
{ "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, greatest(0, (sum(ProfileEvent_OSReadChars) - sum(ProfileEvent_OSReadBytes)) / (sum(ProfileEvent_OSReadChars) + sum(ProfileEvent_ReadBufferFromS3Bytes))) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Network receive bytes/sec" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value)\nFROM (\n SELECT event_time, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric LIKE 'NetworkReceiveBytes%'\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
},
{
{ "dashboard", "Cloud overview" },
{ "title", "Network send bytes/sec" },
{ "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(value)\nFROM (\n SELECT event_time, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric LIKE 'NetworkSendBytes%'\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" }
}
};

View File

@ -16,6 +16,7 @@
#include <Storages/StorageDummy.h>
#include <Planner/Utils.h>
#include <Processors/Executors/CompletedPipelineExecutor.h>
#include <Processors/Transforms/SquashingChunksTransform.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
@ -283,7 +284,16 @@ TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node,
auto optimization_settings = QueryPlanOptimizationSettings::fromContext(mutable_context);
auto build_pipeline_settings = BuildQueryPipelineSettings::fromContext(mutable_context);
auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*query_plan.buildQueryPipeline(optimization_settings, build_pipeline_settings)));
auto builder = query_plan.buildQueryPipeline(optimization_settings, build_pipeline_settings);
size_t min_block_size_rows = mutable_context->getSettingsRef().min_external_table_block_size_rows;
size_t min_block_size_bytes = mutable_context->getSettingsRef().min_external_table_block_size_bytes;
auto squashing = std::make_shared<SimpleSquashingChunksTransform>(builder->getHeader(), min_block_size_rows, min_block_size_bytes);
builder->resize(1);
builder->addTransform(std::move(squashing));
auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
pipeline.complete(std::move(table_out));
CompletedPipelineExecutor executor(pipeline);
@ -295,10 +305,8 @@ TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node,
}
QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeNodePtr query_tree_to_modify)
QueryTreeNodePtr buildQueryTreeForShard(const PlannerContextPtr & planner_context, QueryTreeNodePtr query_tree_to_modify)
{
auto & planner_context = query_info.planner_context;
CollectColumnSourceToColumnsVisitor collect_column_source_to_columns_visitor;
collect_column_source_to_columns_visitor.visit(query_tree_to_modify);
@ -378,16 +386,47 @@ QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeN
return query_tree_to_modify;
}
class RewriteJoinToGlobalJoinVisitor : public InDepthQueryTreeVisitor<RewriteJoinToGlobalJoinVisitor>
class CollectStoragesVisitor : public InDepthQueryTreeVisitor<CollectStoragesVisitor>
{
public:
using Base = InDepthQueryTreeVisitor<RewriteJoinToGlobalJoinVisitor>;
using Base = InDepthQueryTreeVisitor<CollectStoragesVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
{
if (auto * table_node = node->as<TableNode>())
storages.push_back(table_node->getStorage());
}
std::vector<StoragePtr> storages;
};
class RewriteJoinToGlobalJoinVisitor : public InDepthQueryTreeVisitorWithContext<RewriteJoinToGlobalJoinVisitor>
{
public:
using Base = InDepthQueryTreeVisitorWithContext<RewriteJoinToGlobalJoinVisitor>;
using Base::Base;
static bool allStoragesAreMergeTree(QueryTreeNodePtr & node)
{
CollectStoragesVisitor collect_storages;
collect_storages.visit(node);
for (const auto & storage : collect_storages.storages)
if (!storage->isMergeTree())
return false;
return true;
}
void enterImpl(QueryTreeNodePtr & node)
{
if (auto * join_node = node->as<JoinNode>())
join_node->setLocality(JoinLocality::Global);
{
bool prefer_local_join = getContext()->getSettingsRef().parallel_replicas_prefer_local_join;
bool should_use_global_join = !prefer_local_join || !allStoragesAreMergeTree(join_node->getRightTableExpression());
if (should_use_global_join)
join_node->setLocality(JoinLocality::Global);
}
}
static bool needChildVisit(QueryTreeNodePtr & parent, QueryTreeNodePtr & child)
@ -400,9 +439,9 @@ public:
}
};
void rewriteJoinToGlobalJoin(QueryTreeNodePtr query_tree_to_modify)
void rewriteJoinToGlobalJoin(QueryTreeNodePtr query_tree_to_modify, ContextPtr context)
{
RewriteJoinToGlobalJoinVisitor visitor;
RewriteJoinToGlobalJoinVisitor visitor(context);
visitor.visit(query_tree_to_modify);
}

View File

@ -10,8 +10,14 @@ struct SelectQueryInfo;
class IQueryTreeNode;
using QueryTreeNodePtr = std::shared_ptr<IQueryTreeNode>;
QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeNodePtr query_tree_to_modify);
class PlannerContext;
using PlannerContextPtr = std::shared_ptr<PlannerContext>;
void rewriteJoinToGlobalJoin(QueryTreeNodePtr query_tree_to_modify);
class Context;
using ContextPtr = std::shared_ptr<const Context>;
QueryTreeNodePtr buildQueryTreeForShard(const PlannerContextPtr & planner_context, QueryTreeNodePtr query_tree_to_modify);
void rewriteJoinToGlobalJoin(QueryTreeNodePtr query_tree_to_modify, ContextPtr context);
}

View File

@ -42,7 +42,6 @@ protected:
virtual String getFormatFromFirstArgument();
String filename;
String path_to_archive;
String format = "auto";
String structure = "auto";
String compression_method = "auto";

View File

@ -26,6 +26,7 @@ public:
protected:
int fd = -1;
String path_to_archive;
void parseFirstArguments(const ASTPtr & arg, const ContextPtr & context) override;
String getFormatFromFirstArgument() override;

View File

@ -6,10 +6,7 @@ test_executable_table_function/test.py::test_executable_function_input_python
test_mask_sensitive_info/test.py::test_encryption_functions
test_merge_table_over_distributed/test.py::test_global_in
test_merge_table_over_distributed/test.py::test_select_table_name_from_merge_over_distributed
test_mutations_with_merge_tree/test.py::test_mutations_with_merge_background_task
test_passing_max_partitions_to_read_remotely/test.py::test_default_database_on_cluster
test_select_access_rights/test_main.py::test_alias_columns
test_settings_profile/test.py::test_show_profiles
test_shard_level_const_function/test.py::test_remote
test_sql_user_defined_functions_on_cluster/test.py::test_sql_user_defined_functions_on_cluster
test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view

View File

@ -1,5 +1,5 @@
<clickhouse>
<max_table_num_to_warn>10</max_table_num_to_warn>
<max_database_num_to_warn>10</max_database_num_to_warn>
<max_table_num_to_warn>5</max_table_num_to_warn>
<max_database_num_to_warn>2</max_database_num_to_warn>
<max_part_num_to_warn>10</max_part_num_to_warn>
</clickhouse>

View File

@ -19,7 +19,6 @@ Don't use Docker from your system repository.
```
sudo -H pip install \
PyMySQL \
aerospike \
avro \
cassandra-driver \
confluent-kafka \

View File

@ -4,7 +4,6 @@ import os
import uuid
import warnings
import aerospike
import cassandra.cluster
import pymongo
import pymysql.cursors
@ -696,91 +695,3 @@ class SourceRedis(ExternalSource):
or layout.is_complex
and self.storage_type == "hash_map"
)
class SourceAerospike(ExternalSource):
def __init__(
self,
name,
internal_hostname,
internal_port,
docker_hostname,
docker_port,
user,
password,
):
ExternalSource.__init__(
self,
name,
internal_hostname,
internal_port,
docker_hostname,
docker_port,
user,
password,
)
self.namespace = "test"
self.set = "test_set"
def get_source_str(self, table_name):
print("AEROSPIKE get source str")
return """
<aerospike>
<host>{host}</host>
<port>{port}</port>
</aerospike>
""".format(
host=self.docker_hostname,
port=self.docker_port,
)
def prepare(self, structure, table_name, cluster):
config = {"hosts": [(self.internal_hostname, self.internal_port)]}
self.client = aerospike.client(config).connect()
self.prepared = True
print("PREPARED AEROSPIKE")
print(config)
def compatible_with_layout(self, layout):
print("compatible AEROSPIKE")
return layout.is_simple
def _flush_aerospike_db(self):
keys = []
def handle_record(xxx_todo_changeme):
(key, metadata, record) = xxx_todo_changeme
print(("Handle record {} {}".format(key, record)))
keys.append(key)
def print_record(xxx_todo_changeme1):
(key, metadata, record) = xxx_todo_changeme1
print(("Print record {} {}".format(key, record)))
scan = self.client.scan(self.namespace, self.set)
scan.foreach(handle_record)
[self.client.remove(key) for key in keys]
def load_kv_data(self, values):
self._flush_aerospike_db()
print("Load KV Data Aerospike")
if len(values[0]) == 2:
for value in values:
key = (self.namespace, self.set, value[0])
print(key)
self.client.put(
key,
{"bin_value": value[1]},
policy={"key": aerospike.POLICY_KEY_SEND},
)
assert self.client.exists(key)
else:
assert "VALUES SIZE != 2"
# print(values)
def load_data(self, data, table_name):
print("Load Data Aerospike")
# print(data)

View File

@ -1,7 +1,7 @@
<clickhouse>
<profiles>
<default>
<max_expanded_ast_elements>500</max_expanded_ast_elements>
<max_expanded_ast_elements>1800</max_expanded_ast_elements>
<force_index_by_date>1</force_index_by_date>
<force_primary_key>1</force_primary_key>
</default>

View File

@ -43,6 +43,13 @@
</query>
<substitutions>
<substitution>
<name>num_unique_sessions</name>
<values>
<value>1000</value>
<value>1000000</value>
</values>
</substitution>
<substitution>
<name>num_rows</name>
<values>
@ -56,15 +63,15 @@
FROM
(
SELECT
number AS id,
number AS visitor_id
(number % {num_unique_sessions}) AS visitor_id,
number AS id
FROM system.numbers
LIMIT {num_rows}
) AS sessions
ASOF LEFT JOIN
(
SELECT
number AS visitor_id,
(number % {num_unique_sessions}) AS visitor_id,
number AS starting_session_id
FROM system.numbers
LIMIT {num_rows}

View File

@ -22,7 +22,7 @@ CREATE DICTIONARY simple_key_flat_dictionary_01862
value String
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'simple_key_source_table_01862'))
SOURCE(CLICKHOUSE(TABLE 'simple_key_source_table_01862'))
LAYOUT(FLAT())
LIFETIME(MIN 0 MAX 1000);
@ -43,7 +43,7 @@ CREATE DICTIONARY simple_key_direct_dictionary_01862
value String
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'simple_key_source_table_01862'))
SOURCE(CLICKHOUSE(TABLE 'simple_key_source_table_01862'))
LAYOUT(DIRECT());
-- check that found_rate is 0, not nan
@ -65,7 +65,7 @@ CREATE DICTIONARY simple_key_hashed_dictionary_01862
value String
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'simple_key_source_table_01862'))
SOURCE(CLICKHOUSE(TABLE 'simple_key_source_table_01862'))
LAYOUT(HASHED())
LIFETIME(MIN 0 MAX 1000);
@ -85,7 +85,7 @@ CREATE DICTIONARY simple_key_sparse_hashed_dictionary_01862
value String
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'simple_key_source_table_01862'))
SOURCE(CLICKHOUSE(TABLE 'simple_key_source_table_01862'))
LAYOUT(SPARSE_HASHED())
LIFETIME(MIN 0 MAX 1000);
@ -105,7 +105,7 @@ CREATE DICTIONARY simple_key_cache_dictionary_01862
value String
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'simple_key_source_table_01862'))
SOURCE(CLICKHOUSE(TABLE 'simple_key_source_table_01862'))
LAYOUT(CACHE(SIZE_IN_CELLS 100000))
LIFETIME(MIN 0 MAX 1000);
@ -143,7 +143,7 @@ CREATE DICTIONARY complex_key_hashed_dictionary_01862
value String
)
PRIMARY KEY id, id_key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'complex_key_source_table_01862'))
SOURCE(CLICKHOUSE(TABLE 'complex_key_source_table_01862'))
LAYOUT(COMPLEX_KEY_HASHED())
LIFETIME(MIN 0 MAX 1000);
@ -164,7 +164,7 @@ CREATE DICTIONARY complex_key_direct_dictionary_01862
value String
)
PRIMARY KEY id, id_key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'complex_key_source_table_01862'))
SOURCE(CLICKHOUSE(TABLE 'complex_key_source_table_01862'))
LAYOUT(COMPLEX_KEY_DIRECT());
SELECT name, found_rate FROM system.dictionaries WHERE database = currentDatabase() AND name = 'complex_key_direct_dictionary_01862';
@ -184,7 +184,7 @@ CREATE DICTIONARY complex_key_cache_dictionary_01862
value String
)
PRIMARY KEY id, id_key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'complex_key_source_table_01862'))
SOURCE(CLICKHOUSE(TABLE 'complex_key_source_table_01862'))
LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 100000))
LIFETIME(MIN 0 MAX 1000);
@ -223,7 +223,7 @@ CREATE DICTIONARY simple_key_range_hashed_dictionary_01862
last Date
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'range_key_source_table_01862'))
SOURCE(CLICKHOUSE(TABLE 'range_key_source_table_01862'))
LAYOUT(RANGE_HASHED())
RANGE(MIN first MAX last)
LIFETIME(MIN 0 MAX 1000);
@ -259,13 +259,16 @@ CREATE DICTIONARY ip_trie_dictionary_01862
value String
)
PRIMARY KEY prefix
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'ip_trie_source_table_01862'))
SOURCE(CLICKHOUSE(TABLE 'ip_trie_source_table_01862'))
LAYOUT(IP_TRIE())
LIFETIME(MIN 0 MAX 1000);
-- found_rate = 0, because we didn't make any searches.
SELECT name, found_rate FROM system.dictionaries WHERE database = currentDatabase() AND name = 'ip_trie_dictionary_01862';
-- found_rate = 1, because the dictionary covers the 127.0.0.1 address.
SELECT dictGet('ip_trie_dictionary_01862', 'value', tuple(toIPv4('127.0.0.1'))) FORMAT Null;
SELECT name, found_rate FROM system.dictionaries WHERE database = currentDatabase() AND name = 'ip_trie_dictionary_01862';
-- found_rate = 0.5, because the dictionary does not cover 1.1.1.1 and we have two lookups in total as of now.
SELECT dictGet('ip_trie_dictionary_01862', 'value', tuple(toIPv4('1.1.1.1'))) FORMAT Null;
SELECT name, found_rate FROM system.dictionaries WHERE database = currentDatabase() AND name = 'ip_trie_dictionary_01862';
@ -299,7 +302,7 @@ CREATE DICTIONARY polygon_dictionary_01862
name String
)
PRIMARY KEY key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons_01862'))
SOURCE(CLICKHOUSE(USER 'default' TABLE 'polygons_01862'))
LIFETIME(0)
LAYOUT(POLYGON());

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: long
# Tags: long, no-random-settings, no-debug
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -58,8 +58,7 @@ U c 10
UlI+1 10
bX?}ix [ Ny]2 G 10
t<iT X48q:Z]t0 10
0 3 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2` 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`
0 3 SELECT `__table2`.`value1` AS `value1`, `__table2`.`value2` AS `value2`, count() AS `count` FROM `default`.`join_outer_table` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2`
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 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`
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, allow_experimental_analyzer=0;

View File

@ -64,6 +64,7 @@ function run_query_with_pure_parallel_replicas () {
--query_id "${1}_pure" \
--max_parallel_replicas 3 \
--prefer_localhost_replica 1 \
--parallel_replicas_prefer_local_join 0 \
--cluster_for_parallel_replicas "parallel_replicas" \
--allow_experimental_parallel_reading_from_replicas 1 \
--parallel_replicas_for_non_replicated_merge_tree 1 \

View File

@ -23,14 +23,14 @@ function test_alter_profile()
${CLICKHOUSE_CLIENT} -q $"ALTER SETTINGS PROFILE ${PROFILE} SETTINGS max_sessions_for_user = ${max_session_count}"
# Create sesssions with $max_session_count resriction
# Create sessions with $max_session_count restriction
for ((i = 1 ; i <= ${max_session_count} ; i++)); do
local session_id="${SESSION_ID_PREFIX}_${i}"
# Skip output from this query
${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${USER}&session_id=${session_id}&session_check=0" --data-binary "SELECT 1" > /dev/null
done
# Update resriction to $alter_sessions_count
# Update restriction to $alter_sessions_count
${CLICKHOUSE_CLIENT} -q $"ALTER SETTINGS PROFILE ${PROFILE} SETTINGS max_sessions_for_user = ${alter_sessions_count}"
# Simultaneous sessions should use max settings from profile ($alter_sessions_count)

View File

@ -1,3 +1,3 @@
The number of attached tables is more than 10
The number of attached databases is more than 10
The number of attached tables is more than 5
The number of attached databases is more than 2
The number of active parts is more than 10

View File

@ -37,7 +37,7 @@ INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_9 VALUES (1, 'Hello'
INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_10 VALUES (1, 'Hello');
INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_11 VALUES (1, 'Hello');
SELECT * FROM system.warnings where message in ('The number of attached tables is more than 10', 'The number of attached databases is more than 10', 'The number of active parts is more than 10');
SELECT * FROM system.warnings where message in ('The number of attached tables is more than 5', 'The number of attached databases is more than 2', 'The number of active parts is more than 10');
DROP DATABASE IF EXISTS test_max_num_to_warn_02931;
DROP DATABASE IF EXISTS test_max_num_to_warn_1;

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: long
# Tags: long, no-debug
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# reset --log_comment
@ -74,11 +74,11 @@ run 0
$CH_CLIENT -q "drop table test;"
echo "MergeTree compact"
$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, String, Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;"
$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, String, Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity = 8192, index_granularity_bytes = '10Mi';"
run 1
$CH_CLIENT -q "drop table test;"
echo "MergeTree wide"
$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, String, Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, String, Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity = 8192, index_granularity_bytes = '10Mi';"
run 1
$CH_CLIENT -q "drop table test;"

View File

@ -0,0 +1,177 @@
simple join with analyzer
4200000 4200000 4200000 -1400000
4200006 4200006 4200006 -1400002
4200012 4200012 4200012 -1400004
4200018 4200018 4200018 -1400006
4200024 4200024 4200024 -1400008
4200030 4200030 4200030 -1400010
4200036 4200036 4200036 -1400012
4200042 4200042 4200042 -1400014
4200048 4200048 4200048 -1400016
4200054 4200054 4200054 -1400018
simple (global) join with analyzer and parallel replicas
4200000 4200000 4200000 -1400000
4200006 4200006 4200006 -1400002
4200012 4200012 4200012 -1400004
4200018 4200018 4200018 -1400006
4200024 4200024 4200024 -1400008
4200030 4200030 4200030 -1400010
4200036 4200036 4200036 -1400012
4200042 4200042 4200042 -1400014
4200048 4200048 4200048 -1400016
4200054 4200054 4200054 -1400018
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` (stage: WithMergeableState)
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` (stage: WithMergeableState)
<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(700000, 'UInt64'), _CAST(10, 'UInt64') SETTINGS allow_experimental_analyzer = 1, allow_experimental_parallel_reading_from_replicas = 2, send_logs_level = 'trace', 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 (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(700000, 'UInt64'), _CAST(10, 'UInt64') SETTINGS allow_experimental_analyzer = 1, allow_experimental_parallel_reading_from_replicas = 2, send_logs_level = 'trace', 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 (stage: WithMergeableState)
<Debug> DefaultCoordinator: Coordination done
simple (local) join with analyzer and parallel replicas
4200000 4200000 4200000 -1400000
4200006 4200006 4200006 -1400002
4200012 4200012 4200012 -1400004
4200018 4200018 4200018 -1400006
4200024 4200024 4200024 -1400008
4200030 4200030 4200030 -1400010
4200036 4200036 4200036 -1400012
4200042 4200042 4200042 -1400014
4200048 4200048 4200048 -1400016
4200054 4200054 4200054 -1400018
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` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') SETTINGS allow_experimental_analyzer = 1, send_logs_level = 'trace', 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 = 1 (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` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') SETTINGS allow_experimental_analyzer = 1, send_logs_level = 'trace', 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 = 1 (stage: WithMergeableState)
<Debug> DefaultCoordinator: Coordination done
simple (local) join with analyzer and parallel replicas and full sorting merge join
4200000 4200000 4200000 -1400000
4200006 4200006 4200006 -1400002
4200012 4200012 4200012 -1400004
4200018 4200018 4200018 -1400006
4200024 4200024 4200024 -1400008
4200030 4200030 4200030 -1400010
4200036 4200036 4200036 -1400012
4200042 4200042 4200042 -1400014
4200048 4200048 4200048 -1400016
4200054 4200054 4200054 -1400018
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` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') SETTINGS allow_experimental_analyzer = 1, join_algorithm = 'full_sorting_merge', send_logs_level = 'trace', 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 = 1 (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` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') SETTINGS allow_experimental_analyzer = 1, join_algorithm = 'full_sorting_merge', send_logs_level = 'trace', 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 = 1 (stage: WithMergeableState)
<Debug> WithOrderCoordinator: Coordination done
nested join with analyzer
420000 420000 420000 -140000
420042 420042 420042 -140014
420084 420084 420084 -140028
420126 420126 420126 -140042
420168 420168 420168 -140056
420210 420210 420210 -140070
420252 420252 420252 -140084
420294 420294 420294 -140098
420336 420336 420336 -140112
420378 420378 420378 -140126
nested join with analyzer and parallel replicas, both local
420000 420000 420000 -140000
420042 420042 420042 -140014
420084 420084 420084 -140028
420126 420126 420126 -140042
420168 420168 420168 -140056
420210 420210 420210 -140070
420252 420252 420252 -140084
420294 420294 420294 -140098
420336 420336 420336 -140112
420378 420378 420378 -140126
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` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4` ALL INNER JOIN (SELECT `__table6`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table6`) AS `__table5` ON `__table4`.`key` = `__table5`.`key` SETTINGS parallel_replicas_prefer_local_join = 1) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') SETTINGS allow_experimental_analyzer = 1, join_algorithm = 'full_sorting_merge', send_logs_level = 'trace', 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 = 1 (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` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4` ALL INNER JOIN (SELECT `__table6`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table6`) AS `__table5` ON `__table4`.`key` = `__table5`.`key` SETTINGS parallel_replicas_prefer_local_join = 1) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') SETTINGS allow_experimental_analyzer = 1, join_algorithm = 'full_sorting_merge', send_logs_level = 'trace', 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 = 1 (stage: WithMergeableState)
<Debug> WithOrderCoordinator: Coordination done
nested join with analyzer and parallel replicas, both global
420000 420000 420000 -140000
420042 420042 420042 -140014
420084 420084 420084 -140028
420126 420126 420126 -140042
420168 420168 420168 -140056
420210 420210 420210 -140070
420252 420252 420252 -140084
420294 420294 420294 -140098
420336 420336 420336 -140112
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` SETTINGS parallel_replicas_prefer_local_join = 0 (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` SETTINGS parallel_replicas_prefer_local_join = 0 (stage: WithMergeableState)
<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') SETTINGS allow_experimental_analyzer = 1, send_logs_level = 'trace', 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 (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') SETTINGS allow_experimental_analyzer = 1, send_logs_level = 'trace', 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 (stage: WithMergeableState)
<Debug> DefaultCoordinator: Coordination done
nested join with analyzer and parallel replicas, global + local
420000 420000 420000 -140000
420042 420042 420042 -140014
420084 420084 420084 -140028
420126 420126 420126 -140042
420168 420168 420168 -140056
420210 420210 420210 -140070
420252 420252 420252 -140084
420294 420294 420294 -140098
420336 420336 420336 -140112
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` SETTINGS parallel_replicas_prefer_local_join = 1 (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` SETTINGS parallel_replicas_prefer_local_join = 1 (stage: WithMergeableState)
<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') SETTINGS allow_experimental_analyzer = 1, send_logs_level = 'trace', 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 (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') SETTINGS allow_experimental_analyzer = 1, send_logs_level = 'trace', 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 (stage: WithMergeableState)
<Debug> DefaultCoordinator: Coordination done
nested join with analyzer and parallel replicas, both local, both full sorting merge join
420000 420000 420000 -140000
420042 420042 420042 -140014
420084 420084 420084 -140028
420126 420126 420126 -140042
420168 420168 420168 -140056
420210 420210 420210 -140070
420252 420252 420252 -140084
420294 420294 420294 -140098
420336 420336 420336 -140112
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` SETTINGS join_algorithm = 'full_sorting_merge' (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` SETTINGS join_algorithm = 'full_sorting_merge' (stage: WithMergeableState)
<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') SETTINGS allow_experimental_analyzer = 1, parallel_replicas_prefer_local_join = 0, send_logs_level = 'trace', 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' (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') SETTINGS allow_experimental_analyzer = 1, parallel_replicas_prefer_local_join = 0, send_logs_level = 'trace', 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' (stage: WithMergeableState)
<Debug> WithOrderCoordinator: Coordination done
nested join with analyzer and parallel replicas, both local, both full sorting and hash join
420000 420000 420000 -140000
420042 420042 420042 -140014
420084 420084 420084 -140028
420126 420126 420126 -140042
420168 420168 420168 -140056
420210 420210 420210 -140070
420252 420252 420252 -140084
420294 420294 420294 -140098
420336 420336 420336 -140112
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` SETTINGS join_algorithm = 'hash' (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` SETTINGS join_algorithm = 'hash' (stage: WithMergeableState)
<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') SETTINGS allow_experimental_analyzer = 1, parallel_replicas_prefer_local_join = 0, send_logs_level = 'trace', 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' (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') SETTINGS allow_experimental_analyzer = 1, parallel_replicas_prefer_local_join = 0, send_logs_level = 'trace', 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' (stage: WithMergeableState)
<Debug> WithOrderCoordinator: Coordination done
nested join with analyzer and parallel replicas, both local, both full sorting and hash join
420000 420000 420000 -140000
420042 420042 420042 -140014
420084 420084 420084 -140028
420126 420126 420126 -140042
420168 420168 420168 -140056
420210 420210 420210 -140070
420252 420252 420252 -140084
420294 420294 420294 -140098
420336 420336 420336 -140112
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` SETTINGS join_algorithm = 'full_sorting_merge' (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` SETTINGS join_algorithm = 'full_sorting_merge' (stage: WithMergeableState)
<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') SETTINGS allow_experimental_analyzer = 1, parallel_replicas_prefer_local_join = 0, send_logs_level = 'trace', 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' (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') SETTINGS allow_experimental_analyzer = 1, parallel_replicas_prefer_local_join = 0, send_logs_level = 'trace', 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' (stage: WithMergeableState)
<Debug> DefaultCoordinator: Coordination done

View File

@ -0,0 +1,263 @@
#!/usr/bin/env bash
# Tags: long, no-random-settings, no-random-merge-tree-settings
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -nm -q "
drop table if exists num_1;
drop table if exists num_2;
create table num_1 (key UInt64, value String) engine = MergeTree order by key;
create table num_2 (key UInt64, value Int64) engine = MergeTree order by key;
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);
"
##############
echo
echo "simple join with analyzer"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
inner join (select key, value from num_2) r on l.key = r.key
order by l.key limit 10 offset 700000
SETTINGS allow_experimental_analyzer=1"
##############
echo
echo "simple (global) join with analyzer and parallel replicas"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
inner join (select key, value from num_2) r on l.key = r.key
order by l.key limit 10 offset 700000
SETTINGS allow_experimental_analyzer=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"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
inner join (select key, value from num_2) r on l.key = r.key
order by l.key limit 10 offset 700000
SETTINGS allow_experimental_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, send_logs_level='trace',
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 -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
##############
echo
echo "simple (local) join with analyzer and parallel replicas"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
inner join (select key, value from num_2) r on l.key = r.key
order by l.key limit 10 offset 700000
SETTINGS allow_experimental_analyzer=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=1"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
inner join (select key, value from num_2) r on l.key = r.key
order by l.key limit 10 offset 700000
SETTINGS allow_experimental_analyzer=1, send_logs_level='trace',
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=1" 2>&1 |
grep "executeQuery\|<Debug>.*Coordinator: Coordination done" |
grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
##############
echo
echo "simple (local) join with analyzer and parallel replicas and full sorting merge join"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
inner join (select key, value from num_2) r on l.key = r.key
order by l.key limit 10 offset 700000
SETTINGS allow_experimental_analyzer=1, 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', parallel_replicas_prefer_local_join=1"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
inner join (select key, value from num_2) r on l.key = r.key
order by l.key limit 10 offset 700000
SETTINGS allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', send_logs_level='trace',
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=1" 2>&1 |
grep "executeQuery\|<Debug>.*Coordinator: Coordination done" |
grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
##############
echo
echo "nested join with analyzer"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
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
on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS allow_experimental_analyzer=1"
##############
echo
echo "nested join with analyzer and parallel replicas, both local"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
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
on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS allow_experimental_analyzer=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=1"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
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
on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS allow_experimental_analyzer=1, join_algorithm='full_sorting_merge', send_logs_level='trace',
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=1" 2>&1 |
grep "executeQuery\|<Debug>.*Coordinator: Coordination done" |
grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
##############
echo
echo "nested join with analyzer and parallel replicas, both global"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
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
on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS allow_experimental_analyzer=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"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
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
on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS allow_experimental_analyzer=1, send_logs_level='trace',
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 -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
##############
echo
echo "nested join with analyzer and parallel replicas, global + local"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
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
on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS allow_experimental_analyzer=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"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
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
on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS allow_experimental_analyzer=1, send_logs_level='trace',
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 -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
##############
echo
echo "nested join with analyzer and parallel replicas, both local, both full sorting merge join"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
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
on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS allow_experimental_analyzer=1, 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', join_algorithm='full_sorting_merge'"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
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
on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace',
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 -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
##############
echo
echo "nested join with analyzer and parallel replicas, both local, both full sorting and hash join"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
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
on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS allow_experimental_analyzer=1, 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', join_algorithm='full_sorting_merge'"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
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
on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace',
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 -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
##############
echo
echo "nested join with analyzer and parallel replicas, both local, both full sorting and hash join"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
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
on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS allow_experimental_analyzer=1, 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', join_algorithm='hash'"
$CLICKHOUSE_CLIENT -q "
select * from (select key, value from num_1) l
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
on l.key = r.key order by l.key limit 10 offset 10000
SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace',
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 -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'

View File

@ -0,0 +1,502 @@
-- { echoOn }
set parallel_replicas_prefer_local_join = 0;
-- 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', allow_experimental_analyzer=1;
0 0 0 0 0 0
1 1 0 0 0 0
3 3 0 0 0 0
4 4 0 0 0 0
5 5 0 0 0 0
6 6 6 6 0 0
7 7 0 0 0 0
8 8 8 8 0 0
9 9 0 0 0 0
10 10 10 10 0 0
11 11 0 0 0 0
12 12 12 12 12 12
13 13 0 0 0 0
14 14 14 14 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', allow_experimental_analyzer=1;
Expression
ReadFromRemoteParallelReplicas
--
-- The same query with cte;
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
0 0 0 0 0 0
1 1 0 0 0 0
3 3 0 0 0 0
4 4 0 0 0 0
5 5 0 0 0 0
6 6 6 6 0 0
7 7 0 0 0 0
8 8 8 8 0 0
9 9 0 0 0 0
10 10 10 10 0 0
11 11 0 0 0 0
12 12 12 12 12 12
13 13 0 0 0 0
14 14 14 14 0 0
15 15 0 0 0 0
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
Expression
Sorting
Expression
ReadFromRemoteParallelReplicas
--
-- GROUP BY should work up to WithMergableStage
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key)
select * from sub5 order by key
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', allow_experimental_analyzer=1;
54 54 50 50 12 12 0
64 64 0 0 0 0 1
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key)
select * from sub5 order by key
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', allow_experimental_analyzer=1;
Expression
Sorting
Expression
MergingAggregated
Expression
ReadFromRemoteParallelReplicas
--
-- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
0 0 0 0 0 0
1 1 0 0 0 0
3 3 0 0 0 0
4 4 0 0 0 0
5 5 0 0 0 0
6 6 6 6 0 0
7 7 0 0 0 0
8 8 8 8 0 0
9 9 0 0 0 0
10 10 10 10 0 0
11 11 0 0 0 0
12 12 12 12 12 12
13 13 0 0 0 0
14 14 14 14 0 0
15 15 0 0 0 0
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
Expression
Sorting
Expression
Join
Expression
ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
--
-- ORDER BY in sub1 : sub1 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2 order by y),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
0 0 0 0 0 0
1 1 0 0 0 0
3 3 0 0 0 0
4 4 0 0 0 0
5 5 0 0 0 0
6 6 6 6 0 0
7 7 0 0 0 0
8 8 8 8 0 0
9 9 0 0 0 0
10 10 10 10 0 0
11 11 0 0 0 0
12 12 12 12 12 12
13 13 0 0 0 0
14 14 14 14 0 0
15 15 0 0 0 0
explain description=0
with sub1 as (select x, y from tab1 where x != 2 order by y),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
Expression
Sorting
Expression
Join
Expression
Join
Expression
ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
--
-- RIGHT JOIN in sub3: sub3 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5
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', allow_experimental_analyzer=1;
0 0 0 0 0 0
6 6 6 6 0 0
8 8 8 8 0 0
10 10 10 10 0 0
12 12 12 12 12 12
14 14 14 14 0 0
4 4 0 0 0 0
3 3 0 0 0 0
5 5 0 0 0 0
1 1 0 0 0 0
7 7 0 0 0 0
9 9 0 0 0 0
15 15 0 0 0 0
11 11 0 0 0 0
13 13 0 0 0 0
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5
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', allow_experimental_analyzer=1;
Expression
Join
Expression
Join
Expression
ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
--
-- RIGHT JOIN in sub5: sub5 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z)
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', allow_experimental_analyzer=1;
0 0 0 0 0 0
0 0 1 1 0 0
0 0 3 3 0 0
0 0 4 4 0 0
0 0 5 5 0 0
0 0 6 6 6 6
0 0 7 7 0 0
0 0 8 8 8 8
0 0 9 9 0 0
0 0 10 10 10 10
0 0 11 11 0 0
12 12 12 12 12 12
0 0 13 13 0 0
0 0 14 14 14 14
0 0 15 15 0 0
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z)
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', allow_experimental_analyzer=1;-- { echoOn }
Expression
Sorting
Expression
Join
Expression
ReadFromRemoteParallelReplicas
Expression
Join
Expression
ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
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.
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', allow_experimental_analyzer=1;
0 0 0 0 0 0
1 1 0 0 0 0
3 3 0 0 0 0
4 4 0 0 0 0
5 5 0 0 0 0
6 6 6 6 0 0
7 7 0 0 0 0
8 8 8 8 0 0
9 9 0 0 0 0
10 10 10 10 0 0
11 11 0 0 0 0
12 12 12 12 12 12
13 13 0 0 0 0
14 14 14 14 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', allow_experimental_analyzer=1;
Expression
ReadFromRemoteParallelReplicas
--
-- The same query with cte;
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
0 0 0 0 0 0
1 1 0 0 0 0
3 3 0 0 0 0
4 4 0 0 0 0
5 5 0 0 0 0
6 6 6 6 0 0
7 7 0 0 0 0
8 8 8 8 0 0
9 9 0 0 0 0
10 10 10 10 0 0
11 11 0 0 0 0
12 12 12 12 12 12
13 13 0 0 0 0
14 14 14 14 0 0
15 15 0 0 0 0
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
Expression
Sorting
Expression
ReadFromRemoteParallelReplicas
--
-- GROUP BY should work up to WithMergableStage
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key)
select * from sub5 order by key
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', allow_experimental_analyzer=1;
54 54 50 50 12 12 0
64 64 0 0 0 0 1
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key)
select * from sub5 order by key
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', allow_experimental_analyzer=1;
Expression
Sorting
Expression
MergingAggregated
Expression
ReadFromRemoteParallelReplicas
--
-- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
0 0 0 0 0 0
1 1 0 0 0 0
3 3 0 0 0 0
4 4 0 0 0 0
5 5 0 0 0 0
6 6 6 6 0 0
7 7 0 0 0 0
8 8 8 8 0 0
9 9 0 0 0 0
10 10 10 10 0 0
11 11 0 0 0 0
12 12 12 12 12 12
13 13 0 0 0 0
14 14 14 14 0 0
15 15 0 0 0 0
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
Expression
Sorting
Expression
Join
Expression
ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
--
-- ORDER BY in sub1 : sub1 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2 order by y),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
0 0 0 0 0 0
1 1 0 0 0 0
3 3 0 0 0 0
4 4 0 0 0 0
5 5 0 0 0 0
6 6 6 6 0 0
7 7 0 0 0 0
8 8 8 8 0 0
9 9 0 0 0 0
10 10 10 10 0 0
11 11 0 0 0 0
12 12 12 12 12 12
13 13 0 0 0 0
14 14 14 14 0 0
15 15 0 0 0 0
explain description=0
with sub1 as (select x, y from tab1 where x != 2 order by y),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
Expression
Sorting
Expression
Join
Expression
Join
Expression
ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
--
-- RIGHT JOIN in sub3: sub3 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5
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', allow_experimental_analyzer=1;
0 0 0 0 0 0
6 6 6 6 0 0
8 8 8 8 0 0
10 10 10 10 0 0
12 12 12 12 12 12
14 14 14 14 0 0
4 4 0 0 0 0
3 3 0 0 0 0
5 5 0 0 0 0
1 1 0 0 0 0
7 7 0 0 0 0
9 9 0 0 0 0
15 15 0 0 0 0
11 11 0 0 0 0
13 13 0 0 0 0
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5
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', allow_experimental_analyzer=1;
Expression
Join
Expression
Join
Expression
ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas
--
-- RIGHT JOIN in sub5: sub5 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z)
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', allow_experimental_analyzer=1;
0 0 0 0 0 0
0 0 1 1 0 0
0 0 3 3 0 0
0 0 4 4 0 0
0 0 5 5 0 0
0 0 6 6 6 6
0 0 7 7 0 0
0 0 8 8 8 8
0 0 9 9 0 0
0 0 10 10 10 10
0 0 11 11 0 0
12 12 12 12 12 12
0 0 13 13 0 0
0 0 14 14 14 14
0 0 15 15 0 0
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z)
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', allow_experimental_analyzer=1;
Expression
Sorting
Expression
Join
Expression
ReadFromRemoteParallelReplicas
Expression
Join
Expression
ReadFromRemoteParallelReplicas
Expression
ReadFromRemoteParallelReplicas

View File

@ -0,0 +1,129 @@
drop table if exists tab1;
drop table if exists tab2;
drop table if exists tab3;
create table tab1 (x UInt32, y UInt32, shard UInt32) engine = MergeTree order by shard;
create table tab2 (y UInt32, z UInt32) engine = MergeTree order by tuple();
create table tab3 (z UInt32, a UInt32) engine = MergeTree order by tuple();
insert into tab1 select number, number, number from numbers(16);
insert into tab2 select number * 2, number * 2 from numbers(8);
insert into tab3 select number * 4, number * 4 from numbers(4);
{% for use_global_in in [0, 1] -%}
-- { echoOn }
set parallel_replicas_prefer_local_join = {{use_global_in}};
-- 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', allow_experimental_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', allow_experimental_analyzer=1;
--
-- The same query with cte;
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
--
-- GROUP BY should work up to WithMergableStage
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key)
select * from sub5 order by key
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', allow_experimental_analyzer=1;
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key)
select * from sub5 order by key
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', allow_experimental_analyzer=1;
--
-- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
--
-- ORDER BY in sub1 : sub1 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2 order by y),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
explain description=0
with sub1 as (select x, y from tab1 where x != 2 order by y),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
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', allow_experimental_analyzer=1;
--
-- RIGHT JOIN in sub3: sub3 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5
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', allow_experimental_analyzer=1;
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5
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', allow_experimental_analyzer=1;
--
-- RIGHT JOIN in sub5: sub5 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z)
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', allow_experimental_analyzer=1;
explain description=0
with sub1 as (select x, y from tab1 where x != 2),
sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8),
sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z)
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', allow_experimental_analyzer=1;
{%- endfor %}

View File

@ -1,46 +0,0 @@
-- Tags: no-parallel
SET async_insert = 1;
SET insert_deduplicate = 1;
SET deduplicate_blocks_in_dependent_materialized_views = 1;
DROP TABLE IF EXISTS 02985_test;
CREATE TABLE 02985_test
(
d Date,
value UInt64
) ENGINE = MergeTree ORDER BY tuple() SETTINGS non_replicated_deduplication_window = 1000;
DROP VIEW IF EXISTS 02985_mv;
CREATE MATERIALIZED VIEW 02985_mv
ENGINE = SummingMergeTree ORDER BY d AS
SELECT
d, sum(value) s
FROM 02985_test GROUP BY d;
-- Inserts are synchronous.
INSERT INTO 02985_test (*)
VALUES ('2024-01-01', 1), ('2024-01-01', 2), ('2024-01-02', 1);
SYSTEM FLUSH LOGS;
SELECT format, status, rows, data_kind FROM system.asynchronous_insert_log
WHERE database = currentDatabase() AND table = '02985_test';
SET deduplicate_blocks_in_dependent_materialized_views = 0;
-- Set a large value for async_insert_busy_timeout_max_ms to avoid flushing the entry synchronously.
INSERT INTO 02985_test (*)
SETTINGS
async_insert_busy_timeout_min_ms=200,
async_insert_busy_timeout_max_ms=100000
VALUES ('2024-01-01', 1), ('2024-01-01', 2), ('2024-01-02', 1), ('2024-01-02', 4);
SYSTEM FLUSH LOGS;
SELECT format, status, rows, data_kind
FROM system.asynchronous_insert_log
WHERE database = currentDatabase() AND table = '02985_test';
DROP VIEW IF EXISTS 02985_mv;
DROP TABLE IF EXISTS 02985_test;

View File

@ -0,0 +1,3 @@
0
1
2

View File

@ -0,0 +1 @@
SELECT number FROM numbers(3) GROUP BY (number, number % 2) ORDER BY number;

View File

@ -0,0 +1,14 @@
Row 1:
──────
name: metadata
version: 1
--
Row 1:
──────
name: metadata_version
value: 1
--
id UInt64
value String
insert_time DateTime
insert_time_updated DateTime

View File

@ -0,0 +1,40 @@
-- Tags: zookeeper
DROP TABLE IF EXISTS test_table_replicated;
CREATE TABLE test_table_replicated
(
id UInt64,
value String
) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_table_replicated', '1_replica') ORDER BY id;
ALTER TABLE test_table_replicated ADD COLUMN insert_time DateTime;
SELECT name, version FROM system.zookeeper
WHERE path = '/clickhouse/tables/' || currentDatabase() ||'/test_table_replicated/'
AND name = 'metadata' FORMAT Vertical;
DROP TABLE IF EXISTS test_table_replicated_second;
CREATE TABLE test_table_replicated_second
(
id UInt64,
value String,
insert_time DateTime
) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_table_replicated', '2_replica') ORDER BY id;
DROP TABLE test_table_replicated;
SELECT '--';
SELECT name, value FROM system.zookeeper
WHERE path = '/clickhouse/tables/' || currentDatabase() ||'/test_table_replicated/replicas/2_replica'
AND name = 'metadata_version' FORMAT Vertical;
SYSTEM RESTART REPLICA test_table_replicated_second;
ALTER TABLE test_table_replicated_second ADD COLUMN insert_time_updated DateTime;
SELECT '--';
DESCRIBE test_table_replicated_second;
DROP TABLE test_table_replicated_second;

View File

@ -1 +1,2 @@
SET max_bytes_before_external_group_by = '1G';
SELECT k, any(u) AS u, uniqMerge(us) AS us FROM (SELECT domain(URL) AS k, uniq(UserID) AS u, uniqState(UserID) AS us FROM test.hits GROUP BY k) GROUP BY k ORDER BY u DESC, k ASC LIMIT 100

View File

@ -1 +1,2 @@
SET max_bytes_before_external_group_by = '1G';
SELECT topLevelDomain(concat('http://', k)) AS tld, sum(u) AS u, uniqMerge(us) AS us FROM (SELECT domain(URL) AS k, uniq(UserID) AS u, uniqState(UserID) AS us FROM test.hits GROUP BY k) GROUP BY tld ORDER BY u DESC, tld ASC LIMIT 100

View File

@ -1,5 +1,8 @@
-- Tags: no-tsan, no-parallel
-- Suppress "ReadWriteBufferFromHTTP: HTTP request to `{}` failed at try 1/10 with bytes read: 311149/378695. Error: DB::HTTPException: Received error from remote server {}. (Current backoff wait is 100/10000 ms)" errors
SET send_logs_level='error';
DROP TABLE IF EXISTS test.hits_1m;
CREATE TABLE test.hits_1m AS test.hits

View File

@ -274,6 +274,11 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -F '!!!' | grep -P '.' && echo "Too many exclamation marks (looks dirty, unconfident)."
# Exclamation mark in a message
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -F '!",' | grep -P '.' && echo "No need for an exclamation mark (looks dirty, unconfident)."
# Trailing whitespaces
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |

View File

@ -1,3 +1,4 @@
v24.1.5.6-stable 2024-02-14
v24.1.4.20-stable 2024-02-14
v24.1.3.31-stable 2024-02-09
v24.1.2.5-stable 2024-02-02

1 v24.1.4.20-stable v24.1.5.6-stable 2024-02-14
1 v24.1.5.6-stable 2024-02-14
2 v24.1.4.20-stable v24.1.4.20-stable 2024-02-14
3 v24.1.3.31-stable v24.1.3.31-stable 2024-02-09
4 v24.1.2.5-stable v24.1.2.5-stable 2024-02-02