From 5c847264f706954d39b188bf570275e4a01fd811 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Tue, 11 Jan 2022 20:19:41 +0800 Subject: [PATCH 01/18] Fix bug for select from VIEW with format and limit settings --- src/Interpreters/InterpreterSelectQuery.cpp | 1 + src/Interpreters/InterpreterSelectWithUnionQuery.cpp | 6 +++--- src/Interpreters/SelectQueryOptions.h | 7 +++++-- src/Storages/SelectQueryInfo.h | 1 + src/Storages/StorageView.cpp | 3 ++- .../02169_fix_view_offset_limit_setting.reference | 12 ++++++++++++ .../02169_fix_view_offset_limit_setting.sql | 12 ++++++++++++ 7 files changed, 36 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02169_fix_view_offset_limit_setting.reference create mode 100644 tests/queries/0_stateless/02169_fix_view_offset_limit_setting.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 5c0322ac1d9..a47fc216dad 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1977,6 +1977,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete)) quota = context->getQuota(); + query_info.settings_limit_offset_done = options.settings_limit_offset_done; storage->read(query_plan, required_columns, metadata_snapshot, query_info, context, processing_stage, max_block_size, max_streams); if (context->hasQueryContext() && !options.is_internal) diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 6779093a53d..e4b3e62c358 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -83,7 +83,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( } } - if (num_children == 1 && settings_limit_offset_needed) + if (num_children == 1 && settings_limit_offset_needed && !options.settings_limit_offset_done) { const ASTPtr first_select_ast = ast->list_of_selects->children.at(0); ASTSelectQuery * select_query = dynamic_cast(first_select_ast.get()); @@ -127,7 +127,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( select_query->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, std::move(new_limit_length_ast)); } - settings_limit_offset_done = true; + options.settings_limit_offset_done = true; } } @@ -305,7 +305,7 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan) } } - if (settings_limit_offset_needed && !settings_limit_offset_done) + if (settings_limit_offset_needed && !options.settings_limit_offset_done) { if (settings.limit > 0) { diff --git a/src/Interpreters/SelectQueryOptions.h b/src/Interpreters/SelectQueryOptions.h index bc95a940c18..ee708b064bd 100644 --- a/src/Interpreters/SelectQueryOptions.h +++ b/src/Interpreters/SelectQueryOptions.h @@ -48,6 +48,7 @@ struct SelectQueryOptions bool is_internal = false; bool is_subquery = false; // non-subquery can also have subquery_depth > 0, e.g. insert select bool with_all_cols = false; /// asterisk include materialized and aliased columns + bool settings_limit_offset_done = false; /// These two fields are used to evaluate shardNum() and shardCount() function when /// prefer_localhost_replica == 1 and local instance is selected. They are needed because local @@ -58,8 +59,10 @@ struct SelectQueryOptions SelectQueryOptions( QueryProcessingStage::Enum stage = QueryProcessingStage::Complete, size_t depth = 0, - bool is_subquery_ = false) - : to_stage(stage), subquery_depth(depth), is_subquery(is_subquery_) + bool is_subquery_ = false, + bool settings_limit_offset_done_ = false) + : to_stage(stage), subquery_depth(depth), is_subquery(is_subquery_), + settings_limit_offset_done(settings_limit_offset_done_) {} SelectQueryOptions copy() const { return *this; } diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index e53f5adec52..5df50ab9a7c 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -169,6 +169,7 @@ struct SelectQueryInfo bool ignore_projections = false; bool is_projection_query = false; bool merge_tree_empty_result = false; + bool settings_limit_offset_done = false; Block minmax_count_projection_block; MergeTreeDataSelectAnalysisResultPtr merge_tree_select_result_ptr; }; diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index dcf664db6fe..bcf7d7856cf 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -140,7 +140,8 @@ void StorageView::read( current_inner_query = query_info.view_query->clone(); } - InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, {}, column_names); + auto options = SelectQueryOptions(QueryProcessingStage::Complete, 0, false, query_info.settings_limit_offset_done); + InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, options, column_names); interpreter.buildQueryPlan(query_plan); /// It's expected that the columns read from storage are not constant. diff --git a/tests/queries/0_stateless/02169_fix_view_offset_limit_setting.reference b/tests/queries/0_stateless/02169_fix_view_offset_limit_setting.reference new file mode 100644 index 00000000000..32c54e3eeea --- /dev/null +++ b/tests/queries/0_stateless/02169_fix_view_offset_limit_setting.reference @@ -0,0 +1,12 @@ +5 +6 +7 +8 +9 +10 +0 +1 +2 +3 +4 +5 diff --git a/tests/queries/0_stateless/02169_fix_view_offset_limit_setting.sql b/tests/queries/0_stateless/02169_fix_view_offset_limit_setting.sql new file mode 100644 index 00000000000..8ac88ebc5c0 --- /dev/null +++ b/tests/queries/0_stateless/02169_fix_view_offset_limit_setting.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS counter; +CREATE TABLE counter (id UInt64, createdAt DateTime) ENGINE = MergeTree() ORDER BY id; +INSERT INTO counter SELECT number, now() FROM numbers(500); + +DROP TABLE IF EXISTS vcounter; +CREATE VIEW vcounter AS SELECT intDiv(id, 10) AS tens, max(createdAt) AS maxid FROM counter GROUP BY tens; + +SELECT tens FROM vcounter ORDER BY tens ASC LIMIT 100 SETTINGS limit = 6, offset = 5; + +SELECT tens FROM vcounter ORDER BY tens ASC LIMIT 100 SETTINGS limit = 6, offset = 0; +DROP TABLE vcounter; +DROP TABLE counter; From 457a38a1ade45212bb5ca07e1186d068e62cfe31 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 11 Jan 2022 18:45:41 +0300 Subject: [PATCH 02/18] HashTable constant fix --- src/Common/CombinedCardinalityEstimator.h | 2 +- src/Common/HashTable/StringHashTable.h | 2 +- src/Common/HashTable/TwoLevelStringHashTable.h | 2 +- src/Common/Throttler.cpp | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/CombinedCardinalityEstimator.h b/src/Common/CombinedCardinalityEstimator.h index 55afb028247..8cf35436840 100644 --- a/src/Common/CombinedCardinalityEstimator.h +++ b/src/Common/CombinedCardinalityEstimator.h @@ -323,7 +323,7 @@ private: UInt64 address = 0; }; static const UInt64 mask = 0xFFFFFFFFFFFFFFFC; - static const UInt32 medium_set_size_max = 1UL << medium_set_power2_max; + static const UInt32 medium_set_size_max = 1ULL << medium_set_power2_max; }; } diff --git a/src/Common/HashTable/StringHashTable.h b/src/Common/HashTable/StringHashTable.h index d30271d65db..7d704e4bdc7 100644 --- a/src/Common/HashTable/StringHashTable.h +++ b/src/Common/HashTable/StringHashTable.h @@ -280,7 +280,7 @@ public: if ((reinterpret_cast(p) & 2048) == 0) { memcpy(&n[0], p, 8); - n[0] &= -1ul >> s; + n[0] &= -1ULL >> s; } else { diff --git a/src/Common/HashTable/TwoLevelStringHashTable.h b/src/Common/HashTable/TwoLevelStringHashTable.h index 93bbcb2835d..871becc86a4 100644 --- a/src/Common/HashTable/TwoLevelStringHashTable.h +++ b/src/Common/HashTable/TwoLevelStringHashTable.h @@ -114,7 +114,7 @@ public: if ((reinterpret_cast(p) & 2048) == 0) { memcpy(&n[0], p, 8); - n[0] &= -1ul >> s; + n[0] &= -1ULL >> s; } else { diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index f02001e338a..95baf40f2c0 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -23,7 +23,7 @@ static constexpr auto NS = 1000000000UL; /// Tracking window. Actually the size is not really important. We just want to avoid /// throttles when there are no actions for a long period time. -static const double window_ns = 1UL * NS; +static const double window_ns = 1ULL * NS; void Throttler::add(size_t amount) { From 80a452cca0d439b8a68e5753f9e55ce45dfffeef Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 11 Jan 2022 12:23:09 +0100 Subject: [PATCH 03/18] Apply black on build_check.py and tee_popen.py --- tests/ci/build_check.py | 179 +++++++++++++++++++++++++++------------- tests/ci/tee_popen.py | 14 +++- 2 files changed, 131 insertions(+), 62 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index f37ea49e387..a9ec27d25a0 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -20,8 +20,8 @@ from tee_popen import TeePopen def get_build_config(build_check_name, build_name): - if build_check_name == 'ClickHouse build check (actions)': - build_config_name = 'build_config' + if build_check_name == "ClickHouse build check (actions)": + build_config_name = "build_config" else: raise Exception(f"Unknown build check name {build_check_name}") @@ -29,57 +29,69 @@ def get_build_config(build_check_name, build_name): def _can_export_binaries(build_config): - if build_config['package_type'] != 'deb': + if build_config["package_type"] != "deb": return False - if build_config['bundled'] != "bundled": + if build_config["bundled"] != "bundled": return False - if build_config['splitted'] == 'splitted': + if build_config["splitted"] == "splitted": return False - if build_config['sanitizer'] != '': + if build_config["sanitizer"] != "": return True - if build_config['build_type'] != '': + if build_config["build_type"] != "": return True return False -def get_packager_cmd(build_config, packager_path, output_path, build_version, image_version, ccache_path, pr_info): - package_type = build_config['package_type'] - comp = build_config['compiler'] - cmd = f"cd {packager_path} && ./packager --output-dir={output_path} --package-type={package_type} --compiler={comp}" +def get_packager_cmd( + build_config, + packager_path, + output_path, + build_version, + image_version, + ccache_path, + pr_info, +): + package_type = build_config["package_type"] + comp = build_config["compiler"] + cmd = ( + f"cd {packager_path} && ./packager --output-dir={output_path} " + f"--package-type={package_type} --compiler={comp}" + ) - if build_config['build_type']: - cmd += ' --build-type={}'.format(build_config['build_type']) - if build_config['sanitizer']: - cmd += ' --sanitizer={}'.format(build_config['sanitizer']) - if build_config['splitted'] == 'splitted': - cmd += ' --split-binary' - if build_config['tidy'] == 'enable': - cmd += ' --clang-tidy' + if build_config["build_type"]: + cmd += " --build-type={}".format(build_config["build_type"]) + if build_config["sanitizer"]: + cmd += " --sanitizer={}".format(build_config["sanitizer"]) + if build_config["splitted"] == "splitted": + cmd += " --split-binary" + if build_config["tidy"] == "enable": + cmd += " --clang-tidy" - cmd += ' --cache=ccache' - cmd += ' --ccache_dir={}'.format(ccache_path) + cmd += " --cache=ccache" + cmd += " --ccache_dir={}".format(ccache_path) - if 'alien_pkgs' in build_config and build_config['alien_pkgs']: - if pr_info.number == 0 or 'release' in pr_info.labels: - cmd += ' --alien-pkgs rpm tgz' + if "alien_pkgs" in build_config and build_config["alien_pkgs"]: + if pr_info.number == 0 or "release" in pr_info.labels: + cmd += " --alien-pkgs rpm tgz" - cmd += ' --docker-image-version={}'.format(image_version) - cmd += ' --version={}'.format(build_version) + cmd += " --docker-image-version={}".format(image_version) + cmd += " --version={}".format(build_version) if _can_export_binaries(build_config): - cmd += ' --with-binaries=tests' + cmd += " --with-binaries=tests" return cmd + def get_image_name(build_config): - if build_config['package_type'] != 'deb': - return 'clickhouse/binary-builder' + if build_config["package_type"] != "deb": + return "clickhouse/binary-builder" else: - return 'clickhouse/deb-builder' + return "clickhouse/deb-builder" def build_clickhouse(packager_cmd, logs_path, build_output_path): - build_log_path = os.path.join(logs_path, 'build_log.log') + build_log_path = os.path.join(logs_path, "build_log.log") with TeePopen(packager_cmd, build_log_path) as process: retcode = process.wait() if os.path.exists(build_output_path): @@ -91,7 +103,9 @@ def build_clickhouse(packager_cmd, logs_path, build_output_path): if len(build_results) != 0: logging.info("Built successfully") else: - logging.info("Success exit code, but no build artifacts => build failed") + logging.info( + "Success exit code, but no build artifacts => build failed" + ) else: logging.info("Build failed") return build_log_path, retcode == 0 and len(build_results) > 0 @@ -105,8 +119,13 @@ def get_build_results_if_exists(s3_helper, s3_prefix): logging.info("Got exception %s listing %s", ex, s3_prefix) return None -def create_json_artifact(temp_path, build_name, log_url, build_urls, build_config, elapsed, success): - subprocess.check_call(f"echo 'BUILD_NAME=build_urls_{build_name}' >> $GITHUB_ENV", shell=True) + +def create_json_artifact( + temp_path, build_name, log_url, build_urls, build_config, elapsed, success +): + subprocess.check_call( + f"echo 'BUILD_NAME=build_urls_{build_name}' >> $GITHUB_ENV", shell=True + ) result = { "log_url": log_url, @@ -116,11 +135,18 @@ def create_json_artifact(temp_path, build_name, log_url, build_urls, build_confi "status": success, } - json_name = "build_urls_" + build_name + '.json' + json_name = "build_urls_" + build_name + ".json" - print ("Dump json report", result, "to", json_name, "with env", "build_urls_{build_name}") + print( + "Dump json report", + result, + "to", + json_name, + "with env", + "build_urls_{build_name}", + ) - with open(os.path.join(temp_path, json_name), 'w') as build_links: + with open(os.path.join(temp_path, json_name), "w") as build_links: json.dump(result, build_links) @@ -143,14 +169,14 @@ if __name__ == "__main__": logging.info("Repo copy path %s", repo_path) gh = Github(get_best_robot_token()) - s3_helper = S3Helper('https://s3.amazonaws.com') + s3_helper = S3Helper("https://s3.amazonaws.com") version = get_version_from_repo(repo_path) release_or_pr = None - if 'release' in pr_info.labels or 'release-lts' in pr_info.labels: + if "release" in pr_info.labels or "release-lts" in pr_info.labels: # for release pull requests we use branch names prefixes, not pr numbers release_or_pr = pr_info.head_ref - elif pr_info.number == 0 and build_config['package_type'] != "performance": + elif pr_info.number == 0 and build_config["package_type"] != "performance": # for pushes to master - major version, but not for performance builds # they havily relies on a fixed path for build package and nobody going # to deploy them somewhere, so it's ok. @@ -167,13 +193,26 @@ if __name__ == "__main__": if build_results is not None and len(build_results) > 0: logging.info("Some build results found %s", build_results) build_urls = [] - log_url = '' + log_url = "" for url in build_results: - if 'build_log.log' in url: - log_url = 'https://s3.amazonaws.com/clickhouse-builds/' + url.replace('+', '%2B').replace(' ', '%20') + if "build_log.log" in url: + log_url = "https://s3.amazonaws.com/clickhouse-builds/" + url.replace( + "+", "%2B" + ).replace(" ", "%20") else: - build_urls.append('https://s3.amazonaws.com/clickhouse-builds/' + url.replace('+', '%2B').replace(' ', '%20')) - create_json_artifact(temp_path, build_name, log_url, build_urls, build_config, 0, len(build_urls) > 0) + build_urls.append( + "https://s3.amazonaws.com/clickhouse-builds/" + + url.replace("+", "%2B").replace(" ", "%20") + ) + create_json_artifact( + temp_path, + build_name, + log_url, + build_urls, + build_config, + 0, + len(build_urls) > 0, + ) sys.exit(0) image_name = get_image_name(build_config) @@ -182,9 +221,9 @@ if __name__ == "__main__": logging.info("Got version from repo %s", version.get_version_string()) - version_type = 'testing' - if 'release' in pr_info.labels or 'release-lts' in pr_info.labels: - version_type = 'stable' + version_type = "testing" + if "release" in pr_info.labels or "release-lts" in pr_info.labels: + version_type = "stable" update_version_local(repo_path, pr_info.sha, version, version_type) @@ -196,7 +235,7 @@ if __name__ == "__main__": if not os.path.exists(build_output_path): os.makedirs(build_output_path) - ccache_path = os.path.join(caches_path, build_name + '_ccache') + ccache_path = os.path.join(caches_path, build_name + "_ccache") logging.info("Will try to fetch cache for our build") get_ccache_if_not_exists(ccache_path, s3_helper, pr_info.number, temp_path) @@ -205,11 +244,21 @@ if __name__ == "__main__": logging.info("cache was not fetched, will create empty dir") os.makedirs(ccache_path) - if build_config['package_type'] == "performance" and pr_info.number != 0: + if build_config["package_type"] == "performance" and pr_info.number != 0: # because perf tests store some information about git commits - subprocess.check_call(f"cd {repo_path} && git fetch origin master:master", shell=True) + subprocess.check_call( + f"cd {repo_path} && git fetch origin master:master", shell=True + ) - packager_cmd = get_packager_cmd(build_config, os.path.join(repo_path, "docker/packager"), build_output_path, version.get_version_string(), image_version, ccache_path, pr_info) + packager_cmd = get_packager_cmd( + build_config, + os.path.join(repo_path, "docker/packager"), + build_output_path, + version.get_version_string(), + image_version, + ccache_path, + pr_info, + ) logging.info("Going to run packager with %s", packager_cmd) build_clickhouse_log = os.path.join(temp_path, "build_log") @@ -217,30 +266,42 @@ if __name__ == "__main__": os.makedirs(build_clickhouse_log) start = time.time() - log_path, success = build_clickhouse(packager_cmd, build_clickhouse_log, build_output_path) + log_path, success = build_clickhouse( + packager_cmd, build_clickhouse_log, build_output_path + ) elapsed = int(time.time() - start) - subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {build_output_path}", shell=True) + subprocess.check_call( + f"sudo chown -R ubuntu:ubuntu {build_output_path}", shell=True + ) subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {ccache_path}", shell=True) logging.info("Build finished with %s, log path %s", success, log_path) - logging.info("Will upload cache") upload_ccache(ccache_path, s3_helper, pr_info.number, temp_path) if os.path.exists(log_path): - log_url = s3_helper.upload_build_file_to_s3(log_path, s3_path_prefix + "/" + os.path.basename(log_path)) + log_url = s3_helper.upload_build_file_to_s3( + log_path, s3_path_prefix + "/" + os.path.basename(log_path) + ) logging.info("Log url %s", log_url) else: logging.info("Build log doesn't exist") - build_urls = s3_helper.upload_build_folder_to_s3(build_output_path, s3_path_prefix, keep_dirs_in_s3_path=False, upload_symlinks=False) + build_urls = s3_helper.upload_build_folder_to_s3( + build_output_path, + s3_path_prefix, + keep_dirs_in_s3_path=False, + upload_symlinks=False, + ) logging.info("Got build URLs %s", build_urls) - print("::notice ::Build URLs: {}".format('\n'.join(build_urls))) + print("::notice ::Build URLs: {}".format("\n".join(build_urls))) print("::notice ::Log URL: {}".format(log_url)) - create_json_artifact(temp_path, build_name, log_url, build_urls, build_config, elapsed, success) + create_json_artifact( + temp_path, build_name, log_url, build_urls, build_config, elapsed, success + ) # Fail build job if not successeded if not success: sys.exit(1) diff --git a/tests/ci/tee_popen.py b/tests/ci/tee_popen.py index cbb915e6de7..20302dacb97 100644 --- a/tests/ci/tee_popen.py +++ b/tests/ci/tee_popen.py @@ -15,11 +15,19 @@ class TeePopen: self.command = command self.log_file = log_file self.env = env + self.process = None def __enter__(self): - # pylint: disable=W0201 - self.process = Popen(self.command, shell=True, universal_newlines=True, env=self.env, stderr=STDOUT, stdout=PIPE, bufsize=1) - self.log_file = open(self.log_file, 'w', encoding='utf-8') + self.process = Popen( + self.command, + shell=True, + universal_newlines=True, + env=self.env, + stderr=STDOUT, + stdout=PIPE, + bufsize=1, + ) + self.log_file = open(self.log_file, "w", encoding="utf-8") return self def __exit__(self, t, value, traceback): From 211f1dd4825599b19f86a17ced7a8fef547d35ad Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 12 Jan 2022 00:27:42 +0300 Subject: [PATCH 04/18] Fix --- src/Client/LocalConnection.cpp | 6 ++++++ ...ckhouse_local_interactive_exception.expect | 21 +++++++++++++++++++ ...ouse_local_interactive_exception.reference | 0 3 files changed, 27 insertions(+) create mode 100755 tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect create mode 100644 tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.reference diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 8ad853950b2..528c38f9b76 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -214,6 +214,12 @@ bool LocalConnection::poll(size_t) if (next_packet_type) return true; + if (state->exception) + { + next_packet_type = Protocol::Server::Exception; + return true; + } + if (!state->is_finished) { if (send_progress && (state->after_send_progress.elapsedMicroseconds() >= query_context->getSettingsRef().interactive_delay)) diff --git a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect new file mode 100755 index 00000000000..4f006b926bd --- /dev/null +++ b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect @@ -0,0 +1,21 @@ +#!/usr/bin/expect -f + +log_user 0 +set timeout 20 +match_max 100000 + +expect_after { + eof { exp_continue } + timeout { exit 1 } +} + +set basedir [file dirname $argv0] +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion" + +expect ":) " +send -- "insert into table function null() format TSV some trash here 123 \n 456\r" +expect -re ".*DB::Exception: Table function 'null' requires 'structure'.*\r" +expect ":) " + +send -- "" +expect eof diff --git a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.reference b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.reference new file mode 100644 index 00000000000..e69de29bb2d From 7ea75b308249472a78baaa49bf1958a0fda753a8 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 12 Jan 2022 12:59:40 +0300 Subject: [PATCH 05/18] Fix parsing queries with FROM INFILE statement again --- src/Parsers/ASTInsertQuery.cpp | 13 ++++---- src/Parsers/ParserInsertQuery.cpp | 32 +++++++------------ .../02165_insert_from_infile.reference | 5 +++ .../0_stateless/02165_insert_from_infile.sql | 4 +-- 4 files changed, 25 insertions(+), 29 deletions(-) diff --git a/src/Parsers/ASTInsertQuery.cpp b/src/Parsers/ASTInsertQuery.cpp index c733398a32b..7e1d48d7f55 100644 --- a/src/Parsers/ASTInsertQuery.cpp +++ b/src/Parsers/ASTInsertQuery.cpp @@ -79,6 +79,13 @@ void ASTInsertQuery::formatImpl(const FormatSettings & settings, FormatState & s settings.ostr << ")"; } + if (infile) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM INFILE " << (settings.hilite ? hilite_none : "") << infile->as().value.safeGet(); + if (compression) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " COMPRESSION " << (settings.hilite ? hilite_none : "") << compression->as().value.safeGet(); + } + if (select) { settings.ostr << " "; @@ -91,12 +98,6 @@ void ASTInsertQuery::formatImpl(const FormatSettings & settings, FormatState & s } else { - if (infile) - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM INFILE " << (settings.hilite ? hilite_none : "") << infile->as().value.safeGet(); - if (compression) - settings.ostr << (settings.hilite ? hilite_keyword : "") << " COMPRESSION " << (settings.hilite ? hilite_none : "") << compression->as().value.safeGet(); - } if (!format.empty()) { settings.ostr << (settings.hilite ? hilite_keyword : "") << " FORMAT " << (settings.hilite ? hilite_none : "") << format; diff --git a/src/Parsers/ParserInsertQuery.cpp b/src/Parsers/ParserInsertQuery.cpp index fc606b15e9f..568f486a5cf 100644 --- a/src/Parsers/ParserInsertQuery.cpp +++ b/src/Parsers/ParserInsertQuery.cpp @@ -113,10 +113,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - Pos before_values = pos; - String format_str; - - /// VALUES or FROM INFILE or FORMAT or SELECT + /// Check if file is a source of data. if (s_from_infile.ignore(pos, expected)) { /// Read file name to process it later @@ -131,17 +128,14 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!compression_p.parse(pos, compression, expected)) return false; } - - /// Check if we have FORMAT statement - if (s_format.ignore(pos, expected)) - { - if (!name_p.parse(pos, format, expected)) - return false; - - tryGetIdentifierNameInto(format, format_str); - } } - else if (s_values.ignore(pos, expected)) + + Pos before_values = pos; + String format_str; + + /// VALUES or FORMAT or SELECT or WITH or WATCH. + /// After FROM INFILE we expect FORMAT, SELECT, WITH or nothing. + if (!infile && s_values.ignore(pos, expected)) { /// If VALUES is defined in query, everything except setting will be parsed as data data = pos->begin; @@ -169,21 +163,17 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) tryGetIdentifierNameInto(format, format_str); } - else if (s_watch.ignore(pos, expected)) + else if (!infile && s_watch.ignore(pos, expected)) { /// If WATCH is defined, return to position before WATCH and parse /// rest of query as WATCH query. pos = before_values; ParserWatchQuery watch_p; watch_p.parse(pos, watch, expected); - - /// FORMAT section is expected if we have input() in SELECT part - if (s_format.ignore(pos, expected) && !name_p.parse(pos, format, expected)) - return false; } - else + else if (!infile) { - /// If all previous conditions were false, query is incorrect + /// If all previous conditions were false and it's not FROM INFILE, query is incorrect return false; } diff --git a/tests/queries/0_stateless/02165_insert_from_infile.reference b/tests/queries/0_stateless/02165_insert_from_infile.reference index e69de29bb2d..2a00a8faa31 100644 --- a/tests/queries/0_stateless/02165_insert_from_infile.reference +++ b/tests/queries/0_stateless/02165_insert_from_infile.reference @@ -0,0 +1,5 @@ +INSERT INTO test FROM INFILE data.file SELECT x +FROM input(\'x UInt32\') +INSERT INTO test FROM INFILE data.file WITH number AS x +SELECT number +FROM input(\'number UInt32\') diff --git a/tests/queries/0_stateless/02165_insert_from_infile.sql b/tests/queries/0_stateless/02165_insert_from_infile.sql index 1979c618c06..8cc851fa4e5 100644 --- a/tests/queries/0_stateless/02165_insert_from_infile.sql +++ b/tests/queries/0_stateless/02165_insert_from_infile.sql @@ -1,4 +1,4 @@ -EXPLAIN SYNTAX INSERT INTO test FROM INFILE 'data.file' SELECT 1; -- { clientError SYNTAX_ERROR } +EXPLAIN SYNTAX INSERT INTO test FROM INFILE 'data.file' SELECT x from input('x UInt32') FORMAT TSV; EXPLAIN SYNTAX INSERT INTO test FROM INFILE 'data.file' WATCH view; -- { clientError SYNTAX_ERROR } EXPLAIN SYNTAX INSERT INTO test FROM INFILE 'data.file' VALUES (1) -- { clientError SYNTAX_ERROR } -EXPLAIN SYNTAX INSERT INTO test FROM INFILE 'data.file' WITH number AS x SELECT number FROM numbers(10); -- { clientError SYNTAX_ERROR } +EXPLAIN SYNTAX INSERT INTO test FROM INFILE 'data.file' WITH number AS x SELECT number FROM input('number UInt32'); From b98e0be8690544736c91f3f4108873f9d3f48d6d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 12 Jan 2022 12:29:37 +0100 Subject: [PATCH 06/18] Use a proper branch name for PRInfo w/ number==0 --- tests/ci/pr_info.py | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 48464439dbc..8de09547b0e 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -41,6 +41,9 @@ class PRInfo: github_event = {'commits': 1, 'after': 'HEAD', 'ref': None} self.event = github_event self.changed_files = set([]) + ref = github_event.get("ref", "refs/head/master") + if ref.startswith('refs/heads/'): + ref = ref[11:] # workflow completed event, used for PRs only if 'action' in github_event and github_event['action'] == 'completed': @@ -93,10 +96,10 @@ class PRInfo: if pull_request is None or pull_request['state'] == 'closed': # it's merged PR to master self.number = 0 self.labels = {} - self.pr_html_url = f"{repo_prefix}/commits/master" - self.base_ref = "master" + self.pr_html_url = f"{repo_prefix}/commits/{ref}" + self.base_ref = ref self.base_name = self.repo_full_name - self.head_ref = "master" + self.head_ref = ref self.head_name = self.repo_full_name self.diff_url = \ f"https://api.github.com/repos/{GITHUB_REPOSITORY}/compare/{github_event['before']}...{self.sha}" @@ -126,10 +129,10 @@ class PRInfo: self.task_url = f"{repo_prefix}/actions/runs/{GITHUB_RUN_ID or '0'}" self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" self.repo_full_name = GITHUB_REPOSITORY - self.pr_html_url = f"{repo_prefix}/commits/master" - self.base_ref = "master" + self.pr_html_url = f"{repo_prefix}/commits/{ref}" + self.base_ref = ref self.base_name = self.repo_full_name - self.head_ref = "master" + self.head_ref = ref self.head_name = self.repo_full_name if need_changed_files: From eaf3bdabf50b3afd90aec4195f125289674ecfe6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 12 Jan 2022 13:14:31 +0100 Subject: [PATCH 07/18] Add typing to CI_CONFIG --- tests/ci/ci_config.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index d5f8757ffdf..ba31e8e803c 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -1,5 +1,10 @@ #!/usr/bin/env python3 +from typing import Dict, TypeVar + +ConfValue = TypeVar("ConfValue", str, bool) +BuildConfig = Dict[str, ConfValue] + CI_CONFIG = { "build_config": { "package_release": { @@ -334,4 +339,4 @@ CI_CONFIG = { "required_build": "performance", }, }, -} +} # type: dict From 3614b0c4ed71083423db6acfd9b9587bd3314308 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 12 Jan 2022 13:17:21 +0100 Subject: [PATCH 08/18] Slightly refactor build_check.py --- tests/ci/build_check.py | 123 +++++++++++++++++++++++----------------- 1 file changed, 71 insertions(+), 52 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index a9ec27d25a0..714686b67b9 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -6,20 +6,23 @@ import json import os import sys import time -from github import Github +from typing import List, Optional, Tuple from env_helper import REPO_COPY, TEMP_PATH, CACHES_PATH, IMAGES_PATH from s3_helper import S3Helper from pr_info import PRInfo -from get_robot_token import get_best_robot_token -from version_helper import get_version_from_repo, update_version_local +from version_helper import ( + ClickHouseVersion, + get_version_from_repo, + update_version_local, +) from ccache_utils import get_ccache_if_not_exists, upload_ccache -from ci_config import CI_CONFIG +from ci_config import CI_CONFIG, BuildConfig from docker_pull_helper import get_image_with_version from tee_popen import TeePopen -def get_build_config(build_check_name, build_name): +def get_build_config(build_check_name: str, build_name: str) -> BuildConfig: if build_check_name == "ClickHouse build check (actions)": build_config_name = "build_config" else: @@ -28,7 +31,7 @@ def get_build_config(build_check_name, build_name): return CI_CONFIG[build_config_name][build_name] -def _can_export_binaries(build_config): +def _can_export_binaries(build_config: BuildConfig) -> bool: if build_config["package_type"] != "deb": return False if build_config["bundled"] != "bundled": @@ -43,14 +46,14 @@ def _can_export_binaries(build_config): def get_packager_cmd( - build_config, - packager_path, - output_path, - build_version, - image_version, - ccache_path, - pr_info, -): + build_config: BuildConfig, + packager_path: str, + output_path: str, + build_version: str, + image_version: str, + ccache_path: str, + pr_info: PRInfo, +) -> str: package_type = build_config["package_type"] comp = build_config["compiler"] cmd = ( @@ -83,15 +86,18 @@ def get_packager_cmd( return cmd -def get_image_name(build_config): +def get_image_name(build_config: BuildConfig) -> str: if build_config["package_type"] != "deb": return "clickhouse/binary-builder" else: return "clickhouse/deb-builder" -def build_clickhouse(packager_cmd, logs_path, build_output_path): +def build_clickhouse( + packager_cmd: str, logs_path: str, build_output_path: str +) -> Tuple[str, bool]: build_log_path = os.path.join(logs_path, "build_log.log") + success = False with TeePopen(packager_cmd, build_log_path) as process: retcode = process.wait() if os.path.exists(build_output_path): @@ -100,7 +106,8 @@ def build_clickhouse(packager_cmd, logs_path, build_output_path): build_results = [] if retcode == 0: - if len(build_results) != 0: + if len(build_results) > 0: + success = True logging.info("Built successfully") else: logging.info( @@ -108,10 +115,12 @@ def build_clickhouse(packager_cmd, logs_path, build_output_path): ) else: logging.info("Build failed") - return build_log_path, retcode == 0 and len(build_results) > 0 + return build_log_path, success -def get_build_results_if_exists(s3_helper, s3_prefix): +def get_build_results_if_exists( + s3_helper: S3Helper, s3_prefix: str +) -> Optional[List[str]]: try: content = s3_helper.list_prefix(s3_prefix) return content @@ -121,7 +130,13 @@ def get_build_results_if_exists(s3_helper, s3_prefix): def create_json_artifact( - temp_path, build_name, log_url, build_urls, build_config, elapsed, success + temp_path: str, + build_name: str, + log_url: str, + build_urls: List[str], + build_config: BuildConfig, + elapsed: int, + success: bool, ): subprocess.check_call( f"echo 'BUILD_NAME=build_urls_{build_name}' >> $GITHUB_ENV", shell=True @@ -150,40 +165,40 @@ def create_json_artifact( json.dump(result, build_links) -if __name__ == "__main__": +def get_release_or_pr( + pr_info: PRInfo, build_config: BuildConfig, version: ClickHouseVersion +) -> str: + if "release" in pr_info.labels or "release-lts" in pr_info.labels: + # for release pull requests we use branch names prefixes, not pr numbers + return pr_info.head_ref + elif pr_info.number == 0 and build_config["package_type"] != "performance": + # for pushes to master - major version, but not for performance builds + # they havily relies on a fixed path for build package and nobody going + # to deploy them somewhere, so it's ok. + return ".".join(version.as_tuple()[:2]) + # PR number for anything else + return str(pr_info.number) + + +def main(): logging.basicConfig(level=logging.INFO) - repo_path = REPO_COPY - temp_path = TEMP_PATH - caches_path = CACHES_PATH build_check_name = sys.argv[1] build_name = sys.argv[2] build_config = get_build_config(build_check_name, build_name) - if not os.path.exists(temp_path): - os.makedirs(temp_path) + if not os.path.exists(TEMP_PATH): + os.makedirs(TEMP_PATH) pr_info = PRInfo() - logging.info("Repo copy path %s", repo_path) + logging.info("Repo copy path %s", REPO_COPY) - gh = Github(get_best_robot_token()) s3_helper = S3Helper("https://s3.amazonaws.com") - version = get_version_from_repo(repo_path) - release_or_pr = None - if "release" in pr_info.labels or "release-lts" in pr_info.labels: - # for release pull requests we use branch names prefixes, not pr numbers - release_or_pr = pr_info.head_ref - elif pr_info.number == 0 and build_config["package_type"] != "performance": - # for pushes to master - major version, but not for performance builds - # they havily relies on a fixed path for build package and nobody going - # to deploy them somewhere, so it's ok. - release_or_pr = ".".join(version.as_tuple()[:2]) - else: - # PR number for anything else - release_or_pr = str(pr_info.number) + version = get_version_from_repo(REPO_COPY) + release_or_pr = get_release_or_pr(pr_info, build_config, version) s3_path_prefix = "/".join((release_or_pr, pr_info.sha, build_name)) @@ -205,7 +220,7 @@ if __name__ == "__main__": + url.replace("+", "%2B").replace(" ", "%20") ) create_json_artifact( - temp_path, + TEMP_PATH, build_name, log_url, build_urls, @@ -213,7 +228,7 @@ if __name__ == "__main__": 0, len(build_urls) > 0, ) - sys.exit(0) + return image_name = get_image_name(build_config) docker_image = get_image_with_version(IMAGES_PATH, image_name) @@ -225,20 +240,20 @@ if __name__ == "__main__": if "release" in pr_info.labels or "release-lts" in pr_info.labels: version_type = "stable" - update_version_local(repo_path, pr_info.sha, version, version_type) + update_version_local(REPO_COPY, pr_info.sha, version, version_type) logging.info("Updated local files with version") logging.info("Build short name %s", build_name) - build_output_path = os.path.join(temp_path, build_name) + build_output_path = os.path.join(TEMP_PATH, build_name) if not os.path.exists(build_output_path): os.makedirs(build_output_path) - ccache_path = os.path.join(caches_path, build_name + "_ccache") + ccache_path = os.path.join(CACHES_PATH, build_name + "_ccache") logging.info("Will try to fetch cache for our build") - get_ccache_if_not_exists(ccache_path, s3_helper, pr_info.number, temp_path) + get_ccache_if_not_exists(ccache_path, s3_helper, pr_info.number, TEMP_PATH) if not os.path.exists(ccache_path): logging.info("cache was not fetched, will create empty dir") @@ -247,12 +262,12 @@ if __name__ == "__main__": if build_config["package_type"] == "performance" and pr_info.number != 0: # because perf tests store some information about git commits subprocess.check_call( - f"cd {repo_path} && git fetch origin master:master", shell=True + f"cd {REPO_COPY} && git fetch origin master:master", shell=True ) packager_cmd = get_packager_cmd( build_config, - os.path.join(repo_path, "docker/packager"), + os.path.join(REPO_COPY, "docker/packager"), build_output_path, version.get_version_string(), image_version, @@ -261,7 +276,7 @@ if __name__ == "__main__": ) logging.info("Going to run packager with %s", packager_cmd) - build_clickhouse_log = os.path.join(temp_path, "build_log") + build_clickhouse_log = os.path.join(TEMP_PATH, "build_log") if not os.path.exists(build_clickhouse_log): os.makedirs(build_clickhouse_log) @@ -277,7 +292,7 @@ if __name__ == "__main__": logging.info("Build finished with %s, log path %s", success, log_path) logging.info("Will upload cache") - upload_ccache(ccache_path, s3_helper, pr_info.number, temp_path) + upload_ccache(ccache_path, s3_helper, pr_info.number, TEMP_PATH) if os.path.exists(log_path): log_url = s3_helper.upload_build_file_to_s3( @@ -300,8 +315,12 @@ if __name__ == "__main__": print("::notice ::Log URL: {}".format(log_url)) create_json_artifact( - temp_path, build_name, log_url, build_urls, build_config, elapsed, success + TEMP_PATH, build_name, log_url, build_urls, build_config, elapsed, success ) # Fail build job if not successeded if not success: sys.exit(1) + + +if __name__ == "__main__": + main() From 325c786f6856432f89b73264c0da47e233dca228 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 12 Jan 2022 13:17:54 +0100 Subject: [PATCH 09/18] Add uploading binary builds from master to static URLs --- tests/ci/build_check.py | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 714686b67b9..60a53156d3c 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -180,6 +180,28 @@ def get_release_or_pr( return str(pr_info.number) +def upload_master_static_binaries( + pr_info: PRInfo, + build_config: BuildConfig, + s3_helper: S3Helper, + build_output_path: str, +): + """Upload binary artifacts to a static S3 links""" + if pr_info.number != 0: + return + elif build_config["package_type"] != "binary": + return + elif pr_info.base_ref == "master": + return + + s3_path = "/".join( + (pr_info.base_ref, os.path.basename(build_output_path), "clickhouse") + ) + binary = os.path.join(build_output_path, "clickhouse") + url = s3_helper.upload_build_file_to_s3(binary, s3_path) + print(f"::notice ::Binary static URL: {url}") + + def main(): logging.basicConfig(level=logging.INFO) @@ -317,6 +339,8 @@ def main(): create_json_artifact( TEMP_PATH, build_name, log_url, build_urls, build_config, elapsed, success ) + + upload_master_static_binaries(pr_info, build_config, s3_helper, build_output_path) # Fail build job if not successeded if not success: sys.exit(1) From fc097173ef61b7a5542ef08fc5e4a0de8ab9f072 Mon Sep 17 00:00:00 2001 From: zhanghuajie <310336422@qq.com> Date: Wed, 29 Dec 2021 22:10:13 +0800 Subject: [PATCH 10/18] fix hang up with command 'drop table system.query_log sync' --- src/Interpreters/SystemLog.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index d6342e3973e..46254d0c3a2 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -168,6 +168,8 @@ public: void shutdown() override { stopFlushThread(); + + auto table = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); if (table) table->flushAndShutdown(); } @@ -186,7 +188,6 @@ private: /* Saving thread data */ const StorageID table_id; const String storage_def; - StoragePtr table; String create_query; String old_create_query; bool is_prepared = false; @@ -525,7 +526,7 @@ void SystemLog::prepareTable() { String description = table_id.getNameForLogs(); - table = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); + auto table = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); if (table) { From c5aa0bd23d07b9e6110ca8deb3813f5c61d44c7b Mon Sep 17 00:00:00 2001 From: zhanghuajie <310336422@qq.com> Date: Wed, 12 Jan 2022 21:05:42 +0800 Subject: [PATCH 11/18] fix hang up with command 'drop table system.query_log sync' -- add test case --- .../02167_hang_in_drop_query_log_sync.reference | 0 .../0_stateless/02167_hang_in_drop_query_log_sync.sh | 7 +++++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/02167_hang_in_drop_query_log_sync.reference create mode 100644 tests/queries/0_stateless/02167_hang_in_drop_query_log_sync.sh diff --git a/tests/queries/0_stateless/02167_hang_in_drop_query_log_sync.reference b/tests/queries/0_stateless/02167_hang_in_drop_query_log_sync.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02167_hang_in_drop_query_log_sync.sh b/tests/queries/0_stateless/02167_hang_in_drop_query_log_sync.sh new file mode 100644 index 00000000000..7e6f625388c --- /dev/null +++ b/tests/queries/0_stateless/02167_hang_in_drop_query_log_sync.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +timeout 3s $CLICKHOUSE_CLIENT --receive_timeout=2 -q "DROP TABLE system.query_log sync;" 2>&1 | grep -o "Timeout exceeded" From 9188b8afc284540f6d78b293ff67726715977051 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 12 Jan 2022 14:30:32 +0100 Subject: [PATCH 12/18] Fix logic for upload upload_master_static_binaries --- tests/ci/build_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 60a53156d3c..c78f818e1ec 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -191,7 +191,7 @@ def upload_master_static_binaries( return elif build_config["package_type"] != "binary": return - elif pr_info.base_ref == "master": + elif pr_info.base_ref != "master": return s3_path = "/".join( From 830edf76ccd8682dc9924080c48db1484ef47ae7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 12 Jan 2022 14:57:11 +0100 Subject: [PATCH 13/18] Don't upload splitted binaries to builds.clickhouse.com --- tests/ci/build_check.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index c78f818e1ec..0362732403f 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -191,6 +191,8 @@ def upload_master_static_binaries( return elif build_config["package_type"] != "binary": return + elif build_config["splitted"] == "splitted": + return elif pr_info.base_ref != "master": return From 72b67f37fc5e1f2f3ed96e9c2df1bb9e33dfb290 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 12 Jan 2022 18:20:22 +0300 Subject: [PATCH 14/18] fix log message and flaky test --- src/Processors/Transforms/TTLTransform.cpp | 6 ++++-- tests/queries/0_stateless/01282_system_parts_ttl_info.sql | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/TTLTransform.cpp b/src/Processors/Transforms/TTLTransform.cpp index 7d0da3dca91..e79dcb34c41 100644 --- a/src/Processors/Transforms/TTLTransform.cpp +++ b/src/Processors/Transforms/TTLTransform.cpp @@ -139,8 +139,10 @@ void TTLTransform::finalize() if (delete_algorithm) { - size_t rows_removed = all_data_dropped ? data_part->rows_count : delete_algorithm->getNumberOfRemovedRows(); - LOG_DEBUG(log, "Removed {} rows with expired TTL from part {}", rows_removed, data_part->name); + if (all_data_dropped) + LOG_DEBUG(log, "Removed all rows from part {} due to expired TTL", data_part->name); + else + LOG_DEBUG(log, "Removed {} rows with expired TTL from part {}", delete_algorithm->getNumberOfRemovedRows(), data_part->name); } } diff --git a/tests/queries/0_stateless/01282_system_parts_ttl_info.sql b/tests/queries/0_stateless/01282_system_parts_ttl_info.sql index dfa340636b3..ede5350ddd4 100644 --- a/tests/queries/0_stateless/01282_system_parts_ttl_info.sql +++ b/tests/queries/0_stateless/01282_system_parts_ttl_info.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS ttl; -CREATE TABLE ttl (d DateTime) ENGINE = MergeTree ORDER BY tuple() TTL d + INTERVAL 10 DAY; +CREATE TABLE ttl (d DateTime) ENGINE = MergeTree ORDER BY tuple() TTL d + INTERVAL 10 DAY SETTINGS remove_empty_parts=0; SYSTEM STOP MERGES ttl; INSERT INTO ttl VALUES ('2000-01-01 01:02:03'), ('2000-02-03 04:05:06'); SELECT rows, delete_ttl_info_min, delete_ttl_info_max, move_ttl_info.expression, move_ttl_info.min, move_ttl_info.max FROM system.parts WHERE database = currentDatabase() AND table = 'ttl'; From 9ceaaeaf91c52b9ac61455620c0ec9c6e96fd274 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 12 Jan 2022 20:19:24 +0300 Subject: [PATCH 15/18] Revert "fix hang up with command 'drop table system.query_log sync' -- add test case" This reverts commit c5aa0bd23d07b9e6110ca8deb3813f5c61d44c7b. --- .../02167_hang_in_drop_query_log_sync.reference | 0 .../0_stateless/02167_hang_in_drop_query_log_sync.sh | 7 ------- 2 files changed, 7 deletions(-) delete mode 100644 tests/queries/0_stateless/02167_hang_in_drop_query_log_sync.reference delete mode 100644 tests/queries/0_stateless/02167_hang_in_drop_query_log_sync.sh diff --git a/tests/queries/0_stateless/02167_hang_in_drop_query_log_sync.reference b/tests/queries/0_stateless/02167_hang_in_drop_query_log_sync.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02167_hang_in_drop_query_log_sync.sh b/tests/queries/0_stateless/02167_hang_in_drop_query_log_sync.sh deleted file mode 100644 index 7e6f625388c..00000000000 --- a/tests/queries/0_stateless/02167_hang_in_drop_query_log_sync.sh +++ /dev/null @@ -1,7 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -timeout 3s $CLICKHOUSE_CLIENT --receive_timeout=2 -q "DROP TABLE system.query_log sync;" 2>&1 | grep -o "Timeout exceeded" From 25db53a3c57f2586bb073720f345554c47a37112 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 12 Jan 2022 21:08:08 +0300 Subject: [PATCH 16/18] add an integration test --- .../test_system_logs_recreate/test.py | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/tests/integration/test_system_logs_recreate/test.py b/tests/integration/test_system_logs_recreate/test.py index 3ab0269b42e..c0afa8cd555 100644 --- a/tests/integration/test_system_logs_recreate/test.py +++ b/tests/integration/test_system_logs_recreate/test.py @@ -68,3 +68,26 @@ def test_system_logs_recreate(): # IOW that the table created only when the structure is indeed different. for table in system_logs: assert len(node.query(f"SHOW TABLES FROM system LIKE '{table}%'").strip().split('\n')) == 3 + + +def test_drop_system_log(): + node.exec_in_container(['bash', '-c', f"""echo " + + + 1000000 + + + " > /etc/clickhouse-server/config.d/yyy-override-query_log.xml + """]) + node.restart_clickhouse() + node.query("select 1") + node.query("system flush logs") + node.query("select 2") + node.query("system flush logs") + assert node.query("select count() > 0 from system.query_log") == "1\n" + node.query("drop table system.query_log sync") + node.query("select 3") + node.query("system flush logs") + assert node.query("select count() > 0 from system.query_log") == "1\n" + node.exec_in_container(['rm', f'/etc/clickhouse-server/config.d/yyy-override-query_log.xml']) + node.restart_clickhouse() From 60bb0e43f482f82e6f724d1fabf0aad1a0670d61 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 12 Jan 2022 23:51:40 +0300 Subject: [PATCH 17/18] fix test with wide parts --- .../02156_storage_merge_prewhere.reference | 8 ++++---- .../0_stateless/02156_storage_merge_prewhere.sql | 12 ++++++------ 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference index 6df60403ae0..2dc83f1eaa5 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference @@ -1,13 +1,13 @@ SELECT count() FROM t_02156_merge1 -PREWHERE k = 1 -WHERE (k = 1) AND notEmpty(v) +PREWHERE k = 3 +WHERE (k = 3) AND notEmpty(v) 2 SELECT count() FROM t_02156_merge2 -WHERE (k = 1) AND notEmpty(v) +WHERE (k = 3) AND notEmpty(v) 2 SELECT count() FROM t_02156_merge3 -WHERE (k = 1) AND notEmpty(v) +WHERE (k = 3) AND notEmpty(v) 2 diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql index e033005e014..69fa9ac5ee2 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql @@ -20,14 +20,14 @@ INSERT INTO t_02156_mt1 SELECT number, toString(number) FROM numbers(10000); INSERT INTO t_02156_mt2 SELECT number, toString(number) FROM numbers(10000); INSERT INTO t_02156_log SELECT number, toString(number) FROM numbers(10000); -EXPLAIN SYNTAX SELECT count() FROM t_02156_merge1 WHERE k = 1 AND notEmpty(v); -SELECT count() FROM t_02156_merge1 WHERE k = 1 AND notEmpty(v); +EXPLAIN SYNTAX SELECT count() FROM t_02156_merge1 WHERE k = 3 AND notEmpty(v); +SELECT count() FROM t_02156_merge1 WHERE k = 3 AND notEmpty(v); -EXPLAIN SYNTAX SELECT count() FROM t_02156_merge2 WHERE k = 1 AND notEmpty(v); -SELECT count() FROM t_02156_merge2 WHERE k = 1 AND notEmpty(v); +EXPLAIN SYNTAX SELECT count() FROM t_02156_merge2 WHERE k = 3 AND notEmpty(v); +SELECT count() FROM t_02156_merge2 WHERE k = 3 AND notEmpty(v); -EXPLAIN SYNTAX SELECT count() FROM t_02156_merge3 WHERE k = 1 AND notEmpty(v); -SELECT count() FROM t_02156_merge3 WHERE k = 1 AND notEmpty(v); +EXPLAIN SYNTAX SELECT count() FROM t_02156_merge3 WHERE k = 3 AND notEmpty(v); +SELECT count() FROM t_02156_merge3 WHERE k = 3 AND notEmpty(v); DROP TABLE IF EXISTS t_02156_mt1; DROP TABLE IF EXISTS t_02156_mt2; From 5d2324368f0276ece354a5aaa370bb278f5dacd1 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 13 Jan 2022 09:37:57 +0300 Subject: [PATCH 18/18] Update mongodb.md --- docs/en/engines/table-engines/integrations/mongodb.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/mongodb.md b/docs/en/engines/table-engines/integrations/mongodb.md index 52876674475..475416ffb94 100644 --- a/docs/en/engines/table-engines/integrations/mongodb.md +++ b/docs/en/engines/table-engines/integrations/mongodb.md @@ -66,4 +66,14 @@ SELECT COUNT() FROM mongo_table; └─────────┘ ``` +You can also adjust connection timeout: + +``` sql +CREATE TABLE mongo_table +( + key UInt64, + data String +) ENGINE = MongoDB('mongo2:27017', 'test', 'simple_table', 'testuser', 'clickhouse', 'connectTimeoutMS=100000'); +``` + [Original article](https://clickhouse.com/docs/en/engines/table-engines/integrations/mongodb/)