From 4a005579ccdd2f9a2309fc98b2cf275b401a4d4d Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 28 Jan 2022 14:26:50 +0300 Subject: [PATCH 001/165] Randomize some settings in functional tests --- tests/clickhouse-test | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 4566cabf1e7..e3acb9cb99a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -336,6 +336,24 @@ class FailureReason(enum.Enum): INTERNAL_ERROR = "Test internal error: " +class SettingsRandomizer: + settings = { + "convert_query_to_cnf": lambda: random.randint(0, 1), + "max_block_size": lambda: random.randint(8000, 100000), + "max_threads": lambda: random.randint(1, 64), + "input_format_parallel_parsing": lambda: random.randint(0, 1), + "max_read_buffer_size": lambda: random.randint(1, 20) if random.random() < 0.1 else random.randint(500000, 1048576), + "use_uncompressed_cache": lambda: random.randint(0, 1), + } + + @staticmethod + def add_random_settings(client_options): + client_options += " " + for setting, generator in SettingsRandomizer.settings.items(): + client_options += "--" + setting + "=\"" + str(generator()) + "\" " + return client_options + + class TestResult: def __init__(self, case_name: str, status: TestStatus, reason: Optional[FailureReason], total_time: float, description: str): self.case_name: str = case_name @@ -668,6 +686,7 @@ class TestCase: self.runs_count += 1 self.testcase_args = self.configure_testcase_args(args, self.case_file, suite.suite_tmp_path) + client_options = SettingsRandomizer.add_random_settings(client_options) proc, stdout, stderr, total_time = self.run_single_test(server_logs_level, client_options) result = self.process_result_impl(proc, stdout, stderr, total_time) From 1cdf49efede870b70c2b6e4374d8a89c916b6414 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 28 Jan 2022 15:26:55 +0300 Subject: [PATCH 002/165] More random settings Co-authored-by: tavplubix --- tests/clickhouse-test | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index e3acb9cb99a..7ca9870ac34 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -342,6 +342,10 @@ class SettingsRandomizer: "max_block_size": lambda: random.randint(8000, 100000), "max_threads": lambda: random.randint(1, 64), "input_format_parallel_parsing": lambda: random.randint(0, 1), + "min_chunk_bytes_for_parallel_parsing": lambda: max(1024, int(random.gauss(10 * 1024 * 1024, 5 * 1000 * 1000))), + "output_format_parallel_formatting": lambda: random.randint(0, 1), + "fsync_metadata": lambda: random.randint(0, 1), + "priority": lambda: int(abs(random.gauss(0, 2))), "max_read_buffer_size": lambda: random.randint(1, 20) if random.random() < 0.1 else random.randint(500000, 1048576), "use_uncompressed_cache": lambda: random.randint(0, 1), } From 4bd6cd0b806fb389e3ec923976f8218f6f7fa97b Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 28 Jan 2022 16:46:00 +0300 Subject: [PATCH 003/165] Add more settings, print settings if tests fails, add random settings to CLICKHOUSE_URL_PARAMS --- tests/clickhouse-test | 45 +++++++++++++++++++++++++++++++------------ 1 file changed, 33 insertions(+), 12 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 7ca9870ac34..9929f802b86 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -348,14 +348,21 @@ class SettingsRandomizer: "priority": lambda: int(abs(random.gauss(0, 2))), "max_read_buffer_size": lambda: random.randint(1, 20) if random.random() < 0.1 else random.randint(500000, 1048576), "use_uncompressed_cache": lambda: random.randint(0, 1), + "optimize_move_to_prewhere": lambda: random.randint(0, 1), + "prefer_localhost_replica": lambda: random.randint(0, 1), + "group_by_two_level_threshold": lambda: 1 if random.random() < 0.1 else 2 ** 60 if random.random() < 0.11 else 100000, + "group_by_two_level_threshold_bytes": lambda: 1 if random.random() < 0.1 else 2 ** 60 if random.random() < 0.11 else 50000000, + "distributed_aggregation_memory_efficient": lambda: random.randint(0, 1), + "allow_experimental_parallel_reading_from_replicas": lambda: random.randint(0, 1), + "max_insert_threads": lambda: 0 if random.random() < 0.5 else random.randint(1, 16), } @staticmethod - def add_random_settings(client_options): - client_options += " " + def get_random_settings(): + random_settings = [] for setting, generator in SettingsRandomizer.settings.items(): - client_options += "--" + setting + "=\"" + str(generator()) + "\" " - return client_options + random_settings.append(setting + "=" + str(generator()) + "") + return random_settings class TestResult: @@ -438,6 +445,12 @@ class TestCase: return testcase_args + def configure_test_settings(self, client_options): + return client_options + " --" + " --".join(self.random_settings) + + def add_info_about_settings(self, description): + return description + "\n" + "Settings used in the test: " + ", ".join(self.random_settings) + "\n" + def __init__(self, suite, case: str, args, is_concurrent: bool): self.case: str = case # case file name self.tags: Set[str] = suite.all_tags[case] if case in suite.all_tags else set() @@ -452,6 +465,7 @@ class TestCase: self.testcase_args = None self.runs_count = 0 + self.random_settings = SettingsRandomizer.get_random_settings() # should skip test, should increment skipped_total, skip reason def should_skip_test(self, suite) -> Optional[FailureReason]: @@ -690,11 +704,13 @@ class TestCase: self.runs_count += 1 self.testcase_args = self.configure_testcase_args(args, self.case_file, suite.suite_tmp_path) - client_options = SettingsRandomizer.add_random_settings(client_options) + client_options = self.configure_test_settings(client_options) proc, stdout, stderr, total_time = self.run_single_test(server_logs_level, client_options) result = self.process_result_impl(proc, stdout, stderr, total_time) result.check_if_need_retry(args, stdout, stderr, self.runs_count) + if result.status == TestStatus.FAIL: + result.description = self.add_info_about_settings(result.description) return result except KeyboardInterrupt as e: raise e @@ -1480,6 +1496,16 @@ if __name__ == '__main__': else: args.client_database = 'default' + # Add random settings to url parameters + if 'CLICKHOUSE_URL_PARAMS' in os.environ: + os.environ['CLICKHOUSE_URL_PARAMS'] += '&' + else: + os.environ['CLICKHOUSE_URL_PARAMS'] = '' + + random_settings = SettingsRandomizer.get_random_settings() + print("Add random settings to CLICKHOUSE_URL_PARAMS:", ', '.join(random_settings)) + os.environ['CLICKHOUSE_URL_PARAMS'] += '&'.join(random_settings) + if args.client_option: # Set options for client if 'CLICKHOUSE_CLIENT_OPT' in os.environ: @@ -1489,15 +1515,10 @@ if __name__ == '__main__': os.environ['CLICKHOUSE_CLIENT_OPT'] += get_additional_client_options(args) - # Set options for curl - if 'CLICKHOUSE_URL_PARAMS' in os.environ: - os.environ['CLICKHOUSE_URL_PARAMS'] += '&' - else: - os.environ['CLICKHOUSE_URL_PARAMS'] = '' - client_options_query_str = get_additional_client_options_url(args) args.client_options_query_str = client_options_query_str + '&' - os.environ['CLICKHOUSE_URL_PARAMS'] += client_options_query_str + # Set options for curl + os.environ['CLICKHOUSE_URL_PARAMS'] += '&' + client_options_query_str else: args.client_options_query_str = '' From 33c3b2d856fd2cd4a55f071a1f3ce897efd0000c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 8 Feb 2022 18:38:41 +0300 Subject: [PATCH 004/165] Better message --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 9929f802b86..b49ac7a92c3 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -449,7 +449,7 @@ class TestCase: return client_options + " --" + " --".join(self.random_settings) def add_info_about_settings(self, description): - return description + "\n" + "Settings used in the test: " + ", ".join(self.random_settings) + "\n" + return description + "\n" + "Settings used in the test: " + "--" + " --".join(self.random_settings) + "\n" def __init__(self, suite, case: str, args, is_concurrent: bool): self.case: str = case # case file name From 880780c7a8e4f20afeb06987a48afc7e8dbbf4ae Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 11 Feb 2022 17:15:56 +0300 Subject: [PATCH 005/165] Fix some tests --- programs/local/LocalServer.cpp | 1 + programs/server/config.xml | 2 +- src/Databases/DatabaseOnDisk.cpp | 2 ++ tests/clickhouse-test | 6 ++++++ .../0_stateless/00474_readonly_settings.reference | 10 ++++------ tests/queries/0_stateless/00474_readonly_settings.sh | 12 +++++++----- .../0_stateless/00808_not_optimize_predicate.sql | 1 + .../0_stateless/00826_cross_to_inner_join.sql | 2 ++ .../0_stateless/00849_multiple_comma_join_2.sql | 1 + .../00965_shard_unresolvable_addresses.sql | 2 ++ .../0_stateless/01017_uniqCombined_memory_usage.sql | 4 ++++ .../0_stateless/01034_sample_final_distributed.sql | 2 ++ .../0_stateless/01056_predicate_optimizer_bugs.sql | 1 + .../0_stateless/01083_cross_to_inner_with_like.sql | 2 ++ .../01099_parallel_distributed_insert_select.sql | 2 ++ .../01155_rename_move_materialized_view.sql | 2 ++ ...arithmetic_operations_in_aggr_func_with_alias.sql | 1 + tests/queries/0_stateless/01293_show_settings.sql | 2 ++ .../0_stateless/01293_system_distribution_queue.sql | 1 + .../0_stateless/01517_select_final_distributed.sql | 2 ++ .../01557_max_parallel_replicas_no_sample.sql | 2 ++ .../01582_move_to_prewhere_compact_parts.sql | 3 +++ .../01605_adaptive_granularity_block_borders.sql | 2 ++ .../01671_aggregate_function_group_bitmap_data.sql | 2 ++ ...01737_move_order_key_to_prewhere_select_final.sql | 3 +++ .../0_stateless/01763_max_distributed_depth.sql | 5 +++++ .../01824_move_to_prewhere_many_columns.sql | 3 +++ tests/queries/0_stateless/01943_query_id_check.sql | 2 ++ tests/queries/0_stateless/02030_tuple_filter.sql | 1 + tests/queries/0_stateless/02136_scalar_progress.sh | 2 +- .../0_stateless/02156_storage_merge_prewhere.sql | 2 ++ tests/queries/1_stateful/00154_avro.sql | 2 +- 32 files changed, 73 insertions(+), 14 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index ce39f22e978..fe105a8b96b 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -333,6 +333,7 @@ std::string LocalServer::getInitialCreateTableQuery() /// Use regular file auto file_name = config().getString("table-file"); table_file = quoteString(file_name); + std::cerr << file_name << "\n"; format_from_file_name = FormatFactory::instance().getFormatFromFileName(file_name, false); } diff --git a/programs/server/config.xml b/programs/server/config.xml index d88773a3fc4..a3f72a6c629 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -176,7 +176,7 @@ - users without password have readonly access. See also: https://www.shodan.io/search?query=clickhouse --> - + :: diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 165bad950f5..46bf2d17f1e 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -118,6 +118,8 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) if (!create->is_dictionary) create->attach = true; +// create->columns_list->c + /// We remove everything that is not needed for ATTACH from the query. assert(!create->temporary); create->database.reset(); diff --git a/tests/clickhouse-test b/tests/clickhouse-test index b49ac7a92c3..3bd8d7263e8 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -446,9 +446,15 @@ class TestCase: return testcase_args def configure_test_settings(self, client_options): + if self.tags and 'no-random-settings' in self.tags: + return client_options + return client_options + " --" + " --".join(self.random_settings) def add_info_about_settings(self, description): + if self.tags and 'no-random-settings' in self.tags: + return description + return description + "\n" + "Settings used in the test: " + "--" + " --".join(self.random_settings) + "\n" def __init__(self, suite, case: str, args, is_concurrent: bool): diff --git a/tests/queries/0_stateless/00474_readonly_settings.reference b/tests/queries/0_stateless/00474_readonly_settings.reference index b1da40ce414..e2b45931965 100644 --- a/tests/queries/0_stateless/00474_readonly_settings.reference +++ b/tests/queries/0_stateless/00474_readonly_settings.reference @@ -2,13 +2,11 @@ "value": 4611686018427387904 "name": "value", "value": "4611686018427387904" -value -value -Cannot modify 'output_format_json_quote_64bit_integers' setting in readonly mode +OK +OK "name": "value", "value": "9223372036854775808" "name": "value", "value": 9223372036854775808 -value -value -Cannot modify 'output_format_json_quote_64bit_integers' setting in readonly mode +OK +OK diff --git a/tests/queries/0_stateless/00474_readonly_settings.sh b/tests/queries/0_stateless/00474_readonly_settings.sh index 0887ecfa14e..07b78c64a7e 100755 --- a/tests/queries/0_stateless/00474_readonly_settings.sh +++ b/tests/queries/0_stateless/00474_readonly_settings.sh @@ -9,13 +9,15 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=0 | grep value $CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=1 | grep value -$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode' -$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode' +$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL" +$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL" + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" | grep value ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" | grep value -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&session_timeout=3600" -d 'SET readonly = 1' +#${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&session_timeout=3600" -d 'SET readonly = 1' + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode.' && echo "OK" || echo "FAIL" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL" -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode.' -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode' diff --git a/tests/queries/0_stateless/00808_not_optimize_predicate.sql b/tests/queries/0_stateless/00808_not_optimize_predicate.sql index 7c1e57706e2..ba8f5eb5753 100644 --- a/tests/queries/0_stateless/00808_not_optimize_predicate.sql +++ b/tests/queries/0_stateless/00808_not_optimize_predicate.sql @@ -1,4 +1,5 @@ SET send_logs_level = 'fatal'; +SET convert_query_to_cnf = 0; DROP TABLE IF EXISTS test_00808; CREATE TABLE test_00808(date Date, id Int8, name String, value Int64, sign Int8) ENGINE = CollapsingMergeTree(sign) ORDER BY (id, date); diff --git a/tests/queries/0_stateless/00826_cross_to_inner_join.sql b/tests/queries/0_stateless/00826_cross_to_inner_join.sql index 392ade02ab7..ce0c8ea2bfc 100644 --- a/tests/queries/0_stateless/00826_cross_to_inner_join.sql +++ b/tests/queries/0_stateless/00826_cross_to_inner_join.sql @@ -1,4 +1,6 @@ SET enable_optimize_predicate_expression = 0; +SET optimize_move_to_prewhere = 1; +SET convert_query_to_cnf = 0; select * from system.one l cross join system.one r; diff --git a/tests/queries/0_stateless/00849_multiple_comma_join_2.sql b/tests/queries/0_stateless/00849_multiple_comma_join_2.sql index 58535f556d9..eabede3ff00 100644 --- a/tests/queries/0_stateless/00849_multiple_comma_join_2.sql +++ b/tests/queries/0_stateless/00849_multiple_comma_join_2.sql @@ -1,4 +1,5 @@ SET enable_optimize_predicate_expression = 0; +SET convert_query_to_cnf = 0; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql b/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql index 0d82519e4d3..555e7a98380 100644 --- a/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql +++ b/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql @@ -1,5 +1,7 @@ -- Tags: shard +SET prefer_localhost_replica = 1; + SELECT count() FROM remote('127.0.0.1,localhos', system.one); -- { serverError 198 } SELECT count() FROM remote('127.0.0.1|localhos', system.one); diff --git a/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql b/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql index d47dc6b8d5f..4aa59cfedf5 100644 --- a/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql +++ b/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql @@ -7,6 +7,8 @@ -- sizeof(HLL) is (2^K * 6 / 8) -- hence max_memory_usage for 100 rows = (96<<10)*100 = 9830400 +SET use_uncompressed_cache = 0; + -- HashTable for UInt32 (used until (1<<13) elements), hence 8192 elements SELECT 'UInt32'; SET max_memory_usage = 4000000; @@ -19,6 +21,8 @@ SELECT 'UInt64'; SET max_memory_usage = 4000000; SELECT sum(u) FROM (SELECT intDiv(number, 4096) AS k, uniqCombined(reinterpretAsString(number % 4096)) u FROM numbers(4096 * 100) GROUP BY k); -- { serverError 241 } SET max_memory_usage = 9830400; + + SELECT sum(u) FROM (SELECT intDiv(number, 4096) AS k, uniqCombined(reinterpretAsString(number % 4096)) u FROM numbers(4096 * 100) GROUP BY k); SELECT 'K=16'; diff --git a/tests/queries/0_stateless/01034_sample_final_distributed.sql b/tests/queries/0_stateless/01034_sample_final_distributed.sql index b784b35cbb3..a81fef645db 100644 --- a/tests/queries/0_stateless/01034_sample_final_distributed.sql +++ b/tests/queries/0_stateless/01034_sample_final_distributed.sql @@ -1,5 +1,7 @@ -- Tags: distributed +set allow_experimental_parallel_reading_from_replicas = 0; + drop table if exists sample_final; create table sample_final (CounterID UInt32, EventDate Date, EventTime DateTime, UserID UInt64, Sign Int8) engine = CollapsingMergeTree(Sign) order by (CounterID, EventDate, intHash32(UserID), EventTime) sample by intHash32(UserID); insert into sample_final select number / (8192 * 4), toDate('2019-01-01'), toDateTime('2019-01-01 00:00:01') + number, number / (8192 * 2), number % 3 = 1 ? -1 : 1 from numbers(1000000); diff --git a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index d59b8fc30ac..6d2bb2964d6 100644 --- a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -1,5 +1,6 @@ SET enable_optimize_predicate_expression = 1; SET joined_subquery_requires_alias = 0; +SET convert_query_to_cnf = 0; -- https://github.com/ClickHouse/ClickHouse/issues/3885 -- https://github.com/ClickHouse/ClickHouse/issues/5485 diff --git a/tests/queries/0_stateless/01083_cross_to_inner_with_like.sql b/tests/queries/0_stateless/01083_cross_to_inner_with_like.sql index 644190cbddf..6ec6e80692c 100644 --- a/tests/queries/0_stateless/01083_cross_to_inner_with_like.sql +++ b/tests/queries/0_stateless/01083_cross_to_inner_with_like.sql @@ -1,3 +1,5 @@ +SET convert_query_to_cnf = 0; + DROP TABLE IF EXISTS n; DROP TABLE IF EXISTS r; diff --git a/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql b/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql index 4e011bf6b31..de93166d891 100644 --- a/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql +++ b/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql @@ -2,6 +2,8 @@ -- set insert_distributed_sync = 1; -- see https://github.com/ClickHouse/ClickHouse/issues/18971 +SET allow_experimental_parallel_reading_from_replicas = 0; -- see https://github.com/ClickHouse/ClickHouse/issues/34525 + DROP TABLE IF EXISTS local_01099_a; DROP TABLE IF EXISTS local_01099_b; DROP TABLE IF EXISTS distributed_01099_a; diff --git a/tests/queries/0_stateless/01155_rename_move_materialized_view.sql b/tests/queries/0_stateless/01155_rename_move_materialized_view.sql index 0b672cbddbf..e0546ec8117 100644 --- a/tests/queries/0_stateless/01155_rename_move_materialized_view.sql +++ b/tests/queries/0_stateless/01155_rename_move_materialized_view.sql @@ -1,5 +1,7 @@ -- Tags: no-parallel +SET prefer_localhost_replica = 1; + DROP DATABASE IF EXISTS test_01155_ordinary; DROP DATABASE IF EXISTS test_01155_atomic; diff --git a/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func_with_alias.sql b/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func_with_alias.sql index 73b87817bb3..242a253e67c 100644 --- a/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func_with_alias.sql +++ b/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func_with_alias.sql @@ -1,4 +1,5 @@ set optimize_arithmetic_operations_in_aggregate_functions = 1; +SET convert_query_to_cnf = 0; explain syntax select min((n as a) + (1 as b)) c from (select number n from numbers(10)) where a > 0 and b > 0 having c > 0; select min((n as a) + (1 as b)) c from (select number n from numbers(10)) where a > 0 and b > 0 having c > 0; diff --git a/tests/queries/0_stateless/01293_show_settings.sql b/tests/queries/0_stateless/01293_show_settings.sql index 08f00ed201c..3e55ffb58d7 100644 --- a/tests/queries/0_stateless/01293_show_settings.sql +++ b/tests/queries/0_stateless/01293_show_settings.sql @@ -1,3 +1,5 @@ +-- Tags: no-random-settings + show settings like 'send_timeout'; SHOW SETTINGS ILIKE '%CONNECT_timeout%'; SHOW CHANGED SETTINGS ILIKE '%MEMORY%'; diff --git a/tests/queries/0_stateless/01293_system_distribution_queue.sql b/tests/queries/0_stateless/01293_system_distribution_queue.sql index 34158fb081c..9997f18f61d 100644 --- a/tests/queries/0_stateless/01293_system_distribution_queue.sql +++ b/tests/queries/0_stateless/01293_system_distribution_queue.sql @@ -1,4 +1,5 @@ -- Tags: no-parallel +set prefer_localhost_replica = 1; drop table if exists null_01293; drop table if exists dist_01293; diff --git a/tests/queries/0_stateless/01517_select_final_distributed.sql b/tests/queries/0_stateless/01517_select_final_distributed.sql index a3d1fcfc185..701828b0b38 100644 --- a/tests/queries/0_stateless/01517_select_final_distributed.sql +++ b/tests/queries/0_stateless/01517_select_final_distributed.sql @@ -1,5 +1,7 @@ -- Tags: distributed +SET allow_experimental_parallel_reading_from_replicas = 0; + DROP TABLE IF EXISTS test5346; CREATE TABLE test5346 (`Id` String, `Timestamp` DateTime, `updated` DateTime) diff --git a/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql index 2b1a66147a4..04777f5b31c 100644 --- a/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql +++ b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql @@ -1,5 +1,7 @@ -- Tags: replica +SET allow_experimental_parallel_reading_from_replicas=0; + DROP TABLE IF EXISTS t; CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY x; INSERT INTO t VALUES ('Hello'); diff --git a/tests/queries/0_stateless/01582_move_to_prewhere_compact_parts.sql b/tests/queries/0_stateless/01582_move_to_prewhere_compact_parts.sql index 788c99da76d..bd3e651e0dc 100644 --- a/tests/queries/0_stateless/01582_move_to_prewhere_compact_parts.sql +++ b/tests/queries/0_stateless/01582_move_to_prewhere_compact_parts.sql @@ -1,3 +1,6 @@ +SET optimize_move_to_prewhere = 1; +SET convert_query_to_cnf = 0; + DROP TABLE IF EXISTS prewhere_move; CREATE TABLE prewhere_move (x Int, y String) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO prewhere_move SELECT number, toString(number) FROM numbers(1000); diff --git a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql index a73045f5a6f..9f26302e564 100644 --- a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql +++ b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql @@ -1,3 +1,5 @@ +SET use_uncompressed_cache = 0; + DROP TABLE IF EXISTS adaptive_table; --- If granularity of consequent blocks differs a lot, then adaptive diff --git a/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.sql b/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.sql index 3f5c5c2f25b..d70665655ca 100644 --- a/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.sql +++ b/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.sql @@ -1,3 +1,5 @@ +SET group_by_two_level_threshold = 10000; + CREATE TABLE group_bitmap_data_test ( `pickup_date` Date, diff --git a/tests/queries/0_stateless/01737_move_order_key_to_prewhere_select_final.sql b/tests/queries/0_stateless/01737_move_order_key_to_prewhere_select_final.sql index ecc11c625e3..0528988ef0e 100644 --- a/tests/queries/0_stateless/01737_move_order_key_to_prewhere_select_final.sql +++ b/tests/queries/0_stateless/01737_move_order_key_to_prewhere_select_final.sql @@ -1,3 +1,6 @@ +SET optimize_move_to_prewhere = 1; +SET convert_query_to_cnf = 1; + DROP TABLE IF EXISTS prewhere_move_select_final; CREATE TABLE prewhere_move_select_final (x Int, y Int, z Int) ENGINE = ReplacingMergeTree() ORDER BY (x, y); diff --git a/tests/queries/0_stateless/01763_max_distributed_depth.sql b/tests/queries/0_stateless/01763_max_distributed_depth.sql index 12b2e368007..0e89d8d296b 100644 --- a/tests/queries/0_stateless/01763_max_distributed_depth.sql +++ b/tests/queries/0_stateless/01763_max_distributed_depth.sql @@ -1,5 +1,7 @@ -- Tags: distributed +SET prefer_localhost_replica = 0; + DROP TABLE IF EXISTS tt6; CREATE TABLE tt6 @@ -13,6 +15,8 @@ CREATE TABLE tt6 ) ENGINE = Distributed('test_shard_localhost', '', 'tt7', rand()); +DROP TABLE IF EXISTS tt7; + CREATE TABLE tt7 as tt6 ENGINE = Distributed('test_shard_localhost', '', 'tt6', rand()); INSERT INTO tt6 VALUES (1, 1, 1, 1, 'ok'); -- { serverError 581 } @@ -28,3 +32,4 @@ INSERT INTO tt6 VALUES (1, 1, 1, 1, 'ok'); -- { serverError 306} SELECT * FROM tt6; -- { serverError 306 } DROP TABLE tt6; +DROP TABLE tt7; diff --git a/tests/queries/0_stateless/01824_move_to_prewhere_many_columns.sql b/tests/queries/0_stateless/01824_move_to_prewhere_many_columns.sql index e03972e818d..c4ef5516fc8 100644 --- a/tests/queries/0_stateless/01824_move_to_prewhere_many_columns.sql +++ b/tests/queries/0_stateless/01824_move_to_prewhere_many_columns.sql @@ -1,3 +1,6 @@ +SET optimize_move_to_prewhere = 1; +SET convert_query_to_cnf = 0; + DROP TABLE IF EXISTS t_move_to_prewhere; CREATE TABLE t_move_to_prewhere (id UInt32, a UInt8, b UInt8, c UInt8, fat_string String) diff --git a/tests/queries/0_stateless/01943_query_id_check.sql b/tests/queries/0_stateless/01943_query_id_check.sql index cb2ef090854..ad9e88e0478 100644 --- a/tests/queries/0_stateless/01943_query_id_check.sql +++ b/tests/queries/0_stateless/01943_query_id_check.sql @@ -1,6 +1,8 @@ -- Tags: no-replicated-database -- Tag no-replicated-database: Different query_id +SET prefer_localhost_replica = 1; + DROP TABLE IF EXISTS tmp; CREATE TABLE tmp ENGINE = TinyLog AS SELECT queryID(); diff --git a/tests/queries/0_stateless/02030_tuple_filter.sql b/tests/queries/0_stateless/02030_tuple_filter.sql index 5efedeb8c0d..c19f538b8e1 100644 --- a/tests/queries/0_stateless/02030_tuple_filter.sql +++ b/tests/queries/0_stateless/02030_tuple_filter.sql @@ -5,6 +5,7 @@ CREATE TABLE test_tuple_filter (id UInt32, value String, log_date Date) Engine=M INSERT INTO test_tuple_filter VALUES (1,'A','2021-01-01'),(2,'B','2021-01-01'),(3,'C','2021-01-01'),(4,'D','2021-01-02'),(5,'E','2021-01-02'); SET force_primary_key = 1; +SET optimize_move_to_prewhere = 1; SELECT * FROM test_tuple_filter WHERE (id, value) = (1, 'A'); SELECT * FROM test_tuple_filter WHERE (1, 'A') = (id, value); diff --git a/tests/queries/0_stateless/02136_scalar_progress.sh b/tests/queries/0_stateless/02136_scalar_progress.sh index 4608031f83d..9f4429b0caa 100755 --- a/tests/queries/0_stateless/02136_scalar_progress.sh +++ b/tests/queries/0_stateless/02136_scalar_progress.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CURL -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d "SELECT (SELECT max(number), count(number) FROM numbers(100000));" -v 2>&1 | grep -E "X-ClickHouse-Summary|X-ClickHouse-Progress" +$CLICKHOUSE_CURL -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d "SELECT (SELECT max(number), count(number) FROM numbers(100000) settings max_block_size=65505);" -v 2>&1 | grep -E "X-ClickHouse-Summary|X-ClickHouse-Progress" diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql index 69fa9ac5ee2..b75d3fa22e5 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql @@ -1,3 +1,5 @@ +SET optimize_move_to_prewhere = 1; + DROP TABLE IF EXISTS t_02156_mt1; DROP TABLE IF EXISTS t_02156_mt2; DROP TABLE IF EXISTS t_02156_log; diff --git a/tests/queries/1_stateful/00154_avro.sql b/tests/queries/1_stateful/00154_avro.sql index ea5d665a3b4..f608da629d2 100644 --- a/tests/queries/1_stateful/00154_avro.sql +++ b/tests/queries/1_stateful/00154_avro.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS test.avro; -SET max_threads = 1, max_block_size = 8192, min_insert_block_size_rows = 8192, min_insert_block_size_bytes = 1048576; -- lower memory usage +SET max_threads = 1, max_insert_threads = 0, max_block_size = 8192, min_insert_block_size_rows = 8192, min_insert_block_size_bytes = 1048576; -- lower memory usage CREATE TABLE test.avro AS test.hits ENGINE = File(Avro); INSERT INTO test.avro SELECT * FROM test.hits LIMIT 10000; From b4440131d00a6242c0404872adaafd499b268454 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 15 Feb 2022 16:25:07 +0800 Subject: [PATCH 006/165] add metrics for clickhouse-local --- src/Client/LocalConnection.cpp | 100 +++++++++++++++++++++++++- src/Client/LocalConnection.h | 11 ++- src/Interpreters/ProfileEventsExt.cpp | 45 ++++++++++++ src/Interpreters/ProfileEventsExt.h | 18 +++++ src/Server/TCPHandler.cpp | 73 +------------------ src/Server/TCPHandler.h | 11 +-- 6 files changed, 180 insertions(+), 78 deletions(-) diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 8ee4b9e1c1f..037111d4b1a 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -6,6 +6,9 @@ #include #include #include +#include +#include +#include namespace DB @@ -18,10 +21,11 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -LocalConnection::LocalConnection(ContextPtr context_, bool send_progress_) +LocalConnection::LocalConnection(ContextPtr context_, bool send_progress_, bool send_profile_events_) : WithContext(context_) , session(getContext(), ClientInfo::Interface::LOCAL) , send_progress(send_progress_) + , send_profile_events(send_profile_events_) { /// Authenticate and create a context to execute queries. session.authenticate("default", "", Poco::Net::SocketAddress{}); @@ -58,6 +62,88 @@ void LocalConnection::updateProgress(const Progress & value) state->progress.incrementPiecewiseAtomically(value); } +void LocalConnection::updateProfileEvents(Block & block) +{ + static const NamesAndTypesList column_names_and_types = { + {"host_name", std::make_shared()}, + {"current_time", std::make_shared()}, + {"thread_id", std::make_shared()}, + {"type", ProfileEvents::TypeEnum}, + {"name", std::make_shared()}, + {"value", std::make_shared()}, + }; + + ColumnsWithTypeAndName temp_columns; + for (auto const & name_and_type : column_names_and_types) + temp_columns.emplace_back(name_and_type.type, name_and_type.name); + + using namespace ProfileEvents; + block = Block(std::move(temp_columns)); + MutableColumns columns = block.mutateColumns(); + auto thread_group = CurrentThread::getGroup(); + auto const current_thread_id = CurrentThread::get().thread_id; + std::vector snapshots; + ThreadIdToCountersSnapshot new_snapshots; + ProfileEventsSnapshot group_snapshot; + { + auto stats = thread_group->getProfileEventsCountersAndMemoryForThreads(); + snapshots.reserve(stats.size()); + + for (auto & stat : stats) + { + auto const thread_id = stat.thread_id; + if (thread_id == current_thread_id) + continue; + auto current_time = time(nullptr); + auto previous_snapshot = last_sent_snapshots.find(thread_id); + auto increment = + previous_snapshot != last_sent_snapshots.end() + ? CountersIncrement(stat.counters, previous_snapshot->second) + : CountersIncrement(stat.counters); + snapshots.push_back(ProfileEventsSnapshot{ + thread_id, + std::move(increment), + stat.memory_usage, + current_time + }); + new_snapshots[thread_id] = std::move(stat.counters); + } + + group_snapshot.thread_id = 0; + group_snapshot.current_time = time(nullptr); + group_snapshot.memory_usage = thread_group->memory_tracker.get(); + auto group_counters = thread_group->performance_counters.getPartiallyAtomicSnapshot(); + auto prev_group_snapshot = last_sent_snapshots.find(0); + group_snapshot.counters = + prev_group_snapshot != last_sent_snapshots.end() + ? CountersIncrement(group_counters, prev_group_snapshot->second) + : CountersIncrement(group_counters); + new_snapshots[0] = std::move(group_counters); + } + last_sent_snapshots = std::move(new_snapshots); + + const String server_display_name = "localhost"; + for (auto & snapshot : snapshots) + { + dumpProfileEvents(snapshot, columns, server_display_name); + dumpMemoryTracker(snapshot, columns, server_display_name); + } + dumpProfileEvents(group_snapshot, columns, server_display_name); + dumpMemoryTracker(group_snapshot, columns, server_display_name); + + MutableColumns logs_columns; + Block curr_block; + size_t rows = 0; + + for (; state->profile_queue->tryPop(curr_block); ++rows) + { + auto curr_columns = curr_block.getColumns(); + for (size_t j = 0; j < curr_columns.size(); ++j) + columns[j]->insertRangeFrom(*curr_columns[j], 0, curr_columns[j]->size()); + } + +} + void LocalConnection::sendQuery( const ConnectionTimeouts &, const String & query, @@ -85,10 +171,15 @@ void LocalConnection::sendQuery( state->query_id = query_id; state->query = query; state->stage = QueryProcessingStage::Enum(stage); + state->profile_queue = std::make_shared(std::numeric_limits::max()); + CurrentThread::attachInternalProfileEventsQueue(state->profile_queue); if (send_progress) state->after_send_progress.restart(); + if (send_profile_events) + state->after_send_profile_events.restart(); + next_packet_type.reset(); try @@ -231,6 +322,13 @@ bool LocalConnection::poll(size_t) return true; } + if (send_profile_events && (state->after_send_profile_events.elapsedMicroseconds() >= query_context->getSettingsRef().interactive_delay)) + { + state->after_send_profile_events.restart(); + next_packet_type = Protocol::Server::ProfileEvents; + return true; + } + try { pollImpl(); diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index b85022cf183..9f0b6989c47 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -29,6 +30,7 @@ struct LocalQueryState std::unique_ptr executor; std::unique_ptr pushing_executor; std::unique_ptr pushing_async_executor; + InternalProfileEventsQueuePtr profile_queue; std::optional exception; @@ -50,13 +52,15 @@ struct LocalQueryState Progress progress; /// Time after the last check to stop the request and send the progress. Stopwatch after_send_progress; + Stopwatch after_send_profile_events; + }; class LocalConnection : public IServerConnection, WithContext { public: - explicit LocalConnection(ContextPtr context_, bool send_progress_ = false); + explicit LocalConnection(ContextPtr context_, bool send_progress_ = false, bool send_profile_events_ = false); ~LocalConnection() override; @@ -129,12 +133,15 @@ private: void updateProgress(const Progress & value); + void updateProfileEvents(Block & block); + bool pollImpl(); ContextMutablePtr query_context; Session session; bool send_progress; + bool send_profile_events; String description = "clickhouse-local"; std::optional state; @@ -144,5 +151,7 @@ private: std::optional next_packet_type; String current_database; + + ProfileEvents::ThreadIdToCountersSnapshot last_sent_snapshots; }; } diff --git a/src/Interpreters/ProfileEventsExt.cpp b/src/Interpreters/ProfileEventsExt.cpp index 472efc109fb..263ee9acec6 100644 --- a/src/Interpreters/ProfileEventsExt.cpp +++ b/src/Interpreters/ProfileEventsExt.cpp @@ -1,5 +1,6 @@ #include "ProfileEventsExt.h" #include +#include #include #include #include @@ -45,4 +46,48 @@ void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, offsets.push_back(offsets.back() + size); } + +void dumpProfileEvents(ProfileEventsSnapshot const & snapshot, DB::MutableColumns & columns, String const & host_name) +{ + size_t rows = 0; + auto & name_column = columns[NAME_COLUMN_INDEX]; + auto & value_column = columns[VALUE_COLUMN_INDEX]; + for (ProfileEvents::Event event = 0; event < ProfileEvents::Counters::num_counters; ++event) + { + Int64 value = snapshot.counters[event]; + + if (value == 0) + continue; + + const char * desc = ProfileEvents::getName(event); + name_column->insertData(desc, strlen(desc)); + value_column->insert(value); + rows++; + } + + // Fill the rest of the columns with data + for (size_t row = 0; row < rows; ++row) + { + size_t i = 0; + columns[i++]->insertData(host_name.data(), host_name.size()); + columns[i++]->insert(UInt64(snapshot.current_time)); + columns[i++]->insert(UInt64{snapshot.thread_id}); + columns[i++]->insert(ProfileEvents::Type::INCREMENT); + } +} + +void dumpMemoryTracker(ProfileEventsSnapshot const & snapshot, DB::MutableColumns & columns, String const & host_name) +{ + { + size_t i = 0; + columns[i++]->insertData(host_name.data(), host_name.size()); + columns[i++]->insert(UInt64(snapshot.current_time)); + columns[i++]->insert(UInt64{snapshot.thread_id}); + columns[i++]->insert(ProfileEvents::Type::GAUGE); + + columns[i++]->insertData(MemoryTracker::USAGE_EVENT_NAME, strlen(MemoryTracker::USAGE_EVENT_NAME)); + columns[i++]->insert(snapshot.memory_usage); + } +} + } diff --git a/src/Interpreters/ProfileEventsExt.h b/src/Interpreters/ProfileEventsExt.h index 8a92eadec79..0edaec64d25 100644 --- a/src/Interpreters/ProfileEventsExt.h +++ b/src/Interpreters/ProfileEventsExt.h @@ -7,9 +7,27 @@ namespace ProfileEvents { +constexpr size_t NAME_COLUMN_INDEX = 4; +constexpr size_t VALUE_COLUMN_INDEX = 5; + +struct ProfileEventsSnapshot +{ + UInt64 thread_id; + ProfileEvents::CountersIncrement counters; + Int64 memory_usage; + time_t current_time; +}; + +using ThreadIdToCountersSnapshot = std::unordered_map; + /// Dumps profile events to columns Map(String, UInt64) void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, bool nonzero_only = true); +/// Add records about provided non-zero ProfileEvents::Counters. +void dumpProfileEvents(ProfileEventsSnapshot const & snapshot, DB::MutableColumns & columns, String const & host_name); + +void dumpMemoryTracker(ProfileEventsSnapshot const & snapshot, DB::MutableColumns & columns, String const & host_name); + /// This is for ProfileEvents packets. enum Type : int8_t { diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 99523ff09e3..1198cc9271f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -853,82 +853,12 @@ void TCPHandler::sendExtremes(const Block & extremes) } } - -namespace -{ - using namespace ProfileEvents; - - constexpr size_t NAME_COLUMN_INDEX = 4; - constexpr size_t VALUE_COLUMN_INDEX = 5; - - struct ProfileEventsSnapshot - { - UInt64 thread_id; - ProfileEvents::CountersIncrement counters; - Int64 memory_usage; - time_t current_time; - }; - - /* - * Add records about provided non-zero ProfileEvents::Counters. - */ - void dumpProfileEvents( - ProfileEventsSnapshot const & snapshot, - MutableColumns & columns, - String const & host_name) - { - size_t rows = 0; - auto & name_column = columns[NAME_COLUMN_INDEX]; - auto & value_column = columns[VALUE_COLUMN_INDEX]; - for (ProfileEvents::Event event = 0; event < ProfileEvents::Counters::num_counters; ++event) - { - Int64 value = snapshot.counters[event]; - - if (value == 0) - continue; - - const char * desc = ProfileEvents::getName(event); - name_column->insertData(desc, strlen(desc)); - value_column->insert(value); - rows++; - } - - // Fill the rest of the columns with data - for (size_t row = 0; row < rows; ++row) - { - size_t i = 0; - columns[i++]->insertData(host_name.data(), host_name.size()); - columns[i++]->insert(UInt64(snapshot.current_time)); - columns[i++]->insert(UInt64{snapshot.thread_id}); - columns[i++]->insert(ProfileEvents::Type::INCREMENT); - } - } - - void dumpMemoryTracker( - ProfileEventsSnapshot const & snapshot, - MutableColumns & columns, - String const & host_name) - { - { - size_t i = 0; - columns[i++]->insertData(host_name.data(), host_name.size()); - columns[i++]->insert(UInt64(snapshot.current_time)); - columns[i++]->insert(UInt64{snapshot.thread_id}); - columns[i++]->insert(ProfileEvents::Type::GAUGE); - - columns[i++]->insertData(MemoryTracker::USAGE_EVENT_NAME, strlen(MemoryTracker::USAGE_EVENT_NAME)); - columns[i++]->insert(snapshot.memory_usage); - } - } -} - - void TCPHandler::sendProfileEvents() { if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS) return; - NamesAndTypesList column_names_and_types = { + static const NamesAndTypesList column_names_and_types = { { "host_name", std::make_shared() }, { "current_time", std::make_shared() }, { "thread_id", std::make_shared() }, @@ -943,6 +873,7 @@ void TCPHandler::sendProfileEvents() Block block(std::move(temp_columns)); + using namespace ProfileEvents; MutableColumns columns = block.mutateColumns(); auto thread_group = CurrentThread::getGroup(); auto const current_thread_id = CurrentThread::get().thread_id; diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 6afda654e6a..b6ce9fa7507 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -3,9 +3,10 @@ #include #include -#include "Common/ProfileEvents.h" +#include #include #include +#include #include #include #include @@ -13,7 +14,7 @@ #include #include #include -#include +#include #include @@ -36,6 +37,8 @@ struct Settings; class ColumnsDescription; struct ProfileInfo; class TCPServer; +class NativeWriter; +class NativeReader; /// State of query processing. struct QueryState @@ -189,9 +192,7 @@ private: CurrentMetrics::Increment metric_increment{CurrentMetrics::TCPConnection}; - using ThreadIdToCountersSnapshot = std::unordered_map; - - ThreadIdToCountersSnapshot last_sent_snapshots; + ProfileEvents::ThreadIdToCountersSnapshot last_sent_snapshots; /// It is the name of the server that will be sent to the client. String server_display_name; From 24bd47e556f7258a2dbd92679f9b21ab8fb8282b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 15 Feb 2022 20:11:13 +0800 Subject: [PATCH 007/165] finish dev --- programs/local/LocalServer.cpp | 2 +- src/Client/ClientBase.h | 1 + src/Client/LocalConnection.cpp | 13 ++++++++----- src/Client/LocalConnection.h | 9 +++++++-- src/Client/Suggest.cpp | 1 + 5 files changed, 18 insertions(+), 8 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index db0015882b0..a090a007219 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -411,7 +411,7 @@ void LocalServer::setupUsers() void LocalServer::connect() { connection_parameters = ConnectionParameters(config()); - connection = LocalConnection::createConnection(connection_parameters, global_context, need_render_progress); + connection = LocalConnection::createConnection(connection_parameters, global_context, need_render_progress, need_render_profile_events); } diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index e74a6a47d76..08316168f98 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -212,6 +212,7 @@ protected: ProgressIndication progress_indication; bool need_render_progress = true; + bool need_render_profile_events = true; bool written_first_block = false; size_t processed_rows = 0; /// How many rows have been read or written. diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 037111d4b1a..e9cf335bd76 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -62,7 +62,7 @@ void LocalConnection::updateProgress(const Progress & value) state->progress.incrementPiecewiseAtomically(value); } -void LocalConnection::updateProfileEvents(Block & block) +void LocalConnection::getProfileEvents(Block & block) { static const NamesAndTypesList column_names_and_types = { {"host_name", std::make_shared()}, @@ -141,7 +141,6 @@ void LocalConnection::updateProfileEvents(Block & block) for (size_t j = 0; j < curr_columns.size(); ++j) columns[j]->insertRangeFrom(*curr_columns[j], 0, curr_columns[j]->size()); } - } void LocalConnection::sendQuery( @@ -163,7 +162,8 @@ void LocalConnection::sendQuery( if (!current_database.empty()) query_context->setCurrentDatabase(current_database); - CurrentThread::QueryScope query_scope_holder(query_context); + query_scope_holder.reset(); + query_scope_holder = std::make_unique(query_context); state.reset(); state.emplace(); @@ -324,8 +324,11 @@ bool LocalConnection::poll(size_t) if (send_profile_events && (state->after_send_profile_events.elapsedMicroseconds() >= query_context->getSettingsRef().interactive_delay)) { + Block block; state->after_send_profile_events.restart(); next_packet_type = Protocol::Server::ProfileEvents; + getProfileEvents(block); + state->block.emplace(std::move(block)); return true; } @@ -557,9 +560,9 @@ void LocalConnection::sendMergeTreeReadTaskResponse(const PartitionReadResponse throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented"); } -ServerConnectionPtr LocalConnection::createConnection(const ConnectionParameters &, ContextPtr current_context, bool send_progress) +ServerConnectionPtr LocalConnection::createConnection(const ConnectionParameters &, ContextPtr current_context, bool send_progress, bool send_profile_events) { - return std::make_unique(current_context, send_progress); + return std::make_unique(current_context, send_progress, send_profile_events); } diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index 9f0b6989c47..7edb791a177 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -66,7 +66,11 @@ public: IServerConnection::Type getConnectionType() const override { return IServerConnection::Type::LOCAL; } - static ServerConnectionPtr createConnection(const ConnectionParameters & connection_parameters, ContextPtr current_context, bool send_progress = false); + static ServerConnectionPtr createConnection( + const ConnectionParameters & connection_parameters, + ContextPtr current_context, + bool send_progress = false, + bool send_profile_events = false); void setDefaultDatabase(const String & database) override; @@ -133,7 +137,7 @@ private: void updateProgress(const Progress & value); - void updateProfileEvents(Block & block); + void getProfileEvents(Block & block); bool pollImpl(); @@ -153,5 +157,6 @@ private: String current_database; ProfileEvents::ThreadIdToCountersSnapshot last_sent_snapshots; + std::unique_ptr query_scope_holder; }; } diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 738c98d2119..b711008e233 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -103,6 +103,7 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p { loading_thread = std::thread([context=Context::createCopy(context), connection_parameters, suggestion_limit, this] { + ThreadStatus thread_status; for (size_t retry = 0; retry < 10; ++retry) { try From e6b29167da0e9d51d7956cc1e2825d462ccd224b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 15 Feb 2022 20:26:53 +0800 Subject: [PATCH 008/165] fix style --- src/Client/LocalConnection.cpp | 5 +++-- src/Interpreters/ProfileEventsExt.cpp | 10 +++++----- src/Interpreters/ProfileEventsExt.h | 4 ++-- src/Server/TCPHandler.cpp | 5 +++-- 4 files changed, 13 insertions(+), 11 deletions(-) diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index e9cf335bd76..08ba485d5fc 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -64,11 +64,13 @@ void LocalConnection::updateProgress(const Progress & value) void LocalConnection::getProfileEvents(Block & block) { + using namespace ProfileEvents; + static const NamesAndTypesList column_names_and_types = { {"host_name", std::make_shared()}, {"current_time", std::make_shared()}, {"thread_id", std::make_shared()}, - {"type", ProfileEvents::TypeEnum}, + {"type", TypeEnum}, {"name", std::make_shared()}, {"value", std::make_shared()}, }; @@ -77,7 +79,6 @@ void LocalConnection::getProfileEvents(Block & block) for (auto const & name_and_type : column_names_and_types) temp_columns.emplace_back(name_and_type.type, name_and_type.name); - using namespace ProfileEvents; block = Block(std::move(temp_columns)); MutableColumns columns = block.mutateColumns(); auto thread_group = CurrentThread::getGroup(); diff --git a/src/Interpreters/ProfileEventsExt.cpp b/src/Interpreters/ProfileEventsExt.cpp index 263ee9acec6..173df507c65 100644 --- a/src/Interpreters/ProfileEventsExt.cpp +++ b/src/Interpreters/ProfileEventsExt.cpp @@ -37,7 +37,7 @@ void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, if (nonzero_only && 0 == value) continue; - const char * desc = ProfileEvents::getName(event); + const char * desc = getName(event); key_column.insertData(desc, strlen(desc)); value_column.insert(value); size++; @@ -52,14 +52,14 @@ void dumpProfileEvents(ProfileEventsSnapshot const & snapshot, DB::MutableColumn size_t rows = 0; auto & name_column = columns[NAME_COLUMN_INDEX]; auto & value_column = columns[VALUE_COLUMN_INDEX]; - for (ProfileEvents::Event event = 0; event < ProfileEvents::Counters::num_counters; ++event) + for (Event event = 0; event < Counters::num_counters; ++event) { Int64 value = snapshot.counters[event]; if (value == 0) continue; - const char * desc = ProfileEvents::getName(event); + const char * desc = getName(event); name_column->insertData(desc, strlen(desc)); value_column->insert(value); rows++; @@ -72,7 +72,7 @@ void dumpProfileEvents(ProfileEventsSnapshot const & snapshot, DB::MutableColumn columns[i++]->insertData(host_name.data(), host_name.size()); columns[i++]->insert(UInt64(snapshot.current_time)); columns[i++]->insert(UInt64{snapshot.thread_id}); - columns[i++]->insert(ProfileEvents::Type::INCREMENT); + columns[i++]->insert(Type::INCREMENT); } } @@ -83,7 +83,7 @@ void dumpMemoryTracker(ProfileEventsSnapshot const & snapshot, DB::MutableColumn columns[i++]->insertData(host_name.data(), host_name.size()); columns[i++]->insert(UInt64(snapshot.current_time)); columns[i++]->insert(UInt64{snapshot.thread_id}); - columns[i++]->insert(ProfileEvents::Type::GAUGE); + columns[i++]->insert(Type::GAUGE); columns[i++]->insertData(MemoryTracker::USAGE_EVENT_NAME, strlen(MemoryTracker::USAGE_EVENT_NAME)); columns[i++]->insert(snapshot.memory_usage); diff --git a/src/Interpreters/ProfileEventsExt.h b/src/Interpreters/ProfileEventsExt.h index 0edaec64d25..ebb6981405f 100644 --- a/src/Interpreters/ProfileEventsExt.h +++ b/src/Interpreters/ProfileEventsExt.h @@ -13,12 +13,12 @@ constexpr size_t VALUE_COLUMN_INDEX = 5; struct ProfileEventsSnapshot { UInt64 thread_id; - ProfileEvents::CountersIncrement counters; + CountersIncrement counters; Int64 memory_usage; time_t current_time; }; -using ThreadIdToCountersSnapshot = std::unordered_map; +using ThreadIdToCountersSnapshot = std::unordered_map; /// Dumps profile events to columns Map(String, UInt64) void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, bool nonzero_only = true); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 1198cc9271f..2f6882643ba 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -855,6 +855,8 @@ void TCPHandler::sendExtremes(const Block & extremes) void TCPHandler::sendProfileEvents() { + using namespace ProfileEvents; + if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS) return; @@ -862,7 +864,7 @@ void TCPHandler::sendProfileEvents() { "host_name", std::make_shared() }, { "current_time", std::make_shared() }, { "thread_id", std::make_shared() }, - { "type", ProfileEvents::TypeEnum }, + { "type", TypeEnum }, { "name", std::make_shared() }, { "value", std::make_shared() }, }; @@ -873,7 +875,6 @@ void TCPHandler::sendProfileEvents() Block block(std::move(temp_columns)); - using namespace ProfileEvents; MutableColumns columns = block.mutateColumns(); auto thread_group = CurrentThread::getGroup(); auto const current_thread_id = CurrentThread::get().thread_id; From 8f6ca035d94feab5aa0f460c9deecc8dc14dc1be Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 15 Feb 2022 14:36:31 +0000 Subject: [PATCH 009/165] Fix next batch of tests --- programs/local/LocalServer.cpp | 1 - programs/server/config.xml | 2 +- src/Databases/DatabaseOnDisk.cpp | 2 -- .../0_stateless/01386_negative_float_constant_key_condition.sql | 2 ++ .../0_stateless/01415_overlimiting_threads_for_repica_bug.sql | 1 + .../01737_move_order_key_to_prewhere_select_final.sql | 2 +- tests/queries/0_stateless/01763_max_distributed_depth.sql | 2 +- tests/queries/0_stateless/01822_short_circuit.sql | 1 + tests/queries/0_stateless/01917_prewhere_column_type.sql | 2 ++ 9 files changed, 9 insertions(+), 6 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index fe105a8b96b..ce39f22e978 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -333,7 +333,6 @@ std::string LocalServer::getInitialCreateTableQuery() /// Use regular file auto file_name = config().getString("table-file"); table_file = quoteString(file_name); - std::cerr << file_name << "\n"; format_from_file_name = FormatFactory::instance().getFormatFromFileName(file_name, false); } diff --git a/programs/server/config.xml b/programs/server/config.xml index a3f72a6c629..d88773a3fc4 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -176,7 +176,7 @@ - users without password have readonly access. See also: https://www.shodan.io/search?query=clickhouse --> - :: + diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 46bf2d17f1e..165bad950f5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -118,8 +118,6 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) if (!create->is_dictionary) create->attach = true; -// create->columns_list->c - /// We remove everything that is not needed for ATTACH from the query. assert(!create->temporary); create->database.reset(); diff --git a/tests/queries/0_stateless/01386_negative_float_constant_key_condition.sql b/tests/queries/0_stateless/01386_negative_float_constant_key_condition.sql index c2191d6ab96..b45b9c84b18 100644 --- a/tests/queries/0_stateless/01386_negative_float_constant_key_condition.sql +++ b/tests/queries/0_stateless/01386_negative_float_constant_key_condition.sql @@ -1,3 +1,5 @@ +SET convert_query_to_cnf = 0; + DROP TABLE IF EXISTS t0; CREATE TABLE t0 diff --git a/tests/queries/0_stateless/01415_overlimiting_threads_for_repica_bug.sql b/tests/queries/0_stateless/01415_overlimiting_threads_for_repica_bug.sql index 306d94387a4..6b5c2ac8ffd 100644 --- a/tests/queries/0_stateless/01415_overlimiting_threads_for_repica_bug.sql +++ b/tests/queries/0_stateless/01415_overlimiting_threads_for_repica_bug.sql @@ -1,5 +1,6 @@ set log_queries = 1; set max_threads = 16; +set prefer_localhost_replica = 1; select sum(number) from remote('127.0.0.{1|2}', numbers_mt(1000000)) group by number % 2 order by number % 2; diff --git a/tests/queries/0_stateless/01737_move_order_key_to_prewhere_select_final.sql b/tests/queries/0_stateless/01737_move_order_key_to_prewhere_select_final.sql index 0528988ef0e..789892dbd38 100644 --- a/tests/queries/0_stateless/01737_move_order_key_to_prewhere_select_final.sql +++ b/tests/queries/0_stateless/01737_move_order_key_to_prewhere_select_final.sql @@ -1,5 +1,5 @@ SET optimize_move_to_prewhere = 1; -SET convert_query_to_cnf = 1; +SET convert_query_to_cnf = 0; DROP TABLE IF EXISTS prewhere_move_select_final; diff --git a/tests/queries/0_stateless/01763_max_distributed_depth.sql b/tests/queries/0_stateless/01763_max_distributed_depth.sql index 0e89d8d296b..f50d15e7121 100644 --- a/tests/queries/0_stateless/01763_max_distributed_depth.sql +++ b/tests/queries/0_stateless/01763_max_distributed_depth.sql @@ -1,6 +1,6 @@ -- Tags: distributed -SET prefer_localhost_replica = 0; +SET prefer_localhost_replica = 1; DROP TABLE IF EXISTS tt6; diff --git a/tests/queries/0_stateless/01822_short_circuit.sql b/tests/queries/0_stateless/01822_short_circuit.sql index 48fff04921b..c7379d210eb 100644 --- a/tests/queries/0_stateless/01822_short_circuit.sql +++ b/tests/queries/0_stateless/01822_short_circuit.sql @@ -1,4 +1,5 @@ set short_circuit_function_evaluation = 'enable'; +set convert_query_to_cnf = 0; select if(number > 0, intDiv(number + 100, number), throwIf(number)) from numbers(10); select multiIf(number == 0, 0, number == 1, intDiv(1, number), number == 2, intDiv(1, number - 1), number == 3, intDiv(1, number - 2), intDiv(1, number - 3)) from numbers(10); diff --git a/tests/queries/0_stateless/01917_prewhere_column_type.sql b/tests/queries/0_stateless/01917_prewhere_column_type.sql index 5147e6093a9..c0bc0c3e36b 100644 --- a/tests/queries/0_stateless/01917_prewhere_column_type.sql +++ b/tests/queries/0_stateless/01917_prewhere_column_type.sql @@ -1,3 +1,5 @@ +SET optimize_move_to_prewhere = 1; + DROP TABLE IF EXISTS t1; CREATE TABLE t1 ( s String, f Float32, e UInt16 ) ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = '100G'; From be5efc33d41a125b368d10f1d9832666436029a0 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 16 Feb 2022 10:35:46 +0000 Subject: [PATCH 010/165] Fix more settings, add random settings for .sh tests --- tests/clickhouse-test | 42 ++++++++++++------- ...4_shard_distributed_with_many_replicas.sql | 1 + .../01017_uniqCombined_memory_usage.sql | 2 +- ...308_row_policy_and_trivial_count_query.sql | 2 + ...457_create_as_table_function_structure.sql | 2 + .../0_stateless/01475_read_subcolumns.sql | 2 + .../0_stateless/01533_multiple_nested.sql | 1 + .../01951_distributed_push_down_limit.sql | 2 + ...mize_distributed_group_by_sharding_key.sql | 1 + 9 files changed, 39 insertions(+), 16 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 3bd8d7263e8..793c700ab3f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -445,11 +445,22 @@ class TestCase: return testcase_args - def configure_test_settings(self, client_options): + def add_random_settings(self, client_options): if self.tags and 'no-random-settings' in self.tags: return client_options - return client_options + " --" + " --".join(self.random_settings) + if len(self.base_url_params) == 0: + os.environ['CLICKHOUSE_URL_PARAMS'] = '&'.join(self.random_settings) + else: + os.environ['CLICKHOUSE_URL_PARAMS'] = self.base_url_params + '&' + '&'.join(self.random_settings) + + new_options = " --" + " --".join(self.random_settings) + os.environ['CLICKHOUSE_CLIENT_OPT'] = self.base_client_options + new_options + return client_options + new_options + + def remove_random_settings_from_env(self): + os.environ['CLICKHOUSE_URL_PARAMS'] = self.base_url_params + os.environ['CLICKHOUSE_CLIENT_OPT'] = self.base_client_options def add_info_about_settings(self, description): if self.tags and 'no-random-settings' in self.tags: @@ -471,7 +482,10 @@ class TestCase: self.testcase_args = None self.runs_count = 0 + self.random_settings = SettingsRandomizer.get_random_settings() + self.base_url_params = os.environ['CLICKHOUSE_URL_PARAMS'] if 'CLICKHOUSE_URL_PARAMS' in os.environ else '' + self.base_client_options = os.environ['CLICKHOUSE_CLIENT_OPT'] if 'CLICKHOUSE_CLIENT_OPT' in os.environ else '' # should skip test, should increment skipped_total, skip reason def should_skip_test(self, suite) -> Optional[FailureReason]: @@ -710,7 +724,7 @@ class TestCase: self.runs_count += 1 self.testcase_args = self.configure_testcase_args(args, self.case_file, suite.suite_tmp_path) - client_options = self.configure_test_settings(client_options) + client_options = self.add_random_settings(client_options) proc, stdout, stderr, total_time = self.run_single_test(server_logs_level, client_options) result = self.process_result_impl(proc, stdout, stderr, total_time) @@ -735,6 +749,9 @@ class TestCase: FailureReason.INTERNAL_ERROR, 0., self.get_description_from_exception_info(sys.exc_info())) + finally: + self.remove_random_settings_from_env() + class TestSuite: @staticmethod @@ -1502,16 +1519,6 @@ if __name__ == '__main__': else: args.client_database = 'default' - # Add random settings to url parameters - if 'CLICKHOUSE_URL_PARAMS' in os.environ: - os.environ['CLICKHOUSE_URL_PARAMS'] += '&' - else: - os.environ['CLICKHOUSE_URL_PARAMS'] = '' - - random_settings = SettingsRandomizer.get_random_settings() - print("Add random settings to CLICKHOUSE_URL_PARAMS:", ', '.join(random_settings)) - os.environ['CLICKHOUSE_URL_PARAMS'] += '&'.join(random_settings) - if args.client_option: # Set options for client if 'CLICKHOUSE_CLIENT_OPT' in os.environ: @@ -1521,10 +1528,15 @@ if __name__ == '__main__': os.environ['CLICKHOUSE_CLIENT_OPT'] += get_additional_client_options(args) + # Set options for curl + if 'CLICKHOUSE_URL_PARAMS' in os.environ: + os.environ['CLICKHOUSE_URL_PARAMS'] += '&' + else: + os.environ['CLICKHOUSE_URL_PARAMS'] = '' + client_options_query_str = get_additional_client_options_url(args) args.client_options_query_str = client_options_query_str + '&' - # Set options for curl - os.environ['CLICKHOUSE_URL_PARAMS'] += '&' + client_options_query_str + os.environ['CLICKHOUSE_URL_PARAMS'] += client_options_query_str else: args.client_options_query_str = '' diff --git a/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql b/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql index f9cbf92db41..e29a166c1ee 100644 --- a/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql +++ b/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql @@ -1,5 +1,6 @@ -- Tags: replica, distributed +SET allow_experimental_parallel_reading_from_replicas = 0; SET max_parallel_replicas = 2; DROP TABLE IF EXISTS report; diff --git a/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql b/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql index 4aa59cfedf5..69bd15e3f54 100644 --- a/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql +++ b/tests/queries/0_stateless/01017_uniqCombined_memory_usage.sql @@ -1,4 +1,4 @@ --- Tags: no-tsan, no-asan, no-msan, no-replicated-database +-- Tags: no-tsan, no-asan, no-msan, no-replicated-database, no-random-settings -- Tag no-tsan: Fine thresholds on memory usage -- Tag no-asan: Fine thresholds on memory usage -- Tag no-msan: Fine thresholds on memory usage diff --git a/tests/queries/0_stateless/01308_row_policy_and_trivial_count_query.sql b/tests/queries/0_stateless/01308_row_policy_and_trivial_count_query.sql index cd41bb227eb..81bd2ad97a9 100644 --- a/tests/queries/0_stateless/01308_row_policy_and_trivial_count_query.sql +++ b/tests/queries/0_stateless/01308_row_policy_and_trivial_count_query.sql @@ -1,3 +1,5 @@ +SET optimize_move_to_prewhere = 1; + DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt8) ENGINE = MergeTree ORDER BY x; diff --git a/tests/queries/0_stateless/01457_create_as_table_function_structure.sql b/tests/queries/0_stateless/01457_create_as_table_function_structure.sql index d7c681dc615..bc677698d88 100644 --- a/tests/queries/0_stateless/01457_create_as_table_function_structure.sql +++ b/tests/queries/0_stateless/01457_create_as_table_function_structure.sql @@ -1,5 +1,7 @@ -- Tags: no-parallel +SET prefer_localhost_replica = 1; + DROP DATABASE IF EXISTS test_01457; CREATE DATABASE test_01457; diff --git a/tests/queries/0_stateless/01475_read_subcolumns.sql b/tests/queries/0_stateless/01475_read_subcolumns.sql index 6e2c8d458ae..fa849c889b4 100644 --- a/tests/queries/0_stateless/01475_read_subcolumns.sql +++ b/tests/queries/0_stateless/01475_read_subcolumns.sql @@ -1,3 +1,5 @@ +SET use_uncompressed_cache = 0; + SELECT '====array===='; DROP TABLE IF EXISTS t_arr; CREATE TABLE t_arr (a Array(UInt32)) ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; diff --git a/tests/queries/0_stateless/01533_multiple_nested.sql b/tests/queries/0_stateless/01533_multiple_nested.sql index 82049243006..40aef40eaca 100644 --- a/tests/queries/0_stateless/01533_multiple_nested.sql +++ b/tests/queries/0_stateless/01533_multiple_nested.sql @@ -1,6 +1,7 @@ DROP TABLE IF EXISTS nested; SET flatten_nested = 0; +SET use_uncompressed_cache = 0; CREATE TABLE nested ( diff --git a/tests/queries/0_stateless/01951_distributed_push_down_limit.sql b/tests/queries/0_stateless/01951_distributed_push_down_limit.sql index fa2fc1800c1..184e6321988 100644 --- a/tests/queries/0_stateless/01951_distributed_push_down_limit.sql +++ b/tests/queries/0_stateless/01951_distributed_push_down_limit.sql @@ -1,5 +1,7 @@ -- Tags: distributed +set prefer_localhost_replica = 1; + -- { echo } explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql index d1f80b42e75..74b55b95315 100644 --- a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql @@ -2,6 +2,7 @@ set optimize_skip_unused_shards=1; set optimize_distributed_group_by_sharding_key=1; +set prefer_localhost_replica=1; -- { echo } explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized From 1e797c1a8f214b8015e9a1f2479e5d6bb9435758 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 16 Feb 2022 15:10:02 +0000 Subject: [PATCH 011/165] Fix some tests --- programs/client/Client.cpp | 2 +- src/Client/ClientBase.cpp | 2 +- src/Core/Settings.cpp | 16 ++++++++++++++++ src/Core/Settings.h | 6 ++++++ .../01010_pmj_right_table_memory_limits.sql | 2 +- tests/queries/0_stateless/01666_blns_long.sql | 2 ++ ...746_long_zstd_http_compression_json_format.sh | 2 +- 7 files changed, 28 insertions(+), 4 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 2cbf39a61da..8f0da85a62a 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1119,7 +1119,7 @@ void Client::processOptions(const OptionsDescription & options_description, { const auto & name = setting.getName(); if (options.count(name)) - config().setString(name, options[name].as()); + config().setString(name, options[name].as().back()); } if (options.count("config-file") && options.count("config")) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 6fd8c0cb9bc..3b42d0fb043 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1873,7 +1873,7 @@ void ClientBase::readArguments( void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) { - cmd_settings.addProgramOptions(options_description.main_description.value()); + cmd_settings.addProgramOptionsAsMultitokens(options_description.main_description.value()); /// Parse main commandline options. auto parser = po::command_line_parser(arguments).options(options_description.main_description.value()).allow_unregistered(); po::parsed_options parsed = parser.run(); diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 87d7eee0daa..411e73bdf1a 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -89,6 +89,14 @@ void Settings::addProgramOptions(boost::program_options::options_description & o } } +void Settings::addProgramOptionsAsMultitokens(boost::program_options::options_description & options) +{ + for (const auto & field : all()) + { + addProgramOptionAsMultitoken(options, field); + } +} + void Settings::addProgramOption(boost::program_options::options_description & options, const SettingFieldRef & field) { const std::string_view name = field.getName(); @@ -97,6 +105,14 @@ void Settings::addProgramOption(boost::program_options::options_description & op name.data(), boost::program_options::value()->composing()->notifier(on_program_option), field.getDescription()))); } +void Settings::addProgramOptionAsMultitoken(boost::program_options::options_description & options, const SettingFieldRef & field) +{ + const std::string_view name = field.getName(); + auto on_program_option = boost::function1([this, name](const Strings & values) { set(name, values.back()); }); + options.add(boost::shared_ptr(new boost::program_options::option_description( + name.data(), boost::program_options::value()->multitoken()->composing()->notifier(on_program_option), field.getDescription()))); +} + void Settings::checkNoSettingNamesAtTopLevel(const Poco::Util::AbstractConfiguration & config, const String & config_path) { if (config.getBool("skip_check_for_incorrect_settings", false)) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 25c3ddbe582..1626fd0f74a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -721,6 +721,10 @@ struct Settings : public BaseSettings, public IHints<2, Settings /// (Don't forget to call notify() on the `variables_map` after parsing it!) void addProgramOptions(boost::program_options::options_description & options); + /// Adds program options as to set the settings from a command line. + /// (Don't forget to call notify() on the `variables_map` after parsing it!) + void addProgramOptionsAsMultitokens(boost::program_options::options_description & options); + /// Check that there is no user-level settings at the top level in config. /// This is a common source of mistake (user don't know where to write user-level setting). static void checkNoSettingNamesAtTopLevel(const Poco::Util::AbstractConfiguration & config, const String & config_path); @@ -728,6 +732,8 @@ struct Settings : public BaseSettings, public IHints<2, Settings std::vector getAllRegisteredNames() const override; void addProgramOption(boost::program_options::options_description & options, const SettingFieldRef & field); + + void addProgramOptionAsMultitoken(boost::program_options::options_description & options, const SettingFieldRef & field); }; /* diff --git a/tests/queries/0_stateless/01010_pmj_right_table_memory_limits.sql b/tests/queries/0_stateless/01010_pmj_right_table_memory_limits.sql index af747c93678..7804ce32a5a 100644 --- a/tests/queries/0_stateless/01010_pmj_right_table_memory_limits.sql +++ b/tests/queries/0_stateless/01010_pmj_right_table_memory_limits.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest +-- Tags: no-parallel, no-fasttest, no-random-settings SET max_memory_usage = 32000000; SET join_on_disk_max_files_to_merge = 4; diff --git a/tests/queries/0_stateless/01666_blns_long.sql b/tests/queries/0_stateless/01666_blns_long.sql index fd959cf0a73..74054551b18 100644 --- a/tests/queries/0_stateless/01666_blns_long.sql +++ b/tests/queries/0_stateless/01666_blns_long.sql @@ -27,6 +27,8 @@ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ +SET max_insert_threads = 0; + DROP TABLE IF EXISTS test; CREATE TABLE test diff --git a/tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.sh b/tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.sh index 02943cad583..e10032e04fd 100755 --- a/tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.sh +++ b/tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: zstd' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | zstd -d | tail -n30 | head -n23 +${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: zstd' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) SETTINGS max_block_size=65505 FORMAT JSON" | zstd -d | tail -n30 | head -n23 From 48d036d3f9edd34e676af0426447bd0c1b194efb Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 17 Feb 2022 11:10:38 +0000 Subject: [PATCH 012/165] Fix next batch of tests --- tests/clickhouse-test | 4 ++-- .../01034_prewhere_max_parallel_replicas_distributed.sql | 2 ++ tests/queries/0_stateless/01187_set_profile_as_setting.sh | 1 + tests/queries/0_stateless/01786_explain_merge_tree.sh | 2 ++ tests/queries/0_stateless/01798_uniq_theta_sketch.sql | 2 ++ tests/queries/0_stateless/02050_client_profile_events.sh | 2 +- tests/queries/0_stateless/02136_scalar_read_rows_json.sh | 2 +- 7 files changed, 11 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 0969a1c1697..9d1c9a5b649 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -743,12 +743,12 @@ class TestCase: return TestResult(self.name, TestStatus.FAIL, FailureReason.INTERNAL_QUERY_FAIL, 0., - self.get_description_from_exception_info(sys.exc_info())) + self.add_info_about_settings(self.get_description_from_exception_info(sys.exc_info()))) except (ConnectionRefusedError, ConnectionResetError): return TestResult(self.name, TestStatus.FAIL, FailureReason.SERVER_DIED, 0., - self.get_description_from_exception_info(sys.exc_info())) + self.add_info_about_settings(self.get_description_from_exception_info(sys.exc_info()))) except: return TestResult(self.name, TestStatus.UNKNOWN, FailureReason.INTERNAL_ERROR, diff --git a/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql b/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql index 4eea4fd47c7..3a0b0448110 100644 --- a/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql +++ b/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql @@ -1,5 +1,7 @@ -- Tags: replica, distributed +set allow_experimental_parallel_reading_from_replicas=1; + drop table if exists test_max_parallel_replicas_lr; -- If you wonder why the table is named with "_lr" suffix in this test. diff --git a/tests/queries/0_stateless/01187_set_profile_as_setting.sh b/tests/queries/0_stateless/01187_set_profile_as_setting.sh index ec07f4d3687..dacb609d790 100755 --- a/tests/queries/0_stateless/01187_set_profile_as_setting.sh +++ b/tests/queries/0_stateless/01187_set_profile_as_setting.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-random-settings unset CLICKHOUSE_LOG_COMMENT diff --git a/tests/queries/0_stateless/01786_explain_merge_tree.sh b/tests/queries/0_stateless/01786_explain_merge_tree.sh index 6be86f9ce02..eb47f065044 100755 --- a/tests/queries/0_stateless/01786_explain_merge_tree.sh +++ b/tests/queries/0_stateless/01786_explain_merge_tree.sh @@ -4,6 +4,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +CLICKHOUSE_CLIENT="$CLICKHOUSE_CLIENT --optimize_move_to_prewhere=1 --convert_query_to_cnf=0" + $CLICKHOUSE_CLIENT -q "drop table if exists test_index" $CLICKHOUSE_CLIENT -q "drop table if exists idx" diff --git a/tests/queries/0_stateless/01798_uniq_theta_sketch.sql b/tests/queries/0_stateless/01798_uniq_theta_sketch.sql index bb400c5de14..eace83d5cfa 100644 --- a/tests/queries/0_stateless/01798_uniq_theta_sketch.sql +++ b/tests/queries/0_stateless/01798_uniq_theta_sketch.sql @@ -1,5 +1,7 @@ -- Tags: no-fasttest +SET max_block_size = 65505; + SELECT 'uniqTheta many agrs'; SELECT diff --git a/tests/queries/0_stateless/02050_client_profile_events.sh b/tests/queries/0_stateless/02050_client_profile_events.sh index 459e8505e22..f8bcea0d1bb 100755 --- a/tests/queries/0_stateless/02050_client_profile_events.sh +++ b/tests/queries/0_stateless/02050_client_profile_events.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # do not print any ProfileEvents packets $CLICKHOUSE_CLIENT -q 'select * from numbers(1e5) format Null' |& grep -c 'SelectedRows' # print only last (and also number of rows to provide more info in case of failures) -$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers(1e5)' 2> >(grep -o -e '\[ 0 \] SelectedRows: .*$' -e Exception) 1> >(wc -l) +$CLICKHOUSE_CLIENT --max_block_size=65505 --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers(1e5)' 2> >(grep -o -e '\[ 0 \] SelectedRows: .*$' -e Exception) 1> >(wc -l) # print everything profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events -q 'select sleep(1) from numbers(2) format Null' |& grep -c 'SelectedRows')" test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" diff --git a/tests/queries/0_stateless/02136_scalar_read_rows_json.sh b/tests/queries/0_stateless/02136_scalar_read_rows_json.sh index d589cb60086..34b4b6909b5 100755 --- a/tests/queries/0_stateless/02136_scalar_read_rows_json.sh +++ b/tests/queries/0_stateless/02136_scalar_read_rows_json.sh @@ -7,4 +7,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo "#1" ${CLICKHOUSE_CLIENT} --query='SELECT count() FROM numbers(100) FORMAT JSON;' | grep -a -v "elapsed" echo "#2" -${CLICKHOUSE_CLIENT} --query='SELECT (SELECT max(number), count(number) FROM numbers(100000) as n) FORMAT JSON;' | grep -a -v "elapsed" | grep -v "_subquery" +${CLICKHOUSE_CLIENT} --query='SELECT (SELECT max(number), count(number) FROM numbers(100000) as n) SETTINGS max_block_size = 65505 FORMAT JSON;' | grep -a -v "elapsed" | grep -v "_subquery" From 26f808bd46158902c57ff4599ce26874503f4f27 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 17 Feb 2022 13:46:51 +0000 Subject: [PATCH 013/165] Fix next batch of tests --- .../01034_prewhere_max_parallel_replicas_distributed.sql | 2 +- tests/queries/0_stateless/01655_plan_optimizations.sh | 4 ++-- .../02131_multiply_row_policies_on_same_column.sql | 2 ++ 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql b/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql index 3a0b0448110..6d1c7fd5ef6 100644 --- a/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql +++ b/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql @@ -1,6 +1,6 @@ -- Tags: replica, distributed -set allow_experimental_parallel_reading_from_replicas=1; +set allow_experimental_parallel_reading_from_replicas=0; drop table if exists test_max_parallel_replicas_lr; diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index de3d3ac3eb6..37cf0ecfe14 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -64,7 +64,7 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> one condition of filter should be pushed down after aggregating, other two conditions are ANDed" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --convert_query_to_cnf=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 8 and s - 4 @@ -127,7 +127,7 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> filter is pushed down before sorting steps" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --convert_query_to_cnf=0 -q " explain actions = 1 select x, y from ( select number % 2 as x, number % 3 as y from numbers(6) order by y desc ) where x != 0 and y != 0 diff --git a/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql b/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql index 75f7f737e85..d0a55c6ba65 100644 --- a/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql +++ b/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql @@ -1,3 +1,5 @@ +SET optimize_move_to_prewhere = 1; + DROP TABLE IF EXISTS 02131_multiply_row_policies_on_same_column; CREATE TABLE 02131_multiply_row_policies_on_same_column (x UInt8) ENGINE = MergeTree ORDER BY x; INSERT INTO 02131_multiply_row_policies_on_same_column VALUES (1), (2), (3), (4); From 89f411023254bd6b6b14d042d33bc0be6cf10981 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 18 Feb 2022 12:05:44 +0000 Subject: [PATCH 014/165] Fix more tests --- tests/queries/0_stateless/01275_parallel_mv.sql | 2 ++ tests/queries/0_stateless/01300_group_by_other_keys.sql | 2 ++ tests/queries/0_stateless/01655_plan_optimizations.sh | 2 +- 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01275_parallel_mv.sql b/tests/queries/0_stateless/01275_parallel_mv.sql index 32b43ce616f..11e5ff41417 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.sql +++ b/tests/queries/0_stateless/01275_parallel_mv.sql @@ -1,3 +1,5 @@ +set max_threads = 0; + drop table if exists testX; drop table if exists testXA; drop table if exists testXB; diff --git a/tests/queries/0_stateless/01300_group_by_other_keys.sql b/tests/queries/0_stateless/01300_group_by_other_keys.sql index 22cff012e71..0e37ef55a6a 100644 --- a/tests/queries/0_stateless/01300_group_by_other_keys.sql +++ b/tests/queries/0_stateless/01300_group_by_other_keys.sql @@ -1,3 +1,5 @@ +set max_block_size = 65505; + set optimize_group_by_function_keys = 1; SELECT round(max(log(2) * number), 6) AS k FROM numbers(10000000) GROUP BY number % 2, number % 3, (number % 2 + number % 3) % 2 ORDER BY k; diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index 37cf0ecfe14..b66d788a338 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -77,7 +77,7 @@ $CLICKHOUSE_CLIENT -q " settings enable_optimize_predicate_expression=0" echo "> two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased" -$CLICKHOUSE_CLIENT -q " +$CLICKHOUSE_CLIENT --convert_query_to_cnf=0 -q " explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 8 and y - 4 From 90603ae54944a8022521fc6ba1c76e75ec2093e0 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 18 Feb 2022 12:22:06 +0000 Subject: [PATCH 015/165] Randomize just first batch of settings --- tests/clickhouse-test | 19 +++++-------------- 1 file changed, 5 insertions(+), 14 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 9d1c9a5b649..a729a216f9a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -339,23 +339,14 @@ class FailureReason(enum.Enum): class SettingsRandomizer: settings = { - "convert_query_to_cnf": lambda: random.randint(0, 1), - "max_block_size": lambda: random.randint(8000, 100000), - "max_threads": lambda: random.randint(1, 64), - "input_format_parallel_parsing": lambda: random.randint(0, 1), - "min_chunk_bytes_for_parallel_parsing": lambda: max(1024, int(random.gauss(10 * 1024 * 1024, 5 * 1000 * 1000))), - "output_format_parallel_formatting": lambda: random.randint(0, 1), - "fsync_metadata": lambda: random.randint(0, 1), - "priority": lambda: int(abs(random.gauss(0, 2))), - "max_read_buffer_size": lambda: random.randint(1, 20) if random.random() < 0.1 else random.randint(500000, 1048576), - "use_uncompressed_cache": lambda: random.randint(0, 1), - "optimize_move_to_prewhere": lambda: random.randint(0, 1), - "prefer_localhost_replica": lambda: random.randint(0, 1), + "max_insert_threads": lambda: 0 if random.random() < 0.5 else random.randint(1, 16), "group_by_two_level_threshold": lambda: 1 if random.random() < 0.1 else 2 ** 60 if random.random() < 0.11 else 100000, "group_by_two_level_threshold_bytes": lambda: 1 if random.random() < 0.1 else 2 ** 60 if random.random() < 0.11 else 50000000, "distributed_aggregation_memory_efficient": lambda: random.randint(0, 1), - "allow_experimental_parallel_reading_from_replicas": lambda: random.randint(0, 1), - "max_insert_threads": lambda: 0 if random.random() < 0.5 else random.randint(1, 16), + "fsync_metadata": lambda: random.randint(0, 1), + "priority": lambda: int(abs(random.gauss(0, 2))), + "output_format_parallel_formatting": lambda: random.randint(0, 1), + "input_format_parallel_parsing": lambda: random.randint(0, 1), } @staticmethod From dd37b237ab4f949e27d5bdbfb9a4bc51bc5e801d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 18 Jan 2022 12:41:33 +0100 Subject: [PATCH 016/165] Changing repository url to packages.clickhouse.com --- docker/server/Dockerfile | 4 ++-- docs/_includes/install/deb.sh | 4 ++-- docs/_includes/install/rpm.sh | 7 +++---- docs/_includes/install/tgz.sh | 27 +++++++++++++------------- docs/en/getting-started/install.md | 29 +++++----------------------- docs/ja/getting-started/install.md | 29 +++++----------------------- docs/ru/getting-started/install.md | 31 +++++------------------------- docs/zh/getting-started/install.md | 29 +++++----------------------- 8 files changed, 41 insertions(+), 119 deletions(-) diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 5b10d1fc490..5b7990ab030 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -4,7 +4,7 @@ FROM ubuntu:20.04 ARG apt_archive="http://archive.ubuntu.com" RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list -ARG repository="deb https://repo.clickhouse.com/deb/stable/ main/" +ARG repository="deb https://packages.clickhouse.com/deb stable main" ARG version=22.1.1.* # set non-empty deb_location_url url to create a docker image @@ -58,7 +58,7 @@ RUN groupadd -r clickhouse --gid=101 \ wget \ tzdata \ && mkdir -p /etc/apt/sources.list.d \ - && apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 \ + && apt-key adv --keyserver keyserver.ubuntu.com --recv 8919F6BD2B48D754 \ && echo $repository > /etc/apt/sources.list.d/clickhouse.list \ && if [ -n "$deb_location_url" ]; then \ echo "installing from custom url with deb packages: $deb_location_url" \ diff --git a/docs/_includes/install/deb.sh b/docs/_includes/install/deb.sh index 21106e9fc47..9dceef4c245 100644 --- a/docs/_includes/install/deb.sh +++ b/docs/_includes/install/deb.sh @@ -1,7 +1,7 @@ sudo apt-get install apt-transport-https ca-certificates dirmngr -sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv E0C56BD4 +sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754 -echo "deb https://repo.clickhouse.com/deb/stable/ main/" | sudo tee \ +echo "deb https://packages.clickhouse.com/deb stable main/" | sudo tee \ /etc/apt/sources.list.d/clickhouse.list sudo apt-get update diff --git a/docs/_includes/install/rpm.sh b/docs/_includes/install/rpm.sh index e3fd1232047..ff99018f872 100644 --- a/docs/_includes/install/rpm.sh +++ b/docs/_includes/install/rpm.sh @@ -1,7 +1,6 @@ -sudo yum install yum-utils -sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG -sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/clickhouse.repo -sudo yum install clickhouse-server clickhouse-client +sudo yum install -y yum-utils +sudo yum-config-manager --add-repo https://packages.clickhouse.com/rpm/clickhouse.repo +sudo yum install -y clickhouse-server clickhouse-client sudo /etc/init.d/clickhouse-server start clickhouse-client # or "clickhouse-client --password" if you set up a password. diff --git a/docs/_includes/install/tgz.sh b/docs/_includes/install/tgz.sh index 0994510755b..4ba5890b32b 100644 --- a/docs/_includes/install/tgz.sh +++ b/docs/_includes/install/tgz.sh @@ -1,19 +1,20 @@ -export LATEST_VERSION=$(curl -s https://repo.clickhouse.com/tgz/stable/ | \ +LATEST_VERSION=$(curl -s https://packages.clickhouse.com/tgz/stable/ | \ grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | sort -V -r | head -n 1) -curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-dbg-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-server-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-client-$LATEST_VERSION.tgz +export LATEST_VERSION +curl -O "https://packages.clickhouse.com/tgz/stable/clickhouse-common-static-$LATEST_VERSION.tgz" +curl -O "https://packages.clickhouse.com/tgz/stable/clickhouse-common-static-dbg-$LATEST_VERSION.tgz" +curl -O "https://packages.clickhouse.com/tgz/stable/clickhouse-server-$LATEST_VERSION.tgz" +curl -O "https://packages.clickhouse.com/tgz/stable/clickhouse-client-$LATEST_VERSION.tgz" -tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz -sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh +tar -xzvf "clickhouse-common-static-$LATEST_VERSION.tgz" +sudo "clickhouse-common-static-$LATEST_VERSION/install/doinst.sh" -tar -xzvf clickhouse-common-static-dbg-$LATEST_VERSION.tgz -sudo clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh +tar -xzvf "clickhouse-common-static-dbg-$LATEST_VERSION.tgz" +sudo "clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh" -tar -xzvf clickhouse-server-$LATEST_VERSION.tgz -sudo clickhouse-server-$LATEST_VERSION/install/doinst.sh +tar -xzvf "clickhouse-server-$LATEST_VERSION.tgz" +sudo "clickhouse-server-$LATEST_VERSION/install/doinst.sh" sudo /etc/init.d/clickhouse-server start -tar -xzvf clickhouse-client-$LATEST_VERSION.tgz -sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh +tar -xzvf "clickhouse-client-$LATEST_VERSION.tgz" +sudo "clickhouse-client-$LATEST_VERSION/install/doinst.sh" diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index c03daf45b02..da6e225f745 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -29,7 +29,7 @@ It is recommended to use official pre-compiled `deb` packages for Debian or Ubun You can replace `stable` with `lts` or `testing` to use different [release trains](../faq/operations/production.md) based on your needs. -You can also download and install packages manually from [here](https://repo.clickhouse.com/deb/stable/main/). +You can also download and install packages manually from [here](https://packages.clickhouse.com/deb/pool/stable). #### Packages {#packages} @@ -49,9 +49,7 @@ It is recommended to use official pre-compiled `rpm` packages for CentOS, RedHat First, you need to add the official repository: ``` bash -sudo yum install yum-utils -sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG -sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64 +{% include 'install/rpm.sh' %} ``` If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). `prestable` is sometimes also available. @@ -62,34 +60,17 @@ Then run these commands to install packages: sudo yum install clickhouse-server clickhouse-client ``` -You can also download and install packages manually from [here](https://repo.clickhouse.com/rpm/stable/x86_64). +You can also download and install packages manually from [here](https://packages.clickhouse.com/rpm/stable). ### From Tgz Archives {#from-tgz-archives} It is recommended to use official pre-compiled `tgz` archives for all Linux distributions, where installation of `deb` or `rpm` packages is not possible. -The required version can be downloaded with `curl` or `wget` from repository https://repo.clickhouse.com/tgz/. +The required version can be downloaded with `curl` or `wget` from repository https://packages.clickhouse.com/tgz/. After that downloaded archives should be unpacked and installed with installation scripts. Example for the latest stable version: ``` bash -export LATEST_VERSION=`curl https://api.github.com/repos/ClickHouse/ClickHouse/tags 2>/dev/null | grep stable | grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | head -n 1` -curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-dbg-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-server-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-client-$LATEST_VERSION.tgz - -tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz -sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh - -tar -xzvf clickhouse-common-static-dbg-$LATEST_VERSION.tgz -sudo clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh - -tar -xzvf clickhouse-server-$LATEST_VERSION.tgz -sudo clickhouse-server-$LATEST_VERSION/install/doinst.sh -sudo /etc/init.d/clickhouse-server start - -tar -xzvf clickhouse-client-$LATEST_VERSION.tgz -sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh +{% include 'install/tgz.sh' %} ``` For production environments, it’s recommended to use the latest `stable`-version. You can find its number on GitHub page https://github.com/ClickHouse/ClickHouse/tags with postfix `-stable`. diff --git a/docs/ja/getting-started/install.md b/docs/ja/getting-started/install.md index 7a2a822fe52..575506c3c4b 100644 --- a/docs/ja/getting-started/install.md +++ b/docs/ja/getting-started/install.md @@ -30,7 +30,7 @@ Debian や Ubuntu 用にコンパイル済みの公式パッケージ `deb` を 最新版を使いたい場合は、`stable`を`testing`に置き換えてください。(テスト環境ではこれを推奨します) -同様に、[こちら](https://repo.clickhouse.com/deb/stable/main/)からパッケージをダウンロードして、手動でインストールすることもできます。 +同様に、[こちら](https://packages.clickhouse.com/deb/pool/stable)からパッケージをダウンロードして、手動でインストールすることもできます。 #### パッケージ {#packages} @@ -46,9 +46,7 @@ CentOS、RedHat、その他すべてのrpmベースのLinuxディストリビュ まず、公式リポジトリを追加する必要があります: ``` bash -sudo yum install yum-utils -sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG -sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64 +{% include 'install/rpm.sh' %} ``` 最新版を使いたい場合は `stable` を `testing` に置き換えてください。(テスト環境ではこれが推奨されています)。`prestable` もしばしば同様に利用できます。 @@ -59,33 +57,16 @@ sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64 sudo yum install clickhouse-server clickhouse-client ``` -同様に、[こちら](https://repo.clickhouse.com/rpm/stable/x86_64) からパッケージをダウンロードして、手動でインストールすることもできます。 +同様に、[こちら](https://packages.clickhouse.com/rpm/stable) からパッケージをダウンロードして、手動でインストールすることもできます。 ### Tgzアーカイブから {#from-tgz-archives} すべての Linux ディストリビューションで、`deb` や `rpm` パッケージがインストールできない場合は、公式のコンパイル済み `tgz` アーカイブを使用することをお勧めします。 -必要なバージョンは、リポジトリ https://repo.clickhouse.com/tgz/ から `curl` または `wget` でダウンロードできます。その後、ダウンロードしたアーカイブを解凍し、インストールスクリプトでインストールしてください。最新版の例は以下です: +必要なバージョンは、リポジトリ https://packages.clickhouse.com/tgz/ から `curl` または `wget` でダウンロードできます。その後、ダウンロードしたアーカイブを解凍し、インストールスクリプトでインストールしてください。最新版の例は以下です: ``` bash -export LATEST_VERSION=`curl https://api.github.com/repos/ClickHouse/ClickHouse/tags 2>/dev/null | grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | head -n 1` -curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-dbg-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/clickhouse-server-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/clickhouse-client-$LATEST_VERSION.tgz - -tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz -sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh - -tar -xzvf clickhouse-common-static-dbg-$LATEST_VERSION.tgz -sudo clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh - -tar -xzvf clickhouse-server-$LATEST_VERSION.tgz -sudo clickhouse-server-$LATEST_VERSION/install/doinst.sh -sudo /etc/init.d/clickhouse-server start - -tar -xzvf clickhouse-client-$LATEST_VERSION.tgz -sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh +{% include 'install/tgz.sh' %} ``` 本番環境では、最新の `stable` バージョンを使うことをお勧めします。GitHub のページ https://github.com/ClickHouse/ClickHouse/tags で 接尾辞 `-stable` となっているバージョン番号として確認できます。 diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index a12773a75b0..84f9bc576e9 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -27,11 +27,9 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su {% include 'install/deb.sh' %} ``` -Также эти пакеты можно скачать и установить вручную отсюда: https://repo.clickhouse.com/deb/stable/main/. - Чтобы использовать различные [версии ClickHouse](../faq/operations/production.md) в зависимости от ваших потребностей, вы можете заменить `stable` на `lts` или `testing`. -Также вы можете вручную скачать и установить пакеты из [репозитория](https://repo.clickhouse.com/deb/stable/main/). +Также вы можете вручную скачать и установить пакеты из [репозитория](https://packages.clickhouse.com/deb/pool/stable). #### Пакеты {#packages} @@ -51,9 +49,7 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su Сначала нужно подключить официальный репозиторий: ``` bash -sudo yum install yum-utils -sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG -sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64 +{% include 'install/rpm.sh' %} ``` Для использования наиболее свежих версий нужно заменить `stable` на `testing` (рекомендуется для тестовых окружений). Также иногда доступен `prestable`. @@ -64,34 +60,17 @@ sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64 sudo yum install clickhouse-server clickhouse-client ``` -Также есть возможность установить пакеты вручную, скачав отсюда: https://repo.clickhouse.com/rpm/stable/x86_64. +Также есть возможность установить пакеты вручную, скачав отсюда: https://packages.clickhouse.com/rpm/stable. ### Из Tgz архивов {#from-tgz-archives} Команда ClickHouse в Яндексе рекомендует использовать предкомпилированные бинарники из `tgz` архивов для всех дистрибутивов, где невозможна установка `deb` и `rpm` пакетов. -Интересующую версию архивов можно скачать вручную с помощью `curl` или `wget` из репозитория https://repo.clickhouse.com/tgz/. +Интересующую версию архивов можно скачать вручную с помощью `curl` или `wget` из репозитория https://packages.clickhouse.com/tgz/. После этого архивы нужно распаковать и воспользоваться скриптами установки. Пример установки самой свежей версии: ``` bash -export LATEST_VERSION=`curl https://api.github.com/repos/ClickHouse/ClickHouse/tags 2>/dev/null | grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | head -n 1` -curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-dbg-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/clickhouse-server-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/clickhouse-client-$LATEST_VERSION.tgz - -tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz -sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh - -tar -xzvf clickhouse-common-static-dbg-$LATEST_VERSION.tgz -sudo clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh - -tar -xzvf clickhouse-server-$LATEST_VERSION.tgz -sudo clickhouse-server-$LATEST_VERSION/install/doinst.sh -sudo /etc/init.d/clickhouse-server start - -tar -xzvf clickhouse-client-$LATEST_VERSION.tgz -sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh +{% include 'install/tgz.sh' %} ``` Для production окружений рекомендуется использовать последнюю `stable`-версию. Её номер также можно найти на github с на вкладке https://github.com/ClickHouse/ClickHouse/tags c постфиксом `-stable`. diff --git a/docs/zh/getting-started/install.md b/docs/zh/getting-started/install.md index eec3aabe2a1..6a966355fae 100644 --- a/docs/zh/getting-started/install.md +++ b/docs/zh/getting-started/install.md @@ -29,7 +29,7 @@ $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not 如果您想使用最新的版本,请用`testing`替代`stable`(我们只推荐您用于测试环境)。 -你也可以从这里手动下载安装包:[下载](https://repo.clickhouse.com/deb/stable/main/)。 +你也可以从这里手动下载安装包:[下载](https://packages.clickhouse.com/deb/pool/stable)。 安装包列表: @@ -45,9 +45,7 @@ $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not 首先,您需要添加官方存储库: ``` bash -sudo yum install yum-utils -sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG -sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64 +{% include 'install/rpm.sh' %} ``` 如果您想使用最新的版本,请用`testing`替代`stable`(我们只推荐您用于测试环境)。`prestable`有时也可用。 @@ -58,35 +56,18 @@ sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64 sudo yum install clickhouse-server clickhouse-client ``` -你也可以从这里手动下载安装包:[下载](https://repo.clickhouse.com/rpm/stable/x86_64)。 +你也可以从这里手动下载安装包:[下载](https://packages.clickhouse.com/rpm/stable)。 ### `Tgz`安装包 {#from-tgz-archives} 如果您的操作系统不支持安装`deb`或`rpm`包,建议使用官方预编译的`tgz`软件包。 -所需的版本可以通过`curl`或`wget`从存储库`https://repo.clickhouse.com/tgz/`下载。 +所需的版本可以通过`curl`或`wget`从存储库`https://packages.clickhouse.com/tgz/`下载。 下载后解压缩下载资源文件并使用安装脚本进行安装。以下是一个最新稳定版本的安装示例: ``` bash -export LATEST_VERSION=`curl https://api.github.com/repos/ClickHouse/ClickHouse/tags 2>/dev/null | grep stable | grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | head -n 1` -curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-dbg-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-server-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-client-$LATEST_VERSION.tgz - -tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz -sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh - -tar -xzvf clickhouse-common-static-dbg-$LATEST_VERSION.tgz -sudo clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh - -tar -xzvf clickhouse-server-$LATEST_VERSION.tgz -sudo clickhouse-server-$LATEST_VERSION/install/doinst.sh -sudo /etc/init.d/clickhouse-server start - -tar -xzvf clickhouse-client-$LATEST_VERSION.tgz -sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh +{% include 'install/tgz.sh' %} ``` 对于生产环境,建议使用最新的`stable`版本。你可以在GitHub页面https://github.com/ClickHouse/ClickHouse/tags找到它,它以后缀`-stable`标志。 From 01bd5858c7f9cfce8daf39a3a635acda6dee7e4a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 23 Feb 2022 15:36:37 +0100 Subject: [PATCH 017/165] Add clickhouse-rpm.repo to repository --- packages/clickhouse-rpm.repo | 31 +++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) create mode 100644 packages/clickhouse-rpm.repo diff --git a/packages/clickhouse-rpm.repo b/packages/clickhouse-rpm.repo new file mode 100644 index 00000000000..27321123dc1 --- /dev/null +++ b/packages/clickhouse-rpm.repo @@ -0,0 +1,31 @@ +[clickhouse-stable] +name=ClickHouse - Stable Repository +baseurl=https://packages.clickhouse.com/rpm/stable/ +gpgkey=https://packages.clickhouse.com/rpm/stable/repodata/repomd.xml.key +gpgcheck=0 +repo_gpgcheck=1 +enabled=0 + +[clickhouse-lts] +name=ClickHouse - LTS Repository +baseurl=https://packages.clickhouse.com/rpm/lts/ +gpgkey=https://packages.clickhouse.com/rpm/lts/repodata/repomd.xml.key +gpgcheck=0 +repo_gpgcheck=1 +enabled=0 + +[clickhouse-prestable] +name=ClickHouse - Pre-stable Repository +baseurl=https://packages.clickhouse.com/rpm/prestable/ +gpgkey=https://packages.clickhouse.com/rpm/prestable/repodata/repomd.xml.key +gpgcheck=0 +repo_gpgcheck=1 +enabled=0 + +[clickhouse-testing] +name=ClickHouse - Testing Repository +baseurl=https://packages.clickhouse.com/rpm/testing/ +gpgkey=https://packages.clickhouse.com/rpm/testing/repodata/repomd.xml.key +gpgcheck=0 +repo_gpgcheck=1 +enabled=1 From b7817b4cea0bee1ca14eb503944014d8ab277361 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 24 Feb 2022 00:41:16 +0300 Subject: [PATCH 018/165] Ignore obsolete grants in ATTACH GRANT statements. --- src/Access/AccessEntityIO.cpp | 35 ++++++++++++++----- src/Access/AccessEntityIO.h | 2 +- src/Access/DiskAccessStorage.cpp | 2 +- src/Parsers/Access/ParserGrantQuery.cpp | 9 +++-- .../test_access_for_functions/test.py | 22 ++++++++++-- 5 files changed, 55 insertions(+), 15 deletions(-) diff --git a/src/Access/AccessEntityIO.cpp b/src/Access/AccessEntityIO.cpp index e944dc71114..acf2a972b13 100644 --- a/src/Access/AccessEntityIO.cpp +++ b/src/Access/AccessEntityIO.cpp @@ -91,7 +91,7 @@ String serializeAccessEntity(const IAccessEntity & entity) return buf.str(); } -AccessEntityPtr deserializeAccessEntity(const String & definition, const String & path) +AccessEntityPtr deserializeAccessEntityImpl(const String & definition) { ASTs queries; ParserAttachAccessEntity parser; @@ -118,43 +118,42 @@ AccessEntityPtr deserializeAccessEntity(const String & definition, const String if (auto * create_user_query = query->as()) { if (res) - throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); + throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); res = user = std::make_unique(); InterpreterCreateUserQuery::updateUserFromQuery(*user, *create_user_query); } else if (auto * create_role_query = query->as()) { if (res) - throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); + throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); res = role = std::make_unique(); InterpreterCreateRoleQuery::updateRoleFromQuery(*role, *create_role_query); } else if (auto * create_policy_query = query->as()) { if (res) - throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); + throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); res = policy = std::make_unique(); InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(*policy, *create_policy_query); } else if (auto * create_quota_query = query->as()) { if (res) - throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); + throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); res = quota = std::make_unique(); InterpreterCreateQuotaQuery::updateQuotaFromQuery(*quota, *create_quota_query); } else if (auto * create_profile_query = query->as()) { if (res) - throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); + throw Exception("Two access entities attached in the same file", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); res = profile = std::make_unique(); InterpreterCreateSettingsProfileQuery::updateSettingsProfileFromQuery(*profile, *create_profile_query); } else if (auto * grant_query = query->as()) { if (!user && !role) - throw Exception( - "A user or role should be attached before grant in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); + throw Exception("A user or role should be attached before grant", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); if (user) InterpreterGrantQuery::updateUserFromQuery(*user, *grant_query); else @@ -165,9 +164,27 @@ AccessEntityPtr deserializeAccessEntity(const String & definition, const String } if (!res) - throw Exception("No access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); + throw Exception("No access entities attached", ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); return res; } + +AccessEntityPtr deserializeAccessEntity(const String & definition, const String & file_path) +{ + if (file_path.empty()) + return deserializeAccessEntityImpl(definition); + + try + { + return deserializeAccessEntityImpl(definition); + } + catch (Exception & e) + { + e.addMessage("Could not parse " + file_path); + e.rethrow(); + __builtin_unreachable(); + } +} + } diff --git a/src/Access/AccessEntityIO.h b/src/Access/AccessEntityIO.h index aa0a3e7cf63..457b490affb 100644 --- a/src/Access/AccessEntityIO.h +++ b/src/Access/AccessEntityIO.h @@ -10,6 +10,6 @@ using AccessEntityPtr = std::shared_ptr; String serializeAccessEntity(const IAccessEntity & entity); -AccessEntityPtr deserializeAccessEntity(const String & definition, const String & path); +AccessEntityPtr deserializeAccessEntity(const String & definition, const String & file_path = ""); } diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 5215139b50c..7393fcd8d36 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -48,7 +48,7 @@ namespace } catch (...) { - tryLogCurrentException(&log, "Could not parse " + file_path); + tryLogCurrentException(&log); return nullptr; } } diff --git a/src/Parsers/Access/ParserGrantQuery.cpp b/src/Parsers/Access/ParserGrantQuery.cpp index 9f7e8535a14..43e1cedd34d 100644 --- a/src/Parsers/Access/ParserGrantQuery.cpp +++ b/src/Parsers/Access/ParserGrantQuery.cpp @@ -156,7 +156,7 @@ namespace } - void eraseNonGrantable(AccessRightsElements & elements) + void throwIfNotGrantable(AccessRightsElements & elements) { boost::range::remove_erase_if(elements, [](AccessRightsElement & element) { @@ -303,7 +303,12 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } if (!is_revoke) - eraseNonGrantable(elements); + { + if (attach_mode) + elements.eraseNonGrantable(); + else + throwIfNotGrantable(elements); + } auto query = std::make_shared(); node = query; diff --git a/tests/integration/test_access_for_functions/test.py b/tests/integration/test_access_for_functions/test.py index ebd0f6bd907..0abe74e31a3 100644 --- a/tests/integration/test_access_for_functions/test.py +++ b/tests/integration/test_access_for_functions/test.py @@ -1,8 +1,9 @@ import pytest +import uuid from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance') +instance = cluster.add_instance('instance', stay_alive=True) @pytest.fixture(scope="module", autouse=True) @@ -14,7 +15,8 @@ def started_cluster(): finally: cluster.shutdown() -def test_access_rights_for_funtion(): + +def test_access_rights_for_function(): create_function_query = "CREATE FUNCTION MySum AS (a, b) -> a + b" instance.query("CREATE USER A") @@ -37,3 +39,19 @@ def test_access_rights_for_funtion(): instance.query("DROP USER IF EXISTS A") instance.query("DROP USER IF EXISTS B") + + +def test_ignore_obsolete_grant_on_database(): + instance.stop_clickhouse() + + user_id = uuid.uuid4() + instance.exec_in_container(["bash", "-c" , f""" + cat > /var/lib/clickhouse/access/{user_id}.sql << EOF +ATTACH USER X; +ATTACH GRANT CREATE FUNCTION, SELECT ON mydb.* TO X; +EOF"""]) + + instance.exec_in_container(["bash", "-c" , "touch /var/lib/clickhouse/access/need_rebuild_lists.mark"]) + instance.start_clickhouse() + + assert instance.query("SHOW GRANTS FOR X") == "GRANT SELECT ON mydb.* TO X\n" From d148cc30cc5872ac9eeac6e37a75ef0a1436d1b6 Mon Sep 17 00:00:00 2001 From: Jianmei Zhang Date: Thu, 24 Feb 2022 14:37:35 +0800 Subject: [PATCH 019/165] Support non-table DDLs on cross replicated cluster --- src/Interpreters/DDLTask.cpp | 14 +++++++++----- .../test.py | 8 ++++++++ 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 64b9bf88ae9..a490d7bed43 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -259,13 +259,17 @@ bool DDLTask::tryFindHostInCluster() * */ is_circular_replicated = true; auto * query_with_table = dynamic_cast(query.get()); - if (!query_with_table || !query_with_table->database) + + /// For other DDLs like CREATE USER, there is no database name and should be executed successfully. + if (query_with_table) { - throw Exception(ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION, - "For a distributed DDL on circular replicated cluster its table name must be qualified by database name."); + if (!query_with_table->database) + throw Exception(ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION, + "For a distributed DDL on circular replicated cluster its table name must be qualified by database name."); + + if (default_database == query_with_table->getDatabase()) + return true; } - if (default_database == query_with_table->getDatabase()) - return true; } } found_exact_match = true; diff --git a/tests/integration/test_distributed_ddl_on_cross_replication/test.py b/tests/integration/test_distributed_ddl_on_cross_replication/test.py index 833a3fb1f04..b61bfc5d83f 100644 --- a/tests/integration/test_distributed_ddl_on_cross_replication/test.py +++ b/tests/integration/test_distributed_ddl_on_cross_replication/test.py @@ -104,3 +104,11 @@ def test_atomic_database(started_cluster): node1.query("INSERT INTO replica_1.rmt VALUES (1, 'test')") node2.query("SYSTEM SYNC REPLICA replica_2.rmt", timeout=5) assert_eq_with_retry(node2, "SELECT * FROM replica_2.rmt", '1\ttest') + +def test_non_query_with_table_ddl(started_cluster): + node1.query("CREATE USER A ON CLUSTER cross_3shards_2replicas") + + assert node1.query("SELECT 1", user='A') == "1\n" + assert node2.query("SELECT 1", user='A') == "1\n" + + node2.query("DROP USER A ON CLUSTER cross_3shards_2replicas") From eaf6d8c5f60b2f514b41c3cc4496d7bac1d6714a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 24 Feb 2022 17:35:29 +0100 Subject: [PATCH 020/165] Update DiskS3.cpp --- src/Disks/S3/DiskS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index aff4985a4f1..5d61285981b 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -286,7 +286,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, settings->s3_upload_part_size_multiply_parts_count_threshold, settings->s3_max_single_part_upload_size, std::move(object_metadata), - buf_size /*, std::move(schedule) */); + buf_size, std::move(schedule)); auto create_metadata_callback = [this, path, blob_name, mode] (size_t count) { From 2629614dfe7863f9829ffb656bf3f9cccf432e04 Mon Sep 17 00:00:00 2001 From: Lars Eidnes Date: Fri, 25 Feb 2022 15:36:36 +0100 Subject: [PATCH 021/165] Allow LowCardinality strings for ngrambf_v1/tokenbf_v1 indexes. Fixes #21865 --- .../MergeTree/MergeTreeIndexFullText.cpp | 7 +- ...dinality_text_bloom_filter_index.reference | 24 +++++++ ...ow_cardinality_text_bloom_filter_index.sql | 69 +++++++++++++++++++ 3 files changed, 99 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02226_low_cardinality_text_bloom_filter_index.reference create mode 100644 tests/queries/0_stateless/02226_low_cardinality_text_bloom_filter_index.sql diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index f87584c9cd6..f5a63097a24 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -709,10 +709,15 @@ void bloomFilterIndexValidator(const IndexDescription & index, bool /*attach*/) { const auto & array_type = assert_cast(*index_data_type); data_type = WhichDataType(array_type.getNestedType()); + } + else if (data_type.isLowCarnality()) + { + const auto & low_cardinality = assert_cast(*index_data_type); + data_type = WhichDataType(low_cardinality.getDictionaryType()); } if (!data_type.isString() && !data_type.isFixedString()) - throw Exception("Bloom filter index can be used only with `String`, `FixedString` column or Array with `String` or `FixedString` values column.", ErrorCodes::INCORRECT_QUERY); + throw Exception("Bloom filter index can be used only with `String`, `FixedString`, `LowCardinality(String)`, `LowCardinality(FixedString)` column or Array with `String` or `FixedString` values column.", ErrorCodes::INCORRECT_QUERY); } if (index.type == NgramTokenExtractor::getName()) diff --git a/tests/queries/0_stateless/02226_low_cardinality_text_bloom_filter_index.reference b/tests/queries/0_stateless/02226_low_cardinality_text_bloom_filter_index.reference new file mode 100644 index 00000000000..a1533c4e44a --- /dev/null +++ b/tests/queries/0_stateless/02226_low_cardinality_text_bloom_filter_index.reference @@ -0,0 +1,24 @@ +lc_bf_tokenbf +1 K1 K1ZZZZZZ +2 K2 K2ZZZZZZ +lc_fixed_bf_tokenbf +1 K1 K1ZZZZZZ +2 K2 K2ZZZZZZ +lc_ngram +1 K1 K1ZZZZZZ +2 K2 K2ZZZZZZ +lc_fixed_ngram +1 K1 K1ZZZZZZ +2 K2 K2ZZZZZZ +lc_bf_tokenbf +3 abCD3ef abCD3ef\0 +4 abCD4ef abCD4ef\0 +lc_fixed_bf_tokenbf +3 abCD3ef abCD3ef\0 +4 abCD4ef abCD4ef\0 +lc_ngram +3 abCD3ef abCD3ef\0 +4 abCD4ef abCD4ef\0 +lc_fixed_ngram +3 abCD3ef abCD3ef\0 +4 abCD4ef abCD4ef\0 diff --git a/tests/queries/0_stateless/02226_low_cardinality_text_bloom_filter_index.sql b/tests/queries/0_stateless/02226_low_cardinality_text_bloom_filter_index.sql new file mode 100644 index 00000000000..d2b30f5e8f4 --- /dev/null +++ b/tests/queries/0_stateless/02226_low_cardinality_text_bloom_filter_index.sql @@ -0,0 +1,69 @@ +DROP TABLE IF EXISTS bf_tokenbf_lowcard_test; +DROP TABLE IF EXISTS bf_ngram_lowcard_test; + +CREATE TABLE bf_tokenbf_lowcard_test +( + row_id UInt32, + lc LowCardinality(String), + lc_fixed LowCardinality(FixedString(8)), + INDEX lc_bf_tokenbf lc TYPE tokenbf_v1(256,2,0) GRANULARITY 1, + INDEX lc_fixed_bf_tokenbf lc_fixed TYPE tokenbf_v1(256,2,0) GRANULARITY 1 +) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 1; + +CREATE TABLE bf_ngram_lowcard_test +( + row_id UInt32, + lc LowCardinality(String), + lc_fixed LowCardinality(FixedString(8)), + INDEX lc_ngram lc TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1, + INDEX lc_fixed_ngram lc_fixed TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1 +) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 1; + +INSERT INTO bf_tokenbf_lowcard_test VALUES (1, 'K1', 'K1ZZZZZZ'), (2, 'K2', 'K2ZZZZZZ'); +INSERT INTO bf_ngram_lowcard_test VALUES (1, 'K1', 'K1ZZZZZZ'), (2, 'K2', 'K2ZZZZZZ'); +INSERT INTO bf_tokenbf_lowcard_test VALUES (3, 'abCD3ef', 'abCD3ef'), (4, 'abCD4ef', 'abCD4ef'); +INSERT INTO bf_ngram_lowcard_test VALUES (3, 'abCD3ef', 'abCD3ef'), (4, 'abCD4ef', 'abCD4ef'); + +SELECT 'lc_bf_tokenbf'; +SELECT * FROM bf_tokenbf_lowcard_test WHERE like(lc, 'K1') SETTINGS force_data_skipping_indices='lc_bf_tokenbf'; +SELECT * FROM bf_tokenbf_lowcard_test WHERE like(lc, 'K2') SETTINGS force_data_skipping_indices='lc_bf_tokenbf'; +SELECT * FROM bf_tokenbf_lowcard_test WHERE like(lc, 'K3') SETTINGS force_data_skipping_indices='lc_bf_tokenbf'; + +SELECT 'lc_fixed_bf_tokenbf'; +SELECT * FROM bf_tokenbf_lowcard_test WHERE like(lc_fixed, 'K1ZZZZZZ') SETTINGS force_data_skipping_indices='lc_fixed_bf_tokenbf'; +SELECT * FROM bf_tokenbf_lowcard_test WHERE like(lc_fixed, 'K2ZZZZZZ') SETTINGS force_data_skipping_indices='lc_fixed_bf_tokenbf'; +SELECT * FROM bf_tokenbf_lowcard_test WHERE like(lc_fixed, 'K3ZZZZZZ') SETTINGS force_data_skipping_indices='lc_fixed_bf_tokenbf'; + +SELECT 'lc_ngram'; +SELECT * FROM bf_ngram_lowcard_test WHERE like(lc, 'K1') SETTINGS force_data_skipping_indices='lc_ngram'; +SELECT * FROM bf_ngram_lowcard_test WHERE like(lc, 'K2') SETTINGS force_data_skipping_indices='lc_ngram'; +SELECT * FROM bf_ngram_lowcard_test WHERE like(lc, 'K3') SETTINGS force_data_skipping_indices='lc_ngram'; + +SELECT 'lc_fixed_ngram'; +SELECT * FROM bf_ngram_lowcard_test WHERE like(lc_fixed, 'K1ZZZZZZ') SETTINGS force_data_skipping_indices='lc_fixed_ngram'; +SELECT * FROM bf_ngram_lowcard_test WHERE like(lc_fixed, 'K2ZZZZZZ') SETTINGS force_data_skipping_indices='lc_fixed_ngram'; +SELECT * FROM bf_ngram_lowcard_test WHERE like(lc_fixed, 'K3ZZZZZZ') SETTINGS force_data_skipping_indices='lc_fixed_ngram'; + + +SELECT 'lc_bf_tokenbf'; +SELECT * FROM bf_tokenbf_lowcard_test WHERE like(lc, '%CD3%') SETTINGS force_data_skipping_indices='lc_bf_tokenbf'; +SELECT * FROM bf_tokenbf_lowcard_test WHERE like(lc, '%CD4%') SETTINGS force_data_skipping_indices='lc_bf_tokenbf'; +SELECT * FROM bf_tokenbf_lowcard_test WHERE like(lc, '%CD5%') SETTINGS force_data_skipping_indices='lc_bf_tokenbf'; + +SELECT 'lc_fixed_bf_tokenbf'; +SELECT * FROM bf_tokenbf_lowcard_test WHERE like(lc_fixed, '%CD3%') SETTINGS force_data_skipping_indices='lc_fixed_bf_tokenbf'; +SELECT * FROM bf_tokenbf_lowcard_test WHERE like(lc_fixed, '%CD4%') SETTINGS force_data_skipping_indices='lc_fixed_bf_tokenbf'; +SELECT * FROM bf_tokenbf_lowcard_test WHERE like(lc_fixed, '%CD5%') SETTINGS force_data_skipping_indices='lc_fixed_bf_tokenbf'; + +SELECT 'lc_ngram'; +SELECT * FROM bf_ngram_lowcard_test WHERE like(lc, '%CD3%') SETTINGS force_data_skipping_indices='lc_ngram'; +SELECT * FROM bf_ngram_lowcard_test WHERE like(lc, '%CD4%') SETTINGS force_data_skipping_indices='lc_ngram'; +SELECT * FROM bf_ngram_lowcard_test WHERE like(lc, '%CD5%') SETTINGS force_data_skipping_indices='lc_ngram'; + +SELECT 'lc_fixed_ngram'; +SELECT * FROM bf_ngram_lowcard_test WHERE like(lc_fixed, '%CD3%') SETTINGS force_data_skipping_indices='lc_fixed_ngram'; +SELECT * FROM bf_ngram_lowcard_test WHERE like(lc_fixed, '%CD4%') SETTINGS force_data_skipping_indices='lc_fixed_ngram'; +SELECT * FROM bf_ngram_lowcard_test WHERE like(lc_fixed, '%CD5%') SETTINGS force_data_skipping_indices='lc_fixed_ngram'; + +DROP TABLE bf_tokenbf_lowcard_test; +DROP TABLE bf_ngram_lowcard_test; From 6c8401bfbd4ae820c7e91c3e53bfc1a31fe39925 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 25 Feb 2022 16:35:37 +0100 Subject: [PATCH 022/165] Fix --- src/Storages/FileLog/StorageFileLog.cpp | 36 ++++++++++++------------- src/Storages/FileLog/StorageFileLog.h | 5 ++-- 2 files changed, 20 insertions(+), 21 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index dac70e362ed..4a9da5cbf87 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -53,6 +53,7 @@ StorageFileLog::StorageFileLog( ContextPtr context_, const ColumnsDescription & columns_, const String & path_, + const String & metadata_base_path_, const String & format_name_, std::unique_ptr settings, const String & comment, @@ -61,6 +62,7 @@ StorageFileLog::StorageFileLog( , WithContext(context_->getGlobalContext()) , filelog_settings(std::move(settings)) , path(path_) + , metadata_base_path(std::filesystem::path(metadata_base_path_) / "metadata") , format_name(format_name_) , log(&Poco::Logger::get("StorageFileLog (" + table_id_.table_name + ")")) , milliseconds_to_wait(filelog_settings->poll_directory_watch_events_backoff_init.totalMilliseconds()) @@ -94,18 +96,13 @@ StorageFileLog::StorageFileLog( void StorageFileLog::loadMetaFiles(bool attach) { - const auto & storage = getStorageID(); - /// FIXME Why do we need separate directory? Why not to use data directory? - root_meta_path - = std::filesystem::path(getContext()->getPath()) / "stream_engines/filelog/" / DatabaseCatalog::getPathForUUID(storage.uuid); - /// Attach table if (attach) { /// Meta file may lost, log and create directory - if (!std::filesystem::exists(root_meta_path)) + if (!std::filesystem::exists(metadata_base_path)) { - /// Create root_meta_path directory when store meta data + /// Create metadata_base_path directory when store meta data LOG_ERROR(log, "Metadata files of table {} are lost.", getStorageID().getTableName()); } /// Load all meta info to file_infos; @@ -114,14 +111,14 @@ void StorageFileLog::loadMetaFiles(bool attach) /// Create table, just create meta data directory else { - if (std::filesystem::exists(root_meta_path)) + if (std::filesystem::exists(metadata_base_path)) { throw Exception( ErrorCodes::TABLE_METADATA_ALREADY_EXISTS, "Metadata files already exist by path: {}, remove them manually if it is intended", - root_meta_path); + metadata_base_path); } - /// We do not create the root_meta_path directory at creation time, create it at the moment of serializing + /// We do not create the metadata_base_path directory at creation time, create it at the moment of serializing /// meta files, such that can avoid unnecessarily create this directory if create table failed. } } @@ -212,9 +209,9 @@ void StorageFileLog::loadFiles() void StorageFileLog::serialize() const { - if (!std::filesystem::exists(root_meta_path)) + if (!std::filesystem::exists(metadata_base_path)) { - std::filesystem::create_directories(root_meta_path); + std::filesystem::create_directories(metadata_base_path); } for (const auto & [inode, meta] : file_infos.meta_by_inode) { @@ -236,9 +233,9 @@ void StorageFileLog::serialize() const void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const { - if (!std::filesystem::exists(root_meta_path)) + if (!std::filesystem::exists(metadata_base_path)) { - std::filesystem::create_directories(root_meta_path); + std::filesystem::create_directories(metadata_base_path); } auto full_name = getFullMetaPath(file_meta.file_name); if (!std::filesystem::exists(full_name)) @@ -257,11 +254,11 @@ void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const void StorageFileLog::deserialize() { - if (!std::filesystem::exists(root_meta_path)) + if (!std::filesystem::exists(metadata_base_path)) return; /// In case of single file (not a watched directory), /// iterated directory always has one file inside. - for (const auto & dir_entry : std::filesystem::directory_iterator{root_meta_path}) + for (const auto & dir_entry : std::filesystem::directory_iterator{metadata_base_path}) { if (!dir_entry.is_regular_file()) { @@ -269,7 +266,7 @@ void StorageFileLog::deserialize() ErrorCodes::BAD_FILE_TYPE, "The file {} under {} is not a regular file when deserializing meta files", dir_entry.path().c_str(), - root_meta_path); + metadata_base_path); } ReadBufferFromFile in(dir_entry.path().c_str()); @@ -373,8 +370,8 @@ void StorageFileLog::drop() { try { - if (std::filesystem::exists(root_meta_path)) - std::filesystem::remove_all(root_meta_path); + if (std::filesystem::exists(metadata_base_path)) + std::filesystem::remove_all(metadata_base_path); } catch (...) { @@ -802,6 +799,7 @@ void registerStorageFileLog(StorageFactory & factory) args.getContext(), args.columns, path, + args.relative_data_path, format, std::move(filelog_settings), args.comment, diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index f7e67747965..1d1ae30f58b 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -89,7 +89,7 @@ public: auto & getFileInfos() { return file_infos; } - String getFullMetaPath(const String & file_name) const { return std::filesystem::path(root_meta_path) / file_name; } + String getFullMetaPath(const String & file_name) const { return std::filesystem::path(metadata_base_path) / file_name; } String getFullDataPath(const String & file_name) const { return std::filesystem::path(root_data_path) / file_name; } NamesAndTypesList getVirtuals() const override; @@ -131,6 +131,7 @@ protected: ContextPtr context_, const ColumnsDescription & columns_, const String & path_, + const String & metadata_base_path_, const String & format_name_, std::unique_ptr settings, const String & comment, @@ -145,7 +146,7 @@ private: /// If path argument of the table is a regular file, it equals to user_files_path /// otherwise, it equals to user_files_path/ + path_argument/, e.g. path String root_data_path; - String root_meta_path; + String metadata_base_path; FileInfos file_infos; From 2ba9010a347c721f7d90ea8b0d0819a04e848c23 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 25 Feb 2022 17:53:19 +0100 Subject: [PATCH 023/165] Fix --- .../FileLog/ReadBufferFromFileLog.cpp | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 49 +++++++++++++------ src/Storages/FileLog/StorageFileLog.h | 3 +- 3 files changed, 36 insertions(+), 18 deletions(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 86caac57a53..ebb0925f128 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -134,7 +134,7 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ UInt64 current_position = reader.tellg(); StorageFileLog::assertStreamGood(reader); - file_meta.last_writen_position = current_position; + file_meta.last_written_position = current_position; /// stream reach to end if (current_position == file_meta.last_open_end) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 4a9da5cbf87..604116808ab 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -222,12 +222,12 @@ void StorageFileLog::serialize() const } else { - checkOffsetIsValid(full_name, meta.last_writen_position); + checkOffsetIsValid(full_name, meta.last_written_position); } WriteBufferFromFile out(full_name); writeIntText(inode, out); writeChar('\n', out); - writeIntText(meta.last_writen_position, out); + writeIntText(meta.last_written_position, out); } } @@ -244,12 +244,12 @@ void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const } else { - checkOffsetIsValid(full_name, file_meta.last_writen_position); + checkOffsetIsValid(full_name, file_meta.last_written_position); } WriteBufferFromFile out(full_name); writeIntText(inode, out); writeChar('\n', out); - writeIntText(file_meta.last_writen_position, out); + writeIntText(file_meta.last_written_position, out); } void StorageFileLog::deserialize() @@ -284,7 +284,7 @@ void StorageFileLog::deserialize() } meta.file_name = dir_entry.path().filename(); - meta.last_writen_position = last_written_pos; + meta.last_written_position = last_written_pos; file_infos.meta_by_inode.emplace(inode, meta); } @@ -435,7 +435,17 @@ void StorageFileLog::openFilesAndSetPos() auto & file_ctx = findInMap(file_infos.context_by_name, file); if (file_ctx.status != FileStatus::NO_CHANGE) { - file_ctx.reader.emplace(getFullDataPath(file)); + auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); + + auto current_path = getFullDataPath(file); + if (!std::filesystem::exists(file) && meta.last_written_position != 0) + { + file_ctx.status = FileStatus::REMOVED; + continue; + } + + file_ctx.reader.emplace(current_path); + auto & reader = file_ctx.reader.value(); assertStreamGood(reader); @@ -445,23 +455,24 @@ void StorageFileLog::openFilesAndSetPos() auto file_end = reader.tellg(); assertStreamGood(reader); - auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); - if (meta.last_writen_position > static_cast(file_end)) + if (meta.last_written_position > static_cast(file_end)) { throw Exception( ErrorCodes::CANNOT_READ_ALL_DATA, "Last saved offsset for File {} is bigger than file size ({} > {})", file, - meta.last_writen_position, + meta.last_written_position, file_end); } /// update file end at the moment, used in ReadBuffer and serialize meta.last_open_end = file_end; - reader.seekg(meta.last_writen_position); + reader.seekg(meta.last_written_position); assertStreamGood(reader); } } + + removeInvalidFiles(); serialize(); } @@ -927,6 +938,18 @@ bool StorageFileLog::updateFileInfos() } } } + + removeInvalidFiles(); + + /// These file infos should always have same size(one for one) + assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); + assert(file_infos.file_names.size() == file_infos.context_by_name.size()); + + return events.empty() || file_infos.file_names.empty(); +} + +void StorageFileLog::removeInvalidFiles() +{ std::vector valid_files; /// Remove file infos with REMOVE status @@ -956,12 +979,6 @@ bool StorageFileLog::updateFileInfos() } } file_infos.file_names.swap(valid_files); - - /// These file infos should always have same size(one for one) - assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); - assert(file_infos.file_names.size() == file_infos.context_by_name.size()); - - return events.empty() || file_infos.file_names.empty(); } NamesAndTypesList StorageFileLog::getVirtuals() const diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 1d1ae30f58b..2dd45cbee6d 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -72,7 +72,7 @@ public: struct FileMeta { String file_name; - UInt64 last_writen_position = 0; + UInt64 last_written_position = 0; UInt64 last_open_end = 0; }; @@ -205,6 +205,7 @@ private: void deserialize(); static void checkOffsetIsValid(const String & full_name, UInt64 offset); + void removeInvalidFiles(); }; } From 2176d74cd1292f1580157162d717f59afb415103 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 28 Feb 2022 15:11:38 +0800 Subject: [PATCH 024/165] Use connection pool in HiveMetastoreClient 1. remove lock for hive metastore client access 2. auo reconnect when connection is broken --- src/Storages/Hive/HiveCommon.cpp | 134 ++++++++++++++++++------------- src/Storages/Hive/HiveCommon.h | 63 ++++++++++++--- 2 files changed, 130 insertions(+), 67 deletions(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index aa19ff042e2..4000e5b8981 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -1,3 +1,4 @@ +#include #include #if USE_HIVE @@ -5,6 +6,7 @@ #include #include #include +#include namespace DB @@ -15,6 +17,8 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +const unsigned ThriftHiveMetastoreClientPool::max_connections = 16; + bool HiveMetastoreClient::shouldUpdateTableMetadata( const String & db_name, const String & table_name, const std::vector & partitions) { @@ -40,25 +44,42 @@ bool HiveMetastoreClient::shouldUpdateTableMetadata( return false; } +void HiveMetastoreClient::tryCallHiveClient(std::function func) +{ + int i = 0; + String err_msg; + for (; i < max_retry; ++i) + { + auto client = client_pool.get(get_client_timeout); + try + { + func(client); + } + catch (apache::thrift::transport::TTransportException & e) + { + client.expire(); + err_msg = e.what(); + continue; + } + break; + } + if (i >= max_retry) + throw Exception(ErrorCodes::NO_HIVEMETASTORE, "Hive Metastore expired because {}", err_msg); +} + HiveMetastoreClient::HiveTableMetadataPtr HiveMetastoreClient::getTableMetadata(const String & db_name, const String & table_name) { LOG_TRACE(log, "Get table metadata for {}.{}", db_name, table_name); - std::lock_guard lock{mutex}; auto table = std::make_shared(); std::vector partitions; - try + auto client_call = [&](ThriftHiveMetastoreClientPool::Entry & client) { client->get_table(*table, db_name, table_name); - /// Query the latest partition info to check new change. client->get_partitions(partitions, db_name, table_name, -1); - } - catch (apache::thrift::transport::TTransportException & e) - { - setExpired(); - throw Exception(ErrorCodes::NO_HIVEMETASTORE, "Hive Metastore expired because {}", String(e.what())); - } + }; + tryCallHiveClient(client_call); bool update_cache = shouldUpdateTableMetadata(db_name, table_name, partitions); String cache_key = getCacheKey(db_name, table_name); @@ -103,23 +124,26 @@ HiveMetastoreClient::HiveTableMetadataPtr HiveMetastoreClient::getTableMetadata( return metadata; } +std::shared_ptr HiveMetastoreClient::getHiveTable(const String & db_name, const String & table_name) +{ + auto table = std::make_shared(); + auto client_call = [&](ThriftHiveMetastoreClientPool::Entry & client) + { + client->get_table(*table, db_name, table_name); + }; + tryCallHiveClient(client_call); + return table; +} + void HiveMetastoreClient::clearTableMetadata(const String & db_name, const String & table_name) { String cache_key = getCacheKey(db_name, table_name); - std::lock_guard lock{mutex}; HiveTableMetadataPtr metadata = table_metadata_cache.get(cache_key); if (metadata) table_metadata_cache.remove(cache_key); } -void HiveMetastoreClient::setClient(std::shared_ptr client_) -{ - std::lock_guard lock{mutex}; - client = client_; - clearExpired(); -} - bool HiveMetastoreClient::PartitionInfo::haveSameParameters(const Apache::Hadoop::Hive::Partition & other) const { /// Parameters include keys:numRows,numFiles,rawDataSize,totalSize,transient_lastDdlTime @@ -192,53 +216,55 @@ HiveMetastoreClientFactory & HiveMetastoreClientFactory::instance() return factory; } +using namespace apache::thrift; +using namespace apache::thrift::protocol; +using namespace apache::thrift::transport; +using namespace Apache::Hadoop::Hive; + HiveMetastoreClientPtr HiveMetastoreClientFactory::getOrCreate(const String & name, ContextPtr context) { - using namespace apache::thrift; - using namespace apache::thrift::protocol; - using namespace apache::thrift::transport; - using namespace Apache::Hadoop::Hive; std::lock_guard lock(mutex); auto it = clients.find(name); - if (it == clients.end() || it->second->isExpired()) + if (it == clients.end()) { - /// Connect to hive metastore - Poco::URI hive_metastore_url(name); - const auto & host = hive_metastore_url.getHost(); - auto port = hive_metastore_url.getPort(); - - std::shared_ptr socket = std::make_shared(host, port); - socket->setKeepAlive(true); - socket->setConnTimeout(conn_timeout_ms); - socket->setRecvTimeout(recv_timeout_ms); - socket->setSendTimeout(send_timeout_ms); - std::shared_ptr transport(new TBufferedTransport(socket)); - std::shared_ptr protocol(new TBinaryProtocol(transport)); - std::shared_ptr thrift_client = std::make_shared(protocol); - try + auto builder = [name]() { - transport->open(); - } - catch (TException & tx) - { - throw Exception("connect to hive metastore:" + name + " failed." + tx.what(), ErrorCodes::BAD_ARGUMENTS); - } - - if (it == clients.end()) - { - HiveMetastoreClientPtr client = std::make_shared(std::move(thrift_client), context); - clients[name] = client; - return client; - } - else - { - it->second->setClient(std::move(thrift_client)); - return it->second; - } + return createThriftHiveMetastoreClient(name); + }; + auto client = std::make_shared(builder, context->getGlobalContext()); + clients[name] = client; + return client; } return it->second; } +const int HiveMetastoreClientFactory::conn_timeout_ms = 10000; +const int HiveMetastoreClientFactory::recv_timeout_ms = 10000; +const int HiveMetastoreClientFactory::send_timeout_ms = 10000; +std::shared_ptr HiveMetastoreClientFactory::createThriftHiveMetastoreClient(const String &name) +{ + Poco::URI hive_metastore_url(name); + const auto & host = hive_metastore_url.getHost(); + auto port = hive_metastore_url.getPort(); + + std::shared_ptr socket = std::make_shared(host, port); + socket->setKeepAlive(true); + socket->setConnTimeout(conn_timeout_ms); + socket->setRecvTimeout(recv_timeout_ms); + socket->setSendTimeout(send_timeout_ms); + std::shared_ptr transport(new TBufferedTransport(socket)); + std::shared_ptr protocol(new TBinaryProtocol(transport)); + std::shared_ptr thrift_client = std::make_shared(protocol); + try + { + transport->open(); + } + catch (TException & tx) + { + throw Exception("connect to hive metastore:" + name + " failed." + tx.what(), ErrorCodes::BAD_ARGUMENTS); + } + return thrift_client; +} } #endif diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index e88e67b0257..ec3d0b052b4 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -1,5 +1,6 @@ #pragma once +#include #include #if USE_HIVE @@ -10,12 +11,40 @@ #include #include +#include #include namespace DB { +using ThriftHiveMetastoreClientBuilder = std::function()>; + +class ThriftHiveMetastoreClientPool : public PoolBase +{ +public: + using Object = Apache::Hadoop::Hive::ThriftHiveMetastoreClient; + using ObjectPtr = std::shared_ptr; + using Entry = PoolBase::Entry; + explicit ThriftHiveMetastoreClientPool(ThriftHiveMetastoreClientBuilder builder_) + : PoolBase(max_connections, &Poco::Logger::get("ThriftHiveMetastoreClientPool")) + , builder(builder_) + { + + } + +protected: + ObjectPtr allocObject() override + { + return builder(); + } + +private: + ThriftHiveMetastoreClientBuilder builder; + + const static unsigned max_connections; + +}; class HiveMetastoreClient : public WithContext { public: @@ -26,7 +55,9 @@ public: UInt64 last_modify_time; /// In ms size_t size; - FileInfo() = default; + explicit FileInfo() = default; + FileInfo & operator = (const FileInfo &) = default; + FileInfo(const FileInfo &) = default; FileInfo(const String & path_, UInt64 last_modify_time_, size_t size_) : path(path_), last_modify_time(last_modify_time_), size(size_) { @@ -94,17 +125,18 @@ public: using HiveTableMetadataPtr = std::shared_ptr; - explicit HiveMetastoreClient(std::shared_ptr client_, ContextPtr context_) - : WithContext(context_), client(client_), table_metadata_cache(1000) + explicit HiveMetastoreClient(ThriftHiveMetastoreClientBuilder builder_, ContextPtr context_) + : WithContext(context_) + , table_metadata_cache(1000) + , client_pool(builder_) { } + HiveTableMetadataPtr getTableMetadata(const String & db_name, const String & table_name); + // Access hive table information by hive client + std::shared_ptr getHiveTable(const String & db_name, const String & table_name); void clearTableMetadata(const String & db_name, const String & table_name); - void setClient(std::shared_ptr client_); - bool isExpired() const { return expired; } - void setExpired() { expired = true; } - void clearExpired() { expired = false; } private: static String getCacheKey(const String & db_name, const String & table_name) { return db_name + "." + table_name; } @@ -112,12 +144,15 @@ private: bool shouldUpdateTableMetadata( const String & db_name, const String & table_name, const std::vector & partitions); - std::shared_ptr client; + void tryCallHiveClient(std::function func); + LRUCache table_metadata_cache; - mutable std::mutex mutex; - std::atomic expired{false}; + ThriftHiveMetastoreClientPool client_pool; Poco::Logger * log = &Poco::Logger::get("HiveMetastoreClient"); + + const int max_retry = 3; + const UInt64 get_client_timeout = 1000000; }; using HiveMetastoreClientPtr = std::shared_ptr; @@ -128,13 +163,15 @@ public: HiveMetastoreClientPtr getOrCreate(const String & name, ContextPtr context); + static std::shared_ptr createThriftHiveMetastoreClient(const String & name); + private: std::mutex mutex; std::map clients; - const int conn_timeout_ms = 10000; - const int recv_timeout_ms = 10000; - const int send_timeout_ms = 10000; + const static int conn_timeout_ms; + const static int recv_timeout_ms; + const static int send_timeout_ms; }; } From c5e02be44ede2d96466981f07fc54bcddf150f60 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 28 Feb 2022 15:22:54 +0800 Subject: [PATCH 025/165] fixed code-style --- src/Storages/Hive/HiveCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 4000e5b8981..80110a517b9 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -47,7 +47,7 @@ bool HiveMetastoreClient::shouldUpdateTableMetadata( void HiveMetastoreClient::tryCallHiveClient(std::function func) { int i = 0; - String err_msg; + String err_msg; for (; i < max_retry; ++i) { auto client = client_pool.get(get_client_timeout); From 5885cfd86953487d1f7764ff87ac9c63c03f9b5c Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 28 Feb 2022 16:47:50 +0800 Subject: [PATCH 026/165] fixed bug : unexpected result when using in clause for filtering partitions --- .../integration/hive_server/prepare_hive_data.sh | 5 ++++- .../runner/compose/docker_compose_hive.yml | 2 +- src/Storages/Hive/StorageHive.h | 2 +- tests/integration/test_hive_query/test.py | 12 +++++++++++- 4 files changed, 17 insertions(+), 4 deletions(-) diff --git a/docker/test/integration/hive_server/prepare_hive_data.sh b/docker/test/integration/hive_server/prepare_hive_data.sh index afecbb91c5d..fa67f5dbb77 100755 --- a/docker/test/integration/hive_server/prepare_hive_data.sh +++ b/docker/test/integration/hive_server/prepare_hive_data.sh @@ -2,5 +2,8 @@ hive -e "create database test" hive -e "create table test.demo(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'; create table test.demo_orc(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'; " +hive -e "create table test.parquet_demo(id string, score int) PARTITIONED BY(day string, hour) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'" hive -e "create table test.demo_text(id string, score int, day string)row format delimited fields terminated by ','; load data local inpath '/demo_data.txt' into table test.demo_text " - hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.demo partition(day) select * from test.demo_text; insert into test.demo_orc partition(day) select * from test.demo_text" +hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.demo partition(day) select * from test.demo_text; insert into test.demo_orc partition(day) select * from test.demo_text" +hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '00' as hour from test.demo;" +hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '01' as hour from test.demo;" diff --git a/docker/test/integration/runner/compose/docker_compose_hive.yml b/docker/test/integration/runner/compose/docker_compose_hive.yml index 44f23655d2a..459e8481d0b 100644 --- a/docker/test/integration/runner/compose/docker_compose_hive.yml +++ b/docker/test/integration/runner/compose/docker_compose_hive.yml @@ -1,7 +1,7 @@ version: '2.3' services: hdfs1: - image: lgboustc/hive_test:v1.0 + image: lgboustc/hive_test:v2.0 hostname: hivetest restart: always entrypoint: bash /start.sh diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 9629629e057..1377e4a4504 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -36,7 +36,7 @@ public: ContextPtr /* query_context */, const StorageMetadataPtr & /* metadata_snapshot */) const override { - return false; + return true; } diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index a68ae0b066d..3e89bc418d8 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -30,7 +30,8 @@ def test_create_parquet_table(started_cluster): node.query("set input_format_parquet_allow_missing_columns = true") result = node.query(""" DROP TABLE IF EXISTS default.demo_parquet; - CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) + CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day); + CREATE TABLE default.demo_parquet_parts (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String), `hour` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'parquet_demo') PARTITION BY(day, hour); """) logging.info("create result {}".format(result)) time.sleep(120) @@ -70,6 +71,15 @@ def test_parquet_groupby(started_cluster): 2021-11-16 2 """ assert result == expected_result + +def test_parquet_in_filter(started_cluster): + logging.info('Start testing groupby ...') + node = started_cluster.instances['h0_0_0'] + result = node.query(""" + SELECT day, count(*) FROM default.demo_parquet_parts where day = '2021-11-05' and hour in ('00') + """) + expected_result = """2021-11-05 2""" + assert result == expected_result def test_orc_groupby(started_cluster): logging.info('Start testing groupby ...') node = started_cluster.instances['h0_0_0'] From 6473767c9972faa481fafa716fba7326e3e25622 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 28 Feb 2022 17:10:56 +0800 Subject: [PATCH 027/165] fixed code style --- src/Storages/Hive/HiveCommon.cpp | 27 +++++++++++++++++---------- src/Storages/Hive/HiveCommon.h | 17 +---------------- 2 files changed, 18 insertions(+), 26 deletions(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 80110a517b9..ac4abfb0f3d 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -17,7 +17,17 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -const unsigned ThriftHiveMetastoreClientPool::max_connections = 16; +static const unsigned max_hive_metastore_client_connections = 16; +static const int max_hive_metastore_client_retry = 3; +static const UInt64 get_hive_metastore_client_timeout = 1000000; +static const int hive_metastore_client_conn_timeout_ms = 10000; +static const int hive_metastore_client_recv_timeout_ms = 10000; +static const int hive_metastore_client_send_timeout_ms = 10000; + +ThriftHiveMetastoreClientPool::ThriftHiveMetastoreClientPool(ThriftHiveMetastoreClientBuilder builder_) + : PoolBase(max_hive_metastore_client_connections, &Poco::Logger::get("ThriftHiveMetastoreClientPool")), builder(builder_) +{ +} bool HiveMetastoreClient::shouldUpdateTableMetadata( const String & db_name, const String & table_name, const std::vector & partitions) @@ -48,9 +58,9 @@ void HiveMetastoreClient::tryCallHiveClient(std::function= max_retry) + if (i >= max_hive_metastore_client_retry) throw Exception(ErrorCodes::NO_HIVEMETASTORE, "Hive Metastore expired because {}", err_msg); } @@ -238,9 +248,6 @@ HiveMetastoreClientPtr HiveMetastoreClientFactory::getOrCreate(const String & na } return it->second; } -const int HiveMetastoreClientFactory::conn_timeout_ms = 10000; -const int HiveMetastoreClientFactory::recv_timeout_ms = 10000; -const int HiveMetastoreClientFactory::send_timeout_ms = 10000; std::shared_ptr HiveMetastoreClientFactory::createThriftHiveMetastoreClient(const String &name) { Poco::URI hive_metastore_url(name); @@ -249,9 +256,9 @@ std::shared_ptr HiveMetastoreClientFactory::createThr std::shared_ptr socket = std::make_shared(host, port); socket->setKeepAlive(true); - socket->setConnTimeout(conn_timeout_ms); - socket->setRecvTimeout(recv_timeout_ms); - socket->setSendTimeout(send_timeout_ms); + socket->setConnTimeout(hive_metastore_client_conn_timeout_ms); + socket->setRecvTimeout(hive_metastore_client_recv_timeout_ms); + socket->setSendTimeout(hive_metastore_client_send_timeout_ms); std::shared_ptr transport(new TBufferedTransport(socket)); std::shared_ptr protocol(new TBinaryProtocol(transport)); std::shared_ptr thrift_client = std::make_shared(protocol); diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index ec3d0b052b4..b8075457a02 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -26,12 +26,7 @@ public: using Object = Apache::Hadoop::Hive::ThriftHiveMetastoreClient; using ObjectPtr = std::shared_ptr; using Entry = PoolBase::Entry; - explicit ThriftHiveMetastoreClientPool(ThriftHiveMetastoreClientBuilder builder_) - : PoolBase(max_connections, &Poco::Logger::get("ThriftHiveMetastoreClientPool")) - , builder(builder_) - { - - } + explicit ThriftHiveMetastoreClientPool(ThriftHiveMetastoreClientBuilder builder_); protected: ObjectPtr allocObject() override @@ -41,9 +36,6 @@ protected: private: ThriftHiveMetastoreClientBuilder builder; - - const static unsigned max_connections; - }; class HiveMetastoreClient : public WithContext { @@ -150,9 +142,6 @@ private: ThriftHiveMetastoreClientPool client_pool; Poco::Logger * log = &Poco::Logger::get("HiveMetastoreClient"); - - const int max_retry = 3; - const UInt64 get_client_timeout = 1000000; }; using HiveMetastoreClientPtr = std::shared_ptr; @@ -168,10 +157,6 @@ public: private: std::mutex mutex; std::map clients; - - const static int conn_timeout_ms; - const static int recv_timeout_ms; - const static int send_timeout_ms; }; } From afad4bf4144f35a5773d939205c3b419eef57153 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 28 Feb 2022 17:14:56 +0800 Subject: [PATCH 028/165] update test --- tests/integration/test_hive_query/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index 3e89bc418d8..3be42f30473 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -76,7 +76,7 @@ def test_parquet_in_filter(started_cluster): logging.info('Start testing groupby ...') node = started_cluster.instances['h0_0_0'] result = node.query(""" - SELECT day, count(*) FROM default.demo_parquet_parts where day = '2021-11-05' and hour in ('00') + SELECT count(*) FROM default.demo_parquet_parts where day = '2021-11-05' and hour in ('00') """) expected_result = """2021-11-05 2""" assert result == expected_result From 53190ed4d43b939277ae71ef6a2bfd8b00831251 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 28 Feb 2022 17:40:08 +0800 Subject: [PATCH 029/165] updat test --- tests/integration/test_hive_query/test.py | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index 3be42f30473..5f7f5aa61ca 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -31,6 +31,17 @@ def test_create_parquet_table(started_cluster): result = node.query(""" DROP TABLE IF EXISTS default.demo_parquet; CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day); + """) + logging.info("create result {}".format(result)) + time.sleep(120) + assert result.strip() == '' + +def test_create_parquet_table_1(started_cluster): + logging.info('Start testing creating hive table ...') + node = started_cluster.instances['h0_0_0'] + node.query("set input_format_parquet_allow_missing_columns = true") + result = node.query(""" + DROP TABLE IF EXISTS default.demo_parquet_parts; CREATE TABLE default.demo_parquet_parts (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String), `hour` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'parquet_demo') PARTITION BY(day, hour); """) logging.info("create result {}".format(result)) @@ -78,7 +89,7 @@ def test_parquet_in_filter(started_cluster): result = node.query(""" SELECT count(*) FROM default.demo_parquet_parts where day = '2021-11-05' and hour in ('00') """) - expected_result = """2021-11-05 2""" + expected_result = """2""" assert result == expected_result def test_orc_groupby(started_cluster): logging.info('Start testing groupby ...') From aa8db563d684a9595d20422a029201996bf95469 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 28 Feb 2022 18:01:47 +0800 Subject: [PATCH 030/165] update test --- docker/test/integration/hive_server/Dockerfile | 1 + docker/test/integration/hive_server/prepare_hive_data.sh | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/integration/hive_server/Dockerfile b/docker/test/integration/hive_server/Dockerfile index fa6e4bf6313..4056f8aee45 100644 --- a/docker/test/integration/hive_server/Dockerfile +++ b/docker/test/integration/hive_server/Dockerfile @@ -38,6 +38,7 @@ COPY hadoop-env.sh /hadoop-3.1.0/etc/hadoop/ #COPY core-site.xml /hadoop-3.1.0/etc/hadoop COPY core-site.xml.template /hadoop-3.1.0/etc/hadoop COPY hive-site.xml /apache-hive-2.3.9-bin/conf +RUN echo "123" COPY prepare_hive_data.sh / COPY demo_data.txt / diff --git a/docker/test/integration/hive_server/prepare_hive_data.sh b/docker/test/integration/hive_server/prepare_hive_data.sh index fa67f5dbb77..b5ba2a08194 100755 --- a/docker/test/integration/hive_server/prepare_hive_data.sh +++ b/docker/test/integration/hive_server/prepare_hive_data.sh @@ -2,7 +2,7 @@ hive -e "create database test" hive -e "create table test.demo(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'; create table test.demo_orc(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'; " -hive -e "create table test.parquet_demo(id string, score int) PARTITIONED BY(day string, hour) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'" +hive -e "create table test.parquet_demo(id string, score int) PARTITIONED BY(day string, hour string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'" hive -e "create table test.demo_text(id string, score int, day string)row format delimited fields terminated by ','; load data local inpath '/demo_data.txt' into table test.demo_text " hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.demo partition(day) select * from test.demo_text; insert into test.demo_orc partition(day) select * from test.demo_text" hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '00' as hour from test.demo;" From b8a2deba0f95847d40a4a8e223597baa3432b7e5 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 28 Feb 2022 20:11:59 +0800 Subject: [PATCH 031/165] update tests --- docker/test/integration/hive_server/prepare_hive_data.sh | 1 + tests/integration/test_hive_query/test.py | 5 ++++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docker/test/integration/hive_server/prepare_hive_data.sh b/docker/test/integration/hive_server/prepare_hive_data.sh index b5ba2a08194..8126b975612 100755 --- a/docker/test/integration/hive_server/prepare_hive_data.sh +++ b/docker/test/integration/hive_server/prepare_hive_data.sh @@ -5,5 +5,6 @@ hive -e "create table test.demo(id string, score int) PARTITIONED BY(day string) hive -e "create table test.parquet_demo(id string, score int) PARTITIONED BY(day string, hour string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'" hive -e "create table test.demo_text(id string, score int, day string)row format delimited fields terminated by ','; load data local inpath '/demo_data.txt' into table test.demo_text " hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.demo partition(day) select * from test.demo_text; insert into test.demo_orc partition(day) select * from test.demo_text" + hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '00' as hour from test.demo;" hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '01' as hour from test.demo;" diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index 5f7f5aa61ca..621614813da 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -89,7 +89,9 @@ def test_parquet_in_filter(started_cluster): result = node.query(""" SELECT count(*) FROM default.demo_parquet_parts where day = '2021-11-05' and hour in ('00') """) - expected_result = """2""" + expected_result = """2 +""" + logging.info("query result:{}".format(result)) assert result == expected_result def test_orc_groupby(started_cluster): logging.info('Start testing groupby ...') @@ -150,4 +152,5 @@ def test_cache_read_bytes(started_cluster): assert result == expected_result result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") logging.info("Read bytes from cache:{}".format(result)) + assert result.strip() != '0' From dbb29dd3944be9af6e809086b4134a9d58ea33b4 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 28 Feb 2022 12:17:44 +0000 Subject: [PATCH 032/165] Fix tests --- tests/clickhouse-test | 2 +- tests/queries/0_stateless/00505_secure.sh | 1 + tests/queries/0_stateless/01339_client_unrecognized_option.sh | 2 +- .../1_stateful/00168_parallel_processing_on_replicas_part_1.sh | 2 +- 4 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a729a216f9a..a990e0c2986 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -447,7 +447,7 @@ class TestCase: os.environ['CLICKHOUSE_URL_PARAMS'] = self.base_url_params + '&' + '&'.join(self.random_settings) new_options = " --" + " --".join(self.random_settings) - os.environ['CLICKHOUSE_CLIENT_OPT'] = self.base_client_options + new_options + os.environ['CLICKHOUSE_CLIENT_OPT'] = self.base_client_options + new_options + ' ' return client_options + new_options def remove_random_settings_from_env(self): diff --git a/tests/queries/0_stateless/00505_secure.sh b/tests/queries/0_stateless/00505_secure.sh index e69515253ed..a8bc2a0f44a 100755 --- a/tests/queries/0_stateless/00505_secure.sh +++ b/tests/queries/0_stateless/00505_secure.sh @@ -7,6 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +echo $CLICKHOUSE_CLIENT_SECURE $CLICKHOUSE_CLIENT_SECURE -q "SELECT 2;" #disable test diff --git a/tests/queries/0_stateless/01339_client_unrecognized_option.sh b/tests/queries/0_stateless/01339_client_unrecognized_option.sh index 00c153ec915..9f827ccb13e 100755 --- a/tests/queries/0_stateless/01339_client_unrecognized_option.sh +++ b/tests/queries/0_stateless/01339_client_unrecognized_option.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-random-settings CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh index 699700bcd3e..4b1a03e8722 100755 --- a/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh +++ b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan +# Tags: no-tsan, no-ranfom-settings CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 24caa70243c237ccbc88555b6f4d964ccf7c0e9f Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 28 Feb 2022 20:23:00 +0800 Subject: [PATCH 033/165] remove meaningless codes --- docker/test/integration/hive_server/Dockerfile | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/test/integration/hive_server/Dockerfile b/docker/test/integration/hive_server/Dockerfile index 4056f8aee45..fa6e4bf6313 100644 --- a/docker/test/integration/hive_server/Dockerfile +++ b/docker/test/integration/hive_server/Dockerfile @@ -38,7 +38,6 @@ COPY hadoop-env.sh /hadoop-3.1.0/etc/hadoop/ #COPY core-site.xml /hadoop-3.1.0/etc/hadoop COPY core-site.xml.template /hadoop-3.1.0/etc/hadoop COPY hive-site.xml /apache-hive-2.3.9-bin/conf -RUN echo "123" COPY prepare_hive_data.sh / COPY demo_data.txt / From 91a45d799e8e1af8a29e34cb040f081d6fe25210 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 1 Mar 2022 15:22:07 +0800 Subject: [PATCH 034/165] optimization for first time to read a random access readbuffer --- .../Cache/ExternalDataSourceCache.cpp | 79 +++++++++++++++++-- src/Storages/Cache/ExternalDataSourceCache.h | 10 ++- src/Storages/Cache/RemoteCacheController.cpp | 1 + src/Storages/Cache/RemoteCacheController.h | 17 ++++ src/Storages/Hive/StorageHive.cpp | 10 ++- 5 files changed, 106 insertions(+), 11 deletions(-) diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index 0f78c8d3511..0b7b5948cf4 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -27,7 +27,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -LocalFileHolder::LocalFileHolder(RemoteFileCacheType::MappedHolderPtr cache_controller) : file_cache_controller(std::move(cache_controller)) +LocalFileHolder::LocalFileHolder(RemoteFileCacheType::MappedHolderPtr cache_controller) + : file_cache_controller(std::move(cache_controller)), file_buffer(nullptr), original_readbuffer(nullptr), thread_pool(nullptr) { file_buffer = file_cache_controller->value().allocFile(); if (!file_buffer) @@ -35,18 +36,43 @@ LocalFileHolder::LocalFileHolder(RemoteFileCacheType::MappedHolderPtr cache_cont ErrorCodes::LOGICAL_ERROR, "Create file readbuffer failed. {}", file_cache_controller->value().getLocalPath().string()); } +LocalFileHolder::LocalFileHolder( + RemoteFileCacheType::MappedHolderPtr cache_controller, + std::unique_ptr original_readbuffer_, + BackgroundSchedulePool * thread_pool_) + : file_cache_controller(std::move(cache_controller)) + , file_buffer(nullptr) + , original_readbuffer(std::move(original_readbuffer_)) + , thread_pool(thread_pool_) +{ +} + +LocalFileHolder::~LocalFileHolder() +{ + if (original_readbuffer) + { + dynamic_cast(original_readbuffer.get())->seek(0, SEEK_SET); + file_cache_controller->value().startBackgroundDownload(std::move(original_readbuffer), *thread_pool); + } +} + RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory(buff_size) { } std::unique_ptr RemoteReadBuffer::create( - ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer, size_t buff_size) + ContextPtr context, + IRemoteFileMetadataPtr remote_file_metadata, + std::unique_ptr read_buffer, + size_t buff_size, + bool is_random_accessed) + { auto remote_path = remote_file_metadata->remote_path; auto remote_read_buffer = std::make_unique(buff_size); std::tie(remote_read_buffer->local_file_holder, read_buffer) - = ExternalDataSourceCache::instance().createReader(context, remote_file_metadata, read_buffer); + = ExternalDataSourceCache::instance().createReader(context, remote_file_metadata, read_buffer, is_random_accessed); if (remote_read_buffer->local_file_holder == nullptr) return read_buffer; remote_read_buffer->remote_file_size = remote_file_metadata->file_size; @@ -55,6 +81,19 @@ std::unique_ptr RemoteReadBuffer::create( bool RemoteReadBuffer::nextImpl() { + if (local_file_holder->original_readbuffer) + { + auto status = local_file_holder->original_readbuffer->next(); + if (status) + { + BufferBase::set( + local_file_holder->original_readbuffer->buffer().begin(), + local_file_holder->original_readbuffer->buffer().size(), + local_file_holder->original_readbuffer->offset()); + } + return status; + } + auto start_offset = local_file_holder->file_buffer->getPosition(); auto end_offset = start_offset + local_file_holder->file_buffer->internalBuffer().size(); local_file_holder->file_cache_controller->value().waitMoreData(start_offset, end_offset); @@ -73,6 +112,16 @@ bool RemoteReadBuffer::nextImpl() off_t RemoteReadBuffer::seek(off_t offset, int whence) { + if (local_file_holder->original_readbuffer) + { + auto ret = dynamic_cast(local_file_holder->original_readbuffer.get())->seek(offset, whence); + BufferBase::set( + local_file_holder->original_readbuffer->buffer().begin(), + local_file_holder->original_readbuffer->buffer().size(), + local_file_holder->original_readbuffer->offset()); + return ret; + } + if (!local_file_holder->file_buffer) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot call seek() in this buffer. It's a bug!"); /* @@ -88,6 +137,10 @@ off_t RemoteReadBuffer::seek(off_t offset, int whence) off_t RemoteReadBuffer::getPosition() { + if (local_file_holder->original_readbuffer) + { + return dynamic_cast(local_file_holder->original_readbuffer.get())->getPosition(); + } return local_file_holder->file_buffer->getPosition(); } @@ -164,7 +217,7 @@ String ExternalDataSourceCache::calculateLocalPath(IRemoteFileMetadataPtr metada } std::pair, std::unique_ptr> ExternalDataSourceCache::createReader( - ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer) + ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer, bool is_random_accessed) { // If something is wrong on startup, rollback to read from the original ReadBuffer. if (!isInitialized()) @@ -180,7 +233,12 @@ std::pair, std::unique_ptr> Externa auto cache = lru_caches->get(local_path); if (cache) { - // The remote file has been updated, need to redownload. + if (!cache->value().isEnable()) + { + return {nullptr, std::move(read_buffer)}; + } + + // the remote file has been updated, need to redownload if (!cache->value().isValid() || cache->value().isModified(remote_file_metadata)) { LOG_TRACE( @@ -216,6 +274,17 @@ std::pair, std::unique_ptr> Externa lru_caches->weight()); return {nullptr, std::move(read_buffer)}; } + /* + If read_buffer is seekable, use read_buffer directly inside LocalFileHolder. And once LocalFileHolder is released, + start the download process in background. + The cache is marked disable until the download process finish. + For reading parquet files from hdfs, with this optimization, the speedup can reach 3x. + */ + if (dynamic_cast(read_buffer.get()) && is_random_accessed) + { + new_cache->value().disable(); + return {std::make_unique(std::move(new_cache), std::move(read_buffer), &context->getSchedulePool()), nullptr}; + } new_cache->value().startBackgroundDownload(std::move(read_buffer), context->getSchedulePool()); return {std::make_unique(std::move(new_cache)), nullptr}; } diff --git a/src/Storages/Cache/ExternalDataSourceCache.h b/src/Storages/Cache/ExternalDataSourceCache.h index c555198e4c4..a25686b49c1 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.h +++ b/src/Storages/Cache/ExternalDataSourceCache.h @@ -34,10 +34,13 @@ class LocalFileHolder { public: explicit LocalFileHolder(RemoteFileCacheType::MappedHolderPtr cache_controller); - ~LocalFileHolder() = default; + explicit LocalFileHolder(RemoteFileCacheType::MappedHolderPtr cache_controller, std::unique_ptr original_readbuffer_, BackgroundSchedulePool * thread_pool_); + ~LocalFileHolder(); RemoteFileCacheType::MappedHolderPtr file_cache_controller; std::unique_ptr file_buffer; + std::unique_ptr original_readbuffer; + BackgroundSchedulePool * thread_pool; }; class RemoteReadBuffer : public BufferWithOwnMemory @@ -45,7 +48,7 @@ class RemoteReadBuffer : public BufferWithOwnMemory public: explicit RemoteReadBuffer(size_t buff_size); ~RemoteReadBuffer() override = default; - static std::unique_ptr create(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer, size_t buff_size); + static std::unique_ptr create(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer, size_t buff_size, bool is_random_accessed = false); bool nextImpl() override; off_t seek(off_t off, int whence) override; @@ -70,7 +73,8 @@ public: inline bool isInitialized() const { return initialized; } std::pair, std::unique_ptr> - createReader(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer); + createReader(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer, bool is_random_accessed); + void updateTotalSize(size_t size) { total_size += size; } diff --git a/src/Storages/Cache/RemoteCacheController.cpp b/src/Storages/Cache/RemoteCacheController.cpp index b5fc38fffcd..b72f5336ea4 100644 --- a/src/Storages/Cache/RemoteCacheController.cpp +++ b/src/Storages/Cache/RemoteCacheController.cpp @@ -169,6 +169,7 @@ void RemoteCacheController::backgroundDownload(ReadBufferPtr remote_read_buffer) file_status = DOWNLOADED; flush(true); data_file_writer.reset(); + is_enable = true; lock.unlock(); more_data_signal.notify_all(); ExternalDataSourceCache::instance().updateTotalSize(file_metadata_ptr->file_size); diff --git a/src/Storages/Cache/RemoteCacheController.h b/src/Storages/Cache/RemoteCacheController.h index ca2cb837e34..4e122565641 100644 --- a/src/Storages/Cache/RemoteCacheController.h +++ b/src/Storages/Cache/RemoteCacheController.h @@ -62,6 +62,22 @@ public: { std::lock_guard lock(mutex); return valid; + } + inline bool isEnable() + { + std::lock_guard lock(mutex); + return is_enable; + + } + inline void disable() + { + std::lock_guard lock(mutex); + is_enable = false; + } + inline void enable() + { + std::lock_guard lock(mutex); + is_enable = true; } IRemoteFileMetadataPtr getFileMetadata() { return file_metadata_ptr; } inline size_t getFileSize() const { return file_metadata_ptr->file_size; } @@ -83,6 +99,7 @@ private: IRemoteFileMetadataPtr file_metadata_ptr; std::filesystem::path local_path; + bool is_enable = true; bool valid = true; size_t local_cache_bytes_read_before_flush; size_t current_offset; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 3040ad23283..b0d3ae02d9a 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -171,9 +171,13 @@ public: size_t buff_size = raw_read_buf->internalBuffer().size(); if (buff_size == 0) buff_size = DBMS_DEFAULT_BUFFER_SIZE; - remote_read_buf = RemoteReadBuffer::create(getContext(), - std::make_shared("Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getSize(), curr_file->getLastModTs()), - std::move(raw_read_buf), buff_size); + remote_read_buf = RemoteReadBuffer::create( + getContext(), + std::make_shared( + "Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getSize(), curr_file->getLastModTs()), + std::move(raw_read_buf), + buff_size, + format == "Parquet" || format == "ORC"); } else remote_read_buf = std::move(raw_read_buf); From 6e568c1530bc1b234157da6952069e0e7f2e616b Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 1 Mar 2022 15:24:40 +0800 Subject: [PATCH 035/165] update codes --- src/Storages/Cache/ExternalDataSourceCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index 0b7b5948cf4..2d754b252e9 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -238,7 +238,7 @@ std::pair, std::unique_ptr> Externa return {nullptr, std::move(read_buffer)}; } - // the remote file has been updated, need to redownload + // The remote file has been updated, need to redownload. if (!cache->value().isValid() || cache->value().isModified(remote_file_metadata)) { LOG_TRACE( From 5ae99df87cd6a165e17e83ae4250ab10ecee5bf2 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 1 Mar 2022 15:35:57 +0800 Subject: [PATCH 036/165] fxied code style --- src/Storages/Cache/RemoteCacheController.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Cache/RemoteCacheController.h b/src/Storages/Cache/RemoteCacheController.h index 4e122565641..95f46f13bdc 100644 --- a/src/Storages/Cache/RemoteCacheController.h +++ b/src/Storages/Cache/RemoteCacheController.h @@ -63,7 +63,7 @@ public: std::lock_guard lock(mutex); return valid; } - inline bool isEnable() + inline bool isEnable() { std::lock_guard lock(mutex); return is_enable; From f83132bad29fb44ad0ab0bb79664f44cb6e6fb85 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 1 Mar 2022 15:54:23 +0800 Subject: [PATCH 037/165] finish dev --- src/Client/ClientBase.cpp | 2 +- src/Client/LocalConnection.cpp | 80 +---------------------- src/Interpreters/ProfileEventsExt.cpp | 94 ++++++++++++++++++++++++++- src/Interpreters/ProfileEventsExt.h | 10 +-- src/Server/TCPHandler.cpp | 87 +------------------------ 5 files changed, 102 insertions(+), 171 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 7dfa60ad560..b93fef04be4 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -866,7 +866,7 @@ void ClientBase::onProfileEvents(Block & block) if (rows == 0) return; - if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS) + if (getName() == "local" || server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS) { const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); const auto & names = typeid_cast(*block.getByName("name").column); diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 08ba485d5fc..02b437d7ce6 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -8,7 +8,6 @@ #include #include #include -#include namespace DB @@ -64,84 +63,7 @@ void LocalConnection::updateProgress(const Progress & value) void LocalConnection::getProfileEvents(Block & block) { - using namespace ProfileEvents; - - static const NamesAndTypesList column_names_and_types = { - {"host_name", std::make_shared()}, - {"current_time", std::make_shared()}, - {"thread_id", std::make_shared()}, - {"type", TypeEnum}, - {"name", std::make_shared()}, - {"value", std::make_shared()}, - }; - - ColumnsWithTypeAndName temp_columns; - for (auto const & name_and_type : column_names_and_types) - temp_columns.emplace_back(name_and_type.type, name_and_type.name); - - block = Block(std::move(temp_columns)); - MutableColumns columns = block.mutateColumns(); - auto thread_group = CurrentThread::getGroup(); - auto const current_thread_id = CurrentThread::get().thread_id; - std::vector snapshots; - ThreadIdToCountersSnapshot new_snapshots; - ProfileEventsSnapshot group_snapshot; - { - auto stats = thread_group->getProfileEventsCountersAndMemoryForThreads(); - snapshots.reserve(stats.size()); - - for (auto & stat : stats) - { - auto const thread_id = stat.thread_id; - if (thread_id == current_thread_id) - continue; - auto current_time = time(nullptr); - auto previous_snapshot = last_sent_snapshots.find(thread_id); - auto increment = - previous_snapshot != last_sent_snapshots.end() - ? CountersIncrement(stat.counters, previous_snapshot->second) - : CountersIncrement(stat.counters); - snapshots.push_back(ProfileEventsSnapshot{ - thread_id, - std::move(increment), - stat.memory_usage, - current_time - }); - new_snapshots[thread_id] = std::move(stat.counters); - } - - group_snapshot.thread_id = 0; - group_snapshot.current_time = time(nullptr); - group_snapshot.memory_usage = thread_group->memory_tracker.get(); - auto group_counters = thread_group->performance_counters.getPartiallyAtomicSnapshot(); - auto prev_group_snapshot = last_sent_snapshots.find(0); - group_snapshot.counters = - prev_group_snapshot != last_sent_snapshots.end() - ? CountersIncrement(group_counters, prev_group_snapshot->second) - : CountersIncrement(group_counters); - new_snapshots[0] = std::move(group_counters); - } - last_sent_snapshots = std::move(new_snapshots); - - const String server_display_name = "localhost"; - for (auto & snapshot : snapshots) - { - dumpProfileEvents(snapshot, columns, server_display_name); - dumpMemoryTracker(snapshot, columns, server_display_name); - } - dumpProfileEvents(group_snapshot, columns, server_display_name); - dumpMemoryTracker(group_snapshot, columns, server_display_name); - - MutableColumns logs_columns; - Block curr_block; - size_t rows = 0; - - for (; state->profile_queue->tryPop(curr_block); ++rows) - { - auto curr_columns = curr_block.getColumns(); - for (size_t j = 0; j < curr_columns.size(); ++j) - columns[j]->insertRangeFrom(*curr_columns[j], 0, curr_columns[j]->size()); - } + ProfileEvents::getProfileEvents("local", state->profile_queue, block, last_sent_snapshots); } void LocalConnection::sendQuery( diff --git a/src/Interpreters/ProfileEventsExt.cpp b/src/Interpreters/ProfileEventsExt.cpp index 173df507c65..6961d70529e 100644 --- a/src/Interpreters/ProfileEventsExt.cpp +++ b/src/Interpreters/ProfileEventsExt.cpp @@ -1,6 +1,7 @@ #include "ProfileEventsExt.h" #include #include +#include #include #include #include @@ -46,8 +47,8 @@ void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, offsets.push_back(offsets.back() + size); } - -void dumpProfileEvents(ProfileEventsSnapshot const & snapshot, DB::MutableColumns & columns, String const & host_name) +/// Add records about provided non-zero ProfileEvents::Counters. +static void dumpProfileEvents(ProfileEventsSnapshot const & snapshot, DB::MutableColumns & columns, String const & host_name) { size_t rows = 0; auto & name_column = columns[NAME_COLUMN_INDEX]; @@ -76,7 +77,7 @@ void dumpProfileEvents(ProfileEventsSnapshot const & snapshot, DB::MutableColumn } } -void dumpMemoryTracker(ProfileEventsSnapshot const & snapshot, DB::MutableColumns & columns, String const & host_name) +static void dumpMemoryTracker(ProfileEventsSnapshot const & snapshot, DB::MutableColumns & columns, String const & host_name) { { size_t i = 0; @@ -90,4 +91,91 @@ void dumpMemoryTracker(ProfileEventsSnapshot const & snapshot, DB::MutableColumn } } +void getProfileEvents( + const String & server_display_name, + DB::InternalProfileEventsQueuePtr profile_queue, + DB::Block & block, + ThreadIdToCountersSnapshot & last_sent_snapshots) +{ + using namespace DB; + static const NamesAndTypesList column_names_and_types = { + {"host_name", std::make_shared()}, + {"current_time", std::make_shared()}, + {"thread_id", std::make_shared()}, + {"type", TypeEnum}, + {"name", std::make_shared()}, + {"value", std::make_shared()}, + }; + + ColumnsWithTypeAndName temp_columns; + for (auto const & name_and_type : column_names_and_types) + temp_columns.emplace_back(name_and_type.type, name_and_type.name); + + block = std::move(temp_columns); + MutableColumns columns = block.mutateColumns(); + auto thread_group = CurrentThread::getGroup(); + auto const current_thread_id = CurrentThread::get().thread_id; + std::vector snapshots; + ThreadIdToCountersSnapshot new_snapshots; + ProfileEventsSnapshot group_snapshot; + { + auto stats = thread_group->getProfileEventsCountersAndMemoryForThreads(); + snapshots.reserve(stats.size()); + + for (auto & stat : stats) + { + auto const thread_id = stat.thread_id; + if (thread_id == current_thread_id) + continue; + auto current_time = time(nullptr); + auto previous_snapshot = last_sent_snapshots.find(thread_id); + auto increment = + previous_snapshot != last_sent_snapshots.end() + ? CountersIncrement(stat.counters, previous_snapshot->second) + : CountersIncrement(stat.counters); + snapshots.push_back(ProfileEventsSnapshot{ + thread_id, + std::move(increment), + stat.memory_usage, + current_time + }); + new_snapshots[thread_id] = std::move(stat.counters); + } + + group_snapshot.thread_id = 0; + group_snapshot.current_time = time(nullptr); + group_snapshot.memory_usage = thread_group->memory_tracker.get(); + auto group_counters = thread_group->performance_counters.getPartiallyAtomicSnapshot(); + auto prev_group_snapshot = last_sent_snapshots.find(0); + group_snapshot.counters = + prev_group_snapshot != last_sent_snapshots.end() + ? CountersIncrement(group_counters, prev_group_snapshot->second) + : CountersIncrement(group_counters); + new_snapshots[0] = std::move(group_counters); + } + last_sent_snapshots = std::move(new_snapshots); + + for (auto & snapshot : snapshots) + { + dumpProfileEvents(snapshot, columns, server_display_name); + dumpMemoryTracker(snapshot, columns, server_display_name); + } + dumpProfileEvents(group_snapshot, columns, server_display_name); + dumpMemoryTracker(group_snapshot, columns, server_display_name); + + Block curr_block; + size_t rows = 0; + + for (; profile_queue->tryPop(curr_block); ++rows) + { + auto curr_columns = curr_block.getColumns(); + for (size_t j = 0; j < curr_columns.size(); ++j) + columns[j]->insertRangeFrom(*curr_columns[j], 0, curr_columns[j]->size()); + } + + bool empty = columns[0]->empty(); + if (!empty) + block.setColumns(std::move(columns)); +} + } diff --git a/src/Interpreters/ProfileEventsExt.h b/src/Interpreters/ProfileEventsExt.h index ebb6981405f..7d9fc512d15 100644 --- a/src/Interpreters/ProfileEventsExt.h +++ b/src/Interpreters/ProfileEventsExt.h @@ -1,5 +1,6 @@ #pragma once #include +#include #include #include @@ -23,10 +24,11 @@ using ThreadIdToCountersSnapshot = std::unordered_map #include #include -#include #include #include #include @@ -855,93 +854,13 @@ void TCPHandler::sendExtremes(const Block & extremes) void TCPHandler::sendProfileEvents() { - using namespace ProfileEvents; - if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS) return; - static const NamesAndTypesList column_names_and_types = { - { "host_name", std::make_shared() }, - { "current_time", std::make_shared() }, - { "thread_id", std::make_shared() }, - { "type", TypeEnum }, - { "name", std::make_shared() }, - { "value", std::make_shared() }, - }; - - ColumnsWithTypeAndName temp_columns; - for (auto const & name_and_type : column_names_and_types) - temp_columns.emplace_back(name_and_type.type, name_and_type.name); - - Block block(std::move(temp_columns)); - - MutableColumns columns = block.mutateColumns(); - auto thread_group = CurrentThread::getGroup(); - auto const current_thread_id = CurrentThread::get().thread_id; - std::vector snapshots; - ThreadIdToCountersSnapshot new_snapshots; - ProfileEventsSnapshot group_snapshot; + Block block; + ProfileEvents::getProfileEvents(server_display_name, state.profile_queue, block, last_sent_snapshots); + if (!!block.rows()) { - auto stats = thread_group->getProfileEventsCountersAndMemoryForThreads(); - snapshots.reserve(stats.size()); - - for (auto & stat : stats) - { - auto const thread_id = stat.thread_id; - if (thread_id == current_thread_id) - continue; - auto current_time = time(nullptr); - auto previous_snapshot = last_sent_snapshots.find(thread_id); - auto increment = - previous_snapshot != last_sent_snapshots.end() - ? CountersIncrement(stat.counters, previous_snapshot->second) - : CountersIncrement(stat.counters); - snapshots.push_back(ProfileEventsSnapshot{ - thread_id, - std::move(increment), - stat.memory_usage, - current_time - }); - new_snapshots[thread_id] = std::move(stat.counters); - } - - group_snapshot.thread_id = 0; - group_snapshot.current_time = time(nullptr); - group_snapshot.memory_usage = thread_group->memory_tracker.get(); - auto group_counters = thread_group->performance_counters.getPartiallyAtomicSnapshot(); - auto prev_group_snapshot = last_sent_snapshots.find(0); - group_snapshot.counters = - prev_group_snapshot != last_sent_snapshots.end() - ? CountersIncrement(group_counters, prev_group_snapshot->second) - : CountersIncrement(group_counters); - new_snapshots[0] = std::move(group_counters); - } - last_sent_snapshots = std::move(new_snapshots); - - for (auto & snapshot : snapshots) - { - dumpProfileEvents(snapshot, columns, server_display_name); - dumpMemoryTracker(snapshot, columns, server_display_name); - } - dumpProfileEvents(group_snapshot, columns, server_display_name); - dumpMemoryTracker(group_snapshot, columns, server_display_name); - - MutableColumns logs_columns; - Block curr_block; - size_t rows = 0; - - for (; state.profile_queue->tryPop(curr_block); ++rows) - { - auto curr_columns = curr_block.getColumns(); - for (size_t j = 0; j < curr_columns.size(); ++j) - columns[j]->insertRangeFrom(*curr_columns[j], 0, curr_columns[j]->size()); - } - - bool empty = columns[0]->empty(); - if (!empty) - { - block.setColumns(std::move(columns)); - initProfileEventsBlockOutput(block); writeVarUInt(Protocol::Server::ProfileEvents, *out); From 5ed41bda9b824d8de499e478a35cf3097ecde99e Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 1 Mar 2022 17:20:32 +0800 Subject: [PATCH 038/165] fixed code style --- src/Storages/Hive/HiveCommon.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index ac4abfb0f3d..a9d0c22d6a5 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -259,8 +259,8 @@ std::shared_ptr HiveMetastoreClientFactory::createThr socket->setConnTimeout(hive_metastore_client_conn_timeout_ms); socket->setRecvTimeout(hive_metastore_client_recv_timeout_ms); socket->setSendTimeout(hive_metastore_client_send_timeout_ms); - std::shared_ptr transport(new TBufferedTransport(socket)); - std::shared_ptr protocol(new TBinaryProtocol(transport)); + std::shared_ptr transport = std::make_shared(socket); + std::shared_ptr protocol = std::make_shared(transport); std::shared_ptr thrift_client = std::make_shared(protocol); try { @@ -268,7 +268,7 @@ std::shared_ptr HiveMetastoreClientFactory::createThr } catch (TException & tx) { - throw Exception("connect to hive metastore:" + name + " failed." + tx.what(), ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "connect to hive metastore: {} failed. {}", name, tx.what()); } return thrift_client; } From ca470e1b94aae83d9bf6edd0bb7aa7f58bbaecb8 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 1 Mar 2022 19:01:39 +0800 Subject: [PATCH 039/165] lazy initialization about getting hive metadata in HiveStorage --- src/Storages/Hive/StorageHive.cpp | 19 +++++++++++++++---- src/Storages/Hive/StorageHive.h | 5 +++++ 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 3040ad23283..63445c4a24c 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -286,14 +286,22 @@ StorageHive::StorageHive( storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment_); setInMemoryMetadata(storage_metadata); +} + +void StorageHive::lazyInitialize() +{ + std::lock_guard lock{init_mutex}; + if (has_initialized) + return; + auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); - auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); + auto hive_table_metadata = hive_metastore_client->getHiveTable(hive_database, hive_table); - hdfs_namenode_url = getNameNodeUrl(hive_table_metadata->getTable()->sd.location); - table_schema = hive_table_metadata->getTable()->sd.cols; + hdfs_namenode_url = getNameNodeUrl(hive_table_metadata->sd.location); + table_schema = hive_table_metadata->sd.cols; - FileFormat hdfs_file_format = IHiveFile::toFileFormat(hive_table_metadata->getTable()->sd.inputFormat); + FileFormat hdfs_file_format = IHiveFile::toFileFormat(hive_table_metadata->sd.inputFormat); switch (hdfs_file_format) { case FileFormat::TEXT: @@ -331,6 +339,7 @@ StorageHive::StorageHive( } initMinMaxIndexExpression(); + has_initialized = true; } void StorageHive::initMinMaxIndexExpression() @@ -552,6 +561,8 @@ Pipe StorageHive::read( size_t max_block_size, unsigned num_streams) { + lazyInitialize(); + HDFSBuilderWrapper builder = createHDFSBuilder(hdfs_namenode_url, context_->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 9629629e057..f6968ebb7c6 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -94,6 +94,9 @@ private: String hive_database; String hive_table; + std::mutex init_mutex; + bool has_initialized = false; + /// Hive table meta std::vector table_schema; Names text_input_field_names; /// Defines schema of hive file, only used when text input format is TEXT @@ -116,6 +119,8 @@ private: std::shared_ptr storage_settings; Poco::Logger * log = &Poco::Logger::get("StorageHive"); + + void lazyInitialize(); }; } From 2b77416bf135e6faded12295f31123c6618749c0 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 1 Mar 2022 11:35:17 +0000 Subject: [PATCH 040/165] Fix tests --- tests/queries/0_stateless/00505_secure.sh | 1 - tests/queries/0_stateless/02006_test_positional_arguments.sql | 1 + tests/queries/1_stateful/00084_external_aggregation.sql | 2 ++ .../1_stateful/00168_parallel_processing_on_replicas_part_1.sh | 2 +- 4 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00505_secure.sh b/tests/queries/0_stateless/00505_secure.sh index a8bc2a0f44a..e69515253ed 100755 --- a/tests/queries/0_stateless/00505_secure.sh +++ b/tests/queries/0_stateless/00505_secure.sh @@ -7,7 +7,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -echo $CLICKHOUSE_CLIENT_SECURE $CLICKHOUSE_CLIENT_SECURE -q "SELECT 2;" #disable test diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.sql b/tests/queries/0_stateless/02006_test_positional_arguments.sql index 54b55c4a9f8..7442ca6bbf6 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.sql +++ b/tests/queries/0_stateless/02006_test_positional_arguments.sql @@ -1,3 +1,4 @@ +set group_by_two_level_threshold = 100000; set enable_positional_arguments = 1; drop table if exists test; diff --git a/tests/queries/1_stateful/00084_external_aggregation.sql b/tests/queries/1_stateful/00084_external_aggregation.sql index b3922eae049..816d95f4b8b 100644 --- a/tests/queries/1_stateful/00084_external_aggregation.sql +++ b/tests/queries/1_stateful/00084_external_aggregation.sql @@ -1,3 +1,5 @@ +-- Tags: no-random-settings + SET max_bytes_before_external_group_by = 200000000; SET max_memory_usage = 1500000000; diff --git a/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh index 4b1a03e8722..276fc0274c2 100755 --- a/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh +++ b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-ranfom-settings +# Tags: no-tsan, no-random-settings CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 755e63ed0387aeca5726096895f6b4d0dd36d482 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 1 Mar 2022 19:21:59 +0100 Subject: [PATCH 041/165] Keep compatibility --- src/Storages/FileLog/StorageFileLog.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 604116808ab..48a729febb2 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -99,8 +99,18 @@ void StorageFileLog::loadMetaFiles(bool attach) /// Attach table if (attach) { + const auto & storage = getStorageID(); + + auto metadata_path_exist = std::filesystem::exists(metadata_base_path); + auto previous_path = std::filesystem::path(getContext()->getPath()) / ".filelog_storage_metadata" / storage.getDatabaseName() / storage.getTableName(); + + /// For compatibility with the previous path version. + if (std::filesystem::exists(previous_path) && !metadata_path_exist) + { + std::filesystem::copy(previous_path, metadata_base_path, std::filesystem::copy_options::recursive); + } /// Meta file may lost, log and create directory - if (!std::filesystem::exists(metadata_base_path)) + else if (!metadata_path_exist) { /// Create metadata_base_path directory when store meta data LOG_ERROR(log, "Metadata files of table {} are lost.", getStorageID().getTableName()); From 7efe4abe2a960fd9b3f9821af5b46e6ccd001eb3 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 2 Mar 2022 09:46:32 +0800 Subject: [PATCH 042/165] update tests --- tests/integration/test_hive_query/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index a68ae0b066d..0661a8cafc1 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -117,6 +117,7 @@ def test_cache_read_bytes(started_cluster): result = node.query(""" SELECT day, count(*) FROM default.demo_parquet group by day order by day """) + time.sleep(10) result = node.query(""" SELECT day, count(*) FROM default.demo_parquet group by day order by day """) @@ -128,5 +129,7 @@ def test_cache_read_bytes(started_cluster): time.sleep(120) assert result == expected_result result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") + time.sleep(10) + result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") logging.info("Read bytes from cache:{}".format(result)) assert result.strip() != '0' From 2ff7ff90fa6f6c4ad4f6bdb88eb755137ba10a85 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 2 Mar 2022 14:17:04 +0800 Subject: [PATCH 043/165] update tests --- .../test/integration/hive_server/Dockerfile | 5 +- .../hive_server/http_api_server.py | 72 +++++++++++++++++++ docker/test/integration/hive_server/start.sh | 3 +- 3 files changed, 77 insertions(+), 3 deletions(-) create mode 100644 docker/test/integration/hive_server/http_api_server.py diff --git a/docker/test/integration/hive_server/Dockerfile b/docker/test/integration/hive_server/Dockerfile index fa6e4bf6313..391f9a5e22f 100644 --- a/docker/test/integration/hive_server/Dockerfile +++ b/docker/test/integration/hive_server/Dockerfile @@ -42,6 +42,9 @@ COPY prepare_hive_data.sh / COPY demo_data.txt / ENV PATH=/apache-hive-2.3.9-bin/bin:/hadoop-3.1.0/bin:/hadoop-3.1.0/sbin:$PATH - +RUN service ssh start && sed s/HOSTNAME/$HOSTNAME/ /hadoop-3.1.0/etc/hadoop/core-site.xml.template > /hadoop-3.1.0/etc/hadoop/core-site.xml && hdfs namenode -format +RUN apt install -y python3 python3-pip +RUN pip3 install flask requests +COPY http_api_server.py / COPY start.sh / diff --git a/docker/test/integration/hive_server/http_api_server.py b/docker/test/integration/hive_server/http_api_server.py new file mode 100644 index 00000000000..74b511a3a35 --- /dev/null +++ b/docker/test/integration/hive_server/http_api_server.py @@ -0,0 +1,72 @@ +import os +import subprocess +import datetime +from flask import Flask, flash, request, redirect, url_for + +def run_command(command, wait=False): + print("{} - execute shell command:{}".format(datetime.datetime.now(), command)) + lines = [] + p = subprocess.Popen(command, + stdout=subprocess.PIPE, + stderr=subprocess.STDOUT, + shell=True) + if wait: + for l in iter(p.stdout.readline, b''): + lines.append(l) + p.poll() + return (lines, p.returncode) + else: + return(iter(p.stdout.readline, b''), 0) + + +UPLOAD_FOLDER = './' +ALLOWED_EXTENSIONS = {'txt', 'pdf', 'png', 'jpg', 'jpeg', 'gif', 'sh'} +app = Flask(__name__) +app.config['UPLOAD_FOLDER'] = UPLOAD_FOLDER + +@app.route('/') +def hello_world(): + return 'Hello World' + + +def allowed_file(filename): + return '.' in filename and \ + filename.rsplit('.', 1)[1].lower() in ALLOWED_EXTENSIONS + + +@app.route('/upload', methods=['GET', 'POST']) +def upload_file(): + if request.method == 'POST': + # check if the post request has the file part + if 'file' not in request.files: + flash('No file part') + return redirect(request.url) + file = request.files['file'] + # If the user does not select a file, the browser submits an + # empty file without a filename. + if file.filename == '': + flash('No selected file') + return redirect(request.url) + if file and allowed_file(file.filename): + filename = file.filename + file.save(os.path.join(app.config['UPLOAD_FOLDER'], filename)) + return redirect(url_for('upload_file', name=filename)) + return ''' + + Upload new File +

Upload new File

+
+ + +
+ ''' +@app.route('/run', methods=['GET', 'POST']) +def parse_request(): + data = request.data # data is empty + print(data) + run_command(data, wait=True) + return 'Ok' + # need posted data here + +if __name__ == '__main__': + app.run(port=5011) diff --git a/docker/test/integration/hive_server/start.sh b/docker/test/integration/hive_server/start.sh index e01f28542af..4224b8126e6 100755 --- a/docker/test/integration/hive_server/start.sh +++ b/docker/test/integration/hive_server/start.sh @@ -1,6 +1,5 @@ service ssh start sed s/HOSTNAME/$HOSTNAME/ /hadoop-3.1.0/etc/hadoop/core-site.xml.template > /hadoop-3.1.0/etc/hadoop/core-site.xml -hadoop namenode -format start-all.sh service mysql start mysql -u root -e "CREATE USER \"test\"@\"localhost\" IDENTIFIED BY \"test\"" @@ -9,4 +8,4 @@ schematool -initSchema -dbType mysql #nohup hiveserver2 & nohup hive --service metastore & bash /prepare_hive_data.sh -while true; do sleep 1000; done +python3 http_api_server.py From 21c164ad2c63d8d73f7bc78e173d7a26034524fe Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 2 Mar 2022 14:20:59 +0800 Subject: [PATCH 044/165] update tests --- docker/test/integration/hive_server/http_api_server.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docker/test/integration/hive_server/http_api_server.py b/docker/test/integration/hive_server/http_api_server.py index 74b511a3a35..4818b785c89 100644 --- a/docker/test/integration/hive_server/http_api_server.py +++ b/docker/test/integration/hive_server/http_api_server.py @@ -20,7 +20,7 @@ def run_command(command, wait=False): UPLOAD_FOLDER = './' -ALLOWED_EXTENSIONS = {'txt', 'pdf', 'png', 'jpg', 'jpeg', 'gif', 'sh'} +ALLOWED_EXTENSIONS = {'txt', 'sh'} app = Flask(__name__) app.config['UPLOAD_FOLDER'] = UPLOAD_FOLDER @@ -63,10 +63,8 @@ def upload_file(): @app.route('/run', methods=['GET', 'POST']) def parse_request(): data = request.data # data is empty - print(data) run_command(data, wait=True) return 'Ok' - # need posted data here if __name__ == '__main__': app.run(port=5011) From 53116faeeb99bc0bd74bbf5b281457524ef92a4e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 2 Mar 2022 11:08:35 +0100 Subject: [PATCH 045/165] Update MergeTreeIndexFullText.cpp --- src/Storages/MergeTree/MergeTreeIndexFullText.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index f5a63097a24..5ecb7b537e2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -709,7 +709,7 @@ void bloomFilterIndexValidator(const IndexDescription & index, bool /*attach*/) { const auto & array_type = assert_cast(*index_data_type); data_type = WhichDataType(array_type.getNestedType()); - } + } else if (data_type.isLowCarnality()) { const auto & low_cardinality = assert_cast(*index_data_type); From ef344a581b213eaa0ab1a4d43583cf1892ca12d5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 2 Mar 2022 15:28:17 +0100 Subject: [PATCH 046/165] Fix bug in FileLog storage --- src/Storages/FileLog/StorageFileLog.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 48a729febb2..07853677222 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -837,6 +837,9 @@ void registerStorageFileLog(StorageFactory & factory) bool StorageFileLog::updateFileInfos() { + if (file_infos.file_names.empty()) + return false; + if (!directory_watch) { /// For table just watch one file, we can not use directory monitor to watch it From d25d83580a2b7ea3446d2c881b834bfe4c294171 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 2 Mar 2022 16:33:21 +0000 Subject: [PATCH 047/165] Fix tests --- programs/client/Client.cpp | 7 ++++++- src/Client/ClientBase.cpp | 7 ++++++- src/Client/ClientBase.h | 2 ++ src/Core/Settings.h | 1 + tests/clickhouse-test | 2 +- 5 files changed, 16 insertions(+), 3 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index f09133f22f1..ec8742a937b 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1119,7 +1119,12 @@ void Client::processOptions(const OptionsDescription & options_description, { const auto & name = setting.getName(); if (options.count(name)) - config().setString(name, options[name].as().back()); + { + if (allow_repeated_settings) + config().setString(name, options[name].as().back()); + else + config().setString(name, options[name].as()); + } } if (options.count("config-file") && options.count("config")) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 9ff139cdc9c..35690a77b13 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1868,6 +1868,8 @@ void ClientBase::readArguments( prev_port_arg = port_arg; } } + else if (arg == "--allow_repeated_settings"sv) + allow_repeated_settings = true; else common_arguments.emplace_back(arg); } @@ -1880,7 +1882,10 @@ void ClientBase::readArguments( void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) { - cmd_settings.addProgramOptionsAsMultitokens(options_description.main_description.value()); + if (allow_repeated_settings) + cmd_settings.addProgramOptionsAsMultitokens(options_description.main_description.value()); + else + cmd_settings.addProgramOptions(options_description.main_description.value()); /// Parse main commandline options. auto parser = po::command_line_parser(arguments).options(options_description.main_description.value()).allow_unregistered(); po::parsed_options parsed = parser.run(); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 0a11745b996..7dfd8117181 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -259,6 +259,8 @@ protected: }; std::vector hosts_and_ports{}; + + bool allow_repeated_settings = false; }; } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b646e7ca025..8cda17af1cf 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -721,6 +721,7 @@ struct Settings : public BaseSettings, public IHints<2, Settings void addProgramOptions(boost::program_options::options_description & options); /// Adds program options as to set the settings from a command line. + /// Allows to set one setting multiple times, the last value will be used. /// (Don't forget to call notify() on the `variables_map` after parsing it!) void addProgramOptionsAsMultitokens(boost::program_options::options_description & options); diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a990e0c2986..c314fafbe52 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -446,7 +446,7 @@ class TestCase: else: os.environ['CLICKHOUSE_URL_PARAMS'] = self.base_url_params + '&' + '&'.join(self.random_settings) - new_options = " --" + " --".join(self.random_settings) + new_options = "--allow_repeated_settings --" + " --".join(self.random_settings) os.environ['CLICKHOUSE_CLIENT_OPT'] = self.base_client_options + new_options + ' ' return client_options + new_options From 192b839cab020c08d061879b6ce0a8b75bc5730f Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 2 Mar 2022 19:11:45 +0100 Subject: [PATCH 048/165] Fix --- src/Disks/DiskCacheWrapper.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Disks/DiskCacheWrapper.cpp b/src/Disks/DiskCacheWrapper.cpp index 5d5eb89691e..a893e9dfae6 100644 --- a/src/Disks/DiskCacheWrapper.cpp +++ b/src/Disks/DiskCacheWrapper.cpp @@ -144,6 +144,9 @@ DiskCacheWrapper::readFile( } } + auto current_read_settings = settings; + current_read_settings.remote_fs_method = RemoteFSReadMethod::read; + if (metadata->status == DOWNLOADING) { FileDownloadStatus result_status = DOWNLOADED; @@ -158,7 +161,7 @@ DiskCacheWrapper::readFile( auto tmp_path = path + ".tmp"; { - auto src_buffer = DiskDecorator::readFile(path, settings, read_hint, file_size); + auto src_buffer = DiskDecorator::readFile(path, current_read_settings, read_hint, file_size); auto dst_buffer = cache_disk->writeFile(tmp_path, settings.local_fs_buffer_size, WriteMode::Rewrite); copyData(*src_buffer, *dst_buffer); } @@ -184,7 +187,7 @@ DiskCacheWrapper::readFile( if (metadata->status == DOWNLOADED) return cache_disk->readFile(path, settings, read_hint, file_size); - return DiskDecorator::readFile(path, settings, read_hint, file_size); + return DiskDecorator::readFile(path, current_read_settings, read_hint, file_size); } std::unique_ptr From 74ff3ee4bcf28796a33c81f55518ea9769ccfff2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Mar 2022 20:02:02 +0100 Subject: [PATCH 049/165] Add sanity check for localhost --- src/Common/isLocalAddress.cpp | 28 ++++++++++++++++--------- src/Common/isLocalAddress.h | 1 + src/Coordination/KeeperStateManager.cpp | 27 ++++++++++++++++++++++++ 3 files changed, 46 insertions(+), 10 deletions(-) diff --git a/src/Common/isLocalAddress.cpp b/src/Common/isLocalAddress.cpp index d79e4cebd15..40907ddbe04 100644 --- a/src/Common/isLocalAddress.cpp +++ b/src/Common/isLocalAddress.cpp @@ -79,17 +79,8 @@ struct NetworkInterfaces } -bool isLocalAddress(const Poco::Net::IPAddress & address) +bool isLoopback(const Poco::Net::IPAddress & address) { - /** 127.0.0.1 is treat as local address unconditionally. - * ::1 is also treat as local address unconditionally. - * - * 127.0.0.{2..255} are not treat as local addresses, because they are used in tests - * to emulate distributed queries across localhost. - * - * But 127.{0,1}.{0,1}.{0,1} are treat as local addresses, - * because they are used in Debian for localhost. - */ if (address.isLoopback()) { if (address.family() == Poco::Net::AddressFamily::IPv4) @@ -111,6 +102,23 @@ bool isLocalAddress(const Poco::Net::IPAddress & address) } } + return false; +} + +bool isLocalAddress(const Poco::Net::IPAddress & address) +{ + /** 127.0.0.1 is treat as local address unconditionally. + * ::1 is also treat as local address unconditionally. + * + * 127.0.0.{2..255} are not treat as local addresses, because they are used in tests + * to emulate distributed queries across localhost. + * + * But 127.{0,1}.{0,1}.{0,1} are treat as local addresses, + * because they are used in Debian for localhost. + */ + if (isLoopback(address)) + return true; + NetworkInterfaces interfaces; return interfaces.hasAddress(address); } diff --git a/src/Common/isLocalAddress.h b/src/Common/isLocalAddress.h index 42977775b09..9cae0e0460f 100644 --- a/src/Common/isLocalAddress.h +++ b/src/Common/isLocalAddress.h @@ -25,6 +25,7 @@ namespace DB bool isLocalAddress(const Poco::Net::SocketAddress & address, UInt16 clickhouse_port); bool isLocalAddress(const Poco::Net::SocketAddress & address); bool isLocalAddress(const Poco::Net::IPAddress & address); + bool isLoopback(const Poco::Net::IPAddress & address); /// Returns number of different bytes in hostnames, used for load balancing size_t getHostNameDifference(const std::string & local_hostname, const std::string & host); diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index c2d4274f972..d646695cda3 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace DB { @@ -12,6 +14,16 @@ namespace ErrorCodes extern const int RAFT_ERROR; } +namespace +{ + +bool isLocalhost(const std::string & hostname) +{ + return isLoopback(DNSResolver::instance().resolveHost(hostname)); +} + +} + KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const { KeeperConfigurationWrapper result; @@ -25,6 +37,8 @@ KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersC std::unordered_map check_duplicated_hostnames; size_t total_servers = 0; + std::string local_hostname; + std::string non_local_hostname; for (const auto & server_key : keys) { if (!startsWith(server_key, "server")) @@ -38,6 +52,11 @@ KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersC int32_t priority = config.getInt(full_prefix + ".priority", 1); bool start_as_follower = config.getBool(full_prefix + ".start_as_follower", false); + if (isLocalhost(hostname)) + local_hostname = hostname; + else + non_local_hostname = hostname; + if (start_as_follower) result.servers_start_as_followers.insert(new_server_id); @@ -77,6 +96,14 @@ KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersC if (result.servers_start_as_followers.size() == total_servers) throw Exception(ErrorCodes::RAFT_ERROR, "At least one of servers should be able to start as leader (without )"); + if (!local_hostname.empty() && !non_local_hostname.empty()) + { + throw Exception( + ErrorCodes::RAFT_ERROR, + "Mixing local and non-local hostnames ('{}' and '{}') in raft_configuration is not allowed. Different hosts can resolve it to themselves so it's not allowed.", + local_hostname, non_local_hostname); + } + return result; } From 5fa7cc6e1821824e739a390387620569c5512d1f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 2 Mar 2022 19:04:14 +0000 Subject: [PATCH 050/165] Fix test. --- tests/queries/0_stateless/01275_parallel_mv.reference | 8 ++++---- tests/queries/0_stateless/01275_parallel_mv.sql | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01275_parallel_mv.reference b/tests/queries/0_stateless/01275_parallel_mv.reference index a5987acafde..9021ae2bb1a 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.reference +++ b/tests/queries/0_stateless/01275_parallel_mv.reference @@ -2,8 +2,8 @@ set parallel_view_processing=1; insert into testX select number from numbers(10) settings log_queries=1; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select length(thread_ids) from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and Settings['parallel_view_processing'] = '1'; -8 +select length(thread_ids) >= 8 from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and Settings['parallel_view_processing'] = '1'; +1 select count() from testX; 10 select count() from testXA; @@ -15,8 +15,8 @@ select count() from testXC; set parallel_view_processing=0; insert into testX select number from numbers(10) settings log_queries=1; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select length(thread_ids) from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and Settings['parallel_view_processing'] = '0'; -5 +select length(thread_ids) >= 5 from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and Settings['parallel_view_processing'] = '0'; +1 select count() from testX; 20 select count() from testXA; diff --git a/tests/queries/0_stateless/01275_parallel_mv.sql b/tests/queries/0_stateless/01275_parallel_mv.sql index 32b43ce616f..d1fa8bbcae6 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.sql +++ b/tests/queries/0_stateless/01275_parallel_mv.sql @@ -13,7 +13,7 @@ create materialized view testXC engine=MergeTree order by tuple() as select slee set parallel_view_processing=1; insert into testX select number from numbers(10) settings log_queries=1; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select length(thread_ids) from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and Settings['parallel_view_processing'] = '1'; +select length(thread_ids) >= 8 from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and Settings['parallel_view_processing'] = '1'; select count() from testX; select count() from testXA; @@ -23,7 +23,7 @@ select count() from testXC; set parallel_view_processing=0; insert into testX select number from numbers(10) settings log_queries=1; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select length(thread_ids) from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and Settings['parallel_view_processing'] = '0'; +select length(thread_ids) >= 5 from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and Settings['parallel_view_processing'] = '0'; select count() from testX; select count() from testXA; From 04d4c52e01ed92512d933193fce2f250346a91ee Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Mar 2022 20:37:59 +0100 Subject: [PATCH 051/165] Add sanity checks for keeper --- src/Coordination/KeeperStateManager.cpp | 23 +++++++++++++++++++++-- 1 file changed, 21 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index d646695cda3..066dc0d63e8 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -24,6 +24,12 @@ bool isLocalhost(const std::string & hostname) } +/// this function quite long because contains a lot of sanity checks in config: +/// 1. No duplicate endpoints +/// 2. No "localhost" or "127.0.0.1" or another local addresses mixed with normal addresses +/// 3. Raft internal port is equal to client port +/// 4. No duplicate IDs +/// 5. Our ID present in hostnames list KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const { KeeperConfigurationWrapper result; @@ -31,6 +37,13 @@ KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersC Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix + ".raft_configuration", keys); + std::unordered_set client_ports; + if (config.has(config_prefix + ".tcp_port")) + client_ports.insert(config.getUInt64(config_prefix + ".tcp_port")); + + if (config.has(config_prefix + ".tcp_port_secure")) + client_ports.insert(config.getUInt64(config_prefix + ".tcp_port_secure")); + /// Sometimes (especially in cloud envs) users can provide incorrect /// configuration with duplicated raft ids or endpoints. We check them /// on config parsing stage and never commit to quorum. @@ -52,6 +65,12 @@ KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersC int32_t priority = config.getInt(full_prefix + ".priority", 1); bool start_as_follower = config.getBool(full_prefix + ".start_as_follower", false); + if (client_ports.contains(port)) + { + throw Exception(ErrorCodes::RAFT_ERROR, "Raft config contains hostname '{}' with port '{}' which is equal to client port on current machine", + hostname, port); + } + if (isLocalhost(hostname)) local_hostname = hostname; else @@ -63,7 +82,7 @@ KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersC auto endpoint = hostname + ":" + std::to_string(port); if (check_duplicated_hostnames.count(endpoint)) { - throw Exception(ErrorCodes::RAFT_ERROR, "Raft config contain duplicate endpoints: " + throw Exception(ErrorCodes::RAFT_ERROR, "Raft config contains duplicate endpoints: " "endpoint {} has been already added with id {}, but going to add it one more time with id {}", endpoint, check_duplicated_hostnames[endpoint], new_server_id); } @@ -73,7 +92,7 @@ KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersC for (const auto & [id_endpoint, id] : check_duplicated_hostnames) { if (new_server_id == id) - throw Exception(ErrorCodes::RAFT_ERROR, "Raft config contain duplicate ids: id {} has been already added with endpoint {}, " + throw Exception(ErrorCodes::RAFT_ERROR, "Raft config contains duplicate ids: id {} has been already added with endpoint {}, " "but going to add it one more time with endpoint {}", id, id_endpoint, endpoint); } check_duplicated_hostnames.emplace(endpoint, new_server_id); From 9ee0d2c8a0af81ffbea40f6f1b631cd53d2cc3e2 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Thu, 3 Mar 2022 14:30:22 +0800 Subject: [PATCH 052/165] keeper atomic stat --- src/Coordination/FourLetterCommand.cpp | 6 +-- src/Coordination/KeeperConnectionStats.cpp | 52 +++++++++++----------- src/Coordination/KeeperConnectionStats.h | 20 +++++---- src/Coordination/KeeperDispatcher.cpp | 1 - src/Coordination/KeeperDispatcher.h | 7 +-- src/Server/KeeperTCPHandler.cpp | 25 +++-------- src/Server/KeeperTCPHandler.h | 3 +- 7 files changed, 48 insertions(+), 66 deletions(-) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 4c76d052f9b..feeec222c0f 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -202,7 +202,7 @@ void print(IFourLetterCommand::StringBuffer & buf, const String & key, uint64_t String MonitorCommand::run() { - KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats(); + auto & stats = keeper_dispatcher.getKeeperConnectionStats(); Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo(); if (!keeper_info.has_leader) @@ -288,7 +288,7 @@ String ServerStatCommand::run() writeText('\n', buf); }; - KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats(); + auto & stats = keeper_dispatcher.getKeeperConnectionStats(); Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo(); write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); @@ -314,7 +314,7 @@ String StatCommand::run() auto write = [&buf] (const String & key, const String & value) { buf << key << ": " << value << '\n'; }; - KeeperConnectionStats stats = keeper_dispatcher.getKeeperConnectionStats(); + auto & stats = keeper_dispatcher.getKeeperConnectionStats(); Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo(); write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); diff --git a/src/Coordination/KeeperConnectionStats.cpp b/src/Coordination/KeeperConnectionStats.cpp index 1e9f2c051a7..26455de25f5 100644 --- a/src/Coordination/KeeperConnectionStats.cpp +++ b/src/Coordination/KeeperConnectionStats.cpp @@ -1,3 +1,4 @@ +#include #include namespace DB @@ -5,61 +6,58 @@ namespace DB uint64_t KeeperConnectionStats::getMinLatency() const { - return min_latency; + return min_latency.load(std::memory_order_relaxed); } uint64_t KeeperConnectionStats::getMaxLatency() const { - return max_latency; + return max_latency.load(std::memory_order_relaxed); } uint64_t KeeperConnectionStats::getAvgLatency() const { - if (count != 0) - return total_latency / count; + auto cnt = count.load(std::memory_order_relaxed); + if (cnt) + return total_latency.load(std::memory_order_relaxed) / cnt; return 0; } uint64_t KeeperConnectionStats::getLastLatency() const { - return last_latency; + return last_latency.load(std::memory_order_relaxed); } uint64_t KeeperConnectionStats::getPacketsReceived() const { - return packets_received; + return packets_received.load(std::memory_order_relaxed); } uint64_t KeeperConnectionStats::getPacketsSent() const { - return packets_sent; + return packets_sent.load(std::memory_order_relaxed); } void KeeperConnectionStats::incrementPacketsReceived() { - packets_received++; + packets_received.fetch_add(1, std::memory_order_relaxed); } void KeeperConnectionStats::incrementPacketsSent() { - packets_sent++; + packets_sent.fetch_add(1, std::memory_order_relaxed); } void KeeperConnectionStats::updateLatency(uint64_t latency_ms) { - last_latency = latency_ms; - total_latency += (latency_ms); - count++; + last_latency.store(latency_ms, std::memory_order_relaxed); + total_latency.fetch_add(latency_ms, std::memory_order_relaxed); + count.fetch_add(1, std::memory_order_relaxed); - if (latency_ms < min_latency) - { - min_latency = latency_ms; - } + if (latency_ms < min_latency.load(std::memory_order_relaxed)) + min_latency.store(latency_ms, std::memory_order_relaxed); - if (latency_ms > max_latency) - { - max_latency = latency_ms; - } + if (latency_ms > max_latency.load(std::memory_order_relaxed)) + max_latency.store(latency_ms, std::memory_order_relaxed); } void KeeperConnectionStats::reset() @@ -70,17 +68,17 @@ void KeeperConnectionStats::reset() void KeeperConnectionStats::resetLatency() { - total_latency = 0; - count = 0; - max_latency = 0; - min_latency = 0; - last_latency = 0; + total_latency.store(0, std::memory_order_relaxed); + count.store(0, std::memory_order_relaxed); + max_latency.store(0, std::memory_order_relaxed); + min_latency.store(0, std::memory_order_relaxed); + last_latency.store(0, std::memory_order_relaxed); } void KeeperConnectionStats::resetRequestCounters() { - packets_received = 0; - packets_sent = 0; + packets_received.store(0, std::memory_order_relaxed); + packets_sent.store(0, std::memory_order_relaxed); } } diff --git a/src/Coordination/KeeperConnectionStats.h b/src/Coordination/KeeperConnectionStats.h index 3cd881d553b..ff276def0c9 100644 --- a/src/Coordination/KeeperConnectionStats.h +++ b/src/Coordination/KeeperConnectionStats.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -11,7 +12,10 @@ namespace DB class KeeperConnectionStats { public: - KeeperConnectionStats() = default; + KeeperConnectionStats() + { + reset(); + } uint64_t getMinLatency() const; uint64_t getMaxLatency() const; @@ -33,20 +37,20 @@ private: void resetRequestCounters(); /// all response with watch response included - uint64_t packets_sent = 0; + std::atomic_uint64_t packets_sent; /// All user requests - uint64_t packets_received = 0; + std::atomic_uint64_t packets_received; /// For consistent with zookeeper measured by millisecond, /// otherwise maybe microsecond is better - uint64_t total_latency = 0; - uint64_t max_latency = 0; - uint64_t min_latency = 0; + std::atomic_uint64_t total_latency; + std::atomic_uint64_t max_latency; + std::atomic_uint64_t min_latency; /// last operation latency - uint64_t last_latency = 0; + std::atomic_uint64_t last_latency; - uint64_t count = 0; + std::atomic_uint64_t count; }; } diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 86e22b834d4..a4dcb0acc52 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -594,7 +594,6 @@ void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfigurati void KeeperDispatcher::updateKeeperStatLatency(uint64_t process_time_ms) { - std::lock_guard lock(keeper_stats_mutex); keeper_stats.updateLatency(process_time_ms); } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 646fab5d80f..cc680f8d78f 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -68,7 +68,6 @@ private: /// RAFT wrapper. std::unique_ptr server; - mutable std::mutex keeper_stats_mutex; KeeperConnectionStats keeper_stats; KeeperConfigurationAndSettingsPtr configuration_and_settings; @@ -159,9 +158,8 @@ public: uint64_t getSnapDirSize() const; /// Request statistics such as qps, latency etc. - KeeperConnectionStats getKeeperConnectionStats() const + KeeperConnectionStats & getKeeperConnectionStats() { - std::lock_guard lock(keeper_stats_mutex); return keeper_stats; } @@ -179,19 +177,16 @@ public: void incrementPacketsSent() { - std::lock_guard lock(keeper_stats_mutex); keeper_stats.incrementPacketsSent(); } void incrementPacketsReceived() { - std::lock_guard lock(keeper_stats_mutex); keeper_stats.incrementPacketsReceived(); } void resetConnectionStats() { - std::lock_guard lock(keeper_stats_mutex); keeper_stats.reset(); } }; diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 07964c29577..3e354cfd18f 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -544,19 +544,13 @@ std::pair KeeperTCPHandler::receiveReque void KeeperTCPHandler::packageSent() { - { - std::lock_guard lock(conn_stats_mutex); - conn_stats.incrementPacketsSent(); - } + conn_stats.incrementPacketsSent(); keeper_dispatcher->incrementPacketsSent(); } void KeeperTCPHandler::packageReceived() { - { - std::lock_guard lock(conn_stats_mutex); - conn_stats.incrementPacketsReceived(); - } + conn_stats.incrementPacketsReceived(); keeper_dispatcher->incrementPacketsReceived(); } @@ -566,10 +560,7 @@ void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response if (response->xid != Coordination::WATCH_XID && response->getOpNum() != Coordination::OpNum::Heartbeat) { Int64 elapsed = (Poco::Timestamp() - operations[response->xid]) / 1000; - { - std::lock_guard lock(conn_stats_mutex); - conn_stats.updateLatency(elapsed); - } + conn_stats.updateLatency(elapsed); operations.erase(response->xid); keeper_dispatcher->updateKeeperStatLatency(elapsed); @@ -584,15 +575,14 @@ void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response } -KeeperConnectionStats KeeperTCPHandler::getConnectionStats() const +KeeperConnectionStats & KeeperTCPHandler::getConnectionStats() { - std::lock_guard lock(conn_stats_mutex); return conn_stats; } void KeeperTCPHandler::dumpStats(WriteBufferFromOwnString & buf, bool brief) { - KeeperConnectionStats stats = getConnectionStats(); + auto & stats = getConnectionStats(); writeText(' ', buf); writeText(socket().peerAddress().toString(), buf); @@ -641,10 +631,7 @@ void KeeperTCPHandler::dumpStats(WriteBufferFromOwnString & buf, bool brief) void KeeperTCPHandler::resetStats() { - { - std::lock_guard lock(conn_stats_mutex); - conn_stats.reset(); - } + conn_stats.reset(); last_op.set(std::make_unique(EMPTY_LAST_OP)); } diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index 7953dfd2cbe..b8cccafeca5 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -51,7 +51,7 @@ public: KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_); void run() override; - KeeperConnectionStats getConnectionStats() const; + KeeperConnectionStats & getConnectionStats(); void dumpStats(WriteBufferFromOwnString & buf, bool brief); void resetStats(); @@ -100,7 +100,6 @@ private: LastOpMultiVersion last_op; - mutable std::mutex conn_stats_mutex; KeeperConnectionStats conn_stats; }; From 299e11bb2b94b58211058a534aaf324cd28dbd64 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 3 Mar 2022 11:11:57 +0100 Subject: [PATCH 053/165] Update DiskCacheWrapper.cpp --- src/Disks/DiskCacheWrapper.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Disks/DiskCacheWrapper.cpp b/src/Disks/DiskCacheWrapper.cpp index a893e9dfae6..7cacab98af5 100644 --- a/src/Disks/DiskCacheWrapper.cpp +++ b/src/Disks/DiskCacheWrapper.cpp @@ -144,6 +144,9 @@ DiskCacheWrapper::readFile( } } + /// Do not use RemoteFSReadMethod::threadpool for index and mark files. + /// Here it does not make sense since the files are small. + /// Note: enabling `threadpool` read requires to call setReadUntilEnd(). auto current_read_settings = settings; current_read_settings.remote_fs_method = RemoteFSReadMethod::read; From f24fa161845a5b910238c12e328021d4ff8908c3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 3 Mar 2022 11:29:43 +0100 Subject: [PATCH 054/165] Fix address check --- src/Coordination/KeeperStateManager.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 066dc0d63e8..b24fa6cdb5e 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -19,7 +19,15 @@ namespace bool isLocalhost(const std::string & hostname) { - return isLoopback(DNSResolver::instance().resolveHost(hostname)); + try + { + return isLoopback(DNSResolver::instance().resolveHost(hostname)); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + return false; } } From a6c510ceb1b6a5c370c74f9b64e4ba47a5f79305 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 3 Mar 2022 12:40:42 +0000 Subject: [PATCH 055/165] ExecutableUDF function deterministic in scope of query fix --- src/Interpreters/UserDefinedExecutableFunctionFactory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp index 8d40dc6dfc8..6d7dee7a4c7 100644 --- a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp +++ b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp @@ -47,6 +47,7 @@ public: bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForNulls() const override { return true; } bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes &) const override { From d3ee3dee85e8ddbedb78175c0e4cec372573815d Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 1 Mar 2022 12:13:46 +0000 Subject: [PATCH 056/165] Catch exception in ~WriteBufferFromS3 --- src/IO/WriteBufferFromS3.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index ec7b1772a78..eda7bb6f8ae 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -116,7 +116,14 @@ void WriteBufferFromS3::allocateBuffer() WriteBufferFromS3::~WriteBufferFromS3() { - finalize(); + try + { + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } void WriteBufferFromS3::preFinalize() From 8bb095c67e657887f9643d9436870fca37f0aba6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 1 Mar 2022 16:38:20 +0000 Subject: [PATCH 057/165] cancel processon in QueryPipeline dtor --- src/QueryPipeline/QueryPipeline.cpp | 16 +++++++++++++++- src/QueryPipeline/QueryPipeline.h | 1 + 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 0412049bd58..ac230ab7871 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -26,7 +26,11 @@ namespace ErrorCodes QueryPipeline::QueryPipeline() = default; QueryPipeline::QueryPipeline(QueryPipeline &&) noexcept = default; QueryPipeline & QueryPipeline::operator=(QueryPipeline &&) noexcept = default; -QueryPipeline::~QueryPipeline() = default; + +QueryPipeline::~QueryPipeline() +{ + cancel(); +} static void checkInput(const InputPort & input, const ProcessorPtr & processor) { @@ -528,6 +532,16 @@ void QueryPipeline::addStorageHolder(StoragePtr storage) resources.storage_holders.emplace_back(std::move(storage)); } + +void QueryPipeline::cancel() +{ + for (auto & proc : processors) + { + if (proc) + proc->cancel(); + } +} + void QueryPipeline::reset() { QueryPipeline to_remove = std::move(*this); diff --git a/src/QueryPipeline/QueryPipeline.h b/src/QueryPipeline/QueryPipeline.h index 29b5dd76017..bcda9567ec5 100644 --- a/src/QueryPipeline/QueryPipeline.h +++ b/src/QueryPipeline/QueryPipeline.h @@ -100,6 +100,7 @@ public: const Processors & getProcessors() const { return processors; } void reset(); + void cancel(); private: PipelineResourcesHolder resources; From 6c29a574a3b0d736851f5433c3b450c82f4af7c6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 3 Mar 2022 11:49:24 +0000 Subject: [PATCH 058/165] Revert "cancel processon in QueryPipeline dtor" This reverts commit 7783bb24ea0c109c918faf24343082fc564f22d1. --- src/QueryPipeline/QueryPipeline.cpp | 16 +--------------- src/QueryPipeline/QueryPipeline.h | 1 - 2 files changed, 1 insertion(+), 16 deletions(-) diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index ac230ab7871..0412049bd58 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -26,11 +26,7 @@ namespace ErrorCodes QueryPipeline::QueryPipeline() = default; QueryPipeline::QueryPipeline(QueryPipeline &&) noexcept = default; QueryPipeline & QueryPipeline::operator=(QueryPipeline &&) noexcept = default; - -QueryPipeline::~QueryPipeline() -{ - cancel(); -} +QueryPipeline::~QueryPipeline() = default; static void checkInput(const InputPort & input, const ProcessorPtr & processor) { @@ -532,16 +528,6 @@ void QueryPipeline::addStorageHolder(StoragePtr storage) resources.storage_holders.emplace_back(std::move(storage)); } - -void QueryPipeline::cancel() -{ - for (auto & proc : processors) - { - if (proc) - proc->cancel(); - } -} - void QueryPipeline::reset() { QueryPipeline to_remove = std::move(*this); diff --git a/src/QueryPipeline/QueryPipeline.h b/src/QueryPipeline/QueryPipeline.h index bcda9567ec5..29b5dd76017 100644 --- a/src/QueryPipeline/QueryPipeline.h +++ b/src/QueryPipeline/QueryPipeline.h @@ -100,7 +100,6 @@ public: const Processors & getProcessors() const { return processors; } void reset(); - void cancel(); private: PipelineResourcesHolder resources; From e953e482ec643830c982e30d0081c62fd9c91834 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 17 Jan 2022 16:34:44 +0800 Subject: [PATCH 059/165] Implement more functions for Map data type --- src/Columns/ColumnMap.h | 4 +- src/Functions/FunctionMapMapped.h | 198 ++++++++++++++++++ src/Functions/map.cpp | 110 ++++++++++ src/Functions/mapFilter.cpp | 124 +++++++++++ .../registerFunctionsHigherOrder.cpp | 4 + .../0_stateless/02169_map_functions.reference | 21 ++ .../0_stateless/02169_map_functions.sql | 8 + 7 files changed, 467 insertions(+), 2 deletions(-) create mode 100644 src/Functions/FunctionMapMapped.h create mode 100644 src/Functions/mapFilter.cpp create mode 100644 tests/queries/0_stateless/02169_map_functions.reference create mode 100644 tests/queries/0_stateless/02169_map_functions.sql diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index fb69541c363..b33630b993c 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -36,8 +36,8 @@ public: static Ptr create(const ColumnPtr & column) { return ColumnMap::create(column->assumeMutable()); } static Ptr create(ColumnPtr && arg) { return create(arg); } - template ::value>::type> - static MutablePtr create(Arg && arg) { return Base::create(std::forward(arg)); } + template ::value>::type> + static MutablePtr create(Args &&... args) { return Base::create(std::forward(args)...); } std::string getName() const override; const char * getFamilyName() const override { return "Map"; } diff --git a/src/Functions/FunctionMapMapped.h b/src/Functions/FunctionMapMapped.h new file mode 100644 index 00000000000..fb92db22325 --- /dev/null +++ b/src/Functions/FunctionMapMapped.h @@ -0,0 +1,198 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +/** Higher-order functions for map. + * These functions optionally apply a map by lambda function, + * and return some result based on that transformation. + * + * Examples: + * mapMap(x1,...,xn -> expression, map) - apply the expression to the map. + * mapFilter((k, v) -> predicate, map) - leave in the map only the kv elements for which the expression is true. + */ +template +class FunctionMapMapped : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + /// Called if at least one function argument is a lambda expression. + /// For argument-lambda expressions, it defines the types of arguments of these expressions. + void getLambdaArgumentTypes(DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception("Function " + getName() + " needs at least one argument; passed " + + toString(arguments.size()) + ".", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (arguments.size() == 1) + throw Exception("Function " + getName() + " needs at least one map argument.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + DataTypes nested_types((arguments.size() - 1) * 2); + for (size_t i = 0; i < arguments.size() - 1; ++i) + { + const DataTypeMap * map_type = checkAndGetDataType(&*arguments[i + 1]); + if (!map_type) + throw Exception("Argument " + toString(i + 2) + " of function " + getName() + " must be map. Found " + + arguments[i + 1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + nested_types[i * 2] = recursiveRemoveLowCardinality(map_type->getKeyType()); + nested_types[i * 2 + 1] = recursiveRemoveLowCardinality(map_type->getValueType()); + } + + const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); + if (!function_type || function_type->getArgumentTypes().size() != nested_types.size()) + throw Exception("First argument for this overload of " + getName() + " must be a function with " + + toString(nested_types.size()) + " arguments. Found " + + arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + arguments[0] = std::make_shared(nested_types); + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + size_t min_args = Impl::needExpression() ? 2 : 1; + if (arguments.size() < min_args) + throw Exception("Function " + getName() + " needs at least " + + toString(min_args) + " argument; passed " + + toString(arguments.size()) + ".", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + if (arguments.size() > 2 && Impl::needOneMap()) + throw Exception("Function " + getName() + " needs one map argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); + + if (!data_type_function) + throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + /// The types of the remaining arguments are already checked in getLambdaArgumentTypes. + + DataTypePtr return_type = removeLowCardinality(data_type_function->getReturnType()); + if (Impl::needBoolean() && !WhichDataType(return_type).isUInt8()) + throw Exception( + "Expression for function " + getName() + " must return UInt8, found " + return_type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto * map_type = checkAndGetDataType(arguments[1].type.get()); + if (!map_type) + throw Exception("Second argument for function " + getName() + " must be a map.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return Impl::getReturnType(return_type, map_type->getKeyValueTypes()); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + const auto & column_with_type_and_name = arguments[0]; + + if (!column_with_type_and_name.column) + throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto * column_function = typeid_cast(column_with_type_and_name.column.get()); + + if (!column_function) + throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + ColumnPtr offsets_column; + + ColumnPtr column_first_map_ptr; + const ColumnMap * column_first_map = nullptr; + + ColumnsWithTypeAndName maps; + maps.reserve(arguments.size() - 1); + + for (size_t i = 1; i < arguments.size(); ++i) + { + const auto & map_with_type_and_name = arguments[i]; + + ColumnPtr column_map_ptr = map_with_type_and_name.column; + const auto * column_map = checkAndGetColumn(column_map_ptr.get()); + + const DataTypePtr & map_type_ptr = map_with_type_and_name.type; + const auto * map_type = checkAndGetDataType(map_type_ptr.get()); + + if (!column_map) + { + const ColumnConst * column_const_map = checkAndGetColumnConst(column_map_ptr.get()); + if (!column_const_map) + throw Exception("Expected map column, found " + column_map_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); + column_map_ptr = recursiveRemoveLowCardinality(column_const_map->convertToFullColumn()); + column_map = checkAndGetColumn(column_map_ptr.get()); + } + + if (!map_type) + throw Exception("Expected map type, found " + map_type_ptr->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!offsets_column) + { + offsets_column = column_map->getNestedColumn().getOffsetsPtr(); + } + else + { + /// The first condition is optimization: do not compare data if the pointers are equal. + if (column_map->getNestedColumn().getOffsetsPtr() != offsets_column + && column_map->getNestedColumn().getOffsets() != typeid_cast(*offsets_column).getData()) + throw Exception("maps passed to " + getName() + " must have equal size", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + } + + if (i == 1) + { + column_first_map_ptr = column_map_ptr; + column_first_map = column_map; + } + + maps.emplace_back(ColumnWithTypeAndName( + column_map->getNestedData().getColumnPtr(0), recursiveRemoveLowCardinality(map_type->getKeyType()), map_with_type_and_name.name+".key")); + maps.emplace_back(ColumnWithTypeAndName( + column_map->getNestedData().getColumnPtr(1), recursiveRemoveLowCardinality(map_type->getValueType()), map_with_type_and_name.name+".value")); + } + + /// Put all the necessary columns multiplied by the sizes of maps into the columns. + auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(column_first_map->getNestedColumn().getOffsets())); + auto * replicated_column_function = typeid_cast(replicated_column_function_ptr.get()); + replicated_column_function->appendArguments(maps); + + auto lambda_result = replicated_column_function->reduce().column; + if (lambda_result->lowCardinality()) + lambda_result = lambda_result->convertToFullColumnIfLowCardinality(); + + return Impl::execute(*column_first_map, lambda_result); + } +}; + +} diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 4e242c4348b..95f464c951f 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -518,6 +518,115 @@ public: } }; +class FunctionMapReplace : public IFunction +{ +public: + static constexpr auto name = "mapReplace"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 2; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 2) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 2", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const DataTypeMap * left = checkAndGetDataType(arguments[0].type.get()); + const DataTypeMap * right = checkAndGetDataType(arguments[1].type.get()); + + if (!left || !right) + throw Exception{"The two arguments for function " + getName() + " must be both Map type", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + if (!left->getKeyType()->equals(*right->getKeyType()) || !left->getValueType()->equals(*right->getValueType())) + throw Exception{"The Key And Value type of Map for function " + getName() + " must be the same", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(left->getKeyType(), left->getValueType()); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + const ColumnMap * col_map_left = typeid_cast(arguments[0].column.get()); + const auto * col_const_map_left = checkAndGetColumnConst(arguments[0].column.get()); + if (col_const_map_left) + col_map_left = typeid_cast(&col_const_map_left->getDataColumn()); + if (!col_map_left) + return nullptr; + + const ColumnMap * col_map_right = typeid_cast(arguments[1].column.get()); + const auto * col_const_map_right = checkAndGetColumnConst(arguments[1].column.get()); + if (col_const_map_right) + col_map_right = typeid_cast(&col_const_map_right->getDataColumn()); + if (!col_map_right) + return nullptr; + + auto & nested_column_left = col_map_left->getNestedColumn(); + auto & keys_data_left = col_map_left->getNestedData().getColumn(0); + auto & values_data_left = col_map_left->getNestedData().getColumn(1); + auto & offsets_left = nested_column_left.getOffsets(); + + const auto & nested_column_right = col_map_right->getNestedColumn(); + const auto & keys_data_right = col_map_right->getNestedData().getColumn(0); + const auto & values_data_right = col_map_right->getNestedData().getColumn(1); + const auto & offsets_right = nested_column_right.getOffsets(); + + const auto & result_type_map = static_cast(*result_type); + const DataTypePtr & key_type = result_type_map.getKeyType(); + const DataTypePtr & value_type = result_type_map.getValueType(); + MutableColumnPtr keys_data = key_type->createColumn(); + MutableColumnPtr values_data = value_type->createColumn(); + MutableColumnPtr offsets = DataTypeNumber().createColumn(); + + IColumn::Offset current_offset = 0; + for (size_t idx = 0; idx < input_rows_count; ++idx) + { + for (size_t i = offsets_left[idx - 1]; i < offsets_left[idx]; ++i) + { + bool matched = false; + auto key = keys_data_left.getDataAt(i); + for (size_t j = offsets_right[idx - 1]; j < offsets_right[idx]; ++j) + { + if (keys_data_right.getDataAt(j).toString() == key.toString()) + { + matched = true; + break; + } + } + if (!matched) + { + keys_data->insertFrom(keys_data_left, i); + values_data->insertFrom(values_data_left, i); + ++current_offset; + } + } + for (size_t j = offsets_right[idx - 1]; j < offsets_right[idx]; ++j) + { + keys_data->insertFrom(keys_data_right, j); + values_data->insertFrom(values_data_right, j); + ++current_offset; + } + offsets->insert(current_offset); + } + + auto nested_column = ColumnArray::create( + ColumnTuple::create(Columns{std::move(keys_data), std::move(values_data)}), + std::move(offsets)); + + return ColumnMap::create(nested_column); + } +}; + } void registerFunctionsMap(FunctionFactory & factory) @@ -528,6 +637,7 @@ void registerFunctionsMap(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp new file mode 100644 index 00000000000..6b90d2552cc --- /dev/null +++ b/src/Functions/mapFilter.cpp @@ -0,0 +1,124 @@ +#include +#include +#include +#include +#include "FunctionMapMapped.h" +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +/** MapFilter(x -> predicate, array) - leave in the array only the elements for which the expression is true. + */ +struct MapFilterImpl +{ + static bool needBoolean() { return true; } + static bool needExpression() { return true; } + static bool needOneMap() { return true; } + + static DataTypePtr getReturnType(const DataTypePtr & /*expression_return*/, const DataTypes & elems) + { + return std::make_shared(elems); + } + + /// If there are several arrays, the first one is passed here. + static ColumnPtr execute(const ColumnMap & map_column, ColumnPtr mapped) + { + const ColumnUInt8 * column_filter = typeid_cast(&*mapped); + + if (!column_filter) + { + const auto * column_filter_const = checkAndGetColumnConst(&*mapped); + + if (!column_filter_const) + throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN); + + if (column_filter_const->getValue()) + return map_column.clone(); + else + return ColumnMap::create( + map_column.getNestedColumnPtr()->cloneEmpty()); + } + + const IColumn::Filter & filter = column_filter->getData(); + ColumnPtr filtered = map_column.getNestedColumn().getData().filter(filter, -1); + + const IColumn::Offsets & in_offsets = map_column.getNestedColumn().getOffsets(); + auto column_offsets = ColumnArray::ColumnOffsets::create(in_offsets.size()); + IColumn::Offsets & out_offsets = column_offsets->getData(); + + size_t in_pos = 0; + size_t out_pos = 0; + for (size_t i = 0; i < in_offsets.size(); ++i) + { + for (; in_pos < in_offsets[i]; ++in_pos) + { + if (filter[in_pos]) + ++out_pos; + } + out_offsets[i] = out_pos; + } + + return ColumnMap::create(ColumnArray::create(filtered, std::move(column_offsets))); + } +}; + +struct NameMapFilter { static constexpr auto name = "mapFilter"; }; +using FunctionMapFilter = FunctionMapMapped; + +void registerFunctionMapFilter(FunctionFactory & factory) +{ + factory.registerFunction(); +} + + +/** mapMap((k,v) -> expression, map) - apply the expression to the map. + */ +struct MapMapImpl +{ + /// true if the expression (for an overload of f(expression, maps)) or a map (for f(map)) should be boolean. + static bool needBoolean() { return false; } + /// true if the f(map) overload is unavailable. + static bool needExpression() { return true; } + /// true if the map must be exactly one. + static bool needOneMap() { return true; } + + static DataTypePtr getReturnType(const DataTypePtr & , const DataTypes & elems) + { + return std::make_shared(elems); + } + + static ColumnPtr execute(const ColumnMap & map, ColumnPtr mapped) + { + const auto * column_tuple = checkAndGetColumn(mapped.get()); + if (!column_tuple) + { + const ColumnConst * column_const_tuple = checkAndGetColumnConst(mapped.get()); + if (!column_const_tuple) + throw Exception("Expected tuple column, found " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN); + ColumnPtr column_tuple_ptr = recursiveRemoveLowCardinality(column_const_tuple->convertToFullColumn()); + column_tuple = checkAndGetColumn(column_tuple_ptr.get()); + } + + return ColumnMap::create(column_tuple->getColumnPtr(0), column_tuple->getColumnPtr(1), + map.getNestedColumn().getOffsetsPtr()); + } +}; + +struct NameMapMap { static constexpr auto name = "mapMap"; }; +using FunctionMapMap = FunctionMapMapped; + +void registerFunctionMapMap(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + + diff --git a/src/Functions/registerFunctionsHigherOrder.cpp b/src/Functions/registerFunctionsHigherOrder.cpp index d3621a03ecd..0a5d729facd 100644 --- a/src/Functions/registerFunctionsHigherOrder.cpp +++ b/src/Functions/registerFunctionsHigherOrder.cpp @@ -18,6 +18,8 @@ void registerFunctionsArraySort(FunctionFactory & factory); void registerFunctionArrayCumSum(FunctionFactory & factory); void registerFunctionArrayCumSumNonNegative(FunctionFactory & factory); void registerFunctionArrayDifference(FunctionFactory & factory); +void registerFunctionMapFilter(FunctionFactory & factory); +void registerFunctionMapMap(FunctionFactory & factory); void registerFunctionsHigherOrder(FunctionFactory & factory) { @@ -36,6 +38,8 @@ void registerFunctionsHigherOrder(FunctionFactory & factory) registerFunctionArrayCumSum(factory); registerFunctionArrayCumSumNonNegative(factory); registerFunctionArrayDifference(factory); + registerFunctionMapFilter(factory); + registerFunctionMapMap(factory); } } diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference new file mode 100644 index 00000000000..0f47bdce042 --- /dev/null +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -0,0 +1,21 @@ +{} +{} +{} +{'key3':103} +{} +{} +{} +{'key3':100,'key2':101,'key4':102} {'key4':102} +{'key3':101,'key2':102,'key4':103} {'key2':102,'key4':103} +{'key3':102,'key2':103,'key4':104} {'key3':102,'key2':103,'key4':104} +{'key3':103,'key2':104,'key4':105} {'key3':103,'key2':104,'key4':105} +{'key1':1111,'key2':2222} {'key2':2222} +{'key1':1112,'key2':2224} {'key1':1112,'key2':2224} +{'key1':1113,'key2':2226} {'key1':1113,'key2':2226} +{'key3':101,'key2':102,'key4':103} +{'key3':102,'key2':103,'key4':104} +{'key3':103,'key2':104,'key4':105} +{'key3':104,'key2':105,'key4':106} +{'key1':1112,'key2':2223} +{'key1':1113,'key2':2225} +{'key1':1114,'key2':2227} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql new file mode 100644 index 00000000000..6b4d796043f --- /dev/null +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -0,0 +1,8 @@ +DROP TABLE IF EXISTS table_map; +create TABLE table_map (id UInt32, col Map(String, UInt64)) engine = MergeTree() ORDER BY tuple(); +INSERT INTO table_map SELECT number, map('key1', number, 'key2', number * 2) FROM numbers(1111, 3); +INSERT INTO table_map SELECT number, map('key3', number, 'key2', number + 1, 'key4', number + 2) FROM numbers(100, 4); + +SELECT mapFilter((k,v)->k like '%3' and v > 102, col) FROM table_map ORDER BY id; +SELECT col, mapFilter((k, v) -> ((v % 10) > 1), col) FROM table_map ORDER BY id ASC; +SELECT mapMap((k,v)->(k,v+1), col) FROM table_map ORDER BY id; From d6d943f4d41da97d4c80cc8e4f4ceedb346324ba Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 17 Jan 2022 16:49:10 +0800 Subject: [PATCH 060/165] Rectify test case --- tests/queries/0_stateless/02169_map_functions.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index 6b4d796043f..420bbaaf3d5 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -6,3 +6,4 @@ INSERT INTO table_map SELECT number, map('key3', number, 'key2', number + 1, 'ke SELECT mapFilter((k,v)->k like '%3' and v > 102, col) FROM table_map ORDER BY id; SELECT col, mapFilter((k, v) -> ((v % 10) > 1), col) FROM table_map ORDER BY id ASC; SELECT mapMap((k,v)->(k,v+1), col) FROM table_map ORDER BY id; +DROP TABLE table_map; From ed39edda676dd6f74deb0a01ea379f2acdf3e807 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 24 Jan 2022 16:17:02 +0800 Subject: [PATCH 061/165] Update src/Functions/mapFilter.cpp Co-authored-by: Vladimir C --- src/Functions/mapFilter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp index 6b90d2552cc..2a98f0d3fca 100644 --- a/src/Functions/mapFilter.cpp +++ b/src/Functions/mapFilter.cpp @@ -2,7 +2,7 @@ #include #include #include -#include "FunctionMapMapped.h" +#include #include #include From 36939f1d5cf1d6f343530214db7e43720712dc96 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 24 Jan 2022 16:21:28 +0800 Subject: [PATCH 062/165] Update src/Functions/FunctionMapMapped.h Co-authored-by: Vladimir C --- src/Functions/FunctionMapMapped.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionMapMapped.h b/src/Functions/FunctionMapMapped.h index fb92db22325..3801432029f 100644 --- a/src/Functions/FunctionMapMapped.h +++ b/src/Functions/FunctionMapMapped.h @@ -94,7 +94,7 @@ public: + toString(arguments.size()) + ".", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (arguments.size() > 2 && Impl::needOneMap()) - throw Exception("Function " + getName() + " needs one map argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function '{}' needs one map argument", getName()); const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); From 582e6fa1c55c993c9155ddd5a35e045b6fa222ee Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 24 Jan 2022 16:18:34 +0800 Subject: [PATCH 063/165] Fix build error --- src/Functions/map.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 95f464c951f..4e660f2e0b1 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -571,10 +571,10 @@ public: if (!col_map_right) return nullptr; - auto & nested_column_left = col_map_left->getNestedColumn(); - auto & keys_data_left = col_map_left->getNestedData().getColumn(0); - auto & values_data_left = col_map_left->getNestedData().getColumn(1); - auto & offsets_left = nested_column_left.getOffsets(); + const auto & nested_column_left = col_map_left->getNestedColumn(); + const auto & keys_data_left = col_map_left->getNestedData().getColumn(0); + const auto & values_data_left = col_map_left->getNestedData().getColumn(1); + const auto & offsets_left = nested_column_left.getOffsets(); const auto & nested_column_right = col_map_right->getNestedColumn(); const auto & keys_data_right = col_map_right->getNestedData().getColumn(0); From a435c3a1e81b709a30d314d4dfdc0a3aaa5bfaa8 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 24 Jan 2022 18:33:13 +0800 Subject: [PATCH 064/165] Fix mapFilter bug --- src/Functions/mapFilter.cpp | 9 +++++++-- tests/queries/0_stateless/02169_map_functions.reference | 7 +++++++ tests/queries/0_stateless/02169_map_functions.sql | 1 + 3 files changed, 15 insertions(+), 2 deletions(-) diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp index 2a98f0d3fca..570c9fe8d36 100644 --- a/src/Functions/mapFilter.cpp +++ b/src/Functions/mapFilter.cpp @@ -42,8 +42,13 @@ struct MapFilterImpl if (column_filter_const->getValue()) return map_column.clone(); else - return ColumnMap::create( - map_column.getNestedColumnPtr()->cloneEmpty()); + { + const auto * column_array = typeid_cast(map_column.getNestedColumnPtr().get()); + const auto * column_tuple = typeid_cast(column_array->getDataPtr().get()); + ColumnPtr keys = column_tuple->getColumnPtr(0)->cloneEmpty(); + ColumnPtr values = column_tuple->getColumnPtr(1)->cloneEmpty(); + return ColumnMap::create(keys, values, ColumnArray::ColumnOffsets::create(map_column.size(), 0)); + } } const IColumn::Filter & filter = column_filter->getData(); diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index 0f47bdce042..50c35637b84 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -19,3 +19,10 @@ {'key1':1112,'key2':2223} {'key1':1113,'key2':2225} {'key1':1114,'key2':2227} +{} +{} +{} +{} +{} +{} +{} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index 420bbaaf3d5..ade266324d7 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -6,4 +6,5 @@ INSERT INTO table_map SELECT number, map('key3', number, 'key2', number + 1, 'ke SELECT mapFilter((k,v)->k like '%3' and v > 102, col) FROM table_map ORDER BY id; SELECT col, mapFilter((k, v) -> ((v % 10) > 1), col) FROM table_map ORDER BY id ASC; SELECT mapMap((k,v)->(k,v+1), col) FROM table_map ORDER BY id; +SELECT mapFilter((k,v)->0, col) from table_map; DROP TABLE table_map; From 71aa4117460c2e502627abee075ca0a9654cb755 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Tue, 15 Feb 2022 17:59:44 +0800 Subject: [PATCH 065/165] Fix bug --- ...nctionMapMapped.h => FunctionMapAdvance.h} | 121 +++++------------- src/Functions/map.cpp | 8 +- src/Functions/mapFilter.cpp | 30 ++--- .../registerFunctionsHigherOrder.cpp | 4 +- .../0_stateless/02169_map_functions.reference | 2 + .../0_stateless/02169_map_functions.sql | 4 +- 6 files changed, 57 insertions(+), 112 deletions(-) rename src/Functions/{FunctionMapMapped.h => FunctionMapAdvance.h} (51%) diff --git a/src/Functions/FunctionMapMapped.h b/src/Functions/FunctionMapAdvance.h similarity index 51% rename from src/Functions/FunctionMapMapped.h rename to src/Functions/FunctionMapAdvance.h index 3801432029f..9f6660d030b 100644 --- a/src/Functions/FunctionMapMapped.h +++ b/src/Functions/FunctionMapAdvance.h @@ -23,7 +23,6 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -33,15 +32,15 @@ namespace ErrorCodes * and return some result based on that transformation. * * Examples: - * mapMap(x1,...,xn -> expression, map) - apply the expression to the map. + * mapApply(x1,...,xn -> expression, map) - apply the expression to the map. * mapFilter((k, v) -> predicate, map) - leave in the map only the kv elements for which the expression is true. */ template -class FunctionMapMapped : public IFunction +class FunctionMapAdvance : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -56,30 +55,21 @@ public: /// For argument-lambda expressions, it defines the types of arguments of these expressions. void getLambdaArgumentTypes(DataTypes & arguments) const override { - if (arguments.empty()) - throw Exception("Function " + getName() + " needs at least one argument; passed " - + toString(arguments.size()) + ".", + if (arguments.size() != 2) + throw Exception("Function " + getName() + " needs 2 arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (arguments.size() == 1) - throw Exception("Function " + getName() + " needs at least one map argument.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - DataTypes nested_types((arguments.size() - 1) * 2); - for (size_t i = 0; i < arguments.size() - 1; ++i) - { - const DataTypeMap * map_type = checkAndGetDataType(&*arguments[i + 1]); - if (!map_type) - throw Exception("Argument " + toString(i + 2) + " of function " + getName() + " must be map. Found " - + arguments[i + 1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - nested_types[i * 2] = recursiveRemoveLowCardinality(map_type->getKeyType()); - nested_types[i * 2 + 1] = recursiveRemoveLowCardinality(map_type->getValueType()); - } + DataTypes nested_types(2); + const DataTypeMap * map_type = checkAndGetDataType(&*arguments[1]); + if (!map_type) + throw Exception("The second argument of function " + getName() + " must be map. Found " + + arguments[1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + nested_types[0] = recursiveRemoveLowCardinality(map_type->getKeyType()); + nested_types[1] = recursiveRemoveLowCardinality(map_type->getValueType()); const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); - if (!function_type || function_type->getArgumentTypes().size() != nested_types.size()) - throw Exception("First argument for this overload of " + getName() + " must be a function with " - + toString(nested_types.size()) + " arguments. Found " + if (!function_type || function_type->getArgumentTypes().size() != 2) + throw Exception("First argument for this overload of " + getName() + " must be a function with 2 arguments. Found " + arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); arguments[0] = std::make_shared(nested_types); @@ -87,14 +77,10 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - size_t min_args = Impl::needExpression() ? 2 : 1; - if (arguments.size() < min_args) - throw Exception("Function " + getName() + " needs at least " - + toString(min_args) + " argument; passed " + if (arguments.size() != 2) + throw Exception("Function " + getName() + " needs at least 2 argument; passed " + toString(arguments.size()) + ".", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (arguments.size() > 2 && Impl::needOneMap()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function '{}' needs one map argument", getName()); const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); @@ -102,7 +88,6 @@ public: throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); /// The types of the remaining arguments are already checked in getLambdaArgumentTypes. - DataTypePtr return_type = removeLowCardinality(data_type_function->getReturnType()); if (Impl::needBoolean() && !WhichDataType(return_type).isUInt8()) throw Exception( @@ -119,79 +104,41 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { const auto & column_with_type_and_name = arguments[0]; - if (!column_with_type_and_name.column) throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); const auto * column_function = typeid_cast(column_with_type_and_name.column.get()); - if (!column_function) throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - ColumnPtr offsets_column; - - ColumnPtr column_first_map_ptr; - const ColumnMap * column_first_map = nullptr; - - ColumnsWithTypeAndName maps; - maps.reserve(arguments.size() - 1); - - for (size_t i = 1; i < arguments.size(); ++i) + ColumnPtr column_map_ptr = arguments[1].column; + const auto * column_map = checkAndGetColumn(column_map_ptr.get()); + if (!column_map) { - const auto & map_with_type_and_name = arguments[i]; - - ColumnPtr column_map_ptr = map_with_type_and_name.column; - const auto * column_map = checkAndGetColumn(column_map_ptr.get()); - - const DataTypePtr & map_type_ptr = map_with_type_and_name.type; - const auto * map_type = checkAndGetDataType(map_type_ptr.get()); - - if (!column_map) - { - const ColumnConst * column_const_map = checkAndGetColumnConst(column_map_ptr.get()); - if (!column_const_map) - throw Exception("Expected map column, found " + column_map_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); - column_map_ptr = recursiveRemoveLowCardinality(column_const_map->convertToFullColumn()); - column_map = checkAndGetColumn(column_map_ptr.get()); - } - - if (!map_type) - throw Exception("Expected map type, found " + map_type_ptr->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!offsets_column) - { - offsets_column = column_map->getNestedColumn().getOffsetsPtr(); - } - else - { - /// The first condition is optimization: do not compare data if the pointers are equal. - if (column_map->getNestedColumn().getOffsetsPtr() != offsets_column - && column_map->getNestedColumn().getOffsets() != typeid_cast(*offsets_column).getData()) - throw Exception("maps passed to " + getName() + " must have equal size", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); - } - - if (i == 1) - { - column_first_map_ptr = column_map_ptr; - column_first_map = column_map; - } - - maps.emplace_back(ColumnWithTypeAndName( - column_map->getNestedData().getColumnPtr(0), recursiveRemoveLowCardinality(map_type->getKeyType()), map_with_type_and_name.name+".key")); - maps.emplace_back(ColumnWithTypeAndName( - column_map->getNestedData().getColumnPtr(1), recursiveRemoveLowCardinality(map_type->getValueType()), map_with_type_and_name.name+".value")); + const ColumnConst * column_const_map = checkAndGetColumnConst(column_map_ptr.get()); + if (!column_const_map) + throw Exception("Expected map column, found " + column_map_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); + column_map_ptr = recursiveRemoveLowCardinality(column_const_map->convertToFullColumn()); + column_map = checkAndGetColumn(column_map_ptr.get()); } + const DataTypePtr & map_type_ptr = arguments[1].type; + const auto * map_type = checkAndGetDataType(map_type_ptr.get()); + const auto & offsets_column = column_map->getNestedColumn().getOffsets(); + /// Put all the necessary columns multiplied by the sizes of maps into the columns. - auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(column_first_map->getNestedColumn().getOffsets())); + auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(offsets_column)); auto * replicated_column_function = typeid_cast(replicated_column_function_ptr.get()); - replicated_column_function->appendArguments(maps); + const ColumnsWithTypeAndName args { + {column_map->getNestedData().getColumnPtr(0), recursiveRemoveLowCardinality(map_type->getKeyType()), arguments[1].name + ".key"}, + {column_map->getNestedData().getColumnPtr(1), recursiveRemoveLowCardinality(map_type->getValueType()), arguments[1].name + ".value"}}; + replicated_column_function->appendArguments(args); auto lambda_result = replicated_column_function->reduce().column; if (lambda_result->lowCardinality()) lambda_result = lambda_result->convertToFullColumnIfLowCardinality(); - return Impl::execute(*column_first_map, lambda_result); + return Impl::execute(*column_map, lambda_result); } }; diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 4e660f2e0b1..471d6fc575c 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -518,11 +518,11 @@ public: } }; -class FunctionMapReplace : public IFunction +class FunctionMapUpdate : public IFunction { public: - static constexpr auto name = "mapReplace"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + static constexpr auto name = "mapUpdate"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { @@ -637,7 +637,7 @@ void registerFunctionsMap(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp index 570c9fe8d36..5373b7e3b38 100644 --- a/src/Functions/mapFilter.cpp +++ b/src/Functions/mapFilter.cpp @@ -1,8 +1,7 @@ -#include -#include +#include #include #include -#include +#include #include #include @@ -19,8 +18,6 @@ namespace ErrorCodes struct MapFilterImpl { static bool needBoolean() { return true; } - static bool needExpression() { return true; } - static bool needOneMap() { return true; } static DataTypePtr getReturnType(const DataTypePtr & /*expression_return*/, const DataTypes & elems) { @@ -75,7 +72,7 @@ struct MapFilterImpl }; struct NameMapFilter { static constexpr auto name = "mapFilter"; }; -using FunctionMapFilter = FunctionMapMapped; +using FunctionMapFilter = FunctionMapAdvance; void registerFunctionMapFilter(FunctionFactory & factory) { @@ -83,20 +80,17 @@ void registerFunctionMapFilter(FunctionFactory & factory) } -/** mapMap((k,v) -> expression, map) - apply the expression to the map. +/** mapApply((k,v) -> expression, map) - apply the expression to the map. */ -struct MapMapImpl +struct MapApplyImpl { /// true if the expression (for an overload of f(expression, maps)) or a map (for f(map)) should be boolean. static bool needBoolean() { return false; } - /// true if the f(map) overload is unavailable. - static bool needExpression() { return true; } - /// true if the map must be exactly one. - static bool needOneMap() { return true; } - static DataTypePtr getReturnType(const DataTypePtr & , const DataTypes & elems) + static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypes & ) { - return std::make_shared(elems); + const auto * date_type_tuple = typeid_cast(&*expression_return); + return std::make_shared(date_type_tuple->getElements()); } static ColumnPtr execute(const ColumnMap & map, ColumnPtr mapped) @@ -116,12 +110,12 @@ struct MapMapImpl } }; -struct NameMapMap { static constexpr auto name = "mapMap"; }; -using FunctionMapMap = FunctionMapMapped; +struct NameMapApply { static constexpr auto name = "mapApply"; }; +using FunctionMapApply = FunctionMapAdvance; -void registerFunctionMapMap(FunctionFactory & factory) +void registerFunctionMapApply(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/registerFunctionsHigherOrder.cpp b/src/Functions/registerFunctionsHigherOrder.cpp index 0a5d729facd..00b300b18b9 100644 --- a/src/Functions/registerFunctionsHigherOrder.cpp +++ b/src/Functions/registerFunctionsHigherOrder.cpp @@ -19,7 +19,7 @@ void registerFunctionArrayCumSum(FunctionFactory & factory); void registerFunctionArrayCumSumNonNegative(FunctionFactory & factory); void registerFunctionArrayDifference(FunctionFactory & factory); void registerFunctionMapFilter(FunctionFactory & factory); -void registerFunctionMapMap(FunctionFactory & factory); +void registerFunctionMapApply(FunctionFactory & factory); void registerFunctionsHigherOrder(FunctionFactory & factory) { @@ -39,7 +39,7 @@ void registerFunctionsHigherOrder(FunctionFactory & factory) registerFunctionArrayCumSumNonNegative(factory); registerFunctionArrayDifference(factory); registerFunctionMapFilter(factory); - registerFunctionMapMap(factory); + registerFunctionMapApply(factory); } } diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index 50c35637b84..c570ba4e724 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -19,6 +19,7 @@ {'key1':1112,'key2':2223} {'key1':1113,'key2':2225} {'key1':1114,'key2':2227} +{1:2,2:3} {} {} {} @@ -26,3 +27,4 @@ {} {} {} +{3:2,1:0,2:0} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index ade266324d7..c7a11b46357 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -5,6 +5,8 @@ INSERT INTO table_map SELECT number, map('key3', number, 'key2', number + 1, 'ke SELECT mapFilter((k,v)->k like '%3' and v > 102, col) FROM table_map ORDER BY id; SELECT col, mapFilter((k, v) -> ((v % 10) > 1), col) FROM table_map ORDER BY id ASC; -SELECT mapMap((k,v)->(k,v+1), col) FROM table_map ORDER BY id; +SELECT mapApply((k,v)->(k,v+1), col) FROM table_map ORDER BY id; +SELECT mapApply((x, y) -> (x, x + 1), map(1, 0, 2, 0)); SELECT mapFilter((k,v)->0, col) from table_map; +SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0)); DROP TABLE table_map; From aeec4a62e760938e24245837ce6b77138319d10f Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Wed, 16 Feb 2022 11:28:21 +0800 Subject: [PATCH 066/165] fix bug --- src/Functions/FunctionMapAdvance.h | 6 +++--- src/Functions/mapFilter.cpp | 11 ++++++++--- tests/queries/0_stateless/02169_map_functions.sql | 1 + 3 files changed, 12 insertions(+), 6 deletions(-) diff --git a/src/Functions/FunctionMapAdvance.h b/src/Functions/FunctionMapAdvance.h index 9f6660d030b..d72cd4d62e5 100644 --- a/src/Functions/FunctionMapAdvance.h +++ b/src/Functions/FunctionMapAdvance.h @@ -70,7 +70,7 @@ public: const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); if (!function_type || function_type->getArgumentTypes().size() != 2) throw Exception("First argument for this overload of " + getName() + " must be a function with 2 arguments. Found " - + arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); arguments[0] = std::make_shared(nested_types); } @@ -79,8 +79,8 @@ public: { if (arguments.size() != 2) throw Exception("Function " + getName() + " needs at least 2 argument; passed " - + toString(arguments.size()) + ".", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + toString(arguments.size()) + ".", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp index 5373b7e3b38..69bbf0c00f3 100644 --- a/src/Functions/mapFilter.cpp +++ b/src/Functions/mapFilter.cpp @@ -11,6 +11,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } /** MapFilter(x -> predicate, array) - leave in the array only the elements for which the expression is true. @@ -87,10 +88,14 @@ struct MapApplyImpl /// true if the expression (for an overload of f(expression, maps)) or a map (for f(map)) should be boolean. static bool needBoolean() { return false; } - static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypes & ) + static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypes & /*elems*/) { - const auto * date_type_tuple = typeid_cast(&*expression_return); - return std::make_shared(date_type_tuple->getElements()); + const auto & tuple_types = typeid_cast(&*expression_return)->getElements(); + if (tuple_types.size() != 2) + throw Exception("Expected 2 columns as map's key and value, but found " + + toString(tuple_types.size()) + " columns", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return std::make_shared(tuple_types); } static ColumnPtr execute(const ColumnMap & map, ColumnPtr mapped) diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index c7a11b46357..1d8f90e8a90 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -9,4 +9,5 @@ SELECT mapApply((k,v)->(k,v+1), col) FROM table_map ORDER BY id; SELECT mapApply((x, y) -> (x, x + 1), map(1, 0, 2, 0)); SELECT mapFilter((k,v)->0, col) from table_map; SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0)); +SELECT mapApply((k, v) -> tuple(v + 9223372036854775806), col) FROM table_map; -- { serverError 42 } DROP TABLE table_map; From 82a76d47ffa2f47076c181b4beb254fbdc46ab53 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 17 Feb 2022 14:56:29 +0000 Subject: [PATCH 067/165] Use FunctionArrayMapped for Map --- src/Functions/FunctionMapAdvance.h | 145 ------------------ src/Functions/array/FunctionArrayMapped.h | 132 ++++++++++++---- src/Functions/array/arrayAggregation.cpp | 23 ++- src/Functions/array/arrayAll.cpp | 7 +- src/Functions/array/arrayCompact.cpp | 13 +- src/Functions/array/arrayCount.cpp | 8 +- src/Functions/array/arrayCumSum.cpp | 12 +- .../array/arrayCumSumNonNegative.cpp | 11 +- src/Functions/array/arrayDifference.cpp | 12 +- src/Functions/array/arrayExists.cpp | 8 +- src/Functions/array/arrayFill.cpp | 8 +- src/Functions/array/arrayFilter.cpp | 8 +- src/Functions/array/arrayFirst.cpp | 8 +- src/Functions/array/arrayFirstLastIndex.cpp | 3 + src/Functions/array/arrayMap.cpp | 8 +- src/Functions/array/arraySort.cpp | 6 + src/Functions/array/arraySplit.cpp | 8 +- src/Functions/mapFilter.cpp | 30 +++- 18 files changed, 227 insertions(+), 223 deletions(-) delete mode 100644 src/Functions/FunctionMapAdvance.h diff --git a/src/Functions/FunctionMapAdvance.h b/src/Functions/FunctionMapAdvance.h deleted file mode 100644 index d72cd4d62e5..00000000000 --- a/src/Functions/FunctionMapAdvance.h +++ /dev/null @@ -1,145 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - - -/** Higher-order functions for map. - * These functions optionally apply a map by lambda function, - * and return some result based on that transformation. - * - * Examples: - * mapApply(x1,...,xn -> expression, map) - apply the expression to the map. - * mapFilter((k, v) -> predicate, map) - leave in the map only the kv elements for which the expression is true. - */ -template -class FunctionMapAdvance : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - - String getName() const override - { - return name; - } - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - - /// Called if at least one function argument is a lambda expression. - /// For argument-lambda expressions, it defines the types of arguments of these expressions. - void getLambdaArgumentTypes(DataTypes & arguments) const override - { - if (arguments.size() != 2) - throw Exception("Function " + getName() + " needs 2 arguments.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - DataTypes nested_types(2); - const DataTypeMap * map_type = checkAndGetDataType(&*arguments[1]); - if (!map_type) - throw Exception("The second argument of function " + getName() + " must be map. Found " - + arguments[1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - nested_types[0] = recursiveRemoveLowCardinality(map_type->getKeyType()); - nested_types[1] = recursiveRemoveLowCardinality(map_type->getValueType()); - - const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); - if (!function_type || function_type->getArgumentTypes().size() != 2) - throw Exception("First argument for this overload of " + getName() + " must be a function with 2 arguments. Found " - + arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - arguments[0] = std::make_shared(nested_types); - } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - if (arguments.size() != 2) - throw Exception("Function " + getName() + " needs at least 2 argument; passed " - + toString(arguments.size()) + ".", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); - - if (!data_type_function) - throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - /// The types of the remaining arguments are already checked in getLambdaArgumentTypes. - DataTypePtr return_type = removeLowCardinality(data_type_function->getReturnType()); - if (Impl::needBoolean() && !WhichDataType(return_type).isUInt8()) - throw Exception( - "Expression for function " + getName() + " must return UInt8, found " + return_type->getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - const auto * map_type = checkAndGetDataType(arguments[1].type.get()); - if (!map_type) - throw Exception("Second argument for function " + getName() + " must be a map.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return Impl::getReturnType(return_type, map_type->getKeyValueTypes()); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override - { - const auto & column_with_type_and_name = arguments[0]; - if (!column_with_type_and_name.column) - throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - const auto * column_function = typeid_cast(column_with_type_and_name.column.get()); - if (!column_function) - throw Exception("First argument for function " + getName() + " must be a function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - ColumnPtr column_map_ptr = arguments[1].column; - const auto * column_map = checkAndGetColumn(column_map_ptr.get()); - if (!column_map) - { - const ColumnConst * column_const_map = checkAndGetColumnConst(column_map_ptr.get()); - if (!column_const_map) - throw Exception("Expected map column, found " + column_map_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); - column_map_ptr = recursiveRemoveLowCardinality(column_const_map->convertToFullColumn()); - column_map = checkAndGetColumn(column_map_ptr.get()); - } - - const DataTypePtr & map_type_ptr = arguments[1].type; - const auto * map_type = checkAndGetDataType(map_type_ptr.get()); - const auto & offsets_column = column_map->getNestedColumn().getOffsets(); - - /// Put all the necessary columns multiplied by the sizes of maps into the columns. - auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(offsets_column)); - auto * replicated_column_function = typeid_cast(replicated_column_function_ptr.get()); - const ColumnsWithTypeAndName args { - {column_map->getNestedData().getColumnPtr(0), recursiveRemoveLowCardinality(map_type->getKeyType()), arguments[1].name + ".key"}, - {column_map->getNestedData().getColumnPtr(1), recursiveRemoveLowCardinality(map_type->getValueType()), arguments[1].name + ".value"}}; - replicated_column_function->appendArguments(args); - - auto lambda_result = replicated_column_function->reduce().column; - if (lambda_result->lowCardinality()) - lambda_result = lambda_result->convertToFullColumnIfLowCardinality(); - - return Impl::execute(*column_map, lambda_result); - } -}; - -} diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index 029e33db0cf..32fccd89244 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -1,18 +1,29 @@ #pragma once +#include + +#include +#include +#include +#include +#include + +#include +#include +#include + #include #include #include -#include -#include -#include -#include -#include -#include +#include + #include -#include +#include + #include +#include + namespace DB { @@ -26,6 +37,32 @@ namespace ErrorCodes } +template +ColumnPtr getOffsetsPtr(const T & column) +{ + if constexpr (std::is_same_v) + { + return column.getOffsetsPtr(); + } + else // ColumnMap + { + return column.getNestedColumn().getOffsetsPtr(); + } +} + +template +const IColumn::Offsets & getOffsets(const T & column) +{ + if constexpr (std::is_same_v) + { + return column.getOffsets(); + } + else // ColumnMap + { + return column.getNestedColumn().getOffsets(); + } +} + /** Higher-order functions for arrays. * These functions optionally apply a map (transform) to array (or multiple arrays of identical size) by lambda function, * and return some result based on that transformation. @@ -68,14 +105,23 @@ public: throw Exception("Function " + getName() + " needs at least one array argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - DataTypes nested_types(arguments.size() - 1); - for (size_t i = 0; i < nested_types.size(); ++i) + size_t nested_types_count = std::is_same_v ? (arguments.size() - 1) * 2 : (arguments.size() - 1); + DataTypes nested_types(nested_types_count); + for (size_t i = 0; i < arguments.size() - 1; ++i) { - const DataTypeArray * array_type = checkAndGetDataType(&*arguments[i + 1]); + const auto * array_type = checkAndGetDataType(&*arguments[i + 1]); if (!array_type) throw Exception("Argument " + toString(i + 2) + " of function " + getName() + " must be array. Found " + arguments[i + 1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - nested_types[i] = recursiveRemoveLowCardinality(array_type->getNestedType()); + if constexpr (std::is_same_v) + { + nested_types[2 * i] = recursiveRemoveLowCardinality(array_type->getKeyType()); + nested_types[2 * i + 1] = recursiveRemoveLowCardinality(array_type->getValueType()); + } + else if constexpr (std::is_same_v) + { + nested_types[i] = recursiveRemoveLowCardinality(array_type->getNestedType()); + } } const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); @@ -98,7 +144,7 @@ public: if (arguments.size() == 1) { - const auto * array_type = checkAndGetDataType(arguments[0].type.get()); + const auto * array_type = checkAndGetDataType(arguments[0].type.get()); if (!array_type) throw Exception("The only argument for function " + getName() + " must be array. Found " @@ -110,7 +156,10 @@ public: throw Exception("The only argument for function " + getName() + " must be array of UInt8. Found " + arguments[0].type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return Impl::getReturnType(nested_type, nested_type); + if constexpr (std::is_same_v) + return Impl::getReturnType(nested_type, nested_type); + else + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unreachable code reached"); } else { @@ -131,9 +180,11 @@ public: throw Exception("Expression for function " + getName() + " must return UInt8, found " + return_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - const auto * first_array_type = checkAndGetDataType(arguments[1].type.get()); - - return Impl::getReturnType(return_type, first_array_type->getNestedType()); + const auto * first_array_type = checkAndGetDataType(arguments[1].type.get()); + if constexpr (std::is_same_v) + return Impl::getReturnType(return_type, first_array_type->getNestedType()); + else + return Impl::getReturnType(return_type, first_array_type->getKeyValueTypes()); } } @@ -142,18 +193,25 @@ public: if (arguments.size() == 1) { ColumnPtr column_array_ptr = arguments[0].column; - const auto * column_array = checkAndGetColumn(column_array_ptr.get()); + const auto * column_array = checkAndGetColumn(column_array_ptr.get()); if (!column_array) { - const ColumnConst * column_const_array = checkAndGetColumnConst(column_array_ptr.get()); + const ColumnConst * column_const_array = checkAndGetColumnConst(column_array_ptr.get()); if (!column_const_array) throw Exception("Expected array column, found " + column_array_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); column_array_ptr = column_const_array->convertToFullColumn(); - column_array = assert_cast(column_array_ptr.get()); + column_array = assert_cast(column_array_ptr.get()); } - return Impl::execute(*column_array, column_array->getDataPtr()); + if constexpr (std::is_same_v) + { + return Impl::execute(*column_array, column_array->getNestedColumn().getDataPtr()); + } + else + { + return Impl::execute(*column_array, column_array->getDataPtr()); + } } else { @@ -172,7 +230,7 @@ public: ColumnPtr offsets_column; ColumnPtr column_first_array_ptr; - const ColumnArray * column_first_array = nullptr; + const typename Impl::column_type * column_first_array = nullptr; ColumnsWithTypeAndName arrays; arrays.reserve(arguments.size() - 1); @@ -182,18 +240,18 @@ public: const auto & array_with_type_and_name = arguments[i]; ColumnPtr column_array_ptr = array_with_type_and_name.column; - const auto * column_array = checkAndGetColumn(column_array_ptr.get()); + const auto * column_array = checkAndGetColumn(column_array_ptr.get()); const DataTypePtr & array_type_ptr = array_with_type_and_name.type; - const auto * array_type = checkAndGetDataType(array_type_ptr.get()); + const auto * array_type = checkAndGetDataType(array_type_ptr.get()); if (!column_array) { - const ColumnConst * column_const_array = checkAndGetColumnConst(column_array_ptr.get()); + const ColumnConst * column_const_array = checkAndGetColumnConst(column_array_ptr.get()); if (!column_const_array) throw Exception("Expected array column, found " + column_array_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); column_array_ptr = recursiveRemoveLowCardinality(column_const_array->convertToFullColumn()); - column_array = checkAndGetColumn(column_array_ptr.get()); + column_array = checkAndGetColumn(column_array_ptr.get()); } if (!array_type) @@ -201,13 +259,13 @@ public: if (!offsets_column) { - offsets_column = column_array->getOffsetsPtr(); + offsets_column = getOffsetsPtr(*column_array); } else { /// The first condition is optimization: do not compare data if the pointers are equal. - if (column_array->getOffsetsPtr() != offsets_column - && column_array->getOffsets() != typeid_cast(*offsets_column).getData()) + if (getOffsetsPtr(*column_array) != offsets_column + && getOffsets(*column_array) != typeid_cast(*offsets_column).getData()) throw Exception("Arrays passed to " + getName() + " must have equal size", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); } @@ -217,13 +275,23 @@ public: column_first_array = column_array; } - arrays.emplace_back(ColumnWithTypeAndName(column_array->getDataPtr(), - recursiveRemoveLowCardinality(array_type->getNestedType()), - array_with_type_and_name.name)); + if constexpr (std::is_same_v) + { + arrays.emplace_back(ColumnWithTypeAndName( + column_array->getNestedData().getColumnPtr(0), recursiveRemoveLowCardinality(array_type->getKeyType()), array_with_type_and_name.name+".key")); + arrays.emplace_back(ColumnWithTypeAndName( + column_array->getNestedData().getColumnPtr(1), recursiveRemoveLowCardinality(array_type->getValueType()), array_with_type_and_name.name+".value")); + } + else + { + arrays.emplace_back(ColumnWithTypeAndName(column_array->getDataPtr(), + recursiveRemoveLowCardinality(array_type->getNestedType()), + array_with_type_and_name.name)); + } } /// Put all the necessary columns multiplied by the sizes of arrays into the columns. - auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(column_first_array->getOffsets())); + auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(getOffsets(*column_first_array))); auto * replicated_column_function = typeid_cast(replicated_column_function_ptr.get()); replicated_column_function->appendArguments(arrays); diff --git a/src/Functions/array/arrayAggregation.cpp b/src/Functions/array/arrayAggregation.cpp index ee08c4f7f37..97a2f9c4c17 100644 --- a/src/Functions/array/arrayAggregation.cpp +++ b/src/Functions/array/arrayAggregation.cpp @@ -1,12 +1,18 @@ -#include -#include -#include -#include -#include -#include "FunctionArrayMapped.h" -#include #include +#include +#include +#include + +#include +#include +#include +#include + +#include + +#include "FunctionArrayMapped.h" + namespace DB { @@ -83,6 +89,9 @@ using ArrayAggregateResult = typename ArrayAggregateResultImpl struct ArrayAggregateImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayAll.cpp b/src/Functions/array/arrayAll.cpp index 34deafdffdf..0f7ae797dc9 100644 --- a/src/Functions/array/arrayAll.cpp +++ b/src/Functions/array/arrayAll.cpp @@ -1,8 +1,8 @@ -#include #include -#include "FunctionArrayMapped.h" +#include #include +#include "FunctionArrayMapped.h" namespace DB { @@ -16,6 +16,9 @@ namespace ErrorCodes */ struct ArrayAllImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return true; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayCompact.cpp b/src/Functions/array/arrayCompact.cpp index c2908e37e12..aed5e7275d8 100644 --- a/src/Functions/array/arrayCompact.cpp +++ b/src/Functions/array/arrayCompact.cpp @@ -1,10 +1,10 @@ -#include -#include -#include #include -#include -#include +#include +#include +#include #include +#include +#include namespace DB @@ -16,6 +16,9 @@ namespace ErrorCodes struct ArrayCompactImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayCount.cpp b/src/Functions/array/arrayCount.cpp index 377a6eb8fb1..df45783323b 100644 --- a/src/Functions/array/arrayCount.cpp +++ b/src/Functions/array/arrayCount.cpp @@ -1,8 +1,9 @@ -#include #include -#include "FunctionArrayMapped.h" +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -16,6 +17,9 @@ namespace ErrorCodes */ struct ArrayCountImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return true; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayCumSum.cpp b/src/Functions/array/arrayCumSum.cpp index 467d9ad3951..98ffa09820b 100644 --- a/src/Functions/array/arrayCumSum.cpp +++ b/src/Functions/array/arrayCumSum.cpp @@ -1,10 +1,11 @@ -#include -#include -#include #include -#include "FunctionArrayMapped.h" +#include +#include +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -17,6 +18,9 @@ namespace ErrorCodes struct ArrayCumSumImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayCumSumNonNegative.cpp b/src/Functions/array/arrayCumSumNonNegative.cpp index 476bbd08163..cd8393b7a5f 100644 --- a/src/Functions/array/arrayCumSumNonNegative.cpp +++ b/src/Functions/array/arrayCumSumNonNegative.cpp @@ -1,10 +1,10 @@ -#include -#include -#include #include -#include "FunctionArrayMapped.h" +#include +#include +#include #include +#include "FunctionArrayMapped.h" namespace DB { @@ -19,6 +19,9 @@ namespace ErrorCodes */ struct ArrayCumSumNonNegativeImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayDifference.cpp b/src/Functions/array/arrayDifference.cpp index c5fdf27100b..8af0e8b04f9 100644 --- a/src/Functions/array/arrayDifference.cpp +++ b/src/Functions/array/arrayDifference.cpp @@ -1,10 +1,11 @@ -#include -#include -#include #include -#include "FunctionArrayMapped.h" +#include +#include +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -20,6 +21,9 @@ namespace ErrorCodes */ struct ArrayDifferenceImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayExists.cpp b/src/Functions/array/arrayExists.cpp index 34ea71af259..ea39cc0dc0b 100644 --- a/src/Functions/array/arrayExists.cpp +++ b/src/Functions/array/arrayExists.cpp @@ -1,8 +1,9 @@ -#include #include -#include "FunctionArrayMapped.h" +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -16,6 +17,9 @@ namespace ErrorCodes */ struct ArrayExistsImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return true; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayFill.cpp b/src/Functions/array/arrayFill.cpp index d4b36a89ba5..22b9e9a657b 100644 --- a/src/Functions/array/arrayFill.cpp +++ b/src/Functions/array/arrayFill.cpp @@ -1,8 +1,9 @@ -#include #include -#include "FunctionArrayMapped.h" +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -19,6 +20,9 @@ namespace ErrorCodes template struct ArrayFillImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return true; } static bool needExpression() { return true; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayFilter.cpp b/src/Functions/array/arrayFilter.cpp index 1291989f9a2..89a9de44532 100644 --- a/src/Functions/array/arrayFilter.cpp +++ b/src/Functions/array/arrayFilter.cpp @@ -1,8 +1,9 @@ -#include #include -#include "FunctionArrayMapped.h" +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -15,6 +16,9 @@ namespace ErrorCodes */ struct ArrayFilterImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return true; } static bool needExpression() { return true; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayFirst.cpp b/src/Functions/array/arrayFirst.cpp index edbf7ef6269..693aea746f5 100644 --- a/src/Functions/array/arrayFirst.cpp +++ b/src/Functions/array/arrayFirst.cpp @@ -1,8 +1,9 @@ -#include #include -#include "FunctionArrayMapped.h" +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -20,6 +21,9 @@ enum class ArrayFirstLastStrategy template struct ArrayFirstLastImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return true; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayFirstLastIndex.cpp b/src/Functions/array/arrayFirstLastIndex.cpp index 467678f3faa..f5570d37ba4 100644 --- a/src/Functions/array/arrayFirstLastIndex.cpp +++ b/src/Functions/array/arrayFirstLastIndex.cpp @@ -20,6 +20,9 @@ enum class ArrayFirstLastIndexStrategy template struct ArrayFirstLastIndexImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return true; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arrayMap.cpp b/src/Functions/array/arrayMap.cpp index e3afaf7fb66..ec1973d573b 100644 --- a/src/Functions/array/arrayMap.cpp +++ b/src/Functions/array/arrayMap.cpp @@ -1,14 +1,18 @@ -#include "FunctionArrayMapped.h" #include +#include "FunctionArrayMapped.h" + namespace DB { -/** arrayMap(x1,...,xn -> expression, array1,...,arrayn) - apply the expression to each element of the array (or set of parallel arrays). +/** arrayMap(x1, ..., xn -> expression, array1, ..., arrayn) - apply the expression to each element of the array (or set of parallel arrays). */ struct ArrayMapImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + /// true if the expression (for an overload of f(expression, arrays)) or an array (for f(array)) should be boolean. static bool needBoolean() { return false; } /// true if the f(array) overload is unavailable. diff --git a/src/Functions/array/arraySort.cpp b/src/Functions/array/arraySort.cpp index 476dfb46f07..d87b941ab21 100644 --- a/src/Functions/array/arraySort.cpp +++ b/src/Functions/array/arraySort.cpp @@ -1,7 +1,10 @@ #include "FunctionArrayMapped.h" + #include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -11,6 +14,9 @@ namespace DB template struct ArraySortImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return false; } static bool needExpression() { return false; } static bool needOneArray() { return false; } diff --git a/src/Functions/array/arraySplit.cpp b/src/Functions/array/arraySplit.cpp index 2e5f2d8432e..c818be97f60 100644 --- a/src/Functions/array/arraySplit.cpp +++ b/src/Functions/array/arraySplit.cpp @@ -1,8 +1,9 @@ -#include #include -#include "FunctionArrayMapped.h" +#include #include +#include "FunctionArrayMapped.h" + namespace DB { @@ -14,6 +15,9 @@ namespace ErrorCodes template struct ArraySplitImpl { + using column_type = ColumnArray; + using data_type = DataTypeArray; + static bool needBoolean() { return true; } static bool needExpression() { return true; } static bool needOneArray() { return false; } diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp index 69bbf0c00f3..2308b22d3db 100644 --- a/src/Functions/mapFilter.cpp +++ b/src/Functions/mapFilter.cpp @@ -1,24 +1,37 @@ -#include -#include +#include #include -#include +#include +#include #include #include +#include namespace DB { + namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -/** MapFilter(x -> predicate, array) - leave in the array only the elements for which the expression is true. +/** Higher-order functions for map. + * These functions optionally apply a map by lambda function, + * and return some result based on that transformation. + */ + + +/** mapFilter((k, v) -> predicate, map) - leave in the map only the kv elements for which the expression is true. */ struct MapFilterImpl { + using data_type = DataTypeMap; + using column_type = ColumnMap; + static bool needBoolean() { return true; } + static bool needExpression() { return true; } + static bool needOneArray() { return false; } static DataTypePtr getReturnType(const DataTypePtr & /*expression_return*/, const DataTypes & elems) { @@ -73,7 +86,7 @@ struct MapFilterImpl }; struct NameMapFilter { static constexpr auto name = "mapFilter"; }; -using FunctionMapFilter = FunctionMapAdvance; +using FunctionMapFilter = FunctionArrayMapped; void registerFunctionMapFilter(FunctionFactory & factory) { @@ -85,8 +98,13 @@ void registerFunctionMapFilter(FunctionFactory & factory) */ struct MapApplyImpl { + using data_type = DataTypeMap; + using column_type = ColumnMap; + /// true if the expression (for an overload of f(expression, maps)) or a map (for f(map)) should be boolean. static bool needBoolean() { return false; } + static bool needExpression() { return true; } + static bool needOneArray() { return false; } static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypes & /*elems*/) { @@ -116,7 +134,7 @@ struct MapApplyImpl }; struct NameMapApply { static constexpr auto name = "mapApply"; }; -using FunctionMapApply = FunctionMapAdvance; +using FunctionMapApply = FunctionArrayMapped; void registerFunctionMapApply(FunctionFactory & factory) { From 939a15d29a0e2b2d4739d0b7fb32e2bfdc4f0560 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 17 Feb 2022 15:40:26 +0000 Subject: [PATCH 068/165] Upd FunctionArrayMapped for Map --- src/Functions/array/FunctionArrayMapped.h | 19 +++++++++- src/Functions/mapFilter.cpp | 38 +++++++++---------- .../registerFunctionsHigherOrder.cpp | 2 - .../0_stateless/02169_map_functions.reference | 5 ++- .../0_stateless/02169_map_functions.sql | 19 +++++++--- 5 files changed, 53 insertions(+), 30 deletions(-) diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index 32fccd89244..28540354b94 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -180,11 +180,28 @@ public: throw Exception("Expression for function " + getName() + " must return UInt8, found " + return_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + static_assert( + std::is_same_v || + std::is_same_v, + "unsupported type"); + + if (arguments.size() < 2) + { + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "{}", arguments.size()); + } + const auto * first_array_type = checkAndGetDataType(arguments[1].type.get()); + + if (!first_array_type) + throw DB::Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Unsupported type {}", arguments[1].type->getName()); + if constexpr (std::is_same_v) return Impl::getReturnType(return_type, first_array_type->getNestedType()); - else + + if constexpr (std::is_same_v) return Impl::getReturnType(return_type, first_array_type->getKeyValueTypes()); + + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unreachable code reached"); } } diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp index 2308b22d3db..78a7934b2ba 100644 --- a/src/Functions/mapFilter.cpp +++ b/src/Functions/mapFilter.cpp @@ -13,6 +13,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -29,6 +30,8 @@ struct MapFilterImpl using data_type = DataTypeMap; using column_type = ColumnMap; + static constexpr auto name = "mapFilter"; + static bool needBoolean() { return true; } static bool needExpression() { return true; } static bool needOneArray() { return false; } @@ -85,14 +88,6 @@ struct MapFilterImpl } }; -struct NameMapFilter { static constexpr auto name = "mapFilter"; }; -using FunctionMapFilter = FunctionArrayMapped; - -void registerFunctionMapFilter(FunctionFactory & factory) -{ - factory.registerFunction(); -} - /** mapApply((k,v) -> expression, map) - apply the expression to the map. */ @@ -101,6 +96,8 @@ struct MapApplyImpl using data_type = DataTypeMap; using column_type = ColumnMap; + static constexpr auto name = "mapApply"; + /// true if the expression (for an overload of f(expression, maps)) or a map (for f(map)) should be boolean. static bool needBoolean() { return false; } static bool needExpression() { return true; } @@ -108,12 +105,15 @@ struct MapApplyImpl static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypes & /*elems*/) { - const auto & tuple_types = typeid_cast(&*expression_return)->getElements(); - if (tuple_types.size() != 2) - throw Exception("Expected 2 columns as map's key and value, but found " - + toString(tuple_types.size()) + " columns", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + const auto * tuple_types = typeid_cast(expression_return.get()); + if (!tuple_types) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Expected return type is tuple, got {}", expression_return->getName()); + if (tuple_types->getElements().size() != 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Expected 2 columns as map's key and value, but found {}", tuple_types->getElements().size()); - return std::make_shared(tuple_types); + return std::make_shared(tuple_types->getElements()); } static ColumnPtr execute(const ColumnMap & map, ColumnPtr mapped) @@ -123,9 +123,9 @@ struct MapApplyImpl { const ColumnConst * column_const_tuple = checkAndGetColumnConst(mapped.get()); if (!column_const_tuple) - throw Exception("Expected tuple column, found " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN); - ColumnPtr column_tuple_ptr = recursiveRemoveLowCardinality(column_const_tuple->convertToFullColumn()); - column_tuple = checkAndGetColumn(column_tuple_ptr.get()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected tuple column, found {}", mapped->getName()); + auto cols = convertConstTupleToConstantElements(*column_const_tuple); + return ColumnMap::create(cols[0]->convertToFullColumnIfConst(), cols[1]->convertToFullColumnIfConst(), map.getNestedColumn().getOffsetsPtr()); } return ColumnMap::create(column_tuple->getColumnPtr(0), column_tuple->getColumnPtr(1), @@ -133,12 +133,10 @@ struct MapApplyImpl } }; -struct NameMapApply { static constexpr auto name = "mapApply"; }; -using FunctionMapApply = FunctionArrayMapped; - void registerFunctionMapApply(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction>(); + factory.registerFunction>(); } } diff --git a/src/Functions/registerFunctionsHigherOrder.cpp b/src/Functions/registerFunctionsHigherOrder.cpp index 00b300b18b9..00bea58b918 100644 --- a/src/Functions/registerFunctionsHigherOrder.cpp +++ b/src/Functions/registerFunctionsHigherOrder.cpp @@ -18,7 +18,6 @@ void registerFunctionsArraySort(FunctionFactory & factory); void registerFunctionArrayCumSum(FunctionFactory & factory); void registerFunctionArrayCumSumNonNegative(FunctionFactory & factory); void registerFunctionArrayDifference(FunctionFactory & factory); -void registerFunctionMapFilter(FunctionFactory & factory); void registerFunctionMapApply(FunctionFactory & factory); void registerFunctionsHigherOrder(FunctionFactory & factory) @@ -38,7 +37,6 @@ void registerFunctionsHigherOrder(FunctionFactory & factory) registerFunctionArrayCumSum(factory); registerFunctionArrayCumSumNonNegative(factory); registerFunctionArrayDifference(factory); - registerFunctionMapFilter(factory); registerFunctionMapApply(factory); } diff --git a/tests/queries/0_stateless/02169_map_functions.reference b/tests/queries/0_stateless/02169_map_functions.reference index c570ba4e724..160aebbc852 100644 --- a/tests/queries/0_stateless/02169_map_functions.reference +++ b/tests/queries/0_stateless/02169_map_functions.reference @@ -19,7 +19,6 @@ {'key1':1112,'key2':2223} {'key1':1113,'key2':2225} {'key1':1114,'key2':2227} -{1:2,2:3} {} {} {} @@ -28,3 +27,7 @@ {} {} {3:2,1:0,2:0} +{1:2,2:3} +{1:2,2:3} +{'x':'y','x':'y'} +{'x':'y','x':'y'} diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index 1d8f90e8a90..ee2e70f82cd 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -1,13 +1,20 @@ DROP TABLE IF EXISTS table_map; -create TABLE table_map (id UInt32, col Map(String, UInt64)) engine = MergeTree() ORDER BY tuple(); +CREATE TABLE table_map (id UInt32, col Map(String, UInt64)) engine = MergeTree() ORDER BY tuple(); INSERT INTO table_map SELECT number, map('key1', number, 'key2', number * 2) FROM numbers(1111, 3); INSERT INTO table_map SELECT number, map('key3', number, 'key2', number + 1, 'key4', number + 2) FROM numbers(100, 4); -SELECT mapFilter((k,v)->k like '%3' and v > 102, col) FROM table_map ORDER BY id; +SELECT mapFilter((k, v) -> k like '%3' and v > 102, col) FROM table_map ORDER BY id; SELECT col, mapFilter((k, v) -> ((v % 10) > 1), col) FROM table_map ORDER BY id ASC; -SELECT mapApply((k,v)->(k,v+1), col) FROM table_map ORDER BY id; -SELECT mapApply((x, y) -> (x, x + 1), map(1, 0, 2, 0)); -SELECT mapFilter((k,v)->0, col) from table_map; +SELECT mapApply((k, v) -> (k, v + 1), col) FROM table_map ORDER BY id; +SELECT mapFilter((k, v) -> 0, col) from table_map; +SELECT mapApply((k, v) -> tuple(v + 9223372036854775806), col) FROM table_map; -- { serverError 42 } + SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0)); -SELECT mapApply((k, v) -> tuple(v + 9223372036854775806), col) FROM table_map; -- { serverError 42 } +SELECT mapApply((x, y) -> (x, x + 1), map(1, 0, 2, 0)); +SELECT mapApply((x, y) -> (x, x + 1), materialize(map(1, 0, 2, 0))); +SELECT mapApply((x, y) -> ('x', 'y'), map(1, 0, 2, 0)); +SELECT mapApply((x, y) -> ('x', 'y'), materialize(map(1, 0, 2, 0))); +SELECT mapApply((x, y) -> (x), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapApply((x, y) -> ('x'), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + DROP TABLE table_map; From f7f002139a6d266eb1c9a30c5c6e39d257380a78 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 23 Feb 2022 09:22:40 +0000 Subject: [PATCH 069/165] Upd FunctionArrayMapped --- src/Functions/array/FunctionArrayMapped.h | 2 +- src/Functions/mapFilter.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index 28540354b94..e32b4c8e1f3 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -142,7 +142,7 @@ public: + toString(arguments.size()) + ".", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (arguments.size() == 1) + if ((arguments.size() == 1) && std::is_same_v) { const auto * array_type = checkAndGetDataType(arguments[0].type.get()); diff --git a/src/Functions/mapFilter.cpp b/src/Functions/mapFilter.cpp index 78a7934b2ba..f38f8f8b4d1 100644 --- a/src/Functions/mapFilter.cpp +++ b/src/Functions/mapFilter.cpp @@ -34,7 +34,7 @@ struct MapFilterImpl static bool needBoolean() { return true; } static bool needExpression() { return true; } - static bool needOneArray() { return false; } + static bool needOneArray() { return true; } static DataTypePtr getReturnType(const DataTypePtr & /*expression_return*/, const DataTypes & elems) { @@ -101,7 +101,7 @@ struct MapApplyImpl /// true if the expression (for an overload of f(expression, maps)) or a map (for f(map)) should be boolean. static bool needBoolean() { return false; } static bool needExpression() { return true; } - static bool needOneArray() { return false; } + static bool needOneArray() { return true; } static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypes & /*elems*/) { From 95530d0d70e3ae80c340721c422dbc523ab09a53 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Wed, 23 Feb 2022 15:18:18 +0800 Subject: [PATCH 070/165] fix style error --- src/Functions/array/FunctionArrayMapped.h | 1 + src/Functions/array/arrayCompact.cpp | 5 ++++- src/Functions/array/arrayFirstLastIndex.cpp | 3 ++- 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index e32b4c8e1f3..df34abccab9 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -32,6 +32,7 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int LOGICAL_ERROR; extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } diff --git a/src/Functions/array/arrayCompact.cpp b/src/Functions/array/arrayCompact.cpp index aed5e7275d8..a2cfb1cdccd 100644 --- a/src/Functions/array/arrayCompact.cpp +++ b/src/Functions/array/arrayCompact.cpp @@ -1,10 +1,13 @@ #include #include + +#include + #include #include + #include #include -#include namespace DB diff --git a/src/Functions/array/arrayFirstLastIndex.cpp b/src/Functions/array/arrayFirstLastIndex.cpp index f5570d37ba4..9392cbdc840 100644 --- a/src/Functions/array/arrayFirstLastIndex.cpp +++ b/src/Functions/array/arrayFirstLastIndex.cpp @@ -1,8 +1,9 @@ #include #include -#include "FunctionArrayMapped.h" #include +#include "FunctionArrayMapped.h" + namespace DB { From 33baefb12bcd4b56ce60ed5d20d03e9f843c174a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 3 Mar 2022 14:35:52 +0100 Subject: [PATCH 071/165] Done --- src/Common/Config/ConfigProcessor.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index b86e8ed3e40..1070430b842 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -664,6 +664,10 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, new_path.erase(0, main_config_path.size()); std::replace(new_path.begin(), new_path.end(), '/', '_'); + /// If we have config file in YAML format, the preprocessed config will inherit .yaml extension + /// but will contain config in XML format, so some tools like clickhouse extract-from-config won't work + new_path = fs::path(new_path).replace_extension(".xml").string(); + if (preprocessed_dir.empty()) { if (!loaded_config.configuration->has("path")) From c33808d3db3ad9b637578e17ef365cc9de4e64e7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 3 Mar 2022 15:18:03 +0100 Subject: [PATCH 072/165] Fix read old records from logs --- src/Coordination/KeeperStateMachine.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index e3d99d4775b..a9b5fc85828 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -39,7 +39,9 @@ namespace request_for_session.request->xid = xid; request_for_session.request->readImpl(buffer); - readIntBinary(request_for_session.time, buffer); + if (!buffer.eof()) + readIntBinary(request_for_session.time, buffer); + return request_for_session; } } From d19f199e93ccd64fb07e19f5e0f4cc3e5ab2963f Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 3 Mar 2022 15:25:27 +0100 Subject: [PATCH 073/165] Revert --- .../FileLog/ReadBufferFromFileLog.cpp | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 49 ++++++------------- src/Storages/FileLog/StorageFileLog.h | 3 +- 3 files changed, 18 insertions(+), 36 deletions(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index ebb0925f128..86caac57a53 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -134,7 +134,7 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ UInt64 current_position = reader.tellg(); StorageFileLog::assertStreamGood(reader); - file_meta.last_written_position = current_position; + file_meta.last_writen_position = current_position; /// stream reach to end if (current_position == file_meta.last_open_end) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 07853677222..b43f47eb6c5 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -232,12 +232,12 @@ void StorageFileLog::serialize() const } else { - checkOffsetIsValid(full_name, meta.last_written_position); + checkOffsetIsValid(full_name, meta.last_writen_position); } WriteBufferFromFile out(full_name); writeIntText(inode, out); writeChar('\n', out); - writeIntText(meta.last_written_position, out); + writeIntText(meta.last_writen_position, out); } } @@ -254,12 +254,12 @@ void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const } else { - checkOffsetIsValid(full_name, file_meta.last_written_position); + checkOffsetIsValid(full_name, file_meta.last_writen_position); } WriteBufferFromFile out(full_name); writeIntText(inode, out); writeChar('\n', out); - writeIntText(file_meta.last_written_position, out); + writeIntText(file_meta.last_writen_position, out); } void StorageFileLog::deserialize() @@ -294,7 +294,7 @@ void StorageFileLog::deserialize() } meta.file_name = dir_entry.path().filename(); - meta.last_written_position = last_written_pos; + meta.last_writen_position = last_written_pos; file_infos.meta_by_inode.emplace(inode, meta); } @@ -445,17 +445,7 @@ void StorageFileLog::openFilesAndSetPos() auto & file_ctx = findInMap(file_infos.context_by_name, file); if (file_ctx.status != FileStatus::NO_CHANGE) { - auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); - - auto current_path = getFullDataPath(file); - if (!std::filesystem::exists(file) && meta.last_written_position != 0) - { - file_ctx.status = FileStatus::REMOVED; - continue; - } - - file_ctx.reader.emplace(current_path); - + file_ctx.reader.emplace(getFullDataPath(file)); auto & reader = file_ctx.reader.value(); assertStreamGood(reader); @@ -465,24 +455,23 @@ void StorageFileLog::openFilesAndSetPos() auto file_end = reader.tellg(); assertStreamGood(reader); - if (meta.last_written_position > static_cast(file_end)) + auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); + if (meta.last_writen_position > static_cast(file_end)) { throw Exception( ErrorCodes::CANNOT_READ_ALL_DATA, "Last saved offsset for File {} is bigger than file size ({} > {})", file, - meta.last_written_position, + meta.last_writen_position, file_end); } /// update file end at the moment, used in ReadBuffer and serialize meta.last_open_end = file_end; - reader.seekg(meta.last_written_position); + reader.seekg(meta.last_writen_position); assertStreamGood(reader); } } - - removeInvalidFiles(); serialize(); } @@ -951,18 +940,6 @@ bool StorageFileLog::updateFileInfos() } } } - - removeInvalidFiles(); - - /// These file infos should always have same size(one for one) - assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); - assert(file_infos.file_names.size() == file_infos.context_by_name.size()); - - return events.empty() || file_infos.file_names.empty(); -} - -void StorageFileLog::removeInvalidFiles() -{ std::vector valid_files; /// Remove file infos with REMOVE status @@ -992,6 +969,12 @@ void StorageFileLog::removeInvalidFiles() } } file_infos.file_names.swap(valid_files); + + /// These file infos should always have same size(one for one) + assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); + assert(file_infos.file_names.size() == file_infos.context_by_name.size()); + + return events.empty() || file_infos.file_names.empty(); } NamesAndTypesList StorageFileLog::getVirtuals() const diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 9eaac3e8eb3..98915f10a05 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -72,7 +72,7 @@ public: struct FileMeta { String file_name; - UInt64 last_written_position = 0; + UInt64 last_writen_position = 0; UInt64 last_open_end = 0; }; @@ -205,7 +205,6 @@ private: void deserialize(); static void checkOffsetIsValid(const String & full_name, UInt64 offset); - void removeInvalidFiles(); }; } From 18fabccff25cb78e700c713bf8f4b69b12f92710 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 3 Mar 2022 16:13:05 +0000 Subject: [PATCH 074/165] Functions dictGetHierarchy, dictIsIn, dictGetChildren, dictGetDescendants support implicit key cast and constant arguments --- src/Functions/FunctionsExternalDictionaries.h | 156 +++++++++--------- ...erarchical_dictionaries_constant.reference | 32 ++++ ...231_hierarchical_dictionaries_constant.sql | 54 ++++++ 3 files changed, 165 insertions(+), 77 deletions(-) create mode 100644 tests/queries/0_stateless/02231_hierarchical_dictionaries_constant.reference create mode 100644 tests/queries/0_stateless/02231_hierarchical_dictionaries_constant.sql diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index fb0dbdfff5c..7e26de574aa 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -90,6 +90,22 @@ public: return getDictionary(dict_name_col->getValue()); } + static const DictionaryAttribute & getDictionaryHierarchicalAttribute(const std::shared_ptr & dictionary) + { + const auto & dictionary_structure = dictionary->getStructure(); + auto hierarchical_attribute_index_optional = dictionary_structure.hierarchical_attribute_index; + + if (!dictionary->hasHierarchy() || !hierarchical_attribute_index_optional.has_value()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Dictionary {} does not support hierarchy", + dictionary->getFullName()); + + size_t hierarchical_attribute_index = *hierarchical_attribute_index_optional; + const auto & hierarchical_attribute = dictionary_structure.attributes[hierarchical_attribute_index]; + + return hierarchical_attribute; + } + bool isDictGetFunctionInjective(const Block & sample_columns) { /// Assume non-injective by default @@ -939,39 +955,38 @@ private: bool useDefaultImplementationForConstants() const final { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isString(arguments[0])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type of first argument of function {}. Expected String. Actual type {}", - getName(), - arguments[0]->getName()); - - if (!WhichDataType(arguments[1]).isUInt64()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type of second argument of function {}. Expected UInt64. Actual type {}", - getName(), - arguments[1]->getName()); - - return std::make_shared(std::make_shared()); - } - bool isDeterministic() const override { return false; } + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + String dictionary_name; + if (const auto * name_col = checkAndGetColumnConst(arguments[0].column.get())) + dictionary_name = name_col->getValue(); + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, expected a const string.", + arguments[0].type->getName(), + getName()); + + auto dictionary = helper.getDictionary(arguments[0].column); + const auto & hierarchical_attribute = helper.getDictionaryHierarchicalAttribute(dictionary); + + return std::make_shared(hierarchical_attribute.type); + } + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { if (input_rows_count == 0) return result_type->createColumn(); auto dictionary = helper.getDictionary(arguments[0].column); + const auto & hierarchical_attribute = helper.getDictionaryHierarchicalAttribute(dictionary); - if (!dictionary->hasHierarchy()) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Dictionary {} does not support hierarchy", - dictionary->getFullName()); + auto key_column = ColumnWithTypeAndName{arguments[1].column, arguments[1].type, arguments[1].name}; + auto key_column_casted = castColumnAccurate(key_column, hierarchical_attribute.type); + + ColumnPtr result = dictionary->getHierarchy(key_column_casted, hierarchical_attribute.type); - ColumnPtr result = dictionary->getHierarchy(arguments[1].column, std::make_shared()); return result; } @@ -1009,18 +1024,6 @@ private: getName(), arguments[0]->getName()); - if (!WhichDataType(arguments[1]).isUInt64()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type of second argument of function {}. Expected UInt64. Actual type {}", - getName(), - arguments[1]->getName()); - - if (!WhichDataType(arguments[2]).isUInt64()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type of third argument of function {}. Expected UInt64. Actual type {}", - getName(), - arguments[2]->getName()); - return std::make_shared(); } @@ -1031,16 +1034,18 @@ private: if (input_rows_count == 0) return result_type->createColumn(); - auto dict = helper.getDictionary(arguments[0].column); + auto dictionary = helper.getDictionary(arguments[0].column); + const auto & hierarchical_attribute = helper.getDictionaryHierarchicalAttribute(dictionary); - if (!dict->hasHierarchy()) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Dictionary {} does not support hierarchy", - dict->getFullName()); + auto key_column = ColumnWithTypeAndName{arguments[1].column->convertToFullColumnIfConst(), arguments[1].type, arguments[2].name}; + auto in_key_column = ColumnWithTypeAndName{arguments[2].column->convertToFullColumnIfConst(), arguments[2].type, arguments[2].name}; - ColumnPtr res = dict->isInHierarchy(arguments[1].column, arguments[2].column, std::make_shared()); + auto key_column_casted = castColumnAccurate(key_column, hierarchical_attribute.type); + auto in_key_column_casted = castColumnAccurate(in_key_column, hierarchical_attribute.type); - return res; + ColumnPtr result = dictionary->isInHierarchy(key_column_casted, in_key_column_casted, hierarchical_attribute.type); + + return result; } mutable FunctionDictHelper helper; @@ -1069,21 +1074,18 @@ private: bool isDeterministic() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isString(arguments[0])) + if (!isString(arguments[0].type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type of first argument of function {}. Expected String. Actual type {}", getName(), - arguments[0]->getName()); + arguments[0].type->getName()); - if (!WhichDataType(arguments[1]).isUInt64()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type of second argument of function {}. Expected UInt64. Actual type {}", - getName(), - arguments[1]->getName()); + auto dictionary = helper.getDictionary(arguments[0].column); + const auto & hierarchical_attribute = helper.getDictionaryHierarchicalAttribute(dictionary); - return std::make_shared(std::make_shared()); + return std::make_shared(hierarchical_attribute.type); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override @@ -1092,13 +1094,12 @@ private: return result_type->createColumn(); auto dictionary = helper.getDictionary(arguments[0].column); + const auto & hierarchical_attribute = helper.getDictionaryHierarchicalAttribute(dictionary); - if (!dictionary->hasHierarchy()) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Dictionary {} does not support hierarchy", - dictionary->getFullName()); + auto key_column = ColumnWithTypeAndName{arguments[1].column->convertToFullColumnIfConst(), arguments[1].type, arguments[1].name}; + auto key_column_casted = castColumnAccurate(key_column, hierarchical_attribute.type); - ColumnPtr result = dictionary->getDescendants(arguments[1].column, std::make_shared(), 1); + ColumnPtr result = dictionary->getDescendants(key_column_casted, hierarchical_attribute.type, 1); return result; } @@ -1126,12 +1127,11 @@ private: bool isVariadic() const override { return true; } bool useDefaultImplementationForConstants() const final { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 2}; } bool isDeterministic() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { size_t arguments_size = arguments.size(); if (arguments_size < 2 || arguments_size > 3) @@ -1142,27 +1142,24 @@ private: arguments_size); } - if (!isString(arguments[0])) + if (!isString(arguments[0].type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type of first argument of function {}. Expected const String. Actual type {}", getName(), - arguments[0]->getName()); + arguments[0].type->getName()); - if (!WhichDataType(arguments[1]).isUInt64()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type of second argument of function {}. Expected UInt64. Actual type {}", - getName(), - arguments[1]->getName()); - - if (arguments.size() == 3 && !isUnsignedInteger(arguments[2])) + if (arguments.size() == 3 && !isInteger(arguments[2].type)) { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type of third argument of function {}. Expected const unsigned integer. Actual type {}", getName(), - arguments[2]->getName()); + arguments[2].type->getName()); } - return std::make_shared(std::make_shared()); + auto dictionary = helper.getDictionary(arguments[0].column); + const auto & hierarchical_attribute = helper.getDictionaryHierarchicalAttribute(dictionary); + + return std::make_shared(hierarchical_attribute.type); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override @@ -1171,6 +1168,7 @@ private: return result_type->createColumn(); auto dictionary = helper.getDictionary(arguments[0].column); + const auto & hierarchical_attribute = helper.getDictionaryHierarchicalAttribute(dictionary); size_t level = 0; @@ -1181,17 +1179,21 @@ private: "Illegal type of third argument of function {}. Expected const unsigned integer.", getName()); - level = static_cast(arguments[2].column->get64(0)); + auto value = static_cast(arguments[2].column->getInt(0)); + if (value < 0) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type of third argument of function {}. Expected const unsigned integer.", + getName()); + + level = static_cast(value); } - if (!dictionary->hasHierarchy()) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Dictionary {} does not support hierarchy", - dictionary->getFullName()); + auto key_column = ColumnWithTypeAndName{arguments[1].column->convertToFullColumnIfConst(), arguments[1].type, arguments[1].name}; + auto key_column_casted = castColumnAccurate(key_column, hierarchical_attribute.type); - ColumnPtr res = dictionary->getDescendants(arguments[1].column, std::make_shared(), level); + ColumnPtr result = dictionary->getDescendants(key_column_casted, hierarchical_attribute.type, level); - return res; + return result; } mutable FunctionDictHelper helper; diff --git a/tests/queries/0_stateless/02231_hierarchical_dictionaries_constant.reference b/tests/queries/0_stateless/02231_hierarchical_dictionaries_constant.reference new file mode 100644 index 00000000000..bccd9864b30 --- /dev/null +++ b/tests/queries/0_stateless/02231_hierarchical_dictionaries_constant.reference @@ -0,0 +1,32 @@ +Get hierarchy +[] +[1] +[2,1] +[3,1] +[4,2,1] +[] +Get is in hierarchy +1 +1 +0 +Get children +[1] +[2,3] +[4] +[] +[] +[] +Get all descendants +[1,2,3,4] +[2,3,4] +[4] +[] +[] +[] +Get descendants at first level +[1] +[2,3] +[4] +[] +[] +[] diff --git a/tests/queries/0_stateless/02231_hierarchical_dictionaries_constant.sql b/tests/queries/0_stateless/02231_hierarchical_dictionaries_constant.sql new file mode 100644 index 00000000000..bc01b447338 --- /dev/null +++ b/tests/queries/0_stateless/02231_hierarchical_dictionaries_constant.sql @@ -0,0 +1,54 @@ +DROP TABLE IF EXISTS hierarchy_source_table; +CREATE TABLE hierarchy_source_table (id UInt64, parent_id UInt64) ENGINE = TinyLog; +INSERT INTO hierarchy_source_table VALUES (1, 0), (2, 1), (3, 1), (4, 2); + +DROP DICTIONARY IF EXISTS hierarchy_flat_dictionary; +CREATE DICTIONARY hierarchy_flat_dictionary +( + id UInt64, + parent_id UInt64 HIERARCHICAL +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE 'hierarchy_source_table')) +LAYOUT(FLAT()) +LIFETIME(MIN 1 MAX 1000); + +SELECT 'Get hierarchy'; +SELECT dictGetHierarchy('hierarchy_flat_dictionary', 0); +SELECT dictGetHierarchy('hierarchy_flat_dictionary', 1); +SELECT dictGetHierarchy('hierarchy_flat_dictionary', 2); +SELECT dictGetHierarchy('hierarchy_flat_dictionary', 3); +SELECT dictGetHierarchy('hierarchy_flat_dictionary', 4); +SELECT dictGetHierarchy('hierarchy_flat_dictionary', 5); + +SELECT 'Get is in hierarchy'; +SELECT dictIsIn('hierarchy_flat_dictionary', 1, 1); +SELECT dictIsIn('hierarchy_flat_dictionary', 2, 1); +SELECT dictIsIn('hierarchy_flat_dictionary', 2, 0); + +SELECT 'Get children'; +SELECT dictGetChildren('hierarchy_flat_dictionary', 0); +SELECT dictGetChildren('hierarchy_flat_dictionary', 1); +SELECT dictGetChildren('hierarchy_flat_dictionary', 2); +SELECT dictGetChildren('hierarchy_flat_dictionary', 3); +SELECT dictGetChildren('hierarchy_flat_dictionary', 4); +SELECT dictGetChildren('hierarchy_flat_dictionary', 5); + +SELECT 'Get all descendants'; +SELECT dictGetDescendants('hierarchy_flat_dictionary', 0); +SELECT dictGetDescendants('hierarchy_flat_dictionary', 1); +SELECT dictGetDescendants('hierarchy_flat_dictionary', 2); +SELECT dictGetDescendants('hierarchy_flat_dictionary', 3); +SELECT dictGetDescendants('hierarchy_flat_dictionary', 4); +SELECT dictGetDescendants('hierarchy_flat_dictionary', 5); + +SELECT 'Get descendants at first level'; +SELECT dictGetDescendants('hierarchy_flat_dictionary', 0, 1); +SELECT dictGetDescendants('hierarchy_flat_dictionary', 1, 1); +SELECT dictGetDescendants('hierarchy_flat_dictionary', 2, 1); +SELECT dictGetDescendants('hierarchy_flat_dictionary', 3, 1); +SELECT dictGetDescendants('hierarchy_flat_dictionary', 4, 1); +SELECT dictGetDescendants('hierarchy_flat_dictionary', 5, 1); + +DROP DICTIONARY hierarchy_flat_dictionary; +DROP TABLE hierarchy_source_table; From 2aa6685baf9354cc8a71977503a30f8e105b7136 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 3 Mar 2022 18:04:17 +0000 Subject: [PATCH 075/165] Add cases to 02169_map_functions, update error messages in FunctionArrayMapped --- src/Functions/array/FunctionArrayMapped.h | 41 ++++++++++--------- .../0_stateless/02169_map_functions.sql | 21 +++++++++- 2 files changed, 42 insertions(+), 20 deletions(-) diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index df34abccab9..58e6db86f75 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -98,13 +98,16 @@ public: void getLambdaArgumentTypes(DataTypes & arguments) const override { if (arguments.empty()) - throw Exception("Function " + getName() + " needs at least one argument; passed " - + toString(arguments.size()) + ".", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} needs at least one argument, passed {}", getName(), arguments.size()); if (arguments.size() == 1) - throw Exception("Function " + getName() + " needs at least one array argument.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} needs at least one argument with data", getName()); + + if (arguments.size() > 2 && Impl::needOneArray()) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} needs one argument with data", getName()); size_t nested_types_count = std::is_same_v ? (arguments.size() - 1) * 2 : (arguments.size() - 1); DataTypes nested_types(nested_types_count); @@ -127,9 +130,10 @@ public: const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); if (!function_type || function_type->getArgumentTypes().size() != nested_types.size()) - throw Exception("First argument for this overload of " + getName() + " must be a function with " - + toString(nested_types.size()) + " arguments. Found " - + arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for this overload of {} must be a function with {} arguments, found {} instead", + getName(), nested_types.size(), arguments[0]->getName()); arguments[0] = std::make_shared(nested_types); } @@ -138,24 +142,23 @@ public: { size_t min_args = Impl::needExpression() ? 2 : 1; if (arguments.size() < min_args) - throw Exception("Function " + getName() + " needs at least " - + toString(min_args) + " argument; passed " - + toString(arguments.size()) + ".", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} needs at least {} argument, passed {}", + getName(), min_args, arguments.size()); if ((arguments.size() == 1) && std::is_same_v) { - const auto * array_type = checkAndGetDataType(arguments[0].type.get()); + const auto * data_type = checkAndGetDataType(arguments[0].type.get()); - if (!array_type) + if (!data_type) throw Exception("The only argument for function " + getName() + " must be array. Found " - + arguments[0].type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + arguments[0].type->getName() + " instead", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - DataTypePtr nested_type = array_type->getNestedType(); + DataTypePtr nested_type = data_type->getNestedType(); if (Impl::needBoolean() && !WhichDataType(nested_type).isUInt8()) throw Exception("The only argument for function " + getName() + " must be array of UInt8. Found " - + arguments[0].type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + arguments[0].type->getName() + " instead", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if constexpr (std::is_same_v) return Impl::getReturnType(nested_type, nested_type); @@ -165,13 +168,13 @@ public: else { if (arguments.size() > 2 && Impl::needOneArray()) - throw Exception("Function " + getName() + " needs one array argument.", + throw Exception("Function " + getName() + " needs one argument with data", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); if (!data_type_function) - throw Exception("First argument for function " + getName() + " must be a function.", + throw Exception("First argument for function " + getName() + " must be a function", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); /// The types of the remaining arguments are already checked in getLambdaArgumentTypes. diff --git a/tests/queries/0_stateless/02169_map_functions.sql b/tests/queries/0_stateless/02169_map_functions.sql index ee2e70f82cd..4cccaa56722 100644 --- a/tests/queries/0_stateless/02169_map_functions.sql +++ b/tests/queries/0_stateless/02169_map_functions.sql @@ -7,14 +7,33 @@ SELECT mapFilter((k, v) -> k like '%3' and v > 102, col) FROM table_map ORDER BY SELECT col, mapFilter((k, v) -> ((v % 10) > 1), col) FROM table_map ORDER BY id ASC; SELECT mapApply((k, v) -> (k, v + 1), col) FROM table_map ORDER BY id; SELECT mapFilter((k, v) -> 0, col) from table_map; -SELECT mapApply((k, v) -> tuple(v + 9223372036854775806), col) FROM table_map; -- { serverError 42 } +SELECT mapApply((k, v) -> tuple(v + 9223372036854775806), col) FROM table_map; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0)); SELECT mapApply((x, y) -> (x, x + 1), map(1, 0, 2, 0)); SELECT mapApply((x, y) -> (x, x + 1), materialize(map(1, 0, 2, 0))); SELECT mapApply((x, y) -> ('x', 'y'), map(1, 0, 2, 0)); SELECT mapApply((x, y) -> ('x', 'y'), materialize(map(1, 0, 2, 0))); + +SELECT mapApply(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT mapApply((x, y) -> (x), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT mapApply((x, y) -> ('x'), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapApply((x) -> (x, x), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapApply((x, y) -> (x, 1, 2), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT mapApply((x, y) -> (x, x + 1)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT mapApply(map(1, 0, 2, 0), (x, y) -> (x, x + 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapApply((x, y) -> (x, x+1), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT mapFilter(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT mapFilter((x, y) -> (toInt32(x)), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapFilter((x, y) -> ('x'), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapFilter((x) -> (x, x), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapFilter((x, y) -> (x, 1, 2), map(1, 0, 2, 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapFilter((x, y) -> (x, x + 1)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT mapFilter(map(1, 0, 2, 0), (x, y) -> (x > 0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT mapFilter((x, y) -> (x, x + 1), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT mapUpdate(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT mapUpdate(map(1, 3, 3, 2), map(1, 0, 2, 0), map(1, 0, 2, 0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } DROP TABLE table_map; From dda6632932d1afbcfc76cb24c7f170f8723f0213 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Mar 2022 21:28:30 +0100 Subject: [PATCH 076/165] Avoid division by zero in Query Profiler if Linux kernel has a bug --- src/Common/QueryProfiler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 9718d15c072..1f305e30c33 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -48,13 +48,13 @@ namespace if (overrun_count) { /// But pass with some frequency to avoid drop of all traces. - if (write_trace_iteration % (overrun_count + 1) == 0) + if (overrun_count > 0 && write_trace_iteration % (overrun_count + 1) == 0) { ProfileEvents::increment(ProfileEvents::QueryProfilerSignalOverruns, overrun_count); } else { - ProfileEvents::increment(ProfileEvents::QueryProfilerSignalOverruns, overrun_count + 1); + ProfileEvents::increment(ProfileEvents::QueryProfilerSignalOverruns, std::max(0, overrun_count) + 1); return; } } From 5935453bf07f76c091a3bba4e1419e2e1d905b1f Mon Sep 17 00:00:00 2001 From: Anish Bhanwala Date: Fri, 4 Mar 2022 11:54:56 +0530 Subject: [PATCH 077/165] Incorrect wording fixed I think the word "because" was misplaced by mistake. --- docs/en/faq/general/columnar-database.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/faq/general/columnar-database.md b/docs/en/faq/general/columnar-database.md index cbc5f77d0ba..11bbd2e63f6 100644 --- a/docs/en/faq/general/columnar-database.md +++ b/docs/en/faq/general/columnar-database.md @@ -22,4 +22,4 @@ Here is the illustration of the difference between traditional row-oriented syst **Columnar** ![Columnar](https://clickhouse.com/docs/en/images/column-oriented.gif#) -A columnar database is a preferred choice for analytical applications because it allows to have many columns in a table just in case, but do not pay the cost for unused columns on read query execution time. Column-oriented databases are designed for big data processing because and data warehousing, they often natively scale using distributed clusters of low-cost hardware to increase throughput. ClickHouse does it with combination of [distributed](../../engines/table-engines/special/distributed.md) and [replicated](../../engines/table-engines/mergetree-family/replication.md) tables. +A columnar database is a preferred choice for analytical applications because it allows to have many columns in a table just in case, but do not pay the cost for unused columns on read query execution time. Column-oriented databases are designed for big data processing and data warehousing, because they often natively scale using distributed clusters of low-cost hardware to increase throughput. ClickHouse does it with combination of [distributed](../../engines/table-engines/special/distributed.md) and [replicated](../../engines/table-engines/mergetree-family/replication.md) tables. From 69b5fbed64f6246c2ebd75cb3cf8e46289e00b7a Mon Sep 17 00:00:00 2001 From: cnmade Date: Fri, 4 Mar 2022 16:37:18 +0800 Subject: [PATCH 078/165] Translate zh/sql-reference/statements/alter/row-policy: rename old file --- .../statements/alter/{row-policy.md => row-policy.md.bak} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename docs/zh/sql-reference/statements/alter/{row-policy.md => row-policy.md.bak} (100%) diff --git a/docs/zh/sql-reference/statements/alter/row-policy.md b/docs/zh/sql-reference/statements/alter/row-policy.md.bak similarity index 100% rename from docs/zh/sql-reference/statements/alter/row-policy.md rename to docs/zh/sql-reference/statements/alter/row-policy.md.bak From 1d2707e7eae51083124c6c72618bc945b369bd08 Mon Sep 17 00:00:00 2001 From: cnmade Date: Fri, 4 Mar 2022 16:39:12 +0800 Subject: [PATCH 079/165] Translate zh/sql-reference/statements/alter/row-policy: reimport file --- .../statements/alter/row-policy.md | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) create mode 100644 docs/zh/sql-reference/statements/alter/row-policy.md diff --git a/docs/zh/sql-reference/statements/alter/row-policy.md b/docs/zh/sql-reference/statements/alter/row-policy.md new file mode 100644 index 00000000000..bbf9f317737 --- /dev/null +++ b/docs/zh/sql-reference/statements/alter/row-policy.md @@ -0,0 +1,19 @@ +--- +toc_priority: 47 +toc_title: ROW POLICY +--- + +# ALTER ROW POLICY {#alter-row-policy-statement} + +Changes row policy. + +Syntax: + +``` sql +ALTER [ROW] POLICY [IF EXISTS] name1 [ON CLUSTER cluster_name1] ON [database1.]table1 [RENAME TO new_name1] + [, name2 [ON CLUSTER cluster_name2] ON [database2.]table2 [RENAME TO new_name2] ...] + [AS {PERMISSIVE | RESTRICTIVE}] + [FOR SELECT] + [USING {condition | NONE}][,...] + [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] +``` From 44d626af6fcdad817b24881a997eb83efe769b7c Mon Sep 17 00:00:00 2001 From: cnmade Date: Fri, 4 Mar 2022 16:41:07 +0800 Subject: [PATCH 080/165] Translate zh/sql-reference/statements/alter/row-policy: translate to zh --- docs/zh/sql-reference/statements/alter/row-policy.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/zh/sql-reference/statements/alter/row-policy.md b/docs/zh/sql-reference/statements/alter/row-policy.md index bbf9f317737..0cdba239b84 100644 --- a/docs/zh/sql-reference/statements/alter/row-policy.md +++ b/docs/zh/sql-reference/statements/alter/row-policy.md @@ -1,13 +1,13 @@ --- toc_priority: 47 -toc_title: ROW POLICY +toc_title: 行策略 --- -# ALTER ROW POLICY {#alter-row-policy-statement} +# 操作行策略 {#alter-row-policy-statement} -Changes row policy. +修改行策略. -Syntax: +语法: ``` sql ALTER [ROW] POLICY [IF EXISTS] name1 [ON CLUSTER cluster_name1] ON [database1.]table1 [RENAME TO new_name1] From 4167f82c57a06912ace1ff0047e09a919d2aee82 Mon Sep 17 00:00:00 2001 From: cnmade Date: Fri, 4 Mar 2022 16:45:39 +0800 Subject: [PATCH 081/165] Translate zh/sql-reference/statements/alter/row-policy: remove old file --- docs/zh/sql-reference/statements/alter/row-policy.md.bak | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/sql-reference/statements/alter/row-policy.md.bak diff --git a/docs/zh/sql-reference/statements/alter/row-policy.md.bak b/docs/zh/sql-reference/statements/alter/row-policy.md.bak deleted file mode 120000 index 09ad2d301f3..00000000000 --- a/docs/zh/sql-reference/statements/alter/row-policy.md.bak +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/alter/row-policy.md \ No newline at end of file From e9dee63fe2a0a6d533afdfec93d3dfbe4b29b77a Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 4 Mar 2022 10:49:43 +0000 Subject: [PATCH 082/165] style fix --- src/Functions/array/arraySort.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Functions/array/arraySort.cpp b/src/Functions/array/arraySort.cpp index d87b941ab21..5421185211e 100644 --- a/src/Functions/array/arraySort.cpp +++ b/src/Functions/array/arraySort.cpp @@ -3,9 +3,6 @@ #include #include -#include "FunctionArrayMapped.h" - - namespace DB { From c836a57000c4eb36bbdee8dfda6ef22148e91577 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 4 Mar 2022 11:46:15 +0000 Subject: [PATCH 083/165] fix reading from system.asynchronous_inserts table --- .../System/StorageSystemAsynchronousInserts.cpp | 15 +++++++++++++-- .../02015_async_inserts_stress_long.sh | 10 ++++++++++ 2 files changed, 23 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemAsynchronousInserts.cpp b/src/Storages/System/StorageSystemAsynchronousInserts.cpp index a16b76411bb..80fc070c83a 100644 --- a/src/Storages/System/StorageSystemAsynchronousInserts.cpp +++ b/src/Storages/System/StorageSystemAsynchronousInserts.cpp @@ -62,8 +62,19 @@ void StorageSystemAsynchronousInserts::fillData(MutableColumns & res_columns, Co size_t i = 0; res_columns[i++]->insert(queryToString(insert_query)); - res_columns[i++]->insert(insert_query.table_id.getDatabaseName()); - res_columns[i++]->insert(insert_query.table_id.getTableName()); + + /// If query is "INSERT INTO FUNCTION" then table_id is empty. + if (insert_query.table_id) + { + res_columns[i++]->insert(insert_query.table_id.getDatabaseName()); + res_columns[i++]->insert(insert_query.table_id.getTableName()); + } + else + { + res_columns[i++]->insertDefault(); + res_columns[i++]->insertDefault(); + } + res_columns[i++]->insert(insert_query.format); res_columns[i++]->insert(time_in_microseconds(elem->data->first_update)); res_columns[i++]->insert(time_in_microseconds(elem->data->last_update)); diff --git a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh index f9a58818404..31258883194 100755 --- a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh +++ b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh @@ -24,6 +24,14 @@ function insert2() done } +function insert3() +{ + url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" + while true; do + ${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO FUNCTION remote('127.0.0.1', $CLICKHOUSE_DATABASE, async_inserts) VALUES (7, 'g') (8, 'h')" + done +} + function select1() { while true; do @@ -53,6 +61,7 @@ TIMEOUT=10 export -f insert1 export -f insert2 +export -f insert3 export -f select1 export -f select2 export -f truncate1 @@ -60,6 +69,7 @@ export -f truncate1 for _ in {1..5}; do timeout $TIMEOUT bash -c insert1 & timeout $TIMEOUT bash -c insert2 & + timeout $TIMEOUT bash -c insert3 & done timeout $TIMEOUT bash -c select1 & From 2ab920a404203513053ba3f2d90e7ad9b47f3f49 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Mar 2022 13:14:38 +0100 Subject: [PATCH 084/165] Review fixes --- src/Common/isLocalAddress.cpp | 28 +++++--------- src/Common/isLocalAddress.h | 1 - src/Coordination/KeeperStateManager.cpp | 51 +++++++++++++++++++++---- 3 files changed, 53 insertions(+), 27 deletions(-) diff --git a/src/Common/isLocalAddress.cpp b/src/Common/isLocalAddress.cpp index 40907ddbe04..d79e4cebd15 100644 --- a/src/Common/isLocalAddress.cpp +++ b/src/Common/isLocalAddress.cpp @@ -79,8 +79,17 @@ struct NetworkInterfaces } -bool isLoopback(const Poco::Net::IPAddress & address) +bool isLocalAddress(const Poco::Net::IPAddress & address) { + /** 127.0.0.1 is treat as local address unconditionally. + * ::1 is also treat as local address unconditionally. + * + * 127.0.0.{2..255} are not treat as local addresses, because they are used in tests + * to emulate distributed queries across localhost. + * + * But 127.{0,1}.{0,1}.{0,1} are treat as local addresses, + * because they are used in Debian for localhost. + */ if (address.isLoopback()) { if (address.family() == Poco::Net::AddressFamily::IPv4) @@ -102,23 +111,6 @@ bool isLoopback(const Poco::Net::IPAddress & address) } } - return false; -} - -bool isLocalAddress(const Poco::Net::IPAddress & address) -{ - /** 127.0.0.1 is treat as local address unconditionally. - * ::1 is also treat as local address unconditionally. - * - * 127.0.0.{2..255} are not treat as local addresses, because they are used in tests - * to emulate distributed queries across localhost. - * - * But 127.{0,1}.{0,1}.{0,1} are treat as local addresses, - * because they are used in Debian for localhost. - */ - if (isLoopback(address)) - return true; - NetworkInterfaces interfaces; return interfaces.hasAddress(address); } diff --git a/src/Common/isLocalAddress.h b/src/Common/isLocalAddress.h index 9cae0e0460f..42977775b09 100644 --- a/src/Common/isLocalAddress.h +++ b/src/Common/isLocalAddress.h @@ -25,7 +25,6 @@ namespace DB bool isLocalAddress(const Poco::Net::SocketAddress & address, UInt16 clickhouse_port); bool isLocalAddress(const Poco::Net::SocketAddress & address); bool isLocalAddress(const Poco::Net::IPAddress & address); - bool isLoopback(const Poco::Net::IPAddress & address); /// Returns number of different bytes in hostnames, used for load balancing size_t getHostNameDifference(const std::string & local_hostname, const std::string & host); diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index b24fa6cdb5e..c5d719919c8 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -17,16 +17,31 @@ namespace ErrorCodes namespace { -bool isLocalhost(const std::string & hostname) +bool isLoopback(const std::string & hostname) { try { - return isLoopback(DNSResolver::instance().resolveHost(hostname)); + return DNSResolver::instance().resolveHost(hostname).isLoopback(); } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } + + return false; +} + +bool isLocalhost(const std::string & hostname) +{ + try + { + return isLocalAddress(DNSResolver::instance().resolveHost(hostname)); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + return false; } @@ -58,8 +73,9 @@ KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersC std::unordered_map check_duplicated_hostnames; size_t total_servers = 0; - std::string local_hostname; + std::string loopback_hostname; std::string non_local_hostname; + size_t local_address_counter = 0; for (const auto & server_key : keys) { if (!startsWith(server_key, "server")) @@ -79,10 +95,19 @@ KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersC hostname, port); } - if (isLocalhost(hostname)) - local_hostname = hostname; + if (isLoopback(hostname)) + { + loopback_hostname = hostname; + local_address_counter++; + } + else if (isLocalhost(hostname)) + { + local_address_counter++; + } else + { non_local_hostname = hostname; + } if (start_as_follower) result.servers_start_as_followers.insert(new_server_id); @@ -123,12 +148,22 @@ KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersC if (result.servers_start_as_followers.size() == total_servers) throw Exception(ErrorCodes::RAFT_ERROR, "At least one of servers should be able to start as leader (without )"); - if (!local_hostname.empty() && !non_local_hostname.empty()) + if (!loopback_hostname.empty() && !non_local_hostname.empty()) { throw Exception( ErrorCodes::RAFT_ERROR, - "Mixing local and non-local hostnames ('{}' and '{}') in raft_configuration is not allowed. Different hosts can resolve it to themselves so it's not allowed.", - local_hostname, non_local_hostname); + "Mixing loopback and non-local hostnames ('{}' and '{}') in raft_configuration is not allowed. " + "Different hosts can resolve it to themselves so it's not allowed.", + loopback_hostname, non_local_hostname); + } + + if (!non_local_hostname.empty() && local_address_counter > 1) + { + throw Exception( + ErrorCodes::RAFT_ERROR, + "Local address specified more than once ({} times) and non-local hostnames also exists ('{}') in raft_configuration. " + "Such configuration is not allowed because single host can vote multiple times.", + local_address_counter, non_local_hostname); } return result; From 5db9018ae0341c101effbd6ff9e4cde4e357cab7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Mar 2022 13:26:56 +0100 Subject: [PATCH 085/165] Fix unitialized variable --- src/Coordination/KeeperStateMachine.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index a9b5fc85828..35e56ba1e30 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -41,6 +41,9 @@ namespace if (!buffer.eof()) readIntBinary(request_for_session.time, buffer); + else /// backward compatibility + request_for_session.time = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + return request_for_session; } From c426eef07dbd284b1fdf6a3b0a2a34867582f769 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 23 Feb 2022 07:15:56 +0300 Subject: [PATCH 086/165] Fix generating USE_* for system.build_options Signed-off-by: Azat Khuzhin --- src/Formats/CMakeLists.txt | 21 +-------------------- src/Formats/configure_config.cmake | 20 ++++++++++++++++++++ src/Storages/System/CMakeLists.txt | 8 +++++++- 3 files changed, 28 insertions(+), 21 deletions(-) create mode 100644 src/Formats/configure_config.cmake diff --git a/src/Formats/CMakeLists.txt b/src/Formats/CMakeLists.txt index 6e6aa6d4553..44883c271f4 100644 --- a/src/Formats/CMakeLists.txt +++ b/src/Formats/CMakeLists.txt @@ -1,21 +1,2 @@ -if (TARGET ch_contrib::avrocpp) - set(USE_AVRO 1) -endif() -if (TARGET ch_contrib::parquet) - set(USE_PARQUET 1) - set(USE_ARROW 1) - set(USE_ORC 1) -endif() -if (TARGET ch_contrib::snappy) - set(USE_SNAPPY 1) -endif() -if (TARGET ch_contrib::protobuf) - set(USE_PROTOBUF 1) -endif() -if (TARGET ch_contrib::msgpack) - set(USE_MSGPACK 1) -endif() -if (TARGET ch_contrib::capnp) - set(USE_CAPNP 1) -endif() +include(configure_config.cmake) configure_file(config_formats.h.in ${ConfigIncludePath}/config_formats.h) diff --git a/src/Formats/configure_config.cmake b/src/Formats/configure_config.cmake new file mode 100644 index 00000000000..3a11f3c6448 --- /dev/null +++ b/src/Formats/configure_config.cmake @@ -0,0 +1,20 @@ +if (TARGET ch_contrib::avrocpp) + set(USE_AVRO 1) +endif() +if (TARGET ch_contrib::parquet) + set(USE_PARQUET 1) + set(USE_ARROW 1) + set(USE_ORC 1) +endif() +if (TARGET ch_contrib::snappy) + set(USE_SNAPPY 1) +endif() +if (TARGET ch_contrib::protobuf) + set(USE_PROTOBUF 1) +endif() +if (TARGET ch_contrib::msgpack) + set(USE_MSGPACK 1) +endif() +if (TARGET ch_contrib::capnp) + set(USE_CAPNP 1) +endif() diff --git a/src/Storages/System/CMakeLists.txt b/src/Storages/System/CMakeLists.txt index 133761cbe22..efc4c0ed37b 100644 --- a/src/Storages/System/CMakeLists.txt +++ b/src/Storages/System/CMakeLists.txt @@ -39,7 +39,13 @@ if(Git_FOUND) ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) endif() -configure_file (StorageSystemBuildOptions.generated.cpp.in ${CONFIG_BUILD}) +function(generate_system_build_options) + include(${ClickHouse_SOURCE_DIR}/src/configure_config.cmake) + include(${ClickHouse_SOURCE_DIR}/src/Functions/configure_config.cmake) + include(${ClickHouse_SOURCE_DIR}/src/Formats/configure_config.cmake) + configure_file(StorageSystemBuildOptions.generated.cpp.in ${CONFIG_BUILD}) +endfunction() +generate_system_build_options() include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") add_headers_and_sources(storages_system .) From 494fe91f865970328025ceef5b534f945230a0cd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 23 Feb 2022 12:14:38 +0300 Subject: [PATCH 087/165] Fix LINK_FLAGS in system.build_options Fixes: 79f6f5a202ce8c9bf3ed8609293538aecde27fa3 Signed-off-by: Azat Khuzhin --- src/Storages/System/StorageSystemBuildOptions.generated.cpp.in | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 5c25322b4f0..98d992e4247 100644 --- a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -19,7 +19,7 @@ const char * auto_config_build[] "CXX_COMPILER_VERSION", "@CMAKE_CXX_COMPILER_VERSION@", "C_FLAGS", "@FULL_C_FLAGS_NORMALIZED@", "CXX_FLAGS", "@FULL_CXX_FLAGS_NORMALIZED@", - "LINK_FLAGS", "@CMAKE_EXE_LINKER_FLAGS_NORMALIZED@", + "LINK_FLAGS", "@FULL_EXE_LINKER_FLAGS_NORMALIZED@", "BUILD_COMPILE_DEFINITIONS", "@BUILD_COMPILE_DEFINITIONS@", "STATIC", "@USE_STATIC_LIBRARIES@", "SPLIT_BINARY", "@CLICKHOUSE_SPLIT_BINARY@", From b0f964a14fd53941806f00e989855e416b62428a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 23 Feb 2022 12:21:39 +0300 Subject: [PATCH 088/165] Remove LIBRARY_ARCHITECTURE from system.build_options CMAKE_LIBRARY_ARCHITECTURE and it is useless, since it is reported only if the compiler reports subdir arch triplet [1] [1]: https://bugzilla.redhat.com/show_bug.cgi?id=1531678 Signed-off-by: Azat Khuzhin --- src/Storages/System/StorageSystemBuildOptions.generated.cpp.in | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 98d992e4247..d7034cf828b 100644 --- a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -11,7 +11,6 @@ const char * auto_config_build[] "VERSION_DATE", "@VERSION_DATE@", "BUILD_TYPE", "@CMAKE_BUILD_TYPE@", "SYSTEM_PROCESSOR", "@CMAKE_SYSTEM_PROCESSOR@", - "LIBRARY_ARCHITECTURE", "@CMAKE_LIBRARY_ARCHITECTURE@", "CMAKE_VERSION", "@CMAKE_VERSION@", "C_COMPILER", "@CMAKE_C_COMPILER@", "C_COMPILER_VERSION", "@CMAKE_C_COMPILER_VERSION@", From 4a404532fb7763e2dfa95f46b0411bbb3b8453bc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 23 Feb 2022 12:23:31 +0300 Subject: [PATCH 089/165] Remove VERSION_DATE from system.build_options It was set only bu utils/release/release_lib.sh, and seems that this script is not used anymore, at least that part of it. Also note, that GIT_DATE is the same, and it is date time, not only date. Plus VERSION_DATE is not installed for releases anyway. Signed-off-by: Azat Khuzhin --- src/Common/config_version.h.in | 1 - src/Storages/System/StorageSystemBuildOptions.generated.cpp.in | 1 - utils/release/release_lib.sh | 2 -- 3 files changed, 4 deletions(-) diff --git a/src/Common/config_version.h.in b/src/Common/config_version.h.in index 3b0700b8a8a..ea6a95d9946 100644 --- a/src/Common/config_version.h.in +++ b/src/Common/config_version.h.in @@ -17,7 +17,6 @@ #cmakedefine VERSION_DESCRIBE "@VERSION_DESCRIBE@" #cmakedefine VERSION_GITHASH "@VERSION_GITHASH@" #cmakedefine VERSION_INTEGER @VERSION_INTEGER@ -#cmakedefine VERSION_DATE @VERSION_DATE@ #if defined(VERSION_MAJOR) #define DBMS_VERSION_MAJOR VERSION_MAJOR diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index d7034cf828b..7179c44bb59 100644 --- a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -8,7 +8,6 @@ const char * auto_config_build[] "SYSTEM", "@CMAKE_SYSTEM_NAME@", "VERSION_GITHASH", "@VERSION_GITHASH@", "VERSION_REVISION", "@VERSION_REVISION@", - "VERSION_DATE", "@VERSION_DATE@", "BUILD_TYPE", "@CMAKE_BUILD_TYPE@", "SYSTEM_PROCESSOR", "@CMAKE_SYSTEM_PROCESSOR@", "CMAKE_VERSION", "@CMAKE_VERSION@", diff --git a/utils/release/release_lib.sh b/utils/release/release_lib.sh index 538d596d263..da910011b17 100644 --- a/utils/release/release_lib.sh +++ b/utils/release/release_lib.sh @@ -91,12 +91,10 @@ function gen_revision_author { git_describe=`git describe` git_hash=`git rev-parse HEAD` - VERSION_DATE=`git show -s --format=%cs $git_hash` sed -i -e "s/SET(VERSION_REVISION [^) ]*/SET(VERSION_REVISION $VERSION_REVISION/g;" \ -e "s/SET(VERSION_DESCRIBE [^) ]*/SET(VERSION_DESCRIBE $git_describe/g;" \ -e "s/SET(VERSION_GITHASH [^) ]*/SET(VERSION_GITHASH $git_hash/g;" \ - -e "s/SET(VERSION_DATE [^) ]*/SET(VERSION_DATE $VERSION_DATE/g;" \ -e "s/SET(VERSION_MAJOR [^) ]*/SET(VERSION_MAJOR $VERSION_MAJOR/g;" \ -e "s/SET(VERSION_MINOR [^) ]*/SET(VERSION_MINOR $VERSION_MINOR/g;" \ -e "s/SET(VERSION_PATCH [^) ]*/SET(VERSION_PATCH $VERSION_PATCH/g;" \ From 2ef9d324487caa3d8b01b079f9d4fe89f65f0ee5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 4 Mar 2022 15:32:04 +0300 Subject: [PATCH 090/165] Revert "Remove VERSION_DATE from system.build_options" As requested by @kitaisreal This reverts commit 4a404532fb7763e2dfa95f46b0411bbb3b8453bc. Signed-off-by: Azat Khuzhin --- src/Common/config_version.h.in | 1 + src/Storages/System/StorageSystemBuildOptions.generated.cpp.in | 1 + utils/release/release_lib.sh | 2 ++ 3 files changed, 4 insertions(+) diff --git a/src/Common/config_version.h.in b/src/Common/config_version.h.in index ea6a95d9946..3b0700b8a8a 100644 --- a/src/Common/config_version.h.in +++ b/src/Common/config_version.h.in @@ -17,6 +17,7 @@ #cmakedefine VERSION_DESCRIBE "@VERSION_DESCRIBE@" #cmakedefine VERSION_GITHASH "@VERSION_GITHASH@" #cmakedefine VERSION_INTEGER @VERSION_INTEGER@ +#cmakedefine VERSION_DATE @VERSION_DATE@ #if defined(VERSION_MAJOR) #define DBMS_VERSION_MAJOR VERSION_MAJOR diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 7179c44bb59..d7034cf828b 100644 --- a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -8,6 +8,7 @@ const char * auto_config_build[] "SYSTEM", "@CMAKE_SYSTEM_NAME@", "VERSION_GITHASH", "@VERSION_GITHASH@", "VERSION_REVISION", "@VERSION_REVISION@", + "VERSION_DATE", "@VERSION_DATE@", "BUILD_TYPE", "@CMAKE_BUILD_TYPE@", "SYSTEM_PROCESSOR", "@CMAKE_SYSTEM_PROCESSOR@", "CMAKE_VERSION", "@CMAKE_VERSION@", diff --git a/utils/release/release_lib.sh b/utils/release/release_lib.sh index da910011b17..538d596d263 100644 --- a/utils/release/release_lib.sh +++ b/utils/release/release_lib.sh @@ -91,10 +91,12 @@ function gen_revision_author { git_describe=`git describe` git_hash=`git rev-parse HEAD` + VERSION_DATE=`git show -s --format=%cs $git_hash` sed -i -e "s/SET(VERSION_REVISION [^) ]*/SET(VERSION_REVISION $VERSION_REVISION/g;" \ -e "s/SET(VERSION_DESCRIBE [^) ]*/SET(VERSION_DESCRIBE $git_describe/g;" \ -e "s/SET(VERSION_GITHASH [^) ]*/SET(VERSION_GITHASH $git_hash/g;" \ + -e "s/SET(VERSION_DATE [^) ]*/SET(VERSION_DATE $VERSION_DATE/g;" \ -e "s/SET(VERSION_MAJOR [^) ]*/SET(VERSION_MAJOR $VERSION_MAJOR/g;" \ -e "s/SET(VERSION_MINOR [^) ]*/SET(VERSION_MINOR $VERSION_MINOR/g;" \ -e "s/SET(VERSION_PATCH [^) ]*/SET(VERSION_PATCH $VERSION_PATCH/g;" \ From d45b6ae03c8c183d31d31ff5289d34db449833cc Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 4 Mar 2022 21:23:19 +0300 Subject: [PATCH 091/165] Extract schema only once on table creation and add it to metadata (#34684) * Add columns description to metadata in case of schema inference * Make better * Remove unnecessary code * Fix tests * More tests * Add tag no-fasttest * Fix test * Fix test Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- src/Interpreters/InterpreterCreateQuery.cpp | 25 ++++++++++++++ src/Interpreters/InterpreterCreateQuery.h | 4 +++ ...e_table_without_columns_metadata.reference | 3 ++ ...2_create_table_without_columns_metadata.sh | 33 +++++++++++++++++++ 4 files changed, 65 insertions(+) create mode 100644 tests/queries/0_stateless/02222_create_table_without_columns_metadata.reference create mode 100755 tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 3d0bab1bb68..ed996430996 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1190,6 +1190,9 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, properties.columns, properties.constraints, false); + + /// If schema wes inferred while storage creation, add columns description to create query. + addColumnsDescriptionToCreateQueryIfNecessary(query_ptr->as(), res); } if (from_path && !res->storesDataOnDisk()) @@ -1489,4 +1492,26 @@ void InterpreterCreateQuery::extendQueryLogElemImpl(QueryLogElement & elem, cons } } +void InterpreterCreateQuery::addColumnsDescriptionToCreateQueryIfNecessary(ASTCreateQuery & create, const StoragePtr & storage) +{ + if (create.is_dictionary || (create.columns_list && create.columns_list->columns && !create.columns_list->columns->children.empty())) + return; + + auto ast_storage = std::make_shared(); + auto query_from_storage = DB::getCreateQueryFromStorage(storage, ast_storage, false, + getContext()->getSettingsRef().max_parser_depth, true); + auto & create_query_from_storage = query_from_storage->as(); + + if (!create.columns_list) + { + ASTPtr columns_list = std::make_shared(*create_query_from_storage.columns_list); + create.set(create.columns_list, columns_list); + } + else + { + ASTPtr columns = std::make_shared(*create_query_from_storage.columns_list->columns); + create.columns_list->set(create.columns_list->columns, columns); + } +} + } diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 5804d817fe2..b6c8e10668a 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -96,6 +96,10 @@ private: void assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const; + /// Update create query with columns description from storage if query doesn't have it. + /// It's used to prevent automatic schema inference while table creation on each server startup. + void addColumnsDescriptionToCreateQueryIfNecessary(ASTCreateQuery & create, const StoragePtr & storage); + ASTPtr query_ptr; /// Skip safety threshold when loading tables. diff --git a/tests/queries/0_stateless/02222_create_table_without_columns_metadata.reference b/tests/queries/0_stateless/02222_create_table_without_columns_metadata.reference new file mode 100644 index 00000000000..9e9e0082cb3 --- /dev/null +++ b/tests/queries/0_stateless/02222_create_table_without_columns_metadata.reference @@ -0,0 +1,3 @@ +CREATE TABLE default.test\n(\n `y` Nullable(String),\n `x` Nullable(Float64)\n)\nENGINE = File(\'JSONEachRow\', \'data.jsonl\') +OK +OK diff --git a/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh b/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh new file mode 100755 index 00000000000..1ba67fa77ea --- /dev/null +++ b/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh @@ -0,0 +1,33 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel + +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}') + +$CLICKHOUSE_CLIENT -q "insert into table function file(data.jsonl, 'JSONEachRow', 'x UInt32 default 42, y String') select number as x, 'String' as y from numbers(10)" + +$CLICKHOUSE_CLIENT -q "drop table if exists test" +$CLICKHOUSE_CLIENT -q "create table test engine=File(JSONEachRow, 'data.jsonl')" +$CLICKHOUSE_CLIENT -q "show create table test" +$CLICKHOUSE_CLIENT -q "detach table test" + +rm $USER_FILES_PATH/data.jsonl + +$CLICKHOUSE_CLIENT -q "attach table test" +$CLICKHOUSE_CLIENT -q "select * from test" 2>&1 | grep -q "FILE_DOESNT_EXIST" && echo "OK" || echo "FAIL" + + +$CLICKHOUSE_CLIENT -q "drop table test" +$CLICKHOUSE_CLIENT -q "create table test (x UInt64) engine=Memory()" + +$CLICKHOUSE_CLIENT -q "drop table if exists test_dist" +$CLICKHOUSE_CLIENT -q "create table test_dist engine=Distributed('test_shard_localhost', currentDatabase(), 'test')" + +$CLICKHOUSE_CLIENT -q "detach table test_dist" +$CLICKHOUSE_CLIENT -q "drop table test" +$CLICKHOUSE_CLIENT -q "attach table test_dist" +$CLICKHOUSE_CLIENT -q "select * from test_dist" 2>&1 | grep -q "UNKNOWN_TABLE" && echo "OK" || echo "FAIL" + From d312a943b116edd98a86027b11dc970f83298d62 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Fri, 4 Mar 2022 11:46:35 -0800 Subject: [PATCH 092/165] Add redirect for cloud page --- docs/redirects.txt | 1 + docs/tools/redirects.py | 7 ++++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/redirects.txt b/docs/redirects.txt index d0d4d4d6c2c..949b9d48ca8 100644 --- a/docs/redirects.txt +++ b/docs/redirects.txt @@ -6,6 +6,7 @@ changelog/2017.md whats-new/changelog/2017.md changelog/2018.md whats-new/changelog/2018.md changelog/2019.md whats-new/changelog/2019.md changelog/index.md whats-new/changelog/index.md +commercial/cloud.md https://clickhouse.com/cloud/ data_types/array.md sql-reference/data-types/array.md data_types/boolean.md sql-reference/data-types/boolean.md data_types/date.md sql-reference/data-types/date.md diff --git a/docs/tools/redirects.py b/docs/tools/redirects.py index 20e3ec7aa6f..1f0a3bb4b74 100644 --- a/docs/tools/redirects.py +++ b/docs/tools/redirects.py @@ -31,7 +31,12 @@ def build_redirect_html(args, base_prefix, lang, output_dir, from_path, to_path) from_path.replace('/index.md', '/index.html').replace('.md', '/index.html') ) target_path = to_path.replace('/index.md', '/').replace('.md', '/') - to_url = f'/{base_prefix}/{lang}/{target_path}' + + if target_path[0:7] != 'http://' and target_path[0:8] != 'https://': + to_url = f'/{base_prefix}/{lang}/{target_path}' + else: + to_url = target_path + to_url = to_url.strip() write_redirect_html(out_path, to_url) From fc572dcb4148a2c144c9de59107a0522881f37f0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 4 Mar 2022 23:32:38 +0100 Subject: [PATCH 093/165] Remove useless define --- src/Core/ProtocolDefines.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 93f44b02ce3..6ee491f3ab5 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -8,7 +8,6 @@ #define DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME 54372 #define DBMS_MIN_REVISION_WITH_VERSION_PATCH 54401 #define DBMS_MIN_REVISION_WITH_SERVER_LOGS 54406 -#define DBMS_MIN_REVISION_WITH_CLIENT_SUPPORT_EMBEDDED_DATA 54415 /// Minimum revision with exactly the same set of aggregation methods and rules to select them. /// Two-level (bucketed) aggregation is incompatible if servers are inconsistent in these rules /// (keys will be placed in different buckets and result will not be fully aggregated). From 7da2c5f9275efdc07fca880d36e5e9f1676eb3e5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 5 Mar 2022 02:04:49 +0300 Subject: [PATCH 094/165] Update play.html --- programs/server/play.html | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/programs/server/play.html b/programs/server/play.html index f7154643504..8411e0eaadb 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -87,13 +87,6 @@ color: var(--text-color); } - /* Otherwise scrollbar may appear dynamically and it will alter viewport height, - then relative heights of elements will change suddenly, and it will break overall impression. */ - /* html - { - overflow-x: scroll; - }*/ - div { width: 100%; @@ -382,7 +375,7 @@ From 1b9987ceb120666c3bfc8b09c030724c5d199bda Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 5 Mar 2022 14:22:56 +0800 Subject: [PATCH 096/165] fix mistake in check-style --- programs/local/LocalServer.cpp | 3 ++- src/Client/LocalConnection.cpp | 17 +++++++++++------ src/Client/LocalConnection.h | 9 ++++++--- src/Interpreters/ProfileEventsExt.cpp | 16 +++++++--------- src/Server/TCPHandler.cpp | 2 +- utils/check-style/check-style | 2 ++ 6 files changed, 29 insertions(+), 20 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index a2d36954165..eb562dfd9eb 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -411,7 +411,8 @@ void LocalServer::setupUsers() void LocalServer::connect() { connection_parameters = ConnectionParameters(config()); - connection = LocalConnection::createConnection(connection_parameters, global_context, need_render_progress, need_render_profile_events); + connection = LocalConnection::createConnection( + connection_parameters, global_context, need_render_progress, need_render_profile_events, server_display_name); } diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 02b437d7ce6..55505fce987 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -20,11 +20,12 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -LocalConnection::LocalConnection(ContextPtr context_, bool send_progress_, bool send_profile_events_) +LocalConnection::LocalConnection(ContextPtr context_, bool send_progress_, bool send_profile_events_, const String & server_display_name_) : WithContext(context_) , session(getContext(), ClientInfo::Interface::LOCAL) , send_progress(send_progress_) , send_profile_events(send_profile_events_) + , server_display_name(server_display_name_) { /// Authenticate and create a context to execute queries. session.authenticate("default", "", Poco::Net::SocketAddress{}); @@ -63,7 +64,7 @@ void LocalConnection::updateProgress(const Progress & value) void LocalConnection::getProfileEvents(Block & block) { - ProfileEvents::getProfileEvents("local", state->profile_queue, block, last_sent_snapshots); + ProfileEvents::getProfileEvents(server_display_name, state->profile_queue, block, last_sent_snapshots); } void LocalConnection::sendQuery( @@ -85,14 +86,13 @@ void LocalConnection::sendQuery( if (!current_database.empty()) query_context->setCurrentDatabase(current_database); - query_scope_holder.reset(); - query_scope_holder = std::make_unique(query_context); state.reset(); state.emplace(); state->query_id = query_id; state->query = query; + state->query_scope_holder = std::make_unique(query_context); state->stage = QueryProcessingStage::Enum(stage); state->profile_queue = std::make_shared(std::numeric_limits::max()); CurrentThread::attachInternalProfileEventsQueue(state->profile_queue); @@ -483,9 +483,14 @@ void LocalConnection::sendMergeTreeReadTaskResponse(const PartitionReadResponse throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented"); } -ServerConnectionPtr LocalConnection::createConnection(const ConnectionParameters &, ContextPtr current_context, bool send_progress, bool send_profile_events) +ServerConnectionPtr LocalConnection::createConnection( + const ConnectionParameters &, + ContextPtr current_context, + bool send_progress, + bool send_profile_events, + const String & server_display_name) { - return std::make_unique(current_context, send_progress, send_profile_events); + return std::make_unique(current_context, send_progress, send_profile_events, server_display_name); } diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index 7edb791a177..62e95cdfee6 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -54,13 +54,15 @@ struct LocalQueryState Stopwatch after_send_progress; Stopwatch after_send_profile_events; + std::unique_ptr query_scope_holder; }; class LocalConnection : public IServerConnection, WithContext { public: - explicit LocalConnection(ContextPtr context_, bool send_progress_ = false, bool send_profile_events_ = false); + explicit LocalConnection( + ContextPtr context_, bool send_progress_ = false, bool send_profile_events_ = false, const String & server_display_name_ = ""); ~LocalConnection() override; @@ -70,7 +72,8 @@ public: const ConnectionParameters & connection_parameters, ContextPtr current_context, bool send_progress = false, - bool send_profile_events = false); + bool send_profile_events = false, + const String & server_display_name = ""); void setDefaultDatabase(const String & database) override; @@ -146,6 +149,7 @@ private: bool send_progress; bool send_profile_events; + String server_display_name; String description = "clickhouse-local"; std::optional state; @@ -157,6 +161,5 @@ private: String current_database; ProfileEvents::ThreadIdToCountersSnapshot last_sent_snapshots; - std::unique_ptr query_scope_holder; }; } diff --git a/src/Interpreters/ProfileEventsExt.cpp b/src/Interpreters/ProfileEventsExt.cpp index 6961d70529e..ea87d565854 100644 --- a/src/Interpreters/ProfileEventsExt.cpp +++ b/src/Interpreters/ProfileEventsExt.cpp @@ -79,16 +79,14 @@ static void dumpProfileEvents(ProfileEventsSnapshot const & snapshot, DB::Mutabl static void dumpMemoryTracker(ProfileEventsSnapshot const & snapshot, DB::MutableColumns & columns, String const & host_name) { - { - size_t i = 0; - columns[i++]->insertData(host_name.data(), host_name.size()); - columns[i++]->insert(UInt64(snapshot.current_time)); - columns[i++]->insert(UInt64{snapshot.thread_id}); - columns[i++]->insert(Type::GAUGE); + size_t i = 0; + columns[i++]->insertData(host_name.data(), host_name.size()); + columns[i++]->insert(UInt64(snapshot.current_time)); + columns[i++]->insert(UInt64{snapshot.thread_id}); + columns[i++]->insert(Type::GAUGE); - columns[i++]->insertData(MemoryTracker::USAGE_EVENT_NAME, strlen(MemoryTracker::USAGE_EVENT_NAME)); - columns[i++]->insert(snapshot.memory_usage); - } + columns[i++]->insertData(MemoryTracker::USAGE_EVENT_NAME, strlen(MemoryTracker::USAGE_EVENT_NAME)); + columns[i++]->insert(snapshot.memory_usage); } void getProfileEvents( diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 5be340386d8..f4592a8b2c9 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -859,7 +859,7 @@ void TCPHandler::sendProfileEvents() Block block; ProfileEvents::getProfileEvents(server_display_name, state.profile_queue, block, last_sent_snapshots); - if (!!block.rows()) + if (block.rows() != 0) { initProfileEventsBlockOutput(block); diff --git a/utils/check-style/check-style b/utils/check-style/check-style index d178778a410..6ebf53cb932 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -74,6 +74,8 @@ EXTERN_TYPES_EXCLUDES=( ProfileEvents::Type ProfileEvents::TypeEnum ProfileEvents::dumpToMapColumn + ProfileEvents::getProfileEvents + ProfileEvents::ThreadIdToCountersSnapshot ProfileEvents::LOCAL_NAME ProfileEvents::CountersIncrement From 240b5e8199248c700df342179cf61fd4a24843b7 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 5 Mar 2022 13:23:52 +0300 Subject: [PATCH 097/165] Update ym-dict-functions.md --- .../sql-reference/functions/ym-dict-functions.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/functions/ym-dict-functions.md b/docs/en/sql-reference/functions/ym-dict-functions.md index f947c81c7a9..1e6c9cbd0b4 100644 --- a/docs/en/sql-reference/functions/ym-dict-functions.md +++ b/docs/en/sql-reference/functions/ym-dict-functions.md @@ -1,11 +1,11 @@ --- toc_priority: 59 -toc_title: Yandex.Metrica Dictionaries +toc_title: Embedded Dictionaries --- -# Functions for Working with Yandex.Metrica Dictionaries {#functions-for-working-with-yandex-metrica-dictionaries} +# Functions for Working with Embedded Dictionaries -In order for the functions below to work, the server config must specify the paths and addresses for getting all the Yandex.Metrica dictionaries. The dictionaries are loaded at the first call of any of these functions. If the reference lists can’t be loaded, an exception is thrown. +In order for the functions below to work, the server config must specify the paths and addresses for getting all the embedded dictionaries. The dictionaries are loaded at the first call of any of these functions. If the reference lists can’t be loaded, an exception is thrown. For information about creating reference lists, see the section “Dictionaries”. @@ -33,7 +33,7 @@ regionToCountry(RegionID, 'ua') – Uses the dictionary for the 'ua' key: /opt/g ### regionToCity(id\[, geobase\]) {#regiontocityid-geobase} -Accepts a UInt32 number – the region ID from the Yandex geobase. If this region is a city or part of a city, it returns the region ID for the appropriate city. Otherwise, returns 0. +Accepts a UInt32 number – the region ID from the geobase. If this region is a city or part of a city, it returns the region ID for the appropriate city. Otherwise, returns 0. ### regionToArea(id\[, geobase\]) {#regiontoareaid-geobase} @@ -117,7 +117,7 @@ regionToTopContinent(id[, geobase]) **Arguments** -- `id` — Region ID from the Yandex geobase. [UInt32](../../sql-reference/data-types/int-uint.md). +- `id` — Region ID from the geobase. [UInt32](../../sql-reference/data-types/int-uint.md). - `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../sql-reference/data-types/string.md). Optional. **Returned value** @@ -132,7 +132,7 @@ Type: `UInt32`. Gets the population for a region. The population can be recorded in files with the geobase. See the section “External dictionaries”. If the population is not recorded for the region, it returns 0. -In the Yandex geobase, the population might be recorded for child regions, but not for parent regions. +In the geobase, the population might be recorded for child regions, but not for parent regions. ### regionIn(lhs, rhs\[, geobase\]) {#regioninlhs-rhs-geobase} @@ -141,12 +141,12 @@ The relationship is reflexive – any region also belongs to itself. ### regionHierarchy(id\[, geobase\]) {#regionhierarchyid-geobase} -Accepts a UInt32 number – the region ID from the Yandex geobase. Returns an array of region IDs consisting of the passed region and all parents along the chain. +Accepts a UInt32 number – the region ID from the geobase. Returns an array of region IDs consisting of the passed region and all parents along the chain. Example: `regionHierarchy(toUInt32(213)) = [213,1,3,225,10001,10000]`. ### regionToName(id\[, lang\]) {#regiontonameid-lang} -Accepts a UInt32 number – the region ID from the Yandex geobase. A string with the name of the language can be passed as a second argument. Supported languages are: ru, en, ua, uk, by, kz, tr. If the second argument is omitted, the language ‘ru’ is used. If the language is not supported, an exception is thrown. Returns a string – the name of the region in the corresponding language. If the region with the specified ID does not exist, an empty string is returned. +Accepts a UInt32 number – the region ID from the geobase. A string with the name of the language can be passed as a second argument. Supported languages are: ru, en, ua, uk, by, kz, tr. If the second argument is omitted, the language ‘ru’ is used. If the language is not supported, an exception is thrown. Returns a string – the name of the region in the corresponding language. If the region with the specified ID does not exist, an empty string is returned. `ua` and `uk` both mean Ukrainian. From 8a12a4c214238c88f66e6f1651e4ea2d6ec8f4c2 Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 5 Mar 2022 16:17:08 +0000 Subject: [PATCH 098/165] Try to fix failed tests --- .../0_stateless/00135_duplicate_group_by_keys_segfault.sql | 2 ++ tests/queries/0_stateless/01926_order_by_desc_limit.sql | 2 ++ tests/queries/0_stateless/02015_async_inserts_stress_long.sh | 3 +++ 3 files changed, 7 insertions(+) diff --git a/tests/queries/0_stateless/00135_duplicate_group_by_keys_segfault.sql b/tests/queries/0_stateless/00135_duplicate_group_by_keys_segfault.sql index 16356046a36..c54593056cf 100644 --- a/tests/queries/0_stateless/00135_duplicate_group_by_keys_segfault.sql +++ b/tests/queries/0_stateless/00135_duplicate_group_by_keys_segfault.sql @@ -1,3 +1,5 @@ +-- Tags: no-random-settings + SET max_rows_to_read = 1000000; SET read_overflow_mode = 'break'; SELECT concat(toString(number % 256 AS n), '') AS s, n, max(s) FROM system.numbers_mt GROUP BY s, n, n, n, n, n, n, n, n, n ORDER BY s, n; diff --git a/tests/queries/0_stateless/01926_order_by_desc_limit.sql b/tests/queries/0_stateless/01926_order_by_desc_limit.sql index 7ea102e11e9..9ee7f4a6aff 100644 --- a/tests/queries/0_stateless/01926_order_by_desc_limit.sql +++ b/tests/queries/0_stateless/01926_order_by_desc_limit.sql @@ -1,3 +1,5 @@ +-- Tags: no-random-settings + DROP TABLE IF EXISTS order_by_desc; CREATE TABLE order_by_desc (u UInt32, s String) diff --git a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh index f9a58818404..e773024bb12 100755 --- a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh +++ b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh @@ -1,10 +1,13 @@ #!/usr/bin/env bash +# Tags: no-random-settings + set -e CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh + function insert1() { url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" From 722e0ea214ab50d32d5e2c4d586ecbae63fa9ad7 Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 5 Mar 2022 16:46:14 +0000 Subject: [PATCH 099/165] Fix clickhouse-test --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 4c94e4d6493..121a283d0e4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -446,7 +446,7 @@ class TestCase: else: os.environ['CLICKHOUSE_URL_PARAMS'] = self.base_url_params + '&' + '&'.join(self.random_settings) - new_options = "--allow_repeated_settings --" + " --".join(self.random_settings) + new_options = " --allow_repeated_settings --" + " --".join(self.random_settings) os.environ['CLICKHOUSE_CLIENT_OPT'] = self.base_client_options + new_options + ' ' return client_options + new_options From 3ec6cd31280c0c01b3e0bea8cb54a53f21c1ecc3 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 6 Mar 2022 11:03:22 +0100 Subject: [PATCH 100/165] Update StorageFileLog.cpp --- src/Storages/FileLog/StorageFileLog.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index b43f47eb6c5..700b35a5a48 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -108,6 +108,7 @@ void StorageFileLog::loadMetaFiles(bool attach) if (std::filesystem::exists(previous_path) && !metadata_path_exist) { std::filesystem::copy(previous_path, metadata_base_path, std::filesystem::copy_options::recursive); + std::filesystem::remove_all(previous_path); } /// Meta file may lost, log and create directory else if (!metadata_path_exist) From bc224dee3601ef22015b432d976eeea7d149e8dc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 6 Mar 2022 13:39:49 +0300 Subject: [PATCH 101/165] Do not hide exceptions during mutations system.mutations includes only the message, but not stacktrace, and it is not always obvious to understand the culprit w/o stacktrace. Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 8f254b4790e..e3fa07dd0c0 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -94,6 +94,7 @@ bool MutatePlainMergeTreeTask::executeStep() { storage.updateMutationEntriesErrors(future_part, false, getCurrentExceptionMessage(false)); write_part_log(ExecutionStatus::fromCurrentException()); + tryLogCurrentException(__PRETTY_FUNCTION__); return false; } } From 081f9caa040b793c6479bde1477a9380f6ad8455 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 6 Mar 2022 13:23:50 +0300 Subject: [PATCH 102/165] Avoid possible deadlock on server shutdown Here is an example for deadlock during shutting down DDLWorker: Server main thread: 6 ThreadFromGlobalPool::join () at ../src/Common/ThreadPool.h:217 7 DB::DDLWorker::shutdown () at ../src/Interpreters/DDLWorker.cpp:123 8 DB::DDLWorker::~DDLWorker () at ../src/Interpreters/DDLWorker.cpp:131 9 DB::DDLWorker::~DDLWorker () at ../src/Interpreters/DDLWorker.cpp:130 10 std::__1::default_delete::operator() () at ../contrib/libcxx/include/memory:1397 11 std::__1::unique_ptr<>::reset (this=0x7f7521d44fd0, __p=0x0) at ../contrib/libcxx/include/memory:1658 12 DB::ContextSharedPart::shutdown () at ../src/Interpreters/Context.cpp:380 From 9efc8a1d3852e13232e68735ad2fd8dd99e7dd65 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Mon, 7 Mar 2022 11:11:14 +0800 Subject: [PATCH 115/165] Fix min/max stat --- src/Coordination/KeeperConnectionStats.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Coordination/KeeperConnectionStats.cpp b/src/Coordination/KeeperConnectionStats.cpp index 26455de25f5..b4edfe45159 100644 --- a/src/Coordination/KeeperConnectionStats.cpp +++ b/src/Coordination/KeeperConnectionStats.cpp @@ -53,11 +53,11 @@ void KeeperConnectionStats::updateLatency(uint64_t latency_ms) total_latency.fetch_add(latency_ms, std::memory_order_relaxed); count.fetch_add(1, std::memory_order_relaxed); - if (latency_ms < min_latency.load(std::memory_order_relaxed)) - min_latency.store(latency_ms, std::memory_order_relaxed); + uint64_t prev_val = min_latency.load(std::memory_order_relaxed); + while (prev_val > latency_ms && !min_latency.compare_exchange_weak(prev_val, latency_ms, std::memory_order_relaxed)) {} - if (latency_ms > max_latency.load(std::memory_order_relaxed)) - max_latency.store(latency_ms, std::memory_order_relaxed); + prev_val = max_latency.load(std::memory_order_relaxed); + while (prev_val < latency_ms && !max_latency.compare_exchange_weak(prev_val, latency_ms, std::memory_order_relaxed)) {} } void KeeperConnectionStats::reset() From 4507cc58aa111745431639d3ba79676cf0421bc0 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 7 Mar 2022 12:05:07 +0800 Subject: [PATCH 116/165] update codes --- .../Cache/ExternalDataSourceCache.cpp | 2 +- src/Storages/Cache/RemoteCacheController.h | 11 ++++ tests/integration/test_hive_query/test.py | 59 +++++++++++-------- 3 files changed, 47 insertions(+), 25 deletions(-) diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index 2d754b252e9..18607c16ffa 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -28,7 +28,7 @@ namespace ErrorCodes } LocalFileHolder::LocalFileHolder(RemoteFileCacheType::MappedHolderPtr cache_controller) - : file_cache_controller(std::move(cache_controller)), file_buffer(nullptr), original_readbuffer(nullptr), thread_pool(nullptr) + : file_cache_controller(std::move(cache_controller)), original_readbuffer(nullptr), thread_pool(nullptr) { file_buffer = file_cache_controller->value().allocFile(); if (!file_buffer) diff --git a/src/Storages/Cache/RemoteCacheController.h b/src/Storages/Cache/RemoteCacheController.h index 95f46f13bdc..b1e0e37fbba 100644 --- a/src/Storages/Cache/RemoteCacheController.h +++ b/src/Storages/Cache/RemoteCacheController.h @@ -99,6 +99,17 @@ private: IRemoteFileMetadataPtr file_metadata_ptr; std::filesystem::path local_path; + /** + * is_enable = true, only when the remotereadbuffer has been cached at local disk. + * + * The first time to access a remotebuffer which is not cached at local disk, we use the original remotebuffer directly and mark RemoteCacheController::is_enable = false. + * When the first time access is finished, LocalFileHolder will start a background download process by reusing the same remotebuffer object. After the download process + * finish, is_enable is set true. + * + * So when is_enable=false, if there is anther thread trying to access the same remote file, it would fail to use the local file buffer and use the original remotebuffer + * instead. Avoid multi threads trying to save the same file in to disk at the same time. + * + */ bool is_enable = true; bool valid = true; size_t local_cache_bytes_read_before_flush; diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index 0661a8cafc1..5ded8c842cd 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -27,25 +27,36 @@ def started_cluster(): def test_create_parquet_table(started_cluster): logging.info('Start testing creating hive table ...') node = started_cluster.instances['h0_0_0'] - node.query("set input_format_parquet_allow_missing_columns = true") - result = node.query(""" - DROP TABLE IF EXISTS default.demo_parquet; - CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) + test_passed = False + for i in range(10): + node.query("set input_format_parquet_allow_missing_columns = true") + result = node.query(""" +DROP TABLE IF EXISTS default.demo_parquet; +CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) """) - logging.info("create result {}".format(result)) - time.sleep(120) - assert result.strip() == '' + logging.info("create result {}".format(result)) + if result.strip() == '': + test_passed = True + break + time.sleep(60) + assert test_passed def test_create_orc_table(started_cluster): logging.info('Start testing creating hive table ...') node = started_cluster.instances['h0_0_0'] - result = node.query(""" + test_passed = False + for i in range(10): + result = node.query(""" DROP TABLE IF EXISTS default.demo_orc; CREATE TABLE default.demo_orc (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day) """) - logging.info("create result {}".format(result)) + logging.info("create result {}".format(result)) + if result.strip() == '': + test_passed = True + break + time.sleep(60) - assert result.strip() == '' + assert test_passed def test_create_text_table(started_cluster): logging.info('Start testing creating hive table ...') @@ -114,22 +125,22 @@ def test_cache_read_bytes(started_cluster): DROP TABLE IF EXISTS default.demo_parquet; CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) """) - result = node.query(""" - SELECT day, count(*) FROM default.demo_parquet group by day order by day - """) - time.sleep(10) - result = node.query(""" - SELECT day, count(*) FROM default.demo_parquet group by day order by day - """) + test_passed = False expected_result = """2021-11-01 1 2021-11-05 2 2021-11-11 1 2021-11-16 2 """ - time.sleep(120) - assert result == expected_result - result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") - time.sleep(10) - result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") - logging.info("Read bytes from cache:{}".format(result)) - assert result.strip() != '0' + for i in range(10): + result = node.query(""" + SELECT day, count(*) FROM default.demo_parquet group by day order by day + """) + if result != expected_result: + time.sleep(30) + continue + result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") + if result.strip() == '0': + time.sleep(60) + continue + test_passed = True + assert test_passed From a016ce357661440ff3e269c99d95d71620078bb0 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 7 Mar 2022 12:15:20 +0800 Subject: [PATCH 117/165] updat codes --- tests/integration/test_hive_query/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index 5ded8c842cd..4c881b8a1f4 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -143,4 +143,5 @@ def test_cache_read_bytes(started_cluster): time.sleep(60) continue test_passed = True + break assert test_passed From cfeedd2cb53d6e781d60d3673855fd8f7ea54d21 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 7 Mar 2022 12:28:31 +0800 Subject: [PATCH 118/165] fixed code style --- src/Storages/Hive/StorageHive.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index f3381b09958..90f339cb8ec 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -52,7 +52,7 @@ public: SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) override; NamesAndTypesList getVirtuals() const override; - + bool isColumnOriented() const override; protected: From 0c83b96d8c9ebd27cc87b5ac9fac08e9ba652e22 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 7 Mar 2022 13:13:33 +0800 Subject: [PATCH 119/165] fixed code style --- src/Storages/Cache/RemoteCacheController.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/Cache/RemoteCacheController.h b/src/Storages/Cache/RemoteCacheController.h index b1e0e37fbba..9abfa2c2e09 100644 --- a/src/Storages/Cache/RemoteCacheController.h +++ b/src/Storages/Cache/RemoteCacheController.h @@ -105,10 +105,9 @@ private: * The first time to access a remotebuffer which is not cached at local disk, we use the original remotebuffer directly and mark RemoteCacheController::is_enable = false. * When the first time access is finished, LocalFileHolder will start a background download process by reusing the same remotebuffer object. After the download process * finish, is_enable is set true. - * + * * So when is_enable=false, if there is anther thread trying to access the same remote file, it would fail to use the local file buffer and use the original remotebuffer * instead. Avoid multi threads trying to save the same file in to disk at the same time. - * */ bool is_enable = true; bool valid = true; From eab925554db3f3c7f07e2c8b009c4e4de89b1885 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 7 Mar 2022 14:01:03 +0800 Subject: [PATCH 120/165] fixed code styles --- src/Storages/Cache/RemoteCacheController.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Cache/RemoteCacheController.h b/src/Storages/Cache/RemoteCacheController.h index 9abfa2c2e09..5f9d92c1349 100644 --- a/src/Storages/Cache/RemoteCacheController.h +++ b/src/Storages/Cache/RemoteCacheController.h @@ -106,7 +106,7 @@ private: * When the first time access is finished, LocalFileHolder will start a background download process by reusing the same remotebuffer object. After the download process * finish, is_enable is set true. * - * So when is_enable=false, if there is anther thread trying to access the same remote file, it would fail to use the local file buffer and use the original remotebuffer + * So when is_enable=false, if there is anther thread trying to access the same remote file, it would fail to use the local file buffer and use the original remotebuffer * instead. Avoid multi threads trying to save the same file in to disk at the same time. */ bool is_enable = true; From f3226745773efc03bb1b2e732a276228ace96dbd Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 7 Mar 2022 17:22:55 +0800 Subject: [PATCH 121/165] update tests --- tests/integration/test_hive_query/test.py | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index 4c881b8a1f4..ebe584dae75 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -120,26 +120,18 @@ def test_parquet_groupby_with_cache(started_cluster): assert result == expected_result def test_cache_read_bytes(started_cluster): node = started_cluster.instances['h0_0_0'] - node.query("set input_format_parquet_allow_missing_columns = true") result = node.query(""" DROP TABLE IF EXISTS default.demo_parquet; CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) """) test_passed = False - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" for i in range(10): result = node.query(""" - SELECT day, count(*) FROM default.demo_parquet group by day order by day + SELECT day, count(*) FROM default.demo_parquet group by day order by day settings input_format_parquet_allow_missing_columns = true """) - if result != expected_result: - time.sleep(30) - continue result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") if result.strip() == '0': + logging.info("ProfileEvent_ExternalDataSourceLocalCacheReadBytes == 0") time.sleep(60) continue test_passed = True From 8ae5296ee81d17a6a02e91c3acd7892c099c9fca Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 7 Mar 2022 17:26:48 +0800 Subject: [PATCH 122/165] fixed compile errors --- src/Storages/Hive/StorageHive.cpp | 2 +- src/Storages/Hive/StorageHive.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 4db2b8b98d1..540089ecf62 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -548,7 +548,7 @@ bool StorageHive::isColumnOriented() const return format_name == "Parquet" || format_name == "ORC"; } -Block StorageHive::getActualColumnsToRead(Block sample_block, const Block & header_block, const NameSet & partition_columns) +Block StorageHive::getActualColumnsToRead(Block sample_block, const Block & header_block, const NameSet & partition_columns) const { if (!isColumnOriented()) return header_block; diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 90f339cb8ec..eccd04a6759 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -90,7 +90,7 @@ private: HiveFilePtr createHiveFileIfNeeded(const FileInfo & file_info, const FieldVector & fields, SelectQueryInfo & query_info, ContextPtr context_); - Block getActualColumnsToRead(Block sample_block, const Block & header_block, const NameSet & partition_columns); + Block getActualColumnsToRead(Block sample_block, const Block & header_block, const NameSet & partition_columns) const; String hive_metastore_url; From 0f40a5a52d3bf617eda27b1513841ed365703ee0 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 7 Mar 2022 17:31:27 +0800 Subject: [PATCH 123/165] update tests --- tests/integration/test_hive_query/test.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index ebe584dae75..ab130af7476 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -121,13 +121,12 @@ def test_parquet_groupby_with_cache(started_cluster): def test_cache_read_bytes(started_cluster): node = started_cluster.instances['h0_0_0'] result = node.query(""" - DROP TABLE IF EXISTS default.demo_parquet; - CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) + CREATE TABLE IF NOT EXSISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) """) test_passed = False for i in range(10): result = node.query(""" - SELECT day, count(*) FROM default.demo_parquet group by day order by day settings input_format_parquet_allow_missing_columns = true + SELECT day, count(*) FROM default.demo_parquet_1 group by day order by day settings input_format_parquet_allow_missing_columns = true """) result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") if result.strip() == '0': From 5423c5a45cc42ebaddedee38986e1d11695fe4c8 Mon Sep 17 00:00:00 2001 From: 1lann Date: Mon, 7 Mar 2022 18:29:10 +0800 Subject: [PATCH 124/165] Fix typo of update_lag In external dictionary providers, the allowed keys for configuration seemed to have a typo of "update_lag" as "update_tag", preventing the use of "update_lag". This change fixes that. --- src/Dictionaries/ClickHouseDictionarySource.cpp | 2 +- src/Dictionaries/MySQLDictionarySource.cpp | 2 +- src/Dictionaries/PostgreSQLDictionarySource.cpp | 2 +- src/Storages/ExternalDataSourceConfiguration.cpp | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index deecc3c983e..5a18dcffb22 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -30,7 +30,7 @@ namespace ErrorCodes static const std::unordered_set dictionary_allowed_keys = { "host", "port", "user", "password", "db", "database", "table", - "update_field", "update_tag", "invalidate_query", "query", "where", "name", "secure"}; + "update_field", "update_lag", "invalidate_query", "query", "where", "name", "secure"}; namespace { diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 29d70f3a7c4..6578f91aa73 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -34,7 +34,7 @@ static const std::unordered_set dictionary_allowed_keys = { "host", "port", "user", "password", "db", "database", "table", "schema", "update_field", "invalidate_query", "priority", - "update_tag", "dont_check_update_time", + "update_lag", "dont_check_update_time", "query", "where", "name" /* name_collection */, "socket", "share_connection", "fail_on_connection_loss", "close_connection", "ssl_ca", "ssl_cert", "ssl_key", diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 6fdf486fdbf..511d6a7288e 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -30,7 +30,7 @@ static const UInt64 max_block_size = 8192; static const std::unordered_set dictionary_allowed_keys = { "host", "port", "user", "password", "db", "database", "table", "schema", - "update_field", "update_tag", "invalidate_query", "query", "where", "name", "priority"}; + "update_field", "update_lag", "invalidate_query", "query", "where", "name", "priority"}; namespace { diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 2d4b05c51b5..5549a816a06 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -34,7 +34,7 @@ IMPLEMENT_SETTINGS_TRAITS(EmptySettingsTraits, EMPTY_SETTINGS) static const std::unordered_set dictionary_allowed_keys = { "host", "port", "user", "password", "db", "database", "table", "schema", "replica", - "update_field", "update_tag", "invalidate_query", "query", + "update_field", "update_lag", "invalidate_query", "query", "where", "name", "secure", "uri", "collection"}; From 202ac18e764c82497742bf5db1b9bae8801e8dc5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 4 Mar 2022 15:50:21 +0000 Subject: [PATCH 125/165] Skip 01086_odbc_roundtrip for aarch, disable force_tests --- tests/ci/ci_config.py | 2 -- tests/queries/0_stateless/01086_odbc_roundtrip.sh | 3 ++- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 000d3d9a000..b45a4ce90c6 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -231,7 +231,6 @@ CI_CONFIG = { }, "Stateful tests (aarch64, actions)": { "required_build": "package_aarch64", - "force_tests": True, }, "Stateful tests (release, DatabaseOrdinary, actions)": { "required_build": "package_release", @@ -259,7 +258,6 @@ CI_CONFIG = { }, "Stateless tests (aarch64, actions)": { "required_build": "package_aarch64", - "force_tests": True, }, "Stateless tests (release, wide parts enabled, actions)": { "required_build": "package_release", diff --git a/tests/queries/0_stateless/01086_odbc_roundtrip.sh b/tests/queries/0_stateless/01086_odbc_roundtrip.sh index 705746032f8..20066c6b34c 100755 --- a/tests/queries/0_stateless/01086_odbc_roundtrip.sh +++ b/tests/queries/0_stateless/01086_odbc_roundtrip.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: no-asan, no-msan, no-fasttest +# Tags: no-asan, no-msan, no-fasttest, no-cpu-aarch64 # Tag no-msan: can't pass because odbc libraries are not instrumented +# Tag no-cpu-aarch64: clickhouse-odbc is not setup for arm CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 688493373f0cb66bc3ebfd3ecdd33c8e8ea501b4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 4 Mar 2022 10:36:37 +0000 Subject: [PATCH 126/165] Try to fix trim function --- src/Parsers/ExpressionElementParsers.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index e00e0aba7b3..c51201750c5 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -442,9 +442,9 @@ namespace pattern_list_args->children = { std::make_shared("^["), to_remove, - std::make_shared("]*|["), + std::make_shared("]+|["), to_remove, - std::make_shared("]*$") + std::make_shared("]+$") }; func_name = "replaceRegexpAll"; } @@ -455,7 +455,7 @@ namespace pattern_list_args->children = { std::make_shared("^["), to_remove, - std::make_shared("]*") + std::make_shared("]+") }; } else @@ -464,7 +464,7 @@ namespace pattern_list_args->children = { std::make_shared("["), to_remove, - std::make_shared("]*$") + std::make_shared("]+$") }; } func_name = "replaceRegexpOne"; From e81bbfb5b538de09e66afb769f6d5a1238e474ef Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 7 Mar 2022 12:13:37 +0100 Subject: [PATCH 127/165] Review fixes --- src/Coordination/KeeperStateManager.cpp | 40 ++++++++++++++++++------- 1 file changed, 30 insertions(+), 10 deletions(-) diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index ca50d14bca6..f9bfea5e69a 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -45,12 +45,37 @@ bool isLocalhost(const std::string & hostname) return false; } +std::unordered_map getClientPorts(const Poco::Util::AbstractConfiguration & config) +{ + static const char * config_port_names[] = { + "keeper_server.tcp_port", + "keeper_server.tcp_port_secure", + "interserver_http_port", + "interserver_https_port", + "tcp_port", + "tcp_with_proxy_port", + "tcp_port_secure", + "mysql_port", + "postgresql_port", + "grpc_port", + "prometheus.port", + }; + + std::unordered_map ports; + for (const auto & config_port_name : config_port_names) + { + if (config.has(config_port_name)) + ports[config.getUInt64(config_port_name)] = config_port_name; + } + return ports; +} + } /// this function quite long because contains a lot of sanity checks in config: /// 1. No duplicate endpoints /// 2. No "localhost" or "127.0.0.1" or another local addresses mixed with normal addresses -/// 3. Raft internal port is equal to client port +/// 3. Raft internal port is not equal to any other port for client /// 4. No duplicate IDs /// 5. Our ID present in hostnames list KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const @@ -60,12 +85,7 @@ KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersC Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix + ".raft_configuration", keys); - std::unordered_set client_ports; - if (config.has(config_prefix + ".tcp_port")) - client_ports.insert(config.getUInt64(config_prefix + ".tcp_port")); - - if (config.has(config_prefix + ".tcp_port_secure")) - client_ports.insert(config.getUInt64(config_prefix + ".tcp_port_secure")); + auto client_ports = getClientPorts(config); /// Sometimes (especially in cloud envs) users can provide incorrect /// configuration with duplicated raft ids or endpoints. We check them @@ -89,10 +109,10 @@ KeeperStateManager::KeeperConfigurationWrapper KeeperStateManager::parseServersC int32_t priority = config.getInt(full_prefix + ".priority", 1); bool start_as_follower = config.getBool(full_prefix + ".start_as_follower", false); - if (client_ports.contains(port)) + if (client_ports.count(port) != 0) { - throw Exception(ErrorCodes::RAFT_ERROR, "Raft config contains hostname '{}' with port '{}' which is equal to client port on current machine", - hostname, port); + throw Exception(ErrorCodes::RAFT_ERROR, "Raft configuration contains hostname '{}' with port '{}' which is equal to '{}' in server configuration", + hostname, port, client_ports[port]); } if (isLoopback(hostname)) From 20478e9088779799b522220d8e26c53c35d57cf3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 4 Mar 2022 10:42:31 +0000 Subject: [PATCH 128/165] add testcase to 02100_replaceRegexpAll_bug --- tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference | 1 + tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql | 2 ++ 2 files changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference index 993dd9b1cde..4dff9ef38ef 100644 --- a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference +++ b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference @@ -9,3 +9,4 @@ 1 1 1 +1 diff --git a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql index 32f7f63f6d0..66ccb044549 100644 --- a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql +++ b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql @@ -12,3 +12,5 @@ SELECT '1,,' == replaceRegexpOne('1,,', '^[,]*|[,]*$', '') x; SELECT '5935,5998,6014' == trim(BOTH ', ' FROM '5935,5998,6014, ') x; SELECT '5935,5998,6014' == replaceRegexpAll('5935,5998,6014, ', concat('^[', regexpQuoteMeta(', '), ']*|[', regexpQuoteMeta(', '), ']*$'), '') AS x; + +SELECT trim(BOTH '"' FROM '2') == '2' From b0f9cc4838e1e0af2eb779f80c6693aeea369ab3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 7 Mar 2022 12:20:55 +0100 Subject: [PATCH 129/165] Keep deprecated installation methods --- docs/_includes/install/deb_repo.sh | 11 +++++++++++ docs/_includes/install/rpm_repo.sh | 7 +++++++ docs/_includes/install/tgz_repo.sh | 19 +++++++++++++++++++ docs/en/getting-started/install.md | 24 ++++++++++++++++++++++++ docs/ja/getting-started/install.md | 24 ++++++++++++++++++++++++ docs/ru/getting-started/install.md | 24 ++++++++++++++++++++++++ docs/zh/getting-started/install.md | 24 ++++++++++++++++++++++++ 7 files changed, 133 insertions(+) create mode 100644 docs/_includes/install/deb_repo.sh create mode 100644 docs/_includes/install/rpm_repo.sh create mode 100644 docs/_includes/install/tgz_repo.sh diff --git a/docs/_includes/install/deb_repo.sh b/docs/_includes/install/deb_repo.sh new file mode 100644 index 00000000000..21106e9fc47 --- /dev/null +++ b/docs/_includes/install/deb_repo.sh @@ -0,0 +1,11 @@ +sudo apt-get install apt-transport-https ca-certificates dirmngr +sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv E0C56BD4 + +echo "deb https://repo.clickhouse.com/deb/stable/ main/" | sudo tee \ + /etc/apt/sources.list.d/clickhouse.list +sudo apt-get update + +sudo apt-get install -y clickhouse-server clickhouse-client + +sudo service clickhouse-server start +clickhouse-client # or "clickhouse-client --password" if you set up a password. diff --git a/docs/_includes/install/rpm_repo.sh b/docs/_includes/install/rpm_repo.sh new file mode 100644 index 00000000000..e3fd1232047 --- /dev/null +++ b/docs/_includes/install/rpm_repo.sh @@ -0,0 +1,7 @@ +sudo yum install yum-utils +sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG +sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/clickhouse.repo +sudo yum install clickhouse-server clickhouse-client + +sudo /etc/init.d/clickhouse-server start +clickhouse-client # or "clickhouse-client --password" if you set up a password. diff --git a/docs/_includes/install/tgz_repo.sh b/docs/_includes/install/tgz_repo.sh new file mode 100644 index 00000000000..0994510755b --- /dev/null +++ b/docs/_includes/install/tgz_repo.sh @@ -0,0 +1,19 @@ +export LATEST_VERSION=$(curl -s https://repo.clickhouse.com/tgz/stable/ | \ + grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | sort -V -r | head -n 1) +curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-dbg-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-server-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-client-$LATEST_VERSION.tgz + +tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz +sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh + +tar -xzvf clickhouse-common-static-dbg-$LATEST_VERSION.tgz +sudo clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh + +tar -xzvf clickhouse-server-$LATEST_VERSION.tgz +sudo clickhouse-server-$LATEST_VERSION/install/doinst.sh +sudo /etc/init.d/clickhouse-server start + +tar -xzvf clickhouse-client-$LATEST_VERSION.tgz +sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index da6e225f745..da75a991b0b 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -27,6 +27,14 @@ It is recommended to use official pre-compiled `deb` packages for Debian or Ubun {% include 'install/deb.sh' %} ``` +
+ +Deprecated Method for installing deb-packages +``` bash +{% include 'install/deb_repo.sh' %} +``` +
+ You can replace `stable` with `lts` or `testing` to use different [release trains](../faq/operations/production.md) based on your needs. You can also download and install packages manually from [here](https://packages.clickhouse.com/deb/pool/stable). @@ -52,6 +60,14 @@ First, you need to add the official repository: {% include 'install/rpm.sh' %} ``` +
+ +Deprecated Method for installing rpm-packages +``` bash +{% include 'install/rpm_repo.sh' %} +``` +
+ If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). `prestable` is sometimes also available. Then run these commands to install packages: @@ -73,6 +89,14 @@ After that downloaded archives should be unpacked and installed with installatio {% include 'install/tgz.sh' %} ``` +
+ +Deprecated Method for installing tgz archives +``` bash +{% include 'install/tgz_repo.sh' %} +``` +
+ For production environments, it’s recommended to use the latest `stable`-version. You can find its number on GitHub page https://github.com/ClickHouse/ClickHouse/tags with postfix `-stable`. ### From Docker Image {#from-docker-image} diff --git a/docs/ja/getting-started/install.md b/docs/ja/getting-started/install.md index 575506c3c4b..10dd2d74f5d 100644 --- a/docs/ja/getting-started/install.md +++ b/docs/ja/getting-started/install.md @@ -28,6 +28,14 @@ Debian や Ubuntu 用にコンパイル済みの公式パッケージ `deb` を {% include 'install/deb.sh' %} ``` +
+ +Deprecated Method for installing deb-packages +``` bash +{% include 'install/deb_repo.sh' %} +``` +
+ 最新版を使いたい場合は、`stable`を`testing`に置き換えてください。(テスト環境ではこれを推奨します) 同様に、[こちら](https://packages.clickhouse.com/deb/pool/stable)からパッケージをダウンロードして、手動でインストールすることもできます。 @@ -49,6 +57,14 @@ CentOS、RedHat、その他すべてのrpmベースのLinuxディストリビュ {% include 'install/rpm.sh' %} ``` +
+ +Deprecated Method for installing rpm-packages +``` bash +{% include 'install/rpm_repo.sh' %} +``` +
+ 最新版を使いたい場合は `stable` を `testing` に置き換えてください。(テスト環境ではこれが推奨されています)。`prestable` もしばしば同様に利用できます。 そして、以下のコマンドを実行してパッケージをインストールします: @@ -69,6 +85,14 @@ sudo yum install clickhouse-server clickhouse-client {% include 'install/tgz.sh' %} ``` +
+ +Deprecated Method for installing tgz archives +``` bash +{% include 'install/tgz_repo.sh' %} +``` +
+ 本番環境では、最新の `stable` バージョンを使うことをお勧めします。GitHub のページ https://github.com/ClickHouse/ClickHouse/tags で 接尾辞 `-stable` となっているバージョン番号として確認できます。 ### Dockerイメージから {#from-docker-image} diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index 84f9bc576e9..64c5cac26df 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -27,6 +27,14 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su {% include 'install/deb.sh' %} ``` +
+ +Устаревший способ установки deb-пакетов +``` bash +{% include 'install/deb_repo.sh' %} +``` +
+ Чтобы использовать различные [версии ClickHouse](../faq/operations/production.md) в зависимости от ваших потребностей, вы можете заменить `stable` на `lts` или `testing`. Также вы можете вручную скачать и установить пакеты из [репозитория](https://packages.clickhouse.com/deb/pool/stable). @@ -52,6 +60,14 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su {% include 'install/rpm.sh' %} ``` +
+ +Устаревший способ установки rpm-пакетов +``` bash +{% include 'install/rpm_repo.sh' %} +``` +
+ Для использования наиболее свежих версий нужно заменить `stable` на `testing` (рекомендуется для тестовых окружений). Также иногда доступен `prestable`. Для, собственно, установки пакетов необходимо выполнить следующие команды: @@ -73,6 +89,14 @@ sudo yum install clickhouse-server clickhouse-client {% include 'install/tgz.sh' %} ``` +
+ +Устаревший способ установки из архивов tgz +``` bash +{% include 'install/tgz_repo.sh' %} +``` +
+ Для production окружений рекомендуется использовать последнюю `stable`-версию. Её номер также можно найти на github с на вкладке https://github.com/ClickHouse/ClickHouse/tags c постфиксом `-stable`. ### Из Docker образа {#from-docker-image} diff --git a/docs/zh/getting-started/install.md b/docs/zh/getting-started/install.md index 6a966355fae..a6a8b05483a 100644 --- a/docs/zh/getting-started/install.md +++ b/docs/zh/getting-started/install.md @@ -27,6 +27,14 @@ $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not {% include 'install/deb.sh' %} ``` +
+ +Deprecated Method for installing deb-packages +``` bash +{% include 'install/deb_repo.sh' %} +``` +
+ 如果您想使用最新的版本,请用`testing`替代`stable`(我们只推荐您用于测试环境)。 你也可以从这里手动下载安装包:[下载](https://packages.clickhouse.com/deb/pool/stable)。 @@ -48,6 +56,14 @@ $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not {% include 'install/rpm.sh' %} ``` +
+ +Deprecated Method for installing rpm-packages +``` bash +{% include 'install/rpm_repo.sh' %} +``` +
+ 如果您想使用最新的版本,请用`testing`替代`stable`(我们只推荐您用于测试环境)。`prestable`有时也可用。 然后运行命令安装: @@ -70,6 +86,14 @@ sudo yum install clickhouse-server clickhouse-client {% include 'install/tgz.sh' %} ``` +
+ +Deprecated Method for installing tgz archives +``` bash +{% include 'install/tgz_repo.sh' %} +``` +
+ 对于生产环境,建议使用最新的`stable`版本。你可以在GitHub页面https://github.com/ClickHouse/ClickHouse/tags找到它,它以后缀`-stable`标志。 ### `Docker`安装包 {#from-docker-image} From aae13ed9123486ce574e634af08d32111540a9e6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 7 Mar 2022 15:18:57 +0100 Subject: [PATCH 130/165] Supress move partition long for storage S3 --- tests/queries/0_stateless/01154_move_partition_long.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01154_move_partition_long.sh b/tests/queries/0_stateless/01154_move_partition_long.sh index 6b0b0773cb6..7cefac28e22 100755 --- a/tests/queries/0_stateless/01154_move_partition_long.sh +++ b/tests/queries/0_stateless/01154_move_partition_long.sh @@ -1,5 +1,7 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: long, no-parallel, no-s3-storage +# FIXME: s3 storage should work OK, it +# reproduces bug which exists not only in S3 version. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 84e22fb32bc6e4f54444d4356413865276712d57 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 7 Mar 2022 18:59:00 +0300 Subject: [PATCH 131/165] Update DiskLocal.cpp --- src/Disks/DiskLocal.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 57bfaf405e0..e49e9cf6726 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -325,7 +326,7 @@ DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) void DiskLocal::moveFile(const String & from_path, const String & to_path) { - fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path); + renameNoReplace(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path); } void DiskLocal::replaceFile(const String & from_path, const String & to_path) From fe4534d4646e51e40ac04fec83e4a321c4ce19ea Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 7 Mar 2022 21:51:34 +0800 Subject: [PATCH 132/165] Get rid of duplicate query planing. --- src/Interpreters/InterpreterSelectQuery.cpp | 20 ++++++++----------- .../InterpreterSelectWithUnionQuery.cpp | 4 +++- src/Interpreters/JoinedTables.cpp | 4 +++- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 97a06842d97..f2fc17fbf9a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1902,20 +1902,16 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc else if (interpreter_subquery) { /// Subquery. - /// If we need less number of columns that subquery have - update the interpreter. - if (required_columns.size() < source_header.columns()) - { - ASTPtr subquery = extractTableExpression(query, 0); - if (!subquery) - throw Exception("Subquery expected", ErrorCodes::LOGICAL_ERROR); + ASTPtr subquery = extractTableExpression(query, 0); + if (!subquery) + throw Exception("Subquery expected", ErrorCodes::LOGICAL_ERROR); - interpreter_subquery = std::make_unique( - subquery, getSubqueryContext(context), - options.copy().subquery().noModify(), required_columns); + interpreter_subquery = std::make_unique( + subquery, getSubqueryContext(context), + options.copy().subquery().noModify(), required_columns); - if (query_analyzer->hasAggregation()) - interpreter_subquery->ignoreWithTotals(); - } + if (query_analyzer->hasAggregation()) + interpreter_subquery->ignoreWithTotals(); interpreter_subquery->buildQueryPlan(query_plan); query_plan.addInterpreterContext(context); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 723db59f04b..130b3aae58d 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -208,8 +208,10 @@ Block InterpreterSelectWithUnionQuery::getCurrentChildResultHeader(const ASTPtr if (ast_ptr_->as()) return InterpreterSelectWithUnionQuery(ast_ptr_, context, options.copy().analyze().noModify(), required_result_column_names) .getSampleBlock(); - else + else if (ast_ptr_->as()) return InterpreterSelectQuery(ast_ptr_, context, options.copy().analyze().noModify()).getSampleBlock(); + else + return InterpreterSelectIntersectExceptQuery(ast_ptr_, context, options.copy().analyze().noModify()).getSampleBlock(); } std::unique_ptr diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index 3aae3982758..482a813bfef 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -183,7 +183,9 @@ std::unique_ptr JoinedTables::makeLeftTableSubq { if (!isLeftTableSubquery()) return {}; - return std::make_unique(left_table_expression, context, select_options); + + /// Only build dry_run interpreter during analysis. We will reconstruct the subquery interpreter during plan building. + return std::make_unique(left_table_expression, context, select_options.copy().analyze()); } StoragePtr JoinedTables::getLeftTableStorage() From 148109e92949741570c29b98fbfa021c739a4a79 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 8 Mar 2022 09:36:02 +0800 Subject: [PATCH 133/165] update tests --- tests/integration/test_hive_query/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index ab130af7476..8039a42bedc 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -121,7 +121,7 @@ def test_parquet_groupby_with_cache(started_cluster): def test_cache_read_bytes(started_cluster): node = started_cluster.instances['h0_0_0'] result = node.query(""" - CREATE TABLE IF NOT EXSISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) + CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) """) test_passed = False for i in range(10): @@ -131,7 +131,7 @@ def test_cache_read_bytes(started_cluster): result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") if result.strip() == '0': logging.info("ProfileEvent_ExternalDataSourceLocalCacheReadBytes == 0") - time.sleep(60) + time.sleep(10) continue test_passed = True break From a8cfc2458a8a5db2a946096fad0bc2299cf5eb10 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 8 Mar 2022 11:55:15 +0800 Subject: [PATCH 134/165] update codes --- src/Storages/Hive/StorageHive.cpp | 30 +++++++++++++++++------------- src/Storages/Hive/StorageHive.h | 2 +- 2 files changed, 18 insertions(+), 14 deletions(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 540089ecf62..a11488cf3cf 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -115,12 +115,12 @@ public: , format(std::move(format_)) , compression_method(compression_method_) , max_block_size(max_block_size_) - , sample_block(sample_block_) - , to_read_block(sample_block) + , sample_block(std::move(sample_block_)) , columns_description(getColumnsDescription(sample_block, source_info)) , text_input_field_names(text_input_field_names_) , format_settings(getFormatSettings(getContext())) { + to_read_block = sample_block; /// Initialize to_read_block, which is used to read data from HDFS. for (const auto & name_type : source_info->partition_name_types) { @@ -206,12 +206,16 @@ public: /// Enrich with partition columns. auto types = source_info->partition_name_types.getTypes(); + auto names = source_info->partition_name_types.getNames(); + auto fields = source_info->hive_files[current_idx]->getPartitionValues(); for (size_t i = 0; i < types.size(); ++i) { - if (!sample_block.has(source_info->partition_name_types.getNames()[i])) + // Only add the required partition columns. partition columns are not read from readbuffer + // the column must be in sample_block, otherwise sample_block.getPositionByName(names[i]) will throw an exception + if (!sample_block.has(names[i])) continue; - auto column = types[i]->createColumnConst(num_rows, source_info->hive_files[current_idx]->getPartitionValues()[i]); - auto previous_idx = sample_block.getPositionByName(source_info->partition_name_types.getNames()[i]); + auto column = types[i]->createColumnConst(num_rows, fields[i]); + auto previous_idx = sample_block.getPositionByName(names[i]); columns.insert(columns.begin() + previous_idx, column); } @@ -548,28 +552,28 @@ bool StorageHive::isColumnOriented() const return format_name == "Parquet" || format_name == "ORC"; } -Block StorageHive::getActualColumnsToRead(Block sample_block, const Block & header_block, const NameSet & partition_columns) const +void StorageHive::getActualColumnsToRead(Block & sample_block, const Block & header_block, const NameSet & partition_columns) const { if (!isColumnOriented()) - return header_block; - Block result_block = sample_block; + sample_block = header_block; + UInt32 erased_columns = 0; for (const auto & column : partition_columns) { - sample_block.erase(column); + if (sample_block.has(column)) + erased_columns++; } - if (!sample_block.columns()) + if (erased_columns == sample_block.columns()) { for (size_t i = 0; i < header_block.columns(); ++i) { const auto & col = header_block.getByPosition(i); if (!partition_columns.count(col.name)) { - result_block.insert(col); + sample_block.insert(col); break; } } } - return result_block; } Pipe StorageHive::read( const Names & column_names, @@ -646,7 +650,7 @@ Pipe StorageHive::read( sources_info->need_file_column = true; } - sample_block = getActualColumnsToRead(sample_block, header_block, NameSet{partition_names.begin(), partition_names.end()}); + getActualColumnsToRead(sample_block, header_block, NameSet{partition_names.begin(), partition_names.end()}); if (num_streams > sources_info->hive_files.size()) num_streams = sources_info->hive_files.size(); diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index eccd04a6759..71d17750190 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -90,7 +90,7 @@ private: HiveFilePtr createHiveFileIfNeeded(const FileInfo & file_info, const FieldVector & fields, SelectQueryInfo & query_info, ContextPtr context_); - Block getActualColumnsToRead(Block sample_block, const Block & header_block, const NameSet & partition_columns) const; + void getActualColumnsToRead(Block & sample_block, const Block & header_block, const NameSet & partition_columns) const; String hive_metastore_url; From caffc144b5a1c31e93c3db7cae0eb22494217b45 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 24 Feb 2022 15:23:26 +0300 Subject: [PATCH 135/165] Fix possible "Part directory doesn't exist" during INSERT In #33291 final part commit had been defered, and now it can take significantly more time, that may lead to "Part directory doesn't exist" error during INSERT: 2022.02.21 18:18:06.979881 [ 11329 ] {insert} executeQuery: (from 127.1:24572, user: default) INSERT INTO db.table (...) VALUES 2022.02.21 20:58:03.933593 [ 11329 ] {insert} db.table: Renaming temporary part tmp_insert_20220214_18044_18044_0 to 20220214_270654_270654_0. 2022.02.21 21:16:50.961917 [ 11329 ] {insert} db.table: Renaming temporary part tmp_insert_20220214_18197_18197_0 to 20220214_270689_270689_0. ... 2022.02.22 21:16:57.632221 [ 64878 ] {} db.table: Removing temporary directory /clickhouse/data/db/table/tmp_insert_20220214_18232_18232_0/ ... 2022.02.23 12:23:56.277480 [ 11329 ] {insert} db.table: Renaming temporary part tmp_insert_20220214_18232_18232_0 to 20220214_273459_273459_0. 2022.02.23 12:23:56.299218 [ 11329 ] {insert} executeQuery: Code: 107. DB::Exception: Part directory /clickhouse/data/db/table/tmp_insert_20220214_18232_18232_0/ doesn't exist. Most likely it is a logical error. (FILE_DOESNT_EXIST) (version 22.2.1.1) (from 127.1:24572) (in query: INSERT INTO db.table (...) VALUES), Stack trace (when copying this message, always include the lines below): Follow-up for: #28760 Refs: #33291 Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTask.cpp | 8 ++--- src/Storages/MergeTree/MergeTreeData.cpp | 6 ++-- src/Storages/MergeTree/MergeTreeData.h | 28 ++++++++-------- .../MergeTree/MergeTreeDataMergerMutator.cpp | 6 ---- .../MergeTree/MergeTreeDataMergerMutator.h | 20 ----------- .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- src/Storages/MergeTree/TemporaryParts.cpp | 24 ++++++++++++++ src/Storages/MergeTree/TemporaryParts.h | 33 +++++++++++++++++++ src/Storages/StorageMergeTree.cpp | 4 +-- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 10 files changed, 81 insertions(+), 52 deletions(-) create mode 100644 src/Storages/MergeTree/TemporaryParts.cpp create mode 100644 src/Storages/MergeTree/TemporaryParts.h diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 89fb27cc89c..8b5c2e0dc6e 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -126,13 +126,9 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() if (ctx->disk->exists(local_new_part_tmp_path)) throw Exception("Directory " + fullPath(ctx->disk, local_new_part_tmp_path) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS); - { - std::lock_guard lock(global_ctx->mutator->tmp_parts_lock); - global_ctx->mutator->tmp_parts.emplace(local_tmp_part_basename); - } + global_ctx->data->temporary_parts.add(local_tmp_part_basename); SCOPE_EXIT( - std::lock_guard lock(global_ctx->mutator->tmp_parts_lock); - global_ctx->mutator->tmp_parts.erase(local_tmp_part_basename); + global_ctx->data->temporary_parts.remove(local_tmp_part_basename); ); global_ctx->all_column_names = global_ctx->metadata_snapshot->getColumns().getNamesOfPhysical(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8ea9d0a31d0..a15da7578e8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1386,7 +1386,7 @@ static bool isOldPartDirectory(const DiskPtr & disk, const String & directory_pa } -size_t MergeTreeData::clearOldTemporaryDirectories(const MergeTreeDataMergerMutator & merger_mutator, size_t custom_directories_lifetime_seconds) +size_t MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lifetime_seconds) { /// If the method is already called from another thread, then we don't need to do anything. std::unique_lock lock(clear_old_temporary_directories_mutex, std::defer_lock); @@ -1418,9 +1418,9 @@ size_t MergeTreeData::clearOldTemporaryDirectories(const MergeTreeDataMergerMuta { if (disk->isDirectory(it->path()) && isOldPartDirectory(disk, it->path(), deadline)) { - if (merger_mutator.hasTemporaryPart(basename)) + if (temporary_parts.contains(basename)) { - LOG_WARNING(log, "{} is an active destination for one of merge/mutation (consider increasing temporary_directories_lifetime setting)", full_path); + LOG_WARNING(log, "{} is in use (by merge/mutation/INSERT) (consider increasing temporary_directories_lifetime setting)", full_path); continue; } else diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 1a04b2a389b..1e7f127c85b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -3,30 +3,31 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include #include #include #include #include #include #include +#include +#include +#include +#include +#include +#include #include #include -#include #include #include #include -#include -#include +#include +#include +#include +#include #include #include -#include +#include +#include #include @@ -566,7 +567,7 @@ public: /// Delete all directories which names begin with "tmp" /// Must be called with locked lockForShare() because it's using relative_data_path. - size_t clearOldTemporaryDirectories(const MergeTreeDataMergerMutator & merger_mutator, size_t custom_directories_lifetime_seconds); + size_t clearOldTemporaryDirectories(size_t custom_directories_lifetime_seconds); size_t clearEmptyParts(); @@ -906,7 +907,6 @@ public: mutable std::mutex currently_submerging_emerging_mutex; protected: - friend class IMergeTreeDataPart; friend class MergeTreeDataMergerMutator; friend struct ReplicatedMergeTreeTableMetadata; @@ -1200,6 +1200,8 @@ private: /// Create zero-copy exclusive lock for part and disk. Useful for coordination of /// distributed operations which can lead to data duplication. Implemented only in ReplicatedMergeTree. virtual std::optional tryCreateZeroCopyExclusiveLock(const String &, const DiskPtr &) { return std::nullopt; } + + TemporaryParts temporary_parts; }; /// RAII struct to record big parts that are submerging or emerging. diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 2f097b69fc4..a6cda0016a8 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -782,10 +782,4 @@ ExecuteTTLType MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadat } -bool MergeTreeDataMergerMutator::hasTemporaryPart(const std::string & basename) const -{ - std::lock_guard lock(tmp_parts_lock); - return tmp_parts.contains(basename); -} - } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 82cad873dce..ae09e2c916c 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -192,26 +192,6 @@ private: ITTLMergeSelector::PartitionIdToTTLs next_recompress_ttl_merge_times_by_partition; /// Performing TTL merges independently for each partition guarantees that /// there is only a limited number of TTL merges and no partition stores data, that is too stale - -public: - /// Returns true if passed part name is active. - /// (is the destination for one of active mutation/merge). - /// - /// NOTE: that it accept basename (i.e. dirname), not the path, - /// since later requires canonical form. - bool hasTemporaryPart(const std::string & basename) const; - -private: - /// Set of active temporary paths that is used as the destination. - /// List of such paths is required to avoid trying to remove them during cleanup. - /// - /// NOTE: It is pretty short, so use STL is fine. - std::unordered_set tmp_parts; - /// Lock for "tmp_parts". - /// - /// NOTE: mutable is required to mark hasTemporaryPath() const - mutable std::mutex tmp_parts_lock; - }; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 26bfd951d3d..3b6c727cd02 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -64,7 +64,7 @@ void ReplicatedMergeTreeCleanupThread::iterate() /// Both use relative_data_path which changes during rename, so we /// do it under share lock storage.clearOldWriteAheadLogs(); - storage.clearOldTemporaryDirectories(storage.merger_mutator, storage.getSettings()->temporary_directories_lifetime.totalSeconds()); + storage.clearOldTemporaryDirectories(storage.getSettings()->temporary_directories_lifetime.totalSeconds()); } /// This is loose condition: no problem if we actually had lost leadership at this moment diff --git a/src/Storages/MergeTree/TemporaryParts.cpp b/src/Storages/MergeTree/TemporaryParts.cpp new file mode 100644 index 00000000000..4239c8232e5 --- /dev/null +++ b/src/Storages/MergeTree/TemporaryParts.cpp @@ -0,0 +1,24 @@ +#include + +namespace DB +{ + +bool TemporaryParts::contains(const std::string & basename) const +{ + std::lock_guard lock(mutex); + return parts.contains(basename); +} + +void TemporaryParts::add(std::string basename) +{ + std::lock_guard lock(mutex); + parts.emplace(std::move(basename)); +} + +void TemporaryParts::remove(const std::string & basename) +{ + std::lock_guard lock(mutex); + parts.erase(basename); +} + +} diff --git a/src/Storages/MergeTree/TemporaryParts.h b/src/Storages/MergeTree/TemporaryParts.h new file mode 100644 index 00000000000..bc9d270856f --- /dev/null +++ b/src/Storages/MergeTree/TemporaryParts.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +/// Manages set of active temporary paths that should not be cleaned by background thread. +class TemporaryParts : private boost::noncopyable +{ +private: + /// To add const qualifier for contains() + mutable std::mutex mutex; + + /// NOTE: It is pretty short, so use STL is fine. + std::unordered_set parts; + +public: + /// Returns true if passed part name is active. + /// (is the destination for one of active mutation/merge). + /// + /// NOTE: that it accept basename (i.e. dirname), not the path, + /// since later requires canonical form. + bool contains(const std::string & basename) const; + + void add(std::string basename); + void remove(const std::string & basename); +}; + +} diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2db93def004..a05ed04a66c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -108,7 +108,7 @@ void StorageMergeTree::startup() /// Temporary directories contain incomplete results of merges (after forced restart) /// and don't allow to reinitialize them, so delete each of them immediately - clearOldTemporaryDirectories(merger_mutator, 0); + clearOldTemporaryDirectories(0); /// NOTE background task will also do the above cleanups periodically. time_after_previous_cleanup_parts.restart(); @@ -1062,7 +1062,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign assignee.scheduleCommonTask(ExecutableLambdaAdapter::create( [this, share_lock] () { - return clearOldTemporaryDirectories(merger_mutator, getSettings()->temporary_directories_lifetime.totalSeconds()); + return clearOldTemporaryDirectories(getSettings()->temporary_directories_lifetime.totalSeconds()); }, common_assignee_trigger, getStorageID()), /* need_trigger */ false); scheduled = true; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ab42396f8da..82bddddb32d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -451,7 +451,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( } /// Temporary directories contain uninitialized results of Merges or Fetches (after forced restart), /// don't allow to reinitialize them, delete each of them immediately. - clearOldTemporaryDirectories(merger_mutator, 0); + clearOldTemporaryDirectories(0); clearOldWriteAheadLogs(); } From 6499fc2c455289ec9e74689dd20d4ec0e8da8ab1 Mon Sep 17 00:00:00 2001 From: cnmade Date: Tue, 8 Mar 2022 17:03:46 +0800 Subject: [PATCH 136/165] Translate zh/sql-reference/statements/alter/settings-profile: rename old file --- .../alter/{settings-profile.md => settings-profile.md.bak} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename docs/zh/sql-reference/statements/alter/{settings-profile.md => settings-profile.md.bak} (100%) diff --git a/docs/zh/sql-reference/statements/alter/settings-profile.md b/docs/zh/sql-reference/statements/alter/settings-profile.md.bak similarity index 100% rename from docs/zh/sql-reference/statements/alter/settings-profile.md rename to docs/zh/sql-reference/statements/alter/settings-profile.md.bak From 80a8e4aa10c8a69901b31dc0eb93ad25953fe281 Mon Sep 17 00:00:00 2001 From: cnmade Date: Tue, 8 Mar 2022 17:10:54 +0800 Subject: [PATCH 137/165] Translate zh/sql-reference/statements/alter/settings-profile: reimport file --- .../statements/alter/settings-profile.md | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) create mode 100644 docs/zh/sql-reference/statements/alter/settings-profile.md diff --git a/docs/zh/sql-reference/statements/alter/settings-profile.md b/docs/zh/sql-reference/statements/alter/settings-profile.md new file mode 100644 index 00000000000..57d12142c48 --- /dev/null +++ b/docs/zh/sql-reference/statements/alter/settings-profile.md @@ -0,0 +1,16 @@ +--- +toc_priority: 48 +toc_title: SETTINGS PROFILE +--- + +## ALTER SETTINGS PROFILE {#alter-settings-profile-statement} + +Changes settings profiles. + +Syntax: + +``` sql +ALTER SETTINGS PROFILE [IF EXISTS] TO name1 [ON CLUSTER cluster_name1] [RENAME TO new_name1] + [, name2 [ON CLUSTER cluster_name2] [RENAME TO new_name2] ...] + [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | INHERIT 'profile_name'] [,...] +``` From 0d668e4b15caf9175ec809158f95f738c60d76fd Mon Sep 17 00:00:00 2001 From: cnmade Date: Tue, 8 Mar 2022 17:13:36 +0800 Subject: [PATCH 138/165] Translate zh/sql-reference/statements/alter/settings-profile: translate to zh --- .../zh/sql-reference/statements/alter/settings-profile.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/zh/sql-reference/statements/alter/settings-profile.md b/docs/zh/sql-reference/statements/alter/settings-profile.md index 57d12142c48..045b2461e8c 100644 --- a/docs/zh/sql-reference/statements/alter/settings-profile.md +++ b/docs/zh/sql-reference/statements/alter/settings-profile.md @@ -1,13 +1,13 @@ --- toc_priority: 48 -toc_title: SETTINGS PROFILE +toc_title: 配置文件设置 --- -## ALTER SETTINGS PROFILE {#alter-settings-profile-statement} +## 更改配置文件设置 {#alter-settings-profile-statement} -Changes settings profiles. +更改配置文件设置。 -Syntax: +语法: ``` sql ALTER SETTINGS PROFILE [IF EXISTS] TO name1 [ON CLUSTER cluster_name1] [RENAME TO new_name1] From e0ab2c7ca2a2dc235f6f1048d1cc5dccdb50daba Mon Sep 17 00:00:00 2001 From: cnmade Date: Tue, 8 Mar 2022 17:14:06 +0800 Subject: [PATCH 139/165] Translate zh/sql-reference/statements/alter/settings-profile: remove old file --- docs/zh/sql-reference/statements/alter/settings-profile.md.bak | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/sql-reference/statements/alter/settings-profile.md.bak diff --git a/docs/zh/sql-reference/statements/alter/settings-profile.md.bak b/docs/zh/sql-reference/statements/alter/settings-profile.md.bak deleted file mode 120000 index 0e71ac4e831..00000000000 --- a/docs/zh/sql-reference/statements/alter/settings-profile.md.bak +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/alter/settings-profile.md \ No newline at end of file From 225b0bd9140c9457e3f51ba0b648915ece4de097 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 8 Mar 2022 16:14:09 +0800 Subject: [PATCH 140/165] fixed bug: call need_more_input repeatly, overwrite the buffer --- src/IO/HadoopSnappyReadBuffer.cpp | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/src/IO/HadoopSnappyReadBuffer.cpp b/src/IO/HadoopSnappyReadBuffer.cpp index 324df67e900..cac05b4827b 100644 --- a/src/IO/HadoopSnappyReadBuffer.cpp +++ b/src/IO/HadoopSnappyReadBuffer.cpp @@ -11,7 +11,6 @@ #include "HadoopSnappyReadBuffer.h" - namespace DB { namespace ErrorCodes @@ -32,11 +31,11 @@ inline bool HadoopSnappyDecoder::checkAvailIn(size_t avail_in, int min) inline void HadoopSnappyDecoder::copyToBuffer(size_t * avail_in, const char ** next_in) { - assert(*avail_in <= sizeof(buffer)); + assert(*avail_in + buffer_length <= sizeof(buffer)); - memcpy(buffer, *next_in, *avail_in); + memcpy(buffer + buffer_length, *next_in, *avail_in); - buffer_length = *avail_in; + buffer_length += *avail_in; *next_in += *avail_in; *avail_in = 0; } @@ -78,14 +77,21 @@ inline HadoopSnappyDecoder::Status HadoopSnappyDecoder::readLength(size_t * avai inline HadoopSnappyDecoder::Status HadoopSnappyDecoder::readBlockLength(size_t * avail_in, const char ** next_in) { if (block_length < 0) + { return readLength(avail_in, next_in, &block_length); + } return Status::OK; } inline HadoopSnappyDecoder::Status HadoopSnappyDecoder::readCompressedLength(size_t * avail_in, const char ** next_in) { if (compressed_length < 0) - return readLength(avail_in, next_in, &compressed_length); + { + auto status = readLength(avail_in, next_in, &compressed_length); + if (unlikely(compressed_length > 0 && static_cast(compressed_length) > sizeof(buffer))) + throw Exception(ErrorCodes::SNAPPY_UNCOMPRESS_FAILED, "Too large snappy compressed block. buffer size: {}, compressed block size: {}", sizeof(buffer), compressed_length); + return status; + } return Status::OK; } @@ -111,7 +117,6 @@ HadoopSnappyDecoder::readCompressedData(size_t * avail_in, const char ** next_in { compressed = const_cast(*next_in); } - size_t uncompressed_length = *avail_out; auto status = snappy_uncompress(compressed, compressed_length, *next_out, &uncompressed_length); if (status != SNAPPY_OK) @@ -154,7 +159,9 @@ HadoopSnappyDecoder::Status HadoopSnappyDecoder::readBlock(size_t * avail_in, co return status; } if (total_uncompressed_length != block_length) + { return Status::INVALID_INPUT; + } return Status::OK; } From d98ef45a50d0bc5af1fda0c5c9a4e377e0cf1eff Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 8 Mar 2022 18:22:53 +0800 Subject: [PATCH 141/165] update tests --- tests/integration/test_hive_query/test.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index aa0616d6ac0..aa247e2c0af 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -44,10 +44,11 @@ CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32 def test_create_parquet_table_1(started_cluster): logging.info('Start testing creating hive table ...') node = started_cluster.instances['h0_0_0'] - node.query("set input_format_parquet_allow_missing_columns = true") - result = node.query(""" - DROP TABLE IF EXISTS default.demo_parquet_parts; - CREATE TABLE default.demo_parquet_parts (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String), `hour` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'parquet_demo') PARTITION BY(day, hour); + for i in range(10): + node.query("set input_format_parquet_allow_missing_columns = true") + result = node.query(""" +DROP TABLE IF EXISTS default.demo_parquet_parts; +CREATE TABLE default.demo_parquet_parts (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String), `hour` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'parquet_demo') PARTITION BY(day, hour); """) logging.info("create result {}".format(result)) if result.strip() == '': From 7f89a1bcf3b64f4f424931098e4ec2ee07dafc7e Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 8 Mar 2022 20:00:39 +0800 Subject: [PATCH 142/165] add some usage test --- src/IO/examples/hadoop_snappy_read_buffer.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/IO/examples/hadoop_snappy_read_buffer.cpp b/src/IO/examples/hadoop_snappy_read_buffer.cpp index 9cb01e6d697..eeac3db40a7 100644 --- a/src/IO/examples/hadoop_snappy_read_buffer.cpp +++ b/src/IO/examples/hadoop_snappy_read_buffer.cpp @@ -38,6 +38,11 @@ int main() return 1; } } + if (uncompress(256) != output) + { + std::cout << "test hadoop snappy read buffer failed, buf_size:" << 256 << std::endl; + return 1; + } std::cout << "test hadoop snappy read buffer success" << std::endl; return 0; } From c4b634285363093fb71a7e57fcd3273a0d52d91d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 8 Mar 2022 17:24:39 +0300 Subject: [PATCH 143/165] Improvements for `parallel_distributed_insert_select` (and related) (#34728) * Add a warning if parallel_distributed_insert_select was ignored Signed-off-by: Azat Khuzhin * Respect max_distributed_depth for parallel_distributed_insert_select Signed-off-by: Azat Khuzhin * Print warning for non applied parallel_distributed_insert_select only for initial query Signed-off-by: Azat Khuzhin * Remove Cluster::getHashOfAddresses() Signed-off-by: Azat Khuzhin * Forbid parallel_distributed_insert_select for remote()/cluster() with different addresses Before it uses empty cluster name (getClusterName()) which is not correct, compare all addresses instead. Signed-off-by: Azat Khuzhin * Fix max_distributed_depth check max_distributed_depth=1 must mean not more then one distributed query, not two, since max_distributed_depth=0 means no limit, and distribute_depth is 0 for the first query. Signed-off-by: Azat Khuzhin * Fix INSERT INTO remote()/cluster() with parallel_distributed_insert_select Signed-off-by: Azat Khuzhin * Add a test for parallel_distributed_insert_select with cluster()/remote() Signed-off-by: Azat Khuzhin * Return instead of empty cluster name in Distributed engine Signed-off-by: Azat Khuzhin * Make user with sharding_key and w/o in remote()/cluster() identical Before with sharding_key the user was "default", while w/o it it was empty. Signed-off-by: Azat Khuzhin --- src/Interpreters/Cluster.h | 2 -- .../ClusterProxy/executeQuery.cpp | 2 +- src/Storages/Distributed/DistributedSink.cpp | 2 +- .../ExternalDataSourceConfiguration.h | 2 +- src/Storages/StorageDistributed.cpp | 33 ++++++++++++++++-- src/Storages/StorageDistributed.h | 3 +- ...istributed_insert_select_cluster.reference | 27 +++++++++++++++ ...llel_distributed_insert_select_cluster.sql | 34 +++++++++++++++++++ 8 files changed, 95 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02224_parallel_distributed_insert_select_cluster.reference create mode 100644 tests/queries/0_stateless/02224_parallel_distributed_insert_select_cluster.sql diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index 3773dadaf13..248d212ebf0 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -207,7 +207,6 @@ public: using ShardsInfo = std::vector; - String getHashOfAddresses() const { return hash_of_addresses; } const ShardsInfo & getShardsInfo() const { return shards_info; } const AddressesWithFailover & getShardsAddresses() const { return addresses_with_failover; } @@ -263,7 +262,6 @@ private: /// Inter-server secret String secret; - String hash_of_addresses; /// Description of the cluster shards. ShardsInfo shards_info; /// Any remote shard. diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 0db07267231..884b8445732 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -116,7 +116,7 @@ void executeQuery( const Settings & settings = context->getSettingsRef(); - if (settings.max_distributed_depth && context->getClientInfo().distributed_depth > settings.max_distributed_depth) + if (settings.max_distributed_depth && context->getClientInfo().distributed_depth >= settings.max_distributed_depth) throw Exception("Maximum distributed depth exceeded", ErrorCodes::TOO_LARGE_DISTRIBUTED_DEPTH); std::vector plans; diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 9951fb436b5..aa703bcbb89 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -126,7 +126,7 @@ DistributedSink::DistributedSink( , log(&Poco::Logger::get("DistributedBlockOutputStream")) { const auto & settings = context->getSettingsRef(); - if (settings.max_distributed_depth && context->getClientInfo().distributed_depth > settings.max_distributed_depth) + if (settings.max_distributed_depth && context->getClientInfo().distributed_depth >= settings.max_distributed_depth) throw Exception("Maximum distributed depth exceeded", ErrorCodes::TOO_LARGE_DISTRIBUTED_DEPTH); context->getClientInfo().distributed_depth += 1; random_shard_insert = settings.insert_distributed_one_random_shard && !storage.has_sharding_key; diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index 1e08b088b1d..cc3e136ba50 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -16,7 +16,7 @@ struct ExternalDataSourceConfiguration { String host; UInt16 port = 0; - String username; + String username = "default"; String password; String database; String table; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index da648aa4e5c..fcbf22bbd33 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -118,6 +118,7 @@ namespace ErrorCodes extern const int ALTER_OF_COLUMN_IS_FORBIDDEN; extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int TOO_LARGE_DISTRIBUTED_DEPTH; } namespace ActionLocks @@ -705,6 +706,9 @@ SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadata QueryPipelineBuilderPtr StorageDistributed::distributedWrite(const ASTInsertQuery & query, ContextPtr local_context) { const Settings & settings = local_context->getSettingsRef(); + if (settings.max_distributed_depth && local_context->getClientInfo().distributed_depth >= settings.max_distributed_depth) + throw Exception("Maximum distributed depth exceeded", ErrorCodes::TOO_LARGE_DISTRIBUTED_DEPTH); + std::shared_ptr storage_src; auto & select = query.select->as(); auto new_query = std::dynamic_pointer_cast(query.clone()); @@ -733,14 +737,34 @@ QueryPipelineBuilderPtr StorageDistributed::distributedWrite(const ASTInsertQuer } } - if (!storage_src || storage_src->getClusterName() != getClusterName()) + const Cluster::AddressesWithFailover & src_addresses = storage_src ? storage_src->getCluster()->getShardsAddresses() : Cluster::AddressesWithFailover{}; + const Cluster::AddressesWithFailover & dst_addresses = getCluster()->getShardsAddresses(); + /// Compare addresses instead of cluster name, to handle remote()/cluster(). + /// (since for remote()/cluster() the getClusterName() is empty string) + if (src_addresses != dst_addresses) { + /// The warning should be produced only for root queries, + /// since in case of parallel_distributed_insert_select=1, + /// it will produce warning for the rewritten insert, + /// since destination table is still Distributed there. + if (local_context->getClientInfo().distributed_depth == 0) + { + LOG_WARNING(log, + "Parallel distributed INSERT SELECT is not possible " + "(source cluster={} ({} addresses), destination cluster={} ({} addresses))", + storage_src ? storage_src->getClusterName() : "", + src_addresses.size(), + getClusterName(), + dst_addresses.size()); + } return nullptr; } if (settings.parallel_distributed_insert_select == PARALLEL_DISTRIBUTED_INSERT_SELECT_ALL) { new_query->table_id = StorageID(getRemoteDatabaseName(), getRemoteTableName()); + /// Reset table function for INSERT INTO remote()/cluster() + new_query->table_function.reset(); } const auto & cluster = getCluster(); @@ -757,12 +781,15 @@ QueryPipelineBuilderPtr StorageDistributed::distributedWrite(const ASTInsertQuer new_query_str = buf.str(); } + ContextMutablePtr query_context = Context::createCopy(local_context); + ++query_context->getClientInfo().distributed_depth; + for (size_t shard_index : collections::range(0, shards_info.size())) { const auto & shard_info = shards_info[shard_index]; if (shard_info.isLocal()) { - InterpreterInsertQuery interpreter(new_query, local_context); + InterpreterInsertQuery interpreter(new_query, query_context); pipelines.emplace_back(std::make_unique()); pipelines.back()->init(interpreter.execute().pipeline); } @@ -776,7 +803,7 @@ QueryPipelineBuilderPtr StorageDistributed::distributedWrite(const ASTInsertQuer /// INSERT SELECT query returns empty block auto remote_query_executor - = std::make_shared(shard_info.pool, std::move(connections), new_query_str, Block{}, local_context); + = std::make_shared(shard_info.pool, std::move(connections), new_query_str, Block{}, query_context); pipelines.emplace_back(std::make_unique()); pipelines.back()->init(Pipe(std::make_shared(remote_query_executor, false, settings.async_socket_for_remote))); pipelines.back()->setSinks([](const Block & header, QueryPipelineBuilder::StreamType) -> ProcessorPtr diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index e47e0fddd6c..45b1cd640ee 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -114,8 +114,6 @@ public: /// Used by InterpreterInsertQuery std::string getRemoteDatabaseName() const { return remote_database; } std::string getRemoteTableName() const { return remote_table; } - /// Returns empty string if tables is used by TableFunctionRemote - std::string getClusterName() const { return cluster_name; } ClusterPtr getCluster() const; /// Used by InterpreterSystemQuery @@ -201,6 +199,7 @@ private: std::optional getOptimizedQueryProcessingStage(const SelectQueryInfo & query_info, const Settings & settings) const; size_t getRandomShardIndex(const Cluster::ShardsInfo & shards); + std::string getClusterName() const { return cluster_name.empty() ? "" : cluster_name; } const DistributedSettings & getDistributedSettingsRef() const { return distributed_settings; } diff --git a/tests/queries/0_stateless/02224_parallel_distributed_insert_select_cluster.reference b/tests/queries/0_stateless/02224_parallel_distributed_insert_select_cluster.reference new file mode 100644 index 00000000000..05fbb680c65 --- /dev/null +++ b/tests/queries/0_stateless/02224_parallel_distributed_insert_select_cluster.reference @@ -0,0 +1,27 @@ +-- { echoOn } +truncate table dst_02224; +insert into function cluster('test_cluster_two_shards', currentDatabase(), dst_02224, key) +select * from cluster('test_cluster_two_shards', currentDatabase(), src_02224, key) +settings parallel_distributed_insert_select=1, max_distributed_depth=1; -- { serverError TOO_LARGE_DISTRIBUTED_DEPTH } +select * from dst_02224; +truncate table dst_02224; +insert into function cluster('test_cluster_two_shards', currentDatabase(), dst_02224, key) +select * from cluster('test_cluster_two_shards', currentDatabase(), src_02224, key) +settings parallel_distributed_insert_select=1, max_distributed_depth=2; +select * from dst_02224; +1 +1 +truncate table dst_02224; +insert into function cluster('test_cluster_two_shards', currentDatabase(), dst_02224, key) +select * from cluster('test_cluster_two_shards', currentDatabase(), src_02224, key) +settings parallel_distributed_insert_select=2, max_distributed_depth=1; +select * from dst_02224; +1 +1 +truncate table dst_02224; +insert into function remote('127.{1,2}', currentDatabase(), dst_02224, key) +select * from remote('127.{1,2}', currentDatabase(), src_02224, key) +settings parallel_distributed_insert_select=2, max_distributed_depth=1; +select * from dst_02224; +1 +1 diff --git a/tests/queries/0_stateless/02224_parallel_distributed_insert_select_cluster.sql b/tests/queries/0_stateless/02224_parallel_distributed_insert_select_cluster.sql new file mode 100644 index 00000000000..023f220e930 --- /dev/null +++ b/tests/queries/0_stateless/02224_parallel_distributed_insert_select_cluster.sql @@ -0,0 +1,34 @@ +drop table if exists dst_02224; +drop table if exists src_02224; +create table dst_02224 (key Int) engine=Memory(); +create table src_02224 (key Int) engine=Memory(); +insert into src_02224 values (1); + +-- { echoOn } +truncate table dst_02224; +insert into function cluster('test_cluster_two_shards', currentDatabase(), dst_02224, key) +select * from cluster('test_cluster_two_shards', currentDatabase(), src_02224, key) +settings parallel_distributed_insert_select=1, max_distributed_depth=1; -- { serverError TOO_LARGE_DISTRIBUTED_DEPTH } +select * from dst_02224; + +truncate table dst_02224; +insert into function cluster('test_cluster_two_shards', currentDatabase(), dst_02224, key) +select * from cluster('test_cluster_two_shards', currentDatabase(), src_02224, key) +settings parallel_distributed_insert_select=1, max_distributed_depth=2; +select * from dst_02224; + +truncate table dst_02224; +insert into function cluster('test_cluster_two_shards', currentDatabase(), dst_02224, key) +select * from cluster('test_cluster_two_shards', currentDatabase(), src_02224, key) +settings parallel_distributed_insert_select=2, max_distributed_depth=1; +select * from dst_02224; + +truncate table dst_02224; +insert into function remote('127.{1,2}', currentDatabase(), dst_02224, key) +select * from remote('127.{1,2}', currentDatabase(), src_02224, key) +settings parallel_distributed_insert_select=2, max_distributed_depth=1; +select * from dst_02224; +-- { echoOff } + +drop table src_02224; +drop table dst_02224; From 52ed751d58d228325d17b15961a3f3ea7c2d6ee8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 8 Mar 2022 16:29:42 +0100 Subject: [PATCH 144/165] Fix installation documentation typo --- docs/_includes/install/deb.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/_includes/install/deb.sh b/docs/_includes/install/deb.sh index 9dceef4c245..0daf12a132f 100644 --- a/docs/_includes/install/deb.sh +++ b/docs/_includes/install/deb.sh @@ -1,11 +1,11 @@ -sudo apt-get install apt-transport-https ca-certificates dirmngr +sudo apt-get install -y apt-transport-https ca-certificates dirmngr sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754 -echo "deb https://packages.clickhouse.com/deb stable main/" | sudo tee \ +echo "deb https://packages.clickhouse.com/deb stable main" | sudo tee \ /etc/apt/sources.list.d/clickhouse.list sudo apt-get update sudo apt-get install -y clickhouse-server clickhouse-client sudo service clickhouse-server start -clickhouse-client # or "clickhouse-client --password" if you set up a password. +clickhouse-client # or "clickhouse-client --password" if you've set up a password. From a871036361ea8e57660ecd88f1da5dea29b5ebf4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 8 Mar 2022 18:42:29 +0300 Subject: [PATCH 145/165] Fix `parallel_reading_from_replicas` with `clickhouse-bechmark` (#34751) * Use INITIAL_QUERY for clickhouse-benchmark Signed-off-by: Azat Khuzhin * Fix parallel_reading_from_replicas with clickhouse-bechmark Before it produces the following error: $ clickhouse-benchmark --stacktrace -i1 --query "select * from remote('127.1', default.data_mt) limit 10" --allow_experimental_parallel_reading_from_replicas=1 --max_parallel_replicas=3 Loaded 1 queries. Logical error: 'Coordinator for parallel reading from replicas is not initialized'. Aborted (core dumped) Since it uses the same code, i.e RemoteQueryExecutor -> MultiplexedConnections, which enables coordinator if it was requested from settings, but it should be done only for non-initial queries, i.e. when server send connection to another server. Signed-off-by: Azat Khuzhin * Fix 02226_parallel_reading_from_replicas_benchmark for older shellcheck By shellcheck 0.8 does not complains, while on CI shellcheck 0.7.0 and it does complains [1]: In 02226_parallel_reading_from_replicas_benchmark.sh line 17: --allow_experimental_parallel_reading_from_replicas=1 ^-- SC2191: The = here is literal. To assign by index, use ( [index]=value ) with no spaces. To keep as literal, quote it. Did you mean: "--allow_experimental_parallel_reading_from_replicas=1" [1]: https://s3.amazonaws.com/clickhouse-test-reports/34751/d883af711822faf294c876b017cbf745b1cda1b3/style_check__actions_/shellcheck_output.txt Signed-off-by: Azat Khuzhin --- programs/benchmark/Benchmark.cpp | 2 ++ src/Client/MultiplexedConnections.cpp | 7 ++++- src/QueryPipeline/RemoteQueryExecutor.cpp | 10 ++----- src/QueryPipeline/RemoteQueryExecutor.h | 8 ++++- ..._reading_from_replicas_benchmark.reference | 0 ...arallel_reading_from_replicas_benchmark.sh | 29 +++++++++++++++++++ 6 files changed, 46 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.reference create mode 100755 tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 35ffb97b8e2..60e5ca92f77 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -435,6 +435,8 @@ private: Progress progress; executor.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); }); + executor.sendQuery(ClientInfo::QueryKind::INITIAL_QUERY); + ProfileInfo info; while (Block block = executor.read()) info.update(block); diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index d1873ac038d..31fbc609bdc 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -133,7 +133,12 @@ void MultiplexedConnections::sendQuery( modified_settings.group_by_two_level_threshold_bytes = 0; } - if (settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas) + bool parallel_reading_from_replicas = settings.max_parallel_replicas > 1 + && settings.allow_experimental_parallel_reading_from_replicas + /// To avoid trying to coordinate with clickhouse-benchmark, + /// since it uses the same code. + && client_info.query_kind != ClientInfo::QueryKind::INITIAL_QUERY; + if (parallel_reading_from_replicas) { client_info.collaborate_with_initiator = true; client_info.count_participating_replicas = replica_info.all_replicas_count; diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 142e56ceb25..d1275444b84 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -210,7 +210,7 @@ static Block adaptBlockStructure(const Block & block, const Block & header) return res; } -void RemoteQueryExecutor::sendQuery() +void RemoteQueryExecutor::sendQuery(ClientInfo::QueryKind query_kind) { if (sent_query) return; @@ -237,13 +237,7 @@ void RemoteQueryExecutor::sendQuery() auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings); ClientInfo modified_client_info = context->getClientInfo(); - modified_client_info.query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - /// Set initial_query_id to query_id for the clickhouse-benchmark. - /// - /// (since first query of clickhouse-benchmark will be issued as SECONDARY_QUERY, - /// due to it executes queries via RemoteBlockInputStream) - if (modified_client_info.initial_query_id.empty()) - modified_client_info.initial_query_id = query_id; + modified_client_info.query_kind = query_kind; if (CurrentThread::isInitialized()) { modified_client_info.client_trace_context = CurrentThread::get().thread_trace_context; diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 655bd5603de..78bc9f611ab 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -83,7 +83,13 @@ public: ~RemoteQueryExecutor(); /// Create connection and send query, external tables and scalars. - void sendQuery(); + /// + /// @param query_kind - kind of query, usually it is SECONDARY_QUERY, + /// since this is the queries between servers + /// (for which this code was written in general). + /// But clickhouse-benchmark uses the same code, + /// and it should pass INITIAL_QUERY. + void sendQuery(ClientInfo::QueryKind query_kind = ClientInfo::QueryKind::SECONDARY_QUERY); /// Query is resent to a replica, the query itself can be modified. std::atomic resent_query { false }; diff --git a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.reference b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh new file mode 100755 index 00000000000..2a163746e20 --- /dev/null +++ b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh @@ -0,0 +1,29 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -nm -q " +drop table if exists data_02226; +create table data_02226 (key Int) engine=MergeTree() order by key +as select * from numbers(1); +" + +# Regression for: +# +# Logical error: 'Coordinator for parallel reading from replicas is not initialized'. +opts=( + --allow_experimental_parallel_reading_from_replicas 1 + --max_parallel_replicas 3 + + --iterations 1 +) +$CLICKHOUSE_BENCHMARK --query "select * from remote('127.1', $CLICKHOUSE_DATABASE, data_02226)" "${opts[@]}" >& /dev/null +ret=$? + +$CLICKHOUSE_CLIENT -nm -q " +drop table data_02226; +" + +exit $ret From ced34dea84246c24d776dd58206f79c26ccf7533 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 5 Mar 2022 14:43:06 +0300 Subject: [PATCH 146/165] Take flush_time into account for scheduling background flush of the Buffer Signed-off-by: Azat Khuzhin --- src/Storages/StorageBuffer.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index f97c09471c3..c1f2e14da7c 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -1000,7 +1000,8 @@ void StorageBuffer::reschedule() size_t min = std::max(min_thresholds.time - time_passed, 1); size_t max = std::max(max_thresholds.time - time_passed, 1); - flush_handle->scheduleAfter(std::min(min, max) * 1000); + size_t flush = std::max(flush_thresholds.time - time_passed, 1); + flush_handle->scheduleAfter(std::min({min, max, flush}) * 1000); } void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const From 132bbce29cf26f2e93de852e6ddce9f2e3f0023d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 18 Feb 2022 17:42:32 +0300 Subject: [PATCH 147/165] Add ability to get SELECT query from TableFunctionView Signed-off-by: Azat Khuzhin --- src/TableFunctions/TableFunctionView.cpp | 6 ++++++ src/TableFunctions/TableFunctionView.h | 3 +++ 2 files changed, 9 insertions(+) diff --git a/src/TableFunctions/TableFunctionView.cpp b/src/TableFunctions/TableFunctionView.cpp index 2cab8aeca25..e9fcbb219a3 100644 --- a/src/TableFunctions/TableFunctionView.cpp +++ b/src/TableFunctions/TableFunctionView.cpp @@ -15,6 +15,12 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } + +const ASTSelectWithUnionQuery & TableFunctionView::getSelectQuery() const +{ + return *create.select; +} + void TableFunctionView::parseArguments(const ASTPtr & ast_function, ContextPtr /*context*/) { const auto * function = ast_function->as(); diff --git a/src/TableFunctions/TableFunctionView.h b/src/TableFunctions/TableFunctionView.h index c20b45e7546..4afb049e738 100644 --- a/src/TableFunctions/TableFunctionView.h +++ b/src/TableFunctions/TableFunctionView.h @@ -16,6 +16,9 @@ class TableFunctionView : public ITableFunction public: static constexpr auto name = "view"; std::string getName() const override { return name; } + + const ASTSelectWithUnionQuery & getSelectQuery() const; + private: StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns) const override; const char * getStorageTypeName() const override { return "View"; } From fd3f7347f3e5f9dbc4f1f7279be8687ff85a0560 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 19 Feb 2022 10:48:06 +0300 Subject: [PATCH 148/165] Remove unused DBMS_COMMON_LIBRARIES Fixes: 4f8438bb346 ("cmake: do not allow internal libstdc++ usage") Signed-off-by: Azat Khuzhin --- src/CMakeLists.txt | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index b99ffd7ee18..0a831950d5e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -253,18 +253,16 @@ if (TARGET ch_contrib::nuraft) add_object_library(clickhouse_coordination Coordination) endif() -set (DBMS_COMMON_LIBRARIES) - if (USE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) add_library (dbms STATIC ${dbms_headers} ${dbms_sources}) - target_link_libraries (dbms PRIVATE ch_contrib::libdivide ${DBMS_COMMON_LIBRARIES}) + target_link_libraries (dbms PRIVATE ch_contrib::libdivide) if (TARGET ch_contrib::jemalloc) target_link_libraries (dbms PRIVATE ch_contrib::jemalloc) endif() set (all_modules dbms) else() add_library (dbms SHARED ${dbms_headers} ${dbms_sources}) - target_link_libraries (dbms PUBLIC ${all_modules} ${DBMS_COMMON_LIBRARIES}) + target_link_libraries (dbms PUBLIC ${all_modules}) target_link_libraries (clickhouse_interpreters PRIVATE ch_contrib::libdivide) if (TARGET ch_contrib::jemalloc) target_link_libraries (clickhouse_interpreters PRIVATE ch_contrib::jemalloc) From 75da778d10fd005703523ddd9837617803be0d69 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 19 Feb 2022 11:02:45 +0300 Subject: [PATCH 149/165] Tiny cmake refactoring Signed-off-by: Azat Khuzhin --- src/CMakeLists.txt | 22 +++++++++++++++++----- src/TableFunctions/CMakeLists.txt | 16 +++++++++------- 2 files changed, 26 insertions(+), 12 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 0a831950d5e..ebcd027ed2b 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -144,7 +144,6 @@ list (APPEND dbms_sources AggregateFunctions/AggregateFunctionState.cpp AggregateFunctions/AggregateFunctionCount.cpp AggregateFunctions/parseAggregateFunctionParameters.cpp) - list (APPEND dbms_headers AggregateFunctions/IAggregateFunction.h AggregateFunctions/IAggregateFunctionCombinator.h @@ -155,10 +154,23 @@ list (APPEND dbms_headers AggregateFunctions/FactoryHelpers.h AggregateFunctions/parseAggregateFunctionParameters.h) -list (APPEND dbms_sources TableFunctions/ITableFunction.cpp TableFunctions/TableFunctionFactory.cpp) -list (APPEND dbms_headers TableFunctions/ITableFunction.h TableFunctions/TableFunctionFactory.h) -list (APPEND dbms_sources Dictionaries/DictionaryFactory.cpp Dictionaries/DictionarySourceFactory.cpp Dictionaries/DictionaryStructure.cpp Dictionaries/getDictionaryConfigurationFromAST.cpp) -list (APPEND dbms_headers Dictionaries/DictionaryFactory.h Dictionaries/DictionarySourceFactory.h Dictionaries/DictionaryStructure.h Dictionaries/getDictionaryConfigurationFromAST.h) +list (APPEND dbms_sources + TableFunctions/ITableFunction.cpp + TableFunctions/TableFunctionFactory.cpp) +list (APPEND dbms_headers + TableFunctions/ITableFunction.h + TableFunctions/TableFunctionFactory.h) + +list (APPEND dbms_sources + Dictionaries/DictionaryFactory.cpp + Dictionaries/DictionarySourceFactory.cpp + Dictionaries/DictionaryStructure.cpp + Dictionaries/getDictionaryConfigurationFromAST.cpp) +list (APPEND dbms_headers + Dictionaries/DictionaryFactory.h + Dictionaries/DictionarySourceFactory.h + Dictionaries/DictionaryStructure.h + Dictionaries/getDictionaryConfigurationFromAST.h) if (NOT ENABLE_SSL) list (REMOVE_ITEM clickhouse_common_io_sources Common/OpenSSLHelpers.cpp) diff --git a/src/TableFunctions/CMakeLists.txt b/src/TableFunctions/CMakeLists.txt index c9948a4b131..21d329667a5 100644 --- a/src/TableFunctions/CMakeLists.txt +++ b/src/TableFunctions/CMakeLists.txt @@ -4,14 +4,16 @@ if (TARGET ch_contrib::hivemetastore) add_headers_and_sources(clickhouse_table_functions Hive) endif () -list(REMOVE_ITEM clickhouse_table_functions_sources ITableFunction.cpp TableFunctionFactory.cpp) -list(REMOVE_ITEM clickhouse_table_functions_headers ITableFunction.h TableFunctionFactory.h) +list(REMOVE_ITEM clickhouse_table_functions_sources + ITableFunction.cpp + TableFunctionFactory.cpp) +list(REMOVE_ITEM clickhouse_table_functions_headers + ITableFunction.h + TableFunctionFactory.h) add_library(clickhouse_table_functions ${clickhouse_table_functions_sources}) + +target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_parsers clickhouse_storages_system dbms) if (TARGET ch_contrib::hivemetastore) - target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_parsers clickhouse_storages_system dbms ch_contrib::hivemetastore ch_contrib::hdfs) -else () - target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_parsers clickhouse_storages_system dbms) + target_link_libraries(clickhouse_table_functions PRIVATE ch_contrib::hivemetastore ch_contrib::hdfs) endif () - - From 4843e210c322b07fcd3a899afa35cf7a31109441 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 18 Feb 2022 17:42:48 +0300 Subject: [PATCH 150/165] Support view() for parallel_distributed_insert_select Signed-off-by: Azat Khuzhin --- src/CMakeLists.txt | 2 ++ src/Storages/StorageDistributed.cpp | 26 ++++++++++---- src/TableFunctions/CMakeLists.txt | 2 ++ ...l_distributed_insert_select_view.reference | 4 +++ ...parallel_distributed_insert_select_view.sh | 35 +++++++++++++++++++ 5 files changed, 63 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.reference create mode 100755 tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index ebcd027ed2b..e18914740ff 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -156,9 +156,11 @@ list (APPEND dbms_headers list (APPEND dbms_sources TableFunctions/ITableFunction.cpp + TableFunctions/TableFunctionView.cpp TableFunctions/TableFunctionFactory.cpp) list (APPEND dbms_headers TableFunctions/ITableFunction.h + TableFunctions/TableFunctionView.h TableFunctions/TableFunctionFactory.h) list (APPEND dbms_sources diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index fcbf22bbd33..5bfb3b4ce45 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -56,6 +56,8 @@ #include #include #include +#include +#include #include #include @@ -723,15 +725,27 @@ QueryPipelineBuilderPtr StorageDistributed::distributedWrite(const ASTInsertQuer storage_src = std::dynamic_pointer_cast(joined_tables.getLeftTableStorage()); if (storage_src) { - const auto select_with_union_query = std::make_shared(); - select_with_union_query->list_of_selects = std::make_shared(); + /// Unwrap view() function. + if (storage_src->remote_table_function_ptr) + { + const TableFunctionPtr src_table_function = + TableFunctionFactory::instance().get(storage_src->remote_table_function_ptr, local_context); + const TableFunctionView * view_function = + assert_cast(src_table_function.get()); + new_query->select = view_function->getSelectQuery().clone(); + } + else + { + const auto select_with_union_query = std::make_shared(); + select_with_union_query->list_of_selects = std::make_shared(); - auto new_select_query = std::dynamic_pointer_cast(select_query->clone()); - select_with_union_query->list_of_selects->children.push_back(new_select_query); + auto new_select_query = std::dynamic_pointer_cast(select_query->clone()); + select_with_union_query->list_of_selects->children.push_back(new_select_query); - new_select_query->replaceDatabaseAndTable(storage_src->getRemoteDatabaseName(), storage_src->getRemoteTableName()); + new_select_query->replaceDatabaseAndTable(storage_src->getRemoteDatabaseName(), storage_src->getRemoteTableName()); - new_query->select = select_with_union_query; + new_query->select = select_with_union_query; + } } } } diff --git a/src/TableFunctions/CMakeLists.txt b/src/TableFunctions/CMakeLists.txt index 21d329667a5..c58f93e310a 100644 --- a/src/TableFunctions/CMakeLists.txt +++ b/src/TableFunctions/CMakeLists.txt @@ -6,9 +6,11 @@ endif () list(REMOVE_ITEM clickhouse_table_functions_sources ITableFunction.cpp + TableFunctionView.cpp TableFunctionFactory.cpp) list(REMOVE_ITEM clickhouse_table_functions_headers ITableFunction.h + TableFunctionView.h TableFunctionFactory.h) add_library(clickhouse_table_functions ${clickhouse_table_functions_sources}) diff --git a/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.reference b/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.reference new file mode 100644 index 00000000000..98fb6a68656 --- /dev/null +++ b/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.reference @@ -0,0 +1,4 @@ +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh b/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh new file mode 100755 index 00000000000..376a49fd820 --- /dev/null +++ b/tests/queries/0_stateless/02225_parallel_distributed_insert_select_view.sh @@ -0,0 +1,35 @@ +#!/usr/bin/env bash + +# NOTE: sh test is required since view() does not have current database + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -nm -q " +drop table if exists dst_02225; +drop table if exists src_02225; +create table dst_02225 (key Int) engine=Memory(); +create table src_02225 (key Int) engine=Memory(); +insert into src_02225 values (1); +" + +$CLICKHOUSE_CLIENT --param_database=$CLICKHOUSE_DATABASE -nm -q " +truncate table dst_02225; +insert into function remote('127.{1,2}', currentDatabase(), dst_02225, key) +select * from remote('127.{1,2}', view(select * from {database:Identifier}.src_02225), key) +settings parallel_distributed_insert_select=2, max_distributed_depth=1; +select * from dst_02225; + +-- w/o sharding key +truncate table dst_02225; +insert into function remote('127.{1,2}', currentDatabase(), dst_02225, key) +select * from remote('127.{1,2}', view(select * from {database:Identifier}.src_02225)) +settings parallel_distributed_insert_select=2, max_distributed_depth=1; +select * from dst_02225; +" + +$CLICKHOUSE_CLIENT -nm -q " +drop table src_02225; +drop table dst_02225; +" From 8dc8c87fd1cea08e7e9875dd7774b02b1f3eef63 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 9 Mar 2022 10:03:04 +0800 Subject: [PATCH 151/165] add a test case --- src/IO/tests/gtest_hadoop_snappy_decoder.cpp | 38 ++++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 src/IO/tests/gtest_hadoop_snappy_decoder.cpp diff --git a/src/IO/tests/gtest_hadoop_snappy_decoder.cpp b/src/IO/tests/gtest_hadoop_snappy_decoder.cpp new file mode 100644 index 00000000000..81174689566 --- /dev/null +++ b/src/IO/tests/gtest_hadoop_snappy_decoder.cpp @@ -0,0 +1,38 @@ +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +using namespace DB; +TEST(HadoopSnappyDecoder, repeat_need_more_input) +{ + String snappy_base64_content = "AAAl6gAAB67qSzQyMDIxLTA2LTAxAXh4eGIEACQzMTkuNzQyNDMKnjEAHDQyLjgyMTcynjEAIDI5Ni4yODQwNqIxABgyNy43MjYzqpMAGDMuNTIyMzSiYgAcNjUuNDk1OTeiMQAYOTQuNTg1NaYxABg4OC40NzgyojEAHDMyMS4zOTE1osQAHDM0Ni4xNTI3qjEAGDEuMjA3MTWm9QAQMi41MjamYQAcMjIuNTEyNDieYQAcMzMwLjI5MTKiGgIcMzIzLjAzNDiiwwAcMzE1LjA1MDmiYgAcNDM1Ljc2ODaqxAAUMS45NDA5nvQACDAuMP4rAEorABwzMDMuMjAyNaYZARgwOC4xOTEypugAGDQ2LjQ0MjKilQMcMjc4Ljk3MTiiMQAcMzUwLjc3NTeirAGqSwEcMzI5LjkyMzGiXAAcMzMxLjc2NzamwAMUMTMuNjM4pjEAGDI3NC4yMzK2MQAINDg0qrMBFDExLjgzNqbbBRgyNDkuNTI5qtsFGDUwLjE4ODmi5AGlSAgwNjWmiAEUMjIuNjU4pqcCBDUzYcCqdgIYMDEuMzcxNbbPBgQ5Na5TBBA0Ljc1OaIiBMGdDDM0OTGeJwQcMjg3LjIyNTmm/AMhVAAyopAAGDMxOC4wMjGmMAAB8AQ0OKpGAhgyMC42MTM4poMBFDg3LjEzOKoxABA5My4xNaZSARQ5NS41ODemTgVhOQwwODg2osIAGDMyNi45NTSmMQAcMjc3LjgxNDmqjQcMNS42MqpqA0F3DDg2MDamzAPhKwQ4OKJWARgzMDYuMTc1qi0EGDgwLjIwNTSihAUYMjk3LjY5NaYiBRAyOTAuM6aNBBgyMzkuMzI5pkIJwdOi7wcYMzcxLjIyNqpiBxQ0NS44NzGq9woEODAOZAoANqJ+BRgyNzYuMjExpnYCIYIMMjIyOKKnAmVrBDc0psQAEDMwOS4xqtEJGDMwNC45MzSq8wAMNC40OKomCyG3DDE4MTGi/AMhJAQxMKqjBhgyNjEuNDQ4rqMGFDIuOTEwN6I5AwQzN7JMCQw2LjcwqqoMGDI2MC44NzOmdwIOTAkMNDgzMqLSBhQyNTkuMjGmYweBiwg3MzOmyQMYNDM3Ljg1N6ZyBq5QARQzMy43MjSqKw4UMTIuNzkxpkkFDmgNDDc4MzCmUAEUOTUuOTQypnoFDiQIDDI2ODmmBQMUNTEuMjc2qikEDtkJBDA1qgUDFDA3LjE1N6ZiAOGUCDMwOaoxABA3NC42NqqmAhA5Ni45N6rIAxwzMDcuMjkzMaL+ChQyNzUuODau/QoANOExpugBGDI0Ny4xODSm5wEYOTEuNDE3MZ7MChQzMzUuNjWquQQUNTMuODg1psMHDu8SCDIyOaYJDoFbCDk4M6aWDhwzNDEuNTcyMKK1AUF4ADSqCwoQMzg1LjSujBIB9Aw0MDUwotoJDi4PCDc0N6aHARgyMjMuODMxpgYRwmcRGDIxMi4xNjWqSgIQMDkuODmuzgMYMTkuNTg0MaK7CMFFADmuZQcQMDYuMzKqXwAIOS4zrl8ADu4PBDQ0qtQUGDQ3LjAzODGmFwIYMTAuOTIwMKLDAKG0DDM3MDOiYgCqNgcORgkEMzeuGwWqXQAhqwg2MDWmSQUYMjY0LjE2N6aZFBIgFgQyM6aiCRQwNi41NTSm9AcYMjczLjczNqqSABg0NS45OTIzpugPFDIxLjc3MqZ4EBwyODYuMDkyNKZAAhg0OS4yMjQzom8GDu0LCDEwNKaTBwAzDiUIADimGQkUMzM4Ljc2qlITADcOmBUAOaYNBhwyNzAuODA4N6qrCQw3LjAwppkYwT4IMjYzrg0GDDMuOTmq/xEQMjIuODOqRgkEMjQOXxKmQA0IMzAwDggVqjwREDY1LjYxsh8aCDQuOKrCBxgyNTQuNjQ2phMUISQENzmqsAwOLgsENTWqeAIQOTEuNTiuzREANw55CQAwpp8GEDI2My44rgsRFDI0LjMxNqZuBhIrFgAxqswDGDI4OS4zMzCqXwQANoHyADCmbAMUMzI4LjM2ppsDDDY1LjKBj57+Cg5PFwQ1NaoVBmFrADaqwgccMjk5LjgxMTCqdwYQMy4wODKmZwcEMzIOqBQAMaaCBRgyMjUuMTE2qtkJADEOLw8AMKYwBBgyMzAuMTQyprwPGDMwMi4wMjemiAEOzQ4MODA0M6YaAhA1NC4yNKYkBWEMDsELqmEAFDIuNjE4N6LNBxgyODMuNTM1qqUfFDk5Ljc4NKaaGQ5UEAgyNjSuqw2usgkYNDMuMDY0MZ5rAyHkCDMzOa6sHg6+CwAwpnYGDnseCDk1MqaoAsHYDDgzNjeiLgsYMjg1LjkzMqZ1EQ67IQgyNTmmMQBB2Qg0OTamuhMUMjcxLjkzqpMWBDMyDoohADSmYgChhAg2NjimeAIQMzkxLjiqyw4IOTkuDt8bpoYBDDk0LjamaQMO4hAIOTI3qqQYFDQyLjk1M6oxAAQ4NA7GHaZKIg6YCwwxNzYzpiQXFDkwLjk0OKqqAhQ5Ny4yNzSmvwQANg54GKq/CA4AIQg1MzOm/wMUNTYuNzQ2phcCHDM0NS4wOTEyoswHDoAQCDA5M6rOGRA5MS42N6ZPGyQyNzUuNzExMTIK"; + String snappy_content; + Poco::MemoryInputStream istr(snappy_base64_content.data(), snappy_base64_content.size()); + Poco::Base64Decoder decoder(istr); + Poco::StreamCopier::copyToString(decoder, snappy_content); + auto file_writer = std::make_unique("./test.snappy"); + file_writer->write(snappy_content.c_str(), snappy_content.size()); + file_writer->close(); + std::unique_ptr in = std::make_unique("./test.snappy", 128); + HadoopSnappyReadBuffer read_buffer(std::move(in)); + String output; + WriteBufferFromString out(output); + copyData(read_buffer, out); + UInt128 hashcode = sipHash128(output.c_str(), output.size()); + String hashcode_str = getHexUIntLowercase(hashcode); + ASSERT_EQ(hashcode_str, "593afe14f61866915cc00b8c7bd86046"); +} From 46c4b3a69fa8ca71d824925df961e3f53bafb812 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 9 Mar 2022 11:02:33 +0800 Subject: [PATCH 152/165] retry on exception --- tests/integration/test_hive_query/test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index aa247e2c0af..da1496ed1d7 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -169,7 +169,12 @@ def test_cache_read_bytes(started_cluster): result = node.query(""" SELECT day, count(*) FROM default.demo_parquet_1 group by day order by day settings input_format_parquet_allow_missing_columns = true """) - result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") + try: + result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") + except Exception as e: + logging.info(e) + time.sleep(10) + continue if result.strip() == '0': logging.info("ProfileEvent_ExternalDataSourceLocalCacheReadBytes == 0") time.sleep(10) From bd7e48e87cf717a7decf756c07f7cc8a561e799f Mon Sep 17 00:00:00 2001 From: kashwy Date: Tue, 8 Mar 2022 19:06:21 -0800 Subject: [PATCH 153/165] Upgrade icu to icu-release-70-1 upgrade due to vulnerabilities of WS-2019-0513 and CVE-2020-10531 --- contrib/icu | 2 +- contrib/icu-cmake/CMakeLists.txt | 18 ++++++++++++++---- contrib/icudata | 2 +- docs/en/development/contrib.md | 4 ++-- 4 files changed, 18 insertions(+), 8 deletions(-) diff --git a/contrib/icu b/contrib/icu index faa2f9f9e1f..a56dde820dc 160000 --- a/contrib/icu +++ b/contrib/icu @@ -1 +1 @@ -Subproject commit faa2f9f9e1fe74c5ed00eba371d2830134cdbea1 +Subproject commit a56dde820dc35665a66f2e9ee8ba58e75049b668 diff --git a/contrib/icu-cmake/CMakeLists.txt b/contrib/icu-cmake/CMakeLists.txt index ae19ef20e38..9c34228e2a0 100644 --- a/contrib/icu-cmake/CMakeLists.txt +++ b/contrib/icu-cmake/CMakeLists.txt @@ -212,7 +212,9 @@ set(ICUUC_SOURCES "${ICU_SOURCE_DIR}/common/ubiditransform.cpp" "${ICU_SOURCE_DIR}/common/pluralmap.cpp" "${ICU_SOURCE_DIR}/common/static_unicode_sets.cpp" -"${ICU_SOURCE_DIR}/common/restrace.cpp") +"${ICU_SOURCE_DIR}/common/restrace.cpp" +"${ICU_SOURCE_DIR}/common/emojiprops.cpp" +"${ICU_SOURCE_DIR}/common/lstmbe.cpp") set(ICUI18N_SOURCES "${ICU_SOURCE_DIR}/i18n/ucln_in.cpp" @@ -398,7 +400,6 @@ set(ICUI18N_SOURCES "${ICU_SOURCE_DIR}/i18n/sharedbreakiterator.cpp" "${ICU_SOURCE_DIR}/i18n/scientificnumberformatter.cpp" "${ICU_SOURCE_DIR}/i18n/dayperiodrules.cpp" -"${ICU_SOURCE_DIR}/i18n/nounit.cpp" "${ICU_SOURCE_DIR}/i18n/number_affixutils.cpp" "${ICU_SOURCE_DIR}/i18n/number_compact.cpp" "${ICU_SOURCE_DIR}/i18n/number_decimalquantity.cpp" @@ -446,12 +447,21 @@ set(ICUI18N_SOURCES "${ICU_SOURCE_DIR}/i18n/formattedvalue.cpp" "${ICU_SOURCE_DIR}/i18n/formattedval_iterimpl.cpp" "${ICU_SOURCE_DIR}/i18n/formattedval_sbimpl.cpp" -"${ICU_SOURCE_DIR}/i18n/formatted_string_builder.cpp") +"${ICU_SOURCE_DIR}/i18n/formatted_string_builder.cpp" +"${ICU_SOURCE_DIR}/i18n/measunit_extra.cpp" +"${ICU_SOURCE_DIR}/i18n/number_symbolswrapper.cpp" +"${ICU_SOURCE_DIR}/i18n/number_usageprefs.cpp" +"${ICU_SOURCE_DIR}/i18n/numrange_capi.cpp" +"${ICU_SOURCE_DIR}/i18n/pluralranges.cpp" +"${ICU_SOURCE_DIR}/i18n/units_complexconverter.cpp" +"${ICU_SOURCE_DIR}/i18n/units_converter.cpp" +"${ICU_SOURCE_DIR}/i18n/units_data.cpp" +"${ICU_SOURCE_DIR}/i18n/units_router.cpp") file(GENERATE OUTPUT "${CMAKE_CURRENT_BINARY_DIR}/empty.cpp" CONTENT " ") enable_language(ASM) set(ICUDATA_SOURCES - "${ICUDATA_SOURCE_DIR}/icudt66l_dat.S" + "${ICUDATA_SOURCE_DIR}/icudt70l_dat.S" "${CMAKE_CURRENT_BINARY_DIR}/empty.cpp" # Without this cmake can incorrectly detects library type (OBJECT) instead of SHARED/STATIC ) diff --git a/contrib/icudata b/contrib/icudata index f020820388e..72d9a4a7feb 160000 --- a/contrib/icudata +++ b/contrib/icudata @@ -1 +1 @@ -Subproject commit f020820388e3faafb44cc643574a2d563dfde572 +Subproject commit 72d9a4a7febc904e2b0a534ccb25ae40fac5f1e5 diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index 07969f8ef6a..6c12a3d9055 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -40,8 +40,8 @@ The list of third-party libraries: | grpc | [Apache](https://github.com/ClickHouse-Extras/grpc/blob/60c986e15cae70aade721d26badabab1f822fdd6/LICENSE) | | h3 | [Apache](https://github.com/ClickHouse-Extras/h3/blob/c7f46cfd71fb60e2fefc90e28abe81657deff735/LICENSE) | | hyperscan | [Boost](https://github.com/ClickHouse-Extras/hyperscan/blob/e9f08df0213fc637aac0a5bbde9beeaeba2fe9fa/LICENSE) | -| icu | [Public Domain](https://github.com/unicode-org/icu/blob/faa2f9f9e1fe74c5ed00eba371d2830134cdbea1/icu4c/LICENSE) | -| icudata | [Public Domain](https://github.com/ClickHouse-Extras/icudata/blob/f020820388e3faafb44cc643574a2d563dfde572/LICENSE) | +| icu | [Public Domain](https://github.com/unicode-org/icu/blob/a56dde820dc35665a66f2e9ee8ba58e75049b668/icu4c/LICENSE) | +| icudata | [Public Domain](https://github.com/ClickHouse-Extras/icudata/blob/72d9a4a7febc904e2b0a534ccb25ae40fac5f1e5/LICENSE) | | jemalloc | [BSD 2-clause](https://github.com/ClickHouse-Extras/jemalloc/blob/e6891d9746143bf2cf617493d880ba5a0b9a3efd/COPYING) | | krb5 | [MIT](https://github.com/ClickHouse-Extras/krb5/blob/5149dea4e2be0f67707383d2682b897c14631374/src/lib/gssapi/LICENSE) | | libc-headers | [LGPL](https://github.com/ClickHouse-Extras/libc-headers/blob/a720b7105a610acbd7427eea475a5b6810c151eb/LICENSE) | From 95d8f28aa0a6e8242b72fedc6a54634b9b156e1c Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 9 Mar 2022 15:42:57 +0800 Subject: [PATCH 154/165] update test.py --- tests/integration/test_hive_query/test.py | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index da1496ed1d7..20b6a6cb8f2 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -169,12 +169,8 @@ def test_cache_read_bytes(started_cluster): result = node.query(""" SELECT day, count(*) FROM default.demo_parquet_1 group by day order by day settings input_format_parquet_allow_missing_columns = true """) - try: - result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") - except Exception as e: - logging.info(e) - time.sleep(10) - continue + node.query("system flush logs") + result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") if result.strip() == '0': logging.info("ProfileEvent_ExternalDataSourceLocalCacheReadBytes == 0") time.sleep(10) From a19224bc9b3305a8381c8b758315a7b0ac998da7 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 9 Mar 2022 15:48:42 +0800 Subject: [PATCH 155/165] Fix partition pruner: non-monotonic function IN --- src/Functions/IFunction.h | 4 ++-- src/Interpreters/Set.cpp | 5 +++-- src/Interpreters/Set.h | 2 +- src/Storages/MergeTree/KeyCondition.cpp | 4 ++-- .../02232_partition_pruner_single_point.reference | 2 ++ .../02232_partition_pruner_single_point.sql | 14 ++++++++++++++ 6 files changed, 24 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02232_partition_pruner_single_point.reference create mode 100644 tests/queries/0_stateless/02232_partition_pruner_single_point.sql diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 8063ad77ad0..71af6149774 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -267,7 +267,7 @@ public: */ virtual Monotonicity getMonotonicityForRange(const IDataType & /*type*/, const Field & /*left*/, const Field & /*right*/) const { - throw Exception("Function " + getName() + " has no information about its monotonicity.", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("Function " + getName() + " has no information about its monotonicity", ErrorCodes::NOT_IMPLEMENTED); } }; @@ -452,7 +452,7 @@ public: using Monotonicity = IFunctionBase::Monotonicity; virtual Monotonicity getMonotonicityForRange(const IDataType & /*type*/, const Field & /*left*/, const Field & /*right*/) const { - throw Exception("Function " + getName() + " has no information about its monotonicity.", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("Function " + getName() + " has no information about its monotonicity", ErrorCodes::NOT_IMPLEMENTED); } /// For non-variadic functions, return number of arguments; otherwise return zero (that should be ignored). diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 32dac7f9e9b..7af3e23d0d4 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -445,7 +445,7 @@ MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector & key_ranges, const DataTypes & data_types) const +BoolMask MergeTreeSetIndex::checkInRange(const std::vector & key_ranges, const DataTypes & data_types, bool single_point) const { size_t tuple_size = indexes_mapping.size(); @@ -468,7 +468,8 @@ BoolMask MergeTreeSetIndex::checkInRange(const std::vector & key_ranges, std::optional new_range = KeyCondition::applyMonotonicFunctionsChainToRange( key_ranges[indexes_mapping[i].key_index], indexes_mapping[i].functions, - data_types[indexes_mapping[i].key_index]); + data_types[indexes_mapping[i].key_index], + single_point); if (!new_range) return {true, true}; diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 3146b6af03f..2eecb0211a4 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -214,7 +214,7 @@ public: bool hasMonotonicFunctionsChain() const; - BoolMask checkInRange(const std::vector & key_ranges, const DataTypes & data_types) const; + BoolMask checkInRange(const std::vector & key_ranges, const DataTypes & data_types, bool single_point = false) const; private: // If all arguments in tuple are key columns, we can optimize NOT IN when there is only one element. diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 323b59e2902..c17eb5a981e 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -448,7 +448,7 @@ KeyCondition::KeyCondition( { for (size_t i = 0, size = key_column_names.size(); i < size; ++i) { - std::string name = key_column_names[i]; + const auto & name = key_column_names[i]; if (!key_columns.count(name)) key_columns[name] = i; } @@ -1999,7 +1999,7 @@ BoolMask KeyCondition::checkInHyperrectangle( if (!element.set_index) throw Exception("Set for IN is not created yet", ErrorCodes::LOGICAL_ERROR); - rpn_stack.emplace_back(element.set_index->checkInRange(hyperrectangle, data_types)); + rpn_stack.emplace_back(element.set_index->checkInRange(hyperrectangle, data_types, single_point)); if (element.function == RPNElement::FUNCTION_NOT_IN_SET) rpn_stack.back() = !rpn_stack.back(); } diff --git a/tests/queries/0_stateless/02232_partition_pruner_single_point.reference b/tests/queries/0_stateless/02232_partition_pruner_single_point.reference new file mode 100644 index 00000000000..1191247b6d9 --- /dev/null +++ b/tests/queries/0_stateless/02232_partition_pruner_single_point.reference @@ -0,0 +1,2 @@ +1 +2 diff --git a/tests/queries/0_stateless/02232_partition_pruner_single_point.sql b/tests/queries/0_stateless/02232_partition_pruner_single_point.sql new file mode 100644 index 00000000000..0400d0e1b59 --- /dev/null +++ b/tests/queries/0_stateless/02232_partition_pruner_single_point.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS lower_test; + +CREATE TABLE lower_test ( + a Int32, + b String +) ENGINE=MergeTree +PARTITION BY b +ORDER BY a; + +INSERT INTO lower_test (a,b) VALUES (1,'A'),(2,'B'),(3,'C'); + +SELECT a FROM lower_test WHERE lower(b) IN ('a','b') order by a; + +DROP TABLE lower_test; From d749295222fbfae16ea0bd82c4b16cdbd5b2f95f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 9 Mar 2022 12:35:23 +0100 Subject: [PATCH 156/165] Fix hardcoded page size (#35129) --- src/Common/PODArray.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index b312fbda21c..9b15782a231 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -196,7 +197,7 @@ protected: /// The operation is slow and performed only for debug builds. void protectImpl(int prot) { - static constexpr size_t PROTECT_PAGE_SIZE = 4096; + static size_t PROTECT_PAGE_SIZE = ::getPageSize(); char * left_rounded_up = reinterpret_cast((reinterpret_cast(c_start) - pad_left + PROTECT_PAGE_SIZE - 1) / PROTECT_PAGE_SIZE * PROTECT_PAGE_SIZE); char * right_rounded_down = reinterpret_cast((reinterpret_cast(c_end_of_storage) + pad_right) / PROTECT_PAGE_SIZE * PROTECT_PAGE_SIZE); From af9d8d278e53557e48badaaa78cca88e34900734 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 9 Mar 2022 19:25:43 +0100 Subject: [PATCH 157/165] Fix --- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 5 +++-- .../test_postgresql_replica_database_engine_2/test.py | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index c72dec824f0..669dbdf3eb2 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -382,8 +382,6 @@ ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery( PostgreSQLTableStructurePtr table_structure, const ASTTableOverride * table_override) { auto create_table_query = std::make_shared(); - if (table_override) - applyTableOverrideToCreateQuery(*table_override, create_table_query.get()); auto table_id = getStorageID(); create_table_query->setTable(getNestedTableName()); @@ -502,6 +500,9 @@ ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery( create_table_query->set(create_table_query->storage, storage); + if (table_override) + applyTableOverrideToCreateQuery(*table_override, create_table_query.get()); + /// Add columns _sign and _version, so that they can be accessed from nested ReplacingMergeTree table if needed. ordinary_columns_and_types.push_back({"_sign", std::make_shared()}); ordinary_columns_and_types.push_back({"_version", std::make_shared()}); diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 3226c040e8e..69c9d037cce 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -399,7 +399,7 @@ def test_table_override(started_cluster): create_postgres_table(cursor, table_name, template=postgres_table_template_5); instance.query(f"create table {table_name}(key Int32, value UUID) engine = PostgreSQL (postgres1, table={table_name})") instance.query(f"insert into {table_name} select number, generateUUIDv4() from numbers(10)") - table_overrides = f" TABLE OVERRIDE {table_name} (COLUMNS (key Int32, value UUID))" + table_overrides = f" TABLE OVERRIDE {table_name} (COLUMNS (key Int32, value UUID) PARTITION BY key)" pg_manager.create_materialized_db( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, settings=[f"materialized_postgresql_tables_list = '{table_name}'"], @@ -407,7 +407,7 @@ def test_table_override(started_cluster): assert_nested_table_is_created(instance, table_name, materialized_database) result = instance.query(f"show create table {materialized_database}.{table_name}") print(result) - expected = "CREATE TABLE test_database.table_override\\n(\\n `key` Int32,\\n `value` UUID,\\n `_sign` Int8() MATERIALIZED 1,\\n `_version` UInt64() MATERIALIZED 1\\n)\\nENGINE = ReplacingMergeTree(_version)\\nORDER BY tuple(key)" + expected = "CREATE TABLE test_database.table_override\\n(\\n `key` Int32,\\n `value` UUID,\\n `_sign` Int8() MATERIALIZED 1,\\n `_version` UInt64() MATERIALIZED 1\\n)\\nENGINE = ReplacingMergeTree(_version)\\nPARTITION BY key\\nORDER BY tuple(key)" assert(result.strip() == expected) time.sleep(5) query = f"select * from {materialized_database}.{table_name} order by key" From fbd82865efa5cf7888513b9e1b51e5dde9e3ca70 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Wed, 9 Mar 2022 11:34:30 -0700 Subject: [PATCH 158/165] Crb restructure css (#35033) * Remove greenhouse iframe css * Edit build to include css, js, and images in /docs --- docs/tools/webpack.config.js | 1 - docs/tools/website.py | 46 ++++++++++++++++----------- website/benchmark/dbms/index.html | 2 +- website/benchmark/hardware/index.html | 2 +- website/css/greenhouse.css | 1 - website/js/base.js | 2 +- website/src/scss/greenhouse.scss | 27 ---------------- website/templates/common_css.html | 2 +- website/templates/common_js.html | 2 +- website/templates/common_meta.html | 4 +-- website/templates/docs/amp.html | 2 +- website/templates/docs/nav.html | 12 +++---- website/templates/docs/sidebar.html | 4 +-- 13 files changed, 43 insertions(+), 64 deletions(-) delete mode 100644 website/css/greenhouse.css delete mode 100644 website/src/scss/greenhouse.scss diff --git a/docs/tools/webpack.config.js b/docs/tools/webpack.config.js index fcb3e7bf32d..e0dea964101 100644 --- a/docs/tools/webpack.config.js +++ b/docs/tools/webpack.config.js @@ -14,7 +14,6 @@ module.exports = { entry: [ path.resolve(scssPath, 'bootstrap.scss'), - path.resolve(scssPath, 'greenhouse.scss'), path.resolve(scssPath, 'main.scss'), path.resolve(jsPath, 'main.js'), ], diff --git a/docs/tools/website.py b/docs/tools/website.py index 11772fe7a73..de4cc14670c 100644 --- a/docs/tools/website.py +++ b/docs/tools/website.py @@ -151,6 +151,11 @@ def build_website(args): ) ) + shutil.copytree( + os.path.join(args.website_dir, 'images'), + os.path.join(args.output_dir, 'docs', 'images') + ) + # This file can be requested to check for available ClickHouse releases. shutil.copy2( os.path.join(args.src_dir, 'utils', 'list-versions', 'version_date.tsv'), @@ -231,28 +236,31 @@ def minify_file(path, css_digest, js_digest): def minify_website(args): - # Output greenhouse css separately from main bundle to be included via the greenhouse iframe - command = f"cat '{args.website_dir}/css/greenhouse.css' > '{args.output_dir}/css/greenhouse.css'" - logging.info(command) - output = subprocess.check_output(command, shell=True) - logging.debug(output) - css_in = ' '.join(get_css_in(args)) - css_out = f'{args.output_dir}/css/base.css' - if args.minify: + css_out = f'{args.output_dir}/docs/css/base.css' + os.makedirs(f'{args.output_dir}/docs/css') + + if args.minify and False: # TODO: return closure command = f"purifycss -w '*algolia*' --min {css_in} '{args.output_dir}/*.html' " \ f"'{args.output_dir}/docs/en/**/*.html' '{args.website_dir}/js/**/*.js' > {css_out}" - else: - command = f'cat {css_in} > {css_out}' + logging.info(css_in) + logging.info(command) + output = subprocess.check_output(command, shell=True) + logging.debug(output) + + else: + command = f"cat {css_in}" + output = subprocess.check_output(command, shell=True) + with open(css_out, 'wb+') as f: + f.write(output) - logging.info(command) - output = subprocess.check_output(command, shell=True) - logging.debug(output) with open(css_out, 'rb') as f: css_digest = hashlib.sha3_224(f.read()).hexdigest()[0:8] - js_in = get_js_in(args) - js_out = f'{args.output_dir}/js/base.js' + js_in = ' '.join(get_js_in(args)) + js_out = f'{args.output_dir}/docs/js/base.js' + os.makedirs(f'{args.output_dir}/docs/js') + if args.minify and False: # TODO: return closure js_in = [js[1:-1] for js in js_in] closure_args = [ @@ -271,11 +279,11 @@ def minify_website(args): f.write(js_content) else: - js_in = ' '.join(js_in) - command = f'cat {js_in} > {js_out}' - logging.info(command) + command = f"cat {js_in}" output = subprocess.check_output(command, shell=True) - logging.debug(output) + with open(js_out, 'wb+') as f: + f.write(output) + with open(js_out, 'rb') as f: js_digest = hashlib.sha3_224(f.read()).hexdigest()[0:8] logging.info(js_digest) diff --git a/website/benchmark/dbms/index.html b/website/benchmark/dbms/index.html index b4e29098ead..a856bbb0502 100644 --- a/website/benchmark/dbms/index.html +++ b/website/benchmark/dbms/index.html @@ -15,7 +15,7 @@
- ClickHouse + ClickHouse

Performance comparison of analytical DBMS

diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index 06878eb077c..42c87c334c0 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -15,7 +15,7 @@
- ClickHouse + ClickHouse

{{ title }}

diff --git a/website/css/greenhouse.css b/website/css/greenhouse.css deleted file mode 100644 index 76812a169e8..00000000000 --- a/website/css/greenhouse.css +++ /dev/null @@ -1 +0,0 @@ -#main{padding-bottom:0;padding-top:0}#wrapper{max-width:1078px;padding:0}body>#wrapper>#main>#wrapper>#content,body>#wrapper>#main>#wrapper>#logo,body>#wrapper>#main>#wrapper>h1{display:none}body>#wrapper>#main>#wrapper>#board_title{margin-top:0}body>#wrapper>#main>#logo{margin-top:80px}body>#wrapper>#main>:last-child{margin-bottom:120px} \ No newline at end of file diff --git a/website/js/base.js b/website/js/base.js index 52b801eb98f..d953d5f6a1f 100644 --- a/website/js/base.js +++ b/website/js/base.js @@ -55,7 +55,7 @@ $('pre').each(function(_, element) { $(element).prepend( - 'Copy' + 'Copy' ); }); diff --git a/website/src/scss/greenhouse.scss b/website/src/scss/greenhouse.scss deleted file mode 100644 index 710b606fa15..00000000000 --- a/website/src/scss/greenhouse.scss +++ /dev/null @@ -1,27 +0,0 @@ -#main { - padding-bottom: 0; - padding-top: 0; -} - -#wrapper { - max-width: 1078px; - padding: 0; -} - -body > #wrapper > #main > #wrapper > #logo, -body > #wrapper > #main > #wrapper > h1, -body > #wrapper > #main > #wrapper > #content { - display: none; -} - -body > #wrapper > #main > #wrapper > #board_title { - margin-top: 0; -} - -body > #wrapper > #main > #logo { - margin-top: 80px; -} - -body > #wrapper > #main > :last-child { - margin-bottom: 120px; -} diff --git a/website/templates/common_css.html b/website/templates/common_css.html index ac10b233f25..b26b2bf973e 100644 --- a/website/templates/common_css.html +++ b/website/templates/common_css.html @@ -1,4 +1,4 @@ - + {% for src in extra_css %} diff --git a/website/templates/common_js.html b/website/templates/common_js.html index 72421f00562..93e35d37918 100644 --- a/website/templates/common_js.html +++ b/website/templates/common_js.html @@ -1,4 +1,4 @@ - + {% for src in extra_js %} diff --git a/website/templates/common_meta.html b/website/templates/common_meta.html index 018d533e893..07aa05d28b1 100644 --- a/website/templates/common_meta.html +++ b/website/templates/common_meta.html @@ -7,7 +7,7 @@ {% if title %}{{ title }}{% else %}{{ _('ClickHouse - fast open-source OLAP DBMS') }}{% endif %} - + @@ -15,7 +15,7 @@ {% if page and page.meta.image %} {% else %} - + {% endif %} diff --git a/website/templates/docs/amp.html b/website/templates/docs/amp.html index 5d2777af188..dc7dd7acb49 100644 --- a/website/templates/docs/amp.html +++ b/website/templates/docs/amp.html @@ -20,7 +20,7 @@ diff --git a/website/templates/docs/nav.html b/website/templates/docs/nav.html index 4d57d282796..afac39c2fab 100644 --- a/website/templates/docs/nav.html +++ b/website/templates/docs/nav.html @@ -1,7 +1,7 @@