diff --git a/programs/client/Suggest.cpp b/programs/client/Suggest.cpp index 87083c2c27b..dfa7048349e 100644 --- a/programs/client/Suggest.cpp +++ b/programs/client/Suggest.cpp @@ -1,5 +1,6 @@ #include "Suggest.h" +#include #include #include @@ -86,6 +87,9 @@ Suggest::Suggest() void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeouts, size_t suggestion_limit) { + /// NOTE: Once you will update the completion list, + /// do not forget to update 01676_clickhouse_client_autocomplete.sh + std::stringstream query; // STYLE_CHECK_ALLOW_STD_STRING_STREAM query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" "SELECT name FROM system.functions" @@ -104,6 +108,18 @@ void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeo " UNION ALL " "SELECT cluster FROM system.clusters" " UNION ALL " + "SELECT name FROM system.errors" + " UNION ALL " + "SELECT event FROM system.events" + " UNION ALL " + "SELECT metric FROM system.asynchronous_metrics" + " UNION ALL " + "SELECT metric FROM system.metrics" + " UNION ALL " + "SELECT macro FROM system.macros" + " UNION ALL " + "SELECT policy_name FROM system.storage_policies" + " UNION ALL " "SELECT concat(func.name, comb.name) FROM system.functions AS func CROSS JOIN system.aggregate_function_combinators AS comb WHERE is_aggregate"; /// The user may disable loading of databases, tables, columns by setting suggestion_limit to zero. @@ -123,12 +139,17 @@ void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeo query << ") WHERE notEmpty(res)"; - fetch(connection, timeouts, query.str()); + Settings settings; + /// To show all rows from: + /// - system.errors + /// - system.events + settings.system_events_show_zero_values = true; + fetch(connection, timeouts, query.str(), settings); } -void Suggest::fetch(Connection & connection, const ConnectionTimeouts & timeouts, const std::string & query) +void Suggest::fetch(Connection & connection, const ConnectionTimeouts & timeouts, const std::string & query, Settings & settings) { - connection.sendQuery(timeouts, query); + connection.sendQuery(timeouts, query, "" /* query_id */, QueryProcessingStage::Complete, &settings); while (true) { diff --git a/programs/client/Suggest.h b/programs/client/Suggest.h index 03332088cbe..0049bc08ebf 100644 --- a/programs/client/Suggest.h +++ b/programs/client/Suggest.h @@ -33,7 +33,7 @@ public: private: void loadImpl(Connection & connection, const ConnectionTimeouts & timeouts, size_t suggestion_limit); - void fetch(Connection & connection, const ConnectionTimeouts & timeouts, const std::string & query); + void fetch(Connection & connection, const ConnectionTimeouts & timeouts, const std::string & query, Settings & settings); void fillWordsFromBlock(const Block & block); /// Words are fetched asynchronously. diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index 88c2221be76..f727b1d6b48 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -58,6 +58,9 @@ public: /// Separate method allows to initialize the `servers` variable beforehand. void start() { + /// Update once right now, to make metrics available just after server start + /// (without waiting for asynchronous_metrics_update_period_s). + update(); thread = std::make_unique([this] { run(); }); } diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 475bfc81801..c3492e9ea8a 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1613,7 +1613,10 @@ private: GRPCServer::GRPCServer(IServer & iserver_, const Poco::Net::SocketAddress & address_to_listen_) - : iserver(iserver_), address_to_listen(address_to_listen_), log(&Poco::Logger::get("GRPCServer")) + : iserver(iserver_) + , address_to_listen(address_to_listen_) + , log(&Poco::Logger::get("GRPCServer")) + , runner(std::make_unique(*this)) {} GRPCServer::~GRPCServer() @@ -1644,7 +1647,6 @@ void GRPCServer::start() queue = builder.AddCompletionQueue(); grpc_server = builder.BuildAndStart(); - runner = std::make_unique(*this); runner->start(); } diff --git a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.reference b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh new file mode 100755 index 00000000000..08e07044841 --- /dev/null +++ b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh @@ -0,0 +1,114 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function test_completion_word() +{ + local w=$1 && shift + + local w_len=${#w} + local compword_begin=${w:0:$((w_len-3))} + local compword_end=${w:$((w_len-3))} + + # NOTE: here and below you should escape variables of the expect. + timeout 22s expect << EOF +log_user 0 +set timeout 3 +match_max 100000 +# A default timeout action is to do nothing, change it to fail +expect_after { + timeout { + exit 1 + } +} + +spawn bash -c "$CLICKHOUSE_CLIENT_BINARY $CLICKHOUSE_CLIENT_OPT" +expect ":) " + +# Make a query +send -- "SET $compword_begin" +expect "SET $compword_begin" + +# Wait for suggestions to load, they are loaded in background +set is_done 0 +while {\$is_done == 0} { + send -- "\\t" + expect { + "$compword_begin$compword_end" { + set is_done 1 + } + default { + sleep 1 + } + } +} + +send -- "\\3\\4" +expect eof +EOF +} + +# last 3 bytes will be completed, +# so take this in mind when you will update the list. +compwords_positive=( + # system.functions + concatAssumeInjective + # system.table_engines + ReplacingMergeTree + # system.formats + JSONEachRow + # system.table_functions + clusterAllReplicas + # system.data_type_families + SimpleAggregateFunction + # system.merge_tree_settings + write_ahead_log_interval_ms_to_fsync + # system.settings + max_concurrent_queries_for_all_users + # system.clusters + test_shard_localhost + # system.errors, also it is very rare to cover system_events_show_zero_values + CONDITIONAL_TREE_PARENT_NOT_FOUND + # system.events, also it is very rare to cover system_events_show_zero_values + WriteBufferFromFileDescriptorWriteFailed + # system.asynchronous_metrics, also this metric has zero value + # + # NOTE: that there is no ability to complete metrics like + # jemalloc.background_thread.num_runs, due to "." is used as a word breaker + # (and this cannot be changed -- db.table) + ReplicasMaxAbsoluteDelay + # system.metrics + PartsPreCommitted + # system.macros + default_path_test + # system.storage_policies, egh not uniq + default + # system.aggregate_function_combinators + uniqCombined64ForEach + + # FIXME: one may add separate case for suggestion_limit + # system.databases + system + # system.tables + aggregate_function_combinators + # system.columns + primary_key_bytes_in_memory_allocated + # system.dictionaries + # FIXME: none +) +for w in "${compwords_positive[@]}"; do + test_completion_word "$w" || echo "[FAIL] $w (positive)" +done + +# One negative is enough +compwords_negative=( + # system.clusters + test_shard_localhost_no_such_cluster +) +for w in "${compwords_negative[@]}"; do + test_completion_word "$w" && echo "[FAIL] $w (negative)" +done + +exit 0 diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 5b8256bb5af..a33ff98032b 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -197,5 +197,6 @@ 01658_test_base64Encode_mysql_compatibility 01659_test_base64Decode_mysql_compatibility 01675_data_type_coroutine +01676_clickhouse_client_autocomplete 01671_aggregate_function_group_bitmap_data 01674_executable_dictionary_implicit_key diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 26c1c71de62..d973c09acef 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -90,6 +90,7 @@ "01300_client_save_history_when_terminated", "orc_output", "01370_client_autocomplete_word_break_characters", + "01676_clickhouse_client_autocomplete", "01193_metadata_loading", "01455_time_zones" ],