Merge pull request #19584 from azat/client-complete-error-codes

client: more suggestions
This commit is contained in:
alexey-milovidov 2021-02-02 11:05:17 +03:00 committed by GitHub
commit f5dc38a33c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 148 additions and 6 deletions

View File

@ -1,5 +1,6 @@
#include "Suggest.h"
#include <Core/Settings.h>
#include <Columns/ColumnString.h>
#include <Common/typeid_cast.h>
@ -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)
{

View File

@ -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.

View File

@ -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<ThreadFromGlobalPool>([this] { run(); });
}

View File

@ -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<Runner>(*this))
{}
GRPCServer::~GRPCServer()
@ -1644,7 +1647,6 @@ void GRPCServer::start()
queue = builder.AddCompletionQueue();
grpc_server = builder.BuildAndStart();
runner = std::make_unique<Runner>(*this);
runner->start();
}

View File

@ -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

View File

@ -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

View File

@ -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"
],