2021-03-29 08:24:56 +00:00
# include <Coordination/KeeperStorage.h>
2020-10-30 19:57:30 +00:00
# include <Common/ZooKeeper/IKeeper.h>
2020-11-03 14:49:30 +00:00
# include <Common/setThreadName.h>
2020-11-04 18:54:55 +00:00
# include <Common/StringUtils/StringUtils.h>
2022-01-10 19:01:41 +00:00
# include <Common/hex.h>
# include <IO/WriteHelpers.h>
# include <IO/Operators.h>
2021-05-21 21:19:22 +00:00
# include <Poco/SHA1Engine.h>
2021-05-22 16:07:47 +00:00
# include <Poco/Base64Encoder.h>
# include <boost/algorithm/string.hpp>
2021-12-30 16:21:49 +00:00
# include <Coordination/pathUtils.h>
2022-01-10 19:01:41 +00:00
# include <sstream>
# include <iomanip>
# include <mutex>
# include <functional>
# include <base/logger_useful.h>
2020-10-30 14:16:47 +00:00
2020-11-11 13:55:28 +00:00
namespace DB
{
2020-11-25 13:19:09 +00:00
namespace ErrorCodes
{
extern const int LOGICAL_ERROR ;
extern const int BAD_ARGUMENTS ;
}
2021-05-22 16:07:47 +00:00
static String base64Encode ( const String & decoded )
{
std : : ostringstream ostr ; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
ostr . exceptions ( std : : ios : : failbit ) ;
Poco : : Base64Encoder encoder ( ostr ) ;
encoder . rdbuf ( ) - > setLineLength ( 0 ) ;
encoder < < decoded ;
encoder . close ( ) ;
return ostr . str ( ) ;
}
2021-05-21 21:19:22 +00:00
static String getSHA1 ( const String & userdata )
{
Poco : : SHA1Engine engine ;
engine . update ( userdata ) ;
const auto & digest_id = engine . digest ( ) ;
return String { digest_id . begin ( ) , digest_id . end ( ) } ;
}
2021-05-22 16:07:47 +00:00
static String generateDigest ( const String & userdata )
{
std : : vector < String > user_password ;
boost : : split ( user_password , userdata , [ ] ( char c ) { return c = = ' : ' ; } ) ;
2021-06-21 13:58:39 +00:00
return user_password [ 0 ] + " : " + base64Encode ( getSHA1 ( userdata ) ) ;
2021-05-22 16:07:47 +00:00
}
static bool checkACL ( int32_t permission , const Coordination : : ACLs & node_acls , const std : : vector < KeeperStorage : : AuthID > & session_auths )
2021-05-21 21:19:22 +00:00
{
if ( node_acls . empty ( ) )
return true ;
2021-05-24 12:18:04 +00:00
for ( const auto & session_auth : session_auths )
2021-05-22 16:07:47 +00:00
if ( session_auth . scheme = = " super " )
return true ;
2021-05-24 15:52:48 +00:00
for ( const auto & node_acl : node_acls )
2021-05-21 21:19:22 +00:00
{
2021-05-24 15:52:48 +00:00
if ( node_acl . permissions & permission )
2021-05-22 16:07:47 +00:00
{
2021-05-24 15:52:48 +00:00
if ( node_acl . scheme = = " world " & & node_acl . id = = " anyone " )
2021-05-22 16:07:47 +00:00
return true ;
2021-05-21 21:19:22 +00:00
2021-05-24 15:52:48 +00:00
for ( const auto & session_auth : session_auths )
2021-06-21 13:58:39 +00:00
{
2021-05-24 15:52:48 +00:00
if ( node_acl . scheme = = session_auth . scheme & & node_acl . id = = session_auth . id )
2021-05-24 12:18:04 +00:00
return true ;
2021-06-21 13:58:39 +00:00
}
2021-05-22 16:07:47 +00:00
}
2021-05-21 21:19:22 +00:00
}
return false ;
}
2021-05-22 16:21:52 +00:00
static bool fixupACL (
const std : : vector < Coordination : : ACL > & request_acls ,
const std : : vector < KeeperStorage : : AuthID > & current_ids ,
2021-12-28 09:17:01 +00:00
std : : vector < Coordination : : ACL > & result_acls )
2021-05-22 16:21:52 +00:00
{
if ( request_acls . empty ( ) )
2021-05-24 12:18:04 +00:00
return true ;
2021-05-22 16:21:52 +00:00
2021-05-24 12:18:04 +00:00
bool valid_found = false ;
2021-05-22 16:21:52 +00:00
for ( const auto & request_acl : request_acls )
{
2021-05-24 12:18:04 +00:00
if ( request_acl . scheme = = " auth " )
2021-05-22 16:21:52 +00:00
{
for ( const auto & current_id : current_ids )
{
2021-05-24 12:18:04 +00:00
valid_found = true ;
2021-05-22 16:21:52 +00:00
Coordination : : ACL new_acl = request_acl ;
new_acl . scheme = current_id . scheme ;
new_acl . id = current_id . id ;
result_acls . push_back ( new_acl ) ;
}
}
2021-05-24 12:18:04 +00:00
else if ( request_acl . scheme = = " world " & & request_acl . id = = " anyone " )
{
2021-05-27 08:15:46 +00:00
/// We don't need to save default ACLs
2021-05-24 12:18:04 +00:00
valid_found = true ;
}
else if ( request_acl . scheme = = " digest " )
{
Coordination : : ACL new_acl = request_acl ;
/// Bad auth
if ( std : : count ( new_acl . id . begin ( ) , new_acl . id . end ( ) , ' : ' ) ! = 1 )
return false ;
valid_found = true ;
result_acls . push_back ( new_acl ) ;
}
2021-05-22 16:21:52 +00:00
}
2021-05-24 12:18:04 +00:00
return valid_found ;
2021-05-22 16:21:52 +00:00
}
2021-03-29 08:24:56 +00:00
static KeeperStorage : : ResponsesForSessions processWatchesImpl ( const String & path , KeeperStorage : : Watches & watches , KeeperStorage : : Watches & list_watches , Coordination : : Event event_type )
2020-11-10 13:43:10 +00:00
{
2021-03-29 08:24:56 +00:00
KeeperStorage : : ResponsesForSessions result ;
2020-12-08 13:28:39 +00:00
auto it = watches . find ( path ) ;
2020-11-10 13:43:10 +00:00
if ( it ! = watches . end ( ) )
{
2020-12-08 13:28:39 +00:00
std : : shared_ptr < Coordination : : ZooKeeperWatchResponse > watch_response = std : : make_shared < Coordination : : ZooKeeperWatchResponse > ( ) ;
watch_response - > path = path ;
2021-01-21 11:07:55 +00:00
watch_response - > xid = Coordination : : WATCH_XID ;
2020-12-08 13:28:39 +00:00
watch_response - > zxid = - 1 ;
watch_response - > type = event_type ;
watch_response - > state = Coordination : : State : : CONNECTED ;
2021-01-19 14:22:28 +00:00
for ( auto watcher_session : it - > second )
2021-03-29 08:24:56 +00:00
result . push_back ( KeeperStorage : : ResponseForSession { watcher_session , watch_response } ) ;
2020-11-10 13:43:10 +00:00
watches . erase ( it ) ;
}
2020-12-08 13:28:39 +00:00
auto parent_path = parentPath ( path ) ;
2021-08-26 11:50:08 +00:00
Strings paths_to_check_for_list_watches ;
if ( event_type = = Coordination : : Event : : CREATED )
2020-11-10 13:43:10 +00:00
{
2022-01-19 11:46:29 +00:00
paths_to_check_for_list_watches . push_back ( parent_path . toString ( ) ) ; /// Trigger list watches for parent
2021-08-26 11:50:08 +00:00
}
else if ( event_type = = Coordination : : Event : : DELETED )
{
paths_to_check_for_list_watches . push_back ( path ) ; /// Trigger both list watches for this path
2022-01-19 11:46:29 +00:00
paths_to_check_for_list_watches . push_back ( parent_path . toString ( ) ) ; /// And for parent path
2021-08-26 11:50:08 +00:00
}
/// CHANGED event never trigger list wathes
2020-11-10 13:43:10 +00:00
2021-08-26 11:50:08 +00:00
for ( const auto & path_to_check : paths_to_check_for_list_watches )
{
it = list_watches . find ( path_to_check ) ;
if ( it ! = list_watches . end ( ) )
{
std : : shared_ptr < Coordination : : ZooKeeperWatchResponse > watch_list_response = std : : make_shared < Coordination : : ZooKeeperWatchResponse > ( ) ;
watch_list_response - > path = path_to_check ;
watch_list_response - > xid = Coordination : : WATCH_XID ;
watch_list_response - > zxid = - 1 ;
if ( path_to_check = = parent_path )
watch_list_response - > type = Coordination : : Event : : CHILD ;
else
watch_list_response - > type = Coordination : : Event : : DELETED ;
watch_list_response - > state = Coordination : : State : : CONNECTED ;
for ( auto watcher_session : it - > second )
result . push_back ( KeeperStorage : : ResponseForSession { watcher_session , watch_list_response } ) ;
list_watches . erase ( it ) ;
}
2020-11-10 13:43:10 +00:00
}
2021-01-19 14:22:28 +00:00
return result ;
2020-11-10 13:43:10 +00:00
}
2021-05-23 17:54:42 +00:00
KeeperStorage : : KeeperStorage ( int64_t tick_time_ms , const String & superdigest_ )
2021-02-03 20:32:15 +00:00
: session_expiry_queue ( tick_time_ms )
2021-05-23 17:54:42 +00:00
, superdigest ( superdigest_ )
2020-11-03 14:49:30 +00:00
{
2021-02-26 13:53:34 +00:00
container . insert ( " / " , Node ( ) ) ;
2020-11-03 14:49:30 +00:00
}
2020-10-30 19:57:30 +00:00
using Undo = std : : function < void ( ) > ;
2021-08-24 12:30:31 +00:00
struct KeeperStorageRequestProcessor
2020-10-30 19:57:30 +00:00
{
Coordination : : ZooKeeperRequestPtr zk_request ;
2021-08-24 12:30:31 +00:00
explicit KeeperStorageRequestProcessor ( const Coordination : : ZooKeeperRequestPtr & zk_request_ )
2020-10-30 19:57:30 +00:00
: zk_request ( zk_request_ )
{ }
2022-01-06 13:14:45 +00:00
virtual std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & storage , int64_t zxid , int64_t session_id , int64_t time ) const = 0 ;
2021-03-29 08:24:56 +00:00
virtual KeeperStorage : : ResponsesForSessions processWatches ( KeeperStorage : : Watches & /*watches*/ , KeeperStorage : : Watches & /*list_watches*/ ) const { return { } ; }
2021-05-21 21:19:22 +00:00
virtual bool checkAuth ( KeeperStorage & /*storage*/ , int64_t /*session_id*/ ) const { return true ; }
2020-11-10 13:43:10 +00:00
2021-08-24 12:30:31 +00:00
virtual ~ KeeperStorageRequestProcessor ( ) = default ;
2020-11-20 12:36:10 +00:00
} ;
2020-10-30 19:57:30 +00:00
2021-08-24 12:30:31 +00:00
struct KeeperStorageHeartbeatRequestProcessor final : public KeeperStorageRequestProcessor
2020-11-03 14:49:30 +00:00
{
2021-08-24 12:30:31 +00:00
using KeeperStorageRequestProcessor : : KeeperStorageRequestProcessor ;
2022-01-06 13:14:45 +00:00
std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & /* storage */ , int64_t /* zxid */ , int64_t /* session_id */ , int64_t /* time */ ) const override
2020-11-03 14:49:30 +00:00
{
return { zk_request - > makeResponse ( ) , { } } ;
}
} ;
2021-08-24 12:30:31 +00:00
struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProcessor
2021-02-10 13:01:05 +00:00
{
2021-08-24 12:30:31 +00:00
using KeeperStorageRequestProcessor : : KeeperStorageRequestProcessor ;
2022-01-06 13:14:45 +00:00
std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & /* storage */ , int64_t /* zxid */ , int64_t /* session_id */ , int64_t /* time */ ) const override
2021-02-10 13:01:05 +00:00
{
auto response = zk_request - > makeResponse ( ) ;
2021-05-08 14:10:06 +00:00
dynamic_cast < Coordination : : ZooKeeperSyncResponse & > ( * response ) . path
= dynamic_cast < Coordination : : ZooKeeperSyncRequest & > ( * zk_request ) . path ;
2021-02-10 13:01:05 +00:00
return { response , { } } ;
}
} ;
2021-08-24 12:30:31 +00:00
struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestProcessor
2020-10-30 19:57:30 +00:00
{
2021-08-24 12:30:31 +00:00
using KeeperStorageRequestProcessor : : KeeperStorageRequestProcessor ;
2020-11-10 13:43:10 +00:00
2021-03-29 08:24:56 +00:00
KeeperStorage : : ResponsesForSessions processWatches ( KeeperStorage : : Watches & watches , KeeperStorage : : Watches & list_watches ) const override
2020-11-10 13:43:10 +00:00
{
2021-01-19 14:22:28 +00:00
return processWatchesImpl ( zk_request - > getPath ( ) , watches , list_watches , Coordination : : Event : : CREATED ) ;
2020-11-10 13:43:10 +00:00
}
2021-05-21 21:19:22 +00:00
bool checkAuth ( KeeperStorage & storage , int64_t session_id ) const override
2020-10-30 19:57:30 +00:00
{
2021-05-21 21:19:22 +00:00
auto & container = storage . container ;
2022-01-22 15:29:36 +00:00
auto path = zk_request - > getPath ( ) ;
auto parent_path = parentPath ( path ) ;
2021-05-21 21:19:22 +00:00
auto it = container . find ( parent_path ) ;
if ( it = = container . end ( ) )
return true ;
2021-05-28 11:52:19 +00:00
const auto & node_acls = storage . acl_map . convertNumber ( it - > value . acl_id ) ;
2021-05-27 08:15:46 +00:00
if ( node_acls . empty ( ) )
return true ;
2021-05-21 21:19:22 +00:00
const auto & session_auths = storage . session_and_auth [ session_id ] ;
return checkACL ( Coordination : : ACL : : Create , node_acls , session_auths ) ;
}
2022-01-06 13:14:45 +00:00
std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & storage , int64_t zxid , int64_t session_id , int64_t time ) const override
2021-05-21 21:19:22 +00:00
{
auto & container = storage . container ;
auto & ephemerals = storage . ephemerals ;
2020-10-30 19:57:30 +00:00
Coordination : : ZooKeeperResponsePtr response_ptr = zk_request - > makeResponse ( ) ;
Undo undo ;
Coordination : : ZooKeeperCreateResponse & response = dynamic_cast < Coordination : : ZooKeeperCreateResponse & > ( * response_ptr ) ;
Coordination : : ZooKeeperCreateRequest & request = dynamic_cast < Coordination : : ZooKeeperCreateRequest & > ( * zk_request ) ;
2021-08-03 09:59:08 +00:00
auto parent_path = parentPath ( request . path ) ;
auto it = container . find ( parent_path ) ;
if ( it = = container . end ( ) )
{
response . error = Coordination : : Error : : ZNONODE ;
return { response_ptr , undo } ;
}
else if ( it - > value . stat . ephemeralOwner ! = 0 )
{
response . error = Coordination : : Error : : ZNOCHILDRENFOREPHEMERALS ;
return { response_ptr , undo } ;
}
std : : string path_created = request . path ;
if ( request . is_sequential )
{
auto seq_num = it - > value . seq_num ;
std : : stringstream seq_num_str ; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
seq_num_str . exceptions ( std : : ios : : failbit ) ;
seq_num_str < < std : : setw ( 10 ) < < std : : setfill ( ' 0 ' ) < < seq_num ;
path_created + = seq_num_str . str ( ) ;
}
if ( container . contains ( path_created ) )
2020-10-30 19:57:30 +00:00
{
response . error = Coordination : : Error : : ZNODEEXISTS ;
2021-08-03 09:59:08 +00:00
return { response_ptr , undo } ;
2020-10-30 19:57:30 +00:00
}
2022-01-19 11:46:29 +00:00
if ( getBaseName ( path_created ) . size = = 0 )
2020-10-30 19:57:30 +00:00
{
2021-08-03 09:59:08 +00:00
response . error = Coordination : : Error : : ZBADARGUMENTS ;
return { response_ptr , undo } ;
}
2020-10-30 19:57:30 +00:00
2021-08-03 09:59:08 +00:00
auto & session_auth_ids = storage . session_and_auth [ session_id ] ;
KeeperStorage : : Node created_node ;
Coordination : : ACLs node_acls ;
2021-12-28 09:17:01 +00:00
if ( ! fixupACL ( request . acls , session_auth_ids , node_acls ) )
2021-08-03 09:59:08 +00:00
{
response . error = Coordination : : Error : : ZINVALIDACL ;
return { response_ptr , { } } ;
2020-10-30 19:57:30 +00:00
}
2021-08-03 09:59:08 +00:00
uint64_t acl_id = storage . acl_map . convertACLs ( node_acls ) ;
storage . acl_map . addUsage ( acl_id ) ;
created_node . acl_id = acl_id ;
created_node . stat . czxid = zxid ;
created_node . stat . mzxid = zxid ;
created_node . stat . pzxid = zxid ;
2022-01-06 13:14:45 +00:00
created_node . stat . ctime = time ;
created_node . stat . mtime = time ;
2021-08-03 09:59:08 +00:00
created_node . stat . numChildren = 0 ;
created_node . stat . dataLength = request . data . length ( ) ;
created_node . stat . ephemeralOwner = request . is_ephemeral ? session_id : 0 ;
created_node . data = request . data ;
created_node . is_sequental = request . is_sequential ;
2022-03-02 17:22:12 +00:00
auto [ map_key , _ ] = container . insert ( path_created , created_node ) ;
2022-01-21 14:26:50 +00:00
/// Take child path from key owned by map.
2022-01-21 13:35:28 +00:00
auto child_path = getBaseName ( map_key - > getKey ( ) ) ;
2022-01-19 11:46:29 +00:00
2021-08-03 09:59:08 +00:00
int32_t parent_cversion = request . parent_cversion ;
int64_t prev_parent_zxid ;
int32_t prev_parent_cversion ;
container . updateValue ( parent_path , [ child_path , zxid , & prev_parent_zxid ,
parent_cversion , & prev_parent_cversion ] ( KeeperStorage : : Node & parent )
{
parent . children . insert ( child_path ) ;
2022-01-19 11:46:29 +00:00
parent . size_bytes + = child_path . size ;
2021-08-03 09:59:08 +00:00
prev_parent_cversion = parent . stat . cversion ;
prev_parent_zxid = parent . stat . pzxid ;
/// Increment sequential number even if node is not sequential
+ + parent . seq_num ;
if ( parent_cversion = = - 1 )
+ + parent . stat . cversion ;
else if ( parent_cversion > parent . stat . cversion )
parent . stat . cversion = parent_cversion ;
if ( zxid > parent . stat . pzxid )
parent . stat . pzxid = zxid ;
+ + parent . stat . numChildren ;
} ) ;
response . path_created = path_created ;
if ( request . is_ephemeral )
ephemerals [ session_id ] . emplace ( path_created ) ;
undo = [ & storage , prev_parent_zxid , prev_parent_cversion , session_id , path_created , is_ephemeral = request . is_ephemeral , parent_path , child_path , acl_id ]
{
storage . acl_map . removeUsage ( acl_id ) ;
if ( is_ephemeral )
storage . ephemerals [ session_id ] . erase ( path_created ) ;
storage . container . updateValue ( parent_path , [ child_path , prev_parent_zxid , prev_parent_cversion ] ( KeeperStorage : : Node & undo_parent )
{
- - undo_parent . stat . numChildren ;
- - undo_parent . seq_num ;
undo_parent . stat . cversion = prev_parent_cversion ;
undo_parent . stat . pzxid = prev_parent_zxid ;
undo_parent . children . erase ( child_path ) ;
2022-01-19 11:46:29 +00:00
undo_parent . size_bytes - = child_path . size ;
2021-08-03 09:59:08 +00:00
} ) ;
2022-01-19 11:46:29 +00:00
storage . container . erase ( path_created ) ;
2021-08-03 09:59:08 +00:00
} ;
response . error = Coordination : : Error : : ZOK ;
2020-10-30 19:57:30 +00:00
return { response_ptr , undo } ;
}
} ;
2021-08-24 12:30:31 +00:00
struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProcessor
2020-11-03 14:49:30 +00:00
{
2021-05-21 21:19:22 +00:00
bool checkAuth ( KeeperStorage & storage , int64_t session_id ) const override
{
auto & container = storage . container ;
2021-05-22 07:38:50 +00:00
auto it = container . find ( zk_request - > getPath ( ) ) ;
2021-05-21 21:19:22 +00:00
if ( it = = container . end ( ) )
return true ;
2021-05-28 11:52:19 +00:00
const auto & node_acls = storage . acl_map . convertNumber ( it - > value . acl_id ) ;
2021-05-27 08:15:46 +00:00
if ( node_acls . empty ( ) )
return true ;
2021-05-21 21:19:22 +00:00
const auto & session_auths = storage . session_and_auth [ session_id ] ;
return checkACL ( Coordination : : ACL : : Read , node_acls , session_auths ) ;
}
2021-08-24 12:30:31 +00:00
using KeeperStorageRequestProcessor : : KeeperStorageRequestProcessor ;
2022-01-06 13:14:45 +00:00
std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & storage , int64_t /* zxid */ , int64_t /* session_id */ , int64_t /* time */ ) const override
2020-11-03 14:49:30 +00:00
{
2021-05-21 21:19:22 +00:00
auto & container = storage . container ;
2020-11-03 14:49:30 +00:00
Coordination : : ZooKeeperResponsePtr response_ptr = zk_request - > makeResponse ( ) ;
Coordination : : ZooKeeperGetResponse & response = dynamic_cast < Coordination : : ZooKeeperGetResponse & > ( * response_ptr ) ;
Coordination : : ZooKeeperGetRequest & request = dynamic_cast < Coordination : : ZooKeeperGetRequest & > ( * zk_request ) ;
auto it = container . find ( request . path ) ;
if ( it = = container . end ( ) )
{
response . error = Coordination : : Error : : ZNONODE ;
}
else
{
2021-02-26 13:53:34 +00:00
response . stat = it - > value . stat ;
response . data = it - > value . data ;
2020-11-03 14:49:30 +00:00
response . error = Coordination : : Error : : ZOK ;
}
return { response_ptr , { } } ;
}
} ;
2021-07-10 08:42:25 +00:00
namespace
{
/// Garbage required to apply log to "fuzzy" zookeeper snapshot
void updateParentPzxid ( const std : : string & child_path , int64_t zxid , KeeperStorage : : Container & container )
{
auto parent_path = parentPath ( child_path ) ;
auto parent_it = container . find ( parent_path ) ;
if ( parent_it ! = container . end ( ) )
{
container . updateValue ( parent_path , [ zxid ] ( KeeperStorage : : Node & parent )
{
if ( parent . stat . pzxid < zxid )
parent . stat . pzxid = zxid ;
} ) ;
}
}
}
2021-08-24 12:30:31 +00:00
struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor
2020-11-04 18:54:55 +00:00
{
2021-05-21 21:19:22 +00:00
bool checkAuth ( KeeperStorage & storage , int64_t session_id ) const override
{
auto & container = storage . container ;
2021-05-24 12:18:04 +00:00
auto it = container . find ( parentPath ( zk_request - > getPath ( ) ) ) ;
2021-05-21 21:19:22 +00:00
if ( it = = container . end ( ) )
return true ;
2021-05-28 11:52:19 +00:00
const auto & node_acls = storage . acl_map . convertNumber ( it - > value . acl_id ) ;
2021-05-27 08:15:46 +00:00
if ( node_acls . empty ( ) )
return true ;
2021-05-21 21:19:22 +00:00
const auto & session_auths = storage . session_and_auth [ session_id ] ;
return checkACL ( Coordination : : ACL : : Delete , node_acls , session_auths ) ;
}
2021-08-24 12:30:31 +00:00
using KeeperStorageRequestProcessor : : KeeperStorageRequestProcessor ;
2022-01-06 13:14:45 +00:00
std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & storage , int64_t zxid , int64_t /*session_id*/ , int64_t /* time */ ) const override
2020-11-04 18:54:55 +00:00
{
2021-05-21 21:19:22 +00:00
auto & container = storage . container ;
auto & ephemerals = storage . ephemerals ;
2020-11-04 18:54:55 +00:00
Coordination : : ZooKeeperResponsePtr response_ptr = zk_request - > makeResponse ( ) ;
Coordination : : ZooKeeperRemoveResponse & response = dynamic_cast < Coordination : : ZooKeeperRemoveResponse & > ( * response_ptr ) ;
Coordination : : ZooKeeperRemoveRequest & request = dynamic_cast < Coordination : : ZooKeeperRemoveRequest & > ( * zk_request ) ;
Undo undo ;
auto it = container . find ( request . path ) ;
if ( it = = container . end ( ) )
{
2021-07-09 13:00:50 +00:00
if ( request . restored_from_zookeeper_log )
updateParentPzxid ( request . path , zxid , container ) ;
2020-11-04 18:54:55 +00:00
response . error = Coordination : : Error : : ZNONODE ;
}
2021-02-26 13:53:34 +00:00
else if ( request . version ! = - 1 & & request . version ! = it - > value . stat . version )
2020-11-04 18:54:55 +00:00
{
response . error = Coordination : : Error : : ZBADVERSION ;
}
2021-02-26 13:53:34 +00:00
else if ( it - > value . stat . numChildren )
2020-11-04 18:54:55 +00:00
{
response . error = Coordination : : Error : : ZNOTEMPTY ;
}
else
{
2021-07-09 13:00:50 +00:00
if ( request . restored_from_zookeeper_log )
updateParentPzxid ( request . path , zxid , container ) ;
2021-02-26 13:53:34 +00:00
auto prev_node = it - > value ;
2021-03-03 12:29:00 +00:00
if ( prev_node . stat . ephemeralOwner ! = 0 )
2021-03-24 08:12:37 +00:00
{
auto ephemerals_it = ephemerals . find ( prev_node . stat . ephemeralOwner ) ;
ephemerals_it - > second . erase ( request . path ) ;
if ( ephemerals_it - > second . empty ( ) )
ephemerals . erase ( ephemerals_it ) ;
}
2020-11-19 16:06:19 +00:00
2021-05-28 11:52:19 +00:00
storage . acl_map . removeUsage ( prev_node . acl_id ) ;
2022-01-21 14:26:50 +00:00
container . updateValue ( parentPath ( request . path ) , [ child_basename = getBaseName ( it - > key ) ] ( KeeperStorage : : Node & parent )
2021-02-26 13:53:34 +00:00
{
- - parent . stat . numChildren ;
+ + parent . stat . cversion ;
parent . children . erase ( child_basename ) ;
2022-01-19 11:46:29 +00:00
parent . size_bytes - = child_basename . size ;
2021-02-26 13:53:34 +00:00
} ) ;
2020-11-04 18:54:55 +00:00
response . error = Coordination : : Error : : ZOK ;
2022-01-21 14:26:50 +00:00
/// Erase full path from container after child removed from parent
2021-02-26 13:53:34 +00:00
container . erase ( request . path ) ;
2021-02-19 07:05:52 +00:00
2022-01-19 11:46:29 +00:00
undo = [ prev_node , & storage , path = request . path ]
2020-11-04 18:54:55 +00:00
{
2021-03-03 12:29:00 +00:00
if ( prev_node . stat . ephemeralOwner ! = 0 )
2021-05-28 11:52:19 +00:00
storage . ephemerals [ prev_node . stat . ephemeralOwner ] . emplace ( path ) ;
storage . acl_map . addUsage ( prev_node . acl_id ) ;
2020-11-20 08:37:16 +00:00
2022-01-21 14:26:50 +00:00
/// Dangerous place: we are adding StringRef to child into children unordered_hash set.
/// That's why we are taking getBaseName from inserted key, not from the path from request object.
2022-01-19 11:46:29 +00:00
auto [ map_key , _ ] = storage . container . insert ( path , prev_node ) ;
2022-01-21 13:35:28 +00:00
storage . container . updateValue ( parentPath ( path ) , [ child_name = getBaseName ( map_key - > getKey ( ) ) ] ( KeeperStorage : : Node & parent )
2021-02-26 13:53:34 +00:00
{
+ + parent . stat . numChildren ;
- - parent . stat . cversion ;
2022-01-19 11:46:29 +00:00
parent . children . insert ( child_name ) ;
parent . size_bytes + = child_name . size ;
2021-02-26 13:53:34 +00:00
} ) ;
2020-11-04 18:54:55 +00:00
} ;
}
return { response_ptr , undo } ;
}
2020-11-10 13:43:10 +00:00
2021-03-29 08:24:56 +00:00
KeeperStorage : : ResponsesForSessions processWatches ( KeeperStorage : : Watches & watches , KeeperStorage : : Watches & list_watches ) const override
2020-11-10 13:43:10 +00:00
{
2021-01-19 14:22:28 +00:00
return processWatchesImpl ( zk_request - > getPath ( ) , watches , list_watches , Coordination : : Event : : DELETED ) ;
2020-11-10 13:43:10 +00:00
}
2020-11-04 18:54:55 +00:00
} ;
2021-08-24 12:30:31 +00:00
struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestProcessor
2020-11-04 18:54:55 +00:00
{
2021-08-24 12:30:31 +00:00
using KeeperStorageRequestProcessor : : KeeperStorageRequestProcessor ;
2022-01-06 13:14:45 +00:00
std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & storage , int64_t /*zxid*/ , int64_t /* session_id */ , int64_t /* time */ ) const override
2020-11-04 18:54:55 +00:00
{
2021-05-21 21:19:22 +00:00
auto & container = storage . container ;
2020-11-04 18:54:55 +00:00
Coordination : : ZooKeeperResponsePtr response_ptr = zk_request - > makeResponse ( ) ;
Coordination : : ZooKeeperExistsResponse & response = dynamic_cast < Coordination : : ZooKeeperExistsResponse & > ( * response_ptr ) ;
Coordination : : ZooKeeperExistsRequest & request = dynamic_cast < Coordination : : ZooKeeperExistsRequest & > ( * zk_request ) ;
auto it = container . find ( request . path ) ;
if ( it ! = container . end ( ) )
{
2021-02-26 13:53:34 +00:00
response . stat = it - > value . stat ;
2020-11-04 18:54:55 +00:00
response . error = Coordination : : Error : : ZOK ;
}
else
{
response . error = Coordination : : Error : : ZNONODE ;
}
return { response_ptr , { } } ;
}
} ;
2021-08-24 12:30:31 +00:00
struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProcessor
2020-11-04 18:54:55 +00:00
{
2021-05-21 21:19:22 +00:00
bool checkAuth ( KeeperStorage & storage , int64_t session_id ) const override
{
auto & container = storage . container ;
2021-05-22 07:38:50 +00:00
auto it = container . find ( zk_request - > getPath ( ) ) ;
2021-05-21 21:19:22 +00:00
if ( it = = container . end ( ) )
return true ;
2021-05-28 11:52:19 +00:00
const auto & node_acls = storage . acl_map . convertNumber ( it - > value . acl_id ) ;
2021-05-27 08:15:46 +00:00
if ( node_acls . empty ( ) )
return true ;
2021-05-21 21:19:22 +00:00
const auto & session_auths = storage . session_and_auth [ session_id ] ;
return checkACL ( Coordination : : ACL : : Write , node_acls , session_auths ) ;
}
2021-08-24 12:30:31 +00:00
using KeeperStorageRequestProcessor : : KeeperStorageRequestProcessor ;
2022-01-06 13:14:45 +00:00
std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & storage , int64_t zxid , int64_t /* session_id */ , int64_t time ) const override
2020-11-04 18:54:55 +00:00
{
2021-05-21 21:19:22 +00:00
auto & container = storage . container ;
2020-11-04 18:54:55 +00:00
Coordination : : ZooKeeperResponsePtr response_ptr = zk_request - > makeResponse ( ) ;
Coordination : : ZooKeeperSetResponse & response = dynamic_cast < Coordination : : ZooKeeperSetResponse & > ( * response_ptr ) ;
Coordination : : ZooKeeperSetRequest & request = dynamic_cast < Coordination : : ZooKeeperSetRequest & > ( * zk_request ) ;
Undo undo ;
auto it = container . find ( request . path ) ;
if ( it = = container . end ( ) )
{
response . error = Coordination : : Error : : ZNONODE ;
}
2021-02-26 13:53:34 +00:00
else if ( request . version = = - 1 | | request . version = = it - > value . stat . version )
2020-11-04 18:54:55 +00:00
{
2021-06-18 18:36:19 +00:00
2021-02-26 13:53:34 +00:00
auto prev_node = it - > value ;
2022-01-06 13:14:45 +00:00
auto itr = container . updateValue ( request . path , [ zxid , request , time ] ( KeeperStorage : : Node & value )
2021-02-26 13:53:34 +00:00
{
value . stat . version + + ;
value . stat . mzxid = zxid ;
2022-01-06 13:14:45 +00:00
value . stat . mtime = time ;
2021-02-26 13:53:34 +00:00
value . stat . dataLength = request . data . length ( ) ;
2021-12-16 01:25:38 +00:00
value . size_bytes = value . size_bytes + request . data . size ( ) - value . data . size ( ) ;
2021-02-26 13:53:34 +00:00
value . data = request . data ;
} ) ;
2021-03-29 08:24:56 +00:00
container . updateValue ( parentPath ( request . path ) , [ ] ( KeeperStorage : : Node & parent )
2021-02-26 13:53:34 +00:00
{
parent . stat . cversion + + ;
} ) ;
response . stat = itr - > value . stat ;
2020-11-04 18:54:55 +00:00
response . error = Coordination : : Error : : ZOK ;
undo = [ prev_node , & container , path = request . path ]
{
2021-03-29 08:24:56 +00:00
container . updateValue ( path , [ & prev_node ] ( KeeperStorage : : Node & value ) { value = prev_node ; } ) ;
container . updateValue ( parentPath ( path ) , [ ] ( KeeperStorage : : Node & parent )
2021-02-26 13:53:34 +00:00
{
parent . stat . cversion - - ;
} ) ;
2020-11-04 18:54:55 +00:00
} ;
}
else
{
response . error = Coordination : : Error : : ZBADVERSION ;
}
2020-11-11 13:07:06 +00:00
return { response_ptr , undo } ;
2020-11-04 18:54:55 +00:00
}
2020-11-10 13:43:10 +00:00
2021-03-29 08:24:56 +00:00
KeeperStorage : : ResponsesForSessions processWatches ( KeeperStorage : : Watches & watches , KeeperStorage : : Watches & list_watches ) const override
2020-11-10 13:43:10 +00:00
{
2021-01-19 14:22:28 +00:00
return processWatchesImpl ( zk_request - > getPath ( ) , watches , list_watches , Coordination : : Event : : CHANGED ) ;
2020-11-10 13:43:10 +00:00
}
2020-11-04 18:54:55 +00:00
} ;
2021-08-24 12:30:31 +00:00
struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProcessor
2020-11-04 18:54:55 +00:00
{
2021-05-21 21:19:22 +00:00
bool checkAuth ( KeeperStorage & storage , int64_t session_id ) const override
{
auto & container = storage . container ;
2021-05-22 07:38:50 +00:00
auto it = container . find ( zk_request - > getPath ( ) ) ;
2021-05-21 21:19:22 +00:00
if ( it = = container . end ( ) )
return true ;
2021-05-28 11:52:19 +00:00
const auto & node_acls = storage . acl_map . convertNumber ( it - > value . acl_id ) ;
2021-05-27 08:15:46 +00:00
if ( node_acls . empty ( ) )
return true ;
2021-05-21 21:19:22 +00:00
const auto & session_auths = storage . session_and_auth [ session_id ] ;
return checkACL ( Coordination : : ACL : : Read , node_acls , session_auths ) ;
}
2021-08-24 12:30:31 +00:00
using KeeperStorageRequestProcessor : : KeeperStorageRequestProcessor ;
2022-01-06 13:14:45 +00:00
std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & storage , int64_t /*zxid*/ , int64_t /*session_id*/ , int64_t /* time */ ) const override
2020-11-04 18:54:55 +00:00
{
2021-05-21 21:19:22 +00:00
auto & container = storage . container ;
2020-11-04 18:54:55 +00:00
Coordination : : ZooKeeperResponsePtr response_ptr = zk_request - > makeResponse ( ) ;
Coordination : : ZooKeeperListResponse & response = dynamic_cast < Coordination : : ZooKeeperListResponse & > ( * response_ptr ) ;
Coordination : : ZooKeeperListRequest & request = dynamic_cast < Coordination : : ZooKeeperListRequest & > ( * zk_request ) ;
2022-01-22 19:36:23 +00:00
2020-11-04 18:54:55 +00:00
auto it = container . find ( request . path ) ;
if ( it = = container . end ( ) )
{
response . error = Coordination : : Error : : ZNONODE ;
}
else
{
auto path_prefix = request . path ;
if ( path_prefix . empty ( ) )
2020-11-25 13:19:09 +00:00
throw DB : : Exception ( " Logical error: path cannot be empty " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-11-04 18:54:55 +00:00
2022-01-19 11:46:29 +00:00
response . names . reserve ( it - > value . children . size ( ) ) ;
for ( const auto child : it - > value . children )
response . names . push_back ( child . toString ( ) ) ;
2020-11-04 18:54:55 +00:00
2021-02-26 13:53:34 +00:00
response . stat = it - > value . stat ;
2020-11-04 18:54:55 +00:00
response . error = Coordination : : Error : : ZOK ;
}
return { response_ptr , { } } ;
}
} ;
2021-08-24 12:30:31 +00:00
struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestProcessor
2020-11-04 18:54:55 +00:00
{
2021-05-21 21:19:22 +00:00
bool checkAuth ( KeeperStorage & storage , int64_t session_id ) const override
{
auto & container = storage . container ;
2021-05-22 07:38:50 +00:00
auto it = container . find ( zk_request - > getPath ( ) ) ;
2021-05-21 21:19:22 +00:00
if ( it = = container . end ( ) )
return true ;
2021-05-28 11:52:19 +00:00
const auto & node_acls = storage . acl_map . convertNumber ( it - > value . acl_id ) ;
2021-05-27 08:15:46 +00:00
if ( node_acls . empty ( ) )
return true ;
2021-05-21 21:19:22 +00:00
const auto & session_auths = storage . session_and_auth [ session_id ] ;
return checkACL ( Coordination : : ACL : : Read , node_acls , session_auths ) ;
}
2021-08-24 12:30:31 +00:00
using KeeperStorageRequestProcessor : : KeeperStorageRequestProcessor ;
2022-01-06 13:14:45 +00:00
std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & storage , int64_t /*zxid*/ , int64_t /*session_id*/ , int64_t /* time */ ) const override
2020-11-04 18:54:55 +00:00
{
2021-05-21 21:19:22 +00:00
auto & container = storage . container ;
2020-11-04 18:54:55 +00:00
Coordination : : ZooKeeperResponsePtr response_ptr = zk_request - > makeResponse ( ) ;
Coordination : : ZooKeeperCheckResponse & response = dynamic_cast < Coordination : : ZooKeeperCheckResponse & > ( * response_ptr ) ;
Coordination : : ZooKeeperCheckRequest & request = dynamic_cast < Coordination : : ZooKeeperCheckRequest & > ( * zk_request ) ;
auto it = container . find ( request . path ) ;
if ( it = = container . end ( ) )
{
response . error = Coordination : : Error : : ZNONODE ;
}
2021-02-26 13:53:34 +00:00
else if ( request . version ! = - 1 & & request . version ! = it - > value . stat . version )
2020-11-04 18:54:55 +00:00
{
response . error = Coordination : : Error : : ZBADVERSION ;
}
else
{
response . error = Coordination : : Error : : ZOK ;
}
return { response_ptr , { } } ;
}
} ;
2021-06-22 10:49:35 +00:00
2021-08-24 12:30:31 +00:00
struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestProcessor
2021-06-22 10:49:35 +00:00
{
bool checkAuth ( KeeperStorage & storage , int64_t session_id ) const override
{
auto & container = storage . container ;
auto it = container . find ( zk_request - > getPath ( ) ) ;
if ( it = = container . end ( ) )
return true ;
const auto & node_acls = storage . acl_map . convertNumber ( it - > value . acl_id ) ;
if ( node_acls . empty ( ) )
return true ;
const auto & session_auths = storage . session_and_auth [ session_id ] ;
return checkACL ( Coordination : : ACL : : Admin , node_acls , session_auths ) ;
}
2021-08-24 12:30:31 +00:00
using KeeperStorageRequestProcessor : : KeeperStorageRequestProcessor ;
2021-06-22 10:49:35 +00:00
2022-01-06 13:14:45 +00:00
std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & storage , int64_t /*zxid*/ , int64_t session_id , int64_t /* time */ ) const override
2021-06-22 10:49:35 +00:00
{
auto & container = storage . container ;
Coordination : : ZooKeeperResponsePtr response_ptr = zk_request - > makeResponse ( ) ;
Coordination : : ZooKeeperSetACLResponse & response = dynamic_cast < Coordination : : ZooKeeperSetACLResponse & > ( * response_ptr ) ;
Coordination : : ZooKeeperSetACLRequest & request = dynamic_cast < Coordination : : ZooKeeperSetACLRequest & > ( * zk_request ) ;
auto it = container . find ( request . path ) ;
if ( it = = container . end ( ) )
{
response . error = Coordination : : Error : : ZNONODE ;
}
else if ( request . version ! = - 1 & & request . version ! = it - > value . stat . aversion )
{
response . error = Coordination : : Error : : ZBADVERSION ;
}
else
{
auto & session_auth_ids = storage . session_and_auth [ session_id ] ;
Coordination : : ACLs node_acls ;
2021-12-28 09:17:01 +00:00
if ( ! fixupACL ( request . acls , session_auth_ids , node_acls ) )
2021-06-22 10:49:35 +00:00
{
response . error = Coordination : : Error : : ZINVALIDACL ;
return { response_ptr , { } } ;
}
uint64_t acl_id = storage . acl_map . convertACLs ( node_acls ) ;
storage . acl_map . addUsage ( acl_id ) ;
storage . container . updateValue ( request . path , [ acl_id ] ( KeeperStorage : : Node & node )
{
node . acl_id = acl_id ;
+ + node . stat . aversion ;
} ) ;
response . stat = it - > value . stat ;
response . error = Coordination : : Error : : ZOK ;
}
/// It cannot be used insied multitransaction?
return { response_ptr , { } } ;
}
} ;
2021-08-24 12:30:31 +00:00
struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestProcessor
2021-06-22 10:49:35 +00:00
{
bool checkAuth ( KeeperStorage & storage , int64_t session_id ) const override
{
auto & container = storage . container ;
auto it = container . find ( zk_request - > getPath ( ) ) ;
if ( it = = container . end ( ) )
return true ;
const auto & node_acls = storage . acl_map . convertNumber ( it - > value . acl_id ) ;
if ( node_acls . empty ( ) )
return true ;
const auto & session_auths = storage . session_and_auth [ session_id ] ;
/// LOL, GetACL require more permissions, then SetACL...
return checkACL ( Coordination : : ACL : : Admin | Coordination : : ACL : : Read , node_acls , session_auths ) ;
}
2021-08-24 12:30:31 +00:00
using KeeperStorageRequestProcessor : : KeeperStorageRequestProcessor ;
2021-06-22 10:49:35 +00:00
2022-01-06 13:14:45 +00:00
std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & storage , int64_t /*zxid*/ , int64_t /*session_id*/ , int64_t /* time */ ) const override
2021-06-22 10:49:35 +00:00
{
Coordination : : ZooKeeperResponsePtr response_ptr = zk_request - > makeResponse ( ) ;
Coordination : : ZooKeeperGetACLResponse & response = dynamic_cast < Coordination : : ZooKeeperGetACLResponse & > ( * response_ptr ) ;
Coordination : : ZooKeeperGetACLRequest & request = dynamic_cast < Coordination : : ZooKeeperGetACLRequest & > ( * zk_request ) ;
auto & container = storage . container ;
auto it = container . find ( request . path ) ;
if ( it = = container . end ( ) )
{
response . error = Coordination : : Error : : ZNONODE ;
}
else
{
response . stat = it - > value . stat ;
response . acl = storage . acl_map . convertNumber ( it - > value . acl_id ) ;
}
return { response_ptr , { } } ;
}
} ;
2021-08-24 12:30:31 +00:00
struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestProcessor
2020-11-04 18:54:55 +00:00
{
2021-05-21 21:19:22 +00:00
bool checkAuth ( KeeperStorage & storage , int64_t session_id ) const override
{
for ( const auto & concrete_request : concrete_requests )
if ( ! concrete_request - > checkAuth ( storage , session_id ) )
return false ;
return true ;
}
2021-08-24 12:30:31 +00:00
std : : vector < KeeperStorageRequestProcessorPtr > concrete_requests ;
explicit KeeperStorageMultiRequestProcessor ( const Coordination : : ZooKeeperRequestPtr & zk_request_ )
: KeeperStorageRequestProcessor ( zk_request_ )
2020-11-04 18:54:55 +00:00
{
Coordination : : ZooKeeperMultiRequest & request = dynamic_cast < Coordination : : ZooKeeperMultiRequest & > ( * zk_request ) ;
concrete_requests . reserve ( request . requests . size ( ) ) ;
2020-11-25 13:19:09 +00:00
for ( const auto & sub_request : request . requests )
2020-11-04 18:54:55 +00:00
{
2020-12-10 21:33:13 +00:00
auto sub_zk_request = std : : dynamic_pointer_cast < Coordination : : ZooKeeperRequest > ( sub_request ) ;
2020-11-25 13:19:09 +00:00
if ( sub_zk_request - > getOpNum ( ) = = Coordination : : OpNum : : Create )
2020-11-04 18:54:55 +00:00
{
2021-08-24 12:30:31 +00:00
concrete_requests . push_back ( std : : make_shared < KeeperStorageCreateRequestProcessor > ( sub_zk_request ) ) ;
2020-11-04 18:54:55 +00:00
}
2020-11-25 13:19:09 +00:00
else if ( sub_zk_request - > getOpNum ( ) = = Coordination : : OpNum : : Remove )
2020-11-04 18:54:55 +00:00
{
2021-08-24 12:30:31 +00:00
concrete_requests . push_back ( std : : make_shared < KeeperStorageRemoveRequestProcessor > ( sub_zk_request ) ) ;
2020-11-04 18:54:55 +00:00
}
2020-11-25 13:19:09 +00:00
else if ( sub_zk_request - > getOpNum ( ) = = Coordination : : OpNum : : Set )
2020-11-04 18:54:55 +00:00
{
2021-08-24 12:30:31 +00:00
concrete_requests . push_back ( std : : make_shared < KeeperStorageSetRequestProcessor > ( sub_zk_request ) ) ;
2020-11-04 18:54:55 +00:00
}
2020-11-25 13:19:09 +00:00
else if ( sub_zk_request - > getOpNum ( ) = = Coordination : : OpNum : : Check )
2020-11-04 18:54:55 +00:00
{
2021-08-24 12:30:31 +00:00
concrete_requests . push_back ( std : : make_shared < KeeperStorageCheckRequestProcessor > ( sub_zk_request ) ) ;
2020-11-04 18:54:55 +00:00
}
else
2020-11-25 13:19:09 +00:00
throw DB : : Exception ( ErrorCodes : : BAD_ARGUMENTS , " Illegal command as part of multi ZooKeeper request {} " , sub_zk_request - > getOpNum ( ) ) ;
2020-11-04 18:54:55 +00:00
}
}
2022-01-06 13:14:45 +00:00
std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & storage , int64_t zxid , int64_t session_id , int64_t time ) const override
2020-11-04 18:54:55 +00:00
{
Coordination : : ZooKeeperResponsePtr response_ptr = zk_request - > makeResponse ( ) ;
Coordination : : ZooKeeperMultiResponse & response = dynamic_cast < Coordination : : ZooKeeperMultiResponse & > ( * response_ptr ) ;
std : : vector < Undo > undo_actions ;
try
{
2020-11-09 17:50:39 +00:00
size_t i = 0 ;
2020-11-04 18:54:55 +00:00
for ( const auto & concrete_request : concrete_requests )
{
2022-01-06 13:14:45 +00:00
auto [ cur_response , undo_action ] = concrete_request - > process ( storage , zxid , session_id , time ) ;
2020-12-08 13:28:39 +00:00
2020-11-09 17:50:39 +00:00
response . responses [ i ] = cur_response ;
2020-11-04 18:54:55 +00:00
if ( cur_response - > error ! = Coordination : : Error : : ZOK )
{
2020-12-08 13:28:39 +00:00
for ( size_t j = 0 ; j < = i ; + + j )
{
auto response_error = response . responses [ j ] - > error ;
response . responses [ j ] = std : : make_shared < Coordination : : ZooKeeperErrorResponse > ( ) ;
response . responses [ j ] - > error = response_error ;
}
for ( size_t j = i + 1 ; j < response . responses . size ( ) ; + + j )
{
response . responses [ j ] = std : : make_shared < Coordination : : ZooKeeperErrorResponse > ( ) ;
response . responses [ j ] - > error = Coordination : : Error : : ZRUNTIMEINCONSISTENCY ;
}
2020-11-04 18:54:55 +00:00
for ( auto it = undo_actions . rbegin ( ) ; it ! = undo_actions . rend ( ) ; + + it )
if ( * it )
( * it ) ( ) ;
return { response_ptr , { } } ;
}
else
undo_actions . emplace_back ( std : : move ( undo_action ) ) ;
2020-12-08 13:28:39 +00:00
2020-11-09 17:50:39 +00:00
+ + i ;
2020-11-04 18:54:55 +00:00
}
response . error = Coordination : : Error : : ZOK ;
return { response_ptr , { } } ;
}
catch ( . . . )
{
for ( auto it = undo_actions . rbegin ( ) ; it ! = undo_actions . rend ( ) ; + + it )
if ( * it )
( * it ) ( ) ;
throw ;
}
}
2020-11-10 13:43:10 +00:00
2021-03-29 08:24:56 +00:00
KeeperStorage : : ResponsesForSessions processWatches ( KeeperStorage : : Watches & watches , KeeperStorage : : Watches & list_watches ) const override
2020-11-10 13:43:10 +00:00
{
2021-03-29 08:24:56 +00:00
KeeperStorage : : ResponsesForSessions result ;
2020-11-10 13:43:10 +00:00
for ( const auto & generic_request : concrete_requests )
2021-01-19 14:22:28 +00:00
{
auto responses = generic_request - > processWatches ( watches , list_watches ) ;
result . insert ( result . end ( ) , responses . begin ( ) , responses . end ( ) ) ;
}
return result ;
2020-11-10 13:43:10 +00:00
}
2020-11-04 18:54:55 +00:00
} ;
2021-08-24 12:30:31 +00:00
struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestProcessor
2020-11-19 16:06:19 +00:00
{
2021-08-24 12:30:31 +00:00
using KeeperStorageRequestProcessor : : KeeperStorageRequestProcessor ;
2022-01-06 13:14:45 +00:00
std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & , int64_t , int64_t , int64_t /* time */ ) const override
2020-11-19 16:06:19 +00:00
{
2020-11-25 13:19:09 +00:00
throw DB : : Exception ( " Called process on close request " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-11-19 16:06:19 +00:00
}
} ;
2021-08-24 12:30:31 +00:00
struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProcessor
2021-05-15 15:01:00 +00:00
{
2021-08-24 12:30:31 +00:00
using KeeperStorageRequestProcessor : : KeeperStorageRequestProcessor ;
2022-01-06 13:14:45 +00:00
std : : pair < Coordination : : ZooKeeperResponsePtr , Undo > process ( KeeperStorage & storage , int64_t /*zxid*/ , int64_t session_id , int64_t /* time */ ) const override
2021-05-15 15:01:00 +00:00
{
2021-05-21 21:19:22 +00:00
Coordination : : ZooKeeperAuthRequest & auth_request = dynamic_cast < Coordination : : ZooKeeperAuthRequest & > ( * zk_request ) ;
2021-05-15 15:01:00 +00:00
Coordination : : ZooKeeperResponsePtr response_ptr = zk_request - > makeResponse ( ) ;
2021-05-21 21:19:22 +00:00
Coordination : : ZooKeeperAuthResponse & auth_response = dynamic_cast < Coordination : : ZooKeeperAuthResponse & > ( * response_ptr ) ;
2021-05-22 16:07:47 +00:00
auto & sessions_and_auth = storage . session_and_auth ;
2021-05-21 21:19:22 +00:00
2021-05-24 12:18:04 +00:00
if ( auth_request . scheme ! = " digest " | | std : : count ( auth_request . data . begin ( ) , auth_request . data . end ( ) , ' : ' ) ! = 1 )
{
auth_response . error = Coordination : : Error : : ZAUTHFAILED ;
}
else
2021-05-22 16:07:47 +00:00
{
2021-05-24 12:18:04 +00:00
auto digest = generateDigest ( auth_request . data ) ;
if ( digest = = storage . superdigest )
2021-05-22 16:07:47 +00:00
{
KeeperStorage : : AuthID auth { " super " , " " } ;
sessions_and_auth [ session_id ] . emplace_back ( auth ) ;
}
else
{
2021-05-24 12:18:04 +00:00
KeeperStorage : : AuthID auth { auth_request . scheme , digest } ;
auto & session_ids = sessions_and_auth [ session_id ] ;
if ( std : : find ( session_ids . begin ( ) , session_ids . end ( ) , auth ) = = session_ids . end ( ) )
sessions_and_auth [ session_id ] . emplace_back ( auth ) ;
2021-05-22 16:07:47 +00:00
}
2021-05-24 12:18:04 +00:00
2021-05-21 21:19:22 +00:00
}
2021-05-15 15:01:00 +00:00
return { response_ptr , { } } ;
}
} ;
2021-03-29 08:24:56 +00:00
void KeeperStorage : : finalize ( )
2020-11-03 14:49:30 +00:00
{
2021-01-19 14:22:28 +00:00
if ( finalized )
throw DB : : Exception ( " Testkeeper storage already finalized " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-11-10 13:43:10 +00:00
2021-01-19 14:22:28 +00:00
finalized = true ;
2020-11-03 14:49:30 +00:00
2021-02-04 13:22:30 +00:00
for ( const auto & [ session_id , ephemerals_paths ] : ephemerals )
for ( const String & ephemeral_path : ephemerals_paths )
2021-02-04 12:07:41 +00:00
container . erase ( ephemeral_path ) ;
ephemerals . clear ( ) ;
2020-11-18 20:36:25 +00:00
2021-01-19 14:22:28 +00:00
watches . clear ( ) ;
list_watches . clear ( ) ;
sessions_and_watchers . clear ( ) ;
2021-02-04 12:07:41 +00:00
session_expiry_queue . clear ( ) ;
2020-11-04 18:54:55 +00:00
}
2021-08-24 12:30:31 +00:00
class KeeperStorageRequestProcessorsFactory final : private boost : : noncopyable
2020-11-04 18:54:55 +00:00
{
public :
2021-08-24 12:30:31 +00:00
using Creator = std : : function < KeeperStorageRequestProcessorPtr ( const Coordination : : ZooKeeperRequestPtr & ) > ;
2020-11-11 13:07:06 +00:00
using OpNumToRequest = std : : unordered_map < Coordination : : OpNum , Creator > ;
2020-11-04 18:54:55 +00:00
2021-08-24 12:30:31 +00:00
static KeeperStorageRequestProcessorsFactory & instance ( )
2020-11-04 18:54:55 +00:00
{
2021-08-24 12:30:31 +00:00
static KeeperStorageRequestProcessorsFactory factory ;
2020-11-04 18:54:55 +00:00
return factory ;
}
2021-08-24 12:30:31 +00:00
KeeperStorageRequestProcessorPtr get ( const Coordination : : ZooKeeperRequestPtr & zk_request ) const
2020-11-04 18:54:55 +00:00
{
auto it = op_num_to_request . find ( zk_request - > getOpNum ( ) ) ;
if ( it = = op_num_to_request . end ( ) )
2020-11-25 13:19:09 +00:00
throw DB : : Exception ( " Unknown operation type " + toString ( zk_request - > getOpNum ( ) ) , ErrorCodes : : LOGICAL_ERROR ) ;
2020-11-04 18:54:55 +00:00
return it - > second ( zk_request ) ;
}
2020-11-03 14:49:30 +00:00
2020-11-11 13:07:06 +00:00
void registerRequest ( Coordination : : OpNum op_num , Creator creator )
2020-11-04 18:54:55 +00:00
{
if ( ! op_num_to_request . try_emplace ( op_num , creator ) . second )
throw DB : : Exception ( ErrorCodes : : LOGICAL_ERROR , " Request with op num {} already registered " , op_num ) ;
}
private :
OpNumToRequest op_num_to_request ;
2021-08-24 12:30:31 +00:00
KeeperStorageRequestProcessorsFactory ( ) ;
2020-11-04 18:54:55 +00:00
} ;
2020-11-11 13:07:06 +00:00
template < Coordination : : OpNum num , typename RequestT >
2021-08-24 12:30:31 +00:00
void registerKeeperRequestProcessor ( KeeperStorageRequestProcessorsFactory & factory )
2020-11-04 18:54:55 +00:00
{
factory . registerRequest ( num , [ ] ( const Coordination : : ZooKeeperRequestPtr & zk_request ) { return std : : make_shared < RequestT > ( zk_request ) ; } ) ;
}
2021-08-24 12:30:31 +00:00
KeeperStorageRequestProcessorsFactory : : KeeperStorageRequestProcessorsFactory ( )
2020-11-04 18:54:55 +00:00
{
2021-08-24 12:30:31 +00:00
registerKeeperRequestProcessor < Coordination : : OpNum : : Heartbeat , KeeperStorageHeartbeatRequestProcessor > ( * this ) ;
registerKeeperRequestProcessor < Coordination : : OpNum : : Sync , KeeperStorageSyncRequestProcessor > ( * this ) ;
registerKeeperRequestProcessor < Coordination : : OpNum : : Auth , KeeperStorageAuthRequestProcessor > ( * this ) ;
registerKeeperRequestProcessor < Coordination : : OpNum : : Close , KeeperStorageCloseRequestProcessor > ( * this ) ;
registerKeeperRequestProcessor < Coordination : : OpNum : : Create , KeeperStorageCreateRequestProcessor > ( * this ) ;
registerKeeperRequestProcessor < Coordination : : OpNum : : Remove , KeeperStorageRemoveRequestProcessor > ( * this ) ;
registerKeeperRequestProcessor < Coordination : : OpNum : : Exists , KeeperStorageExistsRequestProcessor > ( * this ) ;
registerKeeperRequestProcessor < Coordination : : OpNum : : Get , KeeperStorageGetRequestProcessor > ( * this ) ;
registerKeeperRequestProcessor < Coordination : : OpNum : : Set , KeeperStorageSetRequestProcessor > ( * this ) ;
registerKeeperRequestProcessor < Coordination : : OpNum : : List , KeeperStorageListRequestProcessor > ( * this ) ;
registerKeeperRequestProcessor < Coordination : : OpNum : : SimpleList , KeeperStorageListRequestProcessor > ( * this ) ;
registerKeeperRequestProcessor < Coordination : : OpNum : : Check , KeeperStorageCheckRequestProcessor > ( * this ) ;
registerKeeperRequestProcessor < Coordination : : OpNum : : Multi , KeeperStorageMultiRequestProcessor > ( * this ) ;
registerKeeperRequestProcessor < Coordination : : OpNum : : SetACL , KeeperStorageSetACLRequestProcessor > ( * this ) ;
registerKeeperRequestProcessor < Coordination : : OpNum : : GetACL , KeeperStorageGetACLRequestProcessor > ( * this ) ;
2020-11-03 14:49:30 +00:00
}
2022-01-06 13:14:45 +00:00
KeeperStorage : : ResponsesForSessions KeeperStorage : : processRequest ( const Coordination : : ZooKeeperRequestPtr & zk_request , int64_t session_id , int64_t time , std : : optional < int64_t > new_last_zxid , bool check_acl )
2020-11-03 14:49:30 +00:00
{
2021-03-29 08:24:56 +00:00
KeeperStorage : : ResponsesForSessions results ;
2021-02-25 08:34:05 +00:00
if ( new_last_zxid )
{
if ( zxid > = * new_last_zxid )
2021-03-04 11:22:59 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Got new ZXID {} smaller or equal than current {}. It's a bug " , * new_last_zxid , zxid ) ;
2021-02-25 08:34:05 +00:00
zxid = * new_last_zxid ;
}
2021-09-02 11:54:32 +00:00
/// ZooKeeper update sessions expirity for each request, not only for heartbeats
2021-09-02 11:43:34 +00:00
session_expiry_queue . addNewSessionOrUpdate ( session_id , session_and_timeout [ session_id ] ) ;
2021-08-24 12:30:31 +00:00
if ( zk_request - > getOpNum ( ) = = Coordination : : OpNum : : Close ) /// Close request is special
2020-11-03 14:49:30 +00:00
{
2021-01-19 14:22:28 +00:00
auto it = ephemerals . find ( session_id ) ;
if ( it ! = ephemerals . end ( ) )
{
for ( const auto & ephemeral_path : it - > second )
{
2021-03-29 08:24:56 +00:00
container . updateValue ( parentPath ( ephemeral_path ) , [ & ephemeral_path ] ( KeeperStorage : : Node & parent )
2021-03-16 12:36:54 +00:00
{
- - parent . stat . numChildren ;
+ + parent . stat . cversion ;
2021-12-30 16:21:49 +00:00
auto base_name = getBaseName ( ephemeral_path ) ;
parent . children . erase ( base_name ) ;
2022-01-19 11:46:29 +00:00
parent . size_bytes - = base_name . size ;
2021-03-16 12:36:54 +00:00
} ) ;
2022-01-22 19:36:23 +00:00
container . erase ( ephemeral_path ) ;
2021-01-19 14:22:28 +00:00
auto responses = processWatchesImpl ( ephemeral_path , watches , list_watches , Coordination : : Event : : DELETED ) ;
results . insert ( results . end ( ) , responses . begin ( ) , responses . end ( ) ) ;
}
ephemerals . erase ( it ) ;
}
clearDeadWatches ( session_id ) ;
2021-05-27 08:15:46 +00:00
auto auth_it = session_and_auth . find ( session_id ) ;
if ( auth_it ! = session_and_auth . end ( ) )
session_and_auth . erase ( auth_it ) ;
2021-01-19 14:22:28 +00:00
/// Finish connection
auto response = std : : make_shared < Coordination : : ZooKeeperCloseResponse > ( ) ;
response - > xid = zk_request - > xid ;
response - > zxid = getZXID ( ) ;
2021-02-03 20:32:15 +00:00
session_expiry_queue . remove ( session_id ) ;
session_and_timeout . erase ( session_id ) ;
results . push_back ( ResponseForSession { session_id , response } ) ;
}
2021-08-24 12:30:31 +00:00
else if ( zk_request - > getOpNum ( ) = = Coordination : : OpNum : : Heartbeat ) /// Heartbeat request is also special
2021-02-03 20:32:15 +00:00
{
2021-08-24 12:30:31 +00:00
KeeperStorageRequestProcessorPtr storage_request = KeeperStorageRequestProcessorsFactory : : instance ( ) . get ( zk_request ) ;
2022-01-06 13:14:45 +00:00
auto [ response , _ ] = storage_request - > process ( * this , zxid , session_id , time ) ;
2021-02-03 20:32:15 +00:00
response - > xid = zk_request - > xid ;
response - > zxid = getZXID ( ) ;
2021-01-19 14:45:45 +00:00
results . push_back ( ResponseForSession { session_id , response } ) ;
2020-11-03 14:49:30 +00:00
}
2021-08-24 12:30:31 +00:00
else /// normal requests proccession
2020-11-03 14:49:30 +00:00
{
2021-08-24 12:30:31 +00:00
KeeperStorageRequestProcessorPtr request_processor = KeeperStorageRequestProcessorsFactory : : instance ( ) . get ( zk_request ) ;
2021-05-21 21:19:22 +00:00
Coordination : : ZooKeeperResponsePtr response ;
2021-08-24 12:30:31 +00:00
if ( check_acl & & ! request_processor - > checkAuth ( * this , session_id ) )
2021-05-21 21:19:22 +00:00
{
response = zk_request - > makeResponse ( ) ;
2021-05-24 12:18:04 +00:00
/// Original ZooKeeper always throws no auth, even when user provided some credentials
response - > error = Coordination : : Error : : ZNOAUTH ;
2021-05-21 21:19:22 +00:00
}
else
{
2022-01-06 13:14:45 +00:00
std : : tie ( response , std : : ignore ) = request_processor - > process ( * this , zxid , session_id , time ) ;
2021-05-21 21:19:22 +00:00
}
2021-01-19 14:22:28 +00:00
2021-08-24 12:30:31 +00:00
/// Watches for this requests are added to the watches lists
2021-01-19 14:22:28 +00:00
if ( zk_request - > has_watch )
{
if ( response - > error = = Coordination : : Error : : ZOK )
{
auto & watches_type = zk_request - > getOpNum ( ) = = Coordination : : OpNum : : List | | zk_request - > getOpNum ( ) = = Coordination : : OpNum : : SimpleList
? list_watches
: watches ;
watches_type [ zk_request - > getPath ( ) ] . emplace_back ( session_id ) ;
sessions_and_watchers [ session_id ] . emplace ( zk_request - > getPath ( ) ) ;
}
else if ( response - > error = = Coordination : : Error : : ZNONODE & & zk_request - > getOpNum ( ) = = Coordination : : OpNum : : Exists )
{
watches [ zk_request - > getPath ( ) ] . emplace_back ( session_id ) ;
sessions_and_watchers [ session_id ] . emplace ( zk_request - > getPath ( ) ) ;
}
}
2021-08-24 12:30:31 +00:00
/// If this requests processed successfully we need to check watches
2021-01-19 14:22:28 +00:00
if ( response - > error = = Coordination : : Error : : ZOK )
{
2021-08-24 12:30:31 +00:00
auto watch_responses = request_processor - > processWatches ( watches , list_watches ) ;
2021-01-19 14:22:28 +00:00
results . insert ( results . end ( ) , watch_responses . begin ( ) , watch_responses . end ( ) ) ;
}
response - > xid = zk_request - > xid ;
response - > zxid = getZXID ( ) ;
2021-01-19 14:45:45 +00:00
results . push_back ( ResponseForSession { session_id , response } ) ;
2020-11-03 14:49:30 +00:00
}
2021-01-19 14:22:28 +00:00
return results ;
2020-11-03 14:49:30 +00:00
}
2021-01-19 14:22:28 +00:00
2021-03-29 08:24:56 +00:00
void KeeperStorage : : clearDeadWatches ( int64_t session_id )
2020-11-26 14:57:32 +00:00
{
2021-08-25 09:31:02 +00:00
/// Clear all watches for this session
2020-11-26 14:57:32 +00:00
auto watches_it = sessions_and_watchers . find ( session_id ) ;
if ( watches_it ! = sessions_and_watchers . end ( ) )
{
for ( const auto & watch_path : watches_it - > second )
{
2021-08-24 12:30:31 +00:00
/// Maybe it's a normal watch
2020-11-26 14:57:32 +00:00
auto watch = watches . find ( watch_path ) ;
if ( watch ! = watches . end ( ) )
{
auto & watches_for_path = watch - > second ;
for ( auto w_it = watches_for_path . begin ( ) ; w_it ! = watches_for_path . end ( ) ; )
{
2021-01-19 14:22:28 +00:00
if ( * w_it = = session_id )
2020-11-26 14:57:32 +00:00
w_it = watches_for_path . erase ( w_it ) ;
else
+ + w_it ;
}
if ( watches_for_path . empty ( ) )
watches . erase ( watch ) ;
}
2020-12-14 16:01:29 +00:00
2021-08-24 12:30:31 +00:00
/// Maybe it's a list watch
2020-12-14 16:01:29 +00:00
auto list_watch = list_watches . find ( watch_path ) ;
if ( list_watch ! = list_watches . end ( ) )
{
auto & list_watches_for_path = list_watch - > second ;
for ( auto w_it = list_watches_for_path . begin ( ) ; w_it ! = list_watches_for_path . end ( ) ; )
{
2021-01-19 14:22:28 +00:00
if ( * w_it = = session_id )
2020-12-14 16:01:29 +00:00
w_it = list_watches_for_path . erase ( w_it ) ;
else
+ + w_it ;
}
if ( list_watches_for_path . empty ( ) )
list_watches . erase ( list_watch ) ;
}
2020-11-26 14:57:32 +00:00
}
2021-08-24 12:30:31 +00:00
2020-11-26 14:57:32 +00:00
sessions_and_watchers . erase ( watches_it ) ;
}
}
2021-11-05 10:21:34 +00:00
void KeeperStorage : : dumpWatches ( WriteBufferFromOwnString & buf ) const
{
2021-11-18 20:17:22 +00:00
for ( const auto & [ session_id , watches_paths ] : sessions_and_watchers )
2021-11-05 10:21:34 +00:00
{
2021-11-18 20:17:22 +00:00
buf < < " 0x " < < getHexUIntLowercase ( session_id ) < < " \n " ;
for ( const String & path : watches_paths )
2021-11-12 12:48:42 +00:00
buf < < " \t " < < path < < " \n " ;
2021-11-05 10:21:34 +00:00
}
}
void KeeperStorage : : dumpWatchesByPath ( WriteBufferFromOwnString & buf ) const
{
2021-11-18 20:17:22 +00:00
auto write_int_vec = [ & buf ] ( const std : : vector < int64_t > & session_ids )
2021-11-05 10:21:34 +00:00
{
2021-11-18 20:17:22 +00:00
for ( int64_t session_id : session_ids )
2021-11-05 10:21:34 +00:00
{
2021-11-18 20:17:22 +00:00
buf < < " \t 0x " < < getHexUIntLowercase ( session_id ) < < " \n " ;
2021-11-05 10:21:34 +00:00
}
} ;
2021-11-18 20:17:22 +00:00
for ( const auto & [ watch_path , sessions ] : watches )
2021-11-05 10:21:34 +00:00
{
2021-11-18 20:17:22 +00:00
buf < < watch_path < < " \n " ;
write_int_vec ( sessions ) ;
2021-11-05 10:21:34 +00:00
}
2021-11-18 20:17:22 +00:00
for ( const auto & [ watch_path , sessions ] : list_watches )
2021-11-05 10:21:34 +00:00
{
2021-11-18 20:17:22 +00:00
buf < < watch_path < < " \n " ;
write_int_vec ( sessions ) ;
2021-11-05 10:21:34 +00:00
}
}
2021-11-18 20:17:22 +00:00
void KeeperStorage : : dumpSessionsAndEphemerals ( WriteBufferFromOwnString & buf ) const
2021-11-05 10:21:34 +00:00
{
2021-11-18 20:17:22 +00:00
auto write_str_set = [ & buf ] ( const std : : unordered_set < String > & ephemeral_paths )
2021-11-05 10:21:34 +00:00
{
2021-11-18 20:17:22 +00:00
for ( const String & path : ephemeral_paths )
2021-11-05 10:21:34 +00:00
{
2021-11-18 20:17:22 +00:00
buf < < " \t " < < path < < " \n " ;
2021-11-05 10:21:34 +00:00
}
} ;
2021-11-18 20:17:22 +00:00
buf < < " Sessions dump ( " < < session_and_timeout . size ( ) < < " ): \n " ;
for ( const auto & [ session_id , _ ] : session_and_timeout )
{
buf < < " 0x " < < getHexUIntLowercase ( session_id ) < < " \n " ;
}
buf < < " Sessions with Ephemerals ( " < < getSessionWithEphemeralNodesCount ( ) < < " ): \n " ;
for ( const auto & [ session_id , ephemeral_paths ] : ephemerals )
2021-11-05 10:21:34 +00:00
{
2021-11-18 20:17:22 +00:00
buf < < " 0x " < < getHexUIntLowercase ( session_id ) < < " \n " ;
write_str_set ( ephemeral_paths ) ;
2021-11-05 10:21:34 +00:00
}
}
2021-11-18 20:17:22 +00:00
uint64_t KeeperStorage : : getTotalWatchesCount ( ) const
{
uint64_t ret = 0 ;
for ( const auto & [ path , subscribed_sessions ] : watches )
ret + = subscribed_sessions . size ( ) ;
for ( const auto & [ path , subscribed_sessions ] : list_watches )
ret + = subscribed_sessions . size ( ) ;
return ret ;
}
uint64_t KeeperStorage : : getSessionsWithWatchesCount ( ) const
{
std : : unordered_set < int64_t > counter ;
for ( const auto & [ path , subscribed_sessions ] : watches )
counter . insert ( subscribed_sessions . begin ( ) , subscribed_sessions . end ( ) ) ;
for ( const auto & [ path , subscribed_sessions ] : list_watches )
counter . insert ( subscribed_sessions . begin ( ) , subscribed_sessions . end ( ) ) ;
return counter . size ( ) ;
}
uint64_t KeeperStorage : : getTotalEphemeralNodesCount ( ) const
{
uint64_t ret = 0 ;
for ( const auto & [ session_id , nodes ] : ephemerals )
ret + = nodes . size ( ) ;
return ret ;
}
2020-10-30 14:16:47 +00:00
}