From 7aaf7e0e03d1d1e192a68277c4f6836ce92587e4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 30 Apr 2024 10:44:28 +0200 Subject: [PATCH 001/105] Fix 02488_zero_copy_detached_parts_drop_table test --- .../0_stateless/02488_zero_copy_detached_parts_drop_table.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh b/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh index b01f16e1cad..60cec5caea3 100755 --- a/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh +++ b/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT -q "select throwIf(substring('$path', 1, 1) != '/', 'Path is rm -f $path/count.txt $CLICKHOUSE_CLIENT -q "detach table rmt2 sync" -$CLICKHOUSE_CLIENT --send_logs_level='fatal' -q "attach table rmt2" +$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level='fatal' -q "attach table rmt2" $CLICKHOUSE_CLIENT -q "select reason, name from system.detached_parts where database='$CLICKHOUSE_DATABASE' and table='rmt2'" From 6556bd28dcb7912add1c192e318f4e9dd0f9cd31 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 30 Apr 2024 15:23:36 +0200 Subject: [PATCH 002/105] Fix parallel write for Azure --- src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index f52ab803012..cd154ec0f79 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -163,6 +163,8 @@ public: return client.get(); } + bool supportParallelWrite() const override { return true; } + private: using SharedAzureClientPtr = std::shared_ptr; void removeObjectImpl(const StoredObject & object, const SharedAzureClientPtr & client_ptr, bool if_exists); From 697b5556a3e16e8b0cbba43d5226a2003ef21f0a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 30 Apr 2024 22:10:36 +0200 Subject: [PATCH 003/105] Enable azure test --- tests/ci/functional_test_check.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index e898138fb3a..6418ee4e0ce 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -318,9 +318,6 @@ def main(): state, description, test_results, additional_logs = process_results( result_path, server_log_path ) - # FIXME (alesapin) - if "azure" in check_name: - state = "success" else: print( "This is validate bugfix or flaky check run, but no changes test to run - skip with success" From ccabc292ad6647855d5b99c1631b8ee77413b855 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 1 May 2024 08:48:24 +0200 Subject: [PATCH 004/105] Temporarily remove running azure tests only for master --- tests/ci/ci_config.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index bdfff12db0b..f7b32e9177c 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -1205,7 +1205,7 @@ CI_CONFIG = CIConfig( ), JobNames.STATELESS_TEST_AZURE_ASAN: TestConfig( Build.PACKAGE_ASAN, - job_config=JobConfig(num_batches=4, **statless_test_common_params, release_only=True), # type: ignore + job_config=JobConfig(num_batches=4, **statless_test_common_params), # type: ignore ), JobNames.STATELESS_TEST_S3_TSAN: TestConfig( Build.PACKAGE_TSAN, @@ -1230,7 +1230,7 @@ CI_CONFIG = CIConfig( Build.PACKAGE_ASAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore ), JobNames.STRESS_TEST_AZURE_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(**stress_test_common_params, release_only=True) # type: ignore + Build.PACKAGE_TSAN, job_config=JobConfig(**stress_test_common_params) # type: ignore ), JobNames.UPGRADE_TEST_TSAN: TestConfig( Build.PACKAGE_TSAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore From 068ddc2d65f2ba4bce98aaafe134a94f843d5974 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 1 May 2024 11:03:52 +0200 Subject: [PATCH 005/105] Add allow_repeated_settings before appending send_logs_level in tests --- .../0_stateless/02494_query_cache_nested_query_bug.sh | 4 ++-- .../0_stateless/02535_max_parallel_replicas_custom_key.sh | 2 +- .../0_stateless/02922_deduplication_with_zero_copy.sh | 8 ++++---- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index 8712c7c84c6..59b107d06fe 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -19,7 +19,7 @@ SETTINGS="SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer # Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated # with EXPLAIN PLAN, we need to check the logs. -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --allow_repeated_settings --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --allow_repeated_settings --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l ${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index 9850406eb3a..480517bf17f 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -41,6 +41,6 @@ run_count_with_custom_key "y" run_count_with_custom_key "cityHash64(y)" run_count_with_custom_key "cityHash64(y) + 1" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --allow_repeated_settings --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with" $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" diff --git a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh index bb013dccb65..dad4e6747e0 100755 --- a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh +++ b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh @@ -58,9 +58,9 @@ function filter_temporary_locks() function insert_duplicates() { - $CLICKHOUSE_CLIENT -q "insert into r1 values(1);" --send_logs_level="error" & + $CLICKHOUSE_CLIENT -q "insert into r1 values(1);" --allow_repeated_settings --send_logs_level="error" & - $CLICKHOUSE_CLIENT -q "insert into r2 values(1);" --send_logs_level="error" + $CLICKHOUSE_CLIENT -q "insert into r2 values(1);" --allow_repeated_settings --send_logs_level="error" wait @@ -137,8 +137,8 @@ function list_keeper_nodes() { list_keeper_nodes "${table_shared_id}" -$CLICKHOUSE_CLIENT -nm -q "drop table r1;" --send_logs_level="error" & -$CLICKHOUSE_CLIENT -nm -q "drop table r2;" --send_logs_level="error" & +$CLICKHOUSE_CLIENT -nm -q "drop table r1;" --allow_repeated_settings --send_logs_level="error" & +$CLICKHOUSE_CLIENT -nm -q "drop table r2;" --allow_repeated_settings --send_logs_level="error" & wait list_keeper_nodes "${table_shared_id}" From 95f65cf4220318cc85b94b29588aaa8b04bc0221 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 14 May 2024 16:58:05 +0000 Subject: [PATCH 006/105] Use right comments for system tables (cherry picked from commit 6c5a35d08943c1492cbf7ee861130ce80541879a) --- .../table-engines/integrations/s3queue.md | 2 +- src/Databases/DatabaseOnDisk.cpp | 2 +- .../02117_show_create_table_system.reference | 128 +++++++++--------- .../02118_show_create_table_rocksdb.reference | 2 +- 4 files changed, 67 insertions(+), 67 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 8ebab80423f..deb915f2060 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -268,7 +268,7 @@ For introspection use `system.s3queue` stateless table and `system.s3queue_log` `exception` String ) ENGINE = SystemS3Queue -COMMENT 'SYSTEM TABLE is built on the fly.' │ +COMMENT 'Contains in-memory state of S3Queue metadata and currently processed rows per file.' │ └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 674e9afa8ac..ac45e5be7dc 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -785,7 +785,7 @@ ASTPtr DatabaseOnDisk::getCreateQueryFromStorage(const String & table_name, cons throw_on_error); create_table_query->set(create_table_query->as()->comment, - std::make_shared("SYSTEM TABLE is built on the fly.")); + std::make_shared(storage->getInMemoryMetadata().comment)); return create_table_query; } diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index d625feb94d3..d15c1545135 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -4,7 +4,7 @@ CREATE TABLE system.aggregate_function_combinators `is_internal` UInt8 ) ENGINE = SystemAggregateFunctionCombinators -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available aggregate function combinators, which could be applied to aggregate functions and change the way they work.' CREATE TABLE system.asynchronous_inserts ( `query` String, @@ -17,7 +17,7 @@ CREATE TABLE system.asynchronous_inserts `entries.bytes` Array(UInt64) ) ENGINE = SystemAsynchronousInserts -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about pending asynchronous inserts in queue in server\'s memory.' CREATE TABLE system.asynchronous_metrics ( `metric` String, @@ -25,14 +25,14 @@ CREATE TABLE system.asynchronous_metrics `description` String ) ENGINE = SystemAsynchronousMetrics -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains metrics that are calculated periodically in the background. For example, the amount of RAM in use.' CREATE TABLE system.build_options ( `name` String, `value` String ) ENGINE = SystemBuildOptions -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all build flags, compiler options and commit hash for used build.' CREATE TABLE system.clusters ( `cluster` String, @@ -55,14 +55,14 @@ CREATE TABLE system.clusters `name` String ALIAS cluster ) ENGINE = SystemClusters -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about clusters defined in the configuration file or generated by a Replicated database.' CREATE TABLE system.collations ( `name` String, `language` Nullable(String) ) ENGINE = SystemTableCollations -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available collations for alphabetical comparison of strings.' CREATE TABLE system.columns ( `database` String, @@ -88,13 +88,13 @@ CREATE TABLE system.columns `datetime_precision` Nullable(UInt64) ) ENGINE = SystemColumns -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Lists all columns from all tables of the current server.' CREATE TABLE system.contributors ( `name` String ) ENGINE = SystemContributors -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all ClickHouse contributors <3' CREATE TABLE system.current_roles ( `role_name` String, @@ -102,7 +102,7 @@ CREATE TABLE system.current_roles `is_default` UInt8 ) ENGINE = SystemCurrentRoles -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains active roles of a current user. SET ROLE changes the contents of this table.' CREATE TABLE system.data_skipping_indices ( `database` String, @@ -117,7 +117,7 @@ CREATE TABLE system.data_skipping_indices `marks` UInt64 ) ENGINE = SystemDataSkippingIndices -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains all the information about all the data skipping indices in tables, similar to system.columns.' CREATE TABLE system.data_type_families ( `name` String, @@ -125,7 +125,7 @@ CREATE TABLE system.data_type_families `alias_to` String ) ENGINE = SystemTableDataTypeFamilies -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available native data types along with all the aliases used for compatibility with other DBMS.' CREATE TABLE system.databases ( `name` String, @@ -138,7 +138,7 @@ CREATE TABLE system.databases `database` String ALIAS name ) ENGINE = SystemDatabases -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Lists all databases of the current server.' CREATE TABLE system.detached_parts ( `database` String, @@ -155,7 +155,7 @@ CREATE TABLE system.detached_parts `level` Nullable(UInt32) ) ENGINE = SystemDetachedParts -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all parts which are being found in /detached directory along with a reason why it was detached. ClickHouse server doesn\'t use such parts anyhow.' CREATE TABLE system.dictionaries ( `database` String, @@ -185,7 +185,7 @@ CREATE TABLE system.dictionaries `comment` String ) ENGINE = SystemDictionaries -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about dictionaries.' CREATE TABLE system.disks ( `name` String, @@ -205,7 +205,7 @@ CREATE TABLE system.disks `cache_path` String ) ENGINE = SystemDisks -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about disks defined in the server configuration.' CREATE TABLE system.distributed_ddl_queue ( `entry` String, @@ -225,7 +225,7 @@ CREATE TABLE system.distributed_ddl_queue `query_duration_ms` Nullable(UInt64) ) ENGINE = SystemDDLWorkerQueue -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about distributed DDL queries (ON CLUSTER clause) that were executed on a cluster.' CREATE TABLE system.distribution_queue ( `database` String, @@ -241,7 +241,7 @@ CREATE TABLE system.distribution_queue `last_exception_time` DateTime ) ENGINE = SystemDistributionQueue -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about local files that are in the queue to be sent to the shards. These local files contain new parts that are created by inserting new data into the Distributed table in asynchronous mode.' CREATE TABLE system.enabled_roles ( `role_name` String, @@ -250,7 +250,7 @@ CREATE TABLE system.enabled_roles `is_default` UInt8 ) ENGINE = SystemEnabledRoles -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains all active roles at the moment, including current role of the current user and granted roles for current role.' CREATE TABLE system.errors ( `name` String, @@ -262,7 +262,7 @@ CREATE TABLE system.errors `remote` UInt8 ) ENGINE = SystemErrors -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all errors which have ever happened including the error code, last time and message with unsymbolized stacktrace.' CREATE TABLE system.events ( `event` String, @@ -271,7 +271,7 @@ CREATE TABLE system.events `name` String ALIAS event ) ENGINE = SystemEvents -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains profiling events and their current value.' CREATE TABLE system.formats ( `name` String, @@ -281,7 +281,7 @@ CREATE TABLE system.formats `supports_parallel_formatting` UInt8 ) ENGINE = SystemFormats -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all the formats along with flags whether a format is suitable for input/output or whether it supports parallelization.' CREATE TABLE system.functions ( `name` String, @@ -299,7 +299,7 @@ CREATE TABLE system.functions `categories` String ) ENGINE = SystemFunctions -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available ordinary and aggregate functions with their descriptions.' CREATE TABLE system.graphite_retentions ( `config_name` String, @@ -314,7 +314,7 @@ CREATE TABLE system.graphite_retentions `Tables.table` Array(String) ) ENGINE = SystemGraphite -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about parameters graphite_rollup which are used in tables with *GraphiteMergeTree engines.' CREATE TABLE system.licenses ( `library_name` String, @@ -323,14 +323,14 @@ CREATE TABLE system.licenses `license_text` String ) ENGINE = SystemLicenses -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains licenses of third-party libraries that are located in the contrib directory of ClickHouse sources.' CREATE TABLE system.macros ( `macro` String, `substitution` String ) ENGINE = SystemMacros -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all macros defined in server configuration.' CREATE TABLE system.merge_tree_settings ( `name` String, @@ -344,7 +344,7 @@ CREATE TABLE system.merge_tree_settings `is_obsolete` UInt8 ) ENGINE = SystemMergeTreeSettings -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all MergeTree engine specific settings, their current and default values along with descriptions. You may change any of them in SETTINGS section in CREATE query.' CREATE TABLE system.merges ( `database` String, @@ -373,7 +373,7 @@ CREATE TABLE system.merges `merge_algorithm` String ) ENGINE = SystemMerges -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of merges currently executing merges of MergeTree tables and their progress. Each merge operation is represented by a single row.' CREATE TABLE system.metrics ( `metric` String, @@ -382,7 +382,7 @@ CREATE TABLE system.metrics `name` String ALIAS metric ) ENGINE = SystemMetrics -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains metrics which can be calculated instantly, or have a current value. For example, the number of simultaneously processed queries or the current replica delay. This table is always up to date.' CREATE TABLE system.moves ( `database` String, @@ -395,7 +395,7 @@ CREATE TABLE system.moves `thread_id` UInt64 ) ENGINE = SystemMoves -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about in-progress data part moves of MergeTree tables. Each data part movement is represented by a single row.' CREATE TABLE system.mutations ( `database` String, @@ -414,25 +414,25 @@ CREATE TABLE system.mutations `latest_fail_reason` String ) ENGINE = SystemMutations -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of mutations and their progress. Each mutation command is represented by a single row.' CREATE TABLE system.numbers ( `number` UInt64 ) ENGINE = SystemNumbers -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Generates all natural numbers, starting from 0 (to 2^64 - 1, and then again) in sorted order.' CREATE TABLE system.numbers_mt ( `number` UInt64 ) ENGINE = SystemNumbers -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Multithreaded version of `system.numbers`. Numbers order is not guaranteed.' CREATE TABLE system.one ( `dummy` UInt8 ) ENGINE = SystemOne -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`.' CREATE TABLE system.part_moves_between_shards ( `database` String, @@ -451,7 +451,7 @@ CREATE TABLE system.part_moves_between_shards `last_exception` String ) ENGINE = SystemShardMoves -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about parts which are currently in a process of moving between shards and their progress.' CREATE TABLE system.parts ( `partition` String, @@ -522,7 +522,7 @@ CREATE TABLE system.parts `part_name` String ALIAS name ) ENGINE = SystemParts -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of currently existing (both active and inactive) parts of all *-MergeTree tables. Each part is represented by a single row.' CREATE TABLE system.parts_columns ( `partition` String, @@ -582,7 +582,7 @@ CREATE TABLE system.parts_columns `part_name` String ALIAS name ) ENGINE = SystemPartsColumns -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of columns of all currently existing parts of all MergeTree tables. Each column is represented by a single row.' CREATE TABLE system.processes ( `is_initial_query` UInt8, @@ -630,7 +630,7 @@ CREATE TABLE system.processes `Settings.Values` Array(String) ALIAS mapValues(Settings) ) ENGINE = SystemProcesses -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of currently executing processes (queries) with their progress.' CREATE TABLE system.projection_parts ( `partition` String, @@ -698,7 +698,7 @@ CREATE TABLE system.projection_parts `part_name` String ALIAS name ) ENGINE = SystemProjectionParts -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of currently existing projection parts (a copy of some part containing aggregated data or just sorted in different order) created for all the projections for all tables within a cluster.' CREATE TABLE system.projection_parts_columns ( `partition` String, @@ -754,7 +754,7 @@ CREATE TABLE system.projection_parts_columns `part_name` String ALIAS name ) ENGINE = SystemProjectionPartsColumns -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of columns of all currently existing projection parts of all MergeTree tables. Each column is represented by a single row.' CREATE TABLE system.quota_limits ( `quota_name` String, @@ -773,7 +773,7 @@ CREATE TABLE system.quota_limits `max_failed_sequential_authentications` Nullable(UInt64) ) ENGINE = SystemQuotaLimits -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about maximums for all intervals of all quotas. Any number of rows or zero can correspond to specific quota.' CREATE TABLE system.quota_usage ( `quota_name` String, @@ -805,7 +805,7 @@ CREATE TABLE system.quota_usage `max_failed_sequential_authentications` Nullable(UInt64) ) ENGINE = SystemQuotaUsage -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains quota usage by the current user: how much is used and how much is left.' CREATE TABLE system.quotas ( `name` String, @@ -818,7 +818,7 @@ CREATE TABLE system.quotas `apply_to_except` Array(String) ) ENGINE = SystemQuotas -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about quotas.' CREATE TABLE system.quotas_usage ( `quota_name` String, @@ -851,7 +851,7 @@ CREATE TABLE system.quotas_usage `max_failed_sequential_authentications` Nullable(UInt64) ) ENGINE = SystemQuotasUsage -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains quota usage by all users.' CREATE TABLE system.replicas ( `database` String, @@ -891,7 +891,7 @@ CREATE TABLE system.replicas `replica_is_active` Map(String, UInt8) ) ENGINE = SystemReplicas -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information and status of all table replicas on current server. Each replica is represented by a single row.' CREATE TABLE system.replicated_fetches ( `database` String, @@ -912,7 +912,7 @@ CREATE TABLE system.replicated_fetches `thread_id` UInt64 ) ENGINE = SystemReplicatedFetches -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about currently running background fetches.' CREATE TABLE system.replicated_merge_tree_settings ( `name` String, @@ -926,7 +926,7 @@ CREATE TABLE system.replicated_merge_tree_settings `is_obsolete` UInt8 ) ENGINE = SystemReplicatedMergeTreeSettings -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all ReplicatedMergeTree engine specific settings, their current and default values along with descriptions. You may change any of them in SETTINGS section in CREATE query. ' CREATE TABLE system.replication_queue ( `database` String, @@ -952,7 +952,7 @@ CREATE TABLE system.replication_queue `merge_type` String ) ENGINE = SystemReplicationQueue -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about tasks from replication queues stored in ClickHouse Keeper, or ZooKeeper, for each table replica.' CREATE TABLE system.role_grants ( `user_name` Nullable(String), @@ -963,7 +963,7 @@ CREATE TABLE system.role_grants `with_admin_option` UInt8 ) ENGINE = SystemRoleGrants -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains the role grants for users and roles. To add entries to this table, use `GRANT role TO user`. Using this table you may find out which roles are assigned to which users or which roles a user has.' CREATE TABLE system.roles ( `name` String, @@ -971,7 +971,7 @@ CREATE TABLE system.roles `storage` String ) ENGINE = SystemRoles -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all roles created at the server.' CREATE TABLE system.row_policies ( `name` String, @@ -987,7 +987,7 @@ CREATE TABLE system.row_policies `apply_to_except` Array(String) ) ENGINE = SystemRowPolicies -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains filters for one particular table, as well as a list of roles and/or users which should use this row policy.' CREATE TABLE system.settings ( `name` String, @@ -1003,7 +1003,7 @@ CREATE TABLE system.settings `is_obsolete` UInt8 ) ENGINE = SystemSettings -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all user-level settings (which can be modified in a scope of query or session), their current and default values along with descriptions.' CREATE TABLE system.settings_profile_elements ( `profile_name` Nullable(String), @@ -1018,7 +1018,7 @@ CREATE TABLE system.settings_profile_elements `inherit_profile` Nullable(String) ) ENGINE = SystemSettingsProfileElements -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Describes the content of each settings profile configured on the server. Including settings constraints, roles and users for which the settings are applied, and parent settings profiles.' CREATE TABLE system.settings_profiles ( `name` String, @@ -1030,7 +1030,7 @@ CREATE TABLE system.settings_profiles `apply_to_except` Array(String) ) ENGINE = SystemSettingsProfiles -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains properties of configured setting profiles.' CREATE TABLE system.stack_trace ( `thread_name` String, @@ -1039,7 +1039,7 @@ CREATE TABLE system.stack_trace `trace` Array(UInt64) ) ENGINE = SystemStackTrace -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Allows to obtain an unsymbolized stacktrace from all the threads of the server process.' CREATE TABLE system.storage_policies ( `policy_name` String, @@ -1054,7 +1054,7 @@ CREATE TABLE system.storage_policies `load_balancing` Enum8('ROUND_ROBIN' = 0, 'LEAST_USED' = 1) ) ENGINE = SystemStoragePolicies -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains information about storage policies and volumes defined in the server configuration.' CREATE TABLE system.table_engines ( `name` String, @@ -1068,7 +1068,7 @@ CREATE TABLE system.table_engines `supports_parallel_insert` UInt8 ) ENGINE = SystemTableEngines -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available table engines along with information whether a particular table engine supports some specific features (e.g. settings, skipping indices, projections, replication, TTL, deduplication, parallel insert, etc.)' CREATE TABLE system.table_functions ( `name` String, @@ -1076,7 +1076,7 @@ CREATE TABLE system.table_functions `allow_readonly` UInt8 ) ENGINE = SystemTableFunctions -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all available table functions with their descriptions.' CREATE TABLE system.tables ( `database` String, @@ -1115,13 +1115,13 @@ CREATE TABLE system.tables `table` String ALIAS name ) ENGINE = SystemTables -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Lists all tables of the current server.' CREATE TABLE system.time_zones ( `time_zone` String ) ENGINE = SystemTimeZones -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of time zones that are supported by the ClickHouse server. This list of timezones might vary depending on the version of ClickHouse.' CREATE TABLE system.user_directories ( `name` String, @@ -1130,7 +1130,7 @@ CREATE TABLE system.user_directories `precedence` UInt64 ) ENGINE = SystemUserDirectories -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains the information about configured user directories - directories on the file system from which ClickHouse server is allowed to read user provided data.' CREATE TABLE system.users ( `name` String, @@ -1151,22 +1151,22 @@ CREATE TABLE system.users `default_database` String ) ENGINE = SystemUsers -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains a list of all users profiles either configured at the server through a configuration file or created via SQL.' CREATE TABLE system.warnings ( `message` String ) ENGINE = SystemWarnings -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Contains warnings about server configuration to be displayed by clickhouse-client right after it connects to the server.' CREATE TABLE system.zeros ( `zero` UInt8 ) ENGINE = SystemZeros -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Produces unlimited number of non-materialized zeros.' CREATE TABLE system.zeros_mt ( `zero` UInt8 ) ENGINE = SystemZeros -COMMENT 'SYSTEM TABLE is built on the fly.' +COMMENT 'Multithreaded version of system.zeros.' diff --git a/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference b/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference index 848abb332bb..e1e1ab3f441 100644 --- a/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference +++ b/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference @@ -1 +1 @@ -CREATE TABLE system.rocksdb\n(\n `database` String,\n `table` String,\n `name` String,\n `value` UInt64\n)\nENGINE = SystemRocksDB\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.rocksdb\n(\n `database` String,\n `table` String,\n `name` String,\n `value` UInt64\n)\nENGINE = SystemRocksDB\nCOMMENT \'Contains a list of metrics exposed from embedded RocksDB.\' From 042c36313da07f77d3f2acae09d2ffb858fbfe37 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 16 May 2024 14:42:26 +0200 Subject: [PATCH 007/105] Updated to use required memory and prevent creating extra buffer --- .../IO/WriteBufferFromAzureBlobStorage.cpp | 127 +++++++++++++++--- .../IO/WriteBufferFromAzureBlobStorage.h | 14 +- 2 files changed, 120 insertions(+), 21 deletions(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 2c90e3a9003..ebb3fc4a82d 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -23,7 +23,6 @@ struct WriteBufferFromAzureBlobStorage::PartData { Memory<> memory; size_t data_size = 0; - std::string block_id; }; BufferAllocationPolicyPtr createBufferAllocationPolicy(const AzureObjectStorageSettings & settings) @@ -119,22 +118,32 @@ void WriteBufferFromAzureBlobStorage::preFinalize() // This function should not be run again is_prefinalized = true; + hidePartialData(); + + if (hidden_size > 0) + detachBuffer(); + + setFakeBufferWhenPreFinalized(); + /// If there is only one block and size is less than or equal to max_single_part_upload_size /// then we use single part upload instead of multi part upload - if (buffer_allocation_policy->getBufferNumber() == 1) + if (detached_part_data.size() == 1) { - size_t data_size = size_t(position() - memory.data()); - if (data_size <= max_single_part_upload_size) + if (detached_part_data.front().data_size <= max_single_part_upload_size) { + auto part_data = std::move(detached_part_data.front()); auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); - Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(memory.data()), data_size); - execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, data_size); + Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(part_data.memory.data()), part_data.data_size); + execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, part_data.data_size); LOG_TRACE(log, "Committed single block for blob `{}`", blob_path); + + detached_part_data.pop_front(); + block_ids.clear(); return; } } - writePart(); + writeMultipartUpload(); } void WriteBufferFromAzureBlobStorage::finalizeImpl() @@ -144,9 +153,13 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() if (!is_prefinalized) preFinalize(); + chassert(offset() == 0); + chassert(hidden_size == 0); + + task_tracker->waitAll(); + if (!block_ids.empty()) { - task_tracker->waitAll(); auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, max_unexpected_write_error_retries); LOG_TRACE(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path); @@ -156,13 +169,60 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() void WriteBufferFromAzureBlobStorage::nextImpl() { task_tracker->waitIfAny(); - writePart(); + + hidePartialData(); + + reallocateFirstBuffer(); + + if (available() > 0) + return; + + detachBuffer(); + + if (detached_part_data.size() > 1) + writeMultipartUpload(); + allocateBuffer(); } +void WriteBufferFromAzureBlobStorage::hidePartialData() +{ + if (write_settings.remote_throttler) + write_settings.remote_throttler->add(offset(), ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); + + chassert(memory.size() >= hidden_size + offset()); + + hidden_size += offset(); + chassert(memory.data() + hidden_size == working_buffer.begin() + offset()); + chassert(memory.data() + hidden_size == position()); + + WriteBuffer::set(memory.data() + hidden_size, memory.size() - hidden_size); + chassert(offset() == 0); +} + +void WriteBufferFromAzureBlobStorage::reallocateFirstBuffer() +{ + chassert(offset() == 0); + + if (buffer_allocation_policy->getBufferNumber() > 1 || available() > 0) + return; + + const size_t max_first_buffer = buffer_allocation_policy->getBufferSize(); + if (memory.size() == max_first_buffer) + return; + + size_t size = std::min(memory.size() * 2, max_first_buffer); + memory.resize(size); + + WriteBuffer::set(memory.data() + hidden_size, memory.size() - hidden_size); + chassert(offset() == 0); +} + void WriteBufferFromAzureBlobStorage::allocateBuffer() { buffer_allocation_policy->nextBuffer(); + chassert(0 == hidden_size); + auto size = buffer_allocation_policy->getBufferSize(); if (buffer_allocation_policy->getBufferNumber() == 1) @@ -172,30 +232,57 @@ void WriteBufferFromAzureBlobStorage::allocateBuffer() WriteBuffer::set(memory.data(), memory.size()); } -void WriteBufferFromAzureBlobStorage::writePart() +void WriteBufferFromAzureBlobStorage::detachBuffer() { - auto data_size = size_t(position() - memory.data()); + size_t data_size = size_t(position() - memory.data()); if (data_size == 0) return; - const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64)); - std::shared_ptr part_data = std::make_shared(std::move(memory), data_size, block_id); - WriteBuffer::set(nullptr, 0); + chassert(data_size == hidden_size); - auto upload_worker = [this, part_data] () + auto buf = std::move(memory); + + WriteBuffer::set(nullptr, 0); + total_size += hidden_size; + hidden_size = 0; + + detached_part_data.push_back({std::move(buf), data_size}); + WriteBuffer::set(nullptr, 0); +} + +void WriteBufferFromAzureBlobStorage::writePart(WriteBufferFromAzureBlobStorage::PartData && part_data) +{ + const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64)); + auto worker_data = std::make_shared>(block_id, std::move(part_data)); + + auto upload_worker = [this, worker_data] () { + auto & data_size = std::get<1>(*worker_data).data_size; + auto & data_block_id = std::get<0>(*worker_data); auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); - Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(part_data->memory.data()), part_data->data_size); - execWithRetry([&](){ block_blob_client.StageBlock(part_data->block_id, memory_stream); }, max_unexpected_write_error_retries, part_data->data_size); - - if (write_settings.remote_throttler) - write_settings.remote_throttler->add(part_data->data_size, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); + Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(std::get<1>(*worker_data).memory.data()), data_size); + execWithRetry([&](){ block_blob_client.StageBlock(data_block_id, memory_stream); }, max_unexpected_write_error_retries, data_size); }; task_tracker->add(std::move(upload_worker)); } +void WriteBufferFromAzureBlobStorage::setFakeBufferWhenPreFinalized() +{ + WriteBuffer::set(fake_buffer_when_prefinalized, sizeof(fake_buffer_when_prefinalized)); +} + +void WriteBufferFromAzureBlobStorage::writeMultipartUpload() +{ + while (!detached_part_data.empty()) + { + writePart(std::move(detached_part_data.front())); + detached_part_data.pop_front(); + } +} + } #endif + diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index 96ba6acefff..25d515df8c8 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -48,8 +48,13 @@ public: private: struct PartData; - void writePart(); + void writeMultipartUpload(); + void writePart(PartData && part_data); + void detachBuffer(); + void reallocateFirstBuffer(); void allocateBuffer(); + void hidePartialData(); + void setFakeBufferWhenPreFinalized(); void finalizeImpl() override; void execWithRetry(std::function func, size_t num_tries, size_t cost = 0); @@ -77,9 +82,16 @@ private: MemoryBufferPtr allocateBuffer() const; + char fake_buffer_when_prefinalized[1] = {}; + bool first_buffer=true; + size_t total_size = 0; + size_t hidden_size = 0; + std::unique_ptr task_tracker; + + std::deque detached_part_data; }; } From 7e5bf14135604ae6d2b3b5861d93583093993482 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 11 Apr 2024 00:01:48 +0200 Subject: [PATCH 008/105] Add support for LineString WKT --- .../en/sql-reference/functions/geo/polygon.md | 30 ++++++++++++++- src/DataTypes/DataTypeCustomGeo.cpp | 7 ++++ src/DataTypes/DataTypeCustomGeo.h | 6 +++ src/Functions/geometryConverters.h | 38 +++++++++++++++++++ src/Functions/readWkt.cpp | 6 +++ 5 files changed, 86 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/geo/polygon.md b/docs/en/sql-reference/functions/geo/polygon.md index c2572779ada..25a7a1fac8e 100644 --- a/docs/en/sql-reference/functions/geo/polygon.md +++ b/docs/en/sql-reference/functions/geo/polygon.md @@ -142,6 +142,34 @@ SELECT readWKTPoint('POINT (1.2 3.4)'); (1.2,3.4) ``` +## readWKTLineString + +Parses a Well-Known Text (WKT) representation of a LineString geometry and returns it in the internal ClickHouse format. + +### Syntax + +```sql +readWKTLineString(wkt_string) +``` + +### Arguments + +- `wkt_string`: The input WKT string representing a LineString geometry. + +### Returned value + +The function returns a ClickHouse internal representation of the linestring geometry. + +### Example + +```sql +SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'); +``` + +```response +[(1,1),(2,2),(3,3),(1,1)] +``` + ## readWKTRing Parses a Well-Known Text (WKT) representation of a Polygon geometry and returns a ring (closed linestring) in the internal ClickHouse format. @@ -163,7 +191,7 @@ The function returns a ClickHouse internal representation of the ring (closed li ### Example ```sql -SELECT readWKTRing('LINESTRING (1 1, 2 2, 3 3, 1 1)'); +SELECT readWKTRing('POLYGON ((1 1, 2 2, 3 3, 1 1))'); ``` ```response diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp index f7d05fa3be6..0736d837d46 100644 --- a/src/DataTypes/DataTypeCustomGeo.cpp +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -17,6 +17,13 @@ void registerDataTypeDomainGeo(DataTypeFactory & factory) std::make_unique(std::make_unique())); }); + // Custom type for simple line which consists from several segments. + factory.registerSimpleDataTypeCustom("LineString", [] + { + return std::make_pair(DataTypeFactory::instance().get("Array(Point)"), + std::make_unique(std::make_unique())); + }); + // Custom type for simple polygon without holes stored as Array(Point) factory.registerSimpleDataTypeCustom("Ring", [] { diff --git a/src/DataTypes/DataTypeCustomGeo.h b/src/DataTypes/DataTypeCustomGeo.h index c2a83b3e577..0a1c83e4638 100644 --- a/src/DataTypes/DataTypeCustomGeo.h +++ b/src/DataTypes/DataTypeCustomGeo.h @@ -11,6 +11,12 @@ public: DataTypePointName() : DataTypeCustomFixedName("Point") {} }; +class DataTypeLineStringName : public DataTypeCustomFixedName +{ +public: + DataTypeLineStringName() : DataTypeCustomFixedName("LineString") {} +}; + class DataTypeRingName : public DataTypeCustomFixedName { public: diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 97162fa9dd0..98865a9048f 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -28,6 +28,9 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +template +using LineString = boost::geometry::model::linestring; + template using Ring = boost::geometry::model::ring; @@ -38,11 +41,13 @@ template using MultiPolygon = boost::geometry::model::multi_polygon>; using CartesianPoint = boost::geometry::model::d2::point_xy; +using CartesianLineString = LineString; using CartesianRing = Ring; using CartesianPolygon = Polygon; using CartesianMultiPolygon = MultiPolygon; using SphericalPoint = boost::geometry::model::point>; +using SphericalLineString = LineString; using SphericalRing = Ring; using SphericalPolygon = Polygon; using SphericalMultiPolygon = MultiPolygon; @@ -208,6 +213,39 @@ private: ColumnFloat64::Container & second_container; }; +/// Serialize Point, LineString as LineString +template +class LineStringSerializer +{ +public: + LineStringSerializer() + : offsets(ColumnUInt64::create()) + {} + + explicit LineStringSerializer(size_t n) + : offsets(ColumnUInt64::create(n)) + {} + + void add(const LineString & ring) + { + size += ring.size(); + offsets->insertValue(size); + for (const auto & point : ring) + point_serializer.add(point); + } + + ColumnPtr finalize() + { + return ColumnArray::create(point_serializer.finalize(), std::move(offsets)); + } + +private: + size_t size = 0; + PointSerializer point_serializer; + ColumnUInt64::MutablePtr offsets; +}; + +/// Almost the same as LineStringSerializer /// Serialize Point, Ring as Ring template class RingSerializer diff --git a/src/Functions/readWkt.cpp b/src/Functions/readWkt.cpp index ddc847b1ca5..f4975e85d0d 100644 --- a/src/Functions/readWkt.cpp +++ b/src/Functions/readWkt.cpp @@ -82,6 +82,11 @@ struct ReadWKTPointNameHolder static constexpr const char * name = "readWKTPoint"; }; +struct ReadWKTLineStringNameHolder +{ + static constexpr const char * name = "readWKTLineString"; +}; + struct ReadWKTRingNameHolder { static constexpr const char * name = "readWKTRing"; @@ -102,6 +107,7 @@ struct ReadWKTMultiPolygonNameHolder REGISTER_FUNCTION(ReadWKT) { factory.registerFunction, ReadWKTPointNameHolder>>(); + factory.registerFunction, ReadWKTLineStringNameHolder>>(); factory.registerFunction, ReadWKTRingNameHolder>>(); factory.registerFunction, ReadWKTPolygonNameHolder>>(); factory.registerFunction, ReadWKTMultiPolygonNameHolder>>(); From 40c069dc28c8d08c5477202db9dc98ba53edec2d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 11 Apr 2024 17:26:11 +0000 Subject: [PATCH 009/105] Fix style --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 1c601bc200a..31bf406ded1 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -485,6 +485,7 @@ Liao LibFuzzer LightHouse LineAsString +LineString Linf LinfDistance LinfNorm @@ -2275,6 +2276,7 @@ randomStringUTF rankCorr rapidjson rawblob +readWKTLineString readWKTMultiPolygon readWKTPoint readWKTPolygon From 54f2549ef59ca72478ef98a5289264e60790dcc1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 12 Apr 2024 12:40:29 +0000 Subject: [PATCH 010/105] Save --- src/Functions/geometryConverters.h | 26 +++++++++++++++++++++++++ src/Functions/polygonsIntersection.cpp | 2 ++ src/Functions/polygonsSymDifference.cpp | 2 ++ src/Functions/polygonsUnion.cpp | 2 ++ src/Functions/polygonsWithin.cpp | 2 ++ 5 files changed, 34 insertions(+) diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 98865a9048f..ec74d415824 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -90,6 +90,29 @@ struct ColumnToPointsConverter } }; + +/** + * Class which converts Column with type Array(Tuple(Float64, Float64)) to a vector of boost linestring type. +*/ +template +struct ColumnToLineStringsConverter +{ + static std::vector> convert(ColumnPtr col) + { + const IColumn::Offsets & offsets = typeid_cast(*col).getOffsets(); + size_t prev_offset = 0; + std::vector> answer; + answer.reserve(offsets.size()); + auto tmp = ColumnToPointsConverter::convert(typeid_cast(*col).getDataPtr()); + for (size_t offset : offsets) + { + answer.emplace_back(tmp.begin() + prev_offset, tmp.begin() + offset); + prev_offset = offset; + } + return answer; + } +}; + /** * Class which converts Column with type Array(Tuple(Float64, Float64)) to a vector of boost ring type. */ @@ -382,6 +405,9 @@ static void callOnGeometryDataType(DataTypePtr type, F && f) /// There is no Point type, because for most of geometry functions it is useless. if (factory.get("Point")->equals(*type)) return f(ConverterType>()); + /// There is some ambiguity, we don't take into consideration a name of a custom type. + // else if (factory.get("LineString")->equals(*type)) + // return f(ConverterType>()); else if (factory.get("Ring")->equals(*type)) return f(ConverterType>()); else if (factory.get("Polygon")->equals(*type)) diff --git a/src/Functions/polygonsIntersection.cpp b/src/Functions/polygonsIntersection.cpp index 77484e7e63c..329242e762e 100644 --- a/src/Functions/polygonsIntersection.cpp +++ b/src/Functions/polygonsIntersection.cpp @@ -73,6 +73,8 @@ public: if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); diff --git a/src/Functions/polygonsSymDifference.cpp b/src/Functions/polygonsSymDifference.cpp index 194b7f2cfd7..3c219d0facb 100644 --- a/src/Functions/polygonsSymDifference.cpp +++ b/src/Functions/polygonsSymDifference.cpp @@ -71,6 +71,8 @@ public: if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); diff --git a/src/Functions/polygonsUnion.cpp b/src/Functions/polygonsUnion.cpp index 37d865af50a..969eb2f78fb 100644 --- a/src/Functions/polygonsUnion.cpp +++ b/src/Functions/polygonsUnion.cpp @@ -71,6 +71,8 @@ public: if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); diff --git a/src/Functions/polygonsWithin.cpp b/src/Functions/polygonsWithin.cpp index 35a9e17cdfd..c63ad5ef868 100644 --- a/src/Functions/polygonsWithin.cpp +++ b/src/Functions/polygonsWithin.cpp @@ -75,6 +75,8 @@ public: if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be Point", getName()); + else if constexpr (std::is_same_v, LeftConverter> || std::is_same_v, RightConverter>) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Any argument of function {} must not be LineString", getName()); else { auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); From 6510f804478303fb5d420d43459201018bea6dc1 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 23 May 2024 16:59:07 +0200 Subject: [PATCH 011/105] Updated test to fix unrecognized option error --- tests/queries/0_stateless/02941_variant_type_1.sh | 6 +++--- tests/queries/0_stateless/02941_variant_type_2.sh | 6 +++--- tests/queries/0_stateless/02941_variant_type_3.sh | 6 +++--- tests/queries/0_stateless/02941_variant_type_4.sh | 6 +++--- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02941_variant_type_1.sh b/tests/queries/0_stateless/02941_variant_type_1.sh index 22ca909a26e..723de45eaad 100755 --- a/tests/queries/0_stateless/02941_variant_type_1.sh +++ b/tests/queries/0_stateless/02941_variant_type_1.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test1_insert() { @@ -115,11 +115,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), 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(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), 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(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_2.sh b/tests/queries/0_stateless/02941_variant_type_2.sh index 91ba0285bd8..f43cd2bb0d6 100755 --- a/tests/queries/0_stateless/02941_variant_type_2.sh +++ b/tests/queries/0_stateless/02941_variant_type_2.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test4_insert() { @@ -61,11 +61,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), 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(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), 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(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_3.sh b/tests/queries/0_stateless/02941_variant_type_3.sh index 8a039a02d6d..f4b2b304f56 100755 --- a/tests/queries/0_stateless/02941_variant_type_3.sh +++ b/tests/queries/0_stateless/02941_variant_type_3.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192 " +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test5_insert() { @@ -63,11 +63,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), 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(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), 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(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_4.sh b/tests/queries/0_stateless/02941_variant_type_4.sh index b003bcdcef1..f9a16847864 100755 --- a/tests/queries/0_stateless/02941_variant_type_4.sh +++ b/tests/queries/0_stateless/02941_variant_type_4.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192 " +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test6_insert() { @@ -58,11 +58,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), 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(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), 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(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" From ac6dae516547afe8390c868787219c225a08bdd3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 23 May 2024 15:26:21 +0000 Subject: [PATCH 012/105] Correctly compare custom types --- src/Functions/geometryConverters.h | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index ec74d415824..1478de14c4d 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -405,10 +405,11 @@ static void callOnGeometryDataType(DataTypePtr type, F && f) /// There is no Point type, because for most of geometry functions it is useless. if (factory.get("Point")->equals(*type)) return f(ConverterType>()); - /// There is some ambiguity, we don't take into consideration a name of a custom type. - // else if (factory.get("LineString")->equals(*type)) - // return f(ConverterType>()); - else if (factory.get("Ring")->equals(*type)) + /// We should take the name into consideration to avoid ambiguity. + /// Because for example both Ring and LineString are resolved to Array. + else if (factory.get("LineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "Ring") + return f(ConverterType>()); + else if (factory.get("Ring")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "Ring") return f(ConverterType>()); else if (factory.get("Polygon")->equals(*type)) return f(ConverterType>()); From aa93ace6d8d92cca4dedcf5a2bebf7c8b2b966d6 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 24 May 2024 10:42:59 +0200 Subject: [PATCH 013/105] Fixed more tests with unrecognized option error --- tests/queries/0_stateless/02941_variant_type_4.sh | 2 +- .../0_stateless/03037_dynamic_merges_1_horizontal.sh | 6 +++--- .../queries/0_stateless/03037_dynamic_merges_1_vertical.sh | 6 +++--- .../0_stateless/03039_dynamic_all_merge_algorithms_2.sh | 6 +++--- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02941_variant_type_4.sh b/tests/queries/0_stateless/02941_variant_type_4.sh index b90f9090b07..f9a16847864 100755 --- a/tests/queries/0_stateless/02941_variant_type_4.sh +++ b/tests/queries/0_stateless/02941_variant_type_4.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1"g +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test6_insert() { diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh index 7c1ac41cfdc..0e4c2a0cc5e 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --index_granularity_bytes 10485760 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760 --index_granularity 8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760" function test() { @@ -41,12 +41,12 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh index 927ceac72b5..e47b11614af 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh @@ -9,7 +9,7 @@ CLICKHOUSE_LOG_COMMENT= -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --index_granularity_bytes 10485760 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760 --index_granularity 8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760" function test() { echo "test" @@ -41,11 +41,11 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh b/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh index 02362012960..0be0e973d01 100755 --- a/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh +++ b/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --index_granularity_bytes 10485760 --index_granularity 8128 --merge_max_block_size 8128" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8128" function test() @@ -44,7 +44,7 @@ echo "MergeTree wide + horizontal merge" test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1" echo "MergeTree compact + vertical merge" -test "min_rows_for_wide_part=100000000000, min_bytes_for_wide_part=1000000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1" +test "min_rows_for_wide_part=100000000000, min_bytes_for_wide_part=1000000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128;" echo "MergeTree wide + vertical merge" -test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1" +test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128;" From d2739d495157026fa3df6e7c140bb2800fad7cea Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 24 May 2024 13:37:57 +0200 Subject: [PATCH 014/105] Fixed merge_max_block_size setting --- tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh | 1 + .../0_stateless/03037_dynamic_merges_1_horizontal.sh | 6 +++--- .../queries/0_stateless/03037_dynamic_merges_1_vertical.sh | 6 +++--- .../0_stateless/03039_dynamic_all_merge_algorithms_2.sh | 6 +++--- 4 files changed, 10 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index caa600298ce..b100f96befa 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -24,6 +24,7 @@ function check_refcnt_for_table() local log_file log_file=$(mktemp "$CUR_DIR/clickhouse-tests.XXXXXX.log") local args=( + --allow_repeated_settings --format Null --max_threads 1 --max_block_size 1 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh index 0e4c2a0cc5e..0c9060d261a 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size_bytes=10485760" function test() { @@ -41,12 +41,12 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh index e47b11614af..2384350fa85 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh @@ -9,7 +9,7 @@ CLICKHOUSE_LOG_COMMENT= -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size_bytes=10485760" function test() { echo "test" @@ -41,11 +41,11 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh b/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh index 0be0e973d01..68cc72faf59 100755 --- a/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh +++ b/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8128" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1" function test() @@ -44,7 +44,7 @@ echo "MergeTree wide + horizontal merge" test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1" echo "MergeTree compact + vertical merge" -test "min_rows_for_wide_part=100000000000, min_bytes_for_wide_part=1000000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128;" +test "min_rows_for_wide_part=100000000000, min_bytes_for_wide_part=1000000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128, merge_max_block_size=8128;" echo "MergeTree wide + vertical merge" -test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128;" +test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128, merge_max_block_size=8128;" From 33306993c132508d93a9bd2c4d063e73d1d6165d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 24 May 2024 18:02:45 +0000 Subject: [PATCH 015/105] Better --- src/Functions/geometryConverters.h | 10 +++++++--- src/Functions/readWkt.cpp | 25 ++++++++++++++++++++++++- src/Functions/wkt.cpp | 8 ++++++++ 3 files changed, 39 insertions(+), 4 deletions(-) diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 1478de14c4d..03831d37e0c 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -405,12 +405,16 @@ static void callOnGeometryDataType(DataTypePtr type, F && f) /// There is no Point type, because for most of geometry functions it is useless. if (factory.get("Point")->equals(*type)) return f(ConverterType>()); + /// We should take the name into consideration to avoid ambiguity. - /// Because for example both Ring and LineString are resolved to Array. - else if (factory.get("LineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "Ring") + /// Because for example both Ring and LineString are resolved to Array(Tuple(Point)). + else if (factory.get("LineString")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "LineString") return f(ConverterType>()); - else if (factory.get("Ring")->equals(*type) && type->getCustomName() && type->getCustomName()->getName() == "Ring") + + /// For backward compatibility if we call this function not on a custom type, we will consider Array(Tuple(Point)) as type Ring. + else if (factory.get("Ring")->equals(*type)) return f(ConverterType>()); + else if (factory.get("Polygon")->equals(*type)) return f(ConverterType>()); else if (factory.get("MultiPolygon")->equals(*type)) diff --git a/src/Functions/readWkt.cpp b/src/Functions/readWkt.cpp index f4975e85d0d..ccd969af58b 100644 --- a/src/Functions/readWkt.cpp +++ b/src/Functions/readWkt.cpp @@ -107,7 +107,30 @@ struct ReadWKTMultiPolygonNameHolder REGISTER_FUNCTION(ReadWKT) { factory.registerFunction, ReadWKTPointNameHolder>>(); - factory.registerFunction, ReadWKTLineStringNameHolder>>(); + factory.registerFunction, ReadWKTLineStringNameHolder>>(FunctionDocumentation + { + .description=R"( +Parses a Well-Known Text (WKT) representation of a LineString geometry and returns it in the internal ClickHouse format. +)", + .syntax = "readWKTLineString(wkt_string)", + .arguments{ + {"wkt_string", "The input WKT string representing a LineString geometry."} + }, + .returned_value = "The function returns a ClickHouse internal representation of the linestring geometry.", + .examples{ + {"first call", "SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)');", R"( + ┌─readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')─┐ +1. │ [(1,1),(2,2),(3,3),(1,1)] │ + └──────────────────────────────────────────────────────┘ + )"}, + {"second call", "SELECT toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'));", R"( + ┌─toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'))─┐ +1. │ LineString │ + └──────────────────────────────────────────────────────────────────┘ + )"}, + }, + .categories{"Unique identifiers"} + }); factory.registerFunction, ReadWKTRingNameHolder>>(); factory.registerFunction, ReadWKTPolygonNameHolder>>(); factory.registerFunction, ReadWKTMultiPolygonNameHolder>>(); diff --git a/src/Functions/wkt.cpp b/src/Functions/wkt.cpp index afcfabd0bf4..678ec02d229 100644 --- a/src/Functions/wkt.cpp +++ b/src/Functions/wkt.cpp @@ -41,6 +41,14 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + /* + * Functions like recursiveRemoveLowCardinality don't pay enough attention to custom types and just erase + * the information about it during type conversions. + * While it is a big problem the quick solution would be just to disable default low cardinality implementation + * because it doesn't make a lot of sense for geo types. + */ + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override { auto res_column = ColumnString::create(); From 1fa17f1035012c4f9a215846a35fe74bbb21d3c4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 24 May 2024 19:09:39 +0000 Subject: [PATCH 016/105] Fixed style --- src/Functions/readWkt.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Functions/readWkt.cpp b/src/Functions/readWkt.cpp index ccd969af58b..eb262777b0d 100644 --- a/src/Functions/readWkt.cpp +++ b/src/Functions/readWkt.cpp @@ -119,14 +119,14 @@ Parses a Well-Known Text (WKT) representation of a LineString geometry and retur .returned_value = "The function returns a ClickHouse internal representation of the linestring geometry.", .examples{ {"first call", "SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)');", R"( - ┌─readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')─┐ -1. │ [(1,1),(2,2),(3,3),(1,1)] │ - └──────────────────────────────────────────────────────┘ +┌─readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')─┐ +│ [(1,1),(2,2),(3,3),(1,1)] │ +└──────────────────────────────────────────────────────┘ )"}, {"second call", "SELECT toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'));", R"( - ┌─toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'))─┐ -1. │ LineString │ - └──────────────────────────────────────────────────────────────────┘ +┌─toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'))─┐ +│ LineString │ +└──────────────────────────────────────────────────────────────────┘ )"}, }, .categories{"Unique identifiers"} From 8484decf4891c7c8939688e3261ea3b5e4c53584 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 27 May 2024 10:32:17 +0200 Subject: [PATCH 017/105] Fix test fails --- .../0_stateless/03037_dynamic_merges_1_horizontal.sh | 6 +++--- .../queries/0_stateless/03037_dynamic_merges_1_vertical.sh | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh index 0c9060d261a..887b2ed94d7 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size_bytes=10485760" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1" function test() { @@ -41,12 +41,12 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh index 2384350fa85..371ae87c2ef 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh @@ -9,7 +9,7 @@ CLICKHOUSE_LOG_COMMENT= -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size_bytes=10485760" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1" function test() { echo "test" @@ -41,11 +41,11 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" From 2a433112ee4e695d59ee79445c96a87f1d0caea3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 27 May 2024 14:01:52 +0000 Subject: [PATCH 018/105] Add test --- .../0_stateless/03164_linestring_geometry.reference | 11 +++++++++++ .../queries/0_stateless/03164_linestring_geometry.sql | 8 ++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/03164_linestring_geometry.reference create mode 100644 tests/queries/0_stateless/03164_linestring_geometry.sql diff --git a/tests/queries/0_stateless/03164_linestring_geometry.reference b/tests/queries/0_stateless/03164_linestring_geometry.reference new file mode 100644 index 00000000000..1f68df04614 --- /dev/null +++ b/tests/queries/0_stateless/03164_linestring_geometry.reference @@ -0,0 +1,11 @@ +-- { echoOn } +SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'); +[(1,1),(2,2),(3,3),(1,1)] +SELECT toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')); +LineString +SELECT wkt(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')); +LINESTRING(1 1,2 2,3 3,1 1) +-- Native Array(Tuple(Float64, Float64)) is threated as Ring, not as LineString. +WITH wkt(CAST([(1, 1), (2, 2), (3, 3)], 'Array(Tuple(Float64, Float64))')) as x +SELECT x, toTypeName(x), readWKTRing(x) as y, toTypeName(y); +POLYGON((1 1,2 2,3 3)) String [(1,1),(2,2),(3,3)] Ring diff --git a/tests/queries/0_stateless/03164_linestring_geometry.sql b/tests/queries/0_stateless/03164_linestring_geometry.sql new file mode 100644 index 00000000000..e4f1d1295e7 --- /dev/null +++ b/tests/queries/0_stateless/03164_linestring_geometry.sql @@ -0,0 +1,8 @@ +-- { echoOn } +SELECT readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)'); +SELECT toTypeName(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')); +SELECT wkt(readWKTLineString('LINESTRING (1 1, 2 2, 3 3, 1 1)')); + +-- Native Array(Tuple(Float64, Float64)) is threated as Ring, not as LineString. +WITH wkt(CAST([(1, 1), (2, 2), (3, 3)], 'Array(Tuple(Float64, Float64))')) as x +SELECT x, toTypeName(x), readWKTRing(x) as y, toTypeName(y); From 49962e7b6e4c444bb9f2599d7a380bfdf273b231 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 30 May 2024 18:12:32 +0200 Subject: [PATCH 019/105] Docs: add info about analyzer incompatibilities --- docs/en/operations/analyzer.md | 92 ++++++++++++++++++++++++++++++++++ 1 file changed, 92 insertions(+) create mode 100644 docs/en/operations/analyzer.md diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md new file mode 100644 index 00000000000..05f7856c5e3 --- /dev/null +++ b/docs/en/operations/analyzer.md @@ -0,0 +1,92 @@ +--- +slug: /en/operations/analyzer +sidebar_label: Analyzer +--- + +# Analyzer + + + +## Known incompatibilities + +In ClickHouse version `24.3`, new query analysis was enabled by default. +Despite fixing a large number of bugs and introducing new optimizations, it also introduces some breaking changes in ClickHouse behaviour. + +### Invalid before optimization queries + +The previous query planning infrastructure applied AST-level optimizations before the query validation step. +Optimizations could rewrite the initial query so it becomes valid and can be executed. + +In the new infrastructure, query validation takes place before the optimization step. +This means that invalid queries that were possible to execute before are now unsupported. + +**Example:** + +The following query uses column `number` in the projection list when only `toString(number)` is available after the aggregation. +In the old infrastructure, `GROUP BY toString(number)` was optimized into `GROUP BY number,` making the query valid. + +```sql +SELECT number +FROM numbers(1) +GROUP BY toString(number) +``` + +### Known incompatibilities for JOIN clause + +* Using expression from `SELECT` list in `JOIN` key as an expression from LEFT table. Example. Fix (best effort, should be under compatibility flag). +* Similar issue ^. Alias for column (in select list) now applied to JOIN result (and not to left table). Example from Denny Crane. New behavior is the correct one. Will try to add best-effort compatibility setting. +* Columns names are changed for some queries. This might breaks some scripts. Example. + + +### Projection column names changes + +During projection names computation aliases are not substituted. + +```sql +SELECT + 1 + 1 AS x, + x + 1 +SETTINGS allow_experimental_analyzer = 0 +FORMAT PrettyCompact + + ┌─x─┬─plus(plus(1, 1), 1)─┐ +1. │ 2 │ 3 │ + └───┴─────────────────────┘ + +SELECT + 1 + 1 AS x, + x + 1 +SETTINGS allow_experimental_analyzer = 1 +FORMAT PrettyCompact + +Query id: 2a5e39a3-3b64-49fd-bad3-0e351931ac99 + + ┌─x─┬─plus(x, 1)─┐ +1. │ 2 │ 3 │ + └───┴────────────┘ +``` + +### Incompatible function arguments types + +In the new infrastructure type inference happens during initial query analysis. +This change means that type checks are done before short-circuit evaluation; thus, `if` function arguments must always have a common supertype. + +**Example:** + +The following query fails with `There is no supertype for types Array(UInt8), String because some of them are Array and some of them are not`: + +```sql +SELECT toTypeName(if(0, [2, 3, 4], 'String')) +``` + +### Heterogeneous clusters + +The new analyzer significantly changed the communication protocol between servers in the cluster. Thus, it's impossible to run distributed queries on servers with different `allow_experimental_analyzer` setting values. + +### Unsupported features + +The list of features new analyzer currently doesn't support: + +- Annoy index. +- Hypothesis index. Work in progress [here](https://github.com/ClickHouse/ClickHouse/pull/48381). +- Window view is not supported. There are no plans to support it in the future. From 35d3cddf993822b31a9cdf21418295f9289be961 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 30 May 2024 21:36:12 +0200 Subject: [PATCH 020/105] Added condition for single block commit --- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index ebb3fc4a82d..8333bf45e67 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -127,7 +127,7 @@ void WriteBufferFromAzureBlobStorage::preFinalize() /// If there is only one block and size is less than or equal to max_single_part_upload_size /// then we use single part upload instead of multi part upload - if (detached_part_data.size() == 1) + if (detached_part_data.size() == 1 && block_ids.empty()) { if (detached_part_data.front().data_size <= max_single_part_upload_size) { @@ -168,6 +168,11 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() void WriteBufferFromAzureBlobStorage::nextImpl() { + if (is_prefinalized) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot write to prefinalized buffer for Azure Blob Storage, the file could have been created"); + task_tracker->waitIfAny(); hidePartialData(); From 216c67c9f8cb8dcb79fd061f92bc3e9576be1fd9 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 31 May 2024 10:05:15 +0200 Subject: [PATCH 021/105] Fixed style check --- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 8333bf45e67..16bd70cf2e5 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -19,6 +19,11 @@ namespace ProfileEvents namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + struct WriteBufferFromAzureBlobStorage::PartData { Memory<> memory; From 76ee999cae00b1644980cb45aa20502efdd0cd7a Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 31 May 2024 13:57:58 +0200 Subject: [PATCH 022/105] Add an example --- docs/en/operations/analyzer.md | 26 +++++++++++++++++++------- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index 05f7856c5e3..62bb45eeb7d 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -20,10 +20,7 @@ Optimizations could rewrite the initial query so it becomes valid and can be exe In the new infrastructure, query validation takes place before the optimization step. This means that invalid queries that were possible to execute before are now unsupported. -**Example:** - -The following query uses column `number` in the projection list when only `toString(number)` is available after the aggregation. -In the old infrastructure, `GROUP BY toString(number)` was optimized into `GROUP BY number,` making the query valid. +**Example 1:** ```sql SELECT number @@ -31,7 +28,24 @@ FROM numbers(1) GROUP BY toString(number) ``` -### Known incompatibilities for JOIN clause +The following query uses column `number` in the projection list when only `toString(number)` is available after the aggregation. +In the old infrastructure, `GROUP BY toString(number)` was optimized into `GROUP BY number,` making the query valid. + +**Example 2:** + +```sql +SELECT + number % 2 AS n, + sum(number) +FROM numbers(10) +GROUP BY n +HAVING number > 5 +``` + +The same problem occurs in this query: column `number` is used after aggregation with another key. +The previous query analyzer fixed this query by moving the `number > 5` filter from the `HAVING` clause to the `WHERE` clause. + +### Known incompatibilities of JOIN clause * Using expression from `SELECT` list in `JOIN` key as an expression from LEFT table. Example. Fix (best effort, should be under compatibility flag). * Similar issue ^. Alias for column (in select list) now applied to JOIN result (and not to left table). Example from Denny Crane. New behavior is the correct one. Will try to add best-effort compatibility setting. @@ -59,8 +73,6 @@ SELECT SETTINGS allow_experimental_analyzer = 1 FORMAT PrettyCompact -Query id: 2a5e39a3-3b64-49fd-bad3-0e351931ac99 - ┌─x─┬─plus(x, 1)─┐ 1. │ 2 │ 3 │ └───┴────────────┘ From edbef6ac9f0e6ae3e94a4732a453de0dcabb3a93 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 4 Jun 2024 14:22:21 +0200 Subject: [PATCH 023/105] Add 'Known incompatibilities of JOIN clause' --- docs/en/operations/analyzer.md | 42 +++++++++++++++++++++++++++++++--- 1 file changed, 39 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index 62bb45eeb7d..c559924314b 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -47,9 +47,45 @@ The previous query analyzer fixed this query by moving the `number > 5` filter f ### Known incompatibilities of JOIN clause -* Using expression from `SELECT` list in `JOIN` key as an expression from LEFT table. Example. Fix (best effort, should be under compatibility flag). -* Similar issue ^. Alias for column (in select list) now applied to JOIN result (and not to left table). Example from Denny Crane. New behavior is the correct one. Will try to add best-effort compatibility setting. -* Columns names are changed for some queries. This might breaks some scripts. Example. +#### Join using column from projection + +Alias from the `SELECT` list can not be used as a `JOIN USING` key by default. + +A new setting, `analyzer_compatibility_join_using_top_level_identifier`, when enabled, alters the behavior of `JOIN USING` to prefer to resolve identifiers based on expressions from the projection list of the SELECT query, rather than using the columns from left table directly. + +*Example:* + +```sql +SELECT a + 1 AS b, t2.s +FROM Values('a UInt64, b UInt64', (1, 1)) AS t1 +JOIN Values('b UInt64, s String', (1, 'one'), (2, 'two')) t2 +USING (b); +``` + +With `analyzer_compatibility_join_using_top_level_identifier` set to `true`, the join condition is interpreted as `t1.a + 1 = t2.b`, matching the behavior of earlier versions. So, the result will be `2, 'two'` +When the setting is `false`, the join condition defaults to `t1.b = t2.b`, and the query will return `2, 'one'`. +In case then `b` is not present in `t1`, the query will fail with an error. + +#### Changes in Behavior with `JOIN USING` and `ALIAS/MATERIALIZED` Columns + +In the new analyzer, using `*` in a `JOIN USING` query that involves `ALIAS` or `MATERIALIZED` columns will include that columns in the result set by default. + +*Example:* + +```sql +CREATE TABLE t1 (id UInt64, payload ALIAS sipHash64(id)) ENGINE = MergeTree ORDER BY id; +INSERT INTO t1 VALUES (1), (2); + +CREATE TABLE t2 (id UInt64, payload ALIAS sipHash64(id)) ENGINE = MergeTree ORDER BY id; +INSERT INTO t2 VALUES (2), (3); + +SELECT * FROM t1 +FULL JOIN t2 USING (payload); +``` + +In the new analyzer, the result of this query will include the `payload` column along with `id` from both tables. In contrast, the previous analyzer would only include these `ALIAS` columns if specific settings (`asterisk_include_alias_columns` or `asterisk_include_materialized_columns`) were enabled, and the columns might appear in a different order. + +To ensure consistent and expected results, especially when migrating old queries to the new analyzer, it is advisable to specify columns explicitly in the `SELECT` clause rather than using `*`. ### Projection column names changes From e7541224751137e193941aa06f71a80c44bfd1c3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 4 Jun 2024 14:41:30 +0200 Subject: [PATCH 024/105] Handling of Type Modifiers for columns in `USING` Clause --- docs/en/operations/analyzer.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index c559924314b..f5b900d01f7 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -87,6 +87,23 @@ In the new analyzer, the result of this query will include the `payload` column To ensure consistent and expected results, especially when migrating old queries to the new analyzer, it is advisable to specify columns explicitly in the `SELECT` clause rather than using `*`. +#### Handling of Type Modifiers for columns in `USING` Clause + +In the new version of the analyzer, the rules for determining the common supertype for columns specified in the `USING` clause have been standardized to produce more predictable outcomes, especially when dealing with type modifiers like `LowCardinality` and `Nullable`. + +- `LowCardinality(T)` and `T`: When a column of type `LowCardinality(T)` is joined with a column of type `T`, the resulting common supertype will be `T`, effectively discarding the `LowCardinality` modifier. + +- `Nullable(T)` and `T`: When a column of type `Nullable(T)` is joined with a column of type `T`, the resulting common supertype will be `Nullable(T)`, ensuring that the nullable property is preserved. + +*Example:* + +```sql +SELECT id, toTypeName(id) FROM Values('id LowCardinality(String)', ('a')) AS t1 +FULL OUTER JOIN Values('id String', ('b')) AS t2 +USING (id); +``` + +In this query, the common supertype for `id` is determined as `String`, discarding the `LowCardinality` modifier from `t1`. ### Projection column names changes From 086fc73f42f7d29d2082b66b24ab769230f47528 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 13:52:10 +0000 Subject: [PATCH 025/105] Remove aliases from fiter condition. --- src/Interpreters/ActionsDAG.cpp | 31 ++++++++++++++++++++++++- src/Interpreters/ActionsDAG.h | 2 ++ src/Processors/QueryPlan/FilterStep.cpp | 2 ++ 3 files changed, 34 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index cfccc835d29..cb88aad5a10 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -596,6 +596,34 @@ void ActionsDAG::removeUnusedActions(const std::unordered_set & us std::erase_if(inputs, [&](const Node * node) { return !visited_nodes.contains(node); }); } + +void ActionsDAG::removeAliasesForFilter(const std::string & filter_name) +{ + const auto & filter_node = findInOutputs(filter_name); + std::stack stack; + stack.push(const_cast(&filter_node)); + + std::unordered_set visited; + visited.insert(stack.top()); + + while (!stack.empty()) + { + auto * node = stack.top(); + stack.pop(); + for (auto & child : node->children) + { + while (child->type == ActionType::ALIAS) + child = child->children.front(); + + if (!visited.contains(child)) + { + stack.push(const_cast(child)); + visited.insert(child); + } + } + } +} + ActionsDAGPtr ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases) { auto actions = std::make_shared(); @@ -1830,7 +1858,8 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split input_node.result_name = child->result_name; child_data.to_second = &second_nodes.emplace_back(std::move(input_node)); - new_inputs.push_back(child); + if (child->type != ActionType::INPUT) + new_inputs.push_back(child); } } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 8c0e3f0e576..97549088cdf 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -195,6 +195,8 @@ public: /// Remove actions that are not needed to compute output nodes with required names void removeUnusedActions(const NameSet & required_names, bool allow_remove_inputs = true, bool allow_constant_folding = true); + void removeAliasesForFilter(const std::string & filter_name); + /// Transform the current DAG in a way that leaf nodes get folded into their parents. It's done /// because each projection can provide some columns as inputs to substitute certain sub-DAGs /// (expressions). Consider the following example: diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 56b31b2c8ba..72934665b5c 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -50,6 +50,8 @@ FilterStep::FilterStep( , filter_column_name(std::move(filter_column_name_)) , remove_filter_column(remove_filter_column_) { + actions_dag = actions_dag->clone(); + actions_dag->removeAliasesForFilter(filter_column_name); } void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) From 830acd3a8927e49c94a4d82817a1364c4dfd0604 Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Tue, 4 Jun 2024 15:32:38 +0000 Subject: [PATCH 026/105] Support non-const scale arguments in rounding functions --- .../functions/rounding-functions.md | 8 +- .../functions/rounding-functions.md | 2 +- src/Functions/FunctionsRound.h | 251 +- .../03165_round_scale_as_column.reference | 2165 +++++++++++++++++ .../03165_round_scale_as_column.sql | 125 + 5 files changed, 2475 insertions(+), 76 deletions(-) create mode 100644 tests/queries/0_stateless/03165_round_scale_as_column.reference create mode 100644 tests/queries/0_stateless/03165_round_scale_as_column.sql diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index d18185c5013..55a4b3ce9df 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -9,8 +9,8 @@ sidebar_label: Rounding ## floor(x\[, N\]) Returns the largest round number that is less than or equal to `x`. A round number is a multiple of 1/10N, or the nearest number of the appropriate data type if 1 / 10N isn’t exact. -‘N’ is an integer constant, optional parameter. By default it is zero, which means to round to an integer. -‘N’ may be negative. +`N` is an integer, optional parameter. By default it is zero, which means to round to an integer. +`N` may be negative. Examples: `floor(123.45, 1) = 123.4, floor(123.45, -1) = 120.` @@ -70,7 +70,7 @@ round(expression [, decimal_places]) **Arguments** - `expression` — A number to be rounded. Can be any [expression](../../sql-reference/syntax.md#syntax-expressions) returning the numeric [data type](../data-types/index.md#data_types). -- `decimal-places` — An integer value. +- `decimal-places` — The number of decimal places to round to. - If `decimal-places > 0` then the function rounds the value to the right of the decimal point. - If `decimal-places < 0` then the function rounds the value to the left of the decimal point. - If `decimal-places = 0` then the function rounds the value to integer. In this case the argument can be omitted. @@ -264,7 +264,7 @@ Result: ## roundDuration -Accepts a number. If the number is less than one, it returns `0`. Otherwise, it rounds the number down to numbers from the set of commonly used durations: `1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000`. +Accepts a number. If the number is less than one, it returns `0`. Otherwise, it rounds the number down to numbers from the set of commonly used durations: `1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000`. **Syntax** diff --git a/docs/ru/sql-reference/functions/rounding-functions.md b/docs/ru/sql-reference/functions/rounding-functions.md index cc939f69afc..f1dd57505ea 100644 --- a/docs/ru/sql-reference/functions/rounding-functions.md +++ b/docs/ru/sql-reference/functions/rounding-functions.md @@ -10,7 +10,7 @@ sidebar_label: "Функции округления" Возвращает наибольшее круглое число, которое меньше или равно, чем x. Круглым называется число, кратное 1 / 10N или ближайшее к нему число соответствующего типа данных, если 1 / 10N не представимо точно. -N - целочисленная константа, не обязательный параметр. По умолчанию - ноль, что означает - округлять до целого числа. +N - целочисленный аргумент, не обязательный параметр. По умолчанию - ноль, что означает - округлять до целого числа. N может быть отрицательным. Примеры: `floor(123.45, 1) = 123.4, floor(123.45, -1) = 120.` diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 1f20fbff24e..978b6d88d05 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -187,13 +187,13 @@ struct IntegerRoundingComputation }; -#ifdef __SSE4_1__ - -template +template class BaseFloatRoundingComputation; +#ifdef __SSE4_1__ + template <> -class BaseFloatRoundingComputation +class BaseFloatRoundingComputation { public: using ScalarType = Float32; @@ -214,7 +214,7 @@ public: }; template <> -class BaseFloatRoundingComputation +class BaseFloatRoundingComputation { public: using ScalarType = Float64; @@ -234,9 +234,9 @@ public: } }; -#else +#endif -/// Implementation for ARM. Not vectorized. +/// Implementation for ARM/columnar scale argument. Not vectorized. inline float roundWithMode(float x, RoundingMode mode) { @@ -265,7 +265,7 @@ inline double roundWithMode(double x, RoundingMode mode) } template -class BaseFloatRoundingComputation +class BaseFloatRoundingComputation { public: using ScalarType = T; @@ -285,15 +285,13 @@ public: } }; -#endif - /** Implementation of low-level round-off functions for floating-point values. */ -template -class FloatRoundingComputation : public BaseFloatRoundingComputation +template +class FloatRoundingComputation : public BaseFloatRoundingComputation { - using Base = BaseFloatRoundingComputation; + using Base = BaseFloatRoundingComputation; public: static void compute(const T * __restrict in, const typename Base::VectorType & scale, T * __restrict out) @@ -325,15 +323,22 @@ struct FloatRoundingImpl private: static_assert(!is_decimal); - using Op = FloatRoundingComputation; - using Data = std::array; + template + using Op = FloatRoundingComputation; + using Data = std::array::data_count>; using ColumnType = ColumnVector; using Container = typename ColumnType::Container; public: static NO_INLINE void apply(const Container & in, size_t scale, Container & out) { - auto mm_scale = Op::prepare(scale); + auto mm_scale = Op<>::prepare(scale); const size_t data_count = std::tuple_size(); @@ -345,7 +350,7 @@ public: while (p_in < limit) { - Op::compute(p_in, mm_scale, p_out); + Op<>::compute(p_in, mm_scale, p_out); p_in += data_count; p_out += data_count; } @@ -358,10 +363,17 @@ public: size_t tail_size_bytes = (end_in - p_in) * sizeof(*p_in); memcpy(&tmp_src, p_in, tail_size_bytes); - Op::compute(reinterpret_cast(&tmp_src), mm_scale, reinterpret_cast(&tmp_dst)); + Op<>::compute(reinterpret_cast(&tmp_src), mm_scale, reinterpret_cast(&tmp_dst)); memcpy(p_out, &tmp_dst, tail_size_bytes); } } + + static void applyOne(const T* __restrict in, size_t scale, T* __restrict out) + { + using ScalarOp = Op; + auto s = ScalarOp::prepare(scale); + ScalarOp::compute(in, s, out); + } }; template @@ -417,6 +429,11 @@ public: throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected 'scale' parameter passed to function"); } } + + static void applyOne(const T* __restrict in, size_t scale, T* __restrict out) + { + Op::compute(in, scale, out); + } }; @@ -452,11 +469,40 @@ public: memcpy(out.data(), in.data(), in.size() * sizeof(T)); } } -}; + static void applyOne(const NativeType* __restrict in, UInt32 in_scale, NativeType* __restrict out, Scale scale_arg) + { + scale_arg = in_scale - scale_arg; + if (scale_arg > 0) + { + auto scale = intExp10OfSize(scale_arg); + Op::compute(in, scale, out); + } + else + { + memcpy(out, in, sizeof(T)); + } + } +}; /** Select the appropriate processing algorithm depending on the scale. */ +inline void validateScale(Int64 scale64) +{ + if (scale64 > std::numeric_limits::max() || scale64 < std::numeric_limits::min()) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale argument for rounding function is too large"); +} + +inline Scale getScaleArg(const ColumnConst* scale_col) +{ + const auto& scale_field = scale_col->getField(); + + Int64 scale64 = scale_field.get(); + validateScale(scale64); + + return scale64; +} + template struct Dispatcher { @@ -465,9 +511,10 @@ struct Dispatcher FloatRoundingImpl, IntegerRoundingImpl>; - static ColumnPtr apply(const IColumn * col_general, Scale scale_arg) + template + static ColumnPtr apply(const IColumn * data_col, const IColumn * scale_col = nullptr) { - const auto & col = checkAndGetColumn>(*col_general); + const auto & col = checkAndGetColumn>(*data_col); auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_res = col_res->getData(); @@ -475,20 +522,62 @@ struct Dispatcher if (!vec_res.empty()) { - if (scale_arg == 0) + using ColVecScale = ColumnVector; + if (scale_col == nullptr || isColumnConst(*scale_col)) { - size_t scale = 1; - FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst(scale_col)); + if (scale_arg == 0) + { + size_t scale = 1; + FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + } + else if (scale_arg > 0) + { + size_t scale = intExp10(scale_arg); + FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + } + else + { + size_t scale = intExp10(-scale_arg); + FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + } } - else if (scale_arg > 0) + else if (const auto scale_typed = checkAndGetColumn(scale_col)) { - size_t scale = intExp10(scale_arg); - FunctionRoundingImpl::apply(col.getData(), scale, vec_res); - } - else - { - size_t scale = intExp10(-scale_arg); - FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + const auto & in = col.getData(); + const auto & scale_data = scale_typed->getData(); + const size_t count = in.size(); + + const T * end_in = in.data() + count; + const T * __restrict p_in = in.data(); + const ScaleType * __restrict p_scale = scale_data.data(); + T * __restrict p_out = vec_res.data(); + + while (p_in < end_in) + { + Int64 scale64 = *p_scale; + validateScale(scale64); + Scale raw_scale = scale64; + + if (raw_scale == 0) + { + size_t scale = 1; + FunctionRoundingImpl::applyOne(p_in, scale, p_out); + } + else if (raw_scale > 0) + { + size_t scale = intExp10(raw_scale); + FunctionRoundingImpl::applyOne(p_in, scale, p_out); + } + else + { + size_t scale = intExp10(-raw_scale); + FunctionRoundingImpl::applyOne(p_in, scale, p_out); + } + ++p_in; + ++p_scale; + ++p_out; + } } } @@ -500,16 +589,46 @@ template { public: - static ColumnPtr apply(const IColumn * col_general, Scale scale_arg) + template + static ColumnPtr apply(const IColumn * data_col, const IColumn* scale_col = nullptr) { - const auto & col = checkAndGetColumn>(*col_general); + const auto & col = checkAndGetColumn>(*data_col); const typename ColumnDecimal::Container & vec_src = col.getData(); auto col_res = ColumnDecimal::create(vec_src.size(), col.getScale()); auto & vec_res = col_res->getData(); if (!vec_res.empty()) - DecimalRoundingImpl::apply(col.getData(), col.getScale(), vec_res, scale_arg); + { + using ColVecScale = ColumnVector; + if (scale_col == nullptr || isColumnConst(*scale_col)) + { + auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst(scale_col)); + DecimalRoundingImpl::apply(col.getData(), col.getScale(), vec_res, scale_arg); + } + else if (const auto scale_typed = checkAndGetColumn(scale_col)) + { + const auto & scale = scale_typed->getData(); + const size_t count = vec_src.size(); + + using NativeType = typename T::NativeType; + const NativeType * __restrict p_in = reinterpret_cast(vec_src.data()); + const ScaleType * __restrict p_scale = scale.data(); + const NativeType * end_in = p_in + count; + NativeType * __restrict p_out = reinterpret_cast(vec_res.data()); + while (p_in < end_in) + { + Int64 scale64 = *p_scale; + validateScale(scale64); + Scale raw_scale = scale64; + + DecimalRoundingImpl::applyOne(p_in, col.getScale(), p_out, raw_scale); + ++p_in; + ++p_scale; + ++p_out; + } + } + } return col_res; } @@ -550,50 +669,41 @@ public: return arguments[0]; } - static Scale getScaleArg(const ColumnsWithTypeAndName & arguments) - { - if (arguments.size() == 2) - { - const IColumn & scale_column = *arguments[1].column; - if (!isColumnConst(scale_column)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must be constant"); - - Field scale_field = assert_cast(scale_column).getField(); - if (scale_field.getType() != Field::Types::UInt64 - && scale_field.getType() != Field::Types::Int64) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must have integer type"); - - Int64 scale64 = scale_field.get(); - if (scale64 > std::numeric_limits::max() - || scale64 < std::numeric_limits::min()) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Scale argument for rounding function is too large"); - - return scale64; - } - return 0; - } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { - const ColumnWithTypeAndName & column = arguments[0]; - Scale scale_arg = getScaleArg(arguments); + const ColumnWithTypeAndName & data_column = arguments[0]; ColumnPtr res; - auto call = [&](const auto & types) -> bool + auto callData = [&](const auto & types) -> bool { using Types = std::decay_t; - using DataType = typename Types::LeftType; + using DataType = typename Types::RightType; - if constexpr (IsDataTypeNumber || IsDataTypeDecimal) + if (arguments.size() > 1) { - using FieldType = typename DataType::FieldType; - res = Dispatcher::apply(column.column.get(), scale_arg); + const ColumnWithTypeAndName & scale_column = arguments[1]; + + auto callScale = [&](const auto & scaleTypes) -> bool + { + using ScaleTypes = std::decay_t; + using ScaleType = typename ScaleTypes::RightType; + + if (isColumnConst(*data_column.column) && !isColumnConst(*scale_column.column)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale column must be const for const data column"); + + res = Dispatcher::template apply(data_column.column.get(), scale_column.column.get()); + return true; + }; + + TypeIndex right_index = scale_column.type->getTypeId(); + if (!callOnBasicType(right_index, callScale)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must have integer type"); return true; } - return false; + res = Dispatcher::template apply(data_column.column.get()); + return true; }; #if !defined(__SSE4_1__) @@ -605,10 +715,9 @@ public: throw Exception(ErrorCodes::CANNOT_SET_ROUNDING_MODE, "Cannot set floating point rounding mode"); #endif - if (!callOnIndexAndDataType(column.type->getTypeId(), call)) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", column.name, getName()); - } + TypeIndex left_index = data_column.type->getTypeId(); + if (!callOnBasicType(left_index, callData)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", data_column.name, getName()); return res; } diff --git a/tests/queries/0_stateless/03165_round_scale_as_column.reference b/tests/queries/0_stateless/03165_round_scale_as_column.reference new file mode 100644 index 00000000000..9ad25ed466a --- /dev/null +++ b/tests/queries/0_stateless/03165_round_scale_as_column.reference @@ -0,0 +1,2165 @@ +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +10 10 10 10 10 10 +11 11 11 11 11 11 +12 12 12 12 12 12 +13 13 13 13 13 13 +14 14 14 14 14 14 +15 15 15 15 15 15 +16 16 16 16 16 16 +17 17 17 17 17 17 +18 18 18 18 18 18 +19 19 19 19 19 19 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +10 10 10 10 10 10 +11 11 11 11 11 11 +12 12 12 12 12 12 +13 13 13 13 13 13 +14 14 14 14 14 14 +15 15 15 15 15 15 +16 16 16 16 16 16 +17 17 17 17 17 17 +18 18 18 18 18 18 +19 19 19 19 19 19 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +10 10 10 10 10 10 +11 11 11 11 11 11 +12 12 12 12 12 12 +13 13 13 13 13 13 +14 14 14 14 14 14 +15 15 15 15 15 15 +16 16 16 16 16 16 +17 17 17 17 17 17 +18 18 18 18 18 18 +19 19 19 19 19 19 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +10 10 10 10 10 10 +11 11 11 11 11 11 +12 12 12 12 12 12 +13 13 13 13 13 13 +14 14 14 14 14 14 +15 15 15 15 15 15 +16 16 16 16 16 16 +17 17 17 17 17 17 +18 18 18 18 18 18 +19 19 19 19 19 19 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-10 -10 -10 -10 -10 -10 +-9 -9 -9 -9 -9 -9 +-8 -8 -8 -8 -8 -8 +-7 -7 -7 -7 -7 -7 +-6 -6 -6 -6 -6 -6 +-5 -5 -5 -5 -5 -5 +-4 -4 -4 -4 -4 -4 +-3 -3 -3 -3 -3 -3 +-2 -2 -2 -2 -2 -2 +-1 -1 -1 -1 -1 -1 +0 0 0 0 0 0 +1 1 1 1 1 1 +2 2 2 2 2 2 +3 3 3 3 3 3 +4 4 4 4 4 4 +5 5 5 5 5 5 +6 6 6 6 6 6 +7 7 7 7 7 7 +8 8 8 8 8 8 +9 9 9 9 9 9 +-1 -1 -1 -1 -1 -1 +-0.9 -1 -1 -1 -0 -0 +-0.8 -1 -1 -1 -0 -0 +-0.7 -1 -1 -1 -0 -0 +-0.6 -1 -1 -1 -0 -0 +-0.5 -0 -0 -1 -0 -0 +-0.4 -0 -0 -1 -0 -0 +-0.3 -0 -0 -1 -0 -0 +-0.2 -0 -0 -1 -0 -0 +-0.1 -0 -0 -1 -0 -0 +0 0 0 0 0 0 +0.1 0 0 0 1 0 +0.2 0 0 0 1 0 +0.3 0 0 0 1 0 +0.4 0 0 0 1 0 +0.5 0 0 0 1 0 +0.6 1 1 0 1 0 +0.7 1 1 0 1 0 +0.8 1 1 0 1 0 +0.9 1 1 0 1 0 +-1 -1 -1 -1 -1 -1 +-0.9 -1 -1 -1 -0 -0 +-0.8 -1 -1 -1 -0 -0 +-0.7 -1 -1 -1 -0 -0 +-0.6 -1 -1 -1 -0 -0 +-0.5 -0 -0 -1 -0 -0 +-0.4 -0 -0 -1 -0 -0 +-0.3 -0 -0 -1 -0 -0 +-0.2 -0 -0 -1 -0 -0 +-0.1 -0 -0 -1 -0 -0 +0 0 0 0 0 0 +0.1 0 0 0 1 0 +0.2 0 0 0 1 0 +0.3 0 0 0 1 0 +0.4 0 0 0 1 0 +0.5 0 0 0 1 0 +0.6 1 1 0 1 0 +0.7 1 1 0 1 0 +0.8 1 1 0 1 0 +0.9 1 1 0 1 0 +-1 -1 -1 -1 -1 -1 +-0.9 -0.9 -0.9 -0.9 -0.9 -0.9 +-0.8 -0.8 -0.8 -0.8 -0.8 -0.8 +-0.7 -0.7 -0.7 -0.7 -0.7 -0.7 +-0.6 -0.6 -0.6 -0.6 -0.6 -0.6 +-0.5 -0.5 -0.5 -0.5 -0.5 -0.5 +-0.4 -0.4 -0.4 -0.4 -0.4 -0.4 +-0.3 -0.3 -0.3 -0.3 -0.3 -0.3 +-0.2 -0.2 -0.2 -0.2 -0.2 -0.2 +-0.1 -0.1 -0.1 -0.1 -0.1 -0.1 +0 0 0 0 0 0 +0.1 0.1 0.1 0.1 0.1 0.1 +0.2 0.2 0.2 0.2 0.2 0.2 +0.3 0.3 0.3 0.3 0.3 0.3 +0.4 0.4 0.4 0.4 0.4 0.4 +0.5 0.5 0.5 0.5 0.5 0.5 +0.6 0.6 0.6 0.6 0.6 0.6 +0.7 0.7 0.7 0.7 0.7 0.7 +0.8 0.8 0.8 0.8 0.8 0.8 +0.9 0.9 0.9 0.9 0.9 0.9 +-1 -1 -1 -1 -1 -1 +-0.9 -0.9 -0.9 -0.9 -0.9 -0.9 +-0.8 -0.8 -0.8 -0.8 -0.8 -0.8 +-0.7 -0.7 -0.7 -0.7 -0.7 -0.7 +-0.6 -0.6 -0.6 -0.6 -0.6 -0.6 +-0.5 -0.5 -0.5 -0.5 -0.5 -0.5 +-0.4 -0.4 -0.4 -0.4 -0.4 -0.4 +-0.3 -0.3 -0.3 -0.3 -0.3 -0.3 +-0.2 -0.2 -0.2 -0.2 -0.2 -0.2 +-0.1 -0.1 -0.1 -0.1 -0.1 -0.1 +0 0 0 0 0 0 +0.1 0.1 0.1 0.1 0.1 0.1 +0.2 0.2 0.2 0.2 0.2 0.2 +0.3 0.3 0.3 0.3 0.3 0.3 +0.4 0.4 0.4 0.4 0.4 0.4 +0.5 0.5 0.5 0.5 0.5 0.5 +0.6 0.6 0.6 0.6 0.6 0.6 +0.7 0.7 0.7 0.7 0.7 0.7 +0.8 0.8 0.8 0.8 0.8 0.8 +0.9 0.9 0.9 0.9 0.9 0.9 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +10 10 10 10 10 10 +11 10 10 10 20 10 +12 10 10 10 20 10 +13 10 10 10 20 10 +14 10 10 10 20 10 +15 20 20 10 20 10 +16 20 20 10 20 10 +17 20 20 10 20 10 +18 20 20 10 20 10 +19 20 20 10 20 10 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +10 10 10 10 10 10 +11 10 10 10 20 10 +12 10 10 10 20 10 +13 10 10 10 20 10 +14 10 10 10 20 10 +15 20 20 10 20 10 +16 20 20 10 20 10 +17 20 20 10 20 10 +18 20 20 10 20 10 +19 20 20 10 20 10 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +10 10 10 10 10 10 +11 10 10 10 20 10 +12 10 10 10 20 10 +13 10 10 10 20 10 +14 10 10 10 20 10 +15 20 20 10 20 10 +16 20 20 10 20 10 +17 20 20 10 20 10 +18 20 20 10 20 10 +19 20 20 10 20 10 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +10 10 10 10 10 10 +11 10 10 10 20 10 +12 10 10 10 20 10 +13 10 10 10 20 10 +14 10 10 10 20 10 +15 20 20 10 20 10 +16 20 20 10 20 10 +17 20 20 10 20 10 +18 20 20 10 20 10 +19 20 20 10 20 10 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 0 0 +-8 -10 -10 -10 0 0 +-7 -10 -10 -10 0 0 +-6 -10 -10 -10 0 0 +-5 -10 0 -10 0 0 +-4 0 0 -10 0 0 +-3 0 0 -10 0 0 +-2 0 0 -10 0 0 +-1 0 0 -10 0 0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 0 0 +-8 -10 -10 -10 0 0 +-7 -10 -10 -10 0 0 +-6 -10 -10 -10 0 0 +-5 -10 0 -10 0 0 +-4 0 0 -10 0 0 +-3 0 0 -10 0 0 +-2 0 0 -10 0 0 +-1 0 0 -10 0 0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 0 0 +-8 -10 -10 -10 0 0 +-7 -10 -10 -10 0 0 +-6 -10 -10 -10 0 0 +-5 -10 0 -10 0 0 +-4 0 0 -10 0 0 +-3 0 0 -10 0 0 +-2 0 0 -10 0 0 +-1 0 0 -10 0 0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 0 0 +-8 -10 -10 -10 0 0 +-7 -10 -10 -10 0 0 +-6 -10 -10 -10 0 0 +-5 -10 0 -10 0 0 +-4 0 0 -10 0 0 +-3 0 0 -10 0 0 +-2 0 0 -10 0 0 +-1 0 0 -10 0 0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 10 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 -0 -0 +-8 -10 -10 -10 -0 -0 +-7 -10 -10 -10 -0 -0 +-6 -10 -10 -10 -0 -0 +-5 -0 -0 -10 -0 -0 +-4 -0 -0 -10 -0 -0 +-3 -0 -0 -10 -0 -0 +-2 -0 -0 -10 -0 -0 +-1 -0 -0 -10 -0 -0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 0 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +-10 -10 -10 -10 -10 -10 +-9 -10 -10 -10 -0 -0 +-8 -10 -10 -10 -0 -0 +-7 -10 -10 -10 -0 -0 +-6 -10 -10 -10 -0 -0 +-5 -0 -0 -10 -0 -0 +-4 -0 -0 -10 -0 -0 +-3 -0 -0 -10 -0 -0 +-2 -0 -0 -10 -0 -0 +-1 -0 -0 -10 -0 -0 +0 0 0 0 0 0 +1 0 0 0 10 0 +2 0 0 0 10 0 +3 0 0 0 10 0 +4 0 0 0 10 0 +5 0 0 0 10 0 +6 10 10 0 10 0 +7 10 10 0 10 0 +8 10 10 0 10 0 +9 10 10 0 10 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +10 0 0 0 100 0 +11 0 0 0 100 0 +12 0 0 0 100 0 +13 0 0 0 100 0 +14 0 0 0 100 0 +15 0 0 0 100 0 +16 0 0 0 100 0 +17 0 0 0 100 0 +18 0 0 0 100 0 +19 0 0 0 100 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +10 0 0 0 100 0 +11 0 0 0 100 0 +12 0 0 0 100 0 +13 0 0 0 100 0 +14 0 0 0 100 0 +15 0 0 0 100 0 +16 0 0 0 100 0 +17 0 0 0 100 0 +18 0 0 0 100 0 +19 0 0 0 100 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +10 0 0 0 100 0 +11 0 0 0 100 0 +12 0 0 0 100 0 +13 0 0 0 100 0 +14 0 0 0 100 0 +15 0 0 0 100 0 +16 0 0 0 100 0 +17 0 0 0 100 0 +18 0 0 0 100 0 +19 0 0 0 100 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +10 0 0 0 100 0 +11 0 0 0 100 0 +12 0 0 0 100 0 +13 0 0 0 100 0 +14 0 0 0 100 0 +15 0 0 0 100 0 +16 0 0 0 100 0 +17 0 0 0 100 0 +18 0 0 0 100 0 +19 0 0 0 100 0 +-10 0 0 -100 0 0 +-9 0 0 -100 0 0 +-8 0 0 -100 0 0 +-7 0 0 -100 0 0 +-6 0 0 -100 0 0 +-5 0 0 -100 0 0 +-4 0 0 -100 0 0 +-3 0 0 -100 0 0 +-2 0 0 -100 0 0 +-1 0 0 -100 0 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 0 0 -100 0 0 +-9 0 0 -100 0 0 +-8 0 0 -100 0 0 +-7 0 0 -100 0 0 +-6 0 0 -100 0 0 +-5 0 0 -100 0 0 +-4 0 0 -100 0 0 +-3 0 0 -100 0 0 +-2 0 0 -100 0 0 +-1 0 0 -100 0 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 0 0 -100 0 0 +-9 0 0 -100 0 0 +-8 0 0 -100 0 0 +-7 0 0 -100 0 0 +-6 0 0 -100 0 0 +-5 0 0 -100 0 0 +-4 0 0 -100 0 0 +-3 0 0 -100 0 0 +-2 0 0 -100 0 0 +-1 0 0 -100 0 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 0 0 -100 0 0 +-9 0 0 -100 0 0 +-8 0 0 -100 0 0 +-7 0 0 -100 0 0 +-6 0 0 -100 0 0 +-5 0 0 -100 0 0 +-4 0 0 -100 0 0 +-3 0 0 -100 0 0 +-2 0 0 -100 0 0 +-1 0 0 -100 0 0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 -0 -0 -100 -0 -0 +-9 -0 -0 -100 -0 -0 +-8 -0 -0 -100 -0 -0 +-7 -0 -0 -100 -0 -0 +-6 -0 -0 -100 -0 -0 +-5 -0 -0 -100 -0 -0 +-4 -0 -0 -100 -0 -0 +-3 -0 -0 -100 -0 -0 +-2 -0 -0 -100 -0 -0 +-1 -0 -0 -100 -0 -0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +-10 -0 -0 -100 -0 -0 +-9 -0 -0 -100 -0 -0 +-8 -0 -0 -100 -0 -0 +-7 -0 -0 -100 -0 -0 +-6 -0 -0 -100 -0 -0 +-5 -0 -0 -100 -0 -0 +-4 -0 -0 -100 -0 -0 +-3 -0 -0 -100 -0 -0 +-2 -0 -0 -100 -0 -0 +-1 -0 -0 -100 -0 -0 +0 0 0 0 0 0 +1 0 0 0 100 0 +2 0 0 0 100 0 +3 0 0 0 100 0 +4 0 0 0 100 0 +5 0 0 0 100 0 +6 0 0 0 100 0 +7 0 0 0 100 0 +8 0 0 0 100 0 +9 0 0 0 100 0 +CHECKPOINT1 +id u8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 10 0 10 10 10 10 10 +21 11 0 11 11 11 11 11 +22 12 0 12 12 12 12 12 +23 13 0 13 13 13 13 13 +24 14 0 14 14 14 14 14 +25 15 0 15 15 15 15 15 +26 16 0 16 16 16 16 16 +27 17 0 17 17 17 17 17 +28 18 0 18 18 18 18 18 +29 19 0 19 19 19 19 19 +30 20 0 20 20 20 20 20 +31 21 0 21 21 21 21 21 +32 22 0 22 22 22 22 22 +33 23 0 23 23 23 23 23 +34 24 0 24 24 24 24 24 +35 25 0 25 25 25 25 25 +36 26 0 26 26 26 26 26 +37 27 0 27 27 27 27 27 +38 28 0 28 28 28 28 28 +39 29 0 29 29 29 29 29 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 10 -1 10 10 10 10 10 +61 11 -1 10 10 10 20 10 +62 12 -1 10 10 10 20 10 +63 13 -1 10 10 10 20 10 +64 14 -1 10 10 10 20 10 +65 15 -1 20 20 10 20 10 +66 16 -1 20 20 10 20 10 +67 17 -1 20 20 10 20 10 +68 18 -1 20 20 10 20 10 +69 19 -1 20 20 10 20 10 +70 20 -1 20 20 20 20 20 +71 21 -1 20 20 20 30 20 +72 22 -1 20 20 20 30 20 +73 23 -1 20 20 20 30 20 +74 24 -1 20 20 20 30 20 +75 25 -1 30 20 20 30 20 +76 26 -1 30 30 20 30 20 +77 27 -1 30 30 20 30 20 +78 28 -1 30 30 20 30 20 +79 29 -1 30 30 20 30 20 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 10 -2 0 0 0 100 0 +101 11 -2 0 0 0 100 0 +102 12 -2 0 0 0 100 0 +103 13 -2 0 0 0 100 0 +104 14 -2 0 0 0 100 0 +105 15 -2 0 0 0 100 0 +106 16 -2 0 0 0 100 0 +107 17 -2 0 0 0 100 0 +108 18 -2 0 0 0 100 0 +109 19 -2 0 0 0 100 0 +110 20 -2 0 0 0 100 0 +111 21 -2 0 0 0 100 0 +112 22 -2 0 0 0 100 0 +113 23 -2 0 0 0 100 0 +114 24 -2 0 0 0 100 0 +115 25 -2 0 0 0 100 0 +116 26 -2 0 0 0 100 0 +117 27 -2 0 0 0 100 0 +118 28 -2 0 0 0 100 0 +119 29 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 244 4 244 244 244 244 244 +id u16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 10 0 10 10 10 10 10 +21 11 0 11 11 11 11 11 +22 12 0 12 12 12 12 12 +23 13 0 13 13 13 13 13 +24 14 0 14 14 14 14 14 +25 15 0 15 15 15 15 15 +26 16 0 16 16 16 16 16 +27 17 0 17 17 17 17 17 +28 18 0 18 18 18 18 18 +29 19 0 19 19 19 19 19 +30 20 0 20 20 20 20 20 +31 21 0 21 21 21 21 21 +32 22 0 22 22 22 22 22 +33 23 0 23 23 23 23 23 +34 24 0 24 24 24 24 24 +35 25 0 25 25 25 25 25 +36 26 0 26 26 26 26 26 +37 27 0 27 27 27 27 27 +38 28 0 28 28 28 28 28 +39 29 0 29 29 29 29 29 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 10 -1 10 10 10 10 10 +61 11 -1 10 10 10 20 10 +62 12 -1 10 10 10 20 10 +63 13 -1 10 10 10 20 10 +64 14 -1 10 10 10 20 10 +65 15 -1 20 20 10 20 10 +66 16 -1 20 20 10 20 10 +67 17 -1 20 20 10 20 10 +68 18 -1 20 20 10 20 10 +69 19 -1 20 20 10 20 10 +70 20 -1 20 20 20 20 20 +71 21 -1 20 20 20 30 20 +72 22 -1 20 20 20 30 20 +73 23 -1 20 20 20 30 20 +74 24 -1 20 20 20 30 20 +75 25 -1 30 20 20 30 20 +76 26 -1 30 30 20 30 20 +77 27 -1 30 30 20 30 20 +78 28 -1 30 30 20 30 20 +79 29 -1 30 30 20 30 20 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 10 -2 0 0 0 100 0 +101 11 -2 0 0 0 100 0 +102 12 -2 0 0 0 100 0 +103 13 -2 0 0 0 100 0 +104 14 -2 0 0 0 100 0 +105 15 -2 0 0 0 100 0 +106 16 -2 0 0 0 100 0 +107 17 -2 0 0 0 100 0 +108 18 -2 0 0 0 100 0 +109 19 -2 0 0 0 100 0 +110 20 -2 0 0 0 100 0 +111 21 -2 0 0 0 100 0 +112 22 -2 0 0 0 100 0 +113 23 -2 0 0 0 100 0 +114 24 -2 0 0 0 100 0 +115 25 -2 0 0 0 100 0 +116 26 -2 0 0 0 100 0 +117 27 -2 0 0 0 100 0 +118 28 -2 0 0 0 100 0 +119 29 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id u32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 10 0 10 10 10 10 10 +21 11 0 11 11 11 11 11 +22 12 0 12 12 12 12 12 +23 13 0 13 13 13 13 13 +24 14 0 14 14 14 14 14 +25 15 0 15 15 15 15 15 +26 16 0 16 16 16 16 16 +27 17 0 17 17 17 17 17 +28 18 0 18 18 18 18 18 +29 19 0 19 19 19 19 19 +30 20 0 20 20 20 20 20 +31 21 0 21 21 21 21 21 +32 22 0 22 22 22 22 22 +33 23 0 23 23 23 23 23 +34 24 0 24 24 24 24 24 +35 25 0 25 25 25 25 25 +36 26 0 26 26 26 26 26 +37 27 0 27 27 27 27 27 +38 28 0 28 28 28 28 28 +39 29 0 29 29 29 29 29 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 10 -1 10 10 10 10 10 +61 11 -1 10 10 10 20 10 +62 12 -1 10 10 10 20 10 +63 13 -1 10 10 10 20 10 +64 14 -1 10 10 10 20 10 +65 15 -1 20 20 10 20 10 +66 16 -1 20 20 10 20 10 +67 17 -1 20 20 10 20 10 +68 18 -1 20 20 10 20 10 +69 19 -1 20 20 10 20 10 +70 20 -1 20 20 20 20 20 +71 21 -1 20 20 20 30 20 +72 22 -1 20 20 20 30 20 +73 23 -1 20 20 20 30 20 +74 24 -1 20 20 20 30 20 +75 25 -1 30 20 20 30 20 +76 26 -1 30 30 20 30 20 +77 27 -1 30 30 20 30 20 +78 28 -1 30 30 20 30 20 +79 29 -1 30 30 20 30 20 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 10 -2 0 0 0 100 0 +101 11 -2 0 0 0 100 0 +102 12 -2 0 0 0 100 0 +103 13 -2 0 0 0 100 0 +104 14 -2 0 0 0 100 0 +105 15 -2 0 0 0 100 0 +106 16 -2 0 0 0 100 0 +107 17 -2 0 0 0 100 0 +108 18 -2 0 0 0 100 0 +109 19 -2 0 0 0 100 0 +110 20 -2 0 0 0 100 0 +111 21 -2 0 0 0 100 0 +112 22 -2 0 0 0 100 0 +113 23 -2 0 0 0 100 0 +114 24 -2 0 0 0 100 0 +115 25 -2 0 0 0 100 0 +116 26 -2 0 0 0 100 0 +117 27 -2 0 0 0 100 0 +118 28 -2 0 0 0 100 0 +119 29 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id u64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 10 0 10 10 10 10 10 +21 11 0 11 11 11 11 11 +22 12 0 12 12 12 12 12 +23 13 0 13 13 13 13 13 +24 14 0 14 14 14 14 14 +25 15 0 15 15 15 15 15 +26 16 0 16 16 16 16 16 +27 17 0 17 17 17 17 17 +28 18 0 18 18 18 18 18 +29 19 0 19 19 19 19 19 +30 20 0 20 20 20 20 20 +31 21 0 21 21 21 21 21 +32 22 0 22 22 22 22 22 +33 23 0 23 23 23 23 23 +34 24 0 24 24 24 24 24 +35 25 0 25 25 25 25 25 +36 26 0 26 26 26 26 26 +37 27 0 27 27 27 27 27 +38 28 0 28 28 28 28 28 +39 29 0 29 29 29 29 29 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 10 -1 10 10 10 10 10 +61 11 -1 10 10 10 20 10 +62 12 -1 10 10 10 20 10 +63 13 -1 10 10 10 20 10 +64 14 -1 10 10 10 20 10 +65 15 -1 20 20 10 20 10 +66 16 -1 20 20 10 20 10 +67 17 -1 20 20 10 20 10 +68 18 -1 20 20 10 20 10 +69 19 -1 20 20 10 20 10 +70 20 -1 20 20 20 20 20 +71 21 -1 20 20 20 30 20 +72 22 -1 20 20 20 30 20 +73 23 -1 20 20 20 30 20 +74 24 -1 20 20 20 30 20 +75 25 -1 30 20 20 30 20 +76 26 -1 30 30 20 30 20 +77 27 -1 30 30 20 30 20 +78 28 -1 30 30 20 30 20 +79 29 -1 30 30 20 30 20 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 10 -2 0 0 0 100 0 +101 11 -2 0 0 0 100 0 +102 12 -2 0 0 0 100 0 +103 13 -2 0 0 0 100 0 +104 14 -2 0 0 0 100 0 +105 15 -2 0 0 0 100 0 +106 16 -2 0 0 0 100 0 +107 17 -2 0 0 0 100 0 +108 18 -2 0 0 0 100 0 +109 19 -2 0 0 0 100 0 +110 20 -2 0 0 0 100 0 +111 21 -2 0 0 0 100 0 +112 22 -2 0 0 0 100 0 +113 23 -2 0 0 0 100 0 +114 24 -2 0 0 0 100 0 +115 25 -2 0 0 0 100 0 +116 26 -2 0 0 0 100 0 +117 27 -2 0 0 0 100 0 +118 28 -2 0 0 0 100 0 +119 29 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id i8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -10 0 -10 -10 -10 -10 -10 +21 -9 0 -9 -9 -9 -9 -9 +22 -8 0 -8 -8 -8 -8 -8 +23 -7 0 -7 -7 -7 -7 -7 +24 -6 0 -6 -6 -6 -6 -6 +25 -5 0 -5 -5 -5 -5 -5 +26 -4 0 -4 -4 -4 -4 -4 +27 -3 0 -3 -3 -3 -3 -3 +28 -2 0 -2 -2 -2 -2 -2 +29 -1 0 -1 -1 -1 -1 -1 +30 0 0 0 0 0 0 0 +31 1 0 1 1 1 1 1 +32 2 0 2 2 2 2 2 +33 3 0 3 3 3 3 3 +34 4 0 4 4 4 4 4 +35 5 0 5 5 5 5 5 +36 6 0 6 6 6 6 6 +37 7 0 7 7 7 7 7 +38 8 0 8 8 8 8 8 +39 9 0 9 9 9 9 9 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -10 -1 -10 -10 -10 -10 -10 +61 -9 -1 -10 -10 -10 0 0 +62 -8 -1 -10 -10 -10 0 0 +63 -7 -1 -10 -10 -10 0 0 +64 -6 -1 -10 -10 -10 0 0 +65 -5 -1 -10 0 -10 0 0 +66 -4 -1 0 0 -10 0 0 +67 -3 -1 0 0 -10 0 0 +68 -2 -1 0 0 -10 0 0 +69 -1 -1 0 0 -10 0 0 +70 0 -1 0 0 0 0 0 +71 1 -1 0 0 0 10 0 +72 2 -1 0 0 0 10 0 +73 3 -1 0 0 0 10 0 +74 4 -1 0 0 0 10 0 +75 5 -1 10 0 0 10 0 +76 6 -1 10 10 0 10 0 +77 7 -1 10 10 0 10 0 +78 8 -1 10 10 0 10 0 +79 9 -1 10 10 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -10 -2 0 0 -100 0 0 +101 -9 -2 0 0 -100 0 0 +102 -8 -2 0 0 -100 0 0 +103 -7 -2 0 0 -100 0 0 +104 -6 -2 0 0 -100 0 0 +105 -5 -2 0 0 -100 0 0 +106 -4 -2 0 0 -100 0 0 +107 -3 -2 0 0 -100 0 0 +108 -2 -2 0 0 -100 0 0 +109 -1 -2 0 0 -100 0 0 +110 0 -2 0 0 0 0 0 +111 1 -2 0 0 0 100 0 +112 2 -2 0 0 0 100 0 +113 3 -2 0 0 0 100 0 +114 4 -2 0 0 0 100 0 +115 5 -2 0 0 0 100 0 +116 6 -2 0 0 0 100 0 +117 7 -2 0 0 0 100 0 +118 8 -2 0 0 0 100 0 +119 9 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 -56 4 -56 -56 -56 -56 -56 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 -12 4 -12 -12 -12 -12 -12 +id i16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -10 0 -10 -10 -10 -10 -10 +21 -9 0 -9 -9 -9 -9 -9 +22 -8 0 -8 -8 -8 -8 -8 +23 -7 0 -7 -7 -7 -7 -7 +24 -6 0 -6 -6 -6 -6 -6 +25 -5 0 -5 -5 -5 -5 -5 +26 -4 0 -4 -4 -4 -4 -4 +27 -3 0 -3 -3 -3 -3 -3 +28 -2 0 -2 -2 -2 -2 -2 +29 -1 0 -1 -1 -1 -1 -1 +30 0 0 0 0 0 0 0 +31 1 0 1 1 1 1 1 +32 2 0 2 2 2 2 2 +33 3 0 3 3 3 3 3 +34 4 0 4 4 4 4 4 +35 5 0 5 5 5 5 5 +36 6 0 6 6 6 6 6 +37 7 0 7 7 7 7 7 +38 8 0 8 8 8 8 8 +39 9 0 9 9 9 9 9 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -10 -1 -10 -10 -10 -10 -10 +61 -9 -1 -10 -10 -10 0 0 +62 -8 -1 -10 -10 -10 0 0 +63 -7 -1 -10 -10 -10 0 0 +64 -6 -1 -10 -10 -10 0 0 +65 -5 -1 -10 0 -10 0 0 +66 -4 -1 0 0 -10 0 0 +67 -3 -1 0 0 -10 0 0 +68 -2 -1 0 0 -10 0 0 +69 -1 -1 0 0 -10 0 0 +70 0 -1 0 0 0 0 0 +71 1 -1 0 0 0 10 0 +72 2 -1 0 0 0 10 0 +73 3 -1 0 0 0 10 0 +74 4 -1 0 0 0 10 0 +75 5 -1 10 0 0 10 0 +76 6 -1 10 10 0 10 0 +77 7 -1 10 10 0 10 0 +78 8 -1 10 10 0 10 0 +79 9 -1 10 10 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -10 -2 0 0 -100 0 0 +101 -9 -2 0 0 -100 0 0 +102 -8 -2 0 0 -100 0 0 +103 -7 -2 0 0 -100 0 0 +104 -6 -2 0 0 -100 0 0 +105 -5 -2 0 0 -100 0 0 +106 -4 -2 0 0 -100 0 0 +107 -3 -2 0 0 -100 0 0 +108 -2 -2 0 0 -100 0 0 +109 -1 -2 0 0 -100 0 0 +110 0 -2 0 0 0 0 0 +111 1 -2 0 0 0 100 0 +112 2 -2 0 0 0 100 0 +113 3 -2 0 0 0 100 0 +114 4 -2 0 0 0 100 0 +115 5 -2 0 0 0 100 0 +116 6 -2 0 0 0 100 0 +117 7 -2 0 0 0 100 0 +118 8 -2 0 0 0 100 0 +119 9 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id i32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -10 0 -10 -10 -10 -10 -10 +21 -9 0 -9 -9 -9 -9 -9 +22 -8 0 -8 -8 -8 -8 -8 +23 -7 0 -7 -7 -7 -7 -7 +24 -6 0 -6 -6 -6 -6 -6 +25 -5 0 -5 -5 -5 -5 -5 +26 -4 0 -4 -4 -4 -4 -4 +27 -3 0 -3 -3 -3 -3 -3 +28 -2 0 -2 -2 -2 -2 -2 +29 -1 0 -1 -1 -1 -1 -1 +30 0 0 0 0 0 0 0 +31 1 0 1 1 1 1 1 +32 2 0 2 2 2 2 2 +33 3 0 3 3 3 3 3 +34 4 0 4 4 4 4 4 +35 5 0 5 5 5 5 5 +36 6 0 6 6 6 6 6 +37 7 0 7 7 7 7 7 +38 8 0 8 8 8 8 8 +39 9 0 9 9 9 9 9 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -10 -1 -10 -10 -10 -10 -10 +61 -9 -1 -10 -10 -10 0 0 +62 -8 -1 -10 -10 -10 0 0 +63 -7 -1 -10 -10 -10 0 0 +64 -6 -1 -10 -10 -10 0 0 +65 -5 -1 -10 0 -10 0 0 +66 -4 -1 0 0 -10 0 0 +67 -3 -1 0 0 -10 0 0 +68 -2 -1 0 0 -10 0 0 +69 -1 -1 0 0 -10 0 0 +70 0 -1 0 0 0 0 0 +71 1 -1 0 0 0 10 0 +72 2 -1 0 0 0 10 0 +73 3 -1 0 0 0 10 0 +74 4 -1 0 0 0 10 0 +75 5 -1 10 0 0 10 0 +76 6 -1 10 10 0 10 0 +77 7 -1 10 10 0 10 0 +78 8 -1 10 10 0 10 0 +79 9 -1 10 10 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -10 -2 0 0 -100 0 0 +101 -9 -2 0 0 -100 0 0 +102 -8 -2 0 0 -100 0 0 +103 -7 -2 0 0 -100 0 0 +104 -6 -2 0 0 -100 0 0 +105 -5 -2 0 0 -100 0 0 +106 -4 -2 0 0 -100 0 0 +107 -3 -2 0 0 -100 0 0 +108 -2 -2 0 0 -100 0 0 +109 -1 -2 0 0 -100 0 0 +110 0 -2 0 0 0 0 0 +111 1 -2 0 0 0 100 0 +112 2 -2 0 0 0 100 0 +113 3 -2 0 0 0 100 0 +114 4 -2 0 0 0 100 0 +115 5 -2 0 0 0 100 0 +116 6 -2 0 0 0 100 0 +117 7 -2 0 0 0 100 0 +118 8 -2 0 0 0 100 0 +119 9 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id i64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -10 0 -10 -10 -10 -10 -10 +21 -9 0 -9 -9 -9 -9 -9 +22 -8 0 -8 -8 -8 -8 -8 +23 -7 0 -7 -7 -7 -7 -7 +24 -6 0 -6 -6 -6 -6 -6 +25 -5 0 -5 -5 -5 -5 -5 +26 -4 0 -4 -4 -4 -4 -4 +27 -3 0 -3 -3 -3 -3 -3 +28 -2 0 -2 -2 -2 -2 -2 +29 -1 0 -1 -1 -1 -1 -1 +30 0 0 0 0 0 0 0 +31 1 0 1 1 1 1 1 +32 2 0 2 2 2 2 2 +33 3 0 3 3 3 3 3 +34 4 0 4 4 4 4 4 +35 5 0 5 5 5 5 5 +36 6 0 6 6 6 6 6 +37 7 0 7 7 7 7 7 +38 8 0 8 8 8 8 8 +39 9 0 9 9 9 9 9 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 10 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -10 -1 -10 -10 -10 -10 -10 +61 -9 -1 -10 -10 -10 0 0 +62 -8 -1 -10 -10 -10 0 0 +63 -7 -1 -10 -10 -10 0 0 +64 -6 -1 -10 -10 -10 0 0 +65 -5 -1 -10 0 -10 0 0 +66 -4 -1 0 0 -10 0 0 +67 -3 -1 0 0 -10 0 0 +68 -2 -1 0 0 -10 0 0 +69 -1 -1 0 0 -10 0 0 +70 0 -1 0 0 0 0 0 +71 1 -1 0 0 0 10 0 +72 2 -1 0 0 0 10 0 +73 3 -1 0 0 0 10 0 +74 4 -1 0 0 0 10 0 +75 5 -1 10 0 0 10 0 +76 6 -1 10 10 0 10 0 +77 7 -1 10 10 0 10 0 +78 8 -1 10 10 0 10 0 +79 9 -1 10 10 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -10 -2 0 0 -100 0 0 +101 -9 -2 0 0 -100 0 0 +102 -8 -2 0 0 -100 0 0 +103 -7 -2 0 0 -100 0 0 +104 -6 -2 0 0 -100 0 0 +105 -5 -2 0 0 -100 0 0 +106 -4 -2 0 0 -100 0 0 +107 -3 -2 0 0 -100 0 0 +108 -2 -2 0 0 -100 0 0 +109 -1 -2 0 0 -100 0 0 +110 0 -2 0 0 0 0 0 +111 1 -2 0 0 0 100 0 +112 2 -2 0 0 0 100 0 +113 3 -2 0 0 0 100 0 +114 4 -2 0 0 0 100 0 +115 5 -2 0 0 0 100 0 +116 6 -2 0 0 0 100 0 +117 7 -2 0 0 0 100 0 +118 8 -2 0 0 0 100 0 +119 9 -2 0 0 0 100 0 +200 0 0 0 0 0 0 0 +201 0 -1 0 0 0 0 0 +202 0 -2 0 0 0 0 0 +203 0 -3 0 0 0 0 0 +204 0 -4 0 0 0 0 0 +205 0 -5 0 0 0 0 0 +206 0 -6 0 0 0 0 0 +207 0 -7 0 0 0 0 0 +208 0 -8 0 0 0 0 0 +209 0 -9 0 0 0 0 0 +210 0 0 0 0 0 0 0 +211 0 -1 0 0 0 0 0 +212 0 -2 0 0 0 0 0 +213 0 -3 0 0 0 0 0 +214 0 -4 0 0 0 0 0 +215 0 -5 0 0 0 0 0 +216 0 -6 0 0 0 0 0 +217 0 -7 0 0 0 0 0 +218 0 -8 0 0 0 0 0 +219 0 -9 0 0 0 0 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id f32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -1 0 -1 -1 -1 -1 -1 +21 -0.9 0 -1 -1 -1 -0 -0 +22 -0.8 0 -1 -1 -1 -0 -0 +23 -0.7 0 -1 -1 -1 -0 -0 +24 -0.6 0 -1 -1 -1 -0 -0 +25 -0.5 0 -0 -0 -1 -0 -0 +26 -0.4 0 -0 -0 -1 -0 -0 +27 -0.3 0 -0 -0 -1 -0 -0 +28 -0.2 0 -0 -0 -1 -0 -0 +29 -0.1 0 -0 -0 -1 -0 -0 +30 0 0 0 0 0 0 0 +31 0.1 0 0 0 0 1 0 +32 0.2 0 0 0 0 1 0 +33 0.3 0 0 0 0 1 0 +34 0.4 0 0 0 0 1 0 +35 0.5 0 0 0 0 1 0 +36 0.6 0 1 1 0 1 0 +37 0.7 0 1 1 0 1 0 +38 0.8 0 1 1 0 1 0 +39 0.9 0 1 1 0 1 0 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 0 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -1 -1 -0 -0 -10 -0 -0 +61 -0.9 -1 -0 -0 -10 -0 -0 +62 -0.8 -1 -0 -0 -10 -0 -0 +63 -0.7 -1 -0 -0 -10 -0 -0 +64 -0.6 -1 -0 -0 -10 -0 -0 +65 -0.5 -1 -0 -0 -10 -0 -0 +66 -0.4 -1 -0 -0 -10 -0 -0 +67 -0.3 -1 -0 -0 -10 -0 -0 +68 -0.2 -1 -0 -0 -10 -0 -0 +69 -0.1 -1 -0 -0 -10 -0 -0 +70 0 -1 0 0 0 0 0 +71 0.1 -1 0 0 0 10 0 +72 0.2 -1 0 0 0 10 0 +73 0.3 -1 0 0 0 10 0 +74 0.4 -1 0 0 0 10 0 +75 0.5 -1 0 0 0 10 0 +76 0.6 -1 0 0 0 10 0 +77 0.7 -1 0 0 0 10 0 +78 0.8 -1 0 0 0 10 0 +79 0.9 -1 0 0 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -1 -2 -0 -0 -100 -0 -0 +101 -0.9 -2 -0 -0 -100 -0 -0 +102 -0.8 -2 -0 -0 -100 -0 -0 +103 -0.7 -2 -0 -0 -100 -0 -0 +104 -0.6 -2 -0 -0 -100 -0 -0 +105 -0.5 -2 -0 -0 -100 -0 -0 +106 -0.4 -2 -0 -0 -100 -0 -0 +107 -0.3 -2 -0 -0 -100 -0 -0 +108 -0.2 -2 -0 -0 -100 -0 -0 +109 -0.1 -2 -0 -0 -100 -0 -0 +110 0 -2 0 0 0 0 0 +111 0.1 -2 0 0 0 100 0 +112 0.2 -2 0 0 0 100 0 +113 0.3 -2 0 0 0 100 0 +114 0.4 -2 0 0 0 100 0 +115 0.5 -2 0 0 0 100 0 +116 0.6 -2 0 0 0 100 0 +117 0.7 -2 0 0 0 100 0 +118 0.8 -2 0 0 0 100 0 +119 0.9 -2 0 0 0 100 0 +200 12345.679 0 12346 12346 12345 12346 12345 +201 12345.679 -1 12350 12350 12340 12350 12340 +202 12345.679 -2 12300 12300 12300 12400 12300 +203 12345.679 -3 12000 12000 12000 13000 12000 +204 12345.679 -4 10000 10000 10000 20000 10000 +205 12345.679 -5 0 0 0 100000 0 +206 12345.679 -6 0 0 0 1000000 0 +207 12345.679 -7 0 0 0 10000000 0 +208 12345.679 -8 0 0 0 100000000 0 +209 12345.679 -9 0 0 0 1000000000 0 +210 12345.679 0 12346 12346 12345 12346 12345 +211 12345.679 -1 12350 12350 12340 12350 12340 +212 12345.679 -2 12300 12300 12300 12400 12300 +213 12345.679 -3 12000 12000 12000 13000 12000 +214 12345.679 -4 10000 10000 10000 20000 10000 +215 12345.679 -5 0 0 0 100000 0 +216 12345.679 -6 0 0 0 1000000 0 +217 12345.679 -7 0 0 0 10000000 0 +218 12345.679 -8 0 0 0 100000000 0 +219 12345.679 -9 0 0 0 1000000000 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +id f64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale) +0 0 0 0 0 0 0 0 +1 1 0 1 1 1 1 1 +2 2 0 2 2 2 2 2 +3 3 0 3 3 3 3 3 +4 4 0 4 4 4 4 4 +5 5 0 5 5 5 5 5 +6 6 0 6 6 6 6 6 +7 7 0 7 7 7 7 7 +8 8 0 8 8 8 8 8 +9 9 0 9 9 9 9 9 +10 10 0 10 10 10 10 10 +11 11 0 11 11 11 11 11 +12 12 0 12 12 12 12 12 +13 13 0 13 13 13 13 13 +14 14 0 14 14 14 14 14 +15 15 0 15 15 15 15 15 +16 16 0 16 16 16 16 16 +17 17 0 17 17 17 17 17 +18 18 0 18 18 18 18 18 +19 19 0 19 19 19 19 19 +20 -1 0 -1 -1 -1 -1 -1 +21 -0.9 0 -1 -1 -1 -0 -0 +22 -0.8 0 -1 -1 -1 -0 -0 +23 -0.7 0 -1 -1 -1 -0 -0 +24 -0.6 0 -1 -1 -1 -0 -0 +25 -0.5 0 -0 -0 -1 -0 -0 +26 -0.4 0 -0 -0 -1 -0 -0 +27 -0.3 0 -0 -0 -1 -0 -0 +28 -0.2 0 -0 -0 -1 -0 -0 +29 -0.1 0 -0 -0 -1 -0 -0 +30 0 0 0 0 0 0 0 +31 0.1 0 0 0 0 1 0 +32 0.2 0 0 0 0 1 0 +33 0.3 0 0 0 0 1 0 +34 0.4 0 0 0 0 1 0 +35 0.5 0 0 0 0 1 0 +36 0.6 0 1 1 0 1 0 +37 0.7 0 1 1 0 1 0 +38 0.8 0 1 1 0 1 0 +39 0.9 0 1 1 0 1 0 +40 0 -1 0 0 0 0 0 +41 1 -1 0 0 0 10 0 +42 2 -1 0 0 0 10 0 +43 3 -1 0 0 0 10 0 +44 4 -1 0 0 0 10 0 +45 5 -1 0 0 0 10 0 +46 6 -1 10 10 0 10 0 +47 7 -1 10 10 0 10 0 +48 8 -1 10 10 0 10 0 +49 9 -1 10 10 0 10 0 +50 10 -1 10 10 10 10 10 +51 11 -1 10 10 10 20 10 +52 12 -1 10 10 10 20 10 +53 13 -1 10 10 10 20 10 +54 14 -1 10 10 10 20 10 +55 15 -1 20 20 10 20 10 +56 16 -1 20 20 10 20 10 +57 17 -1 20 20 10 20 10 +58 18 -1 20 20 10 20 10 +59 19 -1 20 20 10 20 10 +60 -1 -1 -0 -0 -10 -0 -0 +61 -0.9 -1 -0 -0 -10 -0 -0 +62 -0.8 -1 -0 -0 -10 -0 -0 +63 -0.7 -1 -0 -0 -10 -0 -0 +64 -0.6 -1 -0 -0 -10 -0 -0 +65 -0.5 -1 -0 -0 -10 -0 -0 +66 -0.4 -1 -0 -0 -10 -0 -0 +67 -0.3 -1 -0 -0 -10 -0 -0 +68 -0.2 -1 -0 -0 -10 -0 -0 +69 -0.1 -1 -0 -0 -10 -0 -0 +70 0 -1 0 0 0 0 0 +71 0.1 -1 0 0 0 10 0 +72 0.2 -1 0 0 0 10 0 +73 0.3 -1 0 0 0 10 0 +74 0.4 -1 0 0 0 10 0 +75 0.5 -1 0 0 0 10 0 +76 0.6 -1 0 0 0 10 0 +77 0.7 -1 0 0 0 10 0 +78 0.8 -1 0 0 0 10 0 +79 0.9 -1 0 0 0 10 0 +80 0 -2 0 0 0 0 0 +81 1 -2 0 0 0 100 0 +82 2 -2 0 0 0 100 0 +83 3 -2 0 0 0 100 0 +84 4 -2 0 0 0 100 0 +85 5 -2 0 0 0 100 0 +86 6 -2 0 0 0 100 0 +87 7 -2 0 0 0 100 0 +88 8 -2 0 0 0 100 0 +89 9 -2 0 0 0 100 0 +90 10 -2 0 0 0 100 0 +91 11 -2 0 0 0 100 0 +92 12 -2 0 0 0 100 0 +93 13 -2 0 0 0 100 0 +94 14 -2 0 0 0 100 0 +95 15 -2 0 0 0 100 0 +96 16 -2 0 0 0 100 0 +97 17 -2 0 0 0 100 0 +98 18 -2 0 0 0 100 0 +99 19 -2 0 0 0 100 0 +100 -1 -2 -0 -0 -100 -0 -0 +101 -0.9 -2 -0 -0 -100 -0 -0 +102 -0.8 -2 -0 -0 -100 -0 -0 +103 -0.7 -2 -0 -0 -100 -0 -0 +104 -0.6 -2 -0 -0 -100 -0 -0 +105 -0.5 -2 -0 -0 -100 -0 -0 +106 -0.4 -2 -0 -0 -100 -0 -0 +107 -0.3 -2 -0 -0 -100 -0 -0 +108 -0.2 -2 -0 -0 -100 -0 -0 +109 -0.1 -2 -0 -0 -100 -0 -0 +110 0 -2 0 0 0 0 0 +111 0.1 -2 0 0 0 100 0 +112 0.2 -2 0 0 0 100 0 +113 0.3 -2 0 0 0 100 0 +114 0.4 -2 0 0 0 100 0 +115 0.5 -2 0 0 0 100 0 +116 0.6 -2 0 0 0 100 0 +117 0.7 -2 0 0 0 100 0 +118 0.8 -2 0 0 0 100 0 +119 0.9 -2 0 0 0 100 0 +200 12345.6789 0 12346 12346 12345 12346 12345 +201 12345.6789 -1 12350 12350 12340 12350 12340 +202 12345.6789 -2 12300 12300 12300 12400 12300 +203 12345.6789 -3 12000 12000 12000 13000 12000 +204 12345.6789 -4 10000 10000 10000 20000 10000 +205 12345.6789 -5 0 0 0 100000 0 +206 12345.6789 -6 0 0 0 1000000 0 +207 12345.6789 -7 0 0 0 10000000 0 +208 12345.6789 -8 0 0 0 100000000 0 +209 12345.6789 -9 0 0 0 1000000000 0 +210 12345.6789 0 12346 12346 12345 12346 12345 +211 12345.6789 -1 12350 12350 12340 12350 12340 +212 12345.6789 -2 12300 12300 12300 12400 12300 +213 12345.6789 -3 12000 12000 12000 13000 12000 +214 12345.6789 -4 10000 10000 10000 20000 10000 +215 12345.6789 -5 0 0 0 100000 0 +216 12345.6789 -6 0 0 0 1000000 0 +217 12345.6789 -7 0 0 0 10000000 0 +218 12345.6789 -8 0 0 0 100000000 0 +219 12345.6789 -9 0 0 0 1000000000 0 +300 2 4 2 2 2 2 2 +301 20 4 20 20 20 20 20 +302 200 4 200 200 200 200 200 +303 5 4 5 5 5 5 5 +304 50 4 50 50 50 50 50 +305 500 4 500 500 500 500 500 +CHECKPOINT2 +1 42.42 42.42 42.42 42.42 +2 0.0084 0.0084 0.0084 0.0084 +3 6.513 6.513 6.513 6.513 +4 115.6011 115.6011 115.6011 115.6011 +5 1.6029 1.6029 1.6029 1.6029 +6 -0.9999 -0.9999 -0.9999 -0.9999 +7 0.0084 0.0084 0.0084 0.0084 +8 3.7476 3.7476 3.7476 3.7476 +9 5.4066 5.4066 5.4066 5.4066 +10 1.6275 1.6275 1.6275 1.6275 +1 +1 +1 diff --git a/tests/queries/0_stateless/03165_round_scale_as_column.sql b/tests/queries/0_stateless/03165_round_scale_as_column.sql new file mode 100644 index 00000000000..60940b22fc1 --- /dev/null +++ b/tests/queries/0_stateless/03165_round_scale_as_column.sql @@ -0,0 +1,125 @@ +-- Regression test that functions round(), roundBankers(), floor(), ceil() and trunc() work with default 'scale' argument (= the 2nd argument) +SELECT toUInt8(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toUInt16(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toUInt32(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toUInt64(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toInt8(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toInt16(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toInt32(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toInt64(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toFloat32(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toFloat64(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; + +SELECT toFloat32((number - 10) / 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; +SELECT toFloat64((number - 10) / 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; + +-- Test that functions round(), roundBankers(), floor(), ceil() and trunc() accept non-const 'scale' argument +SELECT toFloat32((number - 10) / 10) AS x, round(x, materialize(1)), roundBankers(x, materialize(1)), floor(x, materialize(1)), ceil(x, materialize(1)), trunc(x, materialize(1)) FROM system.numbers LIMIT 20; +SELECT toFloat64((number - 10) / 10) AS x, round(x, materialize(1)), roundBankers(x, materialize(1)), floor(x, materialize(1)), ceil(x, materialize(1)), trunc(x, materialize(1)) FROM system.numbers LIMIT 20; +SELECT toUInt8(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toUInt16(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toUInt32(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toUInt64(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; + +SELECT toInt8(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toInt16(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toInt32(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toInt64(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toFloat32(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; +SELECT toFloat64(number - 10) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; + +SELECT toUInt8(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toUInt16(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toUInt32(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toUInt64(number) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toInt8(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toInt16(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toInt32(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toInt64(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toFloat32(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; +SELECT toFloat64(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, materialize(-2)), floor(x, materialize(-2)), ceil(x, materialize(-2)), trunc(x, materialize(-2)) FROM system.numbers LIMIT 20; + +SELECT toString('CHECKPOINT1'); + +DROP TABLE IF EXISTS tround; + +CREATE TABLE tround ( + id Int32, + scale Int16, + u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, + i8 Int8, i16 Int16, i32 Int32, i64 Int64, + f32 Float32, f64 Float64 +) ENGINE = Memory; + +INSERT INTO tround SELECT number , 0, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tround SELECT number+20 , 0, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; +INSERT INTO tround SELECT number+40 , -1, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tround SELECT number+60 , -1, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; +INSERT INTO tround SELECT number+80 , -2, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tround SELECT number+100, -2, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; + +INSERT INTO tround SELECT number+200, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10; +INSERT INTO tround SELECT number+210, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10; + +INSERT INTO tround VALUES (300, 4, 2, 2, 2, 2, 2, 2, 2, 2, 2.0, 2.0); +INSERT INTO tround VALUES (301, 4, 20, 20, 20, 20, 20, 20, 20, 20, 20.0, 20.0); +INSERT INTO tround VALUES (302, 4, 200, 200, 200, 200, 200, 200, 200, 200, 200.0, 200.0); +INSERT INTO tround VALUES (303, 4, 5, 5, 5, 5, 5, 5, 5, 5, 5.0, 5.0); +INSERT INTO tround VALUES (304, 4, 50, 50, 50, 50, 50, 50, 50, 50, 50.0, 50.0); +INSERT INTO tround VALUES (305, 4, 500, 500, 500, 500, 500, 500, 500, 500, 500.0, 500.0); + +SELECT toString('id u8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, u8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id u16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, u16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id u32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, u32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id u64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, u64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id i8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, i8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id i16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, i16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id i32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, i32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id i64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, i64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id f32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, f32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT toString('id f64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); +SELECT id, f64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; + +DROP TABLE IF EXISTS tround; +-- +SELECT toString('CHECKPOINT2'); + +DROP TABLE IF EXISTS tround2; + +CREATE TABLE tround2 ( + id Int32, + scale Int16, + d32 Decimal32(4), d64 Decimal64(4), d128 Decimal128(4), d256 Decimal256(4) +) ENGINE = Memory; + +INSERT INTO tround2 VALUES (1, 6, toDecimal32('42.42', 4), toDecimal64('42.42', 4), toDecimal128('42.42', 4), toDecimal256('42.42', 4)); +INSERT INTO tround2 SELECT 2 , 6, cos(d32), cos(d64), cos(d128), cos(d256) FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 3 , 6, sqrt(d32), sqrt(d64), sqrt(d128), sqrt(d256) FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 4 , 6, lgamma(d32), lgamma(d64), lgamma(d128), lgamma(d256) FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 5 , 6, tgamma(d32)/1e50, tgamma(d64)/1e50, tgamma(d128)/1e50, tgamma(d256)/1e50 FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 6 , 8, sin(d32), sin(d64), sin(d128), sin(d256) FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 7 , 8, cos(d32), cos(d64), cos(d128), cos(d256) FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 8 , 8, log(d32), log(d64), log(d128), log(d256) FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 9 , 8, log2(d32), log2(d64), log2(d128), log2(d256) FROM tround2 WHERE id = 1; +INSERT INTO tround2 SELECT 10, 8, log10(d32), log10(d64), log10(d128), log10(d256) FROM tround2 WHERE id = 1; + +SELECT id, round(d32, scale), round(d64, scale), round(d128, scale), round(d256, scale) FROM tround2 ORDER BY id; + +DROP TABLE IF EXISTS tround2; + +SELECT round(1, 1); +SELECT round(materialize(1), materialize(1)); +SELECT round(1, materialize(1)); --{serverError ILLEGAL_COLUMN} +SELECT round(materialize(1), 1); + + + From ff84232033b7f0421f0516fd38a2bb69d025c8aa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 15:52:27 +0000 Subject: [PATCH 027/105] Fixing some tests. --- .../01655_plan_optimizations.reference | 8 -- ...ct_in_order_optimization_explain.reference | 2 +- ...dicate_push_down_filled_join_fix.reference | 8 +- ...filter_push_down_equivalent_sets.reference | 84 +++++++------- ...convert_outer_join_to_inner_join.reference | 104 ++++++++---------- .../03156_analyzer_array_join_distributed.sql | 18 +++ 6 files changed, 111 insertions(+), 113 deletions(-) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 1b9755a74d5..edc6412c932 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -44,12 +44,10 @@ Filter 9 10 1 > one condition of filter should be pushed down after aggregating, other condition is aliased Filter column -ALIAS notEquals(s, 4) :: 1 -> and(notEquals(y, 0), notEquals(s, 4)) Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased Filter column -ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8)) Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -63,12 +61,10 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > one condition of filter should be pushed down after aggregating, other condition is casted Filter column -FUNCTION and(minus(s, 4) :: 1, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) UInt8 : 2 Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted Filter column -FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2 Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -82,12 +78,10 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > one condition of filter should be pushed down after aggregating, other two conditions are ANDed Filter column -FUNCTION and(minus(s, 8) :: 1, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4)) Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed Filter column -FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8)) Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -100,12 +94,10 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased Filter column -ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4)) Aggregating Filter column: and(notEquals(y, 0), minus(y, 4)) > (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased Filter column -ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8)) Aggregating Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8)) 0 1 diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index 85e8a802bdc..9b9885478cd 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -87,7 +87,7 @@ Sorting (Stream): a ASC, b ASC Sorting (Stream): __table1.a ASC, __table1.b ASC Sorting (Stream): __table1.a ASC, __table1.b ASC Sorting (Stream): __table1.a ASC, __table1.b ASC -Sorting (Stream): __table1.a ASC, b ASC +Sorting (Stream): a ASC, b ASC -- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns Sorting (Stream): __table1.a ASC Sorting (Stream): __table1.a ASC diff --git a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference index e6c4d5768af..6e8325a2ff3 100644 --- a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference +++ b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference @@ -24,10 +24,10 @@ Positions: 3 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 0_UInt8 :: 2) -> equals(__table1.id, 0_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 0_UInt8 :: 2) -> equals(__table1.id, 0_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table) Header: id UInt64 value String diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference index 00740e6380f..80f4e309505 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference @@ -33,10 +33,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -50,10 +50,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table2.id UInt64 : 3 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String @@ -100,10 +100,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -117,10 +117,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table2.id UInt64 : 3 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String @@ -168,12 +168,12 @@ Positions: 4 2 0 1 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 2 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 3 - ALIAS id :: 0 -> __table1.id UInt64 : 4 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 4, 6_UInt8 :: 2) -> equals(__table1.id, 6_UInt8) UInt8 : 1 - FUNCTION equals(__table1.id : 4, 5_UInt8 :: 3) -> equals(__table1.id, 5_UInt8) UInt8 : 2 + ALIAS id : 0 -> __table1.id UInt64 : 4 + ALIAS value :: 1 -> __table1.value String : 5 + FUNCTION equals(id : 0, 6_UInt8 :: 2) -> equals(__table1.id, 6_UInt8) UInt8 : 1 + FUNCTION equals(id :: 0, 5_UInt8 :: 3) -> equals(__table1.id, 5_UInt8) UInt8 : 2 FUNCTION and(equals(__table1.id, 5_UInt8) :: 2, equals(__table1.id, 6_UInt8) :: 1) -> and(equals(__table1.id, 5_UInt8), equals(__table1.id, 6_UInt8)) UInt8 : 3 - Positions: 3 4 0 + Positions: 3 4 5 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -188,12 +188,12 @@ Positions: 4 2 0 1 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 3 - ALIAS id :: 0 -> __table2.id UInt64 : 4 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 4, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - FUNCTION equals(__table2.id : 4, 6_UInt8 :: 3) -> equals(__table2.id, 6_UInt8) UInt8 : 2 + ALIAS id : 0 -> __table2.id UInt64 : 4 + ALIAS value :: 1 -> __table2.value String : 5 + FUNCTION equals(id : 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + FUNCTION equals(id :: 0, 6_UInt8 :: 3) -> equals(__table2.id, 6_UInt8) UInt8 : 2 FUNCTION and(equals(__table2.id, 6_UInt8) :: 2, equals(__table2.id, 5_UInt8) :: 1) -> and(equals(__table2.id, 6_UInt8), equals(__table2.id, 5_UInt8)) UInt8 : 3 - Positions: 3 4 0 + Positions: 3 4 5 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String @@ -237,10 +237,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -254,10 +254,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table2.id UInt64 : 3 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String @@ -452,10 +452,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value :: 1 -> __table1.value String : 0 - FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table1.id UInt64 : 3 + ALIAS value :: 1 -> __table1.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_1) Header: id UInt64 value String @@ -469,10 +469,10 @@ Positions: 4 2 0 1 Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2 - ALIAS id :: 0 -> __table2.id UInt64 : 3 - ALIAS value :: 1 -> __table2.value String : 0 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 - Positions: 1 3 0 + ALIAS id : 0 -> __table2.id UInt64 : 3 + ALIAS value :: 1 -> __table2.value String : 4 + FUNCTION equals(id :: 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1 + Positions: 1 3 4 ReadFromMergeTree (default.test_table_2) Header: id UInt64 value String diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference index 6ca5b6ef572..d35bdeff98b 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference @@ -25,13 +25,12 @@ Positions: 4 0 2 1 Header: __table1.id UInt64 __table1.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table1.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table1.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table1.value String : 2 + ALIAS id :: 1 -> __table1.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_1) - Header: __table1.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -42,20 +41,18 @@ Positions: 4 0 2 1 Prewhere filter column: notEquals(__table1.id, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table1.id UInt64 : 2 - FUNCTION notEquals(__table1.id : 2, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3 - Positions: 2 0 3 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 2 + Positions: 0 2 Expression Header: __table2.id UInt64 __table2.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table2.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table2.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table2.value String : 2 + ALIAS id :: 1 -> __table2.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_2) - Header: __table2.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -66,9 +63,8 @@ Positions: 4 0 2 1 Prewhere filter column: notEquals(__table2.id, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table2.id UInt64 : 2 - FUNCTION notEquals(__table2.id : 2, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3 - Positions: 2 0 3 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 2 + Positions: 0 2 -- 2 Value_2 2 Value_2 -- @@ -99,13 +95,12 @@ Positions: 4 0 2 1 Header: __table1.id UInt64 __table1.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table1.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table1.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table1.value String : 2 + ALIAS id :: 1 -> __table1.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_1) - Header: __table1.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -116,20 +111,18 @@ Positions: 4 0 2 1 Prewhere filter column: notEquals(__table1.id, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table1.id UInt64 : 2 - FUNCTION notEquals(__table1.id : 2, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3 - Positions: 2 0 3 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 2 + Positions: 0 2 Expression Header: __table2.id UInt64 __table2.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table2.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table2.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table2.value String : 2 + ALIAS id :: 1 -> __table2.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_2) - Header: __table2.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -140,9 +133,8 @@ Positions: 4 0 2 1 Prewhere filter column: notEquals(__table2.id, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table2.id UInt64 : 2 - FUNCTION notEquals(__table2.id : 2, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3 - Positions: 2 0 3 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 2 + Positions: 0 2 -- 2 Value_2 2 Value_2 -- @@ -173,13 +165,12 @@ Positions: 4 0 2 1 Header: __table1.id UInt64 __table1.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table1.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table1.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table1.value String : 2 + ALIAS id :: 1 -> __table1.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_1) - Header: __table1.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -190,22 +181,20 @@ Positions: 4 0 2 1 Prewhere filter column: and(notEquals(__table1.id, 0_UInt8), notEquals(__table1.id, 0_UInt8)) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table1.id UInt64 : 2 - FUNCTION notEquals(__table1.id : 2, 0_UInt8 : 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3 - FUNCTION notEquals(__table1.id : 2, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 4 - FUNCTION and(notEquals(__table1.id, 0_UInt8) :: 4, notEquals(__table1.id, 0_UInt8) :: 3) -> and(notEquals(__table1.id, 0_UInt8), notEquals(__table1.id, 0_UInt8)) UInt8 : 1 - Positions: 2 0 1 + FUNCTION notEquals(id : 0, 0_UInt8 : 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 2 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table1.id, 0_UInt8) UInt8 : 3 + FUNCTION and(notEquals(__table1.id, 0_UInt8) :: 3, notEquals(__table1.id, 0_UInt8) :: 2) -> and(notEquals(__table1.id, 0_UInt8), notEquals(__table1.id, 0_UInt8)) UInt8 : 1 + Positions: 0 1 Expression Header: __table2.id UInt64 __table2.value String Actions: INPUT : 1 -> value String : 0 - INPUT :: 0 -> __table2.id UInt64 : 1 - INPUT :: 2 -> id UInt64 : 2 - ALIAS value :: 0 -> __table2.value String : 3 - Positions: 1 3 + INPUT : 0 -> id UInt64 : 1 + ALIAS value :: 0 -> __table2.value String : 2 + ALIAS id :: 1 -> __table2.id UInt64 : 0 + Positions: 0 2 ReadFromMergeTree (default.test_table_2) - Header: __table2.id UInt64 - id UInt64 + Header: id UInt64 value String ReadType: Default Parts: 1 @@ -216,10 +205,9 @@ Positions: 4 0 2 1 Prewhere filter column: and(notEquals(__table2.id, 0_UInt8), notEquals(__table2.id, 0_UInt8)) (removed) Actions: INPUT : 0 -> id UInt64 : 0 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 1 - ALIAS id : 0 -> __table2.id UInt64 : 2 - FUNCTION notEquals(__table2.id : 2, 0_UInt8 : 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3 - FUNCTION notEquals(__table2.id : 2, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 4 - FUNCTION and(notEquals(__table2.id, 0_UInt8) :: 4, notEquals(__table2.id, 0_UInt8) :: 3) -> and(notEquals(__table2.id, 0_UInt8), notEquals(__table2.id, 0_UInt8)) UInt8 : 1 - Positions: 2 0 1 + FUNCTION notEquals(id : 0, 0_UInt8 : 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 2 + FUNCTION notEquals(id : 0, 0_UInt8 :: 1) -> notEquals(__table2.id, 0_UInt8) UInt8 : 3 + FUNCTION and(notEquals(__table2.id, 0_UInt8) :: 3, notEquals(__table2.id, 0_UInt8) :: 2) -> and(notEquals(__table2.id, 0_UInt8), notEquals(__table2.id, 0_UInt8)) UInt8 : 1 + Positions: 0 1 -- 2 Value_2 2 Value_2 diff --git a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql index f605a369822..f4ebba0b1b2 100644 --- a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql +++ b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql @@ -8,3 +8,21 @@ SELECT s, arr, a FROM remote('127.0.0.{1,2}', currentDatabase(), arrays_test) AR SELECT s, arr FROM remote('127.0.0.2', currentDatabase(), arrays_test) ARRAY JOIN arr WHERE arr < 3 ORDER BY arr; SELECT s, arr FROM remote('127.0.0.{1,2}', currentDatabase(), arrays_test) ARRAY JOIN arr WHERE arr < 3 ORDER BY arr; + +create table hourly( + hour datetime, + `metric.names` Array(String), + `metric.values` Array(Int64) +) Engine=Memory +as select '2020-01-01', ['a', 'b'], [1,2]; + +SELECT + toDate(hour) AS day, + `metric.names`, + sum(`metric.values`) +FROM remote('127.0.0.{1,2}', currentDatabase(), hourly) +ARRAY JOIN metric +GROUP BY + day, + metric.names; +ORDER BY metric.names; From 2db0eddc0f8df5d5600f9ccb10fcc2c65ca0b114 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 16:11:45 +0000 Subject: [PATCH 028/105] Update tests. --- .../01655_plan_optimizations.reference | 8 ++++++++ .../0_stateless/01655_plan_optimizations.sh | 16 ++++++++-------- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index edc6412c932..edf93b4b39f 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -44,10 +44,12 @@ Filter 9 10 1 > one condition of filter should be pushed down after aggregating, other condition is aliased Filter column +ALIAS notEquals(s, 4) :: 4 -> and(notEquals(y, 0), notEquals(s, 4)) UInt8 : 2 Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased Filter column +ALIAS notEquals(__table1.s, 4_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8)) Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -61,10 +63,12 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > one condition of filter should be pushed down after aggregating, other condition is casted Filter column +FUNCTION and(minus(s, 4) :: 5, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted Filter column +FUNCTION and(minus(__table1.s, 4_UInt8) :: 1, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -78,10 +82,12 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > one condition of filter should be pushed down after aggregating, other two conditions are ANDed Filter column +FUNCTION and(minus(s, 8) :: 5, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4)) Aggregating Filter column: notEquals(y, 0) > (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed Filter column +FUNCTION and(minus(__table1.s, 8_UInt8) :: 1, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8)) Aggregating Filter column: notEquals(__table1.y, 0_UInt8) 0 1 @@ -94,10 +100,12 @@ Filter column: notEquals(__table1.y, 0_UInt8) 9 10 > two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased Filter column +ALIAS notEquals(s, 8) :: 4 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4)) Aggregating Filter column: and(notEquals(y, 0), minus(y, 4)) > (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased Filter column +ALIAS notEquals(__table1.s, 8_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8)) Aggregating Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8)) 0 1 diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index 864dd69412a..4bd0eb7d908 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -49,14 +49,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|ALIAS notEquals(s, 4) :: 1 -> and(notEquals(y, 0), notEquals(s, 4))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|ALIAS notEquals(s, 4) :: 4 -> and(notEquals(y, 0), notEquals(s, 4)) UInt8 : 2" echo "> (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased" $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|ALIAS notEquals(__table1.s, 4_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -69,14 +69,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 1, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) UInt8 : 2" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 5, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4))" echo "> (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted" $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 4_UInt8) :: 1, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -89,14 +89,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --convert_query_to_cnf=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 8 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 8) :: 1, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 8) :: 5, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))" echo "> (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed" $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 8 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 8_UInt8) :: 1, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -109,14 +109,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --convert_query_to_cnf=0 -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 8 and y - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y, 0), minus(y, 4))\|ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))" + grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y, 0), minus(y, 4))\|ALIAS notEquals(s, 8) :: 4 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))" echo "> (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased" $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 8 and y - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))\|ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))" + grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))\|ALIAS notEquals(__table1.s, 8_UInt8) :: 1 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y From 8cb846c28b9fff0571140e31dd8a7dedf5563bc8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 16:12:59 +0000 Subject: [PATCH 029/105] Revert test. --- .../03156_analyzer_array_join_distributed.sql | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql index f4ebba0b1b2..f605a369822 100644 --- a/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql +++ b/tests/queries/0_stateless/03156_analyzer_array_join_distributed.sql @@ -8,21 +8,3 @@ SELECT s, arr, a FROM remote('127.0.0.{1,2}', currentDatabase(), arrays_test) AR SELECT s, arr FROM remote('127.0.0.2', currentDatabase(), arrays_test) ARRAY JOIN arr WHERE arr < 3 ORDER BY arr; SELECT s, arr FROM remote('127.0.0.{1,2}', currentDatabase(), arrays_test) ARRAY JOIN arr WHERE arr < 3 ORDER BY arr; - -create table hourly( - hour datetime, - `metric.names` Array(String), - `metric.values` Array(Int64) -) Engine=Memory -as select '2020-01-01', ['a', 'b'], [1,2]; - -SELECT - toDate(hour) AS day, - `metric.names`, - sum(`metric.values`) -FROM remote('127.0.0.{1,2}', currentDatabase(), hourly) -ARRAY JOIN metric -GROUP BY - day, - metric.names; -ORDER BY metric.names; From 3d7beae8fd4140f90917a592f3aae1bfecb90c5a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 4 Jun 2024 19:29:26 +0200 Subject: [PATCH 030/105] Simplify 03023_zeros_generate_random_with_limit_progress_bar --- ...rate_random_with_limit_progress_bar.expect | 49 ------------------- ...e_random_with_limit_progress_bar.reference | 3 ++ ...generate_random_with_limit_progress_bar.sh | 16 ++++++ 3 files changed, 19 insertions(+), 49 deletions(-) delete mode 100755 tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect create mode 100755 tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect deleted file mode 100755 index de15a199132..00000000000 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect +++ /dev/null @@ -1,49 +0,0 @@ -#!/usr/bin/expect -f - -set basedir [file dirname $argv0] -set basename [file tail $argv0] -if {[info exists env(CLICKHOUSE_TMP)]} { - set CLICKHOUSE_TMP $env(CLICKHOUSE_TMP) -} else { - set CLICKHOUSE_TMP "." -} -exp_internal -f $CLICKHOUSE_TMP/$basename.debuglog 0 - -log_user 0 -set timeout 60 -match_max 100000 -set stty_init "rows 25 cols 120" - -expect_after { - -i $any_spawn_id eof { exp_continue } - -i $any_spawn_id timeout { exit 1 } -} - -spawn clickhouse-local -expect ":) " - -# Trivial SELECT with LIMIT from system.zeros shows progress bar. -send "SELECT * FROM system.zeros LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" -expect "Progress: " -expect "█" -send "\3" -expect "Query was cancelled." -expect ":) " - -send "SELECT * FROM system.zeros_mt LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" -expect "Progress: " -expect "█" -send "\3" -expect "Query was cancelled." -expect ":) " - -# As well as from generateRandom -send "SELECT * FROM generateRandom() LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" -expect "Progress: " -expect "█" -send "\3" -expect "Query was cancelled." -expect ":) " - -send "exit\r" -expect eof diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference index e69de29bb2d..6ca5ae94f9a 100644 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference @@ -0,0 +1,3 @@ +Matched +Matched +Matched diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh new file mode 100755 index 00000000000..4bb8fc8880d --- /dev/null +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function run_with_progress_and_match_total_rows() +{ + echo "$1" | \ + ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1 | \ + grep -q '"total_rows_to_read":"100"' && echo "Matched" || echo "Expected total_rows_to_read not found" +} + +run_with_progress_and_match_total_rows 'SELECT * FROM system.zeros LIMIT 100' +run_with_progress_and_match_total_rows 'SELECT * FROM system.zeros_mt LIMIT 100' +run_with_progress_and_match_total_rows 'SELECT * FROM generateRandom() LIMIT 100' From 1628c2985d588613d6e1356720e96ec904a15651 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 5 Jun 2024 09:42:26 +0000 Subject: [PATCH 031/105] Some fixups --- .../functions/rounding-functions.md | 160 ++++++++++----- src/Functions/FunctionsRound.h | 191 +++++++++--------- .../03165_round_scale_as_column.sql | 91 ++++----- 3 files changed, 245 insertions(+), 197 deletions(-) diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index 55a4b3ce9df..6495a43fc85 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -6,49 +6,90 @@ sidebar_label: Rounding # Rounding Functions -## floor(x\[, N\]) +## floor -Returns the largest round number that is less than or equal to `x`. A round number is a multiple of 1/10N, or the nearest number of the appropriate data type if 1 / 10N isn’t exact. -`N` is an integer, optional parameter. By default it is zero, which means to round to an integer. -`N` may be negative. +Returns the largest rounded number less than or equal `x`. +A rounded number is a multiple of 1 / 10 * N, or the nearest number of the appropriate data type if 1 / 10 * N isn’t exact. -Examples: `floor(123.45, 1) = 123.4, floor(123.45, -1) = 120.` +Integer arguments may be rounded with negative `N` argument, with non-negative `N` the function returns `x`, i.e. does nothing. -`x` is any numeric type. The result is a number of the same type. -For integer arguments, it makes sense to round with a negative `N` value (for non-negative `N`, the function does not do anything). -If rounding causes overflow (for example, floor(-128, -1)), an implementation-specific result is returned. +If rounding causes an overflow (for example, `floor(-128, -1)`), the result is undefined. -## ceil(x\[, N\]), ceiling(x\[, N\]) +**Syntax** -Returns the smallest round number that is greater than or equal to `x`. In every other way, it is the same as the `floor` function (see above). +``` sql +floor(x[, N]) +``` -## trunc(x\[, N\]), truncate(x\[, N\]) +**Parameters** -Returns the round number with largest absolute value that has an absolute value less than or equal to `x`‘s. In every other way, it is the same as the ’floor’ function (see above). +- `x` - The value to round. [Float*](../data-types/float.md), [Decimal*](../data-types/decimal.md), or [(U)Int*](../data-types/int-uint.md). +- `N` . [(U)Int*](../data-types/int-uint.md). The default is zero, which means rounding to an integer. Can be negative. + +**Returned value** + +A rounded number of the same type as `x`. + +**Examples** + +Query: + +```sql +SELECT floor(123.45, 1) AS rounded +``` + +Result: + +``` +┌─rounded─┐ +│ 123.4 │ +└─────────┘ +``` + +Query: + +```sql +SELECT floor(123.45, -1) +``` + +Result: + +``` +┌─rounded─┐ +│ 120 │ +└─────────┘ +``` + +## ceiling + +Like `floor` but returns the smallest rounded number greater than or equal `x`. + +**Syntax** + +``` sql +ceiling(x[, N]) +``` + +Alias: `ceil` + +## truncate + +Like `floor` but returns the rounded number with largest absolute value that has an absolute value less than or equal to `x`‘s. **Syntax** ```sql -trunc(input, precision) +truncate(x[, N]) ``` -Alias: `truncate`. - -**Parameters** - -- `input`: A numeric type ([Float](../data-types/float.md), [Decimal](../data-types/decimal.md) or [Integer](../data-types/int-uint.md)). -- `precision`: An [Integer](../data-types/int-uint.md) type. - -**Returned value** - -- A data type of `input`. +Alias: `trunc`. **Example** Query: ```sql -SELECT trunc(123.499, 1) as res; +SELECT truncate(123.499, 1) as res; ``` ```response @@ -57,37 +98,40 @@ SELECT trunc(123.499, 1) as res; └───────┘ ``` -## round(x\[, N\]) +## round Rounds a value to a specified number of decimal places. -The function returns the nearest number of the specified order. In case when given number has equal distance to surrounding numbers, the function uses banker’s rounding for float number types and rounds away from zero for the other number types (Decimal). +The function returns the nearest number of the specified order. +If the input value has equal distance to two neighboring numbers, the function uses banker’s rounding for [Float*](../data-types/float.md) inputs and rounds away from zero for the other number types ([Decimal*](../data-types/decimal.md). + +**Syntax** ``` sql -round(expression [, decimal_places]) +round(x[, N]) ``` **Arguments** -- `expression` — A number to be rounded. Can be any [expression](../../sql-reference/syntax.md#syntax-expressions) returning the numeric [data type](../data-types/index.md#data_types). -- `decimal-places` — The number of decimal places to round to. - - If `decimal-places > 0` then the function rounds the value to the right of the decimal point. - - If `decimal-places < 0` then the function rounds the value to the left of the decimal point. - - If `decimal-places = 0` then the function rounds the value to integer. In this case the argument can be omitted. +- `x` — A number to round. [Float*](../data-types/float.md), [Decimal*](../data-types/decimal.md), or [(U)Int*](../data-types/int-uint.md). +- `N` — The number of decimal places to round to. Integer. Defaults to `0`. + - If `N > 0`, the function rounds to the right of the decimal point. + - If `N < 0`, the function rounds to the left of the decimal point. + - If `N = 0`, the function rounds to the next integer. **Returned value:** -The rounded number of the same type as the input number. +A rounded number of the same type as `x`. **Examples** -Example of usage with Float: +Example with `Float` inputs: -``` sql +```sql SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3; ``` -``` text +``` ┌───x─┬─round(divide(number, 2))─┐ │ 0 │ 0 │ │ 0.5 │ 0 │ @@ -95,13 +139,13 @@ SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3; └─────┴──────────────────────────┘ ``` -Example of usage with Decimal: +Example with `Decimal` inputs: -``` sql +```sql SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3; ``` -``` text +``` ┌───x─┬─round(CAST(divide(number, 2), 'Decimal(10, 4)'))─┐ │ 0 │ 0 │ │ 0.5 │ 1 │ @@ -109,14 +153,14 @@ SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIM └─────┴──────────────────────────────────────────────────┘ ``` -If you want to keep the trailing zeros, you need to enable `output_format_decimal_trailing_zeros` +To retain trailing zeros, enable setting `output_format_decimal_trailing_zeros`: -``` sql +```sql SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3 settings output_format_decimal_trailing_zeros=1; ``` -``` text +``` ┌──────x─┬─round(CAST(divide(number, 2), 'Decimal(10, 4)'))─┐ │ 0.0000 │ 0.0000 │ │ 0.5000 │ 1.0000 │ @@ -151,9 +195,15 @@ round(3.65, 1) = 3.6 Rounds a number to a specified decimal position. -- If the rounding number is halfway between two numbers, the function uses banker’s rounding. Banker's rounding is a method of rounding fractional numbers. When the rounding number is halfway between two numbers, it's rounded to the nearest even digit at the specified decimal position. For example: 3.5 rounds up to 4, 2.5 rounds down to 2. It's the default rounding method for floating point numbers defined in [IEEE 754](https://en.wikipedia.org/wiki/IEEE_754#Roundings_to_nearest). The [round](#rounding_functions-round) function performs the same rounding for floating point numbers. The `roundBankers` function also rounds integers the same way, for example, `roundBankers(45, -1) = 40`. +If the rounding number is halfway between two numbers, the function uses banker’s rounding. +Banker's rounding is a method of rounding fractional numbers +When the rounding number is halfway between two numbers, it's rounded to the nearest even digit at the specified decimal position. +For example: 3.5 rounds up to 4, 2.5 rounds down to 2. +It's the default rounding method for floating point numbers defined in [IEEE 754](https://en.wikipedia.org/wiki/IEEE_754#Roundings_to_nearest). +The [round](#rounding_functions-round) function performs the same rounding for floating point numbers. +The `roundBankers` function also rounds integers the same way, for example, `roundBankers(45, -1) = 40`. -- In other cases, the function rounds numbers to the nearest integer. +In other cases, the function rounds numbers to the nearest integer. Using banker’s rounding, you can reduce the effect that rounding numbers has on the results of summing or subtracting these numbers. @@ -166,16 +216,20 @@ For example, sum numbers 1.5, 2.5, 3.5, 4.5 with different rounding: **Syntax** ``` sql -roundBankers(expression [, decimal_places]) +roundBankers(x [, N]) ``` **Arguments** -- `expression` — A number to be rounded. Can be any [expression](../../sql-reference/syntax.md#syntax-expressions) returning the numeric [data type](../data-types/index.md#data_types). -- `decimal-places` — Decimal places. An integer number. - - `decimal-places > 0` — The function rounds the number to the given position right of the decimal point. Example: `roundBankers(3.55, 1) = 3.6`. - - `decimal-places < 0` — The function rounds the number to the given position left of the decimal point. Example: `roundBankers(24.55, -1) = 20`. - - `decimal-places = 0` — The function rounds the number to an integer. In this case the argument can be omitted. Example: `roundBankers(2.5) = 2`. + - `N > 0` — The function rounds the number to the given position right of the decimal point. Example: `roundBankers(3.55, 1) = 3.6`. + - `N < 0` — The function rounds the number to the given position left of the decimal point. Example: `roundBankers(24.55, -1) = 20`. + - `N = 0` — The function rounds the number to an integer. In this case the argument can be omitted. Example: `roundBankers(2.5) = 2`. + +- `x` — A number to round. [Float*](../data-types/float.md), [Decimal*](../data-types/decimal.md), or [(U)Int*](../data-types/int-uint.md). +- `N` — The number of decimal places to round to. Integer. Defaults to `0`. + - If `N > 0`, the function rounds to the right of the decimal point. + - If `N < 0`, the function rounds to the left of the decimal point. + - If `N = 0`, the function rounds to the next integer. **Returned value** @@ -185,13 +239,13 @@ A value rounded by the banker’s rounding method. Query: -``` sql +```sql SELECT number / 2 AS x, roundBankers(x, 0) AS b fROM system.numbers limit 10 ``` Result: -``` text +``` ┌───x─┬─b─┐ │ 0 │ 0 │ │ 0.5 │ 0 │ @@ -208,7 +262,7 @@ Result: Examples of Banker’s rounding: -``` text +``` roundBankers(0.4) = 0 roundBankers(-3.5) = -4 roundBankers(4.5) = 4 diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 978b6d88d05..d6eabdb6e88 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -40,26 +40,22 @@ namespace ErrorCodes } -/** Rounding Functions: - * round(x, N) - rounding to nearest (N = 0 by default). Use banker's rounding for floating point numbers. - * roundBankers(x, N) - rounding to nearest (N = 0 by default). Use banker's rounding for all numbers. - * floor(x, N) is the largest number <= x (N = 0 by default). - * ceil(x, N) is the smallest number >= x (N = 0 by default). - * trunc(x, N) - is the largest by absolute value number that is not greater than x by absolute value (N = 0 by default). - * - * The value of the parameter N (scale): - * - N > 0: round to the number with N decimal places after the decimal point - * - N < 0: round to an integer with N zero characters - * - N = 0: round to an integer - * - * Type of the result is the type of argument. - * For integer arguments, when passing negative scale, overflow can occur. - * In that case, the behavior is implementation specific. - */ +/// Rounding Functions: +/// - round(x, N) - rounding to nearest (N = 0 by default). Use banker's rounding for floating point numbers. +/// - roundBankers(x, N) - rounding to nearest (N = 0 by default). Use banker's rounding for all numbers. +/// - floor(x, N) is the largest number <= x (N = 0 by default). +/// - ceil(x, N) is the smallest number >= x (N = 0 by default). +/// - trunc(x, N) - is the largest by absolute value number that is not greater than x by absolute value (N = 0 by default). +/// The value of the parameter N (scale): +/// - N > 0: round to the number with N decimal places after the decimal point +/// - N < 0: round to an integer with N zero characters +/// - N = 0: round to an integer -/** This parameter controls the behavior of the rounding functions. - */ +/// Type of the result is the type of argument. +/// For integer arguments, when passing negative scale, overflow can occur. In that case, the behavior is undefined. + +/// Controls the behavior of the rounding functions. enum class ScaleMode : uint8_t { Positive, // round to a number with N decimal places after the decimal point @@ -75,7 +71,7 @@ enum class RoundingMode : uint8_t Ceil = _MM_FROUND_TO_POS_INF | _MM_FROUND_NO_EXC, Trunc = _MM_FROUND_TO_ZERO | _MM_FROUND_NO_EXC, #else - Round = 8, /// Values are correspond to above just in case. + Round = 8, /// Values correspond to above values, just in case. Floor = 9, Ceil = 10, Trunc = 11, @@ -84,16 +80,21 @@ enum class RoundingMode : uint8_t enum class TieBreakingMode : uint8_t { - Auto, // use banker's rounding for floating point numbers, round up otherwise - Bankers, // use banker's rounding + Auto, /// banker's rounding for floating point numbers, round up otherwise + Bankers, /// banker's rounding +}; + +enum class Vectorize : uint8_t +{ + No, + Yes }; /// For N, no more than the number of digits in the largest type. using Scale = Int16; -/** Rounding functions for integer values. - */ +/// Rounding functions for integer values. template struct IntegerRoundingComputation { @@ -150,7 +151,7 @@ struct IntegerRoundingComputation } } - UNREACHABLE(); + std::unreachable(); } static ALWAYS_INLINE T compute(T x, T scale) @@ -164,10 +165,11 @@ struct IntegerRoundingComputation return computeImpl(x, scale); } - UNREACHABLE(); + std::unreachable(); } - static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) requires std::integral + static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) + requires std::integral { if constexpr (sizeof(T) <= sizeof(scale) && scale_mode == ScaleMode::Negative) { @@ -180,20 +182,23 @@ struct IntegerRoundingComputation *out = compute(*in, static_cast(scale)); } - static ALWAYS_INLINE void compute(const T * __restrict in, T scale, T * __restrict out) requires(!std::integral) + static ALWAYS_INLINE void compute(const T * __restrict in, T scale, T * __restrict out) + requires(!std::integral) { *out = compute(*in, scale); } }; -template -class BaseFloatRoundingComputation; +template +class FloatRoundingComputationBase; #ifdef __SSE4_1__ +/// Vectorized implementation for x86. + template <> -class BaseFloatRoundingComputation +class FloatRoundingComputationBase { public: using ScalarType = Float32; @@ -214,7 +219,7 @@ public: }; template <> -class BaseFloatRoundingComputation +class FloatRoundingComputationBase { public: using ScalarType = Float64; @@ -236,7 +241,7 @@ public: #endif -/// Implementation for ARM/columnar scale argument. Not vectorized. +/// Sequential implementation for ARM. Also used for scalar arguments. inline float roundWithMode(float x, RoundingMode mode) { @@ -248,7 +253,7 @@ inline float roundWithMode(float x, RoundingMode mode) case RoundingMode::Trunc: return truncf(x); } - UNREACHABLE(); + std::unreachable(); } inline double roundWithMode(double x, RoundingMode mode) @@ -261,11 +266,11 @@ inline double roundWithMode(double x, RoundingMode mode) case RoundingMode::Trunc: return trunc(x); } - UNREACHABLE(); + std::unreachable(); } template -class BaseFloatRoundingComputation +class FloatRoundingComputationBase { public: using ScalarType = T; @@ -288,10 +293,10 @@ public: /** Implementation of low-level round-off functions for floating-point values. */ -template -class FloatRoundingComputation : public BaseFloatRoundingComputation +template +class FloatRoundingComputation : public FloatRoundingComputationBase { - using Base = BaseFloatRoundingComputation; + using Base = FloatRoundingComputationBase; public: static void compute(const T * __restrict in, const typename Base::VectorType & scale, T * __restrict out) @@ -323,14 +328,14 @@ struct FloatRoundingImpl private: static_assert(!is_decimal); - template - using Op = FloatRoundingComputation; + using Op = FloatRoundingComputation; using Data = std::array::data_count>; using ColumnType = ColumnVector; using Container = typename ColumnType::Container; @@ -370,7 +375,7 @@ public: static void applyOne(const T* __restrict in, size_t scale, T* __restrict out) { - using ScalarOp = Op; + using ScalarOp = Op; auto s = ScalarOp::prepare(scale); ScalarOp::compute(in, s, out); } @@ -485,8 +490,7 @@ public: } }; -/** Select the appropriate processing algorithm depending on the scale. - */ +/// Select the appropriate processing algorithm depending on the scale. inline void validateScale(Int64 scale64) { if (scale64 > std::numeric_limits::max() || scale64 < std::numeric_limits::min()) @@ -495,7 +499,7 @@ inline void validateScale(Int64 scale64) inline Scale getScaleArg(const ColumnConst* scale_col) { - const auto& scale_field = scale_col->getField(); + const auto & scale_field = scale_col->getField(); Int64 scale64 = scale_field.get(); validateScale(scale64); @@ -503,6 +507,7 @@ inline Scale getScaleArg(const ColumnConst* scale_col) return scale64; } +/// Generic dispatcher template struct Dispatcher { @@ -512,44 +517,44 @@ struct Dispatcher IntegerRoundingImpl>; template - static ColumnPtr apply(const IColumn * data_col, const IColumn * scale_col = nullptr) + static ColumnPtr apply(const IColumn * value_col, const IColumn * scale_col = nullptr) { - const auto & col = checkAndGetColumn>(*data_col); + const auto & value_col_typed = checkAndGetColumn>(*value_col); auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_res = col_res->getData(); - vec_res.resize(col.getData().size()); + vec_res.resize(value_col_typed.getData().size()); if (!vec_res.empty()) { - using ColVecScale = ColumnVector; if (scale_col == nullptr || isColumnConst(*scale_col)) { - auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst(scale_col)); + auto scale_arg = (scale_col == nullptr) ? 0 : getScaleArg(checkAndGetColumnConst>(scale_col)); if (scale_arg == 0) { size_t scale = 1; - FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + FunctionRoundingImpl::apply(value_col_typed.getData(), scale, vec_res); } else if (scale_arg > 0) { size_t scale = intExp10(scale_arg); - FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + FunctionRoundingImpl::apply(value_col_typed.getData(), scale, vec_res); } else { size_t scale = intExp10(-scale_arg); - FunctionRoundingImpl::apply(col.getData(), scale, vec_res); + FunctionRoundingImpl::apply(value_col_typed.getData(), scale, vec_res); } } - else if (const auto scale_typed = checkAndGetColumn(scale_col)) + /// Non-const scale argument: + else if (const auto * scale_col_typed = checkAndGetColumn>(scale_col)) { - const auto & in = col.getData(); - const auto & scale_data = scale_typed->getData(); - const size_t count = in.size(); + const auto & value_data = value_col_typed.getData(); + const auto & scale_data = scale_col_typed->getData(); + const size_t rows = value_data.size(); - const T * end_in = in.data() + count; - const T * __restrict p_in = in.data(); + const T * end_in = value_data.data() + rows; + const T * __restrict p_in = value_data.data(); const ScaleType * __restrict p_scale = scale_data.data(); T * __restrict p_out = vec_res.data(); @@ -585,36 +590,37 @@ struct Dispatcher } }; +/// Dispatcher for Decimal inputs template struct Dispatcher { public: template - static ColumnPtr apply(const IColumn * data_col, const IColumn* scale_col = nullptr) + static ColumnPtr apply(const IColumn * value_col, const IColumn* scale_col = nullptr) { - const auto & col = checkAndGetColumn>(*data_col); - const typename ColumnDecimal::Container & vec_src = col.getData(); + const auto & value_col_typed = checkAndGetColumn>(*value_col); + const typename ColumnDecimal::Container & vec_src = value_col_typed.getData(); - auto col_res = ColumnDecimal::create(vec_src.size(), col.getScale()); + auto col_res = ColumnDecimal::create(vec_src.size(), value_col_typed.getScale()); auto & vec_res = col_res->getData(); if (!vec_res.empty()) { - using ColVecScale = ColumnVector; if (scale_col == nullptr || isColumnConst(*scale_col)) { - auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst(scale_col)); - DecimalRoundingImpl::apply(col.getData(), col.getScale(), vec_res, scale_arg); + auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst>(scale_col)); + DecimalRoundingImpl::apply(value_col_typed.getData(), value_col_typed.getScale(), vec_res, scale_arg); } - else if (const auto scale_typed = checkAndGetColumn(scale_col)) + /// Non-cosnt scale argument + else if (const auto * scale_col_typed = checkAndGetColumn>(scale_col)) { - const auto & scale = scale_typed->getData(); - const size_t count = vec_src.size(); + const auto & scale = scale_col_typed->getData(); + const size_t rows = vec_src.size(); using NativeType = typename T::NativeType; const NativeType * __restrict p_in = reinterpret_cast(vec_src.data()); const ScaleType * __restrict p_scale = scale.data(); - const NativeType * end_in = p_in + count; + const NativeType * end_in = p_in + rows; NativeType * __restrict p_out = reinterpret_cast(vec_res.data()); while (p_in < end_in) { @@ -622,7 +628,7 @@ public: validateScale(scale64); Scale raw_scale = scale64; - DecimalRoundingImpl::applyOne(p_in, col.getScale(), p_out, raw_scale); + DecimalRoundingImpl::applyOne(p_in, value_col_typed.getScale(), p_out, raw_scale); ++p_in; ++p_scale; ++p_out; @@ -634,9 +640,8 @@ public: } }; -/** A template for functions that round the value of an input parameter of type - * (U)Int8/16/32/64, Float32/64 or Decimal32/64/128, and accept an additional optional parameter (default is 0). - */ +/// Functions that round the value of an input parameter of type (U)Int8/16/32/64, Float32/64 or Decimal32/64/128. +/// Accept an additional optional parameter (0 by default). template class FunctionRounding : public IFunction { @@ -644,16 +649,12 @@ public: static constexpr auto name = Name::name; static FunctionPtr create(ContextPtr) { return std::make_shared(); } - String getName() const override - { - return name; - } - + String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } - /// Get result types by argument types. If the function does not apply to these arguments, throw an exception. DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if ((arguments.empty()) || (arguments.size() > 2)) @@ -669,14 +670,12 @@ public: return arguments[0]; } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { - const ColumnWithTypeAndName & data_column = arguments[0]; + const ColumnWithTypeAndName & value_arg = arguments[0]; ColumnPtr res; - auto callData = [&](const auto & types) -> bool + auto call_data = [&](const auto & types) -> bool { using Types = std::decay_t; using DataType = typename Types::RightType; @@ -685,24 +684,24 @@ public: { const ColumnWithTypeAndName & scale_column = arguments[1]; - auto callScale = [&](const auto & scaleTypes) -> bool + auto call_scale = [&](const auto & scaleTypes) -> bool { using ScaleTypes = std::decay_t; using ScaleType = typename ScaleTypes::RightType; - if (isColumnConst(*data_column.column) && !isColumnConst(*scale_column.column)) + if (isColumnConst(*value_arg.column) && !isColumnConst(*scale_column.column)) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale column must be const for const data column"); - res = Dispatcher::template apply(data_column.column.get(), scale_column.column.get()); + res = Dispatcher::template apply(value_arg.column.get(), scale_column.column.get()); return true; }; TypeIndex right_index = scale_column.type->getTypeId(); - if (!callOnBasicType(right_index, callScale)) + if (!callOnBasicType(right_index, call_scale)) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Scale argument for rounding functions must have integer type"); return true; } - res = Dispatcher::template apply(data_column.column.get()); + res = Dispatcher::template apply(value_arg.column.get()); return true; }; @@ -715,9 +714,9 @@ public: throw Exception(ErrorCodes::CANNOT_SET_ROUNDING_MODE, "Cannot set floating point rounding mode"); #endif - TypeIndex left_index = data_column.type->getTypeId(); - if (!callOnBasicType(left_index, callData)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", data_column.name, getName()); + TypeIndex left_index = value_arg.type->getTypeId(); + if (!callOnBasicType(left_index, call_data)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", value_arg.name, getName()); return res; } @@ -734,9 +733,8 @@ public: }; -/** Rounds down to a number within explicitly specified array. - * If the value is less than the minimal bound - returns the minimal bound. - */ +/// Rounds down to a number within explicitly specified array. +/// If the value is less than the minimal bound - returns the minimal bound. class FunctionRoundDown : public IFunction { public: @@ -744,7 +742,6 @@ public: static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } - bool isVariadic() const override { return false; } size_t getNumberOfArguments() const override { return 2; } bool useDefaultImplementationForConstants() const override { return true; } diff --git a/tests/queries/0_stateless/03165_round_scale_as_column.sql b/tests/queries/0_stateless/03165_round_scale_as_column.sql index 60940b22fc1..2304d2730a9 100644 --- a/tests/queries/0_stateless/03165_round_scale_as_column.sql +++ b/tests/queries/0_stateless/03165_round_scale_as_column.sql @@ -1,4 +1,4 @@ --- Regression test that functions round(), roundBankers(), floor(), ceil() and trunc() work with default 'scale' argument (= the 2nd argument) +-- Tests functions round(), roundBankers(), floor(), ceil() and trunc() with default 'scale' argument SELECT toUInt8(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; SELECT toUInt16(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; SELECT toUInt32(number) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; @@ -7,13 +7,13 @@ SELECT toInt8(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), t SELECT toInt16(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; SELECT toInt32(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; SELECT toInt64(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; + SELECT toFloat32(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; SELECT toFloat64(number - 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; - SELECT toFloat32((number - 10) / 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; SELECT toFloat64((number - 10) / 10) AS x, round(x), roundBankers(x), floor(x), ceil(x), trunc(x) FROM system.numbers LIMIT 20; --- Test that functions round(), roundBankers(), floor(), ceil() and trunc() accept non-const 'scale' argument +-- Functions round(), roundBankers(), floor(), ceil() and trunc() accept non-const 'scale' arguments SELECT toFloat32((number - 10) / 10) AS x, round(x, materialize(1)), roundBankers(x, materialize(1)), floor(x, materialize(1)), ceil(x, materialize(1)), trunc(x, materialize(1)) FROM system.numbers LIMIT 20; SELECT toFloat64((number - 10) / 10) AS x, round(x, materialize(1)), roundBankers(x, materialize(1)), floor(x, materialize(1)), ceil(x, materialize(1)), trunc(x, materialize(1)) FROM system.numbers LIMIT 20; SELECT toUInt8(number) AS x, round(x, materialize(-1)), roundBankers(x, materialize(-1)), floor(x, materialize(-1)), ceil(x, materialize(-1)), trunc(x, materialize(-1)) FROM system.numbers LIMIT 20; @@ -41,9 +41,9 @@ SELECT toFloat64(number - 10) AS x, round(x, materialize(-2)), roundBankers(x, m SELECT toString('CHECKPOINT1'); -DROP TABLE IF EXISTS tround; +DROP TABLE IF EXISTS tab; -CREATE TABLE tround ( +CREATE TABLE tab ( id Int32, scale Int16, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, @@ -51,75 +51,72 @@ CREATE TABLE tround ( f32 Float32, f64 Float64 ) ENGINE = Memory; -INSERT INTO tround SELECT number , 0, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; -INSERT INTO tround SELECT number+20 , 0, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; -INSERT INTO tround SELECT number+40 , -1, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; -INSERT INTO tround SELECT number+60 , -1, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; -INSERT INTO tround SELECT number+80 , -2, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; -INSERT INTO tround SELECT number+100, -2, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number , 0, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+20 , 0, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+40 , -1, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+60 , -1, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+80 , -2, number, number, number, number, number, number, number, number, number, number, FROM system.numbers LIMIT 20; +INSERT INTO tab SELECT number+100, -2, number+10, number+10, number+10, number+10, number-10, number-10, number-10, number-10, (toFloat32(number)-10)/10, (toFloat64(number)-10)/10, FROM system.numbers LIMIT 20; -INSERT INTO tround SELECT number+200, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10; -INSERT INTO tround SELECT number+210, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10; +INSERT INTO tab SELECT number+200, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10; +INSERT INTO tab SELECT number+210, -number, 0, 0, 0, 0, 0, 0, 0, 0, 12345.6789, 12345.6789, FROM system.numbers LIMIT 10; -INSERT INTO tround VALUES (300, 4, 2, 2, 2, 2, 2, 2, 2, 2, 2.0, 2.0); -INSERT INTO tround VALUES (301, 4, 20, 20, 20, 20, 20, 20, 20, 20, 20.0, 20.0); -INSERT INTO tround VALUES (302, 4, 200, 200, 200, 200, 200, 200, 200, 200, 200.0, 200.0); -INSERT INTO tround VALUES (303, 4, 5, 5, 5, 5, 5, 5, 5, 5, 5.0, 5.0); -INSERT INTO tround VALUES (304, 4, 50, 50, 50, 50, 50, 50, 50, 50, 50.0, 50.0); -INSERT INTO tround VALUES (305, 4, 500, 500, 500, 500, 500, 500, 500, 500, 500.0, 500.0); +INSERT INTO tab VALUES (300, 4, 2, 2, 2, 2, 2, 2, 2, 2, 2.0, 2.0); +INSERT INTO tab VALUES (301, 4, 20, 20, 20, 20, 20, 20, 20, 20, 20.0, 20.0); +INSERT INTO tab VALUES (302, 4, 200, 200, 200, 200, 200, 200, 200, 200, 200.0, 200.0); +INSERT INTO tab VALUES (303, 4, 5, 5, 5, 5, 5, 5, 5, 5, 5.0, 5.0); +INSERT INTO tab VALUES (304, 4, 50, 50, 50, 50, 50, 50, 50, 50, 50.0, 50.0); +INSERT INTO tab VALUES (305, 4, 500, 500, 500, 500, 500, 500, 500, 500, 500.0, 500.0); SELECT toString('id u8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, u8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, u8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id u16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, u16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, u16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id u32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, u32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, u32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id u64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, u64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, u64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id i8 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, i8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, i8 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id i16 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, i16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, i16 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id i32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, i32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, i32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id i64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, i64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, i64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id f32 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, f32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, f32 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; SELECT toString('id f64 scale round(u8, scale) roundBankers(x, scale) floor(x, scale) ceil(x, scale) trunc(x, scale)'); -SELECT id, f64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tround ORDER BY id; +SELECT id, f64 AS x, scale, round(x, scale), roundBankers(x, scale), floor(x, scale), ceil(x, scale), trunc(x, scale) FROM tab ORDER BY id; -DROP TABLE IF EXISTS tround; +DROP TABLE tab; -- SELECT toString('CHECKPOINT2'); -DROP TABLE IF EXISTS tround2; +DROP TABLE IF EXISTS tab; -CREATE TABLE tround2 ( +CREATE TABLE tab ( id Int32, scale Int16, d32 Decimal32(4), d64 Decimal64(4), d128 Decimal128(4), d256 Decimal256(4) ) ENGINE = Memory; -INSERT INTO tround2 VALUES (1, 6, toDecimal32('42.42', 4), toDecimal64('42.42', 4), toDecimal128('42.42', 4), toDecimal256('42.42', 4)); -INSERT INTO tround2 SELECT 2 , 6, cos(d32), cos(d64), cos(d128), cos(d256) FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 3 , 6, sqrt(d32), sqrt(d64), sqrt(d128), sqrt(d256) FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 4 , 6, lgamma(d32), lgamma(d64), lgamma(d128), lgamma(d256) FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 5 , 6, tgamma(d32)/1e50, tgamma(d64)/1e50, tgamma(d128)/1e50, tgamma(d256)/1e50 FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 6 , 8, sin(d32), sin(d64), sin(d128), sin(d256) FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 7 , 8, cos(d32), cos(d64), cos(d128), cos(d256) FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 8 , 8, log(d32), log(d64), log(d128), log(d256) FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 9 , 8, log2(d32), log2(d64), log2(d128), log2(d256) FROM tround2 WHERE id = 1; -INSERT INTO tround2 SELECT 10, 8, log10(d32), log10(d64), log10(d128), log10(d256) FROM tround2 WHERE id = 1; +INSERT INTO tab VALUES (1, 6, toDecimal32('42.42', 4), toDecimal64('42.42', 4), toDecimal128('42.42', 4), toDecimal256('42.42', 4)); +INSERT INTO tab SELECT 2 , 6, cos(d32), cos(d64), cos(d128), cos(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 3 , 6, sqrt(d32), sqrt(d64), sqrt(d128), sqrt(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 4 , 6, lgamma(d32), lgamma(d64), lgamma(d128), lgamma(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 5 , 6, tgamma(d32)/1e50, tgamma(d64)/1e50, tgamma(d128)/1e50, tgamma(d256)/1e50 FROM tab WHERE id = 1; +INSERT INTO tab SELECT 6 , 8, sin(d32), sin(d64), sin(d128), sin(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 7 , 8, cos(d32), cos(d64), cos(d128), cos(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 8 , 8, log(d32), log(d64), log(d128), log(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 9 , 8, log2(d32), log2(d64), log2(d128), log2(d256) FROM tab WHERE id = 1; +INSERT INTO tab SELECT 10, 8, log10(d32), log10(d64), log10(d128), log10(d256) FROM tab WHERE id = 1; -SELECT id, round(d32, scale), round(d64, scale), round(d128, scale), round(d256, scale) FROM tround2 ORDER BY id; +SELECT id, round(d32, scale), round(d64, scale), round(d128, scale), round(d256, scale) FROM tab ORDER BY id; -DROP TABLE IF EXISTS tround2; +DROP TABLE tab; SELECT round(1, 1); SELECT round(materialize(1), materialize(1)); SELECT round(1, materialize(1)); --{serverError ILLEGAL_COLUMN} SELECT round(materialize(1), 1); - - - From 7ce67265c70075009617c1b43700d351bf43a9ad Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Wed, 5 Jun 2024 14:06:31 +0000 Subject: [PATCH 032/105] PR post-review fixes --- src/Functions/FunctionsRound.h | 51 ++++++++++++---------------------- 1 file changed, 18 insertions(+), 33 deletions(-) diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index b31d1bcee8a..a05619b54bb 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -373,11 +373,11 @@ public: } } - static void applyOne(const T* __restrict in, size_t scale, T* __restrict out) + static void applyOne(T in, size_t scale, T& out) { using ScalarOp = Op; auto s = ScalarOp::prepare(scale); - ScalarOp::compute(in, s, out); + ScalarOp::compute(&in, s, &out); } }; @@ -435,9 +435,9 @@ public: } } - static void applyOne(const T* __restrict in, size_t scale, T* __restrict out) + static void applyOne(T in, size_t scale, T& out) { - Op::compute(in, scale, out); + Op::compute(&in, scale, &out); } }; @@ -475,17 +475,17 @@ public: } } - static void applyOne(const NativeType* __restrict in, UInt32 in_scale, NativeType* __restrict out, Scale scale_arg) + static void applyOne(NativeType in, UInt32 in_scale, NativeType& out, Scale scale_arg) { scale_arg = in_scale - scale_arg; if (scale_arg > 0) { auto scale = intExp10OfSize(scale_arg); - Op::compute(in, scale, out); + Op::compute(&in, scale, &out); } else { - memcpy(out, in, sizeof(T)); + memcpy(&out, &in, sizeof(T)); } } }; @@ -553,35 +553,27 @@ struct Dispatcher const auto & scale_data = scale_col_typed->getData(); const size_t rows = value_data.size(); - const T * end_in = value_data.data() + rows; - const T * __restrict p_in = value_data.data(); - const ScaleType * __restrict p_scale = scale_data.data(); - T * __restrict p_out = vec_res.data(); - - while (p_in < end_in) + for (size_t i = 0; i < rows; ++i) { - Int64 scale64 = *p_scale; + Int64 scale64 = scale_data[i]; validateScale(scale64); Scale raw_scale = scale64; if (raw_scale == 0) { size_t scale = 1; - FunctionRoundingImpl::applyOne(p_in, scale, p_out); + FunctionRoundingImpl::applyOne(value_data[i], scale, vec_res[i]); } else if (raw_scale > 0) { size_t scale = intExp10(raw_scale); - FunctionRoundingImpl::applyOne(p_in, scale, p_out); + FunctionRoundingImpl::applyOne(value_data[i], scale, vec_res[i]); } else { size_t scale = intExp10(-raw_scale); - FunctionRoundingImpl::applyOne(p_in, scale, p_out); + FunctionRoundingImpl::applyOne(value_data[i], scale, vec_res[i]); } - ++p_in; - ++p_scale; - ++p_out; } } } @@ -596,7 +588,7 @@ struct Dispatcher { public: template - static ColumnPtr apply(const IColumn * value_col, const IColumn* scale_col = nullptr) + static ColumnPtr apply(const IColumn * value_col, const IColumn * scale_col = nullptr) { const auto & value_col_typed = checkAndGetColumn>(*value_col); const typename ColumnDecimal::Container & vec_src = value_col_typed.getData(); @@ -611,27 +603,20 @@ public: auto scale_arg = scale_col == nullptr ? 0 : getScaleArg(checkAndGetColumnConst>(scale_col)); DecimalRoundingImpl::apply(value_col_typed.getData(), value_col_typed.getScale(), vec_res, scale_arg); } - /// Non-cosnt scale argument + /// Non-const scale argument else if (const auto * scale_col_typed = checkAndGetColumn>(scale_col)) { const auto & scale = scale_col_typed->getData(); const size_t rows = vec_src.size(); - using NativeType = typename T::NativeType; - const NativeType * __restrict p_in = reinterpret_cast(vec_src.data()); - const ScaleType * __restrict p_scale = scale.data(); - const NativeType * end_in = p_in + rows; - NativeType * __restrict p_out = reinterpret_cast(vec_res.data()); - while (p_in < end_in) + for (size_t i = 0; i < rows; ++i) { - Int64 scale64 = *p_scale; + Int64 scale64 = scale[i]; validateScale(scale64); Scale raw_scale = scale64; - DecimalRoundingImpl::applyOne(p_in, value_col_typed.getScale(), p_out, raw_scale); - ++p_in; - ++p_scale; - ++p_out; + DecimalRoundingImpl::applyOne(value_col_typed.getElement(i), value_col_typed.getScale(), + reinterpret_cast::NativeT&>(col_res->getElement(i)), raw_scale); } } } From ee5100171e383c2c82cdaacdedd9e78146981bb5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 5 Jun 2024 15:50:27 +0000 Subject: [PATCH 033/105] Avoid duplicating names for new inputs of ActionsDAG::split result. --- src/Interpreters/ActionsDAG.cpp | 34 +++++++++++++++++-- ...mv_prewhere_duplicating_name_bug.reference | 2 ++ ...03166_mv_prewhere_duplicating_name_bug.sql | 7 ++++ 3 files changed, 41 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.reference create mode 100644 tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index cfccc835d29..6915f82d067 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1718,6 +1718,13 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split /// List of nodes from current actions which are not inputs, but will be in second part. NodeRawConstPtrs new_inputs; + /// Avoid new inputs to have the same name as existing inputs. + /// It's allowed for DAG but may break Block invariant 'columns with identical name must have identical structure'. + std::unordered_set duplicate_inputs; + size_t duplicate_counter = 0; + for (const auto * input : inputs) + duplicate_inputs.insert(input->result_name); + struct Frame { const Node * node = nullptr; @@ -1830,7 +1837,8 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split input_node.result_name = child->result_name; child_data.to_second = &second_nodes.emplace_back(std::move(input_node)); - new_inputs.push_back(child); + if (child->type != ActionType::INPUT) + new_inputs.push_back(child); } } @@ -1886,7 +1894,29 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split for (const auto * input : new_inputs) { - const auto & cur = data[input]; + auto & cur = data[input]; + + bool is_name_updated = false; + while (!duplicate_inputs.insert(cur.to_first->result_name).second) + { + is_name_updated = true; + cur.to_first->result_name = fmt::format("{}_{}", input->result_name, duplicate_counter); + ++duplicate_counter; + } + + if (is_name_updated) + { + Node input_node; + input_node.type = ActionType::INPUT; + input_node.result_type = cur.to_first->result_type; + input_node.result_name = cur.to_first->result_name; + + auto * new_input = &second_nodes.emplace_back(std::move(input_node)); + cur.to_second->type = ActionType::ALIAS; + cur.to_second->children = {new_input}; + cur.to_second = new_input; + } + second_inputs.push_back(cur.to_second); first_outputs.push_back(cur.to_first); } diff --git a/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.reference b/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.reference new file mode 100644 index 00000000000..b50fdcee209 --- /dev/null +++ b/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.reference @@ -0,0 +1,2 @@ +a b +a b diff --git a/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql b/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql new file mode 100644 index 00000000000..e32d23920dd --- /dev/null +++ b/tests/queries/0_stateless/03166_mv_prewhere_duplicating_name_bug.sql @@ -0,0 +1,7 @@ +create table src (x Int64) engine = Log; +create table dst (s String, lc LowCardinality(String)) engine MergeTree order by s; +create materialized view mv to dst (s String, lc String) as select 'a' as s, toLowCardinality('b') as lc from src; +insert into src values (1); + +select s, lc from mv where not ignore(lc) settings allow_experimental_analyzer=0; +select s, lc from mv where not ignore(lc) settings allow_experimental_analyzer=1; From d99f8b4acc3991cbcb8c269fc27773146d47c6cf Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Wed, 5 Jun 2024 20:35:22 +0330 Subject: [PATCH 034/105] Added Named collections for accessing Kafka --- docs/en/operations/named-collections.md | 56 +++++++++++++++++++++++++ 1 file changed, 56 insertions(+) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index c9d94dd95ee..5ba9029152e 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -443,3 +443,59 @@ SELECT dictGet('dict', 'b', 1); │ a │ └─────────────────────────┘ ``` + +## Named collections for accessing Kafka + +The description of parameters see [Kafka](../engines/table-engines/integrations/kafka.md). + +### DDL example + +```sql +CREATE TABLE queue( + timestamp UInt64, + level String, + message String +)ENGINE = Kafka SETTINGS kafka_broker_list = 'localhost:9092', + kafka_topic_list = 'kafka_topic', + kafka_group_name = 'consumer_group', + kafka_format = 'JSONEachRow', + kafka_max_block_size = 1048576; +``` +### XML example + +```xml + + + + localhost:9092 + kafka_topic + consumer_group + JSONEachRow + 1048576 + + + +``` + +### kafka Table, named collection examples + +Both of the following examples use the same named collection `my_kafka_cluster`: + + +```sql +CREATE TABLE queue( + timestamp UInt64, + level String, + message String +)ENGINE = Kafka(my_kafka_cluster) + + +CREATE TABLE queue( + timestamp UInt64, + level String, + message String +)ENGINE = Kafka(my_kafka_cluster) +SETTINGS kafka_num_consumers = 4, + kafka_thread_per_consumer = 1; + +``` From 8bbf46ce2fe7f3d4e6c65b3268e44bb29642eb0e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 5 Jun 2024 17:15:48 +0000 Subject: [PATCH 035/105] Trying to fix a test. --- src/Interpreters/ActionsDAG.cpp | 42 ++++++++++--------- src/Interpreters/ActionsDAG.h | 2 +- .../Optimizations/optimizePrewhere.cpp | 2 +- 3 files changed, 25 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 6915f82d067..840aff9ebfb 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1704,7 +1704,7 @@ void ActionsDAG::mergeNodes(ActionsDAG && second, NodeRawConstPtrs * out_outputs } } -ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split_nodes, bool create_split_nodes_mapping) const +ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split_nodes, bool create_split_nodes_mapping, bool avoid_duplicate_inputs) const { /// Split DAG into two parts. /// (first_nodes, first_outputs) is a part which will have split_list in result. @@ -1722,8 +1722,9 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split /// It's allowed for DAG but may break Block invariant 'columns with identical name must have identical structure'. std::unordered_set duplicate_inputs; size_t duplicate_counter = 0; - for (const auto * input : inputs) - duplicate_inputs.insert(input->result_name); + if (avoid_duplicate_inputs) + for (const auto * input : inputs) + duplicate_inputs.insert(input->result_name); struct Frame { @@ -1896,25 +1897,28 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split { auto & cur = data[input]; - bool is_name_updated = false; - while (!duplicate_inputs.insert(cur.to_first->result_name).second) + if (avoid_duplicate_inputs) { - is_name_updated = true; - cur.to_first->result_name = fmt::format("{}_{}", input->result_name, duplicate_counter); - ++duplicate_counter; - } + bool is_name_updated = false; + while (!duplicate_inputs.insert(cur.to_first->result_name).second) + { + is_name_updated = true; + cur.to_first->result_name = fmt::format("{}_{}", input->result_name, duplicate_counter); + ++duplicate_counter; + } - if (is_name_updated) - { - Node input_node; - input_node.type = ActionType::INPUT; - input_node.result_type = cur.to_first->result_type; - input_node.result_name = cur.to_first->result_name; + if (is_name_updated) + { + Node input_node; + input_node.type = ActionType::INPUT; + input_node.result_type = cur.to_first->result_type; + input_node.result_name = cur.to_first->result_name; - auto * new_input = &second_nodes.emplace_back(std::move(input_node)); - cur.to_second->type = ActionType::ALIAS; - cur.to_second->children = {new_input}; - cur.to_second = new_input; + auto * new_input = &second_nodes.emplace_back(std::move(input_node)); + cur.to_second->type = ActionType::ALIAS; + cur.to_second->children = {new_input}; + cur.to_second = new_input; + } } second_inputs.push_back(cur.to_second); diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 8c0e3f0e576..7c6753ac9de 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -343,7 +343,7 @@ public: /// initial DAG : (a, b, c, d, e) -> (w, x, y, z) | 1 a 2 b 3 c 4 d 5 e 6 -> 1 2 3 4 5 6 w x y z /// split (first) : (a, c, d) -> (i, j, k, w, y) | 1 a 2 b 3 c 4 d 5 e 6 -> 1 2 b 3 4 5 e 6 i j k w y /// split (second) : (i, j, k, y, b, e) -> (x, y, z) | 1 2 b 3 4 5 e 6 i j k w y -> 1 2 3 4 5 6 w x y z - SplitResult split(std::unordered_set split_nodes, bool create_split_nodes_mapping = false) const; + SplitResult split(std::unordered_set split_nodes, bool create_split_nodes_mapping = false, bool avoid_duplicate_inputs = false) const; /// Splits actions into two parts. Returned first half may be swapped with ARRAY JOIN. SplitResult splitActionsBeforeArrayJoin(const NameSet & array_joined_columns) const; diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index fbd9b451ddc..74da7b8190f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -118,7 +118,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) outputs.resize(size); } - auto split_result = filter_step->getExpression()->split(optimize_result.prewhere_nodes, true); + auto split_result = filter_step->getExpression()->split(optimize_result.prewhere_nodes, true, true); /// This is the leak of abstraction. /// Splited actions may have inputs which are needed only for PREWHERE. From 966b08f192928cb799c12e18a62b6fa6a8dea2ad Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 5 Jun 2024 19:26:20 +0200 Subject: [PATCH 036/105] Fix segfault for a very tricky case --- src/Storages/TTLDescription.cpp | 12 +++-- .../__init__.py | 1 + .../configs/enable_parallel_replicas.xml | 11 ++++ .../configs/node1_macro.xml | 6 +++ .../test.py | 51 +++++++++++++++++++ 5 files changed, 78 insertions(+), 3 deletions(-) create mode 100644 tests/integration/test_replicated_merge_tree_replicated_db_ttl/__init__.py create mode 100644 tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml create mode 100644 tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/node1_macro.xml create mode 100644 tests/integration/test_replicated_merge_tree_replicated_db_ttl/test.py diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 6e7ea32ee59..f831465277d 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -166,8 +166,14 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType { ExpressionAndSets result; auto ttl_string = queryToString(ast); - auto syntax_analyzer_result = TreeRewriter(context).analyze(ast, columns); - ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context); + auto context_copy = Context::createCopy(context); + /// FIXME All code here will work with old analyzer, however for TTL + /// with subqueries it's possible that new analyzer will be enabled in ::read method + /// of underlying storage when all other parts of infra are not ready for it + /// (built with old analyzer). + context_copy->setSetting("allow_experimental_analyzer", Field{0}); + auto syntax_analyzer_result = TreeRewriter(context_copy).analyze(ast, columns); + ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context_copy); auto dag = analyzer.getActionsDAG(false); const auto * col = &dag->findInOutputs(ast->getColumnName()); @@ -177,7 +183,7 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType dag->getOutputs() = {col}; dag->removeUnusedActions(); - result.expression = std::make_shared(dag, ExpressionActionsSettings::fromContext(context)); + result.expression = std::make_shared(dag, ExpressionActionsSettings::fromContext(context_copy)); result.sets = analyzer.getPreparedSets(); return result; diff --git a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/__init__.py b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml new file mode 100644 index 00000000000..c654074740a --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/enable_parallel_replicas.xml @@ -0,0 +1,11 @@ + + + + 1 + 1 + default + 100 + 0 + + + diff --git a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/node1_macro.xml b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/node1_macro.xml new file mode 100644 index 00000000000..86e392f09ca --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/configs/node1_macro.xml @@ -0,0 +1,6 @@ + + + node1 + default + + diff --git a/tests/integration/test_replicated_merge_tree_replicated_db_ttl/test.py b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/test.py new file mode 100644 index 00000000000..1c6d15d9c7b --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_replicated_db_ttl/test.py @@ -0,0 +1,51 @@ +#!/usr/bin/env python3 + +import logging +import random +import string +import time + +import pytest +from multiprocessing.dummy import Pool +from helpers.cluster import ClickHouseCluster +import minio + + +cluster = ClickHouseCluster(__file__) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.add_instance( + "node1", + main_configs=["configs/node1_macro.xml"], + user_configs=[ + "configs/enable_parallel_replicas.xml", + ], + with_minio=True, + with_zookeeper=True, + ) + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_replicated_db_and_ttl(started_cluster): + node1 = cluster.instances["node1"] + node1.query("DROP DATABASE default") + node1.query("CREATE DATABASE default ENGINE Replicated('/replicated')") + + node1.query( + "CREATE TABLE 02908_main (a UInt32) ENGINE = ReplicatedMergeTree ORDER BY a" + ) + node1.query( + "CREATE TABLE 02908_dependent (a UInt32, ts DateTime) ENGINE = ReplicatedMergeTree ORDER BY a TTL ts + 1 WHERE a IN (SELECT a FROM 02908_main)" + ) + + node1.query("INSERT INTO 02908_main VALUES (1)") + node1.query("INSERT INTO 02908_dependent VALUES (1, now())") + + node1.query("SELECT * FROM 02908_dependent") From 46f14046e4e410f67ef7b436562a03c5bb605a98 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Wed, 5 Jun 2024 21:34:53 +0330 Subject: [PATCH 037/105] Added example on how to create named collection for kafka using ddl --- docs/en/operations/named-collections.md | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 5ba9029152e..9b9b682481d 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -451,15 +451,13 @@ The description of parameters see [Kafka](../engines/table-engines/integrations/ ### DDL example ```sql -CREATE TABLE queue( - timestamp UInt64, - level String, - message String -)ENGINE = Kafka SETTINGS kafka_broker_list = 'localhost:9092', - kafka_topic_list = 'kafka_topic', - kafka_group_name = 'consumer_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 1048576; +CREATE NAMED COLLECTION my_kafka_cluster AS +kafka_broker_list = 'localhost:9092', +kafka_topic_list = 'kafka_topic', +kafka_group_name = 'consumer_group', +kafka_format = 'JSONEachRow', +kafka_max_block_size = '1048576'; + ``` ### XML example From 927720a02033239cb71568008c0a90f5e5482fc0 Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 21:40:13 +0330 Subject: [PATCH 038/105] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 9b9b682481d..12d68b266c6 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -488,11 +488,13 @@ CREATE TABLE queue( )ENGINE = Kafka(my_kafka_cluster) -CREATE TABLE queue( +CREATE TABLE queue +( timestamp UInt64, level String, message String -)ENGINE = Kafka(my_kafka_cluster) +) +ENGINE = Kafka(my_kafka_cluster) SETTINGS kafka_num_consumers = 4, kafka_thread_per_consumer = 1; From 0316fa701fcaf2a81360fa0c7f16e4e83942d22d Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 21:40:23 +0330 Subject: [PATCH 039/105] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 12d68b266c6..4a3d303c114 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -475,7 +475,7 @@ kafka_max_block_size = '1048576'; ``` -### kafka Table, named collection examples +### Example of using named collections with a Kafka table Both of the following examples use the same named collection `my_kafka_cluster`: From 082efe2668bac79c13642ef4f9ea103e34fc0b62 Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 21:40:28 +0330 Subject: [PATCH 040/105] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 4a3d303c114..a563ed626fe 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -481,11 +481,13 @@ Both of the following examples use the same named collection `my_kafka_cluster`: ```sql -CREATE TABLE queue( +CREATE TABLE queue +( timestamp UInt64, level String, message String -)ENGINE = Kafka(my_kafka_cluster) +) +ENGINE = Kafka(my_kafka_cluster) CREATE TABLE queue From 8aea0c3fecf852cbed7c2dd55f98777fcb71e97d Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 22:05:19 +0330 Subject: [PATCH 041/105] Omitted extra spaces Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index a563ed626fe..0438fcae3b2 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -465,11 +465,11 @@ kafka_max_block_size = '1048576'; - localhost:9092 - kafka_topic - consumer_group - JSONEachRow - 1048576 + localhost:9092 + kafka_topic + consumer_group + JSONEachRow + 1048576 From 844a94383b7473d2174075257c3a99fa4a54c978 Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 22:05:36 +0330 Subject: [PATCH 042/105] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 0438fcae3b2..734b8dbe24a 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -489,7 +489,6 @@ CREATE TABLE queue ) ENGINE = Kafka(my_kafka_cluster) - CREATE TABLE queue ( timestamp UInt64, From 007a8ca32edba3b83f32ddffd95b4c2a13f78069 Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 22:05:45 +0330 Subject: [PATCH 043/105] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 734b8dbe24a..1d261d935af 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -498,5 +498,4 @@ CREATE TABLE queue ENGINE = Kafka(my_kafka_cluster) SETTINGS kafka_num_consumers = 4, kafka_thread_per_consumer = 1; - ``` From 42097398b38cc73fe3a99cdb515c138f665f1a5d Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 28 May 2024 16:33:24 +0200 Subject: [PATCH 044/105] Do not remove trailing space in *.reference test files (by modern IDEs) by adding it to .editorconfig --- .editorconfig | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.editorconfig b/.editorconfig index 8ac06debb5a..8ecaf9b0267 100644 --- a/.editorconfig +++ b/.editorconfig @@ -19,3 +19,7 @@ charset = utf-8 indent_style = space indent_size = 4 trim_trailing_whitespace = true + +# Some SQL results have trailing whitespace which is removed by IDEs +[tests/queries/**.reference] +trim_trailing_whitespace = false From 54bc5d12effe75708774d7a70c7b94296cf3f771 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 28 May 2024 16:40:43 +0200 Subject: [PATCH 045/105] Add randomization of new settings in clickhouse-test prefer_external_sort_block_bytes cross_join_min_rows_to_compress cross_join_min_bytes_to_compress max_parsing_threads min_external_table_block_size_bytes --- tests/clickhouse-test | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index af203563d58..36870d59c3a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -808,10 +808,10 @@ class SettingsRandomizer: "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), "optimize_distinct_in_order": lambda: random.randint(0, 1), "max_bytes_before_external_sort": threshold_generator( - 0.3, 0.5, 1, 10 * 1024 * 1024 * 1024 + 0.3, 0.5, 0, 10 * 1024 * 1024 * 1024 ), "max_bytes_before_external_group_by": threshold_generator( - 0.3, 0.5, 1, 10 * 1024 * 1024 * 1024 + 0.3, 0.5, 0, 10 * 1024 * 1024 * 1024 ), "max_bytes_before_remerge_sort": lambda: random.randint(1, 3000000000), "min_compress_block_size": lambda: random.randint(1, 1048576 * 3), @@ -850,6 +850,11 @@ class SettingsRandomizer: "merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability": lambda: round( random.random(), 2 ), + "prefer_external_sort_block_bytes": lambda: random.choice([0, 1, 100000000]), + "cross_join_min_rows_to_compress": lambda: random.choice([0, 1, 100000000]), + "cross_join_min_bytes_to_compress": lambda: random.choice([0, 1, 100000000]), + "min_external_table_block_size_bytes": lambda: random.choice([0, 1, 100000000]), + "max_parsing_threads": lambda: random.choice([0, 1, 10]), } @staticmethod From d59ccbdb5040f8623600c3c5d2c113adb5c64ba4 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 28 May 2024 18:33:08 +0200 Subject: [PATCH 046/105] stateless tests: Add test for unequal JOINs --- ...006_join_on_inequal_expression_2.reference | 290 ++++++++++++++++++ .../03006_join_on_inequal_expression_2.sql.j2 | 82 +++++ ...006_join_on_inequal_expression_3.reference | 90 ++++++ .../03006_join_on_inequal_expression_3.sql.j2 | 44 +++ ...006_join_on_inequal_expression_4.reference | 123 ++++++++ .../03006_join_on_inequal_expression_4.sql.j2 | 47 +++ 6 files changed, 676 insertions(+) create mode 100644 tests/queries/0_stateless/03006_join_on_inequal_expression_2.reference create mode 100644 tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 create mode 100644 tests/queries/0_stateless/03006_join_on_inequal_expression_3.reference create mode 100644 tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 create mode 100644 tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference create mode 100644 tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_2.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_2.reference new file mode 100644 index 00000000000..bab1fbd050f --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_2.reference @@ -0,0 +1,290 @@ +-- { echoOn } +-- inequality operation +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +-- +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +-- +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +-- BETWEEN +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +1 10 alpha 0 0 +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +1 10 alpha 0 0 +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +0 0 1 5 ALPHA +0 0 4 25 delta +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +0 0 1 5 ALPHA +0 0 4 25 delta +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +0 0 1 5 ALPHA +0 0 4 25 delta +1 10 alpha 0 0 +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +0 0 1 5 ALPHA +0 0 4 25 delta +1 10 alpha 0 0 +2 15 beta 2 10 beta +3 20 gamma 0 0 +-- +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +-- Stupid condition +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 2 10 beta +3 20 gamma 0 0 +-- Window functions with stupid condition +SET join_algorithm='hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA 15 3 +2 15 beta 2 10 beta 15 3 +3 20 gamma 0 0 15 3 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 LEFT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA 15 3 +2 15 beta 2 10 beta 15 3 +3 20 gamma 0 0 15 3 +SET join_algorithm='hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA 12.5 3 +2 15 beta 2 10 beta 12.5 3 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 INNER JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +1 10 alpha 1 5 ALPHA 12.5 3 +2 15 beta 2 10 beta 12.5 3 +SET join_algorithm='hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta 8.333333333333334 7 +1 10 alpha 1 5 ALPHA 8.333333333333334 7 +2 15 beta 2 10 beta 8.333333333333334 7 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 RIGHT JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta 8.333333333333334 7 +1 10 alpha 1 5 ALPHA 8.333333333333334 7 +2 15 beta 2 10 beta 8.333333333333334 7 +SET join_algorithm='hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta 11.25 7 +1 10 alpha 1 5 ALPHA 11.25 7 +2 15 beta 2 10 beta 11.25 7 +3 20 gamma 0 0 11.25 7 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 FULL JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +0 0 4 25 delta 11.25 7 +1 10 alpha 1 5 ALPHA 11.25 7 +2 15 beta 2 10 beta 11.25 7 +3 20 gamma 0 0 11.25 7 +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 new file mode 100644 index 00000000000..f15fced161c --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_2.sql.j2 @@ -0,0 +1,82 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + + +CREATE TABLE t1 ( + key UInt32, + a UInt32, + attr String +) ENGINE = MergeTree ORDER BY key; + +CREATE TABLE t2 ( + key UInt32, + a UInt32, + attr String +) ENGINE = MergeTree ORDER BY key; + +INSERT INTO t1 (key, a, attr) VALUES (1, 10, 'alpha'), (2, 15, 'beta'), (3, 20, 'gamma'); +INSERT INTO t2 (key, a, attr) VALUES (1, 5, 'ALPHA'), (2, 10, 'beta'), (4, 25, 'delta'); + + +SET allow_experimental_analyzer=1; +SET allow_experimental_join_condition=1; +SET join_use_nulls=0; +-- { echoOn } +-- inequality operation +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND t1.a > t2.key AND t1.key + t2.a > 1 ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.key < t2.a OR t1.a % 2 = 0) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- BETWEEN +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t2.a BETWEEN 8 AND t1.a) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a IN (SELECT a FROM t2 WHERE a = 10)) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- Stupid condition +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- Window functions with stupid condition +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.*, AVG(t1.a) OVER () AS avg_b, SUM(t2.key) OVER () AS sum_c FROM t1 {{ join_type }} JOIN t2 ON t1.key == t2.key AND (t1.a * length(t2.attr) / length(t1.attr) <> t2.a + t1.key - t2.key) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_3.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_3.reference new file mode 100644 index 00000000000..1ec6f911897 --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_3.reference @@ -0,0 +1,90 @@ +-- { echoOn } + +-- Support for query lower +SET join_algorithm='hash'; +SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='hash'; +SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='grace_hash'; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +SET join_algorithm='hash'; +SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +0 0 2 10 beta +0 0 4 25 delta +1 10 alpha 1 5 ALPHA +2 15 beta 0 0 +3 20 gamma 0 0 +-- Subquery JOIN +SET join_algorithm='hash'; +SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) LEFT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +1 10 alpha 1 5 ALPHA 1 100 +2 15 beta 0 0 0 0 +3 20 gamma 0 0 0 0 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) LEFT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +1 10 alpha 1 5 ALPHA 1 100 +2 15 beta 0 0 0 0 +3 20 gamma 0 0 0 0 +SET join_algorithm='hash'; +SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) INNER JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +1 10 alpha 1 5 ALPHA 1 100 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) INNER JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +1 10 alpha 1 5 ALPHA 1 100 +SET join_algorithm='hash'; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) RIGHT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +0 0 0 0 0 10 +0 0 0 0 2 1000 +1 10 alpha 1 5 ALPHA 1 100 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) RIGHT JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +0 0 0 0 0 10 +0 0 0 0 2 1000 +1 10 alpha 1 5 ALPHA 1 100 +SET join_algorithm='hash'; +SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) FULL JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +0 0 0 0 0 10 +0 0 0 0 2 1000 +0 0 2 10 beta 0 0 +0 0 4 25 delta 0 0 +1 10 alpha 1 5 ALPHA 1 100 +2 15 beta 0 0 0 0 +3 20 gamma 0 0 0 0 +SET join_algorithm='grace_hash'; +SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) FULL JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +0 0 0 0 0 10 +0 0 0 0 2 1000 +0 0 2 10 beta 0 0 +0 0 4 25 delta 0 0 +1 10 alpha 1 5 ALPHA 1 100 +2 15 beta 0 0 0 0 +3 20 gamma 0 0 0 0 +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 new file mode 100644 index 00000000000..a97153ce3aa --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_3.sql.j2 @@ -0,0 +1,44 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + + +CREATE TABLE t1 ( + key UInt32, + a UInt32, + attr String +) ENGINE = MergeTree ORDER BY key; + +CREATE TABLE t2 ( + key UInt32, + a UInt32, + attr String +) ENGINE = MergeTree ORDER BY key; + +INSERT INTO t1 (key, a, attr) VALUES (1, 10, 'alpha'), (2, 15, 'beta'), (3, 20, 'gamma'); +INSERT INTO t2 (key, a, attr) VALUES (1, 5, 'ALPHA'), (2, 10, 'beta'), (4, 25, 'delta'); + + +SET allow_experimental_analyzer=1; +SET allow_experimental_join_condition=1; +SET join_use_nulls=0; +-- { echoOn } + +-- Support for query lower +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT * FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + + +-- Subquery JOIN +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT * FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.attr != t2.attr) {{ join_type }} JOIN (SELECT * FROM VALUES('key UInt64, a UInt64', (0, 10), (1, 100), (2, 1000))) t3 ON t1.key=t3.key AND t2.key=t3.key AND t3.a!=t1.a AND t3.a!=t2.a ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference new file mode 100644 index 00000000000..f2a5901077a --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference @@ -0,0 +1,123 @@ +-- { echoOn } + +-- These queries work +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 0 0 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 0 0 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 2 10 +0 0 4 25 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 2 10 +0 0 4 25 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 2 10 +0 0 4 25 +1 10 0 0 +2 15 0 0 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 2 10 +0 0 4 25 +1 10 0 0 +2 15 0 0 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 new file mode 100644 index 00000000000..59df0b2abc2 --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 @@ -0,0 +1,47 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + + +CREATE TABLE t1 ( + key UInt32, + a UInt32 +) ENGINE = MergeTree ORDER BY key; + +CREATE TABLE t2 ( + key UInt32, + a UInt32 +) ENGINE = MergeTree ORDER BY key; + +INSERT INTO t1 (key, a) VALUES (1, 10), (2, 15), (3, 20); +INSERT INTO t2 (key, a) VALUES (1, 5), (2, 10), (4, 25); + +SET allow_experimental_analyzer=1; +SET allow_experimental_join_condition=1; +SET join_algorithm='hash'; +-- { echoOn } + +-- These queries work +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND ((t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 AND (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +{% endfor -%} +{% endfor -%} + +-- LOGICAL_ERROR Not-ready Set is passed as the second argument for function 'in' +-- https://github.com/ClickHouse/ClickHouse/issues/64513 +-- SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; From 103a448e710fc2fa406574b9d29e93e4a0665733 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 29 May 2024 12:11:00 +0200 Subject: [PATCH 047/105] stateless tests: complete parallel_parsing test --- .../0_stateless/01701_parallel_parsing_infinite_segmentation.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01701_parallel_parsing_infinite_segmentation.sh b/tests/queries/0_stateless/01701_parallel_parsing_infinite_segmentation.sh index 9284348dd62..d2dcd501428 100755 --- a/tests/queries/0_stateless/01701_parallel_parsing_infinite_segmentation.sh +++ b/tests/queries/0_stateless/01701_parallel_parsing_infinite_segmentation.sh @@ -7,6 +7,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -q "create table insert_big_json(a String, b String) engine=MergeTree() order by tuple()"; -python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --min_chunk_bytes_for_parallel_parsing=10485760 --max_threads=0 --input_format_parallel_parsing=1 --max_memory_usage=0 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||: +python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --min_chunk_bytes_for_parallel_parsing=10485760 --max_threads=0 --input_format_parallel_parsing=1 --max_memory_usage=0 --max_parsing_threads=2 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||: ${CLICKHOUSE_CLIENT} -q "drop table insert_big_json" From e6bf468e563f45cf228ed98c3ce872de205aebff Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 29 May 2024 12:11:15 +0200 Subject: [PATCH 048/105] stateless tests: add missing order by --- tests/queries/0_stateless/00576_nested_and_prewhere.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00576_nested_and_prewhere.sql b/tests/queries/0_stateless/00576_nested_and_prewhere.sql index 5916e679f1e..f5d9f0d6240 100644 --- a/tests/queries/0_stateless/00576_nested_and_prewhere.sql +++ b/tests/queries/0_stateless/00576_nested_and_prewhere.sql @@ -4,10 +4,10 @@ CREATE TABLE nested (x UInt64, filter UInt8, n Nested(a UInt64)) ENGINE = MergeT INSERT INTO nested SELECT number, number % 2, range(number % 10) FROM system.numbers LIMIT 100000; ALTER TABLE nested ADD COLUMN n.b Array(UInt64); -SELECT DISTINCT n.b FROM nested PREWHERE filter; +SELECT DISTINCT n.b FROM nested PREWHERE filter ORDER BY ALL; ALTER TABLE nested ADD COLUMN n.c Array(UInt64) DEFAULT arrayMap(x -> x * 2, n.a); -SELECT DISTINCT n.c FROM nested PREWHERE filter; -SELECT DISTINCT n.a, n.c FROM nested PREWHERE filter; +SELECT DISTINCT n.c FROM nested PREWHERE filter ORDER BY ALL; +SELECT DISTINCT n.a, n.c FROM nested PREWHERE filter ORDER BY ALL; DROP TABLE nested; From 9ce824d41c2323f6556dd6f4015d9ec99272083c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 6 Jun 2024 11:04:38 +0200 Subject: [PATCH 049/105] Addressed review comments --- .../IO/WriteBufferFromAzureBlobStorage.cpp | 27 +++++++++---------- 1 file changed, 12 insertions(+), 15 deletions(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 16bd70cf2e5..a2d21cf49c2 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -132,23 +132,21 @@ void WriteBufferFromAzureBlobStorage::preFinalize() /// If there is only one block and size is less than or equal to max_single_part_upload_size /// then we use single part upload instead of multi part upload - if (detached_part_data.size() == 1 && block_ids.empty()) + if (block_ids.empty() && detached_part_data.size() == 1 && detached_part_data.front().data_size <= max_single_part_upload_size) { - if (detached_part_data.front().data_size <= max_single_part_upload_size) - { - auto part_data = std::move(detached_part_data.front()); - auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); - Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(part_data.memory.data()), part_data.data_size); - execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, part_data.data_size); - LOG_TRACE(log, "Committed single block for blob `{}`", blob_path); + auto part_data = std::move(detached_part_data.front()); + auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); + Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(part_data.memory.data()), part_data.data_size); + execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, part_data.data_size); + LOG_TRACE(log, "Committed single block for blob `{}`", blob_path); - detached_part_data.pop_front(); - block_ids.clear(); - return; - } + detached_part_data.pop_front(); + return; + } + else + { + writeMultipartUpload(); } - - writeMultipartUpload(); } void WriteBufferFromAzureBlobStorage::finalizeImpl() @@ -295,4 +293,3 @@ void WriteBufferFromAzureBlobStorage::writeMultipartUpload() } #endif - From 1d3cf170534b1ae8382397d8d2672868e237f33f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 6 Jun 2024 11:11:08 +0200 Subject: [PATCH 050/105] Fix global trace collector --- programs/server/Server.cpp | 99 +++++++++++++++-------------- src/Common/QueryProfiler.cpp | 6 +- src/Interpreters/Context.cpp | 17 ++++- src/Interpreters/Context.h | 2 + src/Interpreters/TraceCollector.cpp | 30 +++++++-- src/Interpreters/TraceCollector.h | 11 +++- 6 files changed, 105 insertions(+), 60 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8fcb9d87a93..f47cd53f9e9 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -773,7 +773,51 @@ try LOG_INFO(log, "Available CPU instruction sets: {}", cpu_info); #endif - bool will_have_trace_collector = hasPHDRCache() && config().has("trace_log"); + bool has_trace_collector = false; + /// Disable it if we collect test coverage information, because it will work extremely slow. +#if !WITH_COVERAGE + /// Profilers cannot work reliably with any other libunwind or without PHDR cache. + if (hasPHDRCache() && config().has("trace_log")) + { + has_trace_collector = true; + + /// Set up server-wide memory profiler (for total memory tracker). + if (server_settings.total_memory_profiler_step) + { + total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); + } + + if (server_settings.total_memory_tracker_sample_probability > 0.0) + { + total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); + } + + if (server_settings.total_memory_profiler_sample_min_allocation_size) + { + total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); + } + + if (server_settings.total_memory_profiler_sample_max_allocation_size) + { + total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); + } + } +#endif + + /// Describe multiple reasons when query profiler cannot work. + +#if WITH_COVERAGE + LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage."); +#endif + +#if defined(SANITIZER) + LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" + " when two different stack unwinding methods will interfere with each other."); +#endif + + if (!hasPHDRCache()) + LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" + " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); // Initialize global thread pool. Do it before we fetch configs from zookeeper // nodes (`from_zk`), because ZooKeeper interface uses the pool. We will @@ -782,8 +826,12 @@ try server_settings.max_thread_pool_size, server_settings.max_thread_pool_free_size, server_settings.thread_pool_queue_size, - will_have_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0, - will_have_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); + has_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0, + has_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); + + if (has_trace_collector) + global_context->createTraceCollector(); + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; @@ -1950,52 +1998,9 @@ try LOG_DEBUG(log, "Loaded metadata."); - /// Init trace collector only after trace_log system table was created - /// Disable it if we collect test coverage information, because it will work extremely slow. -#if !WITH_COVERAGE - /// Profilers cannot work reliably with any other libunwind or without PHDR cache. - if (hasPHDRCache()) - { + if (has_trace_collector) global_context->initializeTraceCollector(); - /// Set up server-wide memory profiler (for total memory tracker). - if (server_settings.total_memory_profiler_step) - { - total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); - } - - if (server_settings.total_memory_tracker_sample_probability > 0.0) - { - total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); - } - - if (server_settings.total_memory_profiler_sample_min_allocation_size) - { - total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); - } - - if (server_settings.total_memory_profiler_sample_max_allocation_size) - { - total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); - } - } -#endif - - /// Describe multiple reasons when query profiler cannot work. - -#if WITH_COVERAGE - LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage."); -#endif - -#if defined(SANITIZER) - LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" - " when two different stack unwinding methods will interfere with each other."); -#endif - - if (!hasPHDRCache()) - LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" - " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); - #if defined(OS_LINUX) auto tasks_stats_provider = TasksStatsCounters::findBestAvailableProvider(); if (tasks_stats_provider == TasksStatsCounters::MetricsProvider::None) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index c3affbdd968..746010b5462 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -228,9 +228,9 @@ void Timer::cleanup() #endif template -QueryProfilerBase::QueryProfilerBase([[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt32 period, [[maybe_unused]] int pause_signal_) - : log(getLogger("QueryProfiler")) - , pause_signal(pause_signal_) +QueryProfilerBase::QueryProfilerBase( + [[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt32 period, [[maybe_unused]] int pause_signal_) + : log(getLogger("QueryProfiler")), pause_signal(pause_signal_) { #if defined(SANITIZER) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "QueryProfiler disabled because they cannot work under sanitizers"); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 06b3adb328d..b1fb6a68618 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -740,12 +740,18 @@ struct ContextSharedPart : boost::noncopyable void initializeTraceCollector(std::shared_ptr trace_log) { - if (!trace_log) - return; + if (!trace_collector.has_value()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "TraceCollector needs to be first created before initialization"); + + trace_collector->initialize(trace_log); + } + + void createTraceCollector() + { if (hasTraceCollector()) return; - trace_collector.emplace(std::move(trace_log)); + trace_collector.emplace(); } void addWarningMessage(const String & message) TSA_REQUIRES(mutex) @@ -3891,6 +3897,11 @@ void Context::initializeSystemLogs() }); } +void Context::createTraceCollector() +{ + shared->createTraceCollector(); +} + void Context::initializeTraceCollector() { shared->initializeTraceCollector(getTraceLog()); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 87a7baa0469..68f37377926 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1077,6 +1077,8 @@ public: void initializeSystemLogs(); /// Call after initialization before using trace collector. + void createTraceCollector(); + void initializeTraceCollector(); /// Call after unexpected crash happen. diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index 8e9c397b7a1..77f70d754c8 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -1,5 +1,4 @@ -#include "TraceCollector.h" - +#include #include #include #include @@ -14,8 +13,12 @@ namespace DB { -TraceCollector::TraceCollector(std::shared_ptr trace_log_) - : trace_log(std::move(trace_log_)) +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +TraceCollector::TraceCollector() { TraceSender::pipe.open(); @@ -28,6 +31,23 @@ TraceCollector::TraceCollector(std::shared_ptr trace_log_) thread = ThreadFromGlobalPool(&TraceCollector::run, this); } +void TraceCollector::initialize(std::shared_ptr trace_log_) +{ + if (is_trace_log_initialized) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "TraceCollector is already initialized"); + + trace_log_ptr = trace_log_; + is_trace_log_initialized.store(true, std::memory_order_release); +} + +std::shared_ptr TraceCollector::getTraceLog() +{ + if (!is_trace_log_initialized.load(std::memory_order_acquire)) + return nullptr; + + return trace_log_ptr; +} + void TraceCollector::tryClosePipe() { try @@ -120,7 +140,7 @@ void TraceCollector::run() ProfileEvents::Count increment; readPODBinary(increment, in); - if (trace_log) + if (auto trace_log = getTraceLog()) { // time and time_in_microseconds are both being constructed from the same timespec so that the // times will be equal up to the precision of a second. diff --git a/src/Interpreters/TraceCollector.h b/src/Interpreters/TraceCollector.h index 382e7511ac6..db1ce286b0e 100644 --- a/src/Interpreters/TraceCollector.h +++ b/src/Interpreters/TraceCollector.h @@ -1,4 +1,5 @@ #pragma once +#include #include class StackTrace; @@ -16,11 +17,17 @@ class TraceLog; class TraceCollector { public: - explicit TraceCollector(std::shared_ptr trace_log_); + explicit TraceCollector(); ~TraceCollector(); + void initialize(std::shared_ptr trace_log_); + private: - std::shared_ptr trace_log; + std::shared_ptr getTraceLog(); + + std::atomic is_trace_log_initialized = false; + std::shared_ptr trace_log_ptr; + ThreadFromGlobalPool thread; void tryClosePipe(); From fcee260b259b7750f18389e26debc90de22017a5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 6 Jun 2024 11:22:48 +0200 Subject: [PATCH 051/105] Update src/Interpreters/TraceCollector.h Co-authored-by: alesapin --- src/Interpreters/TraceCollector.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/TraceCollector.h b/src/Interpreters/TraceCollector.h index db1ce286b0e..c2894394dd0 100644 --- a/src/Interpreters/TraceCollector.h +++ b/src/Interpreters/TraceCollector.h @@ -17,7 +17,7 @@ class TraceLog; class TraceCollector { public: - explicit TraceCollector(); + TraceCollector(); ~TraceCollector(); void initialize(std::shared_ptr trace_log_); From bfb1c4c79396bb234b63a0a963fc643af292956d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 6 Jun 2024 11:29:42 +0200 Subject: [PATCH 052/105] better --- programs/server/Server.cpp | 41 +++++++++++++++----------------------- 1 file changed, 16 insertions(+), 25 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f47cd53f9e9..84ff0e94cef 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -777,31 +777,7 @@ try /// Disable it if we collect test coverage information, because it will work extremely slow. #if !WITH_COVERAGE /// Profilers cannot work reliably with any other libunwind or without PHDR cache. - if (hasPHDRCache() && config().has("trace_log")) - { - has_trace_collector = true; - - /// Set up server-wide memory profiler (for total memory tracker). - if (server_settings.total_memory_profiler_step) - { - total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); - } - - if (server_settings.total_memory_tracker_sample_probability > 0.0) - { - total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); - } - - if (server_settings.total_memory_profiler_sample_min_allocation_size) - { - total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); - } - - if (server_settings.total_memory_profiler_sample_max_allocation_size) - { - total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); - } - } + has_trace_collector = hasPHDRCache() && config().has("trace_log"); #endif /// Describe multiple reasons when query profiler cannot work. @@ -830,8 +806,23 @@ try has_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); if (has_trace_collector) + { global_context->createTraceCollector(); + /// Set up server-wide memory profiler (for total memory tracker). + if (server_settings.total_memory_profiler_step) + total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); + + if (server_settings.total_memory_tracker_sample_probability > 0.0) + total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); + + if (server_settings.total_memory_profiler_sample_min_allocation_size) + total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); + + if (server_settings.total_memory_profiler_sample_max_allocation_size) + total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); + } + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; From 2f50e070efcceae8eba19287443e96ecdbfaf6b3 Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Thu, 6 Jun 2024 10:34:59 +0000 Subject: [PATCH 053/105] Scale argument restricted to types (U)Int8 - (U)Int64 --- src/Functions/FunctionsRound.h | 22 +++++++++---------- .../03165_round_scale_as_column.sql | 1 + 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index a05619b54bb..1cc0b410514 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -626,7 +626,7 @@ public: }; /// Functions that round the value of an input parameter of type (U)Int8/16/32/64, Float32/64 or Decimal32/64/128. -/// Accept an additional optional parameter (0 by default). +/// Accept an additional optional parameter of type (U)Int8/16/32/64 (0 by default). template class FunctionRounding : public IFunction { @@ -640,19 +640,17 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if ((arguments.empty()) || (arguments.size() > 2)) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 1 or 2.", - getName(), arguments.size()); + FunctionArgumentDescriptors mandatory_args{ + {"x", static_cast(&isNumber), nullptr, "A number to round"}, + }; + FunctionArgumentDescriptors optional_args{ + {"N", static_cast(&isNativeInteger), nullptr, "The number of decimal places to round to"}, + }; + validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); - for (const auto & type : arguments) - if (!isNumber(type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", - arguments[0]->getName(), getName()); - - return arguments[0]; + return arguments[0].type; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override diff --git a/tests/queries/0_stateless/03165_round_scale_as_column.sql b/tests/queries/0_stateless/03165_round_scale_as_column.sql index 2304d2730a9..229f705808d 100644 --- a/tests/queries/0_stateless/03165_round_scale_as_column.sql +++ b/tests/queries/0_stateless/03165_round_scale_as_column.sql @@ -120,3 +120,4 @@ SELECT round(1, 1); SELECT round(materialize(1), materialize(1)); SELECT round(1, materialize(1)); --{serverError ILLEGAL_COLUMN} SELECT round(materialize(1), 1); +SELECT materialize(10.1) AS x, ceil(x, toUInt256(123)); --{serverError ILLEGAL_TYPE_OF_ARGUMENT} From 54ed3354f6be7aab304f55e253d8175a03d9a2b0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jun 2024 12:56:43 +0200 Subject: [PATCH 054/105] Better debugging --- ...23_zeros_generate_random_with_limit_progress_bar.sh | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh index 4bb8fc8880d..8aedf0bc0ff 100755 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-random-settings CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -6,11 +7,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function run_with_progress_and_match_total_rows() { - echo "$1" | \ - ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1 | \ - grep -q '"total_rows_to_read":"100"' && echo "Matched" || echo "Expected total_rows_to_read not found" + CURL_RESPONSE=$(echo "$1" | \ + ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1) + + echo "$CURL_RESPONSE" | grep -q '"total_rows_to_read":"100"' && echo "Matched" || echo "Expected total_rows_to_read not found: ${CURL_RESPONSE}" } run_with_progress_and_match_total_rows 'SELECT * FROM system.zeros LIMIT 100' run_with_progress_and_match_total_rows 'SELECT * FROM system.zeros_mt LIMIT 100' -run_with_progress_and_match_total_rows 'SELECT * FROM generateRandom() LIMIT 100' +run_with_progress_and_match_total_rows "SELECT * FROM generateRandom('number UInt64') LIMIT 100" From f0c21da1d8ed3327d4eb53f1e30bbc5b63c2e950 Mon Sep 17 00:00:00 2001 From: Mikhail Gorshkov Date: Thu, 6 Jun 2024 11:06:16 +0000 Subject: [PATCH 055/105] Style fixed --- src/Functions/FunctionsRound.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 1cc0b410514..08e257de8ac 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -31,7 +31,6 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ARGUMENT_OUT_OF_BOUND; extern const int ILLEGAL_COLUMN; From 407b86012c2f7c768202f0af0b54cd5b0426adbd Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 6 Jun 2024 13:23:48 +0200 Subject: [PATCH 056/105] stateless tests: add case for unequal joins --- ...006_join_on_inequal_expression_4.reference | 40 +++++++++++++++++++ .../03006_join_on_inequal_expression_4.sql.j2 | 10 +++-- 2 files changed, 47 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference index f2a5901077a..0c747d04c58 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.reference @@ -121,3 +121,43 @@ SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a = 1 10 0 0 2 15 2 10 3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +2 15 2 10 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +0 0 1 5 +0 0 4 25 +1 10 0 0 +2 15 2 10 +3 20 0 0 diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 index 59df0b2abc2..3235019821b 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_4.sql.j2 @@ -42,6 +42,10 @@ SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 {% endfor -%} {% endfor -%} --- LOGICAL_ERROR Not-ready Set is passed as the second argument for function 'in' --- https://github.com/ClickHouse/ClickHouse/issues/64513 --- SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; + +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2 OR (t2.a IN (SELECT a FROM t1 WHERE a = 10))) ORDER BY ALL; +{% endfor -%} +{% endfor -%} From a7230e3c6c3c35ff56836ed799f3f15bd037f8a0 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Mon, 27 May 2024 10:16:28 -0700 Subject: [PATCH 057/105] Add settings to allow parallel replicas custom key with range filter to use a custom range --- docs/en/operations/settings/settings.md | 17 ++++++ src/Core/Settings.h | 2 + src/Interpreters/InterpreterSelectQuery.cpp | 4 +- .../getCustomKeyFilterForParallelReplicas.cpp | 43 +++++++++----- .../getCustomKeyFilterForParallelReplicas.h | 10 +++- src/Planner/PlannerJoinTree.cpp | 4 +- src/Storages/StorageDistributed.cpp | 4 +- ...el_replicas_range_filter_min_max.reference | 7 +++ ...parallel_replicas_range_filter_min_max.sql | 57 +++++++++++++++++++ 9 files changed, 129 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference create mode 100644 tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ffaf53085c4..0dbd349525d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1590,6 +1590,23 @@ Possible values: Default value: `default`. +## parallel_replicas_custom_key_range_lower {#parallel_replicas_custom_key_range_lower} + +Allows the filter type `range` to split the work evenly between replicas based the custom range `[parallel_replicas_custom_key_range_lower, INT_MAX]`. + +When used in conjuction with [parallel_replicas_custom_key_range_upper](#parallel_replicas_custom_key_range_upper), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. + +Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` when parallelizing the query. + +## parallel_replicas_custom_key_range_upper {#parallel_replicas_custom_key_range_upper} + + +Allows the filter type `range` to split the work evenly between replicas based the custom range `[0, parallel_replicas_custom_key_range_upper]`. + +When used in conjuction with [parallel_replicas_custom_key_range_lower](#parallel_replicas_custom_key_range_lower), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. + +Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` when parallelizing the query. + ## allow_experimental_parallel_reading_from_replicas Enables or disables sending SELECT queries to all replicas of a table (up to `max_parallel_replicas`). Reading is parallelized and coordinated dynamically. It will work for any kind of MergeTree table. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 27ce54c03a7..d6e4f0ae92b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -202,6 +202,8 @@ class IColumn; M(UInt64, parallel_replica_offset, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas.", 0) \ M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \ M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ + M(UInt64, parallel_replicas_custom_key_range_lower, 0, "Lower bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ + M(UInt64, parallel_replicas_custom_key_range_upper, std::numeric_limits::max(), "Upper bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ \ M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \ M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 09f987a1c24..a418a4c9729 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -578,7 +578,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( settings.parallel_replicas_count, settings.parallel_replica_offset, std::move(custom_key_ast), - settings.parallel_replicas_custom_key_filter_type, + {settings.parallel_replicas_custom_key_filter_type, + settings.parallel_replicas_custom_key_range_lower, + settings.parallel_replicas_custom_key_range_upper}, storage->getInMemoryMetadataPtr()->columns, context); } diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index d78b6ab0c4d..31669efb698 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -7,7 +7,6 @@ #include -#include #include @@ -18,18 +17,19 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; + extern const int INVALID_SETTING_VALUE; } ASTPtr getCustomKeyFilterForParallelReplica( size_t replicas_count, size_t replica_num, ASTPtr custom_key_ast, - ParallelReplicasCustomKeyFilterType filter_type, + ParallelReplicasCustomKeyFilter filter, const ColumnsDescription & columns, const ContextPtr & context) { chassert(replicas_count > 1); - if (filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT) + if (filter.filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT) { // first we do modulo with replica count auto modulo_function = makeASTFunction("positiveModulo", custom_key_ast, std::make_shared(replicas_count)); @@ -40,35 +40,48 @@ ASTPtr getCustomKeyFilterForParallelReplica( return equals_function; } - assert(filter_type == ParallelReplicasCustomKeyFilterType::RANGE); + assert(filter.filter_type == ParallelReplicasCustomKeyFilterType::RANGE); KeyDescription custom_key_description = KeyDescription::getKeyFromAST(custom_key_ast, columns, context); using RelativeSize = boost::rational; - RelativeSize size_of_universum = 0; + // get + RelativeSize range_upper = RelativeSize(filter.range_upper) + RelativeSize(1); + RelativeSize range_lower = RelativeSize(filter.range_lower); DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); if (custom_key_description.data_types.size() == 1) { if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); } - if (size_of_universum == RelativeSize(0)) + if (range_upper == RelativeSize(0)) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid custom key column type: {}. Must be one unsigned integer type", custom_key_column_type->getName()); + if (range_lower < 0) + throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range Min must be a postive"); + + if (range_lower > range_upper) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Invalid custom key filter range: Range max {} must be larger than range min {}", + range_lower, + range_upper); + + RelativeSize size_of_universum = range_upper - range_lower; + RelativeSize relative_range_size = RelativeSize(1) / replicas_count; RelativeSize relative_range_offset = relative_range_size * RelativeSize(replica_num); @@ -76,16 +89,16 @@ ASTPtr getCustomKeyFilterForParallelReplica( bool has_lower_limit = false; bool has_upper_limit = false; - RelativeSize lower_limit_rational = relative_range_offset * size_of_universum; - RelativeSize upper_limit_rational = (relative_range_offset + relative_range_size) * size_of_universum; + RelativeSize lower_limit_rational = range_lower + relative_range_offset * size_of_universum; + RelativeSize upper_limit_rational = range_lower + (relative_range_offset + relative_range_size) * size_of_universum; UInt64 lower = boost::rational_cast(lower_limit_rational); UInt64 upper = boost::rational_cast(upper_limit_rational); - if (lower > 0) + if (lower > range_lower) has_lower_limit = true; - if (upper_limit_rational < size_of_universum) + if (upper < range_upper) has_upper_limit = true; assert(has_lower_limit || has_upper_limit); diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h index 1506c1992c0..36198be8e51 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h @@ -6,16 +6,24 @@ #include #include #include +#include namespace DB { +struct ParallelReplicasCustomKeyFilter +{ + ParallelReplicasCustomKeyFilterType filter_type; + UInt64 range_lower; + UInt64 range_upper; +}; + /// Get AST for filter created from custom_key /// replica_num is the number of the replica for which we are generating filter starting from 0 ASTPtr getCustomKeyFilterForParallelReplica( size_t replicas_count, size_t replica_num, ASTPtr custom_key_ast, - ParallelReplicasCustomKeyFilterType filter_type, + ParallelReplicasCustomKeyFilter filter, const ColumnsDescription & columns, const ContextPtr & context); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 83b6f4f2c26..b092f3d0c26 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -501,7 +501,9 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, settings.parallel_replicas_count, settings.parallel_replica_offset, std::move(custom_key_ast), - settings.parallel_replicas_custom_key_filter_type, + {settings.parallel_replicas_custom_key_filter_type, + settings.parallel_replicas_custom_key_range_lower, + settings.parallel_replicas_custom_key_range_upper}, storage->getInMemoryMetadataPtr()->columns, query_context); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 9c58468c4a4..5048ef4788e 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -904,11 +904,13 @@ void StorageDistributed::read( [my_custom_key_ast = std::move(custom_key_ast), column_description = this->getInMemoryMetadataPtr()->columns, custom_key_type = settings.parallel_replicas_custom_key_filter_type.value, + custom_key_range_lower = settings.parallel_replicas_custom_key_range_lower.value, + custom_key_range_upper = settings.parallel_replicas_custom_key_range_upper.value, context = local_context, replica_count = modified_query_info.getCluster()->getShardsInfo().front().per_replica_pools.size()](uint64_t replica_num) -> ASTPtr { return getCustomKeyFilterForParallelReplica( - replica_count, replica_num - 1, my_custom_key_ast, custom_key_type, column_description, context); + replica_count, replica_num - 1, my_custom_key_ast, {custom_key_type, custom_key_range_lower, custom_key_range_upper}, column_description, context); }; } } diff --git a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference new file mode 100644 index 00000000000..5ba3f6bc471 --- /dev/null +++ b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference @@ -0,0 +1,7 @@ +10 +10 +10 +10 +10 +10 +10 diff --git a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql new file mode 100644 index 00000000000..c9588d931a8 --- /dev/null +++ b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql @@ -0,0 +1,57 @@ +DROP TABLE IF EXISTS range_filter_custom_range_test; + +CREATE TABLE range_filter_custom_range_test (k Int64) ENGINE=MergeTree ORDER BY k; + +INSERT INTO range_filter_custom_range_test SELECT number + 5 from numbers(10); + +SELECT count() FROM +(SELECT * +FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) +SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, +parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', +parallel_replicas_custom_key_range_lower = 5, parallel_replicas_custom_key_range_upper=15); + +SELECT count() FROM +(SELECT * +FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) +SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, +parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', +parallel_replicas_custom_key_range_lower = 4, parallel_replicas_custom_key_range_upper=14); + +SELECT count() FROM +(SELECT * +FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) +SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, +parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', +parallel_replicas_custom_key_range_lower = 6, parallel_replicas_custom_key_range_upper=17); + + +SELECT count() FROM +(SELECT * +FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) +SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, +parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', +parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper=15); + +SELECT count() FROM +(SELECT * +FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) +SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, +parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', +parallel_replicas_custom_key_range_lower = 15, parallel_replicas_custom_key_range_upper=25); + +SELECT count() FROM +(SELECT * +FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) +SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, +parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', +parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper=5); + +SELECT count() FROM +(SELECT * +FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) +SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, +parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', +parallel_replicas_custom_key_range_lower = 500, parallel_replicas_custom_key_range_upper=10000); + +DROP TABLE range_filter_custom_range_test; \ No newline at end of file From d455116b7c9e363dc6873221dd28c91bc6bf5f1e Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 29 May 2024 15:20:18 -0700 Subject: [PATCH 058/105] fix style --- docs/en/operations/settings/settings.md | 4 ++-- src/Interpreters/InterpreterSelectQuery.cpp | 6 +++--- .../getCustomKeyFilterForParallelReplicas.cpp | 5 ++--- .../getCustomKeyFilterForParallelReplicas.h | 4 ++-- src/Planner/PlannerJoinTree.cpp | 16 ++++++++-------- 5 files changed, 17 insertions(+), 18 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 0dbd349525d..ff0da9eaa12 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1596,7 +1596,7 @@ Allows the filter type `range` to split the work evenly between replicas based t When used in conjuction with [parallel_replicas_custom_key_range_upper](#parallel_replicas_custom_key_range_upper), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. -Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` when parallelizing the query. +Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing. ## parallel_replicas_custom_key_range_upper {#parallel_replicas_custom_key_range_upper} @@ -1605,7 +1605,7 @@ Allows the filter type `range` to split the work evenly between replicas based t When used in conjuction with [parallel_replicas_custom_key_range_lower](#parallel_replicas_custom_key_range_lower), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. -Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` when parallelizing the query. +Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing. ## allow_experimental_parallel_reading_from_replicas diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a418a4c9729..b72399df2c1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -578,9 +578,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( settings.parallel_replicas_count, settings.parallel_replica_offset, std::move(custom_key_ast), - {settings.parallel_replicas_custom_key_filter_type, - settings.parallel_replicas_custom_key_range_lower, - settings.parallel_replicas_custom_key_range_upper}, + {settings.parallel_replicas_custom_key_filter_type, + settings.parallel_replicas_custom_key_range_lower, + settings.parallel_replicas_custom_key_range_upper}, storage->getInMemoryMetadataPtr()->columns, context); } diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index 31669efb698..6e45e806019 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -47,7 +47,6 @@ ASTPtr getCustomKeyFilterForParallelReplica( using RelativeSize = boost::rational; - // get RelativeSize range_upper = RelativeSize(filter.range_upper) + RelativeSize(1); RelativeSize range_lower = RelativeSize(filter.range_lower); DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; @@ -71,12 +70,12 @@ ASTPtr getCustomKeyFilterForParallelReplica( custom_key_column_type->getName()); if (range_lower < 0) - throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range Min must be a postive"); + throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range lower bound must be a positive"); if (range_lower > range_upper) throw Exception( ErrorCodes::INVALID_SETTING_VALUE, - "Invalid custom key filter range: Range max {} must be larger than range min {}", + "Invalid custom key filter range: Range upper bound {} must be larger than range lower bound {}", range_lower, range_upper); diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h index 36198be8e51..dfee5123ecb 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h @@ -14,8 +14,8 @@ struct ParallelReplicasCustomKeyFilter { ParallelReplicasCustomKeyFilterType filter_type; UInt64 range_lower; - UInt64 range_upper; -}; + UInt64 range_upper; +}; /// Get AST for filter created from custom_key /// replica_num is the number of the replica for which we are generating filter starting from 0 diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index b092f3d0c26..efc449402b9 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -498,14 +498,14 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, LOG_TRACE(getLogger("Planner"), "Processing query on a replica using custom_key '{}'", settings.parallel_replicas_custom_key.value); auto parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica( - settings.parallel_replicas_count, - settings.parallel_replica_offset, - std::move(custom_key_ast), - {settings.parallel_replicas_custom_key_filter_type, - settings.parallel_replicas_custom_key_range_lower, - settings.parallel_replicas_custom_key_range_upper}, - storage->getInMemoryMetadataPtr()->columns, - query_context); + settings.parallel_replicas_count, + settings.parallel_replica_offset, + std::move(custom_key_ast), + {settings.parallel_replicas_custom_key_filter_type, + settings.parallel_replicas_custom_key_range_lower, + settings.parallel_replicas_custom_key_range_upper}, + storage->getInMemoryMetadataPtr()->columns, + query_context); return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info.table_expression, planner_context); } From 9ed52af56f51b24a9754a2d4d42060f5e9586712 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 29 May 2024 17:10:08 -0700 Subject: [PATCH 059/105] Add new settings to SettingsChangesHistory.h --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d6e4f0ae92b..092cbe35503 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -203,7 +203,7 @@ class IColumn; M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \ M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ M(UInt64, parallel_replicas_custom_key_range_lower, 0, "Lower bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ - M(UInt64, parallel_replicas_custom_key_range_upper, std::numeric_limits::max(), "Upper bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ + M(UInt64, parallel_replicas_custom_key_range_upper, std::numeric_limits::max(), "Upper bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ \ M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \ M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 814c08c5705..144cb9100e5 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -101,6 +101,8 @@ static const std::map Date: Thu, 30 May 2024 06:31:06 -0700 Subject: [PATCH 060/105] Fix issue with rational cast --- src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index 6e45e806019..3cd8f3564a6 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -94,10 +94,10 @@ ASTPtr getCustomKeyFilterForParallelReplica( UInt64 lower = boost::rational_cast(lower_limit_rational); UInt64 upper = boost::rational_cast(upper_limit_rational); - if (lower > range_lower) + if (lower_limit_rational > range_lower) has_lower_limit = true; - if (upper < range_upper) + if (upper_limit_rational < range_upper) has_upper_limit = true; assert(has_lower_limit || has_upper_limit); From 0302f218e43fa0a6b8ba8c0696ec25169c508d47 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Thu, 30 May 2024 07:07:59 -0700 Subject: [PATCH 061/105] Add additional check to ensure range is larger than the number of replicas --- src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index 3cd8f3564a6..a69d6cd6d21 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -72,7 +72,7 @@ ASTPtr getCustomKeyFilterForParallelReplica( if (range_lower < 0) throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range lower bound must be a positive"); - if (range_lower > range_upper) + if (range_lower >= range_upper) throw Exception( ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range upper bound {} must be larger than range lower bound {}", @@ -81,6 +81,10 @@ ASTPtr getCustomKeyFilterForParallelReplica( RelativeSize size_of_universum = range_upper - range_lower; + if (size_of_universum <= RelativeSize(replicas_count)) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range must be larger than than the number of replicas"); + RelativeSize relative_range_size = RelativeSize(1) / replicas_count; RelativeSize relative_range_offset = relative_range_size * RelativeSize(replica_num); From 677b2de5ed47f7940f100f9b53e171caa035899a Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Mon, 3 Jun 2024 08:43:09 -0400 Subject: [PATCH 062/105] Update docs/en/operations/settings/settings.md Co-authored-by: Antonio Andelic --- docs/en/operations/settings/settings.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ff0da9eaa12..6c94de3ac40 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1600,7 +1600,6 @@ Note: This setting will not cause any additional data to be filtered during quer ## parallel_replicas_custom_key_range_upper {#parallel_replicas_custom_key_range_upper} - Allows the filter type `range` to split the work evenly between replicas based the custom range `[0, parallel_replicas_custom_key_range_upper]`. When used in conjuction with [parallel_replicas_custom_key_range_lower](#parallel_replicas_custom_key_range_lower), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. From bde8d19f45f64492167c3b57e303b8d610a7f4aa Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Mon, 3 Jun 2024 10:12:24 -0700 Subject: [PATCH 063/105] Address review comments --- src/Core/Settings.h | 2 +- .../getCustomKeyFilterForParallelReplicas.cpp | 46 +++++++++++++++---- 2 files changed, 39 insertions(+), 9 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 092cbe35503..43879485f23 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -203,7 +203,7 @@ class IColumn; M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \ M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ M(UInt64, parallel_replicas_custom_key_range_lower, 0, "Lower bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ - M(UInt64, parallel_replicas_custom_key_range_upper, std::numeric_limits::max(), "Upper bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ + M(UInt64, parallel_replicas_custom_key_range_upper, 0, "Upper bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ \ M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \ M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index a69d6cd6d21..065460abda4 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -47,20 +47,53 @@ ASTPtr getCustomKeyFilterForParallelReplica( using RelativeSize = boost::rational; - RelativeSize range_upper = RelativeSize(filter.range_upper) + RelativeSize(1); + RelativeSize range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1) + : RelativeSize(std::numeric_limits::max()) + RelativeSize(1); RelativeSize range_lower = RelativeSize(filter.range_lower); DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; if (custom_key_description.data_types.size() == 1) { if (typeid_cast(custom_key_column_type.get())) - range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); + { + range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1) + : RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + if (range_upper > RelativeSize(std::numeric_limits::max()) + RelativeSize(1)) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Invalid custom key range upper bound: {}. Value must be smaller than custom key column type (UInt64) max value", + range_upper); + } else if (typeid_cast(custom_key_column_type.get())) - range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); + { + range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1) + : RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + if (range_upper > RelativeSize(std::numeric_limits::max()) + RelativeSize(1)) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Invalid custom key range upper bound: {}. Value must be smaller than custom key column type (UInt32) max value", + range_upper); + } else if (typeid_cast(custom_key_column_type.get())) - range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); + { + range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1) + : RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + if (range_upper > RelativeSize(std::numeric_limits::max()) + RelativeSize(1)) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Invalid custom key range upper bound: {}. Value must be smaller than custom key column type (UInt16) max value", + range_upper); + } else if (typeid_cast(custom_key_column_type.get())) - range_upper = std::min(RelativeSize(std::numeric_limits::max()) + RelativeSize(1), range_upper); + { + range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1) + : RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + if (range_upper > RelativeSize(std::numeric_limits::max()) + RelativeSize(1)) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Invalid custom key range upper bound: {}. Value must be smaller than custom key column type (UInt8) max value", + range_upper); + } } if (range_upper == RelativeSize(0)) @@ -69,9 +102,6 @@ ASTPtr getCustomKeyFilterForParallelReplica( "Invalid custom key column type: {}. Must be one unsigned integer type", custom_key_column_type->getName()); - if (range_lower < 0) - throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid custom key filter range: Range lower bound must be a positive"); - if (range_lower >= range_upper) throw Exception( ErrorCodes::INVALID_SETTING_VALUE, From 0c0c6bee323dca926b7358bac927c52a815c3bdc Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Tue, 4 Jun 2024 06:32:38 -0700 Subject: [PATCH 064/105] Address review comments --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- .../getCustomKeyFilterForParallelReplicas.cpp | 6 +- ...parallel_replicas_range_filter_min_max.sql | 105 +++++++++++------- 4 files changed, 68 insertions(+), 47 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 43879485f23..cf572459cbc 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -203,7 +203,7 @@ class IColumn; M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \ M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ M(UInt64, parallel_replicas_custom_key_range_lower, 0, "Lower bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ - M(UInt64, parallel_replicas_custom_key_range_upper, 0, "Upper bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ + M(UInt64, parallel_replicas_custom_key_range_upper, 0, "Upper bound for the universe that the parallel replicas custom range filter is calculated over. A value of 0 disables the upper bound, setting it to the max value of the custom key expression", 0) \ \ M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \ M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 144cb9100e5..35266249a2d 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -102,7 +102,7 @@ static const std::map lower_function; @@ -156,7 +156,7 @@ ASTPtr getCustomKeyFilterForParallelReplica( return upper_function; } - assert(upper_function && lower_function); + chassert(upper_function && lower_function); return makeASTFunction("and", std::move(lower_function), std::move(upper_function)); } diff --git a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql index c9588d931a8..b37948eccbd 100644 --- a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql +++ b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql @@ -4,54 +4,75 @@ CREATE TABLE range_filter_custom_range_test (k Int64) ENGINE=MergeTree ORDER BY INSERT INTO range_filter_custom_range_test SELECT number + 5 from numbers(10); -SELECT count() FROM -(SELECT * -FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) -SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, -parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', -parallel_replicas_custom_key_range_lower = 5, parallel_replicas_custom_key_range_upper=15); +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 5, parallel_replicas_custom_key_range_upper = 15 +); -SELECT count() FROM -(SELECT * -FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) -SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, -parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', -parallel_replicas_custom_key_range_lower = 4, parallel_replicas_custom_key_range_upper=14); +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 4, parallel_replicas_custom_key_range_upper = 14 +); -SELECT count() FROM -(SELECT * -FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) -SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, -parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', -parallel_replicas_custom_key_range_lower = 6, parallel_replicas_custom_key_range_upper=17); +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 6, parallel_replicas_custom_key_range_upper = 17 +); -SELECT count() FROM -(SELECT * -FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) -SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, -parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', -parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper=15); +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 15 +); -SELECT count() FROM -(SELECT * -FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) -SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, -parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', -parallel_replicas_custom_key_range_lower = 15, parallel_replicas_custom_key_range_upper=25); +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 15, parallel_replicas_custom_key_range_upper = 25 +); -SELECT count() FROM -(SELECT * -FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) -SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, -parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', -parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper=5); +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 5 +); -SELECT count() FROM -(SELECT * -FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) -SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge=0, -parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', -parallel_replicas_custom_key_range_lower = 500, parallel_replicas_custom_key_range_upper=10000); +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 500, parallel_replicas_custom_key_range_upper = 10000 +); DROP TABLE range_filter_custom_range_test; \ No newline at end of file From e3e5f304b9b13a3b964d4a3e4483c45393973a37 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Tue, 4 Jun 2024 06:34:56 -0700 Subject: [PATCH 065/105] Update docs --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 6c94de3ac40..65b0fa3f385 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1592,7 +1592,7 @@ Default value: `default`. ## parallel_replicas_custom_key_range_lower {#parallel_replicas_custom_key_range_lower} -Allows the filter type `range` to split the work evenly between replicas based the custom range `[parallel_replicas_custom_key_range_lower, INT_MAX]`. +Allows the filter type `range` to split the work evenly between replicas based on the custom range `[parallel_replicas_custom_key_range_lower, INT_MAX]`. When used in conjuction with [parallel_replicas_custom_key_range_upper](#parallel_replicas_custom_key_range_upper), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. @@ -1600,7 +1600,7 @@ Note: This setting will not cause any additional data to be filtered during quer ## parallel_replicas_custom_key_range_upper {#parallel_replicas_custom_key_range_upper} -Allows the filter type `range` to split the work evenly between replicas based the custom range `[0, parallel_replicas_custom_key_range_upper]`. +Allows the filter type `range` to split the work evenly between replicas based on the custom range `[0, parallel_replicas_custom_key_range_upper]`. A value of 0 disables the upper bound, setting it the max value of the custom key expression. When used in conjuction with [parallel_replicas_custom_key_range_lower](#parallel_replicas_custom_key_range_lower), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. From e495bf074212bc3c5671b9fe754029eb1254eddf Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Wed, 5 Jun 2024 08:24:59 -0400 Subject: [PATCH 066/105] Update src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp Co-authored-by: Antonio Andelic --- src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index 548aeba5398..7a908783f20 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -47,8 +47,7 @@ ASTPtr getCustomKeyFilterForParallelReplica( using RelativeSize = boost::rational; - RelativeSize range_upper = filter.range_upper > 0 ? RelativeSize(filter.range_upper) + RelativeSize(1) - : RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + RelativeSize range_upper = 0; RelativeSize range_lower = RelativeSize(filter.range_lower); DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; From 095508edb764af98706d860bd016f5309168c808 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 5 Jun 2024 06:57:02 -0700 Subject: [PATCH 067/105] Add additional tests --- .../getCustomKeyFilterForParallelReplicas.cpp | 2 +- ...el_replicas_range_filter_min_max.reference | 3 ++ ...parallel_replicas_range_filter_min_max.sql | 51 ++++++++++++++++++- 3 files changed, 53 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index 7a908783f20..43be7c5f043 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -47,7 +47,7 @@ ASTPtr getCustomKeyFilterForParallelReplica( using RelativeSize = boost::rational; - RelativeSize range_upper = 0; + RelativeSize range_upper = RelativeSize(0); RelativeSize range_lower = RelativeSize(filter.range_lower); DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; diff --git a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference index 5ba3f6bc471..4dded9eda81 100644 --- a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference +++ b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.reference @@ -5,3 +5,6 @@ 10 10 10 +10 +13 +4 diff --git a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql index b37948eccbd..58143395e44 100644 --- a/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql +++ b/tests/queries/0_stateless/03164_parallel_replicas_range_filter_min_max.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS range_filter_custom_range_test; -CREATE TABLE range_filter_custom_range_test (k Int64) ENGINE=MergeTree ORDER BY k; +CREATE TABLE range_filter_custom_range_test (k UInt64) ENGINE=MergeTree ORDER BY k; INSERT INTO range_filter_custom_range_test SELECT number + 5 from numbers(10); @@ -75,4 +75,51 @@ FROM parallel_replicas_custom_key_range_lower = 500, parallel_replicas_custom_key_range_upper = 10000 ); -DROP TABLE range_filter_custom_range_test; \ No newline at end of file + +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 3, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 10, parallel_replicas_custom_key_range_upper = 13 +); + +DROP TABLE range_filter_custom_range_test; + +DROP TABLE IF EXISTS range_filter_custom_range_test_2; + +CREATE TABLE range_filter_custom_range_test_2 (k UInt64) ENGINE=MergeTree ORDER BY k; + +INSERT INTO range_filter_custom_range_test_2 SELECT number from numbers(13); + +SELECT count() +FROM +( + SELECT * + FROM cluster(parallel_replicas, currentDatabase(), range_filter_custom_range_test_2) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 12, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 13 +); + +DROP TABLE range_filter_custom_range_test_2; + +DROP TABLE IF EXISTS range_filter_custom_range_test_3; + +CREATE TABLE range_filter_custom_range_test_3 (k UInt64) ENGINE=MergeTree ORDER BY k; + +INSERT INTO range_filter_custom_range_test_3 SELECT number from numbers(4); + +SELECT count() +FROM +( + SELECT * + FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), range_filter_custom_range_test_3) + SETTINGS prefer_localhost_replica = 0, max_parallel_replicas = 12, distributed_group_by_no_merge = 0, + parallel_replicas_custom_key = 'k', parallel_replicas_custom_key_filter_type = 'range', + parallel_replicas_custom_key_range_lower = 0, parallel_replicas_custom_key_range_upper = 4 +); + +DROP TABLE range_filter_custom_range_test_3; \ No newline at end of file From 59f50ca2357e81c19c226b27a301c70b8ce09ffd Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 5 Jun 2024 10:53:48 -0700 Subject: [PATCH 068/105] Fix existing tests to use UInt rather than Int --- .../test_parallel_replicas_custom_key_failover/test.py | 4 ++-- .../0_stateless/02535_max_parallel_replicas_custom_key.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_parallel_replicas_custom_key_failover/test.py b/tests/integration/test_parallel_replicas_custom_key_failover/test.py index bbb8fd5abb0..3ba3ce092c3 100644 --- a/tests/integration/test_parallel_replicas_custom_key_failover/test.py +++ b/tests/integration/test_parallel_replicas_custom_key_failover/test.py @@ -28,10 +28,10 @@ def create_tables(cluster, table_name): node3.query(f"DROP TABLE IF EXISTS {table_name} SYNC") node1.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" + f"CREATE TABLE IF NOT EXISTS {table_name} (key UInt64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" ) node3.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" + f"CREATE TABLE IF NOT EXISTS {table_name} (key UInt64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" ) # populate data diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index 9850406eb3a..63644a51f8f 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -30,7 +30,7 @@ run_with_custom_key "SELECT * FROM cluster(test_cluster_one_shard_three_replicas $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" -$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key (x String, y Int32) ENGINE = MergeTree ORDER BY cityHash64(x)" +$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key (x String, y UInt32) ENGINE = MergeTree ORDER BY cityHash64(x)" $CLICKHOUSE_CLIENT --query="INSERT INTO 02535_custom_key SELECT toString(number), number % 3 FROM numbers(1000)" function run_count_with_custom_key { From 76db904de1d620f07d28135b53202ef8fcd14238 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Thu, 6 Jun 2024 05:15:41 -0700 Subject: [PATCH 069/105] Update integration tests to use UInt rather than Int --- tests/integration/test_parallel_replicas_custom_key/test.py | 2 +- .../test_parallel_replicas_custom_key_load_balancing/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_parallel_replicas_custom_key/test.py b/tests/integration/test_parallel_replicas_custom_key/test.py index c646a678512..07a9e2badff 100644 --- a/tests/integration/test_parallel_replicas_custom_key/test.py +++ b/tests/integration/test_parallel_replicas_custom_key/test.py @@ -26,7 +26,7 @@ def create_tables(cluster): n1.query(f"DROP TABLE IF EXISTS test_table ON CLUSTER {cluster}") n1.query( - f"CREATE TABLE test_table ON CLUSTER {cluster} (key Int32, value String) Engine=MergeTree ORDER BY (key, sipHash64(value))" + f"CREATE TABLE test_table ON CLUSTER {cluster} (key UInt32, value String) Engine=MergeTree ORDER BY (key, sipHash64(value))" ) n1.query( f""" diff --git a/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py b/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py index d5e17103296..ddfd07a0864 100644 --- a/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py +++ b/tests/integration/test_parallel_replicas_custom_key_load_balancing/test.py @@ -33,7 +33,7 @@ def create_tables(table_name): for i in range(0, 4): nodes[i].query(f"DROP TABLE IF EXISTS {table_name} SYNC") nodes[i].query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r{i+1}') ORDER BY (key)" + f"CREATE TABLE IF NOT EXISTS {table_name} (key UInt64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r{i+1}') ORDER BY (key)" ) # populate data From fab23d2a16573e3a9613cb53c2aba4639f8e33eb Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 6 Jun 2024 15:02:33 +0200 Subject: [PATCH 070/105] Add lowCardinalityKeys, lowCardinalityIndices and an example for blockSize --- .../functions/other-functions.md | 149 +++++++++++++++++- 1 file changed, 148 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 31df9e5627d..4639f9d0991 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -212,7 +212,7 @@ toTypeName(x) ## blockSize {#blockSize} -In ClickHouse, queries are processed in blocks (chunks). +In ClickHouse, queries are processed in [blocks](../../development/architecture.md/#block-block) (chunks). This function returns the size (row count) of the block the function is called on. **Syntax** @@ -221,6 +221,38 @@ This function returns the size (row count) of the block the function is called o blockSize() ``` +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test; +CREATE TABLE test (n UInt8) ENGINE = Memory; +-- Insert 3 blocks: +INSERT INTO test VALUES (1); +INSERT INTO test VALUES (1),(2); +INSERT INTO test VALUES (1),(2),(3); + +SELECT blockSize(), n FROM (SELECT * FROM test); +``` + +Result: + +```response + ┌─blockSize()─┬─n─┐ +1. │ 1 │ 1 │ + └─────────────┴───┘ + ┌─blockSize()─┬─n─┐ +2. │ 3 │ 1 │ +3. │ 3 │ 2 │ +4. │ 3 │ 3 │ + └─────────────┴───┘ + ┌─blockSize()─┬─n─┐ +5. │ 2 │ 1 │ +6. │ 2 │ 2 │ + └─────────────┴───┘ +``` + ## byteSize Returns an estimation of uncompressed byte size of its arguments in memory. @@ -3688,3 +3720,118 @@ Result: ```response {'version':'1','serial_number':'2D9071D64530052D48308473922C7ADAFA85D6C5','signature_algo':'sha256WithRSAEncryption','issuer':'/CN=marsnet.local CA','not_before':'May 7 17:01:21 2024 GMT','not_after':'May 7 17:01:21 2025 GMT','subject':'/CN=chnode1','pkey_algo':'rsaEncryption'} ``` + +## lowCardinalityIndices + +For each row in the current [block](../../development/architecture.md/#block-block), returns the index of the value in the dictionary of unique values for columns of [LowCardinality](../data-types/lowcardinality.md) type. + +:::note +The first unique value encountered in the block is enumerated from 1. +::: + +**Syntax** + +```sql +lowCardinalityIndices(col) +``` + +**Arguments** + +- `col` — a low cardinality column. [LowCardinality](../data-types/lowcardinality.md). + +**Returned value** + +- returns the index of the value in the dictionary of unique values, for each row in the current block. [UInt64](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test; +CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory; + +-- insert two blocks of data: + +INSERT INTO test VALUES ('one'),('two'),('one'),('one'),('two'); +INSERT INTO test VALUES ('three'),('two'),('one'),('two'),('two'),('three'); + +SELECT s, lowCardinalityIndices(s) FROM test; +``` + +Result: + +```response + ┌─s───┬─lowCardinalityIndices(s)─┐ +1. │ one │ 1 │ +2. │ two │ 2 │ +3. │ one │ 1 │ +4. │ one │ 1 │ +5. │ two │ 2 │ + └─────┴──────────────────────────┘ + ┌─s─────┬─lowCardinalityIndices(s)─┐ + 6. │ three │ 1 │ + 7. │ two │ 2 │ + 8. │ one │ 3 │ + 9. │ two │ 2 │ +10. │ two │ 2 │ +11. │ three │ 1 │ + └───────┴──────────────────────────┘ +``` +## lowCardinalityKeys + +For each row in the current [block](../../development/architecture.md/#block-block), returns the keys (unique values) in the dictionary of unique values for columns of [LowCardinality](../data-types/lowcardinality.md) type. + +:::note +If the column size is less than the dictionary size, then values will be cut. If it is greater, then defaults will be added. +::: + +**Syntax** + +```sql +lowCardinalityIndices(col) +``` + +**Arguments** + +- `col` — a low cardinality column. [LowCardinality](../data-types/lowcardinality.md). + +**Returned value** + +- returns the keys of the dictionary, for each row in the current block. [UInt64](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test; +CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory; + +-- insert two blocks of data: + +INSERT INTO test VALUES ('one'),('two'),('one'),('one'),('two'); +INSERT INTO test VALUES ('three'),('two'),('one'),('two'),('two'),('three'); + +SELECT s, lowCardinalityKeys(s) FROM test; +``` + +Result: + +```response + ┌─s───┬─lowCardinalityKeys(s)─┐ +1. │ one │ │ +2. │ two │ one │ +3. │ one │ two │ +4. │ one │ │ +5. │ two │ │ + └─────┴───────────────────────┘ + ┌─s─────┬─lowCardinalityKeys(s)─┐ + 6. │ three │ │ + 7. │ two │ three │ + 8. │ one │ two │ + 9. │ two │ one │ +10. │ two │ │ +11. │ three │ │ + └───────┴───────────────────────┘ +``` From 387869488d80db6c645346c61b2470d37a026fe6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jun 2024 15:54:02 +0200 Subject: [PATCH 071/105] Stabilize and adjust number of streams --- src/Storages/StorageGenerateRandom.cpp | 58 +++++++++++++++---- src/Storages/System/StorageSystemZeros.cpp | 33 ++++++----- ...generate_random_with_limit_progress_bar.sh | 2 +- 3 files changed, 65 insertions(+), 28 deletions(-) diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index cdbade51695..5aceef78238 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -50,6 +50,12 @@ namespace ErrorCodes namespace { +struct GenerateRandomState +{ + std::atomic add_total_rows = 0; +}; +using GenerateRandomStatePtr = std::shared_ptr; + void fillBufferWithRandomData(char * __restrict data, size_t limit, size_t size_of_type, pcg64 & rng, [[maybe_unused]] bool flip_bytes = false) { size_t size = limit * size_of_type; @@ -529,10 +535,24 @@ ColumnPtr fillColumnWithRandomData( class GenerateSource : public ISource { public: - GenerateSource(UInt64 block_size_, UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_, Block block_header_, ContextPtr context_) + GenerateSource( + UInt64 block_size_, + UInt64 max_array_length_, + UInt64 max_string_length_, + UInt64 random_seed_, + Block block_header_, + ContextPtr context_, + GenerateRandomStatePtr state_) : ISource(Nested::flattenNested(prepareBlockToFill(block_header_))) - , block_size(block_size_), max_array_length(max_array_length_), max_string_length(max_string_length_) - , block_to_fill(std::move(block_header_)), rng(random_seed_), context(context_) {} + , block_size(block_size_) + , max_array_length(max_array_length_) + , max_string_length(max_string_length_) + , block_to_fill(std::move(block_header_)) + , rng(random_seed_) + , context(context_) + , shared_state(state_) + { + } String getName() const override { return "GenerateRandom"; } @@ -546,7 +566,15 @@ protected: columns.emplace_back(fillColumnWithRandomData(elem.type, block_size, max_array_length, max_string_length, rng, context)); columns = Nested::flattenNested(block_to_fill.cloneWithColumns(columns)).getColumns(); - return {std::move(columns), block_size}; + + UInt64 total_rows = shared_state->add_total_rows.fetch_and(0); + if (total_rows) + addTotalRowsApprox(total_rows); + + auto chunk = Chunk{std::move(columns), block_size}; + progress(chunk.getNumRows(), chunk.bytes()); + + return chunk; } private: @@ -558,6 +586,7 @@ private: pcg64 rng; ContextPtr context; + GenerateRandomStatePtr shared_state; static Block & prepareBlockToFill(Block & block) { @@ -645,9 +674,6 @@ Pipe StorageGenerateRandom::read( { storage_snapshot->check(column_names); - Pipes pipes; - pipes.reserve(num_streams); - const ColumnsDescription & our_columns = storage_snapshot->metadata->getColumns(); Block block_header; for (const auto & name : column_names) @@ -676,16 +702,24 @@ Pipe StorageGenerateRandom::read( } } + UInt64 query_limit = query_info.limit; + if (query_limit && num_streams * max_block_size < query_limit) + { + /// We want to avoid spawning more streams than necessary + num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); + } + Pipes pipes; + pipes.reserve(num_streams); + /// Will create more seed values for each source from initial seed. pcg64 generate(random_seed); + auto shared_state = std::make_shared(query_info.limit); + for (UInt64 i = 0; i < num_streams; ++i) { - auto source = std::make_shared(max_block_size, max_array_length, max_string_length, generate(), block_header, context); - - if (i == 0 && query_info.limit) - source->addTotalRowsApprox(query_info.limit); - + auto source = std::make_shared( + max_block_size, max_array_length, max_string_length, generate(), block_header, context, shared_state); pipes.emplace_back(std::move(source)); } diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index a48b109fbbe..40faf2e265f 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -16,7 +16,9 @@ namespace struct ZerosState { + ZerosState(UInt64 limit) : add_total_rows(limit) { } std::atomic num_generated_rows = 0; + std::atomic add_total_rows = 0; }; using ZerosStatePtr = std::shared_ptr; @@ -49,6 +51,10 @@ protected: if (generated_rows >= limit) return {}; + UInt64 total_rows = state->add_total_rows.fetch_and(0); + if (total_rows) + addTotalRowsApprox(total_rows); + if (generated_rows + column_size > limit) { column_size = limit - generated_rows; @@ -105,10 +111,16 @@ Pipe StorageSystemZeros::read( bool use_multiple_streams = multithreaded; - if (limit && *limit < max_block_size) + UInt64 query_limit = limit ? *limit : query_info.limit; + if (query_limit && query_limit > max_block_size) + max_block_size = query_limit; + + if (use_multiple_streams && query_limit && num_streams * max_block_size < query_limit) { - max_block_size = static_cast(*limit); - use_multiple_streams = false; + /// We want to avoid spawning more streams than necessary + num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); + if (num_streams <= 1) + use_multiple_streams = false; } if (!use_multiple_streams) @@ -118,21 +130,12 @@ Pipe StorageSystemZeros::read( ZerosStatePtr state; - if (limit) - state = std::make_shared(); + if (query_limit) + state = std::make_shared(query_limit); for (size_t i = 0; i < num_streams; ++i) { - auto source = std::make_shared(max_block_size, limit ? *limit : 0, state); - - if (i == 0) - { - if (limit) - source->addTotalRowsApprox(*limit); - else if (query_info.limit) - source->addTotalRowsApprox(query_info.limit); - } - + auto source = std::make_shared(max_block_size, query_limit, state); res.addSource(std::move(source)); } diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh index 8aedf0bc0ff..500a12587a2 100755 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh @@ -8,7 +8,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function run_with_progress_and_match_total_rows() { CURL_RESPONSE=$(echo "$1" | \ - ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1) + ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&wait_end_of_query=1&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1) echo "$CURL_RESPONSE" | grep -q '"total_rows_to_read":"100"' && echo "Matched" || echo "Expected total_rows_to_read not found: ${CURL_RESPONSE}" } From 00167da1191bec5c2ab6df0323ccae34f2204016 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 6 Jun 2024 16:27:00 +0200 Subject: [PATCH 072/105] Disable transactions for unsupported storages even for materialized views --- src/Interpreters/InterpreterInsertQuery.cpp | 2 ++ ...03167_transactions_are_really_disabled.reference | 0 .../03167_transactions_are_really_disabled.sql | 13 +++++++++++++ 3 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/03167_transactions_are_really_disabled.reference create mode 100644 tests/queries/0_stateless/03167_transactions_are_really_disabled.sql diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 128854e87ba..927bafe4bfb 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -279,6 +279,8 @@ Chain InterpreterInsertQuery::buildChain( std::atomic_uint64_t * elapsed_counter_ms, bool check_access) { + IInterpreter::checkStorageSupportsTransactionsIfNeeded(table, getContext()); + ProfileEvents::increment(ProfileEvents::InsertQueriesWithSubqueries); ProfileEvents::increment(ProfileEvents::QueriesWithSubqueries); diff --git a/tests/queries/0_stateless/03167_transactions_are_really_disabled.reference b/tests/queries/0_stateless/03167_transactions_are_really_disabled.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03167_transactions_are_really_disabled.sql b/tests/queries/0_stateless/03167_transactions_are_really_disabled.sql new file mode 100644 index 00000000000..e3c86a2d5be --- /dev/null +++ b/tests/queries/0_stateless/03167_transactions_are_really_disabled.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS mv_table; +DROP TABLE IF EXISTS null_table; + +CREATE TABLE null_table (str String) ENGINE = Null; +CREATE MATERIALIZED VIEW mv_table (str String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/transactions_disabled_rmt', '{replica}') ORDER BY str AS SELECT str AS str FROM null_table; + +SET implicit_transaction=1; +set throw_on_unsupported_query_inside_transaction=0; + +INSERT INTO null_table VALUES ('test'); --{serverError NOT_IMPLEMENTED} + +DROP TABLE IF EXISTS mv_table; +DROP TABLE IF EXISTS null_table; From 833f031721cf40fc28d155a5df7cf23079bde9a9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 5 Jun 2024 18:32:04 +0200 Subject: [PATCH 073/105] Handle logs from rocksdb by ClickHouse internal logging Someone may not know about /path/to/rocksdb/LOG, and besides it requires an access to data dir, instead of logs dir. v2: fix use-after-free due to destruction order (https://s3.amazonaws.com/clickhouse-test-reports/64856/8cefc1a1ba5ddfdda033694a14e1f1847d497092/stateless_tests__asan__[2_4]/stderr.log) Signed-off-by: Azat Khuzhin --- programs/server/config.xml | 1 + .../RocksDB/StorageEmbeddedRocksDB.cpp | 71 ++++++++++++++++++- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 6 +- 3 files changed, 75 insertions(+), 3 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 4b3248d9d1c..b7a4b8dd0e9 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1561,6 +1561,7 @@ 8 + DEBUG_LEVEL 2 diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index c3b7ae64c7e..b9d3e071b6c 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -25,6 +25,7 @@ #include #include +#include #include #include #include @@ -42,6 +43,7 @@ #include #include +#include #include @@ -185,11 +187,11 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, bool read_only_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) + , log(getLogger(fmt::format("StorageEmbeddedRocksDB ({})", getStorageID().getNameForLogs()))) , primary_key{primary_key_} , rocksdb_dir(std::move(rocksdb_dir_)) , ttl(ttl_) , read_only(read_only_) - , log(getLogger(fmt::format("StorageEmbeddedRocksDB ({})", getStorageID().getNameForLogs()))) { setInMemoryMetadata(metadata_); setSettings(std::move(settings_)); @@ -352,6 +354,72 @@ bool StorageEmbeddedRocksDB::optimize( return true; } +static_assert(rocksdb::DEBUG_LEVEL == 0); +static_assert(rocksdb::HEADER_LEVEL == 5); +static constexpr std::array, 6> rocksdb_logger_map = { + std::make_pair(DB::LogsLevel::debug, Poco::Message::Priority::PRIO_DEBUG), + std::make_pair(DB::LogsLevel::information, Poco::Message::Priority::PRIO_INFORMATION), + std::make_pair(DB::LogsLevel::warning, Poco::Message::Priority::PRIO_WARNING), + std::make_pair(DB::LogsLevel::error, Poco::Message::Priority::PRIO_ERROR), + std::make_pair(DB::LogsLevel::fatal, Poco::Message::Priority::PRIO_FATAL), + /// Same as default logger does for HEADER_LEVEL + std::make_pair(DB::LogsLevel::information, Poco::Message::Priority::PRIO_INFORMATION), +}; +class StorageEmbeddedRocksDBLogger : public rocksdb::Logger +{ +public: + explicit StorageEmbeddedRocksDBLogger(const rocksdb::InfoLogLevel log_level, LoggerRawPtr log_) + : rocksdb::Logger(log_level) + , log(log_) + {} + + void Logv(const char * format, va_list ap) override + __attribute__((format(printf, 2, 0))) + { + Logv(rocksdb::InfoLogLevel::DEBUG_LEVEL, format, ap); + } + + void Logv(const rocksdb::InfoLogLevel log_level, const char * format, va_list ap) override + __attribute__((format(printf, 3, 0))) + { + if (log_level < GetInfoLogLevel()) + return; + + auto level = rocksdb_logger_map[log_level]; + + /// stack buffer was enough + { + va_list backup_ap; + va_copy(backup_ap, ap); + std::array stack; + if (vsnprintf(stack.data(), stack.size(), format, backup_ap) < static_cast(stack.size())) + { + va_end(backup_ap); + LOG_IMPL(log, level.first, level.second, "{}", stack.data()); + return; + } + va_end(backup_ap); + } + + /// let's try with a bigger dynamic buffer (but not too huge, since + /// some of rocksdb internal code has also such a limitation, i..e + /// HdfsLogger) + { + va_list backup_ap; + va_copy(backup_ap, ap); + static constexpr int buffer_size = 30000; + std::unique_ptr buffer(new char[buffer_size]); + if (vsnprintf(buffer.get(), buffer_size, format, backup_ap) >= buffer_size) + buffer[buffer_size - 1] = 0; + va_end(backup_ap); + LOG_IMPL(log, level.first, level.second, "{}", buffer.get()); + } + } + +private: + LoggerRawPtr log; +}; + void StorageEmbeddedRocksDB::initDB() { rocksdb::Status status; @@ -448,6 +516,7 @@ void StorageEmbeddedRocksDB::initDB() } } + merged.info_log = std::make_shared(merged.info_log_level, log.get()); merged.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); if (ttl > 0) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 61592398954..a6aa1ba36a4 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -114,17 +114,19 @@ public: private: SinkToStoragePtr getSink(ContextPtr context, const StorageMetadataPtr & metadata_snapshot); + LoggerPtr log; + MultiVersion storage_settings; const String primary_key; + using RocksDBPtr = std::unique_ptr; RocksDBPtr rocksdb_ptr; + mutable SharedMutex rocksdb_ptr_mx; String rocksdb_dir; Int32 ttl; bool read_only; void initDB(); - - LoggerPtr log; }; } From d10027cc3b7737c524f4cfce262d46753fd03036 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 5 Jun 2024 18:33:13 +0200 Subject: [PATCH 074/105] Enable debug logging for rocksdb on CI To understand the root cause of 02956_rocksdb_bulk_sink failures, this is likely due to some background merge had been done I would say, but it is hard to say without extra information. Signed-off-by: Azat Khuzhin --- tests/config/config.d/rocksdb.xml | 7 +++++++ tests/config/install.sh | 1 + 2 files changed, 8 insertions(+) create mode 100644 tests/config/config.d/rocksdb.xml diff --git a/tests/config/config.d/rocksdb.xml b/tests/config/config.d/rocksdb.xml new file mode 100644 index 00000000000..a3790a3dc1d --- /dev/null +++ b/tests/config/config.d/rocksdb.xml @@ -0,0 +1,7 @@ + + + + DEBUG_LEVEL + + + diff --git a/tests/config/install.sh b/tests/config/install.sh index 6536683b6c2..e04392d893b 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -68,6 +68,7 @@ ln -sf $SRC_PATH/config.d/zero_copy_destructive_operations.xml $DEST_SERVER_PATH ln -sf $SRC_PATH/config.d/block_number.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/handlers.yaml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/serverwide_trace_collector.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/rocksdb.xml $DEST_SERVER_PATH/config.d/ # Not supported with fasttest. if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ] From 4ba025c3ba8ceaa1551670f29b328983741f8052 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 Jun 2024 15:48:36 +0000 Subject: [PATCH 075/105] Fix duplicating Delete events in blob_storage_log --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index ae719f5cde4..afc13251f5b 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -382,6 +382,7 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e { std::vector current_chunk; String keys; + size_t first_position = current_position; for (; current_position < objects.size() && current_chunk.size() < chunk_size_limit; ++current_position) { Aws::S3::Model::ObjectIdentifier obj; @@ -407,9 +408,9 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e { const auto * outcome_error = outcome.IsSuccess() ? nullptr : &outcome.GetError(); auto time_now = std::chrono::system_clock::now(); - for (const auto & object : objects) + for (size_t i = first_position; i < current_position; ++i) blob_storage_log->addEvent(BlobStorageLogElement::EventType::Delete, - uri.bucket, object.remote_path, object.local_path, object.bytes_size, + uri.bucket, objects[i].remote_path, objects[i].local_path, objects[i].bytes_size, outcome_error, time_now); } From 38d45a6bc99e738a051ccb386e3c0db706051c32 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 Jun 2024 15:49:14 +0000 Subject: [PATCH 076/105] Verbose message for errors on inserting to system log --- src/Interpreters/SystemLog.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 5e0ce2cb0de..24e478053bc 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -499,7 +499,10 @@ void SystemLog::flushImpl(const std::vector & to_flush, auto log_element_names_and_types = LogElement::getColumnsDescription(); for (const auto & name_and_type : log_element_names_and_types.getAll()) - log_element_columns.emplace_back(name_and_type.type, name_and_type.name); + { + auto & column = log_element_columns.emplace_back(name_and_type.type, name_and_type.name); + column.column->reserve(to_flush.size()); + } Block block(std::move(log_element_columns)); @@ -532,7 +535,8 @@ void SystemLog::flushImpl(const std::vector & to_flush, } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Failed to flush system log {} with {} entries up to offset {}", + table_id.getNameForLogs(), to_flush.size(), to_flush_end)); } queue->confirm(to_flush_end); From 81fe4c48944eda05c8aad628e0de65b47aeddead Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jun 2024 17:59:16 +0200 Subject: [PATCH 077/105] Fix number of stream decision --- src/Storages/StorageGenerateRandom.cpp | 2 +- src/Storages/System/StorageSystemZeros.cpp | 35 ++++++++-------------- 2 files changed, 14 insertions(+), 23 deletions(-) diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 5aceef78238..ca9c6fb3226 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -703,7 +703,7 @@ Pipe StorageGenerateRandom::read( } UInt64 query_limit = query_info.limit; - if (query_limit && num_streams * max_block_size < query_limit) + if (query_limit && num_streams * max_block_size > query_limit) { /// We want to avoid spawning more streams than necessary num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index 40faf2e265f..7e1d7a14d0e 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -44,17 +44,16 @@ protected: auto column_ptr = column; size_t column_size = column_ptr->size(); - if (state) + UInt64 total_rows = state->add_total_rows.fetch_and(0); + if (total_rows) + addTotalRowsApprox(total_rows); + + if (limit) { auto generated_rows = state->num_generated_rows.fetch_add(column_size, std::memory_order_acquire); - if (generated_rows >= limit) return {}; - UInt64 total_rows = state->add_total_rows.fetch_and(0); - if (total_rows) - addTotalRowsApprox(total_rows); - if (generated_rows + column_size > limit) { column_size = limit - generated_rows; @@ -109,30 +108,22 @@ Pipe StorageSystemZeros::read( { storage_snapshot->check(column_names); - bool use_multiple_streams = multithreaded; + UInt64 query_limit = limit ? *limit : 0; + if (query_info.limit) + query_limit = query_limit ? std::min(query_limit, query_info.limit) : query_info.limit; - UInt64 query_limit = limit ? *limit : query_info.limit; - if (query_limit && query_limit > max_block_size) + if (query_limit && query_limit < max_block_size) max_block_size = query_limit; - if (use_multiple_streams && query_limit && num_streams * max_block_size < query_limit) - { + if (!multithreaded) + num_streams = 1; + else if (query_limit && num_streams * max_block_size > query_limit) /// We want to avoid spawning more streams than necessary num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); - if (num_streams <= 1) - use_multiple_streams = false; - } - if (!use_multiple_streams) - num_streams = 1; + ZerosStatePtr state = std::make_shared(query_limit); Pipe res; - - ZerosStatePtr state; - - if (query_limit) - state = std::make_shared(query_limit); - for (size_t i = 0; i < num_streams; ++i) { auto source = std::make_shared(max_block_size, query_limit, state); From 1881107f0773d4aa0e195ef5cd6bb8f8a17b8cc3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 Jun 2024 16:04:46 +0000 Subject: [PATCH 078/105] fix build --- src/Interpreters/SystemLog.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 24e478053bc..3b25deeb59d 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -499,14 +499,15 @@ void SystemLog::flushImpl(const std::vector & to_flush, auto log_element_names_and_types = LogElement::getColumnsDescription(); for (const auto & name_and_type : log_element_names_and_types.getAll()) - { - auto & column = log_element_columns.emplace_back(name_and_type.type, name_and_type.name); - column.column->reserve(to_flush.size()); - } + log_element_columns.emplace_back(name_and_type.type, name_and_type.name); Block block(std::move(log_element_columns)); MutableColumns columns = block.mutateColumns(); + + for (auto & column : columns) + column->reserve(to_flush.size()); + for (const auto & elem : to_flush) elem.appendToBlock(columns); From e42791e9e8dc619efc12e24c505bc14d7a1a2399 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 6 Jun 2024 18:45:43 +0200 Subject: [PATCH 079/105] Add info about MutationsInterpreter --- docs/en/operations/analyzer.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index f5b900d01f7..f89a4416375 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -148,6 +148,12 @@ SELECT toTypeName(if(0, [2, 3, 4], 'String')) The new analyzer significantly changed the communication protocol between servers in the cluster. Thus, it's impossible to run distributed queries on servers with different `allow_experimental_analyzer` setting values. +### Mutations are interpreted by previous analyzer + +Mutations are still using the old analyzer. +This means some new ClickHouse SQL features can't be used in mutations. For example, the `QUALIFY` clause. +Status can be checked [here](https://github.com/ClickHouse/ClickHouse/issues/61563). + ### Unsupported features The list of features new analyzer currently doesn't support: From ef9ef53fde6d3d1c543fd33a6c278956b7b43450 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 6 Jun 2024 18:53:28 +0200 Subject: [PATCH 080/105] Add info about fix --- docs/en/operations/analyzer.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index f89a4416375..2a79653fa00 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -19,6 +19,7 @@ Optimizations could rewrite the initial query so it becomes valid and can be exe In the new infrastructure, query validation takes place before the optimization step. This means that invalid queries that were possible to execute before are now unsupported. +In such cases, the query must be fixed manually. **Example 1:** @@ -45,6 +46,16 @@ HAVING number > 5 The same problem occurs in this query: column `number` is used after aggregation with another key. The previous query analyzer fixed this query by moving the `number > 5` filter from the `HAVING` clause to the `WHERE` clause. +To fix the query, you should move all conditions that apply to non-aggregated columns to the `WHERE` section to conform to standard SQL syntax: +```sql +SELECT + number % 2 AS n, + sum(number) +FROM numbers(10) +WHERE number > 5 +GROUP BY n +``` + ### Known incompatibilities of JOIN clause #### Join using column from projection From 21294013f2df27b9abb56b0fb5a017176689f8ee Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Thu, 6 Jun 2024 10:03:52 -0700 Subject: [PATCH 081/105] Update SettingsChangesHistory --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 35266249a2d..7eb14047ace 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -100,7 +100,7 @@ static const std::map Date: Thu, 6 Jun 2024 19:17:34 +0200 Subject: [PATCH 082/105] Add info about invalid VIEWs --- docs/en/operations/analyzer.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index 2a79653fa00..7b3e18666ec 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -56,6 +56,23 @@ WHERE number > 5 GROUP BY n ``` +### CREATE VIEW with invalid query + +The new infrastructure always performs type-checking. +Previously, it was possible to create a `VIEW` with an invalid `SELECT` query, and it'd fail during the first SELECT or insert (in the case of `MATERIALIZED VIEW`). + +Now, it's not possible to create such `VIEW`s anymore. + +**Example:** + +```sql +CREATE TABLE source (data String) ENGINE=MergeTree ORDER BY tuple(); + +CREATE VIEW some_view +AS SELECT JSONExtract(data, 'test', 'DateTime64(3)') +FROM source; +``` + ### Known incompatibilities of JOIN clause #### Join using column from projection From d36bf5bac86fb9bfde4fe761a1ab2ab7079f4f1a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jun 2024 21:40:24 +0200 Subject: [PATCH 083/105] Compiler complains --- src/Storages/System/StorageSystemZeros.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index 7e1d7a14d0e..09a2bb5d963 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -16,7 +16,7 @@ namespace struct ZerosState { - ZerosState(UInt64 limit) : add_total_rows(limit) { } + explicit ZerosState(UInt64 limit) : add_total_rows(limit) { } std::atomic num_generated_rows = 0; std::atomic add_total_rows = 0; }; @@ -119,7 +119,7 @@ Pipe StorageSystemZeros::read( num_streams = 1; else if (query_limit && num_streams * max_block_size > query_limit) /// We want to avoid spawning more streams than necessary - num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); + num_streams = std::min(num_streams, static_cast(((query_limit + max_block_size - 1) / max_block_size))); ZerosStatePtr state = std::make_shared(query_limit); From 05ea297c0d79f55f65ded88ec5284fa9e8c34753 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Jun 2024 03:47:35 +0200 Subject: [PATCH 084/105] Fix error message (it was strange) --- src/Functions/FunctionHelpers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionHelpers.cpp b/src/Functions/FunctionHelpers.cpp index c7a0c3c58ca..593646240ca 100644 --- a/src/Functions/FunctionHelpers.cpp +++ b/src/Functions/FunctionHelpers.cpp @@ -314,7 +314,7 @@ void checkFunctionArgumentSizes(const ColumnsWithTypeAndName & arguments, size_t if (current_size != input_rows_count) throw Exception( ErrorCodes::LOGICAL_ERROR, - "Expected the argument nº#{} ('{}' of type {}) to have {} rows, but it has {}", + "Expected the argument №{} ('{}' of type {}) to have {} rows, but it has {}", i + 1, arguments[i].name, arguments[i].type->getName(), From 47578c7243c4379fea1677e32e0e3a2733d8b074 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Fri, 7 Jun 2024 08:43:49 +0200 Subject: [PATCH 085/105] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 49f43615c7e..826c6953a10 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1934,6 +1934,8 @@ london lookups loongarch lowcardinality +lowCardinalityIndices +lowCardinalityKeys lowerUTF lowercased lttb From f2700f551034f5c45f80ab4aeb5777198625a74d Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Fri, 7 Jun 2024 01:48:02 -0700 Subject: [PATCH 086/105] Analyzer docs changes --- docs/en/operations/analyzer.md | 41 ++++++++++++++++++---------------- 1 file changed, 22 insertions(+), 19 deletions(-) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index 7b3e18666ec..e85c5fa9e8d 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -1,6 +1,9 @@ --- slug: /en/operations/analyzer sidebar_label: Analyzer +title: Analyzer +description: Details about ClickHouse's query analyzer +keywords: [analyzer] --- # Analyzer @@ -9,15 +12,15 @@ sidebar_label: Analyzer ## Known incompatibilities -In ClickHouse version `24.3`, new query analysis was enabled by default. -Despite fixing a large number of bugs and introducing new optimizations, it also introduces some breaking changes in ClickHouse behaviour. +In ClickHouse version `24.3`, the new query analyzer was enabled by default. +Despite fixing a large number of bugs and introducing new optimizations, it also introduces some breaking changes in ClickHouse behaviour. Please read the following changes to determine how to rewrite your queries for the new analyzer. -### Invalid before optimization queries +### Invalid queries are no longer optimized The previous query planning infrastructure applied AST-level optimizations before the query validation step. Optimizations could rewrite the initial query so it becomes valid and can be executed. -In the new infrastructure, query validation takes place before the optimization step. +In the new analyzer, query validation takes place before the optimization step. This means that invalid queries that were possible to execute before are now unsupported. In such cases, the query must be fixed manually. @@ -30,7 +33,7 @@ GROUP BY toString(number) ``` The following query uses column `number` in the projection list when only `toString(number)` is available after the aggregation. -In the old infrastructure, `GROUP BY toString(number)` was optimized into `GROUP BY number,` making the query valid. +In the old analyzer, `GROUP BY toString(number)` was optimized into `GROUP BY number,` making the query valid. **Example 2:** @@ -58,8 +61,8 @@ GROUP BY n ### CREATE VIEW with invalid query -The new infrastructure always performs type-checking. -Previously, it was possible to create a `VIEW` with an invalid `SELECT` query, and it'd fail during the first SELECT or insert (in the case of `MATERIALIZED VIEW`). +The new analyzer always performs type-checking. +Previously, it was possible to create a `VIEW` with an invalid `SELECT` query. It would then fail during the first `SELECT` or `INSERT` (in the case of `MATERIALIZED VIEW`). Now, it's not possible to create such `VIEW`s anymore. @@ -73,15 +76,15 @@ AS SELECT JSONExtract(data, 'test', 'DateTime64(3)') FROM source; ``` -### Known incompatibilities of JOIN clause +### Known incompatibilities of the `JOIN` clause #### Join using column from projection -Alias from the `SELECT` list can not be used as a `JOIN USING` key by default. +Alias from the `SELECT` list can not be used as a `JOIN USING` key by default. -A new setting, `analyzer_compatibility_join_using_top_level_identifier`, when enabled, alters the behavior of `JOIN USING` to prefer to resolve identifiers based on expressions from the projection list of the SELECT query, rather than using the columns from left table directly. +A new setting, `analyzer_compatibility_join_using_top_level_identifier`, when enabled, alters the behavior of `JOIN USING` to prefer to resolve identifiers based on expressions from the projection list of the `SELECT` query, rather than using the columns from left table directly. -*Example:* +**Example:** ```sql SELECT a + 1 AS b, t2.s @@ -90,15 +93,15 @@ JOIN Values('b UInt64, s String', (1, 'one'), (2, 'two')) t2 USING (b); ``` -With `analyzer_compatibility_join_using_top_level_identifier` set to `true`, the join condition is interpreted as `t1.a + 1 = t2.b`, matching the behavior of earlier versions. So, the result will be `2, 'two'` +With `analyzer_compatibility_join_using_top_level_identifier` set to `true`, the join condition is interpreted as `t1.a + 1 = t2.b`, matching the behavior of earlier versions. So, the result will be `2, 'two'`. When the setting is `false`, the join condition defaults to `t1.b = t2.b`, and the query will return `2, 'one'`. -In case then `b` is not present in `t1`, the query will fail with an error. +If `b` is not present in `t1`, the query will fail with an error. -#### Changes in Behavior with `JOIN USING` and `ALIAS/MATERIALIZED` Columns +#### Changes in behavior with `JOIN USING` and `ALIAS`/`MATERIALIZED` columns -In the new analyzer, using `*` in a `JOIN USING` query that involves `ALIAS` or `MATERIALIZED` columns will include that columns in the result set by default. +In the new analyzer, using `*` in a `JOIN USING` query that involves `ALIAS` or `MATERIALIZED` columns will include those columns in the result set by default. -*Example:* +**Example:** ```sql CREATE TABLE t1 (id UInt64, payload ALIAS sipHash64(id)) ENGINE = MergeTree ORDER BY id; @@ -123,7 +126,7 @@ In the new version of the analyzer, the rules for determining the common superty - `Nullable(T)` and `T`: When a column of type `Nullable(T)` is joined with a column of type `T`, the resulting common supertype will be `Nullable(T)`, ensuring that the nullable property is preserved. -*Example:* +**Example:** ```sql SELECT id, toTypeName(id) FROM Values('id LowCardinality(String)', ('a')) AS t1 @@ -135,7 +138,7 @@ In this query, the common supertype for `id` is determined as `String`, discardi ### Projection column names changes -During projection names computation aliases are not substituted. +During projection names, computation aliases are not substituted. ```sql SELECT @@ -161,7 +164,7 @@ FORMAT PrettyCompact ### Incompatible function arguments types -In the new infrastructure type inference happens during initial query analysis. +In the new analyzer, type inference happens during initial query analysis. This change means that type checks are done before short-circuit evaluation; thus, `if` function arguments must always have a common supertype. **Example:** From 43e6482af0a6e14a2aee3abe007ff23a084c7acc Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 7 Jun 2024 11:36:24 +0200 Subject: [PATCH 087/105] Update docs/en/operations/analyzer.md --- docs/en/operations/analyzer.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/analyzer.md b/docs/en/operations/analyzer.md index e85c5fa9e8d..298c6dacd06 100644 --- a/docs/en/operations/analyzer.md +++ b/docs/en/operations/analyzer.md @@ -138,7 +138,7 @@ In this query, the common supertype for `id` is determined as `String`, discardi ### Projection column names changes -During projection names, computation aliases are not substituted. +During projection names computation, aliases are not substituted. ```sql SELECT From 078f5f4ee0fff1d55a5924df8340a905ef32f0f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 7 Jun 2024 13:10:12 +0200 Subject: [PATCH 088/105] Fix bug in short circuit evaluation --- src/Columns/MaskOperations.cpp | 6 +++++- .../0_stateless/03168_fuzz_multiIf_short_circuit.reference | 0 .../0_stateless/03168_fuzz_multiIf_short_circuit.sql | 6 ++++++ 3 files changed, 11 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.reference create mode 100644 tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql diff --git a/src/Columns/MaskOperations.cpp b/src/Columns/MaskOperations.cpp index 1f5f94beee9..873a4060872 100644 --- a/src/Columns/MaskOperations.cpp +++ b/src/Columns/MaskOperations.cpp @@ -289,10 +289,14 @@ void executeColumnIfNeeded(ColumnWithTypeAndName & column, bool empty) if (!column_function) return; + size_t original_size = column.column->size(); + if (!empty) column = column_function->reduce(); else - column.column = column_function->getResultType()->createColumn(); + column.column = column_function->getResultType()->createColumnConstWithDefaultValue(original_size)->convertToFullColumnIfConst(); + + chassert(column.column->size() == original_size); } int checkShortCircuitArguments(const ColumnsWithTypeAndName & arguments) diff --git a/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.reference b/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql b/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql new file mode 100644 index 00000000000..4e4cc291e9b --- /dev/null +++ b/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql @@ -0,0 +1,6 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/64946 +SELECT + multiIf((number % toLowCardinality(toNullable(toUInt128(2)))) = (number % toNullable(2)), toInt8(1), (number % materialize(toLowCardinality(3))) = toUInt128(toNullable(0)), toInt8(materialize(materialize(2))), toInt64(toUInt128(3))) +FROM system.numbers +LIMIT 44857 +FORMAT Null; From 4b010dc478310b65d26cbe114e15f3cb73af4bb4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 7 Jun 2024 13:11:52 +0200 Subject: [PATCH 089/105] Disable test with ASAN --- .../0_stateless/02908_many_requests_to_system_replicas.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh index 144831a2cdc..a247c99a818 100755 --- a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh +++ b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper, no-parallel, no-fasttest +# Tags: long, zookeeper, no-parallel, no-fasttest, no-asan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 2d9ac2e8169957ba73f477befd9aef8e753e86f2 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 7 Jun 2024 14:03:56 +0200 Subject: [PATCH 090/105] Revert "Add dynamic untracked memory limits for more precise memory tracking" --- docs/en/operations/settings/settings.md | 12 +----------- src/Common/CurrentMemoryTracker.cpp | 12 +----------- src/Common/CurrentMemoryTracker.h | 2 -- src/Common/ThreadStatus.h | 12 ------------ src/Core/Settings.h | 1 - src/Core/SettingsChangesHistory.h | 1 - src/Interpreters/ThreadStatusExt.cpp | 10 +++------- tests/integration/test_failed_async_inserts/test.py | 4 +++- .../test_settings_constraints_distributed/test.py | 5 +---- .../0_stateless/01017_uniqCombined_memory_usage.sql | 3 +-- .../03030_system_flush_distributed_settings.sql | 2 -- 11 files changed, 10 insertions(+), 54 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b3e9da816ab..ffaf53085c4 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3172,7 +3172,7 @@ Default value: `0`. ## lightweight_deletes_sync {#lightweight_deletes_sync} -The same as 'mutation_sync', but controls only execution of lightweight deletes. +The same as 'mutation_sync', but controls only execution of lightweight deletes. Possible values: @@ -4616,16 +4616,6 @@ Read more about [memory overcommit](memory-overcommit.md). Default value: `1GiB`. -## max_untracked_memory {#max_untracked_memory} -Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'. - -Default value: `4MiB`. - -## min_untracked_memory {#min_untracked_memory} -Lower bound for untracked memory limit which is applied to threads with low memory consumption. Untracked memory limit equals thread memory usage divided by 16 and clamped between `min_untracked_memory` and `max_untracked_memory` for every thread. It guarantees that total untracked memory does not exceed 10% of current memory footprint even with a lot of small threads. To disable dynamic limit for untracked memory set value `4MiB`. - -Default value: `4KiB`. - ## Schema Inference settings See [schema inference](../../interfaces/schema-inference.md#schema-inference-modes) documentation for more details. diff --git a/src/Common/CurrentMemoryTracker.cpp b/src/Common/CurrentMemoryTracker.cpp index b1dcded0b23..02c7dc6e224 100644 --- a/src/Common/CurrentMemoryTracker.cpp +++ b/src/Common/CurrentMemoryTracker.cpp @@ -57,7 +57,6 @@ AllocationTrace CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory { auto res = memory_tracker->allocImpl(will_be, throw_if_memory_exceeded); current_thread->untracked_memory = 0; - current_thread->updateUntrackedMemoryLimit(memory_tracker->get()); return res; } else @@ -85,13 +84,6 @@ void CurrentMemoryTracker::check() std::ignore = memory_tracker->allocImpl(0, true); } -Int64 CurrentMemoryTracker::get() -{ - if (auto * memory_tracker = getMemoryTracker()) - return memory_tracker->get(); - return 0; -} - AllocationTrace CurrentMemoryTracker::alloc(Int64 size) { bool throw_if_memory_exceeded = true; @@ -111,12 +103,10 @@ AllocationTrace CurrentMemoryTracker::free(Int64 size) if (current_thread) { current_thread->untracked_memory -= size; - // Note that we use `max_untracked_memory` and not `untracked_memory_limit` to create hysteresis to avoid track/untrack cycles - if (current_thread->untracked_memory < -current_thread->max_untracked_memory) + if (current_thread->untracked_memory < -current_thread->untracked_memory_limit) { Int64 untracked_memory = current_thread->untracked_memory; current_thread->untracked_memory = 0; - current_thread->updateUntrackedMemoryLimit(memory_tracker->get() + untracked_memory); return memory_tracker->free(-untracked_memory); } } diff --git a/src/Common/CurrentMemoryTracker.h b/src/Common/CurrentMemoryTracker.h index 401eeed93dd..18a1e3f49b1 100644 --- a/src/Common/CurrentMemoryTracker.h +++ b/src/Common/CurrentMemoryTracker.h @@ -12,9 +12,7 @@ struct CurrentMemoryTracker /// This function should be called after memory deallocation. [[nodiscard]] static AllocationTrace free(Int64 size); - static void check(); - [[nodiscard]] static Int64 get(); /// Throws MEMORY_LIMIT_EXCEEDED (if it's allowed to throw exceptions) static void injectFault(); diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index db4854da707..0c02ab8fdb0 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -183,12 +183,6 @@ public: Int64 untracked_memory = 0; /// Each thread could new/delete memory in range of (-untracked_memory_limit, untracked_memory_limit) without access to common counters. Int64 untracked_memory_limit = 4 * 1024 * 1024; - /// To keep total untracked memory limited to `untracked_memory_ratio * RSS` we have to account threads with small and large memory footprint differently. - /// For this purpose we dynamically change `untracked_memory_limit` after every tracking event using a simple formula: - /// untracked_memory_limit = clamp(untracked_memory_ratio * cur_memory_bytes, min_untracked_memory, max_untracked_memory) - /// Note that this values are updated when thread is attached to a group - Int64 min_untracked_memory = 4 * 1024 * 1024; // Default value is kept 4MB mostly for tests and client (should be changed to 4KB as default value a setting) - Int64 max_untracked_memory = 4 * 1024 * 1024; /// Statistics of read and write rows/bytes Progress progress_in; @@ -315,12 +309,6 @@ public: void initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period); - void updateUntrackedMemoryLimit(Int64 current) - { - constexpr Int64 untracked_memory_ratio_bits = 4; // untracked_memory_ratio = 1.0 / (1 << untracked_memory_ratio_bits) = 1.0 / 16 = 6.25% - untracked_memory_limit = std::clamp(current >> untracked_memory_ratio_bits, min_untracked_memory, max_untracked_memory); - } - private: void applyGlobalSettings(); void applyQuerySettings(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d63d5e9d181..27ce54c03a7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -491,7 +491,6 @@ class IColumn; M(UInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \ M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \ M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \ - M(UInt64, min_untracked_memory, (4 * 1024), "Lower bound for untracked memory limit which is applied to threads with low memory consumption. Untracked memory limit equals thread_memory_usage/16 and clamped between min_untracked_memory and max_untracked_memory for every thread.", 0) \ M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ M(UInt64, memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 630ffa54b49..814c08c5705 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -96,7 +96,6 @@ static const std::map #include #include -#include #include #include #include @@ -211,12 +210,9 @@ void ThreadStatus::applyQuerySettings() query_id_from_query_context = query_context_ptr->getCurrentQueryId(); initQueryProfiler(); - max_untracked_memory = settings.max_untracked_memory; - if (settings.memory_profiler_step && settings.memory_profiler_step < static_cast(max_untracked_memory)) - max_untracked_memory = settings.memory_profiler_step; - min_untracked_memory = std::min(settings.min_untracked_memory, max_untracked_memory); - - updateUntrackedMemoryLimit(CurrentMemoryTracker::get()); + untracked_memory_limit = settings.max_untracked_memory; + if (settings.memory_profiler_step && settings.memory_profiler_step < static_cast(untracked_memory_limit)) + untracked_memory_limit = settings.memory_profiler_step; #if defined(OS_LINUX) /// Set "nice" value if required. diff --git a/tests/integration/test_failed_async_inserts/test.py b/tests/integration/test_failed_async_inserts/test.py index e7e504e565f..ecb506c36bc 100644 --- a/tests/integration/test_failed_async_inserts/test.py +++ b/tests/integration/test_failed_async_inserts/test.py @@ -45,7 +45,9 @@ def test_failed_async_inserts(started_cluster): ignore_error=True, ) - select_query = "SELECT value FROM system.events WHERE event == 'FailedAsyncInsertQuery' SETTINGS min_untracked_memory = '4Mi'" + select_query = ( + "SELECT value FROM system.events WHERE event == 'FailedAsyncInsertQuery'" + ) assert node.query(select_query) == "4\n" diff --git a/tests/integration/test_settings_constraints_distributed/test.py b/tests/integration/test_settings_constraints_distributed/test.py index d29b66b43bb..fbebbac276e 100644 --- a/tests/integration/test_settings_constraints_distributed/test.py +++ b/tests/integration/test_settings_constraints_distributed/test.py @@ -136,10 +136,7 @@ def test_select_clamps_settings(): ) assert ( - distributed.query( - query, - settings={"max_memory_usage": 1, "min_untracked_memory": 4 * 1024 * 1024}, - ) + distributed.query(query, settings={"max_memory_usage": 1}) == "node1\tmax_memory_usage\t11111111\n" "node1\treadonly\t0\n" "node2\tmax_memory_usage\t0\n" diff --git a/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql b/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql index de84846c1d7..c13a0859183 100644 --- a/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql +++ b/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql @@ -7,8 +7,7 @@ -- sizeof(HLL) is (2^K * 6 / 8) -- hence max_memory_usage for 100 rows = (96<<10)*100 = 9830400 -SET use_uncompressed_cache = 0; -SET min_untracked_memory = '4Mi'; +SET use_uncompressed_cache = 0; -- HashTable for UInt32 (used until (1<<13) elements), hence 8192 elements SELECT 'UInt32'; diff --git a/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql b/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql index 7961444dbc2..da2a387e07c 100644 --- a/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql +++ b/tests/queries/0_stateless/03030_system_flush_distributed_settings.sql @@ -13,8 +13,6 @@ create table dist_out as data engine=Distributed(test_shard_localhost, currentDa set prefer_localhost_replica=0; -set min_untracked_memory='4Mi'; -- Disable precise memory tracking - insert into dist_in select number/100, number from system.numbers limit 1e6 settings max_memory_usage='20Mi'; system flush distributed dist_in; -- { serverError MEMORY_LIMIT_EXCEEDED } system flush distributed dist_in settings max_memory_usage=0; From 69045adc10df36e0e1bd9097b2865e278ed16aea Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 7 Jun 2024 14:09:54 +0200 Subject: [PATCH 091/105] fix bad test --- .../test_attach_partition_using_copy/test.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_attach_partition_using_copy/test.py b/tests/integration/test_attach_partition_using_copy/test.py index e7163b1eb32..d5b07603dff 100644 --- a/tests/integration/test_attach_partition_using_copy/test.py +++ b/tests/integration/test_attach_partition_using_copy/test.py @@ -98,7 +98,8 @@ def create_destination_table(node, table_name, replicated): ) -def test_both_mergtree(start_cluster): +def test_both_mergetree(start_cluster): + cleanup([replica1, replica2]) create_source_table(replica1, "source", False) create_destination_table(replica1, "destination", False) @@ -120,12 +121,13 @@ def test_both_mergtree(start_cluster): def test_all_replicated(start_cluster): + cleanup([replica1, replica2]) create_source_table(replica1, "source", True) create_destination_table(replica1, "destination", True) create_destination_table(replica2, "destination", True) - replica1.query("SYSTEM SYNC REPLICA destination") replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source") + replica2.query("SYSTEM SYNC REPLICA destination") assert_eq_with_retry( replica1, @@ -154,12 +156,13 @@ def test_all_replicated(start_cluster): def test_only_destination_replicated(start_cluster): + cleanup([replica1, replica2]) create_source_table(replica1, "source", False) create_destination_table(replica1, "destination", True) create_destination_table(replica2, "destination", True) - replica1.query("SYSTEM SYNC REPLICA destination") replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source") + replica2.query("SYSTEM SYNC REPLICA destination") assert_eq_with_retry( replica1, @@ -188,6 +191,7 @@ def test_only_destination_replicated(start_cluster): def test_not_work_on_different_disk(start_cluster): + cleanup([replica1, replica2]) # Replace and move should not work on replace create_source_table(replica1, "source", False) create_destination_table(replica2, "destination", False) From b04fb116a5cd8c6064804a1949b97d31ccc95a56 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 7 Jun 2024 11:26:12 +0200 Subject: [PATCH 092/105] Compiler happy. Developer unhappy --- src/Storages/StorageGenerateRandom.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index ca9c6fb3226..8852e468c5e 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -706,7 +706,7 @@ Pipe StorageGenerateRandom::read( if (query_limit && num_streams * max_block_size > query_limit) { /// We want to avoid spawning more streams than necessary - num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); + num_streams = std::min(num_streams, static_cast(((query_limit + max_block_size - 1) / max_block_size))); } Pipes pipes; pipes.reserve(num_streams); From dd0f38db05a04509de548ec149a48cbb763ed4a6 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 7 Jun 2024 16:37:05 +0200 Subject: [PATCH 093/105] CI: Fix nightly workflow --- tests/ci/ci.py | 37 ++++++++++++++++++++++--------------- 1 file changed, 22 insertions(+), 15 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 55a18a2f335..73d1b1e4155 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -424,6 +424,7 @@ def _configure_jobs( s3: S3Helper, pr_info: PRInfo, ci_settings: CiSettings, + skip_jobs: bool, ) -> CiCache: """ returns CICache instance with configured job's data @@ -434,11 +435,14 @@ def _configure_jobs( """ # get all jobs - job_configs = CI_CONFIG.get_workflow_jobs_with_configs( - is_mq=pr_info.is_merge_queue, - is_docs_only=pr_info.has_changes_in_documentation_only(), - is_master=pr_info.is_master, - ) + if not skip_jobs: + job_configs = CI_CONFIG.get_workflow_jobs_with_configs( + is_mq=pr_info.is_merge_queue, + is_docs_only=pr_info.has_changes_in_documentation_only(), + is_master=pr_info.is_master, + ) + else: + job_configs = {} # filter jobs in accordance with ci settings job_configs = ci_settings.apply( @@ -447,7 +451,9 @@ def _configure_jobs( # check jobs in ci cache ci_cache = CiCache.calc_digests_and_create( - s3, job_configs, cache_enabled=not ci_settings.no_ci_cache and CI + s3, + job_configs, + cache_enabled=not ci_settings.no_ci_cache and not skip_jobs and CI, ) ci_cache.update() ci_cache.apply(job_configs, is_release=pr_info.is_release) @@ -971,6 +977,7 @@ def main() -> int: s3, pr_info, ci_settings, + args.skip_jobs, ) ci_cache.print_status() @@ -989,15 +996,15 @@ def main() -> int: result["ci_settings"] = ci_settings.as_dict() if not args.skip_jobs: result["stages_data"] = _generate_ci_stage_config(ci_cache.jobs_to_do) - result["jobs_data"] = { - "jobs_to_do": list(ci_cache.jobs_to_do), - "jobs_to_skip": ci_cache.jobs_to_skip, - "digests": ci_cache.job_digests, - "jobs_params": { - job: {"batches": config.batches, "num_batches": config.num_batches} - for job, config in ci_cache.jobs_to_do.items() - }, - } + result["jobs_data"] = { + "jobs_to_do": list(ci_cache.jobs_to_do), + "jobs_to_skip": ci_cache.jobs_to_skip, + "digests": ci_cache.job_digests, + "jobs_params": { + job: {"batches": config.batches, "num_batches": config.num_batches} + for job, config in ci_cache.jobs_to_do.items() + }, + } result["docker_data"] = docker_data ### CONFIGURE action: end From f50a951e8e3cf80652ea525b4232833c846507f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 7 Jun 2024 16:49:07 +0200 Subject: [PATCH 094/105] Fix innocuous data race in detectLanguage --- contrib/cld2 | 2 +- tests/queries/0_stateless/03168_cld2_tsan.reference | 2 ++ tests/queries/0_stateless/03168_cld2_tsan.sql | 10 ++++++++++ 3 files changed, 13 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03168_cld2_tsan.reference create mode 100644 tests/queries/0_stateless/03168_cld2_tsan.sql diff --git a/contrib/cld2 b/contrib/cld2 index bc6d493a2f6..217ba8b8805 160000 --- a/contrib/cld2 +++ b/contrib/cld2 @@ -1 +1 @@ -Subproject commit bc6d493a2f64ed1fc1c4c4b4294a542a04e04217 +Subproject commit 217ba8b8805b41557faadaa47bb6e99f2242eea3 diff --git a/tests/queries/0_stateless/03168_cld2_tsan.reference b/tests/queries/0_stateless/03168_cld2_tsan.reference new file mode 100644 index 00000000000..6c3cafd4a6d --- /dev/null +++ b/tests/queries/0_stateless/03168_cld2_tsan.reference @@ -0,0 +1,2 @@ +{'ja':0.62,'fr':0.36} +{'ja':0.62,'fr':0.36} diff --git a/tests/queries/0_stateless/03168_cld2_tsan.sql b/tests/queries/0_stateless/03168_cld2_tsan.sql new file mode 100644 index 00000000000..701a781c472 --- /dev/null +++ b/tests/queries/0_stateless/03168_cld2_tsan.sql @@ -0,0 +1,10 @@ +-- Tags: no-fasttest +-- Tag no-fasttest: depends on cld2 + +-- https://github.com/ClickHouse/ClickHouse/issues/64931 +SELECT detectLanguageMixed(materialize('二兎を追う者は一兎をも得ず二兎を追う者は一兎をも得ず A vaincre sans peril, on triomphe sans gloire.')) +GROUP BY + GROUPING SETS ( + ('a', toUInt256(1)), + (stringToH3(toFixedString(toFixedString('85283473ffffff', 14), 14)))) +SETTINGS allow_experimental_nlp_functions = 1; From 4c2783a0d8ae14b9da78815420f43e7b7eb9bab5 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 7 Jun 2024 18:15:57 +0200 Subject: [PATCH 095/105] CI: Builds in CI settings --- .github/PULL_REQUEST_TEMPLATE.md | 8 +++----- tests/ci/ci_config.py | 13 +++++++++++-- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 51a1a6e2df8..73b2155482e 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -48,19 +48,17 @@ At a minimum, the following information should be added (but add more as needed) - [ ] Allow: Stateful tests - [ ] Allow: Integration Tests - [ ] Allow: Performance tests +- [ ] Allow: Normal Builds +- [ ] Allow: Special Builds - [ ] Allow: All NOT Required Checks - [ ] Allow: batch 1, 2 for multi-batch jobs - [ ] Allow: batch 3, 4, 5, 6 for multi-batch jobs --- - [ ] Exclude: Style check - [ ] Exclude: Fast test -- [ ] Exclude: Integration Tests -- [ ] Exclude: Stateless tests -- [ ] Exclude: Stateful tests -- [ ] Exclude: Performance tests - [ ] Exclude: All with ASAN -- [ ] Exclude: All with Aarch64 - [ ] Exclude: All with TSAN, MSAN, UBSAN, Coverage +- [ ] Exclude: All with aarch64, release, debug --- - [ ] Do not test - [ ] Upload binaries for special builds diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index ca46088eb45..775e1f98b34 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -53,6 +53,8 @@ class CILabels(metaclass=WithIter): CI_SET_SYNC = "ci_set_sync" CI_SET_ARM = "ci_set_arm" CI_SET_REQUIRED = "ci_set_required" + CI_SET_NORMAL_BUILDS = "ci_set_normal_builds" + CI_SET_SPECIAL_BUILDS = "ci_set_special_builds" CI_SET_NON_REQUIRED = "ci_set_non_required" CI_SET_OLD_ANALYZER = "ci_set_old_analyzer" @@ -683,12 +685,13 @@ class CIConfig: return result def get_job_parents(self, check_name: str) -> List[str]: + if check_name in self.builds_report_config: + return self.builds_report_config[check_name].builds + res = [] check_name = normalize_string(check_name) - for config in ( self.build_config, - self.builds_report_config, self.test_configs, self.other_jobs_configs, ): @@ -899,6 +902,12 @@ CI_CONFIG = CIConfig( ] ), CILabels.CI_SET_REQUIRED: LabelConfig(run_jobs=REQUIRED_CHECKS), + CILabels.CI_SET_NORMAL_BUILDS: LabelConfig( + run_jobs=[JobNames.STYLE_CHECK, JobNames.BUILD_CHECK] + ), + CILabels.CI_SET_SPECIAL_BUILDS: LabelConfig( + run_jobs=[JobNames.STYLE_CHECK, JobNames.BUILD_CHECK_SPECIAL] + ), CILabels.CI_SET_NON_REQUIRED: LabelConfig( run_jobs=[job for job in JobNames if job not in REQUIRED_CHECKS] ), From d993fa5cc610f7cc3c68bb9a8b8d52ff60942f8b Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 7 Jun 2024 20:33:31 +0200 Subject: [PATCH 096/105] CI: Fix backports --- tests/ci/cherry_pick.py | 44 ++++++++++++++++++++--------------------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index e470621e2c5..d7a78c1d143 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -127,13 +127,11 @@ close it. to_pop.append(i) elif pr.head.ref.startswith(f"backport/{self.name}"): self.backport_pr = pr + self._backported = True to_pop.append(i) else: assert False, f"BUG! Invalid PR's branch [{pr.head.ref}]" - # Cherry-pick or backport PR found, set @backported flag for current release branch - self._backported = True - for i in reversed(to_pop): # Going from the tail to keep the order and pop greater index first prs.pop(i) @@ -218,6 +216,7 @@ close it. self.name, self.pr.number, ) + self._backported = True return except CalledProcessError: # There are most probably conflicts, they'll be resolved in PR @@ -247,7 +246,6 @@ close it. self.cherrypick_pr.add_to_labels(Labels.PR_CRITICAL_BUGFIX) elif Labels.PR_BUGFIX in [label.name for label in self.pr.labels]: self.cherrypick_pr.add_to_labels(Labels.PR_BUGFIX) - self._backported = True self._assign_new_pr(self.cherrypick_pr) # update cherrypick PR to get the state for PR.mergable self.cherrypick_pr.update() @@ -359,10 +357,10 @@ class Backport: self._fetch_from = fetch_from self.dry_run = dry_run - self.must_create_backport_label = ( - Labels.MUST_BACKPORT + self.must_create_backport_labels = ( + [Labels.MUST_BACKPORT] if self._repo_name == self._fetch_from - else Labels.MUST_BACKPORT_CLOUD + else [Labels.MUST_BACKPORT_CLOUD, Labels.MUST_BACKPORT] ) self.backport_created_label = ( Labels.PR_BACKPORTS_CREATED @@ -468,7 +466,7 @@ class Backport: query_args = { "query": f"type:pr repo:{self._fetch_from} -label:{self.backport_created_label}", "label": ",".join( - self.labels_to_backport + [self.must_create_backport_label] + self.labels_to_backport + self.must_create_backport_labels ), "merged": [since_date, tomorrow], } @@ -492,20 +490,22 @@ class Backport: def process_pr(self, pr: PullRequest) -> None: pr_labels = [label.name for label in pr.labels] - if self.must_create_backport_label in pr_labels: - branches = [ - ReleaseBranch(br, pr, self.repo, self.backport_created_label) - for br in self.release_branches - ] # type: List[ReleaseBranch] - else: - branches = [ - ReleaseBranch(br, pr, self.repo, self.backport_created_label) - for br in [ - label.split("-", 1)[0][1:] # v21.8-must-backport - for label in pr_labels - if label in self.labels_to_backport - ] - ] + # FIXME: currently backport to all branches, for branch-specified backports too + # Handle different branch name formats in cloud + # if self.must_create_backport_label in pr_labels: + branches = [ + ReleaseBranch(br, pr, self.repo, self.backport_created_label) + for br in self.release_branches + ] # type: List[ReleaseBranch] + # else: + # branches = [ + # ReleaseBranch(br, pr, self.repo, self.backport_created_label) + # for br in [ + # label.split("-", 1)[0][1:] # v21.8-must-backport + # for label in pr_labels + # if label in self.labels_to_backport + # ] + # ] assert branches, "BUG!" logging.info( From ca756b4d519f796d2b6a4f6f4a1cf62703d34f4b Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 7 Jun 2024 20:42:07 +0200 Subject: [PATCH 097/105] CI: Fix backports --- tests/ci/cherry_pick.py | 30 ++++++++++++++---------------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index d7a78c1d143..375a5f6214a 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -490,22 +490,20 @@ class Backport: def process_pr(self, pr: PullRequest) -> None: pr_labels = [label.name for label in pr.labels] - # FIXME: currently backport to all branches, for branch-specified backports too - # Handle different branch name formats in cloud - # if self.must_create_backport_label in pr_labels: - branches = [ - ReleaseBranch(br, pr, self.repo, self.backport_created_label) - for br in self.release_branches - ] # type: List[ReleaseBranch] - # else: - # branches = [ - # ReleaseBranch(br, pr, self.repo, self.backport_created_label) - # for br in [ - # label.split("-", 1)[0][1:] # v21.8-must-backport - # for label in pr_labels - # if label in self.labels_to_backport - # ] - # ] + if self.must_create_backport_labels in pr_labels or self._repo_name != self._fetch_from: + branches = [ + ReleaseBranch(br, pr, self.repo, self.backport_created_label) + for br in self.release_branches + ] # type: List[ReleaseBranch] + else: + branches = [ + ReleaseBranch(br, pr, self.repo, self.backport_created_label) + for br in [ + label.split("-", 1)[0][1:] # v21.8-must-backport + for label in pr_labels + if label in self.labels_to_backport + ] + ] assert branches, "BUG!" logging.info( From 18e12c48a464712f406a6654938a45c403adabaf Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 7 Jun 2024 18:49:22 +0000 Subject: [PATCH 098/105] Automatic style fix --- tests/ci/cherry_pick.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 375a5f6214a..47e255b1882 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -490,7 +490,10 @@ class Backport: def process_pr(self, pr: PullRequest) -> None: pr_labels = [label.name for label in pr.labels] - if self.must_create_backport_labels in pr_labels or self._repo_name != self._fetch_from: + if ( + any(label in pr_labels for label in self.must_create_backport_labels) + or self._repo_name != self._fetch_from + ): branches = [ ReleaseBranch(br, pr, self.repo, self.backport_created_label) for br in self.release_branches From c7ec61e8e6044eee5b4823147fd3147e67ee6858 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 7 Jun 2024 17:37:29 -0300 Subject: [PATCH 099/105] Update settings.md --- docs/en/operations/settings/settings.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 65b0fa3f385..79d0ca4f151 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3878,6 +3878,10 @@ Possible values: Default value: 30. +:::note +It's applicable only to the default profile. A server reboot is required for the changes to take effect. +::: + ## http_receive_timeout {#http_receive_timeout} HTTP receive timeout (in seconds). From 3238bcf90af9dfb7d5ce86b834744f15c11cec7f Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 8 Jun 2024 09:48:56 +0200 Subject: [PATCH 100/105] Review changes --- .../functions/other-functions.md | 115 ++++++++---------- 1 file changed, 50 insertions(+), 65 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 4639f9d0991..5e63d9824b4 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -228,29 +228,24 @@ Query: ```sql DROP TABLE IF EXISTS test; CREATE TABLE test (n UInt8) ENGINE = Memory; --- Insert 3 blocks: -INSERT INTO test VALUES (1); -INSERT INTO test VALUES (1),(2); -INSERT INTO test VALUES (1),(2),(3); -SELECT blockSize(), n FROM (SELECT * FROM test); +INSERT INTO test +SELECT * FROM system.numbers LIMIT 5; + +SELECT blockSize() +FROM test; ``` Result: ```response - ┌─blockSize()─┬─n─┐ -1. │ 1 │ 1 │ - └─────────────┴───┘ - ┌─blockSize()─┬─n─┐ -2. │ 3 │ 1 │ -3. │ 3 │ 2 │ -4. │ 3 │ 3 │ - └─────────────┴───┘ - ┌─blockSize()─┬─n─┐ -5. │ 2 │ 1 │ -6. │ 2 │ 2 │ - └─────────────┴───┘ + ┌─blockSize()─┐ +1. │ 5 │ +2. │ 5 │ +3. │ 5 │ +4. │ 5 │ +5. │ 5 │ + └─────────────┘ ``` ## byteSize @@ -3723,11 +3718,7 @@ Result: ## lowCardinalityIndices -For each row in the current [block](../../development/architecture.md/#block-block), returns the index of the value in the dictionary of unique values for columns of [LowCardinality](../data-types/lowcardinality.md) type. - -:::note -The first unique value encountered in the block is enumerated from 1. -::: +Returns the position of a value in the dictionary of a [LowCardinality](../data-types/lowcardinality.md) column. Positions start at 1. Since LowCardinality have per-part dictionaries, this function may return different positions for the same value in different parts. **Syntax** @@ -3741,7 +3732,7 @@ lowCardinalityIndices(col) **Returned value** -- returns the index of the value in the dictionary of unique values, for each row in the current block. [UInt64](../data-types/int-uint.md). +- The position of the value in the dictionary of the current part. [UInt64](../data-types/int-uint.md). **Example** @@ -3751,10 +3742,10 @@ Query: DROP TABLE IF EXISTS test; CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory; --- insert two blocks of data: +-- create two parts: -INSERT INTO test VALUES ('one'),('two'),('one'),('one'),('two'); -INSERT INTO test VALUES ('three'),('two'),('one'),('two'),('two'),('three'); +INSERT INTO test VALUES ('ab'), ('cd'), ('ab'), ('ab'), ('df'); +INSERT INTO test VALUES ('ef'), ('cd'), ('ab'), ('cd'), ('ef'); SELECT s, lowCardinalityIndices(s) FROM test; ``` @@ -3762,29 +3753,24 @@ SELECT s, lowCardinalityIndices(s) FROM test; Result: ```response - ┌─s───┬─lowCardinalityIndices(s)─┐ -1. │ one │ 1 │ -2. │ two │ 2 │ -3. │ one │ 1 │ -4. │ one │ 1 │ -5. │ two │ 2 │ - └─────┴──────────────────────────┘ - ┌─s─────┬─lowCardinalityIndices(s)─┐ - 6. │ three │ 1 │ - 7. │ two │ 2 │ - 8. │ one │ 3 │ - 9. │ two │ 2 │ -10. │ two │ 2 │ -11. │ three │ 1 │ - └───────┴──────────────────────────┘ + ┌─s──┬─lowCardinalityIndices(s)─┐ +1. │ ab │ 1 │ +2. │ cd │ 2 │ +3. │ ab │ 1 │ +4. │ ab │ 1 │ +5. │ df │ 3 │ + └────┴──────────────────────────┘ + ┌─s──┬─lowCardinalityIndices(s)─┐ + 6. │ ef │ 1 │ + 7. │ cd │ 2 │ + 8. │ ab │ 3 │ + 9. │ cd │ 2 │ +10. │ ef │ 1 │ + └────┴──────────────────────────┘ ``` ## lowCardinalityKeys -For each row in the current [block](../../development/architecture.md/#block-block), returns the keys (unique values) in the dictionary of unique values for columns of [LowCardinality](../data-types/lowcardinality.md) type. - -:::note -If the column size is less than the dictionary size, then values will be cut. If it is greater, then defaults will be added. -::: +Returns the dictionary values of a [LowCardinality](../data-types/lowcardinality.md) column. If the block is smaller or larger than the dictionary size, the result will be truncated or extended with default values. Since LowCardinality have per-part dictionaries, this function may return different dictionary values in different parts. **Syntax** @@ -3798,7 +3784,7 @@ lowCardinalityIndices(col) **Returned value** -- returns the keys of the dictionary, for each row in the current block. [UInt64](../data-types/int-uint.md). +- The dictionary keys. [UInt64](../data-types/int-uint.md). **Example** @@ -3808,10 +3794,10 @@ Query: DROP TABLE IF EXISTS test; CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory; --- insert two blocks of data: +-- create two parts: -INSERT INTO test VALUES ('one'),('two'),('one'),('one'),('two'); -INSERT INTO test VALUES ('three'),('two'),('one'),('two'),('two'),('three'); +INSERT INTO test VALUES ('ab'), ('cd'), ('ab'), ('ab'), ('df'); +INSERT INTO test VALUES ('ef'), ('cd'), ('ab'), ('cd'), ('ef'); SELECT s, lowCardinalityKeys(s) FROM test; ``` @@ -3819,19 +3805,18 @@ SELECT s, lowCardinalityKeys(s) FROM test; Result: ```response - ┌─s───┬─lowCardinalityKeys(s)─┐ -1. │ one │ │ -2. │ two │ one │ -3. │ one │ two │ -4. │ one │ │ -5. │ two │ │ - └─────┴───────────────────────┘ - ┌─s─────┬─lowCardinalityKeys(s)─┐ - 6. │ three │ │ - 7. │ two │ three │ - 8. │ one │ two │ - 9. │ two │ one │ -10. │ two │ │ -11. │ three │ │ - └───────┴───────────────────────┘ + ┌─s──┬─lowCardinalityKeys(s)─┐ +1. │ ef │ │ +2. │ cd │ ef │ +3. │ ab │ cd │ +4. │ cd │ ab │ +5. │ ef │ │ + └────┴───────────────────────┘ + ┌─s──┬─lowCardinalityKeys(s)─┐ + 6. │ ab │ │ + 7. │ cd │ ab │ + 8. │ ab │ cd │ + 9. │ ab │ df │ +10. │ df │ │ + └────┴───────────────────────┘ ``` From 9f90f2463aff93765483f65dc6c1884abb1339e6 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 8 Jun 2024 18:06:10 +0200 Subject: [PATCH 101/105] fix --- tests/ci/cherry_pick.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 47e255b1882..629464d0422 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -533,8 +533,8 @@ class Backport: br.process(self.dry_run) for br in branches: - assert br.backported, f"BUG! backport to branch [{br}] failed" - self.mark_pr_backported(pr) + if br.backported: + self.mark_pr_backported(pr) def mark_pr_backported(self, pr: PullRequest) -> None: if self.dry_run: From cdd2957a31fae18e44c2230a73e0770c545f793c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 9 Jun 2024 19:09:33 +0000 Subject: [PATCH 102/105] Move MergeTree setting docs into MergeTree settings docs page --- .../mergetree-family/mergetree.md | 136 ++++------------ .../settings/merge-tree-settings.md | 154 ++++++++++++++---- 2 files changed, 151 insertions(+), 139 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 803b753fe0a..0e53d7525b4 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -6,35 +6,26 @@ sidebar_label: MergeTree # MergeTree -The `MergeTree` engine and other engines of this family (`*MergeTree`) are the most commonly used and most robust ClickHouse table engines. +The `MergeTree` engine and other engines of the `MergeTree` family (e.g. `ReplacingMergeTree`, `AggregatingMergeTree` ) are the most commonly used and most robust table engines in ClickHouse. -Engines in the `MergeTree` family are designed for inserting a very large amount of data into a table. The data is quickly written to the table part by part, then rules are applied for merging the parts in the background. This method is much more efficient than continually rewriting the data in storage during insert. +`MergeTree`-family table engines are designed for high data ingest rates and huge data volumes. +Insert operations create table parts which are merged by a background process with other table parts. -Main features: +Main features of `MergeTree`-family table engines. -- Stores data sorted by primary key. +- The table's primary key determines the sort order within each table part (clustered index). The primary key also does not reference individual rows but blocks of 8192 rows called granules. This makes primary keys of huge data sets small enough to remain loaded in main memory, while still providing fast access to on-disk data. - This allows you to create a small sparse index that helps find data faster. +- Tables can be partitioned using an arbitrary partition expression. Partition pruning ensures partitions are omitted from reading when the query allows it. -- Partitions can be used if the [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md) is specified. +- Data can be replicated across multiple cluster nodes for high availability, failover, and zero downtime upgrades. See [Data replication](/docs/en/engines/table-engines/mergetree-family/replication.md). - ClickHouse supports certain operations with partitions that are more efficient than general operations on the same data with the same result. ClickHouse also automatically cuts off the partition data where the partitioning key is specified in the query. +- `MergeTree` table engines support various statistics kinds and sampling methods to help query optimization. -- Data replication support. - - The family of `ReplicatedMergeTree` tables provides data replication. For more information, see [Data replication](/docs/en/engines/table-engines/mergetree-family/replication.md). - -- Data sampling support. - - If necessary, you can set the data sampling method in the table. - -:::info -The [Merge](/docs/en/engines/table-engines/special/merge.md/#merge) engine does not belong to the `*MergeTree` family. +:::note +Despite a similar name, the [Merge](/docs/en/engines/table-engines/special/merge.md/#merge) engine is different from `*MergeTree` engines. ::: -If you need to update rows frequently, we recommend using the [`ReplacingMergeTree`](/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md) table engine. Using `ALTER TABLE my_table UPDATE` to update rows triggers a mutation, which causes parts to be re-written and uses IO/resources. With `ReplacingMergeTree`, you can simply insert the updated rows and the old rows will be replaced according to the table sorting key. - -## Creating a Table {#table_engine-mergetree-creating-a-table} +## Creating Tables {#table_engine-mergetree-creating-a-table} ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] @@ -59,23 +50,24 @@ ORDER BY expr [SETTINGS name = value, ...] ``` -For a description of parameters, see the [CREATE query description](/docs/en/sql-reference/statements/create/table.md). +For a detailed description of the parameters, see the [CREATE TABLE](/docs/en/sql-reference/statements/create/table.md) statement ### Query Clauses {#mergetree-query-clauses} #### ENGINE -`ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine does not have parameters. +`ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine has no parameters. #### ORDER_BY `ORDER BY` — The sorting key. -A tuple of column names or arbitrary expressions. Example: `ORDER BY (CounterID, EventDate)`. +A tuple of column names or arbitrary expressions. Example: `ORDER BY (CounterID + 1, EventDate)`. -ClickHouse uses the sorting key as a primary key if the primary key is not defined explicitly by the `PRIMARY KEY` clause. +If no primary key is defined (i.e. `PRIMARY KEY` was not specified), ClickHouse uses the the sorting key as primary key. -Use the `ORDER BY tuple()` syntax, if you do not need sorting, or set `create_table_empty_primary_key_by_default` to `true` to use the `ORDER BY tuple()` syntax by default. See [Selecting the Primary Key](#selecting-the-primary-key). +If no sorting is required, you can use syntax `ORDER BY tuple()`. +Alternatively, if setting `create_table_empty_primary_key_by_default` is enabled, `ORDER BY tuple()` is implicitly added to `CREATE TABLE` statements. See [Selecting a Primary Key](#selecting-a-primary-key). #### PARTITION BY @@ -87,100 +79,32 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da `PRIMARY KEY` — The primary key if it [differs from the sorting key](#choosing-a-primary-key-that-differs-from-the-sorting-key). Optional. -By default the primary key is the same as the sorting key (which is specified by the `ORDER BY` clause). Thus in most cases it is unnecessary to specify a separate `PRIMARY KEY` clause. +Specifying a sorting key (using `ORDER BY` clause) implicitly specifies a primary key. +It is usually not necessary to specify the primary key in addition to the primary key. #### SAMPLE BY -`SAMPLE BY` — An expression for sampling. Optional. +`SAMPLE BY` — A sampling expression. Optional. -If a sampling expression is used, the primary key must contain it. The result of a sampling expression must be an unsigned integer. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. +If specified, it must be contained in the primary key. +The sampling expression must result in an unsigned integer. + +Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. #### TTL -`TTL` — A list of rules specifying storage duration of rows and defining logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional. +`TTL` — A list of rules that specify the storage duration of rows and the logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional. -Expression must have one `Date` or `DateTime` column as a result. Example: -``` -TTL date + INTERVAL 1 DAY -``` +Expression must result in a `Date` or `DateTime`, e.g. `TTL date + INTERVAL 1 DAY`. Type of the rule `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'|GROUP BY` specifies an action to be done with the part if the expression is satisfied (reaches current time): removal of expired rows, moving a part (if expression is satisfied for all rows in a part) to specified disk (`TO DISK 'xxx'`) or to volume (`TO VOLUME 'xxx'`), or aggregating values in expired rows. Default type of the rule is removal (`DELETE`). List of multiple rules can be specified, but there should be no more than one `DELETE` rule. + For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl) -### SETTINGS -Additional parameters that control the behavior of the `MergeTree` (optional): +#### Settings -#### index_granularity - -`index_granularity` — Maximum number of data rows between the marks of an index. Default value: 8192. See [Data Storage](#mergetree-data-storage). - -#### index_granularity_bytes - -`index_granularity_bytes` — Maximum size of data granules in bytes. Default value: 10Mb. To restrict the granule size only by number of rows, set to 0 (not recommended). See [Data Storage](#mergetree-data-storage). - -#### min_index_granularity_bytes - -`min_index_granularity_bytes` — Min allowed size of data granules in bytes. Default value: 1024b. To provide a safeguard against accidentally creating tables with very low index_granularity_bytes. See [Data Storage](#mergetree-data-storage). - -#### enable_mixed_granularity_parts - -`enable_mixed_granularity_parts` — Enables or disables transitioning to control the granule size with the `index_granularity_bytes` setting. Before version 19.11, there was only the `index_granularity` setting for restricting granule size. The `index_granularity_bytes` setting improves ClickHouse performance when selecting data from tables with big rows (tens and hundreds of megabytes). If you have tables with big rows, you can enable this setting for the tables to improve the efficiency of `SELECT` queries. - -#### use_minimalistic_part_header_in_zookeeper - -`use_minimalistic_part_header_in_zookeeper` — Storage method of the data parts headers in ZooKeeper. If `use_minimalistic_part_header_in_zookeeper=1`, then ZooKeeper stores less data. For more information, see the [setting description](/docs/en/operations/server-configuration-parameters/settings.md/#server-settings-use_minimalistic_part_header_in_zookeeper) in “Server configuration parameters”. - -#### min_merge_bytes_to_use_direct_io - -`min_merge_bytes_to_use_direct_io` — The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option). If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled. Default value: `10 * 1024 * 1024 * 1024` bytes. - -#### merge_with_ttl_timeout - -`merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with delete TTL. Default value: `14400` seconds (4 hours). -#### merge_with_recompression_ttl_timeout - -`merge_with_recompression_ttl_timeout` — Minimum delay in seconds before repeating a merge with recompression TTL. Default value: `14400` seconds (4 hours). - -#### try_fetch_recompressed_part_timeout - -`try_fetch_recompressed_part_timeout` — Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression. Default value: `7200` seconds (2 hours). - -#### write_final_mark - -`write_final_mark` — Enables or disables writing the final index mark at the end of data part (after the last byte). Default value: 1. Don’t turn it off. - -#### merge_max_block_size - -`merge_max_block_size` — Maximum number of rows in block for merge operations. Default value: 8192. - -#### storage_policy - -`storage_policy` — Storage policy. See [Using Multiple Block Devices for Data Storage](#table_engine-mergetree-multiple-volumes). - -#### min_bytes_for_wide_part - -`min_bytes_for_wide_part`, `min_rows_for_wide_part` — Minimum number of bytes/rows in a data part that can be stored in `Wide` format. You can set one, both or none of these settings. See [Data Storage](#mergetree-data-storage). - -#### max_parts_in_total - -`max_parts_in_total` — Maximum number of parts in all partitions. - -#### max_compress_block_size - -`max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. You can also specify this setting in the global settings (see [max_compress_block_size](/docs/en/operations/settings/settings.md/#max-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. - -#### min_compress_block_size - -`min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. You can also specify this setting in the global settings (see [min_compress_block_size](/docs/en/operations/settings/settings.md/#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. - -#### max_partitions_to_read - -`max_partitions_to_read` — Limits the maximum number of partitions that can be accessed in one query. You can also specify setting [max_partitions_to_read](/docs/en/operations/settings/merge-tree-settings.md/#max-partitions-to-read) in the global setting. - -#### allow_experimental_optimized_row_order - -`allow_experimental_optimized_row_order` - Experimental. Enables the optimization of the row order during inserts to improve the compressability of the data for compression codecs (e.g. LZ4). Analyzes and reorders the data, and thus increases the CPU overhead of inserts. +See [MergeTree Settings](../../../operations/settings/merge-tree-settings.md). **Example of Sections Setting** @@ -270,7 +194,7 @@ ClickHouse does not require a unique primary key. You can insert multiple rows w You can use `Nullable`-typed expressions in the `PRIMARY KEY` and `ORDER BY` clauses but it is strongly discouraged. To allow this feature, turn on the [allow_nullable_key](/docs/en/operations/settings/settings.md/#allow-nullable-key) setting. The [NULLS_LAST](/docs/en/sql-reference/statements/select/order-by.md/#sorting-of-special-values) principle applies for `NULL` values in the `ORDER BY` clause. -### Selecting the Primary Key {#selecting-the-primary-key} +### Selecting a Primary Key {#selecting-a-primary-key} The number of columns in the primary key is not explicitly limited. Depending on the data structure, you can include more or fewer columns in the primary key. This may: diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index c3f303dcd38..d791683ac2b 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -3,9 +3,126 @@ slug: /en/operations/settings/merge-tree-settings title: "MergeTree tables settings" --- -The values of `merge_tree` settings (for all MergeTree tables) can be viewed in the table `system.merge_tree_settings`, they can be overridden in `config.xml` in the `merge_tree` section, or set in the `SETTINGS` section of each table. +System table `system.merge_tree_settings` shows the globally set MergeTree settings. -These are example overrides for `max_suspicious_broken_parts`: +MergeTree settings can be set in the `merge_tree` section of the server config file, or specified for each `MergeTree` table individually in +the `SETTINGS` clause of the `CREATE TABLE` statement. + +Example for customizing setting `max_suspicious_broken_parts`: + +Configure the default for all `MergeTree` tables in the server configuration file: + +``` text + + 5 + +``` + +Set for a particular table: + +``` sql +CREATE TABLE tab +( + `A` Int64 +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS max_suspicious_broken_parts = 500; +``` + +Change the settings for a particular table using `ALTER TABLE ... MODIFY SETTING`: + +```sql +ALTER TABLE tab MODIFY SETTING max_suspicious_broken_parts = 100; + +-- reset to global default (value from system.merge_tree_settings) +ALTER TABLE tab RESET SETTING max_suspicious_broken_parts; +``` + +## index_granularity + +Maximum number of data rows between the marks of an index. + +Default value: 8192. + +## index_granularity_bytes + +Maximum size of data granules in bytes. + +Default value: 10Mb. + +To restrict the granule size only by number of rows, set to 0 (not recommended). + +## min_index_granularity_bytes + +Min allowed size of data granules in bytes. + +Default value: 1024b. + +To provide a safeguard against accidentally creating tables with very low index_granularity_bytes. + +## enable_mixed_granularity_parts + +Enables or disables transitioning to control the granule size with the `index_granularity_bytes` setting. Before version 19.11, there was only the `index_granularity` setting for restricting granule size. The `index_granularity_bytes` setting improves ClickHouse performance when selecting data from tables with big rows (tens and hundreds of megabytes). If you have tables with big rows, you can enable this setting for the tables to improve the efficiency of `SELECT` queries. + +## use_minimalistic_part_header_in_zookeeper + +Storage method of the data parts headers in ZooKeeper. If enabled, ZooKeeper stores less data. For details, see [here](../server-configuration-parameters/settings.md/#server-settings-use_minimalistic_part_header_in_zookeeper). + +## min_merge_bytes_to_use_direct_io + +The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. +When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. +If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option). +If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled. + +Default value: `10 * 1024 * 1024 * 1024` bytes. + +## merge_with_ttl_timeout + +Minimum delay in seconds before repeating a merge with delete TTL. + +Default value: `14400` seconds (4 hours). + +## merge_with_recompression_ttl_timeout + +Minimum delay in seconds before repeating a merge with recompression TTL. + +Default value: `14400` seconds (4 hours). + +## write_final_mark + +Enables or disables writing the final index mark at the end of data part (after the last byte). + +Default value: 1. + +Don’t change or bad things will happen. + +## storage_policy + +Storage policy. + +## min_bytes_for_wide_part + +Minimum number of bytes/rows in a data part that can be stored in `Wide` format. +You can set one, both or none of these settings. + +## max_compress_block_size + +Maximum size of blocks of uncompressed data before compressing for writing to a table. +You can also specify this setting in the global settings (see [max_compress_block_size](/docs/en/operations/settings/settings.md/#max-compress-block-size) setting). +The value specified when table is created overrides the global value for this setting. + +## min_compress_block_size + +Minimum size of blocks of uncompressed data required for compression when writing the next mark. +You can also specify this setting in the global settings (see [min_compress_block_size](/docs/en/operations/settings/settings.md/#min-compress-block-size) setting). +The value specified when table is created overrides the global value for this setting. + +## max_partitions_to_read + +Limits the maximum number of partitions that can be accessed in one query. +You can also specify setting [max_partitions_to_read](/docs/en/operations/settings/merge-tree-settings.md/#max-partitions-to-read) in the global setting. ## max_suspicious_broken_parts @@ -17,37 +134,6 @@ Possible values: Default value: 100. -Override example in `config.xml`: - -``` text - - 5 - -``` - -An example to set in `SETTINGS` for a particular table: - -``` sql -CREATE TABLE foo -( - `A` Int64 -) -ENGINE = MergeTree -ORDER BY tuple() -SETTINGS max_suspicious_broken_parts = 500; -``` - -An example of changing the settings for a specific table with the `ALTER TABLE ... MODIFY SETTING` command: - -``` sql -ALTER TABLE foo - MODIFY SETTING max_suspicious_broken_parts = 100; - --- reset to default (use value from system.merge_tree_settings) -ALTER TABLE foo - RESET SETTING max_suspicious_broken_parts; -``` - ## parts_to_throw_insert {#parts-to-throw-insert} If the number of active parts in a single partition exceeds the `parts_to_throw_insert` value, `INSERT` is interrupted with the `Too many parts (N). Merges are processing significantly slower than inserts` exception. @@ -301,6 +387,8 @@ Default value: 10800 ## try_fetch_recompressed_part_timeout +Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression. + Recompression works slow in most cases, so we don't start merge with recompression until this timeout and trying to fetch recompressed part from replica which assigned this merge with recompression. Possible values: From d59a17014461048ec215a45ce742f813c17f95c6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 07:05:36 +0000 Subject: [PATCH 103/105] Docs for MergeTree: Capitalized SETTINGS --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 0e53d7525b4..f0c4e1b0e34 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -102,7 +102,7 @@ Type of the rule `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'|GROUP BY` specifies an ac For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl) -#### Settings +#### SETTINGS See [MergeTree Settings](../../../operations/settings/merge-tree-settings.md). From d80cba90fb467b40d098c028c163fa1deba07dd9 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 10 Jun 2024 11:22:06 +0200 Subject: [PATCH 104/105] CI: Remove fuzzer build from normal CI run (bugfix) --- tests/ci/ci_config.py | 35 ++++++++++++++++++++++++++++++----- 1 file changed, 30 insertions(+), 5 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 775e1f98b34..412a55b8534 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -685,9 +685,6 @@ class CIConfig: return result def get_job_parents(self, check_name: str) -> List[str]: - if check_name in self.builds_report_config: - return self.builds_report_config[check_name].builds - res = [] check_name = normalize_string(check_name) for config in ( @@ -903,10 +900,38 @@ CI_CONFIG = CIConfig( ), CILabels.CI_SET_REQUIRED: LabelConfig(run_jobs=REQUIRED_CHECKS), CILabels.CI_SET_NORMAL_BUILDS: LabelConfig( - run_jobs=[JobNames.STYLE_CHECK, JobNames.BUILD_CHECK] + run_jobs=[ + JobNames.STYLE_CHECK, + JobNames.BUILD_CHECK, + Build.PACKAGE_RELEASE, + Build.PACKAGE_AARCH64, + Build.PACKAGE_ASAN, + Build.PACKAGE_UBSAN, + Build.PACKAGE_TSAN, + Build.PACKAGE_MSAN, + Build.PACKAGE_DEBUG, + Build.BINARY_RELEASE, + Build.PACKAGE_RELEASE_COVERAGE, + Build.FUZZERS, + ] ), CILabels.CI_SET_SPECIAL_BUILDS: LabelConfig( - run_jobs=[JobNames.STYLE_CHECK, JobNames.BUILD_CHECK_SPECIAL] + run_jobs=[ + JobNames.STYLE_CHECK, + JobNames.BUILD_CHECK_SPECIAL, + Build.BINARY_TIDY, + Build.BINARY_DARWIN, + Build.BINARY_AARCH64, + Build.BINARY_AARCH64_V80COMPAT, + Build.BINARY_FREEBSD, + Build.BINARY_DARWIN_AARCH64, + Build.BINARY_PPC64LE, + Build.BINARY_RISCV64, + Build.BINARY_S390X, + Build.BINARY_LOONGARCH64, + Build.BINARY_AMD64_COMPAT, + Build.BINARY_AMD64_MUSL, + ] ), CILabels.CI_SET_NON_REQUIRED: LabelConfig( run_jobs=[job for job in JobNames if job not in REQUIRED_CHECKS] From 132aa996a3f25fa98f3edc1ad92bbc22725d4c8d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 10 Jun 2024 13:49:39 +0200 Subject: [PATCH 105/105] Revert "Fix duplicating Delete events in blob_storage_log" --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 5 ++--- src/Interpreters/SystemLog.cpp | 7 +------ 2 files changed, 3 insertions(+), 9 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index afc13251f5b..ae719f5cde4 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -382,7 +382,6 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e { std::vector current_chunk; String keys; - size_t first_position = current_position; for (; current_position < objects.size() && current_chunk.size() < chunk_size_limit; ++current_position) { Aws::S3::Model::ObjectIdentifier obj; @@ -408,9 +407,9 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e { const auto * outcome_error = outcome.IsSuccess() ? nullptr : &outcome.GetError(); auto time_now = std::chrono::system_clock::now(); - for (size_t i = first_position; i < current_position; ++i) + for (const auto & object : objects) blob_storage_log->addEvent(BlobStorageLogElement::EventType::Delete, - uri.bucket, objects[i].remote_path, objects[i].local_path, objects[i].bytes_size, + uri.bucket, object.remote_path, object.local_path, object.bytes_size, outcome_error, time_now); } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 3b25deeb59d..5e0ce2cb0de 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -504,10 +504,6 @@ void SystemLog::flushImpl(const std::vector & to_flush, Block block(std::move(log_element_columns)); MutableColumns columns = block.mutateColumns(); - - for (auto & column : columns) - column->reserve(to_flush.size()); - for (const auto & elem : to_flush) elem.appendToBlock(columns); @@ -536,8 +532,7 @@ void SystemLog::flushImpl(const std::vector & to_flush, } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Failed to flush system log {} with {} entries up to offset {}", - table_id.getNameForLogs(), to_flush.size(), to_flush_end)); + tryLogCurrentException(__PRETTY_FUNCTION__); } queue->confirm(to_flush_end);