2022-07-27 13:20:45 +00:00
# include <Storages/StorageKeeperMap.h>
2022-07-26 09:08:55 +00:00
# include <Columns/ColumnString.h>
2022-07-27 13:20:45 +00:00
2022-09-01 10:47:05 +00:00
# include <Databases/DatabaseReplicated.h>
2022-07-27 13:20:45 +00:00
# include <Core/NamesAndTypes.h>
2022-09-01 07:56:06 +00:00
# include <Core/UUID.h>
2022-09-02 07:40:39 +00:00
# include <Core/ServerUUID.h>
2022-07-27 13:20:45 +00:00
2022-07-26 09:08:55 +00:00
# include <DataTypes/DataTypeString.h>
2022-07-27 13:20:45 +00:00
# include <Interpreters/evaluateConstantExpression.h>
# include <Parsers/ASTCreateQuery.h>
# include <Parsers/ASTExpressionList.h>
# include <Parsers/ASTFunction.h>
# include <Parsers/ASTIdentifier.h>
# include <Parsers/ASTSelectQuery.h>
2022-09-08 12:45:36 +00:00
# include <Parsers/formatAST.h>
2022-07-27 13:20:45 +00:00
2022-07-26 09:08:55 +00:00
# include <Processors/ISource.h>
# include <Processors/Sinks/SinkToStorage.h>
2022-07-27 13:20:45 +00:00
# include <Storages/ColumnsDescription.h>
# include <Storages/KVStorageUtils.h>
2022-07-26 09:08:55 +00:00
# include <Storages/StorageFactory.h>
2022-07-27 13:20:45 +00:00
# include <Storages/StorageInMemoryMetadata.h>
# include <Storages/checkAndGetLiteralArgument.h>
2022-09-01 07:56:06 +00:00
# include <Common/Base64.h>
# include <Common/Exception.h>
# include <Common/ZooKeeper/IKeeper.h>
2022-07-26 09:08:55 +00:00
# include <Common/ZooKeeper/KeeperException.h>
# include <Common/ZooKeeper/Types.h>
2022-08-23 13:15:31 +00:00
# include <Common/ZooKeeper/ZooKeeper.h>
2022-07-27 13:20:45 +00:00
# include <Common/ZooKeeper/ZooKeeperConstants.h>
2022-09-01 07:56:06 +00:00
# include <base/types.h>
2022-07-27 13:20:45 +00:00
# include <boost/algorithm/string/classification.hpp>
2022-07-26 09:08:55 +00:00
namespace DB
{
namespace ErrorCodes
{
2022-08-03 13:34:14 +00:00
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH ;
extern const int BAD_ARGUMENTS ;
extern const int KEEPER_EXCEPTION ;
2022-08-03 14:02:14 +00:00
extern const int LOGICAL_ERROR ;
2022-08-31 08:14:28 +00:00
extern const int LIMIT_EXCEEDED ;
2022-07-26 09:08:55 +00:00
}
2022-07-27 13:20:45 +00:00
namespace
2022-07-26 09:08:55 +00:00
{
2022-09-08 12:45:36 +00:00
std : : string formattedAST ( const ASTPtr & ast )
{
if ( ! ast )
return " " ;
2022-09-12 14:43:52 +00:00
return serializeAST ( * ast ) ;
2022-09-08 12:45:36 +00:00
}
void verifyTableId ( const StorageID & table_id )
{
if ( ! table_id . hasUUID ( ) )
{
auto database = DatabaseCatalog : : instance ( ) . getDatabase ( table_id . database_name ) ;
throw Exception (
ErrorCodes : : BAD_ARGUMENTS ,
" KeeperMap cannot be used with '{}' database because it uses {} engine. Please use Atomic or Replicated database " ,
table_id . getDatabaseName ( ) ,
database - > getEngineName ( ) ) ;
}
}
2022-07-26 09:08:55 +00:00
}
class StorageKeeperMapSink : public SinkToStorage
{
StorageKeeperMap & storage ;
std : : unordered_map < std : : string , std : : string > new_values ;
2022-07-27 13:20:45 +00:00
size_t primary_key_pos ;
2022-07-26 09:08:55 +00:00
public :
2022-07-27 13:20:45 +00:00
StorageKeeperMapSink ( StorageKeeperMap & storage_ , const StorageMetadataPtr & metadata_snapshot )
: SinkToStorage ( metadata_snapshot - > getSampleBlock ( ) ) , storage ( storage_ )
{
auto primary_key = storage . getPrimaryKey ( ) ;
assert ( primary_key . size ( ) = = 1 ) ;
2022-09-01 07:56:06 +00:00
primary_key_pos = getHeader ( ) . getPositionByName ( primary_key [ 0 ] ) ;
2022-07-27 13:20:45 +00:00
}
2022-07-26 09:08:55 +00:00
std : : string getName ( ) const override { return " StorageKeeperMapSink " ; }
void consume ( Chunk chunk ) override
{
2022-07-27 13:20:45 +00:00
auto rows = chunk . getNumRows ( ) ;
auto block = getHeader ( ) . cloneWithColumns ( chunk . detachColumns ( ) ) ;
WriteBufferFromOwnString wb_key ;
WriteBufferFromOwnString wb_value ;
2022-07-26 09:08:55 +00:00
for ( size_t i = 0 ; i < rows ; + + i )
{
2022-07-27 13:20:45 +00:00
wb_key . restart ( ) ;
wb_value . restart ( ) ;
2022-07-26 09:08:55 +00:00
2022-07-27 13:20:45 +00:00
size_t idx = 0 ;
for ( const auto & elem : block )
{
elem . type - > getDefaultSerialization ( ) - > serializeBinary ( * elem . column , i , idx = = primary_key_pos ? wb_key : wb_value ) ;
+ + idx ;
}
2022-07-26 09:08:55 +00:00
2022-09-12 14:43:52 +00:00
auto key = base64Encode ( wb_key . str ( ) , /* url_encoding */ true ) ;
2022-07-27 13:20:45 +00:00
new_values [ std : : move ( key ) ] = std : : move ( wb_value . str ( ) ) ;
2022-07-26 09:08:55 +00:00
}
}
void onFinish ( ) override
{
2022-08-31 08:14:28 +00:00
auto zookeeper = storage . getClient ( ) ;
2022-08-08 09:43:29 +00:00
2022-08-31 08:14:28 +00:00
auto keys_limit = storage . keysLimit ( ) ;
size_t current_keys_num = 0 ;
size_t new_keys_num = 0 ;
2022-09-08 12:45:36 +00:00
// We use keys limit as a soft limit so we ignore some cases when it can be still exceeded
// (e.g if parallel insert queries are being run)
2022-08-31 08:14:28 +00:00
if ( keys_limit ! = 0 )
{
2022-09-12 14:43:52 +00:00
Coordination : : Stat data_stat ;
zookeeper - > get ( storage . dataPath ( ) , & data_stat ) ;
current_keys_num = data_stat . numChildren ;
2022-08-31 08:14:28 +00:00
}
2022-09-29 10:39:33 +00:00
std : : vector < std : : string > key_paths ;
key_paths . reserve ( new_values . size ( ) ) ;
for ( const auto & [ key , _ ] : new_values )
key_paths . push_back ( storage . fullPathForKey ( key ) ) ;
2022-08-08 09:43:29 +00:00
2022-09-29 10:39:33 +00:00
auto results = zookeeper - > exists ( key_paths ) ;
2022-08-31 10:40:32 +00:00
2022-09-29 10:39:33 +00:00
Coordination : : Requests requests ;
requests . reserve ( key_paths . size ( ) ) ;
for ( size_t i = 0 ; i < key_paths . size ( ) ; + + i )
2022-08-31 10:40:32 +00:00
{
2022-09-29 10:39:33 +00:00
auto key = fs : : path ( key_paths [ i ] ) . filename ( ) ;
if ( results [ i ] . error = = Coordination : : Error : : ZOK )
2022-08-31 08:14:28 +00:00
{
2022-09-29 10:39:33 +00:00
requests . push_back ( zkutil : : makeSetRequest ( key_paths [ i ] , new_values [ key ] , - 1 ) ) ;
2022-08-31 08:14:28 +00:00
}
2022-08-23 13:15:31 +00:00
else
2022-08-31 08:14:28 +00:00
{
2022-09-29 10:39:33 +00:00
requests . push_back ( zkutil : : makeCreateRequest ( key_paths [ i ] , new_values [ key ] , zkutil : : CreateMode : : Persistent ) ) ;
2022-08-31 08:14:28 +00:00
+ + new_keys_num ;
}
}
if ( new_keys_num ! = 0 )
{
auto will_be = current_keys_num + new_keys_num ;
if ( keys_limit ! = 0 & & will_be > keys_limit )
throw Exception (
ErrorCodes : : LIMIT_EXCEEDED ,
" Limit would be exceeded by inserting {} new key(s). Limit is {}, while the number of keys would be {} " ,
new_keys_num ,
keys_limit ,
will_be ) ;
2022-08-08 09:43:29 +00:00
}
2022-07-26 09:08:55 +00:00
zookeeper - > multi ( requests ) ;
}
} ;
2022-07-27 13:20:45 +00:00
template < typename KeyContainer >
2022-07-26 09:08:55 +00:00
class StorageKeeperMapSource : public ISource
{
2022-07-27 13:20:45 +00:00
const StorageKeeperMap & storage ;
2022-07-26 09:08:55 +00:00
size_t max_block_size ;
2022-07-27 13:20:45 +00:00
using KeyContainerPtr = std : : shared_ptr < KeyContainer > ;
KeyContainerPtr container ;
using KeyContainerIter = typename KeyContainer : : const_iterator ;
KeyContainerIter it ;
KeyContainerIter end ;
2022-07-26 09:08:55 +00:00
2022-07-27 13:20:45 +00:00
public :
StorageKeeperMapSource (
const StorageKeeperMap & storage_ ,
const Block & header ,
size_t max_block_size_ ,
KeyContainerPtr container_ ,
KeyContainerIter begin_ ,
KeyContainerIter end_ )
: ISource ( header ) , storage ( storage_ ) , max_block_size ( max_block_size_ ) , container ( std : : move ( container_ ) ) , it ( begin_ ) , end ( end_ )
{
2022-07-26 09:08:55 +00:00
}
2022-07-27 13:20:45 +00:00
std : : string getName ( ) const override { return " StorageKeeperMapSource " ; }
2022-07-26 09:08:55 +00:00
2022-07-27 13:20:45 +00:00
Chunk generate ( ) override
{
if ( it > = end )
2022-07-26 09:08:55 +00:00
{
2022-07-27 13:20:45 +00:00
it = { } ;
return { } ;
2022-07-26 09:08:55 +00:00
}
2022-07-27 13:20:45 +00:00
using KeyType = typename KeyContainer : : value_type ;
if constexpr ( std : : same_as < KeyType , Field > )
{
const auto & sample_block = getPort ( ) . getHeader ( ) ;
const auto & key_column_type = sample_block . getByName ( storage . getPrimaryKey ( ) . at ( 0 ) ) . type ;
auto raw_keys = serializeKeysToRawString ( it , end , key_column_type , max_block_size ) ;
for ( auto & raw_key : raw_keys )
2022-09-12 14:43:52 +00:00
raw_key = base64Encode ( raw_key , /* url_encoding */ true ) ;
2022-07-26 09:08:55 +00:00
2022-07-27 13:20:45 +00:00
return storage . getBySerializedKeys ( raw_keys , nullptr ) ;
}
else
2022-07-26 09:08:55 +00:00
{
2022-07-27 13:20:45 +00:00
size_t elem_num = std : : min ( max_block_size , static_cast < size_t > ( end - it ) ) ;
auto chunk = storage . getBySerializedKeys ( std : : span { it , it + elem_num } , nullptr ) ;
it + = elem_num ;
return chunk ;
2022-07-26 09:08:55 +00:00
}
}
2022-07-27 13:20:45 +00:00
} ;
2022-07-26 09:08:55 +00:00
2022-07-27 13:20:45 +00:00
StorageKeeperMap : : StorageKeeperMap (
2022-08-23 13:15:31 +00:00
ContextPtr context_ ,
2022-07-27 13:20:45 +00:00
const StorageID & table_id ,
const StorageInMemoryMetadata & metadata ,
2022-08-10 07:24:56 +00:00
bool attach ,
2022-07-27 13:20:45 +00:00
std : : string_view primary_key_ ,
2022-08-23 13:15:31 +00:00
const std : : string & root_path_ ,
2022-08-31 08:14:28 +00:00
UInt64 keys_limit_ )
2022-08-23 13:15:31 +00:00
: IStorage ( table_id )
, WithContext ( context_ - > getGlobalContext ( ) )
, root_path ( zkutil : : extractZooKeeperPath ( root_path_ , false ) )
, primary_key ( primary_key_ )
, zookeeper_name ( zkutil : : extractZooKeeperName ( root_path_ ) )
2022-08-31 08:14:28 +00:00
, keys_limit ( keys_limit_ )
2022-09-08 12:45:36 +00:00
, log ( & Poco : : Logger : : get ( fmt : : format ( " StorageKeeperMap ({}) " , table_id . getNameForLogs ( ) ) ) )
2022-07-27 13:20:45 +00:00
{
2022-09-01 14:39:57 +00:00
std : : string path_prefix = context_ - > getConfigRef ( ) . getString ( " keeper_map_path_prefix " , " " ) ;
if ( path_prefix . empty ( ) )
throw Exception ( ErrorCodes : : BAD_ARGUMENTS , " KeeperMap is disabled because 'keeper_map_path_prefix' config is not defined " ) ;
2022-09-08 12:45:36 +00:00
verifyTableId ( table_id ) ;
2022-08-24 17:27:07 +00:00
2022-07-27 13:20:45 +00:00
setInMemoryMetadata ( metadata ) ;
2022-07-26 09:08:55 +00:00
2022-09-08 12:45:36 +00:00
WriteBufferFromOwnString out ;
out < < " KeeperMap metadata format version: 1 \n "
< < " columns: " < < metadata . columns . toString ( )
< < " primary key: " < < formattedAST ( metadata . getPrimaryKey ( ) . expression_list_ast ) < < " \n " ;
metadata_string = out . str ( ) ;
2022-08-10 08:52:36 +00:00
if ( root_path . empty ( ) )
throw Exception ( " root_path should not be empty " , ErrorCodes : : BAD_ARGUMENTS ) ;
if ( ! root_path . starts_with ( ' / ' ) )
throw Exception ( " root_path should start with '/' " , ErrorCodes : : BAD_ARGUMENTS ) ;
2022-07-26 09:08:55 +00:00
2022-08-31 08:14:28 +00:00
auto config_keys_limit = context_ - > getConfigRef ( ) . getUInt64 ( " keeper_map_keys_limit " , 0 ) ;
2022-09-08 12:45:36 +00:00
if ( config_keys_limit ! = 0 & & ( keys_limit = = 0 | | keys_limit > config_keys_limit ) )
2022-08-31 08:14:28 +00:00
{
LOG_WARNING (
log ,
2022-09-08 12:45:36 +00:00
" Keys limit defined by argument ({}) is larger than the one defined by 'keeper_map_keys_limit' config ({}). Will use "
2022-08-31 08:14:28 +00:00
" config defined value " ,
keys_limit ,
config_keys_limit ) ;
keys_limit = config_keys_limit ;
}
else if ( keys_limit > 0 )
{
2022-09-08 12:45:36 +00:00
LOG_INFO ( log , " Keys limit will be set to {} " , keys_limit ) ;
2022-08-31 08:14:28 +00:00
}
2022-09-01 14:39:57 +00:00
auto root_path_fs = fs : : path ( path_prefix ) / std : : string_view { root_path } . substr ( 1 ) ;
root_path = root_path_fs . generic_string ( ) ;
2022-09-12 14:43:52 +00:00
data_path = root_path_fs / " data " ;
2022-09-08 12:45:36 +00:00
auto metadata_path_fs = root_path_fs / " metadata " ;
2022-08-10 08:52:36 +00:00
metadata_path = metadata_path_fs ;
2022-08-23 13:15:31 +00:00
tables_path = metadata_path_fs / " tables " ;
2022-09-01 10:47:05 +00:00
2022-09-02 07:40:39 +00:00
auto table_unique_id = toString ( table_id . uuid ) + toString ( ServerUUID : : get ( ) ) ;
2022-09-01 10:47:05 +00:00
table_path = fs : : path ( tables_path ) / table_unique_id ;
2022-08-23 13:15:31 +00:00
dropped_path = metadata_path_fs / " dropped " ;
2022-08-24 17:27:07 +00:00
dropped_lock_path = fs : : path ( dropped_path ) / " lock " ;
2022-08-10 08:52:36 +00:00
2022-08-10 07:24:56 +00:00
if ( attach )
{
2022-08-23 13:15:31 +00:00
checkTable < false > ( ) ;
2022-08-10 07:24:56 +00:00
return ;
}
2022-08-23 13:15:31 +00:00
auto client = getClient ( ) ;
2022-09-02 07:42:22 +00:00
if ( root_path ! = " / " & & ! client - > exists ( root_path ) )
2022-07-27 13:20:45 +00:00
{
2022-09-08 12:45:36 +00:00
LOG_TRACE ( log , " Creating root path {} " , root_path ) ;
client - > createAncestors ( root_path ) ;
client - > createIfNotExists ( root_path , " " ) ;
2022-07-26 09:08:55 +00:00
}
2022-08-08 09:43:29 +00:00
2022-08-24 17:27:07 +00:00
for ( size_t i = 0 ; i < 1000 ; + + i )
2022-08-23 13:15:31 +00:00
{
2022-09-26 14:28:03 +00:00
std : : string stored_metadata_string ;
auto exists = client - > tryGet ( metadata_path , stored_metadata_string ) ;
if ( exists )
{
// this requires same name for columns
// maybe we can do a smarter comparison for columns and primary key expression
if ( stored_metadata_string ! = metadata_string )
throw Exception (
ErrorCodes : : BAD_ARGUMENTS ,
" Path {} is already used but the stored table definition doesn't match. Stored metadata: {} " ,
root_path ,
stored_metadata_string ) ;
auto code = client - > tryCreate ( table_path , " " , zkutil : : CreateMode : : Persistent ) ;
// tables_path was removed with drop
if ( code = = Coordination : : Error : : ZNONODE )
{
LOG_INFO ( log , " Metadata nodes were removed by another server, will retry " ) ;
continue ;
}
else if ( code ! = Coordination : : Error : : ZOK )
{
throw zkutil : : KeeperException ( code , " Failed to create table on path {} because a table with same UUID already exists " , root_path ) ;
}
return ;
}
2022-08-24 17:27:07 +00:00
if ( client - > exists ( dropped_path ) )
{
LOG_INFO ( log , " Removing leftover nodes " ) ;
auto code = client - > tryCreate ( dropped_lock_path , " " , zkutil : : CreateMode : : Ephemeral ) ;
2022-08-10 07:24:56 +00:00
2022-09-08 12:45:36 +00:00
if ( code = = Coordination : : Error : : ZNONODE )
2022-08-24 17:27:07 +00:00
{
2022-09-01 14:39:57 +00:00
LOG_INFO ( log , " Someone else removed leftover nodes " ) ;
2022-08-24 17:27:07 +00:00
}
2022-09-08 12:45:36 +00:00
else if ( code = = Coordination : : Error : : ZNODEEXISTS )
{
LOG_INFO ( log , " Someone else is removing leftover nodes " ) ;
continue ;
}
2022-08-24 17:27:07 +00:00
else if ( code ! = Coordination : : Error : : ZOK )
{
throw Coordination : : Exception ( code , dropped_lock_path ) ;
}
else
{
auto metadata_drop_lock = zkutil : : EphemeralNodeHolder : : existing ( dropped_lock_path , * client ) ;
2022-08-31 08:23:44 +00:00
if ( ! dropTable ( client , metadata_drop_lock ) )
2022-08-24 17:27:07 +00:00
continue ;
}
}
2022-08-08 09:43:29 +00:00
2022-09-26 14:28:03 +00:00
Coordination : : Requests create_requests
2022-09-08 12:45:36 +00:00
{
2022-09-26 14:28:03 +00:00
zkutil : : makeCreateRequest ( metadata_path , metadata_string , zkutil : : CreateMode : : Persistent ) ,
zkutil : : makeCreateRequest ( data_path , metadata_string , zkutil : : CreateMode : : Persistent ) ,
zkutil : : makeCreateRequest ( tables_path , " " , zkutil : : CreateMode : : Persistent ) ,
zkutil : : makeCreateRequest ( table_path , " " , zkutil : : CreateMode : : Persistent ) ,
} ;
Coordination : : Responses create_responses ;
auto code = client - > tryMulti ( create_requests , create_responses ) ;
if ( code = = Coordination : : Error : : ZNODEEXISTS )
2022-09-08 12:45:36 +00:00
{
2022-09-27 14:00:44 +00:00
LOG_INFO ( log , " It looks like a table on path {} was created by another server at the same moment, will retry " , root_path ) ;
2022-09-26 14:28:03 +00:00
continue ;
2022-09-08 12:45:36 +00:00
}
2022-09-26 14:28:03 +00:00
else if ( code ! = Coordination : : Error : : ZOK )
2022-08-30 13:41:13 +00:00
{
2022-09-26 14:28:03 +00:00
zkutil : : KeeperMultiException : : check ( code , create_requests , create_responses ) ;
2022-08-30 13:41:13 +00:00
}
2022-08-23 13:15:31 +00:00
2022-09-26 14:28:03 +00:00
table_is_valid = true ;
return ;
2022-08-23 13:15:31 +00:00
}
2022-08-30 13:50:02 +00:00
throw Exception ( ErrorCodes : : BAD_ARGUMENTS , " Cannot create metadata for table, because it is removed concurrently or because of wrong root_path ({}) " , root_path) ;
2022-07-26 09:08:55 +00:00
}
2022-07-27 13:20:45 +00:00
2022-07-26 09:08:55 +00:00
Pipe StorageKeeperMap : : read (
const Names & column_names ,
2022-07-27 13:20:45 +00:00
const StorageSnapshotPtr & storage_snapshot ,
2022-07-26 09:08:55 +00:00
SelectQueryInfo & query_info ,
2022-08-23 13:15:31 +00:00
ContextPtr context_ ,
2022-07-26 09:08:55 +00:00
QueryProcessingStage : : Enum /*processed_stage*/ ,
size_t max_block_size ,
2022-07-27 13:20:45 +00:00
unsigned num_streams )
2022-07-26 09:08:55 +00:00
{
2022-08-23 13:15:31 +00:00
checkTable < true > ( ) ;
2022-07-27 13:20:45 +00:00
storage_snapshot - > check ( column_names ) ;
FieldVectorPtr filtered_keys ;
bool all_scan ;
Block sample_block = storage_snapshot - > metadata - > getSampleBlock ( ) ;
auto primary_key_type = sample_block . getByName ( primary_key ) . type ;
2022-08-23 13:15:31 +00:00
std : : tie ( filtered_keys , all_scan ) = getFilterKeys ( primary_key , primary_key_type , query_info , context_ ) ;
2022-07-27 13:20:45 +00:00
const auto process_keys = [ & ] < typename KeyContainerPtr > ( KeyContainerPtr keys ) - > Pipe
2022-07-26 09:08:55 +00:00
{
2022-07-27 13:20:45 +00:00
if ( keys - > empty ( ) )
return { } ;
: : sort ( keys - > begin ( ) , keys - > end ( ) ) ;
keys - > erase ( std : : unique ( keys - > begin ( ) , keys - > end ( ) ) , keys - > end ( ) ) ;
2022-07-26 09:08:55 +00:00
2022-07-27 13:20:45 +00:00
Pipes pipes ;
2022-07-26 09:08:55 +00:00
2022-07-27 13:20:45 +00:00
size_t num_keys = keys - > size ( ) ;
size_t num_threads = std : : min < size_t > ( num_streams , keys - > size ( ) ) ;
assert ( num_keys < = std : : numeric_limits < uint32_t > : : max ( ) ) ;
assert ( num_threads < = std : : numeric_limits < uint32_t > : : max ( ) ) ;
for ( size_t thread_idx = 0 ; thread_idx < num_threads ; + + thread_idx )
{
size_t begin = num_keys * thread_idx / num_threads ;
size_t end = num_keys * ( thread_idx + 1 ) / num_threads ;
using KeyContainer = typename KeyContainerPtr : : element_type ;
pipes . emplace_back ( std : : make_shared < StorageKeeperMapSource < KeyContainer > > (
* this , sample_block , max_block_size , keys , keys - > begin ( ) + begin , keys - > begin ( ) + end ) ) ;
}
return Pipe : : unitePipes ( std : : move ( pipes ) ) ;
} ;
2022-08-31 08:14:28 +00:00
auto client = getClient ( ) ;
2022-07-27 13:20:45 +00:00
if ( all_scan )
2022-09-12 14:43:52 +00:00
return process_keys ( std : : make_shared < std : : vector < std : : string > > ( client - > getChildren ( data_path ) ) ) ;
2022-07-27 13:20:45 +00:00
return process_keys ( std : : move ( filtered_keys ) ) ;
2022-07-26 09:08:55 +00:00
}
2022-07-27 13:20:45 +00:00
SinkToStoragePtr StorageKeeperMap : : write ( const ASTPtr & /*query*/ , const StorageMetadataPtr & metadata_snapshot , ContextPtr /*context*/ )
2022-07-26 09:08:55 +00:00
{
2022-08-23 13:15:31 +00:00
checkTable < true > ( ) ;
2022-07-27 13:20:45 +00:00
return std : : make_shared < StorageKeeperMapSink > ( * this , metadata_snapshot ) ;
2022-07-26 09:08:55 +00:00
}
2022-08-23 13:15:31 +00:00
void StorageKeeperMap : : truncate ( const ASTPtr & , const StorageMetadataPtr & , ContextPtr , TableExclusiveLockHolder & )
2022-08-10 07:24:56 +00:00
{
2022-08-23 13:15:31 +00:00
checkTable < true > ( ) ;
2022-08-10 07:24:56 +00:00
auto client = getClient ( ) ;
2022-09-12 14:43:52 +00:00
client - > tryRemoveChildrenRecursive ( data_path , true ) ;
2022-08-10 07:24:56 +00:00
}
2022-08-31 08:23:44 +00:00
bool StorageKeeperMap : : dropTable ( zkutil : : ZooKeeperPtr zookeeper , const zkutil : : EphemeralNodeHolder : : Ptr & metadata_drop_lock )
2022-08-24 17:27:07 +00:00
{
2022-09-26 14:28:03 +00:00
zookeeper - > removeChildrenRecursive ( data_path ) ;
2022-08-31 08:23:44 +00:00
2022-09-26 14:28:03 +00:00
bool completely_removed = false ;
2022-08-24 17:27:07 +00:00
Coordination : : Requests ops ;
ops . emplace_back ( zkutil : : makeRemoveRequest ( metadata_drop_lock - > getPath ( ) , - 1 ) ) ;
ops . emplace_back ( zkutil : : makeRemoveRequest ( dropped_path , - 1 ) ) ;
2022-09-12 14:43:52 +00:00
ops . emplace_back ( zkutil : : makeRemoveRequest ( data_path , - 1 ) ) ;
2022-08-24 17:27:07 +00:00
ops . emplace_back ( zkutil : : makeRemoveRequest ( metadata_path , - 1 ) ) ;
Coordination : : Responses responses ;
auto code = zookeeper - > tryMulti ( ops , responses ) ;
using enum Coordination : : Error ;
switch ( code )
{
case ZOK :
{
metadata_drop_lock - > setAlreadyRemoved ( ) ;
2022-09-26 14:28:03 +00:00
completely_removed = true ;
2022-09-12 14:43:52 +00:00
LOG_INFO ( log , " Metadata ({}) and data ({}) was successfully removed from ZooKeeper " , metadata_path , data_path ) ;
2022-08-24 17:27:07 +00:00
break ;
}
case ZNONODE :
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " There is a race condition between creation and removal of metadata. It's a bug " ) ;
case ZNOTEMPTY :
2022-09-26 14:28:03 +00:00
LOG_ERROR ( log , " Metadata was not completely removed from ZooKeeper " ) ;
2022-08-24 17:27:07 +00:00
break ;
default :
zkutil : : KeeperMultiException : : check ( code , ops , responses ) ;
2022-08-30 09:19:59 +00:00
break ;
2022-08-24 17:27:07 +00:00
}
2022-09-26 14:28:03 +00:00
return completely_removed ;
2022-08-24 17:27:07 +00:00
}
2022-08-10 07:24:56 +00:00
void StorageKeeperMap : : drop ( )
{
2022-08-23 13:15:31 +00:00
checkTable < true > ( ) ;
2022-08-10 07:24:56 +00:00
auto client = getClient ( ) ;
2022-08-23 13:15:31 +00:00
2022-09-21 13:08:15 +00:00
// we allow ZNONODE in case we got hardware error on previous drop
if ( auto code = client - > tryRemove ( table_path ) ; code = = Coordination : : Error : : ZNOTEMPTY )
{
throw zkutil : : KeeperException (
code , " {} contains children which shouldn't happen. Please DETACH the table if you want to delete it " , table_path ) ;
}
2022-08-30 09:19:59 +00:00
2022-09-21 13:08:15 +00:00
std : : vector < std : : string > children ;
2022-09-21 13:17:44 +00:00
// if the tables_path is not found, some other table removed it
2022-09-21 13:08:15 +00:00
// if there are children, some other tables are still using this path as storage
if ( auto code = client - > tryGetChildren ( tables_path , children ) ;
code ! = Coordination : : Error : : ZOK | | ! children . empty ( ) )
2022-08-30 09:19:59 +00:00
return ;
Coordination : : Requests ops ;
Coordination : : Responses responses ;
2022-08-23 13:15:31 +00:00
2022-08-30 09:19:59 +00:00
ops . emplace_back ( zkutil : : makeRemoveRequest ( tables_path , - 1 ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( dropped_path , " " , zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( dropped_lock_path , " " , zkutil : : CreateMode : : Ephemeral ) ) ;
auto code = client - > tryMulti ( ops , responses ) ;
if ( code = = Coordination : : Error : : ZNONODE | | code = = Coordination : : Error : : ZNODEEXISTS )
2022-08-23 13:15:31 +00:00
{
2022-08-30 09:19:59 +00:00
LOG_INFO ( log , " Metadata is being removed by another table " ) ;
2022-08-23 13:15:31 +00:00
return ;
}
2022-08-30 09:19:59 +00:00
else if ( code = = Coordination : : Error : : ZNOTEMPTY )
{
LOG_WARNING ( log , " Another table is using the same path, metadata will not be deleted " ) ;
return ;
}
else if ( code ! = Coordination : : Error : : ZOK )
zkutil : : KeeperMultiException : : check ( code , ops , responses ) ;
2022-08-23 13:15:31 +00:00
2022-08-30 09:19:59 +00:00
auto metadata_drop_lock = zkutil : : EphemeralNodeHolder : : existing ( dropped_lock_path , * client ) ;
2022-08-31 08:23:44 +00:00
dropTable ( client , metadata_drop_lock ) ;
2022-08-10 07:24:56 +00:00
}
2022-08-31 08:14:28 +00:00
zkutil : : ZooKeeperPtr StorageKeeperMap : : getClient ( ) const
2022-07-26 09:08:55 +00:00
{
2022-08-23 13:15:31 +00:00
std : : lock_guard lock { zookeeper_mutex } ;
if ( ! zookeeper_client | | zookeeper_client - > expired ( ) )
2022-08-03 13:34:14 +00:00
{
2022-08-23 13:15:31 +00:00
zookeeper_client = nullptr ;
if ( zookeeper_name = = " default " )
zookeeper_client = getContext ( ) - > getZooKeeper ( ) ;
else
zookeeper_client = getContext ( ) - > getAuxiliaryZooKeeper ( zookeeper_name ) ;
2022-08-31 08:29:40 +00:00
2022-09-12 14:43:52 +00:00
zookeeper_client - > sync ( root_path ) ;
2022-08-03 13:34:14 +00:00
}
2022-07-26 09:08:55 +00:00
return zookeeper_client ;
}
2022-09-12 14:43:52 +00:00
const std : : string & StorageKeeperMap : : dataPath ( ) const
2022-07-26 09:08:55 +00:00
{
2022-09-12 14:43:52 +00:00
return data_path ;
2022-07-26 09:08:55 +00:00
}
2022-07-27 13:20:45 +00:00
std : : string StorageKeeperMap : : fullPathForKey ( const std : : string_view key ) const
{
2022-09-12 14:43:52 +00:00
return fs : : path ( data_path ) / key ;
2022-07-27 13:20:45 +00:00
}
2022-08-31 08:14:28 +00:00
UInt64 StorageKeeperMap : : keysLimit ( ) const
{
return keys_limit ;
}
2022-08-23 13:15:31 +00:00
std : : optional < bool > StorageKeeperMap : : isTableValid ( ) const
2022-08-08 09:43:29 +00:00
{
2022-08-23 13:15:31 +00:00
std : : lock_guard lock { init_mutex } ;
if ( table_is_valid . has_value ( ) )
return * table_is_valid ;
2022-09-08 12:45:36 +00:00
[ & ]
2022-08-23 13:15:31 +00:00
{
2022-09-08 12:45:36 +00:00
try
{
auto client = getClient ( ) ;
2022-08-23 13:15:31 +00:00
2022-09-08 12:45:36 +00:00
std : : string stored_metadata_string ;
Coordination : : Stat metadata_stat ;
client - > tryGet ( metadata_path , stored_metadata_string , & metadata_stat ) ;
2022-08-23 13:15:31 +00:00
2022-09-08 12:45:36 +00:00
if ( metadata_stat . numChildren = = 0 )
{
table_is_valid = false ;
return ;
}
if ( metadata_string ! = stored_metadata_string )
{
LOG_ERROR (
log ,
2022-09-08 12:57:34 +00:00
" Table definition does not match to the one stored in the path {}. Stored definition: {} " ,
2022-09-08 12:45:36 +00:00
root_path ,
stored_metadata_string ) ;
table_is_valid = false ;
return ;
}
2022-09-12 14:43:52 +00:00
// validate all metadata and data nodes are present
2022-09-08 12:45:36 +00:00
Coordination : : Requests requests ;
requests . push_back ( zkutil : : makeCheckRequest ( table_path , - 1 ) ) ;
2022-09-12 14:43:52 +00:00
requests . push_back ( zkutil : : makeCheckRequest ( data_path , - 1 ) ) ;
2022-09-08 12:45:36 +00:00
requests . push_back ( zkutil : : makeCheckRequest ( dropped_path , - 1 ) ) ;
Coordination : : Responses responses ;
client - > tryMulti ( requests , responses ) ;
2022-09-12 14:43:52 +00:00
table_is_valid = false ;
if ( responses [ 0 ] - > error ! = Coordination : : Error : : ZOK )
{
LOG_ERROR ( log , " Table node ({}) is missing " , table_path ) ;
return ;
}
if ( responses [ 1 ] - > error ! = Coordination : : Error : : ZOK )
{
LOG_ERROR ( log , " Data node ({}) is missing " , data_path ) ;
return ;
}
if ( responses [ 2 ] - > error = = Coordination : : Error : : ZOK )
{
LOG_ERROR ( log , " Tables with root node {} are being dropped " , root_path ) ;
return ;
}
table_is_valid = true ;
2022-09-08 12:45:36 +00:00
}
catch ( const Coordination : : Exception & e )
{
tryLogCurrentException ( log ) ;
if ( ! Coordination : : isHardwareError ( e . code ) )
table_is_valid = false ;
}
} ( ) ;
2022-08-23 13:15:31 +00:00
return table_is_valid ;
2022-08-08 09:43:29 +00:00
}
2022-08-21 17:09:36 +00:00
Chunk StorageKeeperMap : : getByKeys ( const ColumnsWithTypeAndName & keys , PaddedPODArray < UInt8 > & null_map , const Names & ) const
2022-07-27 13:20:45 +00:00
{
if ( keys . size ( ) ! = 1 )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " StorageKeeperMap supports only one key, got: {} " , keys . size ( ) ) ;
auto raw_keys = serializeKeysToRawString ( keys [ 0 ] ) ;
if ( raw_keys . size ( ) ! = keys [ 0 ] . column - > size ( ) )
2022-09-01 07:56:06 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Assertion failed: {} != {} " , raw_keys . size ( ) , keys [ 0 ] . column - > size ( ) ) ;
2022-07-27 13:20:45 +00:00
return getBySerializedKeys ( raw_keys , & null_map ) ;
}
Chunk StorageKeeperMap : : getBySerializedKeys ( const std : : span < const std : : string > keys , PaddedPODArray < UInt8 > * null_map ) const
{
Block sample_block = getInMemoryMetadataPtr ( ) - > getSampleBlock ( ) ;
MutableColumns columns = sample_block . cloneEmptyColumns ( ) ;
size_t primary_key_pos = getPrimaryKeyPos ( sample_block , getPrimaryKey ( ) ) ;
if ( null_map )
{
null_map - > clear ( ) ;
null_map - > resize_fill ( keys . size ( ) , 1 ) ;
}
auto client = getClient ( ) ;
2022-10-11 09:27:46 +00:00
Strings full_key_paths ;
full_key_paths . reserve ( keys . size ( ) ) ;
2022-07-27 13:20:45 +00:00
for ( const auto & key : keys )
{
2022-10-11 09:27:46 +00:00
full_key_paths . emplace_back ( fullPathForKey ( key ) ) ;
2022-07-27 13:20:45 +00:00
}
2022-10-17 07:29:22 +00:00
auto values = client - > tryGet ( full_key_paths ) ;
2022-07-27 13:20:45 +00:00
for ( size_t i = 0 ; i < keys . size ( ) ; + + i )
{
2022-10-11 09:27:46 +00:00
auto response = values [ i ] ;
2022-07-27 13:20:45 +00:00
Coordination : : Error code = response . error ;
if ( code = = Coordination : : Error : : ZOK )
{
2022-09-01 07:56:06 +00:00
fillColumns ( base64Decode ( keys [ i ] , true ) , response . data , primary_key_pos , sample_block , columns ) ;
2022-07-27 13:20:45 +00:00
}
else if ( code = = Coordination : : Error : : ZNONODE )
{
if ( null_map )
{
( * null_map ) [ i ] = 0 ;
for ( size_t col_idx = 0 ; col_idx < sample_block . columns ( ) ; + + col_idx )
columns [ col_idx ] - > insert ( sample_block . getByPosition ( col_idx ) . type - > getDefault ( ) ) ;
}
}
else
{
throw DB : : Exception ( ErrorCodes : : KEEPER_EXCEPTION , " Failed to fetch value: {} " , code ) ;
}
}
size_t num_rows = columns . at ( 0 ) - > size ( ) ;
return Chunk ( std : : move ( columns ) , num_rows ) ;
}
2022-08-21 17:09:36 +00:00
Block StorageKeeperMap : : getSampleBlock ( const Names & ) const
{
auto metadata = getInMemoryMetadataPtr ( ) ;
2022-09-08 12:45:36 +00:00
return metadata - > getSampleBlock ( ) ;
}
void StorageKeeperMap : : checkTableCanBeRenamed ( const StorageID & new_name ) const
{
verifyTableId ( new_name ) ;
}
void StorageKeeperMap : : rename ( const String & /*new_path_to_table_data*/ , const StorageID & new_table_id )
{
checkTableCanBeRenamed ( new_table_id ) ;
2022-09-12 14:43:52 +00:00
renameInMemory ( new_table_id ) ;
2022-08-21 17:09:36 +00:00
}
2022-07-27 13:20:45 +00:00
namespace
{
2022-08-23 13:15:31 +00:00
2022-08-31 08:14:28 +00:00
StoragePtr create ( const StorageFactory : : Arguments & args )
{
ASTs & engine_args = args . engine_args ;
2022-09-08 12:45:36 +00:00
if ( engine_args . empty ( ) | | engine_args . size ( ) > 2 )
2022-08-31 08:14:28 +00:00
throw Exception (
ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ,
2022-09-01 07:56:06 +00:00
" Storage KeeperMap requires 1-3 arguments: \n "
2022-08-31 08:14:28 +00:00
" root_path: path in the Keeper where the values will be stored (required) \n "
2022-09-01 07:56:06 +00:00
" keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0) " ) ;
2022-08-31 08:14:28 +00:00
2022-09-01 14:39:57 +00:00
const auto root_path_node = evaluateConstantExpressionAsLiteral ( engine_args [ 0 ] , args . getLocalContext ( ) ) ;
auto root_path = checkAndGetLiteralArgument < std : : string > ( root_path_node , " root_path " ) ;
2022-08-31 08:14:28 +00:00
UInt64 keys_limit = 0 ;
2022-09-08 12:45:36 +00:00
if ( engine_args . size ( ) > 1 )
keys_limit = checkAndGetLiteralArgument < UInt64 > ( engine_args [ 1 ] , " keys_limit " ) ;
2022-08-31 08:14:28 +00:00
StorageInMemoryMetadata metadata ;
metadata . setColumns ( args . columns ) ;
metadata . setConstraints ( args . constraints ) ;
if ( ! args . storage_def - > primary_key )
throw Exception ( " StorageKeeperMap requires one column in primary key " , ErrorCodes : : BAD_ARGUMENTS ) ;
metadata . primary_key = KeyDescription : : getKeyFromAST ( args . storage_def - > primary_key - > ptr ( ) , metadata . columns , args . getContext ( ) ) ;
auto primary_key_names = metadata . getColumnsRequiredForPrimaryKey ( ) ;
if ( primary_key_names . size ( ) ! = 1 )
throw Exception ( " StorageKeeperMap requires one column in primary key " , ErrorCodes : : BAD_ARGUMENTS ) ;
return std : : make_shared < StorageKeeperMap > (
2022-09-08 12:45:36 +00:00
args . getContext ( ) , args . table_id , metadata , args . query . attach , primary_key_names [ 0 ] , root_path , keys_limit ) ;
2022-08-31 08:14:28 +00:00
}
2022-08-23 13:15:31 +00:00
2022-07-27 13:20:45 +00:00
}
2022-07-26 09:08:55 +00:00
void registerStorageKeeperMap ( StorageFactory & factory )
{
factory . registerStorage (
" KeeperMap " ,
2022-07-27 13:20:45 +00:00
create ,
2022-07-26 09:08:55 +00:00
{
2022-07-27 13:20:45 +00:00
. supports_sort_order = true ,
. supports_parallel_insert = true ,
} ) ;
2022-07-26 09:08:55 +00:00
}
}