ClickHouse/src/Client/Suggest.cpp

200 lines
7.9 KiB
C++
Raw Normal View History

2020-01-01 19:22:57 +00:00
#include "Suggest.h"
2021-08-03 08:33:54 +00:00
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <Core/Settings.h>
2020-01-01 19:22:57 +00:00
#include <Columns/ColumnString.h>
#include <Common/typeid_cast.h>
2021-08-03 08:33:54 +00:00
#include <Common/Macros.h>
2021-08-30 11:04:59 +00:00
#include "Core/Protocol.h"
2021-06-02 14:18:54 +00:00
#include <IO/Operators.h>
2021-08-03 08:33:54 +00:00
#include <Functions/FunctionFactory.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Storages/StorageFactory.h>
#include <DataTypes/DataTypeFactory.h>
#include <Interpreters/Context.h>
2021-09-11 11:34:22 +00:00
#include <Client/Connection.h>
#include <Client/LocalConnection.h>
2020-01-01 19:22:57 +00:00
namespace DB
{
2020-02-25 18:10:48 +00:00
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_PACKET_FROM_SERVER;
extern const int DEADLOCK_AVOIDED;
2020-02-25 18:10:48 +00:00
}
2020-01-01 19:22:57 +00:00
Suggest::Suggest()
{
/// Keywords may be not up to date with ClickHouse parser.
addWords({
"CREATE", "DATABASE", "IF", "NOT", "EXISTS", "TEMPORARY", "TABLE", "ON", "CLUSTER", "DEFAULT",
"MATERIALIZED", "ALIAS", "ENGINE", "AS", "VIEW", "POPULATE", "SETTINGS", "ATTACH", "DETACH", "DROP",
"RENAME", "TO", "ALTER", "ADD", "MODIFY", "CLEAR", "COLUMN", "AFTER", "COPY", "PROJECT",
"PRIMARY", "KEY", "CHECK", "PARTITION", "PART", "FREEZE", "FETCH", "FROM", "SHOW", "INTO",
"OUTFILE", "FORMAT", "TABLES", "DATABASES", "LIKE", "PROCESSLIST", "CASE", "WHEN", "THEN", "ELSE",
"END", "DESCRIBE", "DESC", "USE", "SET", "OPTIMIZE", "FINAL", "DEDUPLICATE", "INSERT", "VALUES",
"SELECT", "DISTINCT", "SAMPLE", "ARRAY", "JOIN", "GLOBAL", "LOCAL", "ANY", "ALL", "INNER",
"LEFT", "RIGHT", "FULL", "OUTER", "CROSS", "USING", "PREWHERE", "WHERE", "GROUP", "BY",
"WITH", "TOTALS", "HAVING", "ORDER", "COLLATE", "LIMIT", "UNION", "AND", "OR", "ASC",
"IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE",
"PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE",
"IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "RANDOMIZED",
"INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE",
});
2020-01-01 19:22:57 +00:00
}
2021-09-11 11:34:22 +00:00
static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggestion)
2020-01-01 19:22:57 +00:00
{
/// NOTE: Once you will update the completion list,
/// do not forget to update 01676_clickhouse_client_autocomplete.sh
2021-06-02 14:18:54 +00:00
WriteBufferFromOwnString query;
2021-08-23 08:50:12 +00:00
query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM ("
2021-08-03 08:33:54 +00:00
"SELECT name FROM system.functions"
" UNION ALL "
"SELECT name FROM system.table_engines"
" UNION ALL "
"SELECT name FROM system.formats"
" UNION ALL "
"SELECT name FROM system.table_functions"
" UNION ALL "
"SELECT name FROM system.data_type_families"
" UNION ALL "
"SELECT name FROM system.merge_tree_settings"
" UNION ALL "
"SELECT name FROM system.settings"
2021-09-11 11:34:22 +00:00
" UNION ALL ";
if (!basic_suggestion)
{
query << "SELECT cluster FROM system.clusters"
" UNION ALL "
"SELECT macro FROM system.macros"
" UNION ALL "
"SELECT policy_name FROM system.storage_policies"
" UNION ALL ";
}
query << "SELECT concat(func.name, comb.name) FROM system.functions AS func CROSS JOIN system.aggregate_function_combinators AS comb WHERE is_aggregate";
2021-08-03 08:33:54 +00:00
/// The user may disable loading of databases, tables, columns by setting suggestion_limit to zero.
2021-08-02 14:46:21 +00:00
if (suggestion_limit > 0)
{
String limit_str = toString(suggestion_limit);
2021-09-11 11:34:22 +00:00
query << " UNION ALL "
"SELECT name FROM system.databases LIMIT " << limit_str
<< " UNION ALL "
"SELECT DISTINCT name FROM system.tables LIMIT " << limit_str
<< " UNION ALL ";
if (!basic_suggestion)
{
query << "SELECT DISTINCT name FROM system.dictionaries LIMIT " << limit_str
<< " UNION ALL ";
}
query << "SELECT DISTINCT name FROM system.columns LIMIT " << limit_str;
2020-01-01 19:22:57 +00:00
}
query << ") WHERE notEmpty(res)";
2021-08-03 08:33:54 +00:00
2021-08-01 20:28:39 +00:00
return query.str();
}
2020-01-01 19:22:57 +00:00
2021-09-11 11:34:22 +00:00
template <typename ConnectionType>
void Suggest::load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit)
2021-08-01 20:28:39 +00:00
{
2021-09-11 11:34:22 +00:00
loading_thread = std::thread([context=Context::createCopy(context), connection_parameters, suggestion_limit, this]
{
2022-02-15 12:11:13 +00:00
ThreadStatus thread_status;
2021-09-11 11:34:22 +00:00
for (size_t retry = 0; retry < 10; ++retry)
{
try
{
auto connection = ConnectionType::createConnection(connection_parameters, context);
fetch(*connection, connection_parameters.timeouts, getLoadSuggestionQuery(suggestion_limit, std::is_same_v<ConnectionType, LocalConnection>));
}
catch (const Exception & e)
{
if (e.code() == ErrorCodes::DEADLOCK_AVOIDED)
continue;
std::cerr << "Cannot load data for command line suggestions: " << getCurrentExceptionMessage(false, true) << "\n";
}
catch (...)
{
std::cerr << "Cannot load data for command line suggestions: " << getCurrentExceptionMessage(false, true) << "\n";
}
2021-08-01 20:28:39 +00:00
2021-09-11 11:34:22 +00:00
break;
}
/// Note that keyword suggestions are available even if we cannot load data from server.
});
}
void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & timeouts, const std::string & query)
2020-01-01 19:22:57 +00:00
{
2021-08-17 19:59:51 +00:00
connection.sendQuery(timeouts, query, "" /* query_id */, QueryProcessingStage::Complete, nullptr, nullptr, false);
2020-01-01 19:22:57 +00:00
while (true)
{
Packet packet = connection.receivePacket();
switch (packet.type)
{
case Protocol::Server::Data:
fillWordsFromBlock(packet.block);
continue;
case Protocol::Server::Progress:
continue;
case Protocol::Server::ProfileInfo:
continue;
case Protocol::Server::Totals:
continue;
case Protocol::Server::Extremes:
continue;
case Protocol::Server::Log:
continue;
2021-08-30 11:04:59 +00:00
case Protocol::Server::ProfileEvents:
continue;
2020-01-01 19:22:57 +00:00
case Protocol::Server::Exception:
packet.exception->rethrow();
return;
case Protocol::Server::EndOfStream:
return;
default:
throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from server {}",
packet.type, connection.getDescription());
2020-01-01 19:22:57 +00:00
}
}
}
void Suggest::fillWordsFromBlock(const Block & block)
{
if (!block)
return;
if (block.columns() != 1)
throw Exception("Wrong number of columns received for query to read words for suggestion", ErrorCodes::LOGICAL_ERROR);
const ColumnString & column = typeid_cast<const ColumnString &>(*block.getByPosition(0).column);
size_t rows = block.rows();
Words new_words;
new_words.reserve(rows);
2020-01-01 19:22:57 +00:00
for (size_t i = 0; i < rows; ++i)
{
new_words.emplace_back(column.getDataAt(i).toString());
}
addWords(std::move(new_words));
2020-01-01 19:22:57 +00:00
}
2021-09-11 11:34:22 +00:00
template
void Suggest::load<Connection>(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit);
template
void Suggest::load<LocalConnection>(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit);
2020-01-01 19:22:57 +00:00
}