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>
2021-01-25 20:05:41 +00:00
# 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 ;
2020-02-29 19:53:22 +00:00
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.
2022-01-24 19:59:18 +00:00
addWords ( {
2022-01-18 18:03:51 +00:00
" 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 " ,
2022-01-24 19:59:18 +00:00
} ) ;
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
{
2021-01-27 19:44:22 +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 :
2020-08-17 18:37:24 +00:00
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 ( ) ;
2022-01-24 19:59:18 +00:00
Words new_words ;
new_words . reserve ( rows ) ;
2020-01-01 19:22:57 +00:00
for ( size_t i = 0 ; i < rows ; + + i )
2022-01-24 19:59:18 +00:00
{
2022-01-18 18:03:51 +00:00
new_words . emplace_back ( column . getDataAt ( i ) . toString ( ) ) ;
2022-01-24 19:59:18 +00:00
}
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
}