From 3e50ccdb452ca74d2477d2f9a0c090dca211ec16 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 24 Mar 2023 00:27:04 +0000 Subject: [PATCH 001/277] fixed bug with names mapping, better structure propagation --- src/Interpreters/Context.cpp | 69 +++++++++++++----------------------- 1 file changed, 24 insertions(+), 45 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4e882f3ab5b..c6006af5ee2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1394,60 +1394,39 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const } if (getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint() && hasInsertionTable()) { - const auto & structure_hint = DatabaseCatalog::instance().getTable(getInsertionTable(), shared_from_this())->getInMemoryMetadataPtr()->getColumns(); - bool use_columns_from_insert_query = true; + const auto & insert_structure = DatabaseCatalog::instance().getTable(getInsertionTable(), shared_from_this())->getInMemoryMetadataPtr()->getColumns(); + auto table_structure = table_function_ptr->getActualTableStructure(getQueryContext()); - /// use_structure_from_insertion_table_in_table_functions=2 means `auto` - if (select_query_hint && getSettingsRef().use_structure_from_insertion_table_in_table_functions == 2) + /// Insert table matches columns against SELECT expression by position, so we want to map + /// insert table columns to table function columns through names from SELECT expression. + + auto insert_column = insert_structure.begin(); + for (const auto & expression : select_query_hint->select()->as()->children) { - const auto * expression_list = select_query_hint->select()->as(); - std::unordered_set virtual_column_names = table_function_ptr->getVirtualsToCheckBeforeUsingStructureHint(); - Names columns_names; - bool have_asterisk = false; - /// First, check if we have only identifiers, asterisk and literals in select expression, - /// and if no, we cannot use the structure from insertion table. - for (const auto & expression : expression_list->children) + if (auto * identifier = expression->as()) { - if (auto * identifier = expression->as()) + if (table_structure.hasPhysical(identifier->name())) + table_structure.modify(identifier->name(), [&insert_column](ColumnDescription & column){ column.type = insert_column->type; }); + ++insert_column; + } + else if (expression->as()) + { + for (const auto & column : table_structure) { - columns_names.push_back(identifier->name()); - } - else if (expression->as()) - { - have_asterisk = true; - } - else if (!expression->as()) - { - use_columns_from_insert_query = false; - break; + table_structure.modify(column.name, [&insert_column](ColumnDescription & column){ column.type = insert_column->type; }); + ++insert_column; + if (insert_column == insert_structure.end()) + break; } } + else + ++insert_column; - /// Check that all identifiers are column names from insertion table and not virtual column names from storage. - for (const auto & column_name : columns_names) - { - if (!structure_hint.has(column_name) || virtual_column_names.contains(column_name)) - { - use_columns_from_insert_query = false; - break; - } - } - - /// If we don't have asterisk but only subset of columns, we should use - /// structure from insertion table only in case when table function - /// supports reading subset of columns from data. - if (use_columns_from_insert_query && !have_asterisk && !columns_names.empty()) - { - /// For input function we should check if input format supports reading subset of columns. - if (table_function_ptr->getName() == "input") - use_columns_from_insert_query = FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(getInsertFormat()); - else - use_columns_from_insert_query = table_function_ptr->supportsReadingSubsetOfColumns(); - } + if (insert_column == insert_structure.end()) + break; } - if (use_columns_from_insert_query) - table_function_ptr->setStructureHint(structure_hint); + table_function_ptr->setStructureHint(table_structure); } res = table_function_ptr->execute(table_expression, shared_from_this(), table_function_ptr->getName()); From 83ca006fcbfeb2390b4a3437659a0c9227d3fdbc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 24 Mar 2023 13:16:54 +0000 Subject: [PATCH 002/277] same fix for analyzer --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 55 ++++++++++++++--------- 1 file changed, 35 insertions(+), 20 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index f5f577a20ab..eb0aed558b5 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6097,25 +6097,9 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, if (!nested_table_function) expressions_visitor.visit(table_function_node_typed.getArgumentsNode()); - const auto & table_function_factory = TableFunctionFactory::instance(); - const auto & table_function_name = table_function_node_typed.getTableFunctionName(); - auto & scope_context = scope.context; - TableFunctionPtr table_function_ptr = table_function_factory.tryGet(table_function_name, scope_context); - if (!table_function_ptr) - { - auto hints = TableFunctionFactory::instance().getHints(table_function_name); - if (!hints.empty()) - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, - "Unknown table function {}. Maybe you meant: {}", - table_function_name, - DB::toString(hints)); - else - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, - "Unknown table function {}", - table_function_name); - } + TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_function_node_typed.toAST(), scope_context); if (!nested_table_function && scope_context->getSettingsRef().use_structure_from_insertion_table_in_table_functions && @@ -6125,9 +6109,40 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, const auto & insertion_table = scope_context->getInsertionTable(); if (!insertion_table.empty()) { - auto insertion_table_storage = DatabaseCatalog::instance().getTable(insertion_table, scope_context); - const auto & structure_hint = insertion_table_storage->getInMemoryMetadataPtr()->columns; - table_function_ptr->setStructureHint(structure_hint); + auto & expression_list = scope.scope_node->as().getProjection(); + const auto & insert_structure = DatabaseCatalog::instance().getTable(insertion_table, scope_context)->getInMemoryMetadataPtr()->getColumns(); + auto table_structure = table_function_ptr->getActualTableStructure(scope_context); + + /// Insert table matches columns against SELECT expression by position, so we want to map + /// insert table columns to table function columns through names from SELECT expression. + + auto insert_column = insert_structure.begin(); + for (const auto & expression : expression_list) + { + if (auto * identifier_node = expression->as()) + { + if (table_structure.hasPhysical(identifier_node->getIdentifier().getFullName())) + table_structure.modify(identifier_node->getIdentifier().getFullName(), [&insert_column](ColumnDescription & column){ column.type = insert_column->type; }); + ++insert_column; + } + else if (auto * matcher_node = expression->as(); matcher_node && matcher_node->getMatcherType() == MatcherNodeType::ASTERISK) + { + for (const auto & column : table_structure) + { + table_structure.modify(column.name, [&insert_column](ColumnDescription & column){ column.type = insert_column->type; }); + ++insert_column; + if (insert_column == insert_structure.end()) + break; + } + } + else + ++insert_column; + + if (insert_column == insert_structure.end()) + break; + } + + table_function_ptr->setStructureHint(table_structure); } } From e64500e72b3e8537587b6a3c6b980b54bc392ebb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 24 Mar 2023 13:34:11 +0000 Subject: [PATCH 003/277] Automatic style fix --- docker/test/performance-comparison/perf.py | 2 ++ docker/test/performance-comparison/report.py | 1 - tests/ci/clickhouse_helper.py | 1 - tests/ci/docker_images_check.py | 1 - tests/ci/get_previous_release_tag.py | 1 - tests/ci/report.py | 2 +- tests/integration/helpers/cluster.py | 3 +-- tests/integration/helpers/network.py | 2 -- .../pytest_xdist_logging_to_separate_files.py | 1 + .../test_detach_part_wrong_partition_id.py | 1 - .../test_cluster_copier/test_three_nodes.py | 1 - .../test_cluster_copier/test_two_nodes.py | 1 - tests/integration/test_composable_protocols/test.py | 1 - .../test_create_query_constraints/test.py | 2 -- .../common.py | 1 - tests/integration/test_disks_app_func/test.py | 1 - .../test_distributed_ddl_parallel/test.py | 1 + tests/integration/test_fetch_memory_usage/test.py | 1 - .../scripts/stress_test.py | 1 - tests/integration/test_jbod_balancer/test.py | 1 - .../test_keeper_and_access_storage/test.py | 1 + tests/integration/test_keeper_back_to_back/test.py | 2 +- tests/integration/test_keeper_persistent_log/test.py | 1 - .../test_keeper_zookeeper_converter/test.py | 1 - tests/integration/test_merge_tree_load_parts/test.py | 6 +++--- .../s3_endpoint/endpoint.py | 1 - .../test_merge_tree_settings_constraints/test.py | 1 - .../test_old_parts_finally_removed/test.py | 1 - tests/integration/test_partition/test.py | 4 +++- tests/integration/test_password_constraints/test.py | 1 - tests/integration/test_read_only_table/test.py | 1 - .../test_reload_auxiliary_zookeepers/test.py | 1 - .../s3_endpoint/endpoint.py | 1 + tests/integration/test_s3_with_proxy/test.py | 1 + .../integration/test_ssl_cert_authentication/test.py | 1 - tests/integration/test_storage_kafka/kafka_pb2.py | 1 - .../test_storage_kafka/message_with_repeated_pb2.py | 1 - tests/integration/test_storage_kafka/social_pb2.py | 1 - tests/integration/test_storage_kafka/test.py | 12 ++---------- tests/integration/test_storage_nats/nats_pb2.py | 1 - .../test_storage_postgresql_replica/test.py | 1 - .../test_storage_rabbitmq/rabbitmq_pb2.py | 1 - tests/integration/test_storage_rabbitmq/test.py | 3 --- tests/integration/test_storage_s3/test.py | 1 + .../test_storage_s3/test_invalid_env_credentials.py | 1 + tests/integration/test_system_merges/test.py | 1 - tests/integration/test_ttl_move/test.py | 2 +- tests/integration/test_zero_copy_fetch/test.py | 1 - utils/changelog-simple/format-changelog.py | 1 + utils/keeper-overload/keeper-overload.py | 2 +- 50 files changed, 23 insertions(+), 57 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 65bf49c2914..7a4e6386d0d 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -26,6 +26,7 @@ logging.basicConfig( total_start_seconds = time.perf_counter() stage_start_seconds = total_start_seconds + # Thread executor that does not hides exception that happens during function # execution, and rethrows it after join() class SafeThread(Thread): @@ -158,6 +159,7 @@ for e in subst_elems: available_parameters[name] = values + # Takes parallel lists of templates, substitutes them with all combos of # parameters. The set of parameters is determined based on the first list. # Note: keep the order of queries -- sometimes we have DROP IF EXISTS diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 782cf29863c..214f2d550b4 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -670,7 +670,6 @@ if args.report == "main": ) elif args.report == "all-queries": - print((header_template.format())) add_tested_commits() diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index d60a9e6afd1..64b64896f66 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -141,7 +141,6 @@ def prepare_tests_results_for_clickhouse( report_url: str, check_name: str, ) -> List[dict]: - pull_request_url = "https://github.com/ClickHouse/ClickHouse/commits/master" base_ref = "master" head_ref = "master" diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 192d216614e..f2b1105b3b0 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -96,7 +96,6 @@ def get_images_dict(repo_path: str, image_file_path: str) -> ImagesDict: def get_changed_docker_images( pr_info: PRInfo, images_dict: ImagesDict ) -> Set[DockerImage]: - if not images_dict: return set() diff --git a/tests/ci/get_previous_release_tag.py b/tests/ci/get_previous_release_tag.py index c6fe6cd5fb5..c2d279f7fec 100755 --- a/tests/ci/get_previous_release_tag.py +++ b/tests/ci/get_previous_release_tag.py @@ -51,7 +51,6 @@ def find_previous_release( for release in releases: if release.version < server_version: - # Check if the artifact exists on GitHub. # It can be not true for a short period of time # after creating a tag for a new release before uploading the packages. diff --git a/tests/ci/report.py b/tests/ci/report.py index 947fb33d905..ddee035d26f 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -473,7 +473,7 @@ def create_build_html_report( commit_url: str, ) -> str: rows = "" - for (build_result, build_log_url, artifact_urls) in zip( + for build_result, build_log_url, artifact_urls in zip( build_results, build_logs_urls, artifact_urls_list ): row = "" diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index dc5ada81995..a9a996e0a5f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -63,6 +63,7 @@ DEFAULT_ENV_NAME = ".env" SANITIZER_SIGN = "==================" + # to create docker-compose env file def _create_env_file(path, variables): logging.debug(f"Env {variables} stored in {path}") @@ -1454,7 +1455,6 @@ class ClickHouseCluster: config_root_name="clickhouse", extra_configs=[], ) -> "ClickHouseInstance": - """Add an instance to the cluster. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. @@ -3089,7 +3089,6 @@ class ClickHouseInstance: config_root_name="clickhouse", extra_configs=[], ): - self.name = name self.base_cmd = cluster.base_cmd self.docker_id = cluster.get_instance_docker_id(self.name) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index e408c9beec1..471aa2bdc2e 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -216,7 +216,6 @@ class _NetworkManager: container_exit_timeout=60, docker_api_version=os.environ.get("DOCKER_API_VERSION"), ): - self.container_expire_timeout = container_expire_timeout self.container_exit_timeout = container_exit_timeout @@ -232,7 +231,6 @@ class _NetworkManager: def _ensure_container(self): if self._container is None or self._container_expire_time <= time.time(): - for i in range(5): if self._container is not None: try: diff --git a/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py b/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py index d424ad58fa4..370aa23a014 100644 --- a/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py +++ b/tests/integration/helpers/pytest_xdist_logging_to_separate_files.py @@ -1,6 +1,7 @@ import logging import os.path + # Makes the parallel workers of pytest-xdist to log to separate files. # Without this function all workers will log to the same log file # and mix everything together making it much more difficult for troubleshooting. diff --git a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py index 02fccfae4e5..a6f7a8653da 100644 --- a/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py +++ b/tests/integration/test_backward_compatibility/test_detach_part_wrong_partition_id.py @@ -24,7 +24,6 @@ def start_cluster(): def test_detach_part_wrong_partition_id(start_cluster): - # Here we create table with partition by UUID. node_21_6.query( "create table tab (id UUID, value UInt32) engine = MergeTree PARTITION BY (id) order by tuple()" diff --git a/tests/integration/test_cluster_copier/test_three_nodes.py b/tests/integration/test_cluster_copier/test_three_nodes.py index 31d6c0448f4..e7d07757adb 100644 --- a/tests/integration/test_cluster_copier/test_three_nodes.py +++ b/tests/integration/test_cluster_copier/test_three_nodes.py @@ -19,7 +19,6 @@ cluster = ClickHouseCluster(__file__) def started_cluster(): global cluster try: - for name in ["first", "second", "third"]: cluster.add_instance( name, diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 10ab7d03b00..2b6fcf6cac2 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -19,7 +19,6 @@ cluster = ClickHouseCluster(__file__) def started_cluster(): global cluster try: - for name in ["first_of_two", "second_of_two"]: instance = cluster.add_instance( name, diff --git a/tests/integration/test_composable_protocols/test.py b/tests/integration/test_composable_protocols/test.py index bc87fea5296..df74cfffa54 100644 --- a/tests/integration/test_composable_protocols/test.py +++ b/tests/integration/test_composable_protocols/test.py @@ -63,7 +63,6 @@ def netcat(hostname, port, content): def test_connections(): - client = Client(server.ip_address, 9000, command=cluster.client_bin_path) assert client.query("SELECT 1") == "1\n" diff --git a/tests/integration/test_create_query_constraints/test.py b/tests/integration/test_create_query_constraints/test.py index 8df043fd24b..33c41b4f161 100644 --- a/tests/integration/test_create_query_constraints/test.py +++ b/tests/integration/test_create_query_constraints/test.py @@ -25,7 +25,6 @@ def start_cluster(): def test_create_query_const_constraints(): - instance.query("CREATE USER u_const SETTINGS max_threads = 1 CONST") instance.query("GRANT ALL ON *.* TO u_const") @@ -57,7 +56,6 @@ def test_create_query_const_constraints(): def test_create_query_minmax_constraints(): - instance.query("CREATE USER u_minmax SETTINGS max_threads = 4 MIN 2 MAX 6") instance.query("GRANT ALL ON *.* TO u_minmax") diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py index b38e81b0227..01addae2542 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py @@ -348,7 +348,6 @@ class RangedLayoutTester(BaseLayoutTester): self.layouts = LAYOUTS_RANGED def execute(self, layout_name, node): - if layout_name not in self.layout_to_dictionary: raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) diff --git a/tests/integration/test_disks_app_func/test.py b/tests/integration/test_disks_app_func/test.py index 027ef8feed0..2428c53854e 100644 --- a/tests/integration/test_disks_app_func/test.py +++ b/tests/integration/test_disks_app_func/test.py @@ -7,7 +7,6 @@ import pytest def started_cluster(): global cluster try: - cluster = ClickHouseCluster(__file__) cluster.add_instance( "disks_app_test", main_configs=["config.xml"], with_minio=True diff --git a/tests/integration/test_distributed_ddl_parallel/test.py b/tests/integration/test_distributed_ddl_parallel/test.py index 6ebfe472e09..eb98dd3e230 100644 --- a/tests/integration/test_distributed_ddl_parallel/test.py +++ b/tests/integration/test_distributed_ddl_parallel/test.py @@ -10,6 +10,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) + # By default the exceptions that was throwed in threads will be ignored # (they will not mark the test as failed, only printed to stderr). # diff --git a/tests/integration/test_fetch_memory_usage/test.py b/tests/integration/test_fetch_memory_usage/test.py index a4371140150..7591cc0e8a9 100644 --- a/tests/integration/test_fetch_memory_usage/test.py +++ b/tests/integration/test_fetch_memory_usage/test.py @@ -18,7 +18,6 @@ def started_cluster(): def test_huge_column(started_cluster): - if ( node.is_built_with_thread_sanitizer() or node.is_built_with_memory_sanitizer() diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py index b8bafb3d0c1..fe69d72c1c7 100644 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py @@ -13,7 +13,6 @@ number_of_iterations = 100 def perform_request(): - buffer = BytesIO() crl = pycurl.Curl() crl.setopt(pycurl.INTERFACE, client_ip) diff --git a/tests/integration/test_jbod_balancer/test.py b/tests/integration/test_jbod_balancer/test.py index e746698611a..df34a075d5a 100644 --- a/tests/integration/test_jbod_balancer/test.py +++ b/tests/integration/test_jbod_balancer/test.py @@ -45,7 +45,6 @@ def start_cluster(): def check_balance(node, table): - partitions = node.query( """ WITH diff --git a/tests/integration/test_keeper_and_access_storage/test.py b/tests/integration/test_keeper_and_access_storage/test.py index 6ec307f7082..0314825b6b7 100644 --- a/tests/integration/test_keeper_and_access_storage/test.py +++ b/tests/integration/test_keeper_and_access_storage/test.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( "node1", main_configs=["configs/keeper.xml"], stay_alive=True ) + # test that server is able to start @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_keeper_back_to_back/test.py b/tests/integration/test_keeper_back_to_back/test.py index 73fface02b4..b737ac284d2 100644 --- a/tests/integration/test_keeper_back_to_back/test.py +++ b/tests/integration/test_keeper_back_to_back/test.py @@ -546,7 +546,6 @@ def test_random_requests(started_cluster): def test_end_of_session(started_cluster): - fake_zk1 = None fake_zk2 = None genuine_zk1 = None @@ -685,6 +684,7 @@ def test_concurrent_watches(started_cluster): nonlocal watches_created nonlocal all_paths_created fake_zk.ensure_path(global_path + "/" + str(i)) + # new function each time def dumb_watch(event): nonlocal dumb_watch_triggered_counter diff --git a/tests/integration/test_keeper_persistent_log/test.py b/tests/integration/test_keeper_persistent_log/test.py index 70cc14fe26d..4164ffb33d3 100644 --- a/tests/integration/test_keeper_persistent_log/test.py +++ b/tests/integration/test_keeper_persistent_log/test.py @@ -163,7 +163,6 @@ def test_state_duplicate_restart(started_cluster): # http://zookeeper-user.578899.n2.nabble.com/Why-are-ephemeral-nodes-written-to-disk-tp7583403p7583418.html def test_ephemeral_after_restart(started_cluster): - try: node_zk = None node_zk2 = None diff --git a/tests/integration/test_keeper_zookeeper_converter/test.py b/tests/integration/test_keeper_zookeeper_converter/test.py index 063421bf922..de5a9416119 100644 --- a/tests/integration/test_keeper_zookeeper_converter/test.py +++ b/tests/integration/test_keeper_zookeeper_converter/test.py @@ -114,7 +114,6 @@ def start_clickhouse(): def copy_zookeeper_data(make_zk_snapshots): - if make_zk_snapshots: # force zookeeper to create snapshot generate_zk_snapshot() else: diff --git a/tests/integration/test_merge_tree_load_parts/test.py b/tests/integration/test_merge_tree_load_parts/test.py index 777b6f14fc6..dfbe00c8e28 100644 --- a/tests/integration/test_merge_tree_load_parts/test.py +++ b/tests/integration/test_merge_tree_load_parts/test.py @@ -148,17 +148,17 @@ def test_merge_tree_load_parts_corrupted(started_cluster): node1.query("SYSTEM WAIT LOADING PARTS mt_load_parts_2") def check_parts_loading(node, partition, loaded, failed, skipped): - for (min_block, max_block) in loaded: + for min_block, max_block in loaded: part_name = f"{partition}_{min_block}_{max_block}" assert node.contains_in_log(f"Loading Active part {part_name}") assert node.contains_in_log(f"Finished loading Active part {part_name}") - for (min_block, max_block) in failed: + for min_block, max_block in failed: part_name = f"{partition}_{min_block}_{max_block}" assert node.contains_in_log(f"Loading Active part {part_name}") assert not node.contains_in_log(f"Finished loading Active part {part_name}") - for (min_block, max_block) in skipped: + for min_block, max_block in skipped: part_name = f"{partition}_{min_block}_{max_block}" assert not node.contains_in_log(f"Loading Active part {part_name}") assert not node.contains_in_log(f"Finished loading Active part {part_name}") diff --git a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py index b6567dfebc5..4613fdb850b 100644 --- a/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py +++ b/tests/integration/test_merge_tree_s3_failover/s3_endpoint/endpoint.py @@ -42,7 +42,6 @@ def delete(_bucket): @route("/<_bucket>/<_path:path>", ["GET", "POST", "PUT", "DELETE"]) def server(_bucket, _path): - # It's delete query for failed part if _path.endswith("delete"): response.set_header("Location", "http://minio1:9001/" + _bucket + "/" + _path) diff --git a/tests/integration/test_merge_tree_settings_constraints/test.py b/tests/integration/test_merge_tree_settings_constraints/test.py index 0bb0179108d..be6e2a31873 100644 --- a/tests/integration/test_merge_tree_settings_constraints/test.py +++ b/tests/integration/test_merge_tree_settings_constraints/test.py @@ -20,7 +20,6 @@ def start_cluster(): def test_merge_tree_settings_constraints(): - assert "Setting storage_policy should not be changed" in instance.query_and_get_error( f"CREATE TABLE wrong_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS storage_policy = 'secret_policy'" ) diff --git a/tests/integration/test_old_parts_finally_removed/test.py b/tests/integration/test_old_parts_finally_removed/test.py index 108b72c5ccd..5347d433419 100644 --- a/tests/integration/test_old_parts_finally_removed/test.py +++ b/tests/integration/test_old_parts_finally_removed/test.py @@ -63,7 +63,6 @@ def test_part_finally_removed(started_cluster): ) for i in range(60): - if ( node1.query( "SELECT count() from system.parts WHERE table = 'drop_outdated_part'" diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index ae4393fc6f6..a34141c6189 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -528,7 +528,9 @@ def test_make_clone_in_detached(started_cluster): ["cp", "-r", path + "all_0_0_0", path + "detached/broken_all_0_0_0"] ) assert_eq_with_retry(instance, "select * from clone_in_detached", "\n") - assert ["broken_all_0_0_0",] == sorted( + assert [ + "broken_all_0_0_0", + ] == sorted( instance.exec_in_container(["ls", path + "detached/"]).strip().split("\n") ) diff --git a/tests/integration/test_password_constraints/test.py b/tests/integration/test_password_constraints/test.py index e3628861b28..9cdff51caa1 100644 --- a/tests/integration/test_password_constraints/test.py +++ b/tests/integration/test_password_constraints/test.py @@ -17,7 +17,6 @@ def start_cluster(): def test_complexity_rules(start_cluster): - error_message = "DB::Exception: Invalid password. The password should: be at least 12 characters long, contain at least 1 numeric character, contain at least 1 lowercase character, contain at least 1 uppercase character, contain at least 1 special character" assert error_message in node.query_and_get_error( "CREATE USER u_1 IDENTIFIED WITH plaintext_password BY ''" diff --git a/tests/integration/test_read_only_table/test.py b/tests/integration/test_read_only_table/test.py index 914c6a99508..df084f9dbbd 100644 --- a/tests/integration/test_read_only_table/test.py +++ b/tests/integration/test_read_only_table/test.py @@ -49,7 +49,6 @@ def start_cluster(): def test_restart_zookeeper(start_cluster): - for table_id in range(NUM_TABLES): node1.query( f"INSERT INTO test_table_{table_id} VALUES (1), (2), (3), (4), (5);" diff --git a/tests/integration/test_reload_auxiliary_zookeepers/test.py b/tests/integration/test_reload_auxiliary_zookeepers/test.py index bb1455333fc..476c5dee99e 100644 --- a/tests/integration/test_reload_auxiliary_zookeepers/test.py +++ b/tests/integration/test_reload_auxiliary_zookeepers/test.py @@ -20,7 +20,6 @@ def start_cluster(): def test_reload_auxiliary_zookeepers(start_cluster): - node.query( "CREATE TABLE simple (date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', 'node') ORDER BY tuple() PARTITION BY date;" ) diff --git a/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py b/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py index d6a732cc681..1d33ca02f86 100644 --- a/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py +++ b/tests/integration/test_s3_aws_sdk_has_slightly_unreliable_behaviour/s3_endpoint/endpoint.py @@ -1,6 +1,7 @@ #!/usr/bin/env python3 from bottle import request, route, run, response + # Handle for MultipleObjectsDelete. @route("/<_bucket>", ["POST"]) def delete(_bucket): diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 1102d190a87..1af040c3c30 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -5,6 +5,7 @@ import time import pytest from helpers.cluster import ClickHouseCluster + # Runs simple proxy resolver in python env container. def run_resolver(cluster): container_id = cluster.get_container_id("resolver") diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 7c62ca0d8b6..b3570b6e281 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -87,7 +87,6 @@ config = """ def execute_query_native(node, query, user, cert_name): - config_path = f"{SCRIPT_DIR}/configs/client.xml" formatted = config.format( diff --git a/tests/integration/test_storage_kafka/kafka_pb2.py b/tests/integration/test_storage_kafka/kafka_pb2.py index 7de1363bbf1..3e47af6c1e0 100644 --- a/tests/integration/test_storage_kafka/kafka_pb2.py +++ b/tests/integration/test_storage_kafka/kafka_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.kafka_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _KEYVALUEPAIR._serialized_start = 46 _KEYVALUEPAIR._serialized_end = 88 diff --git a/tests/integration/test_storage_kafka/message_with_repeated_pb2.py b/tests/integration/test_storage_kafka/message_with_repeated_pb2.py index 4d1a23c0b43..3715a9bea04 100644 --- a/tests/integration/test_storage_kafka/message_with_repeated_pb2.py +++ b/tests/integration/test_storage_kafka/message_with_repeated_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.message_with_repeated_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None DESCRIPTOR._serialized_options = b"H\001" _MESSAGE._serialized_start = 62 diff --git a/tests/integration/test_storage_kafka/social_pb2.py b/tests/integration/test_storage_kafka/social_pb2.py index 830ade81d33..f91a7bd0539 100644 --- a/tests/integration/test_storage_kafka/social_pb2.py +++ b/tests/integration/test_storage_kafka/social_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.social_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _USER._serialized_start = 47 _USER._serialized_end = 90 diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 51952ac1eb7..3a4fa6c6bfe 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -121,7 +121,7 @@ def kafka_create_topic( def kafka_delete_topic(admin_client, topic, max_retries=50): result = admin_client.delete_topics([topic]) - for (topic, e) in result.topic_error_codes: + for topic, e in result.topic_error_codes: if e == 0: logging.debug(f"Topic {topic} deleted") else: @@ -917,9 +917,7 @@ def describe_consumer_group(kafka_cluster, name): member_info["client_id"] = client_id member_info["client_host"] = client_host member_topics_assignment = [] - for (topic, partitions) in MemberAssignment.decode( - member_assignment - ).assignment: + for topic, partitions in MemberAssignment.decode(member_assignment).assignment: member_topics_assignment.append({"topic": topic, "partitions": partitions}) member_info["assignment"] = member_topics_assignment res.append(member_info) @@ -1537,7 +1535,6 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): def test_kafka_materialized_view(kafka_cluster): - instance.query( """ DROP TABLE IF EXISTS test.view; @@ -2315,7 +2312,6 @@ def test_kafka_virtual_columns2(kafka_cluster): def test_kafka_produce_key_timestamp(kafka_cluster): - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -2444,7 +2440,6 @@ def test_kafka_insert_avro(kafka_cluster): def test_kafka_produce_consume_avro(kafka_cluster): - admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -4031,7 +4026,6 @@ def test_kafka_predefined_configuration(kafka_cluster): # https://github.com/ClickHouse/ClickHouse/issues/26643 def test_issue26643(kafka_cluster): - # for backporting: # admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") admin_client = KafkaAdminClient( @@ -4313,7 +4307,6 @@ def test_row_based_formats(kafka_cluster): "RowBinaryWithNamesAndTypes", "MsgPack", ]: - print(format_name) kafka_create_topic(admin_client, format_name) @@ -4438,7 +4431,6 @@ def test_block_based_formats_2(kafka_cluster): "ORC", "JSONCompactColumns", ]: - kafka_create_topic(admin_client, format_name) instance.query( diff --git a/tests/integration/test_storage_nats/nats_pb2.py b/tests/integration/test_storage_nats/nats_pb2.py index 4330ff57950..e9e5cb72363 100644 --- a/tests/integration/test_storage_nats/nats_pb2.py +++ b/tests/integration/test_storage_nats/nats_pb2.py @@ -31,7 +31,6 @@ ProtoKeyValue = _reflection.GeneratedProtocolMessageType( _sym_db.RegisterMessage(ProtoKeyValue) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _PROTOKEYVALUE._serialized_start = 45 _PROTOKEYVALUE._serialized_end = 88 diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 5df8b9029e6..8666d7ae58c 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -706,7 +706,6 @@ def test_abrupt_connection_loss_while_heavy_replication(started_cluster): def test_abrupt_server_restart_while_heavy_replication(started_cluster): - # FIXME (kssenii) temporary disabled if instance.is_built_with_sanitizer(): pytest.skip("Temporary disabled (FIXME)") diff --git a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py index e017b4e66c2..a5845652eef 100644 --- a/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py +++ b/tests/integration/test_storage_rabbitmq/rabbitmq_pb2.py @@ -21,7 +21,6 @@ _builder.BuildTopDescriptorsAndMessages( DESCRIPTOR, "clickhouse_path.format_schemas.rabbitmq_pb2", globals() ) if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None _KEYVALUEPROTO._serialized_start = 49 _KEYVALUEPROTO._serialized_end = 92 diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 2e54f21787a..53b6c4109ef 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2864,7 +2864,6 @@ def test_rabbitmq_predefined_configuration(rabbitmq_cluster): def test_rabbitmq_msgpack(rabbitmq_cluster): - instance.query( """ drop table if exists rabbit_in; @@ -2908,7 +2907,6 @@ def test_rabbitmq_msgpack(rabbitmq_cluster): def test_rabbitmq_address(rabbitmq_cluster): - instance2.query( """ drop table if exists rabbit_in; @@ -3243,7 +3241,6 @@ def test_block_based_formats_2(rabbitmq_cluster): "ORC", "JSONCompactColumns", ]: - print(format_name) instance.query( diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 8b20727a7b5..4d493d9526b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -18,6 +18,7 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_storage_s3/test_invalid_env_credentials.py b/tests/integration/test_storage_s3/test_invalid_env_credentials.py index 2f5d9349904..aa6479a2ed3 100644 --- a/tests/integration/test_storage_s3/test_invalid_env_credentials.py +++ b/tests/integration/test_storage_s3/test_invalid_env_credentials.py @@ -11,6 +11,7 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index 0a469bd7bbd..ff303afe19e 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -171,7 +171,6 @@ def test_mutation_simple(started_cluster, replicated): starting_block = 0 if replicated else 1 try: - for node in nodes: node.query( f"create table {name} (a Int64) engine={engine} order by tuple()" diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 99978cbf6dc..89824293320 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1863,7 +1863,7 @@ def test_ttl_move_if_exists(started_cluster, name, dest_type): ) ) - for (node, policy) in zip( + for node, policy in zip( [node1, node2], ["only_jbod_1", "small_jbod_with_external"] ): node.query( diff --git a/tests/integration/test_zero_copy_fetch/test.py b/tests/integration/test_zero_copy_fetch/test.py index b71752528d3..9b9aa5e0da7 100644 --- a/tests/integration/test_zero_copy_fetch/test.py +++ b/tests/integration/test_zero_copy_fetch/test.py @@ -16,7 +16,6 @@ cluster = ClickHouseCluster(__file__) @pytest.fixture(scope="module") def started_cluster(): try: - cluster.add_instance( "node1", main_configs=["configs/storage_conf.xml"], diff --git a/utils/changelog-simple/format-changelog.py b/utils/changelog-simple/format-changelog.py index d5e1518270e..01f2694dd0f 100755 --- a/utils/changelog-simple/format-changelog.py +++ b/utils/changelog-simple/format-changelog.py @@ -20,6 +20,7 @@ parser.add_argument( ) args = parser.parse_args() + # This function mirrors the PR description checks in ClickhousePullRequestTrigger. # Returns False if the PR should not be mentioned changelog. def parse_one_pull_request(item): diff --git a/utils/keeper-overload/keeper-overload.py b/utils/keeper-overload/keeper-overload.py index bdb4563c713..0a059b10588 100755 --- a/utils/keeper-overload/keeper-overload.py +++ b/utils/keeper-overload/keeper-overload.py @@ -166,7 +166,7 @@ def main(args): keeper_bench_path = args.keeper_bench_path keepers = [] - for (port, server_id) in zip(PORTS, SERVER_IDS): + for port, server_id in zip(PORTS, SERVER_IDS): keepers.append( Keeper( keeper_binary_path, server_id, port, workdir, args.with_thread_fuzzer From bc107c70fa863237685c5c353937e9d4af9dd674 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 1 Mar 2023 18:50:51 +0100 Subject: [PATCH 004/277] merge and mutation make thread group for setting memory trackers right --- src/Common/MemoryTracker.cpp | 9 +++ src/Common/MemoryTracker.h | 2 + src/Common/ThreadStatus.h | 8 +-- src/Storages/MergeTree/MergeList.cpp | 72 +++++++------------ src/Storages/MergeTree/MergeList.h | 27 +++---- .../MergeTree/MergePlainMergeTreeTask.cpp | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MutatePlainMergeTreeTask.cpp | 4 +- .../ReplicatedMergeMutateTaskBase.cpp | 4 +- 9 files changed, 56 insertions(+), 76 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index e2129e1013e..4cfb7f764e5 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -28,6 +28,7 @@ #include #include #include +#include namespace @@ -118,6 +119,14 @@ MemoryTracker::~MemoryTracker() } } +String MemoryTracker::getDebugLog() const +{ + return fmt::format("MemoryTracker(addr {} level {} peak {} ammount {})", + size_t(this), + magic_enum::enum_name(level), + ReadableSize(getPeak()), + ReadableSize(get())); +} void MemoryTracker::logPeakMemoryUsage() { diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 66b56730b75..66037345eb0 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -215,6 +215,8 @@ public: /// Prints info about peak memory consumption into log. void logPeakMemoryUsage(); + + String getDebugLog() const; }; extern MemoryTracker total_memory_tracker; diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 77c924f9650..4f7e9ca6830 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -40,7 +40,7 @@ class TaskStatsInfoGetter; class InternalTextLogsQueue; struct ViewRuntimeData; class QueryViewsLog; -class MemoryTrackerThreadSwitcher; +class ThreadGroupSwitcher; using InternalTextLogsQueuePtr = std::shared_ptr; using InternalTextLogsQueueWeakPtr = std::weak_ptr; @@ -176,12 +176,6 @@ private: bool performance_counters_finalized = false; String query_id_from_query_context; - /// Requires access to query_id. - friend class MemoryTrackerThreadSwitcher; - void setQueryId(const String & query_id_) - { - query_id_from_query_context = query_id_; - } struct TimePoint { diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index fa1887a02e6..4705733bb80 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -11,38 +11,24 @@ namespace DB { -MemoryTrackerThreadSwitcher::MemoryTrackerThreadSwitcher(MergeListEntry & merge_list_entry_) +ThreadGroupSwitcher::ThreadGroupSwitcher(MergeListEntry & merge_list_entry_) : merge_list_entry(merge_list_entry_) { - // Each merge is executed into separate background processing pool thread - background_thread_memory_tracker = CurrentThread::getMemoryTracker(); - background_thread_memory_tracker_prev_parent = background_thread_memory_tracker->getParent(); - background_thread_memory_tracker->setParent(&merge_list_entry->memory_tracker); + prev_thread_group = CurrentThread::getGroup(); + if (!prev_thread_group) + return; - prev_untracked_memory_limit = current_thread->untracked_memory_limit; - current_thread->untracked_memory_limit = merge_list_entry->max_untracked_memory; - - /// Avoid accounting memory from another mutation/merge - /// (NOTE: consider moving such code to ThreadFromGlobalPool and related places) - prev_untracked_memory = current_thread->untracked_memory; - current_thread->untracked_memory = merge_list_entry->untracked_memory; - - prev_query_id = std::string(current_thread->getQueryId()); - current_thread->setQueryId(merge_list_entry->query_id); + CurrentThread::detachGroupIfNotDetached(); + CurrentThread::attachToGroup(merge_list_entry_->thread_group); } - -MemoryTrackerThreadSwitcher::~MemoryTrackerThreadSwitcher() +ThreadGroupSwitcher::~ThreadGroupSwitcher() { - // Unplug memory_tracker from current background processing pool thread - background_thread_memory_tracker->setParent(background_thread_memory_tracker_prev_parent); + if (!prev_thread_group) + return; - current_thread->untracked_memory_limit = prev_untracked_memory_limit; - - merge_list_entry->untracked_memory = current_thread->untracked_memory; - current_thread->untracked_memory = prev_untracked_memory; - - current_thread->setQueryId(prev_query_id); + CurrentThread::detachGroup(); + CurrentThread::attachTo(prev_thread_group); } MergeListElement::MergeListElement( @@ -55,7 +41,6 @@ MergeListElement::MergeListElement( , result_part_path{future_part->path} , result_part_info{future_part->part_info} , num_parts{future_part->parts.size()} - , max_untracked_memory(settings.max_untracked_memory) , query_id(table_id.getShortName() + "::" + result_part_name) , thread_id{getThreadId()} , merge_type{future_part->merge_type} @@ -78,6 +63,12 @@ MergeListElement::MergeListElement( is_mutation = (result_part_info.getDataVersion() != source_data_version); } + thread_group = std::make_shared(); + + thread_group->master_thread_id = CurrentThread::get().thread_id; + + auto & memory_tracker = thread_group->memory_tracker; + memory_tracker.setDescription(description.c_str()); /// MemoryTracker settings should be set here, because /// later (see MemoryTrackerThreadSwitcher) @@ -97,15 +88,16 @@ MergeListElement::MergeListElement( /// /// NOTE: Remember, that Thread level MemoryTracker does not have any settings, /// so it's parent is required. - MemoryTracker * query_memory_tracker = CurrentThread::getMemoryTracker(); - MemoryTracker * parent_query_memory_tracker; - if (query_memory_tracker->level == VariableContext::Thread && - (parent_query_memory_tracker = query_memory_tracker->getParent()) && - parent_query_memory_tracker != &total_memory_tracker) - { - memory_tracker.setOrRaiseHardLimit(parent_query_memory_tracker->getHardLimit()); - } + MemoryTracker * cur_memory_tracker = CurrentThread::getMemoryTracker(); + if (cur_memory_tracker->level == VariableContext::Thread) + { + MemoryTracker * query_memory_tracker = cur_memory_tracker->getParent(); + if (query_memory_tracker != &total_memory_tracker) + { + memory_tracker.setOrRaiseHardLimit(query_memory_tracker->getHardLimit()); + } + } } MergeInfo MergeListElement::getInfo() const @@ -128,7 +120,7 @@ MergeInfo MergeListElement::getInfo() const res.rows_read = rows_read.load(std::memory_order_relaxed); res.rows_written = rows_written.load(std::memory_order_relaxed); res.columns_written = columns_written.load(std::memory_order_relaxed); - res.memory_usage = memory_tracker.get(); + res.memory_usage = getMemoryTracker().get(); res.thread_id = thread_id; res.merge_type = toString(merge_type); res.merge_algorithm = toString(merge_algorithm.load(std::memory_order_relaxed)); @@ -142,14 +134,4 @@ MergeInfo MergeListElement::getInfo() const return res; } -MergeListElement::~MergeListElement() -{ - if (untracked_memory != 0) - { - CurrentThread::getMemoryTracker()->adjustWithUntrackedMemory(untracked_memory); - untracked_memory = 0; - } -} - - } diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 17a56272a57..b557f745f0d 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -63,23 +64,19 @@ struct Settings; /** * Since merge is executed with multiple threads, this class - * switches the parent MemoryTracker to account all the memory used. + * switches the parent MemoryTracker as part of the thread group to account all the memory used. */ -class MemoryTrackerThreadSwitcher : boost::noncopyable +class ThreadGroupSwitcher : boost::noncopyable { public: - explicit MemoryTrackerThreadSwitcher(MergeListEntry & merge_list_entry_); - ~MemoryTrackerThreadSwitcher(); + explicit ThreadGroupSwitcher(MergeListEntry & merge_list_entry_); + ~ThreadGroupSwitcher(); private: MergeListEntry & merge_list_entry; - MemoryTracker * background_thread_memory_tracker; - MemoryTracker * background_thread_memory_tracker_prev_parent = nullptr; - Int64 prev_untracked_memory_limit; - Int64 prev_untracked_memory; - String prev_query_id; + ThreadGroupStatusPtr prev_thread_group; }; -using MemoryTrackerThreadSwitcherPtr = std::unique_ptr; +using ThreadGroupSwitcherPtr = std::unique_ptr; struct MergeListElement : boost::noncopyable { @@ -113,10 +110,6 @@ struct MergeListElement : boost::noncopyable /// Updated only for Vertical algorithm std::atomic columns_written{}; - /// Used to adjust ThreadStatus::untracked_memory_limit - UInt64 max_untracked_memory; - /// Used to avoid losing any allocation context - UInt64 untracked_memory = 0; /// Used for identifying mutations/merges in trace_log std::string query_id; @@ -128,7 +121,7 @@ struct MergeListElement : boost::noncopyable /// Description used for logging /// Needs to outlive memory_tracker since it's used in its destructor const String description{"Mutate/Merge"}; - MemoryTracker memory_tracker{VariableContext::Process}; + ThreadGroupStatusPtr thread_group; MergeListElement( const StorageID & table_id_, @@ -137,9 +130,9 @@ struct MergeListElement : boost::noncopyable MergeInfo getInfo() const; - MergeListElement * ptr() { return this; } + const MemoryTracker & getMemoryTracker() const { return thread_group->memory_tracker; } - ~MergeListElement(); + MergeListElement * ptr() { return this; } MergeListElement & ref() { return *this; } }; diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 1ccdefd2b6a..c2b3f9dfc8d 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -32,9 +32,9 @@ bool MergePlainMergeTreeTask::executeStep() ProfileEventsScope profile_events_scope(&profile_counters); /// Make out memory tracker a parent of current thread memory tracker - MemoryTrackerThreadSwitcherPtr switcher; + ThreadGroupSwitcherPtr switcher; if (merge_list_entry) - switcher = std::make_unique(*merge_list_entry); + switcher = std::make_unique(*merge_list_entry); switch (state) { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 84c301e5986..2a80dc7fb5e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7415,7 +7415,7 @@ try part_log_elem.rows = (*merge_entry)->rows_written; part_log_elem.bytes_uncompressed = (*merge_entry)->bytes_written_uncompressed; - part_log_elem.peak_memory_usage = (*merge_entry)->memory_tracker.getPeak(); + part_log_elem.peak_memory_usage = (*merge_entry)->getMemoryTracker().getPeak(); } if (profile_counters) diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 9bd0f148d6c..a05a12eabe4 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -68,9 +68,9 @@ bool MutatePlainMergeTreeTask::executeStep() ProfileEventsScope profile_events_scope(&profile_counters); /// Make out memory tracker a parent of current thread memory tracker - MemoryTrackerThreadSwitcherPtr switcher; + ThreadGroupSwitcherPtr switcher; if (merge_list_entry) - switcher = std::make_unique(*merge_list_entry); + switcher = std::make_unique(*merge_list_entry); switch (state) { diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 9ce7eb42666..0af96cec323 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -128,9 +128,9 @@ bool ReplicatedMergeMutateTaskBase::executeStep() bool ReplicatedMergeMutateTaskBase::executeImpl() { - MemoryTrackerThreadSwitcherPtr switcher; + ThreadGroupSwitcherPtr switcher; if (merge_mutate_entry) - switcher = std::make_unique(*merge_mutate_entry); + switcher = std::make_unique(*merge_mutate_entry); auto remove_processed_entry = [&] () -> bool { From da4f2bd9232e5fa19a9d6b5e8119b5a9627af301 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 3 Mar 2023 15:21:53 +0100 Subject: [PATCH 005/277] do not attach empty thread group --- src/Common/MemoryTracker.cpp | 2 +- src/Storages/MergeTree/MergeList.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 4cfb7f764e5..16c0d1e9eb1 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -121,7 +121,7 @@ MemoryTracker::~MemoryTracker() String MemoryTracker::getDebugLog() const { - return fmt::format("MemoryTracker(addr {} level {} peak {} ammount {})", + return fmt::format("MemoryTracker(addr {} level {} peak {} amount {})", size_t(this), magic_enum::enum_name(level), ReadableSize(getPeak()), diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 4705733bb80..a9c55495b4b 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -28,7 +28,7 @@ ThreadGroupSwitcher::~ThreadGroupSwitcher() return; CurrentThread::detachGroup(); - CurrentThread::attachTo(prev_thread_group); + CurrentThread::attachToGroup(prev_thread_group); } MergeListElement::MergeListElement( From 6a6d45e6e76669677378c94dc3997a29dccec912 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 3 Mar 2023 17:21:45 +0100 Subject: [PATCH 006/277] set up performance_counters for thread group --- src/Storages/MergeTree/MergeList.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index a9c55495b4b..65725c1a632 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -65,6 +65,11 @@ MergeListElement::MergeListElement( thread_group = std::make_shared(); + auto p_counters = CurrentThread::get().current_performance_counters; + while (p_counters && p_counters->level != VariableContext::Process) + p_counters = p_counters->getParent(); + thread_group->performance_counters.setParent(p_counters); + thread_group->master_thread_id = CurrentThread::get().thread_id; auto & memory_tracker = thread_group->memory_tracker; From 0fcf7c0363ba5279878fe1c9d3f536a2b28cfe72 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 3 Mar 2023 23:09:36 +0100 Subject: [PATCH 007/277] std::optional instead shared_ptr --- src/Storages/MergeTree/MergeList.cpp | 31 +++++++++++++++++-- src/Storages/MergeTree/MergeList.h | 13 +++++--- .../MergeTree/MergePlainMergeTreeTask.cpp | 4 +-- .../MergeTree/MutatePlainMergeTreeTask.cpp | 4 +-- .../ReplicatedMergeMutateTaskBase.cpp | 4 +-- 5 files changed, 43 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 65725c1a632..f53c6ad81ee 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -11,14 +11,14 @@ namespace DB { -ThreadGroupSwitcher::ThreadGroupSwitcher(MergeListEntry & merge_list_entry_) +ThreadGroupSwitcher::ThreadGroupSwitcher(MergeListEntry * merge_list_entry_) : merge_list_entry(merge_list_entry_) { prev_thread_group = CurrentThread::getGroup(); if (!prev_thread_group) return; - CurrentThread::detachGroupIfNotDetached(); + CurrentThread::detachGroup(); CurrentThread::attachToGroup(merge_list_entry_->thread_group); } @@ -27,10 +27,37 @@ ThreadGroupSwitcher::~ThreadGroupSwitcher() if (!prev_thread_group) return; + if (!merge_list_entry) + return; + CurrentThread::detachGroup(); CurrentThread::attachToGroup(prev_thread_group); } +ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupSwitcher && other) +{ + this->swap(other); +} + +ThreadGroupSwitcher& ThreadGroupSwitcher::operator=(ThreadGroupSwitcher && other) +{ + if (this != &other) + { + auto tmp = ThreadGroupSwitcher(); + tmp.swap(other); + this->swap(tmp); + } + return *this; +} + +void ThreadGroupSwitcher::swap(ThreadGroupSwitcher & other) +{ + std::swap(merge_list_entry, other.merge_list_entry); + std::swap(prev_thread_group, other.prev_thread_group); + std::swap(prev_query_id, other.prev_query_id); +} + + MergeListElement::MergeListElement( const StorageID & table_id_, FutureMergedMutatedPartPtr future_part, diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index b557f745f0d..200a574698c 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -66,18 +66,21 @@ struct Settings; * Since merge is executed with multiple threads, this class * switches the parent MemoryTracker as part of the thread group to account all the memory used. */ -class ThreadGroupSwitcher : boost::noncopyable +class ThreadGroupSwitcher : private boost::noncopyable { public: - explicit ThreadGroupSwitcher(MergeListEntry & merge_list_entry_); + explicit ThreadGroupSwitcher(MergeListEntry * merge_list_entry_); + ThreadGroupSwitcher(ThreadGroupSwitcher && other); + ThreadGroupSwitcher& operator=(ThreadGroupSwitcher && other); ~ThreadGroupSwitcher(); private: - MergeListEntry & merge_list_entry; + ThreadGroupSwitcher() = default; + void swap(ThreadGroupSwitcher & other); + + MergeListEntry * merge_list_entry = nullptr; ThreadGroupStatusPtr prev_thread_group; }; -using ThreadGroupSwitcherPtr = std::unique_ptr; - struct MergeListElement : boost::noncopyable { const StorageID table_id; diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index c2b3f9dfc8d..052c6467b37 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -32,9 +32,9 @@ bool MergePlainMergeTreeTask::executeStep() ProfileEventsScope profile_events_scope(&profile_counters); /// Make out memory tracker a parent of current thread memory tracker - ThreadGroupSwitcherPtr switcher; + std::optional switcher; if (merge_list_entry) - switcher = std::make_unique(*merge_list_entry); + switcher = ThreadGroupSwitcher(merge_list_entry.get()); switch (state) { diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index a05a12eabe4..673bfaa0d47 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -68,9 +68,9 @@ bool MutatePlainMergeTreeTask::executeStep() ProfileEventsScope profile_events_scope(&profile_counters); /// Make out memory tracker a parent of current thread memory tracker - ThreadGroupSwitcherPtr switcher; + std::optional switcher; if (merge_list_entry) - switcher = std::make_unique(*merge_list_entry); + switcher = ThreadGroupSwitcher(merge_list_entry.get()); switch (state) { diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 0af96cec323..fbc6522170e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -128,9 +128,9 @@ bool ReplicatedMergeMutateTaskBase::executeStep() bool ReplicatedMergeMutateTaskBase::executeImpl() { - ThreadGroupSwitcherPtr switcher; + std::optional switcher; if (merge_mutate_entry) - switcher = std::make_unique(*merge_mutate_entry); + switcher = ThreadGroupSwitcher(merge_mutate_entry.get()); auto remove_processed_entry = [&] () -> bool { From da3e744405bc9707ac8b453f0637303a191d4847 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 6 Mar 2023 13:53:19 +0100 Subject: [PATCH 008/277] set context from the master thread --- src/Storages/MergeTree/MergeList.cpp | 11 +++++++---- src/Storages/MergeTree/MergeList.h | 8 ++++---- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index f53c6ad81ee..3a2e5a4ff35 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -34,12 +34,12 @@ ThreadGroupSwitcher::~ThreadGroupSwitcher() CurrentThread::attachToGroup(prev_thread_group); } -ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupSwitcher && other) +ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupSwitcher && other) noexcept { this->swap(other); } -ThreadGroupSwitcher& ThreadGroupSwitcher::operator=(ThreadGroupSwitcher && other) +ThreadGroupSwitcher& ThreadGroupSwitcher::operator=(ThreadGroupSwitcher && other) noexcept { if (this != &other) { @@ -50,7 +50,7 @@ ThreadGroupSwitcher& ThreadGroupSwitcher::operator=(ThreadGroupSwitcher && other return *this; } -void ThreadGroupSwitcher::swap(ThreadGroupSwitcher & other) +void ThreadGroupSwitcher::swap(ThreadGroupSwitcher & other) noexcept { std::swap(merge_list_entry, other.merge_list_entry); std::swap(prev_thread_group, other.prev_thread_group); @@ -92,7 +92,10 @@ MergeListElement::MergeListElement( thread_group = std::make_shared(); - auto p_counters = CurrentThread::get().current_performance_counters; + thread_group->query_context = CurrentThread::get().getQueryContext(); + thread_group->global_context = CurrentThread::get().getGlobalContext(); + + auto * p_counters = CurrentThread::get().current_performance_counters; while (p_counters && p_counters->level != VariableContext::Process) p_counters = p_counters->getParent(); thread_group->performance_counters.setParent(p_counters); diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 200a574698c..2a166470203 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -70,12 +70,12 @@ class ThreadGroupSwitcher : private boost::noncopyable { public: explicit ThreadGroupSwitcher(MergeListEntry * merge_list_entry_); - ThreadGroupSwitcher(ThreadGroupSwitcher && other); - ThreadGroupSwitcher& operator=(ThreadGroupSwitcher && other); + ThreadGroupSwitcher(ThreadGroupSwitcher && other) noexcept; + ThreadGroupSwitcher& operator=(ThreadGroupSwitcher && other) noexcept; ~ThreadGroupSwitcher(); private: - ThreadGroupSwitcher() = default; - void swap(ThreadGroupSwitcher & other); + ThreadGroupSwitcher() noexcept = default; + void swap(ThreadGroupSwitcher & other) noexcept; MergeListEntry * merge_list_entry = nullptr; ThreadGroupStatusPtr prev_thread_group; From aeb8766ad59ec2b813a811142cb6e9cd0aa57572 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 18 Mar 2023 21:14:32 +0100 Subject: [PATCH 009/277] adjust after rebase --- src/Storages/MergeTree/MergeList.cpp | 45 +++---------------- src/Storages/MergeTree/MergeList.h | 22 ++++++--- .../MergeTree/MergePlainMergeTreeTask.cpp | 2 +- .../MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- .../ReplicatedMergeMutateTaskBase.cpp | 2 +- 5 files changed, 25 insertions(+), 48 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 3a2e5a4ff35..c13c5e6105e 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -11,15 +11,16 @@ namespace DB { -ThreadGroupSwitcher::ThreadGroupSwitcher(MergeListEntry * merge_list_entry_) - : merge_list_entry(merge_list_entry_) +ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupStatusPtr thread_group) { + chassert(thread_group); + prev_thread_group = CurrentThread::getGroup(); if (!prev_thread_group) return; - CurrentThread::detachGroup(); - CurrentThread::attachToGroup(merge_list_entry_->thread_group); + CurrentThread::detachFromGroupIfNotDetached(); + CurrentThread::attachToGroup(thread_group); } ThreadGroupSwitcher::~ThreadGroupSwitcher() @@ -27,37 +28,10 @@ ThreadGroupSwitcher::~ThreadGroupSwitcher() if (!prev_thread_group) return; - if (!merge_list_entry) - return; - - CurrentThread::detachGroup(); + CurrentThread::detachFromGroupIfNotDetached(); CurrentThread::attachToGroup(prev_thread_group); } -ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupSwitcher && other) noexcept -{ - this->swap(other); -} - -ThreadGroupSwitcher& ThreadGroupSwitcher::operator=(ThreadGroupSwitcher && other) noexcept -{ - if (this != &other) - { - auto tmp = ThreadGroupSwitcher(); - tmp.swap(other); - this->swap(tmp); - } - return *this; -} - -void ThreadGroupSwitcher::swap(ThreadGroupSwitcher & other) noexcept -{ - std::swap(merge_list_entry, other.merge_list_entry); - std::swap(prev_thread_group, other.prev_thread_group); - std::swap(prev_query_id, other.prev_query_id); -} - - MergeListElement::MergeListElement( const StorageID & table_id_, FutureMergedMutatedPartPtr future_part, @@ -90,18 +64,13 @@ MergeListElement::MergeListElement( is_mutation = (result_part_info.getDataVersion() != source_data_version); } - thread_group = std::make_shared(); - - thread_group->query_context = CurrentThread::get().getQueryContext(); - thread_group->global_context = CurrentThread::get().getGlobalContext(); + thread_group = ThreadGroupStatus::createForQuery(CurrentThread::get().getQueryContext(), {}); auto * p_counters = CurrentThread::get().current_performance_counters; while (p_counters && p_counters->level != VariableContext::Process) p_counters = p_counters->getParent(); thread_group->performance_counters.setParent(p_counters); - thread_group->master_thread_id = CurrentThread::get().thread_id; - auto & memory_tracker = thread_group->memory_tracker; memory_tracker.setDescription(description.c_str()); diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 2a166470203..37587d9d517 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -69,15 +69,23 @@ struct Settings; class ThreadGroupSwitcher : private boost::noncopyable { public: - explicit ThreadGroupSwitcher(MergeListEntry * merge_list_entry_); - ThreadGroupSwitcher(ThreadGroupSwitcher && other) noexcept; - ThreadGroupSwitcher& operator=(ThreadGroupSwitcher && other) noexcept; - ~ThreadGroupSwitcher(); -private: ThreadGroupSwitcher() noexcept = default; - void swap(ThreadGroupSwitcher & other) noexcept; + explicit ThreadGroupSwitcher(ThreadGroupStatusPtr thread_group); + ThreadGroupSwitcher(ThreadGroupSwitcher && other) noexcept + : prev_thread_group(std::move(other.prev_thread_group)) + { + other.prev_thread_group = nullptr; + } + ThreadGroupSwitcher & operator=(ThreadGroupSwitcher && other) noexcept + { + chassert(this != &other); + prev_thread_group = std::move(other.prev_thread_group); + other.prev_thread_group = nullptr; + return *this; + } + ~ThreadGroupSwitcher(); - MergeListEntry * merge_list_entry = nullptr; +private: ThreadGroupStatusPtr prev_thread_group; }; diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 052c6467b37..709a681619a 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -34,7 +34,7 @@ bool MergePlainMergeTreeTask::executeStep() /// Make out memory tracker a parent of current thread memory tracker std::optional switcher; if (merge_list_entry) - switcher = ThreadGroupSwitcher(merge_list_entry.get()); + switcher = ThreadGroupSwitcher((*merge_list_entry)->thread_group); switch (state) { diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 673bfaa0d47..822098e4352 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -70,7 +70,7 @@ bool MutatePlainMergeTreeTask::executeStep() /// Make out memory tracker a parent of current thread memory tracker std::optional switcher; if (merge_list_entry) - switcher = ThreadGroupSwitcher(merge_list_entry.get()); + switcher = ThreadGroupSwitcher((*merge_list_entry)->thread_group); switch (state) { diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index fbc6522170e..9368f7d8c51 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -130,7 +130,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() { std::optional switcher; if (merge_mutate_entry) - switcher = ThreadGroupSwitcher(merge_mutate_entry.get()); + switcher = ThreadGroupSwitcher((*merge_mutate_entry)->thread_group); auto remove_processed_entry = [&] () -> bool { From a4e2d09001b0f4cdd1f187bb3f3d95d2d10d2d79 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 24 Mar 2023 21:58:23 +0000 Subject: [PATCH 010/277] without schema inference from actual data --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 84 +++++++++++++++++------ src/Interpreters/Context.cpp | 66 +++++++++++++----- 2 files changed, 110 insertions(+), 40 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 2c4f7c3dc3b..87891cf148e 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -112,6 +112,8 @@ namespace ErrorCodes extern const int ALIAS_REQUIRED; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int UNKNOWN_TABLE; + extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -6087,9 +6089,24 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, if (!nested_table_function) expressions_visitor.visit(table_function_node_typed.getArgumentsNode()); + const auto & table_function_name = table_function_node_typed.getTableFunctionName(); + auto & scope_context = scope.context; - TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_function_node_typed.toAST(), scope_context); + TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().tryGet(table_function_name, scope_context); + if (!table_function_ptr) + { + auto hints = TableFunctionFactory::instance().getHints(table_function_name); + if (!hints.empty()) + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, + "Unknown table function {}. Maybe you meant: {}", + table_function_name, + DB::toString(hints)); + else + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, + "Unknown table function {}", + table_function_name); + } if (!nested_table_function && scope_context->getSettingsRef().use_structure_from_insertion_table_in_table_functions && @@ -6099,40 +6116,65 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, const auto & insertion_table = scope_context->getInsertionTable(); if (!insertion_table.empty()) { - auto & expression_list = scope.scope_node->as().getProjection(); const auto & insert_structure = DatabaseCatalog::instance().getTable(insertion_table, scope_context)->getInMemoryMetadataPtr()->getColumns(); - auto table_structure = table_function_ptr->getActualTableStructure(scope_context); + DB::ColumnsDescription structure_hint; /// Insert table matches columns against SELECT expression by position, so we want to map /// insert table columns to table function columns through names from SELECT expression. auto insert_column = insert_structure.begin(); - for (const auto & expression : expression_list) + auto insert_structure_end = insert_structure.end(); + auto virtual_column_names = table_function_ptr->getVirtualsToCheckBeforeUsingStructureHint(); + bool asterisk = false; + const auto & expression_list = scope.scope_node->as().getProjection(); + auto expression = expression_list.begin(); + + for (; expression != expression_list.end() && insert_column != insert_structure_end; ++expression) { - if (auto * identifier_node = expression->as()) + if (auto * identifier_node = (*expression)->as()) { - if (table_structure.hasPhysical(identifier_node->getIdentifier().getFullName())) - table_structure.modify(identifier_node->getIdentifier().getFullName(), [&insert_column](ColumnDescription & column){ column.type = insert_column->type; }); - ++insert_column; - } - else if (auto * matcher_node = expression->as(); matcher_node && matcher_node->getMatcherType() == MatcherNodeType::ASTERISK) - { - for (const auto & column : table_structure) + if (!virtual_column_names.contains(identifier_node->getIdentifier().getFullName())) { - table_structure.modify(column.name, [&insert_column](ColumnDescription & column){ column.type = insert_column->type; }); - ++insert_column; - if (insert_column == insert_structure.end()) - break; + if (asterisk) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Asterisk cannot be mixed with column list in INSERT SELECT query."); + + structure_hint.add({ identifier_node->getIdentifier().getFullName(), insert_column->type }); } + + if (asterisk) + --insert_structure_end; + else + ++insert_column; + } + else if (auto * matcher_node = (*expression)->as(); matcher_node && matcher_node->getMatcherType() == MatcherNodeType::ASTERISK) + { + if (asterisk) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Only one asterisk can be used in INSERT SELECT query."); + if (!structure_hint.empty()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Asterisk cannot be mixed with column list in INSERT SELECT query."); + + asterisk = true; } else - ++insert_column; - - if (insert_column == insert_structure.end()) - break; + { + if (asterisk) + --insert_structure_end; + else + ++insert_column; + } } - table_function_ptr->setStructureHint(table_structure); + if (expression != expression_list.end()) + throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, "Number of columns in insert table less than required by SELECT expression."); + + if (asterisk) + { + for (; insert_column != insert_structure_end; ++insert_column) + structure_hint.add({ insert_column->name, insert_column->type }); + } + + if (!structure_hint.empty()) + table_function_ptr->setStructureHint(structure_hint); } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 395c735a70a..3aa0f339cf2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -165,6 +165,8 @@ namespace ErrorCodes extern const int UNKNOWN_READ_METHOD; extern const int NOT_IMPLEMENTED; extern const int UNKNOWN_FUNCTION; + extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; } @@ -1395,38 +1397,64 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const if (getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint() && hasInsertionTable()) { const auto & insert_structure = DatabaseCatalog::instance().getTable(getInsertionTable(), shared_from_this())->getInMemoryMetadataPtr()->getColumns(); - auto table_structure = table_function_ptr->getActualTableStructure(getQueryContext()); + DB::ColumnsDescription structure_hint; /// Insert table matches columns against SELECT expression by position, so we want to map /// insert table columns to table function columns through names from SELECT expression. auto insert_column = insert_structure.begin(); - for (const auto & expression : select_query_hint->select()->as()->children) + auto insert_structure_end = insert_structure.end(); + auto virtual_column_names = table_function_ptr->getVirtualsToCheckBeforeUsingStructureHint(); + bool asterisk = false; + const auto & expression_list = select_query_hint->select()->as()->children; + auto expression = expression_list.begin(); + + for (; expression != expression_list.end() && insert_column != insert_structure_end; ++expression) { - if (auto * identifier = expression->as()) + if (auto * identifier = (*expression)->as()) { - if (table_structure.hasPhysical(identifier->name())) - table_structure.modify(identifier->name(), [&insert_column](ColumnDescription & column){ column.type = insert_column->type; }); - ++insert_column; - } - else if (expression->as()) - { - for (const auto & column : table_structure) + if (!virtual_column_names.contains(identifier->name())) { - table_structure.modify(column.name, [&insert_column](ColumnDescription & column){ column.type = insert_column->type; }); - ++insert_column; - if (insert_column == insert_structure.end()) - break; + if (asterisk) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Asterisk cannot be mixed with column list in INSERT SELECT query."); + + structure_hint.add({ identifier->name(), insert_column->type }); } + + if (asterisk) + --insert_structure_end; + else + ++insert_column; + } + else if ((*expression)->as()) + { + if (asterisk) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Only one asterisk can be used in INSERT SELECT query."); + if (!structure_hint.empty()) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Asterisk cannot be mixed with column list in INSERT SELECT query."); + + asterisk = true; } else - ++insert_column; - - if (insert_column == insert_structure.end()) - break; + { + if (asterisk) + --insert_structure_end; + else + ++insert_column; + } } - table_function_ptr->setStructureHint(table_structure); + if (expression != expression_list.end()) + throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, "Number of columns in insert table less than required by SELECT expression."); + + if (asterisk) + { + for (; insert_column != insert_structure_end; ++insert_column) + structure_hint.add({ insert_column->name, insert_column->type }); + } + + if (!structure_hint.empty()) + table_function_ptr->setStructureHint(structure_hint); } res = table_function_ptr->execute(table_expression, shared_from_this(), table_function_ptr->getName()); From 07502600f7518d81f94b89095b70c2297f48a357 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 25 Mar 2023 05:15:10 +0000 Subject: [PATCH 011/277] clang wants it :\ --- src/Interpreters/Context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 3aa0f339cf2..c787d387632 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1407,7 +1407,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const auto virtual_column_names = table_function_ptr->getVirtualsToCheckBeforeUsingStructureHint(); bool asterisk = false; const auto & expression_list = select_query_hint->select()->as()->children; - auto expression = expression_list.begin(); + const auto * expression = expression_list.begin(); for (; expression != expression_list.end() && insert_column != insert_structure_end; ++expression) { From adede9dcb0f9446adf9375ceca870ce024112bc4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 25 Mar 2023 20:55:10 +0000 Subject: [PATCH 012/277] test fixed --- ...e_structure_from_insertion_table.reference | 6 ++++++ ...458_use_structure_from_insertion_table.sql | 20 +++++++++---------- 2 files changed, 16 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference index 0ca28640270..7a004c58187 100644 --- a/tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference +++ b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference @@ -1,9 +1,15 @@ +\N 0 \N 1 1 2 +1 2 \N 42 \N 42 \N 42 \N 42 +\N 42 +\N 42 +42 +42 \N \N \N diff --git a/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql index a609dc361fe..ac53e003521 100644 --- a/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql +++ b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql @@ -9,17 +9,17 @@ set use_structure_from_insertion_table_in_table_functions=2; insert into test select * from file(02458_data.jsonl); insert into test select x, 1 from file(02458_data.jsonl); insert into test select x, y from file(02458_data.jsonl); -insert into test select x + 1, y from file(02458_data.jsonl); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} -insert into test select x, z from file(02458_data.jsonl); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +insert into test select x + 1, y from file(02458_data.jsonl); -- {serverError UNKNOWN_IDENTIFIER} +insert into test select x, z from file(02458_data.jsonl); insert into test select * from file(02458_data.jsoncompacteachrow); -insert into test select x, 1 from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} -insert into test select x, y from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} -insert into test select x + 1, y from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} -insert into test select x, z from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +insert into test select x, 1 from file(02458_data.jsoncompacteachrow); -- {serverError CANNOT_PARSE_INPUT_ASSERTION_FAILED} +insert into test select x, y from file(02458_data.jsoncompacteachrow); +insert into test select x + 1, y from file(02458_data.jsoncompacteachrow); -- {serverError UNKNOWN_IDENTIFIER} +insert into test select x, z from file(02458_data.jsoncompacteachrow); insert into test select * from input() format CSV 1,2 -insert into test select x, y from input() format CSV 1,2 -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} +insert into test select x, y from input() format CSV 1,2 insert into test select x, y from input() format JSONEachRow {"x" : null, "y" : 42} select * from test order by y; @@ -28,10 +28,10 @@ drop table test; create table test (x Nullable(UInt32)) engine=Memory(); insert into test select * from file(02458_data.jsonl); insert into test select x from file(02458_data.jsonl); -insert into test select y from file(02458_data.jsonl); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} -insert into test select y as x from file(02458_data.jsonl); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +insert into test select y from file(02458_data.jsonl); +insert into test select y as x from file(02458_data.jsonl); -insert into test select c1 from input() format CSV 1,2; -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} +insert into test select c1 from input() format CSV 1,2; -- {clientError INCORRECT_DATA} insert into test select x from input() format JSONEachRow {"x" : null, "y" : 42} select * from test order by x; From 22da93e239ffd4402ba27aee4c982742082cc9fc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Mar 2023 21:41:01 +0000 Subject: [PATCH 013/277] Cosmetics --- src/Functions/formatDateTime.cpp | 99 +++++++++++++++----------------- src/Functions/parseDateTime.cpp | 26 ++++----- 2 files changed, 58 insertions(+), 67 deletions(-) diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index bbb4c3ba5b0..daea8b3a7b0 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -39,21 +39,17 @@ namespace ErrorCodes namespace { -struct FormatDateTimeTraits +enum class SupportInteger { - enum class SupportInteger - { - Yes, - No - }; - - enum class FormatSyntax - { - MySQL, - Joda - }; + Yes, + No }; +enum class FormatSyntax +{ + MySQL, + Joda +}; template struct InstructionValueTypeMap {}; template <> struct InstructionValueTypeMap { using InstructionValueType = UInt32; }; @@ -85,11 +81,9 @@ constexpr std::string_view weekdaysFull[] = {"Sunday", "Monday", "Tuesday", "Wed constexpr std::string_view weekdaysShort[] = {"Sun", "Mon", "Tue", "Wed", "Thu", "Fri", "Sat"}; -constexpr std::string_view monthsFull[] - = {"January", "February", "March", "April", "May", "June", "July", "August", "September", "October", "November", "December"}; +constexpr std::string_view monthsFull[] = {"January", "February", "March", "April", "May", "June", "July", "August", "September", "October", "November", "December"}; -constexpr std::string_view monthsShort[] - = {"Jan", "Feb", "Mar", "Apr", "May", "Jun", "Jul", "Aug", "Sep", "Oct", "Nov", "Dec"}; +constexpr std::string_view monthsShort[] = {"Jan", "Feb", "Mar", "Apr", "May", "Jun", "Jul", "Aug", "Sep", "Oct", "Nov", "Dec"}; /** formatDateTime(time, 'format') * Performs formatting of time, according to provided format. @@ -129,7 +123,7 @@ constexpr std::string_view monthsShort[] * * PS. We can make this function to return FixedString. Currently it returns String. */ -template +template class FunctionFormatDateTimeImpl : public IFunction { private: @@ -157,7 +151,7 @@ private: /// This is the reason why we use raw function pointer in MySQL format and std::function /// in Joda format. using Func = std::conditional_t< - format_syntax == FormatDateTimeTraits::FormatSyntax::MySQL, + format_syntax == FormatSyntax::MySQL, size_t (*)(char *, Time, UInt64, UInt32, const DateLUTImpl &), std::function>; @@ -257,7 +251,10 @@ private: return pos; } public: - static size_t mysqlNoop(char *, Time, UInt64, UInt32, const DateLUTImpl &) { return 0; } + static size_t mysqlNoop(char *, Time, UInt64, UInt32, const DateLUTImpl &) + { + return 0; + } static size_t mysqlCentury(char * dest, Time source, UInt64, UInt32, const DateLUTImpl & timezone) { @@ -430,8 +427,7 @@ private: return writeNumber2(dest, ToSecondImpl::execute(source, timezone)); } - static size_t - mysqlFractionalSecond(char * dest, Time /*source*/, UInt64 fractional_second, UInt32 scale, const DateLUTImpl & /*timezone*/) + static size_t mysqlFractionalSecond(char * dest, Time /*source*/, UInt64 fractional_second, UInt32 scale, const DateLUTImpl & /*timezone*/) { if (scale == 0) scale = 1; @@ -672,7 +668,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if constexpr (support_integer == FormatDateTimeTraits::SupportInteger::Yes) + if constexpr (support_integer == SupportInteger::Yes) { if (arguments.size() != 1 && arguments.size() != 2 && arguments.size() != 3) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, @@ -718,7 +714,7 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, [[maybe_unused]] size_t input_rows_count) const override { ColumnPtr res; - if constexpr (support_integer == FormatDateTimeTraits::SupportInteger::Yes) + if constexpr (support_integer == SupportInteger::Yes) { if (arguments.size() == 1) { @@ -793,7 +789,7 @@ public: using T = typename InstructionValueTypeMap::InstructionValueType; std::vector> instructions; String out_template; - auto result_size = parseFormat(format, instructions, scale, out_template); + size_t out_template_size = parseFormat(format, instructions, scale, out_template); const DateLUTImpl * time_zone_tmp = nullptr; if (castType(arguments[0].type.get(), [&]([[maybe_unused]] const auto & type) { return true; })) @@ -807,26 +803,26 @@ public: const auto & vec = times->getData(); auto col_res = ColumnString::create(); - auto & dst_data = col_res->getChars(); - auto & dst_offsets = col_res->getOffsets(); - dst_data.resize(vec.size() * (result_size + 1)); - dst_offsets.resize(vec.size()); + auto & res_data = col_res->getChars(); + auto & res_offsets = col_res->getOffsets(); + res_data.resize(vec.size() * (out_template_size + 1)); + res_offsets.resize(vec.size()); - if constexpr (format_syntax == FormatDateTimeTraits::FormatSyntax::MySQL) + if constexpr (format_syntax == FormatSyntax::MySQL) { - /// Fill result with literals. + /// Fill result with template. { - UInt8 * begin = dst_data.data(); - UInt8 * end = begin + dst_data.size(); - UInt8 * pos = begin; + const UInt8 * const begin = res_data.data(); + const UInt8 * const end = res_data.data() + res_data.size(); + UInt8 * pos = res_data.data(); if (pos < end) { - memcpy(pos, out_template.data(), result_size + 1); /// With zero terminator. - pos += result_size + 1; + memcpy(pos, out_template.data(), out_template_size + 1); /// With zero terminator. mystring[mystring.size()] = '\0' is guaranteed since C++11. + pos += out_template_size + 1; } - /// Fill by copying exponential growing ranges. + /// Copy exponentially growing ranges. while (pos < end) { size_t bytes_to_copy = std::min(pos - begin, end - pos); @@ -836,7 +832,7 @@ public: } } - auto * begin = reinterpret_cast(dst_data.data()); + auto * begin = reinterpret_cast(res_data.data()); auto * pos = begin; for (size_t i = 0; i < vec.size(); ++i) { @@ -844,9 +840,7 @@ public: { const auto c = DecimalUtils::split(vec[i], scale); for (auto & instruction : instructions) - { instruction.perform(pos, static_cast(c.whole), c.fractional, scale, time_zone); - } } else { @@ -855,21 +849,19 @@ public: } *pos++ = '\0'; - dst_offsets[i] = pos - begin; + res_offsets[i] = pos - begin; } - dst_data.resize(pos - begin); + res_data.resize(pos - begin); return col_res; } template size_t parseFormat(const String & format, std::vector> & instructions, UInt32 scale, String & out_template) const { - static_assert( - format_syntax == FormatDateTimeTraits::FormatSyntax::MySQL || format_syntax == FormatDateTimeTraits::FormatSyntax::Joda, - "format syntax must be one of MySQL or Joda"); + static_assert(format_syntax == FormatSyntax::MySQL || format_syntax == FormatSyntax::Joda); - if constexpr (format_syntax == FormatDateTimeTraits::FormatSyntax::MySQL) + if constexpr (format_syntax == FormatSyntax::MySQL) return parseMySQLFormat(format, instructions, scale, out_template); else return parseJodaFormat(format, instructions, scale, out_template); @@ -914,13 +906,13 @@ public: switch (*pos) { - // Abbreviated weekday [Mon...Sun] + // Abbreviated weekday [Mon-Sun] case 'a': instructions.emplace_back(&Instruction::mysqlDayOfWeekTextShort); out_template += "Mon"; break; - // Abbreviated month [Jan...Dec] + // Abbreviated month [Jan-Dec] case 'b': instructions.emplace_back(&Instruction::mysqlMonthOfYearTextShort); out_template += "Jan"; @@ -958,12 +950,10 @@ public: // Fractional seconds case 'f': - { /// If the time data type has no fractional part, then we print '0' as the fractional part. instructions.emplace_back(&Instruction::mysqlFractionalSecond); out_template += String(std::max(1, scale), '0'); break; - } // Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23 case 'F': @@ -1013,7 +1003,7 @@ public: out_template += "0"; break; - // Full weekday [Monday...Sunday] + // Full weekday [Monday-Sunday] case 'W': instructions.emplace_back(&Instruction::mysqlDayOfWeekTextLong); out_template += "Monday"; @@ -1186,6 +1176,7 @@ public: size_t reserve_size = 0; const char * pos = format.data(); const char * end = format.data() + format.size(); + while (pos < end) { const char * cur_token = pos; @@ -1392,10 +1383,10 @@ struct NameFromUnixTimeInJodaSyntax }; -using FunctionFormatDateTime = FunctionFormatDateTimeImpl; -using FunctionFromUnixTimestamp = FunctionFormatDateTimeImpl; -using FunctionFormatDateTimeInJodaSyntax = FunctionFormatDateTimeImpl; -using FunctionFromUnixTimestampInJodaSyntax = FunctionFormatDateTimeImpl; +using FunctionFormatDateTime = FunctionFormatDateTimeImpl; +using FunctionFromUnixTimestamp = FunctionFormatDateTimeImpl; +using FunctionFormatDateTimeInJodaSyntax = FunctionFormatDateTimeImpl; +using FunctionFromUnixTimestampInJodaSyntax = FunctionFormatDateTimeImpl; } diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index abee7e0d8f8..cd3c0d993d0 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -101,16 +101,16 @@ namespace bool is_year_of_era = false; /// If true, year is calculated from era and year of era, the latter cannot be zero or negative. bool has_year = false; /// Whether year was explicitly specified. - /// If is_clock_hour = true, is_hour_of_half_day = true, hour's range is [1, 12] - /// If is_clock_hour = true, is_hour_of_half_day = false, hour's range is [1, 24] - /// If is_clock_hour = false, is_hour_of_half_day = true, hour's range is [0, 11] - /// If is_clock_hour = false, is_hour_of_half_day = false, hour's range is [0, 23] + /// If hour_starts_at_1 = true, is_hour_of_half_day = true, hour's range is [1, 12] + /// If hour_starts_at_1 = true, is_hour_of_half_day = false, hour's range is [1, 24] + /// If hour_starts_at_1 = false, is_hour_of_half_day = true, hour's range is [0, 11] + /// If hour_starts_at_1 = false, is_hour_of_half_day = false, hour's range is [0, 23] Int32 hour = 0; Int32 minute = 0; /// range [0, 59] Int32 second = 0; /// range [0, 59] bool is_am = true; /// If is_hour_of_half_day = true and is_am = false (i.e. pm) then add 12 hours to the result DateTime - bool is_clock_hour = false; /// Whether the hour is clockhour + bool hour_starts_at_1 = false; /// Whether the hour is clockhour bool is_hour_of_half_day = false; /// Whether the hour is of half day bool has_time_zone_offset = false; /// If true, time zone offset is explicitly specified. @@ -137,7 +137,7 @@ namespace second = 0; is_am = true; - is_clock_hour = false; + hour_starts_at_1 = false; is_hour_of_half_day = false; has_time_zone_offset = false; @@ -275,23 +275,23 @@ namespace throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Unknown half day of day: {}", text); } - void setHour(Int32 hour_, bool is_hour_of_half_day_ = false, bool is_clock_hour_ = false) + void setHour(Int32 hour_, bool is_hour_of_half_day_ = false, bool hour_starts_at_1_ = false) { Int32 max_hour; Int32 min_hour; Int32 new_hour = hour_; - if (!is_hour_of_half_day_ && !is_clock_hour_) + if (!is_hour_of_half_day_ && !hour_starts_at_1_) { max_hour = 23; min_hour = 0; } - else if (!is_hour_of_half_day_ && is_clock_hour_) + else if (!is_hour_of_half_day_ && hour_starts_at_1_) { max_hour = 24; min_hour = 1; new_hour = hour_ % 24; } - else if (is_hour_of_half_day_ && !is_clock_hour_) + else if (is_hour_of_half_day_ && !hour_starts_at_1_) { max_hour = 11; min_hour = 0; @@ -306,16 +306,16 @@ namespace if (hour_ < min_hour || hour_ > max_hour) throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, - "Value {} for hour must be in the range [{}, {}] if_hour_of_half_day={} and is_clock_hour={}", + "Value {} for hour must be in the range [{}, {}] if_hour_of_half_day={} and hour_starts_at_1={}", hour, max_hour, min_hour, is_hour_of_half_day_, - is_clock_hour_); + hour_starts_at_1_); hour = new_hour; is_hour_of_half_day = is_hour_of_half_day_; - is_clock_hour = is_clock_hour_; + hour_starts_at_1 = hour_starts_at_1_; } void setMinute(Int32 minute_) From 3db38dbb5a1a227c749549faf4ec0f140bc267b7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Mar 2023 22:16:10 +0000 Subject: [PATCH 014/277] Replace mySQL date formatter M behavior from minutes to month name --- .../functions/date-time-functions.md | 8 +- src/Functions/formatDateTime.cpp | 576 +++++++++++++----- src/Functions/parseDateTime.cpp | 40 +- .../00718_format_datetime.reference | 4 +- .../0_stateless/00718_format_datetime.sql | 3 + ...00921_datetime64_compatibility_long.python | 2 +- ...21_datetime64_compatibility_long.reference | 2 +- .../0_stateless/01411_from_unixtime.reference | 2 +- .../0_stateless/02564_date_format.reference | 2 +- .../02668_parse_datetime.reference | 7 + .../0_stateless/02668_parse_datetime.sql | 6 +- 11 files changed, 493 insertions(+), 159 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index d06ab253cf7..425d67ed5a0 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1276,16 +1276,16 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %k | hour in 24h format (00-23) | 22 | | %l | hour in 12h format (01-12) | 09 | | %m | month as an integer number (01-12) | 01 | -| %M | minute (00-59) | 33 | +| %M | full month name (January-December) | January | | %n | new-line character (‘’) | | | %p | AM or PM designation | PM | | %Q | Quarter (1-4) | 1 | -| %r | 12-hour HH:MM AM/PM time, equivalent to %H:%M %p | 10:30 PM | -| %R | 24-hour HH:MM time, equivalent to %H:%M | 22:33 | +| %r | 12-hour HH:MM AM/PM time, equivalent to %H:%i %p | 10:30 PM | +| %R | 24-hour HH:MM time, equivalent to %H:%i | 22:33 | | %s | second (00-59) | 44 | | %S | second (00-59) | 44 | | %t | horizontal-tab character (’) | | -| %T | ISO 8601 time format (HH:MM:SS), equivalent to %H:%M:%S | 22:33:44 | +| %T | ISO 8601 time format (HH:MM:SS), equivalent to %H:%i:%S | 22:33:44 | | %u | ISO 8601 weekday as number with Monday as 1 (1-7) | 2 | | %V | ISO 8601 week number (01-53) | 01 | | %w | weekday as a integer number with Sunday as 0 (0-6) | 2 | diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index daea8b3a7b0..c243222db91 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -109,13 +109,13 @@ constexpr std::string_view monthsShort[] = {"Jan", "Feb", "Mar", "Apr", "May", " * * Performance on Intel(R) Core(TM) i7-6700 CPU @ 3.40GHz: * - * WITH formatDateTime(now() + number, '%H:%M:%S') AS x SELECT count() FROM system.numbers WHERE NOT ignore(x); + * WITH formatDateTime(now() + number, '%H:%i:%S') AS x SELECT count() FROM system.numbers WHERE NOT ignore(x); * - 97 million rows per second per core; * * WITH formatDateTime(toDateTime('2018-01-01 00:00:00') + number, '%F %T') AS x SELECT count() FROM system.numbers WHERE NOT ignore(x) * - 71 million rows per second per core; * - * select count() from (select formatDateTime(t, '%m/%d/%Y %H:%M:%S') from (select toDateTime('2018-01-01 00:00:00')+number as t from numbers(100000000))); + * select count() from (select formatDateTime(t, '%m/%d/%Y %H:%i:%S') from (select toDateTime('2018-01-01 00:00:00')+number as t from numbers(100000000))); * - 53 million rows per second per core; * * select count() from (select formatDateTime(t, 'Hello %Y World') from (select toDateTime('2018-01-01 00:00:00')+number as t from numbers(100000000))); @@ -146,26 +146,34 @@ private: class Instruction { public: - /// Using std::function will cause performance degradation in MySQL format by 0.45x. - /// But std::function is required for Joda format to capture extra variables. - /// This is the reason why we use raw function pointer in MySQL format and std::function - /// in Joda format. - using Func = std::conditional_t< - format_syntax == FormatSyntax::MySQL, - size_t (*)(char *, Time, UInt64, UInt32, const DateLUTImpl &), - std::function>; + /// Joda format generally requires capturing extra variables (i.e. holding state) which is more convenient with + /// std::function and std::bind. Unfortunately, std::function causes a performance degradation by 0.45x compared to raw function + /// pointers. For MySQL format, we generally prefer raw function pointers. Because of the special case that not all formatters are + /// fixed-width formatters (see mysqlLiteral), we still need to be able to store state. For that reason, we use member function + /// pointers instead of static function pointers. + using FuncMysql = size_t (Instruction diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index c61cacc9d8c..cf71b423713 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -270,3 +270,29 @@ def test_throttle_retry(cluster): ) == "42\n" ) + + +# Check that loading of parts is retried. +def test_retry_loading_parts(cluster): + node = cluster.instances["node"] + + node.query( + """ + CREATE TABLE s3_retry_loading_parts ( + id Int64 + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS storage_policy='s3_no_retries' + """ + ) + + node.query("INSERT INTO s3_retry_loading_parts VALUES (42)") + node.query("DETACH TABLE s3_retry_loading_parts") + + fail_request(cluster, 5) + node.query("ATTACH TABLE s3_retry_loading_parts") + + assert node.contains_in_log( + "Failed to load data part all_1_1_0 at try 0 with retryable error" + ) + assert node.query("SELECT * FROM s3_retry_loading_parts") == "42\n" From 224f4f92e0e39da5f181111aa329048662f319f7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 5 Apr 2023 21:26:53 +0000 Subject: [PATCH 155/277] fix test --- .../02458_use_structure_from_insertion_table.reference | 3 --- .../02458_use_structure_from_insertion_table.sql | 10 +++++----- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference index 7a004c58187..53abb49c4e1 100644 --- a/tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference +++ b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference @@ -1,9 +1,6 @@ \N 0 \N 1 1 2 -1 2 -\N 42 -\N 42 \N 42 \N 42 \N 42 diff --git a/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql index a199a69cde8..97d493fa031 100644 --- a/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql +++ b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql @@ -13,13 +13,13 @@ insert into test select x + 1, y from file(02458_data.jsonl); -- {serverError ON insert into test select x, z from file(02458_data.jsonl); insert into test select * from file(02458_data.jsoncompacteachrow); -insert into test select x, 1 from file(02458_data.jsoncompacteachrow); -- {serverError CANNOT_PARSE_INPUT_ASSERTION_FAILED} -insert into test select x, y from file(02458_data.jsoncompacteachrow); +insert into test select x, 1 from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +insert into test select x, y from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} insert into test select x + 1, y from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} -insert into test select x, z from file(02458_data.jsoncompacteachrow); +insert into test select x, z from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} insert into test select * from input() format CSV 1,2 -insert into test select x, y from input() format CSV 1,2 +insert into test select x, y from input() format CSV 1,2 -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} insert into test select x, y from input() format JSONEachRow {"x" : null, "y" : 42} select * from test order by y; @@ -31,7 +31,7 @@ insert into test select x from file(02458_data.jsonl); insert into test select y from file(02458_data.jsonl); insert into test select y as x from file(02458_data.jsonl); -insert into test select c1 from input() format CSV 1,2; -- {clientError INCORRECT_DATA} +insert into test select c1 from input() format CSV 1,2; -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} insert into test select x from input() format JSONEachRow {"x" : null, "y" : 42} select * from test order by x; From 785ea8213b95d5874512db302515baf6532b25ab Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Apr 2023 02:29:37 +0200 Subject: [PATCH 156/277] Better exception messages from Keeper client --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 8183569a718..79a975e683f 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -669,8 +669,8 @@ void ZooKeeper::receiveThread() earliest_operation = operations.begin()->second; auto earliest_operation_deadline = earliest_operation->time + std::chrono::microseconds(args.operation_timeout_ms * 1000); if (now > earliest_operation_deadline) - throw Exception(Error::ZOPERATIONTIMEOUT, "Operation timeout (deadline already expired) for path: {}", - earliest_operation->request->getPath()); + throw Exception(Error::ZOPERATIONTIMEOUT, "Operation timeout (deadline of {} ms already expired) for path: {}", + args.operation_timeout_ms, earliest_operation->request->getPath()); max_wait_us = std::chrono::duration_cast(earliest_operation_deadline - now).count(); } } @@ -687,12 +687,12 @@ void ZooKeeper::receiveThread() { if (earliest_operation) { - throw Exception(Error::ZOPERATIONTIMEOUT, "Operation timeout (no response) for request {} for path: {}", - toString(earliest_operation->request->getOpNum()), earliest_operation->request->getPath()); + throw Exception(Error::ZOPERATIONTIMEOUT, "Operation timeout (no response in {} ms) for request {} for path: {}", + args.operation_timeout_ms, toString(earliest_operation->request->getOpNum()), earliest_operation->request->getPath()); } waited_us += max_wait_us; if (waited_us >= args.session_timeout_ms * 1000) - throw Exception(Error::ZOPERATIONTIMEOUT, "Nothing is received in session timeout"); + throw Exception(Error::ZOPERATIONTIMEOUT, "Nothing is received in session timeout of {} ms", args.session_timeout_ms); } @@ -1080,7 +1080,7 @@ void ZooKeeper::pushRequest(RequestInfo && info) if (requests_queue.isFinished()) throw Exception(Error::ZSESSIONEXPIRED, "Session expired"); - throw Exception(Error::ZOPERATIONTIMEOUT, "Cannot push request to queue within operation timeout"); + throw Exception(Error::ZOPERATIONTIMEOUT, "Cannot push request to queue within operation timeout of {} ms", args.operation_timeout_ms); } } catch (...) @@ -1332,7 +1332,7 @@ void ZooKeeper::close() request_info.request = std::make_shared(std::move(request)); if (!requests_queue.tryPush(std::move(request_info), args.operation_timeout_ms)) - throw Exception(Error::ZOPERATIONTIMEOUT, "Cannot push close request to queue within operation timeout"); + throw Exception(Error::ZOPERATIONTIMEOUT, "Cannot push close request to queue within operation timeout of {} ms", args.operation_timeout_ms); ProfileEvents::increment(ProfileEvents::ZooKeeperClose); } From 22af013184d70cb87d6c193476ad30aab6559f7d Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Wed, 5 Apr 2023 20:49:24 -0700 Subject: [PATCH 157/277] Add why clickhouse-local overview --- docs/en/operations/utilities/clickhouse-local.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index a23e0745dec..6363d9cab27 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -6,7 +6,13 @@ sidebar_label: clickhouse-local # clickhouse-local -The `clickhouse-local` program enables you to perform fast processing on local files, without having to deploy and configure the ClickHouse server. It accepts data that represent tables and queries them using [ClickHouse SQL dialect](../../sql-reference/index.md). `clickhouse-local` uses the same core as ClickHouse server, so it supports most of the features and the same set of formats and table engines. +## When to use clickhouse-local vs. ClickHouse + +`clickhouse-local` is an easy-to-use version of ClickHouse that is ideal for developers who need to perform fast processing on local and remote files using SQL without having to install a full database server. With `clickhouse-local`, developers can use SQL commands (using the [ClickHouse SQL dialect](../../sql-reference/index.md)) directly from the command line, providing a simple and efficient way to access ClickHouse features without the need for a full ClickHouse installation. One of the main benefits of `clickhouse-local` is that it is already included when installing [clickhouse-client](https://clickhouse.com/docs/en/integrations/sql-clients/clickhouse-client-local). This means that developers can get started with `clickhouse-local` quickly, without the need for a complex installation process. + +While `clickhouse-local` is a great tool for development and testing purposes, and for processing files, it is not suitable for serving end users or applications. In these scenarios, it is recommended to use the open-source [ClickHouse](https://clickhouse.com/docs/en/install). ClickHouse is a powerful OLAP database that is designed to handle large-scale analytical workloads. It provides fast and efficient processing of complex queries on large datasets, making it ideal for use in production environments where high-performance is critical. Additionally, ClickHouse offers a wide range of features such as replication, sharding, and high availability, which are essential for scaling up to handle large datasets and serving applications. If you need to handle larger datasets or serve end users or applications, we recommend using open-source ClickHouse instead of `clickhouse-local`. + +Please read the docs below that show example use cases for `clickhouse-local`, such as [querying local CSVs](#query-data-in-a-csv-file-using-sql) or [reading a parquet file in S3](#query-data-in-a-parquet-file-in-aws-s3). ## Download clickhouse-local From 5eb31bba8743b33050a5711f7e3b1182e1bb5948 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Apr 2023 07:47:07 +0000 Subject: [PATCH 158/277] MySQL compat: Align behavior of %f in formatDateTime() with parseDateTime() --- .../functions/date-time-functions.md | 4 ++- src/Core/Settings.h | 3 +- src/Core/SettingsChangesHistory.h | 1 + src/Functions/formatDateTime.cpp | 36 ++++++++++++++++--- .../00718_format_datetime.reference | 25 +++++++++---- .../0_stateless/00718_format_datetime.sql | 19 ++++++++-- 6 files changed, 73 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 11036d804dc..b49f8745468 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1264,7 +1264,7 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %d | day of the month, zero-padded (01-31) | 02 | | %D | Short MM/DD/YY date, equivalent to %m/%d/%y | 01/02/18 | | %e | day of the month, space-padded (1-31) |   2 | -| %f | fractional second from the fractional part of DateTime64 | 1234560 | +| %f | fractional second, see below (*) | 1234560 | | %F | short YYYY-MM-DD date, equivalent to %Y-%m-%d | 2018-01-02 | | %g | two-digit year format, aligned to ISO 8601, abbreviated from four-digit notation | 18 | | %G | four-digit year format for ISO week number, calculated from the week-based year [defined by the ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates) standard, normally useful only with %V | 2018 | @@ -1295,6 +1295,8 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %z | Time offset from UTC as +HHMM or -HHMM | -0500 | | %% | a % sign | % | +(*) The behavior of `%f` is to print `000000` (six zeros) if the formatted value is a Date, Date32 or DateTime (which have no fractional seconds) or a DateTime64 with a precision of 0. Earlier versions of ClickHouse printed `0` in this case. The previous behavior can be restored using setting `formatdatetime_f_prints_single_zero = 1`. + **Example** Query: diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b6a149aa4ca..1b565d421eb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -467,7 +467,8 @@ class IColumn; M(Bool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.", 0) \ \ M(Bool, allow_execute_multiif_columnar, true, "Allow execute multiIf function columnar", 0) \ - M(Bool, formatdatetime_parsedatetime_m_is_month_name, true, "Formatter '%M' in function 'formatDateTime' produces the month name instead of minutes.", 0) \ + M(Bool, formatdatetime_f_prints_single_zero, false, "Formatter '%f' in function 'formatDateTime()' produces a single zero instead of six zeros if the formatted value has no fractional seconds.", 0) \ + M(Bool, formatdatetime_parsedatetime_m_is_month_name, true, "Formatter '%M' in functions 'formatDateTime()' and 'parseDateTime()' produces the month name instead of minutes.", 0) \ \ M(UInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.", 0) \ M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 4f89397ed9d..d7f80cc7a49 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -101,6 +101,7 @@ static std::map sett {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, + {"23.4", {{"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}}}, {"23.4", {{"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index d6275a54c75..a015340fc5d 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -449,6 +449,20 @@ private: } size_t mysqlFractionalSecond(char * dest, Time /*source*/, UInt64 fractional_second, UInt32 scale, const DateLUTImpl & /*timezone*/) + { + if (scale == 0) + scale = 6; + + for (Int64 i = scale, value = fractional_second; i > 0; --i) + { + dest[i - 1] += value % 10; + value /= 10; + } + return scale; + } + + /// Same as mysqlFractionalSecond but prints a single zero if the value has no fractional seconds + size_t mysqlFractionalSecondSingleZero(char * dest, Time /*source*/, UInt64 fractional_second, UInt32 scale, const DateLUTImpl & /*timezone*/) { if (scale == 0) scale = 1; @@ -710,6 +724,7 @@ private: } const bool mysql_M_is_month_name; + const bool mysql_f_prints_single_zero; public: static constexpr auto name = Name::name; @@ -718,6 +733,7 @@ public: explicit FunctionFormatDateTimeImpl(ContextPtr context) : mysql_M_is_month_name(context->getSettings().formatdatetime_parsedatetime_m_is_month_name) + , mysql_f_prints_single_zero(context->getSettings().formatdatetime_f_prints_single_zero) { } @@ -1116,11 +1132,21 @@ public: // Fractional seconds case 'f': { - /// If the time data type has no fractional part, then we print '0' as the fractional part. - Instruction instruction; - instruction.setMysqlFunc(&Instruction::mysqlFractionalSecond); - instructions.push_back(std::move(instruction)); - out_template += String(std::max(1, scale), '0'); + /// If the time data type has no fractional part, we print (default) '000000' or (deprecated) '0' as fractional part. + if (mysql_f_prints_single_zero) + { + Instruction instruction; + instruction.setMysqlFunc(&Instruction::mysqlFractionalSecondSingleZero); + instructions.push_back(std::move(instruction)); + out_template += String(scale == 0 ? 1 : scale, '0'); + } + else + { + Instruction instruction; + instruction.setMysqlFunc(&Instruction::mysqlFractionalSecond); + instructions.push_back(std::move(instruction)); + out_template += String(scale == 0 ? 6 : scale, '0'); + } break; } diff --git a/tests/queries/0_stateless/00718_format_datetime.reference b/tests/queries/0_stateless/00718_format_datetime.reference index eb2c23576eb..50874ac9b2e 100644 --- a/tests/queries/0_stateless/00718_format_datetime.reference +++ b/tests/queries/0_stateless/00718_format_datetime.reference @@ -5,6 +5,7 @@ Jan Jan 02 02 01/02/18 01/02/18 2 2 +000000 000000 2018-01-02 2018-01-02 10 12 22 00 @@ -43,11 +44,23 @@ no formatting pattern no formatting pattern -1100 +0300 +0530 -1234560 -000340 +000000 +000000 +000000 +000000 +123 +123456 +123456789 +0 +0 +0 +0 +123 +123456 +123456789 2022-12-08 18:11:29.123400000 2022-12-08 18:11:29.1 -2022-12-08 18:11:29.0 -2022-12-08 18:11:29.0 -2022-12-08 00:00:00.0 -2022-12-08 00:00:00.0 +2022-12-08 18:11:29.000000 +2022-12-08 18:11:29.000000 +2022-12-08 00:00:00.000000 +2022-12-08 00:00:00.000000 diff --git a/tests/queries/0_stateless/00718_format_datetime.sql b/tests/queries/0_stateless/00718_format_datetime.sql index a77578fc7c3..c0db6a4f64e 100644 --- a/tests/queries/0_stateless/00718_format_datetime.sql +++ b/tests/queries/0_stateless/00718_format_datetime.sql @@ -17,6 +17,7 @@ SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%C'), formatDateTime(t SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%d'), formatDateTime(toDate32('2018-01-02'), '%d'); SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%D'), formatDateTime(toDate32('2018-01-02'), '%D'); SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%e'), formatDateTime(toDate32('2018-01-02'), '%e'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%f'), formatDateTime(toDate32('2018-01-02'), '%f'); SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%F'), formatDateTime(toDate32('2018-01-02'), '%F'); SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%h'), formatDateTime(toDate32('2018-01-02'), '%h'); SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%H'), formatDateTime(toDate32('2018-01-02'), '%H'); @@ -66,8 +67,22 @@ SELECT formatDateTime(toDateTime('2020-01-01 01:00:00', 'US/Samoa'), '%z'); SELECT formatDateTime(toDateTime('2020-01-01 01:00:00', 'Europe/Moscow'), '%z'); SELECT formatDateTime(toDateTime('1970-01-01 00:00:00', 'Asia/Kolkata'), '%z'); -select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456', 7), '%f'); -select formatDateTime(toDateTime64('2022-12-08 18:11:29.00034', 6, 'UTC'), '%f'); +-- %f (default settings) +select formatDateTime(toDate('2010-01-04'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +select formatDateTime(toDate32('2010-01-04'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +select formatDateTime(toDateTime('2010-01-04 12:34:56'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +select formatDateTime(toDateTime64('2010-01-04 12:34:56', 0), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +select formatDateTime(toDateTime64('2010-01-04 12:34:56.123', 3), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456', 6), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456789', 9), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +-- %f (legacy settings) +select formatDateTime(toDate('2010-01-04'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1; +select formatDateTime(toDate32('2010-01-04'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1; +select formatDateTime(toDateTime('2010-01-04 12:34:56'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1; +select formatDateTime(toDateTime64('2010-01-04 12:34:56', 0), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1; +select formatDateTime(toDateTime64('2010-01-04 12:34:56.123', 3), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1; +select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456', 6), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456789', 9), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1; select formatDateTime(toDateTime64('2022-12-08 18:11:29.1234', 9, 'UTC'), '%F %T.%f'); select formatDateTime(toDateTime64('2022-12-08 18:11:29.1234', 1, 'UTC'), '%F %T.%f'); From b7be5fd89e825ae10be38e265a7289839d97b244 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Apr 2023 08:40:39 +0000 Subject: [PATCH 159/277] Improve code and tests --- src/Functions/parseDateTime.cpp | 48 ++++++++----------- .../02668_parse_datetime.reference | 6 +-- .../0_stateless/02668_parse_datetime.sql | 6 +-- 3 files changed, 27 insertions(+), 33 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 01239074852..12e2e113af4 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -716,7 +716,7 @@ namespace if constexpr (need_check_space == NeedCheckSpace::Yes) checkSpace(cur, end, 1, "assertChar requires size >= 1", fragment); - if (*cur != expected) + if (*cur != expected) [[unlikely]] throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse fragment {} from {} because char {} is expected but {} provided", @@ -729,6 +729,24 @@ namespace return cur; } + template + static Pos assertNumber(Pos cur, Pos end, const String & fragment) + { + if constexpr (need_check_space == NeedCheckSpace::Yes) + checkSpace(cur, end, 1, "assertChar requires size >= 1", fragment); + + if (*cur < '0' || *cur > '9') [[unlikely]] + throw Exception( + ErrorCodes::CANNOT_PARSE_DATETIME, + "Unable to parse fragment {} from {} because {} is not a number", + fragment, + std::string_view(cur, end - cur), + String(*cur, 1)); + + ++cur; + return cur; + } + static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, const String & fragment, DateTime & date) { checkSpace(cur, end, 3, "mysqlDayOfWeekTextShort requires size >= 3", fragment); @@ -1037,32 +1055,8 @@ namespace static Pos mysqlMicrosecond(Pos cur, Pos end, const String & fragment, DateTime & /*date*/) { - checkSpace(cur, end, 6, "mysqlMicrosecond requires size >= 6", fragment); - - Pos start = cur; - auto check_is_number = [&](Pos pos) - { - if (*pos < '0' || *pos > '9') - throw Exception( - ErrorCodes::CANNOT_PARSE_DATETIME, - "Unable to parse fragment '{}' from '{}' because '{}'' is not a number ", - fragment, - std::string_view(start, end), - *cur); - }; - - check_is_number(cur); - ++cur; - check_is_number(cur); - ++cur; - check_is_number(cur); - ++cur; - check_is_number(cur); - ++cur; - check_is_number(cur); - ++cur; - check_is_number(cur); - ++cur; + for (size_t i = 0; i < 6; ++i) + cur = assertNumber(cur, end, fragment); return cur; } diff --git a/tests/queries/0_stateless/02668_parse_datetime.reference b/tests/queries/0_stateless/02668_parse_datetime.reference index a5b5ad7d109..3a6925ecb70 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.reference +++ b/tests/queries/0_stateless/02668_parse_datetime.reference @@ -198,11 +198,11 @@ select parseDateTime('456789', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', select parseDateTime('42', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- { serverError NOT_ENOUGH_SPACE } select parseDateTime('12ABCD', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- mixed YMD format -select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC'); +select parseDateTime('2021-01-04+23:00:00.654321', '%Y-%m-%d+%H:%i:%s.%f', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC'); 1 -select parseDateTime('2019-07-03 11:04:10', '%Y-%m-%d %H:%i:%s', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +select parseDateTime('2019-07-03 11:04:10.975319', '%Y-%m-%d %H:%i:%s.%f', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); 1 -select parseDateTime('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +select parseDateTime('10:04:11 03-07-2019.242424', '%s:%i:%H %d-%m-%Y.%f', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); 1 -- *OrZero, *OrNull, str_to_date select parseDateTimeOrZero('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index 33e84120521..b18375840c9 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -135,9 +135,9 @@ select parseDateTime('42', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC select parseDateTime('12ABCD', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- mixed YMD format -select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC'); -select parseDateTime('2019-07-03 11:04:10', '%Y-%m-%d %H:%i:%s', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); -select parseDateTime('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +select parseDateTime('2021-01-04+23:00:00.654321', '%Y-%m-%d+%H:%i:%s.%f', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC'); +select parseDateTime('2019-07-03 11:04:10.975319', '%Y-%m-%d %H:%i:%s.%f', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +select parseDateTime('10:04:11 03-07-2019.242424', '%s:%i:%H %d-%m-%Y.%f', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); -- *OrZero, *OrNull, str_to_date select parseDateTimeOrZero('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); From 4b7c0f42f4f2777c6da90602ac28deb2963bf5ee Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Apr 2023 09:49:58 +0000 Subject: [PATCH 160/277] Small documentation follow-up to #47246 --- docs/en/sql-reference/functions/date-time-functions.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 11036d804dc..74ef9a28dc1 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1276,7 +1276,7 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %k | hour in 24h format (00-23) | 22 | | %l | hour in 12h format (01-12) | 09 | | %m | month as an integer number (01-12) | 01 | -| %M | full month name (January-December) | January | +| %M | full month name (January-December), see (*) below | January | | %n | new-line character (‘’) | | | %p | AM or PM designation | PM | | %Q | Quarter (1-4) | 1 | @@ -1295,6 +1295,8 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %z | Time offset from UTC as +HHMM or -HHMM | -0500 | | %% | a % sign | % | +(*) In ClickHouse versions earlier than v23.4, `%M` prints the minute (00-59) instead of the full month name (January-December). The previous behavior can be restored using setting `formatdatetime_parsedatetime_m_is_month_name = 0`. + **Example** Query: From f9fa29342159e032b8054c8759d529a78671727c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 6 Apr 2023 13:49:03 +0300 Subject: [PATCH 161/277] Update 00002_log_and_exception_messages_formatting.sql --- .../00002_log_and_exception_messages_formatting.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index e1409985e41..0638c50ec69 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -53,10 +53,10 @@ create temporary table known_short_messages (s String) as select * from (select ] as arr) array join arr; -- Check that we don't have too many short meaningless message patterns. -select 'messages shorter than 10', max2(countDistinctOrDefault(message_format_string), 0) from logs where length(message_format_string) < 10 and message_format_string not in known_short_messages; +select 'messages shorter than 10', max2(countDistinctOrDefault(message_format_string), 1) from logs where length(message_format_string) < 10 and message_format_string not in known_short_messages; -- Same as above. Feel free to update the threshold or remove this query if really necessary -select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_string), 2) from logs where length(message_format_string) < 16 and message_format_string not in known_short_messages; +select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_string), 3) from logs where length(message_format_string) < 16 and message_format_string not in known_short_messages; -- Same as above, but exceptions must be more informative. Feel free to update the threshold or remove this query if really necessary select 'exceptions shorter than 30', max2(countDistinctOrDefault(message_format_string), 27) from logs where length(message_format_string) < 30 and message ilike '%DB::Exception%' and message_format_string not in known_short_messages; From 2cc7d52bdfae122a733bfee6f8f0e7cf05f67fa1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 6 Apr 2023 13:49:44 +0300 Subject: [PATCH 162/277] Update 00002_log_and_exception_messages_formatting.reference --- .../00002_log_and_exception_messages_formatting.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference index d3991f053f0..1e7b85d6489 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference @@ -1,7 +1,7 @@ runtime messages 0.001 runtime exceptions 0.05 -messages shorter than 10 0 -messages shorter than 16 2 +messages shorter than 10 1 +messages shorter than 16 3 exceptions shorter than 30 27 noisy messages 0.3 noisy Trace messages 0.16 From 3fc952a56af8efcccc670460f9fb1462b77f752a Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 6 Apr 2023 13:01:24 +0200 Subject: [PATCH 163/277] Update CachedOnDiskReadBufferFromFile.cpp --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 72346787cfb..5eaee2e3026 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -1184,7 +1184,7 @@ String CachedOnDiskReadBufferFromFile::getInfoForLog() implementation_buffer_read_range_str = "None"; String current_file_segment_info; - if (current_file_segment_it == file_segments_holder->file_segments.end()) + if (current_file_segment_it != file_segments_holder->file_segments.end()) current_file_segment_info = (*current_file_segment_it)->getInfoForLog(); else current_file_segment_info = "None"; From 1ec9f5e42c03d2a054f8aeecf8b8464de657f46b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 6 Apr 2023 13:20:11 +0200 Subject: [PATCH 164/277] Avoid operation on uninitialised data in readDateTimeTextImpl --- src/IO/ReadHelpers.h | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 20ba73e0fa7..9c0c9525773 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1028,12 +1028,15 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re bool is_ok = true; if constexpr (std::is_same_v) - datetime64 = DecimalUtils::decimalFromComponents(components, scale); + { + datetime64 = DecimalUtils::decimalFromComponents(components, scale) * negative_multiplier; + } else + { is_ok = DecimalUtils::tryGetDecimalFromComponents(components, scale, datetime64); - - datetime64 *= negative_multiplier; - + if (is_ok) + datetime64 *= negative_multiplier; + } return ReturnType(is_ok); } From 56c51043785c4c201816ec5b2b8a1b7f40ae70e4 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 6 Apr 2023 11:43:56 +0000 Subject: [PATCH 165/277] Update tests --- tests/queries/0_stateless/01905_to_json_string.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01905_to_json_string.reference b/tests/queries/0_stateless/01905_to_json_string.reference index 33d435f8e1a..ec4f4e48bde 100644 --- a/tests/queries/0_stateless/01905_to_json_string.reference +++ b/tests/queries/0_stateless/01905_to_json_string.reference @@ -1,3 +1,3 @@ -[] 2947817982 "&" -69802.9769 "w" -1.9158530982937093e25 ["2003-05-15","1988-03-19 06:13:49","2090-04-14 03:58:26.029","91943d2e-480d-66b5-ee4c-1b5bb8eb7256"] "½O" [] -[-115] 481807067 ",{MM" -170235.0663 "o" 3.3808659558052087e155 ["2055-01-12","2070-08-09 03:49:21","2068-11-30 09:36:49.672","20b0e7b5-ad0e-177b-3054-c779b2a8ebe0"] "I\\u001C" ["e57178f9-4d10-2fa1-7c2d-53c5a65c3463"] +[] 2947817982 "&" -69802.9769 "o" 3.3808659558052087e155 ["2142-01-24","2076-06-05 14:54:21","2068-11-30 09:36:49.672","ee4c1b5b-b8eb-7256-20b0-e7b5ad0e177b"] "´á" ["7c2d53c5-a65c-3463-a76e-e26583aca234"] +[-115] 481807067 ",{MM" -45534.1174 "w" 1.711178201812925e-166 ["1994-01-04","1971-12-29 08:41:23","2012-03-25 07:11:39.573","3054c779-b2a8-ebe0-ec50-64cb1c494fbd"] "Ó\\u0000" ["055300b2-b400-653c-1ea0-2413e3a3af76"] {"1234":"5678"} From b199219ec1b4f34bfa0b98983e3efb1103785b35 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 6 Apr 2023 12:24:22 +0000 Subject: [PATCH 166/277] Add reading step for system zookeeper. Analyze path from filter DAG. --- .../System/StorageSystemZooKeeper.cpp | 247 ++++++++++-------- src/Storages/System/StorageSystemZooKeeper.h | 17 +- 2 files changed, 154 insertions(+), 110 deletions(-) diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 9663c76a5c3..ee33253e40f 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -12,11 +12,17 @@ #include #include #include +#include +#include +#include +#include #include #include #include #include #include +#include +#include #include #include #include @@ -155,8 +161,24 @@ public: } }; +class ReadFromSystemZooKeeper final : public SourceStepWithFilter +{ +public: + ReadFromSystemZooKeeper(const Block & header, SelectQueryInfo & query_info_, ContextPtr context_); + + String getName() const override { return "ReadFromSystemZooKeeper"; } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; + +private: + void fillData(MutableColumns & res_columns) const; + + std::shared_ptr storage_limits; + ContextPtr context; +}; + StorageSystemZooKeeper::StorageSystemZooKeeper(const StorageID & table_id_) - : IStorageSystemOneBlock(table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; ColumnsDescription desc; @@ -173,6 +195,26 @@ StorageSystemZooKeeper::StorageSystemZooKeeper(const StorageID & table_id_) setInMemoryMetadata(storage_metadata); } +bool StorageSystemZooKeeper::mayBenefitFromIndexForIn(const ASTPtr & node, ContextPtr, const StorageMetadataPtr &) const +{ + return node->as() && node->getColumnName() == "path"; +} + +void StorageSystemZooKeeper::read( + QueryPlan & query_plan, + const Names & /*column_names*/, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + size_t /*num_streams*/) +{ + auto header = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtuals()); + auto read_step = std::make_unique(header, query_info, context); + query_plan.addStep(std::move(read_step)); +} + SinkToStoragePtr StorageSystemZooKeeper::write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr context) { if (!context->getConfigRef().getBool("allow_zookeeper_write", false)) @@ -229,125 +271,99 @@ static String pathCorrected(const String & path) return path_corrected; } - -static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context, bool allow_unrestricted) +static bool isPathNode(const ActionsDAG::Node * node) { - const auto * function = elem.as(); - if (!function) - return false; + while (node->type == ActionsDAG::ActionType::ALIAS) + node = node->children.at(0); - if (function->name == "and") + return node->result_name == "path"; +} + +static void extractPathImpl(const ActionsDAG::Node & node, Paths & res, ContextPtr context, bool allow_unrestricted) +{ + if (node.type != ActionsDAG::ActionType::FUNCTION) + return; + + auto function_name = node.function_base->getName(); + if (function_name == "and") { - for (const auto & child : function->arguments->children) - if (extractPathImpl(*child, res, context, allow_unrestricted)) - return true; + for (const auto * child : node.children) + extractPathImpl(*child, res, context, allow_unrestricted); - return false; + return; } - const auto & args = function->arguments->as(); - if (args.children.size() != 2) - return false; + if (node.children.size() != 2) + return; - if (function->name == "in") + if (function_name == "in") { - const ASTIdentifier * ident = args.children.at(0)->as(); - if (!ident || ident->name() != "path") - return false; + if (!isPathNode(node.children.at(0))) + return; - ASTPtr value = args.children.at(1); + auto value = node.children.at(1)->column; + if (!value) + return; - if (value->as()) - { - auto interpreter_subquery = interpretSubquery(value, context, {}, {}); - auto pipeline = interpreter_subquery->execute().pipeline; - SizeLimits limites(context->getSettingsRef().max_rows_in_set, context->getSettingsRef().max_bytes_in_set, OverflowMode::THROW); - Set set(limites, true, context->getSettingsRef().transform_null_in); - set.setHeader(pipeline.getHeader().getColumnsWithTypeAndName()); + const IColumn * column = value.get(); + if (const auto * column_const = typeid_cast(column)) + column = &column_const->getDataColumn(); - PullingPipelineExecutor executor(pipeline); - Block block; - while (executor.pull(block)) - { - set.insertFromBlock(block.getColumnsWithTypeAndName()); - } - set.finishInsert(); + const ColumnSet * column_set = typeid_cast(column); + if (!column_set) + return; - set.checkColumnsNumber(1); - const auto & set_column = *set.getSetElements()[0]; - for (size_t row = 0; row < set_column.size(); ++row) - res.emplace_back(set_column[row].safeGet(), ZkPathType::Exact); - } - else - { - auto evaluated = evaluateConstantExpressionAsLiteral(value, context); - const auto * literal = evaluated->as(); - if (!literal) - return false; + auto set = column_set->getData(); + if (!set->isCreated()) + return; - if (String str; literal->value.tryGet(str)) - { - res.emplace_back(str, ZkPathType::Exact); - } - else if (Tuple tuple; literal->value.tryGet(tuple)) - { - for (auto element : tuple) - res.emplace_back(element.safeGet(), ZkPathType::Exact); - } - else - return false; - } + if (!set->hasExplicitSetElements()) + return; - return true; + set->checkColumnsNumber(1); + auto type = set->getElementsTypes()[0]; + if (!isString(removeNullable(removeLowCardinality(type)))) + return; + + auto values = set->getSetElements()[0]; + size_t size = values->size(); + + for (size_t row = 0; row < size; ++row) + res.emplace_back(values->getDataAt(row).toString(), ZkPathType::Exact); } - else if (function->name == "equals") + else if (function_name == "equals") { - const ASTIdentifier * ident; - ASTPtr value; - if ((ident = args.children.at(0)->as())) - value = args.children.at(1); - else if ((ident = args.children.at(1)->as())) - value = args.children.at(0); - else - return false; + if (!isPathNode(node.children.at(0))) + return; - if (ident->name() != "path") - return false; + auto value = node.children.at(1); + if (!value->column) + return; - auto evaluated = evaluateConstantExpressionAsLiteral(value, context); - const auto * literal = evaluated->as(); - if (!literal) - return false; + if (!isString(removeNullable(removeLowCardinality(value->result_type)))) + return; - if (literal->value.getType() != Field::Types::String) - return false; + if (value->column->size() != 1) + return; - res.emplace_back(literal->value.safeGet(), ZkPathType::Exact); - return true; + res.emplace_back(value->column->getDataAt(0).toString(), ZkPathType::Exact); } - else if (allow_unrestricted && function->name == "like") + else if (allow_unrestricted && function_name == "like") { - const ASTIdentifier * ident; - ASTPtr value; - if ((ident = args.children.at(0)->as())) - value = args.children.at(1); - else if ((ident = args.children.at(1)->as())) - value = args.children.at(0); - else - return false; + if (!isPathNode(node.children.at(0))) + return; - if (ident->name() != "path") - return false; + auto value = node.children.at(1); + if (!value->column) + return; - auto evaluated = evaluateConstantExpressionAsLiteral(value, context); - const auto * literal = evaluated->as(); - if (!literal) - return false; + if (!isString(removeNullable(removeLowCardinality(value->result_type)))) + return; - if (literal->value.getType() != Field::Types::String) - return false; + if (value->column->size() != 1) + return; - String pattern = literal->value.safeGet(); + String pattern = value->column->getDataAt(0).toString(); bool has_metasymbol = false; String prefix; // pattern prefix before the first metasymbol occurrence for (size_t i = 0; i < pattern.size(); i++) @@ -376,30 +392,28 @@ static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context, } res.emplace_back(prefix, has_metasymbol ? ZkPathType::Prefix : ZkPathType::Exact); - - return true; } - - return false; } /** Retrieve from the query a condition of the form `path = 'path'`, from conjunctions in the WHERE clause. */ -static Paths extractPath(const ASTPtr & query, ContextPtr context, bool allow_unrestricted) +static Paths extractPath(const ActionsDAG::NodeRawConstPtrs & filter_nodes, ContextPtr context, bool allow_unrestricted) { - const auto & select = query->as(); - if (!select.where()) - return allow_unrestricted ? Paths{{"/", ZkPathType::Recurse}} : Paths(); - Paths res; - return extractPathImpl(*select.where(), res, context, allow_unrestricted) ? res : Paths(); + for (const auto * node : filter_nodes) + extractPathImpl(*node, res, context, allow_unrestricted); + + if (filter_nodes.empty() && allow_unrestricted) + res.emplace_back("/", ZkPathType::Recurse); + + return res; } -void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const +void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns) const { - Paths paths = extractPath(query_info.query, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper); + Paths paths = extractPath(getFilterNodes().nodes, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper); zkutil::ZooKeeperPtr zookeeper = context->getZooKeeper(); @@ -486,5 +500,26 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr c } } +ReadFromSystemZooKeeper::ReadFromSystemZooKeeper(const Block & header, SelectQueryInfo & query_info, ContextPtr context_) + : SourceStepWithFilter({.header = std::move(header)}) + , storage_limits(query_info.storage_limits) + , context(std::move(context_)) +{ +} + +void ReadFromSystemZooKeeper::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + const auto & header = getOutputStream().header; + MutableColumns res_columns = header.cloneEmptyColumns(); + fillData(res_columns); + + UInt64 num_rows = res_columns.at(0)->size(); + Chunk chunk(std::move(res_columns), num_rows); + + auto source = std::make_shared(header, std::move(chunk)); + source->setStorageLimits(storage_limits); + processors.emplace_back(source); + pipeline.init(Pipe(std::move(source))); +} } diff --git a/src/Storages/System/StorageSystemZooKeeper.h b/src/Storages/System/StorageSystemZooKeeper.h index 20ad29af481..c8988d787a0 100644 --- a/src/Storages/System/StorageSystemZooKeeper.h +++ b/src/Storages/System/StorageSystemZooKeeper.h @@ -11,7 +11,7 @@ class Context; /** Implements `zookeeper` system table, which allows you to view the data in ZooKeeper for debugging purposes. */ -class StorageSystemZooKeeper final : public IStorageSystemOneBlock +class StorageSystemZooKeeper final : public IStorage { public: explicit StorageSystemZooKeeper(const StorageID & table_id_); @@ -22,10 +22,19 @@ public: SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override; -protected: - using IStorageSystemOneBlock::IStorageSystemOneBlock; + void read( + QueryPlan & query_plan, + const Names & /*column_names*/, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + size_t /*num_streams*/) override; - void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; + bool isSystemStorage() const override { return true; } + bool supportsIndexForIn() const override { return true; } + bool mayBenefitFromIndexForIn(const ASTPtr & node, ContextPtr, const StorageMetadataPtr &) const override; }; } From 54180851ab2fd62b3dc78d3cee09734cda4cfa9d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 6 Apr 2023 13:13:17 +0000 Subject: [PATCH 167/277] Fix name --- .../Optimizations/optimizeUseAggregateProjection.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 61e6988ded1..09c157a0283 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -140,15 +140,15 @@ std::optional matchAggregateFunctions( size_t num_args = aggregate.argument_names.size(); - DataTypes argumen_types; - argumen_types.reserve(num_args); + DataTypes argument_types; + argument_types.reserve(num_args); auto & candidates = it->second; bool found_match = false; for (size_t idx : candidates) { - argumen_types.clear(); + argument_types.clear(); const auto & candidate = info.aggregates[idx]; /// Note: this check is a bit strict. @@ -226,7 +226,7 @@ std::optional matchAggregateFunctions( break; } - argumen_types.push_back(query_node->result_type); + argument_types.push_back(query_node->result_type); ++next_arg; } @@ -234,7 +234,7 @@ std::optional matchAggregateFunctions( continue; found_match = true; - res.push_back({&candidate, std::move(argumen_types)}); + res.push_back({&candidate, std::move(argument_types)}); break; } From 2d335f82cc8ea3dc18f6d05778de1b19c63aec01 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 6 Apr 2023 15:43:16 +0200 Subject: [PATCH 168/277] Fix another test --- src/Storages/RabbitMQ/RabbitMQConsumer.cpp | 3 +++ src/Storages/RabbitMQ/RabbitMQConsumer.h | 2 ++ src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 3 +++ 3 files changed, 8 insertions(+) diff --git a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp index 835cf82b246..65063e004a5 100644 --- a/src/Storages/RabbitMQ/RabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.cpp @@ -38,7 +38,10 @@ void RabbitMQConsumer::shutdown() { stopped = true; cv.notify_one(); +} +void RabbitMQConsumer::closeConnections() +{ if (consumer_channel) consumer_channel->close(); } diff --git a/src/Storages/RabbitMQ/RabbitMQConsumer.h b/src/Storages/RabbitMQ/RabbitMQConsumer.h index 89e2b192c35..c7adb856212 100644 --- a/src/Storages/RabbitMQ/RabbitMQConsumer.h +++ b/src/Storages/RabbitMQ/RabbitMQConsumer.h @@ -84,6 +84,8 @@ public: cv.wait_for(lock, std::chrono::milliseconds(*timeout_ms), [this]{ return !received.empty() || isConsumerStopped(); }); } + void closeConnections(); + private: void subscribe(); void iterateEventLoop(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 0249e3203c6..7999d4af71a 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -837,6 +837,9 @@ void StorageRabbitMQ::shutdown() /// Just a paranoid try catch, it is not actually needed. try { + for (auto & consumer : consumers_ref) + consumer.lock()->closeConnections(); + if (drop_table) cleanupRabbitMQ(); From e91c8bfea3903d0cb4ab294721d9a598c9348e24 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 6 Apr 2023 14:43:52 +0200 Subject: [PATCH 169/277] Fix deadlock due to debug tracking of memory allocations Since memory tracker should be blocked globally (by default it is blocked only for VariableContext::User).
stacktrace * thread 11, name = 'Collector', stop reason = trace frame 1: 0x00007ffff7e37ea2 libc.so.6`pthread_mutex_lock + 274 frame 6: 0x000000001ee10298 clickhouse`toStringCached(pointers=0x00007fffedfef068, offset=0, size=45) + 88 at StackTrace.cpp:425 frame 7: 0x000000001ee1128e clickhouse`StackTrace::toString(this=0x00007fffedfef058) const + 46 at StackTrace.cpp:444 frame 8: 0x000000001ede58ff clickhouse`(anonymous namespace)::debugLogBigAllocationWithoutCheck(size=25187648) + 479 at MemoryTracker.cpp:98 frame 9: 0x000000001ede5481 clickhouse`MemoryTracker::allocImpl(this=0x0000000030729e68, size=25187648, throw_if_memory_exceeded=false, query_tracker=0x00007ffff3c22440) + 3937 at MemoryTracker.cpp:352 frame 10: 0x000000001ede477f clickhouse`MemoryTracker::allocImpl(this=0x00007ffff3c1e530, size=25187648, throw_if_memory_exceeded=false, query_tracker=0x00007ffff3c22440) + 607 at MemoryTracker.cpp:198 frame 11: 0x000000001ede477f clickhouse`MemoryTracker::allocImpl(this=0x00007ffff3c22440, size=25187648, throw_if_memory_exceeded=false, query_tracker=0x0000000000000000) + 607 at MemoryTracker.cpp:198 frame 12: 0x000000001ede477f clickhouse`MemoryTracker::allocImpl(this=0x00007fffedff3bf0, size=25187648, throw_if_memory_exceeded=false, query_tracker=0x0000000000000000) + 607 at MemoryTracker.cpp:198 frame 13: 0x000000001ed6c4c9 clickhouse`CurrentMemoryTracker::allocImpl(size=25165824, throw_if_memory_exceeded=false) + 265 at CurrentMemoryTracker.cpp:58 frame 14: 0x000000001ed6c6a1 clickhouse`CurrentMemoryTracker::allocNoThrow(size=25165824) + 33 at CurrentMemoryTracker.cpp:91 frame 15: 0x000000001ed63079 clickhouse`operator new(unsigned long) [inlined] void Memory::trackMemory<>(size=25165824) + 89 at memory.h:177 frame 16: 0x000000001ed63034 clickhouse`operator new(size=25165824) + 20 at new_delete.cpp:74 ... frame 27: 0x000000001ee15895 clickhouse`DB::(anonymous namespace)::collectSymbols(info=0x00007ffff7062000, (null)=, data_ptr=0x00007fffec2b7000) + 53 at SymbolIndex.cpp:483 frame 28: 0x000000002e1a5d57 clickhouse`::dl_iterate_phdr(callback=(clickhouse`DB::(anonymous namespace)::collectSymbols(dl_phdr_info*, unsigned long, void*) at SymbolIndex.cpp:480), data=0x00007fffec2b7000)(dl_phdr_info *, size_t, void *), void *) + 183 at phdr_cache.cpp:76 frame 29: 0x000000001ee15549 clickhouse`DB::SymbolIndex::update(this=0x00007fffec2b7000) + 41 at SymbolIndex.cpp:515 frame 30: 0x000000001ee1be35 clickhouse`DB::SymbolIndex::SymbolIndex(this=) + 53 at SymbolIndex.h:23 frame 31: 0x000000001ee17a6c clickhouse`DB::SymbolIndex::instanceImpl() + 108 at SymbolIndex.cpp:555 frame 32: 0x000000001ee17b4a clickhouse`DB::SymbolIndex::instance() + 10 at SymbolIndex.cpp:561 frame 33: 0x000000001ee1070f clickhouse`void toStringEveryLineImpl const&, unsigned long, unsigned long)::$_0>(fatal=false, stack_trace=0x00007fffedff1ef8, callback=0x00007fffedff1ee0)::$_0&&) + 207 at StackTrace.cpp:349 frame 34: 0x000000001ee1040b clickhouse`toStringCached(pointers=0x00007fffedff2928, offset=0, size=43) + 459 at StackTrace.cpp:436 frame 35: 0x000000001ee1128e clickhouse`StackTrace::toString(this=0x00007fffedff2918) const + 46 at StackTrace.cpp:444 frame 36: 0x000000001ede58ff clickhouse`(anonymous namespace)::debugLogBigAllocationWithoutCheck(size=25165824) + 479 at MemoryTracker.cpp:98 frame 37: 0x000000001ede5481 clickhouse`MemoryTracker::allocImpl(this=0x00007fffedff3bf0, size=25165824, throw_if_memory_exceeded=false, query_tracker=0x0000000000000000) + 3937 at MemoryTracker.cpp:352 frame 38: 0x000000001ed6c4c9 clickhouse`CurrentMemoryTracker::allocImpl(size=25165824, throw_if_memory_exceeded=false) + 265 at CurrentMemoryTracker.cpp:58 frame 39: 0x000000001ed6c6a1 clickhouse`CurrentMemoryTracker::allocNoThrow(size=25165824) + 33 at CurrentMemoryTracker.cpp:91 frame 40: 0x000000001ed63079 clickhouse`operator new(unsigned long) [inlined] void Memory::trackMemory<>(size=25165824) + 89 at memory.h:177 frame 41: 0x000000001ed63034 clickhouse`operator new(size=25165824) + 20 at new_delete.cpp:74 ... frame 55: 0x0000000025badace clickhouse`DB::WriteBufferFromS3::nextImpl(this=0x00007ffff3c58f00) + 366 at WriteBufferFromS3.cpp:110
Signed-off-by: Azat Khuzhin --- src/Common/MemoryTracker.cpp | 40 ++++++++++++++++++------------------ src/Common/MemoryTracker.h | 2 ++ 2 files changed, 22 insertions(+), 20 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index ca5c4a745cd..674d8d469af 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -82,26 +82,6 @@ inline std::string_view toDescription(OvercommitResult result) } } -inline void debugLogBigAllocationWithoutCheck(Int64 size [[maybe_unused]]) -{ - /// Big allocations through allocNoThrow (without checking memory limits) may easily lead to OOM (and it's hard to debug). - /// Let's find them. -#ifdef ABORT_ON_LOGICAL_ERROR - if (size < 0) - return; - - constexpr Int64 threshold = 16 * 1024 * 1024; /// The choice is arbitrary (maybe we should decrease it) - if (size < threshold) - return; - - MemoryTrackerBlockerInThread blocker; - LOG_TEST(&Poco::Logger::get("MemoryTracker"), "Too big allocation ({} bytes) without checking memory limits, " - "it may lead to OOM. Stack trace: {}", size, StackTrace().toString()); -#else - return; /// Avoid trash logging in release builds -#endif -} - } namespace ProfileEvents @@ -175,6 +155,26 @@ void MemoryTracker::injectFault() const description ? description : ""); } +void MemoryTracker::debugLogBigAllocationWithoutCheck(Int64 size [[maybe_unused]]) +{ + /// Big allocations through allocNoThrow (without checking memory limits) may easily lead to OOM (and it's hard to debug). + /// Let's find them. +#ifdef ABORT_ON_LOGICAL_ERROR + if (size < 0) + return; + + constexpr Int64 threshold = 16 * 1024 * 1024; /// The choice is arbitrary (maybe we should decrease it) + if (size < threshold) + return; + + MemoryTrackerBlockerInThread blocker(VariableContext::Global); + LOG_TEST(&Poco::Logger::get("MemoryTracker"), "Too big allocation ({} bytes) without checking memory limits, " + "it may lead to OOM. Stack trace: {}", size, StackTrace().toString()); +#else + return; /// Avoid trash logging in release builds +#endif +} + void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker) { if (size < 0) diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 66b56730b75..0d7748856bd 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -215,6 +215,8 @@ public: /// Prints info about peak memory consumption into log. void logPeakMemoryUsage(); + + void debugLogBigAllocationWithoutCheck(Int64 size [[maybe_unused]]); }; extern MemoryTracker total_memory_tracker; From cfd9c4d85e26c2a77280582bcc5405a5c9d3d9cb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Apr 2023 14:01:05 +0000 Subject: [PATCH 170/277] Register aliases of date_diff in system.functions --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- src/Functions/dateDiff.cpp | 5 +++++ .../queries/0_stateless/25342_date_diff_aliases.reference | 5 +++++ tests/queries/0_stateless/25342_date_diff_aliases.sql | 7 +++++++ 4 files changed, 18 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/25342_date_diff_aliases.reference create mode 100644 tests/queries/0_stateless/25342_date_diff_aliases.sql diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 74ef9a28dc1..42a16b7185f 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -645,7 +645,7 @@ For an alternative to `date\_diff`, see function `age`. date_diff('unit', startdate, enddate, [timezone]) ``` -Aliases: `dateDiff`, `DATE_DIFF`. +Aliases: `dateDiff`, `DATE_DIFF`, `timestampDiff`, `timestamp_diff`, `TIMESTAMP_DIFF`. **Arguments** diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index 457b77b9843..b28c97d45b2 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -448,6 +448,11 @@ private: REGISTER_FUNCTION(DateDiff) { factory.registerFunction>({}, FunctionFactory::CaseInsensitive); + factory.registerAlias("date_diff", FunctionDateDiff::name); + factory.registerAlias("DATE_DIFF", FunctionDateDiff::name); + factory.registerAlias("timestampDiff", FunctionDateDiff::name); + factory.registerAlias("timestamp_diff", FunctionDateDiff::name); + factory.registerAlias("TIMESTAMP_DIFF", FunctionDateDiff::name); } REGISTER_FUNCTION(TimeDiff) diff --git a/tests/queries/0_stateless/25342_date_diff_aliases.reference b/tests/queries/0_stateless/25342_date_diff_aliases.reference new file mode 100644 index 00000000000..1eeb5a3a2fa --- /dev/null +++ b/tests/queries/0_stateless/25342_date_diff_aliases.reference @@ -0,0 +1,5 @@ +DATE_DIFF +TIMESTAMP_DIFF +date_diff +timestampDiff +timestamp_diff diff --git a/tests/queries/0_stateless/25342_date_diff_aliases.sql b/tests/queries/0_stateless/25342_date_diff_aliases.sql new file mode 100644 index 00000000000..c6b31c44f95 --- /dev/null +++ b/tests/queries/0_stateless/25342_date_diff_aliases.sql @@ -0,0 +1,7 @@ +SELECT name FROM system.functions +WHERE name = 'date_diff' + OR name = 'DATE_DIFF' + OR name = 'timestampDiff' + OR name = 'timestamp_diff' + OR name = 'TIMESTAMP_DIFF' +ORDER BY name; From e88938d6c8f21b3fe4a3e5d3cf2bbc0c18248d83 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 6 Apr 2023 16:17:45 +0200 Subject: [PATCH 171/277] Change error code. --- src/DataTypes/ObjectUtils.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index edda0235bcc..28f000b6f0d 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -30,6 +30,7 @@ namespace ErrorCodes extern const int TYPE_MISMATCH; extern const int LOGICAL_ERROR; extern const int INCOMPATIBLE_COLUMNS; + extern const int NOT_IMPLEMENTED; } size_t getNumberOfDimensions(const IDataType & type) @@ -121,7 +122,7 @@ DataTypePtr getDataTypeByColumn(const IColumn & column) return makeNullable(getDataTypeByColumn(column_nullable->getNestedColumn())); /// TODO: add more types. - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get data type of column {}", column.getFamilyName()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get data type of column {}", column.getFamilyName()); } template From 7bcb8c4970c37a4f6b5001454636effe7684d4f3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Apr 2023 14:25:27 +0000 Subject: [PATCH 172/277] Register trim aliases in system.functions --- src/Functions/trim.cpp | 3 +++ tests/queries/0_stateless/25343_trim_aliases.reference | 3 +++ tests/queries/0_stateless/25343_trim_aliases.sql | 5 +++++ 3 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/25343_trim_aliases.reference create mode 100644 tests/queries/0_stateless/25343_trim_aliases.sql diff --git a/src/Functions/trim.cpp b/src/Functions/trim.cpp index acfab47a68b..dd51c606ff7 100644 --- a/src/Functions/trim.cpp +++ b/src/Functions/trim.cpp @@ -112,5 +112,8 @@ REGISTER_FUNCTION(Trim) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerAlias("ltrim", FunctionTrimLeft::name); + factory.registerAlias("rtrim", FunctionTrimRight::name); + factory.registerAlias("trim", FunctionTrimBoth::name); } } diff --git a/tests/queries/0_stateless/25343_trim_aliases.reference b/tests/queries/0_stateless/25343_trim_aliases.reference new file mode 100644 index 00000000000..fa0920cd079 --- /dev/null +++ b/tests/queries/0_stateless/25343_trim_aliases.reference @@ -0,0 +1,3 @@ +ltrim +rtrim +trim diff --git a/tests/queries/0_stateless/25343_trim_aliases.sql b/tests/queries/0_stateless/25343_trim_aliases.sql new file mode 100644 index 00000000000..d0d739805fd --- /dev/null +++ b/tests/queries/0_stateless/25343_trim_aliases.sql @@ -0,0 +1,5 @@ +SELECT name FROM system.functions +WHERE name = 'ltrim' + OR name = 'rtrim' + OR name = 'trim' +ORDER BY name; From 851fdf7f4691eeb399ddcc5edbde70ffe9a79c4c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 6 Apr 2023 18:55:36 +0300 Subject: [PATCH 173/277] Update 00002_log_and_exception_messages_formatting.sql --- .../0_stateless/00002_log_and_exception_messages_formatting.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 0638c50ec69..480effec065 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -59,7 +59,7 @@ select 'messages shorter than 10', max2(countDistinctOrDefault(message_format_st select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_string), 3) from logs where length(message_format_string) < 16 and message_format_string not in known_short_messages; -- Same as above, but exceptions must be more informative. Feel free to update the threshold or remove this query if really necessary -select 'exceptions shorter than 30', max2(countDistinctOrDefault(message_format_string), 27) from logs where length(message_format_string) < 30 and message ilike '%DB::Exception%' and message_format_string not in known_short_messages; +select 'exceptions shorter than 30', max2(countDistinctOrDefault(message_format_string), 30) from logs where length(message_format_string) < 30 and message ilike '%DB::Exception%' and message_format_string not in known_short_messages; -- Avoid too noisy messages: top 1 message frequency must be less than 30%. We should reduce the threshold From 8e1d9939f78b2b91b65a1ffd616e6db7ef571d18 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 6 Apr 2023 18:56:10 +0300 Subject: [PATCH 174/277] Update 00002_log_and_exception_messages_formatting.reference --- .../00002_log_and_exception_messages_formatting.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference index 1e7b85d6489..cd9da983785 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference @@ -2,7 +2,7 @@ runtime messages 0.001 runtime exceptions 0.05 messages shorter than 10 1 messages shorter than 16 3 -exceptions shorter than 30 27 +exceptions shorter than 30 30 noisy messages 0.3 noisy Trace messages 0.16 noisy Debug messages 0.09 From 4757d76fbf398f4ba6985ce07e2782f5a8abebf9 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 6 Apr 2023 15:59:13 +0000 Subject: [PATCH 175/277] fix query cache with sparse columns --- src/Interpreters/Cache/QueryCache.cpp | 3 ++- ...02708_query_cache_sparse_columns.reference | 1 + .../02708_query_cache_sparse_columns.sql | 23 +++++++++++++++++++ 3 files changed, 26 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02708_query_cache_sparse_columns.reference create mode 100644 tests/queries/0_stateless/02708_query_cache_sparse_columns.sql diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index ba5388847f5..fede871aba3 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -242,8 +242,9 @@ void QueryCache::Writer::finalizeWrite() Chunks squashed_chunks; size_t rows_remaining_in_squashed = 0; /// how many further rows can the last squashed chunk consume until it reaches max_block_size - for (const auto & chunk : *query_result) + for (auto & chunk : *query_result) { + convertToFullIfSparse(chunk); const size_t rows_chunk = chunk.getNumRows(); size_t rows_chunk_processed = 0; diff --git a/tests/queries/0_stateless/02708_query_cache_sparse_columns.reference b/tests/queries/0_stateless/02708_query_cache_sparse_columns.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02708_query_cache_sparse_columns.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02708_query_cache_sparse_columns.sql b/tests/queries/0_stateless/02708_query_cache_sparse_columns.sql new file mode 100644 index 00000000000..952c3227852 --- /dev/null +++ b/tests/queries/0_stateless/02708_query_cache_sparse_columns.sql @@ -0,0 +1,23 @@ +-- Tags: no-parallel + +DROP TABLE IF EXISTS t_cache_sparse; +SYSTEM DROP QUERY CACHE; + +CREATE TABLE t_cache_sparse (id UInt64, v UInt64) +ENGINE = MergeTree ORDER BY id +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.9; + +SYSTEM STOP MERGES t_cache_sparse; + +INSERT INTO t_cache_sparse SELECT number, number FROM numbers(10000); +INSERT INTO t_cache_sparse SELECT number, 0 FROM numbers(10000); + +SET allow_experimental_query_cache = 1; +SET use_query_cache = 1; +SET max_threads = 1; + +SELECT v FROM t_cache_sparse FORMAT Null; +SELECT v FROM t_cache_sparse FORMAT Null; +SELECT count() FROM system.query_cache WHERE query LIKE 'SELECT v FROM t_cache_sparse%'; + +DROP TABLE t_cache_sparse; From fae4906e112b12d67636028df13983e6a609915f Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 6 Apr 2023 12:07:00 -0400 Subject: [PATCH 176/277] Update docs/en/sql-reference/statements/undrop.md Co-authored-by: Alexander Tokmakov --- docs/en/sql-reference/statements/undrop.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/undrop.md b/docs/en/sql-reference/statements/undrop.md index e56829e7151..89208b3aaba 100644 --- a/docs/en/sql-reference/statements/undrop.md +++ b/docs/en/sql-reference/statements/undrop.md @@ -8,7 +8,7 @@ sidebar_label: UNDROP Cancels the dropping of the table. Beginning with ClickHouse version 23.3 it is possible to UNDROP a table -within 8 minutes of issuing the DROP TABLE statement. Dropped tables are listed in +within `database_atomic_delay_before_drop_table_sec` (8 minutes by default) of issuing the DROP TABLE statement. Dropped tables are listed in a system table called `system.dropped_tables`. :::note From 28f86e7b841ec531f279c8c5d4943fcd118b19be Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 6 Apr 2023 12:07:53 -0400 Subject: [PATCH 177/277] Update docs/en/sql-reference/statements/undrop.md Co-authored-by: Alexander Tokmakov --- docs/en/sql-reference/statements/undrop.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/undrop.md b/docs/en/sql-reference/statements/undrop.md index 89208b3aaba..7f42d92d923 100644 --- a/docs/en/sql-reference/statements/undrop.md +++ b/docs/en/sql-reference/statements/undrop.md @@ -25,7 +25,7 @@ Also see [DROP TABLE](/docs/en/sql-reference/statements/drop.md) Syntax: ``` sql -UNDROP [TEMPORARY] TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] [SYNC] +UNDROP TABLE [db.]name [UUID ''] [ON CLUSTER cluster] ``` **Example** From 66af848f8250a636a1b37b3ef614d20ddebf81ad Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 6 Apr 2023 12:09:34 -0400 Subject: [PATCH 178/277] Update docs/en/sql-reference/statements/undrop.md --- docs/en/sql-reference/statements/undrop.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/undrop.md b/docs/en/sql-reference/statements/undrop.md index 7f42d92d923..0822268f908 100644 --- a/docs/en/sql-reference/statements/undrop.md +++ b/docs/en/sql-reference/statements/undrop.md @@ -7,7 +7,7 @@ sidebar_label: UNDROP Cancels the dropping of the table. -Beginning with ClickHouse version 23.3 it is possible to UNDROP a table +Beginning with ClickHouse version 23.3 it is possible to UNDROP a table in an Atomic database within `database_atomic_delay_before_drop_table_sec` (8 minutes by default) of issuing the DROP TABLE statement. Dropped tables are listed in a system table called `system.dropped_tables`. From cbda8762b2a4a29debe9aadb9239c59961fb36c5 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 6 Apr 2023 12:18:09 -0400 Subject: [PATCH 179/277] add note about MV --- docs/en/sql-reference/statements/undrop.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/statements/undrop.md b/docs/en/sql-reference/statements/undrop.md index 0822268f908..40ac1ab4f99 100644 --- a/docs/en/sql-reference/statements/undrop.md +++ b/docs/en/sql-reference/statements/undrop.md @@ -11,6 +11,8 @@ Beginning with ClickHouse version 23.3 it is possible to UNDROP a table in an At within `database_atomic_delay_before_drop_table_sec` (8 minutes by default) of issuing the DROP TABLE statement. Dropped tables are listed in a system table called `system.dropped_tables`. +If you have a materialized view without a `TO` clause associated with the dropped table, then you will also have to UNDROP the inner table of that view. + :::note UNDROP TABLE is experimental. To use it add this setting: ```sql From b6975d36e9e1eefbe9d8722d1bea6837a74e8e86 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 6 Apr 2023 19:05:32 +0200 Subject: [PATCH 180/277] Fix bytesSize() of zk SetRequest --- src/Common/ZooKeeper/IKeeper.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index a94e367cd70..172714fe04f 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -273,7 +273,7 @@ struct SetRequest : virtual Request void addRootPath(const String & root_path) override; String getPath() const override { return path; } - size_t bytesSize() const override { return data.size() + data.size() + sizeof(version); } + size_t bytesSize() const override { return path.size() + data.size() + sizeof(version); } }; struct SetResponse : virtual Response From 4544abc7d6ae98d06b8fc11b1274fe949362641d Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 6 Apr 2023 11:37:12 -0700 Subject: [PATCH 181/277] Remove dead code and unused dependencies --- src/Bridge/CMakeLists.txt | 2 +- src/Bridge/IBridge.cpp | 5 ----- src/Dictionaries/CMakeLists.txt | 1 - 3 files changed, 1 insertion(+), 7 deletions(-) diff --git a/src/Bridge/CMakeLists.txt b/src/Bridge/CMakeLists.txt index daf38bd6cbc..5f0e97fc630 100644 --- a/src/Bridge/CMakeLists.txt +++ b/src/Bridge/CMakeLists.txt @@ -2,4 +2,4 @@ add_library (bridge IBridge.cpp ) -target_link_libraries (bridge PRIVATE daemon dbms Poco::Data Poco::Data::ODBC) +target_link_libraries (bridge PRIVATE daemon dbms) diff --git a/src/Bridge/IBridge.cpp b/src/Bridge/IBridge.cpp index 1ea77573e5f..0f0efbd5eed 100644 --- a/src/Bridge/IBridge.cpp +++ b/src/Bridge/IBridge.cpp @@ -20,11 +20,6 @@ #include "config.h" -#if USE_ODBC -# include -#endif - - namespace DB { diff --git a/src/Dictionaries/CMakeLists.txt b/src/Dictionaries/CMakeLists.txt index 0260804ab5b..c9dd554a6f1 100644 --- a/src/Dictionaries/CMakeLists.txt +++ b/src/Dictionaries/CMakeLists.txt @@ -26,7 +26,6 @@ target_link_libraries(clickhouse_dictionaries clickhouse_common_io dbms Poco::Data - Poco::Data::ODBC Poco::MongoDB Poco::Redis string_utils From 00c8e6e60d6f69ec7c9abd14f83dfad4711ce4a4 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 6 Apr 2023 11:39:33 -0700 Subject: [PATCH 182/277] Use std::string_view instead of strlen --- src/Functions/extractTextFromHTML.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Functions/extractTextFromHTML.cpp b/src/Functions/extractTextFromHTML.cpp index a15611579bb..c7ef97c7bee 100644 --- a/src/Functions/extractTextFromHTML.cpp +++ b/src/Functions/extractTextFromHTML.cpp @@ -70,16 +70,16 @@ namespace ErrorCodes namespace { -inline bool startsWith(const char * s, const char * end, const char * prefix) +bool startsWith(const char * s, const char * end, const std::string_view prefix) { - return s + strlen(prefix) < end && 0 == memcmp(s, prefix, strlen(prefix)); + return s + prefix.length() < end && 0 == memcmp(s, prefix.data(), prefix.length()); } -inline bool checkAndSkip(const char * __restrict & s, const char * end, const char * prefix) +inline bool checkAndSkip(const char * __restrict & s, const char * end, const std::string_view prefix) { if (startsWith(s, end, prefix)) { - s += strlen(prefix); + s += prefix.length(); return true; } return false; @@ -138,7 +138,7 @@ bool processCDATA(const char * __restrict & src, const char * end, char * __rest return true; } -bool processElementAndSkipContent(const char * __restrict & src, const char * end, const char * tag_name) +bool processElementAndSkipContent(const char * __restrict & src, const char * end, const std::string_view tag_name) { const auto * old_src = src; From 790e570af6c37507d21527c0b5ae0f67d8e474b8 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 6 Apr 2023 11:57:44 -0700 Subject: [PATCH 183/277] Use std::string::starts_with instead of a roll your own variant --- src/Common/mysqlxx/PoolFactory.cpp | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/src/Common/mysqlxx/PoolFactory.cpp b/src/Common/mysqlxx/PoolFactory.cpp index 5fae934a400..9479273e36e 100644 --- a/src/Common/mysqlxx/PoolFactory.cpp +++ b/src/Common/mysqlxx/PoolFactory.cpp @@ -23,12 +23,6 @@ PoolWithFailover PoolFactory::get(const std::string & config_name, unsigned defa return get(Poco::Util::Application::instance().config(), config_name, default_connections, max_connections, max_tries); } -/// Duplicate of code from StringUtils.h. Copied here for less dependencies. -static bool startsWith(const std::string & s, const char * prefix) -{ - return s.size() >= strlen(prefix) && 0 == memcmp(s.data(), prefix, strlen(prefix)); -} - static std::string getPoolEntryName(const Poco::Util::AbstractConfiguration & config, const std::string & config_name) { @@ -55,7 +49,7 @@ static std::string getPoolEntryName(const Poco::Util::AbstractConfiguration & co for (const auto & replica_config_key : replica_keys) { /// There could be another elements in the same level in configuration file, like "user", "port"... - if (startsWith(replica_config_key, "replica")) + if (replica_config_key.starts_with("replica")) { std::string replica_name = config_name + "." + replica_config_key; std::string tmp_host = config.getString(replica_name + ".host", host); From e8fb13518018e7dad2cc1881375c8578c158a62e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 6 Apr 2023 19:07:53 +0000 Subject: [PATCH 184/277] Reduce memory usage for multiple alter delete mutations. --- src/Interpreters/MutationsInterpreter.cpp | 8 ++- .../02125_many_mutations_2.reference | 4 ++ .../0_stateless/02125_many_mutations_2.sh | 51 +++++++++++++++++++ 3 files changed, 62 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02125_many_mutations_2.reference create mode 100755 tests/queries/0_stateless/02125_many_mutations_2.sh diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 0b52a1a51bc..26442f94007 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -38,6 +38,7 @@ #include #include #include +#include namespace DB @@ -965,10 +966,15 @@ void MutationsInterpreter::prepareMutationStages(std::vector & prepared_s ExpressionActionsChain & actions_chain = stage.expressions_chain; - for (const auto & ast : stage.filters) + if (!stage.filters.empty()) { + auto ast = stage.filters.front(); + if (stage.filters.size() > 1) + ast = makeASTForLogicalAnd(std::move(stage.filters)); + if (!actions_chain.steps.empty()) actions_chain.addStep(); + stage.analyzer->appendExpression(actions_chain, ast, dry_run); stage.filter_column_names.push_back(ast->getColumnName()); } diff --git a/tests/queries/0_stateless/02125_many_mutations_2.reference b/tests/queries/0_stateless/02125_many_mutations_2.reference new file mode 100644 index 00000000000..4bdea51dfc1 --- /dev/null +++ b/tests/queries/0_stateless/02125_many_mutations_2.reference @@ -0,0 +1,4 @@ +2000 +20000 +0 +1000 diff --git a/tests/queries/0_stateless/02125_many_mutations_2.sh b/tests/queries/0_stateless/02125_many_mutations_2.sh new file mode 100755 index 00000000000..df170a402c6 --- /dev/null +++ b/tests/queries/0_stateless/02125_many_mutations_2.sh @@ -0,0 +1,51 @@ +#!/usr/bin/env bash +# Tags: long, no-tsan, no-debug, no-asan, no-msan, no-ubsan + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "create table many_mutations (x UInt32, y UInt32) engine = MergeTree order by x" +$CLICKHOUSE_CLIENT -q "insert into many_mutations select number, number + 1 from numbers(2000)" +$CLICKHOUSE_CLIENT -q "system stop merges many_mutations" + +$CLICKHOUSE_CLIENT -q "select count() from many_mutations" + +job() +{ + for i in {1..1000} + do + $CLICKHOUSE_CLIENT -q "alter table many_mutations delete where y = ${i} * 2 settings mutations_sync=0" + done +} + +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & + +wait + +$CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done" +$CLICKHOUSE_CLIENT -q "system start merges many_mutations" +$CLICKHOUSE_CLIENT -q "optimize table many_mutations final" +$CLICKHOUSE_CLIENT -q "system flush logs" +$CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done" +$CLICKHOUSE_CLIENT -q "select count() from many_mutations" +$CLICKHOUSE_CLIENT -q "select * from system.part_log where database = currentDatabase() and table == 'many_mutations' and peak_memory_usage > 1e9" From 57255eb56c075c129cbedcc76082c3642560d737 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 6 Apr 2023 19:22:10 +0000 Subject: [PATCH 185/277] Fix equals. --- src/Storages/System/StorageSystemZooKeeper.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index ee33253e40f..c96c8ec60ae 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -333,11 +333,14 @@ static void extractPathImpl(const ActionsDAG::Node & node, Paths & res, ContextP } else if (function_name == "equals") { - if (!isPathNode(node.children.at(0))) - return; + const ActionsDAG::Node * value = nullptr; - auto value = node.children.at(1); - if (!value->column) + if (isPathNode(node.children.at(0))) + value = node.children.at(1); + else if (isPathNode(node.children.at(1))) + value = node.children.at(0); + + if (!value || !value->column) return; if (!isString(removeNullable(removeLowCardinality(value->result_type)))) From b347464496c04f52218fa5fc6b56229c4da5e381 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Thu, 6 Apr 2023 13:01:34 -0700 Subject: [PATCH 186/277] Update ClickHouse logo --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 61d840ecd34..cd10c29b386 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,4 @@ -[![ClickHouse — open source distributed column-oriented DBMS](https://github.com/ClickHouse/clickhouse-presentations/raw/master/images/logo-400x240.png)](https://clickhouse.com) +[![ClickHouse — open source distributed column-oriented DBMS](https://clickhouse.com/images/clickhouse_gh_logo_400.png)](https://clickhouse.com) ClickHouse® is an open-source column-oriented database management system that allows generating analytical data reports in real-time. From 4449e49e725c0a27b9059d3844f10cd8f9109da4 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 6 Apr 2023 13:28:30 -0700 Subject: [PATCH 187/277] Mark a function inline --- src/Functions/extractTextFromHTML.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/extractTextFromHTML.cpp b/src/Functions/extractTextFromHTML.cpp index c7ef97c7bee..4eefeaa9f86 100644 --- a/src/Functions/extractTextFromHTML.cpp +++ b/src/Functions/extractTextFromHTML.cpp @@ -70,7 +70,7 @@ namespace ErrorCodes namespace { -bool startsWith(const char * s, const char * end, const std::string_view prefix) +inline bool startsWith(const char * s, const char * end, const std::string_view prefix) { return s + prefix.length() < end && 0 == memcmp(s, prefix.data(), prefix.length()); } From 09e8a321fc361df604818104e12eccea9be3869f Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Thu, 6 Apr 2023 14:44:29 -0700 Subject: [PATCH 188/277] Change ClickHouse logo to dark mode --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index cd10c29b386..cee3a945262 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,4 @@ -[![ClickHouse — open source distributed column-oriented DBMS](https://clickhouse.com/images/clickhouse_gh_logo_400.png)](https://clickhouse.com) +[ClickHouse — open source distributed column-oriented DBMS](https://clickhouse.com?utm_source=github) ClickHouse® is an open-source column-oriented database management system that allows generating analytical data reports in real-time. From 2e139c21d25f883626e1dbd2a5b8202fabe9297e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 6 Apr 2023 21:57:03 +0000 Subject: [PATCH 189/277] Parallel reading in FROM file() --- src/Processors/ResizeProcessor.h | 2 +- src/Storages/StorageFile.cpp | 15 ++++++++++++--- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Processors/ResizeProcessor.h b/src/Processors/ResizeProcessor.h index 07d7149ebb4..766c39172a2 100644 --- a/src/Processors/ResizeProcessor.h +++ b/src/Processors/ResizeProcessor.h @@ -10,7 +10,7 @@ namespace DB /** Has arbitrary non zero number of inputs and arbitrary non zero number of outputs. * All of them have the same structure. * - * Pulls data from arbitrary input (whenever it is ready) and pushes it to arbitrary output (whenever is is not full). + * Pulls data from arbitrary input (whenever it is ready) and pushes it to arbitrary output (whenever it is not full). * Doesn't do any heavy calculations. * Doesn't preserve an order of data. * diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 5fd5664b9e6..84f030ce90e 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -40,6 +40,7 @@ #include #include #include +#include "Processors/ResizeProcessor.h" #include #include @@ -700,7 +701,7 @@ Pipe StorageFile::read( ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - size_t num_streams) + const size_t max_num_streams) { if (use_table_fd) { @@ -731,7 +732,8 @@ Pipe StorageFile::read( auto this_ptr = std::static_pointer_cast(shared_from_this()); - if (num_streams > paths.size()) + size_t num_streams = max_num_streams; + if (max_num_streams > paths.size()) num_streams = paths.size(); Pipes pipes; @@ -789,7 +791,14 @@ Pipe StorageFile::read( std::move(read_buffer))); } - return Pipe::unitePipes(std::move(pipes)); + Pipe pipe = Pipe::unitePipes(std::move(pipes)); + /// parallelize output as much as possible + if (num_streams < max_num_streams) + { + pipe.addTransform(std::make_shared(pipe.getHeader(), num_streams, max_num_streams)); + } + + return pipe; } From 96213fa464f31a5d70683f438e2af5442533b742 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 6 Apr 2023 22:17:09 +0000 Subject: [PATCH 190/277] Fix header --- src/Storages/StorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 84f030ce90e..f8b5ea0ced7 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -34,13 +34,13 @@ #include #include #include +#include #include #include #include #include #include -#include "Processors/ResizeProcessor.h" #include #include From 343a07179892efddc48488d6b211dad0fed866a8 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 6 Apr 2023 22:38:58 +0000 Subject: [PATCH 191/277] add `lost_part_count` column to `system.replicas` --- .../MergeTree/ReplicatedTableStatus.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 24 ++++++++++++++++++- src/Storages/System/StorageSystemReplicas.cpp | 3 +++ .../02117_show_create_table_system.reference | 1 + 4 files changed, 28 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedTableStatus.h b/src/Storages/MergeTree/ReplicatedTableStatus.h index b9f84091e9b..46e971f562a 100644 --- a/src/Storages/MergeTree/ReplicatedTableStatus.h +++ b/src/Storages/MergeTree/ReplicatedTableStatus.h @@ -25,6 +25,7 @@ struct ReplicatedTableStatus UInt64 absolute_delay; UInt8 total_replicas; UInt8 active_replicas; + UInt64 lost_part_count; String last_queue_update_exception; /// If the error has happened fetching the info from ZooKeeper, this field will be set. String zookeeper_exception; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index cbfe3f8cab2..600168d2637 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -652,6 +652,8 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes() futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/alter_partition_version", String(), zkutil::CreateMode::Persistent)); /// For deduplication of async inserts futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/async_blocks", String(), zkutil::CreateMode::Persistent)); + /// To track "lost forever" parts count, just for `system.replicas` table + futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/lost_part_count", String(), zkutil::CreateMode::Persistent)); /// As for now, "/temp" node must exist, but we want to be able to remove it in future if (zookeeper->exists(zookeeper_path + "/temp")) @@ -5960,6 +5962,7 @@ void StorageReplicatedMergeTree::getStatus(ReplicatedTableStatus & res, bool wit res.log_pointer = 0; res.total_replicas = 0; res.active_replicas = 0; + res.lost_part_count = 0; res.last_queue_update_exception = getLastQueueUpdateException(); if (with_zk_fields && !res.is_session_expired) @@ -5976,6 +5979,7 @@ void StorageReplicatedMergeTree::getStatus(ReplicatedTableStatus & res, bool wit paths.clear(); paths.push_back(fs::path(replica_path) / "log_pointer"); + paths.push_back(fs::path(zookeeper_path) / "lost_part_count"); for (const String & replica : all_replicas) paths.push_back(fs::path(zookeeper_path) / "replicas" / replica / "is_active"); @@ -5993,10 +5997,14 @@ void StorageReplicatedMergeTree::getStatus(ReplicatedTableStatus & res, bool wit res.log_pointer = log_pointer_str.empty() ? 0 : parse(log_pointer_str); res.total_replicas = all_replicas.size(); + if (get_result[1].error == Coordination::Error::ZNONODE) + res.lost_part_count = 0; + else + res.lost_part_count = get_result[1].data.empty() ? 0 : parse(get_result[1].data); for (size_t i = 0, size = all_replicas.size(); i < size; ++i) { - bool is_replica_active = get_result[i + 1].error != Coordination::Error::ZNONODE; + bool is_replica_active = get_result[i + 2].error != Coordination::Error::ZNONODE; res.active_replicas += static_cast(is_replica_active); res.replica_is_active.emplace(all_replicas[i], is_replica_active); } @@ -8862,6 +8870,20 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP getCommitPartOps(ops, new_data_part); + /// Increment lost_part_count + auto lost_part_count_path = fs::path(zookeeper_path) / "lost_part_count"; + Coordination::Stat lost_part_count_stat; + String lost_part_count_str; + if (zookeeper->tryGet(lost_part_count_path, lost_part_count_str, &lost_part_count_stat)) + { + UInt64 lost_part_count = lost_part_count_str.empty() ? 0 : parse(lost_part_count_str); + ops.emplace_back(zkutil::makeSetRequest(lost_part_count_path, fmt::format("{}", lost_part_count + 1), lost_part_count_stat.version)); + } + else + { + ops.emplace_back(zkutil::makeCreateRequest(lost_part_count_path, "1", zkutil::CreateMode::Persistent)); + } + Coordination::Responses responses; if (auto code = zookeeper->tryMulti(ops, responses); code == Coordination::Error::ZOK) { diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 240d452fe29..1a09c8fb96f 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -59,6 +59,7 @@ StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_) { "absolute_delay", std::make_shared() }, { "total_replicas", std::make_shared() }, { "active_replicas", std::make_shared() }, + { "lost_part_count", std::make_shared() }, { "last_queue_update_exception", std::make_shared() }, { "zookeeper_exception", std::make_shared() }, { "replica_is_active", std::make_shared(std::make_shared(), std::make_shared()) } @@ -112,6 +113,7 @@ Pipe StorageSystemReplicas::read( || column_name == "log_pointer" || column_name == "total_replicas" || column_name == "active_replicas" + || column_name == "lost_part_count" || column_name == "zookeeper_exception" || column_name == "replica_is_active") { @@ -212,6 +214,7 @@ Pipe StorageSystemReplicas::read( res_columns[col_num++]->insert(status.absolute_delay); res_columns[col_num++]->insert(status.total_replicas); res_columns[col_num++]->insert(status.active_replicas); + res_columns[col_num++]->insert(status.lost_part_count); res_columns[col_num++]->insert(status.last_queue_update_exception); res_columns[col_num++]->insert(status.zookeeper_exception); diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index b07d6e01161..c3598788b2c 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -867,6 +867,7 @@ CREATE TABLE system.replicas `absolute_delay` UInt64, `total_replicas` UInt8, `active_replicas` UInt8, + `lost_part_count` UInt64, `last_queue_update_exception` String, `zookeeper_exception` String, `replica_is_active` Map(String, UInt8) From 9976f17e12668346c37482e8acb7901e1d31f117 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 6 Apr 2023 22:54:00 +0000 Subject: [PATCH 192/277] add docs --- docs/en/operations/system-tables/replicas.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/operations/system-tables/replicas.md b/docs/en/operations/system-tables/replicas.md index e711d9a7784..15426eefbcc 100644 --- a/docs/en/operations/system-tables/replicas.md +++ b/docs/en/operations/system-tables/replicas.md @@ -50,6 +50,7 @@ last_queue_update: 2021-10-12 14:50:08 absolute_delay: 99 total_replicas: 5 active_replicas: 5 +lost_part_count: 0 last_queue_update_exception: zookeeper_exception: replica_is_active: {'r1':1,'r2':1} @@ -90,6 +91,7 @@ The next 4 columns have a non-zero value only where there is an active session w - `absolute_delay` (`UInt64`) - How big lag in seconds the current replica has. - `total_replicas` (`UInt8`) - The total number of known replicas of this table. - `active_replicas` (`UInt8`) - The number of replicas of this table that have a session in ClickHouse Keeper (i.e., the number of functioning replicas). +- `lost_part_count` (`UInt64`) - The number of data parts lost in the table by all replicas in total since table creation. Value is persisted in ClickHouse Keeper and can only increase. - `last_queue_update_exception` (`String`) - When the queue contains broken entries. Especially important when ClickHouse breaks backward compatibility between versions and log entries written by newer versions aren't parseable by old versions. - `zookeeper_exception` (`String`) - The last exception message, got if the error happened when fetching the info from ClickHouse Keeper. - `replica_is_active` ([Map(String, UInt8)](../../sql-reference/data-types/map.md)) — Map between replica name and is replica active. From 2d07704243fd8f740ec247c92e50553f95526a49 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 6 Apr 2023 23:50:23 +0000 Subject: [PATCH 193/277] fix zk tests --- .../0_stateless/02221_system_zookeeper_unrestricted.reference | 2 ++ .../02221_system_zookeeper_unrestricted_like.reference | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference index 60d3c78d740..53b44764d5c 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference @@ -36,6 +36,8 @@ log log log_pointer log_pointer +lost_part_count +lost_part_count max_processed_insert_time max_processed_insert_time metadata diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference index c59be6a3af5..ccc3064ccbd 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference @@ -17,6 +17,7 @@ leader_election leader_election-0 log log_pointer +lost_part_count max_processed_insert_time metadata metadata @@ -58,6 +59,7 @@ leader_election leader_election-0 log log_pointer +lost_part_count max_processed_insert_time metadata metadata From f3e3117d24e99d148f478a454335d7f84c021d8b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Apr 2023 07:41:05 +0000 Subject: [PATCH 194/277] Fix test --- .../02103_with_names_and_types_parallel_parsing.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh b/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh index 487282099e2..a6e704093a2 100755 --- a/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh +++ b/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +USER_FILES_PATH=$(clickhouse client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') DATA_FILE=$USER_FILES_PATH/test_02103.data @@ -14,7 +14,7 @@ FORMATS=('TSVWithNames' 'TSVWithNamesAndTypes' 'TSVRawWithNames' 'TSVRawWithName for format in "${FORMATS[@]}" do $CLICKHOUSE_CLIENT -q "SELECT number, range(number + 10) AS array, toString(number) AS string FROM numbers(10) FORMAT $format" > $DATA_FILE - $CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103.data', '$format', 'number UInt64, array Array(UInt64), string String') SETTINGS input_format_parallel_parsing=1, min_chunk_bytes_for_parallel_parsing=40" + $CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103.data', '$format', 'number UInt64, array Array(UInt64), string String') ORDER BY number SETTINGS input_format_parallel_parsing=1, min_chunk_bytes_for_parallel_parsing=40" done rm $DATA_FILE From 36eabc57d25ff992cf6f1b74f6cafcbc8b76f3db Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 7 Apr 2023 07:49:19 +0000 Subject: [PATCH 195/277] Sync replicas after inserts --- tests/integration/test_alternative_keeper_config/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_alternative_keeper_config/test.py b/tests/integration/test_alternative_keeper_config/test.py index d2cfc4fe25e..2d59d2ee8b9 100644 --- a/tests/integration/test_alternative_keeper_config/test.py +++ b/tests/integration/test_alternative_keeper_config/test.py @@ -59,6 +59,8 @@ def test_create_insert(started_cluster): node2.query("INSERT INTO tbl VALUES (1, 'str1')") # Test deduplication node3.query("INSERT INTO tbl VALUES (2, 'str2')") + node1.query("SYSTEM SYNC REPLICA ON CLUSTER 'test_cluster' tbl") + for node in [node1, node2, node3]: expected = [[1, "str1"], [2, "str2"]] assert node.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) From 67e633525060c58a1d067caa2deb8de2869985ff Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 7 Apr 2023 07:17:00 +0000 Subject: [PATCH 196/277] Fix nullptr to memcpy --- src/Common/SpaceSaving.h | 3 +++ .../queries/0_stateless/02710_topk_with_empty_array.reference | 1 + tests/queries/0_stateless/02710_topk_with_empty_array.sql | 1 + 3 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/02710_topk_with_empty_array.reference create mode 100644 tests/queries/0_stateless/02710_topk_with_empty_array.sql diff --git a/src/Common/SpaceSaving.h b/src/Common/SpaceSaving.h index c83e836eb83..f5f66e41307 100644 --- a/src/Common/SpaceSaving.h +++ b/src/Common/SpaceSaving.h @@ -51,6 +51,9 @@ struct SpaceSavingArena { StringRef emplace(StringRef key) { + if (!key.data) + return key; + return copyStringInArena(arena, key); } diff --git a/tests/queries/0_stateless/02710_topk_with_empty_array.reference b/tests/queries/0_stateless/02710_topk_with_empty_array.reference new file mode 100644 index 00000000000..17212447ad8 --- /dev/null +++ b/tests/queries/0_stateless/02710_topk_with_empty_array.reference @@ -0,0 +1 @@ +[[]] diff --git a/tests/queries/0_stateless/02710_topk_with_empty_array.sql b/tests/queries/0_stateless/02710_topk_with_empty_array.sql new file mode 100644 index 00000000000..7de066e9ae4 --- /dev/null +++ b/tests/queries/0_stateless/02710_topk_with_empty_array.sql @@ -0,0 +1 @@ +SELECT topK(emptyArrayInt16()); From 011480924a0e261f520803e7d609c4c52dece89b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 6 Apr 2023 19:00:51 +0200 Subject: [PATCH 197/277] Use forward declaration of ThreadPool Signed-off-by: Azat Khuzhin --- src/Access/DiskAccessStorage.cpp | 12 ++-- src/Access/DiskAccessStorage.h | 4 +- src/Access/ReplicatedAccessStorage.cpp | 8 ++- src/Access/ReplicatedAccessStorage.h | 6 +- src/AggregateFunctions/IAggregateFunction.h | 2 +- src/Backups/BackupCoordinationFileInfos.cpp | 1 + src/Backups/BackupEntriesCollector.cpp | 1 + src/Backups/BackupFileInfo.cpp | 1 + src/Backups/BackupFileInfo.h | 3 +- src/Backups/BackupUtils.h | 1 - src/Backups/BackupsWorker.cpp | 19 +++--- src/Backups/BackupsWorker.h | 7 +- src/Bridge/IBridge.cpp | 1 + src/Common/SystemLogBase.cpp | 13 ++-- src/Common/SystemLogBase.h | 6 +- src/Common/ThreadPool.h | 1 + src/Common/ThreadPool_fwd.h | 13 ++++ src/Coordination/Changelog.h | 1 + src/Coordination/KeeperStorage.cpp | 1 + src/Core/BackgroundSchedulePool.cpp | 5 +- src/Core/BackgroundSchedulePool.h | 4 +- src/Databases/DatabaseFactory.h | 2 +- src/Databases/IDatabase.h | 2 +- src/Disks/IDisk.h | 16 +++-- src/Disks/IO/IOUringReader.cpp | 9 +-- src/Disks/IO/IOUringReader.h | 11 +++- src/Disks/IO/ThreadPoolReader.cpp | 10 ++- src/Disks/IO/ThreadPoolReader.h | 7 +- src/Disks/IO/ThreadPoolRemoteFSReader.cpp | 11 +++- src/Disks/IO/ThreadPoolRemoteFSReader.h | 6 +- src/Disks/ObjectStorages/IObjectStorage.h | 2 +- src/Disks/VolumeJBOD.h | 1 + src/Functions/FunctionShowCertificate.h | 1 + src/IO/BackupsIOThreadPool.cpp | 1 + src/IO/BackupsIOThreadPool.h | 4 +- src/IO/IOThreadPool.cpp | 1 + src/IO/IOThreadPool.h | 4 +- src/IO/ParallelReadBuffer.h | 1 - src/IO/ReadBufferFromFileBase.cpp | 1 + src/IO/WriteBufferFromS3.h | 1 - src/Interpreters/Context.h | 2 +- src/Interpreters/CrashLog.h | 1 + src/Interpreters/DDLWorker.cpp | 12 ++-- src/Interpreters/DDLWorker.h | 6 +- src/Interpreters/DatabaseCatalog.cpp | 5 +- src/Interpreters/MetricLog.cpp | 6 +- src/Interpreters/MetricLog.h | 3 +- src/Interpreters/OpenTelemetrySpanLog.h | 1 + src/Interpreters/QueryThreadLog.h | 7 +- src/Server/MySQLHandler.cpp | 1 + .../MergeTree/BackgroundJobsAssignee.h | 1 - .../MergeTree/MergeTreeBackgroundExecutor.cpp | 66 +++++++++++++++++-- .../MergeTree/MergeTreeBackgroundExecutor.h | 49 ++------------ src/Storages/MergeTree/MergeTreeMarksLoader.h | 2 +- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 1 + .../MergeTree/MergeTreePrefetchedReadPool.h | 3 +- src/Storages/StorageDictionary.h | 1 + .../System/StorageSystemCertificates.cpp | 3 +- 58 files changed, 226 insertions(+), 145 deletions(-) create mode 100644 src/Common/ThreadPool_fwd.h diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index ef88e8a225f..710cf257b95 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -19,6 +20,7 @@ #include #include #include +#include namespace DB @@ -317,15 +319,15 @@ void DiskAccessStorage::scheduleWriteLists(AccessEntityType type) return; /// If the lists' writing thread is still waiting we can update `types_of_lists_to_write` easily, /// without restarting that thread. - if (lists_writing_thread.joinable()) - lists_writing_thread.join(); + if (lists_writing_thread && lists_writing_thread->joinable()) + lists_writing_thread->join(); /// Create the 'need_rebuild_lists.mark' file. /// This file will be used later to find out if writing lists is successful or not. std::ofstream out{getNeedRebuildListsMarkFilePath(directory_path)}; out.close(); - lists_writing_thread = ThreadFromGlobalPool{&DiskAccessStorage::listsWritingThreadFunc, this}; + lists_writing_thread = std::make_unique(&DiskAccessStorage::listsWritingThreadFunc, this); lists_writing_thread_is_waiting = true; } @@ -349,10 +351,10 @@ void DiskAccessStorage::listsWritingThreadFunc() void DiskAccessStorage::stopListsWritingThread() { - if (lists_writing_thread.joinable()) + if (lists_writing_thread && lists_writing_thread->joinable()) { lists_writing_thread_should_exit.notify_one(); - lists_writing_thread.join(); + lists_writing_thread->join(); } } diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index b1ef1d10ba7..069a966c8e9 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include @@ -81,7 +81,7 @@ private: bool failed_to_write_lists TSA_GUARDED_BY(mutex) = false; /// List files are written in a separate thread. - ThreadFromGlobalPool lists_writing_thread; + std::unique_ptr lists_writing_thread; /// Signals `lists_writing_thread` to exit. std::condition_variable lists_writing_thread_should_exit; diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index ddc5e8bfed1..f34e6728ab3 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -15,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -72,7 +74,7 @@ void ReplicatedAccessStorage::startWatchingThread() { bool prev_watching_flag = watching.exchange(true); if (!prev_watching_flag) - watching_thread = ThreadFromGlobalPool(&ReplicatedAccessStorage::runWatchingThread, this); + watching_thread = std::make_unique(&ReplicatedAccessStorage::runWatchingThread, this); } void ReplicatedAccessStorage::stopWatchingThread() @@ -81,8 +83,8 @@ void ReplicatedAccessStorage::stopWatchingThread() if (prev_watching_flag) { watched_queue->finish(); - if (watching_thread.joinable()) - watching_thread.join(); + if (watching_thread && watching_thread->joinable()) + watching_thread->join(); } } diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h index d9d4b628f8d..555d58e6b04 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -2,7 +2,7 @@ #include -#include +#include #include #include #include @@ -21,7 +21,7 @@ public: static constexpr char STORAGE_TYPE[] = "replicated"; ReplicatedAccessStorage(const String & storage_name, const String & zookeeper_path, zkutil::GetZooKeeper get_zookeeper, AccessChangesNotifier & changes_notifier_, bool allow_backup); - virtual ~ReplicatedAccessStorage() override; + ~ReplicatedAccessStorage() override; const char * getStorageType() const override { return STORAGE_TYPE; } @@ -43,7 +43,7 @@ private: std::mutex cached_zookeeper_mutex; std::atomic watching = false; - ThreadFromGlobalPool watching_thread; + std::unique_ptr watching_thread; std::shared_ptr> watched_queue; std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 4a050a58600..ddc0535d0e4 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include "config.h" diff --git a/src/Backups/BackupCoordinationFileInfos.cpp b/src/Backups/BackupCoordinationFileInfos.cpp index 44f00f6c543..eead742b510 100644 --- a/src/Backups/BackupCoordinationFileInfos.cpp +++ b/src/Backups/BackupCoordinationFileInfos.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 1adc4d41fee..ab836487ec0 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include namespace fs = std::filesystem; diff --git a/src/Backups/BackupFileInfo.cpp b/src/Backups/BackupFileInfo.cpp index 24548ca05fe..5a3076d1647 100644 --- a/src/Backups/BackupFileInfo.cpp +++ b/src/Backups/BackupFileInfo.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include diff --git a/src/Backups/BackupFileInfo.h b/src/Backups/BackupFileInfo.h index 96df8ab2e0b..ae6ec83a37b 100644 --- a/src/Backups/BackupFileInfo.h +++ b/src/Backups/BackupFileInfo.h @@ -1,8 +1,9 @@ #pragma once #include -#include +#include +namespace Poco { class Logger; } namespace DB { diff --git a/src/Backups/BackupUtils.h b/src/Backups/BackupUtils.h index f451b003652..3dc0a58d304 100644 --- a/src/Backups/BackupUtils.h +++ b/src/Backups/BackupUtils.h @@ -1,7 +1,6 @@ #pragma once #include -#include namespace DB diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 58f0b3effc5..4b17174a8de 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -23,6 +23,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -182,8 +183,8 @@ namespace BackupsWorker::BackupsWorker(size_t num_backup_threads, size_t num_restore_threads, bool allow_concurrent_backups_, bool allow_concurrent_restores_) - : backups_thread_pool(CurrentMetrics::BackupsThreads, CurrentMetrics::BackupsThreadsActive, num_backup_threads, /* max_free_threads = */ 0, num_backup_threads) - , restores_thread_pool(CurrentMetrics::RestoreThreads, CurrentMetrics::RestoreThreadsActive, num_restore_threads, /* max_free_threads = */ 0, num_restore_threads) + : backups_thread_pool(std::make_unique(CurrentMetrics::BackupsThreads, CurrentMetrics::BackupsThreadsActive, num_backup_threads, /* max_free_threads = */ 0, num_backup_threads)) + , restores_thread_pool(std::make_unique(CurrentMetrics::RestoreThreads, CurrentMetrics::RestoreThreadsActive, num_restore_threads, /* max_free_threads = */ 0, num_restore_threads)) , log(&Poco::Logger::get("BackupsWorker")) , allow_concurrent_backups(allow_concurrent_backups_) , allow_concurrent_restores(allow_concurrent_restores_) @@ -248,7 +249,7 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context if (backup_settings.async) { - backups_thread_pool.scheduleOrThrowOnError( + backups_thread_pool->scheduleOrThrowOnError( [this, backup_query, backup_id, backup_name_for_logging, backup_info, backup_settings, backup_coordination, context_in_use, mutable_context] { doBackup( @@ -435,7 +436,7 @@ void BackupsWorker::buildFileInfosForBackupEntries(const BackupPtr & backup, con LOG_TRACE(log, "{}", Stage::BUILDING_FILE_INFOS); backup_coordination->setStage(Stage::BUILDING_FILE_INFOS, ""); backup_coordination->waitForStage(Stage::BUILDING_FILE_INFOS); - backup_coordination->addFileInfos(::DB::buildFileInfosForBackupEntries(backup_entries, backup->getBaseBackup(), backups_thread_pool)); + backup_coordination->addFileInfos(::DB::buildFileInfosForBackupEntries(backup_entries, backup->getBaseBackup(), *backups_thread_pool)); } @@ -522,7 +523,7 @@ void BackupsWorker::writeBackupEntries(BackupMutablePtr backup, BackupEntries && } }; - if (always_single_threaded || !backups_thread_pool.trySchedule([job] { job(true); })) + if (always_single_threaded || !backups_thread_pool->trySchedule([job] { job(true); })) job(false); } @@ -581,7 +582,7 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt if (restore_settings.async) { - restores_thread_pool.scheduleOrThrowOnError( + restores_thread_pool->scheduleOrThrowOnError( [this, restore_query, restore_id, backup_name_for_logging, backup_info, restore_settings, restore_coordination, context_in_use] { doRestore( @@ -716,7 +717,7 @@ void BackupsWorker::doRestore( } /// Execute the data restoring tasks. - restoreTablesData(restore_id, backup, std::move(data_restore_tasks), restores_thread_pool); + restoreTablesData(restore_id, backup, std::move(data_restore_tasks), *restores_thread_pool); /// We have restored everything, we need to tell other hosts (they could be waiting for it). restore_coordination->setStage(Stage::COMPLETED, ""); @@ -941,8 +942,8 @@ void BackupsWorker::shutdown() if (has_active_backups_and_restores) LOG_INFO(log, "Waiting for {} backups and {} restores to be finished", num_active_backups, num_active_restores); - backups_thread_pool.wait(); - restores_thread_pool.wait(); + backups_thread_pool->wait(); + restores_thread_pool->wait(); if (has_active_backups_and_restores) LOG_INFO(log, "All backup and restore tasks have finished"); diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index d319daf42bd..cbfadc24b7b 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -1,7 +1,8 @@ #pragma once #include -#include +#include +#include #include #include #include @@ -132,8 +133,8 @@ private: void setNumFilesAndSize(const OperationID & id, size_t num_files, UInt64 total_size, size_t num_entries, UInt64 uncompressed_size, UInt64 compressed_size, size_t num_read_files, UInt64 num_read_bytes); - ThreadPool backups_thread_pool; - ThreadPool restores_thread_pool; + std::unique_ptr backups_thread_pool; + std::unique_ptr restores_thread_pool; std::unordered_map infos; std::condition_variable status_changed; diff --git a/src/Bridge/IBridge.cpp b/src/Bridge/IBridge.cpp index 1ea77573e5f..fce705dee5f 100644 --- a/src/Bridge/IBridge.cpp +++ b/src/Bridge/IBridge.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 13150194df2..86adcbbd31b 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -18,6 +18,7 @@ #include #include +#include #include #include @@ -35,20 +36,18 @@ namespace constexpr size_t DBMS_SYSTEM_LOG_QUEUE_SIZE = 1048576; } +ISystemLog::~ISystemLog() = default; + void ISystemLog::stopFlushThread() { { std::lock_guard lock(mutex); - if (!saving_thread.joinable()) - { + if (!saving_thread || !saving_thread->joinable()) return; - } if (is_shutdown) - { return; - } is_shutdown = true; @@ -56,13 +55,13 @@ void ISystemLog::stopFlushThread() flush_event.notify_all(); } - saving_thread.join(); + saving_thread->join(); } void ISystemLog::startup() { std::lock_guard lock(mutex); - saving_thread = ThreadFromGlobalPool([this] { savingThreadFunction(); }); + saving_thread = std::make_unique([this] { savingThreadFunction(); }); } static thread_local bool recursive_add_call = false; diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index c2cedb2ae39..8ac731c34f7 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -10,7 +10,7 @@ #include #include #include -#include +#include #define SYSTEM_LOG_ELEMENTS(M) \ M(AsynchronousMetricLogElement) \ @@ -60,12 +60,12 @@ public: /// Stop the background flush thread before destructor. No more data will be written. virtual void shutdown() = 0; - virtual ~ISystemLog() = default; + virtual ~ISystemLog(); virtual void savingThreadFunction() = 0; protected: - ThreadFromGlobalPool saving_thread; + std::unique_ptr saving_thread; /// Data shared between callers of add()/flush()/shutdown(), and the saving thread std::mutex mutex; diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index b2f77f9693c..68023c8a410 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -17,6 +17,7 @@ #include #include #include +#include #include /** Very simple thread pool similar to boost::threadpool. diff --git a/src/Common/ThreadPool_fwd.h b/src/Common/ThreadPool_fwd.h new file mode 100644 index 00000000000..2782acc9c51 --- /dev/null +++ b/src/Common/ThreadPool_fwd.h @@ -0,0 +1,13 @@ +#pragma once + +template +class ThreadPoolImpl; + +template +class ThreadFromGlobalPoolImpl; + +using ThreadFromGlobalPoolNoTracingContextPropagation = ThreadFromGlobalPoolImpl; + +using ThreadFromGlobalPool = ThreadFromGlobalPoolImpl; + +using ThreadPool = ThreadPoolImpl; diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 288f71bb915..56b0475ba8b 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 41a6af54204..dc6c05e2594 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Core/BackgroundSchedulePool.cpp b/src/Core/BackgroundSchedulePool.cpp index 5384ee7f961..39724ec07fa 100644 --- a/src/Core/BackgroundSchedulePool.cpp +++ b/src/Core/BackgroundSchedulePool.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -160,7 +161,7 @@ BackgroundSchedulePool::BackgroundSchedulePool(size_t size_, CurrentMetrics::Met for (auto & thread : threads) thread = ThreadFromGlobalPoolNoTracingContextPropagation([this] { threadFunction(); }); - delayed_thread = ThreadFromGlobalPoolNoTracingContextPropagation([this] { delayExecutionThreadFunction(); }); + delayed_thread = std::make_unique([this] { delayExecutionThreadFunction(); }); } @@ -198,7 +199,7 @@ BackgroundSchedulePool::~BackgroundSchedulePool() delayed_tasks_cond_var.notify_all(); LOG_TRACE(&Poco::Logger::get("BackgroundSchedulePool/" + thread_name), "Waiting for threads to finish."); - delayed_thread.join(); + delayed_thread->join(); for (auto & thread : threads) thread.join(); diff --git a/src/Core/BackgroundSchedulePool.h b/src/Core/BackgroundSchedulePool.h index ef6fbfa68e9..e97b02e976f 100644 --- a/src/Core/BackgroundSchedulePool.h +++ b/src/Core/BackgroundSchedulePool.h @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include @@ -86,7 +86,7 @@ private: std::condition_variable delayed_tasks_cond_var; std::mutex delayed_tasks_mutex; /// Thread waiting for next delayed task. - ThreadFromGlobalPoolNoTracingContextPropagation delayed_thread; + std::unique_ptr delayed_thread; /// Tasks ordered by scheduled time. DelayedTasks delayed_tasks; diff --git a/src/Databases/DatabaseFactory.h b/src/Databases/DatabaseFactory.h index 8992ea27093..cb631cd76d0 100644 --- a/src/Databases/DatabaseFactory.h +++ b/src/Databases/DatabaseFactory.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index b8880c4c4cc..53a2f372814 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 4e488bbb39a..797235b5fb8 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -254,8 +254,8 @@ public: virtual NameSet getCacheLayersNames() const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Method `getCacheLayersNames()` is not implemented for disk: {}", - getDataSourceDescription().type); + "Method `getCacheLayersNames()` is not implemented for disk: {}", + toString(getDataSourceDescription().type)); } /// Returns a list of storage objects (contains path, size, ...). @@ -263,7 +263,9 @@ public: /// be multiple files in remote fs for single clickhouse file. virtual StoredObjects getStorageObjects(const String &) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getStorageObjects() not implemented for disk: {}`", getDataSourceDescription().type); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Method `getStorageObjects()` not implemented for disk: {}", + toString(getDataSourceDescription().type)); } /// For one local path there might be multiple remote paths in case of Log family engines. @@ -281,8 +283,8 @@ public: virtual void getRemotePathsRecursive(const String &, std::vector &) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Method `getRemotePathsRecursive() not implemented for disk: {}`", - getDataSourceDescription().type); + "Method `getRemotePathsRecursive() not implemented for disk: {}`", + toString(getDataSourceDescription().type)); } /// Batch request to remove multiple files. @@ -398,7 +400,7 @@ public: throw Exception( ErrorCodes::NOT_IMPLEMENTED, "Method getObjectStorage() is not implemented for disk type: {}", - getDataSourceDescription().type); + toString(getDataSourceDescription().type)); } /// Create disk object storage according to disk type. @@ -409,7 +411,7 @@ public: throw Exception( ErrorCodes::NOT_IMPLEMENTED, "Method createDiskObjectStorage() is not implemented for disk type: {}", - getDataSourceDescription().type); + toString(getDataSourceDescription().type)); } virtual bool supportsStat() const { return false; } diff --git a/src/Disks/IO/IOUringReader.cpp b/src/Disks/IO/IOUringReader.cpp index 7bf1982d515..7b68e0ee2de 100644 --- a/src/Disks/IO/IOUringReader.cpp +++ b/src/Disks/IO/IOUringReader.cpp @@ -1,15 +1,16 @@ #include "IOUringReader.h" +#include #if USE_LIBURING #include #include -#include #include #include #include #include #include +#include #include #include @@ -44,7 +45,7 @@ namespace ErrorCodes } IOUringReader::IOUringReader(uint32_t entries_) - : log(&Poco::Logger::get("IOUringReader")) + : log(&Poco::Logger::get("IOUringReader")) { struct io_uring_probe * probe = io_uring_get_probe(); if (!probe) @@ -70,7 +71,7 @@ IOUringReader::IOUringReader(uint32_t entries_) throwFromErrno("Failed initializing io_uring", ErrorCodes::IO_URING_INIT_FAILED, -ret); cq_entries = params.cq_entries; - ring_completion_monitor = ThreadFromGlobalPool([this] { monitorRing(); }); + ring_completion_monitor = std::make_unique([this] { monitorRing(); }); } std::future IOUringReader::submit(Request request) @@ -333,7 +334,7 @@ IOUringReader::~IOUringReader() io_uring_submit(&ring); } - ring_completion_monitor.join(); + ring_completion_monitor->join(); io_uring_queue_exit(&ring); } diff --git a/src/Disks/IO/IOUringReader.h b/src/Disks/IO/IOUringReader.h index e3fcf116448..9b80ac6e5e0 100644 --- a/src/Disks/IO/IOUringReader.h +++ b/src/Disks/IO/IOUringReader.h @@ -4,15 +4,20 @@ #if USE_LIBURING -#include +#include +#include #include #include #include #include +namespace Poco { class Logger; } + namespace DB { +class Exception; + /** Perform reads using the io_uring Linux subsystem. * * The class sets up a single io_uring that clients submit read requests to, they are @@ -30,7 +35,7 @@ private: uint32_t cq_entries; std::atomic cancelled{false}; - ThreadFromGlobalPool ring_completion_monitor; + std::unique_ptr ring_completion_monitor; struct EnqueuedRequest { @@ -74,7 +79,7 @@ public: void wait() override {} - virtual ~IOUringReader() override; + ~IOUringReader() override; }; } diff --git a/src/Disks/IO/ThreadPoolReader.cpp b/src/Disks/IO/ThreadPoolReader.cpp index 3a071d13122..de57fa157da 100644 --- a/src/Disks/IO/ThreadPoolReader.cpp +++ b/src/Disks/IO/ThreadPoolReader.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -87,7 +88,7 @@ static bool hasBugInPreadV2() #endif ThreadPoolReader::ThreadPoolReader(size_t pool_size, size_t queue_size_) - : pool(CurrentMetrics::ThreadPoolFSReaderThreads, CurrentMetrics::ThreadPoolFSReaderThreadsActive, pool_size, pool_size, queue_size_) + : pool(std::make_unique(CurrentMetrics::ThreadPoolFSReaderThreads, CurrentMetrics::ThreadPoolFSReaderThreadsActive, pool_size, pool_size, queue_size_)) { } @@ -200,7 +201,7 @@ std::future ThreadPoolReader::submit(Request reques ProfileEvents::increment(ProfileEvents::ThreadPoolReaderPageCacheMiss); - auto schedule = threadPoolCallbackRunner(pool, "ThreadPoolRead"); + auto schedule = threadPoolCallbackRunner(*pool, "ThreadPoolRead"); return schedule([request, fd]() -> Result { @@ -244,4 +245,9 @@ std::future ThreadPoolReader::submit(Request reques }, request.priority); } +void ThreadPoolReader::wait() +{ + pool->wait(); +} + } diff --git a/src/Disks/IO/ThreadPoolReader.h b/src/Disks/IO/ThreadPoolReader.h index dc754e0a81c..4c55be29bf9 100644 --- a/src/Disks/IO/ThreadPoolReader.h +++ b/src/Disks/IO/ThreadPoolReader.h @@ -1,7 +1,8 @@ #pragma once +#include #include -#include +#include #include @@ -28,14 +29,14 @@ namespace DB class ThreadPoolReader final : public IAsynchronousReader { private: - ThreadPool pool; + std::unique_ptr pool; public: ThreadPoolReader(size_t pool_size, size_t queue_size_); std::future submit(Request request) override; - void wait() override { pool.wait(); } + void wait() override; /// pool automatically waits for all tasks in destructor. }; diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp index 1980f57c876..4d0f39357ab 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp @@ -1,6 +1,7 @@ #include "ThreadPoolRemoteFSReader.h" #include "config.h" +#include #include #include #include @@ -14,6 +15,7 @@ #include #include +#include namespace ProfileEvents @@ -62,7 +64,7 @@ IAsynchronousReader::Result RemoteFSFileDescriptor::readInto(char * data, size_t ThreadPoolRemoteFSReader::ThreadPoolRemoteFSReader(size_t pool_size, size_t queue_size_) - : pool(CurrentMetrics::ThreadPoolRemoteFSReaderThreads, CurrentMetrics::ThreadPoolRemoteFSReaderThreadsActive, pool_size, pool_size, queue_size_) + : pool(std::make_unique(CurrentMetrics::ThreadPoolRemoteFSReaderThreads, CurrentMetrics::ThreadPoolRemoteFSReaderThreadsActive, pool_size, pool_size, queue_size_)) { } @@ -92,7 +94,12 @@ std::future ThreadPoolRemoteFSReader::submit(Reques ProfileEvents::increment(ProfileEvents::ThreadpoolReaderReadBytes, result.size); return Result{ .size = result.size, .offset = result.offset, .execution_watch = std::move(watch) }; - }, pool, "VFSRead", request.priority); + }, *pool, "VFSRead", request.priority); +} + +void ThreadPoolRemoteFSReader::wait() +{ + pool->wait(); } } diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.h b/src/Disks/IO/ThreadPoolRemoteFSReader.h index cd289150ba1..3a765993292 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.h +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include namespace DB @@ -15,10 +15,10 @@ public: std::future submit(Request request) override; - void wait() override { pool.wait(); } + void wait() override; private: - ThreadPool pool; + std::unique_ptr pool; }; class RemoteFSFileDescriptor : public IAsynchronousReader::IFileDescriptor diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 2cfb4d43a43..2f27dc18e4b 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -16,7 +16,7 @@ #include #include #include -#include +#include #include diff --git a/src/Disks/VolumeJBOD.h b/src/Disks/VolumeJBOD.h index 81da64c488d..ef6f215bf18 100644 --- a/src/Disks/VolumeJBOD.h +++ b/src/Disks/VolumeJBOD.h @@ -2,6 +2,7 @@ #include #include +#include #include diff --git a/src/Functions/FunctionShowCertificate.h b/src/Functions/FunctionShowCertificate.h index 3c30d8138e5..5061a198614 100644 --- a/src/Functions/FunctionShowCertificate.h +++ b/src/Functions/FunctionShowCertificate.h @@ -15,6 +15,7 @@ #include #include #include +#include #if USE_SSL #include diff --git a/src/IO/BackupsIOThreadPool.cpp b/src/IO/BackupsIOThreadPool.cpp index 0829553945a..e135ef66ffb 100644 --- a/src/IO/BackupsIOThreadPool.cpp +++ b/src/IO/BackupsIOThreadPool.cpp @@ -1,5 +1,6 @@ #include #include +#include #include namespace CurrentMetrics diff --git a/src/IO/BackupsIOThreadPool.h b/src/IO/BackupsIOThreadPool.h index f5aae5741a8..745bf267300 100644 --- a/src/IO/BackupsIOThreadPool.h +++ b/src/IO/BackupsIOThreadPool.h @@ -1,6 +1,8 @@ #pragma once -#include +#include +#include +#include namespace DB { diff --git a/src/IO/IOThreadPool.cpp b/src/IO/IOThreadPool.cpp index 98bb6ffe6a7..6765deff4d4 100644 --- a/src/IO/IOThreadPool.cpp +++ b/src/IO/IOThreadPool.cpp @@ -1,5 +1,6 @@ #include #include +#include #include namespace CurrentMetrics diff --git a/src/IO/IOThreadPool.h b/src/IO/IOThreadPool.h index 4fcf99b6048..cfe755ed45a 100644 --- a/src/IO/IOThreadPool.h +++ b/src/IO/IOThreadPool.h @@ -1,6 +1,8 @@ #pragma once -#include +#include +#include +#include namespace DB { diff --git a/src/IO/ParallelReadBuffer.h b/src/IO/ParallelReadBuffer.h index d6e9b7989ad..a7b78be442c 100644 --- a/src/IO/ParallelReadBuffer.h +++ b/src/IO/ParallelReadBuffer.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB { diff --git a/src/IO/ReadBufferFromFileBase.cpp b/src/IO/ReadBufferFromFileBase.cpp index d94cf12294b..4181615bc52 100644 --- a/src/IO/ReadBufferFromFileBase.cpp +++ b/src/IO/ReadBufferFromFileBase.cpp @@ -1,4 +1,5 @@ #include +#include #include namespace DB diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index e56d590c57a..7c3c439f9c3 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -10,7 +10,6 @@ #include #include -#include #include #include #include diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index d5ade8c02c7..73c16a35efd 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/CrashLog.h b/src/Interpreters/CrashLog.h index f6061355562..78794574c82 100644 --- a/src/Interpreters/CrashLog.h +++ b/src/Interpreters/CrashLog.h @@ -3,6 +3,7 @@ #include #include #include +#include /// Call this function on crash. diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index c4529af2c51..7c021bd82e6 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -31,9 +31,11 @@ #include #include #include +#include #include #include #include +#include #include @@ -121,8 +123,8 @@ void DDLWorker::startup() { [[maybe_unused]] bool prev_stop_flag = stop_flag.exchange(false); chassert(prev_stop_flag); - main_thread = ThreadFromGlobalPool(&DDLWorker::runMainThread, this); - cleanup_thread = ThreadFromGlobalPool(&DDLWorker::runCleanupThread, this); + main_thread = std::make_unique(&DDLWorker::runMainThread, this); + cleanup_thread = std::make_unique(&DDLWorker::runCleanupThread, this); } void DDLWorker::shutdown() @@ -132,8 +134,10 @@ void DDLWorker::shutdown() { queue_updated_event->set(); cleanup_event->set(); - main_thread.join(); - cleanup_thread.join(); + if (main_thread) + main_thread->join(); + if (cleanup_thread) + cleanup_thread->join(); worker_pool.reset(); } } diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index 6cf034edae8..01ed89907a1 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include @@ -145,8 +145,8 @@ protected: std::atomic initialized = false; std::atomic stop_flag = true; - ThreadFromGlobalPool main_thread; - ThreadFromGlobalPool cleanup_thread; + std::unique_ptr main_thread; + std::unique_ptr cleanup_thread; /// Size of the pool for query execution. size_t pool_size = 1; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index f37e41614b0..8d3fa91a7fe 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -8,16 +8,17 @@ #include #include #include -#include #include #include #include #include #include +#include +#include #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/MetricLog.cpp b/src/Interpreters/MetricLog.cpp index 578cc118a6b..24f77f7d0ba 100644 --- a/src/Interpreters/MetricLog.cpp +++ b/src/Interpreters/MetricLog.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -58,7 +59,7 @@ void MetricLog::startCollectMetric(size_t collect_interval_milliseconds_) { collect_interval_milliseconds = collect_interval_milliseconds_; is_shutdown_metric_thread = false; - metric_flush_thread = ThreadFromGlobalPool([this] { metricThreadFunction(); }); + metric_flush_thread = std::make_unique([this] { metricThreadFunction(); }); } @@ -67,7 +68,8 @@ void MetricLog::stopCollectMetric() bool old_val = false; if (!is_shutdown_metric_thread.compare_exchange_strong(old_val, true)) return; - metric_flush_thread.join(); + if (metric_flush_thread) + metric_flush_thread->join(); } diff --git a/src/Interpreters/MetricLog.h b/src/Interpreters/MetricLog.h index 44fcdced07c..aacdd4f49d2 100644 --- a/src/Interpreters/MetricLog.h +++ b/src/Interpreters/MetricLog.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -50,7 +51,7 @@ public: private: void metricThreadFunction(); - ThreadFromGlobalPool metric_flush_thread; + std::unique_ptr metric_flush_thread; size_t collect_interval_milliseconds; std::atomic is_shutdown_metric_thread{false}; }; diff --git a/src/Interpreters/OpenTelemetrySpanLog.h b/src/Interpreters/OpenTelemetrySpanLog.h index 9305605f30b..7368b184e5e 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.h +++ b/src/Interpreters/OpenTelemetrySpanLog.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include diff --git a/src/Interpreters/QueryThreadLog.h b/src/Interpreters/QueryThreadLog.h index 6cdb3142ba3..684d7fce53e 100644 --- a/src/Interpreters/QueryThreadLog.h +++ b/src/Interpreters/QueryThreadLog.h @@ -2,16 +2,11 @@ #include #include +#include #include #include -namespace ProfileEvents -{ - class Counters; -} - - namespace DB { diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 40cc51f8aae..96566a7d706 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include "config_version.h" diff --git a/src/Storages/MergeTree/BackgroundJobsAssignee.h b/src/Storages/MergeTree/BackgroundJobsAssignee.h index db93b5f710b..27e75a79b97 100644 --- a/src/Storages/MergeTree/BackgroundJobsAssignee.h +++ b/src/Storages/MergeTree/BackgroundJobsAssignee.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 84fa9ec2c8e..65a796456bf 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -2,21 +2,77 @@ #include +#include #include #include #include #include +namespace CurrentMetrics +{ + extern const Metric MergeTreeBackgroundExecutorThreads; + extern const Metric MergeTreeBackgroundExecutorThreadsActive; +} + namespace DB { namespace ErrorCodes { extern const int ABORTED; + extern const int INVALID_CONFIG_PARAMETER; } +template +MergeTreeBackgroundExecutor::MergeTreeBackgroundExecutor( + String name_, + size_t threads_count_, + size_t max_tasks_count_, + CurrentMetrics::Metric metric_, + CurrentMetrics::Metric max_tasks_metric_) + : name(name_) + , threads_count(threads_count_) + , max_tasks_count(max_tasks_count_) + , metric(metric_) + , max_tasks_metric(max_tasks_metric_, 2 * max_tasks_count) // active + pending + , pool(std::make_unique(CurrentMetrics::MergeTreeBackgroundExecutorThreads, CurrentMetrics::MergeTreeBackgroundExecutorThreadsActive)) +{ + if (max_tasks_count == 0) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Task count for MergeTreeBackgroundExecutor must not be zero"); + + pending.setCapacity(max_tasks_count); + active.set_capacity(max_tasks_count); + + pool->setMaxThreads(std::max(1UL, threads_count)); + pool->setMaxFreeThreads(std::max(1UL, threads_count)); + pool->setQueueSize(std::max(1UL, threads_count)); + + for (size_t number = 0; number < threads_count; ++number) + pool->scheduleOrThrowOnError([this] { threadFunction(); }); +} + +template +MergeTreeBackgroundExecutor::MergeTreeBackgroundExecutor( + String name_, + size_t threads_count_, + size_t max_tasks_count_, + CurrentMetrics::Metric metric_, + CurrentMetrics::Metric max_tasks_metric_, + std::string_view policy) + requires requires(Queue queue) { queue.updatePolicy(policy); } // Because we use explicit template instantiation + : MergeTreeBackgroundExecutor(name_, threads_count_, max_tasks_count_, metric_, max_tasks_metric_) +{ + pending.updatePolicy(policy); +} + +template +MergeTreeBackgroundExecutor::~MergeTreeBackgroundExecutor() +{ + wait(); +} + template void MergeTreeBackgroundExecutor::wait() { @@ -26,7 +82,7 @@ void MergeTreeBackgroundExecutor::wait() has_tasks.notify_all(); } - pool.wait(); + pool->wait(); } template @@ -52,12 +108,12 @@ void MergeTreeBackgroundExecutor::increaseThreadsAndMaxTasksCount(size_t pending.setCapacity(new_max_tasks_count); active.set_capacity(new_max_tasks_count); - pool.setMaxThreads(std::max(1UL, new_threads_count)); - pool.setMaxFreeThreads(std::max(1UL, new_threads_count)); - pool.setQueueSize(std::max(1UL, new_threads_count)); + pool->setMaxThreads(std::max(1UL, new_threads_count)); + pool->setMaxFreeThreads(std::max(1UL, new_threads_count)); + pool->setQueueSize(std::max(1UL, new_threads_count)); for (size_t number = threads_count; number < new_threads_count; ++number) - pool.scheduleOrThrowOnError([this] { threadFunction(); }); + pool->scheduleOrThrowOnError([this] { threadFunction(); }); max_tasks_metric.changeTo(2 * new_max_tasks_count); // pending + active max_tasks_count.store(new_max_tasks_count, std::memory_order_relaxed); diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h index a27fb18c0fe..1ed4fc2dabc 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h @@ -15,24 +15,14 @@ #include #include -#include +#include #include #include #include -namespace CurrentMetrics -{ - extern const Metric MergeTreeBackgroundExecutorThreads; - extern const Metric MergeTreeBackgroundExecutorThreadsActive; -} - namespace DB { -namespace ErrorCodes -{ - extern const int INVALID_CONFIG_PARAMETER; -} struct TaskRuntimeData; using TaskRuntimeDataPtr = std::shared_ptr; @@ -255,28 +245,7 @@ public: size_t threads_count_, size_t max_tasks_count_, CurrentMetrics::Metric metric_, - CurrentMetrics::Metric max_tasks_metric_) - : name(name_) - , threads_count(threads_count_) - , max_tasks_count(max_tasks_count_) - , metric(metric_) - , max_tasks_metric(max_tasks_metric_, 2 * max_tasks_count) // active + pending - , pool(CurrentMetrics::MergeTreeBackgroundExecutorThreads, CurrentMetrics::MergeTreeBackgroundExecutorThreadsActive) - { - if (max_tasks_count == 0) - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Task count for MergeTreeBackgroundExecutor must not be zero"); - - pending.setCapacity(max_tasks_count); - active.set_capacity(max_tasks_count); - - pool.setMaxThreads(std::max(1UL, threads_count)); - pool.setMaxFreeThreads(std::max(1UL, threads_count)); - pool.setQueueSize(std::max(1UL, threads_count)); - - for (size_t number = 0; number < threads_count; ++number) - pool.scheduleOrThrowOnError([this] { threadFunction(); }); - } - + CurrentMetrics::Metric max_tasks_metric_); MergeTreeBackgroundExecutor( String name_, size_t threads_count_, @@ -284,16 +253,8 @@ public: CurrentMetrics::Metric metric_, CurrentMetrics::Metric max_tasks_metric_, std::string_view policy) - requires requires(Queue queue) { queue.updatePolicy(policy); } // Because we use explicit template instantiation - : MergeTreeBackgroundExecutor(name_, threads_count_, max_tasks_count_, metric_, max_tasks_metric_) - { - pending.updatePolicy(policy); - } - - ~MergeTreeBackgroundExecutor() - { - wait(); - } + requires requires(Queue queue) { queue.updatePolicy(policy); }; // Because we use explicit template instantiation + ~MergeTreeBackgroundExecutor(); /// Handler for hot-reloading /// Supports only increasing the number of threads and tasks, because @@ -335,7 +296,7 @@ private: mutable std::mutex mutex; std::condition_variable has_tasks TSA_GUARDED_BY(mutex); bool shutdown TSA_GUARDED_BY(mutex) = false; - ThreadPool pool; + std::unique_ptr pool; Poco::Logger * log = &Poco::Logger::get("MergeTreeBackgroundExecutor"); }; diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.h b/src/Storages/MergeTree/MergeTreeMarksLoader.h index 0294cbbf3fc..17e52939d3f 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.h +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index d76b8522f42..114039f7ff7 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index 98cfe28c563..74d76f723a9 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -1,10 +1,11 @@ #pragma once -#include +#include #include #include #include #include +#include #include namespace DB diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index b3442ec2f99..2cbcc58fa65 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -4,6 +4,7 @@ #include #include +#include namespace DB diff --git a/src/Storages/System/StorageSystemCertificates.cpp b/src/Storages/System/StorageSystemCertificates.cpp index c4d262f2f44..e1767e4c03a 100644 --- a/src/Storages/System/StorageSystemCertificates.cpp +++ b/src/Storages/System/StorageSystemCertificates.cpp @@ -6,7 +6,8 @@ #include #include #include -#include "Poco/File.h" +#include +#include #if USE_SSL #include #include "Poco/Net/SSLManager.h" From bea9468285a8df083161ea7c3e4ced027fbd1f5b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Apr 2023 10:47:52 +0000 Subject: [PATCH 198/277] Fix 01548_parallel_parsing_max_memory.sh --- tests/queries/0_stateless/01548_parallel_parsing_max_memory.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01548_parallel_parsing_max_memory.sh b/tests/queries/0_stateless/01548_parallel_parsing_max_memory.sh index 8c4900043d0..308b90dcd55 100755 --- a/tests/queries/0_stateless/01548_parallel_parsing_max_memory.sh +++ b/tests/queries/0_stateless/01548_parallel_parsing_max_memory.sh @@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) yes http://foobarfoobarfoobarfoobarfoobarfoobarfoobar.com | head -c1G > ${CLICKHOUSE_TMP}/1g.csv -$CLICKHOUSE_LOCAL --stacktrace --input_format_parallel_parsing=1 --max_memory_usage=100Mi -q "select count() from file('${CLICKHOUSE_TMP}/1g.csv', 'TSV', 'URL String')" +$CLICKHOUSE_LOCAL --stacktrace --input_format_parallel_parsing=1 --max_memory_usage=50Mi -q "select count() from file('${CLICKHOUSE_TMP}/1g.csv', 'TSV', 'URL String') settings max_threads=1" From 78038a3c2cec705b405947bc6c0f53e07f844666 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Apr 2023 11:34:04 +0000 Subject: [PATCH 199/277] Fix: do not resize pipeline when there is no files to process (globs expands to empty set) --- src/Storages/StorageFile.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index f8b5ea0ced7..665630c3559 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -792,12 +792,13 @@ Pipe StorageFile::read( } Pipe pipe = Pipe::unitePipes(std::move(pipes)); - /// parallelize output as much as possible - if (num_streams < max_num_streams) + /// Parallelize output as much as possible + /// Note: number of streams can be 0 if paths is empty + /// It happens if globs in file(path, ...) expands to empty set i.e. no files to process + if (num_streams > 0 && num_streams < max_num_streams) { pipe.addTransform(std::make_shared(pipe.getHeader(), num_streams, max_num_streams)); } - return pipe; } From 5b2b20a0b06e88ea1165cb59a6ff8f65653cd71f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 7 Apr 2023 15:13:21 +0200 Subject: [PATCH 200/277] Rename ThreadGroupStatus to ThreadGroup There are methods like getThreadGroup() and ThreadGroupSwitcher class, so seems that this is logical. Signed-off-by: Azat Khuzhin --- src/Common/CurrentThread.cpp | 2 +- src/Common/CurrentThread.h | 6 ++-- src/Common/ThreadStatus.cpp | 6 ++-- src/Common/ThreadStatus.h | 24 +++++++-------- src/Interpreters/Aggregator.cpp | 4 +-- src/Interpreters/ExternalLoader.cpp | 2 +- src/Interpreters/ProcessList.cpp | 2 +- src/Interpreters/ProcessList.h | 4 +-- src/Interpreters/ThreadStatusExt.cpp | 30 +++++++++---------- .../Executors/CompletedPipelineExecutor.cpp | 2 +- .../PullingAsyncPipelineExecutor.cpp | 2 +- .../PushingAsyncPipelineExecutor.cpp | 2 +- .../Impl/ParallelFormattingOutputFormat.cpp | 4 +-- .../Impl/ParallelFormattingOutputFormat.h | 4 +-- .../Impl/ParallelParsingInputFormat.cpp | 4 +-- .../Formats/Impl/ParallelParsingInputFormat.h | 4 +-- .../Transforms/buildPushingToViewsChain.cpp | 6 ++-- src/Storages/MergeTree/MergeList.cpp | 4 +-- src/Storages/MergeTree/MergeList.h | 6 ++-- 19 files changed, 59 insertions(+), 59 deletions(-) diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 6ec46d6508c..fd2ad0bbaf1 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -90,7 +90,7 @@ void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr & logs_queue, @@ -69,9 +69,9 @@ public: /// You must call one of these methods when create a query child thread: /// Add current thread to a group associated with the thread group - static void attachToGroup(const ThreadGroupStatusPtr & thread_group); + static void attachToGroup(const ThreadGroupPtr & thread_group); /// Is useful for a ThreadPool tasks - static void attachToGroupIfDetached(const ThreadGroupStatusPtr & thread_group); + static void attachToGroupIfDetached(const ThreadGroupPtr & thread_group); /// Non-master threads call this method in destructor automatically static void detachFromGroupIfNotDetached(); diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 1b783aa9ec4..e00c9b168a9 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -61,7 +61,7 @@ static thread_local ThreadStack alt_stack; static thread_local bool has_alt_stack = false; #endif -ThreadGroupStatus::ThreadGroupStatus() +ThreadGroup::ThreadGroup() : master_thread_id(CurrentThread::get().thread_id) {} @@ -119,7 +119,7 @@ ThreadStatus::ThreadStatus() #endif } -ThreadGroupStatusPtr ThreadStatus::getThreadGroup() const +ThreadGroupPtr ThreadStatus::getThreadGroup() const { return thread_group; } @@ -139,7 +139,7 @@ ContextPtr ThreadStatus::getGlobalContext() const return global_context.lock(); } -void ThreadGroupStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel logs_level) +void ThreadGroup::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel logs_level) { std::lock_guard lock(mutex); shared_data.logs_queue_ptr = logs_queue; diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index cb85aa67b11..16083fe0925 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -58,15 +58,15 @@ using ThreadStatusPtr = ThreadStatus *; * Create via CurrentThread::initializeQuery (for queries) or directly (for various background tasks). * Use via CurrentThread::getGroup. */ -class ThreadGroupStatus; -using ThreadGroupStatusPtr = std::shared_ptr; +class ThreadGroup; +using ThreadGroupPtr = std::shared_ptr; -class ThreadGroupStatus +class ThreadGroup { public: - ThreadGroupStatus(); + ThreadGroup(); using FatalErrorCallback = std::function; - ThreadGroupStatus(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {}); + ThreadGroup(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {}); /// The first thread created this thread group const UInt64 master_thread_id; @@ -104,9 +104,9 @@ public: void attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue); /// When new query starts, new thread group is created for it, current thread becomes master thread of the query - static ThreadGroupStatusPtr createForQuery(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {}); + static ThreadGroupPtr createForQuery(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {}); - static ThreadGroupStatusPtr createForBackgroundProcess(ContextPtr storage_context); + static ThreadGroupPtr createForBackgroundProcess(ContextPtr storage_context); std::vector getInvolvedThreadIds() const; void linkThread(UInt64 thread_it); @@ -163,7 +163,7 @@ public: private: /// Group of threads, to which this thread attached - ThreadGroupStatusPtr thread_group; + ThreadGroupPtr thread_group; /// Is set once ContextWeakPtr global_context; @@ -174,7 +174,7 @@ private: using FatalErrorCallback = std::function; FatalErrorCallback fatal_error_callback; - ThreadGroupStatus::SharedData local_data; + ThreadGroup::SharedData local_data; bool performance_counters_finalized = false; @@ -215,7 +215,7 @@ public: ThreadStatus(); ~ThreadStatus(); - ThreadGroupStatusPtr getThreadGroup() const; + ThreadGroupPtr getThreadGroup() const; const String & getQueryId() const; @@ -239,7 +239,7 @@ public: void setInternalThread(); /// Attaches slave thread to existing thread group - void attachToGroup(const ThreadGroupStatusPtr & thread_group_, bool check_detached = true); + void attachToGroup(const ThreadGroupPtr & thread_group_, bool check_detached = true); /// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped void detachFromGroup(); @@ -287,7 +287,7 @@ private: void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database); - void attachToGroupImpl(const ThreadGroupStatusPtr & thread_group_); + void attachToGroupImpl(const ThreadGroupPtr & thread_group_); }; /** diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index d6fbf072d05..2d5de796e1c 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2315,7 +2315,7 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl( std::atomic next_bucket_to_merge = 0; - auto converter = [&](size_t thread_id, ThreadGroupStatusPtr thread_group) + auto converter = [&](size_t thread_id, ThreadGroupPtr thread_group) { SCOPE_EXIT_SAFE( if (thread_group) @@ -3043,7 +3043,7 @@ void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVari LOG_TRACE(log, "Merging partially aggregated two-level data."); - auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, ThreadGroupStatusPtr thread_group) + auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, ThreadGroupPtr thread_group) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 04a116ec0c7..c1cbd8b75be 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -967,7 +967,7 @@ private: } /// Does the loading, possibly in the separate thread. - void doLoading(const String & name, size_t loading_id, bool forced_to_reload, size_t min_id_to_finish_loading_dependencies_, bool async, ThreadGroupStatusPtr thread_group = {}) + void doLoading(const String & name, size_t loading_id, bool forced_to_reload, size_t min_id_to_finish_loading_dependencies_, bool async, ThreadGroupPtr thread_group = {}) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 51053bd2884..aca474bf152 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -340,7 +340,7 @@ QueryStatus::QueryStatus( const String & query_, const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_, - ThreadGroupStatusPtr && thread_group_, + ThreadGroupPtr && thread_group_, IAST::QueryKind query_kind_, UInt64 watch_start_nanoseconds) : WithContext(context_) diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index a04beac4901..b593bcef395 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -86,7 +86,7 @@ protected: ClientInfo client_info; /// Info about all threads involved in query execution - ThreadGroupStatusPtr thread_group; + ThreadGroupPtr thread_group; Stopwatch watch; @@ -162,7 +162,7 @@ public: const String & query_, const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_, - ThreadGroupStatusPtr && thread_group_, + ThreadGroupPtr && thread_group_, IAST::QueryKind query_kind_, UInt64 watch_start_nanoseconds); diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 72ce08bf653..070cd3f98e1 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -41,14 +41,14 @@ namespace ErrorCodes extern const int CANNOT_SET_THREAD_PRIORITY; } -ThreadGroupStatus::ThreadGroupStatus(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_) +ThreadGroup::ThreadGroup(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_) : master_thread_id(CurrentThread::get().thread_id) , query_context(query_context_) , global_context(query_context_->getGlobalContext()) , fatal_error_callback(fatal_error_callback_) {} -std::vector ThreadGroupStatus::getInvolvedThreadIds() const +std::vector ThreadGroup::getInvolvedThreadIds() const { std::vector res; @@ -60,22 +60,22 @@ std::vector ThreadGroupStatus::getInvolvedThreadIds() const return res; } -void ThreadGroupStatus::linkThread(UInt64 thread_it) +void ThreadGroup::linkThread(UInt64 thread_it) { std::lock_guard lock(mutex); thread_ids.insert(thread_it); } -ThreadGroupStatusPtr ThreadGroupStatus::createForQuery(ContextPtr query_context_, std::function fatal_error_callback_) +ThreadGroupPtr ThreadGroup::createForQuery(ContextPtr query_context_, std::function fatal_error_callback_) { - auto group = std::make_shared(query_context_, std::move(fatal_error_callback_)); + auto group = std::make_shared(query_context_, std::move(fatal_error_callback_)); group->memory_tracker.setDescription("(for query)"); return group; } -ThreadGroupStatusPtr ThreadGroupStatus::createForBackgroundProcess(ContextPtr storage_context) +ThreadGroupPtr ThreadGroup::createForBackgroundProcess(ContextPtr storage_context) { - auto group = std::make_shared(storage_context); + auto group = std::make_shared(storage_context); group->memory_tracker.setDescription("background process to apply mutate/merge in table"); /// However settings from storage context have to be applied @@ -89,7 +89,7 @@ ThreadGroupStatusPtr ThreadGroupStatus::createForBackgroundProcess(ContextPtr st return group; } -void ThreadGroupStatus::attachQueryForLog(const String & query_, UInt64 normalized_hash) +void ThreadGroup::attachQueryForLog(const String & query_, UInt64 normalized_hash) { auto hash = normalized_hash ? normalized_hash : normalizedQueryHash(query_); @@ -109,7 +109,7 @@ void ThreadStatus::attachQueryForLog(const String & query_) thread_group->attachQueryForLog(local_data.query_for_logs, local_data.normalized_query_hash); } -void ThreadGroupStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue) +void ThreadGroup::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue) { std::lock_guard lock(mutex); shared_data.profile_queue_ptr = profile_queue; @@ -168,7 +168,7 @@ void ThreadStatus::applyQuerySettings() #endif } -void ThreadStatus::attachToGroupImpl(const ThreadGroupStatusPtr & thread_group_) +void ThreadStatus::attachToGroupImpl(const ThreadGroupPtr & thread_group_) { /// Attach or init current thread to thread group and copy useful information from it thread_group = thread_group_; @@ -234,7 +234,7 @@ void ThreadStatus::setInternalThread() internal_thread = true; } -void ThreadStatus::attachToGroup(const ThreadGroupStatusPtr & thread_group_, bool check_detached) +void ThreadStatus::attachToGroup(const ThreadGroupPtr & thread_group_, bool check_detached) { if (thread_group && check_detached) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't attach query to the thread, it is already attached"); @@ -541,14 +541,14 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo) views_log->add(element); } -void CurrentThread::attachToGroup(const ThreadGroupStatusPtr & thread_group) +void CurrentThread::attachToGroup(const ThreadGroupPtr & thread_group) { if (unlikely(!current_thread)) return; current_thread->attachToGroup(thread_group, true); } -void CurrentThread::attachToGroupIfDetached(const ThreadGroupStatusPtr & thread_group) +void CurrentThread::attachToGroupIfDetached(const ThreadGroupPtr & thread_group) { if (unlikely(!current_thread)) return; @@ -574,7 +574,7 @@ CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context, std::func if (!query_context->hasQueryContext()) query_context->makeQueryContext(); - auto group = ThreadGroupStatus::createForQuery(query_context, std::move(fatal_error_callback)); + auto group = ThreadGroup::createForQuery(query_context, std::move(fatal_error_callback)); CurrentThread::attachToGroup(group); } @@ -584,7 +584,7 @@ CurrentThread::QueryScope::QueryScope(ContextPtr query_context, std::functiongetPort(IOutputFormat::PortKind::Main).getHeader(); } -static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads) +static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGroupPtr thread_group, size_t num_threads) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 3aec7608e6d..ac40cef35d9 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -97,7 +97,7 @@ struct PushingAsyncPipelineExecutor::Data } }; -static void threadFunction(PushingAsyncPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads) +static void threadFunction(PushingAsyncPipelineExecutor::Data & data, ThreadGroupPtr thread_group, size_t num_threads) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp index 3fc57ca1c1e..62ee4e4a48d 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp @@ -96,7 +96,7 @@ namespace DB } - void ParallelFormattingOutputFormat::collectorThreadFunction(const ThreadGroupStatusPtr & thread_group) + void ParallelFormattingOutputFormat::collectorThreadFunction(const ThreadGroupPtr & thread_group) { SCOPE_EXIT_SAFE( if (thread_group) @@ -157,7 +157,7 @@ namespace DB } - void ParallelFormattingOutputFormat::formatterThreadFunction(size_t current_unit_number, size_t first_row_num, const ThreadGroupStatusPtr & thread_group) + void ParallelFormattingOutputFormat::formatterThreadFunction(size_t current_unit_number, size_t first_row_num, const ThreadGroupPtr & thread_group) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h index 790d05e83dd..4e5aaab5dcb 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h @@ -270,10 +270,10 @@ private: } /// Collects all temporary buffers into main WriteBuffer. - void collectorThreadFunction(const ThreadGroupStatusPtr & thread_group); + void collectorThreadFunction(const ThreadGroupPtr & thread_group); /// This function is executed in ThreadPool and the only purpose of it is to format one Chunk into a continuous buffer in memory. - void formatterThreadFunction(size_t current_unit_number, size_t first_row_num, const ThreadGroupStatusPtr & thread_group); + void formatterThreadFunction(size_t current_unit_number, size_t first_row_num, const ThreadGroupPtr & thread_group); void setRowsBeforeLimit(size_t rows_before_limit) override { diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp index a2e5074efb1..f4d619a263b 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp @@ -8,7 +8,7 @@ namespace DB { -void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr thread_group) +void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupPtr thread_group) { SCOPE_EXIT_SAFE( if (thread_group) @@ -62,7 +62,7 @@ void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr } } -void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupStatusPtr thread_group, size_t current_ticket_number) +void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupPtr thread_group, size_t current_ticket_number) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h index 97df9308dbf..ae9f123d411 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h @@ -317,8 +317,8 @@ private: } } - void segmentatorThreadFunction(ThreadGroupStatusPtr thread_group); - void parserThreadFunction(ThreadGroupStatusPtr thread_group, size_t current_ticket_number); + void segmentatorThreadFunction(ThreadGroupPtr thread_group); + void parserThreadFunction(ThreadGroupPtr thread_group, size_t current_ticket_number); /// Save/log a background exception, set termination flag, wake up all /// threads. This function is used by segmentator and parsed threads. diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 91845bc18ad..e32e2116f71 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -270,14 +270,14 @@ Chain buildPushingToViewsChain( ASTPtr query; Chain out; - /// NOTE: ThreadGroupStatus always should have context attached, + /// NOTE: ThreadGroup always should have context attached, /// otherwise entry to the system.query_views_log will not be added /// (see ThreadStatus::logToQueryViewsLog()) - ThreadGroupStatusPtr running_group; + ThreadGroupPtr running_group; if (current_thread) running_group = current_thread->getThreadGroup(); if (!running_group) - running_group = std::make_shared(context); + running_group = std::make_shared(context); /// We are creating a ThreadStatus per view to store its metrics individually /// Since calling ThreadStatus() changes current_thread we save it and restore it after the calls diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 7970efd1a20..91e1cb1b078 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -11,7 +11,7 @@ namespace DB { -ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupStatusPtr thread_group) +ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupPtr thread_group) { chassert(thread_group); @@ -59,7 +59,7 @@ MergeListElement::MergeListElement( is_mutation = (result_part_info.getDataVersion() != source_data_version); } - thread_group = ThreadGroupStatus::createForBackgroundProcess(context); + thread_group = ThreadGroup::createForBackgroundProcess(context); } MergeInfo MergeListElement::getInfo() const diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index c3ba4f00f7f..308f00feda9 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -69,11 +69,11 @@ struct Settings; class ThreadGroupSwitcher : private boost::noncopyable { public: - explicit ThreadGroupSwitcher(ThreadGroupStatusPtr thread_group); + explicit ThreadGroupSwitcher(ThreadGroupPtr thread_group); ~ThreadGroupSwitcher(); private: - ThreadGroupStatusPtr prev_thread_group; + ThreadGroupPtr prev_thread_group; }; struct MergeListElement : boost::noncopyable @@ -113,7 +113,7 @@ struct MergeListElement : boost::noncopyable /// Detected after merge already started std::atomic merge_algorithm; - ThreadGroupStatusPtr thread_group; + ThreadGroupPtr thread_group; MergeListElement( const StorageID & table_id_, From aacf2a083882bf395d94a6f7dba1fa2d35db2991 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 7 Apr 2023 15:15:10 +0200 Subject: [PATCH 201/277] Move ThreadGroupSwitcher to ThreadStatus.h (out from MergeTree code) Signed-off-by: Azat Khuzhin --- src/Common/ThreadStatus.h | 16 +++++++++++++++- src/Interpreters/ThreadStatusExt.cpp | 18 ++++++++++++++++++ src/Storages/MergeTree/MergeList.cpp | 18 ------------------ src/Storages/MergeTree/MergeList.h | 14 -------------- 4 files changed, 33 insertions(+), 33 deletions(-) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 16083fe0925..600dfc56d2b 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -41,7 +41,6 @@ class TaskStatsInfoGetter; class InternalTextLogsQueue; struct ViewRuntimeData; class QueryViewsLog; -class ThreadGroupSwitcher; using InternalTextLogsQueuePtr = std::shared_ptr; using InternalTextLogsQueueWeakPtr = std::weak_ptr; @@ -120,6 +119,21 @@ private: std::unordered_set thread_ids; }; +/** + * Since merge is executed with multiple threads, this class + * switches the parent MemoryTracker as part of the thread group to account all the memory used. + */ +class ThreadGroupSwitcher : private boost::noncopyable +{ +public: + explicit ThreadGroupSwitcher(ThreadGroupPtr thread_group); + ~ThreadGroupSwitcher(); + +private: + ThreadGroupPtr prev_thread_group; +}; + + /** * We use **constinit** here to tell the compiler the current_thread variable is initialized. * If we didn't help the compiler, then it would most likely add a check before every use of the variable to initialize it if needed. diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 070cd3f98e1..c6e36263e7c 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -115,6 +115,24 @@ void ThreadGroup::attachInternalProfileEventsQueue(const InternalProfileEventsQu shared_data.profile_queue_ptr = profile_queue; } +ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupPtr thread_group) +{ + chassert(thread_group); + + /// might be nullptr + prev_thread_group = CurrentThread::getGroup(); + + CurrentThread::detachFromGroupIfNotDetached(); + CurrentThread::attachToGroup(thread_group); +} + +ThreadGroupSwitcher::~ThreadGroupSwitcher() +{ + CurrentThread::detachFromGroupIfNotDetached(); + if (prev_thread_group) + CurrentThread::attachToGroup(prev_thread_group); +} + void ThreadStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue) { if (!thread_group) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 91e1cb1b078..0bf662921ad 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -11,24 +11,6 @@ namespace DB { -ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupPtr thread_group) -{ - chassert(thread_group); - - /// might be nullptr - prev_thread_group = CurrentThread::getGroup(); - - CurrentThread::detachFromGroupIfNotDetached(); - CurrentThread::attachToGroup(thread_group); -} - -ThreadGroupSwitcher::~ThreadGroupSwitcher() -{ - CurrentThread::detachFromGroupIfNotDetached(); - if (prev_thread_group) - CurrentThread::attachToGroup(prev_thread_group); -} - MergeListElement::MergeListElement( const StorageID & table_id_, FutureMergedMutatedPartPtr future_part, diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 308f00feda9..9c8c2ebd1e4 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -62,20 +62,6 @@ using MergeListEntry = BackgroundProcessListEntry; struct Settings; -/** - * Since merge is executed with multiple threads, this class - * switches the parent MemoryTracker as part of the thread group to account all the memory used. - */ -class ThreadGroupSwitcher : private boost::noncopyable -{ -public: - explicit ThreadGroupSwitcher(ThreadGroupPtr thread_group); - ~ThreadGroupSwitcher(); - -private: - ThreadGroupPtr prev_thread_group; -}; - struct MergeListElement : boost::noncopyable { const StorageID table_id; From 44ae8485f185ba43323d61a2e85007f5067b4e2b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 7 Apr 2023 15:29:51 +0200 Subject: [PATCH 202/277] Use one ThreadGroup while pushing to materialized views Before this patch only the case when ThreadStatus (current_thread) already has ThreadGroup works that way, after they will be identical in this aspect. But this should not affect anything, but it just make sense. Signed-off-by: Azat Khuzhin --- src/Interpreters/InterpreterInsertQuery.cpp | 24 ++++++++++++++++--- src/Interpreters/InterpreterInsertQuery.h | 1 + .../Transforms/buildPushingToViewsChain.cpp | 22 ++++++++--------- .../Transforms/buildPushingToViewsChain.h | 4 ++++ 4 files changed, 36 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index b4a19ea7403..e78a61831a1 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include @@ -233,8 +234,14 @@ Chain InterpreterInsertQuery::buildChain( ThreadStatusesHolderPtr thread_status_holder, std::atomic_uint64_t * elapsed_counter_ms) { + ThreadGroupPtr running_group; + if (current_thread) + running_group = current_thread->getThreadGroup(); + if (!running_group) + running_group = std::make_shared(getContext()); + auto sample = getSampleBlock(columns, table, metadata_snapshot); - return buildChainImpl(table, metadata_snapshot, sample, thread_status_holder, elapsed_counter_ms); + return buildChainImpl(table, metadata_snapshot, sample, thread_status_holder, running_group, elapsed_counter_ms); } Chain InterpreterInsertQuery::buildChainImpl( @@ -242,6 +249,7 @@ Chain InterpreterInsertQuery::buildChainImpl( const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block, ThreadStatusesHolderPtr thread_status_holder, + ThreadGroupPtr running_group, std::atomic_uint64_t * elapsed_counter_ms) { ThreadStatus * thread_status = current_thread; @@ -273,7 +281,9 @@ Chain InterpreterInsertQuery::buildChainImpl( } else { - out = buildPushingToViewsChain(table, metadata_snapshot, context_ptr, query_ptr, no_destination, thread_status_holder, elapsed_counter_ms); + out = buildPushingToViewsChain(table, metadata_snapshot, context_ptr, + query_ptr, no_destination, + thread_status_holder, running_group, elapsed_counter_ms); } /// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order. @@ -461,9 +471,17 @@ BlockIO InterpreterInsertQuery::execute() pipeline = interpreter_watch.buildQueryPipeline(); } + ThreadGroupPtr running_group; + if (current_thread) + running_group = current_thread->getThreadGroup(); + if (!running_group) + running_group = std::make_shared(getContext()); for (size_t i = 0; i < out_streams_size; ++i) { - auto out = buildChainImpl(table, metadata_snapshot, query_sample_block, nullptr, nullptr); + auto out = buildChainImpl(table, metadata_snapshot, query_sample_block, + /* thread_status_holder= */ nullptr, + running_group, + /* elapsed_counter_ms= */ nullptr); out_chains.emplace_back(std::move(out)); } } diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index bb6509a9102..f60d6567d74 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -70,6 +70,7 @@ private: const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block, ThreadStatusesHolderPtr thread_status_holder, + ThreadGroupPtr running_group, std::atomic_uint64_t * elapsed_counter_ms); }; diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index e32e2116f71..0bdd7a88851 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -195,6 +195,7 @@ Chain buildPushingToViewsChain( const ASTPtr & query_ptr, bool no_destination, ThreadStatusesHolderPtr thread_status_holder, + ThreadGroupPtr running_group, std::atomic_uint64_t * elapsed_counter_ms, const Block & live_view_header) { @@ -270,15 +271,6 @@ Chain buildPushingToViewsChain( ASTPtr query; Chain out; - /// NOTE: ThreadGroup always should have context attached, - /// otherwise entry to the system.query_views_log will not be added - /// (see ThreadStatus::logToQueryViewsLog()) - ThreadGroupPtr running_group; - if (current_thread) - running_group = current_thread->getThreadGroup(); - if (!running_group) - running_group = std::make_shared(context); - /// We are creating a ThreadStatus per view to store its metrics individually /// Since calling ThreadStatus() changes current_thread we save it and restore it after the calls /// Later on, before doing any task related to a view, we'll switch to its ThreadStatus, do the work, @@ -354,18 +346,24 @@ Chain buildPushingToViewsChain( runtime_stats->type = QueryViewsLogElement::ViewType::LIVE; query = live_view->getInnerQuery(); // Used only to log in system.query_views_log out = buildPushingToViewsChain( - view, view_metadata_snapshot, insert_context, ASTPtr(), true, thread_status_holder, view_counter_ms, storage_header); + view, view_metadata_snapshot, insert_context, ASTPtr(), + /* no_destination= */ true, + thread_status_holder, running_group, view_counter_ms, storage_header); } else if (auto * window_view = dynamic_cast(view.get())) { runtime_stats->type = QueryViewsLogElement::ViewType::WINDOW; query = window_view->getMergeableQuery(); // Used only to log in system.query_views_log out = buildPushingToViewsChain( - view, view_metadata_snapshot, insert_context, ASTPtr(), true, thread_status_holder, view_counter_ms); + view, view_metadata_snapshot, insert_context, ASTPtr(), + /* no_destination= */ true, + thread_status_holder, running_group, view_counter_ms); } else out = buildPushingToViewsChain( - view, view_metadata_snapshot, insert_context, ASTPtr(), false, thread_status_holder, view_counter_ms); + view, view_metadata_snapshot, insert_context, ASTPtr(), + /* no_destination= */ false, + thread_status_holder, running_group, view_counter_ms); views_data->views.emplace_back(ViewRuntimeData{ std::move(query), diff --git a/src/Processors/Transforms/buildPushingToViewsChain.h b/src/Processors/Transforms/buildPushingToViewsChain.h index 76131a8df6e..a2e7e39ff23 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.h +++ b/src/Processors/Transforms/buildPushingToViewsChain.h @@ -61,6 +61,10 @@ Chain buildPushingToViewsChain( /// We could specify separate thread_status for each view. /// Needed mainly to collect counters separately. Should be improved. ThreadStatusesHolderPtr thread_status_holder, + /// Usually current_thread->getThreadGroup(), but sometimes ThreadStatus + /// may not have ThreadGroup (i.e. Buffer background flush), and in this + /// case it should be passed outside. + ThreadGroupPtr running_group, /// Counter to measure time spent separately per view. Should be improved. std::atomic_uint64_t * elapsed_counter_ms, /// LiveView executes query itself, it needs source block structure. From 396f6a6421b49f775119663718d1fdc98953e48b Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 7 Apr 2023 17:07:36 +0200 Subject: [PATCH 203/277] Update src/Storages/StorageReplicatedMergeTree.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 600168d2637..1f081523ed9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8877,7 +8877,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP if (zookeeper->tryGet(lost_part_count_path, lost_part_count_str, &lost_part_count_stat)) { UInt64 lost_part_count = lost_part_count_str.empty() ? 0 : parse(lost_part_count_str); - ops.emplace_back(zkutil::makeSetRequest(lost_part_count_path, fmt::format("{}", lost_part_count + 1), lost_part_count_stat.version)); + ops.emplace_back(zkutil::makeSetRequest(lost_part_count_path, toString(lost_part_count + 1), lost_part_count_stat.version)); } else { From d0a37f8798a2c87b37e31a4fa4f1b656b11d5982 Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Sat, 8 Apr 2023 01:44:43 +0800 Subject: [PATCH 204/277] fix test test_multiple_disks --- tests/integration/test_multiple_disks/test.py | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 221d830f62e..f6e5d456dfb 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -90,6 +90,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "small_jbod_with_external", @@ -100,6 +102,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "small_jbod_with_external_no_merges", @@ -110,6 +114,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "small_jbod_with_external_no_merges", @@ -120,6 +126,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 1, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "one_more_small_jbod_with_external", @@ -130,6 +138,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "one_more_small_jbod_with_external", @@ -140,6 +150,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "jbods_with_external", @@ -150,6 +162,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "10485760", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "jbods_with_external", @@ -160,6 +174,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "moving_jbod_with_external", @@ -170,6 +186,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.7, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "moving_jbod_with_external", @@ -180,6 +198,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.7, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "default_disk_with_external", @@ -190,6 +210,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "2097152", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "default_disk_with_external", @@ -200,6 +222,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "20971520", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "special_warning_policy", @@ -210,6 +234,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "special_warning_policy", @@ -220,6 +246,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "0", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "special_warning_policy", @@ -230,6 +258,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "1024", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, { "policy_name": "special_warning_policy", @@ -240,6 +270,8 @@ def test_system_tables(start_cluster): "max_data_part_size": "1024000000", "move_factor": 0.1, "prefer_not_to_merge": 0, + "perform_ttl_move_on_insert": 1, + "load_balancing": "ROUND_ROBIN", }, ] From b6261104e700354de56c5e6e5c4a54e0ab25bcaa Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 6 Apr 2023 23:04:51 +0000 Subject: [PATCH 205/277] Fix bracketed-paste mode messing up password input in client reconnect --- src/Client/ClientBase.cpp | 17 +++++++++++++---- src/Client/LineReader.h | 3 +++ src/Client/ReplxxLineReader.cpp | 6 ++++++ src/Client/ReplxxLineReader.h | 1 + 4 files changed, 23 insertions(+), 4 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 120d273aa62..a5296a143e1 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -7,6 +7,7 @@ #include #include +#include #include #include #include @@ -2219,9 +2220,6 @@ void ClientBase::runInteractive() LineReader lr(history_file, config().has("multiline"), query_extenders, query_delimiters); #endif - /// Enable bracketed-paste-mode so that we are able to paste multiline queries as a whole. - lr.enableBracketedPaste(); - static const std::initializer_list> backslash_aliases = { { "\\l", "SHOW DATABASES" }, @@ -2239,7 +2237,18 @@ void ClientBase::runInteractive() do { - auto input = lr.readLine(prompt(), ":-] "); + String input; + { + /// Enable bracketed-paste-mode so that we are able to paste multiline queries as a whole. + /// But keep it disabled outside of query input, because it breaks password input + /// (e.g. if we need to reconnect and show a password prompt). + /// (Alternatively, we could make the password input ignore the control sequences.) + lr.enableBracketedPaste(); + SCOPE_EXIT({ lr.disableBracketedPaste(); }); + + input = lr.readLine(prompt(), ":-] "); + } + if (input.empty()) break; diff --git a/src/Client/LineReader.h b/src/Client/LineReader.h index 321cf41b77e..df64a3a85a9 100644 --- a/src/Client/LineReader.h +++ b/src/Client/LineReader.h @@ -46,7 +46,10 @@ public: /// clickhouse-client so that without -m flag, one can still paste multiline queries, and /// possibly get better pasting performance. See https://cirw.in/blog/bracketed-paste for /// more details. + /// These methods (if implemented) emit the control characters immediately, without waiting + /// for the next readLine() call. virtual void enableBracketedPaste() {} + virtual void disableBracketedPaste() {} protected: enum InputStatus diff --git a/src/Client/ReplxxLineReader.cpp b/src/Client/ReplxxLineReader.cpp index 1979b37a94b..180be77ca1c 100644 --- a/src/Client/ReplxxLineReader.cpp +++ b/src/Client/ReplxxLineReader.cpp @@ -519,4 +519,10 @@ void ReplxxLineReader::enableBracketedPaste() rx.enable_bracketed_paste(); } +void ReplxxLineReader::disableBracketedPaste() +{ + bracketed_paste_enabled = false; + rx.disable_bracketed_paste(); +} + } diff --git a/src/Client/ReplxxLineReader.h b/src/Client/ReplxxLineReader.h index d36a1d0f42c..5cb8e48eb86 100644 --- a/src/Client/ReplxxLineReader.h +++ b/src/Client/ReplxxLineReader.h @@ -19,6 +19,7 @@ public: ~ReplxxLineReader() override; void enableBracketedPaste() override; + void disableBracketedPaste() override; /// If highlight is on, we will set a flag to denote whether the last token is a delimiter. /// This is useful to determine the behavior of key when multiline is enabled. From 18a2eb355e9eae7fbac6cb6b9b8a64505584c452 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 7 Apr 2023 20:16:25 +0200 Subject: [PATCH 206/277] fix a bug in sync replica --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 4 +++- src/Storages/StorageReplicatedMergeTree.cpp | 1 - 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index c8ce55f9600..623210ae04c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1147,7 +1147,8 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( *it, /* is_successful = */ false, min_unprocessed_insert_time_changed, max_processed_insert_time_changed, lock); - (*it)->removed_by_other_entry = true; + LogEntryPtr removing_entry = std::move(*it); /// Make it live a bit longer + removing_entry->removed_by_other_entry = true; it = queue.erase(it); notifySubscribers(queue.size(), &znode_name); ++removed_entries; @@ -2491,6 +2492,7 @@ ReplicatedMergeTreeQueue::addSubscriber(ReplicatedMergeTreeQueue::SubscriberCall || std::find(lightweight_entries.begin(), lightweight_entries.end(), entry->type) != lightweight_entries.end()) out_entry_names.insert(entry->znode_name); } + LOG_TEST(log, "Waiting for {} entries to be processed: {}", out_entry_names.size(), fmt::join(out_entry_names, ", ")); } auto it = subscribers.emplace(subscribers.end(), std::move(callback)); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index cbfe3f8cab2..734da8b85a3 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7616,7 +7616,6 @@ bool StorageReplicatedMergeTree::waitForProcessingQueue(UInt64 max_wait_millisec if (removed_log_entry_id) wait_for_ids.erase(*removed_log_entry_id); - chassert(new_queue_size || wait_for_ids.empty()); if (wait_for_ids.empty()) target_entry_event.set(); }; From 9929185eab1a11e0ee7f937db4c93ef871d74ba5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 7 Apr 2023 20:31:03 +0200 Subject: [PATCH 207/277] fix REPLACE_RANGE --- src/Storages/StorageReplicatedMergeTree.cpp | 62 ++++++++++++--------- 1 file changed, 35 insertions(+), 27 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 734da8b85a3..e538b4fbe6c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2209,35 +2209,43 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) /// Check that we could cover whole range for (PartDescriptionPtr & part_desc : parts_to_add) { - if (adding_parts_active_set.getContainingPart(part_desc->new_part_info).empty()) + if (!adding_parts_active_set.getContainingPart(part_desc->new_part_info).empty()) + continue; + + MergeTreePartInfo covering_drop_range; + if (queue.isGoingToBeDropped(part_desc->new_part_info, &covering_drop_range)) { - /// We should enqueue missing part for check, so it will be replaced with empty one (if needed) - /// and we will be able to execute this REPLACE_RANGE. - /// However, it's quite dangerous, because part may appear in source table. - /// So we enqueue it for check only if no replicas of source table have part either. - bool need_check = true; - if (auto * replicated_src_table = typeid_cast(source_table.get())) - { - String src_replica = replicated_src_table->findReplicaHavingPart(part_desc->src_part_name, false); - if (!src_replica.empty()) - { - LOG_DEBUG(log, "Found part {} on replica {} of source table, will not check part {} required for {}", - part_desc->src_part_name, src_replica, part_desc->new_part_name, entry.znode_name); - need_check = false; - } - } - - if (need_check) - { - LOG_DEBUG(log, "Will check part {} required for {}, because no replicas have it (including replicas of source table)", - part_desc->new_part_name, entry.znode_name); - enqueuePartForCheck(part_desc->new_part_name); - } - - throw Exception(ErrorCodes::NO_REPLICA_HAS_PART, - "Not found part {} (or part covering it) neither source table neither remote replicas", - part_desc->new_part_name); + LOG_WARNING(log, "Will not add part {} (while replacing {}) because it's going to be dropped (DROP_RANGE: {})", + part_desc->new_part_name, entry_replace.drop_range_part_name, covering_drop_range.getPartNameForLogs()); + continue; } + + /// We should enqueue missing part for check, so it will be replaced with empty one (if needed) + /// and we will be able to execute this REPLACE_RANGE. + /// However, it's quite dangerous, because part may appear in source table. + /// So we enqueue it for check only if no replicas of source table have part either. + bool need_check = true; + if (auto * replicated_src_table = typeid_cast(source_table.get())) + { + String src_replica = replicated_src_table->findReplicaHavingPart(part_desc->src_part_name, false); + if (!src_replica.empty()) + { + LOG_DEBUG(log, "Found part {} on replica {} of source table, will not check part {} required for {}", + part_desc->src_part_name, src_replica, part_desc->new_part_name, entry.znode_name); + need_check = false; + } + } + + if (need_check) + { + LOG_DEBUG(log, "Will check part {} required for {}, because no replicas have it (including replicas of source table)", + part_desc->new_part_name, entry.znode_name); + enqueuePartForCheck(part_desc->new_part_name); + } + + throw Exception(ErrorCodes::NO_REPLICA_HAS_PART, + "Not found part {} (or part covering it) neither source table neither remote replicas", + part_desc->new_part_name); } /// Filter covered parts From 8fdc2b33260ef8c02ae79b4d5d98e83df3f233e7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Apr 2023 20:06:11 +0000 Subject: [PATCH 208/277] Perf test --- tests/performance/reading_from_file.xml | 9 +++++++++ 1 file changed, 9 insertions(+) create mode 100644 tests/performance/reading_from_file.xml diff --git a/tests/performance/reading_from_file.xml b/tests/performance/reading_from_file.xml new file mode 100644 index 00000000000..992f1e8acae --- /dev/null +++ b/tests/performance/reading_from_file.xml @@ -0,0 +1,9 @@ + + +INSERT INTO function file(reading_from_file.parquet) SELECT URL FROM test.hits LIMIT 100000 SETTINGS engine_file_truncate_on_insert=1 + +SELECT sum(length(base58Encode(URL))) FROM file(reading_from_file.parquet) FORMAT Null + +INSERT INTO FUNCTION file(reading_from_file.parquet) SELECT * FROM numbers(0) SETTINGS engine_file_truncate_on_insert=1 + + From e39ee3f0af5a2e6e8ab9888ed8e9020752870bbf Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 7 Apr 2023 22:54:49 +0200 Subject: [PATCH 209/277] suppress a bug in filesystem cache --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 85420cabb8d..5ed4987a6d5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -34,6 +34,8 @@ #include #include +#include + namespace CurrentMetrics { @@ -1525,6 +1527,10 @@ bool IMergeTreeDataPart::assertHasValidVersionMetadata() const size_t file_size = getDataPartStorage().getFileSize(TXN_VERSION_METADATA_FILE_NAME); auto buf = getDataPartStorage().readFile(TXN_VERSION_METADATA_FILE_NAME, ReadSettings().adjustBufferSize(file_size), file_size, std::nullopt); + /// FIXME https://github.com/ClickHouse/ClickHouse/issues/48465 + if (dynamic_cast(buf.get())) + return true; + readStringUntilEOF(content, *buf); ReadBufferFromString str_buf{content}; VersionMetadata file; From bf3a08dc98842288d1b433efcfd614103fc8e1cc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 7 Apr 2023 22:58:08 +0200 Subject: [PATCH 210/277] Fix 02477_projection_materialize_and_zero_copy flakienss (due to index granularity randomization) Signed-off-by: Azat Khuzhin --- .../0_stateless/02477_projection_materialize_and_zero_copy.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql index d4c24b31da2..1845919890c 100644 --- a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql +++ b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS t; create table t (c1 Int64, c2 String, c3 DateTime, c4 Int8, c5 String, c6 String, c7 String, c8 String, c9 String, c10 String, c11 String, c12 String, c13 Int8, c14 Int64, c15 String, c16 String, c17 String, c18 Int64, c19 Int64, c20 Int64) engine ReplicatedMergeTree('/clickhouse/test/{database}/test_02477', '1') order by c18 -SETTINGS allow_remote_fs_zero_copy_replication=1; +SETTINGS allow_remote_fs_zero_copy_replication=1, index_granularity=8092, index_granularity_bytes='10Mi'; insert into t (c1, c18) select number, -number from numbers(2000000); From 3633f0b2f3d479b884d1e70c2d4bcb0e817d2d19 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 7 Apr 2023 23:50:09 +0200 Subject: [PATCH 211/277] fix --- .github/workflows/backport_branches.yml | 7 +++++++ .github/workflows/master.yml | 7 +++++++ .github/workflows/release_branches.yml | 7 +++++++ docs/en/development/developer-instruction.md | 2 +- docs/ru/development/developer-instruction.md | 2 +- 5 files changed, 23 insertions(+), 2 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 0d81a7b303c..d69168b01ee 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -349,6 +349,13 @@ jobs: with: clear-repository: true submodules: true + - name: Apply sparse checkout for contrib # in order to check that it doesn't break build + run: | + rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' + git -C "$GITHUB_WORKSPACE" checkout . && echo 'restored' + "$GITHUB_WORKSPACE/contrib/update-submodules.sh" && echo 'OK' + du -hs "$GITHUB_WORKSPACE/contrib" ||: + find "$GITHUB_WORKSPACE/contrib" -type f | wc -l ||: - name: Build run: | sudo rm -fr "$TEMP_PATH" diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index ecd5b85d320..1182481c897 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -487,6 +487,13 @@ jobs: with: clear-repository: true submodules: true + - name: Apply sparse checkout for contrib # in order to check that it doesn't break build + run: | + rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' + git -C "$GITHUB_WORKSPACE" checkout . && echo 'restored' + "$GITHUB_WORKSPACE/contrib/update-submodules.sh" && echo 'OK' + du -hs "$GITHUB_WORKSPACE/contrib" ||: + find "$GITHUB_WORKSPACE/contrib" -type f | wc -l ||: - name: Build run: | sudo rm -fr "$TEMP_PATH" diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 1282dbef50b..21284815583 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -406,6 +406,13 @@ jobs: with: clear-repository: true submodules: true + - name: Apply sparse checkout for contrib # in order to check that it doesn't break build + run: | + rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' + git -C "$GITHUB_WORKSPACE" checkout . && echo 'restored' + "$GITHUB_WORKSPACE/contrib/update-submodules.sh" && echo 'OK' + du -hs "$GITHUB_WORKSPACE/contrib" ||: + find "$GITHUB_WORKSPACE/contrib" -type f | wc -l ||: - name: Build run: | sudo rm -fr "$TEMP_PATH" diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index ea98b2da5e6..6bcdadeb1eb 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -39,7 +39,7 @@ Next, you need to download the source files onto your working machine. This is c In the command line terminal run: - git clone --recursive --shallow-submodules git@github.com:your_github_username/ClickHouse.git + git clone --shallow-submodules git@github.com:your_github_username/ClickHouse.git cd ClickHouse Or (if you'd like to use sparse checkout for submodules and avoid checking out unneeded files): diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index c208439678a..7294bc2ae87 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -41,7 +41,7 @@ ClickHouse не работает и не ÑобираетÑÑ Ð½Ð° 32-битны Выполните в терминале: - git clone --recursive --shallow-submodules git@github.com:your_github_username/ClickHouse.git + git clone --shallow-submodules git@github.com:your_github_username/ClickHouse.git cd ClickHouse Или (еÑли вы хотите иÑпользовать sparse checkout Ð´Ð»Ñ submodules): From ec60a1069428e7a127e1c6d440711abd26c980e7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Apr 2023 00:54:05 +0200 Subject: [PATCH 212/277] Better exception message for ZSTD --- src/IO/ZstdDeflatingAppendableWriteBuffer.cpp | 8 ++++---- src/IO/ZstdDeflatingWriteBuffer.cpp | 2 +- src/IO/ZstdInflatingReadBuffer.cpp | 15 +++++++++++---- 3 files changed, 16 insertions(+), 9 deletions(-) diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp index f8c4d0e2bac..be739c0e654 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp @@ -23,11 +23,11 @@ ZstdDeflatingAppendableWriteBuffer::ZstdDeflatingAppendableWriteBuffer( { cctx = ZSTD_createCCtx(); if (cctx == nullptr) - throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, "zstd stream encoder init failed: zstd version: {}", ZSTD_VERSION_STRING); + throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, "ZSTD stream encoder init failed: ZSTD version: {}", ZSTD_VERSION_STRING); size_t ret = ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, compression_level); if (ZSTD_isError(ret)) throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, - "zstd stream encoder option setting failed: error code: {}; zstd version: {}", + "ZSTD stream encoder option setting failed: error code: {}; zstd version: {}", ret, ZSTD_VERSION_STRING); input = {nullptr, 0, 0}; @@ -64,7 +64,7 @@ void ZstdDeflatingAppendableWriteBuffer::nextImpl() if (ZSTD_isError(compression_result)) throw Exception( ErrorCodes::ZSTD_ENCODER_FAILED, - "Zstd stream encoding failed: error code: {}; zstd version: {}", + "ZSTD stream decoding failed: error code: {}; ZSTD version: {}", ZSTD_getErrorName(compression_result), ZSTD_VERSION_STRING); first_write = false; @@ -138,7 +138,7 @@ void ZstdDeflatingAppendableWriteBuffer::finalizeBefore() { if (ZSTD_isError(remaining)) throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, - "Zstd stream encoder end failed: error: '{}' zstd version: {}", + "ZSTD stream encoder end failed: error: '{}' ZSTD version: {}", ZSTD_getErrorName(remaining), ZSTD_VERSION_STRING); remaining = ZSTD_compressStream2(cctx, &output, &input, ZSTD_e_end); diff --git a/src/IO/ZstdDeflatingWriteBuffer.cpp b/src/IO/ZstdDeflatingWriteBuffer.cpp index a12b59b80f5..c6d2ffc39f9 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingWriteBuffer.cpp @@ -63,7 +63,7 @@ void ZstdDeflatingWriteBuffer::nextImpl() if (ZSTD_isError(compression_result)) throw Exception( ErrorCodes::ZSTD_ENCODER_FAILED, - "Zstd stream encoding failed: error: '{}'; zstd version: {}", + "ZSTD stream encoding failed: error: '{}'; zstd version: {}", ZSTD_getErrorName(compression_result), ZSTD_VERSION_STRING); out->position() = out->buffer().begin() + output.pos; diff --git a/src/IO/ZstdInflatingReadBuffer.cpp b/src/IO/ZstdInflatingReadBuffer.cpp index 6956bdb6710..6f5c8b4dc71 100644 --- a/src/IO/ZstdInflatingReadBuffer.cpp +++ b/src/IO/ZstdInflatingReadBuffer.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB @@ -56,11 +57,17 @@ bool ZstdInflatingReadBuffer::nextImpl() /// Decompress data and check errors. size_t ret = ZSTD_decompressStream(dctx, &output, &input); - if (ZSTD_isError(ret)) + if (ZSTD_getErrorCode(ret)) + { throw Exception( - ErrorCodes::ZSTD_DECODER_FAILED, - "Zstd stream encoding failed: error '{}'; zstd version: {}", - ZSTD_getErrorName(ret), ZSTD_VERSION_STRING); + ErrorCodes::ZSTD_DECODER_FAILED, + "ZSTD stream decoding failed: error '{}'{}; ZSTD version: {}", + ZSTD_getErrorName(ret), + ZSTD_error_frameParameter_windowTooLarge == ret + ? ". You can increase the maximum window size with the 'zstd_window_log_max' setting in ClickHouse. Example: 'SET zstd_window_log_max = 31'" + : "", + ZSTD_VERSION_STRING); + } /// Check that something has changed after decompress (input or output position) assert(in->eof() || output.pos > 0 || in->position() < in->buffer().begin() + input.pos); From 2b00c8cb9b4b62f4b5d251e7dcb3b54e33a43754 Mon Sep 17 00:00:00 2001 From: Artem Pershin Date: Sat, 8 Apr 2023 18:39:04 +0300 Subject: [PATCH 213/277] Fix typo in word acquired, in comment for lockForAlter method --- src/Storages/IStorage.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 426d64f41ee..351e147e6cd 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -275,7 +275,7 @@ public: /// acquiring the lock instead of raising a TABLE_IS_DROPPED exception TableLockHolder tryLockForShare(const String & query_id, const std::chrono::milliseconds & acquire_timeout); - /// Lock table for alter. This lock must be acuqired in ALTER queries to be + /// Lock table for alter. This lock must be acquired in ALTER queries to be /// sure, that we execute only one simultaneous alter. Doesn't affect share lock. using AlterLockHolder = std::unique_lock; AlterLockHolder lockForAlter(const std::chrono::milliseconds & acquire_timeout); From 0dc3193e6bf3a29a50242b1743ef1a652570adcc Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 8 Apr 2023 17:29:08 +0000 Subject: [PATCH 214/277] Fix 02293_formats_json_columns --- .../02293_formats_json_columns.reference | 14 +++++++------- .../0_stateless/02293_formats_json_columns.sh | 16 ++++++++-------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/02293_formats_json_columns.reference b/tests/queries/0_stateless/02293_formats_json_columns.reference index f59f02ad42b..d3fb006a73f 100644 --- a/tests/queries/0_stateless/02293_formats_json_columns.reference +++ b/tests/queries/0_stateless/02293_formats_json_columns.reference @@ -86,18 +86,18 @@ d Nullable(String) \N \N 3 \N \N \N \N String OK -3 -2 1 +2 +3 c1 Nullable(Int64) c2 Nullable(Int64) c3 Nullable(String) 1 1 \N -2 2 \N -3 3 \N -1 \N \N -2 \N \N -3 \N \N 1 2 String +1 \N \N +2 2 \N +2 \N \N +3 3 \N +3 \N \N OK OK diff --git a/tests/queries/0_stateless/02293_formats_json_columns.sh b/tests/queries/0_stateless/02293_formats_json_columns.sh index 7a21f8d9bab..0aaf2abfc45 100755 --- a/tests/queries/0_stateless/02293_formats_json_columns.sh +++ b/tests/queries/0_stateless/02293_formats_json_columns.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +USER_FILES_PATH=$(clickhouse client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') DATA_FILE=$USER_FILES_PATH/data_02293 @@ -17,13 +17,13 @@ echo "JSONColumns" $CLICKHOUSE_CLIENT -q "select * from test_02293 order by a format JSONColumns" $CLICKHOUSE_CLIENT -q "select * from test_02293 order by a format JSONColumns" > $DATA_FILE $CLICKHOUSE_CLIENT -q "desc file(data_02293, JSONColumns)" -$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns)" +$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns) order by a" echo "JSONCompactColumns" $CLICKHOUSE_CLIENT -q "select * from test_02293 order by a format JSONCompactColumns" $CLICKHOUSE_CLIENT -q "select * from test_02293 order by a format JSONCompactColumns" > $DATA_FILE $CLICKHOUSE_CLIENT -q "desc file(data_02293, JSONCompactColumns)" -$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONCompactColumns)" +$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONCompactColumns) order by c1, c2, c3" echo "JSONColumnsWithMetadata" $CLICKHOUSE_CLIENT -q "select sum(a) as sum, avg(a) as avg from test_02293 group by a % 4 with totals order by tuple(sum, avg) format JSONColumnsWithMetadata" --extremes=1 | grep -v "elapsed" @@ -49,9 +49,9 @@ echo ' ' > $DATA_FILE $CLICKHOUSE_CLIENT -q "desc file(data_02293, JSONColumns)" -$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns)" -$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns, 'a UInt32, t String') settings input_format_skip_unknown_fields=0" 2>&1 | grep -F -q 'INCORRECT_DATA' && echo 'OK' || echo 'FAIL' -$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns, 'a UInt32, t String') settings input_format_skip_unknown_fields=1" +$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns) order by b, a, c, d" +$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns, 'a UInt32, t String') order by a, t settings input_format_skip_unknown_fields=0" 2>&1 | grep -F -q 'INCORRECT_DATA' && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONColumns, 'a UInt32, t String') order by a, t settings input_format_skip_unknown_fields=1" echo ' [ @@ -75,8 +75,8 @@ echo ' ' > $DATA_FILE $CLICKHOUSE_CLIENT -q "desc file(data_02293, JSONCompactColumns)" -$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONCompactColumns)" -$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONCompactColumns, 'a UInt32, t UInt32')" 2>&1 | grep -F -q 'INCORRECT_DATA' && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONCompactColumns) order by c1, c2, c3" +$CLICKHOUSE_CLIENT -q "select * from file(data_02293, JSONCompactColumns, 'a UInt32, t UInt32') order by a, t" 2>&1 | grep -F -q 'INCORRECT_DATA' && echo 'OK' || echo 'FAIL' echo ' { From 32ac23803f2904187b0a30cdbe7529012d1b12cf Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 8 Apr 2023 19:02:18 +0000 Subject: [PATCH 215/277] Fix test which can become flaky due to file() parallelization --- ...decompression_with_escape_sequence_at_the_end_of_buffer.sh | 4 ++-- tests/queries/0_stateless/02051_symlinks_to_user_files.sh | 2 +- tests/queries/0_stateless/02211_jsonl_format_extension.sql | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh b/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh index 47cf6e06b48..2f8d8f06dee 100755 --- a/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh +++ b/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh @@ -15,8 +15,8 @@ ${CLICKHOUSE_CLIENT} --multiline --multiquery --query " set min_chunk_bytes_for_parallel_parsing=10485760; set max_read_buffer_size = 65536; set input_format_parallel_parsing = 0; -select * from file('test_01946.zstd', 'JSONEachRow', 'foo String') limit 30 format Null; +select * from file('test_01946.zstd', 'JSONEachRow', 'foo String') order by foo limit 30 format Null; set input_format_parallel_parsing = 1; -select * from file('test_01946.zstd', 'JSONEachRow', 'foo String') limit 30 format Null; +select * from file('test_01946.zstd', 'JSONEachRow', 'foo String') order by foo limit 30 format Null; " diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh index dfdc71e0f0b..22d6d2938cd 100755 --- a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh @@ -28,5 +28,5 @@ function cleanup() trap cleanup EXIT ${CLICKHOUSE_CLIENT} --query="insert into table function file('${symlink_path}', 'Values', 'a String') select 'OK'"; -${CLICKHOUSE_CLIENT} --query="select * from file('${symlink_path}', 'Values', 'a String')"; +${CLICKHOUSE_CLIENT} --query="select * from file('${symlink_path}', 'Values', 'a String') order by a"; diff --git a/tests/queries/0_stateless/02211_jsonl_format_extension.sql b/tests/queries/0_stateless/02211_jsonl_format_extension.sql index 08fff5a11f5..907cdd5cf92 100644 --- a/tests/queries/0_stateless/02211_jsonl_format_extension.sql +++ b/tests/queries/0_stateless/02211_jsonl_format_extension.sql @@ -1,3 +1,3 @@ -- Tags: no-fasttest insert into table function file('data.jsonl', 'JSONEachRow', 'x UInt32') select * from numbers(10); -select * from file('data.jsonl'); +select * from file('data.jsonl') order by x; From 0fbb05bf55ce84d1f998b33816f9a21c22456386 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 8 Apr 2023 19:15:18 +0000 Subject: [PATCH 216/277] Fix 02286_mysql_dump_input_format --- .../02286_mysql_dump_input_format.reference | 2 +- .../02286_mysql_dump_input_format.sh | 92 +++++++++---------- 2 files changed, 47 insertions(+), 47 deletions(-) diff --git a/tests/queries/0_stateless/02286_mysql_dump_input_format.reference b/tests/queries/0_stateless/02286_mysql_dump_input_format.reference index a736358b9b7..25a78651e33 100644 --- a/tests/queries/0_stateless/02286_mysql_dump_input_format.reference +++ b/tests/queries/0_stateless/02286_mysql_dump_input_format.reference @@ -12,13 +12,13 @@ 4 \N 5 \N 6 7 +7 6 \N 1 \N 2 \N 3 \N 3 \N 4 \N 5 -7 6 OK 1 2 diff --git a/tests/queries/0_stateless/02286_mysql_dump_input_format.sh b/tests/queries/0_stateless/02286_mysql_dump_input_format.sh index 891734e9ad3..1139c1ea68c 100755 --- a/tests/queries/0_stateless/02286_mysql_dump_input_format.sh +++ b/tests/queries/0_stateless/02286_mysql_dump_input_format.sh @@ -9,140 +9,140 @@ USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonex cp $CURDIR/data_mysql_dump/dump*.sql $USER_FILES_PATH -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32), y Nullable(Int32)')" -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'a Nullable(Int32), b Nullable(Int32)') settings input_format_mysql_dump_map_column_names = 0" -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'y Nullable(Int32), x Nullable(Int32)') settings input_format_mysql_dump_map_column_names = 1" -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32), z String') settings input_format_skip_unknown_fields = 0" 2>&1 | grep -F -q 'INCORRECT_DATA' && echo 'OK' || echo 'FAIL' -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32), z String') settings input_format_skip_unknown_fields = 1" +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32), y Nullable(Int32)') order by x, y" +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'a Nullable(Int32), b Nullable(Int32)') order by a, b settings input_format_mysql_dump_map_column_names = 0" +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'y Nullable(Int32), x Nullable(Int32)') order by y, x settings input_format_mysql_dump_map_column_names = 1" +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32), z String') order by x, z settings input_format_skip_unknown_fields = 0" 2>&1 | grep -F -q 'INCORRECT_DATA' && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32), z String') order by x, z settings input_format_skip_unknown_fields = 1" echo "dump1" $CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" -$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" +$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32)') settings input_format_mysql_dump_table_name='test 3'" 2>&1 | grep -F -q 'EMPTY_DATA_PASSED' && echo 'OK' || echo 'FAIL' echo "dump2" $CLICKHOUSE_CLIENT -q "desc file(dump2.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump2.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump2.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump2.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump2.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump2.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" echo "dump3" $CLICKHOUSE_CLIENT -q "desc file(dump3.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump3.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump3.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump3.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" -$CLICKHOUSE_CLIENT -q "select * from file(dump3.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" +$CLICKHOUSE_CLIENT -q "select * from file(dump3.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2', max_threads=1" echo "dump4" $CLICKHOUSE_CLIENT -q "desc file(dump4.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump4.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump4.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump4.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" -$CLICKHOUSE_CLIENT -q "select * from file(dump4.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" +$CLICKHOUSE_CLIENT -q "select * from file(dump4.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3', max_threads=1" echo "dump5" $CLICKHOUSE_CLIENT -q "desc file(dump5.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump5.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump5.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump5.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump5.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump5.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" echo "dump6" $CLICKHOUSE_CLIENT -q "desc file(dump6.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" -$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" +$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" -$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" +$CLICKHOUSE_CLIENT -q "select * from file(dump6.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3', max_threads=1" echo "dump7" $CLICKHOUSE_CLIENT -q "desc file(dump7.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump7.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "desc file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump7.sql, MySQLDump) settings max_threads=1" +$CLICKHOUSE_CLIENT -q "desc file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" $CLICKHOUSE_CLIENT -q "select * from file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" $CLICKHOUSE_CLIENT -q "desc file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" -$CLICKHOUSE_CLIENT -q "select * from file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" +$CLICKHOUSE_CLIENT -q "select * from file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" -$CLICKHOUSE_CLIENT -q "select * from file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" +$CLICKHOUSE_CLIENT -q "select * from file(dump7.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3', max_threads=1" echo "dump8" $CLICKHOUSE_CLIENT -q "desc file(dump8.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump8.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump8.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump8.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump2.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump2.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" echo "dump9" $CLICKHOUSE_CLIENT -q "desc file(dump9.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump9.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump9.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump9.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump9.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump9.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" echo "dump10" $CLICKHOUSE_CLIENT -q "desc file(dump10.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump10.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump10.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump10.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump10.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump10.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" echo "dump11" $CLICKHOUSE_CLIENT -q "desc file(dump11.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump11.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump11.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump11.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump11.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump11.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" echo "dump12" $CLICKHOUSE_CLIENT -q "desc file(dump12.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" -$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" +$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" -$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" +$CLICKHOUSE_CLIENT -q "select * from file(dump12.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3', max_threads=1" echo "dump13" $CLICKHOUSE_CLIENT -q "desc file(dump13.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump13.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump13.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump13.sql, MySQLDump) settings input_format_mysql_dump_table_name='fruits'" -$CLICKHOUSE_CLIENT -q "select * from file(dump13.sql, MySQLDump) settings input_format_mysql_dump_table_name='fruits'" +$CLICKHOUSE_CLIENT -q "select * from file(dump13.sql, MySQLDump) settings input_format_mysql_dump_table_name='fruits', max_threads=1" echo "dump14" $CLICKHOUSE_CLIENT -q "desc file(dump14.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" -$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" +$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" -$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" +$CLICKHOUSE_CLIENT -q "select * from file(dump14.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3', max_threads=1" echo "dump15" $CLICKHOUSE_CLIENT -q "desc file(dump15.sql, MySQLDump)" -$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump)" +$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" -$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'" +$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" -$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'" +$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2', max_threads=1" $CLICKHOUSE_CLIENT -q "desc file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" -$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" +$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3', max_threads=1" rm $USER_FILES_PATH/dump*.sql From bcb913e7b362191f36895617fbb97b923264c9c7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 8 Apr 2023 20:13:24 +0000 Subject: [PATCH 217/277] A try to prevent possible tests flakyness due to file() parallelization --- .../0_stateless/02293_arrow_dictionary_indexes.sql | 2 +- .../0_stateless/02314_csv_tsv_skip_first_lines.sql | 4 ++-- .../queries/0_stateless/02383_arrow_dict_special_cases.sh | 8 ++++---- .../0_stateless/02455_one_row_from_csv_memory_usage.sh | 4 ++-- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql b/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql index 3ff6a5ffbb3..ec68d1a4443 100644 --- a/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql +++ b/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql @@ -1,3 +1,3 @@ -- Tags: no-fasttest insert into function file(02293_data.arrow) select toLowCardinality(toString(number)) from numbers(300) settings output_format_arrow_low_cardinality_as_dictionary=1, engine_file_truncate_on_insert=1; -select * from file(02293_data.arrow); +select * from file(02293_data.arrow) settings max_threads=1; diff --git a/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.sql b/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.sql index 4a0cef35310..a569b6e9b84 100644 --- a/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.sql +++ b/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.sql @@ -3,10 +3,10 @@ insert into function file(data_02314.csv) select number, number + 1 from numbers(5) settings engine_file_truncate_on_insert=1; insert into function file(data_02314.csv) select number, number + 1, number + 2 from numbers(5); desc file(data_02314.csv) settings input_format_csv_skip_first_lines=5; -select * from file(data_02314.csv) settings input_format_csv_skip_first_lines=5; +select * from file(data_02314.csv) order by c1 settings input_format_csv_skip_first_lines=5; insert into function file(data_02314.tsv) select number, number + 1 from numbers(5) settings engine_file_truncate_on_insert=1; insert into function file(data_02314.tsv) select number, number + 1, number + 2 from numbers(5); desc file(data_02314.tsv) settings input_format_tsv_skip_first_lines=5; -select * from file(data_02314.tsv) settings input_format_tsv_skip_first_lines=5; +select * from file(data_02314.tsv) order by c1 settings input_format_tsv_skip_first_lines=5; diff --git a/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh b/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh index 86f95873f14..bab0b57f116 100755 --- a/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh +++ b/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh @@ -13,17 +13,17 @@ cp $CURDIR/data_arrow/corrupted.arrow $USER_FILES_PATH/test_02383/ cp $CURDIR/data_arrow/dict_with_nulls.arrow $USER_FILES_PATH/test_02383/ $CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary1.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary1.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary1.arrow') settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary2.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary2.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary2.arrow') settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary3.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary3.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary3.arrow') settings max_threads=1" $CLICKHOUSE_CLIENT -q "desc file('test_02383/corrupted.arrow')" $CLICKHOUSE_CLIENT -q "select * from file('test_02383/corrupted.arrow')" 2>&1 | grep -F -q "INCORRECT_DATA" && echo OK || echo FAIL $CLICKHOUSE_CLIENT -q "desc file('test_02383/dict_with_nulls.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dict_with_nulls.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dict_with_nulls.arrow') settings max_threads=1" rm -rf $USER_FILES_PATH/test_02383 diff --git a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh index 7506e78455d..9076d7a8a5f 100755 --- a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh +++ b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh @@ -7,5 +7,5 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) user_files_path=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -E '^Code: 107.*FILE_DOESNT_EXIST' | head -1 | awk '{gsub("/nonexist.txt","",$9); print $9}') cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $user_files_path/ -${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$user_files_path/10m_rows.csv.xz' , 'CSVWithNames') LIMIT 1 settings max_memory_usage=1000000000" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$user_files_path/10m_rows.csv.xz' , 'CSVWithNames') LIMIT 1 settings max_memory_usage=100000000" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$user_files_path/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=1000000000" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$user_files_path/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=100000000" From b319271d59df6544ab356b465ea1a69ade01e7b9 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Sat, 8 Apr 2023 22:50:24 +0200 Subject: [PATCH 218/277] Remove misleading comment It is no longer "significant" after `NOEXCEPT_SCOPE` macro rework #39229 --- .../MergeTree/MergeTreeBackgroundExecutor.cpp | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 65a796456bf..f2efbb2ba8d 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -217,13 +217,10 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) if (item->is_currently_deleting) { - /// This is significant to order the destructors. - { - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - item->task.reset(); - }); - } + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + item->task.reset(); + }); item->is_done.set(); item = nullptr; return; From d80364ffa0bebd86e25dcc89bf3be325da26830a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 8 Apr 2023 22:51:12 +0000 Subject: [PATCH 219/277] Fix tests: truncate output file on insert --- tests/queries/0_stateless/02187_msg_pack_uuid.sh | 6 +++--- tests/queries/0_stateless/02211_jsonl_format_extension.sql | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02187_msg_pack_uuid.sh b/tests/queries/0_stateless/02187_msg_pack_uuid.sh index 9be92d66790..f04ef09a8c8 100755 --- a/tests/queries/0_stateless/02187_msg_pack_uuid.sh +++ b/tests/queries/0_stateless/02187_msg_pack_uuid.sh @@ -5,13 +5,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "insert into table function file('uuid_str.msgpack', 'MsgPack', 'uuid UUID') select toUUID('5e7084e0-019f-461f-9e70-84e0019f561f') settings output_format_msgpack_uuid_representation='str'" +$CLICKHOUSE_CLIENT -q "insert into table function file('uuid_str.msgpack', 'MsgPack', 'uuid UUID') select toUUID('5e7084e0-019f-461f-9e70-84e0019f561f') settings output_format_msgpack_uuid_representation='str', engine_file_truncate_on_insert=1" $CLICKHOUSE_CLIENT -q "select * from file('uuid_str.msgpack', 'MsgPack', 'uuid UUID')" -$CLICKHOUSE_CLIENT -q "insert into table function file('uuid_bin.msgpack', 'MsgPack', 'uuid UUID') select toUUID('5e7084e0-019f-461f-9e70-84e0019f561f') settings output_format_msgpack_uuid_representation='bin'" +$CLICKHOUSE_CLIENT -q "insert into table function file('uuid_bin.msgpack', 'MsgPack', 'uuid UUID') select toUUID('5e7084e0-019f-461f-9e70-84e0019f561f') settings output_format_msgpack_uuid_representation='bin', engine_file_truncate_on_insert=1" $CLICKHOUSE_CLIENT -q "select * from file('uuid_bin.msgpack', 'MsgPack', 'uuid UUID')" -$CLICKHOUSE_CLIENT -q "insert into table function file('uuid_ext.msgpack', 'MsgPack', 'uuid UUID') select toUUID('5e7084e0-019f-461f-9e70-84e0019f561f') settings output_format_msgpack_uuid_representation='ext'" +$CLICKHOUSE_CLIENT -q "insert into table function file('uuid_ext.msgpack', 'MsgPack', 'uuid UUID') select toUUID('5e7084e0-019f-461f-9e70-84e0019f561f') settings output_format_msgpack_uuid_representation='ext', engine_file_truncate_on_insert=1" $CLICKHOUSE_CLIENT -q "select * from file('uuid_ext.msgpack', 'MsgPack', 'uuid UUID')" $CLICKHOUSE_CLIENT -q "select c1, toTypeName(c1) from file('uuid_ext.msgpack') settings input_format_msgpack_number_of_columns=1" diff --git a/tests/queries/0_stateless/02211_jsonl_format_extension.sql b/tests/queries/0_stateless/02211_jsonl_format_extension.sql index 907cdd5cf92..a95b19f4e3b 100644 --- a/tests/queries/0_stateless/02211_jsonl_format_extension.sql +++ b/tests/queries/0_stateless/02211_jsonl_format_extension.sql @@ -1,3 +1,3 @@ -- Tags: no-fasttest -insert into table function file('data.jsonl', 'JSONEachRow', 'x UInt32') select * from numbers(10); +insert into table function file('data.jsonl', 'JSONEachRow', 'x UInt32') select * from numbers(10) SETTINGS engine_file_truncate_on_insert=1; select * from file('data.jsonl') order by x; From 09ea79aaf7955a9db36126e9a5f18e93b98704ec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Apr 2023 03:04:26 +0200 Subject: [PATCH 220/277] Add support for {server_uuid} macro --- programs/keeper/Keeper.cpp | 1 - programs/server/Server.cpp | 2 +- src/Common/Exception.h | 1 - src/Common/Macros.cpp | 12 +++++++++++- src/Daemon/BaseDaemon.h | 1 - src/Dictionaries/MongoDBDictionarySource.cpp | 1 - src/Disks/ObjectStorages/S3/registerDiskS3.cpp | 9 +++------ src/IO/HTTPCommon.cpp | 2 -- src/IO/ReadWriteBufferFromHTTP.h | 1 - src/Processors/Sources/MongoDBSource.cpp | 1 - src/Storages/StorageMongoDB.cpp | 1 - 11 files changed, 15 insertions(+), 17 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 266b363eb47..3853c955171 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -17,7 +17,6 @@ #include #include #include -#include #include #include #include diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 9ef9f704f61..164e1ce14e5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -981,7 +981,7 @@ try StatusFile status{path / "status", StatusFile::write_full_info}; - DB::ServerUUID::load(path / "uuid", log); + ServerUUID::load(path / "uuid", log); /// Try to increase limit on number of open files. { diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 8e50c1114f4..170e0d32b3c 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -4,7 +4,6 @@ #include #include -#include #include #include diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index e5d4be446c1..18e65bbaede 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -1,8 +1,9 @@ #include #include #include -#include #include +#include +#include namespace DB @@ -105,6 +106,15 @@ String Macros::expand(const String & s, res += toString(info.table_id.uuid); info.expanded_uuid = true; } + else if (macro_name == "server_uuid") + { + auto uuid = ServerUUID::get(); + if (UUIDHelpers::Nil == uuid) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Macro {server_uuid} expanded to zero, which means the UUID is not initialized (most likely it's not a server application)"); + res += toString(uuid); + info.expanded_other = true; + } else if (info.shard && macro_name == "shard") { res += *info.shard; diff --git a/src/Daemon/BaseDaemon.h b/src/Daemon/BaseDaemon.h index d28f9403c16..f90f403364b 100644 --- a/src/Daemon/BaseDaemon.h +++ b/src/Daemon/BaseDaemon.h @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index a9555a94304..922e1e71bbb 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -67,7 +67,6 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) #include #include #include -#include // only after poco // naming conflict: diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index 1c192a0d89c..70f39d893f7 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -8,7 +8,6 @@ #if USE_AWS_S3 -#include #include #include @@ -19,9 +18,7 @@ #include #include #include -#include -#include #include #include @@ -87,10 +84,10 @@ public: private: static String getServerUUID() { - DB::UUID server_uuid = DB::ServerUUID::get(); - if (server_uuid == DB::UUIDHelpers::Nil) + UUID server_uuid = ServerUUID::get(); + if (server_uuid == UUIDHelpers::Nil) throw Exception(ErrorCodes::LOGICAL_ERROR, "Server UUID is not initialized"); - return DB::toString(server_uuid); + return toString(server_uuid); } }; diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 551ce797757..9eabe3573f4 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -7,8 +7,6 @@ #include #include -#include - #include "config.h" #if USE_SSL diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 784110f735e..28f7a7f600b 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -21,7 +21,6 @@ #include #include #include -#include #include #include #include "config.h" diff --git a/src/Processors/Sources/MongoDBSource.cpp b/src/Processors/Sources/MongoDBSource.cpp index a8bfefdf8a6..9e50cdcf6ab 100644 --- a/src/Processors/Sources/MongoDBSource.cpp +++ b/src/Processors/Sources/MongoDBSource.cpp @@ -15,7 +15,6 @@ #include #include #include -#include // only after poco // naming conflict: diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 0bad4ab8bf0..63b8c2d00a1 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include From 6292a08c9c98dd85d28e5a7f10759a858c752421 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Apr 2023 03:25:38 +0200 Subject: [PATCH 221/277] Add a test --- src/Common/Macros.cpp | 8 +++++--- .../0_stateless/02711_server_uuid_macro.reference | 1 + .../0_stateless/02711_server_uuid_macro.sql | 15 +++++++++++++++ 3 files changed, 21 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02711_server_uuid_macro.reference create mode 100644 tests/queries/0_stateless/02711_server_uuid_macro.sql diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index 18e65bbaede..f43fed6c499 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -12,6 +12,8 @@ namespace DB namespace ErrorCodes { extern const int SYNTAX_ERROR; + extern const int BAD_ARGUMENTS; + extern const int NO_ELEMENTS_IN_CONFIG; } Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & root_key, Poco::Logger * log) @@ -96,7 +98,7 @@ String Macros::expand(const String & s, else if (macro_name == "uuid" && !info.expand_special_macros_only) { if (info.table_id.uuid == UUIDHelpers::Nil) - throw Exception(ErrorCodes::SYNTAX_ERROR, "Macro 'uuid' and empty arguments of ReplicatedMergeTree " + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Macro 'uuid' and empty arguments of ReplicatedMergeTree " "are supported only for ON CLUSTER queries with Atomic database engine"); /// For ON CLUSTER queries we don't want to require all macros definitions in initiator's config. /// However, initiator must check that for cross-replication cluster zookeeper_path does not contain {uuid} macro. @@ -135,7 +137,7 @@ String Macros::expand(const String & s, info.has_unknown = true; } else - throw Exception(ErrorCodes::SYNTAX_ERROR, "No macro '{}' in config while processing substitutions in " + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No macro '{}' in config while processing substitutions in " "'{}' at '{}' or macro is not supported here", macro_name, s, toString(begin)); pos = end + 1; @@ -152,7 +154,7 @@ String Macros::getValue(const String & key) const { if (auto it = macros.find(key); it != macros.end()) return it->second; - throw Exception(ErrorCodes::SYNTAX_ERROR, "No macro {} in config", key); + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No macro {} in config", key); } diff --git a/tests/queries/0_stateless/02711_server_uuid_macro.reference b/tests/queries/0_stateless/02711_server_uuid_macro.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02711_server_uuid_macro.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02711_server_uuid_macro.sql b/tests/queries/0_stateless/02711_server_uuid_macro.sql new file mode 100644 index 00000000000..f708156c0ae --- /dev/null +++ b/tests/queries/0_stateless/02711_server_uuid_macro.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test; + +-- You can create a table with the {server_uuid} substituted. +CREATE TABLE test (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test', 'replica-{server_uuid}') ORDER BY x; + +-- The server UUID is correctly substituted. +SELECT engine_full LIKE ('%replica-' || serverUUID()::String || '%') FROM system.tables WHERE database = currentDatabase() AND name = 'test'; + +-- An attempt to create a second table with the same UUID results in error. +CREATE TABLE test2 (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test', 'replica-{server_uuid}') ORDER BY x; -- { serverError REPLICA_ALREADY_EXISTS } + +-- The macro {server_uuid} is special, not a configuration-type macro. It's normal that it is inaccessible with the getMacro function. +SELECT getMacro('server_uuid'); -- { serverError NO_ELEMENTS_IN_CONFIG } + +DROP TABLE test NO DELAY; From d23cd018ec8196184a6a275004559202bef4b9ef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Apr 2023 07:46:27 +0200 Subject: [PATCH 222/277] Update test --- .../0_stateless/01148_zookeeper_path_macros_unfolding.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql index 7234cee96e0..505c406c2cc 100644 --- a/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql +++ b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql @@ -12,7 +12,7 @@ DETACH TABLE rmt1; ATTACH TABLE rmt1; SHOW CREATE TABLE rmt1; -CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('{default_path_test}{uuid}', '{default_name_test}') ORDER BY n; -- { serverError 62 } +CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('{default_path_test}{uuid}', '{default_name_test}') ORDER BY n; -- { serverError 36 } CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('{default_path_test}test_01148', '{default_name_test}') ORDER BY n; SHOW CREATE TABLE rmt; RENAME TABLE rmt TO rmt2; -- { serverError 48 } @@ -24,7 +24,7 @@ SET distributed_ddl_output_mode='none'; DROP DATABASE IF EXISTS test_01148_atomic; CREATE DATABASE test_01148_atomic ENGINE=Atomic; CREATE TABLE test_01148_atomic.rmt2 ON CLUSTER test_shard_localhost (n int, PRIMARY KEY n) ENGINE=ReplicatedMergeTree; -CREATE TABLE test_01148_atomic.rmt3 AS test_01148_atomic.rmt2; -- { serverError 62 } +CREATE TABLE test_01148_atomic.rmt3 AS test_01148_atomic.rmt2; -- { serverError 36 } CREATE TABLE test_01148_atomic.rmt4 ON CLUSTER test_shard_localhost AS test_01148_atomic.rmt2; SHOW CREATE TABLE test_01148_atomic.rmt2; RENAME TABLE test_01148_atomic.rmt4 to test_01148_atomic.rmt3; From c3c07c05ea23e8f9f175c76cc7accbcdf14b50f9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Apr 2023 09:36:20 +0300 Subject: [PATCH 223/277] Update 02207_allow_plaintext_and_no_password.sh --- .../0_stateless/02207_allow_plaintext_and_no_password.sh | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02207_allow_plaintext_and_no_password.sh b/tests/queries/0_stateless/02207_allow_plaintext_and_no_password.sh index 693f1d817e3..0345a0e6394 100755 --- a/tests/queries/0_stateless/02207_allow_plaintext_and_no_password.sh +++ b/tests/queries/0_stateless/02207_allow_plaintext_and_no_password.sh @@ -1,11 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-parallel, no-fasttest -# Tag no-tsan: requires jemalloc to track small allocations -# Tag no-asan: requires jemalloc to track small allocations -# Tag no-ubsan: requires jemalloc to track small allocations -# Tag no-msan: requires jemalloc to track small allocations - - +# Tags: no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 10883d8af5b407c5bbcf3b49bc0db76f558afa14 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sun, 9 Apr 2023 12:52:23 +0000 Subject: [PATCH 224/277] Allow commas with subqueries, add comments --- src/Parsers/ExpressionListParsers.cpp | 96 +++++++++++-------- .../02676_trailing_commas.reference | 3 +- .../0_stateless/02676_trailing_commas.sql | 3 +- 3 files changed, 59 insertions(+), 43 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 6008e89d038..0069821c949 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -779,6 +779,41 @@ protected: }; +struct ParserExpressionImpl +{ + static std::vector> operators_table; + static std::vector> unary_operators_table; + static const char * overlapping_operators_to_skip[]; + + static Operator finish_between_operator; + + ParserCompoundIdentifier identifier_parser{false, true}; + ParserNumber number_parser; + ParserAsterisk asterisk_parser; + ParserLiteral literal_parser; + ParserTupleOfLiterals tuple_literal_parser; + ParserArrayOfLiterals array_literal_parser; + ParserSubstitution substitution_parser; + ParserMySQLGlobalVariable mysql_global_variable_parser; + + ParserKeyword any_parser{"ANY"}; + ParserKeyword all_parser{"ALL"}; + + // Recursion + ParserQualifiedAsterisk qualified_asterisk_parser; + ParserColumnsMatcher columns_matcher_parser; + ParserQualifiedColumnsMatcher qualified_columns_matcher_parser; + ParserSubquery subquery_parser; + + bool parse(std::unique_ptr start, IParser::Pos & pos, ASTPtr & node, Expected & expected); + + using Layers = std::vector>; + + Action tryParseOperand(Layers & layers, IParser::Pos & pos, Expected & expected); + Action tryParseOperator(Layers & layers, IParser::Pos & pos, Expected & expected); +}; + + class ExpressionLayer : public Layer { public: @@ -801,7 +836,7 @@ public: return Layer::getResultImpl(node); } - bool parse(IParser::Pos & pos, Expected & expected, Action & /*action*/) override + bool parse(IParser::Pos & pos, Expected & /*expected*/, Action & /*action*/) override { if (pos->type == TokenType::Comma) { @@ -814,27 +849,40 @@ public: /// - SELECT a, b, c, FROM table /// - SELECT 1, - /// For this purpose we eliminate the following cases: + /// For this purpose we need to eliminate the following cases: /// 1. WITH 1 AS from SELECT 2, from /// 2. SELECT to, from FROM table /// 3. SELECT to, from AS alias FROM table - /// 4. SELECT to, from + to FROM table + /// 4. SELECT to, from + to, from IN [1,2,3], FROM table + Expected test_expected; auto test_pos = pos; ++test_pos; + /// End of query if (test_pos.isValid() && test_pos->type != TokenType::Semicolon) { - if (!ParserKeyword("FROM").ignore(test_pos, expected)) + /// If we can't parse FROM then return + if (!ParserKeyword("FROM").ignore(test_pos, test_expected)) return true; - if (ParserKeyword("FROM").ignore(test_pos, expected)) + /// If we parse a second FROM then the first one was a name of a column + if (ParserKeyword("FROM").ignore(test_pos, test_expected)) return true; - if (ParserAlias(false).ignore(test_pos, expected)) + /// If we parse an explicit alias to FROM, then it was a name of a column + if (ParserAlias(false).ignore(test_pos, test_expected)) return true; - if (!ParserIdentifier(true).ignore(test_pos, expected)) + /// If we parse an operator after FROM then it was a name of a column + auto cur_op = ParserExpressionImpl::operators_table.begin(); + for (; cur_op != ParserExpressionImpl::operators_table.end(); ++cur_op) + { + if (parseOperator(test_pos, cur_op->first, test_expected)) + break; + } + + if (cur_op != ParserExpressionImpl::operators_table.end()) return true; } @@ -2205,40 +2253,6 @@ bool ParseTimestampOperatorExpression(IParser::Pos & pos, ASTPtr & node, Expecte return true; } -struct ParserExpressionImpl -{ - static std::vector> operators_table; - static std::vector> unary_operators_table; - static const char * overlapping_operators_to_skip[]; - - static Operator finish_between_operator; - - ParserCompoundIdentifier identifier_parser{false, true}; - ParserNumber number_parser; - ParserAsterisk asterisk_parser; - ParserLiteral literal_parser; - ParserTupleOfLiterals tuple_literal_parser; - ParserArrayOfLiterals array_literal_parser; - ParserSubstitution substitution_parser; - ParserMySQLGlobalVariable mysql_global_variable_parser; - - ParserKeyword any_parser{"ANY"}; - ParserKeyword all_parser{"ALL"}; - - // Recursion - ParserQualifiedAsterisk qualified_asterisk_parser; - ParserColumnsMatcher columns_matcher_parser; - ParserQualifiedColumnsMatcher qualified_columns_matcher_parser; - ParserSubquery subquery_parser; - - bool parse(std::unique_ptr start, IParser::Pos & pos, ASTPtr & node, Expected & expected); - - using Layers = std::vector>; - - Action tryParseOperand(Layers & layers, IParser::Pos & pos, Expected & expected); - Action tryParseOperator(Layers & layers, IParser::Pos & pos, Expected & expected); -}; - bool ParserExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { diff --git a/tests/queries/0_stateless/02676_trailing_commas.reference b/tests/queries/0_stateless/02676_trailing_commas.reference index 41ace3e47aa..76d173ca23e 100644 --- a/tests/queries/0_stateless/02676_trailing_commas.reference +++ b/tests/queries/0_stateless/02676_trailing_commas.reference @@ -1,4 +1,5 @@ 1 1 1 -1 2 +1 2 0 +1 diff --git a/tests/queries/0_stateless/02676_trailing_commas.sql b/tests/queries/0_stateless/02676_trailing_commas.sql index 5e2dafccb46..048405c4d20 100644 --- a/tests/queries/0_stateless/02676_trailing_commas.sql +++ b/tests/queries/0_stateless/02676_trailing_commas.sql @@ -1,4 +1,5 @@ SELECT 1,; SELECT 1, FROM numbers(1); WITH 1 as a SELECT a, FROM numbers(1); -WITH 1 as from SELECT from, from + from, FROM numbers(1); +WITH 1 as from SELECT from, from + from, from in [0], FROM numbers(1); +SELECT n, FROM (SELECT 1 AS n); From 3bc5c6423b2d522418e553f8050947343dbf6160 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Sun, 9 Apr 2023 16:54:21 +0200 Subject: [PATCH 225/277] Update formats.md --- docs/en/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index de8357345db..5c59d8d74ca 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -78,7 +78,7 @@ The supported formats are: | [Null](#null) | ✗ | ✔ | | [XML](#xml) | ✗ | ✔ | | [CapnProto](#capnproto) | ✔ | ✔ | -| [LineAsString](#lineasstring) | ✔ | ✗ | +| [LineAsString](#lineasstring) | ✔ | ✔ | | [Regexp](#data-format-regexp) | ✔ | ✗ | | [RawBLOB](#rawblob) | ✔ | ✔ | | [MsgPack](#msgpack) | ✔ | ✔ | From 286424be05c6734543b5157287d5359b80dbe385 Mon Sep 17 00:00:00 2001 From: caipengxiang <291458254@qq.com> Date: Sun, 9 Apr 2023 22:59:17 +0800 Subject: [PATCH 226/277] bugfix: compare Bits and sizeof(Arithmetic) * 8 --- base/base/wide_integer_impl.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index 7cdb527f9cf..ed4570d5e3f 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -155,13 +155,13 @@ struct common_type, Arithmetic> std::is_floating_point_v, Arithmetic, std::conditional_t< - sizeof(Arithmetic) < Bits * sizeof(long), + sizeof(Arithmetic) * 8 < Bits, wide::integer, std::conditional_t< - Bits * sizeof(long) < sizeof(Arithmetic), + Bits < sizeof(Arithmetic) * 8, Arithmetic, std::conditional_t< - Bits * sizeof(long) == sizeof(Arithmetic) && (std::is_same_v || std::is_signed_v), + Bits == sizeof(Arithmetic) * 8 && (std::is_same_v || std::is_signed_v), Arithmetic, wide::integer>>>>; }; From 553326c999d2febe1316ad372bab976f1f30043b Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 9 Apr 2023 15:13:40 +0000 Subject: [PATCH 227/277] add test --- .../0_stateless/01165_lost_part_empty_partition.reference | 2 ++ tests/queries/0_stateless/01165_lost_part_empty_partition.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01165_lost_part_empty_partition.reference b/tests/queries/0_stateless/01165_lost_part_empty_partition.reference index e69de29bb2d..6ed281c757a 100644 --- a/tests/queries/0_stateless/01165_lost_part_empty_partition.reference +++ b/tests/queries/0_stateless/01165_lost_part_empty_partition.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/01165_lost_part_empty_partition.sql b/tests/queries/0_stateless/01165_lost_part_empty_partition.sql index dc41b15118f..924798b0050 100644 --- a/tests/queries/0_stateless/01165_lost_part_empty_partition.sql +++ b/tests/queries/0_stateless/01165_lost_part_empty_partition.sql @@ -10,6 +10,7 @@ insert into rmt1 values (now(), rand()); drop table rmt1; system sync replica rmt2; +select lost_part_count from system.replicas where database = currentDatabase() and table = 'rmt2'; drop table rmt2; @@ -21,6 +22,7 @@ insert into rmt1 values (now(), rand()); drop table rmt1; system sync replica rmt2; +select lost_part_count from system.replicas where database = currentDatabase() and table = 'rmt2'; drop table rmt2; From 44e418987cdf12ad54bb9fb220947c9451de88b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Apr 2023 22:26:49 +0300 Subject: [PATCH 228/277] Update tests/queries/0_stateless/02711_server_uuid_macro.sql Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- tests/queries/0_stateless/02711_server_uuid_macro.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02711_server_uuid_macro.sql b/tests/queries/0_stateless/02711_server_uuid_macro.sql index f708156c0ae..f10ed7f8f6f 100644 --- a/tests/queries/0_stateless/02711_server_uuid_macro.sql +++ b/tests/queries/0_stateless/02711_server_uuid_macro.sql @@ -1,13 +1,13 @@ DROP TABLE IF EXISTS test; -- You can create a table with the {server_uuid} substituted. -CREATE TABLE test (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test', 'replica-{server_uuid}') ORDER BY x; +CREATE TABLE test (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test', 'replica-{server_uuid}') ORDER BY x; -- The server UUID is correctly substituted. SELECT engine_full LIKE ('%replica-' || serverUUID()::String || '%') FROM system.tables WHERE database = currentDatabase() AND name = 'test'; -- An attempt to create a second table with the same UUID results in error. -CREATE TABLE test2 (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test', 'replica-{server_uuid}') ORDER BY x; -- { serverError REPLICA_ALREADY_EXISTS } +CREATE TABLE test2 (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test', 'replica-{server_uuid}') ORDER BY x; -- { serverError REPLICA_ALREADY_EXISTS } -- The macro {server_uuid} is special, not a configuration-type macro. It's normal that it is inaccessible with the getMacro function. SELECT getMacro('server_uuid'); -- { serverError NO_ELEMENTS_IN_CONFIG } From 6fe6e1fea5f31dc9f2684ae29df2e07a89d8c6f0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 9 Apr 2023 19:55:45 +0000 Subject: [PATCH 229/277] Adopt tests to run in parallel or mark as no-parallel (for flaky check) --- .../02293_arrow_dictionary_indexes.sql | 2 +- .../02383_arrow_dict_special_cases.sh | 32 +++++++++---------- .../02455_one_row_from_csv_memory_usage.sh | 11 +++++-- 3 files changed, 25 insertions(+), 20 deletions(-) diff --git a/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql b/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql index ec68d1a4443..3ea229a1152 100644 --- a/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql +++ b/tests/queries/0_stateless/02293_arrow_dictionary_indexes.sql @@ -1,3 +1,3 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-parallel insert into function file(02293_data.arrow) select toLowCardinality(toString(number)) from numbers(300) settings output_format_arrow_low_cardinality_as_dictionary=1, engine_file_truncate_on_insert=1; select * from file(02293_data.arrow) settings max_threads=1; diff --git a/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh b/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh index bab0b57f116..40487f16551 100755 --- a/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh +++ b/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh @@ -6,24 +6,24 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +UNIQ_DEST_PATH=$USER_FILES_PATH/test-02383-$RANDOM-$RANDOM +mkdir -p $UNIQ_DEST_PATH -mkdir -p $USER_FILES_PATH/test_02383 -cp $CURDIR/data_arrow/dictionary*.arrow $USER_FILES_PATH/test_02383/ -cp $CURDIR/data_arrow/corrupted.arrow $USER_FILES_PATH/test_02383/ -cp $CURDIR/data_arrow/dict_with_nulls.arrow $USER_FILES_PATH/test_02383/ +cp $CURDIR/data_arrow/dictionary*.arrow $UNIQ_DEST_PATH/ +cp $CURDIR/data_arrow/corrupted.arrow $UNIQ_DEST_PATH/ +cp $CURDIR/data_arrow/dict_with_nulls.arrow $UNIQ_DEST_PATH/ -$CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary1.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary1.arrow') settings max_threads=1" -$CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary2.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary2.arrow') settings max_threads=1" -$CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary3.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary3.arrow') settings max_threads=1" +$CLICKHOUSE_CLIENT -q "desc file('$UNIQ_DEST_PATH/dictionary1.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('$UNIQ_DEST_PATH/dictionary1.arrow') settings max_threads=1" +$CLICKHOUSE_CLIENT -q "desc file('$UNIQ_DEST_PATH/dictionary2.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('$UNIQ_DEST_PATH/dictionary2.arrow') settings max_threads=1" +$CLICKHOUSE_CLIENT -q "desc file('$UNIQ_DEST_PATH/dictionary3.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('$UNIQ_DEST_PATH/dictionary3.arrow') settings max_threads=1" -$CLICKHOUSE_CLIENT -q "desc file('test_02383/corrupted.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/corrupted.arrow')" 2>&1 | grep -F -q "INCORRECT_DATA" && echo OK || echo FAIL +$CLICKHOUSE_CLIENT -q "desc file('$UNIQ_DEST_PATH/corrupted.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('$UNIQ_DEST_PATH/corrupted.arrow')" 2>&1 | grep -F -q "INCORRECT_DATA" && echo OK || echo FAIL -$CLICKHOUSE_CLIENT -q "desc file('test_02383/dict_with_nulls.arrow')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dict_with_nulls.arrow') settings max_threads=1" +$CLICKHOUSE_CLIENT -q "desc file('$UNIQ_DEST_PATH/dict_with_nulls.arrow')" +$CLICKHOUSE_CLIENT -q "select * from file('$UNIQ_DEST_PATH/dict_with_nulls.arrow') settings max_threads=1" - -rm -rf $USER_FILES_PATH/test_02383 +rm -rf $UNIQ_DEST_PATH diff --git a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh index 9076d7a8a5f..0b6f91907d4 100755 --- a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh +++ b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh @@ -5,7 +5,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh user_files_path=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -E '^Code: 107.*FILE_DOESNT_EXIST' | head -1 | awk '{gsub("/nonexist.txt","",$9); print $9}') -cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $user_files_path/ -${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$user_files_path/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=1000000000" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$user_files_path/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=100000000" +UNIQ_DEST_PATH=$user_files_path/test-02455-$RANDOM-$RANDOM +mkdir -p $UNIQ_DEST_PATH +cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $UNIQ_DEST_PATH/ + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$UNIQ_DEST_PATH/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=1000000000" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$UNIQ_DEST_PATH/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=100000000" + +rm -rf $UNIQ_DEST_PATH From 211cea5e7c99119777d387a6b5331e4703e24510 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 9 Apr 2023 22:50:21 +0200 Subject: [PATCH 230/277] Fix uncaught exception in case of parallel loader for hashed dictionaries Since ThreadPool::wait() rethrows the first exception (if any):
stacktrace 2023.04.09 12:53:33.629333 [ 22361 ] {} BaseDaemon: (version 22.13.1.1, build id: 5FB01DCAAFFF19F0A9A61E253567F90685989D2F) (from thread 23032) Terminate called for uncaught exception: 2023.04.09 12:53:33.630179 [ 23645 ] {} BaseDaemon: 2023.04.09 12:53:33.630213 [ 23645 ] {} BaseDaemon: Stack trace: 0x7f68b00baccc 0x7f68b006bef2 0x7f68b0056472 0x112a42fe 0x1c17f2a3 0x1c17f238 0xbf4bc3b 0x13961c6d 0x138ee529 0x138ed6bc 0x138dd2f0 0x138dd9c6 0x1571d0dd 0x16197c1f 0x161a231e 0x1619fc93 0x161a51b9 0x11151759 0x1115454e 0x7f68b00b8fd4 0x7f68b013966c 2023.04.09 12:53:33.630247 [ 23645 ] {} BaseDaemon: 3. ? @ 0x7f68b00baccc in ? 2023.04.09 12:53:33.630263 [ 23645 ] {} BaseDaemon: 4. gsignal @ 0x7f68b006bef2 in ? 2023.04.09 12:53:33.630273 [ 23645 ] {} BaseDaemon: 5. abort @ 0x7f68b0056472 in ? 2023.04.09 12:53:33.648815 [ 23645 ] {} BaseDaemon: 6. ./.build/./src/Daemon/BaseDaemon.cpp:456: terminate_handler() @ 0x112a42fe in /usr/lib/debug/usr/bin/clickhouse.debug 2023.04.09 12:53:33.651484 [ 23645 ] {} BaseDaemon: 7. ./.build/./contrib/llvm-project/libcxxabi/src/cxa_handlers.cpp:61: std::__terminate(void (*)()) @ 0x1c17f2a3 in /usr/lib/debug/usr/bin/clickhouse.debug 2023.04.09 12:53:33.654080 [ 23645 ] {} BaseDaemon: 8. ./.build/./contrib/llvm-project/libcxxabi/src/cxa_handlers.cpp:79: std::terminate() @ 0x1c17f238 in /usr/lib/debug/usr/bin/clickhouse.debug 2023.04.09 12:53:35.025565 [ 23645 ] {} BaseDaemon: 9. ? @ 0xbf4bc3b in /usr/lib/debug/usr/bin/clickhouse.debug 2023.04.09 12:53:36.495557 [ 23645 ] {} BaseDaemon: 10. DB::ParallelDictionaryLoader<(DB::DictionaryKeyType)0, true, true>::~ParallelDictionaryLoader() @ 0x13961c6d in /usr/lib/debug/usr/bin/clickhouse.debug 2023.04.09 12:53:37.833142 [ 23645 ] {} BaseDaemon: 11. DB::HashedDictionary<(DB::DictionaryKeyType)0, true, true>::loadData() @ 0x138ee529 in /usr/lib/debug/usr/bin/clickhouse.debug 2023.04.09 12:53:39.124989 [ 23645 ] {} BaseDaemon: 12. DB::HashedDictionary<(DB::DictionaryKeyType)0, true, true>::HashedDictionary(DB::StorageID const&, DB::DictionaryStructure const&, std::__1::shared_ptr, DB::HashedDictionaryStorageConfiguration const&, std::__1::shared_ptr) @ 0x138ed6bc in /usr/lib/debug/usr/bin/clickhouse.debug
Signed-off-by: Azat Khuzhin --- src/Dictionaries/HashedDictionary.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 0e5d18363e9..5cfac20e572 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -114,9 +114,18 @@ public: ~ParallelDictionaryLoader() { - for (auto & queue : shards_queues) - queue->clearAndFinish(); - pool.wait(); + try + { + for (auto & queue : shards_queues) + queue->clearAndFinish(); + + /// NOTE: It is OK to not pass the exception next, since on success finish() should be called which will call wait() + pool.wait(); + } + catch (...) + { + tryLogCurrentException(dictionary.log, "Exception had been thrown during parallel load of the dictionary"); + } } private: From 1187534545451970bd10eb10a986d5e33a8f0b01 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 9 Apr 2023 21:26:39 +0000 Subject: [PATCH 231/277] Simpler way to resize pipeline --- src/Storages/StorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 665630c3559..1b195d2eb47 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -797,7 +797,7 @@ Pipe StorageFile::read( /// It happens if globs in file(path, ...) expands to empty set i.e. no files to process if (num_streams > 0 && num_streams < max_num_streams) { - pipe.addTransform(std::make_shared(pipe.getHeader(), num_streams, max_num_streams)); + pipe.resize(max_num_streams); } return pipe; } From 98cabda66d32069bc1e143a9d7f85cf6369720c7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 00:15:15 +0200 Subject: [PATCH 232/277] Update CCTZ --- contrib/cctz | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/cctz b/contrib/cctz index 7c78edd52b4..5e05432420f 160000 --- a/contrib/cctz +++ b/contrib/cctz @@ -1 +1 @@ -Subproject commit 7c78edd52b4d65acc103c2f195818ffcabe6fe0d +Subproject commit 5e05432420f9692418e2e12aff09859e420b14a2 From c89eb29b0f203e0262891080464293bf153b7e1d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 9 Apr 2023 19:24:42 -0300 Subject: [PATCH 233/277] Update deltasumtimestamp.md --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 7be933d67d7..c059b2c1e63 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -7,7 +7,7 @@ sidebar_position: 141 Суммирует разницу между поÑледовательными Ñтроками. ЕÑли разница отрицательна — она будет проигнорирована. -Эта Ñ„ÑƒÐ½ÐºÑ†Ð¸Ñ Ð¿Ñ€ÐµÐ´Ð½Ð°Ð·Ð½Ð°Ñ‡ÐµÐ½Ð° в первую очередь Ð´Ð»Ñ [материализованных предÑтавлений](../../../sql-reference/statements/create/view.md#materialized), упорÑдоченных по некоторому временному бакету ÑоглаÑно timestamp, например, по бакету `toStartOfMinute`. ПоÑкольку Ñтроки в таком материализованном предÑтавлении будут иметь одинаковый timestamp, невозможно объединить их в "правом" порÑдке. Ð¤ÑƒÐ½ÐºÑ†Ð¸Ñ Ð¾Ñ‚Ñлеживает `timestamp` наблюдаемых значений, поÑтому возможно правильно упорÑдочить ÑоÑтоÑÐ½Ð¸Ñ Ð²Ð¾ Ð²Ñ€ÐµÐ¼Ñ ÑлиÑниÑ. +Эта Ñ„ÑƒÐ½ÐºÑ†Ð¸Ñ Ð¿Ñ€ÐµÐ´Ð½Ð°Ð·Ð½Ð°Ñ‡ÐµÐ½Ð° в первую очередь Ð´Ð»Ñ [материализованных предÑтавлений](../../../sql-reference/statements/create/view.md#materialized), хранÑщих данные, упорÑдоченные по некоторому временному бакету ÑоглаÑно timestamp (time-series), например, по бакету `toStartOfMinute`. ПоÑкольку Ñтроки в таком материализованном предÑтавлении будут иметь одинаковый timestamp, их невозможно объединить их в правильном порÑдке без Ñ…Ñ€Ð°Ð½ÐµÐ½Ð¸Ñ Ð¸Ñходного, неокругленного Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ timestamp. Ð¤ÑƒÐ½ÐºÑ†Ð¸Ñ `deltaSumTimestamp` отÑлеживает иÑходные `timestamp` наблюдаемых значений, поÑтому Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ (ÑоÑтоÑниÑ) функции правильно вычиÑлÑÑŽÑ‚ÑÑ Ð²Ð¾ Ð²Ñ€ÐµÐ¼Ñ ÑлиÑÐ½Ð¸Ñ ÐºÑƒÑков. Чтобы вычиÑлить разницу между упорÑдоченными поÑледовательными Ñтроками, вы можете иÑпользовать функцию [deltaSum](../../../sql-reference/aggregate-functions/reference/deltasum.md#agg_functions-deltasum) вмеÑто функции `deltaSumTimestamp`. From a4183204f9a6b38e40a8523e3c694e0fc0226756 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 9 Apr 2023 19:27:13 -0300 Subject: [PATCH 234/277] Update deltasumtimestamp.md --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index c059b2c1e63..8b022241dac 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -7,7 +7,7 @@ sidebar_position: 141 Суммирует разницу между поÑледовательными Ñтроками. ЕÑли разница отрицательна — она будет проигнорирована. -Эта Ñ„ÑƒÐ½ÐºÑ†Ð¸Ñ Ð¿Ñ€ÐµÐ´Ð½Ð°Ð·Ð½Ð°Ñ‡ÐµÐ½Ð° в первую очередь Ð´Ð»Ñ [материализованных предÑтавлений](../../../sql-reference/statements/create/view.md#materialized), хранÑщих данные, упорÑдоченные по некоторому временному бакету ÑоглаÑно timestamp (time-series), например, по бакету `toStartOfMinute`. ПоÑкольку Ñтроки в таком материализованном предÑтавлении будут иметь одинаковый timestamp, их невозможно объединить их в правильном порÑдке без Ñ…Ñ€Ð°Ð½ÐµÐ½Ð¸Ñ Ð¸Ñходного, неокругленного Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ timestamp. Ð¤ÑƒÐ½ÐºÑ†Ð¸Ñ `deltaSumTimestamp` отÑлеживает иÑходные `timestamp` наблюдаемых значений, поÑтому Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ (ÑоÑтоÑниÑ) функции правильно вычиÑлÑÑŽÑ‚ÑÑ Ð²Ð¾ Ð²Ñ€ÐµÐ¼Ñ ÑлиÑÐ½Ð¸Ñ ÐºÑƒÑков. +Эта Ñ„ÑƒÐ½ÐºÑ†Ð¸Ñ Ð¿Ñ€ÐµÐ´Ð½Ð°Ð·Ð½Ð°Ñ‡ÐµÐ½Ð° в первую очередь Ð´Ð»Ñ [материализованных предÑтавлений](../../../sql-reference/statements/create/view.md#materialized), хранÑщих данные, упорÑдоченные по некоторому временному бакету ÑоглаÑно timestamp (time-series), например, по бакету `toStartOfMinute`. ПоÑкольку Ñтроки в таком материализованном предÑтавлении будут иметь одинаковый timestamp, их невозможно объединить в правильном порÑдке без Ñ…Ñ€Ð°Ð½ÐµÐ½Ð¸Ñ Ð¸Ñходного, неокругленного Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ timestamp. Ð¤ÑƒÐ½ÐºÑ†Ð¸Ñ `deltaSumTimestamp` отÑлеживает иÑходные `timestamp` наблюдаемых значений, поÑтому Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ (ÑоÑтоÑниÑ) функции правильно вычиÑлÑÑŽÑ‚ÑÑ Ð²Ð¾ Ð²Ñ€ÐµÐ¼Ñ ÑлиÑÐ½Ð¸Ñ ÐºÑƒÑков. Чтобы вычиÑлить разницу между упорÑдоченными поÑледовательными Ñтроками, вы можете иÑпользовать функцию [deltaSum](../../../sql-reference/aggregate-functions/reference/deltasum.md#agg_functions-deltasum) вмеÑто функции `deltaSumTimestamp`. From 028a21658d98e6aad5051de2f2f9b1a94461fbfd Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 9 Apr 2023 19:34:10 -0300 Subject: [PATCH 235/277] Update deltasumtimestamp.md --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index e08e69b7cf6..afcf2a48c23 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -6,7 +6,7 @@ title: deltaSumTimestamp Adds the difference between consecutive rows. If the difference is negative, it is ignored. -This function is primarily for [materialized views](../../../sql-reference/statements/create/view.md#materialized) that are ordered by some time bucket-aligned timestamp, for example, a `toStartOfMinute` bucket. Because the rows in such a materialized view will all have the same timestamp, it is impossible for them to be merged in the "right" order. This function keeps track of the `timestamp` of the values it's seen, so it's possible to order the states correctly during merging. +This function is primarily for [materialized views](../../../sql-reference/statements/create/view.md#materialized) that store data ordered by some time bucket-aligned timestamp, for example, a `toStartOfMinute` bucket. Because the rows in such a materialized view will all have the same timestamp, it is impossible for them to be merged in the correct order, without storing the original, unrounded timestamp value. The `deltaSumTimestamp` function keeps track of the original `timestamp` of the values it's seen, so the values (states) of the function are correctly computed during merging of parts. To calculate the delta sum across an ordered collection you can simply use the [deltaSum](../../../sql-reference/aggregate-functions/reference/deltasum.md#agg_functions-deltasum) function. From 36871fad3498c807b63ab54c2bd9643bef3710b8 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 9 Apr 2023 19:38:08 -0300 Subject: [PATCH 236/277] Update deltasumtimestamp.md --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 8b022241dac..50434419651 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -7,7 +7,7 @@ sidebar_position: 141 Суммирует разницу между поÑледовательными Ñтроками. ЕÑли разница отрицательна — она будет проигнорирована. -Эта Ñ„ÑƒÐ½ÐºÑ†Ð¸Ñ Ð¿Ñ€ÐµÐ´Ð½Ð°Ð·Ð½Ð°Ñ‡ÐµÐ½Ð° в первую очередь Ð´Ð»Ñ [материализованных предÑтавлений](../../../sql-reference/statements/create/view.md#materialized), хранÑщих данные, упорÑдоченные по некоторому временному бакету ÑоглаÑно timestamp (time-series), например, по бакету `toStartOfMinute`. ПоÑкольку Ñтроки в таком материализованном предÑтавлении будут иметь одинаковый timestamp, их невозможно объединить в правильном порÑдке без Ñ…Ñ€Ð°Ð½ÐµÐ½Ð¸Ñ Ð¸Ñходного, неокругленного Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ timestamp. Ð¤ÑƒÐ½ÐºÑ†Ð¸Ñ `deltaSumTimestamp` отÑлеживает иÑходные `timestamp` наблюдаемых значений, поÑтому Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ (ÑоÑтоÑниÑ) функции правильно вычиÑлÑÑŽÑ‚ÑÑ Ð²Ð¾ Ð²Ñ€ÐµÐ¼Ñ ÑлиÑÐ½Ð¸Ñ ÐºÑƒÑков. +Эта Ñ„ÑƒÐ½ÐºÑ†Ð¸Ñ Ð¿Ñ€ÐµÐ´Ð½Ð°Ð·Ð½Ð°Ñ‡ÐµÐ½Ð° в первую очередь Ð´Ð»Ñ [материализованных предÑтавлений](../../../sql-reference/statements/create/view.md#materialized), хранÑщих данные, упорÑдоченные по некоторому округленному временному интервалу, ÑоглаÑно timestamp, например, по бакету `toStartOfMinute`. ПоÑкольку Ñтроки в таком материализованном предÑтавлении будут иметь одинаковый timestamp, их невозможно объединить в правильном порÑдке без Ñ…Ñ€Ð°Ð½ÐµÐ½Ð¸Ñ Ð¸Ñходного, неокругленного Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ timestamp. Ð¤ÑƒÐ½ÐºÑ†Ð¸Ñ `deltaSumTimestamp` отÑлеживает иÑходные `timestamp` наблюдаемых значений, поÑтому Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ (ÑоÑтоÑниÑ) функции правильно вычиÑлÑÑŽÑ‚ÑÑ Ð²Ð¾ Ð²Ñ€ÐµÐ¼Ñ ÑлиÑÐ½Ð¸Ñ ÐºÑƒÑков. Чтобы вычиÑлить разницу между упорÑдоченными поÑледовательными Ñтроками, вы можете иÑпользовать функцию [deltaSum](../../../sql-reference/aggregate-functions/reference/deltasum.md#agg_functions-deltasum) вмеÑто функции `deltaSumTimestamp`. From c06a5ab85f9fa9af39d33c836f51cd8c353a83a8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 01:04:23 +0200 Subject: [PATCH 237/277] Remove slow test from debug builds --- tests/queries/0_stateless/02703_jit_external_aggregation.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02703_jit_external_aggregation.sh b/tests/queries/0_stateless/02703_jit_external_aggregation.sh index d1af5b8b8bc..2d1dda45de0 100755 --- a/tests/queries/0_stateless/02703_jit_external_aggregation.sh +++ b/tests/queries/0_stateless/02703_jit_external_aggregation.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-asan, no-msan, no-tsan, no-ubsan +# Tags: long, no-asan, no-msan, no-tsan, no-ubsan, no-debug CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From e1fa279c9875c35fe655adb460e4836dabf6a782 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 9 Apr 2023 23:40:32 +0000 Subject: [PATCH 238/277] Fix tests for flaky check --- .../0_stateless/02211_jsonl_format_extension.sql | 2 +- .../02455_one_row_from_csv_memory_usage.sh | 13 ++++++------- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02211_jsonl_format_extension.sql b/tests/queries/0_stateless/02211_jsonl_format_extension.sql index a95b19f4e3b..61cc2a408fa 100644 --- a/tests/queries/0_stateless/02211_jsonl_format_extension.sql +++ b/tests/queries/0_stateless/02211_jsonl_format_extension.sql @@ -1,3 +1,3 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-parallel insert into table function file('data.jsonl', 'JSONEachRow', 'x UInt32') select * from numbers(10) SETTINGS engine_file_truncate_on_insert=1; select * from file('data.jsonl') order by x; diff --git a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh index 0b6f91907d4..1b0101e4f06 100755 --- a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh +++ b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh @@ -1,16 +1,15 @@ #!/usr/bin/env bash +# Tags: no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -user_files_path=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -E '^Code: 107.*FILE_DOESNT_EXIST' | head -1 | awk '{gsub("/nonexist.txt","",$9); print $9}') +USER_FILES_PATH=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -E '^Code: 107.*FILE_DOESNT_EXIST' | head -1 | awk '{gsub("/nonexist.txt","",$9); print $9}') -UNIQ_DEST_PATH=$user_files_path/test-02455-$RANDOM-$RANDOM -mkdir -p $UNIQ_DEST_PATH -cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $UNIQ_DEST_PATH/ +cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $USER_FILES_PATH/ -${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$UNIQ_DEST_PATH/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=1000000000" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('$UNIQ_DEST_PATH/10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=100000000" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=1000000000" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings max_memory_usage=100000000" -rm -rf $UNIQ_DEST_PATH +rm $USER_FILES_PATH/10m_rows.csv.xz From 5cb00e13c3bff6721d1ea5dc26f5e39ba904f4a2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 02:31:49 +0200 Subject: [PATCH 239/277] Add a test --- ...12_bool_better_exception_message.reference | 5 ++++ .../02712_bool_better_exception_message.sh | 24 +++++++++++++++++++ 2 files changed, 29 insertions(+) create mode 100644 tests/queries/0_stateless/02712_bool_better_exception_message.reference create mode 100755 tests/queries/0_stateless/02712_bool_better_exception_message.sh diff --git a/tests/queries/0_stateless/02712_bool_better_exception_message.reference b/tests/queries/0_stateless/02712_bool_better_exception_message.reference new file mode 100644 index 00000000000..aba60fe1061 --- /dev/null +++ b/tests/queries/0_stateless/02712_bool_better_exception_message.reference @@ -0,0 +1,5 @@ +true +false +1 +1 +1 diff --git a/tests/queries/0_stateless/02712_bool_better_exception_message.sh b/tests/queries/0_stateless/02712_bool_better_exception_message.sh new file mode 100755 index 00000000000..ce6a4f4874b --- /dev/null +++ b/tests/queries/0_stateless/02712_bool_better_exception_message.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh +SCHEMADIR=$CURDIR/format_schemas + +$CLICKHOUSE_LOCAL <&1 | rg -Fc "'w' character" + SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": wtf}'); +END + +$CLICKHOUSE_LOCAL <&1 | rg -Fc "expected 'false'" + SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": ftw}'); +END + +$CLICKHOUSE_LOCAL <&1 | rg -Fc "'{' character" + SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": {}}'); +END From 1f6b9809dd7bda0a0fa83be5f42e24f4b205128a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 02:32:04 +0200 Subject: [PATCH 240/277] Better exception messages for unparsed Bool --- src/DataTypes/Serializations/SerializationBool.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index 81ad0ec46b1..41b5bf806e5 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -238,12 +238,15 @@ void SerializationBool::deserializeTextJSON(IColumn &column, ReadBuffer &istr, c ColumnUInt8 * col = checkAndGetDeserializeColumnType(column); bool value = false; - if (*istr.position() == 't' || *istr.position() == 'f') + char first_char = *istr.position(); + if (first_char == 't' || first_char == 'f') readBoolTextWord(value, istr); - else if (*istr.position() == '1' || *istr.position() == '0') + else if (first_char == '1' || first_char == '0') readBoolText(value, istr); else - throw Exception(ErrorCodes::CANNOT_PARSE_BOOL, "Invalid boolean value, should be true/false, 1/0."); + throw Exception(ErrorCodes::CANNOT_PARSE_BOOL, + "Invalid boolean value, should be true/false, 1/0, but it starts with the '{}' character.", first_char); + col->insert(value); } From a24cc9580cea69c9fc2e70db030fdfebd3d2a5c2 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 10 Apr 2023 03:11:10 +0200 Subject: [PATCH 241/277] Update syntax.md --- docs/en/sql-reference/syntax.md | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/en/sql-reference/syntax.md b/docs/en/sql-reference/syntax.md index 63c5042f9e8..00d11ccff40 100644 --- a/docs/en/sql-reference/syntax.md +++ b/docs/en/sql-reference/syntax.md @@ -14,7 +14,7 @@ The `INSERT` query uses both parsers: INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') ``` -The `INSERT INTO t VALUES` fragment is parsed by the full parser, and the data `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` is parsed by the fast stream parser. You can also turn on the full parser for the data by using the [input_format_values_interpret_expressions](../operations/settings/settings-formats.md#settings-input_format_values_interpret_expressions) setting. When `input_format_values_interpret_expressions = 1`, ClickHouse first tries to parse values with the fast stream parser. If it fails, ClickHouse tries to use the full parser for the data, treating it like an SQL [expression](#syntax-expressions). +The `INSERT INTO t VALUES` fragment is parsed by the full parser, and the data `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` is parsed by the fast stream parser. You can also turn on the full parser for the data by using the [input_format_values_interpret_expressions](../operations/settings/settings-formats.md#settings-input_format_values_interpret_expressions) setting. When `input_format_values_interpret_expressions = 1`, ClickHouse first tries to parse values with the fast stream parser. If it fails, ClickHouse tries to use the full parser for the data, treating it like an SQL [expression](#expressions). Data can have any format. When a query is received, the server calculates no more than [max_query_size](../operations/settings/settings.md#settings-max_query_size) bytes of the request in RAM (by default, 1 MB), and the rest is stream parsed. It allows for avoiding issues with large `INSERT` queries. @@ -45,7 +45,7 @@ You can check whether a data type name is case-sensitive in the [system.data_typ In contrast to standard SQL, all other keywords (including functions names) are **case-sensitive**. -Keywords are not reserved; they are treated as such only in the corresponding context. If you use [identifiers](#syntax-identifiers) with the same name as the keywords, enclose them into double-quotes or backticks. For example, the query `SELECT "FROM" FROM table_name` is valid if the table `table_name` has column with the name `"FROM"`. +Keywords are not reserved; they are treated as such only in the corresponding context. If you use [identifiers](#identifiers) with the same name as the keywords, enclose them into double-quotes or backticks. For example, the query `SELECT "FROM" FROM table_name` is valid if the table `table_name` has column with the name `"FROM"`. ## Identifiers @@ -54,7 +54,7 @@ Identifiers are: - Cluster, database, table, partition, and column names. - Functions. - Data types. -- [Expression aliases](#syntax-expression_aliases). +- [Expression aliases](#expression_aliases). Identifiers can be quoted or non-quoted. The latter is preferred. @@ -149,7 +149,7 @@ For example, the following SQL defines parameters named `a`, `b`, `c` and `d` - SET param_a = 13; SET param_b = 'str'; SET param_c = '2022-08-04 18:30:53'; -SET param_d = {'10': [11, 12], '13': [14, 15]}'; +SET param_d = {'10': [11, 12], '13': [14, 15]}; SELECT {a: UInt32}, @@ -166,7 +166,7 @@ Result: If you are using `clickhouse-client`, the parameters are specified as `--param_name=value`. For example, the following parameter has the name `message` and it is retrieved as a `String`: -```sql +```bash clickhouse-client --param_message='hello' --query="SELECT {message: String}" ``` @@ -190,7 +190,7 @@ Query parameters are not general text substitutions which can be used in arbitra ## Functions Function calls are written like an identifier with a list of arguments (possibly empty) in round brackets. In contrast to standard SQL, the brackets are required, even for an empty argument list. Example: `now()`. -There are regular and aggregate functions (see the section “Aggregate functionsâ€). Some aggregate functions can contain two lists of arguments in brackets. Example: `quantile (0.9) (x)`. These aggregate functions are called “parametric†functions, and the arguments in the first list are called “parametersâ€. The syntax of aggregate functions without parameters is the same as for regular functions. +There are regular and aggregate functions (see the section [Aggregate functions](/docs/en/sql-reference/aggregate-functions/index.md)). Some aggregate functions can contain two lists of arguments in brackets. Example: `quantile (0.9) (x)`. These aggregate functions are called “parametric†functions, and the arguments in the first list are called “parametersâ€. The syntax of aggregate functions without parameters is the same as for regular functions. ## Operators @@ -199,7 +199,7 @@ For example, the expression `1 + 2 * 3 + 4` is transformed to `plus(plus(1, mult ## Data Types and Database Table Engines -Data types and table engines in the `CREATE` query are written the same way as identifiers or functions. In other words, they may or may not contain an argument list in brackets. For more information, see the sections “Data types,†“Table engines,†and “CREATEâ€. +Data types and table engines in the `CREATE` query are written the same way as identifiers or functions. In other words, they may or may not contain an argument list in brackets. For more information, see the sections [Data types](/docs/en/sql-reference/data-types/index.md), [Table engines](/docs/en/engines/table-engines/index.md), and [CREATE](/docs/en/sql-reference/statements/create/index.md). ## Expression Aliases @@ -211,17 +211,17 @@ expr AS alias - `AS` — The keyword for defining aliases. You can define the alias for a table name or a column name in a `SELECT` clause without using the `AS` keyword. - For example, `SELECT table_name_alias.column_name FROM table_name table_name_alias`. + For example, `SELECT table_name_alias.column_name FROM table_name table_name_alias`. - In the [CAST](./functions/type-conversion-functions.md#type_conversion_function-cast) function, the `AS` keyword has another meaning. See the description of the function. + In the [CAST](./functions/type-conversion-functions.md#type_conversion_function-cast) function, the `AS` keyword has another meaning. See the description of the function. - `expr` — Any expression supported by ClickHouse. - For example, `SELECT column_name * 2 AS double FROM some_table`. + For example, `SELECT column_name * 2 AS double FROM some_table`. -- `alias` — Name for `expr`. Aliases should comply with the [identifiers](#syntax-identifiers) syntax. +- `alias` — Name for `expr`. Aliases should comply with the [identifiers](#identifiers) syntax. - For example, `SELECT "table t".column_name FROM table_name AS "table t"`. + For example, `SELECT "table t".column_name FROM table_name AS "table t"`. ### Notes on Usage @@ -258,7 +258,7 @@ In this example, we declared table `t` with column `b`. Then, when selecting dat ## Asterisk -In a `SELECT` query, an asterisk can replace the expression. For more information, see the section “SELECTâ€. +In a `SELECT` query, an asterisk can replace the expression. For more information, see the section [SELECT](/docs/en/sql-reference/statements/select/index.md#asterisk). ## Expressions From 8eaf20e1294a67344260e5d967a4b02db9ef7dd7 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 10 Apr 2023 03:26:43 +0200 Subject: [PATCH 242/277] Update syntax.md --- docs/en/sql-reference/syntax.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/syntax.md b/docs/en/sql-reference/syntax.md index 00d11ccff40..ea2df235c1a 100644 --- a/docs/en/sql-reference/syntax.md +++ b/docs/en/sql-reference/syntax.md @@ -14,7 +14,7 @@ The `INSERT` query uses both parsers: INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') ``` -The `INSERT INTO t VALUES` fragment is parsed by the full parser, and the data `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` is parsed by the fast stream parser. You can also turn on the full parser for the data by using the [input_format_values_interpret_expressions](../operations/settings/settings-formats.md#settings-input_format_values_interpret_expressions) setting. When `input_format_values_interpret_expressions = 1`, ClickHouse first tries to parse values with the fast stream parser. If it fails, ClickHouse tries to use the full parser for the data, treating it like an SQL [expression](#expressions). +The `INSERT INTO t VALUES` fragment is parsed by the full parser, and the data `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` is parsed by the fast stream parser. You can also turn on the full parser for the data by using the [input_format_values_interpret_expressions](../operations/settings/settings-formats.md#input_format_values_interpret_expressions) setting. When `input_format_values_interpret_expressions = 1`, ClickHouse first tries to parse values with the fast stream parser. If it fails, ClickHouse tries to use the full parser for the data, treating it like an SQL [expression](#expressions). Data can have any format. When a query is received, the server calculates no more than [max_query_size](../operations/settings/settings.md#settings-max_query_size) bytes of the request in RAM (by default, 1 MB), and the rest is stream parsed. It allows for avoiding issues with large `INSERT` queries. @@ -108,7 +108,7 @@ Depending on the data format (input or output), `NULL` may have a different repr There are many nuances to processing `NULL`. For example, if at least one of the arguments of a comparison operation is `NULL`, the result of this operation is also `NULL`. The same is true for multiplication, addition, and other operations. For more information, read the documentation for each operation. -In queries, you can check `NULL` using the [IS NULL](../sql-reference/operators/index.md#operator-is-null) and [IS NOT NULL](../sql-reference/operators/index.md) operators and the related functions `isNull` and `isNotNull`. +In queries, you can check `NULL` using the [IS NULL](../sql-reference/operators/index.md#is-null) and [IS NOT NULL](../sql-reference/operators/index.md#is-not-null) operators and the related functions `isNull` and `isNotNull`. ### Heredoc @@ -213,7 +213,7 @@ expr AS alias For example, `SELECT table_name_alias.column_name FROM table_name table_name_alias`. - In the [CAST](./functions/type-conversion-functions.md#type_conversion_function-cast) function, the `AS` keyword has another meaning. See the description of the function. + In the [CAST](./functions/type-conversion-functions.md#castx-t) function, the `AS` keyword has another meaning. See the description of the function. - `expr` — Any expression supported by ClickHouse. @@ -254,7 +254,7 @@ Received exception from server (version 18.14.17): Code: 184. DB::Exception: Received from localhost:9000, 127.0.0.1. DB::Exception: Aggregate function sum(b) is found inside another aggregate function in query. ``` -In this example, we declared table `t` with column `b`. Then, when selecting data, we defined the `sum(b) AS b` alias. As aliases are global, ClickHouse substituted the literal `b` in the expression `argMax(a, b)` with the expression `sum(b)`. This substitution caused the exception. You can change this default behavior by setting [prefer_column_name_to_alias](../operations/settings/settings.md#prefer_column_name_to_alias) to `1`. +In this example, we declared table `t` with column `b`. Then, when selecting data, we defined the `sum(b) AS b` alias. As aliases are global, ClickHouse substituted the literal `b` in the expression `argMax(a, b)` with the expression `sum(b)`. This substitution caused the exception. You can change this default behavior by setting [prefer_column_name_to_alias](../operations/settings/settings.md#prefer-column-name-to-alias) to `1`. ## Asterisk From 8e90b7d446521af0a40ec3c0096f62e7e9395321 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 03:31:43 +0200 Subject: [PATCH 243/277] Fixup --- .../queries/0_stateless/02712_bool_better_exception_message.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02712_bool_better_exception_message.sh b/tests/queries/0_stateless/02712_bool_better_exception_message.sh index ce6a4f4874b..1da70d7025b 100755 --- a/tests/queries/0_stateless/02712_bool_better_exception_message.sh +++ b/tests/queries/0_stateless/02712_bool_better_exception_message.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -SCHEMADIR=$CURDIR/format_schemas + $CLICKHOUSE_LOCAL < Date: Mon, 10 Apr 2023 03:39:13 +0200 Subject: [PATCH 244/277] Don't use type conversion with String query parameters --- src/Interpreters/ReplaceQueryParameterVisitor.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index fad9d4bbfb2..71f28f9fb2c 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -102,7 +102,10 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) else literal = temp_column[0]; - ast = addTypeConversionToAST(std::make_shared(literal), type_name); + if (typeid_cast(&data_type)) + ast = std::make_shared(literal); + else + ast = addTypeConversionToAST(std::make_shared(literal), type_name); /// Keep the original alias. ast->setAlias(alias); From e71c7fe7449f18591a6a3bf375495fa792939ee8 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 10 Apr 2023 03:48:58 +0200 Subject: [PATCH 245/277] Fix --- src/Interpreters/ReplaceQueryParameterVisitor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index 71f28f9fb2c..7ed56dce9d0 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -102,7 +103,7 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) else literal = temp_column[0]; - if (typeid_cast(&data_type)) + if (typeid_cast(data_type.get())) ast = std::make_shared(literal); else ast = addTypeConversionToAST(std::make_shared(literal), type_name); From 345b33dc1fba2937fc758ddd70e0e5f7cd1af6d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 04:15:13 +0200 Subject: [PATCH 246/277] Slightly optimize Install --- programs/install/Install.cpp | 21 ++++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index b142159fbdf..d83e189f7ef 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -375,15 +375,22 @@ int mainEntryClickHouseInstall(int argc, char ** argv) try { - ReadBufferFromFile in(binary_self_path.string()); - WriteBufferFromFile out(main_bin_tmp_path.string()); - copyData(in, out); - out.sync(); + String source = binary_self_path.string(); + String destination = main_bin_tmp_path.string(); - if (0 != fchmod(out.getFD(), S_IRUSR | S_IRGRP | S_IROTH | S_IXUSR | S_IXGRP | S_IXOTH)) + /// Try to make a hard link first, as an optimization. + /// It is possible if the source and the destination are on the same filesystems. + if (0 != link(source.c_str(), destination.c_str())) + { + ReadBufferFromFile in(binary_self_path.string()); + WriteBufferFromFile out(main_bin_tmp_path.string()); + copyData(in, out); + out.sync(); + out.finalize(); + } + + if (0 != chmod(destination.c_str(), S_IRUSR | S_IRGRP | S_IROTH | S_IXUSR | S_IXGRP | S_IXOTH)) throwFromErrno(fmt::format("Cannot chmod {}", main_bin_tmp_path.string()), ErrorCodes::SYSTEM_ERROR); - - out.finalize(); } catch (const Exception & e) { From 6738ab66600d9b6857af07b8d98bf2f0d5277348 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 04:22:17 +0200 Subject: [PATCH 247/277] Fix TSan report in Kerberos --- contrib/krb5 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/krb5 b/contrib/krb5 index 9453aec0d50..b56ce6ba690 160000 --- a/contrib/krb5 +++ b/contrib/krb5 @@ -1 +1 @@ -Subproject commit 9453aec0d50e5aff9b189051611b321b40935d02 +Subproject commit b56ce6ba690e1f320df1a64afa34980c3e462617 From 8524f6958a82e772919e02f2cb6647e8286d3a02 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 10 Apr 2023 09:09:57 +0200 Subject: [PATCH 248/277] Update src/Interpreters/ReplaceQueryParameterVisitor.cpp Co-authored-by: Alexey Milovidov --- src/Interpreters/ReplaceQueryParameterVisitor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index 7ed56dce9d0..c92161c72c3 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -103,6 +103,9 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) else literal = temp_column[0]; + /// If it's a String, substitute it in the form of a string literal without CAST + /// to enable substitutions in simple queries that don't support expressions + /// (such as CREATE USER). if (typeid_cast(data_type.get())) ast = std::make_shared(literal); else From 000c2b5b9128ce9c2c08d530aa0b14e5f91ed7a6 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Mon, 10 Apr 2023 13:02:31 +0300 Subject: [PATCH 249/277] Correct FixupACL for auth scheme --- src/Coordination/KeeperStorage.cpp | 71 +++++++++++++++++++++--------- src/Coordination/KeeperStorage.h | 4 ++ 2 files changed, 54 insertions(+), 21 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 41a6af54204..6b2696034f1 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -61,16 +61,10 @@ String getSHA1(const String & userdata) return String{digest_id.begin(), digest_id.end()}; } -String generateDigest(const String & userdata) -{ - std::vector user_password; - boost::split(user_password, userdata, [](char character) { return character == ':'; }); - return user_password[0] + ":" + base64Encode(getSHA1(userdata)); -} - bool fixupACL( const std::vector & request_acls, - const std::vector & current_ids, + int64_t session_id, + const KeeperStorage::UncommittedState & uncommitted_state, std::vector & result_acls) { if (request_acls.empty()) @@ -81,14 +75,18 @@ bool fixupACL( { if (request_acl.scheme == "auth") { - for (const auto & current_id : current_ids) - { - valid_found = true; - Coordination::ACL new_acl = request_acl; - new_acl.scheme = current_id.scheme; - new_acl.id = current_id.id; - result_acls.push_back(new_acl); - } + uncommitted_state.forEachAuthInSession( + session_id, + [&](const KeeperStorage::AuthID & auth_id) + { + valid_found = true; + Coordination::ACL new_acl = request_acl; + + new_acl.scheme = auth_id.scheme; + new_acl.id = auth_id.id; + + result_acls.push_back(new_acl); + }); } else if (request_acl.scheme == "world" && request_acl.id == "anyone") { @@ -564,6 +562,32 @@ Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) cons return storage.acl_map.convertNumber(node_it->value.acl_id); } +void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t session_id, std::function func) const +{ + const auto call_for_each_auth = [&func](const auto & auth_ids) + { + for (const auto & auth : auth_ids) + { + using TAuth = std::remove_reference_t; + + const AuthID * auth_ptr = nullptr; + if constexpr (std::is_pointer_v) + auth_ptr = auth; + else + auth_ptr = &auth; + + func(*auth_ptr); + } + }; + + // for committed + if (storage.session_and_auth.contains(session_id)) + call_for_each_auth(storage.session_and_auth.at(session_id)); + // for uncommitted + if (session_and_auth.contains(session_id)) + call_for_each_auth(session_and_auth.at(session_id)); +} + namespace { @@ -927,7 +951,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; Coordination::ACLs node_acls; - if (!fixupACL(request.acls, storage.session_and_auth[session_id], node_acls)) + if (!fixupACL(request.acls, session_id, storage.uncommitted_state, node_acls)) return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; if (request.is_ephemeral) @@ -1533,10 +1557,8 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; - auto & session_auth_ids = storage.session_and_auth[session_id]; Coordination::ACLs node_acls; - - if (!fixupACL(request.acls, session_auth_ids, node_acls)) + if (!fixupACL(request.acls, session_id, uncommitted_state, node_acls)) return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; std::vector new_deltas @@ -1840,7 +1862,7 @@ struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProc return {KeeperStorage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; std::vector new_deltas; - auto auth_digest = generateDigest(auth_request.data); + auto auth_digest = KeeperStorage::generateDigest(auth_request.data); if (auth_digest == storage.superdigest) { KeeperStorage::AuthID auth{"super", ""}; @@ -2420,5 +2442,12 @@ void KeeperStorage::recalculateStats() container.recalculateDataSize(); } +String KeeperStorage::generateDigest(const String & userdata) +{ + std::vector user_password; + boost::split(user_password, userdata, [](char character) { return character == ':'; }); + return user_password[0] + ":" + base64Encode(getSHA1(userdata)); +} + } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index be528072df4..cfacdfc84de 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -105,6 +105,8 @@ public: return first.value == second.value; } + static String generateDigest(const String & userdata); + struct RequestForSession { int64_t session_id; @@ -263,6 +265,8 @@ public: return check_auth(auth_it->second); } + void forEachAuthInSession(int64_t session_id, std::function func) const; + std::shared_ptr tryGetNodeFromStorage(StringRef path) const; std::unordered_map> session_and_auth; From e8bf96d126fe08c7264bcec7effc8410aa1284a8 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Mon, 10 Apr 2023 13:02:43 +0300 Subject: [PATCH 250/277] Add unit tests --- src/Coordination/tests/gtest_coordination.cpp | 107 ++++++++++++++++++ 1 file changed, 107 insertions(+) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 895d563327e..68e68ca1fa7 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1579,6 +1579,113 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) } +TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitted) +{ + using namespace Coordination; + using namespace DB; + + ChangelogDirTest snapshots("./snapshots"); + CoordinationSettingsPtr settings = std::make_shared(); + ResponsesQueue queue(std::numeric_limits::max()); + SnapshotsQueue snapshots_queue{1}; + + auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings, keeper_context, nullptr); + state_machine->init(); + + String user_auth_data = "test_user:test_password"; + String digest = KeeperStorage::generateDigest(user_auth_data); + + std::shared_ptr auth_req = std::make_shared(); + auth_req->scheme = "digest"; + auth_req->data = user_auth_data; + + // Add auth data to the session + auto auth_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), auth_req); + state_machine->pre_commit(1, auth_entry->get_buf()); + + // Create a node with 'auth' scheme for ACL + String node_path = "/hello"; + std::shared_ptr create_req = std::make_shared(); + create_req->path = node_path; + // When 'auth' scheme is used the creator must have been authenticated by the server (for example, using 'digest' scheme) before it can + // create nodes with this ACL. + create_req->acls = {{.permissions = 31, .scheme = "auth", .id = ""}}; + auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req); + state_machine->pre_commit(2, create_entry->get_buf()); + + const auto & uncommitted_state = state_machine->getStorage().uncommitted_state; + ASSERT_TRUE(uncommitted_state.nodes.contains(node_path)); + + // commit log entries + state_machine->commit(1, auth_entry->get_buf()); + state_machine->commit(2, create_entry->get_buf()); + + auto node = uncommitted_state.getNode(node_path); + ASSERT_NE(node, nullptr); + auto acls = uncommitted_state.getACLs(node_path); + ASSERT_EQ(acls.size(), 1); + EXPECT_EQ(acls[0].scheme, "digest"); + EXPECT_EQ(acls[0].id, digest); + EXPECT_EQ(acls[0].permissions, 31); +} + +TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) +{ + using namespace Coordination; + using namespace DB; + + ChangelogDirTest snapshots("./snapshots"); + CoordinationSettingsPtr settings = std::make_shared(); + ResponsesQueue queue(std::numeric_limits::max()); + SnapshotsQueue snapshots_queue{1}; + + auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings, keeper_context, nullptr); + state_machine->init(); + + String user_auth_data = "test_user:test_password"; + String digest = KeeperStorage::generateDigest(user_auth_data); + + std::shared_ptr auth_req = std::make_shared(); + auth_req->scheme = "digest"; + auth_req->data = user_auth_data; + + // Add auth data to the session + auto auth_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), auth_req); + state_machine->pre_commit(1, auth_entry->get_buf()); + + // Create a node + String node_path = "/hello"; + std::shared_ptr create_req = std::make_shared(); + create_req->path = node_path; + auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req); + state_machine->pre_commit(2, create_entry->get_buf()); + + // Set ACL with 'auth' scheme for ACL + std::shared_ptr set_acl_req = std::make_shared(); + set_acl_req->path = node_path; + // When 'auth' scheme is used the creator must have been authenticated by the server (for example, using 'digest' scheme) before it can + // set this ACL. + set_acl_req->acls = {{.permissions = 31, .scheme = "auth", .id = ""}}; + auto set_acl_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), set_acl_req); + state_machine->pre_commit(3, set_acl_entry->get_buf()); + + // commit all entries + state_machine->commit(1, auth_entry->get_buf()); + state_machine->commit(2, create_entry->get_buf()); + state_machine->commit(2, set_acl_entry->get_buf()); + + const auto & uncommitted_state = state_machine->getStorage().uncommitted_state; + auto node = uncommitted_state.getNode(node_path); + + ASSERT_NE(node, nullptr); + auto acls = uncommitted_state.getACLs(node_path); + ASSERT_EQ(acls.size(), 1); + EXPECT_EQ(acls[0].scheme, "digest"); + EXPECT_EQ(acls[0].id, digest); + EXPECT_EQ(acls[0].permissions, 31); +} + + TEST_P(CoordinationTest, TestRotateIntervalChanges) { using namespace Coordination; From d564fe4e6446f503250d02f806c1f3eab30cad9c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 10 Apr 2023 10:54:37 +0000 Subject: [PATCH 251/277] Fixing build --- src/Storages/System/StorageSystemZooKeeper.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index c96c8ec60ae..e21bd887ecb 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -356,7 +356,7 @@ static void extractPathImpl(const ActionsDAG::Node & node, Paths & res, ContextP if (!isPathNode(node.children.at(0))) return; - auto value = node.children.at(1); + const auto * value = node.children.at(1); if (!value->column) return; @@ -504,7 +504,7 @@ void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns) const } ReadFromSystemZooKeeper::ReadFromSystemZooKeeper(const Block & header, SelectQueryInfo & query_info, ContextPtr context_) - : SourceStepWithFilter({.header = std::move(header)}) + : SourceStepWithFilter({.header = header}) , storage_limits(query_info.storage_limits) , context(std::move(context_)) { From 450e8207285b411f614564011a2845b9e22aaa6b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 10 Apr 2023 16:18:48 +0200 Subject: [PATCH 252/277] Docs: Add SETTINGS to executable table function --- docs/en/sql-reference/table-functions/executable.md | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/table-functions/executable.md b/docs/en/sql-reference/table-functions/executable.md index 22c74eb8cfa..5a24c3ab11d 100644 --- a/docs/en/sql-reference/table-functions/executable.md +++ b/docs/en/sql-reference/table-functions/executable.md @@ -20,7 +20,7 @@ A key advantage between ordinary UDF functions and the `executable` table functi The `executable` table function requires three parameters and accepts an optional list of input queries: ```sql -executable(script_name, format, structure, [input_query...]) +executable(script_name, format, structure, [input_query...] [,SETTINGS ...]) ``` - `script_name`: the file name of the script. saved in the `user_scripts` folder (the default folder of the `user_scripts_path` setting) @@ -83,6 +83,15 @@ The response looks like: └────┴────────────┘ ``` +## Settings + +- `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Default value is `false`. +- `pool_size` — Size of pool. If 0 is specified as `pool_size` then there is no pool size restrictions. Default value is `16`. +- `max_command_execution_time` — Maximum executable script command execution time for processing block of data. Specified in seconds. Default value is 10. +- `command_termination_timeout` — executable script should contain main read-write loop. After table function is destroyed, pipe is closed, and executable file will have `command_termination_timeout` seconds to shutdown, before ClickHouse will send SIGTERM signal to child process. Specified in seconds. Default value is 10. +- `command_read_timeout` - timeout for reading data from command stdout in milliseconds. Default value 10000. +- `command_write_timeout` - timeout for writing data to command stdin in milliseconds. Default value 10000. + ## Passing Query Results to a Script Be sure to check out the example in the `Executable` table engine on [how to pass query results to a script](../../engines/table-engines/special/executable.md#passing-query-results-to-a-script). Here is how you execute the same script in that example using the `executable` table function: @@ -94,4 +103,4 @@ SELECT * FROM executable( 'id UInt64, sentiment Float32', (SELECT id, comment FROM hackernews WHERE id > 0 AND comment != '' LIMIT 20) ); -``` \ No newline at end of file +``` From d50c6a9e1b546a109c044cc6a2af3d445f75f519 Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Mon, 10 Apr 2023 15:23:00 +0100 Subject: [PATCH 253/277] update blog references --- docs/en/engines/database-engines/postgresql.md | 1 + docs/en/engines/table-engines/integrations/postgresql.md | 2 ++ .../mergetree-family/aggregatingmergetree.md | 4 ++++ .../table-engines/mergetree-family/invertedindexes.md | 4 ++++ .../table-engines/mergetree-family/summingmergetree.md | 4 ++++ docs/en/engines/table-engines/special/keepermap.md | 4 ++++ docs/en/getting-started/example-datasets/github.md | 8 +++++--- docs/en/operations/opentelemetry.md | 4 ++++ docs/en/operations/query-cache.md | 4 ++++ docs/en/operations/utilities/clickhouse-local.md | 4 ++++ docs/en/sql-reference/aggregate-functions/combinators.md | 5 +++++ docs/en/sql-reference/data-types/aggregatefunction.md | 5 +++++ docs/en/sql-reference/data-types/map.md | 5 +++++ docs/en/sql-reference/statements/alter/delete.md | 4 ++++ docs/en/sql-reference/statements/alter/index.md | 4 ++++ docs/en/sql-reference/statements/alter/update.md | 5 +++++ docs/en/sql-reference/statements/create/view.md | 1 + docs/en/sql-reference/statements/delete.md | 4 ++++ docs/en/sql-reference/statements/select/join.md | 4 ++++ docs/en/sql-reference/table-functions/postgresql.md | 2 ++ 20 files changed, 75 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/database-engines/postgresql.md b/docs/en/engines/database-engines/postgresql.md index 939995a61c5..95d6a6192e3 100644 --- a/docs/en/engines/database-engines/postgresql.md +++ b/docs/en/engines/database-engines/postgresql.md @@ -140,3 +140,4 @@ DESCRIBE TABLE test_database.test_table; ## Related content - Blog: [ClickHouse and PostgreSQL - a match made in data heaven - part 1](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres) +- Blog: [ClickHouse and PostgreSQL - a Match Made in Data Heaven - part 2](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres-part-2) diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index 2222d1fc016..2fcc0b10e78 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -177,4 +177,6 @@ CREATE TABLE pg_table_schema_with_dots (a UInt32) - [Using PostgreSQL as a dictionary source](../../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-postgresql) ## Related content + - Blog: [ClickHouse and PostgreSQL - a match made in data heaven - part 1](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres) +- Blog: [ClickHouse and PostgreSQL - a Match Made in Data Heaven - part 2](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres-part-2) diff --git a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md index 6591f666244..2b8b43802ea 100644 --- a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md @@ -122,3 +122,7 @@ FROM test.mv_visits GROUP BY StartDate ORDER BY StartDate; ``` + +## Related Content + +- Blog: [Using Aggregate Combinators in ClickHouse](https://clickhouse.com/blog/aggregate-functions-combinators-in-clickhouse-for-arrays-maps-and-states) diff --git a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md index 701615495de..31f5a87a2b6 100644 --- a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md @@ -191,3 +191,7 @@ is performance. In practice, users often search for multiple terms at once. For '%big%'` can be evaluated directly using an inverted index by forming the union of the row id lists for terms "little" and "big". This also means that the parameter `GRANULARITY` supplied to index creation has no meaning (it may be removed from the syntax in the future). ::: + +## Related Content + +- Blog: [Introducing Inverted Indices in ClickHouse](https://clickhouse.com/blog/clickhouse-search-with-inverted-indices) diff --git a/docs/en/engines/table-engines/mergetree-family/summingmergetree.md b/docs/en/engines/table-engines/mergetree-family/summingmergetree.md index d0078656b5d..d15323bbc88 100644 --- a/docs/en/engines/table-engines/mergetree-family/summingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/summingmergetree.md @@ -186,3 +186,7 @@ ARRAY JOIN When requesting data, use the [sumMap(key, value)](../../../sql-reference/aggregate-functions/reference/summap.md) function for aggregation of `Map`. For nested data structure, you do not need to specify its columns in the tuple of columns for summation. + +## Related Content + +- Blog: [Using Aggregate Combinators in ClickHouse](https://clickhouse.com/blog/aggregate-functions-combinators-in-clickhouse-for-arrays-maps-and-states) diff --git a/docs/en/engines/table-engines/special/keepermap.md b/docs/en/engines/table-engines/special/keepermap.md index e5c4dea2339..a1c7009b712 100644 --- a/docs/en/engines/table-engines/special/keepermap.md +++ b/docs/en/engines/table-engines/special/keepermap.md @@ -112,3 +112,7 @@ If setting `keeper_map_strict_mode` is set to `true`, fetching and updating data ```sql ALTER TABLE keeper_map_table UPDATE v1 = v1 * 10 + 2 WHERE key LIKE 'some%' AND v3 > 3.1; ``` + +## Related content + +- Blog: [Building a Real-time Analytics Apps with ClickHouse and Hex](https://clickhouse.com/blog/building-real-time-applications-with-clickhouse-and-hex-notebook-keeper-engine) diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index e18c7dec1a6..02965ed5e33 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -2499,7 +2499,9 @@ LIMIT 20 We welcome exact and improved solutions here. -# Related Content +## Related Content -- [Git commits and our community](https://clickhouse.com/blog/clickhouse-git-community-commits) -- [Window and array functions for Git commit sequences](https://clickhouse.com/blog/clickhouse-window-array-functions-git-commits) +- Blog: [Git commits and our community](https://clickhouse.com/blog/clickhouse-git-community-commits) +- Blog: [Window and array functions for Git commit sequences](https://clickhouse.com/blog/clickhouse-window-array-functions-git-commits) +- Blog: [Building a Real-time Analytics Apps with ClickHouse and Hex](https://clickhouse.com/blog/building-real-time-applications-with-clickhouse-and-hex-notebook-keeper-engine) +- Blog: [A Story of Open-source GitHub Activity using ClickHouse + Grafana](https://clickhouse.com/blog/introduction-to-clickhouse-and-grafana-webinar) diff --git a/docs/en/operations/opentelemetry.md b/docs/en/operations/opentelemetry.md index 15185f7ae6b..70f64d08ba3 100644 --- a/docs/en/operations/opentelemetry.md +++ b/docs/en/operations/opentelemetry.md @@ -61,3 +61,7 @@ FROM system.opentelemetry_span_log ``` In case of any errors, the part of the log data for which the error has occurred will be silently lost. Check the server log for error messages if the data does not arrive. + +## Related Content + +- Blog: [Building an Observability Solution with ClickHouse - Part 2 - Traces](https://clickhouse.com/blog/storing-traces-and-spans-open-telemetry-in-clickhouse) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 7b106909cf0..e9301ef3051 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -124,3 +124,7 @@ Finally, entries in the query cache are not shared between users due to security row policy on a table by running the same query as another user B for whom no such policy exists. However, if necessary, cache entries can be marked accessible by other users (i.e. shared) by supplying setting [query_cache_share_between_users](settings/settings.md#query-cache-share-between-users). + +## Related Content + +- Blog: [Introducing the ClickHouse Query Cache](https://clickhouse.com/blog/introduction-to-the-clickhouse-query-cache-and-design) diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index 6363d9cab27..4439f823240 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -6,6 +6,10 @@ sidebar_label: clickhouse-local # clickhouse-local +## Related Content + +- Blog: [Extracting, Converting, and Querying Data in Local Files using clickhouse-local](https://clickhouse.com/blog/extracting-converting-querying-local-files-with-sql-clickhouse-local) + ## When to use clickhouse-local vs. ClickHouse `clickhouse-local` is an easy-to-use version of ClickHouse that is ideal for developers who need to perform fast processing on local and remote files using SQL without having to install a full database server. With `clickhouse-local`, developers can use SQL commands (using the [ClickHouse SQL dialect](../../sql-reference/index.md)) directly from the command line, providing a simple and efficient way to access ClickHouse features without the need for a full ClickHouse installation. One of the main benefits of `clickhouse-local` is that it is already included when installing [clickhouse-client](https://clickhouse.com/docs/en/integrations/sql-clients/clickhouse-client-local). This means that developers can get started with `clickhouse-local` quickly, without the need for a complex installation process. diff --git a/docs/en/sql-reference/aggregate-functions/combinators.md b/docs/en/sql-reference/aggregate-functions/combinators.md index 704e88c6313..a28e58ca880 100644 --- a/docs/en/sql-reference/aggregate-functions/combinators.md +++ b/docs/en/sql-reference/aggregate-functions/combinators.md @@ -285,3 +285,8 @@ FROM people │ [3,2] │ [11.5,12.949999809265137] │ └────────┴───────────────────────────┘ ``` + + +## Related Content + +- Blog: [Using Aggregate Combinators in ClickHouse](https://clickhouse.com/blog/aggregate-functions-combinators-in-clickhouse-for-arrays-maps-and-states) diff --git a/docs/en/sql-reference/data-types/aggregatefunction.md b/docs/en/sql-reference/data-types/aggregatefunction.md index d8547f03714..90dd8bd2311 100644 --- a/docs/en/sql-reference/data-types/aggregatefunction.md +++ b/docs/en/sql-reference/data-types/aggregatefunction.md @@ -63,3 +63,8 @@ SELECT uniqMerge(state) FROM (SELECT uniqState(UserID) AS state FROM table GROUP ## Usage Example See [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) engine description. + + +## Related Content + +- Blog: [Using Aggregate Combinators in ClickHouse](https://clickhouse.com/blog/aggregate-functions-combinators-in-clickhouse-for-arrays-maps-and-states) diff --git a/docs/en/sql-reference/data-types/map.md b/docs/en/sql-reference/data-types/map.md index b0659746ba7..ad99bc75f24 100644 --- a/docs/en/sql-reference/data-types/map.md +++ b/docs/en/sql-reference/data-types/map.md @@ -108,3 +108,8 @@ Result: - [map()](../../sql-reference/functions/tuple-map-functions.md#function-map) function - [CAST()](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) function + + +## Related content + +- Blog: [Building an Observability Solution with ClickHouse - Part 2 - Traces](https://clickhouse.com/blog/storing-traces-and-spans-open-telemetry-in-clickhouse) diff --git a/docs/en/sql-reference/statements/alter/delete.md b/docs/en/sql-reference/statements/alter/delete.md index 30ed96c0b9c..21ae6a1e5d1 100644 --- a/docs/en/sql-reference/statements/alter/delete.md +++ b/docs/en/sql-reference/statements/alter/delete.md @@ -28,3 +28,7 @@ The synchronicity of the query processing is defined by the [mutations_sync](/do - [Mutations](/docs/en/sql-reference/statements/alter/index.md#mutations) - [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) - [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting + +## Related content + +- Blog: [Handling Updates and Deletes in ClickHouse](https://clickhouse.com/blog/handling-updates-and-deletes-in-clickhouse) diff --git a/docs/en/sql-reference/statements/alter/index.md b/docs/en/sql-reference/statements/alter/index.md index 5d7b92bd34d..cbec9572bb1 100644 --- a/docs/en/sql-reference/statements/alter/index.md +++ b/docs/en/sql-reference/statements/alter/index.md @@ -61,3 +61,7 @@ For all `ALTER` queries, if `alter_sync = 2` and some replicas are not active fo ::: For `ALTER TABLE ... UPDATE|DELETE` queries the synchronicity is defined by the [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting. + +## Related content + +- Blog: [Handling Updates and Deletes in ClickHouse](https://clickhouse.com/blog/handling-updates-and-deletes-in-clickhouse) diff --git a/docs/en/sql-reference/statements/alter/update.md b/docs/en/sql-reference/statements/alter/update.md index 92f0f111b92..c0036c060eb 100644 --- a/docs/en/sql-reference/statements/alter/update.md +++ b/docs/en/sql-reference/statements/alter/update.md @@ -27,3 +27,8 @@ The synchronicity of the query processing is defined by the [mutations_sync](/do - [Mutations](/docs/en/sql-reference/statements/alter/index.md#mutations) - [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) - [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting + + +## Related content + +- Blog: [Handling Updates and Deletes in ClickHouse](https://clickhouse.com/blog/handling-updates-and-deletes-in-clickhouse) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 0def42259ab..d7b16a88f6b 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -364,3 +364,4 @@ The window view is useful in the following scenarios: ## Related Content - Blog: [Working with time series data in ClickHouse](https://clickhouse.com/blog/working-with-time-series-data-and-functions-ClickHouse) +- Blog: [Building an Observability Solution with ClickHouse - Part 2 - Traces](https://clickhouse.com/blog/storing-traces-and-spans-open-telemetry-in-clickhouse) diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index 149e7ab371f..d18f94031ae 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -55,3 +55,7 @@ With the described implementation now we can see what can negatively affect 'DEL - Table having a very large number of data parts - Having a lot of data in Compact parts—in a Compact part, all columns are stored in one file. + +## Related content + +- Blog: [Handling Updates and Deletes in ClickHouse](https://clickhouse.com/blog/handling-updates-and-deletes-in-clickhouse) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index ece60961aaf..62947dcefdd 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -18,6 +18,10 @@ FROM Expressions from `ON` clause and columns from `USING` clause are called “join keysâ€. Unless otherwise stated, join produces a [Cartesian product](https://en.wikipedia.org/wiki/Cartesian_product) from rows with matching “join keysâ€, which might produce results with much more rows than the source tables. +## Related Content + +- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Part 1](https://clickhouse.com/blog/clickhouse-fully-supports-joins) + ## Supported Types of JOIN All standard [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) types are supported: diff --git a/docs/en/sql-reference/table-functions/postgresql.md b/docs/en/sql-reference/table-functions/postgresql.md index 6cd13acaa77..975a04fa0de 100644 --- a/docs/en/sql-reference/table-functions/postgresql.md +++ b/docs/en/sql-reference/table-functions/postgresql.md @@ -133,4 +133,6 @@ CREATE TABLE pg_table_schema_with_dots (a UInt32) - [Using PostgreSQL as a dictionary source](../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-postgresql) ## Related content + - Blog: [ClickHouse and PostgreSQL - a match made in data heaven - part 1](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres) +- Blog: [ClickHouse and PostgreSQL - a Match Made in Data Heaven - part 2](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres-part-2) From 79b83c4fd2249daaaa606ab907bd72dc0a2afde8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 8 Apr 2023 06:47:21 +0200 Subject: [PATCH 254/277] Remove superfluous includes of logger_userful.h from headers Signed-off-by: Azat Khuzhin --- .../AggregateFunctionMaxIntersections.h | 1 - .../AggregateFunctionSparkbar.h | 1 - .../AggregateFunctionSumMap.h | 1 - src/BridgeHelper/IBridgeHelper.h | 1 - src/BridgeHelper/XDBCBridgeHelper.h | 1 - src/Client/Connection.cpp | 3 ++- src/Client/Connection.h | 1 - src/Client/LocalConnection.cpp | 1 + src/Client/LocalConnection.h | 1 + src/Common/AsynchronousMetrics.cpp | 1 + src/Common/CacheBase.h | 1 - src/Common/Config/ConfigProcessor.cpp | 1 + src/Common/Config/ConfigProcessor.h | 3 ++- src/Common/Config/YAMLParser.h | 8 +++---- src/Common/Config/configReadClient.cpp | 1 + src/Common/DNSResolver.cpp | 1 + src/Common/DNSResolver.h | 3 ++- src/Common/ErrorHandlers.h | 1 - src/Common/FileChecker.cpp | 6 ++++- src/Common/FileChecker.h | 6 +++-- src/Common/LRUCachePolicy.h | 6 ++--- src/Common/OvercommitTracker.h | 2 +- src/Common/PoolBase.h | 11 ++++++--- src/Common/SLRUCachePolicy.h | 6 ++--- src/Common/ThreadProfileEvents.cpp | 1 + src/Common/ThreadProfileEvents.h | 4 +++- src/Common/ThreadStatus.cpp | 2 ++ src/Common/ZooKeeper/Common.h | 2 -- src/Common/ZooKeeper/ZooKeeper.cpp | 1 + src/Common/ZooKeeper/ZooKeeper.h | 1 - src/Common/ZooKeeper/ZooKeeperLock.cpp | 2 ++ src/Common/ZooKeeper/ZooKeeperLock.h | 3 ++- .../ZooKeeper/ZooKeeperWithFaultInjection.h | 1 + src/Common/scope_guard_safe.h | 1 - src/Coordination/KeeperDispatcher.cpp | 1 + src/Coordination/KeeperDispatcher.h | 1 - src/Coordination/KeeperLogStore.cpp | 1 + src/Coordination/KeeperLogStore.h | 1 - src/Coordination/KeeperSnapshotManagerS3.h | 1 - src/Coordination/KeeperStateMachine.cpp | 3 ++- src/Coordination/KeeperStateMachine.h | 1 - src/Coordination/KeeperStateManager.cpp | 1 + src/Coordination/ZooKeeperDataReader.cpp | 1 + src/Coordination/ZooKeeperDataReader.h | 1 - src/Core/PostgreSQL/PoolWithFailover.cpp | 1 + src/Core/PostgreSQL/PoolWithFailover.h | 1 - src/Core/SortDescription.cpp | 1 + src/Daemon/BaseDaemon.h | 1 - src/Dictionaries/CacheDictionary.h | 1 - src/Dictionaries/ExecutableDictionarySource.h | 1 - .../ExecutablePoolDictionarySource.h | 1 - src/Dictionaries/IPAddressDictionary.cpp | 1 + src/Dictionaries/IPAddressDictionary.h | 1 - .../PostgreSQLDictionarySource.cpp | 1 + src/Dictionaries/PostgreSQLDictionarySource.h | 1 - src/Disks/DiskLocal.h | 1 - .../IO/CachedOnDiskReadBufferFromFile.cpp | 3 ++- src/Disks/IO/CachedOnDiskReadBufferFromFile.h | 1 - src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 1 - src/Disks/StoragePolicy.cpp | 1 + src/Disks/StoragePolicy.h | 1 - src/Functions/FunctionsDecimalArithmetics.h | 1 - src/Functions/array/arrayEnumerateExtended.h | 1 + src/Functions/array/arrayEnumerateRanked.h | 1 + src/Functions/geometryConverters.h | 1 - src/IO/S3/AWSLogger.cpp | 3 +++ src/IO/S3/AWSLogger.h | 4 +++- src/IO/S3/Client.h | 1 - src/IO/S3/Credentials.h | 1 - src/IO/WriteBufferFromS3.h | 1 - src/Interpreters/Aggregator.cpp | 23 ++++++++++--------- src/Interpreters/Aggregator.h | 1 - src/Interpreters/Cache/FileCache.h | 1 - .../Cache/LRUFileCachePriority.cpp | 1 + src/Interpreters/Cache/LRUFileCachePriority.h | 1 - src/Interpreters/DNSCacheUpdater.cpp | 1 + src/Interpreters/DirectJoin.cpp | 4 ++-- src/Interpreters/DirectJoin.h | 1 - src/Interpreters/ExternalLoader.cpp | 3 ++- src/Interpreters/ExternalLoader.h | 3 ++- src/Interpreters/FilesystemCacheLog.h | 1 - src/Interpreters/FullSortingMergeJoin.h | 1 + src/Interpreters/HashJoin.h | 1 - src/Interpreters/IJoin.h | 1 - src/Interpreters/InterserverCredentials.h | 1 - src/Interpreters/MergeJoin.cpp | 1 + src/Interpreters/Set.h | 1 - src/Interpreters/TableJoin.h | 1 - src/Interpreters/ThreadStatusExt.cpp | 1 + src/Planner/Planner.cpp | 1 + .../Executors/CompletedPipelineExecutor.cpp | 2 +- src/Processors/Executors/PipelineExecutor.cpp | 1 + src/Processors/Executors/PipelineExecutor.h | 1 - .../PushingAsyncPipelineExecutor.cpp | 3 ++- src/Processors/PingPongProcessor.h | 1 - .../CreateSetAndFilterOnTheFlyStep.cpp | 1 + src/Processors/Sources/ShellCommandSource.h | 1 - .../AggregatingInOrderTransform.cpp | 1 + .../Transforms/AggregatingTransform.cpp | 1 + .../Transforms/AggregatingTransform.h | 1 + .../Transforms/CreatingSetsTransform.cpp | 1 + .../Transforms/MemoryBoundMerging.h | 1 - .../Transforms/MergeJoinTransform.cpp | 10 ++++++++ .../Transforms/MergeJoinTransform.h | 10 +------- .../Transforms/MergeSortingTransform.cpp | 1 + .../Transforms/MergeSortingTransform.h | 1 - .../Transforms/MergingAggregatedTransform.cpp | 1 + src/Server/HTTPHandlerFactory.h | 1 - src/Server/PostgreSQLHandler.h | 1 - src/Server/ProxyV1Handler.cpp | 1 + src/Server/ProxyV1Handler.h | 1 - src/Storages/Cache/RemoteCacheController.h | 1 - .../FileLog/FileLogDirectoryWatcher.cpp | 1 + .../FileLog/FileLogDirectoryWatcher.h | 1 - src/Storages/HDFS/StorageHDFS.h | 1 - src/Storages/Hive/StorageHive.h | 1 - src/Storages/KVStorageUtils.h | 1 - src/Storages/MergeTree/AsyncBlockIDsCache.h | 1 - src/Storages/MergeTree/DataPartsExchange.cpp | 1 + src/Storages/MergeTree/DataPartsExchange.h | 4 ++-- src/Storages/MergeTree/IDataPartStorage.h | 1 - .../MergeTree/IMergedBlockOutputStream.cpp | 1 + .../MergeTree/MergeTreeBackgroundExecutor.cpp | 3 ++- .../MergeTree/MergeTreeBackgroundExecutor.h | 3 +-- .../MergeTreeBaseSelectProcessor.cpp | 1 + .../MergeTreeInOrderSelectProcessor.h | 1 + .../MergeTree/MergeTreePartsMover.cpp | 1 + .../MergeTree/MergeTreePrefetchedReadPool.cpp | 1 + .../MergeTree/MergeTreeRangeReader.cpp | 1 + src/Storages/MergeTree/MergeTreeRangeReader.h | 1 - src/Storages/MergeTree/MergeTreeReadPool.cpp | 3 ++- .../MergeTreeReverseSelectProcessor.h | 1 + .../MergeTree/MergeTreeSequentialSource.cpp | 1 + .../MergeTree/MergeTreeWriteAheadLog.cpp | 3 ++- .../MergeTree/MergedBlockOutputStream.cpp | 1 + .../PartMovesBetweenShardsOrchestrator.h | 1 - src/Storages/MergeTree/PartitionPruner.cpp | 1 + .../MergeTree/ReplicatedMergeMutateTaskBase.h | 1 - .../ReplicatedMergeTreeAttachThread.h | 1 - .../ReplicatedMergeTreeCleanupThread.h | 1 - .../ReplicatedMergeTreePartCheckThread.h | 1 - .../ReplicatedMergeTreeRestartingThread.h | 1 - .../MaterializedPostgreSQLConsumer.h | 1 - src/Storages/StorageDistributed.h | 1 - src/Storages/StorageExecutable.h | 1 - src/Storages/StorageFile.h | 1 - src/Storages/StorageKeeperMap.h | 1 + src/Storages/StorageS3.h | 1 - src/Storages/StorageSet.cpp | 1 + src/Storages/System/StorageSystemReplicas.cpp | 1 + 150 files changed, 142 insertions(+), 130 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index 2c54293eeec..5074e491f60 100644 --- a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionSparkbar.h b/src/AggregateFunctions/AggregateFunctionSparkbar.h index 78f7e9fcefa..30e107bc4db 100644 --- a/src/AggregateFunctions/AggregateFunctionSparkbar.h +++ b/src/AggregateFunctions/AggregateFunctionSparkbar.h @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index f51ec423c69..b30f5ff5220 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -18,7 +18,6 @@ #include #include #include -#include #include diff --git a/src/BridgeHelper/IBridgeHelper.h b/src/BridgeHelper/IBridgeHelper.h index d4762087cc1..272d97c8a78 100644 --- a/src/BridgeHelper/IBridgeHelper.h +++ b/src/BridgeHelper/IBridgeHelper.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB diff --git a/src/BridgeHelper/XDBCBridgeHelper.h b/src/BridgeHelper/XDBCBridgeHelper.h index 00a661a1fc4..44104f26f63 100644 --- a/src/BridgeHelper/XDBCBridgeHelper.h +++ b/src/BridgeHelper/XDBCBridgeHelper.h @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 6643a94c3bc..11bba4f1448 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -22,7 +22,8 @@ #include #include #include -#include "Core/Block.h" +#include +#include #include #include #include diff --git a/src/Client/Connection.h b/src/Client/Connection.h index b86567e2ed0..5f79b365199 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -1,6 +1,5 @@ #pragma once -#include #include diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 712ff5f5a31..4fb80f0ea04 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index 3e6fc007fb9..fb8f9003364 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 99073d79bcd..32d54b7644e 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index 4ae313d7ecf..b176170cc1f 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -12,7 +12,6 @@ #include #include -#include #include diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index b632ea95928..5bbc8eae0de 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index aa8ac71446f..0ca3e46db88 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -16,9 +16,10 @@ #include #include #include -#include +namespace Poco { class Logger; } + namespace zkutil { class ZooKeeperNodeCache; diff --git a/src/Common/Config/YAMLParser.h b/src/Common/Config/YAMLParser.h index b986fc2d895..a00972b813c 100644 --- a/src/Common/Config/YAMLParser.h +++ b/src/Common/Config/YAMLParser.h @@ -2,11 +2,11 @@ #include "config.h" -#include - +#include +#include +#include #include -#include "Poco/DOM/AutoPtr.h" -#include +#include #if USE_YAML_CPP diff --git a/src/Common/Config/configReadClient.cpp b/src/Common/Config/configReadClient.cpp index e5308bc3bc7..44d338c07af 100644 --- a/src/Common/Config/configReadClient.cpp +++ b/src/Common/Config/configReadClient.cpp @@ -4,6 +4,7 @@ #include "ConfigProcessor.h" #include #include +#include namespace fs = std::filesystem; diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 81e2624d6db..b6a68bdfb45 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index a05456d3de8..84715b392a8 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -5,9 +5,10 @@ #include #include #include -#include +namespace Poco { class Logger; } + namespace DB { diff --git a/src/Common/ErrorHandlers.h b/src/Common/ErrorHandlers.h index f55b6c83a69..301377bff83 100644 --- a/src/Common/ErrorHandlers.h +++ b/src/Common/ErrorHandlers.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index aa6b9c90a4c..a6e37654ff1 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -1,5 +1,7 @@ #include #include +#include +#include #include #include #include @@ -25,7 +27,9 @@ FileChecker::FileChecker(const String & file_info_path_) : FileChecker(nullptr, { } -FileChecker::FileChecker(DiskPtr disk_, const String & file_info_path_) : disk(std::move(disk_)) +FileChecker::FileChecker(DiskPtr disk_, const String & file_info_path_) + : disk(std::move(disk_)) + , log(&Poco::Logger::get("FileChecker")) { setPath(file_info_path_); try diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index 1beab31ec8f..bb0383e4b56 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -1,8 +1,10 @@ #pragma once -#include #include +#include +#include +namespace Poco { class Logger; } namespace DB { @@ -46,7 +48,7 @@ private: size_t getRealFileSize(const String & path_) const; const DiskPtr disk; - const Poco::Logger * log = &Poco::Logger::get("FileChecker"); + const Poco::Logger * log; String files_info_path; std::map map; diff --git a/src/Common/LRUCachePolicy.h b/src/Common/LRUCachePolicy.h index 4aee2135af7..49c2fc4541b 100644 --- a/src/Common/LRUCachePolicy.h +++ b/src/Common/LRUCachePolicy.h @@ -5,8 +5,6 @@ #include #include -#include - namespace DB { /// Cache policy LRU evicts entries which are not used for a long time. @@ -174,7 +172,7 @@ private: auto it = cells.find(key); if (it == cells.end()) { - LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); + // Queue became inconsistent abort(); } @@ -192,7 +190,7 @@ private: if (current_size_in_bytes > (1ull << 63)) { - LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); + // Queue became inconsistent abort(); } } diff --git a/src/Common/OvercommitTracker.h b/src/Common/OvercommitTracker.h index 598b877ef3c..f40a70fe7cd 100644 --- a/src/Common/OvercommitTracker.h +++ b/src/Common/OvercommitTracker.h @@ -1,7 +1,7 @@ #pragma once -#include #include +#include #include #include #include diff --git a/src/Common/PoolBase.h b/src/Common/PoolBase.h index 96a18ee6591..a9c595c440c 100644 --- a/src/Common/PoolBase.h +++ b/src/Common/PoolBase.h @@ -144,12 +144,17 @@ public: return Entry(*items.back()); } - LOG_INFO(log, "No free connections in pool. Waiting."); - if (timeout < 0) + { + LOG_INFO(log, "No free connections in pool. Waiting undefinitelly."); available.wait(lock); + } else - available.wait_for(lock, std::chrono::microseconds(timeout)); + { + auto timeout_ms = std::chrono::microseconds(timeout); + LOG_INFO(log, "No free connections in pool. Waiting {} ms.", timeout_ms.count()); + available.wait_for(lock, timeout_ms); + } } } diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index e36bca83c61..810c8c335a0 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -5,8 +5,6 @@ #include #include -#include - namespace DB { @@ -236,7 +234,7 @@ private: auto it = cells.find(key); if (it == cells.end()) { - LOG_ERROR(&Poco::Logger::get("SLRUCache"), "SLRUCache became inconsistent. There must be a bug in it."); + // Queue became inconsistent abort(); } @@ -264,7 +262,7 @@ private: if (current_size_in_bytes > (1ull << 63)) { - LOG_ERROR(&Poco::Logger::get("SLRUCache"), "SLRUCache became inconsistent. There must be a bug in it."); + // Queue became inconsistent abort(); } } diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index 76a4d8b1adf..a94fd81559a 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -23,6 +23,7 @@ #include #include +#include namespace ProfileEvents diff --git a/src/Common/ThreadProfileEvents.h b/src/Common/ThreadProfileEvents.h index 5a03a9e8555..c4703a69998 100644 --- a/src/Common/ThreadProfileEvents.h +++ b/src/Common/ThreadProfileEvents.h @@ -2,11 +2,13 @@ #include #include +#include +#include #include #include #include #include -#include +#include #if defined(OS_LINUX) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 1b783aa9ec4..ac6cfdc2cd7 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include #include #include diff --git a/src/Common/ZooKeeper/Common.h b/src/Common/ZooKeeper/Common.h index a2956706c8f..1a1328588e3 100644 --- a/src/Common/ZooKeeper/Common.h +++ b/src/Common/ZooKeeper/Common.h @@ -3,12 +3,10 @@ #include #include -#include namespace zkutil { using GetZooKeeper = std::function; -using GetZooKeeperWithFaultInjection = std::function; } diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 78f9e3da83b..a8da0dff0cc 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -15,6 +15,7 @@ #include "Common/ZooKeeper/IKeeper.h" #include #include +#include #include #include diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index fc4fe95d9b6..8e7639b8cc1 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Common/ZooKeeper/ZooKeeperLock.cpp b/src/Common/ZooKeeper/ZooKeeperLock.cpp index a52c942a35f..6ee1c380efb 100644 --- a/src/Common/ZooKeeper/ZooKeeperLock.cpp +++ b/src/Common/ZooKeeper/ZooKeeperLock.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include namespace DB diff --git a/src/Common/ZooKeeper/ZooKeeperLock.h b/src/Common/ZooKeeper/ZooKeeperLock.h index 755ca1333b8..146527c6c94 100644 --- a/src/Common/ZooKeeper/ZooKeeperLock.h +++ b/src/Common/ZooKeeper/ZooKeeperLock.h @@ -3,7 +3,8 @@ #include #include #include -#include + +namespace Poco { class Logger; } namespace zkutil { diff --git a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h index a39a083cf33..130590ceb40 100644 --- a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h +++ b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB diff --git a/src/Common/scope_guard_safe.h b/src/Common/scope_guard_safe.h index f098fd95f00..2befb58870a 100644 --- a/src/Common/scope_guard_safe.h +++ b/src/Common/scope_guard_safe.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include /// Same as SCOPE_EXIT() but block the MEMORY_LIMIT_EXCEEDED errors. diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 1828182751d..2cde7c2465e 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 77b5510cbb3..4b8b134cf8f 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Coordination/KeeperLogStore.cpp b/src/Coordination/KeeperLogStore.cpp index d1bd2f9db18..10619a44517 100644 --- a/src/Coordination/KeeperLogStore.cpp +++ b/src/Coordination/KeeperLogStore.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index 108241e024e..e48e3a32463 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -4,7 +4,6 @@ #include #include #include -#include #include namespace DB diff --git a/src/Coordination/KeeperSnapshotManagerS3.h b/src/Coordination/KeeperSnapshotManagerS3.h index 197f528b192..eff7868bba9 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.h +++ b/src/Coordination/KeeperSnapshotManagerS3.h @@ -9,7 +9,6 @@ #if USE_AWS_S3 #include #include -#include #include #endif diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 632aaec6b54..7c55739a96f 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -8,9 +8,10 @@ #include #include #include -#include "Common/ZooKeeper/ZooKeeperCommon.h" +#include #include #include +#include #include "Coordination/KeeperStorage.h" diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index e4f0295db99..5af5bc05b0f 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -8,7 +8,6 @@ #include #include -#include namespace DB diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index cfb3519e597..70687ba471c 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 5fa67a60b4b..2d4f64e033f 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include diff --git a/src/Coordination/ZooKeeperDataReader.h b/src/Coordination/ZooKeeperDataReader.h index 6da6fd498af..8fd86ba99e2 100644 --- a/src/Coordination/ZooKeeperDataReader.h +++ b/src/Coordination/ZooKeeperDataReader.h @@ -1,7 +1,6 @@ #pragma once #include #include -#include namespace DB { diff --git a/src/Core/PostgreSQL/PoolWithFailover.cpp b/src/Core/PostgreSQL/PoolWithFailover.cpp index 22cd88c0764..3655681c515 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.cpp +++ b/src/Core/PostgreSQL/PoolWithFailover.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include diff --git a/src/Core/PostgreSQL/PoolWithFailover.h b/src/Core/PostgreSQL/PoolWithFailover.h index afef2933d29..bf9c34e6723 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.h +++ b/src/Core/PostgreSQL/PoolWithFailover.h @@ -8,7 +8,6 @@ #include "ConnectionHolder.h" #include #include -#include #include #include diff --git a/src/Core/SortDescription.cpp b/src/Core/SortDescription.cpp index 66ca1539b71..ae6aedf202d 100644 --- a/src/Core/SortDescription.cpp +++ b/src/Core/SortDescription.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #if USE_EMBEDDED_COMPILER #include diff --git a/src/Daemon/BaseDaemon.h b/src/Daemon/BaseDaemon.h index f90f403364b..7aa1e8ad1a0 100644 --- a/src/Daemon/BaseDaemon.h +++ b/src/Daemon/BaseDaemon.h @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index e19c4a66b1f..5203415005f 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -9,7 +9,6 @@ #include -#include #include #include diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index 0456d3cafef..c7067a62893 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.h b/src/Dictionaries/ExecutablePoolDictionarySource.h index 1fc10d18b76..e8cc6e83406 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.h +++ b/src/Dictionaries/ExecutablePoolDictionarySource.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 803f607a3a7..ff1c784750b 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index e1fabb89a7e..67827c6524e 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -11,7 +11,6 @@ #include #include #include -#include #include "DictionaryStructure.h" #include "IDictionary.h" #include "IDictionarySource.h" diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 9f254da0b11..8ec78308392 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #endif diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index 8ecf56a9430..1305333458b 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -8,7 +8,6 @@ #include "ExternalQueryBuilder.h" #include #include -#include #include diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 14e29904422..d6182463ebf 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 5eaee2e3026..f48935da7bf 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -4,8 +4,9 @@ #include #include #include -#include #include +#include +#include #include diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index 14e8ea6c7e7..d3c265a522b 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index bcdc97983be..b49f4dafef0 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -9,7 +9,6 @@ #include #include #include -#include namespace DB diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index ec0f201b801..f4be8b8fe86 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index 7e6aff7bbda..69cfb830818 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -10,7 +10,6 @@ #include #include #include -#include #include #include diff --git a/src/Functions/FunctionsDecimalArithmetics.h b/src/Functions/FunctionsDecimalArithmetics.h index aa4afc68707..79e10d215a9 100644 --- a/src/Functions/FunctionsDecimalArithmetics.h +++ b/src/Functions/FunctionsDecimalArithmetics.h @@ -10,7 +10,6 @@ #include #include -#include #include #include diff --git a/src/Functions/array/arrayEnumerateExtended.h b/src/Functions/array/arrayEnumerateExtended.h index 3f145c05b54..cf38afcfa5a 100644 --- a/src/Functions/array/arrayEnumerateExtended.h +++ b/src/Functions/array/arrayEnumerateExtended.h @@ -8,6 +8,7 @@ #include #include #include +#include #include #include diff --git a/src/Functions/array/arrayEnumerateRanked.h b/src/Functions/array/arrayEnumerateRanked.h index 8a348c07421..0733f1e2d43 100644 --- a/src/Functions/array/arrayEnumerateRanked.h +++ b/src/Functions/array/arrayEnumerateRanked.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include diff --git a/src/Functions/geometryConverters.h b/src/Functions/geometryConverters.h index 85e887dd498..97162fa9dd0 100644 --- a/src/Functions/geometryConverters.h +++ b/src/Functions/geometryConverters.h @@ -18,7 +18,6 @@ #include #include -#include namespace DB { diff --git a/src/IO/S3/AWSLogger.cpp b/src/IO/S3/AWSLogger.cpp index 48c30ccf881..d6162823aee 100644 --- a/src/IO/S3/AWSLogger.cpp +++ b/src/IO/S3/AWSLogger.cpp @@ -2,7 +2,10 @@ #if USE_AWS_S3 +#include +#include #include +#include namespace { diff --git a/src/IO/S3/AWSLogger.h b/src/IO/S3/AWSLogger.h index 897c0e8d964..fdb6eed1f86 100644 --- a/src/IO/S3/AWSLogger.h +++ b/src/IO/S3/AWSLogger.h @@ -4,8 +4,10 @@ #if USE_AWS_S3 #include +#include +#include -#include +namespace Poco { class Logger; } namespace DB::S3 { diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 5c68fca6f04..369558ae105 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -4,7 +4,6 @@ #if USE_AWS_S3 -#include #include #include diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index cd9072f9765..22cca76109f 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -9,7 +9,6 @@ # include # include -# include # include diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 7c3c439f9c3..5fa934b886e 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -9,7 +9,6 @@ #include #include -#include #include #include #include diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index d6fbf072d05..03227d8663a 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -8,9 +8,6 @@ #endif #include -#include -#include -#include #include #include #include @@ -21,14 +18,6 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include @@ -37,6 +26,18 @@ #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index e2f76f31466..5fb94c5f4e8 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -5,7 +5,6 @@ #include #include -#include #include #include diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index afafa39c4c6..85c91e8033e 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -12,7 +12,6 @@ #include #include -#include #include #include #include diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index c20379e1fc1..4987d143f52 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace CurrentMetrics { diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 2345d3c47db..2a3c4f3c801 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -2,7 +2,6 @@ #include #include -#include namespace DB { diff --git a/src/Interpreters/DNSCacheUpdater.cpp b/src/Interpreters/DNSCacheUpdater.cpp index bf88d19b7ef..c6a38d85acb 100644 --- a/src/Interpreters/DNSCacheUpdater.cpp +++ b/src/Interpreters/DNSCacheUpdater.cpp @@ -2,6 +2,7 @@ #include #include +#include namespace DB diff --git a/src/Interpreters/DirectJoin.cpp b/src/Interpreters/DirectJoin.cpp index e148db1d8e6..cfefd7c5a91 100644 --- a/src/Interpreters/DirectJoin.cpp +++ b/src/Interpreters/DirectJoin.cpp @@ -1,7 +1,7 @@ #include - -#include #include +#include +#include namespace DB { diff --git a/src/Interpreters/DirectJoin.h b/src/Interpreters/DirectJoin.h index bdbd155dc36..644b66a9d99 100644 --- a/src/Interpreters/DirectJoin.h +++ b/src/Interpreters/DirectJoin.h @@ -1,6 +1,5 @@ #pragma once -#include #include diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 04a116ec0c7..05418003dba 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -9,8 +9,9 @@ #include #include #include -#include #include +#include +#include #include #include #include diff --git a/src/Interpreters/ExternalLoader.h b/src/Interpreters/ExternalLoader.h index 3ce7c40ef03..49b4ea77e0d 100644 --- a/src/Interpreters/ExternalLoader.h +++ b/src/Interpreters/ExternalLoader.h @@ -6,10 +6,11 @@ #include #include #include -#include #include #include +#include +namespace Poco { class Logger; } namespace DB { diff --git a/src/Interpreters/FilesystemCacheLog.h b/src/Interpreters/FilesystemCacheLog.h index bf5361ef324..1b22d561c51 100644 --- a/src/Interpreters/FilesystemCacheLog.h +++ b/src/Interpreters/FilesystemCacheLog.h @@ -8,7 +8,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index a94d7a7dfc6..7318d1d24a1 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index b29b6e617c8..79fac60d8bc 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -16,7 +16,6 @@ #include #include #include -#include #include #include diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 69d69ce30a6..83067b0eab7 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Interpreters/InterserverCredentials.h b/src/Interpreters/InterserverCredentials.h index c4370c83e9b..33176c943a4 100644 --- a/src/Interpreters/InterserverCredentials.h +++ b/src/Interpreters/InterserverCredentials.h @@ -2,7 +2,6 @@ #include #include -#include #include namespace DB diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 07f5ae31ed6..d31510c2fb5 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 00eff614c7c..e593f71cfbe 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -8,7 +8,6 @@ #include #include -#include namespace DB diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 95471885a2a..99b683b7713 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -19,7 +19,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 72ce08bf653..2e051030e38 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #if defined(OS_LINUX) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index a8e0d80ce8c..08efa6653a7 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1,6 +1,7 @@ #include #include +#include #include diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index e624ecd52de..acbb4098145 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -5,8 +5,8 @@ #include #include #include -#include #include +#include namespace DB { diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 994051fd697..441a66b16d7 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include diff --git a/src/Processors/Executors/PipelineExecutor.h b/src/Processors/Executors/PipelineExecutor.h index 147e5b2744a..e6d33e2849b 100644 --- a/src/Processors/Executors/PipelineExecutor.h +++ b/src/Processors/Executors/PipelineExecutor.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 3aec7608e6d..e026dbc50d6 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -5,8 +5,9 @@ #include #include #include -#include #include +#include +#include namespace DB { diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h index 0f2fffbd70f..976b29f0d8a 100644 --- a/src/Processors/PingPongProcessor.h +++ b/src/Processors/PingPongProcessor.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index 07137e87736..fe362f64b96 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Processors/Sources/ShellCommandSource.h b/src/Processors/Sources/ShellCommandSource.h index 80ba1d59adf..b9afaa345cf 100644 --- a/src/Processors/Sources/ShellCommandSource.h +++ b/src/Processors/Sources/ShellCommandSource.h @@ -2,7 +2,6 @@ #include -#include #include #include diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 95e9b953840..3d5a910f054 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 69eeb76bf53..4bd000797a6 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 048b69adae6..beb23a91c88 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace CurrentMetrics { diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 6a8d08cb661..f65e72c2723 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -6,6 +6,7 @@ #include #include +#include #include diff --git a/src/Processors/Transforms/MemoryBoundMerging.h b/src/Processors/Transforms/MemoryBoundMerging.h index 3193a07a0bd..607087fb39c 100644 --- a/src/Processors/Transforms/MemoryBoundMerging.h +++ b/src/Processors/Transforms/MemoryBoundMerging.h @@ -6,7 +6,6 @@ #include #include -#include namespace DB { diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index abeef0f9a47..46abd5c891d 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -302,6 +302,16 @@ MergeJoinAlgorithm::MergeJoinAlgorithm( } } +void MergeJoinAlgorithm::logElapsed(double seconds) +{ + LOG_TRACE(log, + "Finished pocessing in {} seconds" + ", left: {} blocks, {} rows; right: {} blocks, {} rows" + ", max blocks loaded to memory: {}", + seconds, stat.num_blocks[0], stat.num_rows[0], stat.num_blocks[1], stat.num_rows[1], + stat.max_blocks_loaded); +} + static void prepareChunk(Chunk & chunk) { if (!chunk) diff --git a/src/Processors/Transforms/MergeJoinTransform.h b/src/Processors/Transforms/MergeJoinTransform.h index 58ac652cb18..6bf4484df24 100644 --- a/src/Processors/Transforms/MergeJoinTransform.h +++ b/src/Processors/Transforms/MergeJoinTransform.h @@ -233,15 +233,7 @@ public: virtual void consume(Input & input, size_t source_num) override; virtual Status merge() override; - void logElapsed(double seconds) - { - LOG_TRACE(log, - "Finished pocessing in {} seconds" - ", left: {} blocks, {} rows; right: {} blocks, {} rows" - ", max blocks loaded to memory: {}", - seconds, stat.num_blocks[0], stat.num_rows[0], stat.num_blocks[1], stat.num_rows[1], - stat.max_blocks_loaded); - } + void logElapsed(double seconds); private: std::optional handleAnyJoinState(); diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index eebdd678a4b..ecf14a81c00 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Processors/Transforms/MergeSortingTransform.h b/src/Processors/Transforms/MergeSortingTransform.h index c64c93393ce..e8c180b6903 100644 --- a/src/Processors/Transforms/MergeSortingTransform.h +++ b/src/Processors/Transforms/MergeSortingTransform.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index 9d0be86ff83..e4955d06859 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index ebdfa954bf7..76460dcc4a8 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -6,7 +6,6 @@ #include #include #include -#include #include diff --git a/src/Server/PostgreSQLHandler.h b/src/Server/PostgreSQLHandler.h index 6fc128e3883..f20af3df02c 100644 --- a/src/Server/PostgreSQLHandler.h +++ b/src/Server/PostgreSQLHandler.h @@ -4,7 +4,6 @@ #include "config.h" #include #include -#include #include "IServer.h" #if USE_SSL diff --git a/src/Server/ProxyV1Handler.cpp b/src/Server/ProxyV1Handler.cpp index cd5fe29112a..56621940a23 100644 --- a/src/Server/ProxyV1Handler.cpp +++ b/src/Server/ProxyV1Handler.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include diff --git a/src/Server/ProxyV1Handler.h b/src/Server/ProxyV1Handler.h index e56f4cd3545..b50c2acbc55 100644 --- a/src/Server/ProxyV1Handler.h +++ b/src/Server/ProxyV1Handler.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/Cache/RemoteCacheController.h b/src/Storages/Cache/RemoteCacheController.h index 18732acc273..fafe363bbd4 100644 --- a/src/Storages/Cache/RemoteCacheController.h +++ b/src/Storages/Cache/RemoteCacheController.h @@ -9,7 +9,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 3c5342a1e83..9d488616e85 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 4368f284c9d..9b7afcf8e12 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -2,7 +2,6 @@ #include -#include #include #include diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 585f5df6ceb..b123834e981 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -8,7 +8,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 363042621c7..b4afb2421b1 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -7,7 +7,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/KVStorageUtils.h b/src/Storages/KVStorageUtils.h index 0574539f4c7..c3bb2aefa62 100644 --- a/src/Storages/KVStorageUtils.h +++ b/src/Storages/KVStorageUtils.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/Storages/MergeTree/AsyncBlockIDsCache.h b/src/Storages/MergeTree/AsyncBlockIDsCache.h index a661d00f8a6..91d549a0501 100644 --- a/src/Storages/MergeTree/AsyncBlockIDsCache.h +++ b/src/Storages/MergeTree/AsyncBlockIDsCache.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index c6efe9c9589..a80870448f1 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index 20c15039a2d..57fd0d5cff0 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -1,13 +1,12 @@ #pragma once -#include "Storages/MergeTree/MergeTreePartInfo.h" +#include #include #include #include #include #include #include -#include #include @@ -21,6 +20,7 @@ namespace DB { class StorageReplicatedMergeTree; +class PooledReadWriteBufferFromHTTP; namespace DataPartsExchange { diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 4d7212eb17b..2d61f9ee77b 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -11,7 +11,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 37da6014d1b..21bead2864a 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index f2efbb2ba8d..6f1e41d2791 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -1,12 +1,13 @@ #include +#include #include #include #include #include -#include #include +#include namespace CurrentMetrics diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h index 1ed4fc2dabc..9bfea32c7f9 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h @@ -9,12 +9,11 @@ #include #include - #include #include +#include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index a41b697f9e6..24980cdb82c 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.h b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.h index f7c3f294658..da332c64232 100644 --- a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace DB { diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 6f83a82e4e6..e1da57744b3 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 114039f7ff7..e184f84528b 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 5cba3e34169..3f8a90ca175 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 5ffd464cfe2..f180787c869 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -1,6 +1,5 @@ #pragma once #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 54a040724fc..29be06b4e6a 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -1,8 +1,9 @@ #include #include #include -#include "Common/Stopwatch.h" +#include #include +#include #include diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h index fd25748050a..618f2855695 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 4539e0b36c5..ef6013f5c65 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index fabf2acdad3..93e07a38bb1 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -4,15 +4,16 @@ #include #include #include +#include #include #include #include #include +#include #include #include #include #include -#include "Storages/MergeTree/DataPartStorageOnDiskFull.h" #include namespace DB diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index ced43ae25b0..715c218c2db 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h index 24454b897af..af21022953c 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Storages/MergeTree/PartitionPruner.cpp b/src/Storages/MergeTree/PartitionPruner.cpp index 61293888f10..35b2d5db3b5 100644 --- a/src/Storages/MergeTree/PartitionPruner.cpp +++ b/src/Storages/MergeTree/PartitionPruner.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h index fb8f01437ef..a7bf1290274 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h index a491a06d6a5..222b30b519b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h index 35838625bbe..76b9ee4a575 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h index 170b2ca1f60..e7b0e224d9b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index b5314de9dcc..8f5721e440d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 3412e6e422f..02cbedb4dd5 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -5,7 +5,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 3a7fae44708..417fb6447bf 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -8,7 +8,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageExecutable.h b/src/Storages/StorageExecutable.h index 2393920fa3c..37455385675 100644 --- a/src/Storages/StorageExecutable.h +++ b/src/Storages/StorageExecutable.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 03b3aacb67f..8d622fdf3a2 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -3,7 +3,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index f71ff3cc65a..552e6b35fe8 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index f4d915e9c55..982d4e4681e 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 7c5ba497ec9..e11b71d8777 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 240d452fe29..8832d9d4983 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include From 8b0adf58e52d78c2fe0a734e383bd0b4e4d22c34 Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Mon, 10 Apr 2023 10:17:43 -0600 Subject: [PATCH 255/277] Update formats.md Including an Avro example with DESCRIBE url in S3 --- docs/en/interfaces/formats.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index de8357345db..157a852332f 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1877,6 +1877,13 @@ Column names must: Output Avro file compression and sync interval can be configured with [output_format_avro_codec](/docs/en/operations/settings/settings-formats.md/#output_format_avro_codec) and [output_format_avro_sync_interval](/docs/en/operations/settings/settings-formats.md/#output_format_avro_sync_interval) respectively. +### Example Data {#example-data-avro} + +Using the ClickHouse [DESCRIBE](/docs/en/sql-reference/statements/describe-table) function, you can quickly view the inferred format of an Avro file like the following example. This example includes the URL of a publicly accessible Avro file in the ClickHouse S3 public bucket: + +``` DESCRIBE url('https://clickhouse-public-datasets.s3.eu-central-1.amazonaws.com/hits.avro','Avro'); +``` + ## AvroConfluent {#data-format-avro-confluent} AvroConfluent supports decoding single-object Avro messages commonly used with [Kafka](https://kafka.apache.org/) and [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/index.html). From 7cd4009c3d8811cc5f13244edbbf430da14da854 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 10 Apr 2023 20:23:56 +0200 Subject: [PATCH 256/277] Fix flakiness of test_store_cleanup in case of image rebuild The log level will be substituted from "test" to "trace" in case of the tag is not "latest", the assumption behind this I guess is that it should not try to use "test" log level for older versions. But, it could have per-PR image in case of changes in the Dockerfile, so it is better to check for self.with_installed_binary, since actually any parameters except this will use new clickhouse binary anyway. CI: https://s3.amazonaws.com/clickhouse-test-reports/48596/a1272e8536265929255fdf5020836f057859e425/integration_tests__tsan__[1/6].html Signed-off-by: Azat Khuzhin --- tests/integration/helpers/cluster.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index b2aedfce3ca..a5788ce36eb 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4161,9 +4161,10 @@ class ClickHouseInstance: logging.debug("Copy common configuration from helpers") # The file is named with 0_ prefix to be processed before other configuration overloads. if self.copy_common_configs: - need_fix_log_level = self.tag != "latest" write_embedded_config( - "0_common_instance_config.xml", self.config_d_dir, need_fix_log_level + "0_common_instance_config.xml", + self.config_d_dir, + self.with_installed_binary, ) write_embedded_config("0_common_instance_users.xml", users_d_dir) From a6c99e8e843c324153d9ffeba8a67fb58a14ead9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 10 Apr 2023 18:59:50 +0000 Subject: [PATCH 257/277] add IPv4 to supertype calculation --- src/DataTypes/getLeastSupertype.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 3c33289c304..783326c25e9 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -3,6 +3,7 @@ #include #include #include +#include "Core/Types.h" #include @@ -88,7 +89,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types) maximize(max_bits_of_unsigned_integer, 8); else if (type == TypeIndex::UInt16) maximize(max_bits_of_unsigned_integer, 16); - else if (type == TypeIndex::UInt32) + else if (type == TypeIndex::UInt32 || type == TypeIndex::IPv4) maximize(max_bits_of_unsigned_integer, 32); else if (type == TypeIndex::UInt64) maximize(max_bits_of_unsigned_integer, 64); From e6d96c27f249936ae4caa92650788f7ff0febab4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 10 Apr 2023 19:16:02 +0000 Subject: [PATCH 258/277] unnecessary include --- src/DataTypes/getLeastSupertype.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 783326c25e9..9d42d82ce91 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -3,7 +3,6 @@ #include #include #include -#include "Core/Types.h" #include From 23a087945209ecb92f74f1fdc4c7c952f13841e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 21:17:08 +0200 Subject: [PATCH 259/277] Remove strange code --- src/Storages/StorageFile.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 03b3aacb67f..27cf40a0bf1 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -14,8 +14,6 @@ namespace DB class StorageFile final : public IStorage { -friend class partitionedstoragefilesink; - public: struct CommonArguments : public WithContext { From 32174e9a6143e404643273d63e959357f998bd57 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 10 Apr 2023 22:47:59 +0000 Subject: [PATCH 260/277] test is added --- .../0_stateless/02713_ip4_uint_compare.reference | 1 + tests/queries/0_stateless/02713_ip4_uint_compare.sql | 10 ++++++++++ 2 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/02713_ip4_uint_compare.reference create mode 100644 tests/queries/0_stateless/02713_ip4_uint_compare.sql diff --git a/tests/queries/0_stateless/02713_ip4_uint_compare.reference b/tests/queries/0_stateless/02713_ip4_uint_compare.reference new file mode 100644 index 00000000000..fdc2de3fbcb --- /dev/null +++ b/tests/queries/0_stateless/02713_ip4_uint_compare.reference @@ -0,0 +1 @@ +1 0 1 1 1 1 0 diff --git a/tests/queries/0_stateless/02713_ip4_uint_compare.sql b/tests/queries/0_stateless/02713_ip4_uint_compare.sql new file mode 100644 index 00000000000..141fa1371b6 --- /dev/null +++ b/tests/queries/0_stateless/02713_ip4_uint_compare.sql @@ -0,0 +1,10 @@ +WITH toIPv4('127.0.0.10') AS ip +SELECT + ip = 2130706442::UInt32, + ip = 0::UInt32, + ip < 2130706443::UInt32, + ip > 2130706441::UInt32, + ip <= 2130706442::UInt32, + ip >= 2130706442::UInt32, + ip != 2130706442::UInt32; + \ No newline at end of file From 7581982e0a6af04ad953bb68ae151fefdfc1593f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 10 Apr 2023 22:50:09 +0000 Subject: [PATCH 261/277] fix test --- tests/queries/0_stateless/02713_ip4_uint_compare.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02713_ip4_uint_compare.sql b/tests/queries/0_stateless/02713_ip4_uint_compare.sql index 141fa1371b6..ec8d6584329 100644 --- a/tests/queries/0_stateless/02713_ip4_uint_compare.sql +++ b/tests/queries/0_stateless/02713_ip4_uint_compare.sql @@ -7,4 +7,3 @@ SELECT ip <= 2130706442::UInt32, ip >= 2130706442::UInt32, ip != 2130706442::UInt32; - \ No newline at end of file From 9de55d9ccc6d95daa9df63dccd8c94650634e463 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Apr 2023 06:07:58 +0000 Subject: [PATCH 262/277] Rename tests to fix style check --- ...e_diff_aliases.reference => 02710_date_diff_aliases.reference} | 0 .../{25342_date_diff_aliases.sql => 02710_date_diff_aliases.sql} | 0 ...{25343_trim_aliases.reference => 02711_trim_aliases.reference} | 0 .../{25343_trim_aliases.sql => 02711_trim_aliases.sql} | 0 4 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{25342_date_diff_aliases.reference => 02710_date_diff_aliases.reference} (100%) rename tests/queries/0_stateless/{25342_date_diff_aliases.sql => 02710_date_diff_aliases.sql} (100%) rename tests/queries/0_stateless/{25343_trim_aliases.reference => 02711_trim_aliases.reference} (100%) rename tests/queries/0_stateless/{25343_trim_aliases.sql => 02711_trim_aliases.sql} (100%) diff --git a/tests/queries/0_stateless/25342_date_diff_aliases.reference b/tests/queries/0_stateless/02710_date_diff_aliases.reference similarity index 100% rename from tests/queries/0_stateless/25342_date_diff_aliases.reference rename to tests/queries/0_stateless/02710_date_diff_aliases.reference diff --git a/tests/queries/0_stateless/25342_date_diff_aliases.sql b/tests/queries/0_stateless/02710_date_diff_aliases.sql similarity index 100% rename from tests/queries/0_stateless/25342_date_diff_aliases.sql rename to tests/queries/0_stateless/02710_date_diff_aliases.sql diff --git a/tests/queries/0_stateless/25343_trim_aliases.reference b/tests/queries/0_stateless/02711_trim_aliases.reference similarity index 100% rename from tests/queries/0_stateless/25343_trim_aliases.reference rename to tests/queries/0_stateless/02711_trim_aliases.reference diff --git a/tests/queries/0_stateless/25343_trim_aliases.sql b/tests/queries/0_stateless/02711_trim_aliases.sql similarity index 100% rename from tests/queries/0_stateless/25343_trim_aliases.sql rename to tests/queries/0_stateless/02711_trim_aliases.sql From c3f976c58a4eec4b2cc280fe6ad0f62f2b54db07 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Apr 2023 06:19:10 +0000 Subject: [PATCH 263/277] Move space check out of loop --- src/Functions/parseDateTime.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 12e2e113af4..05d18a736e3 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -1055,8 +1055,11 @@ namespace static Pos mysqlMicrosecond(Pos cur, Pos end, const String & fragment, DateTime & /*date*/) { + checkSpace(cur, end, 6, "mysqlMicrosecond requires size >= 6", fragment); + for (size_t i = 0; i < 6; ++i) - cur = assertNumber(cur, end, fragment); + cur = assertNumber(cur, end, fragment); + return cur; } From 4b61685c6469a674a8f7f492d9a94fa0b98a8b64 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Apr 2023 07:29:38 +0000 Subject: [PATCH 264/277] Minor refactoring of formatDateTime() Don't explicitly specify literal length when the called functions can compute it. --- src/Functions/formatDateTime.cpp | 44 ++++++++++++++++---------------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index d6275a54c75..168404eaf01 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -978,15 +978,15 @@ public: instructions.push_back(std::move(instruction)); }; - auto add_extra_shift_or_literal_instruction = [&](size_t amount, std::string_view literal) + auto add_extra_shift_or_literal_instruction = [&](std::string_view literal) { if (mysql_with_only_fixed_length_formatters) - add_extra_shift(amount); + add_extra_shift(literal.size()); else add_literal_instruction(literal); }; - auto add_time_instruction = [&]([[maybe_unused]] typename Instruction::FuncMysql && func, [[maybe_unused]] size_t amount, [[maybe_unused]] std::string_view literal) + auto add_time_instruction = [&]([[maybe_unused]] typename Instruction::FuncMysql && func, [[maybe_unused]] std::string_view literal) { /// DateTime/DateTime64 --> insert instruction /// Other types cannot provide the requested data --> write out template @@ -997,7 +997,7 @@ public: instructions.push_back(std::move(instruction)); } else - add_extra_shift_or_literal_instruction(amount, literal); + add_extra_shift_or_literal_instruction(literal); }; Pos pos = format.data(); @@ -1012,7 +1012,7 @@ public: if (pos < percent_pos) { /// Handle characters before next % - add_extra_shift_or_literal_instruction(percent_pos - pos, std::string_view(pos, percent_pos - pos)); + add_extra_shift_or_literal_instruction(std::string_view(pos, percent_pos - pos)); out_template += String(pos, percent_pos - pos); } @@ -1107,7 +1107,7 @@ public: else { static constexpr std::string_view val = "00"; - add_time_instruction(&Instruction::mysqlMinute, 2, val); + add_time_instruction(&Instruction::mysqlMinute, val); out_template += val; } break; @@ -1260,7 +1260,7 @@ public: case 'p': { static constexpr std::string_view val = "AM"; - add_time_instruction(&Instruction::mysqlAMPM, 2, val); + add_time_instruction(&Instruction::mysqlAMPM, val); out_template += val; break; } @@ -1269,7 +1269,7 @@ public: case 'r': { static constexpr std::string_view val = "12:00 AM"; - add_time_instruction(&Instruction::mysqlHHMM12, 8, val); + add_time_instruction(&Instruction::mysqlHHMM12, val); out_template += val; break; } @@ -1278,7 +1278,7 @@ public: case 'R': { static constexpr std::string_view val = "00:00"; - add_time_instruction(&Instruction::mysqlHHMM24, 5, val); + add_time_instruction(&Instruction::mysqlHHMM24, val); out_template += val; break; } @@ -1287,7 +1287,7 @@ public: case 's': { static constexpr std::string_view val = "00"; - add_time_instruction(&Instruction::mysqlSecond, 2, val); + add_time_instruction(&Instruction::mysqlSecond, val); out_template += val; break; } @@ -1296,7 +1296,7 @@ public: case 'S': { static constexpr std::string_view val = "00"; - add_time_instruction(&Instruction::mysqlSecond, 2, val); + add_time_instruction(&Instruction::mysqlSecond, val); out_template += val; break; } @@ -1305,7 +1305,7 @@ public: case 'T': { static constexpr std::string_view val = "00:00:00"; - add_time_instruction(&Instruction::mysqlISO8601Time, 8, val); + add_time_instruction(&Instruction::mysqlISO8601Time, val); out_template += val; break; } @@ -1314,7 +1314,7 @@ public: case 'h': { static constexpr std::string_view val = "12"; - add_time_instruction(&Instruction::mysqlHour12, 2, val); + add_time_instruction(&Instruction::mysqlHour12, val); out_template += val; break; } @@ -1323,7 +1323,7 @@ public: case 'H': { static constexpr std::string_view val = "00"; - add_time_instruction(&Instruction::mysqlHour24, 2, val); + add_time_instruction(&Instruction::mysqlHour24, val); out_template += val; break; } @@ -1332,7 +1332,7 @@ public: case 'i': { static constexpr std::string_view val = "00"; - add_time_instruction(&Instruction::mysqlMinute, 2, val); + add_time_instruction(&Instruction::mysqlMinute, val); out_template += val; break; } @@ -1341,7 +1341,7 @@ public: case 'I': { static constexpr std::string_view val = "12"; - add_time_instruction(&Instruction::mysqlHour12, 2, val); + add_time_instruction(&Instruction::mysqlHour12, val); out_template += val; break; } @@ -1350,7 +1350,7 @@ public: case 'k': { static constexpr std::string_view val = "00"; - add_time_instruction(&Instruction::mysqlHour24, 2, val); + add_time_instruction(&Instruction::mysqlHour24, val); out_template += val; break; } @@ -1359,7 +1359,7 @@ public: case 'l': { static constexpr std::string_view val = "12"; - add_time_instruction(&Instruction::mysqlHour12, 2, val); + add_time_instruction(&Instruction::mysqlHour12, val); out_template += val; break; } @@ -1367,7 +1367,7 @@ public: case 't': { static constexpr std::string_view val = "\t"; - add_extra_shift_or_literal_instruction(1, val); + add_extra_shift_or_literal_instruction(val); out_template += val; break; } @@ -1375,7 +1375,7 @@ public: case 'n': { static constexpr std::string_view val = "\n"; - add_extra_shift_or_literal_instruction(1, val); + add_extra_shift_or_literal_instruction(val); out_template += val; break; } @@ -1384,7 +1384,7 @@ public: case '%': { static constexpr std::string_view val = "%"; - add_extra_shift_or_literal_instruction(1, val); + add_extra_shift_or_literal_instruction(val); out_template += val; break; } @@ -1411,7 +1411,7 @@ public: else { /// Handle characters after last % - add_extra_shift_or_literal_instruction(end - pos, std::string_view(pos, end - pos)); + add_extra_shift_or_literal_instruction(std::string_view(pos, end - pos)); out_template += String(pos, end - pos); break; } From dd2364361c08bccab5f872e63a1e267827a7fe44 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov <68555560+aalexfvk@users.noreply.github.com> Date: Tue, 11 Apr 2023 10:45:30 +0300 Subject: [PATCH 265/277] Update src/Coordination/tests/gtest_coordination.cpp Co-authored-by: Antonio Andelic --- src/Coordination/tests/gtest_coordination.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 68e68ca1fa7..b1bea8ddf24 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1672,7 +1672,7 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) // commit all entries state_machine->commit(1, auth_entry->get_buf()); state_machine->commit(2, create_entry->get_buf()); - state_machine->commit(2, set_acl_entry->get_buf()); + state_machine->commit(3, set_acl_entry->get_buf()); const auto & uncommitted_state = state_machine->getStorage().uncommitted_state; auto node = uncommitted_state.getNode(node_path); From 363b97fab8a789742153d63f04ec4738a0c42ec3 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 11 Apr 2023 11:45:29 +0200 Subject: [PATCH 266/277] refine some messages of exception in regexp tree --- src/Dictionaries/RegExpTreeDictionary.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index c072ba78d46..4db88631a2c 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -272,7 +272,7 @@ void RegExpTreeDictionary::initGraph() if (value->parent_id == 0) // this is root node. initTopologyOrder(id, visited, topology_id); if (topology_order.size() != regex_nodes.size()) - throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Invalid Regex tree"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "The topology order cannot match the number of regex nodes. This is likely a internal bug."); } void RegExpTreeDictionary::initTopologyOrder(UInt64 node_idx, std::set & visited, UInt64 & topology_id) @@ -280,7 +280,7 @@ void RegExpTreeDictionary::initTopologyOrder(UInt64 node_idx, std::set & visited.insert(node_idx); for (UInt64 child_idx : regex_nodes[node_idx]->children) if (visited.contains(child_idx)) - throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Invalid Regex tree. The input tree is cyclical"); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "The regexp tree is cyclical. Please check your config."); else initTopologyOrder(child_idx, visited, topology_id); topology_order[node_idx] = topology_id++; From 57a1919594b5affdc27e0093fc93e7c44585438c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 11 Apr 2023 12:40:04 +0200 Subject: [PATCH 267/277] Partially revert e0252db8d and fix pr-bugfix labeling --- tests/ci/run_check.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 8dc136e01f4..de56137b7dd 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -35,8 +35,9 @@ LABELS = { "pr-backward-incompatible": ["Backward Incompatible Change"], "pr-bugfix": [ "Bug Fix", - "Bug Fix (user-visible misbehaviour in official stable release)", - "Bug Fix (user-visible misbehavior in official stable release)", + "Bug Fix (user-visible misbehavior in an official stable release)", + "Bug Fix (user-visible misbehaviour in official stable or prestable release)", + "Bug Fix (user-visible misbehavior in official stable or prestable release)", ], "pr-build": [ "Build/Testing/Packaging Improvement", From 4605d71947077067c0fed62ae3e0cb2179c1ef52 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 11 Apr 2023 11:17:02 +0000 Subject: [PATCH 268/277] Fix build src/Interpreters/InterpreterInsertQuery.h --- src/Interpreters/InterpreterInsertQuery.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index f60d6567d74..5bff472270d 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { From 30659c5b48cf10adf936ade1f1a23c9187febacf Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 11 Apr 2023 11:46:53 +0000 Subject: [PATCH 269/277] Fix build ThreadGroupPtr --- src/Processors/Transforms/buildPushingToViewsChain.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.h b/src/Processors/Transforms/buildPushingToViewsChain.h index a2e7e39ff23..c7effa77d5b 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.h +++ b/src/Processors/Transforms/buildPushingToViewsChain.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace Poco { From 24453759719b15c8681e7f838a44e7e81b9d58ee Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 11 Apr 2023 14:11:24 +0200 Subject: [PATCH 270/277] Fix flaky test test_drop_replica_and_achieve_quorum --- tests/integration/test_quorum_inserts/test.py | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index 779d1a69dcc..4dbd530dd17 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -144,25 +144,6 @@ def test_drop_replica_and_achieve_quorum(started_cluster): ) ) - print("Now we can insert some other data.") - zero.query( - "INSERT INTO test_drop_replica_and_achieve_quorum(a,d) VALUES (2, '2012-02-02')" - ) - - assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV( - zero.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a") - ) - assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV( - first.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a") - ) - assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV( - second.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a") - ) - - zero.query( - "DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum ON CLUSTER cluster" - ) - @pytest.mark.parametrize(("add_new_data"), [False, True]) def test_insert_quorum_with_drop_partition(started_cluster, add_new_data): From c29aa0cd49e635b4b2f929c9519a9f4d055af57f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 11 Apr 2023 15:34:33 +0200 Subject: [PATCH 271/277] Fail the build on a wrong changelog category --- tests/ci/run_check.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index de56137b7dd..b7dc78af4d7 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -31,6 +31,8 @@ SUBMODULE_CHANGED_LABEL = "submodule changed" # They are used in .github/PULL_REQUEST_TEMPLATE.md, keep comments there # updated accordingly +# The following lists are append only, try to avoid editing them +# They atill could be cleaned out after the decent time though. LABELS = { "pr-backward-incompatible": ["Backward Incompatible Change"], "pr-bugfix": [ @@ -190,6 +192,9 @@ def check_pr_description(pr_info: PRInfo) -> Tuple[str, str]: ): return "", category + if category not in CATEGORY_TO_LABEL: + return f"Category '{category}' is not valid", "" + if not entry: return f"Changelog entry required for category '{category}'", category From c6907f2a327894480f93487f3a5e13c3d0613f71 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 11 Apr 2023 09:53:40 -0400 Subject: [PATCH 272/277] add Observability to spell list --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 8f72f07d7ec..4eb0e9dc42d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -108,6 +108,7 @@ NULLIF NVME NYPD NuRaft +Observability OLAP OLTP ObjectId From cdceac662412e968096de800c35c47f650d2996a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 11 Apr 2023 16:10:52 +0200 Subject: [PATCH 273/277] Reduce number of return statements --- tests/ci/run_check.py | 20 +++++++++----------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index b7dc78af4d7..4f38007cb03 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -132,6 +132,7 @@ def check_pr_description(pr_info: PRInfo) -> Tuple[str, str]: category = "" entry = "" + description_error = "" i = 0 while i < len(lines): @@ -183,22 +184,19 @@ def check_pr_description(pr_info: PRInfo) -> Tuple[str, str]: i += 1 if not category: - return "Changelog category is empty", category - + description_error = "Changelog category is empty" # Filter out the PR categories that are not for changelog. - if re.match( + elif re.match( r"(?i)doc|((non|in|not|un)[-\s]*significant)|(not[ ]*for[ ]*changelog)", category, ): - return "", category + pass # to not check the rest of the conditions + elif category not in CATEGORY_TO_LABEL: + description_error, category = f"Category '{category}' is not valid", "" + elif not entry: + description_error = f"Changelog entry required for category '{category}'" - if category not in CATEGORY_TO_LABEL: - return f"Category '{category}' is not valid", "" - - if not entry: - return f"Changelog entry required for category '{category}'", category - - return "", category + return description_error, category if __name__ == "__main__": From bf28be8837f7cbeb9883b943cd04aa74678620c1 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 11 Apr 2023 17:07:44 +0200 Subject: [PATCH 274/277] fix 02504_regexp_dictionary_table_source --- src/Dictionaries/RegExpTreeDictionary.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index 4db88631a2c..c56c611d02f 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -271,14 +271,16 @@ void RegExpTreeDictionary::initGraph() for (const auto & [id, value]: regex_nodes) if (value->parent_id == 0) // this is root node. initTopologyOrder(id, visited, topology_id); + /// If there is a cycle and all nodes have a parent, this condition will be met. if (topology_order.size() != regex_nodes.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "The topology order cannot match the number of regex nodes. This is likely a internal bug."); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "The regexp tree is cyclical. Please check your config."); } void RegExpTreeDictionary::initTopologyOrder(UInt64 node_idx, std::set & visited, UInt64 & topology_id) { visited.insert(node_idx); for (UInt64 child_idx : regex_nodes[node_idx]->children) + /// there is a cycle when dfs the graph. if (visited.contains(child_idx)) throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "The regexp tree is cyclical. Please check your config."); else From 38e89b892a3b95e337441e83b872da3cb919ebe4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Apr 2023 20:31:35 +0000 Subject: [PATCH 275/277] Fix roundAge() --- src/Functions/roundAge.cpp | 3 +-- tests/queries/0_stateless/00968_roundAge.sql | 3 ++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/roundAge.cpp b/src/Functions/roundAge.cpp index d2503bb6938..5c6f9b65279 100644 --- a/src/Functions/roundAge.cpp +++ b/src/Functions/roundAge.cpp @@ -15,8 +15,7 @@ struct RoundAgeImpl static inline ResultType apply(A x) { - return x < 1 ? 0 - : (x < 18 ? 17 + return (x < 18 ? 0 : (x < 25 ? 18 : (x < 35 ? 25 : (x < 45 ? 35 diff --git a/tests/queries/0_stateless/00968_roundAge.sql b/tests/queries/0_stateless/00968_roundAge.sql index c8e5a5579f2..f092dbf6560 100644 --- a/tests/queries/0_stateless/00968_roundAge.sql +++ b/tests/queries/0_stateless/00968_roundAge.sql @@ -1,7 +1,8 @@ SELECT roundAge(0); +SELECT roundAge(12); SELECT roundAge(18); SELECT roundAge(25); SELECT roundAge(35); SELECT roundAge(45); SELECT roundAge(55); -SELECT roundAge(56); \ No newline at end of file +SELECT roundAge(56); From 58d98fa21f2e8de2ada4414c42f65d8314c2d48b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 12 Apr 2023 07:57:00 +0000 Subject: [PATCH 276/277] Revert "Fix roundAge()" This reverts commit 38e89b892a3b95e337441e83b872da3cb919ebe4. --- src/Functions/roundAge.cpp | 3 ++- tests/queries/0_stateless/00968_roundAge.sql | 3 +-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/roundAge.cpp b/src/Functions/roundAge.cpp index 5c6f9b65279..d2503bb6938 100644 --- a/src/Functions/roundAge.cpp +++ b/src/Functions/roundAge.cpp @@ -15,7 +15,8 @@ struct RoundAgeImpl static inline ResultType apply(A x) { - return (x < 18 ? 0 + return x < 1 ? 0 + : (x < 18 ? 17 : (x < 25 ? 18 : (x < 35 ? 25 : (x < 45 ? 35 diff --git a/tests/queries/0_stateless/00968_roundAge.sql b/tests/queries/0_stateless/00968_roundAge.sql index f092dbf6560..c8e5a5579f2 100644 --- a/tests/queries/0_stateless/00968_roundAge.sql +++ b/tests/queries/0_stateless/00968_roundAge.sql @@ -1,8 +1,7 @@ SELECT roundAge(0); -SELECT roundAge(12); SELECT roundAge(18); SELECT roundAge(25); SELECT roundAge(35); SELECT roundAge(45); SELECT roundAge(55); -SELECT roundAge(56); +SELECT roundAge(56); \ No newline at end of file From 9c653197a4630b8a32b2e816ec1deb507c81a370 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 12 Apr 2023 08:05:46 +0000 Subject: [PATCH 277/277] Update roundAge() docs --- docs/en/sql-reference/functions/rounding-functions.md | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index 01ee720cfd3..40ff958e0ae 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -194,7 +194,14 @@ Accepts a number. If the number is less than one, it returns 0. Otherwise, it ro ## roundAge(num) -Accepts a number. If the number is less than 18, it returns 0. Otherwise, it rounds the number down to a number from the set: 18, 25, 35, 45, 55. +Accepts a number. If the number is +- smaller than 1, it returns 0, +- between 1 and 17, it returns 17, +- between 18 and 24, it returns 18, +- between 25 and 34, it returns 25, +- between 35 and 44, it returns 35, +- between 45 and 54, it returns 45, +- larger than 55, it returns 55. ## roundDown(num, arr)