2017-04-01 09:19:00 +00:00
# include <DataTypes/DataTypeString.h>
# include <DataTypes/DataTypesNumber.h>
# include <DataTypes/DataTypeDateTime.h>
# include <Storages/System/StorageSystemZooKeeper.h>
2020-12-10 23:56:57 +00:00
# include <Storages/SelectQueryInfo.h>
2017-04-01 09:19:00 +00:00
# include <Parsers/ASTSelectQuery.h>
# include <Parsers/ASTIdentifier.h>
# include <Parsers/ASTLiteral.h>
# include <Parsers/ASTExpressionList.h>
# include <Parsers/ASTFunction.h>
# include <Interpreters/Context.h>
2020-09-21 21:09:50 +00:00
# include <Interpreters/evaluateConstantExpression.h>
2017-06-19 20:06:35 +00:00
# include <Common/ZooKeeper/ZooKeeper.h>
2017-07-13 20:58:19 +00:00
# include <Common/typeid_cast.h>
2021-02-05 06:45:28 +00:00
# include <Parsers/ASTSubquery.h>
# include <Interpreters/Set.h>
# include <Interpreters/interpretSubquery.h>
2014-10-06 03:49:56 +00:00
namespace DB
{
2018-11-22 21:19:58 +00:00
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS ;
}
2014-10-06 03:49:56 +00:00
2018-07-24 18:40:53 +00:00
NamesAndTypesList StorageSystemZooKeeper : : getNamesAndTypes ( )
{
2018-07-24 14:28:56 +00:00
return {
2017-06-06 18:36:13 +00:00
{ " name " , std : : make_shared < DataTypeString > ( ) } ,
{ " value " , std : : make_shared < DataTypeString > ( ) } ,
{ " czxid " , std : : make_shared < DataTypeInt64 > ( ) } ,
{ " mzxid " , std : : make_shared < DataTypeInt64 > ( ) } ,
{ " ctime " , std : : make_shared < DataTypeDateTime > ( ) } ,
{ " mtime " , std : : make_shared < DataTypeDateTime > ( ) } ,
{ " version " , std : : make_shared < DataTypeInt32 > ( ) } ,
{ " cversion " , std : : make_shared < DataTypeInt32 > ( ) } ,
{ " aversion " , std : : make_shared < DataTypeInt32 > ( ) } ,
{ " ephemeralOwner " , std : : make_shared < DataTypeInt64 > ( ) } ,
{ " dataLength " , std : : make_shared < DataTypeInt32 > ( ) } ,
{ " numChildren " , std : : make_shared < DataTypeInt32 > ( ) } ,
{ " pzxid " , std : : make_shared < DataTypeInt64 > ( ) } ,
{ " path " , std : : make_shared < DataTypeString > ( ) } ,
2018-07-24 14:28:56 +00:00
} ;
2014-10-06 03:49:56 +00:00
}
2021-02-05 06:45:28 +00:00
using Paths = Strings ;
2014-10-06 03:49:56 +00:00
2021-02-05 06:45:28 +00:00
static String pathCorrected ( const String & path )
{
String path_corrected ;
/// path should starts with '/', otherwise ZBADARGUMENTS will be thrown in
/// ZooKeeper::sendThread and the session will fail.
if ( path [ 0 ] ! = ' / ' )
path_corrected = ' / ' ;
path_corrected + = path ;
/// In all cases except the root, path must not end with a slash.
if ( path_corrected ! = " / " & & path_corrected . back ( ) = = ' / ' )
path_corrected . resize ( path_corrected . size ( ) - 1 ) ;
return path_corrected ;
}
2021-04-10 23:33:54 +00:00
static bool extractPathImpl ( const IAST & elem , Paths & res , ContextPtr context )
2014-10-06 03:49:56 +00:00
{
2019-03-11 13:22:51 +00:00
const auto * function = elem . as < ASTFunction > ( ) ;
2017-04-01 07:20:54 +00:00
if ( ! function )
return false ;
if ( function - > name = = " and " )
{
2020-03-09 03:14:24 +00:00
for ( const auto & child : function - > arguments - > children )
2020-09-21 21:09:50 +00:00
if ( extractPathImpl ( * child , res , context ) )
2017-04-01 07:20:54 +00:00
return true ;
return false ;
}
2021-02-05 06:45:28 +00:00
const auto & args = function - > arguments - > as < ASTExpressionList & > ( ) ;
if ( args . children . size ( ) ! = 2 )
return false ;
2017-04-01 07:20:54 +00:00
2021-02-05 06:45:28 +00:00
if ( function - > name = = " in " )
{
const ASTIdentifier * ident = args . children . at ( 0 ) - > as < ASTIdentifier > ( ) ;
if ( ! ident | | ident - > name ( ) ! = " path " )
2017-04-01 07:20:54 +00:00
return false ;
2021-02-05 06:45:28 +00:00
ASTPtr value = args . children . at ( 1 ) ;
if ( value - > as < ASTSubquery > ( ) )
{
auto interpreter_subquery = interpretSubquery ( value , context , { } , { } ) ;
auto stream = interpreter_subquery - > execute ( ) . getInputStream ( ) ;
2021-04-10 23:33:54 +00:00
SizeLimits limites ( context - > getSettingsRef ( ) . max_rows_in_set , context - > getSettingsRef ( ) . max_bytes_in_set , OverflowMode : : THROW ) ;
Set set ( limites , true , context - > getSettingsRef ( ) . transform_null_in ) ;
2021-02-05 06:45:28 +00:00
set . setHeader ( stream - > getHeader ( ) ) ;
stream - > readPrefix ( ) ;
while ( Block block = stream - > read ( ) )
{
set . insertFromBlock ( block ) ;
}
set . finishInsert ( ) ;
stream - > readSuffix ( ) ;
set . checkColumnsNumber ( 1 ) ;
const auto & set_column = * set . getSetElements ( ) [ 0 ] ;
for ( size_t row = 0 ; row < set_column . size ( ) ; + + row )
res . emplace_back ( set_column [ row ] . safeGet < String > ( ) ) ;
}
else
{
auto evaluated = evaluateConstantExpressionAsLiteral ( value , context ) ;
const auto * literal = evaluated - > as < ASTLiteral > ( ) ;
if ( ! literal )
return false ;
if ( String str ; literal - > value . tryGet ( str ) )
{
res . emplace_back ( str ) ;
}
else if ( Tuple tuple ; literal - > value . tryGet ( tuple ) )
{
for ( auto element : tuple )
res . emplace_back ( element . safeGet < String > ( ) ) ;
}
else
return false ;
}
return true ;
}
else if ( function - > name = = " equals " )
{
2017-04-01 07:20:54 +00:00
const ASTIdentifier * ident ;
2021-02-05 06:45:28 +00:00
ASTPtr value ;
2019-03-15 17:09:14 +00:00
if ( ( ident = args . children . at ( 0 ) - > as < ASTIdentifier > ( ) ) )
2020-09-21 21:09:50 +00:00
value = args . children . at ( 1 ) ;
2019-03-15 17:09:14 +00:00
else if ( ( ident = args . children . at ( 1 ) - > as < ASTIdentifier > ( ) ) )
2020-09-21 21:09:50 +00:00
value = args . children . at ( 0 ) ;
2017-04-01 07:20:54 +00:00
else
return false ;
2020-10-24 18:46:10 +00:00
if ( ident - > name ( ) ! = " path " )
2017-04-01 07:20:54 +00:00
return false ;
2020-09-21 21:09:50 +00:00
auto evaluated = evaluateConstantExpressionAsLiteral ( value , context ) ;
const auto * literal = evaluated - > as < ASTLiteral > ( ) ;
2017-04-01 07:20:54 +00:00
if ( ! literal )
return false ;
if ( literal - > value . getType ( ) ! = Field : : Types : : String )
return false ;
2021-02-05 06:45:28 +00:00
res . emplace_back ( literal - > value . safeGet < String > ( ) ) ;
2017-04-01 07:20:54 +00:00
return true ;
}
return false ;
2014-10-06 03:49:56 +00:00
}
2017-03-12 19:18:07 +00:00
/** Retrieve from the query a condition of the form `path = 'path'`, from conjunctions in the WHERE clause.
2014-10-06 03:49:56 +00:00
*/
2021-04-10 23:33:54 +00:00
static Paths extractPath ( const ASTPtr & query , ContextPtr context )
2014-10-06 03:49:56 +00:00
{
2019-03-15 17:09:14 +00:00
const auto & select = query - > as < ASTSelectQuery & > ( ) ;
2019-04-09 14:22:35 +00:00
if ( ! select . where ( ) )
2021-02-05 06:45:28 +00:00
return Paths ( ) ;
2014-10-06 03:49:56 +00:00
2021-02-05 06:45:28 +00:00
Paths res ;
return extractPathImpl ( * select . where ( ) , res , context ) ? res : Paths ( ) ;
2014-10-06 03:49:56 +00:00
}
2021-04-10 23:33:54 +00:00
void StorageSystemZooKeeper : : fillData ( MutableColumns & res_columns , ContextPtr context , const SelectQueryInfo & query_info ) const
2014-10-06 03:49:56 +00:00
{
2021-02-05 06:45:28 +00:00
const Paths & paths = extractPath ( query_info . query , context ) ;
if ( paths . empty ( ) )
throw Exception ( " SELECT from system.zookeeper table must contain condition like path = 'path' or path IN ('path1','path2'...) or path IN ( subquery ) in WHERE clause . " , ErrorCodes::BAD_ARGUMENTS) ;
2017-04-01 07:20:54 +00:00
2021-04-10 23:33:54 +00:00
zkutil : : ZooKeeperPtr zookeeper = context - > getZooKeeper ( ) ;
2017-04-01 07:20:54 +00:00
2021-02-05 06:45:28 +00:00
std : : unordered_set < String > paths_corrected ;
for ( const auto & path : paths )
2017-04-01 07:20:54 +00:00
{
2021-02-05 06:45:28 +00:00
const String & path_corrected = pathCorrected ( path ) ;
auto [ it , inserted ] = paths_corrected . emplace ( path_corrected ) ;
if ( ! inserted ) /// Do not repeat processing.
continue ;
zkutil : : Strings nodes = zookeeper - > getChildren ( path_corrected ) ;
String path_part = path_corrected ;
if ( path_part = = " / " )
path_part . clear ( ) ;
std : : vector < std : : future < Coordination : : GetResponse > > futures ;
futures . reserve ( nodes . size ( ) ) ;
for ( const String & node : nodes )
futures . push_back ( zookeeper - > asyncTryGet ( path_part + ' / ' + node ) ) ;
for ( size_t i = 0 , size = nodes . size ( ) ; i < size ; + + i )
{
auto res = futures [ i ] . get ( ) ;
if ( res . error = = Coordination : : Error : : ZNONODE )
continue ; /// Node was deleted meanwhile.
const Coordination : : Stat & stat = res . stat ;
size_t col_num = 0 ;
res_columns [ col_num + + ] - > insert ( nodes [ i ] ) ;
res_columns [ col_num + + ] - > insert ( res . data ) ;
res_columns [ col_num + + ] - > insert ( stat . czxid ) ;
res_columns [ col_num + + ] - > insert ( stat . mzxid ) ;
res_columns [ col_num + + ] - > insert ( UInt64 ( stat . ctime / 1000 ) ) ;
res_columns [ col_num + + ] - > insert ( UInt64 ( stat . mtime / 1000 ) ) ;
res_columns [ col_num + + ] - > insert ( stat . version ) ;
res_columns [ col_num + + ] - > insert ( stat . cversion ) ;
res_columns [ col_num + + ] - > insert ( stat . aversion ) ;
res_columns [ col_num + + ] - > insert ( stat . ephemeralOwner ) ;
res_columns [ col_num + + ] - > insert ( stat . dataLength ) ;
res_columns [ col_num + + ] - > insert ( stat . numChildren ) ;
res_columns [ col_num + + ] - > insert ( stat . pzxid ) ;
res_columns [ col_num + + ] - > insert (
path ) ; /// This is the original path. In order to process the request, condition in WHERE should be triggered.
}
2017-04-01 07:20:54 +00:00
}
2014-10-06 03:49:56 +00:00
}
}