From 51ff688f9f9d3c6d8d848130531079ca3a923ac5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Fri, 22 Mar 2024 16:04:10 +0300 Subject: [PATCH 01/50] Use storage policy to find convert_to_replicated flag path --- src/Databases/DatabaseOrdinary.cpp | 23 +++- src/Databases/DatabaseOrdinary.h | 2 +- .../test_modify_engine_on_restart/common.py | 10 +- .../configs/config.d/storage_policies.xml | 42 ++++++++ .../test_storage_policies.py | 102 ++++++++++++++++++ 5 files changed, 170 insertions(+), 9 deletions(-) create mode 100644 tests/integration/test_modify_engine_on_restart/configs/config.d/storage_policies.xml create mode 100644 tests/integration/test_modify_engine_on_restart/test_storage_policies.py diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 95bdcfc7dce..65d5f21bbea 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -31,6 +31,7 @@ #include #include #include +#include #include @@ -95,16 +96,21 @@ static void setReplicatedEngine(ASTCreateQuery * create_query, ContextPtr contex create_query->storage->set(create_query->storage->engine, engine->clone()); } -String DatabaseOrdinary::getConvertToReplicatedFlagPath(const String & name, bool tableStarted) +String DatabaseOrdinary::getConvertToReplicatedFlagPath(const String & name, const StoragePolicyPtr storage_policy, bool tableStarted) { fs::path data_path; + if (storage_policy->getDisks().empty()) + data_path = getContext()->getPath(); + else + data_path = storage_policy->getDisks()[0]->getPath(); + if (!tableStarted) { auto create_query = tryGetCreateTableQuery(name, getContext()); - data_path = fs::path(getContext()->getPath()) / getTableDataPath(create_query->as()); + data_path = data_path / getTableDataPath(create_query->as()); } else - data_path = fs::path(getContext()->getPath()) / getTableDataPath(name); + data_path = data_path / getTableDataPath(name); return (data_path / CONVERT_TO_REPLICATED_FLAG_NAME).string(); } @@ -120,7 +126,14 @@ void DatabaseOrdinary::convertMergeTreeToReplicatedIfNeeded(ASTPtr ast, const Qu if (!create_query->storage || !create_query->storage->engine->name.ends_with("MergeTree") || create_query->storage->engine->name.starts_with("Replicated") || create_query->storage->engine->name.starts_with("Shared")) return; - auto convert_to_replicated_flag_path = getConvertToReplicatedFlagPath(qualified_name.table, false); + /// Get table's storage policy + MergeTreeSettings default_settings = getContext()->getMergeTreeSettings(); + auto * query_settings = create_query->storage->settings->as(); + auto policy = getContext()->getStoragePolicy(default_settings.storage_policy); + if (Field * policy_setting = query_settings->changes.tryGet("storage_policy")) + policy = getContext()->getStoragePolicy(policy_setting->get()); + + auto convert_to_replicated_flag_path = getConvertToReplicatedFlagPath(qualified_name.table, policy, false); if (!fs::exists(convert_to_replicated_flag_path)) return; @@ -288,7 +301,7 @@ void DatabaseOrdinary::restoreMetadataAfterConvertingToReplicated(StoragePtr tab if (!rmt) return; - auto convert_to_replicated_flag_path = getConvertToReplicatedFlagPath(name.table, true); + auto convert_to_replicated_flag_path = getConvertToReplicatedFlagPath(name.table, table->getStoragePolicy(), true); if (!fs::exists(convert_to_replicated_flag_path)) return; diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index 7089540337a..9e4bd035576 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -85,7 +85,7 @@ protected: private: void convertMergeTreeToReplicatedIfNeeded(ASTPtr ast, const QualifiedTableName & qualified_name, const String & file_name); void restoreMetadataAfterConvertingToReplicated(StoragePtr table, const QualifiedTableName & name); - String getConvertToReplicatedFlagPath(const String & name, bool tableStarted); + String getConvertToReplicatedFlagPath(const String & name, StoragePolicyPtr storage_policy, bool tableStarted); }; } diff --git a/tests/integration/test_modify_engine_on_restart/common.py b/tests/integration/test_modify_engine_on_restart/common.py index 18b6c2dc1d7..3d8529d70ce 100644 --- a/tests/integration/test_modify_engine_on_restart/common.py +++ b/tests/integration/test_modify_engine_on_restart/common.py @@ -2,9 +2,13 @@ from helpers.cluster import ClickHouseCluster def get_table_path(node, table, database): - return node.query( - sql=f"SELECT data_paths FROM system.tables WHERE table = '{table}' and database = '{database}'" - ).strip("'[]\n") + return ( + node.query( + sql=f"SELECT data_paths FROM system.tables WHERE table = '{table}' and database = '{database}' LIMIT 1" + ) + .split(",")[0] + .strip("'[]\n") + ) def check_flags_deleted(node, database_name, tables): diff --git a/tests/integration/test_modify_engine_on_restart/configs/config.d/storage_policies.xml b/tests/integration/test_modify_engine_on_restart/configs/config.d/storage_policies.xml new file mode 100644 index 00000000000..c456b5c1f12 --- /dev/null +++ b/tests/integration/test_modify_engine_on_restart/configs/config.d/storage_policies.xml @@ -0,0 +1,42 @@ + + + + + 1024 + + + /jbod1/ + + + /jbod2/ + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + + + + jbod1 + jbod2 + + + + + + + s3 + + + + + + + + jbod + + diff --git a/tests/integration/test_modify_engine_on_restart/test_storage_policies.py b/tests/integration/test_modify_engine_on_restart/test_storage_policies.py new file mode 100644 index 00000000000..e49af164ed7 --- /dev/null +++ b/tests/integration/test_modify_engine_on_restart/test_storage_policies.py @@ -0,0 +1,102 @@ +import pytest +from test_modify_engine_on_restart.common import check_flags_deleted, set_convert_flags +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +ch1 = cluster.add_instance( + "ch1", + main_configs=[ + "configs/config.d/clusters.xml", + "configs/config.d/distributed_ddl.xml", + "configs/config.d/storage_policies.xml", + ], + with_zookeeper=True, + with_minio=True, + macros={"replica": "node1"}, + stay_alive=True, +) + +database_name = "modify_engine_storage_policies" + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def q(node, query): + return node.query(database=database_name, sql=query) + + +def create_tables(): + # Implicit jbod (set default in config) + q( + ch1, + "CREATE TABLE jbod_imp ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A;", + ) + + # Explicit jbod + q( + ch1, + """ + CREATE TABLE jbod_exp ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A + SETTINGS storage_policy='jbod'; + """, + ) + + # s3 + q( + ch1, + """ + CREATE TABLE s3 ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A + SETTINGS storage_policy='s3'; + """, + ) + + # Default + q( + ch1, + """ + CREATE TABLE default ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A + SETTINGS storage_policy='default'; + """, + ) + + +def check_tables(converted): + engine_prefix = "" + if converted: + engine_prefix = "Replicated" + + assert ( + q( + ch1, + f"SELECT name, engine FROM system.tables WHERE database = '{database_name}'", + ).strip() + == f"default\t{engine_prefix}MergeTree\njbod_exp\t{engine_prefix}MergeTree\njbod_imp\t{engine_prefix}MergeTree\ns3\t{engine_prefix}MergeTree" + ) + + +def test_modify_engine_on_restart(started_cluster): + ch1.query("CREATE DATABASE " + database_name) + + create_tables() + + check_tables(False) + + ch1.restart_clickhouse() + + check_tables(False) + + set_convert_flags(ch1, database_name, ["default", "jbod_exp", "jbod_imp", "s3"]) + + ch1.restart_clickhouse() + + check_flags_deleted(ch1, database_name, ["default", "jbod_exp", "jbod_imp", "s3"]) + + check_tables(True) From acb623f802e1ceb0e80038609da7781908ea8cc6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Fri, 22 Mar 2024 16:11:23 +0300 Subject: [PATCH 02/50] Mention problem in docs --- docs/en/engines/table-engines/mergetree-family/replication.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index f70e275fd4e..257084d8691 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -304,10 +304,10 @@ We use the term `MergeTree` to refer to all table engines in the `MergeTree fami If you had a `MergeTree` table that was manually replicated, you can convert it to a replicated table. You might need to do this if you have already collected a large amount of data in a `MergeTree` table and now you want to enable replication. -`MergeTree` table can be automatically converted on server restart if `convert_to_replicated` flag is set at the table's data directory (`/var/lib/clickhouse/store/xxx/xxxyyyyy-yyyy-yyyy-yyyy-yyyyyyyyyyyy/` for `Atomic` database). +`MergeTree` table can be automatically converted on server restart if `convert_to_replicated` flag is set at the table's data directory (`/store/xxx/xxxyyyyy-yyyy-yyyy-yyyy-yyyyyyyyyyyy/` for `Atomic` database). Create empty `convert_to_replicated` file and the table will be loaded as replicated on next server restart. -This query can be used to get the table's data path. +This query can be used to get the table's data path. It table has many data paths, you have to use the first one. ```sql SELECT data_paths FROM system.tables WHERE table = 'table_name' AND database = 'database_name'; From f2e0a3be1c917b17cddfb720f24fddd8f89117d9 Mon Sep 17 00:00:00 2001 From: Kirill <71129570+kirillgarbar@users.noreply.github.com> Date: Mon, 25 Mar 2024 01:31:59 +0300 Subject: [PATCH 03/50] Fix typo in docs Co-authored-by: Alexey Milovidov --- docs/en/engines/table-engines/mergetree-family/replication.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index 257084d8691..0fbdbec6fff 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -307,7 +307,7 @@ If you had a `MergeTree` table that was manually replicated, you can convert it `MergeTree` table can be automatically converted on server restart if `convert_to_replicated` flag is set at the table's data directory (`/store/xxx/xxxyyyyy-yyyy-yyyy-yyyy-yyyyyyyyyyyy/` for `Atomic` database). Create empty `convert_to_replicated` file and the table will be loaded as replicated on next server restart. -This query can be used to get the table's data path. It table has many data paths, you have to use the first one. +This query can be used to get the table's data path. If table has many data paths, you have to use the first one. ```sql SELECT data_paths FROM system.tables WHERE table = 'table_name' AND database = 'database_name'; From a47b407495c99ad049f04185b45c66e87f28881b Mon Sep 17 00:00:00 2001 From: peter279k Date: Tue, 26 Mar 2024 13:35:35 +0800 Subject: [PATCH 04/50] Fix PR #60656 for install check tests --- tests/ci/install_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index 71e2198f533..54a18c7e26c 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -33,7 +33,7 @@ set -e trap "bash -ex /packages/preserve_logs.sh" ERR test_env='TEST_THE_DEFAULT_PARAMETER=15' echo "$test_env" >> /etc/default/clickhouse -systemctl start clickhouse-server +systemctl restart clickhouse-server clickhouse-client -q 'SELECT version()' grep "$test_env" /proc/$(cat /var/run/clickhouse-server/clickhouse-server.pid)/environ""" initd_test = r"""#!/bin/bash From 500379801bb8c27c1822fd1540dbcf3c1e16a8aa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Wed, 27 Mar 2024 23:09:14 +0300 Subject: [PATCH 05/50] Check query_settings nullptr --- src/Databases/DatabaseOrdinary.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 65d5f21bbea..f4191727b7c 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -31,7 +31,6 @@ #include #include #include -#include #include @@ -128,10 +127,10 @@ void DatabaseOrdinary::convertMergeTreeToReplicatedIfNeeded(ASTPtr ast, const Qu /// Get table's storage policy MergeTreeSettings default_settings = getContext()->getMergeTreeSettings(); - auto * query_settings = create_query->storage->settings->as(); auto policy = getContext()->getStoragePolicy(default_settings.storage_policy); - if (Field * policy_setting = query_settings->changes.tryGet("storage_policy")) - policy = getContext()->getStoragePolicy(policy_setting->get()); + if (auto * query_settings = create_query->storage->settings->as()) + if (Field * policy_setting = query_settings->changes.tryGet("storage_policy")) + policy = getContext()->getStoragePolicy(policy_setting->safeGet()); auto convert_to_replicated_flag_path = getConvertToReplicatedFlagPath(qualified_name.table, policy, false); From 4dd35b4d4f98899da8cdf64f2d9921b7e32eafc9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Tue, 2 Apr 2024 23:36:49 +0300 Subject: [PATCH 06/50] Remove useless cast --- src/Databases/DatabaseOrdinary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index f4191727b7c..5f3d8983936 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -128,7 +128,7 @@ void DatabaseOrdinary::convertMergeTreeToReplicatedIfNeeded(ASTPtr ast, const Qu /// Get table's storage policy MergeTreeSettings default_settings = getContext()->getMergeTreeSettings(); auto policy = getContext()->getStoragePolicy(default_settings.storage_policy); - if (auto * query_settings = create_query->storage->settings->as()) + if (auto * query_settings = create_query->storage->settings) if (Field * policy_setting = query_settings->changes.tryGet("storage_policy")) policy = getContext()->getStoragePolicy(policy_setting->safeGet()); From e2acd0e93f2a5819bdb22a28b87640935bcee308 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 12 Apr 2024 10:18:43 +0000 Subject: [PATCH 07/50] CI: add FT to MQ remove Style from master --- .github/workflows/master.yml | 21 ++++++++-------- .github/workflows/pull_request.yml | 4 +-- tests/ci/ci.py | 39 ++++++++++++++---------------- tests/ci/sync_pr.py | 2 ++ 4 files changed, 33 insertions(+), 33 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 9a719a205d4..384141b78e7 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -59,16 +59,17 @@ jobs: uses: ./.github/workflows/reusable_docker.yml with: data: ${{ needs.RunConfig.outputs.data }} - StyleCheck: - needs: [RunConfig, BuildDockers] - if: ${{ !failure() && !cancelled() }} - uses: ./.github/workflows/reusable_test.yml - with: - test_name: Style check - runner_type: style-checker - data: ${{ needs.RunConfig.outputs.data }} - run_command: | - python3 style_check.py --no-push + # Tested in MQ + # StyleCheck: + # needs: [RunConfig, BuildDockers] + # if: ${{ !failure() && !cancelled() }} + # uses: ./.github/workflows/reusable_test.yml + # with: + # test_name: Style check + # runner_type: style-checker + # data: ${{ needs.RunConfig.outputs.data }} + # run_command: | + # python3 style_check.py --no-push CompatibilityCheckX86: needs: [RunConfig, BuilderDebRelease] if: ${{ !failure() && !cancelled() }} diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 0eac9a9a722..c2e76de5e14 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -83,7 +83,7 @@ jobs: ${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} RCSK FastTest: - needs: [RunConfig, StyleCheck] + needs: [RunConfig, BuildDockers] if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'Fast test') }} uses: ./.github/workflows/reusable_test.yml with: @@ -97,7 +97,7 @@ jobs: # for main CI chain # Builds_1: - needs: [RunConfig, FastTest] + needs: [RunConfig, StyleCheck, FastTest] if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Builds_1') }} # using callable wf (reusable_stage.yml) allows to group all nested jobs under a tab uses: ./.github/workflows/reusable_build_stage.yml diff --git a/tests/ci/ci.py b/tests/ci/ci.py index f11d62e9136..bd6117dc858 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1261,18 +1261,6 @@ def _print_results(result: Any, outfile: Optional[str], pretty: bool = False) -> raise AssertionError(f"Unexpected type for 'res': {type(result)}") -def _update_config_for_docs_only(jobs_data: dict) -> None: - DOCS_CHECK_JOBS = [JobNames.DOCS_CHECK, JobNames.STYLE_CHECK] - print(f"NOTE: Will keep only docs related jobs: [{DOCS_CHECK_JOBS}]") - jobs_to_do = jobs_data.get("jobs_to_do", []) - jobs_data["jobs_to_do"] = [job for job in jobs_to_do if job in DOCS_CHECK_JOBS] - jobs_data["jobs_to_wait"] = { - job: params - for job, params in jobs_data["jobs_to_wait"].items() - if job in DOCS_CHECK_JOBS - } - - def _configure_docker_jobs(docker_digest_or_latest: bool) -> Dict: print("::group::Docker images check") # generate docker jobs data @@ -1332,8 +1320,20 @@ def _configure_jobs( jobs_to_skip: List[str] = [] digests: Dict[str, str] = {} + # FIXME: find better place for these config variables + DOCS_CHECK_JOBS = [JobNames.DOCS_CHECK, JobNames.STYLE_CHECK] + MQ_JOBS = [JobNames.STYLE_CHECK, JobNames.FAST_TEST] + if pr_info.has_changes_in_documentation_only(): + print(f"WARNING: Only docs are changed - will run only [{DOCS_CHECK_JOBS}]") + if pr_info.is_merge_queue(): + print(f"WARNING: It's a MQ run - will run only [{MQ_JOBS}]") + print("::group::Job Digests") for job in CI_CONFIG.job_generator(pr_info.head_ref if CI else "dummy_branch_name"): + if pr_info.is_merge_queue() and job not in MQ_JOBS: + continue + if pr_info.has_changes_in_documentation_only() and job not in DOCS_CHECK_JOBS: + continue digest = job_digester.get_job_digest(CI_CONFIG.get_digest_config(job)) digests[job] = digest print(f" job [{job.rjust(50)}] has digest [{digest}]") @@ -1436,10 +1436,11 @@ def _configure_jobs( ) if pr_info.is_merge_queue(): - # FIXME: Quick support for MQ workflow which is only StyleCheck for now - jobs_to_do = [JobNames.STYLE_CHECK] - jobs_to_skip = [] - print(f"NOTE: This is Merge Queue CI: set jobs to do: [{jobs_to_do}]") + # no need to run pending job in MQ, since it's pending - it's not affected by current checnge + for job_to_wait in jobs_to_wait: + if job_to_wait in jobs_to_do: + print(f"Remove pending job [{job_to_wait}] from MQ workflow") + jobs_to_do.remove(job_to_wait) return { "digests": digests, @@ -1902,9 +1903,6 @@ def main() -> int: else {} ) - if not args.skip_jobs and pr_info.has_changes_in_documentation_only(): - _update_config_for_docs_only(jobs_data) - if not args.skip_jobs: ci_cache = CiCache(s3, jobs_data["digests"]) @@ -1928,8 +1926,7 @@ def main() -> int: jobs_to_skip.append(job) del jobs_params[job] - # set planned jobs as pending in the CI cache if on the master - if pr_info.is_master(): + # set planned jobs as pending in the CI cache if on the master for job in jobs_data["jobs_to_do"]: config = CI_CONFIG.get_job_config(job) if config.run_always or config.run_by_label: diff --git a/tests/ci/sync_pr.py b/tests/ci/sync_pr.py index acff7ba541b..639436ab3ac 100644 --- a/tests/ci/sync_pr.py +++ b/tests/ci/sync_pr.py @@ -54,10 +54,12 @@ def merge_sync_pr(gh, sync_pr): def set_sync_status(gh, pr_info, sync_pr): if not sync_pr or not sync_pr.mergeable: + print("Sync PR is not mergeable") post_commit_status( get_commit(gh, pr_info.sha), FAILURE, "", "Sync PR failure", "A Sync" ) else: + print("Sync PR is mergeable") post_commit_status(get_commit(gh, pr_info.sha), SUCCESS, "", "", "A Sync") From 830598c83e09f4eeb514e0c5300cb520f365b9e9 Mon Sep 17 00:00:00 2001 From: peter279k Date: Fri, 12 Apr 2024 19:25:24 +0800 Subject: [PATCH 08/50] Add functions tryBase58 and tryBase64Decode usage --- .../functions/string-functions.md | 52 +++++++++++++++++++ 1 file changed, 52 insertions(+) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index f4d66730011..87950d65cd5 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -867,6 +867,34 @@ Result: Like `base58Decode` but returns an empty string in case of error. +**Syntax** + +```sql +tryBase58Decode(encoded) +``` + +**Parameters** + +- `encoded`: [String](../../sql-reference/data-types/string.md) column or constant. If the string is not a valid Base58-encoded value, returns an empty string in case of error. + +**Returned value** + +- A string containing the decoded value of the argument. + +**Examples** + +Query: + +```sql +SELECT tryBase58Decode('3dc8KtHrwM') as res; +``` + +```response +┌─res─────┐ +│ Encoded │ +└─────────┘ +``` + ## base64Encode Encodes a String or FixedString as base64. @@ -883,6 +911,30 @@ Alias: `FROM_BASE64`. Like `base64Decode` but returns an empty string in case of error. +**Syntax** + +```sql +tryBase64Decode(encoded) +``` + +**Parameters** + +- `encoded`: [String](../../sql-reference/data-types/string.md) column or constant. If the string is not a valid Base58-encoded value, returns an empty string in case of error. + +**Examples** + +Query: + +```sql +SELECT tryBase64Decode('RW5jb2RlZA==') as res; +``` + +```response +┌─res─────┐ +│ Encoded │ +└─────────┘ +``` + ## endsWith {#endswith} Returns whether string `str` ends with `suffix`. From 91d4284bdf37efca0c1a2250495e93fcbd8b0850 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 15 Apr 2024 16:21:09 +0200 Subject: [PATCH 09/50] don't allow relative paths when installing --- programs/install/Install.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index f2ef3857d63..6bed114238a 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -729,6 +729,15 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } } + /// Don't allow relative paths because install script may cd to / when installing + /// And having path=./ may break the system + if (log_path.is_relative()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Log path is relative: {}", log_path.string()); + if (data_path.is_relative()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Data path is relative: {}", data_path.string()); + if (pid_path.is_relative()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Pid path is relative: {}", pid_path.string()); + /// Create directories for data and log. if (fs::exists(log_path)) From 5ebdcf06ca97543c9262c4271b48bfe3e0321062 Mon Sep 17 00:00:00 2001 From: Artur Malchanau Date: Sat, 20 Apr 2024 12:33:52 +0300 Subject: [PATCH 10/50] Fix taking full part if part contains less than 'limit' rows Do not take full part if part contains less than 'limit' rows. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 6bdd060513c..662e833f090 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -987,7 +987,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( /// We take full part if it contains enough marks or /// if we know limit and part contains less than 'limit' rows. - bool take_full_part = marks_in_part <= need_marks || (input_order_info->limit && input_order_info->limit < part.getRowsCount()); + bool take_full_part = marks_in_part <= need_marks || (input_order_info->limit && input_order_info->limit > part.getRowsCount()); /// We take the whole part if it is small enough. if (take_full_part) From 0575d02b81de8f4efb59e9ddca88ac91833d96b9 Mon Sep 17 00:00:00 2001 From: Artur Malchanau Date: Sat, 20 Apr 2024 23:20:20 +0300 Subject: [PATCH 11/50] Fix 02499_monotonicity_toUnixTimestamp64 test Fix 02499_monotonicity_toUnixTimestamp64 test by using max_threads=1 to make read_rows more predictable. --- .../0_stateless/02499_monotonicity_toUnixTimestamp64.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02499_monotonicity_toUnixTimestamp64.sh b/tests/queries/0_stateless/02499_monotonicity_toUnixTimestamp64.sh index 59b6e2abb06..aa1f635f380 100755 --- a/tests/queries/0_stateless/02499_monotonicity_toUnixTimestamp64.sh +++ b/tests/queries/0_stateless/02499_monotonicity_toUnixTimestamp64.sh @@ -13,8 +13,7 @@ $CLICKHOUSE_CLIENT -q "create table t(ts DateTime64) engine=MergeTree order by t max_block_size=8192 query_id="${CLICKHOUSE_DATABASE}_02499_$RANDOM$RANDOM" -$CLICKHOUSE_CLIENT --query_id="$query_id" -q "select ts from t order by toUnixTimestamp64Nano(ts) limit 10 format Null settings max_block_size = $max_block_size, optimize_read_in_order = 1;" +$CLICKHOUSE_CLIENT --query_id="$query_id" -q "select ts from t order by toUnixTimestamp64Nano(ts) limit 10 format Null settings max_block_size = $max_block_size, optimize_read_in_order = 1, max_threads = 1;" $CLICKHOUSE_CLIENT -q "system flush logs;" $CLICKHOUSE_CLIENT --param_query_id="$query_id" -q "select read_rows <= $max_block_size from system.query_log where event_date >= yesterday() and current_database = '$CLICKHOUSE_DATABASE' and query_id = {query_id:String} and type = 'QueryFinish';" - From e5339413f3ac2186a85ad780c6fb64862e1ac5a4 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 22 Apr 2024 12:24:26 +0200 Subject: [PATCH 12/50] Correct `tanh` return type Change return type from incorrect Float64 to correct Float32 --- docs/en/sql-reference/functions/math-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index e59878f901b..2d0ae28aefc 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -575,7 +575,7 @@ tanh(x) - Values from the interval: `-1 < tanh(x) < 1`. -Type: [Float64](../../sql-reference/data-types/float.md#float32-float64). +Type: [Float32](../../sql-reference/data-types/float.md#float32-float64). **Example** From d3d0d4f6c02a79f40da6c4ae31c23ae7892a0e89 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 22 Apr 2024 11:36:27 +0100 Subject: [PATCH 13/50] empty commit From ae8b22b6e0c00700dd57475260a05f5133caf1f7 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 22 Apr 2024 13:48:28 +0200 Subject: [PATCH 14/50] Update math-functions argument types. Many functions have argument type listed as `Float64` type when it can be `Float*`,`(U)Int*` or in some cases `Decimal*` --- .../sql-reference/functions/math-functions.md | 68 ++++++++++--------- 1 file changed, 35 insertions(+), 33 deletions(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 2d0ae28aefc..5ebc6191010 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -8,7 +8,7 @@ sidebar_label: Mathematical ## e -Returns e ([Euler's constant](https://en.wikipedia.org/wiki/Euler%27s_constant)) +Returns e ([Euler's constant](https://en.wikipedia.org/wiki/Euler%27s_constant)). **Syntax** @@ -45,7 +45,7 @@ exp(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -65,7 +65,7 @@ Alias: `ln(x)` **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -83,7 +83,7 @@ exp2(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -111,7 +111,7 @@ log2(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -129,7 +129,7 @@ exp10(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -157,7 +157,7 @@ log10(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -173,7 +173,7 @@ sqrt(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -189,7 +189,7 @@ cbrt(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -207,7 +207,7 @@ erf(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -239,7 +239,7 @@ erfc(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -257,7 +257,7 @@ lgamma(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -275,7 +275,7 @@ gamma(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -293,7 +293,7 @@ sin(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -323,7 +323,7 @@ cos(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -341,7 +341,7 @@ tan(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -359,7 +359,7 @@ asin(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -377,7 +377,7 @@ acos(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -395,7 +395,7 @@ atan(x) **Arguments** -- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md) +- `x` - [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -434,7 +434,7 @@ cosh(x) **Arguments** -- `x` — The angle, in radians. Values from the interval: `-∞ < x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — The angle, in radians. Values from the interval: `-∞ < x < +∞`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -468,7 +468,7 @@ acosh(x) **Arguments** -- `x` — Hyperbolic cosine of angle. Values from the interval: `1 <= x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — Hyperbolic cosine of angle. Values from the interval: `1 <= x < +∞`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -502,7 +502,7 @@ sinh(x) **Arguments** -- `x` — The angle, in radians. Values from the interval: `-∞ < x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — The angle, in radians. Values from the interval: `-∞ < x < +∞`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -536,7 +536,7 @@ asinh(x) **Arguments** -- `x` — Hyperbolic sine of angle. Values from the interval: `-∞ < x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — Hyperbolic sine of angle. Values from the interval: `-∞ < x < +∞`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -569,13 +569,13 @@ tanh(x) **Arguments** -- `x` — The angle, in radians. Values from the interval: `-∞ < x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — The angle, in radians. Values from the interval: `-∞ < x < +∞`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** - Values from the interval: `-1 < tanh(x) < 1`. -Type: [Float32](../../sql-reference/data-types/float.md#float32-float64). +Type: [Float*](../../sql-reference/data-types/float.md#float32-float64). **Example** @@ -601,7 +601,7 @@ atanh(x) **Arguments** -- `x` — Hyperbolic tangent of angle. Values from the interval: `–1 < x < 1`. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — Hyperbolic tangent of angle. Values from the interval: `–1 < x < 1`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -635,8 +635,8 @@ atan2(y, x) **Arguments** -- `y` — y-coordinate of the point through which the ray passes. [Float64](../../sql-reference/data-types/float.md#float32-float64). -- `x` — x-coordinate of the point through which the ray passes. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `y` — y-coordinate of the point through which the ray passes. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md). +- `x` — x-coordinate of the point through which the ray passes. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md). **Returned value** @@ -670,8 +670,8 @@ hypot(x, y) **Arguments** -- `x` — The first cathetus of a right-angle triangle. [Float64](../../sql-reference/data-types/float.md#float32-float64). -- `y` — The second cathetus of a right-angle triangle. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — The first cathetus of a right-angle triangle. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md). +- `y` — The second cathetus of a right-angle triangle. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md). **Returned value** @@ -705,7 +705,7 @@ log1p(x) **Arguments** -- `x` — Values from the interval: `-1 < x < +∞`. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — Values from the interval: `-1 < x < +∞`. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -747,6 +747,8 @@ sign(x) - 0 for `x = 0` - 1 for `x > 0` +Type: [Int8](../../sql-reference/data-types/int-uint.md). + **Examples** Sign for the zero value: @@ -803,7 +805,7 @@ degrees(x) **Arguments** -- `x` — Input in radians. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — Input in radians. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** @@ -837,7 +839,7 @@ radians(x) **Arguments** -- `x` — Input in degrees. [Float64](../../sql-reference/data-types/float.md#float32-float64). +- `x` — Input in degrees. [(U)Int*](../../sql-reference/data-types/int-uint.md), [Float*](../../sql-reference/data-types/float.md) or [Decimal*](../../sql-reference/data-types/decimal.md). **Returned value** From c75574a1d66b8e9795cbee1209c3a7b26439ea85 Mon Sep 17 00:00:00 2001 From: Dani Pozo Date: Mon, 22 Apr 2024 18:49:46 +0200 Subject: [PATCH 15/50] Azure: fix building final url from endpoint object --- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h index 20bf05d5ba6..c8e556d012e 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h @@ -21,6 +21,8 @@ struct AzureBlobStorageEndpoint String getEndpoint() { String url = storage_account_url; + if (url.ends_with('/')) + url = url.substr(0, url.size() - 1); if (!account_name.empty()) url += "/" + account_name; From d44062edb29eb8d1512725edfb0c1c1c6799dc73 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 23 Apr 2024 15:11:22 +0800 Subject: [PATCH 16/50] Fix LowCardinality(Nullable) key in hyperrectangle --- src/Storages/MergeTree/KeyCondition.cpp | 16 ++++++++++------ .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 ++++-- ...lity_nullable_non_first_primary_key.reference | 1 + ...ardinality_nullable_non_first_primary_key.sql | 10 ++++++++++ 4 files changed, 25 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.reference create mode 100644 tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 2d57ea40c9c..80f425c3e06 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -2250,9 +2250,11 @@ static BoolMask forAnyHyperrectangle( if (left_bounded && right_bounded) hyperrectangle[prefix_size] = Range(left_keys[prefix_size], true, right_keys[prefix_size], true); else if (left_bounded) - hyperrectangle[prefix_size] = Range::createLeftBounded(left_keys[prefix_size], true, data_types[prefix_size]->isNullable()); + hyperrectangle[prefix_size] + = Range::createLeftBounded(left_keys[prefix_size], true, isNullableOrLowCardinalityNullable(data_types[prefix_size])); else if (right_bounded) - hyperrectangle[prefix_size] = Range::createRightBounded(right_keys[prefix_size], true, data_types[prefix_size]->isNullable()); + hyperrectangle[prefix_size] + = Range::createRightBounded(right_keys[prefix_size], true, isNullableOrLowCardinalityNullable(data_types[prefix_size])); return callback(hyperrectangle); } @@ -2262,13 +2264,15 @@ static BoolMask forAnyHyperrectangle( if (left_bounded && right_bounded) hyperrectangle[prefix_size] = Range(left_keys[prefix_size], false, right_keys[prefix_size], false); else if (left_bounded) - hyperrectangle[prefix_size] = Range::createLeftBounded(left_keys[prefix_size], false, data_types[prefix_size]->isNullable()); + hyperrectangle[prefix_size] + = Range::createLeftBounded(left_keys[prefix_size], false, isNullableOrLowCardinalityNullable(data_types[prefix_size])); else if (right_bounded) - hyperrectangle[prefix_size] = Range::createRightBounded(right_keys[prefix_size], false, data_types[prefix_size]->isNullable()); + hyperrectangle[prefix_size] + = Range::createRightBounded(right_keys[prefix_size], false, isNullableOrLowCardinalityNullable(data_types[prefix_size])); for (size_t i = prefix_size + 1; i < key_size; ++i) { - if (data_types[i]->isNullable()) + if (isNullableOrLowCardinalityNullable(data_types[i])) hyperrectangle[i] = Range::createWholeUniverse(); else hyperrectangle[i] = Range::createWholeUniverseWithoutNull(); @@ -2324,7 +2328,7 @@ BoolMask KeyCondition::checkInRange( key_ranges.reserve(used_key_size); for (size_t i = 0; i < used_key_size; ++i) { - if (data_types[i]->isNullable()) + if (isNullableOrLowCardinalityNullable(data_types[i])) key_ranges.push_back(Range::createWholeUniverse()); else key_ranges.push_back(Range::createWholeUniverseWithoutNull()); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 345872efddf..cd0a7615762 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -383,7 +383,8 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (has_lower_limit) { if (!key_condition.addCondition( - sampling_key.column_names[0], Range::createLeftBounded(lower, true, sampling_key.data_types[0]->isNullable()))) + sampling_key.column_names[0], + Range::createLeftBounded(lower, true, isNullableOrLowCardinalityNullable(sampling_key.data_types[0])))) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Sampling column not in primary key"); ASTPtr args = std::make_shared(); @@ -401,7 +402,8 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (has_upper_limit) { if (!key_condition.addCondition( - sampling_key.column_names[0], Range::createRightBounded(upper, false, sampling_key.data_types[0]->isNullable()))) + sampling_key.column_names[0], + Range::createRightBounded(upper, false, isNullableOrLowCardinalityNullable(sampling_key.data_types[0])))) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Sampling column not in primary key"); ASTPtr args = std::make_shared(); diff --git a/tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.reference b/tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.reference new file mode 100644 index 00000000000..712cd79a3f2 --- /dev/null +++ b/tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.reference @@ -0,0 +1 @@ +1650 diff --git a/tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.sql b/tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.sql new file mode 100644 index 00000000000..457c288da69 --- /dev/null +++ b/tests/queries/0_stateless/03129_low_cardinality_nullable_non_first_primary_key.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS small; + +CREATE TABLE small (`dt` DateTime, `user_email` LowCardinality(Nullable(String))) +ENGINE = MergeTree order by (dt, user_email) settings allow_nullable_key = 1, min_bytes_for_wide_part=0, min_rows_for_wide_part=0; + +INSERT INTO small (dt, user_email) SELECT number, if(number % 3 = 2, NULL, number) FROM numbers(1e2); + +SELECT SUM(dt::int) FROM small WHERE user_email IS NULL; + +DROP TABLE small; From 771219a118f0b348bb27a3b6be84a4dade640e11 Mon Sep 17 00:00:00 2001 From: Dani Pozo Date: Tue, 23 Apr 2024 12:34:26 +0200 Subject: [PATCH 17/50] Apply suggestion to use pop_back --- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h index c8e556d012e..8470b843e10 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h @@ -22,7 +22,7 @@ struct AzureBlobStorageEndpoint { String url = storage_account_url; if (url.ends_with('/')) - url = url.substr(0, url.size() - 1); + url.pop_back(); if (!account_name.empty()) url += "/" + account_name; From 7c13ea020c7894dbabe8e73cf64d2a80a27a3006 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 23 Apr 2024 14:17:30 +0300 Subject: [PATCH 18/50] PlannerJoins remove unused comments --- src/Planner/PlannerJoins.cpp | 112 +++-------------------------------- 1 file changed, 9 insertions(+), 103 deletions(-) diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 94ee249106a..78b0e966ca4 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -125,9 +125,8 @@ TableExpressionSet extractTableExpressionsSet(const QueryTreeNodePtr & node) return res; } -std::optional extractJoinTableSideFromExpression(//const ActionsDAG::Node * expression_root_node, +std::optional extractJoinTableSideFromExpression( const IQueryTreeNode * expression_root_node, - //const std::unordered_set & join_expression_dag_input_nodes, const TableExpressionSet & left_table_expressions, const TableExpressionSet & right_table_expressions, const JoinNode & join_node) @@ -136,30 +135,11 @@ std::optional extractJoinTableSideFromExpression(//const ActionsD std::vector nodes_to_process; nodes_to_process.push_back(expression_root_node); - // std::cerr << "==== extractJoinTableSideFromExpression\n"; - // std::cerr << "inp nodes" << std::endl; - // for (const auto * node : join_expression_dag_input_nodes) - // std::cerr << reinterpret_cast(node) << ' ' << node->result_name << std::endl; - - - // std::cerr << "l names" << std::endl; - // for (const auto & l : left_table_expression_columns_names) - // std::cerr << l << std::endl; - - // std::cerr << "r names" << std::endl; - // for (const auto & r : right_table_expression_columns_names) - // std::cerr << r << std::endl; - - // const auto * left_table_expr = join_node.getLeftTableExpression().get(); - // const auto * right_table_expr = join_node.getRightTableExpression().get(); - while (!nodes_to_process.empty()) { const auto * node_to_process = nodes_to_process.back(); nodes_to_process.pop_back(); - //std::cerr << "... " << reinterpret_cast(node_to_process) << ' ' << node_to_process->result_name << std::endl; - if (const auto * function_node = node_to_process->as()) { for (const auto & child : function_node->getArguments()) @@ -172,22 +152,7 @@ std::optional extractJoinTableSideFromExpression(//const ActionsD if (!column_node) continue; - // if (!join_expression_dag_input_nodes.contains(node_to_process)) - // continue; - const auto & input_name = column_node->getColumnName(); - - // bool left_table_expression_contains_input = left_table_expression_columns_names.contains(input_name); - // bool right_table_expression_contains_input = right_table_expression_columns_names.contains(input_name); - - // if (!left_table_expression_contains_input && !right_table_expression_contains_input) - // throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - // "JOIN {} actions has column {} that do not exist in left {} or right {} table expression columns", - // join_node.formatASTForErrorMessage(), - // input_name, - // boost::join(left_table_expression_columns_names, ", "), - // boost::join(right_table_expression_columns_names, ", ")); - const auto * column_source = column_node->getColumnSource().get(); if (!column_source) throw Exception(ErrorCodes::LOGICAL_ERROR, "No source for column {} in JOIN {}", input_name, join_node.formatASTForErrorMessage()); @@ -235,9 +200,6 @@ void buildJoinClause( ActionsDAGPtr & left_dag, ActionsDAGPtr & right_dag, const PlannerContextPtr & planner_context, - //ActionsDAGPtr join_expression_dag, - //const std::unordered_set & join_expression_dag_input_nodes, - //const ActionsDAG::Node * join_expressions_actions_node, const QueryTreeNodePtr & join_expression, const TableExpressionSet & left_table_expressions, const TableExpressionSet & right_table_expressions, @@ -245,22 +207,16 @@ void buildJoinClause( JoinClause & join_clause) { std::string function_name; - - //std::cerr << join_expression_dag->dumpDAG() << std::endl; auto * function_node = join_expression->as(); if (function_node) function_name = function_node->getFunction()->getName(); - // if (join_expressions_actions_node->function) - // function_name = join_expressions_actions_node->function->getName(); - /// For 'and' function go into children if (function_name == "and") { for (const auto & child : function_node->getArguments()) { - buildJoinClause(//join_expression_dag, - //join_expression_dag_input_nodes, + buildJoinClause( left_dag, right_dag, planner_context, @@ -279,17 +235,15 @@ void buildJoinClause( if (function_name == "equals" || function_name == "isNotDistinctFrom" || is_asof_join_inequality) { - const auto left_child = function_node->getArguments().getNodes().at(0);//join_expressions_actions_node->children.at(0); - const auto right_child = function_node->getArguments().getNodes().at(1); //join_expressions_actions_node->children.at(1); + const auto left_child = function_node->getArguments().getNodes().at(0); + const auto right_child = function_node->getArguments().getNodes().at(1); auto left_expression_side_optional = extractJoinTableSideFromExpression(left_child.get(), - //join_expression_dag_input_nodes, left_table_expressions, right_table_expressions, join_node); auto right_expression_side_optional = extractJoinTableSideFromExpression(right_child.get(), - //join_expression_dag_input_nodes, left_table_expressions, right_table_expressions, join_node); @@ -314,7 +268,6 @@ void buildJoinClause( } else { - // std::cerr << "===============\n"; auto left_expression_side = *left_expression_side_optional; auto right_expression_side = *right_expression_side_optional; @@ -361,8 +314,7 @@ void buildJoinClause( return; } - auto expression_side_optional = extractJoinTableSideFromExpression(//join_expressions_actions_node, - //join_expression_dag_input_nodes, + auto expression_side_optional = extractJoinTableSideFromExpression( join_expression.get(), left_table_expressions, right_table_expressions, @@ -377,32 +329,15 @@ void buildJoinClause( join_clause.addCondition(expression_side, node); } -JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName & join_expression_input_columns, +JoinClausesAndActions buildJoinClausesAndActions( const ColumnsWithTypeAndName & left_table_expression_columns, const ColumnsWithTypeAndName & right_table_expression_columns, const JoinNode & join_node, const PlannerContextPtr & planner_context) { - //ActionsDAGPtr join_expression_actions = std::make_shared(join_expression_input_columns); - ActionsDAGPtr left_join_actions = std::make_shared(left_table_expression_columns); ActionsDAGPtr right_join_actions = std::make_shared(right_table_expression_columns); - // LOG_TRACE(getLogger("Planner"), "buildJoinClausesAndActions cols {} ", left_join_actions->dumpDAG()); - // LOG_TRACE(getLogger("Planner"), "buildJoinClausesAndActions cols {} ", right_join_actions->dumpDAG()); - - /** In ActionsDAG if input node has constant representation additional constant column is added. - * That way we cannot simply check that node has INPUT type during resolution of expression join table side. - * Put all nodes after actions dag initialization in set. - * To check if actions dag node is input column, we check if set contains it. - */ - // const auto & join_expression_actions_nodes = join_expression_actions->getNodes(); - - // std::unordered_set join_expression_dag_input_nodes; - // join_expression_dag_input_nodes.reserve(join_expression_actions_nodes.size()); - // for (const auto & node : join_expression_actions_nodes) - // join_expression_dag_input_nodes.insert(&node); - /** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction. * If we do not ignore it, this function will be replaced by underlying constant. * For example ASOF JOIN does not support JOIN with constants, and we should process it like ordinary JOIN. @@ -411,9 +346,6 @@ JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName * ON (t1.id = t2.id) AND 1 != 1 AND (t1.value >= t1.value); */ auto join_expression = join_node.getJoinExpression(); - // LOG_TRACE(getLogger("Planner"), "buildJoinClausesAndActions expr {} ", join_expression->formatConvertedASTForErrorMessage()); - // LOG_TRACE(getLogger("Planner"), "buildJoinClausesAndActions expr {} ", join_expression->dumpTree()); - auto * constant_join_expression = join_expression->as(); if (constant_join_expression && constant_join_expression->hasSourceExpression()) @@ -425,19 +357,6 @@ JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName "JOIN {} join expression expected function", join_node.formatASTForErrorMessage()); - // PlannerActionsVisitor join_expression_visitor(planner_context); - // auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_expression); - // if (join_expression_dag_node_raw_pointers.size() != 1) - // throw Exception(ErrorCodes::LOGICAL_ERROR, - // "JOIN {} ON clause contains multiple expressions", - // join_node.formatASTForErrorMessage()); - - // const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0]; - // if (!join_expressions_actions_root_node->function) - // throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - // "JOIN {} join expression expected function", - // join_node.formatASTForErrorMessage()); - size_t left_table_expression_columns_size = left_table_expression_columns.size(); Names join_left_actions_names; @@ -470,7 +389,6 @@ JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName auto join_right_table_expressions = extractTableExpressionsSet(join_node.getRightTableExpression()); JoinClausesAndActions result; - //result.join_expression_actions = join_expression_actions; const auto & function_name = function_node->getFunction()->getName(); if (function_name == "or") @@ -479,8 +397,7 @@ JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName { result.join_clauses.emplace_back(); - buildJoinClause(//join_expression_actions, - //join_expression_dag_input_nodes, + buildJoinClause( left_join_actions, right_join_actions, planner_context, @@ -499,9 +416,7 @@ JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName left_join_actions, right_join_actions, planner_context, - //join_expression_actions, - //join_expression_dag_input_nodes, - join_expression, //join_expressions_actions_root_node, + join_expression, join_left_table_expressions, join_right_table_expressions, join_node, @@ -621,12 +536,6 @@ JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName result.left_join_expressions_actions = left_join_actions->clone(); result.left_join_tmp_expression_actions = std::move(left_join_actions); result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names); - - // for (const auto & name : join_right_actions_names) - // std::cerr << ".. " << name << std::endl; - - // std::cerr << right_join_actions->dumpDAG() << std::endl; - result.right_join_expressions_actions = right_join_actions->clone(); result.right_join_tmp_expression_actions = std::move(right_join_actions); result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names); @@ -648,10 +557,7 @@ JoinClausesAndActions buildJoinClausesAndActions( "JOIN {} join does not have ON section", join_node_typed.formatASTForErrorMessage()); - // auto join_expression_input_columns = left_table_expression_columns; - // join_expression_input_columns.insert(join_expression_input_columns.end(), right_table_expression_columns.begin(), right_table_expression_columns.end()); - - return buildJoinClausesAndActions(/*join_expression_input_columns,*/ left_table_expression_columns, right_table_expression_columns, join_node_typed, planner_context); + return buildJoinClausesAndActions(left_table_expression_columns, right_table_expression_columns, join_node_typed, planner_context); } std::optional tryExtractConstantFromJoinNode(const QueryTreeNodePtr & join_node) From a5e6f948ee7fc654717b8deb96f170838e396f6e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 Apr 2024 13:40:10 +0000 Subject: [PATCH 19/50] Docs: Fix documented default of cgroup_memory_watcher_soft_limit_ratio --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 24248209025..3a20c7b758b 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -523,7 +523,7 @@ See settings `cgroups_memory_usage_observer_wait_time` and `cgroup_memory_watche Type: Double -Default: 0.95 +Default: 0.9 ## max_table_size_to_drop From ca4e2f581ab2985f4e26a8894f1b6cf9a1737d9b Mon Sep 17 00:00:00 2001 From: Alexey Katsman Date: Mon, 5 Feb 2024 23:43:39 +0100 Subject: [PATCH 20/50] Add columns for values used in the formatting message into system.text_log --- base/poco/Foundation/include/Poco/Message.h | 10 ++++- base/poco/Foundation/src/Message.cpp | 32 ++++++++++++--- src/Access/AccessControl.cpp | 3 +- src/Common/Exception.cpp | 6 ++- src/Common/Exception.h | 33 ++++++++------- src/Common/LoggingFormatStringHelpers.h | 40 +++++++++++++++++-- src/Common/logger_useful.h | 8 ++-- src/Functions/FunctionsConversion.cpp | 2 +- src/Interpreters/QueryLog.h | 1 + src/Interpreters/TextLog.cpp | 20 ++++++++++ src/Interpreters/TextLog.h | 10 +++++ src/Interpreters/TreeRewriter.cpp | 2 +- src/Interpreters/executeQuery.cpp | 3 ++ src/Loggers/OwnSplitChannel.cpp | 15 +++++++ ...log_format_string_args_not_empty.reference | 3 ++ ..._text_log_format_string_args_not_empty.sql | 13 ++++++ 16 files changed, 167 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.reference create mode 100644 tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql diff --git a/base/poco/Foundation/include/Poco/Message.h b/base/poco/Foundation/include/Poco/Message.h index 282c7fb5fd1..9068e56a93c 100644 --- a/base/poco/Foundation/include/Poco/Message.h +++ b/base/poco/Foundation/include/Poco/Message.h @@ -66,9 +66,11 @@ public: /// The thread and process ids are set. Message( - const std::string & source, const std::string & text, Priority prio, const char * file, int line, std::string_view fmt_str = {}); + const std::string & source, const std::string & text, Priority prio, const char * file, int line, + std::string_view fmt_str = {}, const std::vector & fmt_str_args = {}); Message( - std::string && source, std::string && text, Priority prio, const char * file, int line, std::string_view fmt_str); + std::string && source, std::string && text, Priority prio, const char * file, int line, + std::string_view fmt_str, std::vector && fmt_str_args); /// Creates a Message with the given source, text, priority, /// source file path and line. /// @@ -161,6 +163,9 @@ public: std::string_view getFormatString() const; void setFormatString(std::string_view fmt_str); + const std::vector & getFormatStringArgs() const; + void setFormatStringArgs(const std::vector & fmt_str_args); + int getSourceLine() const; /// Returns the source file line of the statement /// generating the log message. May be 0 @@ -210,6 +215,7 @@ private: int _line; StringMap * _pMap; std::string_view _fmt_str; + std::vector _fmt_str_args; }; diff --git a/base/poco/Foundation/src/Message.cpp b/base/poco/Foundation/src/Message.cpp index 54118cc0fc5..9e9835ac585 100644 --- a/base/poco/Foundation/src/Message.cpp +++ b/base/poco/Foundation/src/Message.cpp @@ -46,7 +46,9 @@ Message::Message(const std::string& source, const std::string& text, Priority pr } -Message::Message(const std::string& source, const std::string& text, Priority prio, const char* file, int line, std::string_view fmt_str): +Message::Message( + const std::string& source, const std::string& text, Priority prio, const char* file, int line, + std::string_view fmt_str, const std::vector& fmt_str_args): _source(source), _text(text), _prio(prio), @@ -54,13 +56,16 @@ Message::Message(const std::string& source, const std::string& text, Priority pr _file(file), _line(line), _pMap(0), - _fmt_str(fmt_str) + _fmt_str(fmt_str), + _fmt_str_args(fmt_str_args) { init(); } -Message::Message(std::string && source, std::string && text, Priority prio, const char * file, int line, std::string_view fmt_str): +Message::Message( + std::string && source, std::string && text, Priority prio, const char * file, int line, + std::string_view fmt_str, std::vector && fmt_str_args): _source(std::move(source)), _text(std::move(text)), _prio(prio), @@ -68,7 +73,8 @@ Message::Message(std::string && source, std::string && text, Priority prio, cons _file(file), _line(line), _pMap(0), - _fmt_str(fmt_str) + _fmt_str(fmt_str), + _fmt_str_args(std::move(fmt_str_args)) { init(); } @@ -83,7 +89,8 @@ Message::Message(const Message& msg): _pid(msg._pid), _file(msg._file), _line(msg._line), - _fmt_str(msg._fmt_str) + _fmt_str(msg._fmt_str), + _fmt_str_args(msg._fmt_str_args) { if (msg._pMap) _pMap = new StringMap(*msg._pMap); @@ -102,7 +109,8 @@ Message::Message(const Message& msg, const std::string& text): _pid(msg._pid), _file(msg._file), _line(msg._line), - _fmt_str(msg._fmt_str) + _fmt_str(msg._fmt_str), + _fmt_str_args(msg._fmt_str_args) { if (msg._pMap) _pMap = new StringMap(*msg._pMap); @@ -154,6 +162,7 @@ void Message::swap(Message& msg) swap(_line, msg._line); swap(_pMap, msg._pMap); swap(_fmt_str, msg._fmt_str); + swap(_fmt_str_args, msg._fmt_str_args); } @@ -227,6 +236,17 @@ void Message::setFormatString(std::string_view fmt_str) } +const std::vector& Message::getFormatStringArgs() const +{ + return _fmt_str_args; +} + +void Message::setFormatStringArgs(const std::vector& fmt_str_args) +{ + _fmt_str_args = fmt_str_args; +} + + bool Message::has(const std::string& param) const { return _pMap && (_pMap->find(param) != _pMap->end()); diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index da047d1cb1d..aca0c4d2d53 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -607,7 +607,8 @@ AuthResult AccessControl::authenticate(const Credentials & credentials, const Po /// We use the same message for all authentication failures because we don't want to give away any unnecessary information for security reasons, /// only the log will show the exact reason. throw Exception(PreformattedMessage{message.str(), - "{}: Authentication failed: password is incorrect, or there is no user with such name.{}"}, + "{}: Authentication failed: password is incorrect, or there is no user with such name.{}", + std::vector{credentials.getUserName()}}, ErrorCodes::AUTHENTICATION_FAILED); } } diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 5d936bbb88c..1f4b0aea8f2 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -391,6 +391,7 @@ PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, b { WriteBufferFromOwnString stream; std::string_view message_format_string; + std::vector message_format_string_args; try { @@ -402,6 +403,7 @@ PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, b << (with_extra_info ? getExtraExceptionInfo(e) : "") << " (version " << VERSION_STRING << VERSION_OFFICIAL << ")"; message_format_string = e.tryGetMessageFormatString(); + message_format_string_args = e.getMessageFormatStringArgs(); } catch (const Poco::Exception & e) { @@ -462,7 +464,7 @@ PreformattedMessage getCurrentExceptionMessageAndPattern(bool with_stacktrace, b catch (...) {} // NOLINT(bugprone-empty-catch) } - return PreformattedMessage{stream.str(), message_format_string}; + return PreformattedMessage{stream.str(), message_format_string, message_format_string_args}; } @@ -581,7 +583,7 @@ PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with } catch (...) {} // NOLINT(bugprone-empty-catch) - return PreformattedMessage{stream.str(), e.tryGetMessageFormatString()}; + return PreformattedMessage{stream.str(), e.tryGetMessageFormatString(), e.getMessageFormatStringArgs()}; } std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace) diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 1b4dabec113..73c50debf33 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -13,6 +13,7 @@ #include #include +#include #include #include @@ -59,6 +60,7 @@ public: std::terminate(); capture_thread_frame_pointers = thread_frame_pointers; message_format_string = msg.format_string; + message_format_string_args = msg.format_string_args; } Exception(PreformattedMessage && msg, int code): Exception(std::move(msg.text), code) @@ -67,6 +69,7 @@ public: std::terminate(); capture_thread_frame_pointers = thread_frame_pointers; message_format_string = msg.format_string; + message_format_string_args = msg.format_string_args; } /// Collect call stacks of all previous jobs' schedulings leading to this thread job's execution @@ -107,12 +110,7 @@ public: // Format message with fmt::format, like the logging functions. template - Exception(int code, FormatStringHelper fmt, Args &&... args) - : Exception(fmt::format(fmt.fmt_str, std::forward(args)...), code) - { - capture_thread_frame_pointers = thread_frame_pointers; - message_format_string = fmt.message_format_string; - } + Exception(int code, FormatStringHelper fmt, Args &&... args) : Exception(fmt.format(std::forward(args)...), code) {} struct CreateFromPocoTag {}; struct CreateFromSTDTag {}; @@ -152,6 +150,8 @@ public: std::string_view tryGetMessageFormatString() const { return message_format_string; } + std::vector getMessageFormatStringArgs() const { return message_format_string_args; } + private: #ifndef STD_EXCEPTION_HAS_STACK_TRACE StackTrace trace; @@ -162,6 +162,7 @@ private: protected: std::string_view message_format_string; + std::vector message_format_string_args; /// Local copy of static per-thread thread_frame_pointers, should be mutable to be unpoisoned on printout mutable std::vector capture_thread_frame_pointers; }; @@ -193,26 +194,29 @@ public: // Format message with fmt::format, like the logging functions. template ErrnoException(int code, FormatStringHelper fmt, Args &&... args) - : Exception(fmt::format(fmt.fmt_str, std::forward(args)...), code), saved_errno(errno) + : Exception(fmt.format(std::forward(args)...), code), saved_errno(errno) + { + addMessage(", {}", errnoToString(saved_errno)); + } + + template + ErrnoException(int code, int with_errno, FormatStringHelper fmt, Args &&... args) + : Exception(fmt.format(std::forward(args)...), code), saved_errno(with_errno) { - capture_thread_frame_pointers = thread_frame_pointers; - message_format_string = fmt.message_format_string; addMessage(", {}", errnoToString(saved_errno)); } template [[noreturn]] static void throwWithErrno(int code, int with_errno, FormatStringHelper fmt, Args &&... args) { - auto e = ErrnoException(fmt::format(fmt.fmt_str, std::forward(args)...), code, with_errno); - e.message_format_string = fmt.message_format_string; + auto e = ErrnoException(code, with_errno, std::move(fmt), std::forward(args)...); throw e; /// NOLINT } template [[noreturn]] static void throwFromPath(int code, const std::string & path, FormatStringHelper fmt, Args &&... args) { - auto e = ErrnoException(fmt::format(fmt.fmt_str, std::forward(args)...), code, errno); - e.message_format_string = fmt.message_format_string; + auto e = ErrnoException(code, errno, std::move(fmt), std::forward(args)...); e.path = path; throw e; /// NOLINT } @@ -221,8 +225,7 @@ public: [[noreturn]] static void throwFromPathWithErrno(int code, const std::string & path, int with_errno, FormatStringHelper fmt, Args &&... args) { - auto e = ErrnoException(fmt::format(fmt.fmt_str, std::forward(args)...), code, with_errno); - e.message_format_string = fmt.message_format_string; + auto e = ErrnoException(code, with_errno, std::move(fmt), std::forward(args)...); e.path = path; throw e; /// NOLINT } diff --git a/src/Common/LoggingFormatStringHelpers.h b/src/Common/LoggingFormatStringHelpers.h index 678948424da..3e64c8dfb8f 100644 --- a/src/Common/LoggingFormatStringHelpers.h +++ b/src/Common/LoggingFormatStringHelpers.h @@ -2,8 +2,11 @@ #include #include +#include +#include #include #include +#include #include #include #include @@ -14,6 +17,10 @@ struct PreformattedMessage; consteval void formatStringCheckArgsNumImpl(std::string_view str, size_t nargs); template constexpr std::string_view tryGetStaticFormatString(T && x); +[[maybe_unused]] inline void tryGetFormattedArgs(std::vector&) {}; +template [[maybe_unused]] inline void tryGetFormattedArgs(std::vector&, T &&, Ts && ...); +template inline std::string tryGetArgsAndFormat(std::vector&, fmt::format_string, Args && ...); + /// Extract format string from a string literal and constructs consteval fmt::format_string template struct FormatStringHelperImpl @@ -39,6 +46,7 @@ struct PreformattedMessage { std::string text; std::string_view format_string; + std::vector format_string_args; template static PreformattedMessage create(FormatStringHelper fmt, Args &&... args); @@ -47,22 +55,26 @@ struct PreformattedMessage operator std::string () && { return std::move(text); } /// NOLINT operator fmt::format_string<> () const { UNREACHABLE(); } /// NOLINT - void apply(std::string & out_text, std::string_view & out_format_string) const & + void apply(std::string & out_text, std::string_view & out_format_string, std::vector & out_format_string_args) const & { out_text = text; out_format_string = format_string; + out_format_string_args = format_string_args; } - void apply(std::string & out_text, std::string_view & out_format_string) && + void apply(std::string & out_text, std::string_view & out_format_string, std::vector & out_format_string_args) && { out_text = std::move(text); out_format_string = format_string; + out_format_string_args = std::move(format_string_args); } }; template PreformattedMessage FormatStringHelperImpl::format(Args && ...args) const { - return PreformattedMessage{fmt::format(fmt_str, std::forward(args)...), message_format_string}; + std::vector out_format_string_args; + std::string msg_text = tryGetArgsAndFormat(out_format_string_args, fmt_str, std::forward(args)...); + return PreformattedMessage{msg_text, message_format_string, out_format_string_args}; } template @@ -113,12 +125,23 @@ template constexpr std::string_view tryGetStaticFormatString(T && x } } +template void tryGetFormattedArgs(std::vector& out, T && x, Ts && ...rest) +{ + if constexpr (std::is_base_of_v>) + out.push_back(fmt::format("{}", std::remove_reference_t(x))); + else + out.push_back(fmt::format("{}", std::forward(x))); + + tryGetFormattedArgs(out, std::forward(rest)...); +} + /// Constexpr ifs are not like ifdefs, and compiler still checks that unneeded code can be compiled /// This template is useful to avoid compilation failures when condition of some "constexpr if" is false template struct ConstexprIfsAreNotIfdefs { template constexpr static std::string_view getStaticFormatString(T &&) { return {}; } template static PreformattedMessage getPreformatted(T &&) { return {}; } + template static std::string getArgsAndFormat(std::vector&, fmt::format_string, Args &&...) { return {}; } }; template<> struct ConstexprIfsAreNotIfdefs @@ -133,8 +156,19 @@ template<> struct ConstexprIfsAreNotIfdefs } template static T && getPreformatted(T && x) { return std::forward(x); } + + template static std::string getArgsAndFormat(std::vector& out, fmt::format_string fmt_str, Args && ...args) + { + return tryGetArgsAndFormat(out, std::move(fmt_str), std::forward(args)...); + } }; +template inline std::string tryGetArgsAndFormat(std::vector& out, fmt::format_string fmt_str, Args && ...args) +{ + tryGetFormattedArgs(out, args...); + return fmt::format(fmt_str, std::forward(args)...); +} + template constexpr size_t numArgs(Ts &&...) { return sizeof...(Ts); } template constexpr auto firstArg(T && x, Ts &&...) { return std::forward(x); } /// For implicit conversion of fmt::basic_runtime<> to char* for std::string ctor diff --git a/src/Common/logger_useful.h b/src/Common/logger_useful.h index 9d6ebaddcc6..eaae7e69dbc 100644 --- a/src/Common/logger_useful.h +++ b/src/Common/logger_useful.h @@ -2,6 +2,7 @@ /// Macros for convenient usage of Poco logger. #include +#include #include #include #include @@ -80,6 +81,7 @@ namespace impl \ std::string_view _format_string; \ std::string _formatted_message; \ + std::vector _format_string_args; \ \ if constexpr (LogTypeInfo::is_static) \ { \ @@ -91,17 +93,17 @@ namespace impl if constexpr (is_preformatted_message) \ { \ static_assert(_nargs == 1 || !is_preformatted_message); \ - ConstexprIfsAreNotIfdefs::getPreformatted(LOG_IMPL_FIRST_ARG(__VA_ARGS__)).apply(_formatted_message, _format_string); \ + ConstexprIfsAreNotIfdefs::getPreformatted(LOG_IMPL_FIRST_ARG(__VA_ARGS__)).apply(_formatted_message, _format_string, _format_string_args); \ } \ else \ { \ - _formatted_message = _nargs == 1 ? firstArg(__VA_ARGS__) : fmt::format(__VA_ARGS__); \ + _formatted_message = _nargs == 1 ? firstArg(__VA_ARGS__) : ConstexprIfsAreNotIfdefs::getArgsAndFormat(_format_string_args, __VA_ARGS__); \ } \ \ std::string _file_function = __FILE__ "; "; \ _file_function += __PRETTY_FUNCTION__; \ Poco::Message _poco_message(_logger->name(), std::move(_formatted_message), \ - (PRIORITY), _file_function.c_str(), __LINE__, _format_string); \ + (PRIORITY), _file_function.c_str(), __LINE__, _format_string, _format_string_args); \ _channel->log(_poco_message); \ } \ catch (const Poco::Exception & logger_exception) \ diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 5e072d406ad..a16ce136b9a 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -794,7 +794,7 @@ inline bool tryParseImpl(DataTypeIPv6::FieldType & x, ReadBuffer & if (isNativeNumber(result_type) && !(result_type.getName() == "IPv4" || result_type.getName() == "IPv6")) message_buf << ". Note: there are to" << result_type.getName() << "OrZero and to" << result_type.getName() << "OrNull functions, which returns zero/NULL instead of throwing exception."; - throw Exception(PreformattedMessage{message_buf.str(), "Cannot parse string {} as {}: syntax error {}"}, ErrorCodes::CANNOT_PARSE_TEXT); + throw Exception(PreformattedMessage{message_buf.str(), "Cannot parse string {} as {}: syntax error {}", {String(read_buffer.buffer().begin(), read_buffer.buffer().size()), result_type.getName()}}, ErrorCodes::CANNOT_PARSE_TEXT); } diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index f9ed8a2fb9e..5072d220160 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -86,6 +86,7 @@ struct QueryLogElement String exception; String stack_trace; std::string_view exception_format_string{}; + std::vector exception_format_string_args{}; ClientInfo client_info; diff --git a/src/Interpreters/TextLog.cpp b/src/Interpreters/TextLog.cpp index d6971bbac54..3c3eff600e4 100644 --- a/src/Interpreters/TextLog.cpp +++ b/src/Interpreters/TextLog.cpp @@ -53,6 +53,16 @@ ColumnsDescription TextLogElement::getColumnsDescription() {"source_line", std::make_shared(), "Source line from which the logging was done."}, {"message_format_string", std::make_shared(std::make_shared()), "A format string that was used to format the message."}, + {"value1", std::make_shared(), "Argument 1 that was used to format the message."}, + {"value2", std::make_shared(), "Argument 2 that was used to format the message."}, + {"value3", std::make_shared(), "Argument 3 that was used to format the message."}, + {"value4", std::make_shared(), "Argument 4 that was used to format the message."}, + {"value5", std::make_shared(), "Argument 5 that was used to format the message."}, + {"value6", std::make_shared(), "Argument 6 that was used to format the message."}, + {"value7", std::make_shared(), "Argument 7 that was used to format the message."}, + {"value8", std::make_shared(), "Argument 8 that was used to format the message."}, + {"value9", std::make_shared(), "Argument 9 that was used to format the message."}, + {"value10", std::make_shared(), "Argument 10 that was used to format the message."}, }; } @@ -79,6 +89,16 @@ void TextLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(source_line); columns[i++]->insert(message_format_string); + columns[i++]->insert(value1); + columns[i++]->insert(value2); + columns[i++]->insert(value3); + columns[i++]->insert(value4); + columns[i++]->insert(value5); + columns[i++]->insert(value6); + columns[i++]->insert(value7); + columns[i++]->insert(value8); + columns[i++]->insert(value9); + columns[i++]->insert(value10); } TextLog::TextLog(ContextPtr context_, diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 90fc4ef36a0..78153b9f81e 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -29,6 +29,16 @@ struct TextLogElement UInt64 source_line{}; std::string_view message_format_string; + String value1; + String value2; + String value3; + String value4; + String value5; + String value6; + String value7; + String value8; + String value9; + String value10; static std::string name() { return "TextLog"; } static ColumnsDescription getColumnsDescription(); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 914b3c3037d..50c28fbc8b2 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1249,7 +1249,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (no_throw) return false; - throw Exception(PreformattedMessage{ss.str(), format_string}, ErrorCodes::UNKNOWN_IDENTIFIER); + throw Exception(PreformattedMessage{ss.str(), format_string, std::vector{}}, ErrorCodes::UNKNOWN_IDENTIFIER); } required_source_columns.swap(source_columns); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index db8753000d8..09c8c459f0a 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -200,6 +200,7 @@ static void logException(ContextPtr context, QueryLogElement & elem, bool log_er /// so we pass elem.exception_format_string as format string instead. PreformattedMessage message; message.format_string = elem.exception_format_string; + message.format_string_args = elem.exception_format_string_args; if (elem.stack_trace.empty() || !log_error) message.text = fmt::format("{} (from {}){} (in query: {})", elem.exception, @@ -504,6 +505,7 @@ void logQueryException( auto exception_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false); elem.exception = std::move(exception_message.text); elem.exception_format_string = exception_message.format_string; + elem.exception_format_string_args = exception_message.format_string_args; QueryStatusPtr process_list_elem = context->getProcessListElement(); @@ -597,6 +599,7 @@ void logExceptionBeforeStart( auto exception_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false); elem.exception = std::move(exception_message.text); elem.exception_format_string = exception_message.format_string; + elem.exception_format_string_args = exception_message.format_string_args; elem.client_info = context->getClientInfo(); diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index 68fad2fbaf6..fee33781c27 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -131,6 +131,21 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) elem.source_line = msg.getSourceLine(); elem.message_format_string = msg.getFormatString(); +#define SET_VALUE_IF_EXISTS(INDEX) if ((INDEX) <= msg.getFormatStringArgs().size()) (elem.value##INDEX) = msg.getFormatStringArgs()[(INDEX) - 1] + + SET_VALUE_IF_EXISTS(1); + SET_VALUE_IF_EXISTS(2); + SET_VALUE_IF_EXISTS(3); + SET_VALUE_IF_EXISTS(4); + SET_VALUE_IF_EXISTS(5); + SET_VALUE_IF_EXISTS(6); + SET_VALUE_IF_EXISTS(7); + SET_VALUE_IF_EXISTS(8); + SET_VALUE_IF_EXISTS(9); + SET_VALUE_IF_EXISTS(10); + +#undef SET_VALUE_IF_EXISTS + std::shared_ptr> text_log_locked{}; text_log_locked = text_log.lock(); if (text_log_locked) diff --git a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.reference b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.reference new file mode 100644 index 00000000000..e8183f05f5d --- /dev/null +++ b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.reference @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql new file mode 100644 index 00000000000..0f4a217a4ae --- /dev/null +++ b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql @@ -0,0 +1,13 @@ +set allow_experimental_analyzer = true; + +select count; -- { serverError 47 } + +select conut(); -- { serverError 46 } + +system flush logs; + +select count() > 0 from system.text_log where message_format_string = 'Peak memory usage{}: {}.' and value1 is not null and value2 like '% MiB'; + +select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier \'{}\' in scope {}{}' and value1 = 'expression' and value3 = 'count' and value4 = 'SELECT count'; + +select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Function with name \'{}\' does not exists. In scope {}{}' and value1 = 'conut' and value2 = 'SELECT conut()' and value3 ilike '%\'count\'%'; From 7490e7b2b507e4b1f168c307dcd27a9ed811242f Mon Sep 17 00:00:00 2001 From: Alexey Katsman Date: Mon, 5 Feb 2024 23:44:55 +0100 Subject: [PATCH 21/50] Add docs for columns with values used in the formatting message in system.text_log --- docs/en/operations/system-tables/text_log.md | 20 ++++++++++++++++++++ docs/ru/operations/system-tables/text_log.md | 20 ++++++++++++++++++++ 2 files changed, 40 insertions(+) diff --git a/docs/en/operations/system-tables/text_log.md b/docs/en/operations/system-tables/text_log.md index 0c39499e190..2f0e1c6bf50 100644 --- a/docs/en/operations/system-tables/text_log.md +++ b/docs/en/operations/system-tables/text_log.md @@ -30,6 +30,16 @@ Columns: - `source_file` (LowCardinality(String)) — Source file from which the logging was done. - `source_line` (UInt64) — Source line from which the logging was done. - `message_format_string` (LowCardinality(String)) — A format string that was used to format the message. +- `value1` (String) - Argument 1 that was used to format the message. +- `value2` (String) - Argument 2 that was used to format the message. +- `value3` (String) - Argument 3 that was used to format the message. +- `value4` (String) - Argument 4 that was used to format the message. +- `value5` (String) - Argument 5 that was used to format the message. +- `value6` (String) - Argument 6 that was used to format the message. +- `value7` (String) - Argument 7 that was used to format the message. +- `value8` (String) - Argument 8 that was used to format the message. +- `value9` (String) - Argument 9 that was used to format the message. +- `value10` (String) - Argument 10 that was used to format the message. **Example** @@ -55,4 +65,14 @@ revision: 54440 source_file: /ClickHouse/src/Interpreters/DNSCacheUpdater.cpp; void DB::DNSCacheUpdater::start() source_line: 45 message_format_string: Update period {} seconds +value1: 15 +value2: +value3: +value4: +value5: +value6: +value7: +value8: +value9: +value10: ``` diff --git a/docs/ru/operations/system-tables/text_log.md b/docs/ru/operations/system-tables/text_log.md index 59ae804d85f..a6ab6f7d268 100644 --- a/docs/ru/operations/system-tables/text_log.md +++ b/docs/ru/operations/system-tables/text_log.md @@ -29,6 +29,16 @@ slug: /ru/operations/system-tables/text_log - `source_file` (LowCardinality(String)) — исходный файл, из которого была сделана запись. - `source_line` (UInt64) — исходная строка, из которой была сделана запись. - `message_format_string` (LowCardinality(String)) — форматная строка, с помощью которой было отформатировано сообщение. +- `value1` (String) - аргумент 1, который использовался для форматирования сообщения. +- `value2` (String) - аргумент 2, который использовался для форматирования сообщения. +- `value3` (String) - аргумент 3, который использовался для форматирования сообщения. +- `value4` (String) - аргумент 4, который использовался для форматирования сообщения. +- `value5` (String) - аргумент 5, который использовался для форматирования сообщения. +- `value6` (String) - аргумент 6, который использовался для форматирования сообщения. +- `value7` (String) - аргумент 7, который использовался для форматирования сообщения. +- `value8` (String) - аргумент 8, который использовался для форматирования сообщения. +- `value9` (String) - аргумент 9, который использовался для форматирования сообщения. +- `value10` (String) - аргумент 10, который использовался для форматирования сообщения. **Пример** @@ -53,4 +63,14 @@ revision: 54440 source_file: /ClickHouse/src/Interpreters/DNSCacheUpdater.cpp; void DB::DNSCacheUpdater::start() source_line: 45 message_format_string: Update period {} seconds +value1: 15 +value2: +value3: +value4: +value5: +value6: +value7: +value8: +value9: +value10: ``` From 5dfd7bfd2d4a6bc4596d6e0910b4672e2f5fd3f0 Mon Sep 17 00:00:00 2001 From: Jhonso7393 <167781426+Jhonso7393@users.noreply.github.com> Date: Tue, 23 Apr 2024 19:48:59 +0300 Subject: [PATCH 22/50] Update README.md Fixing a minor typo --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 2b97bd25d70..84859ac78d2 100644 --- a/README.md +++ b/README.md @@ -39,7 +39,7 @@ Every month we get together with the community (users, contributors, customers, ## Upcoming Events -Keep an eye out for upcoming meetups and eventsaround the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. +Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. * [ClickHouse Meetup in Bellevue](https://www.meetup.com/clickhouse-seattle-user-group/events/298650371/) - Mar 11 * [ClickHouse Meetup at Ramp's Offices in NYC](https://www.meetup.com/clickhouse-new-york-user-group/events/298640542/) - Mar 19 From 2dcd1b673154d678a6e6c04db6f8ff89e3bd2d88 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 23 Apr 2024 19:30:04 +0200 Subject: [PATCH 23/50] Update README.md --- README.md | 7 ------- 1 file changed, 7 deletions(-) diff --git a/README.md b/README.md index 84859ac78d2..0d50813e30a 100644 --- a/README.md +++ b/README.md @@ -41,13 +41,6 @@ Every month we get together with the community (users, contributors, customers, Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. -* [ClickHouse Meetup in Bellevue](https://www.meetup.com/clickhouse-seattle-user-group/events/298650371/) - Mar 11 -* [ClickHouse Meetup at Ramp's Offices in NYC](https://www.meetup.com/clickhouse-new-york-user-group/events/298640542/) - Mar 19 -* [ClickHouse Melbourne Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/299479750/) - Mar 20 -* [ClickHouse Meetup in Paris](https://www.meetup.com/clickhouse-france-user-group/events/298997115/) - Mar 21 -* [ClickHouse Meetup in Bengaluru](https://www.meetup.com/clickhouse-bangalore-user-group/events/299479850/) - Mar 23 -* [ClickHouse Meetup in Zurich](https://www.meetup.com/clickhouse-switzerland-meetup-group/events/299628922/) - Apr 16 -* [ClickHouse Meetup in Copenhagen](https://www.meetup.com/clickhouse-denmark-meetup-group/events/299629133/) - Apr 23 * [ClickHouse Meetup in Dubai](https://www.meetup.com/clickhouse-dubai-meetup-group/events/299629189/) - May 28 From 69b259b40936fc1c03d545d0aa54bb48228d6578 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Tue, 23 Apr 2024 16:34:00 -0700 Subject: [PATCH 24/50] Fix build for openssl dynamic linking --- contrib/openssl-cmake/CMakeLists.txt | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/contrib/openssl-cmake/CMakeLists.txt b/contrib/openssl-cmake/CMakeLists.txt index ddbe9f06b1d..14528fbc16f 100644 --- a/contrib/openssl-cmake/CMakeLists.txt +++ b/contrib/openssl-cmake/CMakeLists.txt @@ -93,6 +93,7 @@ enable_language(ASM) if(COMPILER_CLANG) add_definitions(-Wno-unused-command-line-argument) + set(CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=lld") endif() if(ARCH_AMD64) @@ -960,11 +961,6 @@ set(CRYPTO_SRC ${OPENSSL_SOURCE_DIR}/crypto/x509/x_req.c ${OPENSSL_SOURCE_DIR}/crypto/x509/x_x509.c ${OPENSSL_SOURCE_DIR}/crypto/x509/x_x509a.c - ${OPENSSL_SOURCE_DIR}/engines/e_capi.c - ${OPENSSL_SOURCE_DIR}/engines/e_dasync.c - ${OPENSSL_SOURCE_DIR}/engines/e_loader_attic.c - ${OPENSSL_SOURCE_DIR}/engines/e_ossltest.c - ${OPENSSL_SOURCE_DIR}/engines/e_padlock.c ${OPENSSL_SOURCE_DIR}/providers/baseprov.c ${OPENSSL_SOURCE_DIR}/providers/common/bio_prov.c ${OPENSSL_SOURCE_DIR}/providers/common/capabilities.c @@ -985,8 +981,6 @@ set(CRYPTO_SRC ${OPENSSL_SOURCE_DIR}/providers/common/securitycheck.c ${OPENSSL_SOURCE_DIR}/providers/common/securitycheck_default.c ${OPENSSL_SOURCE_DIR}/providers/defltprov.c - ${OPENSSL_SOURCE_DIR}/providers/fips/fips_entry.c - ${OPENSSL_SOURCE_DIR}/providers/fips/fipsprov.c ${OPENSSL_SOURCE_DIR}/providers/implementations/asymciphers/rsa_enc.c ${OPENSSL_SOURCE_DIR}/providers/implementations/asymciphers/sm2_enc.c ${OPENSSL_SOURCE_DIR}/providers/implementations/ciphers/cipher_aes.c @@ -1145,9 +1139,10 @@ set(CRYPTO_SRC ${OPENSSL_SOURCE_DIR}/providers/implementations/signature/sm2_sig.c ${OPENSSL_SOURCE_DIR}/providers/implementations/storemgmt/file_store.c ${OPENSSL_SOURCE_DIR}/providers/implementations/storemgmt/file_store_any2obj.c - ${OPENSSL_SOURCE_DIR}/providers/legacyprov.c ${OPENSSL_SOURCE_DIR}/providers/nullprov.c ${OPENSSL_SOURCE_DIR}/providers/prov_running.c + ${OPENSSL_SOURCE_DIR}/ssl/record/methods/tls_pad.c + ${OPENSSL_SOURCE_DIR}/ssl/record/methods/ssl3_cbc.c ) if(ARCH_AMD64) From 26308744ac68a6c4da8fc73648551cfa3f109eee Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Wed, 24 Apr 2024 02:50:04 +0200 Subject: [PATCH 25/50] Update run.sh --- docker/test/stateless/run.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 9497b7ecc19..55dfe5e2e08 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -44,8 +44,6 @@ source /utils.lib if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then echo "Azure is disabled" -elif [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then - echo "Azure is disabled" else azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & fi From 76b989ebbc8c8801fa4d491ade3d68010590e072 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Wed, 24 Apr 2024 17:13:08 +0800 Subject: [PATCH 26/50] fix missing-field-initializers --- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 557a59dea66..cf84fe46579 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -188,7 +188,8 @@ Azure::Storage::Blobs::BlobClientOptions getAzureBlobClientOptions(const Poco::U retry_options.MaxRetryDelay = std::chrono::milliseconds(config.getUInt(config_prefix + ".retry_max_backoff_ms", 1000)); using CurlOptions = Azure::Core::Http::CurlTransportOptions; - CurlOptions curl_options{.NoSignal = true}; + CurlOptions curl_options; + curl_options.NoSignal = true; if (config.has(config_prefix + ".curl_ip_resolve")) { From 0d06d69377496841ef253cd36ee2a2be1ed8e7d8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 24 Apr 2024 13:32:13 +0200 Subject: [PATCH 27/50] Fix parsing of nested proto messages --- src/Formats/ProtobufSerializer.cpp | 1 + .../0_stateless/03130_nested_type.reference | 1 + tests/queries/0_stateless/03130_nested_type.sh | 9 +++++++++ .../format_schemas/03130_nested_schema.proto | 16 ++++++++++++++++ 4 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/03130_nested_type.reference create mode 100755 tests/queries/0_stateless/03130_nested_type.sh create mode 100644 tests/queries/0_stateless/format_schemas/03130_nested_schema.proto diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index e14ab8992ec..53ac0d56653 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -3736,6 +3736,7 @@ namespace throw Exception(ErrorCodes::BAD_ARGUMENTS, "ClickHouse doesn't support type recursion ({})", field_descriptor->full_name()); } pending_resolution.emplace(field_descriptor); + SCOPE_EXIT({ pending_resolution.erase(field_descriptor); }); if (allow_repeat && field_descriptor->is_map()) { diff --git a/tests/queries/0_stateless/03130_nested_type.reference b/tests/queries/0_stateless/03130_nested_type.reference new file mode 100644 index 00000000000..e4c5ff4fe62 --- /dev/null +++ b/tests/queries/0_stateless/03130_nested_type.reference @@ -0,0 +1 @@ +inner Tuple(\n a Tuple(\n seconds Int64,\n nanos Int32),\n b Tuple(\n seconds Int64,\n nanos Int32),\n c Tuple(\n seconds Int64,\n nanos Int32)) diff --git a/tests/queries/0_stateless/03130_nested_type.sh b/tests/queries/0_stateless/03130_nested_type.sh new file mode 100755 index 00000000000..91ad34e06cd --- /dev/null +++ b/tests/queries/0_stateless/03130_nested_type.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +SCHEMADIR="$CUR_DIR/format_schemas" +$CLICKHOUSE_LOCAL -q "DESCRIBE TABLE file('nonexist', 'Protobuf') SETTINGS format_schema='$SCHEMADIR/03130_nested_schema.proto:Outer'" diff --git a/tests/queries/0_stateless/format_schemas/03130_nested_schema.proto b/tests/queries/0_stateless/format_schemas/03130_nested_schema.proto new file mode 100644 index 00000000000..92ace7083f1 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/03130_nested_schema.proto @@ -0,0 +1,16 @@ +syntax = "proto3"; + +message Duration { + int64 seconds = 1; + int32 nanos = 2; +} + +message Inner { + Duration a = 7; + Duration b = 8; + Duration c = 9; +} + +message Outer { + Inner inner = 6; +} From 328154f567c2f30b5fbe37b5aab4fa698f718e2b Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 24 Apr 2024 15:05:30 +0200 Subject: [PATCH 28/50] Fix 03013_forbid_attach_table_if_active_replica_already_exists for private --- ...03013_forbid_attach_table_if_active_replica_already_exists.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sh b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sh index 85fb900bf83..03b0c42429e 100755 --- a/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sh +++ b/tests/queries/0_stateless/03013_forbid_attach_table_if_active_replica_already_exists.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-shared-merge-tree CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From e1fc3e6b6c193996751c0f2189271d4444320c42 Mon Sep 17 00:00:00 2001 From: Chandre Van Der Westhuizen <32901682+chandrevdw31@users.noreply.github.com> Date: Wed, 24 Apr 2024 15:23:33 +0200 Subject: [PATCH 29/50] Updated MindsDB Updated MindsDB description. If you have any questions feel free to ask. --- docs/en/interfaces/third-party/integrations.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md index a9f1af93495..5846bf8047a 100644 --- a/docs/en/interfaces/third-party/integrations.md +++ b/docs/en/interfaces/third-party/integrations.md @@ -76,7 +76,7 @@ ClickHouse, Inc. does **not** maintain the tools and libraries listed below and - [clickhouse-maxmind-geoip](https://github.com/AlexeyKupershtokh/clickhouse-maxmind-geoip) - AutoML - [MindsDB](https://mindsdb.com/) - - [MindsDB](https://github.com/mindsdb/mindsdb) - Predictive AI layer for ClickHouse database. + - [MindsDB](https://github.com/mindsdb/mindsdb) - The platform for customizing AI from enterprise data. ## Programming Language Ecosystems {#programming-language-ecosystems} From f323b68675af848d9eee89bbbf6a6fb723a66edc Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Wed, 24 Apr 2024 06:28:40 -0700 Subject: [PATCH 30/50] Remove unused c files in build --- contrib/openssl-cmake/CMakeLists.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/contrib/openssl-cmake/CMakeLists.txt b/contrib/openssl-cmake/CMakeLists.txt index 14528fbc16f..e6f3fde9f35 100644 --- a/contrib/openssl-cmake/CMakeLists.txt +++ b/contrib/openssl-cmake/CMakeLists.txt @@ -1371,8 +1371,6 @@ set(SSL_SRC ${OPENSSL_SOURCE_DIR}/ssl/quic/uint_set.c ${OPENSSL_SOURCE_DIR}/ssl/record/rec_layer_d1.c ${OPENSSL_SOURCE_DIR}/ssl/record/rec_layer_s3.c - ${OPENSSL_SOURCE_DIR}/ssl/record/methods/tls_pad.c - ${OPENSSL_SOURCE_DIR}/ssl/record/methods/ssl3_cbc.c ${OPENSSL_SOURCE_DIR}/ssl/record/methods/dtls_meth.c ${OPENSSL_SOURCE_DIR}/ssl/record/methods/ssl3_meth.c ${OPENSSL_SOURCE_DIR}/ssl/record/methods/tls13_meth.c From 88a25eb49c4cc8592312604e2576564ef06971b4 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 24 Apr 2024 15:33:59 +0200 Subject: [PATCH 31/50] Fix 03015_optimize_final_rmt in private --- tests/queries/0_stateless/03015_optimize_final_rmt.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03015_optimize_final_rmt.sh b/tests/queries/0_stateless/03015_optimize_final_rmt.sh index f822a401c49..d86617abd31 100755 --- a/tests/queries/0_stateless/03015_optimize_final_rmt.sh +++ b/tests/queries/0_stateless/03015_optimize_final_rmt.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-random-settings, no-random-merge-tree-settings +# Tags: long, no-random-settings, no-random-merge-tree-settings, no-shared-merge-tree CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 70135abe702e9756e9fadccce1893560a96cdad1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 24 Apr 2024 15:50:21 +0200 Subject: [PATCH 32/50] Disable optimize_rewrite_aggregate_function_with_if for sum(nullable) --- .../Passes/RewriteAggregateFunctionWithIfPass.cpp | 14 +++++++++++--- .../03131_rewrite_sum_if_nullable.reference | 3 +++ .../0_stateless/03131_rewrite_sum_if_nullable.sql | 5 +++++ 3 files changed, 19 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03131_rewrite_sum_if_nullable.reference create mode 100644 tests/queries/0_stateless/03131_rewrite_sum_if_nullable.sql diff --git a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp index 9f3e6aa4482..9e3840a083e 100644 --- a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp +++ b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp @@ -51,10 +51,15 @@ public: { const auto & second_const_value = second_const_node->getValue(); if (second_const_value.isNull() - || (lower_name == "sum" && isInt64OrUInt64FieldType(second_const_value.getType()) && second_const_value.get() == 0)) + || (lower_name == "sum" && isInt64OrUInt64FieldType(second_const_value.getType()) && second_const_value.get() == 0 + && !function_node->getResultType()->isNullable())) { /// avg(if(cond, a, null)) -> avgIf(a, cond) + /// avg(if(cond, nullable_a, null)) -> avgIfOrNull(a, cond) + /// sum(if(cond, a, 0)) -> sumIf(a, cond) + /// sum(if(cond, nullable_a, 0)) **is not** equivalent to sumIfOrNull(cond, nullable_a) as + /// it changes the output when no rows pass the condition (from 0 to NULL) function_arguments_nodes.resize(2); function_arguments_nodes[0] = std::move(if_arguments_nodes[1]); function_arguments_nodes[1] = std::move(if_arguments_nodes[0]); @@ -66,10 +71,13 @@ public: { const auto & first_const_value = first_const_node->getValue(); if (first_const_value.isNull() - || (lower_name == "sum" && isInt64OrUInt64FieldType(first_const_value.getType()) && first_const_value.get() == 0)) + || (lower_name == "sum" && isInt64OrUInt64FieldType(first_const_value.getType()) && first_const_value.get() == 0 + && !function_node->getResultType()->isNullable())) { - /// avg(if(cond, null, a) -> avgIf(a, !cond)) + /// avg(if(cond, null, a) -> avgIfOrNullable(a, !cond)) + /// sum(if(cond, 0, a) -> sumIf(a, !cond)) + /// sum(if(cond, 0, nullable_a) **is not** sumIf(a, !cond)) -> Same as above auto not_function = std::make_shared("not"); auto & not_function_arguments = not_function->getArguments().getNodes(); not_function_arguments.push_back(std::move(if_arguments_nodes[0])); diff --git a/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.reference b/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.reference new file mode 100644 index 00000000000..bb0b1cf658d --- /dev/null +++ b/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.reference @@ -0,0 +1,3 @@ +0 +0 +0 diff --git a/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.sql b/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.sql new file mode 100644 index 00000000000..7b470c12367 --- /dev/null +++ b/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.sql @@ -0,0 +1,5 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/62890 +SELECT sum(if(materialize(0), toNullable(1), 0)); +SELECT sum(if(dummy, 0, toNullable(0))); + +SELECT sum(if(s == '', v, 0)) b from VALUES ('v Nullable(Int64), s String',(1, 'x')); From f77b5c3c8cef1e6ba10ebeb90a4dc6155cd43a22 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 24 Apr 2024 16:14:28 +0200 Subject: [PATCH 33/50] Update associated tests --- ...3010_sum_to_to_count_if_nullable.reference | 42 ++++++++++--------- 1 file changed, 23 insertions(+), 19 deletions(-) diff --git a/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.reference b/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.reference index d20634acf93..79ebc7a5c0c 100644 --- a/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.reference +++ b/tests/queries/0_stateless/03010_sum_to_to_count_if_nullable.reference @@ -43,30 +43,34 @@ QUERY id: 0 FUNCTION id: 2, function_name: tuple, function_type: ordinary, result_type: Tuple(Nullable(UInt64), Nullable(Nothing)) ARGUMENTS LIST id: 3, nodes: 2 - FUNCTION id: 4, function_name: sumOrNullIf, function_type: aggregate, result_type: Nullable(UInt64) + FUNCTION id: 4, function_name: sum, function_type: aggregate, result_type: Nullable(UInt64) ARGUMENTS - LIST id: 5, nodes: 2 - CONSTANT id: 6, constant_value: UInt64_1, constant_value_type: Nullable(UInt8) - EXPRESSION - FUNCTION id: 7, function_name: toNullable, function_type: ordinary, result_type: Nullable(UInt8) - ARGUMENTS - LIST id: 8, nodes: 1 - CONSTANT id: 9, constant_value: UInt64_1, constant_value_type: UInt8 - FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8 + LIST id: 5, nodes: 1 + FUNCTION id: 6, function_name: if, function_type: ordinary, result_type: Nullable(UInt8) ARGUMENTS - LIST id: 11, nodes: 2 - FUNCTION id: 12, function_name: modulo, function_type: ordinary, result_type: UInt8 + LIST id: 7, nodes: 3 + FUNCTION id: 8, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 13, nodes: 2 - COLUMN id: 14, column_name: number, result_type: UInt64, source_id: 15 - CONSTANT id: 16, constant_value: UInt64_2, constant_value_type: UInt8 - CONSTANT id: 17, constant_value: UInt64_0, constant_value_type: UInt8 - CONSTANT id: 18, constant_value: NULL, constant_value_type: Nullable(Nothing) + LIST id: 9, nodes: 2 + FUNCTION id: 10, function_name: modulo, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + COLUMN id: 12, column_name: number, result_type: UInt64, source_id: 13 + CONSTANT id: 14, constant_value: UInt64_2, constant_value_type: UInt8 + CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 16, constant_value: UInt64_1, constant_value_type: Nullable(UInt8) + EXPRESSION + FUNCTION id: 17, function_name: toNullable, function_type: ordinary, result_type: Nullable(UInt8) + ARGUMENTS + LIST id: 18, nodes: 1 + CONSTANT id: 19, constant_value: UInt64_1, constant_value_type: UInt8 + CONSTANT id: 20, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 21, constant_value: NULL, constant_value_type: Nullable(Nothing) JOIN TREE - TABLE_FUNCTION id: 15, alias: __table1, table_function_name: numbers + TABLE_FUNCTION id: 13, alias: __table1, table_function_name: numbers ARGUMENTS - LIST id: 19, nodes: 1 - CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt8 + LIST id: 22, nodes: 1 + CONSTANT id: 23, constant_value: UInt64_10, constant_value_type: UInt8 ((6150),3) QUERY id: 0 PROJECTION COLUMNS From 2f0fb73d7dc8407217a2951c2f0fc8fae41f3ae6 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Wed, 24 Apr 2024 07:15:16 -0700 Subject: [PATCH 34/50] Add back two fips c files under condition --- contrib/openssl-cmake/CMakeLists.txt | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/contrib/openssl-cmake/CMakeLists.txt b/contrib/openssl-cmake/CMakeLists.txt index e6f3fde9f35..f7ecf7be9f9 100644 --- a/contrib/openssl-cmake/CMakeLists.txt +++ b/contrib/openssl-cmake/CMakeLists.txt @@ -1145,6 +1145,13 @@ set(CRYPTO_SRC ${OPENSSL_SOURCE_DIR}/ssl/record/methods/ssl3_cbc.c ) +if(NOT ENABLE_OPENSSL_DYNAMIC) + set(CRYPTO_SRC ${CRYPTO_SRC} + ${OPENSSL_SOURCE_DIR}/providers/fips/fips_entry.c + ${OPENSSL_SOURCE_DIR}/providers/fips/fipsprov.c + ) +endif() + if(ARCH_AMD64) if (OS_DARWIN) set(CRYPTO_SRC ${CRYPTO_SRC} From bff2f2db65c3a9a7083c563a2449c5aa2dd61c01 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 24 Apr 2024 16:20:03 +0200 Subject: [PATCH 35/50] Add more tests --- .../03131_rewrite_sum_if_nullable.reference | 21 +++++++++++++++++++ .../03131_rewrite_sum_if_nullable.sql | 12 ++++++++++- 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.reference b/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.reference index bb0b1cf658d..4d8c835e921 100644 --- a/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.reference +++ b/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.reference @@ -1,3 +1,24 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/62890 +-- { echoOn } +SELECT sum(if(materialize(0), toNullable(1), 0)); 0 +SELECT sum(if(dummy, 0, toNullable(0))); 0 +SELECT sum(if(s == '', v, 0)) b from VALUES ('v Nullable(Int64), s String',(1, 'x')); 0 +SELECT sumOrNull(if(materialize(0), toNullable(1), 0)); +0 +SELECT sumOrNull(if(materialize(0), 1, 0)); +0 +SELECT sum(if(materialize(0), toNullable(1), 0)) settings aggregate_functions_null_for_empty=1; +0 +SELECT sum(if(materialize(0), 1, 0)) settings aggregate_functions_null_for_empty=1; +0 +SELECT sumOrNull(if(materialize(1), toNullable(1), 10)); +1 +SELECT sumOrNull(if(materialize(1), 1, 10)); +1 +SELECT sum(if(materialize(1), toNullable(1), 10)) settings aggregate_functions_null_for_empty=1; +1 +SELECT sum(if(materialize( 1), 1, 10)) settings aggregate_functions_null_for_empty=1; +1 diff --git a/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.sql b/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.sql index 7b470c12367..a4e175378c1 100644 --- a/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.sql +++ b/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.sql @@ -1,5 +1,15 @@ -- https://github.com/ClickHouse/ClickHouse/issues/62890 +-- { echoOn } SELECT sum(if(materialize(0), toNullable(1), 0)); SELECT sum(if(dummy, 0, toNullable(0))); - SELECT sum(if(s == '', v, 0)) b from VALUES ('v Nullable(Int64), s String',(1, 'x')); + +SELECT sumOrNull(if(materialize(0), toNullable(1), 0)); +SELECT sumOrNull(if(materialize(0), 1, 0)); +SELECT sum(if(materialize(0), toNullable(1), 0)) settings aggregate_functions_null_for_empty=1; +SELECT sum(if(materialize(0), 1, 0)) settings aggregate_functions_null_for_empty=1; + +SELECT sumOrNull(if(materialize(1), toNullable(1), 10)); +SELECT sumOrNull(if(materialize(1), 1, 10)); +SELECT sum(if(materialize(1), toNullable(1), 10)) settings aggregate_functions_null_for_empty=1; +SELECT sum(if(materialize( 1), 1, 10)) settings aggregate_functions_null_for_empty=1; From a4055ae879b989324824f67cba7363c095fb3314 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 23 Apr 2024 13:03:15 +0300 Subject: [PATCH 36/50] SYSTEM DROP uninitialized cache fix --- src/Interpreters/Cache/FileCache.cpp | 5 ++++ src/Interpreters/Cache/FileCache.h | 2 ++ src/Interpreters/InterpreterSystemQuery.cpp | 27 ++++++++++++++------- 3 files changed, 25 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 890046bf751..7ebeefe2a8a 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -128,6 +128,11 @@ const FileCache::UserInfo & FileCache::getInternalUser() return user; } +bool FileCache::isInitialized() const +{ + return is_initialized.load(std::memory_order_seq_cst); +} + const String & FileCache::getBasePath() const { return metadata.getBaseDirectory(); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 684ca83fff8..ebb2947a738 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -80,6 +80,8 @@ public: void initialize(); + bool isInitialized() const; + const String & getBasePath() const; static Key createKeyForPath(const String & path); diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index f6db12e977c..bdbdf468600 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -397,22 +397,31 @@ BlockIO InterpreterSystemQuery::execute() { auto caches = FileCacheFactory::instance().getAll(); for (const auto & [_, cache_data] : caches) + { + if (!cache_data->cache->isInitialized()) + continue; + cache_data->cache->removeAllReleasable(user_id); + } } else { auto cache = FileCacheFactory::instance().getByName(query.filesystem_cache_name)->cache; - if (query.key_to_drop.empty()) + + if (cache->isInitialized()) { - cache->removeAllReleasable(user_id); - } - else - { - auto key = FileCacheKey::fromKeyString(query.key_to_drop); - if (query.offset_to_drop.has_value()) - cache->removeFileSegment(key, query.offset_to_drop.value(), user_id); + if (query.key_to_drop.empty()) + { + cache->removeAllReleasable(user_id); + } else - cache->removeKey(key, user_id); + { + auto key = FileCacheKey::fromKeyString(query.key_to_drop); + if (query.offset_to_drop.has_value()) + cache->removeFileSegment(key, query.offset_to_drop.value(), user_id); + else + cache->removeKey(key, user_id); + } } } break; From da4690f1dab8400d581906c544c409cec4b5a224 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Apr 2024 17:52:23 +0200 Subject: [PATCH 37/50] Add some functions to zookeeper client --- src/Common/ZooKeeper/ZooKeeper.cpp | 47 ++++++++++++++++++++++++-- src/Common/ZooKeeper/ZooKeeper.h | 12 +++++++ src/Common/ZooKeeper/ZooKeeperLock.cpp | 23 ++++++++++--- src/Common/ZooKeeper/ZooKeeperLock.h | 6 ++-- 4 files changed, 78 insertions(+), 10 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index ca0a211c716..c62c5d0c143 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -518,7 +518,8 @@ bool ZooKeeper::existsWatch(const std::string & path, Coordination::Stat * stat, return code != Coordination::Error::ZNONODE; } -Coordination::Error ZooKeeper::getImpl(const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback) +Coordination::Error ZooKeeper::getImpl( + const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallbackPtr watch_callback) { auto future_result = asyncTryGetNoThrow(path, watch_callback); @@ -541,6 +542,11 @@ Coordination::Error ZooKeeper::getImpl(const std::string & path, std::string & r } } +Coordination::Error ZooKeeper::getImpl(const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback) +{ + return getImpl(path, res, stat, watch_callback ? std::make_shared(watch_callback) : Coordination::WatchCallbackPtr{}); +} + std::string ZooKeeper::get(const std::string & path, Coordination::Stat * stat, const EventPtr & watch) { Coordination::Error code = Coordination::Error::ZOK; @@ -561,6 +567,17 @@ std::string ZooKeeper::getWatch(const std::string & path, Coordination::Stat * s throw KeeperException(code, "Can't get data for node '{}': node doesn't exist", path); } + +std::string ZooKeeper::getWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallbackPtr watch_callback) +{ + Coordination::Error code = Coordination::Error::ZOK; + std::string res; + if (tryGetWatch(path, res, stat, watch_callback, &code)) + return res; + else + throw KeeperException(code, "Can't get data for node '{}': node doesn't exist", path); +} + bool ZooKeeper::tryGet( const std::string & path, std::string & res, @@ -571,6 +588,25 @@ bool ZooKeeper::tryGet( return tryGetWatch(path, res, stat, callbackForEvent(watch), return_code); } +bool ZooKeeper::tryGetWatch( + const std::string & path, + std::string & res, + Coordination::Stat * stat, + Coordination::WatchCallbackPtr watch_callback, + Coordination::Error * return_code) +{ + Coordination::Error code = getImpl(path, res, stat, watch_callback); + + if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) + throw KeeperException::fromPath(code, path); + + if (return_code) + *return_code = code; + + return code == Coordination::Error::ZOK; + +} + bool ZooKeeper::tryGetWatch( const std::string & path, std::string & res, @@ -589,6 +625,7 @@ bool ZooKeeper::tryGetWatch( return code == Coordination::Error::ZOK; } + Coordination::Error ZooKeeper::setImpl(const std::string & path, const std::string & data, int32_t version, Coordination::Stat * stat) { @@ -1062,6 +1099,11 @@ std::future ZooKeeper::asyncGet(const std::string & p } std::future ZooKeeper::asyncTryGetNoThrow(const std::string & path, Coordination::WatchCallback watch_callback) +{ + return asyncTryGetNoThrow(path, watch_callback ? std::make_shared(watch_callback) : Coordination::WatchCallbackPtr{}); +} + +std::future ZooKeeper::asyncTryGetNoThrow(const std::string & path, Coordination::WatchCallbackPtr watch_callback) { auto promise = std::make_shared>(); auto future = promise->get_future(); @@ -1071,8 +1113,7 @@ std::future ZooKeeper::asyncTryGetNoThrow(const std:: promise->set_value(response); }; - impl->get(path, std::move(callback), - watch_callback ? std::make_shared(watch_callback) : Coordination::WatchCallbackPtr{}); + impl->get(path, std::move(callback), watch_callback); return future; } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 42c77e5ca72..82ce3f72a53 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -306,6 +306,7 @@ public: std::string get(const std::string & path, Coordination::Stat * stat = nullptr, const EventPtr & watch = nullptr); std::string getWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback); + std::string getWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallbackPtr watch_callback); using MultiGetResponse = MultiReadResponses; using MultiTryGetResponse = MultiReadResponses; @@ -338,6 +339,13 @@ public: Coordination::WatchCallback watch_callback, Coordination::Error * code = nullptr); + bool tryGetWatch( + const std::string & path, + std::string & res, + Coordination::Stat * stat, + Coordination::WatchCallbackPtr watch_callback, + Coordination::Error * code = nullptr); + template MultiTryGetResponse tryGet(TIter start, TIter end) { @@ -520,6 +528,8 @@ public: /// Like the previous one but don't throw any exceptions on future.get() FutureGet asyncTryGetNoThrow(const std::string & path, Coordination::WatchCallback watch_callback = {}); + FutureGet asyncTryGetNoThrow(const std::string & path, Coordination::WatchCallbackPtr watch_callback = {}); + using FutureExists = std::future; FutureExists asyncExists(const std::string & path, Coordination::WatchCallback watch_callback = {}); /// Like the previous one but don't throw any exceptions on future.get() @@ -625,6 +635,8 @@ private: Coordination::Error removeImpl(const std::string & path, int32_t version); Coordination::Error getImpl( const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback); + Coordination::Error getImpl( + const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallbackPtr watch_callback); Coordination::Error setImpl(const std::string & path, const std::string & data, int32_t version, Coordination::Stat * stat); Coordination::Error getChildrenImpl( const std::string & path, diff --git a/src/Common/ZooKeeper/ZooKeeperLock.cpp b/src/Common/ZooKeeper/ZooKeeperLock.cpp index b90bcfd2b55..acd06e0a08c 100644 --- a/src/Common/ZooKeeper/ZooKeeperLock.cpp +++ b/src/Common/ZooKeeper/ZooKeeperLock.cpp @@ -22,13 +22,16 @@ ZooKeeperLock::ZooKeeperLock( const ZooKeeperPtr & zookeeper_, const std::string & lock_prefix_, const std::string & lock_name_, - const std::string & lock_message_) + const std::string & lock_message_, + bool throw_if_lost_) : zookeeper(zookeeper_) , lock_path(fs::path(lock_prefix_) / lock_name_) , lock_message(lock_message_) + , throw_if_lost(throw_if_lost_) , log(getLogger("zkutil::Lock")) { zookeeper->createIfNotExists(lock_prefix_, ""); + LOG_TRACE(log, "Trying to create zookeeper lock on path {} for session {}", lock_path, zookeeper->getClientID()); } ZooKeeperLock::~ZooKeeperLock() @@ -45,7 +48,7 @@ ZooKeeperLock::~ZooKeeperLock() bool ZooKeeperLock::isLocked() const { - return locked; + return locked && !zookeeper->expired(); } const std::string & ZooKeeperLock::getLockPath() const @@ -56,7 +59,10 @@ const std::string & ZooKeeperLock::getLockPath() const void ZooKeeperLock::unlock() { if (!locked) + { + LOG_TRACE(log, "Lock on path {} for session {} is not locked, exiting", lock_path, zookeeper->getClientID()); return; + } locked = false; @@ -71,12 +77,19 @@ void ZooKeeperLock::unlock() bool result = zookeeper->exists(lock_path, &stat); if (result && stat.ephemeralOwner == zookeeper->getClientID()) + { zookeeper->remove(lock_path, -1); + LOG_TRACE(log, "Lock on path {} for session {} is unlocked", lock_path, zookeeper->getClientID()); + } else if (result) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Lock is lost, it has another owner. Path: {}, message: {}, owner: {}, our id: {}", lock_path, lock_message, stat.ephemeralOwner, zookeeper->getClientID()); + else if (throw_if_lost) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Lock is lost, node does not exist. Path: {}, message: {}, our id: {}", + lock_path, lock_message, zookeeper->getClientID()); else - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Lock is lost, node does not exist. Path: {}, message: {}", lock_path, lock_message); + LOG_INFO(log, "Lock is lost, node does not exist. Path: {}, message: {}, our id: {}", + lock_path, lock_message, zookeeper->getClientID()); } bool ZooKeeperLock::tryLock() @@ -96,9 +109,9 @@ bool ZooKeeperLock::tryLock() } std::unique_ptr createSimpleZooKeeperLock( - const ZooKeeperPtr & zookeeper, const String & lock_prefix, const String & lock_name, const String & lock_message) + const ZooKeeperPtr & zookeeper, const String & lock_prefix, const String & lock_name, const String & lock_message, bool throw_if_lost) { - return std::make_unique(zookeeper, lock_prefix, lock_name, lock_message); + return std::make_unique(zookeeper, lock_prefix, lock_name, lock_message, throw_if_lost); } diff --git a/src/Common/ZooKeeper/ZooKeeperLock.h b/src/Common/ZooKeeper/ZooKeeperLock.h index 6271afaf6e2..86e9349fd66 100644 --- a/src/Common/ZooKeeper/ZooKeeperLock.h +++ b/src/Common/ZooKeeper/ZooKeeperLock.h @@ -32,7 +32,8 @@ public: const ZooKeeperPtr & zookeeper_, const std::string & lock_prefix_, const std::string & lock_name_, - const std::string & lock_message_ = ""); + const std::string & lock_message_ = "", + bool throw_if_lost_ = true); ~ZooKeeperLock(); @@ -46,12 +47,13 @@ private: std::string lock_path; std::string lock_message; + bool throw_if_lost{true}; LoggerPtr log; bool locked = false; }; std::unique_ptr createSimpleZooKeeperLock( - const ZooKeeperPtr & zookeeper, const String & lock_prefix, const String & lock_name, const String & lock_message); + const ZooKeeperPtr & zookeeper, const String & lock_prefix, const String & lock_name, const String & lock_message, bool throw_if_lost = true); } From 1fb539704ef8f9ebaa9ab9f42f214fdd7187f67d Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 24 Apr 2024 18:43:44 +0200 Subject: [PATCH 38/50] Update length and lengthUTF8 --- .../functions/string-functions.md | 83 +++++++++++++++++-- 1 file changed, 78 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 400d29364a2..14c52807d1e 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -88,20 +88,93 @@ Result: ## length -Returns the length of a string in bytes (not: in characters or Unicode code points). - -The function also works for arrays. +Returns the length of a string in bytes rather than in characters or Unicode code points. The function also works for arrays. Alias: `OCTET_LENGTH` +**Syntax** + +```sql +length(s) +``` + +**Parameters** + +- `s`: An input string. [String](../data-types/string)/[Array](../data-types/array). + +**Returned value** + +- Length of the string `s` in bytes. [UInt64](../data-types/int-uint). + +**Example** + +Query: + +```sql +SELECT length('Hello, world!'); +``` + +Result: + +```response +┌─length('Hello, world!')─┐ +│ 13 │ +└─────────────────────────┘ +``` + +Query: + +```sql +SELECT length([1, 2, 3, 4]); +``` + +Result: + +```response +┌─length([1, 2, 3, 4])─┐ +│ 4 │ +└──────────────────────┘ +``` + + ## lengthUTF8 -Returns the length of a string in Unicode code points (not: in bytes or characters). It assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. +Returns the length of a string in Unicode code points rather than in bytes or characters. It assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. -Alias: +Aliases: - `CHAR_LENGTH` - `CHARACTER_LENGTH` +**Syntax** + +```sql +lengthUTF8(s) +``` + +**Parameters** + +- `s`: String containing valid UTF-8 encoded text. [String](../data-types/string). + +**Returned value** + +- Length of the string `s` in Unicode code points. [UInt64](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT lengthUTF8('Здравствуй, мир!'); +``` + +Result: + +```response +┌─lengthUTF8('Здравствуй, мир!')─┐ +│ 16 │ +└────────────────────────────────┘ +``` + ## left Returns a substring of string `s` with a specified `offset` starting from the left. From 9d2ee35407781f340b918b833df8787fdd698e04 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 24 Apr 2024 18:53:29 +0200 Subject: [PATCH 39/50] Add more tests --- .../03131_rewrite_sum_if_nullable.reference | 10 ++++++++++ .../0_stateless/03131_rewrite_sum_if_nullable.sql | 5 +++++ 2 files changed, 15 insertions(+) diff --git a/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.reference b/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.reference index 4d8c835e921..ba2fbbfbeaf 100644 --- a/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.reference +++ b/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.reference @@ -2,8 +2,18 @@ -- { echoOn } SELECT sum(if(materialize(0), toNullable(1), 0)); 0 +SELECT sum(if(materialize(0), toNullable(1), materialize(0))); +0 +SELECT sum(if(materialize(0), materialize(toNullable(1)), materialize(0))); +0 +SELECT sum(if(materialize(0), materialize(1), materialize(0))); +0 SELECT sum(if(dummy, 0, toNullable(0))); 0 +SELECT sum(if(dummy, materialize(0), toNullable(0))); +0 +SELECT sum(if(dummy, materialize(0), materialize(toNullable(0)))); +0 SELECT sum(if(s == '', v, 0)) b from VALUES ('v Nullable(Int64), s String',(1, 'x')); 0 SELECT sumOrNull(if(materialize(0), toNullable(1), 0)); diff --git a/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.sql b/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.sql index a4e175378c1..ecc4db6fcf3 100644 --- a/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.sql +++ b/tests/queries/0_stateless/03131_rewrite_sum_if_nullable.sql @@ -1,7 +1,12 @@ -- https://github.com/ClickHouse/ClickHouse/issues/62890 -- { echoOn } SELECT sum(if(materialize(0), toNullable(1), 0)); +SELECT sum(if(materialize(0), toNullable(1), materialize(0))); +SELECT sum(if(materialize(0), materialize(toNullable(1)), materialize(0))); +SELECT sum(if(materialize(0), materialize(1), materialize(0))); SELECT sum(if(dummy, 0, toNullable(0))); +SELECT sum(if(dummy, materialize(0), toNullable(0))); +SELECT sum(if(dummy, materialize(0), materialize(toNullable(0)))); SELECT sum(if(s == '', v, 0)) b from VALUES ('v Nullable(Int64), s String',(1, 'x')); SELECT sumOrNull(if(materialize(0), toNullable(1), 0)); From 15594c58ad49f0c46ff8e48678749289700d438d Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 24 Apr 2024 18:54:07 +0200 Subject: [PATCH 40/50] empty commit From 049c429b1f8c1e4edec077043870b9203e29aef7 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 24 Apr 2024 19:19:27 +0200 Subject: [PATCH 41/50] Revert "CI: add FT to MQ remove Style from master" --- .github/workflows/master.yml | 21 ++++++++++----------- tests/ci/ci.py | 30 +++++++++++++++++------------- 2 files changed, 27 insertions(+), 24 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index c5acd183751..100ec3b3b2c 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -59,17 +59,16 @@ jobs: uses: ./.github/workflows/reusable_docker.yml with: data: ${{ needs.RunConfig.outputs.data }} - # Tested in MQ - # StyleCheck: - # needs: [RunConfig, BuildDockers] - # if: ${{ !failure() && !cancelled() }} - # uses: ./.github/workflows/reusable_test.yml - # with: - # test_name: Style check - # runner_type: style-checker - # data: ${{ needs.RunConfig.outputs.data }} - # run_command: | - # python3 style_check.py --no-push + StyleCheck: + needs: [RunConfig, BuildDockers] + if: ${{ !failure() && !cancelled() }} + uses: ./.github/workflows/reusable_test.yml + with: + test_name: Style check + runner_type: style-checker + data: ${{ needs.RunConfig.outputs.data }} + run_command: | + python3 style_check.py --no-push CompatibilityCheckX86: needs: [RunConfig, BuilderDebRelease] if: ${{ !failure() && !cancelled() }} diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 7a7082c9c4a..24d6d95bd0d 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1261,6 +1261,18 @@ def _print_results(result: Any, outfile: Optional[str], pretty: bool = False) -> raise AssertionError(f"Unexpected type for 'res': {type(result)}") +def _update_config_for_docs_only(jobs_data: dict) -> None: + DOCS_CHECK_JOBS = [JobNames.DOCS_CHECK, JobNames.STYLE_CHECK] + print(f"NOTE: Will keep only docs related jobs: [{DOCS_CHECK_JOBS}]") + jobs_to_do = jobs_data.get("jobs_to_do", []) + jobs_data["jobs_to_do"] = [job for job in jobs_to_do if job in DOCS_CHECK_JOBS] + jobs_data["jobs_to_wait"] = { + job: params + for job, params in jobs_data["jobs_to_wait"].items() + if job in DOCS_CHECK_JOBS + } + + def _configure_docker_jobs(docker_digest_or_latest: bool) -> Dict: print("::group::Docker images check") # generate docker jobs data @@ -1320,20 +1332,8 @@ def _configure_jobs( jobs_to_skip: List[str] = [] digests: Dict[str, str] = {} - # FIXME: find better place for these config variables - DOCS_CHECK_JOBS = [JobNames.DOCS_CHECK, JobNames.STYLE_CHECK] - MQ_JOBS = [JobNames.STYLE_CHECK, JobNames.FAST_TEST] - if pr_info.has_changes_in_documentation_only(): - print(f"WARNING: Only docs are changed - will run only [{DOCS_CHECK_JOBS}]") - if pr_info.is_merge_queue(): - print(f"WARNING: It's a MQ run - will run only [{MQ_JOBS}]") - print("::group::Job Digests") for job in CI_CONFIG.job_generator(pr_info.head_ref if CI else "dummy_branch_name"): - if pr_info.is_merge_queue() and job not in MQ_JOBS: - continue - if pr_info.has_changes_in_documentation_only() and job not in DOCS_CHECK_JOBS: - continue digest = job_digester.get_job_digest(CI_CONFIG.get_digest_config(job)) digests[job] = digest print(f" job [{job.rjust(50)}] has digest [{digest}]") @@ -1902,6 +1902,9 @@ def main() -> int: else {} ) + if not args.skip_jobs and pr_info.has_changes_in_documentation_only(): + _update_config_for_docs_only(jobs_data) + if not args.skip_jobs: ci_cache = CiCache(s3, jobs_data["digests"]) @@ -1925,7 +1928,8 @@ def main() -> int: jobs_to_skip.append(job) del jobs_params[job] - # set planned jobs as in-progress in CI cache + # set planned jobs as pending in the CI cache if on the master + if pr_info.is_master: for job in jobs_data["jobs_to_do"]: config = CI_CONFIG.get_job_config(job) if config.run_always or config.run_by_label: From 30583d594c20c331619f329e91c8689ee6a7862c Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Wed, 24 Apr 2024 19:52:14 +0200 Subject: [PATCH 42/50] Update string-functions.md Add mention of array to parameter --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 14c52807d1e..950f77cc685 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -100,7 +100,7 @@ length(s) **Parameters** -- `s`: An input string. [String](../data-types/string)/[Array](../data-types/array). +- `s`: An input string or array. [String](../data-types/string)/[Array](../data-types/array). **Returned value** From 2d9a1d5259fa32c75676444a656ebf8ce87956cc Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Wed, 24 Apr 2024 19:54:00 +0200 Subject: [PATCH 43/50] Update string-functions.md --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 950f77cc685..7e68b11b71a 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -104,7 +104,7 @@ length(s) **Returned value** -- Length of the string `s` in bytes. [UInt64](../data-types/int-uint). +- Length of the string or array `s` in bytes. [UInt64](../data-types/int-uint). **Example** From c91f8ffc39c1b30f10852185579dac93551f8ec8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 24 Apr 2024 18:28:31 +0000 Subject: [PATCH 44/50] Unflake 02813_func_now_and_alias.sql --- contrib/openssl-cmake/CMakeLists.txt | 2 +- tests/queries/0_stateless/02813_func_now_and_alias.sql | 9 ++++++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/contrib/openssl-cmake/CMakeLists.txt b/contrib/openssl-cmake/CMakeLists.txt index f7ecf7be9f9..81f4febf117 100644 --- a/contrib/openssl-cmake/CMakeLists.txt +++ b/contrib/openssl-cmake/CMakeLists.txt @@ -93,7 +93,7 @@ enable_language(ASM) if(COMPILER_CLANG) add_definitions(-Wno-unused-command-line-argument) - set(CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=lld") + set(CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=lld") # only relevant for -DENABLE_OPENSSL_DYNAMIC=1 endif() if(ARCH_AMD64) diff --git a/tests/queries/0_stateless/02813_func_now_and_alias.sql b/tests/queries/0_stateless/02813_func_now_and_alias.sql index bcda1f26181..6a2acbc03c3 100644 --- a/tests/queries/0_stateless/02813_func_now_and_alias.sql +++ b/tests/queries/0_stateless/02813_func_now_and_alias.sql @@ -1,3 +1,6 @@ -SELECT now() = current_timestamp(); -SELECT now() = CURRENT_TIMESTAMP(); -SELECT now() = current_TIMESTAMP(); +-- "Tests" current_timestamp() which is an alias of now(). +-- Since the function is non-deterministic, only check that no bad things happen (don't check the returned value). + +SELECT count() FROM (SELECT current_timestamp()); +SELECT count() FROM (SELECT CURRENT_TIMESTAMP()); +SELECT count() FROM (SELECT current_TIMESTAMP()); From f0462efb73511944704471eb77e3c7ceb7502364 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 24 Apr 2024 13:29:50 +0100 Subject: [PATCH 45/50] impl --- .../00993_system_parts_race_condition_drop_zookeeper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh index 6025279e570..5aa1281edb6 100755 --- a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh +++ b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh @@ -37,7 +37,7 @@ function thread4() { while true; do REPLICA=$(($RANDOM % 10)) - $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table_$REPLICA FINAL"; + $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table_$REPLICA FINAL SETTINGS receive_timeout=1"; sleep 0.$RANDOM; done } From 7a25e186bb8e949356f0efaba9c5c878c502d445 Mon Sep 17 00:00:00 2001 From: tomershafir Date: Thu, 11 Apr 2024 11:44:54 +0300 Subject: [PATCH 46/50] COW: assert in examples --- src/Common/examples/cow_columns.cpp | 68 ++++++++++++++---------- src/Common/examples/cow_compositions.cpp | 60 +++++++++++++-------- 2 files changed, 78 insertions(+), 50 deletions(-) diff --git a/src/Common/examples/cow_columns.cpp b/src/Common/examples/cow_columns.cpp index 404b478f5a0..a0cfc1f12d1 100644 --- a/src/Common/examples/cow_columns.cpp +++ b/src/Common/examples/cow_columns.cpp @@ -1,5 +1,6 @@ #include #include +#include class IColumn : public COW @@ -15,8 +16,6 @@ public: virtual int get() const = 0; virtual void set(int value) = 0; - - virtual MutablePtr test() const = 0; }; using ColumnPtr = IColumn::Ptr; @@ -31,58 +30,69 @@ private: explicit ConcreteColumn(int data_) : data(data_) {} ConcreteColumn(const ConcreteColumn &) = default; - MutableColumnPtr test() const override - { - MutableColumnPtr res = create(123); - return res; - } - public: int get() const override { return data; } void set(int value) override { data = value; } }; +void print(const ColumnPtr & x, const ColumnPtr & y) +{ + std::cerr << "values: " << x->get() << ", " << y->get() << "\n"; + std::cerr << "refcounts: " << x->use_count() << ", " << y->use_count() << "\n"; + std::cerr << "addresses: " << x.get() << ", " << y.get() << "\n"; +} + +void print(const ColumnPtr & x, const MutableColumnPtr & mut) +{ + std::cerr << "values: " << x->get() << ", " << mut->get() << "\n"; + std::cerr << "refcounts: " << x->use_count() << ", " << mut->use_count() << "\n"; + std::cerr << "addresses: " << x.get() << ", " << mut.get() << "\n"; +} int main(int, char **) { ColumnPtr x = ConcreteColumn::create(1); - ColumnPtr y = x;//x->test(); - - std::cerr << "values: " << x->get() << ", " << y->get() << "\n"; - std::cerr << "refcounts: " << x->use_count() << ", " << y->use_count() << "\n"; - std::cerr << "addresses: " << x.get() << ", " << y.get() << "\n"; + ColumnPtr y = x; + print(x, y); + chassert(x->get() == y->get() == 1); + chassert(x->use_count() == y->use_count() == 2); + chassert(x.get() == y.get()); { MutableColumnPtr mut = IColumn::mutate(std::move(y)); mut->set(2); + print(x, mut); + chassert(x->get() == 1 && mut->get() == 2); + chassert(x->use_count() == mut->use_count() == 1); + chassert(x.get() != mut.get()); - std::cerr << "refcounts: " << x->use_count() << ", " << mut->use_count() << "\n"; - std::cerr << "addresses: " << x.get() << ", " << mut.get() << "\n"; y = std::move(mut); } - - std::cerr << "values: " << x->get() << ", " << y->get() << "\n"; - std::cerr << "refcounts: " << x->use_count() << ", " << y->use_count() << "\n"; - std::cerr << "addresses: " << x.get() << ", " << y.get() << "\n"; + print(x, y); + chassert(x->get() == 1 && y->get() == 2); + chassert(x->use_count() == y->use_count() == 1); + chassert(x.get() != y.get()); x = ConcreteColumn::create(0); - - std::cerr << "values: " << x->get() << ", " << y->get() << "\n"; - std::cerr << "refcounts: " << x->use_count() << ", " << y->use_count() << "\n"; - std::cerr << "addresses: " << x.get() << ", " << y.get() << "\n"; + print(x, y); + chassert(x->get() == 0 && y->get() == 2); + chassert(x->use_count() == y->use_count() == 1); + chassert(x.get() != y.get()); { MutableColumnPtr mut = IColumn::mutate(std::move(y)); mut->set(3); + print(x, mut); + chassert(x->get() == 0 && mut->get() == 3); + chassert(x->use_count() == mut->use_count() == 1); + chassert(x.get() != mut.get()); - std::cerr << "refcounts: " << x->use_count() << ", " << mut->use_count() << "\n"; - std::cerr << "addresses: " << x.get() << ", " << mut.get() << "\n"; y = std::move(mut); } - - std::cerr << "values: " << x->get() << ", " << y->get() << "\n"; - std::cerr << "refcounts: " << x->use_count() << ", " << y->use_count() << "\n"; + print(x, y); + chassert(x->get() == 0 && y->get() == 3); + chassert(x->use_count() == y->use_count() == 1); + chassert(x.get() != y.get()); return 0; } - diff --git a/src/Common/examples/cow_compositions.cpp b/src/Common/examples/cow_compositions.cpp index 05d6fe1d89e..3df196bc6cf 100644 --- a/src/Common/examples/cow_compositions.cpp +++ b/src/Common/examples/cow_compositions.cpp @@ -1,5 +1,6 @@ #include #include +#include class IColumn : public COW @@ -61,47 +62,64 @@ public: void set(int value) override { wrapped->set(value); } }; +void print(const ColumnPtr & x, const ColumnPtr & y) +{ + std::cerr << "values: " << x->get() << ", " << y->get() << "\n"; + std::cerr << "refcounts: " << x->use_count() << ", " << y->use_count() << "\n"; + std::cerr << "addresses: " << x.get() << ", " << y.get() << "\n"; +} + +void print(const ColumnPtr & x, const MutableColumnPtr & mut) +{ + std::cerr << "values: " << x->get() << ", " << mut->get() << "\n"; + std::cerr << "refcounts: " << x->use_count() << ", " << mut->use_count() << "\n"; + std::cerr << "addresses: " << x.get() << ", " << mut.get() << "\n"; +} int main(int, char **) { ColumnPtr x = ColumnComposition::create(1); ColumnPtr y = x; - - std::cerr << "values: " << x->get() << ", " << y->get() << "\n"; - std::cerr << "refcounts: " << x->use_count() << ", " << y->use_count() << "\n"; - std::cerr << "addresses: " << x.get() << ", " << y.get() << "\n"; + print(x, y); + chassert(x->get() == y->get() == 1); + chassert(x->use_count() == y->use_count() == 2); + chassert(x.get() == y.get()); { MutableColumnPtr mut = IColumn::mutate(std::move(y)); mut->set(2); + print(x, mut); + chassert(x->get() == 1 && mut->get() == 2); + chassert(x->use_count() == mut->use_count() == 1); + chassert(x.get() != mut.get()); - std::cerr << "refcounts: " << x->use_count() << ", " << mut->use_count() << "\n"; - std::cerr << "addresses: " << x.get() << ", " << mut.get() << "\n"; y = std::move(mut); } - - std::cerr << "values: " << x->get() << ", " << y->get() << "\n"; - std::cerr << "refcounts: " << x->use_count() << ", " << y->use_count() << "\n"; - std::cerr << "addresses: " << x.get() << ", " << y.get() << "\n"; + print(x, y); + chassert(x->get() == 1 && y->get() == 2); + chassert(x->use_count() == y->use_count() == 1); + chassert(x.get() != y.get()); x = ColumnComposition::create(0); - - std::cerr << "values: " << x->get() << ", " << y->get() << "\n"; - std::cerr << "refcounts: " << x->use_count() << ", " << y->use_count() << "\n"; - std::cerr << "addresses: " << x.get() << ", " << y.get() << "\n"; + print(x, y); + chassert(x->get() == 0 && y->get() == 2); + chassert(x->use_count() == y->use_count() == 1); + chassert(x.get() != y.get()); { MutableColumnPtr mut = IColumn::mutate(std::move(y)); mut->set(3); + print(x, mut); + chassert(x->get() == 0 && mut->get() == 3); + chassert(x->use_count() == mut->use_count() == 1); + chassert(x.get() != mut.get()); - std::cerr << "refcounts: " << x->use_count() << ", " << mut->use_count() << "\n"; - std::cerr << "addresses: " << x.get() << ", " << mut.get() << "\n"; y = std::move(mut); } - - std::cerr << "values: " << x->get() << ", " << y->get() << "\n"; - std::cerr << "refcounts: " << x->use_count() << ", " << y->use_count() << "\n"; - + print(x, y); + chassert(x->get() == 0 && y->get() == 3); + chassert(x->use_count() == y->use_count() == 1); + chassert(x.get() != y.get()); + return 0; } - From 1cbba5c0e2805700637d63ab8a1addd6ff1f6817 Mon Sep 17 00:00:00 2001 From: tomershafir Date: Thu, 11 Apr 2024 15:44:40 +0300 Subject: [PATCH 47/50] fix style --- src/Common/examples/cow_compositions.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/examples/cow_compositions.cpp b/src/Common/examples/cow_compositions.cpp index 3df196bc6cf..335ec1412e7 100644 --- a/src/Common/examples/cow_compositions.cpp +++ b/src/Common/examples/cow_compositions.cpp @@ -120,6 +120,6 @@ int main(int, char **) chassert(x->get() == 0 && y->get() == 3); chassert(x->use_count() == y->use_count() == 1); chassert(x.get() != y.get()); - + return 0; } From ef6585adf5ef2c5d88763d44d15b10e45209295c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 Apr 2024 07:48:57 +0000 Subject: [PATCH 48/50] Fix compiler warnings --- src/Common/examples/cow_columns.cpp | 30 ++++++++++-------------- src/Common/examples/cow_compositions.cpp | 30 ++++++++++-------------- 2 files changed, 24 insertions(+), 36 deletions(-) diff --git a/src/Common/examples/cow_columns.cpp b/src/Common/examples/cow_columns.cpp index a0cfc1f12d1..de639b49b83 100644 --- a/src/Common/examples/cow_columns.cpp +++ b/src/Common/examples/cow_columns.cpp @@ -35,18 +35,12 @@ public: void set(int value) override { data = value; } }; -void print(const ColumnPtr & x, const ColumnPtr & y) +template +void print(const ColumnPtr & x, const ColPtr & y) { - std::cerr << "values: " << x->get() << ", " << y->get() << "\n"; - std::cerr << "refcounts: " << x->use_count() << ", " << y->use_count() << "\n"; - std::cerr << "addresses: " << x.get() << ", " << y.get() << "\n"; -} - -void print(const ColumnPtr & x, const MutableColumnPtr & mut) -{ - std::cerr << "values: " << x->get() << ", " << mut->get() << "\n"; - std::cerr << "refcounts: " << x->use_count() << ", " << mut->use_count() << "\n"; - std::cerr << "addresses: " << x.get() << ", " << mut.get() << "\n"; + std::cerr << "values: " << x->get() << ", " << y->get() << "\n"; + std::cerr << "refcounts: " << x->use_count() << ", " << y->use_count() << "\n"; + std::cerr << "addresses: " << x.get() << ", " << y.get() << "\n"; } int main(int, char **) @@ -54,8 +48,8 @@ int main(int, char **) ColumnPtr x = ConcreteColumn::create(1); ColumnPtr y = x; print(x, y); - chassert(x->get() == y->get() == 1); - chassert(x->use_count() == y->use_count() == 2); + chassert(x->get() == 1 && y->get() == 1); + chassert(x->use_count() == 2 && y->use_count() == 2); chassert(x.get() == y.get()); { @@ -63,20 +57,20 @@ int main(int, char **) mut->set(2); print(x, mut); chassert(x->get() == 1 && mut->get() == 2); - chassert(x->use_count() == mut->use_count() == 1); + chassert(x->use_count() == 1 && mut->use_count() == 1); chassert(x.get() != mut.get()); y = std::move(mut); } print(x, y); chassert(x->get() == 1 && y->get() == 2); - chassert(x->use_count() == y->use_count() == 1); + chassert(x->use_count() == 1 && y->use_count() == 1); chassert(x.get() != y.get()); x = ConcreteColumn::create(0); print(x, y); chassert(x->get() == 0 && y->get() == 2); - chassert(x->use_count() == y->use_count() == 1); + chassert(x->use_count() == 1 && y->use_count() == 1); chassert(x.get() != y.get()); { @@ -84,14 +78,14 @@ int main(int, char **) mut->set(3); print(x, mut); chassert(x->get() == 0 && mut->get() == 3); - chassert(x->use_count() == mut->use_count() == 1); + chassert(x->use_count() == 1 && mut->use_count() == 1); chassert(x.get() != mut.get()); y = std::move(mut); } print(x, y); chassert(x->get() == 0 && y->get() == 3); - chassert(x->use_count() == y->use_count() == 1); + chassert(x->use_count() == 1 && y->use_count() == 1); chassert(x.get() != y.get()); return 0; diff --git a/src/Common/examples/cow_compositions.cpp b/src/Common/examples/cow_compositions.cpp index 335ec1412e7..a45de10f437 100644 --- a/src/Common/examples/cow_compositions.cpp +++ b/src/Common/examples/cow_compositions.cpp @@ -62,18 +62,12 @@ public: void set(int value) override { wrapped->set(value); } }; -void print(const ColumnPtr & x, const ColumnPtr & y) +template +void print(const ColumnPtr & x, const ColPtr & y) { - std::cerr << "values: " << x->get() << ", " << y->get() << "\n"; - std::cerr << "refcounts: " << x->use_count() << ", " << y->use_count() << "\n"; - std::cerr << "addresses: " << x.get() << ", " << y.get() << "\n"; -} - -void print(const ColumnPtr & x, const MutableColumnPtr & mut) -{ - std::cerr << "values: " << x->get() << ", " << mut->get() << "\n"; - std::cerr << "refcounts: " << x->use_count() << ", " << mut->use_count() << "\n"; - std::cerr << "addresses: " << x.get() << ", " << mut.get() << "\n"; + std::cerr << "values: " << x->get() << ", " << y->get() << "\n"; + std::cerr << "refcounts: " << x->use_count() << ", " << y->use_count() << "\n"; + std::cerr << "addresses: " << x.get() << ", " << y.get() << "\n"; } int main(int, char **) @@ -81,8 +75,8 @@ int main(int, char **) ColumnPtr x = ColumnComposition::create(1); ColumnPtr y = x; print(x, y); - chassert(x->get() == y->get() == 1); - chassert(x->use_count() == y->use_count() == 2); + chassert(x->get() == 1 && y->get() == 1); + chassert(x->use_count() == 2 && y->use_count() == 2); chassert(x.get() == y.get()); { @@ -90,20 +84,20 @@ int main(int, char **) mut->set(2); print(x, mut); chassert(x->get() == 1 && mut->get() == 2); - chassert(x->use_count() == mut->use_count() == 1); + chassert(x->use_count() == 1 && mut->use_count() == 1); chassert(x.get() != mut.get()); y = std::move(mut); } print(x, y); chassert(x->get() == 1 && y->get() == 2); - chassert(x->use_count() == y->use_count() == 1); + chassert(x->use_count() == 1 && y->use_count() == 1); chassert(x.get() != y.get()); x = ColumnComposition::create(0); print(x, y); chassert(x->get() == 0 && y->get() == 2); - chassert(x->use_count() == y->use_count() == 1); + chassert(x->use_count() == 1 && y->use_count() == 1); chassert(x.get() != y.get()); { @@ -111,14 +105,14 @@ int main(int, char **) mut->set(3); print(x, mut); chassert(x->get() == 0 && mut->get() == 3); - chassert(x->use_count() == mut->use_count() == 1); + chassert(x->use_count() == 1 && mut->use_count() == 1); chassert(x.get() != mut.get()); y = std::move(mut); } print(x, y); chassert(x->get() == 0 && y->get() == 3); - chassert(x->use_count() == y->use_count() == 1); + chassert(x->use_count() == 1 && y->use_count() == 1); chassert(x.get() != y.get()); return 0; From d1c8f0af82a1240f8224a0ac8c8c3a8415b77228 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 Apr 2024 11:33:36 +0200 Subject: [PATCH 49/50] Better --- src/Common/waitForPid.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Common/waitForPid.cpp b/src/Common/waitForPid.cpp index 0ec10811354..547abcf2a37 100644 --- a/src/Common/waitForPid.cpp +++ b/src/Common/waitForPid.cpp @@ -132,7 +132,9 @@ static PollPidResult pollPid(pid_t pid, int timeout_in_ms) if (kq == -1) return PollPidResult::FAILED; - struct kevent change = {.ident = 0}; + struct kevent change; + change.ident = 0; + EV_SET(&change, pid, EVFILT_PROC, EV_ADD, NOTE_EXIT, 0, NULL); int event_add_result = HANDLE_EINTR(kevent(kq, &change, 1, NULL, 0, NULL)); @@ -144,7 +146,9 @@ static PollPidResult pollPid(pid_t pid, int timeout_in_ms) return PollPidResult::FAILED; } - struct kevent event = {.ident = 0}; + struct kevent event; + event.ident = 0; + struct timespec remaining_timespec = {.tv_sec = timeout_in_ms / 1000, .tv_nsec = (timeout_in_ms % 1000) * 1000000}; int ready = HANDLE_EINTR(kevent(kq, nullptr, 0, &event, 1, &remaining_timespec)); PollPidResult result = ready < 0 ? PollPidResult::FAILED : PollPidResult::RESTART; From 3c3cfae8b3a172c5ba0697569dfd2386bfbc900e Mon Sep 17 00:00:00 2001 From: Chandre Van Der Westhuizen <32901682+chandrevdw31@users.noreply.github.com> Date: Thu, 25 Apr 2024 12:04:34 +0200 Subject: [PATCH 50/50] updated mindsdb --- docs/en/interfaces/third-party/integrations.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md index 5846bf8047a..300c75d9a34 100644 --- a/docs/en/interfaces/third-party/integrations.md +++ b/docs/en/interfaces/third-party/integrations.md @@ -76,7 +76,7 @@ ClickHouse, Inc. does **not** maintain the tools and libraries listed below and - [clickhouse-maxmind-geoip](https://github.com/AlexeyKupershtokh/clickhouse-maxmind-geoip) - AutoML - [MindsDB](https://mindsdb.com/) - - [MindsDB](https://github.com/mindsdb/mindsdb) - The platform for customizing AI from enterprise data. + - [MindsDB](https://github.com/mindsdb/mindsdb) - Integrates with ClickHouse, making data from ClickHouse accessible to a diverse range of AI/ML models. ## Programming Language Ecosystems {#programming-language-ecosystems}