diff options
author | komels <komels@yandex-team.ru> | 2022-04-14 13:10:53 +0300 |
---|---|---|
committer | komels <komels@yandex-team.ru> | 2022-04-14 13:10:53 +0300 |
commit | 21c9b0e6b039e9765eb414c406c2b86e8cea6850 (patch) | |
tree | f40ebc18ff8958dfbd189954ad024043ca983ea5 /kikimr/yndx | |
parent | 9a4effa852abe489707139c2b260dccc6f4f9aa9 (diff) | |
download | ydb-21c9b0e6b039e9765eb414c406c2b86e8cea6850.tar.gz |
Final part on compatibility layer: LOGBROKER-7215
ref:777c67aadbf705d19034a09a792b2df61ba53697
Diffstat (limited to 'kikimr/yndx')
31 files changed, 10048 insertions, 0 deletions
diff --git a/kikimr/yndx/api/grpc/CMakeLists.txt b/kikimr/yndx/api/grpc/CMakeLists.txt new file mode 100644 index 0000000000..adcf9dcf68 --- /dev/null +++ b/kikimr/yndx/api/grpc/CMakeLists.txt @@ -0,0 +1,43 @@ + +# This file was gererated by the build system used internally in the Yandex monorepo. +# Only simple modifications are allowed (adding source-files to targets, adding simple properties +# like target_include_directories). These modifications will be ported to original +# ya.make files by maintainers. Any complex modifications which can't be ported back to the +# original buildsystem will not be accepted. + + + +add_library(api-grpc-yndx) +set_property(TARGET api-grpc-yndx PROPERTY + PROTOC_EXTRA_OUTS .grpc.pb.cc .grpc.pb.h +) +target_link_libraries(api-grpc-yndx PUBLIC + contrib-libs-cxxsupp + yutil + contrib-libs-grpc + api-protos-yndx + api-protos + contrib-libs-protobuf +) +target_proto_messages(api-grpc-yndx PRIVATE + ${CMAKE_SOURCE_DIR}/kikimr/yndx/api/grpc/persqueue.proto + ${CMAKE_SOURCE_DIR}/kikimr/yndx/api/grpc/ydb_yndx_keyvalue_v1.proto + ${CMAKE_SOURCE_DIR}/kikimr/yndx/api/grpc/ydb_yndx_rate_limiter_v1.proto +) +target_proto_addincls(api-grpc-yndx + ./ + ${CMAKE_SOURCE_DIR}/ + ${CMAKE_BINARY_DIR} + ${CMAKE_SOURCE_DIR} + ${CMAKE_SOURCE_DIR}/contrib/libs/protobuf/src + ${CMAKE_BINARY_DIR} + ${CMAKE_SOURCE_DIR}/contrib/libs/protobuf/src +) +target_proto_outs(api-grpc-yndx + --cpp_out=${CMAKE_BINARY_DIR}/ + --cpp_styleguide_out=${CMAKE_BINARY_DIR}/ +) +target_proto_plugin(api-grpc-yndx + grpc_cpp + grpc_cpp +) diff --git a/kikimr/yndx/api/grpc/persqueue.proto b/kikimr/yndx/api/grpc/persqueue.proto new file mode 100644 index 0000000000..313b42ca3d --- /dev/null +++ b/kikimr/yndx/api/grpc/persqueue.proto @@ -0,0 +1,68 @@ +syntax = "proto3"; + +package NPersQueue; + +option java_package = "com.yandex.persqueue"; +option java_outer_classname = "PersqueueGrpc"; + +import "kikimr/yndx/api/protos/persqueue.proto"; + +service PersQueueService { + + /** + * Creates Write Session + * Pipeline: + * client server + * Init(Topic, SourceId, ...) + * ----------------> + * Init(Partition, MaxSeqNo, ...) + * <---------------- + * write(data1, seqNo1) + * ----------------> + * write(data2, seqNo2) + * ----------------> + * ack(seqNo1, offset1, ...) + * <---------------- + * write(data3, seqNo3) + * ----------------> + * ack(seqNo2, offset2, ...) + * <---------------- + * error(description, errorCode) + * <---------------- + */ + + rpc WriteSession(stream WriteRequest) returns (stream WriteResponse); + + /** + * Creates Read Session + * Pipeline: + * client server + * Init(Topics, ClientId, ...) + * ----------------> + * Init(SessionId) + * <---------------- + * read1 + * ----------------> + * read2 + * ----------------> + * lock(Topic1,Partition1, ...) - locks and releases are optional + * <---------------- + * lock(Topic2, Partition2, ...) + * <---------------- + * release(Topic1, Partition1, ...) + * <---------------- + * locked(Topic2, Partition2, ...) - client must respond to lock request with this message. Only after this client will start recieving messages from this partition + * ----------------> + * read result(data, ...) + * <---------------- + * commit(cookie1) + * ----------------> + * commit result(cookie1) + * <---------------- + * error(description, errorCode) + * <---------------- + */ + + rpc ReadSession(stream ReadRequest) returns (stream ReadResponse); + +}
\ No newline at end of file diff --git a/kikimr/yndx/api/grpc/ydb_yndx_keyvalue_v1.proto b/kikimr/yndx/api/grpc/ydb_yndx_keyvalue_v1.proto new file mode 100644 index 0000000000..7b685f5966 --- /dev/null +++ b/kikimr/yndx/api/grpc/ydb_yndx_keyvalue_v1.proto @@ -0,0 +1,43 @@ +syntax = "proto3"; + +package Ydb.Yndx.KeyValue.V1; + +option java_package = "com.yandex.ydb.yndx.keyvalue.v1"; +option java_outer_classname = "KeyValueGrpc"; +option java_multiple_files = true; + +import "kikimr/yndx/api/protos/ydb_yndx_keyvalue.proto"; + +// KeyValue tablets provide a simple key-value storage in a low-overhead and easy-to-shoot-your-leg manner. +// To use KeyValue tablets in an efficient way one must be familiar with the design of both the KeyValue tablet +// and the Distributed Storage underneath it. + +service KeyValueService { + + // Create a keyvalue volume by the path and a count of partitions + rpc CreateVolume(KeyValue.CreateVolumeRequest) returns (KeyValue.CreateVolumeResponse); + + // Drop the keyvalue volume by the path + rpc DropVolume(KeyValue.DropVolumeRequest) returns (KeyValue.DropVolumeResponse); + + // List partitions of keyvalue volume in the local node. + rpc ListLocalPartitions(KeyValue.ListLocalPartitionsRequest) returns (KeyValue.ListLocalPartitionsResponse); + + // Obtains an exclusive lock for the tablet. + rpc AcquireLock(KeyValue.AcquireLockRequest) returns (KeyValue.AcquireLockResponse); + + // Performs one or more actions that modify the state of the tablet as an atomic transaction. + rpc ExecuteTransaction(KeyValue.ExecuteTransactionRequest) returns (KeyValue.ExecuteTransactionResponse); + + // Reads value stored in the item with the key specified. + rpc Read(KeyValue.ReadRequest) returns (KeyValue.ReadResponse); + + // Reads a list of items with the keys in the range specified. + rpc ReadRange(KeyValue.ReadRangeRequest) returns (KeyValue.ReadRangeResponse); + + // List existed items with the keys in the range specified. + rpc ListRange(KeyValue.ListRangeRequest) returns (KeyValue.ListRangeResponse); + + // Gets storage channel status of the tablet. + rpc GetStorageChannelStatus(KeyValue.GetStorageChannelStatusRequest) returns (KeyValue.GetStorageChannelStatusResponse); +} diff --git a/kikimr/yndx/api/grpc/ydb_yndx_rate_limiter_v1.proto b/kikimr/yndx/api/grpc/ydb_yndx_rate_limiter_v1.proto new file mode 100644 index 0000000000..9e0c2e97dd --- /dev/null +++ b/kikimr/yndx/api/grpc/ydb_yndx_rate_limiter_v1.proto @@ -0,0 +1,35 @@ +syntax = "proto3"; + +package Ydb.Yndx.RateLimiter.V1; + +option java_package = "com.yandex.ydb.yndx.rate_limiter.v1"; +option java_outer_classname = "YndxRateLimiterGrpc"; +option java_multiple_files = true; + +import "kikimr/yndx/api/protos/ydb_yndx_rate_limiter.proto"; + +// Service that implements distributed rate limiting and accounting. +// +// To use rate limiter functionality you need an existing coordination node. + +service YndxRateLimiterService { + // Control plane API + + // Create a new resource in existing coordination node. + rpc CreateResource(CreateResourceRequest) returns (CreateResourceResponse); + + // Update a resource in coordination node. + rpc AlterResource(AlterResourceRequest) returns (AlterResourceResponse); + + // Delete a resource from coordination node. + rpc DropResource(DropResourceRequest) returns (DropResourceResponse); + + // List resources in given coordination node. + rpc ListResources(ListResourcesRequest) returns (ListResourcesResponse); + + // Describe properties of resource in coordination node. + rpc DescribeResource(DescribeResourceRequest) returns (DescribeResourceResponse); + + // Take units for usage of a resource in coordination node. + rpc AcquireResource(AcquireResourceRequest) returns (AcquireResourceResponse); +} diff --git a/kikimr/yndx/api/protos/CMakeLists.txt b/kikimr/yndx/api/protos/CMakeLists.txt new file mode 100644 index 0000000000..3f858c162c --- /dev/null +++ b/kikimr/yndx/api/protos/CMakeLists.txt @@ -0,0 +1,40 @@ + +# This file was gererated by the build system used internally in the Yandex monorepo. +# Only simple modifications are allowed (adding source-files to targets, adding simple properties +# like target_include_directories). These modifications will be ported to original +# ya.make files by maintainers. Any complex modifications which can't be ported back to the +# original buildsystem will not be accepted. + + + +add_library(api-protos-yndx) +target_link_libraries(api-protos-yndx PUBLIC + contrib-libs-cxxsupp + yutil + api-protos + tools-enum_parser-enum_serialization_runtime + contrib-libs-protobuf +) +target_proto_messages(api-protos-yndx PRIVATE + ${CMAKE_SOURCE_DIR}/kikimr/yndx/api/protos/persqueue.proto + ${CMAKE_SOURCE_DIR}/kikimr/yndx/api/protos/ydb_yndx_keyvalue.proto + ${CMAKE_SOURCE_DIR}/kikimr/yndx/api/protos/ydb_yndx_rate_limiter.proto +) +generate_enum_serilization(api-protos-yndx + ${CMAKE_BINARY_DIR}/kikimr/yndx/api/protos/persqueue.pb.h + INCLUDE_HEADERS + kikimr/yndx/api/protos/persqueue.pb.h +) +target_proto_addincls(api-protos-yndx + ./ + ${CMAKE_SOURCE_DIR}/ + ${CMAKE_BINARY_DIR} + ${CMAKE_SOURCE_DIR} + ${CMAKE_SOURCE_DIR}/contrib/libs/protobuf/src + ${CMAKE_BINARY_DIR} + ${CMAKE_SOURCE_DIR}/contrib/libs/protobuf/src +) +target_proto_outs(api-protos-yndx + --cpp_out=${CMAKE_BINARY_DIR}/ + --cpp_styleguide_out=${CMAKE_BINARY_DIR}/ +) diff --git a/kikimr/yndx/api/protos/persqueue.proto b/kikimr/yndx/api/protos/persqueue.proto new file mode 100644 index 0000000000..e8da4c4a36 --- /dev/null +++ b/kikimr/yndx/api/protos/persqueue.proto @@ -0,0 +1,335 @@ +syntax = "proto3"; +import "google/protobuf/descriptor.proto"; +import "ydb/public/api/protos/draft/persqueue_common.proto"; + +package NPersQueue; + +option java_package = "com.yandex.ydb.persqueue"; +option cc_enable_arenas = true; + +extend google.protobuf.FileOptions { + bool GenerateYaStyle = 66677; +} + +message Path { + // Path of object (topic/consumer). + string path = 1; +} + +// WRITE REQUEST + +message KeyValue { + string key = 1; + string value = 2; +} + +message MapType { + repeated KeyValue items = 1; +} + +/** + * Request for write session. Contains one of : + * Init - consists of initialization info - Topic, SourceId and so on + * Data - data to be writen + * DataBatch - batch of data to be written + */ +message WriteRequest { + message Init { + string topic = 1; + bytes source_id = 2; + + MapType extra_fields = 7; //server and file inside here + + uint64 proxy_cookie = 8; //cookie provided by ChooseProxy request //change to bytes + + uint32 partition_group = 12; //Group to write to - 0 means any; + + string version = 999; //must be filled by client lib + } + + message Data { + uint64 seq_no = 1; + bytes data = 2; + uint64 create_time_ms = 3; //timestamp in ms + NPersQueueCommon.ECodec codec = 4; + uint32 uncompressed_size = 5; + } + + message DataBatch { + repeated Data data = 1; + } + + oneof request { + //init must be sent as first message + Init init = 1; + Data data = 2; + DataBatch data_batch = 3; + } + + NPersQueueCommon.Credentials credentials = 20; +} + +/** + * Response for write session. Contains one of : + * Error - in any error state - grpc errors, session dies, incorrect Init request and so on + * Init - contains SessionId of created session, MaxSeqNo and Partition + * Ack - acknowlegment of storing corresponding message + * AckBatch - acknowlegment of storing corresponding message batch + */ +message WriteResponse { + message Init { + uint64 max_seq_no = 1; + string session_id = 2; + uint32 partition = 3; + string topic = 4; + } + + message Stat { + uint32 write_time_ms = 1; + uint32 total_time_in_partition_queue_ms = 2; + uint32 partition_quoted_time_ms = 3; + uint32 topic_quoted_time_ms = 4; + } + + message Ack { + uint64 seq_no = 1; + uint64 offset = 2; + bool already_written = 3; + + Stat stat = 4; //not filled in batch case + } + + message AckBatch { + Stat stat = 2; //common statistics for batch storing + + repeated Ack ack = 1; + } + + oneof response { + Init init = 1; + Ack ack = 2; + AckBatch ack_batch = 4; + NPersQueueCommon.Error error = 3; + } +} + +// READ REQUEST + +/** + * Request for read session. Contains one of : + * Init - contains of Topics to be readed, ClientId and other metadata + * Read - request for read batch. Contains of restrictments for result - MaxSize, MaxCount and so on + * Commit - request for commit some read batches. Contains corresponding cookies + * Locked - comfirming to server that client is ready to get data from partition from concreet offset + */ +message ReadRequest { + enum EProtocolVersion { + Base = 0; // Base protocol version + Batching = 1; // Client supports more effective batching structs (TBatchedData instead of TData) + ReadParamsInInit = 2; // Client sets read params in Init request + } + + message Init { + repeated string topics = 1; + bool read_only_local = 2; // ReadOnlyLocal=false - read mirrored topics from other clusters too; will be renamed to read_only_original + + string client_id = 4; + bool clientside_locks_allowed = 5; //if true then partitions Lock signal will be sent from server, + //and reads from partitions will began only after Locked signal recieved by server from client + + uint64 proxy_cookie = 6; //cookie provided by ChooseProxy request + + bool balance_partition_right_now = 8; //if set then do not wait for commits from client on data from partition in case of balancing + + repeated uint32 partition_groups = 9; //Groups to be read - if empty then read from all of them + + uint32 idle_timeout_sec = 10; //TODO: do we need it? + + uint32 commit_interval_ms = 12; // How often server must commit data. If client sends commits faster, + // then server will hold them in order to archive corresponding rate; zero means server default = 1sec + + // Read request params + uint32 max_read_messages_count = 14; // Max messages to give to client in one read request + uint32 max_read_size = 15; // Max size in bytes to give to client in one read request + uint32 max_read_partitions_count = 16; // 0 means not matters // Maximum partitions count to give to client in one read request + uint32 max_time_lag_ms = 17; // Read data only with time lag less than or equal to specified + uint64 read_timestamp_ms = 18; // Read data only after this timestamp + + bool commits_disabled = 19; // Client will never commit + + string version = 999; //must be filled by client lib + + // Protocol version to let server know about new features that client supports + uint32 protocol_version = 13; // version must be integer (not enum) because client may be newer than server + } + + message Read { + // It is not allowed to change these parameters. + // They will be removed in future from TRead structure. + uint32 max_count = 1; + uint32 max_size = 2; + uint32 partitions_at_once = 3; //0 means not matters + uint32 max_time_lag_ms = 5; + uint64 read_timestamp_ms = 6; //read data only after this timestamp + } + + message StartRead { + string topic = 1; + uint32 partition = 2; + + uint64 read_offset = 3; //skip upto this position; if committed position is bigger, then do nothing + bool verify_read_offset = 4; //if true then check that committed position is <= ReadOffset; otherwise it means error in client logic + uint64 generation = 5; + uint64 commit_offset = 6; //all messages BEFORE this position are processed by client + } + + message Commit { + repeated uint64 cookie = 1; + } + + message Status { + uint64 generation = 1; + string topic = 2; + uint32 partition = 3; + } + + oneof request { + //init must be sent as first message + Init init = 1; + Read read = 2; + StartRead start_read = 3; + Commit commit = 4; + Status status = 5; + } + + NPersQueueCommon.Credentials credentials = 20; +} + + +message MessageMeta { + bytes source_id = 1; + uint64 seq_no = 2; + uint64 create_time_ms = 3; + uint64 write_time_ms = 4; + + MapType extra_fields = 7; + NPersQueueCommon.ECodec codec = 8; + string ip = 9; + uint32 uncompressed_size = 10; +} + +/** + * Response for read session. Contains one of : + * Error - in any error state - grpc errors, session dies, incorrect Init request and so on + * Init - contains SessionId of created session + * Data - result of read, contains of messages batch and cookie + * Commit - acknowlegment for commit + * Lock - informs client that server is ready to read data from corresponding partition + * Release - informs client that server will not get data from this partition in future read results, unless other Lock-Locked conversation will be done + */ + +message ReadResponse { + message Init { + string session_id = 2; //for debug only + } + + message Data { + message Message { + MessageMeta meta = 1; //SeqNo ... + bytes data = 2; + //unique value for clientside deduplication - Topic:Partition:Offset + uint64 offset = 3; + bytes broken_packed_data = 4; // TODO: move to pqlib + } + + message MessageBatch { + string topic = 1; + uint32 partition = 2; + repeated Message message = 3; + } + + repeated MessageBatch message_batch = 1; + uint64 cookie = 2; //Cookie to be committed by server + } + + message BatchedData { + message MessageData { + NPersQueueCommon.ECodec codec = 2; + + uint64 offset = 3; //unique value for clientside deduplication - Topic:Partition:Offset + uint64 seq_no = 4; + + uint64 create_time_ms = 5; + uint64 uncompressed_size = 6; + + bytes data = 1; + } + + message Batch { + bytes source_id = 2; + MapType extra_fields = 3; + uint64 write_time_ms = 4; + string ip = 5; + + repeated MessageData message_data = 1; + } + + message PartitionData { + string topic = 2; + uint32 partition = 3; + + repeated Batch batch = 1; + } + + uint64 cookie = 2; //Cookie to be committed by server + + repeated PartitionData partition_data = 1; //not greater than one PartitionData for each partition + } + + message Lock { + string topic = 1; + uint32 partition = 2; + + uint64 read_offset = 3; //offset to read from + uint64 end_offset = 4; //know till this time end offset + uint64 generation = 5; + } + + message Release { + string topic = 1; + uint32 partition = 2; + bool can_commit = 3; //if CanCommit=false then you can not store progress of processing data for that partition at server; + //all commits will have no effect for this partition + //if you rely on committing offsets then just drop all data for this partition without processing - another session will get them later + //if CanCommit=true and you are relying on committing offsets - you can process all data for this partition you got, + //commit cookies and be sure that no other session will ever get this data + uint64 generation = 4; + } + + message Commit { + repeated uint64 cookie = 1; //for debug purposes only + } + + // Response for status requst. + message PartitionStatus { + uint64 generation = 1; + string topic = 2; + uint32 partition = 3; + + uint64 committed_offset = 4; + uint64 end_offset = 5; + uint64 write_watermark_ms = 6; + } + + oneof response { + Init init = 1; + Data data = 2; + BatchedData batched_data = 7; + NPersQueueCommon.Error error = 3; + Lock lock = 4; + Release release = 5; + Commit commit = 6; + PartitionStatus partition_status = 8; + } +} + diff --git a/kikimr/yndx/api/protos/ydb_yndx_keyvalue.proto b/kikimr/yndx/api/protos/ydb_yndx_keyvalue.proto new file mode 100644 index 0000000000..6782a1bfe5 --- /dev/null +++ b/kikimr/yndx/api/protos/ydb_yndx_keyvalue.proto @@ -0,0 +1,460 @@ +syntax = "proto3"; +option cc_enable_arenas = true; + +package Ydb.Yndx.KeyValue; + +option java_package = "com.yandex.ydb.yndx.rate_limiter"; +option java_outer_classname = "YndxkeyValueProtos"; +option java_multiple_files = true; + +import "ydb/public/api/protos/ydb_operation.proto"; + +// +// KeyValue API. +// + +message Flags { + // Free disk space is low. + bool disk_space_cyan = 1; + + // Free disk space is low, it is recommended to stop writing additional data. + bool disk_space_light_yellow_move = 2; + bool disk_space_yellow_stop = 3; + + // Free disk space is very low, clients must stop writing additional data. + bool disk_space_light_orange = 4; + bool disk_space_orange = 5; + + // Free disk space is extremely low, operations other than deletion may not be performed. + bool disk_space_red = 6; + + // No free disk space available. + bool disk_space_black = 7; +} + + +message Statuses { + enum Status { + STATUS_UNSPECIFIED = 0; + STATUS_SUCCESS = 1; + STATUS_NO_DATA = 2; + STATUS_ERROR = 3; + STATUS_OVERRUN = 4; + } +} + + +message StorageChannel { + // XXX + Statuses.Status status = 1; + + // Storage channel index. + uint32 storage_channel = 2; + + // If present, contains the status flags of the storage channel. Empty if status flags could not be obtained. + optional Flags status_flags = 3; +} + + +message Priorities { + enum Priority { + PRIORITY_UNSPECIFIED = 0; + + // High priority for user-initiated operations. + PRIORITY_REALTIME = 1; + + // Low prioroty for background system activity. + PRIORITY_BACKGROUND = 2; + } +} + + +message KVRange { + // The first bound of the range of the keys + // If no one is assigned then specify in order for the range to begin from the lowest key + oneof from_bound { + // Specify in order for the range to include the key specified + string from_key_inclusive = 1; + // Specify in order for the range not to include the key specified + string from_key_exclusive = 2; + } + + // The second bound of the range of the keys + // If no one is assigned then specify in order for the range to end to the highest keys + oneof to_bound { + // Specify in order for the range to include the key specified + string to_key_inclusive = 3; + // Specify in order for the range not to include the key specified + string to_key_exclusive = 4; + } +} + + +message AcquireLockRequest { + Ydb.Operations.OperationParams operation_params = 1; + + // Path to the volume + string path = 2; + uint64 partition_id = 3; +} + + +message AcquireLockResponse { + // Operation contains the result of the request. Check the ydb_operation.proto. + Ydb.Operations.Operation operation = 1; +} + +message AcquireLockResult { + // The generation of the lock to provide as an argument to all the operaions the user performs with the tablet. + uint64 lock_generation = 1; +} + +message ExecuteTransactionRequest { + message Command { + message Rename { + // The key to change. + string old_key = 1; + + // The new key to change the old key to. + string new_key = 2; + } + message Concat { + // Keys to use as the source for the concatenation. + repeated string input_keys = 1; + + // New key to use for the result of the concatenation + string output_key = 2; + + // Input keys are deleted after the concatenation by default. In order to keep both the inputs and the + // output, set keep_inputs to true. + bool keep_inputs = 3; + } + + // Makes a copy of a range of key-value pairs. New keys are formed by removing a prefix and/or prepending a new + // prefix. For example, copy of the key-value pairs [{aaabc,1},{aaaef,2}] can be stripped of the 'aa' prefix and + // prepended with the 'x' so that the new pairs are [{xabc, 1}, {xaef, 2}]. + message CopyRange { + // The range of keys to copy + KVRange range = 1; + + // For each source key that begins with the prefix_to_remove, that prefix is removed from the new key before + // prepending it with the prefix_to_add. Acts as filter if not empty. + string prefix_to_remove = 2; + + // The prefix_to_add prefix is prepended to each new key. + string prefix_to_add = 3; + } + message Write { + enum Tactic { + TACTIC_UNSPECIFIED = 0; + + // Write minimum required redundant data. Does not affect storage durability. + TACTIC_MAX_THROUGHPUT = 1; + + // Write additional redundant data to more disks to reduce operation duration. Does not affect storage + // durability, but will use additional space. + TACTIC_MIN_LATENCY = 2; + } + // Key of the key-value pair to write. + string key = 1; + + // Value of the key-value pair to write. + bytes value = 2; + + // Storage channel to write the value to. Channel numbers begin with 1 and may go up to approximately 250 + // (depends on the channel configuration of each tablet). + // Channel 1 is called the INLINE channel (value is stored in the index table). + // Channel 2 is called the MAIN channel (value is stored as a separate blob in the Distributed Storage). + // Channels 1 and 2 are available for all tablets. + // If the storage channel specified is not configured for the tablet, the value is stored in + // channel 2 (the MAIN channel). + uint32 storage_channel = 3; // (default = 0 is same as 2 or MAIN) + + // Priority to use for the Distributed Storage Get operation. Has no effect for the 1st (inline) storage + // channel. Defaults to PRIORITY_UNSPECIFIED which interpreted like PRIORITY_REALTIME. + Priorities.Priority priority = 4; + + // Tactic to use for the Distributed Storage Put operation. Has no effect for the 1st (inline) storage + // channel. Defaults to TACTIC_UNSPECIFIED which interpreted like TACTIC_MAX_THROUGHPUT. + Tactic tactic = 5; + } + message DeleteRange { + // The range of keys to delete + KVRange range = 1; + } + + oneof action { + // Deletes key-value pairs with keys in the range specified. + DeleteRange delete_range = 1; + + // Changes the key of a key-value pair. + Rename rename = 2; + + // Creates a copy of key-value pairs with keys in the range specified by removin and/or prepending a prefix + // specified to each key. + CopyRange copy_range = 3; + + // Creates a new key-value pair with key specified by concatenating values of multiple other key-value pairs + // with keys specified. + Concat concat = 4; + + // Creates a new key-value pair with key and value specified. + Write write = 5; + } + } + + Ydb.Operations.OperationParams operation_params = 1; + + // Path to the volume + string path = 2; + uint64 partition_id = 3; + + // Generation of the exclusive lock obtained for the tablet as a result of an AcquireLock call. + uint64 lock_generation = 4; + + // Commands to execute as a single atomic transaction. The order of execution of commands is the same as the order + // of commands in the ExecuteTransactionRequest. Order of execution of different transactions is not specified. + repeated Command commands = 5; +} + +message ExecuteTransactionResponse { + // Operation contains the result of the request. Check the ydb_operation.proto. + Ydb.Operations.Operation operation = 1; +} + +message ExecuteTransactionResult { + repeated StorageChannel storage_channel = 1; +} + +message ReadRequest { + Ydb.Operations.OperationParams operation_params = 1; + + // Path to the volume + string path = 2; + uint64 partition_id = 3; + + // Generation of the exclusive lock obtained for the tablet as a result of an AcquireLock call. + uint64 lock_generation = 4; + + // Key of the key-value pair to read. + string key = 5; + + // Offset in bytes from the beginning of the value to read data from. + uint64 offset = 6; + + // Size of the data to read in bytes. 0 means "read to the end of the value". + uint64 size = 7; + + // Result protobuf size limit. If not 0, overrides the default one only with a smaller value. + uint64 limit_bytes = 8; + + // Priority to use for the Distributed Storage Get operation. Has no effect for the 1st (inline) storage + // channel. Defaults to PRIORITY_UNSPECIFIED which interpreted like PRIORITY_REALTIME. + Priorities.Priority priority = 9; +} + +message ReadResponse { + // Operation contains the result of the request. Check the ydb_operation.proto. + Ydb.Operations.Operation operation = 1; +} + +message ReadResult { + // The key of the requested key-value pair + string requested_key = 1; + + // Offset in bytes from the beginning of the value requested + uint64 requested_offset = 2; + + // Size of the data requested + uint64 requested_size = 3; + + // The bytes of the requested part of the value of the requested key-value pair + bytes value = 4; + + // XXX + string msg = 5; + + Statuses.Status status = 6; +} + +message ReadRangeRequest { + Ydb.Operations.OperationParams operation_params = 1; + + // Path to the volume + string path = 2; + uint64 partition_id = 3; + + // Generation of the exclusive lock obtained for the tablet as a result of an AcquireLock call. + uint64 lock_generation = 4; + + // The range of keys to read + KVRange range = 5; + + // Result protobuf size limit. If not 0, overrides the default one only with a smaller value. + uint64 limit_bytes = 6; + + // Priority to use for the Distributed Storage Get operation. Has no effect for the 1st (inline) storage + // channel. Defaults to PRIORITY_UNSPECIFIED which interpreted like PRIORITY_REALTIME. + Priorities.Priority priority = 7; +} + +message ReadRangeResponse { + // Operation contains the result of the request. Check the ydb_operation.proto. + Ydb.Operations.Operation operation = 1; +} + +message ReadRangeResult { + message KeyValuePair { + // The key of the key-value pair. + string key = 1; + + // The value of the key-value pair. Present only if the request was performed with include_data set to true. + bytes value = 2; + + // Full size of the value of the key-value pair. + uint32 value_size = 3; + + // Unix time of the creation of the key-value pair (in ms). + uint64 creation_unix_time = 4; + + // Contains the index of the actualy used storage channel. The actually used storage channel may differ from + // the value specified in the write request for example if there were no such storage channel at the moment + // of execution of the write command. + // For values created as a result of a concatenation or a copy of such values, the storage channel of the first + // part of the value is specified. + uint32 storage_channel = 5; // Returns the _actual_ storage channel + + Statuses.Status status = 6; + } + Statuses.Status status = 1; + + // List of the key-value pairs and metadata requested. + repeated KeyValuePair pair = 2; +} + +message ListRangeRequest { + Ydb.Operations.OperationParams operation_params = 1; + + // Path to the volume + string path = 2; + uint64 partition_id = 3; + + // Generation of the exclusive lock obtained for the tablet as a result of an AcquireLock call. + uint64 lock_generation = 4; + + // The range of keys to read + KVRange range = 5; + + // Result protobuf size limit. If not 0, overrides the default one only with a smaller value. + uint64 limit_bytes = 6; + + // Priority to use for the Distributed Storage Get operation. Has no effect for the 1st (inline) storage + // channel. Defaults to PRIORITY_UNSPECIFIED which interpreted like PRIORITY_REALTIME. + Priorities.Priority priority = 7; +} + +message ListRangeResponse { + // Operation contains the result of the request. Check the ydb_operation.proto. + Ydb.Operations.Operation operation = 1; +} + +message ListRangeResult { + message KeyInfo { + // The key of the key-value pair. + string key = 1; + + // Full size of the value of the key-value pair. + uint32 value_size = 2; + + // Unix time of the creation of the key-value pair (in ms). + uint64 creation_unix_time = 3; + + // Contains the index of the actualy used storage channel. The actually used storage channel may differ from + // the value specified in the write request for example if there were no such storage channel at the moment + // of execution of the write command. + // For values created as a result of a concatenation or a copy of such values, the storage channel of the first + // part of the value is specified. + uint32 storage_channel = 4; // Returns the _actual_ storage channel + } + Statuses.Status status = 1; + + // List of the key-value pairs and metadata requested. + repeated KeyInfo key = 2; +} + +message GetStorageChannelStatusRequest { + Ydb.Operations.OperationParams operation_params = 1; + + // Path to the volume + string path = 2; + uint64 partition_id = 3; + + // Generation of the exclusive lock obtained for the tablet as a result of an AcquireLock call. + uint64 lock_generation = 4; + + // Storage channel index. + repeated uint32 storage_channel = 5; +} + +message GetStorageChannelStatusResponse { + // Operation contains the result of the request. Check the ydb_operation.proto. + Ydb.Operations.Operation operation = 1; +} + +message GetStorageChannelStatusResult { + repeated StorageChannel storage_channel = 1; +} + +message CreateVolumeRequest { + Ydb.Operations.OperationParams operation_params = 1; + + // Path to the volume + string path = 2; + uint32 channel_profile_id = 3; + uint32 partition_count = 4; +} + +message CreateVolumeResponse { + // Operation contains the result of the request. Check the ydb_operation.proto. + Ydb.Operations.Operation operation = 1; +} + +message CreateVolumeResult { +} + +message DropVolumeRequest { + Ydb.Operations.OperationParams operation_params = 1; + + // Path to the volume + string path = 2; +} + +message DropVolumeResponse { + // Operation contains the result of the request. Check the ydb_operation.proto. + Ydb.Operations.Operation operation = 1; +} + +message DropVolumeResult { +} + +message ListLocalPartitionsRequest { + Ydb.Operations.OperationParams operation_params = 1; + + // Path to the volume + string path = 2; + + // If it's zero than it used local node. + uint64 node_id = 3; +} + +message ListLocalPartitionsResponse { + // Operation contains the result of the request. Check the ydb_operation.proto. + Ydb.Operations.Operation operation = 1; +} + +message ListLocalPartitionsResult { + string requested_path = 1; + uint64 node_id = 2; + + repeated uint64 partition_ids = 3; +} diff --git a/kikimr/yndx/api/protos/ydb_yndx_rate_limiter.proto b/kikimr/yndx/api/protos/ydb_yndx_rate_limiter.proto new file mode 100644 index 0000000000..10ba9627da --- /dev/null +++ b/kikimr/yndx/api/protos/ydb_yndx_rate_limiter.proto @@ -0,0 +1,273 @@ +syntax = "proto3"; +option cc_enable_arenas = true; + +package Ydb.Yndx.RateLimiter; + +option java_package = "com.yandex.ydb.yndx.rate_limiter"; +option java_outer_classname = "YndxRateLimiterProtos"; +option java_multiple_files = true; + +import "ydb/public/api/protos/ydb_operation.proto"; + +// +// Rate Limiter control API. +// + +// +// Resource properties. +// + +message AccountingConfig { + // Account consumed resources and send billing metrics. + // Default value is false (not inherited). + bool enabled = 1; + + // Period to report consumption history from clients to kesus + // Default value is inherited from parent or equals 5000 ms for root. + uint64 report_period_ms = 2; + + // Consumption history period that is sent in one message to accounting actor. + // Default value is inherited from parent or equals 1000 ms for root. + uint64 account_period_ms = 3; + + // Time window to collect data from every client. + // Any client account message that is `collect_period` late is discarded (not accounted or billed). + // Default value is inherited from parent or equals 30 seconds for root. + uint64 collect_period_sec = 4; + + // Provisioned consumption limit in units per second. + // Effective value is limited by corresponding `max_units_per_second`. + // Default value is 0 (not inherited). + double provisioned_units_per_second = 5; + + // Provisioned allowed burst equals `provisioned_coefficient * provisioned_units_per_second` units. + // Effective value is limited by corresponding PrefetchCoefficient. + // Default value is inherited from parent or equals 60 for root. + double provisioned_coefficient = 6; + + // On-demand allowed burst equals `overshoot_coefficient * prefetch_coefficient * max_units_per_second` units. + // Should be greater or equal to 1.0 + // Default value is inherited from parent or equals 1.1 for root + double overshoot_coefficient = 7; + + // Billing metric description. + message Metric { + // Send this metric to billing. + // Default value is false (not inherited). + bool enabled = 1; + + // Billing metric period (aligned to hour boundary). + // Default value is inherited from parent or equals 60 seconds for root. + uint64 billing_period_sec = 2; + + // Billing metric JSON fields (inherited from parent if not set) + string version = 3; + string schema = 5; + string cloud_id = 6; + string folder_id = 7; + string resource_id = 8; + string source_id = 9; + } + + // Consumption within provisioned limit. + // Informative metric that should be sent to billing (not billed). + Metric provisioned = 8; + + // Consumption that exceeds provisioned limit is billed as on-demand. + Metric on_demand = 9; + + // Consumption that exceeds even on-demand limit. + // Normally it is free and should not be billed. + Metric overshoot = 10; +} + +// Settings for hierarchical deficit round robin (HDRR) algorithm. +message HierarchicalDrrSettings { + // Resource consumption speed limit. + // Value is required for root resource. + // 0 is equivalent to not set. + // Must be nonnegative. + double max_units_per_second = 1; + + // Maximum burst size of resource consumption across the whole cluster + // divided by max_units_per_second. + // Default value is 1. + // This means that maximum burst size might be equal to max_units_per_second. + // 0 is equivalent to not set. + // Must be nonnegative. + double max_burst_size_coefficient = 2; + + // Prefetch in local bucket up to prefetch_coefficient*max_units_per_second units (full size). + // Default value is inherited from parent or 0.2 for root. + // Disables prefetching if any negative value is set + // (It is useful to avoid bursts in case of large number of local buckets). + double prefetch_coefficient = 3; + + // Prefetching starts if there is less than prefetch_watermark fraction of full local bucket left. + // Default value is inherited from parent or 0.75 for root. + // Must be nonnegative and less than or equal to 1. + double prefetch_watermark = 4; +} + +// Rate limiter resource description. +message Resource { + // Resource path. Elements are separated by slash. + // The first symbol is not slash. + // The first element is root resource name. + // Resource path is the path of resource inside coordination node. + string resource_path = 1; + + oneof type { + // Settings for Hierarchical DRR algorithm. + HierarchicalDrrSettings hierarchical_drr = 2; + } + + AccountingConfig accounting_config = 3; +} + +// +// CreateResource method. +// + +message CreateResourceRequest { + Ydb.Operations.OperationParams operation_params = 1; + + // Path of a coordination node. + string coordination_node_path = 2; + + // Resource properties. + Resource resource = 3; +} + +message CreateResourceResponse { + // Holds CreateResourceResult in case of successful call. + Ydb.Operations.Operation operation = 1; +} + +message CreateResourceResult { +} + +// +// AlterResource method. +// + +message AlterResourceRequest { + Ydb.Operations.OperationParams operation_params = 1; + + // Path of a coordination node. + string coordination_node_path = 2; + + // New resource properties. + Resource resource = 3; +} + +message AlterResourceResponse { + // Holds AlterResourceResult in case of successful call. + Ydb.Operations.Operation operation = 1; +} + +message AlterResourceResult { +} + +// +// DropResource method. +// + +message DropResourceRequest { + Ydb.Operations.OperationParams operation_params = 1; + + // Path of a coordination node. + string coordination_node_path = 2; + + // Path of resource inside a coordination node. + string resource_path = 3; +} + +message DropResourceResponse { + // Holds DropResourceResult in case of successful call. + Ydb.Operations.Operation operation = 1; +} + +message DropResourceResult { +} + +// +// ListResources method. +// + +message ListResourcesRequest { + Ydb.Operations.OperationParams operation_params = 1; + + // Path of a coordination node. + string coordination_node_path = 2; + + // Path of resource inside a coordination node. + // May be empty. + // In that case all root resources will be listed. + string resource_path = 3; + + // List resources recursively. + bool recursive = 4; +} + +message ListResourcesResponse { + // Holds ListResourcesResult in case of successful call. + Ydb.Operations.Operation operation = 1; +} + +message ListResourcesResult { + repeated string resource_paths = 1; +} + +// +// DescribeResource method. +// + +message DescribeResourceRequest { + Ydb.Operations.OperationParams operation_params = 1; + + // Path of a coordination node. + string coordination_node_path = 2; + + // Path of resource inside a coordination node. + string resource_path = 3; +} + +message DescribeResourceResponse { + // Holds DescribeResourceResult in case of successful call. + Ydb.Operations.Operation operation = 1; +} + +message DescribeResourceResult { + Resource resource = 1; +} + +// +// AcquireResource method. +// + +message AcquireResourceRequest { + Ydb.Operations.OperationParams operation_params = 1; + + // Path of a coordination node. + string coordination_node_path = 2; + + // Path of resource inside a coordination node. + string resource_path = 3; + + oneof units { + // Request resource's units for usage. + uint64 required = 4; + + // Actually used resource's units by client. + uint64 used = 5; + } +} + +message AcquireResourceResponse { + // Holds AcquireResourceResult in case of successful call. + Ydb.Operations.Operation operation = 1; +} + +message AcquireResourceResult { +} diff --git a/kikimr/yndx/grpc_services/persqueue/CMakeLists.txt b/kikimr/yndx/grpc_services/persqueue/CMakeLists.txt new file mode 100644 index 0000000000..5b00f89620 --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/CMakeLists.txt @@ -0,0 +1,38 @@ + +# This file was gererated by the build system used internally in the Yandex monorepo. +# Only simple modifications are allowed (adding source-files to targets, adding simple properties +# like target_include_directories). These modifications will be ported to original +# ya.make files by maintainers. Any complex modifications which can't be ported back to the +# original buildsystem will not be accepted. + + + +add_library(yndx-grpc_services-persqueue) +target_link_libraries(yndx-grpc_services-persqueue PUBLIC + contrib-libs-cxxsupp + yutil + api-grpc-yndx + api-protos-yndx + yndx-persqueue-read_batch_converter + ydb-core-base + core-client-server + ydb-core-grpc_services + core-mind-address_classification + ydb-core-persqueue + core-persqueue-events + core-persqueue-writer + ydb-core-protos + ydb-library-aclib + library-persqueue-topic_parser + services-lib-actors + services-lib-sharding + ydb-services-persqueue_v1 +) +target_sources(yndx-grpc_services-persqueue PRIVATE + ${CMAKE_SOURCE_DIR}/kikimr/yndx/grpc_services/persqueue/grpc_pq_clusters_updater_actor.cpp + ${CMAKE_SOURCE_DIR}/kikimr/yndx/grpc_services/persqueue/grpc_pq_read.cpp + ${CMAKE_SOURCE_DIR}/kikimr/yndx/grpc_services/persqueue/grpc_pq_read_actor.cpp + ${CMAKE_SOURCE_DIR}/kikimr/yndx/grpc_services/persqueue/grpc_pq_write.cpp + ${CMAKE_SOURCE_DIR}/kikimr/yndx/grpc_services/persqueue/grpc_pq_write_actor.cpp + ${CMAKE_SOURCE_DIR}/kikimr/yndx/grpc_services/persqueue/persqueue.cpp +) diff --git a/kikimr/yndx/grpc_services/persqueue/grpc_pq_actor.h b/kikimr/yndx/grpc_services/persqueue/grpc_pq_actor.h new file mode 100644 index 0000000000..155c803d0d --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/grpc_pq_actor.h @@ -0,0 +1,928 @@ +#pragma once + +#include "grpc_pq_session.h" +#include "ydb/core/client/server/msgbus_server_pq_metacache.h" +#include "ydb/core/client/server/msgbus_server_persqueue.h" + +#include <ydb/core/base/events.h> +#include <ydb/core/tx/scheme_cache/scheme_cache.h> +#include <kikimr/yndx/api/grpc/persqueue.grpc.pb.h> + +#include <ydb/core/protos/grpc_pq_old.pb.h> +#include <ydb/core/protos/pqconfig.pb.h> +#include <ydb/core/persqueue/writer/source_id_encoding.h> + +#include <library/cpp/actors/core/actor_bootstrapped.h> + +#include <library/cpp/actors/core/hfunc.h> + +#include <ydb/library/persqueue/topic_parser/topic_parser.h> +#include <ydb/core/persqueue/events/global.h> +#include <ydb/core/persqueue/writer/writer.h> +#include <ydb/core/persqueue/percentile_counter.h> +#include <ydb/core/base/appdata.h> +#include <ydb/core/base/tablet_pipe.h> +#include <ydb/core/tx/tx_proxy/proxy.h> +#include <ydb/public/lib/base/msgbus_status.h> +#include <ydb/core/kqp/kqp.h> + +#include <ydb/core/base/ticket_parser.h> +#include <ydb/services/lib/actors/type_definitions.h> +#include <ydb/services/persqueue_v1/actors/read_init_auth_actor.h> +#include <ydb/services/persqueue_v1/actors/read_session_actor.h> + + +#include <util/generic/guid.h> +#include <util/system/compiler.h> + +namespace NKikimr { +namespace NGRpcProxy { + + +static inline TVector<TEvTicketParser::TEvAuthorizeTicket::TEntry> GetTicketParserEntries(const TString& dbId, const TString& folderId) { + static const TVector<TString> permissions = {"ydb.streams.write", "ydb.databases.list", + "ydb.databases.create", "ydb.databases.connect"}; + TVector<std::pair<TString, TString>> attributes; + if (!dbId.empty()) attributes.push_back({"database_id", dbId}); + if (!folderId.empty()) attributes.push_back({"folder_id", folderId}); + if (!attributes.empty()) { + return {{permissions, attributes}}; + } + return {}; +} + + + +static inline bool InternalErrorCode(NPersQueue::NErrorCode::EErrorCode errorCode) { + switch(errorCode) { + case NPersQueue::NErrorCode::UNKNOWN_TOPIC: + case NPersQueue::NErrorCode::ERROR: + case NPersQueue::NErrorCode::INITIALIZING: + case NPersQueue::NErrorCode::OVERLOAD: + case NPersQueue::NErrorCode::WRITE_ERROR_DISK_IS_FULL: + return true; + default: + return false; + } + return false; +} + + + +Ydb::StatusIds::StatusCode ConvertPersQueueInternalCodeToStatus(const NPersQueue::NErrorCode::EErrorCode code); +void FillIssue(Ydb::Issue::IssueMessage* issue, const NPersQueue::NErrorCode::EErrorCode errorCode, const TString& errorReason); + +using IWriteSessionHandlerRef = TIntrusivePtr<ISessionHandler<NPersQueue::TWriteResponse>>; +using IReadSessionHandlerRef = TIntrusivePtr<ISessionHandler<NPersQueue::TReadResponse>>; + +const TString& LocalDCPrefix(); +const TString& MirroredDCPrefix(); + +constexpr ui64 MAGIC_COOKIE_VALUE = 123456789; + +static const TDuration CHECK_ACL_DELAY = TDuration::Minutes(5); + +struct TEvPQProxy { + enum EEv { + EvWriteInit = EventSpaceBegin(TKikimrEvents::ES_PQ_PROXY), + EvWrite, + EvDone, + EvReadInit, + EvRead, + EvCloseSession, + EvPartitionReady, + EvReadResponse, + EvCommit, + EvCommitDone, + EvLocked, + EvReleasePartition, + EvPartitionReleased, + EvLockPartition, + EvRestartPipe, + EvDieCommand, + EvPartitionStatus, + EvAuth, + EvReadSessionStatus, + EvReadSessionStatusResponse, + EvDeadlineExceeded, + EvGetStatus, + EvWriteDone, + EvEnd, + }; + + struct TEvReadSessionStatus : public TEventPB<TEvReadSessionStatus, NKikimrPQ::TReadSessionStatus, EvReadSessionStatus> { + }; + + struct TEvReadSessionStatusResponse : public TEventPB<TEvReadSessionStatusResponse, NKikimrPQ::TReadSessionStatusResponse, EvReadSessionStatusResponse> { + }; + + + + struct TEvWriteInit : public NActors::TEventLocal<TEvWriteInit, EvWriteInit> { + TEvWriteInit(const NPersQueue::TWriteRequest& req, const TString& peerName, const TString& database) + : Request(req) + , PeerName(peerName) + , Database(database) + { } + + NPersQueue::TWriteRequest Request; + TString PeerName; + TString Database; + }; + + struct TEvWrite : public NActors::TEventLocal<TEvWrite, EvWrite> { + explicit TEvWrite(const NPersQueue::TWriteRequest& req) + : Request(req) + { } + + NPersQueue::TWriteRequest Request; + }; + + struct TEvDone : public NActors::TEventLocal<TEvDone, EvDone> { + TEvDone() + { } + }; + + struct TEvWriteDone : public NActors::TEventLocal<TEvWriteDone, EvWriteDone> { + TEvWriteDone(ui64 size) + : Size(size) + { } + + ui64 Size; + }; + + struct TEvReadInit : public NActors::TEventLocal<TEvReadInit, EvReadInit> { + TEvReadInit(const NPersQueue::TReadRequest& req, const TString& peerName, const TString& database) + : Request(req) + , PeerName(peerName) + , Database(database) + { } + + NPersQueue::TReadRequest Request; + TString PeerName; + TString Database; + }; + + struct TEvRead : public NActors::TEventLocal<TEvRead, EvRead> { + explicit TEvRead(const NPersQueue::TReadRequest& req, const TString& guid = CreateGuidAsString()) + : Request(req) + , Guid(guid) + { } + + NPersQueue::TReadRequest Request; + const TString Guid; + }; + struct TEvCloseSession : public NActors::TEventLocal<TEvCloseSession, EvCloseSession> { + TEvCloseSession(const TString& reason, const NPersQueue::NErrorCode::EErrorCode errorCode) + : Reason(reason) + , ErrorCode(errorCode) + { } + + const TString Reason; + NPersQueue::NErrorCode::EErrorCode ErrorCode; + }; + + struct TEvPartitionReady : public NActors::TEventLocal<TEvPartitionReady, EvPartitionReady> { + TEvPartitionReady(const NPersQueue::TTopicConverterPtr& topic, const ui32 partition, const ui64 wTime, const ui64 sizeLag, + const ui64 readOffset, const ui64 endOffset) + : Topic(topic) + , Partition(partition) + , WTime(wTime) + , SizeLag(sizeLag) + , ReadOffset(readOffset) + , EndOffset(endOffset) + { } + + NPersQueue::TTopicConverterPtr Topic; + ui32 Partition; + ui64 WTime; + ui64 SizeLag; + ui64 ReadOffset; + ui64 EndOffset; + }; + + struct TEvReadResponse : public NActors::TEventLocal<TEvReadResponse, EvReadResponse> { + explicit TEvReadResponse( + NPersQueue::TReadResponse&& resp, + ui64 nextReadOffset, + bool fromDisk, + TDuration waitQuotaTime + ) + : Response(std::move(resp)) + , NextReadOffset(nextReadOffset) + , FromDisk(fromDisk) + , WaitQuotaTime(waitQuotaTime) + { } + + NPersQueue::TReadResponse Response; + ui64 NextReadOffset; + bool FromDisk; + TDuration WaitQuotaTime; + }; + + struct TEvCommit : public NActors::TEventLocal<TEvCommit, EvCommit> { + explicit TEvCommit(ui64 readId, ui64 offset = Max<ui64>()) + : ReadId(readId) + , Offset(offset) + { } + + ui64 ReadId; + ui64 Offset; // Actual value for requests to concreete partitions + }; + + struct TEvAuth : public NActors::TEventLocal<TEvAuth, EvAuth> { + TEvAuth(const NPersQueueCommon::TCredentials& auth) + : Auth(auth) + { } + + NPersQueueCommon::TCredentials Auth; + }; + + struct TEvLocked : public NActors::TEventLocal<TEvLocked, EvLocked> { + TEvLocked(const TString& topic, ui32 partition, ui64 readOffset, ui64 commitOffset, bool verifyReadOffset, ui64 generation) + : Topic(topic) + , Partition(partition) + , ReadOffset(readOffset) + , CommitOffset(commitOffset) + , VerifyReadOffset(verifyReadOffset) + , Generation(generation) + { } + + TString Topic; + ui32 Partition; + ui64 ReadOffset; + ui64 CommitOffset; + bool VerifyReadOffset; + ui64 Generation; + }; + + struct TEvGetStatus : public NActors::TEventLocal<TEvGetStatus, EvGetStatus> { + TEvGetStatus(const TString& topic, ui32 partition, ui64 generation) + : Topic(topic) + , Partition(partition) + , Generation(generation) + { } + + TString Topic; + ui32 Partition; + ui64 Generation; + }; + + + + struct TEvCommitDone : public NActors::TEventLocal<TEvCommitDone, EvCommitDone> { + TEvCommitDone(ui64 readId, const NPersQueue::TTopicConverterPtr& topic, const ui32 partition) + : ReadId(readId) + , Topic(topic) + , Partition(partition) + { } + + ui64 ReadId; + NPersQueue::TTopicConverterPtr Topic; + ui32 Partition; + }; + + struct TEvReleasePartition : public NActors::TEventLocal<TEvReleasePartition, EvReleasePartition> { + TEvReleasePartition() + { } + }; + + struct TEvLockPartition : public NActors::TEventLocal<TEvLockPartition, EvLockPartition> { + explicit TEvLockPartition(const ui64 readOffset, const ui64 commitOffset, bool verifyReadOffset, bool startReading) + : ReadOffset(readOffset) + , CommitOffset(commitOffset) + , VerifyReadOffset(verifyReadOffset) + , StartReading(startReading) + { } + + ui64 ReadOffset; + ui64 CommitOffset; + bool VerifyReadOffset; + bool StartReading; + }; + + + struct TEvPartitionReleased : public NActors::TEventLocal<TEvPartitionReleased, EvPartitionReleased> { + TEvPartitionReleased(const NPersQueue::TTopicConverterPtr& topic, const ui32 partition) + : Topic(topic) + , Partition(partition) + { } + + NPersQueue::TTopicConverterPtr Topic; + ui32 Partition; + }; + + + struct TEvRestartPipe : public NActors::TEventLocal<TEvRestartPipe, EvRestartPipe> { + TEvRestartPipe() + { } + }; + + struct TEvDeadlineExceeded : public NActors::TEventLocal<TEvDeadlineExceeded, EvDeadlineExceeded> { + TEvDeadlineExceeded(ui64 cookie) + : Cookie(cookie) + { } + + ui64 Cookie; + }; + + + struct TEvDieCommand : public NActors::TEventLocal<TEvDieCommand, EvDieCommand> { + TEvDieCommand(const TString& reason, const NPersQueue::NErrorCode::EErrorCode errorCode) + : Reason(reason) + , ErrorCode(errorCode) + { } + + TString Reason; + NPersQueue::NErrorCode::EErrorCode ErrorCode; + }; + + struct TEvPartitionStatus : public NActors::TEventLocal<TEvPartitionStatus, EvPartitionStatus> { + TEvPartitionStatus(const NPersQueue::TTopicConverterPtr& topic, const ui32 partition, const ui64 offset, + const ui64 endOffset, ui64 writeTimestampEstimateMs, bool init = true) + : Topic(topic) + , Partition(partition) + , Offset(offset) + , EndOffset(endOffset) + , WriteTimestampEstimateMs(writeTimestampEstimateMs) + , Init(init) + { } + + NPersQueue::TTopicConverterPtr Topic; + ui32 Partition; + ui64 Offset; + ui64 EndOffset; + ui64 WriteTimestampEstimateMs; + bool Init; + }; + +}; + + + +/// WRITE ACTOR +class TWriteSessionActor : public NActors::TActorBootstrapped<TWriteSessionActor> { + using TEvDescribeTopicsRequest = NMsgBusProxy::NPqMetaCacheV2::TEvPqNewMetaCache::TEvDescribeTopicsRequest; + using TEvDescribeTopicsResponse = NMsgBusProxy::NPqMetaCacheV2::TEvPqNewMetaCache::TEvDescribeTopicsResponse; + using TPQGroupInfoPtr = TIntrusiveConstPtr<NSchemeCache::TSchemeCacheNavigate::TPQGroupInfo>; +public: + TWriteSessionActor(IWriteSessionHandlerRef handler, const ui64 cookie, const NActors::TActorId& schemeCache, + TIntrusivePtr<NMonitoring::TDynamicCounters> counters, const TString& localDC, + const TMaybe<TString> clientDC); + ~TWriteSessionActor(); + + void Bootstrap(const NActors::TActorContext& ctx); + + void Die(const NActors::TActorContext& ctx) override; + + static constexpr NKikimrServices::TActivity::EType ActorActivityType() { return NKikimrServices::TActivity::FRONT_PQ_WRITE; } +private: + STFUNC(StateFunc) { + switch (ev->GetTypeRewrite()) { + CFunc(NActors::TEvents::TSystem::Wakeup, HandleWakeup) + + HFunc(TEvTicketParser::TEvAuthorizeTicketResult, Handle); + + HFunc(TEvPQProxy::TEvDieCommand, HandlePoison) + HFunc(TEvPQProxy::TEvWriteInit, Handle) + HFunc(TEvPQProxy::TEvWrite, Handle) + HFunc(TEvPQProxy::TEvDone, Handle) + HFunc(TEvPersQueue::TEvGetPartitionIdForWriteResponse, Handle) + + HFunc(TEvDescribeTopicsResponse, Handle); + + HFunc(NPQ::TEvPartitionWriter::TEvInitResult, Handle); + HFunc(NPQ::TEvPartitionWriter::TEvWriteAccepted, Handle); + HFunc(NPQ::TEvPartitionWriter::TEvWriteResponse, Handle); + HFunc(NPQ::TEvPartitionWriter::TEvDisconnected, Handle); + HFunc(TEvTabletPipe::TEvClientDestroyed, Handle); + HFunc(TEvTabletPipe::TEvClientConnected, Handle); + + HFunc(NKqp::TEvKqp::TEvQueryResponse, Handle); + HFunc(NKqp::TEvKqp::TEvProcessResponse, Handle); + + default: + break; + }; + } + + void Handle(NKqp::TEvKqp::TEvQueryResponse::TPtr &ev, const TActorContext &ctx); + void Handle(NKqp::TEvKqp::TEvProcessResponse::TPtr &ev, const TActorContext &ctx); + + TString CheckSupportedCodec(const ui32 codecId); + void CheckACL(const TActorContext& ctx); + void InitCheckACL(const TActorContext& ctx); + void Handle(TEvTicketParser::TEvAuthorizeTicketResult::TPtr& ev, const TActorContext& ctx); + void Handle(TEvPQProxy::TEvWriteInit::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvWrite::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvDone::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPersQueue::TEvGetPartitionIdForWriteResponse::TPtr& ev, const NActors::TActorContext& ctx); + + void LogSession(const TActorContext& ctx); + + void InitAfterDiscovery(const TActorContext& ctx); + void DiscoverPartition(const NActors::TActorContext& ctx); + void SendSelectPartitionRequest(ui32 hash, const TString& topic, const NActors::TActorContext& ctx); + void UpdatePartition(const NActors::TActorContext& ctx); + void RequestNextPartition(const NActors::TActorContext& ctx); + void ProceedPartition(const ui32 partition, const NActors::TActorContext& ctx); + THolder<NKqp::TEvKqp::TEvQueryRequest> MakeUpdateSourceIdMetadataRequest(ui32 hash, const TString& topic); + + + void Handle(TEvDescribeTopicsResponse::TPtr& ev, const NActors::TActorContext& ctx); + + void Handle(NPQ::TEvPartitionWriter::TEvInitResult::TPtr& ev, const TActorContext& ctx); + void Handle(NPQ::TEvPartitionWriter::TEvWriteAccepted::TPtr& ev, const TActorContext& ctx); + void Handle(NPQ::TEvPartitionWriter::TEvWriteResponse::TPtr& ev, const TActorContext& ctx); + void Handle(NPQ::TEvPartitionWriter::TEvDisconnected::TPtr& ev, const TActorContext& ctx); + void Handle(TEvTabletPipe::TEvClientConnected::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvTabletPipe::TEvClientDestroyed::TPtr& ev, const NActors::TActorContext& ctx); + + void HandlePoison(TEvPQProxy::TEvDieCommand::TPtr& ev, const NActors::TActorContext& ctx); + void HandleWakeup(const NActors::TActorContext& ctx); + + void CloseSession(const TString& errorReason, const NPersQueue::NErrorCode::EErrorCode errorCode, const NActors::TActorContext& ctx); + + void CheckFinish(const NActors::TActorContext& ctx); + + void GenerateNextWriteRequest(const NActors::TActorContext& ctx); + + void SetupCounters(); + void SetupCounters(const TString& cloudId, const TString& dbId, const TString& folderId); + + +private: + IWriteSessionHandlerRef Handler; + + enum EState { + ES_CREATED = 1, + ES_WAIT_SCHEME_1 = 2, + ES_WAIT_SCHEME_2 = 3, + ES_WAIT_TABLE_REQUEST_1 = 4, + ES_WAIT_NEXT_PARTITION = 5, + ES_WAIT_TABLE_REQUEST_2 = 6, + ES_WAIT_TABLE_REQUEST_3 = 7, + ES_WAIT_WRITER_INIT = 8, + ES_INITED = 9, + ES_DYING = 10, + }; + + EState State; + TActorId SchemeCache; + TActorId Writer; + + TString PeerName; + TString Database; + ui64 Cookie; + + ui32 Partition; + bool PartitionFound = false; + ui32 PreferedPartition; + TString SourceId; + ui32 SelectReqsInflight = 0; + ui64 MaxSrcIdAccessTime = 0; + NPQ::NSourceIdEncoding::TEncodedSourceId EncodedSourceId; + TString OwnerCookie; + TString UserAgent; + + ui32 NumReserveBytesRequests; + + struct TWriteRequestBatchInfo: public TSimpleRefCount<TWriteRequestBatchInfo> { + using TPtr = TIntrusivePtr<TWriteRequestBatchInfo>; + + // Source requests from user (grpc session object) + std::deque<THolder<TEvPQProxy::TEvWrite>> UserWriteRequests; + + // Formed write request's size + ui64 ByteSize = 0; + + // Formed write request's cookie + ui64 Cookie = 0; + }; + + // Nonprocessed source client requests + std::deque<THolder<TEvPQProxy::TEvWrite>> Writes; + + // Formed, but not sent, batch requests to partition actor + std::deque<TWriteRequestBatchInfo::TPtr> FormedWrites; + + // Requests that is already sent to partition actor + std::deque<TWriteRequestBatchInfo::TPtr> SentMessages; + + bool WritesDone; + + THashMap<ui32, ui64> PartitionToTablet; + + TIntrusivePtr<NMonitoring::TDynamicCounters> Counters; + + NKikimr::NPQ::TMultiCounter BytesInflight; + NKikimr::NPQ::TMultiCounter BytesInflightTotal; + + ui64 BytesInflight_; + ui64 BytesInflightTotal_; + + bool NextRequestInited; + + NKikimr::NPQ::TMultiCounter SessionsCreated; + NKikimr::NPQ::TMultiCounter SessionsActive; + NKikimr::NPQ::TMultiCounter SessionsWithoutAuth; + + NKikimr::NPQ::TMultiCounter Errors; + + ui64 NextRequestCookie; + + TIntrusivePtr<NACLib::TUserToken> Token; + NPersQueueCommon::TCredentials Auth; + TString AuthStr; + bool ACLCheckInProgress; + bool FirstACLCheck; + bool ForceACLCheck; + bool RequestNotChecked; + TInstant LastACLCheckTimestamp; + TInstant LogSessionDeadline; + + ui64 BalancerTabletId; + TString DatabaseId; + TString FolderId; + TActorId PipeToBalancer; + TIntrusivePtr<TSecurityObject> SecurityObject; + TPQGroupInfoPtr PQInfo; + + NKikimrPQClient::TDataChunk InitMeta; + TString LocalDC; + TString ClientDC; + TString SelectSourceIdQuery; + TString UpdateSourceIdQuery; + TInstant LastSourceIdUpdate; + + ui64 SourceIdCreateTime = 0; + ui32 SourceIdUpdatesInflight = 0; + + + TVector<NPersQueue::TPQLabelsInfo> Aggr; + NKikimr::NPQ::TMultiCounter SLITotal; + NKikimr::NPQ::TMultiCounter SLIErrors; + TInstant StartTime; + NKikimr::NPQ::TPercentileCounter InitLatency; + NKikimr::NPQ::TMultiCounter SLIBigLatency; + + THolder<NPersQueue::TTopicNamesConverterFactory> ConverterFactory; + NPersQueue::TDiscoveryConverterPtr DiscoveryConverter; + NPersQueue::TTopicConverterPtr FullConverter; + + NPersQueue::TWriteRequest::TInit InitRequest; +}; + +class TReadSessionActor : public TActorBootstrapped<TReadSessionActor> { + using TEvDescribeTopicsRequest = NMsgBusProxy::NPqMetaCacheV2::TEvPqNewMetaCache::TEvDescribeTopicsRequest; + using TEvDescribeTopicsResponse = NMsgBusProxy::NPqMetaCacheV2::TEvPqNewMetaCache::TEvDescribeTopicsResponse; +public: + TReadSessionActor(IReadSessionHandlerRef handler, const NPersQueue::TTopicsListController& topicsHandler, const ui64 cookie, + const NActors::TActorId& schemeCache, const NActors::TActorId& newSchemeCache, TIntrusivePtr<NMonitoring::TDynamicCounters> counters, + const TMaybe<TString> clientDC); + ~TReadSessionActor(); + + void Bootstrap(const NActors::TActorContext& ctx); + + void Die(const NActors::TActorContext& ctx) override; + + static constexpr NKikimrServices::TActivity::EType ActorActivityType() { return NKikimrServices::TActivity::FRONT_PQ_READ; } + + + struct TTopicCounters { + NKikimr::NPQ::TMultiCounter PartitionsLocked; + NKikimr::NPQ::TMultiCounter PartitionsReleased; + NKikimr::NPQ::TMultiCounter PartitionsToBeReleased; + NKikimr::NPQ::TMultiCounter PartitionsToBeLocked; + NKikimr::NPQ::TMultiCounter PartitionsInfly; + NKikimr::NPQ::TMultiCounter Errors; + NKikimr::NPQ::TMultiCounter Commits; + NKikimr::NPQ::TMultiCounter WaitsForData; + }; + +private: + STFUNC(StateFunc) { + switch (ev->GetTypeRewrite()) { + CFunc(NActors::TEvents::TSystem::Wakeup, HandleWakeup) + + HFunc(NKikimr::NGRpcProxy::V1::TEvPQProxy::TEvAuthResultOk, Handle); // form auth actor + + HFunc(TEvPQProxy::TEvDieCommand, HandlePoison) + HFunc(TEvPQProxy::TEvReadInit, Handle) //from gRPC + HFunc(TEvPQProxy::TEvReadSessionStatus, Handle) // from read sessions info builder proxy + HFunc(TEvPQProxy::TEvRead, Handle) //from gRPC + HFunc(TEvPQProxy::TEvDone, Handle) //from gRPC + HFunc(TEvPQProxy::TEvWriteDone, Handle) //from gRPC + HFunc(NKikimr::NGRpcProxy::V1::TEvPQProxy::TEvCloseSession, Handle) //from partitionActor + HFunc(TEvPQProxy::TEvCloseSession, Handle) //from partitionActor + + HFunc(TEvPQProxy::TEvPartitionReady, Handle) //from partitionActor + HFunc(TEvPQProxy::TEvPartitionReleased, Handle) //from partitionActor + + HFunc(TEvPQProxy::TEvReadResponse, Handle) //from partitionActor + HFunc(TEvPQProxy::TEvCommit, Handle) //from gRPC + HFunc(TEvPQProxy::TEvLocked, Handle) //from gRPC + HFunc(TEvPQProxy::TEvGetStatus, Handle) //from gRPC + HFunc(TEvPQProxy::TEvAuth, Handle) //from gRPC + + HFunc(TEvPQProxy::TEvCommitDone, Handle) //from PartitionActor + HFunc(TEvPQProxy::TEvPartitionStatus, Handle) //from partitionActor + + HFunc(TEvPersQueue::TEvLockPartition, Handle) //from Balancer + HFunc(TEvPersQueue::TEvReleasePartition, Handle) //from Balancer + HFunc(TEvPersQueue::TEvError, Handle) //from Balancer + + HFunc(TEvTabletPipe::TEvClientDestroyed, Handle); + HFunc(TEvTabletPipe::TEvClientConnected, Handle); + + HFunc(TEvDescribeTopicsResponse, HandleDescribeTopicsResponse); + HFunc(TEvTicketParser::TEvAuthorizeTicketResult, Handle); + + default: + break; + }; + } + + void Handle(TEvPQProxy::TEvReadInit::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvReadSessionStatus::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvRead::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvReadResponse::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvDone::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvWriteDone::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(NKikimr::NGRpcProxy::V1::TEvPQProxy::TEvCloseSession::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvCloseSession::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvPartitionReady::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvPartitionReleased::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvCommit::TPtr& ev, const NActors::TActorContext& ctx); + void MakeCommit(const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvLocked::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvGetStatus::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvAuth::TPtr& ev, const NActors::TActorContext& ctx); + void ProcessAuth(const NPersQueueCommon::TCredentials& auth); + void Handle(TEvPQProxy::TEvCommitDone::TPtr& ev, const NActors::TActorContext& ctx); + void AnswerForCommitsIfCan(const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvPartitionStatus::TPtr& ev, const NActors::TActorContext& ctx); + + void Handle(TEvPersQueue::TEvLockPartition::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPersQueue::TEvReleasePartition::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPersQueue::TEvError::TPtr& ev, const NActors::TActorContext& ctx); + + void Handle(TEvTabletPipe::TEvClientConnected::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvTabletPipe::TEvClientDestroyed::TPtr& ev, const NActors::TActorContext& ctx); + [[nodiscard]] bool ProcessBalancerDead(const ui64 tabletId, const NActors::TActorContext& ctx); // returns false if actor died + + void HandlePoison(TEvPQProxy::TEvDieCommand::TPtr& ev, const NActors::TActorContext& ctx); + void HandleWakeup(const NActors::TActorContext& ctx); + void Handle(NKikimr::NGRpcProxy::V1::TEvPQProxy::TEvAuthResultOk::TPtr& ev, const NActors::TActorContext& ctx); + + void CloseSession(const TString& errorReason, const NPersQueue::NErrorCode::EErrorCode errorCode, + const NActors::TActorContext& ctx); + + void Handle(TEvTicketParser::TEvAuthorizeTicketResult::TPtr& ev, const TActorContext& ctx); + void HandleDescribeTopicsResponse(TEvDescribeTopicsResponse::TPtr& ev, const TActorContext& ctx); + + void SendAuthRequest(const TActorContext& ctx); + void CreateInitAndAuthActor(const TActorContext& ctx); + + void SetupCounters(); + void SetupTopicCounters(const NPersQueue::TTopicConverterPtr& topic); + void SetupTopicCounters(const NPersQueue::TTopicConverterPtr& topic, const TString& cloudId, const TString& dbId, + const TString& folderId); + + [[nodiscard]] bool ProcessReads(const NActors::TActorContext& ctx); // returns false if actor died + struct TFormedReadResponse; + [[nodiscard]] bool ProcessAnswer(const NActors::TActorContext& ctx, TIntrusivePtr<TFormedReadResponse> formedResponse); // returns false if actor died + + void RegisterSessions(const NActors::TActorContext& ctx); + void RegisterSession(const TActorId& pipe, const TString& topic, const TActorContext& ctx); + + struct TPartitionActorInfo; + void DropPartitionIfNeeded(THashMap<std::pair<TString, ui32>, TPartitionActorInfo>::iterator it, const TActorContext& ctx); + + bool ActualPartitionActor(const TActorId& part); + [[nodiscard]] bool ProcessReleasePartition(const THashMap<std::pair<TString, ui32>, TPartitionActorInfo>::iterator& it, + bool kill, bool couldBeReads, const TActorContext& ctx); // returns false if actor died + void InformBalancerAboutRelease(const THashMap<std::pair<TString, ui32>, TPartitionActorInfo>::iterator& it, const TActorContext& ctx); + + // returns false if check failed. + bool CheckAndUpdateReadSettings(const NPersQueue::TReadRequest::TRead& readRequest); + + static ui32 NormalizeMaxReadMessagesCount(ui32 sourceValue); + static ui32 NormalizeMaxReadSize(ui32 sourceValue); + static ui32 NormalizeMaxReadPartitionsCount(ui32 sourceValue); + + static bool RemoveEmptyMessages(NPersQueue::TReadResponse::TBatchedData& data); // returns true if there are nonempty messages + +private: + IReadSessionHandlerRef Handler; + + const TInstant StartTimestamp; + + TActorId PqMetaCache; + TActorId NewSchemeCache; + + TActorId AuthInitActor; + bool AuthInflight; + + TString InternalClientId; + TString ExternalClientId; + const TString ClientDC; + TString ClientPath; + TString Session; + TString PeerName; + TString Database; + + bool ClientsideLocksAllowed; + bool BalanceRightNow; + bool CommitsDisabled; + bool BalancersInitStarted; + + bool InitDone; + + ui32 ProtocolVersion; // from NPersQueue::TReadRequest::EProtocolVersion + // Read settings. + // Can be initialized during Init request (new preferable way) + // or during read request (old way that will be removed in future). + // These settings can't be changed (in that case server closes session). + ui32 MaxReadMessagesCount; + ui32 MaxReadSize; + ui32 MaxReadPartitionsCount; + ui32 MaxTimeLagMs; + ui64 ReadTimestampMs; + bool ReadSettingsInited; + + NPersQueueCommon::TCredentials Auth; + TString AuthStr; + TIntrusivePtr<NACLib::TUserToken> Token; + bool ForceACLCheck; + bool RequestNotChecked; + TInstant LastACLCheckTimestamp; + + struct TPartitionActorInfo { + TActorId Actor; + std::deque<ui64> Commits; + bool Reading; + bool Releasing; + bool Released; + ui64 LockGeneration; + bool LockSent; + NPersQueue::TTopicConverterPtr Converter; + + TPartitionActorInfo(const TActorId& actor, ui64 generation, const NPersQueue::TTopicConverterPtr& topic) + : Actor(actor) + , Reading(false) + , Releasing(false) + , Released(false) + , LockGeneration(generation) + , LockSent(false) + , Converter(topic) + {} + }; + + + THashSet<TActorId> ActualPartitionActors; + THashMap<std::pair<TString, ui32>, TPartitionActorInfo> Partitions; //topic[ClientSideName!]:partition -> info + + THashMap<TString, NPersQueue::TTopicConverterPtr> FullPathToConverter; // PrimaryFullPath -> Converter, for balancer replies matching + THashMap<TString, TTopicHolder> Topics; // PrimaryName ->topic info + + TVector<ui32> Groups; + bool ReadOnlyLocal; + + struct TPartitionInfo { + NPersQueue::TTopicConverterPtr Topic; + ui32 Partition; + ui64 WTime; + ui64 SizeLag; + ui64 MsgLag; + TActorId Actor; + bool operator < (const TPartitionInfo& rhs) const { + return std::tie(WTime, Topic, Partition, Actor) < std::tie(rhs.WTime, rhs.Topic, rhs.Partition, rhs.Actor); + } + }; + + TSet<TPartitionInfo> AvailablePartitions; + + struct TOffsetsInfo { + struct TPartitionOffsetInfo { + TPartitionOffsetInfo(const TActorId& sender, const TString& topic, ui32 partition, ui64 offset) + : Sender(sender) + , Topic(topic) + , Partition(partition) + , Offset(offset) + { + } + + TActorId Sender; + TString Topic; + ui32 Partition; + ui64 Offset; + }; + + // find by read id + bool operator<(ui64 readId) const { + return ReadId < readId; + } + + friend bool operator<(ui64 readId, const TOffsetsInfo& info) { + return readId < info.ReadId; + } + + ui64 ReadId = 0; + std::vector<TPartitionOffsetInfo> PartitionOffsets; + }; + + std::deque<TOffsetsInfo> Offsets; // Sequential read id -> offsets + + struct TFormedReadResponse: public TSimpleRefCount<TFormedReadResponse> { + using TPtr = TIntrusivePtr<TFormedReadResponse>; + + TFormedReadResponse(const TString& guid, const TInstant start) + : Guid(guid) + , Start(start) + , FromDisk(false) + { + } + + NPersQueue::TReadResponse Response; + ui32 RequestsInfly = 0; + i64 ByteSize = 0; + + ui64 RequestedBytes = 0; + + //returns byteSize diff + i64 ApplyResponse(NPersQueue::TReadResponse&& resp); + + TVector<NPersQueue::TReadResponse> ControlMessages; + + THashSet<TActorId> PartitionsTookPartInRead; + TSet<TPartitionInfo> PartitionsBecameAvailable; // Partitions that became available during this read request execution. + // These partitions are bringed back to AvailablePartitions after reply to this read request. + TOffsetsInfo Offsets; // Offsets without assigned read id. + + const TString Guid; + TInstant Start; + bool FromDisk; + TDuration WaitQuotaTime; + }; + + THashMap<TActorId, TFormedReadResponse::TPtr> PartitionToReadResponse; // Partition actor -> TFormedReadResponse answer that has this partition. + // PartitionsTookPartInRead in formed read response contain this actor id. + + ui64 ReadIdToResponse; + ui64 ReadIdCommitted; + TSet<ui64> NextCommits; + TInstant LastCommitTimestamp; + TDuration CommitInterval; + ui32 CommitsInfly; + + std::deque<THolder<TEvPQProxy::TEvRead>> Reads; + + ui64 Cookie; + + struct TCommitInfo { + ui64 StartReadId; + ui32 Partitions; + TInstant StartTime; + }; + + TMap<ui64, TCommitInfo> Commits; //readid->TCommitInfo + + TIntrusivePtr<NMonitoring::TDynamicCounters> Counters; + + NMonitoring::TDynamicCounters::TCounterPtr SessionsCreated; + NMonitoring::TDynamicCounters::TCounterPtr SessionsActive; + NMonitoring::TDynamicCounters::TCounterPtr SessionsWithoutAuth; + NMonitoring::TDynamicCounters::TCounterPtr SessionsWithOldBatchingVersion; // LOGBROKER-3173 + + NMonitoring::TDynamicCounters::TCounterPtr Errors; + NMonitoring::TDynamicCounters::TCounterPtr PipeReconnects; + NMonitoring::TDynamicCounters::TCounterPtr BytesInflight; + ui64 BytesInflight_; + ui64 RequestedBytes; + ui32 ReadsInfly; + + NKikimr::NPQ::TPercentileCounter PartsPerSession; + + THashMap<TString, TTopicCounters> TopicCounters; + THashMap<TString, ui32> NumPartitionsFromTopic; + + TVector<NPersQueue::TPQLabelsInfo> Aggr; + NKikimr::NPQ::TMultiCounter SLITotal; + NKikimr::NPQ::TMultiCounter SLIErrors; + TInstant StartTime; + NKikimr::NPQ::TPercentileCounter InitLatency; + NKikimr::NPQ::TPercentileCounter CommitLatency; + NKikimr::NPQ::TMultiCounter SLIBigLatency; + + NKikimr::NPQ::TPercentileCounter ReadLatency; + NKikimr::NPQ::TPercentileCounter ReadLatencyFromDisk; + NKikimr::NPQ::TMultiCounter SLIBigReadLatency; + NKikimr::NPQ::TMultiCounter ReadsTotal; + + NPersQueue::TTopicsListController TopicsHandler; + NPersQueue::TTopicsToConverter TopicsList; +}; + +} +} diff --git a/kikimr/yndx/grpc_services/persqueue/grpc_pq_clusters_updater_actor.cpp b/kikimr/yndx/grpc_services/persqueue/grpc_pq_clusters_updater_actor.cpp new file mode 100644 index 0000000000..357c535aca --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/grpc_pq_clusters_updater_actor.cpp @@ -0,0 +1,86 @@ +#include "grpc_pq_clusters_updater_actor.h" + +#include <ydb/core/base/appdata.h> +#include <ydb/core/persqueue/pq_database.h> + +namespace NKikimr { +namespace NGRpcProxy { + +static const int CLUSTERS_UPDATER_TIMEOUT_ON_ERROR = 1; + + +TClustersUpdater::TClustersUpdater(IPQClustersUpdaterCallback* callback) + : Callback(callback) + {}; + +void TClustersUpdater::Bootstrap(const NActors::TActorContext& ctx) { + ctx.Send(ctx.SelfID, new TEvPQClustersUpdater::TEvUpdateClusters()); + ctx.Send(NNetClassifier::MakeNetClassifierID(), new NNetClassifier::TEvNetClassifier::TEvSubscribe); + + Become(&TThis::StateFunc); +} + +void TClustersUpdater::Handle(TEvPQClustersUpdater::TEvUpdateClusters::TPtr&, const TActorContext &ctx) { + auto req = MakeHolder<NKqp::TEvKqp::TEvQueryRequest>(); + req->Record.MutableRequest()->SetAction(NKikimrKqp::QUERY_ACTION_EXECUTE); + req->Record.MutableRequest()->SetType(NKikimrKqp::QUERY_TYPE_SQL_DML); + req->Record.MutableRequest()->SetKeepSession(false); + req->Record.MutableRequest()->SetQuery("--!syntax_v1\nSELECT `name`, `local`, `enabled` FROM `" + AppData(ctx)->PQConfig.GetRoot() + "/Config/V2/Cluster`;"); + req->Record.MutableRequest()->SetDatabase(NKikimr::NPQ::GetDatabaseFromConfig(AppData(ctx)->PQConfig)); + req->Record.MutableRequest()->MutableTxControl()->set_commit_tx(true); + req->Record.MutableRequest()->MutableTxControl()->mutable_begin_tx()->mutable_serializable_read_write(); + ctx.Send(NKqp::MakeKqpProxyID(ctx.SelfID.NodeId()), req.Release()); +} + +void TClustersUpdater::Handle(NNetClassifier::TEvNetClassifier::TEvClassifierUpdate::TPtr& ev, const TActorContext&) { + + Callback->NetClassifierUpdated(ev->Get()->Classifier); +} + + + + +void TClustersUpdater::Handle(NKqp::TEvKqp::TEvQueryResponse::TPtr &ev, const TActorContext &ctx) { + auto& record = ev->Get()->Record.GetRef(); + + if (record.GetYdbStatus() == Ydb::StatusIds::SUCCESS) { + auto& t = record.GetResponse().GetResults(0).GetValue().GetStruct(0); + bool local = false; + TVector<TString> clusters; + for (size_t i = 0; i < t.ListSize(); ++i) { + TString dc = t.GetList(i).GetStruct(0).GetOptional().GetText(); + local = t.GetList(i).GetStruct(1).GetOptional().GetBool(); + clusters.push_back(dc); + if (local) { + bool enabled = t.GetList(i).GetStruct(2).GetOptional().GetBool(); + Y_VERIFY(LocalCluster.empty() || LocalCluster == dc); + bool changed = LocalCluster != dc || Enabled != enabled; + if (changed) { + LocalCluster = dc; + Enabled = enabled; + Callback->CheckClusterChange(LocalCluster, Enabled); + } + } + } + if (Clusters != clusters) { + Clusters = clusters; + Callback->CheckClustersListChange(Clusters); + } + ctx.Schedule(TDuration::Seconds(AppData(ctx)->PQConfig.GetClustersUpdateTimeoutSec()), new TEvPQClustersUpdater::TEvUpdateClusters()); + } else { + LOG_ERROR_S(ctx, NKikimrServices::PQ_WRITE_PROXY, "can't update clusters " << record); + ctx.Schedule(TDuration::Seconds(CLUSTERS_UPDATER_TIMEOUT_ON_ERROR), new TEvPQClustersUpdater::TEvUpdateClusters()); + } +} + + +void TClustersUpdater::Handle(NKqp::TEvKqp::TEvProcessResponse::TPtr &ev, const TActorContext &ctx) { + auto& record = ev->Get()->Record; + + LOG_ERROR_S(ctx, NKikimrServices::PQ_WRITE_PROXY, "can't update clusters " << record); + ctx.Schedule(TDuration::Seconds(CLUSTERS_UPDATER_TIMEOUT_ON_ERROR), new TEvPQClustersUpdater::TEvUpdateClusters()); +} + + +} +} diff --git a/kikimr/yndx/grpc_services/persqueue/grpc_pq_clusters_updater_actor.h b/kikimr/yndx/grpc_services/persqueue/grpc_pq_clusters_updater_actor.h new file mode 100644 index 0000000000..6f1b6ade76 --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/grpc_pq_clusters_updater_actor.h @@ -0,0 +1,77 @@ +#pragma once + +#include <library/cpp/actors/core/actor_bootstrapped.h> +#include <library/cpp/actors/core/actor.h> +#include <library/cpp/actors/core/event_local.h> +#include <library/cpp/actors/core/hfunc.h> + +#include <ydb/core/base/events.h> +#include <ydb/core/kqp/kqp.h> +#include <ydb/core/mind/address_classification/net_classifier.h> + +namespace NKikimr { +namespace NGRpcProxy { + +struct TEvPQClustersUpdater { + enum EEv { + EvUpdateClusters = EventSpaceBegin(TKikimrEvents::ES_PQ_CLUSTERS_UPDATER), + EvEnd, + }; + + struct TEvUpdateClusters : public NActors::TEventLocal<TEvUpdateClusters, EvUpdateClusters> { + TEvUpdateClusters() + {} + }; +}; + +class IPQClustersUpdaterCallback { +public: + virtual ~IPQClustersUpdaterCallback() = default; + virtual void CheckClusterChange(const TString& localCluster, const bool enabled) + { + Y_UNUSED(localCluster); + Y_UNUSED(enabled); + } + + virtual void CheckClustersListChange(const TVector<TString>& clusters) + { + Y_UNUSED(clusters); + } + + virtual void NetClassifierUpdated(NAddressClassifier::TLabeledAddressClassifier::TConstPtr classifier) { + Y_UNUSED(classifier); + } +}; + +class TClustersUpdater : public NActors::TActorBootstrapped<TClustersUpdater> { +public: + TClustersUpdater(IPQClustersUpdaterCallback* callback); + + void Bootstrap(const NActors::TActorContext& ctx); + + static constexpr NKikimrServices::TActivity::EType ActorActivityType() { return NKikimrServices::TActivity::FRONT_PQ_WRITE; } // FIXME + +private: + IPQClustersUpdaterCallback* Callback; + TString LocalCluster; + TVector<TString> Clusters; + bool Enabled = false; + + STFUNC(StateFunc) { + switch (ev->GetTypeRewrite()) { + HFunc(TEvPQClustersUpdater::TEvUpdateClusters, Handle); + HFunc(NKqp::TEvKqp::TEvQueryResponse, Handle); + HFunc(NKqp::TEvKqp::TEvProcessResponse, Handle); + HFunc(NNetClassifier::TEvNetClassifier::TEvClassifierUpdate, Handle); + } + } + + void Handle(TEvPQClustersUpdater::TEvUpdateClusters::TPtr &ev, const TActorContext &ctx); + void Handle(NKqp::TEvKqp::TEvQueryResponse::TPtr &ev, const TActorContext &ctx); + void Handle(NKqp::TEvKqp::TEvProcessResponse::TPtr &ev, const TActorContext &ctx); + void Handle(NNetClassifier::TEvNetClassifier::TEvClassifierUpdate::TPtr& ev, const TActorContext& ctx); + +}; + +} +} diff --git a/kikimr/yndx/grpc_services/persqueue/grpc_pq_read.cpp b/kikimr/yndx/grpc_services/persqueue/grpc_pq_read.cpp new file mode 100644 index 0000000000..373b74bcf2 --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/grpc_pq_read.cpp @@ -0,0 +1,268 @@ +#include "grpc_pq_read.h" +#include "grpc_pq_actor.h" +#include "grpc_pq_session.h" +#include "ydb/core/client/server/grpc_proxy_status.h" + +#include <ydb/core/grpc_services/grpc_helper.h> +#include <ydb/core/tx/scheme_board/cache.h> + +using namespace NActors; +using namespace NKikimrClient; + +using grpc::Status; + +namespace NKikimr { +namespace NGRpcProxy { + +/////////////////////////////////////////////////////////////////////////////// + +using namespace NPersQueue; + +void TPQReadService::TSession::OnCreated() { + // Start waiting for new session. + Proxy->WaitReadSession(); + if (Proxy->TooMuchSessions()) { + ReplyWithError("proxy overloaded", NPersQueue::NErrorCode::OVERLOAD); + return; + } + // Create actor for current session. + auto clusters = Proxy->GetClusters(); + auto localCluster = Proxy->GetLocalCluster(); + if (NeedDiscoverClusters && (clusters.empty() || localCluster.empty())) { + //TODO: inc sli errors counter + ReplyWithError("clusters list or local cluster is empty", NPersQueue::NErrorCode::INITIALIZING); + return; + + } + if (!TopicConverterFactory->GetLocalCluster().empty()) { + TopicConverterFactory->SetLocalCluster(localCluster); + } + auto topicsHandler = std::make_unique<NPersQueue::TTopicsListController>( + TopicConverterFactory, clusters + ); + + CreateActor(std::move(topicsHandler)); + ReadyForNextRead(); +} + +void TPQReadService::TSession::OnRead(const NPersQueue::TReadRequest& request) { + switch (request.GetRequestCase()) { + case TReadRequest::kInit: { + SendEvent(new TEvPQProxy::TEvReadInit(request, GetPeerName(), GetDatabase())); + break; + } + case TReadRequest::kRead: { + SendEvent(new TEvPQProxy::TEvRead(request)); + break; + } + case TReadRequest::kStatus: { + Y_VERIFY(ActorId); + const auto& req = request.GetStatus(); + const TString& topic = req.GetTopic(); + const ui32 partition = req.GetPartition(); + const ui64 generation = req.GetGeneration(); + SendEvent(new TEvPQProxy::TEvGetStatus(topic, partition, generation)); + ReadyForNextRead(); + break; + } + case TReadRequest::kStartRead: { + Y_VERIFY(ActorId); + const auto& req = request.GetStartRead(); + const TString& topic = req.GetTopic(); + const ui32 partition = req.GetPartition(); + const ui64 readOffset = req.GetReadOffset(); + const ui64 commitOffset = req.GetCommitOffset(); + const bool verifyReadOffset = req.GetVerifyReadOffset(); + const ui64 generation = req.GetGeneration(); + + if (request.GetCredentials().GetCredentialsCase() != NPersQueueCommon::TCredentials::CREDENTIALS_NOT_SET) { + SendEvent(new TEvPQProxy::TEvAuth(request.GetCredentials())); + } + SendEvent(new TEvPQProxy::TEvLocked(topic, partition, readOffset, commitOffset, verifyReadOffset, generation)); + ReadyForNextRead(); + break; + } + case TReadRequest::kCommit: { + Y_VERIFY(ActorId); + const auto& req = request.GetCommit(); + + if (request.GetCredentials().GetCredentialsCase() != NPersQueueCommon::TCredentials::CREDENTIALS_NOT_SET) { + SendEvent(new TEvPQProxy::TEvAuth(request.GetCredentials())); + } + + // Empty cookies list will lead to no effect. + for (ui32 i = 0; i < req.CookieSize(); ++i) { + SendEvent(new TEvPQProxy::TEvCommit(req.GetCookie(i))); + } + + ReadyForNextRead(); + break; + } + + default: { + SendEvent(new TEvPQProxy::TEvCloseSession("unsupported request", NPersQueue::NErrorCode::BAD_REQUEST)); + break; + } + } +} + +void TPQReadService::TSession::OnDone() { + SendEvent(new TEvPQProxy::TEvDone()); +} + +void TPQReadService::TSession::OnWriteDone(ui64 size) { + SendEvent(new TEvPQProxy::TEvWriteDone(size)); +} + +void TPQReadService::TSession::DestroyStream(const TString& reason, const NPersQueue::NErrorCode::EErrorCode errorCode) { + // Send poison pill to the actor(if it is alive) + SendEvent(new TEvPQProxy::TEvDieCommand("read-session " + ToString<ui64>(Cookie) + ": " + reason, errorCode)); + // Remove reference to session from "cookie -> session" map. + Proxy->ReleaseSession(Cookie); +} + +bool TPQReadService::TSession::IsShuttingDown() const { + return Proxy->IsShuttingDown(); +} + +TPQReadService::TSession::TSession(std::shared_ptr<TPQReadService> proxy, + grpc::ServerCompletionQueue* cq, ui64 cookie, const TActorId& schemeCache, const TActorId& newSchemeCache, + TIntrusivePtr<NMonitoring::TDynamicCounters> counters, bool needDiscoverClusters, + const NPersQueue::TConverterFactoryPtr& converterFactory) + : ISession(cq) + , Proxy(proxy) + , Cookie(cookie) + , ActorId() + , SchemeCache(schemeCache) + , NewSchemeCache(newSchemeCache) + , Counters(counters) + , NeedDiscoverClusters(needDiscoverClusters) + , TopicConverterFactory(converterFactory) +{ +} + +void TPQReadService::TSession::Start() { + if (!Proxy->IsShuttingDown()) { + Proxy->RequestSession(&Context, &Stream, CQ, CQ, new TRequestCreated(this)); + } +} + +void TPQReadService::TSession::SendEvent(IEventBase* ev) { + Proxy->ActorSystem->Send(ActorId, ev); +} + +void TPQReadService::TSession::CreateActor(std::unique_ptr<NPersQueue::TTopicsListController>&& topicsHandler) { + auto classifier = Proxy->GetClassifier(); + + ActorId = Proxy->ActorSystem->Register( + new TReadSessionActor(this, *topicsHandler, Cookie, SchemeCache, NewSchemeCache, Counters, + classifier ? classifier->ClassifyAddress(GetPeerName()) + : "unknown")); +} + + + +ui64 TPQReadService::TSession::GetCookie() const { + return Cookie; +} + +/////////////////////////////////////////////////////////////////////////////// + + +TPQReadService::TPQReadService(NKikimr::NGRpcService::TGRpcPersQueueService* service, grpc::ServerCompletionQueue* cq, + NActors::TActorSystem* as, const TActorId& schemeCache, + TIntrusivePtr<NMonitoring::TDynamicCounters> counters, + const ui32 maxSessions) + : Service(service) + , CQ(cq) + , ActorSystem(as) + , SchemeCache(schemeCache) + , Counters(counters) + , MaxSessions(maxSessions) +{ + auto appData = ActorSystem->AppData<TAppData>(); + auto cacheCounters = GetServiceCounters(counters, "pqproxy|schemecache"); + auto cacheConfig = MakeIntrusive<NSchemeCache::TSchemeCacheConfig>(appData, cacheCounters); + NewSchemeCache = ActorSystem->Register(CreateSchemeBoardSchemeCache(cacheConfig.Get())); + // ToDo[migration]: Other conditions; + NeedDiscoverClusters = !ActorSystem->AppData<TAppData>()->PQConfig.GetTopicsAreFirstClassCitizen(); + TopicConverterFactory = std::make_shared<NPersQueue::TTopicNamesConverterFactory>( + ActorSystem->AppData<TAppData>()->PQConfig, "" + ); + + if (NeedDiscoverClusters) { + ActorSystem->Register(new TClustersUpdater(this)); + } +} + + +ui64 TPQReadService::NextCookie() { + return AtomicIncrement(LastCookie); +} + + +void TPQReadService::ReleaseSession(ui64 cookie) { + auto g(Guard(Lock)); + bool erased = Sessions.erase(cookie); + if (erased) + ActorSystem->Send(MakeGRpcProxyStatusID(ActorSystem->NodeId), new TEvGRpcProxyStatus::TEvUpdateStatus(0,0,-1,0)); + +} + +void TPQReadService::CheckClusterChange(const TString& localCluster, const bool) { + auto g(Guard(Lock)); + LocalCluster = localCluster; + TopicConverterFactory->SetLocalCluster(localCluster); +} + +void TPQReadService::NetClassifierUpdated(NAddressClassifier::TLabeledAddressClassifier::TConstPtr classifier) { + auto g(Guard(Lock)); + if (!DatacenterClassifier) { + for (auto it = Sessions.begin(); it != Sessions.end();) { + auto jt = it++; + jt->second->DestroyStream("datacenter classifier initialized, restart session please", NPersQueue::NErrorCode::INITIALIZING); + } + } + + DatacenterClassifier = classifier; +} + + +void TPQReadService::CheckClustersListChange(const TVector<TString> &clusters) { + auto g(Guard(Lock)); + Clusters = clusters; +} + +void TPQReadService::SetupIncomingRequests() { + WaitReadSession(); +} + + +void TPQReadService::WaitReadSession() { + + const ui64 cookie = NextCookie(); + + ActorSystem->Send(MakeGRpcProxyStatusID(ActorSystem->NodeId), new TEvGRpcProxyStatus::TEvUpdateStatus(0,0,1,0)); + + TSessionRef session(new TSession(shared_from_this(), CQ, cookie, SchemeCache, NewSchemeCache, Counters, + NeedDiscoverClusters, TopicConverterFactory)); + + { + auto g(Guard(Lock)); + Sessions.insert(std::make_pair(cookie, session)); + } + + session->Start(); +} + + + +bool TPQReadService::TooMuchSessions() { + auto g(Guard(Lock)); + return Sessions.size() >= MaxSessions; +} + +/////////////////////////////////////////////////////////////////////////////// + +} +} diff --git a/kikimr/yndx/grpc_services/persqueue/grpc_pq_read.h b/kikimr/yndx/grpc_services/persqueue/grpc_pq_read.h new file mode 100644 index 0000000000..9fbc177e6f --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/grpc_pq_read.h @@ -0,0 +1,146 @@ +#pragma once + +#include "persqueue.h" +#include "grpc_pq_clusters_updater_actor.h" +#include "grpc_pq_session.h" + +#include <ydb/core/client/server/grpc_base.h> +#include <ydb/library/persqueue/topic_parser/topic_parser.h> + +#include <library/cpp/grpc/server/grpc_request.h> +#include <library/cpp/actors/core/actorsystem.h> + +#include <util/generic/hash.h> +#include <util/system/mutex.h> + +namespace NKikimr { +namespace NGRpcProxy { + +class TPQReadService : public IPQClustersUpdaterCallback, public std::enable_shared_from_this<TPQReadService> { + class TSession + : public ISession<NPersQueue::TReadRequest, NPersQueue::TReadResponse> + { + + public: + void OnCreated() override; + void OnRead(const NPersQueue::TReadRequest& request) override; + void OnDone() override; + void OnWriteDone(ui64 size) override; + void DestroyStream(const TString& reason, const NPersQueue::NErrorCode::EErrorCode errorCode) override; + bool IsShuttingDown() const override; + TSession(std::shared_ptr<TPQReadService> proxy, + grpc::ServerCompletionQueue* cq, ui64 cookie, const NActors::TActorId& schemeCache, const NActors::TActorId& newSchemeCache, + TIntrusivePtr<NMonitoring::TDynamicCounters> counters, bool needDiscoverClusters, + const NPersQueue::TConverterFactoryPtr& converterFactory); + void Start() override; + void SendEvent(NActors::IEventBase* ev); + + private: + void CreateActor(std::unique_ptr<NPersQueue::TTopicsListController>&& topicsHandler); + ui64 GetCookie() const; + + private: + std::shared_ptr<TPQReadService> Proxy; + const ui64 Cookie; + + NActors::TActorId ActorId; + + const NActors::TActorId SchemeCache; + const NActors::TActorId NewSchemeCache; + + TIntrusivePtr<NMonitoring::TDynamicCounters> Counters; + bool NeedDiscoverClusters; + + NPersQueue::TConverterFactoryPtr TopicConverterFactory; + + }; + + using TSessionRef = TIntrusivePtr<TSession>; + +public: + + TPQReadService(NGRpcService::TGRpcPersQueueService* service, + grpc::ServerCompletionQueue* cq, + NActors::TActorSystem* as, const NActors::TActorId& schemeCache, TIntrusivePtr<NMonitoring::TDynamicCounters> counters, + const ui32 maxSessions); + + virtual ~TPQReadService() + {} + + void RequestSession(::grpc::ServerContext* context, ::grpc::ServerAsyncReaderWriter< ::NPersQueue::TReadResponse, ::NPersQueue::TReadRequest>* stream, + ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) + { + Service->GetService()->RequestReadSession(context, stream, new_call_cq, notification_cq, tag); + } + + void SetupIncomingRequests(); + + void StopService() { + AtomicSet(ShuttingDown_, 1); + } + + bool IsShuttingDown() const { + return AtomicGet(ShuttingDown_); + } + + TVector<TString> GetClusters() const { + auto g(Guard(Lock)); + return Clusters; + } + TString GetLocalCluster() const { + auto g(Guard(Lock)); + return LocalCluster; + } + + NAddressClassifier::TLabeledAddressClassifier::TConstPtr GetClassifier() const { + auto g(Guard(Lock)); + return DatacenterClassifier; + } + +private: + ui64 NextCookie(); + + void CheckClustersListChange(const TVector<TString>& clusters) override; + void CheckClusterChange(const TString& localCluster, const bool enabled) override; + void NetClassifierUpdated(NAddressClassifier::TLabeledAddressClassifier::TConstPtr classifier) override; + void UpdateTopicsHandler(); + //! Unregistry session object. + void ReleaseSession(ui64 cookie); + + //! Start listening for incoming connections. + void WaitReadSession(); + + bool TooMuchSessions(); + +private: + NKikimr::NGRpcService::TGRpcPersQueueService* Service; + + grpc::ServerContext Context; + grpc::ServerCompletionQueue* CQ; + NActors::TActorSystem* ActorSystem; + NActors::TActorId SchemeCache; + NActors::TActorId NewSchemeCache; + + TAtomic LastCookie = 0; + TMutex Lock; + THashMap<ui64, TSessionRef> Sessions; + + TVector<TString> Clusters; + TString LocalCluster; + + TIntrusivePtr<NMonitoring::TDynamicCounters> Counters; + + ui32 MaxSessions; + + TAtomic ShuttingDown_ = 0; + + NAddressClassifier::TLabeledAddressClassifier::TConstPtr DatacenterClassifier; // Detects client's datacenter by IP. May be null + + bool NeedDiscoverClusters; + NPersQueue::TConverterFactoryPtr TopicConverterFactory; + std::unique_ptr<NPersQueue::TTopicsListController> TopicsHandler; +}; + + +} +} diff --git a/kikimr/yndx/grpc_services/persqueue/grpc_pq_read_actor.cpp b/kikimr/yndx/grpc_services/persqueue/grpc_pq_read_actor.cpp new file mode 100644 index 0000000000..e9e8798713 --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/grpc_pq_read_actor.cpp @@ -0,0 +1,2585 @@ +#include "grpc_pq_actor.h" + +#include <ydb/core/base/path.h> +#include <ydb/core/client/server/msgbus_server_persqueue.h> +#include <ydb/core/protos/services.pb.h> +#include <ydb/core/persqueue/percentile_counter.h> +#include <ydb/core/persqueue/pq_database.h> +#include <ydb/core/persqueue/write_meta.h> +#include <ydb/core/persqueue/writer/source_id_encoding.h> +#include <ydb/library/persqueue/topic_parser/type_definitions.h> +#include <ydb/library/persqueue/topic_parser/topic_parser.h> +#include <ydb/library/persqueue/topic_parser/counters.h> +#include <kikimr/yndx/persqueue/read_batch_converter/read_batch_converter.h> + +#include <library/cpp/actors/core/log.h> +#include <library/cpp/actors/interconnect/interconnect.h> +#include <library/cpp/protobuf/util/repeated_field_utils.h> + +#include <util/string/strip.h> +#include <util/charset/utf8.h> + +#include <algorithm> + +using namespace NActors; +using namespace NKikimrClient; + +namespace NKikimr { + +using namespace NMsgBusProxy; + +namespace NGRpcProxy { + +using namespace NPersQueue; +using namespace NSchemeCache; +#define PQ_LOG_PREFIX "session cookie " << Cookie << " client " << InternalClientId << " session " << Session + + +//11 tries = 10,23 seconds, then each try for 5 seconds , so 21 retries will take near 1 min +static const NTabletPipe::TClientRetryPolicy RetryPolicyForPipes = { + .RetryLimitCount = 21, + .MinRetryTime = TDuration::MilliSeconds(10), + .MaxRetryTime = TDuration::Seconds(5), + .BackoffMultiplier = 2, + .DoFirstRetryInstantly = true +}; + +static const ui64 MAX_INFLY_BYTES = 25_MB; +static const ui32 MAX_INFLY_READS = 10; + +static const TDuration READ_TIMEOUT_DURATION = TDuration::Seconds(1); + +static const TDuration WAIT_DATA = TDuration::Seconds(10); +static const TDuration PREWAIT_DATA = TDuration::Seconds(9); +static const TDuration WAIT_DELTA = TDuration::MilliSeconds(500); + +static const ui64 INIT_COOKIE = Max<ui64>(); //some identifier + +static const ui32 MAX_PIPE_RESTARTS = 100; //after 100 restarts without progress kill session +static const ui32 RESTART_PIPE_DELAY_MS = 100; + +static const ui64 MAX_READ_SIZE = 100 << 20; //100mb; + +static const TDuration DEFAULT_COMMIT_RATE = TDuration::Seconds(1); //1 second; +static const ui32 MAX_COMMITS_INFLY = 3; + +static const double LAG_GROW_MULTIPLIER = 1.2; //assume that 20% more data arrived to partitions + + +//TODO: add here tracking of bytes in/out + +#define LOG_PROTO(FieldName) \ + if (proto.Has##FieldName()) { \ + res << " " << Y_STRINGIZE(FieldName) << " { " << proto.Get##FieldName().ShortDebugString() << " }"; \ + } + +#define LOG_FIELD(proto, FieldName) \ + if (proto.Has##FieldName()) { \ + res << " " << Y_STRINGIZE(FieldName) << ": " << proto.Get##FieldName(); \ + } + +TString PartitionResponseToLog(const NKikimrClient::TPersQueuePartitionResponse& proto) { + if (!proto.HasCmdReadResult()) { + return proto.ShortDebugString(); + } + TStringBuilder res; + res << "{"; + + + if (proto.CmdWriteResultSize() > 0) { + res << " CmdWriteResult {"; + for (const auto& writeRes : proto.GetCmdWriteResult()) { + res << " { " << writeRes.ShortDebugString() << " }"; + } + res << " }"; + } + + LOG_PROTO(CmdGetMaxSeqNoResult); + LOG_PROTO(CmdGetClientOffsetResult); + LOG_PROTO(CmdGetOwnershipResult); + + + if (proto.HasCmdReadResult()) { + const auto& readRes = proto.GetCmdReadResult(); + res << " CmdReadResult {"; + LOG_FIELD(readRes, MaxOffset); + LOG_FIELD(readRes, BlobsFromDisk); + LOG_FIELD(readRes, BlobsFromCache); + //LOG_FIELD(readRes, ErrorCode); + LOG_FIELD(readRes, ErrorReason); + LOG_FIELD(readRes, BlobsCachedSize); + LOG_FIELD(readRes, SizeLag); + LOG_FIELD(readRes, RealReadOffset); + if (readRes.ResultSize() > 0) { + res << " Result {"; + for (const auto &tRes: readRes.GetResult()) { + res << " {"; + LOG_FIELD(tRes, Offset); + LOG_FIELD(tRes, SeqNo); + LOG_FIELD(tRes, PartNo); + LOG_FIELD(tRes, TotalParts); + LOG_FIELD(tRes, TotalSize); + LOG_FIELD(tRes, WriteTimestampMS); + LOG_FIELD(tRes, CreateTimestampMS); + LOG_FIELD(tRes, UncompressedSize); + LOG_FIELD(tRes, PartitionKey); + res << " }"; + } + res << " }"; + } + res << " }"; + } + res << " }"; + return res; +} +#undef LOG_PROTO +#undef LOG_FIELD + +class TPartitionActor : public NActors::TActorBootstrapped<TPartitionActor> { +public: + TPartitionActor(const TActorId& parentId, const TString& clientId, const ui64 cookie, const TString& session, const ui32 generation, + const ui32 step, const NPersQueue::TTopicConverterPtr& topic, const ui32 partition, const ui64 tabletID, + const TReadSessionActor::TTopicCounters& counters, const TString& clientDC); + ~TPartitionActor(); + + void Bootstrap(const NActors::TActorContext& ctx); + void Die(const NActors::TActorContext& ctx) override; + + + static constexpr NKikimrServices::TActivity::EType ActorActivityType() { return NKikimrServices::TActivity::FRONT_PQ_PARTITION; } +private: + STFUNC(StateFunc) { + switch (ev->GetTypeRewrite()) { + CFunc(NActors::TEvents::TSystem::Wakeup, HandleWakeup) + HFunc(TEvPQProxy::TEvDeadlineExceeded, Handle) + + HFunc(NActors::TEvents::TEvPoisonPill, HandlePoison) + HFunc(TEvPQProxy::TEvRead, Handle) + HFunc(TEvPQProxy::TEvCommit, Handle) + HFunc(TEvPQProxy::TEvReleasePartition, Handle) + HFunc(TEvPQProxy::TEvLockPartition, Handle) + HFunc(TEvPQProxy::TEvGetStatus, Handle) + HFunc(TEvPQProxy::TEvRestartPipe, Handle) + + HFunc(TEvTabletPipe::TEvClientDestroyed, Handle); + HFunc(TEvTabletPipe::TEvClientConnected, Handle); + HFunc(TEvPersQueue::TEvResponse, Handle); + HFunc(TEvPersQueue::TEvHasDataInfoResponse, Handle); + default: + break; + }; + } + + + void Handle(TEvPQProxy::TEvReleasePartition::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvLockPartition::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvGetStatus::TPtr& ev, const NActors::TActorContext& ctx); + + void Handle(TEvPQProxy::TEvDeadlineExceeded::TPtr& ev, const NActors::TActorContext& ctx); + + void Handle(TEvPQProxy::TEvRead::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPQProxy::TEvCommit::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(const TEvPQProxy::TEvRestartPipe::TPtr&, const NActors::TActorContext& ctx); + + void Handle(TEvTabletPipe::TEvClientConnected::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvTabletPipe::TEvClientDestroyed::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPersQueue::TEvResponse::TPtr& ev, const NActors::TActorContext& ctx); + void Handle(TEvPersQueue::TEvHasDataInfoResponse::TPtr& ev, const NActors::TActorContext& ctx); + + void HandlePoison(NActors::TEvents::TEvPoisonPill::TPtr& ev, const NActors::TActorContext& ctx); + void HandleWakeup(const NActors::TActorContext& ctx); + + void CheckRelease(const NActors::TActorContext& ctx); + void InitLockPartition(const NActors::TActorContext& ctx); + void InitStartReading(const NActors::TActorContext& ctx); + + void RestartPipe(const NActors::TActorContext& ctx, const TString& reason, const NPersQueue::NErrorCode::EErrorCode errorCode); + void WaitDataInPartition(const NActors::TActorContext& ctx); + void SendCommit(const ui64 readId, const ui64 offset, const TActorContext& ctx); + +private: + const TActorId ParentId; + const TString InternalClientId; + const TString ClientDC; + const ui64 Cookie; + const TString Session; + const ui32 Generation; + const ui32 Step; + + NPersQueue::TTopicConverterPtr Topic; + const ui32 Partition; + + const ui64 TabletID; + + ui64 ReadOffset; + ui64 ClientReadOffset; + ui64 ClientCommitOffset; + bool ClientVerifyReadOffset; + ui64 CommittedOffset; + ui64 WriteTimestampEstimateMs; + + ui64 WTime; + bool InitDone; + bool StartReading; + bool AllPrepareInited; + bool FirstInit; + TActorId PipeClient; + ui32 PipeGeneration; + bool RequestInfly; + NKikimrClient::TPersQueueRequest CurrentRequest; + + ui64 EndOffset; + ui64 SizeLag; + + TString ReadGuid; // empty if not reading + + bool NeedRelease; + bool Released; + + std::set<ui64> WaitDataInfly; + ui64 WaitDataCookie; + bool WaitForData; + + bool LockCounted; + + std::deque<std::pair<ui64, ui64>> CommitsInfly; //ReadId, Offset + + TReadSessionActor::TTopicCounters Counters; +}; + + +TReadSessionActor::TReadSessionActor( + IReadSessionHandlerRef handler, const NPersQueue::TTopicsListController& topicsHandler, const ui64 cookie, + const TActorId& pqMetaCache, const TActorId& newSchemeCache, TIntrusivePtr<NMonitoring::TDynamicCounters> counters, + const TMaybe<TString> clientDC +) + : Handler(handler) + , StartTimestamp(TInstant::Now()) + , PqMetaCache(pqMetaCache) + , NewSchemeCache(newSchemeCache) + , AuthInitActor() + , AuthInflight(false) + , ClientDC(clientDC ? *clientDC : "other") + , ClientPath() + , Session() + , ClientsideLocksAllowed(false) + , BalanceRightNow(false) + , CommitsDisabled(false) + , BalancersInitStarted(false) + , InitDone(false) + , ProtocolVersion(NPersQueue::TReadRequest::Base) + , MaxReadMessagesCount(0) + , MaxReadSize(0) + , MaxReadPartitionsCount(0) + , MaxTimeLagMs(0) + , ReadTimestampMs(0) + , ReadSettingsInited(false) + , ForceACLCheck(false) + , RequestNotChecked(true) + , LastACLCheckTimestamp(TInstant::Zero()) + , ReadOnlyLocal(false) + , ReadIdToResponse(1) + , ReadIdCommitted(0) + , LastCommitTimestamp(TInstant::Zero()) + , CommitInterval(DEFAULT_COMMIT_RATE) + , CommitsInfly(0) + , Cookie(cookie) + , Counters(counters) + , BytesInflight_(0) + , RequestedBytes(0) + , ReadsInfly(0) + , TopicsHandler(topicsHandler) +{ + Y_ASSERT(Handler); +} + + + +TReadSessionActor::~TReadSessionActor() = default; + + +void TReadSessionActor::Bootstrap(const TActorContext& ctx) { + if (!AppData(ctx)->PQConfig.GetTopicsAreFirstClassCitizen()) { + ++(*GetServiceCounters(Counters, "pqproxy|readSession")->GetCounter("SessionsCreatedTotal", true)); + } + StartTime = ctx.Now(); + Become(&TThis::StateFunc); +} + + +void TReadSessionActor::Die(const TActorContext& ctx) { + + ctx.Send(AuthInitActor, new TEvents::TEvPoisonPill()); + + for (auto& p : Partitions) { + ctx.Send(p.second.Actor, new TEvents::TEvPoisonPill()); + + if (!p.second.Released) { + auto it = TopicCounters.find(p.second.Converter->GetInternalName()); + Y_VERIFY(it != TopicCounters.end()); + it->second.PartitionsInfly.Dec(); + it->second.PartitionsReleased.Inc(); + if (p.second.Releasing) + it->second.PartitionsToBeReleased.Dec(); + } + } + + for (auto& t : Topics) { + if (t.second.PipeClient) + NTabletPipe::CloseClient(ctx, t.second.PipeClient); + } + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " is DEAD"); + + if (SessionsActive) { + --(*SessionsActive); + } + if (BytesInflight) { + (*BytesInflight) -= BytesInflight_; + } + if (SessionsActive) { //PartsPerSession is inited too + PartsPerSession.DecFor(Partitions.size(), 1); + } + if (!Handler->IsShuttingDown()) + Handler->Finish(); + TActorBootstrapped<TReadSessionActor>::Die(ctx); +} + +void TReadSessionActor::Handle(TEvPQProxy::TEvDone::TPtr&, const TActorContext& ctx) { + CloseSession(TStringBuilder() << "Reads done signal - closing everything", NPersQueue::NErrorCode::OK, ctx); +} + +void TReadSessionActor::Handle(TEvPQProxy::TEvWriteDone::TPtr& ev, const TActorContext& ctx) { + Y_VERIFY(BytesInflight_ >= ev->Get()->Size); + BytesInflight_ -= ev->Get()->Size; + if (BytesInflight) (*BytesInflight) -= ev->Get()->Size; + + const bool isAlive = ProcessReads(ctx); + Y_UNUSED(isAlive); +} + +void TReadSessionActor::Handle(TEvPQProxy::TEvCommit::TPtr& ev, const TActorContext& ctx) { + RequestNotChecked = true; + + if (CommitsDisabled) { + CloseSession(TStringBuilder() << "commits in session are disabled by client option", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + + const ui64 readId = ev->Get()->ReadId; + if (readId <= ReadIdCommitted) { + CloseSession(TStringBuilder() << "commit of " << ev->Get()->ReadId << " that is already committed", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + if (readId >= ReadIdToResponse) { + CloseSession(TStringBuilder() << "commit of unknown cookie " << readId, NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + if (NextCommits.size() >= AppData(ctx)->PQConfig.GetMaxReadCookies()) { + CloseSession(TStringBuilder() << "got more than " << AppData(ctx)->PQConfig.GetMaxReadCookies() << " unordered cookies to commit " << readId, NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + + bool res = NextCommits.insert(readId).second; + if (!res) { + CloseSession(TStringBuilder() << "double commit of cookie " << readId, NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " commit request from client for " << readId); + MakeCommit(ctx); +} + +void TReadSessionActor::MakeCommit(const TActorContext& ctx) { + if (CommitsDisabled) + return; + if (ctx.Now() - LastCommitTimestamp < CommitInterval) + return; + if (CommitsInfly > MAX_COMMITS_INFLY) + return; + ui64 readId = ReadIdCommitted; + auto it = NextCommits.begin(); + for (;it != NextCommits.end() && (*it) == readId + 1; ++it) { + ++readId; + } + if (readId == ReadIdCommitted) + return; + NextCommits.erase(NextCommits.begin(), it); + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " commit request from " << ReadIdCommitted + 1 << " to " << readId); + + auto& commit = Commits[readId]; + commit.StartReadId = ReadIdCommitted + 1; + commit.Partitions = 0; + commit.StartTime = ctx.Now(); + ReadIdCommitted = readId; + LastCommitTimestamp = ctx.Now(); + ++CommitsInfly; + SLITotal.Inc(); + Y_VERIFY(Commits.size() == CommitsInfly); + + // Find last offset info belonging to our read id and its ancestors. + const auto firstGreater = std::upper_bound(Offsets.begin(), Offsets.end(), readId); + THashSet<std::pair<TString, ui64>> processedPartitions; + + // Iterate from last to first offsets to find partitions' offsets. + // Offsets in queue have nondecreasing values (for each partition), + // so it it sufficient to take only the last offset for each partition. + // Note: reverse_iterator(firstGreater) points to _before_ firstGreater + + for (auto i = std::make_reverse_iterator(firstGreater), end = std::make_reverse_iterator(Offsets.begin()); i != end; ++i) { + const TOffsetsInfo& info = *i; + for (const TOffsetsInfo::TPartitionOffsetInfo& pi : info.PartitionOffsets) { + if (!ActualPartitionActor(pi.Sender)) { + continue; + } + const auto partitionKey = std::make_pair(pi.Topic, pi.Partition); + if (!processedPartitions.insert(partitionKey).second) { + continue; // already processed + } + const auto partitionIt = Partitions.find(partitionKey); + if (partitionIt != Partitions.end() && !partitionIt->second.Released) { + ctx.Send(partitionIt->second.Actor, new TEvPQProxy::TEvCommit(readId, pi.Offset)); + partitionIt->second.Commits.push_back(readId); + ++commit.Partitions; + } + } + } + Offsets.erase(Offsets.begin(), firstGreater); + + AnswerForCommitsIfCan(ctx); //Could be done if all partitions are lost because of balancer dead +} + +void TReadSessionActor::Handle(TEvPQProxy::TEvAuth::TPtr& ev, const TActorContext&) { + ProcessAuth(ev->Get()->Auth); +} + +void TReadSessionActor::Handle(TEvPQProxy::TEvGetStatus::TPtr& ev, const TActorContext& ctx) { + + if (!ClientsideLocksAllowed) { + CloseSession("Partition status available only when ClientsideLocksAllowed is true", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + + auto it = Partitions.find(std::make_pair(ev->Get()->Topic, ev->Get()->Partition)); + + if (it == Partitions.end() || it->second.Releasing || it->second.LockGeneration != ev->Get()->Generation) { + //do nothing - already released partition + LOG_WARN_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " got NOTACTUAL get status request from client for " << ev->Get()->Topic + << ":" << ev->Get()->Partition << " generation " << ev->Get()->Generation); + return; + } + + //proxy request to partition - allow initing + //TODO: add here VerifyReadOffset too and check it against Committed position + ctx.Send(it->second.Actor, new TEvPQProxy::TEvGetStatus(ev->Get()->Topic, ev->Get()->Partition, ev->Get()->Generation)); +} + + +void TReadSessionActor::Handle(TEvPQProxy::TEvLocked::TPtr& ev, const TActorContext& ctx) { + + RequestNotChecked = true; + if (!ClientsideLocksAllowed) { + CloseSession("Locked requests are allowed only when ClientsideLocksAllowed is true", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + auto& topic = ev->Get()->Topic; + if (topic.empty()) { + CloseSession("empty topic in start_read request", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + + } + auto it = Partitions.find(std::make_pair(topic, ev->Get()->Partition)); + + if (it == Partitions.end() || it->second.Releasing || it->second.LockGeneration != ev->Get()->Generation) { + //do nothing - already released partition + LOG_WARN_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " got NOTACTUAL lock from client for " << topic + << ":" << ev->Get()->Partition << " at offset " << ev->Get()->ReadOffset << " generation " << ev->Get()->Generation); + + return; + } + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " got lock from client for " << ev->Get()->Topic + << ":" << ev->Get()->Partition << " at readOffset " << ev->Get()->ReadOffset << " commitOffset " << ev->Get()->CommitOffset << " generation " << ev->Get()->Generation); + + //proxy request to partition - allow initing + //TODO: add here VerifyReadOffset too and check it against Committed position + ctx.Send(it->second.Actor, new TEvPQProxy::TEvLockPartition(ev->Get()->ReadOffset, ev->Get()->CommitOffset, ev->Get()->VerifyReadOffset, true)); +} + +void TReadSessionActor::DropPartitionIfNeeded(THashMap<std::pair<TString, ui32>, TPartitionActorInfo>::iterator it, const TActorContext& ctx) { + if (it->second.Commits.empty() && it->second.Released) { + ctx.Send(it->second.Actor, new TEvents::TEvPoisonPill()); + bool res = ActualPartitionActors.erase(it->second.Actor); + Y_VERIFY(res); + + if (--NumPartitionsFromTopic[it->second.Converter->GetInternalName()] == 0) { + bool res = TopicCounters.erase(it->second.Converter->GetInternalName()); + Y_VERIFY(res); + } + + if (SessionsActive) { + PartsPerSession.DecFor(Partitions.size(), 1); + } + Partitions.erase(it); + if (SessionsActive) { + PartsPerSession.IncFor(Partitions.size(), 1); + } + } +} + + +void TReadSessionActor::Handle(TEvPQProxy::TEvCommitDone::TPtr& ev, const TActorContext& ctx) { + + Y_VERIFY(!CommitsDisabled); + + if (!ActualPartitionActor(ev->Sender)) + return; + + ui64 readId = ev->Get()->ReadId; + + auto it = Commits.find(readId); + Y_VERIFY(it != Commits.end()); + --it->second.Partitions; + + auto jt = Partitions.find(std::make_pair(ev->Get()->Topic->GetClientsideName(), ev->Get()->Partition)); + Y_VERIFY(jt != Partitions.end()); + Y_VERIFY(!jt->second.Commits.empty() && jt->second.Commits.front() == readId); + jt->second.Commits.pop_front(); + + DropPartitionIfNeeded(jt, ctx); + + AnswerForCommitsIfCan(ctx); + + MakeCommit(ctx); +} + +void TReadSessionActor::AnswerForCommitsIfCan(const TActorContext& ctx) { + while (!Commits.empty() && Commits.begin()->second.Partitions == 0) { + auto it = Commits.begin(); + ui64 readId = it->first; + TReadResponse result; + for (ui64 i = it->second.StartReadId; i <= readId; ++i){ + result.MutableCommit()->AddCookie(i); + } + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " replying for commits from " << it->second.StartReadId + << " to " << readId); + ui64 diff = result.ByteSize(); + BytesInflight_ += diff; + if (BytesInflight) (*BytesInflight) += diff; + Handler->Reply(result); + + ui32 commitDurationMs = (ctx.Now() - it->second.StartTime).MilliSeconds(); + CommitLatency.IncFor(commitDurationMs, 1); + if (commitDurationMs >= AppData(ctx)->PQConfig.GetCommitLatencyBigMs()) { + SLIBigLatency.Inc(); + } + Commits.erase(it); + --CommitsInfly; + Y_VERIFY(Commits.size() == CommitsInfly); + } +} + + +void TReadSessionActor::Handle(TEvPQProxy::TEvReadSessionStatus::TPtr& ev, const TActorContext& ctx) { + + THolder<TEvPQProxy::TEvReadSessionStatusResponse> result(new TEvPQProxy::TEvReadSessionStatusResponse()); + result->Record.SetSession(Session); + result->Record.SetTimestamp(StartTimestamp.MilliSeconds()); + + result->Record.SetClientNode(PeerName); + result->Record.SetProxyNodeId(ctx.SelfID.NodeId()); + + for (auto& p : Partitions) { + auto part = result->Record.AddPartition(); + part->SetTopic(p.first.first); + part->SetPartition(p.first.second); + part->SetAssignId(0); + for (auto& c : NextCommits) { + part->AddNextCommits(c); + } + part->SetReadIdCommitted(ReadIdCommitted); + part->SetLastReadId(ReadIdToResponse - 1); + part->SetTimestampMs(0); + } + + ctx.Send(ev->Sender, result.Release()); +} + +void TReadSessionActor::Handle(TEvPQProxy::TEvReadInit::TPtr& ev, const TActorContext& ctx) { + + THolder<TEvPQProxy::TEvReadInit> event(ev->Release()); + + if (!Topics.empty()) { + //answer error + CloseSession("got second init request", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + + const auto& init = event->Request.GetInit(); + + if (!init.TopicsSize()) { + CloseSession("no topics in init request", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + + if (init.GetClientId().empty()) { + CloseSession("no clientId in init request", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + + if (init.GetProxyCookie() != ctx.SelfID.NodeId() && init.GetProxyCookie() != MAGIC_COOKIE_VALUE) { + CloseSession("you must perform ChooseProxy request at first and go to ProxyName server with ProxyCookie", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + + // ToDo[migration] - consider separate consumer conversion logic - ? + if (AppData(ctx)->PQConfig.GetTopicsAreFirstClassCitizen()) { + ClientPath = init.GetClientId(); + ExternalClientId = ClientPath; + InternalClientId = ConvertNewConsumerName(init.GetClientId()); + } else { + ClientPath = NormalizeFullPath(MakeConsumerPath(init.GetClientId())); + ExternalClientId = ClientPath; + InternalClientId = ConvertNewConsumerName(init.GetClientId()); + } + + Auth = event->Request.GetCredentials(); + event->Request.ClearCredentials(); + Y_PROTOBUF_SUPPRESS_NODISCARD Auth.SerializeToString(&AuthStr); + TStringBuilder session; + session << ExternalClientId << "_" << ctx.SelfID.NodeId() << "_" << Cookie << "_" << TAppData::RandomProvider->GenRand64(); + Session = session; + ProtocolVersion = init.GetProtocolVersion(); + CommitsDisabled = init.GetCommitsDisabled(); + + if (ProtocolVersion >= NPersQueue::TReadRequest::ReadParamsInInit) { + ReadSettingsInited = true; + MaxReadMessagesCount = NormalizeMaxReadMessagesCount(init.GetMaxReadMessagesCount()); + MaxReadSize = NormalizeMaxReadSize(init.GetMaxReadSize()); + MaxReadPartitionsCount = NormalizeMaxReadPartitionsCount(init.GetMaxReadPartitionsCount()); + MaxTimeLagMs = init.GetMaxTimeLagMs(); + ReadTimestampMs = init.GetReadTimestampMs(); + } + + PeerName = event->PeerName; + Database = event->Database; + + ReadOnlyLocal = init.GetReadOnlyLocal(); + + if (init.GetCommitIntervalMs()) { + CommitInterval = Min(CommitInterval, TDuration::MilliSeconds(init.GetCommitIntervalMs())); + } + + for (ui32 i = 0; i < init.PartitionGroupsSize(); ++i) { + Groups.push_back(init.GetPartitionGroups(i)); + } + THashSet<TString> topicsToResolve; + for (ui32 i = 0; i < init.TopicsSize(); ++i) { + const auto& t = init.GetTopics(i); + + if (t.empty()) { + CloseSession("empty topic in init request", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + + topicsToResolve.insert(t); + } + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " init: " << event->Request << " from " << PeerName); + + ClientsideLocksAllowed = init.GetClientsideLocksAllowed(); + BalanceRightNow = init.GetBalancePartitionRightNow() || CommitsDisabled; + + if (!AppData(ctx)->PQConfig.GetTopicsAreFirstClassCitizen()) { + SetupCounters(); + } + + if (Auth.GetCredentialsCase() == NPersQueueCommon::TCredentials::CREDENTIALS_NOT_SET) { + LOG_WARN_S(ctx, NKikimrServices::PQ_READ_PROXY, "session without AuthInfo : " << ExternalClientId << " from " << PeerName); + if (SessionsWithoutAuth) { + ++(*SessionsWithoutAuth); + } + if (AppData(ctx)->PQConfig.GetRequireCredentialsInNewProtocol()) { + CloseSession("Unauthenticated access is forbidden, please provide credentials", NPersQueue::NErrorCode::ACCESS_DENIED, ctx); + return; + } + } + TopicsList = TopicsHandler.GetReadTopicsList( + topicsToResolve, ReadOnlyLocal, Database + ); + if (!TopicsList.IsValid) { + return CloseSession( + TopicsList.Reason, + NPersQueue::NErrorCode::BAD_REQUEST, ctx + ); + } + SendAuthRequest(ctx); + + auto subGroup = GetServiceCounters(Counters, "pqproxy|SLI"); + Aggr = {{{{"Account", ClientPath.substr(0, ClientPath.find("/"))}}, {"total"}}}; + SLITotal = NKikimr::NPQ::TMultiCounter(subGroup, Aggr, {}, {"RequestsTotal"}, true, "sensor", false); + SLIErrors = NKikimr::NPQ::TMultiCounter(subGroup, Aggr, {}, {"RequestsError"}, true, "sensor", false); + + SLITotal.Inc(); +} + + +void TReadSessionActor::SendAuthRequest(const TActorContext& ctx) { + AuthInitActor = {}; + AuthInflight = true; + + if (Auth.GetCredentialsCase() == NPersQueueCommon::TCredentials::CREDENTIALS_NOT_SET) { + Token = nullptr; + CreateInitAndAuthActor(ctx); + return; + } + auto database = Database.empty() ? NKikimr::NPQ::GetDatabaseFromConfig(AppData(ctx)->PQConfig) : Database; + Y_VERIFY(TopicsList.IsValid); + TVector<TDiscoveryConverterPtr> topics; + for(const auto& t : TopicsList.Topics) { + if (topics.size() >= 10) { + break; + } + topics.push_back(t.second); + } + ctx.Send(PqMetaCache, new TEvDescribeTopicsRequest(topics, false)); +} + + + +void TReadSessionActor::HandleDescribeTopicsResponse(TEvDescribeTopicsResponse::TPtr& ev, const TActorContext& ctx) { + TString dbId, folderId; + for (const auto& entry : ev->Get()->Result->ResultSet) { + Y_VERIFY(entry.PQGroupInfo); // checked at ProcessMetaCacheTopicResponse() + auto& pqDescr = entry.PQGroupInfo->Description; + dbId = pqDescr.GetPQTabletConfig().GetYdbDatabaseId(); + folderId = pqDescr.GetPQTabletConfig().GetYcFolderId(); + break; + } + + auto entries = GetTicketParserEntries(dbId, folderId); + + TString ticket; + switch (Auth.GetCredentialsCase()) { + case NPersQueueCommon::TCredentials::kTvmServiceTicket: + ticket = Auth.GetTvmServiceTicket(); + break; + case NPersQueueCommon::TCredentials::kOauthToken: + ticket = Auth.GetOauthToken(); + break; + default: + CloseSession("Unknown Credentials case", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + + ctx.Send(MakeTicketParserID(), new TEvTicketParser::TEvAuthorizeTicket({ + .Database = Database, + .Ticket = ticket, + .PeerName = PeerName, + .Entries = entries + })); +} + +void TReadSessionActor::CreateInitAndAuthActor(const TActorContext& ctx) { + auto database = Database.empty() ? NKikimr::NPQ::GetDatabaseFromConfig(AppData(ctx)->PQConfig) : Database; + AuthInitActor = ctx.Register(new V1::TReadInitAndAuthActor( + ctx, ctx.SelfID, InternalClientId, Cookie, Session, PqMetaCache, NewSchemeCache, Counters, Token, + TopicsList, TopicsHandler.GetLocalCluster() + )); +} + +void TReadSessionActor::Handle(TEvTicketParser::TEvAuthorizeTicketResult::TPtr& ev, const TActorContext& ctx) { + TString ticket = ev->Get()->Ticket; + TString maskedTicket = ticket.size() > 5 ? (ticket.substr(0, 5) + "***" + ticket.substr(ticket.size() - 5)) : "***"; + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, "CheckACL ticket " << maskedTicket << " got result from TICKET_PARSER response: error: " << ev->Get()->Error << " user: " + << (ev->Get()->Error.empty() ? ev->Get()->Token->GetUserSID() : "")); + + if (!ev->Get()->Error.empty()) { + CloseSession(TStringBuilder() << "Ticket parsing error: " << ev->Get()->Error, NPersQueue::NErrorCode::ACCESS_DENIED, ctx); + return; + } + Token = ev->Get()->Token; + CreateInitAndAuthActor(ctx); +} + + +void TReadSessionActor::RegisterSession(const TActorId& pipe, const TString& topic, const TActorContext& ctx) { + + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " register session to " << topic); + THolder<TEvPersQueue::TEvRegisterReadSession> request; + request.Reset(new TEvPersQueue::TEvRegisterReadSession); + auto& req = request->Record; + req.SetSession(Session); + req.SetClientNode(PeerName); + ActorIdToProto(pipe, req.MutablePipeClient()); + req.SetClientId(InternalClientId); + + for (ui32 i = 0; i < Groups.size(); ++i) { + req.AddGroups(Groups[i]); + } + + NTabletPipe::SendData(ctx, pipe, request.Release()); +} + +void TReadSessionActor::RegisterSessions(const TActorContext& ctx) { + InitDone = true; + + for (auto& t : Topics) { + auto& topic = t.first; + RegisterSession(t.second.PipeClient, topic, ctx); + NumPartitionsFromTopic[topic] = 0; + } +} + + +void TReadSessionActor::SetupCounters() +{ + if (SessionsCreated) { + return; + } + + auto subGroup = GetServiceCounters(Counters, "pqproxy|readSession")->GetSubgroup("Client", InternalClientId)->GetSubgroup("ConsumerPath", ClientPath); + SessionsCreated = subGroup->GetExpiringCounter("SessionsCreated", true); + SessionsActive = subGroup->GetExpiringCounter("SessionsActive", false); + SessionsWithoutAuth = subGroup->GetExpiringCounter("WithoutAuth", true); + SessionsWithOldBatchingVersion = subGroup->GetExpiringCounter("SessionsWithOldBatchingVersion", true); // monitoring to ensure that old version is not used anymore + Errors = subGroup->GetExpiringCounter("Errors", true); + PipeReconnects = subGroup->GetExpiringCounter("PipeReconnects", true); + + BytesInflight = subGroup->GetExpiringCounter("BytesInflight", false); + + PartsPerSession = NKikimr::NPQ::TPercentileCounter(subGroup->GetSubgroup("sensor", "PartsPerSession"), {}, {}, "Count", + TVector<std::pair<ui64, TString>>{{1, "1"}, {2, "2"}, {5, "5"}, + {10, "10"}, {20, "20"}, {50, "50"}, {70, "70"}, + {100, "100"}, {150, "150"}, {300,"300"}, {99999999, "99999999"}}, false); + + ++(*SessionsCreated); + ++(*SessionsActive); + PartsPerSession.IncFor(Partitions.size(), 1); //for 0 + + if (ProtocolVersion < NPersQueue::TReadRequest::Batching) { + ++(*SessionsWithOldBatchingVersion); + } +} + + +void TReadSessionActor::SetupTopicCounters(const TTopicConverterPtr& topic) +{ + auto& topicCounters = TopicCounters[topic->GetInternalName()]; + auto subGroup = GetServiceCounters(Counters, "pqproxy|readSession"); +//client/consumerPath Account/Producer OriginDC Topic/TopicPath + + auto aggr = GetLabels(topic); + TVector<std::pair<TString, TString>> cons = {{"Client", InternalClientId}, {"ConsumerPath", ClientPath}}; + + topicCounters.PartitionsLocked = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"PartitionsLocked"}, true); + topicCounters.PartitionsReleased = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"PartitionsReleased"}, true); + topicCounters.PartitionsToBeReleased = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"PartitionsToBeReleased"}, false); + topicCounters.PartitionsToBeLocked = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"PartitionsToBeLocked"}, false); + topicCounters.PartitionsInfly = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"PartitionsInfly"}, false); + topicCounters.Errors = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"PartitionsErrors"}, true); + topicCounters.Commits = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"Commits"}, true); + topicCounters.WaitsForData = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"WaitsForData"}, true); +} + +void TReadSessionActor::SetupTopicCounters(const TTopicConverterPtr& topic, const TString& cloudId, + const TString& dbId, const TString& folderId) +{ + auto& topicCounters = TopicCounters[topic->GetInternalName()]; + auto subGroup = NPersQueue::GetCountersForStream(Counters); +//client/consumerPath Account/Producer OriginDC Topic/TopicPath + auto aggr = GetLabelsForStream(topic, cloudId, dbId, folderId); + TVector<std::pair<TString, TString>> cons = {{"consumer", ClientPath}}; + + topicCounters.PartitionsLocked = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"stream.internal_read.partitions_locked_per_second"}, true, "name"); + topicCounters.PartitionsReleased = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"stream.internal_read.partitions_released_per_second"}, true, "name"); + topicCounters.PartitionsToBeReleased = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"stream.internal_read.partitions_to_be_released"}, false, "name"); + topicCounters.PartitionsToBeLocked = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"stream.internal_read.partitions_to_be_locked"}, false, "name"); + topicCounters.PartitionsInfly = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"stream.internal_read.partitions_locked"}, false, "name"); + topicCounters.Errors = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"stream.internal_read.partitions_errors_per_second"}, true, "name"); + topicCounters.Commits = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"stream.internal_read.commits_per_second"}, true, "name"); + topicCounters.WaitsForData = NKikimr::NPQ::TMultiCounter(subGroup, aggr, cons, {"stream.internal_read.waits_for_data"}, true, "name"); +} + +void TReadSessionActor::Handle(V1::TEvPQProxy::TEvAuthResultOk::TPtr& ev, const TActorContext& ctx) { + + LastACLCheckTimestamp = ctx.Now(); + + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " auth ok, got " << ev->Get()->TopicAndTablets.size() << " topics, init done " << InitDone); + + AuthInitActor = TActorId(); + AuthInflight = false; + + if (!InitDone) { + + ui32 initBorder = AppData(ctx)->PQConfig.GetReadInitLatencyBigMs(); + ui32 readBorder = AppData(ctx)->PQConfig.GetReadLatencyBigMs(); + ui32 readBorderFromDisk = AppData(ctx)->PQConfig.GetReadLatencyFromDiskBigMs(); + + auto subGroup = GetServiceCounters(Counters, "pqproxy|SLI"); + InitLatency = NKikimr::NPQ::CreateSLIDurationCounter(subGroup, Aggr, "ReadInit", initBorder, {100, 200, 500, 1000, 1500, 2000, 5000, 10000, 30000, 99999999}); + CommitLatency = NKikimr::NPQ::CreateSLIDurationCounter(subGroup, Aggr, "Commit", AppData(ctx)->PQConfig.GetCommitLatencyBigMs(), {100, 200, 500, 1000, 1500, 2000, 5000, 10000, 30000, 99999999}); + SLIBigLatency = NKikimr::NPQ::TMultiCounter(subGroup, Aggr, {}, {"RequestsBigLatency"}, true, "sensor", false); + ReadLatency = NKikimr::NPQ::CreateSLIDurationCounter(subGroup, Aggr, "Read", readBorder, {100, 200, 500, 1000, 1500, 2000, 5000, 10000, 30000, 99999999}); + ReadLatencyFromDisk = NKikimr::NPQ::CreateSLIDurationCounter(subGroup, Aggr, "ReadFromDisk", readBorderFromDisk, {100, 200, 500, 1000, 1500, 2000, 5000, 10000, 30000, 99999999}); + SLIBigReadLatency = NKikimr::NPQ::TMultiCounter(subGroup, Aggr, {}, {"ReadBigLatency"}, true, "sensor", false); + ReadsTotal = NKikimr::NPQ::TMultiCounter(subGroup, Aggr, {}, {"ReadsTotal"}, true, "sensor", false); + + ui32 initDurationMs = (ctx.Now() - StartTime).MilliSeconds(); + InitLatency.IncFor(initDurationMs, 1); + if (initDurationMs >= initBorder) { + SLIBigLatency.Inc(); + } + + + TReadResponse result; + result.MutableInit()->SetSessionId(Session); + ui64 diff = result.ByteSize(); + BytesInflight_ += diff; + if (BytesInflight) (*BytesInflight) += diff; + + Handler->Reply(result); + + Handler->ReadyForNextRead(); + + Y_VERIFY(!BalancersInitStarted); + BalancersInitStarted = true; + + for (auto& t : ev->Get()->TopicAndTablets) { + auto& topicHolder = Topics[t.TopicNameConverter->GetInternalName()]; + topicHolder.TabletID = t.TabletID; + topicHolder.CloudId = t.CloudId; + topicHolder.DbId = t.DbId; + topicHolder.FolderId = t.FolderId; + topicHolder.FullConverter = t.TopicNameConverter; + FullPathToConverter[t.TopicNameConverter->GetPrimaryPath()] = t.TopicNameConverter; + const auto& second = t.TopicNameConverter->GetSecondaryPath(); + if (!second.empty()) { + FullPathToConverter[second] = t.TopicNameConverter; + } + } + + for (auto& t : Topics) { + NTabletPipe::TClientConfig clientConfig; + + clientConfig.CheckAliveness = false; + + clientConfig.RetryPolicy = RetryPolicyForPipes; + t.second.PipeClient = ctx.RegisterWithSameMailbox(NTabletPipe::CreateClient(ctx.SelfID, t.second.TabletID, clientConfig)); + } + + RegisterSessions(ctx); + + ctx.Schedule(Min(CommitInterval, CHECK_ACL_DELAY), new TEvents::TEvWakeup()); + } else { + for (auto& t : ev->Get()->TopicAndTablets) { + if (Topics.find(t.TopicNameConverter->GetInternalName()) == Topics.end()) { + CloseSession(TStringBuilder() << "list of topics changed - new topic '" << + t.TopicNameConverter->GetPrimaryPath() << "' found", + NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + } + } +} + + +void TReadSessionActor::Handle(TEvPersQueue::TEvLockPartition::TPtr& ev, const TActorContext& ctx) { + + auto& record = ev->Get()->Record; + Y_VERIFY(record.GetSession() == Session); + Y_VERIFY(record.GetClientId() == InternalClientId); + + TActorId pipe = ActorIdFromProto(record.GetPipeClient()); + auto path = record.GetPath(); + if (path.empty()) { + path = record.GetTopic(); + } + + auto converterIter = FullPathToConverter.find(NPersQueue::NormalizeFullPath(path)); + if (converterIter.IsEnd()) { + LOG_ALERT_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << " ignored ev lock for event = " << record.ShortDebugString() << " path not recognized" + ); + CloseSession( + TStringBuilder() << "Internal server error, cannot parse lock event: " << record.ShortDebugString() << ", reason: topic not found", + NPersQueue::NErrorCode::ERROR, ctx + ); + return; + } + //auto topic = converterIter->second->GetClientsideName(); + auto intName = converterIter->second->GetInternalName(); + Y_VERIFY(!intName.empty()); + auto jt = Topics.find(intName); + + if (jt == Topics.end() || pipe != jt->second.PipeClient) { //this is message from old version of pipe + LOG_DEBUG_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << " ignored ev lock for topic = " << converterIter->second->GetPrintableString() + << " path recognized, but topic is unknown, this is unexpected" + ); + return; + } + // ToDo[counters] + if (NumPartitionsFromTopic[intName]++ == 0) { + if (AppData(ctx)->PQConfig.GetTopicsAreFirstClassCitizen()) { + SetupTopicCounters(converterIter->second, jt->second.CloudId, jt->second.DbId, jt->second.FolderId); + } else { + SetupTopicCounters(converterIter->second); + } + } + + auto it = TopicCounters.find(intName); + Y_VERIFY(it != TopicCounters.end()); + + IActor* partitionActor = new TPartitionActor( + ctx.SelfID, InternalClientId, Cookie, Session, record.GetGeneration(), + record.GetStep(), jt->second.FullConverter, record.GetPartition(), record.GetTabletId(), it->second, + ClientDC + ); + + TActorId actorId = ctx.Register(partitionActor); + if (SessionsActive) { + PartsPerSession.DecFor(Partitions.size(), 1); + } + Y_VERIFY(record.GetGeneration() > 0); + //Partitions use clientside name ! + auto pp = Partitions.insert({ + std::make_pair(jt->second.FullConverter->GetClientsideName(), record.GetPartition()), + TPartitionActorInfo{actorId, (((ui64)record.GetGeneration()) << 32) + record.GetStep(), jt->second.FullConverter} + }); + Y_VERIFY(pp.second); + if (SessionsActive) { + PartsPerSession.IncFor(Partitions.size(), 1); + } + + bool res = ActualPartitionActors.insert(actorId).second; + Y_VERIFY(res); + + it->second.PartitionsLocked.Inc(); + it->second.PartitionsInfly.Inc(); + + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " lock: " << record); + + ctx.Send(actorId, new TEvPQProxy::TEvLockPartition(0, 0, false, !ClientsideLocksAllowed)); +} + +void TReadSessionActor::Handle(TEvPQProxy::TEvPartitionStatus::TPtr& ev, const TActorContext&) { + if (!ActualPartitionActor(ev->Sender)) + return; + + auto& evTopic = ev->Get()->Topic; + auto it = Partitions.find(std::make_pair(evTopic->GetClientsideName(), ev->Get()->Partition)); + Y_VERIFY(it != Partitions.end()); + Y_VERIFY(it->second.LockGeneration); + + if (it->second.Releasing) //lock request for already released partition - ignore + return; + + if (ev->Get()->Init) { + Y_VERIFY(!it->second.LockSent); + + it->second.LockSent = true; + auto topicIter = Topics.find(evTopic->GetInternalName()); + Y_VERIFY(topicIter != Topics.end()); + Y_VERIFY(ClientsideLocksAllowed); + TReadResponse result; + auto lock = result.MutableLock(); + lock->SetTopic(topicIter->second.FullConverter->GetClientsideName()); + lock->SetPartition(ev->Get()->Partition); + lock->SetReadOffset(ev->Get()->Offset); + lock->SetEndOffset(ev->Get()->EndOffset); + lock->SetGeneration(it->second.LockGeneration); + auto jt = PartitionToReadResponse.find(it->second.Actor); + if (jt == PartitionToReadResponse.end()) { + ui64 diff = result.ByteSize(); + BytesInflight_ += diff; + if (BytesInflight) (*BytesInflight) += diff; + Handler->Reply(result); + } else { + jt->second->ControlMessages.push_back(result); + } + } else { + Y_VERIFY(it->second.LockSent); + TReadResponse result; + auto status = result.MutablePartitionStatus(); + status->SetTopic(ev->Get()->Topic->GetClientsideName()); + status->SetPartition(ev->Get()->Partition); + status->SetEndOffset(ev->Get()->EndOffset); + status->SetGeneration(it->second.LockGeneration); + status->SetCommittedOffset(ev->Get()->Offset); + status->SetWriteWatermarkMs(ev->Get()->WriteTimestampEstimateMs); + auto jt = PartitionToReadResponse.find(it->second.Actor); + if (jt == PartitionToReadResponse.end()) { + ui64 diff = result.ByteSize(); + BytesInflight_ += diff; + if (BytesInflight) (*BytesInflight) += diff; + Handler->Reply(result); + } else { + jt->second->ControlMessages.push_back(result); + } + + } +} + +void TReadSessionActor::Handle(TEvPersQueue::TEvError::TPtr& ev, const TActorContext& ctx) { + CloseSession(ev->Get()->Record.GetDescription(), ev->Get()->Record.GetCode(), ctx); +} + + +void TReadSessionActor::Handle(TEvPersQueue::TEvReleasePartition::TPtr& ev, const TActorContext& ctx) { + auto& record = ev->Get()->Record; + Y_VERIFY(record.GetSession() == Session); + Y_VERIFY(record.GetClientId() == InternalClientId); + auto topic = record.GetPath(); + if (topic.empty()) { + topic = record.GetTopic(); + } + ui32 group = record.HasGroup() ? record.GetGroup() : 0; + + auto converterIter = FullPathToConverter.find(topic); + if (converterIter.IsEnd()) { + LOG_ALERT_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " Failed to parse balancer response: " << record.ShortDebugString()); + CloseSession( + TStringBuilder() << "Internal server error, cannot parse release event: " << record.ShortDebugString() << ", path not recognized", + NPersQueue::NErrorCode::ERROR, ctx + ); + return; + } + auto name = converterIter->second->GetInternalName(); + + auto it = Topics.find(name); + Y_VERIFY(it != Topics.end()); + + TActorId pipe = ActorIdFromProto(record.GetPipeClient()); + + if (pipe != it->second.PipeClient) { //this is message from old version of pipe + return; + } + + for (ui32 c = 0; c < record.GetCount(); ++c) { + Y_VERIFY(!Partitions.empty()); + + TActorId actorId = TActorId{}; + auto jt = Partitions.begin(); + ui32 i = 0; + for (auto it = Partitions.begin(); it != Partitions.end(); ++it) { + if (it->first.first == name && !it->second.Releasing && (group == 0 || it->first.second + 1 == group)) { + ++i; + if (rand() % i == 0) { //will lead to 1/n probability for each of n partitions + actorId = it->second.Actor; + jt = it; + } + } + } + Y_VERIFY(actorId); + + { + auto it = TopicCounters.find(name); + Y_VERIFY(it != TopicCounters.end()); + it->second.PartitionsToBeReleased.Inc(); + } + + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " releasing " << jt->first.first << ":" << jt->first.second); + jt->second.Releasing = true; + + ctx.Send(actorId, new TEvPQProxy::TEvReleasePartition()); + if (ClientsideLocksAllowed && jt->second.LockSent && !jt->second.Reading) { //locked and no active reads + if (!ProcessReleasePartition(jt, BalanceRightNow, false, ctx)) { // returns false if actor died + return; + } + } + } + AnswerForCommitsIfCan(ctx); // in case of killing partition +} + + +void TReadSessionActor::Handle(TEvPQProxy::TEvPartitionReleased::TPtr& ev, const TActorContext& ctx) { + if (!ActualPartitionActor(ev->Sender)) + return; + + const auto& topic = ev->Get()->Topic; + const ui32 partition = ev->Get()->Partition; + + auto jt = Partitions.find(std::make_pair(topic->GetClientsideName(), partition)); + Y_VERIFY(jt != Partitions.end(), "session %s topic %s part %u", Session.c_str(), topic->GetInternalName().c_str(), partition); + Y_VERIFY(jt->second.Releasing); + jt->second.Released = true; + + { + auto it = TopicCounters.find(topic->GetInternalName()); + Y_VERIFY(it != TopicCounters.end()); + it->second.PartitionsReleased.Inc(); + it->second.PartitionsInfly.Dec(); + it->second.PartitionsToBeReleased.Dec(); + + } + + InformBalancerAboutRelease(jt, ctx); + + DropPartitionIfNeeded(jt, ctx); +} + +void TReadSessionActor::InformBalancerAboutRelease(const THashMap<std::pair<TString, ui32>, TPartitionActorInfo>::iterator& it, const TActorContext& ctx) { + + THolder<TEvPersQueue::TEvPartitionReleased> request; + request.Reset(new TEvPersQueue::TEvPartitionReleased); + auto& req = request->Record; + + auto jt = Topics.find(it->second.Converter->GetInternalName()); + Y_VERIFY(jt != Topics.end()); + + req.SetSession(Session); + ActorIdToProto(jt->second.PipeClient, req.MutablePipeClient()); + req.SetClientId(InternalClientId); + req.SetTopic(it->first.first); + req.SetPartition(it->first.second); + + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " released: " << it->first.first << ":" << it->first.second); + + NTabletPipe::SendData(ctx, jt->second.PipeClient, request.Release()); +} + + +void TReadSessionActor::CloseSession(const TString& errorReason, const NPersQueue::NErrorCode::EErrorCode errorCode, const NActors::TActorContext& ctx) { + + if (errorCode != NPersQueue::NErrorCode::OK) { + + if (InternalErrorCode(errorCode)) { + SLIErrors.Inc(); + } + + if (Errors) { + ++(*Errors); + } else { + if (!AppData(ctx)->PQConfig.GetTopicsAreFirstClassCitizen()) { + ++(*GetServiceCounters(Counters, "pqproxy|readSession")->GetCounter("Errors", true)); + } + } + + TReadResponse result; + + auto error = result.MutableError(); + error->SetDescription(errorReason); + error->SetCode(errorCode); + + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " closed with error reason: " << errorReason); + if (!Handler->IsShuttingDown()) { + ui64 diff = result.ByteSize(); + BytesInflight_ += diff; + if (BytesInflight) (*BytesInflight) += diff; + Handler->Reply(result); + } else { + LOG_WARN_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " GRps is shutting dows, skip reply"); + } + } else { + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " closed"); + } + + Die(ctx); +} + + +void TReadSessionActor::Handle(TEvTabletPipe::TEvClientConnected::TPtr& ev, const TActorContext& ctx) { + TEvTabletPipe::TEvClientConnected *msg = ev->Get(); + if (msg->Status != NKikimrProto::OK) { + if (msg->Dead) { + CloseSession(TStringBuilder() << "one of topics is deleted, tablet " << msg->TabletId, NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + //TODO: remove it + CloseSession(TStringBuilder() << "unable to connect to one of topics, tablet " << msg->TabletId, NPersQueue::NErrorCode::ERROR, ctx); + return; + + const bool isAlive = ProcessBalancerDead(msg->TabletId, ctx); // returns false if actor died + Y_UNUSED(isAlive); + return; + } +} + +bool TReadSessionActor::ActualPartitionActor(const TActorId& part) { + return ActualPartitionActors.contains(part); +} + + +bool TReadSessionActor::ProcessReleasePartition(const THashMap<std::pair<TString, ui32>, TPartitionActorInfo>::iterator& it, + bool kill, bool couldBeReads, const TActorContext& ctx) +{ + //inform client + if (ClientsideLocksAllowed && it->second.LockSent) { + TReadResponse result; + result.MutableRelease()->SetTopic(it->first.first); + result.MutableRelease()->SetPartition(it->first.second); + result.MutableRelease()->SetCanCommit(!kill); + result.MutableRelease()->SetGeneration(it->second.LockGeneration); + auto jt = PartitionToReadResponse.find(it->second.Actor); + if (jt == PartitionToReadResponse.end()) { + ui64 diff = result.ByteSize(); + BytesInflight_ += diff; + if (BytesInflight) (*BytesInflight) += diff; + Handler->Reply(result); + } else { + jt->second->ControlMessages.push_back(result); + } + it->second.LockGeneration = 0; + it->second.LockSent = false; + } + + if (!kill) { + return true; + } + + { + auto jt = TopicCounters.find(it->second.Converter->GetInternalName()); + Y_VERIFY(jt != TopicCounters.end()); + jt->second.PartitionsReleased.Inc(); + jt->second.PartitionsInfly.Dec(); + if (!it->second.Released && it->second.Releasing) { + jt->second.PartitionsToBeReleased.Dec(); + } + } + + //process commits + for (auto& c : it->second.Commits) { + auto kt = Commits.find(c); + Y_VERIFY(kt != Commits.end()); + --kt->second.Partitions; + } + it->second.Commits.clear(); + + Y_VERIFY(couldBeReads || !it->second.Reading); + //process reads + TFormedReadResponse::TPtr formedResponseToAnswer; + if (it->second.Reading) { + const auto readIt = PartitionToReadResponse.find(it->second.Actor); + Y_VERIFY(readIt != PartitionToReadResponse.end()); + if (--readIt->second->RequestsInfly == 0) { + formedResponseToAnswer = readIt->second; + } + } + + InformBalancerAboutRelease(it, ctx); + + it->second.Released = true; //to force drop + DropPartitionIfNeeded(it, ctx); //partition will be dropped + + if (formedResponseToAnswer) { + return ProcessAnswer(ctx, formedResponseToAnswer); // returns false if actor died + } + return true; +} + + +bool TReadSessionActor::ProcessBalancerDead(const ui64 tablet, const TActorContext& ctx) { + for (auto& t : Topics) { + if (t.second.TabletID == tablet) { + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " balancer for topic " << t.first << " is dead, restarting all from this topic"); + + //Drop all partitions from this topic + for (auto it = Partitions.begin(); it != Partitions.end();) { + if (it->second.Converter->GetInternalName() == t.first) { //partition from this topic + // kill actor + auto jt = it; + ++it; + if (!ProcessReleasePartition(jt, true, true, ctx)) { // returns false if actor died + return false; + } + } else { + ++it; + } + } + + AnswerForCommitsIfCan(ctx); + + //reconnect pipe + NTabletPipe::TClientConfig clientConfig; + clientConfig.CheckAliveness = false; + clientConfig.RetryPolicy = RetryPolicyForPipes; + t.second.PipeClient = ctx.RegisterWithSameMailbox(NTabletPipe::CreateClient(ctx.SelfID, t.second.TabletID, clientConfig)); + if (InitDone) { + if (PipeReconnects) { + ++(*PipeReconnects); + ++(*Errors); + } + + RegisterSession(t.second.PipeClient, t.first, ctx); + } + } + } + return true; +} + + +void TReadSessionActor::Handle(TEvTabletPipe::TEvClientDestroyed::TPtr& ev, const TActorContext& ctx) { + const bool isAlive = ProcessBalancerDead(ev->Get()->TabletId, ctx); // returns false if actor died + Y_UNUSED(isAlive); +} + +void TReadSessionActor::ProcessAuth(const NPersQueueCommon::TCredentials& auth) { + TString tmp; + Y_PROTOBUF_SUPPRESS_NODISCARD auth.SerializeToString(&tmp); + if (auth.GetCredentialsCase() != NPersQueueCommon::TCredentials::CREDENTIALS_NOT_SET && tmp != AuthStr) { + Auth = auth; + AuthStr = tmp; + ForceACLCheck = true; + } +} + +void TReadSessionActor::Handle(TEvPQProxy::TEvRead::TPtr& ev, const TActorContext& ctx) { + RequestNotChecked = true; + + THolder<TEvPQProxy::TEvRead> event(ev->Release()); + + Handler->ReadyForNextRead(); + + + ProcessAuth(event->Request.GetCredentials()); + event->Request.ClearCredentials(); + + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " got read request: " << event->Request.GetRead() << " with guid: " << event->Guid); + + Reads.emplace_back(event.Release()); + + const bool isAlive = ProcessReads(ctx); // returns false if actor died + Y_UNUSED(isAlive); +} + + +i64 TReadSessionActor::TFormedReadResponse::ApplyResponse(NPersQueue::TReadResponse&& resp) { + Y_VERIFY(resp.GetBatchedData().PartitionDataSize() == 1); + Response.MutableBatchedData()->AddPartitionData()->Swap(resp.MutableBatchedData()->MutablePartitionData(0)); + i64 prev = Response.ByteSize(); + std::swap<i64>(prev, ByteSize); + return ByteSize - prev; +} + + +void TReadSessionActor::Handle(TEvPQProxy::TEvReadResponse::TPtr& ev, const TActorContext& ctx) { + TActorId sender = ev->Sender; + if (!ActualPartitionActor(sender)) + return; + + THolder<TEvPQProxy::TEvReadResponse> event(ev->Release()); + + Y_VERIFY(event->Response.GetBatchedData().GetCookie() == 0); // cookie is not assigned + Y_VERIFY(event->Response.GetBatchedData().PartitionDataSize() == 1); + + const TString topic = event->Response.GetBatchedData().GetPartitionData(0).GetTopic(); + const ui32 partition = event->Response.GetBatchedData().GetPartitionData(0).GetPartition(); + std::pair<TString, ui32> key(topic, partition); + // Topic is expected to have clientSide name + const auto partitionIt = Partitions.find(key); + Y_VERIFY(partitionIt != Partitions.end()); + Y_VERIFY(partitionIt->second.Reading); + partitionIt->second.Reading = false; + + auto it = PartitionToReadResponse.find(sender); + Y_VERIFY(it != PartitionToReadResponse.end()); + + TFormedReadResponse::TPtr formedResponse = it->second; + + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " read done guid " << formedResponse->Guid + << " " << key.first << ":" << key.second + << " size " << event->Response.ByteSize()); + + const i64 diff = formedResponse->ApplyResponse(std::move(event->Response)); + if (event->FromDisk) { + formedResponse->FromDisk = true; + } + formedResponse->WaitQuotaTime = Max(formedResponse->WaitQuotaTime, event->WaitQuotaTime); + --formedResponse->RequestsInfly; + formedResponse->Offsets.PartitionOffsets.emplace_back(sender, topic, partition, event->NextReadOffset); + + BytesInflight_ += diff; + if (BytesInflight) (*BytesInflight) += diff; + + if (ClientsideLocksAllowed && partitionIt->second.LockSent && partitionIt->second.Releasing) { //locked and need to be released + if (!ProcessReleasePartition(partitionIt, BalanceRightNow, false, ctx)) { // returns false if actor died + return; + } + } + AnswerForCommitsIfCan(ctx); // in case of killing partition + + if (formedResponse->RequestsInfly == 0) { + const bool isAlive = ProcessAnswer(ctx, formedResponse); // returns false if actor died + Y_UNUSED(isAlive); + } +} + + +bool TReadSessionActor::ProcessAnswer(const TActorContext& ctx, TFormedReadResponse::TPtr formedResponse) { + ui32 readDurationMs = (ctx.Now() - formedResponse->Start - formedResponse->WaitQuotaTime).MilliSeconds(); + if (formedResponse->FromDisk) { + ReadLatencyFromDisk.IncFor(readDurationMs, 1); + } else { + ReadLatency.IncFor(readDurationMs, 1); + } + if (readDurationMs >= (formedResponse->FromDisk ? AppData(ctx)->PQConfig.GetReadLatencyFromDiskBigMs() : AppData(ctx)->PQConfig.GetReadLatencyBigMs())) { + SLIBigReadLatency.Inc(); + } + + Y_VERIFY(formedResponse->RequestsInfly == 0); + i64 diff = formedResponse->Response.ByteSize(); + const bool hasMessages = RemoveEmptyMessages(*formedResponse->Response.MutableBatchedData()); + if (hasMessages) { + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " assign read id " << ReadIdToResponse << " to read request " << formedResponse->Guid); + formedResponse->Response.MutableBatchedData()->SetCookie(ReadIdToResponse); + // reply to client + if (ProtocolVersion < NPersQueue::TReadRequest::Batching) { + ConvertToOldBatch(formedResponse->Response); + } + diff -= formedResponse->Response.ByteSize(); // Bytes will be tracked inside handler + Handler->Reply(formedResponse->Response); + } else { + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " empty read result " << formedResponse->Guid << ", start new reading"); + } + + BytesInflight_ -= diff; + if (BytesInflight) (*BytesInflight) -= diff; + + for (auto& r : formedResponse->ControlMessages) { + ui64 diff = r.ByteSize(); + BytesInflight_ += diff; + if (BytesInflight) (*BytesInflight) += diff; + Handler->Reply(r); + } + + for (const TActorId& p : formedResponse->PartitionsTookPartInRead) { + PartitionToReadResponse.erase(p); + } + + // Bring back available partitions. + // If some partition was removed from partitions container, it is not bad because it will be checked during read processing. + AvailablePartitions.insert(formedResponse->PartitionsBecameAvailable.begin(), formedResponse->PartitionsBecameAvailable.end()); + + formedResponse->Offsets.ReadId = ReadIdToResponse; + + RequestedBytes -= formedResponse->RequestedBytes; + + ReadsInfly--; + + if (hasMessages) { + if (!CommitsDisabled) + Offsets.emplace_back(std::move(formedResponse->Offsets)); // even empty responses are needed for correct offsets commit. + ReadIdToResponse++; + } else { + // process new read + NPersQueue::TReadRequest req; + req.MutableRead(); + Reads.emplace_back(new TEvPQProxy::TEvRead(req, formedResponse->Guid)); // Start new reading request with the same guid + } + + return ProcessReads(ctx); // returns false if actor died +} + + +void TReadSessionActor::Handle(TEvPQProxy::TEvCloseSession::TPtr& ev, const TActorContext& ctx) { + CloseSession(ev->Get()->Reason, ev->Get()->ErrorCode, ctx); +} + +void TReadSessionActor::Handle(V1::TEvPQProxy::TEvCloseSession::TPtr& ev, const TActorContext& ctx) { + CloseSession(ev->Get()->Reason, NErrorCode::EErrorCode(ev->Get()->ErrorCode - 500000), ctx); +} + +ui32 TReadSessionActor::NormalizeMaxReadMessagesCount(ui32 sourceValue) { + ui32 count = Min<ui32>(sourceValue, Max<i32>()); + if (count == 0) { + count = Max<i32>(); + } + return count; +} + +ui32 TReadSessionActor::NormalizeMaxReadSize(ui32 sourceValue) { + ui32 size = Min<ui32>(sourceValue, MAX_READ_SIZE); + if (size == 0) { + size = MAX_READ_SIZE; + } + return size; +} + +ui32 TReadSessionActor::NormalizeMaxReadPartitionsCount(ui32 sourceValue) { + ui32 maxPartitions = sourceValue; + if (maxPartitions == 0) { + maxPartitions = Max<ui32>(); + } + return maxPartitions; +} + +bool TReadSessionActor::CheckAndUpdateReadSettings(const NPersQueue::TReadRequest::TRead& readRequest) { + if (ReadSettingsInited) { // already updated. Check that settings are not changed. + const bool hasSettings = readRequest.GetMaxCount() + || readRequest.GetMaxSize() + || readRequest.GetPartitionsAtOnce() + || readRequest.GetMaxTimeLagMs() + || readRequest.GetReadTimestampMs(); + if (!hasSettings) { + return true; + } + + const bool settingsChanged = NormalizeMaxReadMessagesCount(readRequest.GetMaxCount()) != MaxReadMessagesCount + || NormalizeMaxReadSize(readRequest.GetMaxSize()) != MaxReadSize + || NormalizeMaxReadPartitionsCount(readRequest.GetPartitionsAtOnce()) != MaxReadPartitionsCount + || readRequest.GetMaxTimeLagMs() != MaxTimeLagMs + || readRequest.GetReadTimestampMs() != ReadTimestampMs; + return !settingsChanged; + } else { + // Update settings for the first time + ReadSettingsInited = true; + MaxReadMessagesCount = NormalizeMaxReadMessagesCount(readRequest.GetMaxCount()); + MaxReadSize = NormalizeMaxReadSize(readRequest.GetMaxSize()); + MaxReadPartitionsCount = NormalizeMaxReadPartitionsCount(readRequest.GetPartitionsAtOnce()); + MaxTimeLagMs = readRequest.GetMaxTimeLagMs(); + ReadTimestampMs = readRequest.GetReadTimestampMs(); + return true; + } +} + +bool TReadSessionActor::ProcessReads(const TActorContext& ctx) { + while (!Reads.empty() && BytesInflight_ + RequestedBytes < MAX_INFLY_BYTES && ReadsInfly < MAX_INFLY_READS) { + const auto& readRequest = Reads.front()->Request.GetRead(); + if (!CheckAndUpdateReadSettings(readRequest)) { + CloseSession("read settings were changed in read request", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return false; + } + + if (Offsets.size() >= AppData(ctx)->PQConfig.GetMaxReadCookies() + 10) { + CloseSession(TStringBuilder() << "got more than " << AppData(ctx)->PQConfig.GetMaxReadCookies() << " uncommitted reads", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return false; + } + + ui32 count = MaxReadMessagesCount; + ui64 size = MaxReadSize; + ui32 maxPartitions = MaxReadPartitionsCount; + ui32 partitionsAsked = 0; + + TFormedReadResponse::TPtr formedResponse = new TFormedReadResponse(Reads.front()->Guid, ctx.Now()); + while (!AvailablePartitions.empty()) { + auto part = *AvailablePartitions.begin(); + AvailablePartitions.erase(AvailablePartitions.begin()); + + auto it = Partitions.find(std::make_pair(part.Topic->GetClientsideName(), part.Partition)); + if (it == Partitions.end() || it->second.Releasing || it->second.Actor != part.Actor) { //this is already released partition + continue; + } + //add this partition to reading + ++partitionsAsked; + + TAutoPtr<TEvPQProxy::TEvRead> read = new TEvPQProxy::TEvRead(Reads.front()->Request, Reads.front()->Guid); + const ui32 ccount = Min<ui32>(part.MsgLag * LAG_GROW_MULTIPLIER, count); + count -= ccount; + const ui64 csize = (ui64)Min<double>(part.SizeLag * LAG_GROW_MULTIPLIER, size); + size -= csize; + + Y_VERIFY(csize < Max<i32>()); + auto* readR = read->Request.MutableRead(); + readR->SetMaxCount(ccount); + readR->SetMaxSize(csize); + readR->SetMaxTimeLagMs(MaxTimeLagMs); + readR->SetReadTimestampMs(ReadTimestampMs); + + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX + << " performing read request: " << (*readR) << " with guid " << read->Guid + << " from " << part.Topic->GetPrintableString() << ", partition:" << part.Partition + << " count " << ccount << " size " << csize + << " partitionsAsked " << partitionsAsked << " maxTimeLag " << MaxTimeLagMs << "ms"); + + + Y_VERIFY(!it->second.Reading); + it->second.Reading = true; + formedResponse->PartitionsTookPartInRead.insert(it->second.Actor); + + RequestedBytes += csize; + formedResponse->RequestedBytes += csize; + + ctx.Send(it->second.Actor, read.Release()); + const auto insertResult = PartitionToReadResponse.insert(std::make_pair(it->second.Actor, formedResponse)); + Y_VERIFY(insertResult.second); + + if (--maxPartitions == 0 || count == 0 || size == 0) + break; + } + if (partitionsAsked == 0) + break; + ReadsTotal.Inc(); + formedResponse->RequestsInfly = partitionsAsked; + + ReadsInfly++; + + i64 diff = formedResponse->Response.ByteSize(); + BytesInflight_ += diff; + formedResponse->ByteSize = diff; + if (BytesInflight) (*BytesInflight) += diff; + Reads.pop_front(); + } + return true; +} + + +void TReadSessionActor::Handle(TEvPQProxy::TEvPartitionReady::TPtr& ev, const TActorContext& ctx) { + + if (!ActualPartitionActor(ev->Sender)) + return; + + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " " << ev->Get()->Topic->GetPrintableString() + << " partition:" << ev->Get()->Partition << " ready for read with readOffset " + << ev->Get()->ReadOffset << " endOffset " << ev->Get()->EndOffset << " WTime " + << ev->Get()->WTime << " sizeLag " << ev->Get()->SizeLag); + + const auto it = PartitionToReadResponse.find(ev->Sender); // check whether this partition is taking part in read response + auto& container = it != PartitionToReadResponse.end() ? it->second->PartitionsBecameAvailable : AvailablePartitions; + auto res = container.insert({ev->Get()->Topic, ev->Get()->Partition, ev->Get()->WTime, ev->Get()->SizeLag, + ev->Get()->EndOffset - ev->Get()->ReadOffset, ev->Sender}); + Y_VERIFY(res.second); + const bool isAlive = ProcessReads(ctx); // returns false if actor died + Y_UNUSED(isAlive); +} + + +void TReadSessionActor::HandlePoison(TEvPQProxy::TEvDieCommand::TPtr& ev, const TActorContext& ctx) { + CloseSession(ev->Get()->Reason, ev->Get()->ErrorCode, ctx); +} + + +void TReadSessionActor::HandleWakeup(const TActorContext& ctx) { + ctx.Schedule(Min(CommitInterval, CHECK_ACL_DELAY), new TEvents::TEvWakeup()); + MakeCommit(ctx); + if (!AuthInflight && (ForceACLCheck || (ctx.Now() - LastACLCheckTimestamp > TDuration::Seconds(AppData(ctx)->PQConfig.GetACLRetryTimeoutSec()) && RequestNotChecked))) { + ForceACLCheck = false; + RequestNotChecked = false; + Y_VERIFY(!AuthInitActor); + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " checking auth because of timeout"); + + SendAuthRequest(ctx); + } +} + +bool TReadSessionActor::RemoveEmptyMessages(TReadResponse::TBatchedData& data) { + bool hasNonEmptyMessages = false; + auto isMessageEmpty = [&](TReadResponse::TBatchedData::TMessageData& message) -> bool { + if (message.GetData().empty()) { + return true; + } else { + hasNonEmptyMessages = true; + return false; + } + }; + auto batchRemover = [&](TReadResponse::TBatchedData::TBatch& batch) -> bool { + NProtoBuf::RemoveRepeatedFieldItemIf(batch.MutableMessageData(), isMessageEmpty); + return batch.MessageDataSize() == 0; + }; + auto partitionDataRemover = [&](TReadResponse::TBatchedData::TPartitionData& partition) -> bool { + NProtoBuf::RemoveRepeatedFieldItemIf(partition.MutableBatch(), batchRemover); + return partition.BatchSize() == 0; + }; + NProtoBuf::RemoveRepeatedFieldItemIf(data.MutablePartitionData(), partitionDataRemover); + return hasNonEmptyMessages; +} + + +////////////////// PARTITION ACTOR + +TPartitionActor::TPartitionActor( + const TActorId& parentId, const TString& internalClientId, const ui64 cookie, const TString& session, + const ui32 generation, const ui32 step, const NPersQueue::TTopicConverterPtr& topic, const ui32 partition, + const ui64 tabletID, const TReadSessionActor::TTopicCounters& counters, const TString& clientDC +) + : ParentId(parentId) + , InternalClientId(internalClientId) + , ClientDC(clientDC) + , Cookie(cookie) + , Session(session) + , Generation(generation) + , Step(step) + , Topic(topic) + , Partition(partition) + , TabletID(tabletID) + , ReadOffset(0) + , ClientReadOffset(0) + , ClientCommitOffset(0) + , ClientVerifyReadOffset(false) + , CommittedOffset(0) + , WriteTimestampEstimateMs(0) + , WTime(0) + , InitDone(false) + , StartReading(false) + , AllPrepareInited(false) + , FirstInit(true) + , PipeClient() + , PipeGeneration(0) + , RequestInfly(false) + , EndOffset(0) + , SizeLag(0) + , NeedRelease(false) + , Released(false) + , WaitDataCookie(0) + , WaitForData(false) + , LockCounted(false) + , Counters(counters) +{ +} + + +TPartitionActor::~TPartitionActor() = default; + + +void TPartitionActor::Bootstrap(const TActorContext&) { + Become(&TThis::StateFunc); +} + + +void TPartitionActor::CheckRelease(const TActorContext& ctx) { + const bool hasUncommittedData = ReadOffset > ClientCommitOffset && ReadOffset > ClientReadOffset; + if (NeedRelease) { + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " " << Topic->GetPrintableString() << " partition:" << Partition + << " checking release readOffset " << ReadOffset << " committedOffset " << CommittedOffset << " ReadGuid " << ReadGuid + << " CommitsInfly.size " << CommitsInfly.size() << " Released " << Released); + } + + if (NeedRelease && ReadGuid.empty() && CommitsInfly.empty() && !hasUncommittedData && !Released) { + Released = true; + ctx.Send(ParentId, new TEvPQProxy::TEvPartitionReleased(Topic, Partition)); + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " " << Topic->GetPrintableString() << " partition:" << Partition + << " check release done - releasing; readOffset " << ReadOffset << " committedOffset " << CommittedOffset << " ReadGuid " << ReadGuid + << " CommitsInfly.size " << CommitsInfly.size() << " Released " << Released); + + } +} + + +void TPartitionActor::SendCommit(const ui64 readId, const ui64 offset, const TActorContext& ctx) { + NKikimrClient::TPersQueueRequest request; + request.MutablePartitionRequest()->SetTopic(Topic->GetClientsideName()); + request.MutablePartitionRequest()->SetPartition(Partition); + request.MutablePartitionRequest()->SetCookie(readId); + + Y_VERIFY(PipeClient); + + ActorIdToProto(PipeClient, request.MutablePartitionRequest()->MutablePipeClient()); + auto commit = request.MutablePartitionRequest()->MutableCmdSetClientOffset(); + commit->SetClientId(InternalClientId); + commit->SetOffset(offset); + Y_VERIFY(!Session.empty()); + commit->SetSessionId(Session); + + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " " << Topic->GetPrintableString() << " partition:" + << Partition << " committing to position " << offset << " prev " << CommittedOffset + << " end " << EndOffset << " by cookie " << readId); + + TAutoPtr<TEvPersQueue::TEvRequest> req(new TEvPersQueue::TEvRequest); + req->Record.Swap(&request); + + NTabletPipe::SendData(ctx, PipeClient, req.Release()); +} + +void TPartitionActor::RestartPipe(const TActorContext& ctx, const TString& reason, const NPersQueue::NErrorCode::EErrorCode errorCode) { + + if (!PipeClient) + return; + + Counters.Errors.Inc(); + + NTabletPipe::CloseClient(ctx, PipeClient); + PipeClient = TActorId{}; + if (errorCode != NPersQueue::NErrorCode::OVERLOAD) + ++PipeGeneration; + + if (PipeGeneration == MAX_PIPE_RESTARTS) { + ctx.Send(ParentId, new TEvPQProxy::TEvCloseSession("too much attempts to restart pipe", NPersQueue::NErrorCode::ERROR)); + return; + } + + ctx.Schedule(TDuration::MilliSeconds(RESTART_PIPE_DELAY_MS), new TEvPQProxy::TEvRestartPipe()); + + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " " << Topic->GetPrintableString() << " partition:" << Partition + << " schedule pipe restart attempt " << PipeGeneration << " reason: " << reason); +} + + +void TPartitionActor::Handle(const TEvPQProxy::TEvRestartPipe::TPtr&, const TActorContext& ctx) { + + Y_VERIFY(!PipeClient); + + NTabletPipe::TClientConfig clientConfig; + clientConfig.RetryPolicy = { + .RetryLimitCount = 6, + .MinRetryTime = TDuration::MilliSeconds(10), + .MaxRetryTime = TDuration::MilliSeconds(100), + .BackoffMultiplier = 2, + .DoFirstRetryInstantly = true + }; + PipeClient = ctx.RegisterWithSameMailbox(NTabletPipe::CreateClient(ctx.SelfID, TabletID, clientConfig)); + Y_VERIFY(TabletID); + + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " " << Topic->GetPrintableString() << " partition:" << Partition + << " pipe restart attempt " << PipeGeneration << " RequestInfly " << RequestInfly << " ReadOffset " << ReadOffset << " EndOffset " << EndOffset + << " InitDone " << InitDone << " WaitForData " << WaitForData); + + if (RequestInfly) { //got read infly + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " " << Topic->GetPrintableString() << " partition:" << Partition + << " resend " << CurrentRequest); + + TAutoPtr<TEvPersQueue::TEvRequest> event(new TEvPersQueue::TEvRequest); + event->Record = CurrentRequest; + + ActorIdToProto(PipeClient, event->Record.MutablePartitionRequest()->MutablePipeClient()); + + NTabletPipe::SendData(ctx, PipeClient, event.Release()); + } + if (InitDone) { + for (auto& c : CommitsInfly) { //resend all commits + if (c.second != Max<ui64>()) + SendCommit(c.first, c.second, ctx); + } + if (WaitForData) { //resend wait-for-data requests + WaitDataInfly.clear(); + WaitDataInPartition(ctx); + } + } +} + +void TPartitionActor::Handle(TEvPersQueue::TEvResponse::TPtr& ev, const TActorContext& ctx) { + + if (ev->Get()->Record.HasErrorCode() && ev->Get()->Record.GetErrorCode() != NPersQueue::NErrorCode::OK) { + const auto errorCode = ev->Get()->Record.GetErrorCode(); + if (errorCode == NPersQueue::NErrorCode::WRONG_COOKIE || errorCode == NPersQueue::NErrorCode::BAD_REQUEST) { + Counters.Errors.Inc(); + ctx.Send(ParentId, new TEvPQProxy::TEvCloseSession("status is not ok: " + ev->Get()->Record.GetErrorReason(), ev->Get()->Record.GetErrorCode())); + } else { + RestartPipe(ctx, TStringBuilder() << "status is not ok. Code: " << EErrorCode_Name(errorCode) << ". Reason: " << ev->Get()->Record.GetErrorReason(), errorCode); + } + return; + } + + if (ev->Get()->Record.GetStatus() != NMsgBusProxy::MSTATUS_OK) { //this is incorrect answer, die + Y_VERIFY(!ev->Get()->Record.HasErrorCode()); + Counters.Errors.Inc(); + ctx.Send(ParentId, new TEvPQProxy::TEvCloseSession("status is not ok: " + ev->Get()->Record.GetErrorReason(), NPersQueue::NErrorCode::ERROR)); + return; + } + if (!ev->Get()->Record.HasPartitionResponse()) { //this is incorrect answer, die + Counters.Errors.Inc(); + ctx.Send(ParentId, new TEvPQProxy::TEvCloseSession("empty partition in response", NPersQueue::NErrorCode::ERROR)); + return; + } + + const auto& result = ev->Get()->Record.GetPartitionResponse(); + + if (!result.HasCookie()) { //this is incorrect answer, die + Counters.Errors.Inc(); + ctx.Send(ParentId, new TEvPQProxy::TEvCloseSession("no cookie in response", NPersQueue::NErrorCode::ERROR)); + return; + } + + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " " << Topic->GetPrintableString() + << " partition:" << Partition + << " initDone " << InitDone << " event " << PartitionResponseToLog(result)); + + + if (!InitDone) { + if (result.GetCookie() != INIT_COOKIE) { + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " " << Topic->GetPrintableString() + << " partition:" << Partition + << " unwaited response in init with cookie " << result.GetCookie()); + return; + } + Y_VERIFY(RequestInfly); + CurrentRequest.Clear(); + RequestInfly = false; + + Y_VERIFY(result.HasCmdGetClientOffsetResult()); + const auto& resp = result.GetCmdGetClientOffsetResult(); + Y_VERIFY(resp.HasEndOffset()); + EndOffset = resp.GetEndOffset(); + SizeLag = resp.GetSizeLag(); + + ClientCommitOffset = ReadOffset = CommittedOffset = resp.HasOffset() ? resp.GetOffset() : 0; + Y_VERIFY(EndOffset >= CommittedOffset); + + if (resp.HasWriteTimestampMS()) + WTime = resp.GetWriteTimestampMS(); + WriteTimestampEstimateMs = resp.GetWriteTimestampEstimateMS(); + InitDone = true; + PipeGeneration = 0; //reset tries counter - all ok + LOG_INFO_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " INIT DONE " << Topic->GetPrintableString() + << " partition:" << Partition + << " EndOffset " << EndOffset << " readOffset " << ReadOffset << " committedOffset " << CommittedOffset); + + + if (!StartReading) { + ctx.Send(ParentId, new TEvPQProxy::TEvPartitionStatus(Topic, Partition, CommittedOffset, EndOffset, WriteTimestampEstimateMs, true)); + } else { + InitStartReading(ctx); + } + return; + } + + if (!result.HasCmdReadResult()) { //this is commit response + if (CommitsInfly.empty()) { + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " " << Topic->GetPrintableString() + << " partition:" << Partition + << " unwaited commit-response with cookie " << result.GetCookie() << "; waiting for nothing"); + return; + } + ui64 readId = CommitsInfly.front().first; + + if (result.GetCookie() != readId) { + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " " << Topic->GetPrintableString() + << " partition:" << Partition + << " unwaited commit-response with cookie " << result.GetCookie() << "; waiting for " << readId); + return; + } + + Counters.Commits.Inc(); + + CommittedOffset = CommitsInfly.front().second; + CommitsInfly.pop_front(); + if (readId != Max<ui64>()) //this readId is reserved for upcommits on client skipping with ClientCommitOffset + ctx.Send(ParentId, new TEvPQProxy::TEvCommitDone(readId, Topic, Partition)); + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " " << Topic->GetPrintableString() + << " partition:" << Partition + << " commit done to position " << CommittedOffset << " endOffset " << EndOffset << " with cookie " << readId); + + while (!CommitsInfly.empty() && CommitsInfly.front().second == Max<ui64>()) { //this is cookies that have no effect on this partition + readId = CommitsInfly.front().first; + CommitsInfly.pop_front(); + ctx.Send(ParentId, new TEvPQProxy::TEvCommitDone(readId, Topic, Partition)); + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " " << Topic->GetPrintableString() + << "partition :" << Partition + << " commit done with no effect with cookie " << readId); + } + + CheckRelease(ctx); + PipeGeneration = 0; //reset tries counter - all ok + return; + } + + //This is read + + Y_VERIFY(result.HasCmdReadResult()); + const auto& res = result.GetCmdReadResult(); + + if (result.GetCookie() != (ui64)ReadOffset) { + LOG_DEBUG_S(ctx, NKikimrServices::PQ_READ_PROXY, PQ_LOG_PREFIX << " " << Topic->GetPrintableString() + << "partition :" << Partition + << " unwaited read-response with cookie " << result.GetCookie() << "; waiting for " << ReadOffset << "; current read guid is " << ReadGuid); + return; + } + + Y_VERIFY(res.HasMaxOffset()); + EndOffset = res.GetMaxOffset(); + SizeLag = res.GetSizeLag(); + + const ui64 realReadOffset = res.HasRealReadOffset() ? res.GetRealReadOffset() : 0; + + TReadResponse response; + + auto* data = response.MutableBatchedData(); + auto* partitionData = data->AddPartitionData(); + partitionData->SetTopic(Topic->GetClientsideName()); + partitionData->SetPartition(Partition); + + bool hasOffset = false; + + TReadResponse::TBatchedData::TBatch* currentBatch = nullptr; + for (ui32 i = 0; i < res.ResultSize(); ++i) { + const auto& r = res.GetResult(i); + + WTime = r.GetWriteTimestampMS(); + WriteTimestampEstimateMs = Max(WriteTimestampEstimateMs, WTime); + Y_VERIFY(r.GetOffset() >= ReadOffset); + ReadOffset = r.GetOffset() + 1; + hasOffset = true; + + auto proto(GetDeserializedData(r.GetData())); + if (proto.GetChunkType() != NKikimrPQClient::TDataChunk::REGULAR) { + continue; //TODO - no such chunks must be on prod + } + + Y_VERIFY(!r.GetSourceId().empty()); + if (!NPQ::NSourceIdEncoding::IsValidEncoded(r.GetSourceId())) { + LOG_ERROR_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << "read bad sourceId from topic " << Topic->GetPrintableString() + << " partition:" << Partition + << " offset " << r.GetOffset() << " seqNo " << r.GetSeqNo() << " sourceId '" << r.GetSourceId() << "' ReadGuid " << ReadGuid); + } + TString sourceId = NPQ::NSourceIdEncoding::Decode(r.GetSourceId()); + + if (!currentBatch || currentBatch->GetWriteTimeMs() != r.GetWriteTimestampMS() || currentBatch->GetSourceId() != sourceId) { + // If write time and source id are the same, the rest fields will be the same too. + currentBatch = partitionData->AddBatch(); + currentBatch->SetWriteTimeMs(r.GetWriteTimestampMS()); + currentBatch->SetSourceId(sourceId); + + if (proto.HasMeta()) { + const auto& header = proto.GetMeta(); + if (header.HasServer()) { + auto* item = currentBatch->MutableExtraFields()->AddItems(); + item->SetKey("server"); + item->SetValue(header.GetServer()); + } + if (header.HasFile()) { + auto* item = currentBatch->MutableExtraFields()->AddItems(); + item->SetKey("file"); + item->SetValue(header.GetFile()); + } + if (header.HasIdent()) { + auto* item = currentBatch->MutableExtraFields()->AddItems(); + item->SetKey("ident"); + item->SetValue(header.GetIdent()); + } + if (header.HasLogType()) { + auto* item = currentBatch->MutableExtraFields()->AddItems(); + item->SetKey("logtype"); + item->SetValue(header.GetLogType()); + } + } + + if (proto.HasExtraFields()) { + const auto& map = proto.GetExtraFields(); + for (const auto& kv : map.GetItems()) { + auto* item = currentBatch->MutableExtraFields()->AddItems(); + item->SetKey(kv.GetKey()); + item->SetValue(kv.GetValue()); + } + } + + if (proto.HasIp() && IsUtf(proto.GetIp())) { + currentBatch->SetIp(proto.GetIp()); + } + } + + auto* message = currentBatch->AddMessageData(); + message->SetSeqNo(r.GetSeqNo()); + message->SetCreateTimeMs(r.GetCreateTimestampMS()); + message->SetOffset(r.GetOffset()); + message->SetUncompressedSize(r.GetUncompressedSize()); + if (proto.HasCodec()) { + const auto codec = proto.GetCodec(); + if (codec < Min<int>() || codec > Max<int>() || !NPersQueueCommon::ECodec_IsValid(codec)) { + LOG_ERROR_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << "data chunk (topic " << Topic->GetInternalName() << ", partition " << Partition + << ", offset " << r.GetOffset() << ", seqNo " << r.GetSeqNo() << ", sourceId " + << r.GetSourceId() << ") codec (id " << codec + << ") is not valid NPersQueueCommon::ECodec, loss of data compression codec information" + ); + } + message->SetCodec((NPersQueueCommon::ECodec)proto.GetCodec()); + } + message->SetData(proto.GetData()); + } + + if (!hasOffset) { //no data could be read from paritition at offset ReadOffset - no data in partition at all??? + ReadOffset = Min(Max(ReadOffset + 1, realReadOffset + 1), EndOffset); + } + + CurrentRequest.Clear(); + RequestInfly = false; + + Y_VERIFY(!WaitForData); + + if (EndOffset > ReadOffset) { + ctx.Send(ParentId, new TEvPQProxy::TEvPartitionReady(Topic, Partition, WTime, SizeLag, ReadOffset, EndOffset)); + } else { + WaitForData = true; + if (PipeClient) //pipe will be recreated soon + WaitDataInPartition(ctx); + } + + LOG_DEBUG_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << " after read state " << Topic->GetPrintableString() + << " partition:" << Partition + << " EndOffset " << EndOffset << " ReadOffset " << ReadOffset << " ReadGuid " << ReadGuid); + + ReadGuid = TString(); + + auto readResponse = MakeHolder<TEvPQProxy::TEvReadResponse>( + std::move(response), + ReadOffset, + res.GetBlobsFromDisk() > 0, + TDuration::MilliSeconds(res.GetWaitQuotaTimeMs()) + ); + ctx.Send(ParentId, readResponse.Release()); + CheckRelease(ctx); + + PipeGeneration = 0; //reset tries counter - all ok +} + +void TPartitionActor::Handle(TEvTabletPipe::TEvClientConnected::TPtr& ev, const TActorContext& ctx) { + TEvTabletPipe::TEvClientConnected *msg = ev->Get(); + + LOG_INFO_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << " " << Topic->GetPrintableString() << " partition:" << Partition + << " pipe restart attempt " << PipeGeneration << " pipe creation result: " << msg->Status); + + if (msg->Status != NKikimrProto::OK) { + RestartPipe(ctx, TStringBuilder() << "pipe to tablet is dead " << msg->TabletId, NPersQueue::NErrorCode::ERROR); + return; + } +} + +void TPartitionActor::Handle(TEvTabletPipe::TEvClientDestroyed::TPtr& ev, const TActorContext& ctx) { + RestartPipe(ctx, TStringBuilder() << "pipe to tablet is dead " << ev->Get()->TabletId, NPersQueue::NErrorCode::ERROR); +} + + +void TPartitionActor::Handle(TEvPQProxy::TEvReleasePartition::TPtr&, const TActorContext& ctx) { + LOG_INFO_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << " (partition)releasing " << Topic->GetPrintableString() << " partition:" << Partition + << " ReadOffset " << ReadOffset << " ClientCommitOffset " << ClientCommitOffset + << " CommittedOffst " << CommittedOffset + ); + NeedRelease = true; + CheckRelease(ctx); +} + + +void TPartitionActor::Handle(TEvPQProxy::TEvGetStatus::TPtr&, const TActorContext& ctx) { + ctx.Send(ParentId, new TEvPQProxy::TEvPartitionStatus(Topic, Partition, CommittedOffset, EndOffset, WriteTimestampEstimateMs, false)); +} + + +void TPartitionActor::Handle(TEvPQProxy::TEvLockPartition::TPtr& ev, const TActorContext& ctx) { + ClientReadOffset = ev->Get()->ReadOffset; + ClientCommitOffset = ev->Get()->CommitOffset; + ClientVerifyReadOffset = ev->Get()->VerifyReadOffset; + + if (StartReading) { + Y_VERIFY(ev->Get()->StartReading); //otherwise it is signal from actor, this could not be done + ctx.Send(ParentId, new TEvPQProxy::TEvCloseSession("double partition locking", NPersQueue::NErrorCode::BAD_REQUEST)); + return; + } + + StartReading = ev->Get()->StartReading; + InitLockPartition(ctx); +} + +void TPartitionActor::InitStartReading(const TActorContext& ctx) { + + Y_VERIFY(AllPrepareInited); + Y_VERIFY(!WaitForData); + LOG_INFO_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << " Start reading " << Topic->GetPrintableString() << " partition:" << Partition + << " EndOffset " << EndOffset << " readOffset " << ReadOffset << " committedOffset " + << CommittedOffset << " clientCommittedOffset " << ClientCommitOffset + << " clientReadOffset " << ClientReadOffset + ); + + Counters.PartitionsToBeLocked.Dec(); + LockCounted = false; + + ReadOffset = Max<ui64>(CommittedOffset, ClientReadOffset); + + if (ClientVerifyReadOffset) { + if (ClientReadOffset < CommittedOffset) { + ctx.Send(ParentId, new TEvPQProxy::TEvCloseSession(TStringBuilder() + << "trying to read from position that is less than committed: read " << ClientReadOffset << " committed " << CommittedOffset, + NPersQueue::NErrorCode::BAD_REQUEST)); + return; + } + } + + if (ClientCommitOffset > CommittedOffset) { + if (ClientCommitOffset > ReadOffset) { + ctx.Send(ParentId, new TEvPQProxy::TEvCloseSession(TStringBuilder() + << "trying to read from position that is less than provided to commit: read " << ReadOffset << " commit " << ClientCommitOffset, + NPersQueue::NErrorCode::BAD_REQUEST)); + return; + } + if (ClientCommitOffset > EndOffset) { + ctx.Send(ParentId, new TEvPQProxy::TEvCloseSession(TStringBuilder() + << "trying to commit to future: commit " << ClientCommitOffset << " endOffset " << EndOffset, + NPersQueue::NErrorCode::BAD_REQUEST)); + return; + } + Y_VERIFY(CommitsInfly.empty()); + CommitsInfly.push_back(std::pair<ui64, ui64>(Max<ui64>(), ClientCommitOffset)); + if (PipeClient) //pipe will be recreated soon + SendCommit(CommitsInfly.back().first, CommitsInfly.back().second, ctx); + } else { + ClientCommitOffset = CommittedOffset; + } + + if (EndOffset > ReadOffset) { + ctx.Send(ParentId, new TEvPQProxy::TEvPartitionReady(Topic, Partition, WTime, SizeLag, ReadOffset, EndOffset)); + } else { + WaitForData = true; + if (PipeClient) //pipe will be recreated soon + WaitDataInPartition(ctx); + } +} + +void TPartitionActor::InitLockPartition(const TActorContext& ctx) { + if (PipeClient && AllPrepareInited) { + ctx.Send(ParentId, new TEvPQProxy::TEvCloseSession("double partition locking", NPersQueue::NErrorCode::BAD_REQUEST)); + return; + } + if (!LockCounted) { + Counters.PartitionsToBeLocked.Inc(); + LockCounted = true; + } + if (StartReading) + AllPrepareInited = true; + + if (FirstInit) { + Y_VERIFY(!PipeClient); + FirstInit = false; + NTabletPipe::TClientConfig clientConfig; + clientConfig.RetryPolicy = { + .RetryLimitCount = 6, + .MinRetryTime = TDuration::MilliSeconds(10), + .MaxRetryTime = TDuration::MilliSeconds(100), + .BackoffMultiplier = 2, + .DoFirstRetryInstantly = true + }; + PipeClient = ctx.RegisterWithSameMailbox(NTabletPipe::CreateClient(ctx.SelfID, TabletID, clientConfig)); + + NKikimrClient::TPersQueueRequest request; + + request.MutablePartitionRequest()->SetTopic(Topic->GetClientsideName()); + request.MutablePartitionRequest()->SetPartition(Partition); + request.MutablePartitionRequest()->SetCookie(INIT_COOKIE); + + ActorIdToProto(PipeClient, request.MutablePartitionRequest()->MutablePipeClient()); + + auto cmd = request.MutablePartitionRequest()->MutableCmdCreateSession(); + cmd->SetClientId(InternalClientId); + cmd->SetSessionId(Session); + cmd->SetGeneration(Generation); + cmd->SetStep(Step); + + LOG_INFO_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << " INITING " << Topic->GetPrintableString() << " partition:" << Partition); + + TAutoPtr<TEvPersQueue::TEvRequest> req(new TEvPersQueue::TEvRequest); + Y_VERIFY(!RequestInfly); + CurrentRequest = request; + RequestInfly = true; + req->Record.Swap(&request); + + NTabletPipe::SendData(ctx, PipeClient, req.Release()); + } else { + Y_VERIFY(StartReading); //otherwise it is double locking from actor, not client - client makes lock always with StartReading == true + Y_VERIFY(InitDone); + InitStartReading(ctx); + } +} + + +void TPartitionActor::WaitDataInPartition(const TActorContext& ctx) { + + if (WaitDataInfly.size() > 1) //already got 2 requests inflight + return; + Y_VERIFY(InitDone); + + Y_VERIFY(PipeClient); + + if (!WaitForData) + return; + + Y_VERIFY(ReadOffset >= EndOffset); + + TAutoPtr<TEvPersQueue::TEvHasDataInfo> event(new TEvPersQueue::TEvHasDataInfo()); + event->Record.SetPartition(Partition); + event->Record.SetOffset(ReadOffset); + event->Record.SetCookie(++WaitDataCookie); + ui64 deadline = (ctx.Now() + WAIT_DATA - WAIT_DELTA).MilliSeconds(); + event->Record.SetDeadline(deadline); + event->Record.SetClientId(InternalClientId); + + LOG_DEBUG_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << " " << Topic->GetPrintableString() << " partition:" << Partition + << " wait data in partition inited, cookie " << WaitDataCookie + ); + + NTabletPipe::SendData(ctx, PipeClient, event.Release()); + + ctx.Schedule(PREWAIT_DATA, new TEvents::TEvWakeup()); + + ctx.Schedule(WAIT_DATA, new TEvPQProxy::TEvDeadlineExceeded(WaitDataCookie)); + + WaitDataInfly.insert(WaitDataCookie); +} + +void TPartitionActor::Handle(TEvPersQueue::TEvHasDataInfoResponse::TPtr& ev, const TActorContext& ctx) { + const auto& record = ev->Get()->Record; + + WriteTimestampEstimateMs = Max(WriteTimestampEstimateMs, record.GetWriteTimestampEstimateMS()); + + auto it = WaitDataInfly.find(ev->Get()->Record.GetCookie()); + if (it == WaitDataInfly.end()) { + LOG_DEBUG_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << " " << Topic->GetPrintableString() << " partition:" << Partition + << " unwaited response for WaitData " << ev->Get()->Record); + return; + } + WaitDataInfly.erase(it); + if (!WaitForData) + return; + + Counters.WaitsForData.Inc(); + + Y_VERIFY(record.HasEndOffset()); + Y_VERIFY(EndOffset <= record.GetEndOffset()); //end offset could not be changed if no data arrived, but signal will be sended anyway after timeout + Y_VERIFY(ReadOffset >= EndOffset); //otherwise no WaitData were needed + + LOG_DEBUG_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << " " << Topic->GetPrintableString() << " partition:" << Partition + << " wait for data done: " << " readOffset " << ReadOffset << " EndOffset " << EndOffset + << " newEndOffset " << record.GetEndOffset() << " commitOffset " << CommittedOffset + << " clientCommitOffset " << ClientCommitOffset << " cookie " << ev->Get()->Record.GetCookie() + ); + + EndOffset = record.GetEndOffset(); + SizeLag = record.GetSizeLag(); + + if (ReadOffset < EndOffset) { + WaitForData = false; + WaitDataInfly.clear(); + ctx.Send(ParentId, new TEvPQProxy::TEvPartitionReady(Topic, Partition, WTime, SizeLag, ReadOffset, EndOffset)); + LOG_DEBUG_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << " " << Topic->GetPrintableString() << " partition:" << Partition + << " ready for read with readOffset " << ReadOffset << " endOffset " << EndOffset + ); + } else { + if (PipeClient) + WaitDataInPartition(ctx); + } + CheckRelease(ctx); //just for logging purpose +} + + +void TPartitionActor::Handle(TEvPQProxy::TEvRead::TPtr& ev, const TActorContext& ctx) { + LOG_DEBUG_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << " READ FROM " << Topic->GetPrintableString() << " partition:" << Partition + << " event " << ev->Get()->Request << " readOffset " << ReadOffset + << " EndOffset " << EndOffset << " ClientCommitOffset " << ClientCommitOffset + << " committedOffset " << CommittedOffset << " Guid " << ev->Get()->Guid + ); + + Y_VERIFY(!NeedRelease); + Y_VERIFY(!Released); + + Y_VERIFY(ReadGuid.empty()); + Y_VERIFY(!RequestInfly); + + ReadGuid = ev->Get()->Guid; + + const auto& req = ev->Get()->Request.GetRead(); + + NKikimrClient::TPersQueueRequest request; + + request.MutablePartitionRequest()->SetTopic(Topic->GetClientsideName()); + + request.MutablePartitionRequest()->SetPartition(Partition); + request.MutablePartitionRequest()->SetCookie((ui64)ReadOffset); + + ActorIdToProto(PipeClient, request.MutablePartitionRequest()->MutablePipeClient()); + auto read = request.MutablePartitionRequest()->MutableCmdRead(); + read->SetClientId(InternalClientId); + read->SetClientDC(ClientDC); + if (req.GetMaxCount()) { + read->SetCount(req.GetMaxCount()); + } + if (req.GetMaxSize()) { + read->SetBytes(req.GetMaxSize()); + } + if (req.GetMaxTimeLagMs()) { + read->SetMaxTimeLagMs(req.GetMaxTimeLagMs()); + } + if (req.GetReadTimestampMs()) { + read->SetReadTimestampMs(req.GetReadTimestampMs()); + } + + read->SetOffset(ReadOffset); + read->SetTimeoutMs(READ_TIMEOUT_DURATION.MilliSeconds()); + RequestInfly = true; + CurrentRequest = request; + + if (!PipeClient) //Pipe will be recreated soon + return; + + TAutoPtr<TEvPersQueue::TEvRequest> event(new TEvPersQueue::TEvRequest); + event->Record.Swap(&request); + + NTabletPipe::SendData(ctx, PipeClient, event.Release()); +} + + +void TPartitionActor::Handle(TEvPQProxy::TEvCommit::TPtr& ev, const TActorContext& ctx) { + const ui64 readId = ev->Get()->ReadId; + const ui64 offset = ev->Get()->Offset; + Y_VERIFY(offset != Max<ui64>()); // has concreete offset + if (offset < ClientCommitOffset) { + LOG_ERROR_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << " " << Topic->GetPrintableString() << " partition:" << Partition + << " commit done to too small position " << offset + << " committedOffset " << ClientCommitOffset << " cookie " << readId + ); + } + Y_VERIFY(offset >= ClientCommitOffset); + + const bool hasProgress = offset > ClientCommitOffset; + + if (!hasProgress) {//nothing to commit for this partition + if (CommitsInfly.empty()) { + LOG_DEBUG_S( + ctx, NKikimrServices::PQ_READ_PROXY, + PQ_LOG_PREFIX << " " << Topic->GetPrintableString() << " partition:" << Partition + << " commit done with no effect with cookie " << readId + ); + ctx.Send(ParentId, new TEvPQProxy::TEvCommitDone(readId, Topic, Partition)); + CheckRelease(ctx); + } else { + CommitsInfly.push_back(std::pair<ui64, ui64>(readId, Max<ui64>())); + } + return; + } + + ClientCommitOffset = offset; + CommitsInfly.push_back(std::pair<ui64, ui64>(readId, offset)); + + if (PipeClient) //if not then pipe will be recreated soon and SendCommit will be done + SendCommit(readId, offset, ctx); +} + + +void TPartitionActor::Die(const TActorContext& ctx) { + if (PipeClient) + NTabletPipe::CloseClient(ctx, PipeClient); + TActorBootstrapped<TPartitionActor>::Die(ctx); +} + +void TPartitionActor::HandlePoison(TEvents::TEvPoisonPill::TPtr&, const TActorContext& ctx) { + if (LockCounted) + Counters.PartitionsToBeLocked.Dec(); + Die(ctx); +} + +void TPartitionActor::Handle(TEvPQProxy::TEvDeadlineExceeded::TPtr& ev, const TActorContext& ctx) { + + WaitDataInfly.erase(ev->Get()->Cookie); + if (ReadOffset >= EndOffset && WaitDataInfly.size() <= 1 && PipeClient) { + Y_VERIFY(WaitForData); + WaitDataInPartition(ctx); + } + +} + +void TPartitionActor::HandleWakeup(const TActorContext& ctx) { + if (ReadOffset >= EndOffset && WaitDataInfly.size() <= 1 && PipeClient) { + Y_VERIFY(WaitForData); + WaitDataInPartition(ctx); + } +} +} +} diff --git a/kikimr/yndx/grpc_services/persqueue/grpc_pq_session.h b/kikimr/yndx/grpc_services/persqueue/grpc_pq_session.h new file mode 100644 index 0000000000..22e2b61e5b --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/grpc_pq_session.h @@ -0,0 +1,317 @@ +#pragma once + +#include "ydb/core/client/server/grpc_base.h" +#include <library/cpp/grpc/server/grpc_server.h> +#include <util/generic/queue.h> + +using grpc::Status; + + +namespace NKikimr { +namespace NGRpcProxy { + +/////////////////////////////////////////////////////////////////////////////// + +using namespace NKikimrClient; + +template<class TResponse> +class ISessionHandler : public TAtomicRefCount<ISessionHandler<TResponse>> { +public: + virtual ~ISessionHandler() + { } + + /// Finish session. + virtual void Finish() = 0; + + /// Send reply to client. + virtual void Reply(const TResponse& resp) = 0; + + virtual void ReadyForNextRead() = 0; + + virtual bool IsShuttingDown() const = 0; +}; + +template<class TResponse> +using ISessionHandlerRef = TIntrusivePtr<ISessionHandler<TResponse>>; + + +template <class TRequest, class TResponse> +class ISession : public ISessionHandler<TResponse> +{ + + using ISessionRef = TIntrusivePtr<ISession<TRequest, TResponse>>; + +protected: + class TRequestCreated : public NGrpc::IQueueEvent { + public: + TRequestCreated(ISessionRef session) + : Session(session) + { } + + bool Execute(bool ok) override { + if (!ok) { + Session->DestroyStream("waiting stream creating failed"); + return false; + } + + Session->OnCreated(); + return false; + } + + void DestroyRequest() override { + if (!Session->Context.c_call() && Session->ClientDone) { + // AsyncNotifyWhenDone will not appear on the queue. + delete Session->ClientDone; + Session->ClientDone = nullptr; + } + delete this; + } + + ISessionRef Session; + }; + + class TReadDone : public NGrpc::IQueueEvent { + public: + TReadDone(ISessionRef session) + : Session(session) + { } + + bool Execute(bool ok) override { + if (ok) { + Session->OnRead(Request); + } else { + if (Session->IsCancelled()) { + Session->DestroyStream("reading from stream failed"); + } else { + Session->OnDone(); + } + } + return false; + } + + void DestroyRequest() override { + delete this; + } + + TRequest Request; + ISessionRef Session; + }; + + class TWriteDone : public NGrpc::IQueueEvent { + public: + TWriteDone(ISessionRef session, ui64 size) + : Session(session) + , Size(size) + { } + + bool Execute(bool ok) override { + Session->OnWriteDone(Size); + if (!ok) { + Session->DestroyStream("writing to stream failed"); + return false; + } + + TGuard<TSpinLock> lock(Session->Lock); + if (Session->Responses.empty()) { + Session->HaveWriteInflight = false; + if (Session->NeedFinish) { + lock.Release(); + Session->Stream.Finish(Status::OK, new TFinishDone(Session)); + } + } else { + auto resp = Session->Responses.front(); + Session->Responses.pop(); + lock.Release(); + ui64 sz = resp.ByteSize(); + Session->Stream.Write(resp, new TWriteDone(Session, sz)); + } + + return false; + } + + void DestroyRequest() override { + delete this; + } + + ISessionRef Session; + ui64 Size; + }; + + class TFinishDone : public NGrpc::IQueueEvent { + public: + TFinishDone(ISessionRef session) + : Session(session) + { } + + bool Execute(bool) override { + Session->DestroyStream("some stream finished"); + return false; + } + + void DestroyRequest() override { + delete this; + } + + ISessionRef Session; + }; + + class TClientDone : public NGrpc::IQueueEvent { + public: + TClientDone(ISessionRef session) + : Session(session) + { + Session->ClientDone = this; + } + + bool Execute(bool) override { + Session->ClientIsDone = true; + Session->DestroyStream("sesison closed"); + return false; + } + + void DestroyRequest() override { + Y_VERIFY(Session->ClientDone); + Session->ClientDone = nullptr; + delete this; + } + + ISessionRef Session; + }; + +public: + ISession(grpc::ServerCompletionQueue* cq) + : CQ(cq) + , Stream(&Context) + , HaveWriteInflight(false) + , NeedFinish(false) + , ClientIsDone(false) + { + Context.AsyncNotifyWhenDone(new TClientDone(this)); + } + + TString GetDatabase() const { + TString key = "x-ydb-database"; + const auto& clientMetadata = Context.client_metadata(); + const auto range = clientMetadata.equal_range(grpc::string_ref{key.data(), key.size()}); + if (range.first == range.second) { + return ""; + } + + TVector<TStringBuf> values; + values.reserve(std::distance(range.first, range.second)); + + for (auto it = range.first; it != range.second; ++it) { + return TString(it->second.data(), it->second.size()); + } + return ""; + } + + TString GetPeerName() const { + TString res(Context.peer()); + if (res.StartsWith("ipv4:[") || res.StartsWith("ipv6:[")) { + size_t pos = res.find(']'); + Y_VERIFY(pos != TString::npos); + res = res.substr(6, pos - 6); + } else if (res.StartsWith("ipv4:")) { + size_t pos = res.rfind(':'); + if (pos == TString::npos) {//no port + res = res.substr(5); + } else { + res = res.substr(5, pos - 5); + } + } else { + size_t pos = res.rfind(":"); //port + if (pos != TString::npos) { + res = res.substr(0, pos); + } + } + return res; + } + +protected: + + virtual void OnCreated() = 0; + virtual void OnRead(const TRequest& request) = 0; + virtual void OnDone() = 0; + virtual void OnWriteDone(ui64 size) = 0; + + virtual void DestroyStream(const TString& reason, NPersQueue::NErrorCode::EErrorCode code = NPersQueue::NErrorCode::BAD_REQUEST) = 0; + + /// Start accepting session's requests. + virtual void Start() = 0; + + bool IsCancelled() const { + return ClientIsDone && Context.IsCancelled(); + } + + void ReplyWithError(const TString& description, NPersQueue::NErrorCode::EErrorCode code) + { + TResponse response; + response.MutableError()->SetDescription(description); + response.MutableError()->SetCode(code); + Reply(response); + Finish(); + } + + /// Finish session. + void Finish() override { + { + TGuard<TSpinLock> lock(Lock); + if (NeedFinish) + return; + if (HaveWriteInflight || !Responses.empty()) { + NeedFinish = true; + return; + } + HaveWriteInflight = true; + } + + Stream.Finish(Status::OK, new TFinishDone(this)); + } + + /// Send reply to client. + void Reply(const TResponse& resp) override { + { + TGuard<TSpinLock> lock(Lock); + if (NeedFinish) //ignore responses after finish + return; + if (HaveWriteInflight || !Responses.empty()) { + Responses.push(resp); + return; + } else { + HaveWriteInflight = true; + } + } + + ui64 size = resp.ByteSize(); + Stream.Write(resp, new TWriteDone(this, size)); + } + + void ReadyForNextRead() override { + { + TGuard<TSpinLock> lock(Lock); + if (NeedFinish) { + return; + } + } + + auto read = new TReadDone(this); + Stream.Read(&read->Request, read); + } + +protected: + grpc::ServerCompletionQueue* const CQ; + grpc::ServerContext Context; + grpc::ServerAsyncReaderWriter<TResponse, TRequest> + Stream; +private: + TSpinLock Lock; + bool HaveWriteInflight; + bool NeedFinish; + std::atomic<bool> ClientIsDone; + TClientDone* ClientDone; + TQueue<TResponse> Responses; //TODO: if Responses total size is too big - fail this session; +}; + +} +} diff --git a/kikimr/yndx/grpc_services/persqueue/grpc_pq_write.cpp b/kikimr/yndx/grpc_services/persqueue/grpc_pq_write.cpp new file mode 100644 index 0000000000..36ba3fa8f6 --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/grpc_pq_write.cpp @@ -0,0 +1,221 @@ +#include "grpc_pq_write.h" +#include "grpc_pq_actor.h" +#include "grpc_pq_session.h" +#include "ydb/core/client/server/grpc_proxy_status.h" + +#include <ydb/core/base/appdata.h> +#include <util/generic/queue.h> + +using namespace NActors; +using namespace NKikimrClient; + +using grpc::Status; + +namespace NKikimr { +namespace NGRpcProxy { + +using namespace NPersQueue; + +/////////////////////////////////////////////////////////////////////////////// + + +void TPQWriteServiceImpl::TSession::OnCreated() { // Start waiting for new session. + Proxy->WaitWriteSession(); + if (Proxy->TooMuchSessions()) { + ReplyWithError("proxy overloaded", NPersQueue::NErrorCode::OVERLOAD); + return; + } + TMaybe<TString> localCluster = Proxy->AvailableLocalCluster(); + if (NeedDiscoverClusters) { + if (!localCluster.Defined()) { + ReplyWithError("initializing", NPersQueue::NErrorCode::INITIALIZING); + return; + } else if (localCluster->empty()) { + ReplyWithError("cluster disabled", NPersQueue::NErrorCode::CLUSTER_DISABLED); + return; + } else { + CreateActor(*localCluster); + } + } else { + CreateActor(TString()); + } + ReadyForNextRead(); +} + +void TPQWriteServiceImpl::TSession::OnRead(const TWriteRequest& request) { + + switch (request.GetRequestCase()) { + case TWriteRequest::kInit: { + SendEvent(new TEvPQProxy::TEvWriteInit(request, GetPeerName(), GetDatabase())); + break; + } + case TWriteRequest::kDataBatch: + case TWriteRequest::kData: { + SendEvent(new TEvPQProxy::TEvWrite(request)); + break; + } + default: { + ReplyWithError("unsupported request", NPersQueue::NErrorCode::BAD_REQUEST); + } + } +} + +void TPQWriteServiceImpl::TSession::OnDone() { + SendEvent(new TEvPQProxy::TEvDone()); +} + +TPQWriteServiceImpl::TSession::TSession(std::shared_ptr<TPQWriteServiceImpl> proxy, + grpc::ServerCompletionQueue* cq, ui64 cookie, const TActorId& schemeCache, + TIntrusivePtr<NMonitoring::TDynamicCounters> counters, bool needDiscoverClusters) + : ISession(cq) + , Proxy(proxy) + , Cookie(cookie) + , SchemeCache(schemeCache) + , Counters(counters) + , NeedDiscoverClusters(needDiscoverClusters) +{ +} + +void TPQWriteServiceImpl::TSession::Start() { + if (!Proxy->IsShuttingDown()) { + Proxy->RequestSession(&Context, &Stream, CQ, CQ, new TRequestCreated(this)); + } +} + +ui64 TPQWriteServiceImpl::TSession::GetCookie() const { + return Cookie; +} + +void TPQWriteServiceImpl::TSession::DestroyStream(const TString& reason, const NPersQueue::NErrorCode::EErrorCode errorCode) { + // Send poison pill to the actor(if it is alive) + SendEvent(new TEvPQProxy::TEvDieCommand("write-session " + ToString<ui64>(Cookie) + ": " + reason, errorCode)); + // Remove reference to session from "cookie -> session" map. + Proxy->ReleaseSession(this); +} + +bool TPQWriteServiceImpl::TSession::IsShuttingDown() const { + return Proxy->IsShuttingDown(); +} + +void TPQWriteServiceImpl::TSession::CreateActor(const TString &localCluster) { + + auto classifier = Proxy->GetClassifier(); + ActorId = Proxy->ActorSystem->Register( + new TWriteSessionActor(this, Cookie, SchemeCache, Counters, localCluster, + classifier ? classifier->ClassifyAddress(GetPeerName()) + : "unknown"), TMailboxType::Simple, 0 + ); +} + +void TPQWriteServiceImpl::TSession::SendEvent(IEventBase* ev) { + Proxy->ActorSystem->Send(ActorId, ev); +} + +/////////////////////////////////////////////////////////////////////////////// + + +TPQWriteServiceImpl::TPQWriteServiceImpl(grpc::ServerCompletionQueue* cq, + NActors::TActorSystem* as, const TActorId& schemeCache, + TIntrusivePtr<NMonitoring::TDynamicCounters> counters, const ui32 maxSessions) + : CQ(cq) + , ActorSystem(as) + , SchemeCache(schemeCache) + , Counters(counters) + , MaxSessions(maxSessions) + , NeedDiscoverClusters(false) +{ +} + +void TPQWriteServiceImpl::InitClustersUpdater() +{ + TAppData* appData = ActorSystem->AppData<TAppData>(); + NeedDiscoverClusters = !appData->PQConfig.GetTopicsAreFirstClassCitizen(); + if (NeedDiscoverClusters) { + ActorSystem->Register(new TClustersUpdater(this)); + } +} + + +ui64 TPQWriteServiceImpl::NextCookie() { + return AtomicIncrement(LastCookie); +} + + +void TPQWriteServiceImpl::ReleaseSession(TSessionRef session) { + with_lock (Lock) { + bool erased = Sessions.erase(session->GetCookie()); + if (erased) { + ActorSystem->Send(MakeGRpcProxyStatusID(ActorSystem->NodeId), new TEvGRpcProxyStatus::TEvUpdateStatus(0, 0, -1, 0)); + } + } +} + + +void TPQWriteServiceImpl::SetupIncomingRequests() { + WaitWriteSession(); +} + + +void TPQWriteServiceImpl::WaitWriteSession() { + + const ui64 cookie = NextCookie(); + + ActorSystem->Send(MakeGRpcProxyStatusID(ActorSystem->NodeId), new TEvGRpcProxyStatus::TEvUpdateStatus(0,0,1,0)); + + TSessionRef session(new TSession(shared_from_this(), CQ, cookie, SchemeCache, Counters, NeedDiscoverClusters)); + { + with_lock (Lock) { + Sessions[cookie] = session; + } + } + + session->Start(); +} + + +bool TPQWriteServiceImpl::TooMuchSessions() { + with_lock (Lock) { + return Sessions.size() >= MaxSessions; + } +} + + +TMaybe<TString> TPQWriteServiceImpl::AvailableLocalCluster() { + with_lock (Lock) { + return AvailableLocalClusterName; + } +} + + +void TPQWriteServiceImpl::NetClassifierUpdated(NAddressClassifier::TLabeledAddressClassifier::TConstPtr classifier) { + auto g(Guard(Lock)); + if (!DatacenterClassifier) { + for (auto it = Sessions.begin(); it != Sessions.end();) { + auto jt = it++; + jt->second->DestroyStream("datacenter classifier initialized, restart session please", NPersQueue::NErrorCode::INITIALIZING); + } + } + + DatacenterClassifier = classifier; +} + + + +void TPQWriteServiceImpl::CheckClusterChange(const TString &localCluster, const bool enabled) { + with_lock (Lock) { + AvailableLocalClusterName = enabled ? localCluster : TString(); + + if (!enabled) { + for (auto it = Sessions.begin(); it != Sessions.end();) { + auto jt = it++; + jt->second->DestroyStream("cluster disabled", NPersQueue::NErrorCode::CLUSTER_DISABLED); + } + } + } +} + + +/////////////////////////////////////////////////////////////////////////////// + +} +} diff --git a/kikimr/yndx/grpc_services/persqueue/grpc_pq_write.h b/kikimr/yndx/grpc_services/persqueue/grpc_pq_write.h new file mode 100644 index 0000000000..40a0d64e47 --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/grpc_pq_write.h @@ -0,0 +1,148 @@ +#pragma once + +#include "grpc_pq_clusters_updater_actor.h" +#include "grpc_pq_session.h" + +#include <ydb/core/client/server/grpc_base.h> + +#include <kikimr/yndx/api/grpc/persqueue.grpc.pb.h> + +#include <library/cpp/actors/core/actorsystem.h> + +#include <util/generic/hash.h> +#include <util/generic/maybe.h> +#include <util/system/mutex.h> + +namespace NKikimr { +namespace NGRpcProxy { + +// Класс, отвечающий за обработку запросов на запись. + +class TPQWriteServiceImpl : public IPQClustersUpdaterCallback, public std::enable_shared_from_this<TPQWriteServiceImpl> { + + class TSession : public ISession<NPersQueue::TWriteRequest, NPersQueue::TWriteResponse> + { + + void OnCreated() override; + void OnRead(const NPersQueue::TWriteRequest& request) override; + void OnDone() override; + void OnWriteDone(ui64) override {}; + + public: + TSession(std::shared_ptr<TPQWriteServiceImpl> proxy, + grpc::ServerCompletionQueue* cq, ui64 cookie, const TActorId& schemeCache, + TIntrusivePtr<NMonitoring::TDynamicCounters> counters, bool needDiscoverClusters); + void Start() override; + ui64 GetCookie() const; + void DestroyStream(const TString& reason, const NPersQueue::NErrorCode::EErrorCode errorCode) override; + bool IsShuttingDown() const override; + + private: + void CreateActor(const TString& localCluster); + void SendEvent(NActors::IEventBase* ev); + + private: + std::shared_ptr<TPQWriteServiceImpl> Proxy; + const ui64 Cookie; + + NActors::TActorId ActorId; + + const NActors::TActorId SchemeCache; + + TIntrusivePtr<NMonitoring::TDynamicCounters> Counters; + + bool NeedDiscoverClusters; + }; + using TSessionRef = TIntrusivePtr<TSession>; + +public: + TPQWriteServiceImpl(grpc::ServerCompletionQueue* cq, + NActors::TActorSystem* as, const NActors::TActorId& schemeCache, TIntrusivePtr<NMonitoring::TDynamicCounters> counters, + const ui32 maxSessions); + virtual ~TPQWriteServiceImpl() = default; + + void SetupIncomingRequests(); + + virtual void RequestSession(::grpc::ServerContext* context, ::grpc::ServerAsyncReaderWriter< ::NPersQueue::TWriteResponse, ::NPersQueue::TWriteRequest>* stream, + ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) = 0; + + void StopService() { + AtomicSet(ShuttingDown_, 1); + } + + bool IsShuttingDown() const { + return AtomicGet(ShuttingDown_); + } + void InitClustersUpdater(); + +private: + ui64 NextCookie(); + + //! Unregistry session object. + void ReleaseSession(TSessionRef session); + + //! Start listening for incoming connections. + void WaitWriteSession(); + bool TooMuchSessions(); + TMaybe<TString> AvailableLocalCluster(); + NAddressClassifier::TLabeledAddressClassifier::TConstPtr GetClassifier() const { + auto g(Guard(Lock)); + return DatacenterClassifier; + } + void CheckClusterChange(const TString& localCluster, const bool enabled) override; + void NetClassifierUpdated(NAddressClassifier::TLabeledAddressClassifier::TConstPtr classifier) override; + +private: + grpc::ServerContext Context; + grpc::ServerCompletionQueue* CQ; + + NActors::TActorSystem* ActorSystem; + NActors::TActorId SchemeCache; + + TAtomic LastCookie = 0; + + TMutex Lock; + THashMap<ui64, TSessionRef> Sessions; + + TIntrusivePtr<NMonitoring::TDynamicCounters> Counters; + + ui32 MaxSessions; + TMaybe<TString> AvailableLocalClusterName; + TString SelectSourceIdQuery; + TString UpdateSourceIdQuery; + TString DeleteSourceIdQuery; + + TAtomic ShuttingDown_ = 0; + + bool NeedDiscoverClusters; // Legacy mode OR account-mode in multi-cluster setup; + + NAddressClassifier::TLabeledAddressClassifier::TConstPtr DatacenterClassifier; // Detects client's datacenter by IP. May be null +}; + + +class TPQWriteService : public TPQWriteServiceImpl { +public: + TPQWriteService(NPersQueue::PersQueueService::AsyncService* service, + grpc::ServerCompletionQueue* cq, + NActors::TActorSystem* as, const NActors::TActorId& schemeCache, TIntrusivePtr<NMonitoring::TDynamicCounters> counters, + const ui32 maxSessions) + : TPQWriteServiceImpl(cq, as, schemeCache, counters, maxSessions) + , Service(service) + {} + + virtual ~TPQWriteService() + {} + + void RequestSession(::grpc::ServerContext* context, ::grpc::ServerAsyncReaderWriter< ::NPersQueue::TWriteResponse, ::NPersQueue::TWriteRequest>* stream, + ::grpc::CompletionQueue* new_call_cq, ::grpc::ServerCompletionQueue* notification_cq, void *tag) override + { + Service->RequestWriteSession(context, stream, new_call_cq, notification_cq, tag); + } + +private: + NPersQueue::PersQueueService::AsyncService* Service; +}; + + +} +} diff --git a/kikimr/yndx/grpc_services/persqueue/grpc_pq_write_actor.cpp b/kikimr/yndx/grpc_services/persqueue/grpc_pq_write_actor.cpp new file mode 100644 index 0000000000..ae2c3198c0 --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/grpc_pq_write_actor.cpp @@ -0,0 +1,1055 @@ +#include "grpc_pq_actor.h" + +#include <ydb/core/persqueue/pq_database.h> +#include <ydb/core/persqueue/write_meta.h> +#include <ydb/core/protos/services.pb.h> +#include <ydb/public/lib/deprecated/kicli/kicli.h> +#include <ydb/library/persqueue/topic_parser/topic_parser.h> +#include <ydb/library/persqueue/topic_parser/counters.h> +#include <ydb/services/lib/sharding/sharding.h> + +#include <library/cpp/actors/core/log.h> +#include <library/cpp/digest/md5/md5.h> +#include <util/string/hex.h> +#include <util/string/vector.h> +#include <util/string/escape.h> + +using namespace NActors; +using namespace NKikimrClient; + + +namespace NKikimr { +using namespace NMsgBusProxy::NPqMetaCacheV2; +using namespace NSchemeCache; +using namespace NPQ; + +template <> +void FillChunkDataFromReq(NKikimrPQClient::TDataChunk& proto, const NPersQueue::TWriteRequest::TData& data) { + proto.SetData(data.GetData()); + proto.SetSeqNo(data.GetSeqNo()); + proto.SetCreateTime(data.GetCreateTimeMs()); + proto.SetCodec(data.GetCodec()); +} + +template <> +void FillExtraFieldsForDataChunk( + const NPersQueue::TWriteRequest::TInit& init, + NKikimrPQClient::TDataChunk& data, + TString& server, + TString& ident, + TString& logType, + TString& file +) { + for (ui32 i = 0; i < init.GetExtraFields().ItemsSize(); ++i) { + const auto& item = init.GetExtraFields().GetItems(i); + if (item.GetKey() == "server") { + server = item.GetValue(); + } else if (item.GetKey() == "ident") { + ident = item.GetValue(); + } else if (item.GetKey() == "logtype") { + logType = item.GetValue(); + } else if (item.GetKey() == "file") { + file = item.GetValue(); + } else { + auto res = data.MutableExtraFields()->AddItems(); + res->SetKey(item.GetKey()); + res->SetValue(item.GetValue()); + } + } +} + +namespace NGRpcProxy { + +using namespace NPersQueue; + +static const ui32 MAX_RESERVE_REQUESTS_INFLIGHT = 5; + +static const ui32 MAX_BYTES_INFLIGHT = 1_MB; +static const TDuration SOURCEID_UPDATE_PERIOD = TDuration::Hours(1); + +//TODO: add here tracking of bytes in/out + +TWriteSessionActor::TWriteSessionActor(IWriteSessionHandlerRef handler, const ui64 cookie, const TActorId& schemeCache, + TIntrusivePtr<NMonitoring::TDynamicCounters> counters, const TString& localDC, + const TMaybe<TString> clientDC) + : Handler(handler) + , State(ES_CREATED) + , SchemeCache(schemeCache) + , PeerName("") + , Cookie(cookie) + , Partition(0) + , PreferedPartition(Max<ui32>()) + , NumReserveBytesRequests(0) + , WritesDone(false) + , Counters(counters) + , BytesInflight_(0) + , BytesInflightTotal_(0) + , NextRequestInited(false) + , NextRequestCookie(0) + , Token(nullptr) + , ACLCheckInProgress(true) + , FirstACLCheck(true) + , ForceACLCheck(false) + , RequestNotChecked(true) + , LastACLCheckTimestamp(TInstant::Zero()) + , LogSessionDeadline(TInstant::Zero()) + , BalancerTabletId(0) + , PipeToBalancer() + , LocalDC(localDC) + , ClientDC(clientDC ? *clientDC : "other") + , LastSourceIdUpdate(TInstant::Zero()) + , SourceIdCreateTime(0) + , SourceIdUpdatesInflight(0) + +{ + Y_ASSERT(Handler); +} + +TWriteSessionActor::~TWriteSessionActor() = default; + + +void TWriteSessionActor::Bootstrap(const TActorContext& ctx) { + if (!AppData(ctx)->PQConfig.GetTopicsAreFirstClassCitizen()) { + ++(*GetServiceCounters(Counters, "pqproxy|writeSession")->GetCounter("SessionsCreatedTotal", true)); + } + Become(&TThis::StateFunc); + + Database = NKikimr::NPQ::GetDatabaseFromConfig(AppData(ctx)->PQConfig); + const auto& root = AppData(ctx)->PQConfig.GetRoot(); + SelectSourceIdQuery = GetSourceIdSelectQuery(root); + UpdateSourceIdQuery = GetUpdateIdSelectQuery(root); + ConverterFactory = MakeHolder<NPersQueue::TTopicNamesConverterFactory>( + AppData(ctx)->PQConfig, LocalDC + ); + StartTime = ctx.Now(); +} + + +void TWriteSessionActor::Die(const TActorContext& ctx) { + if (State == ES_DYING) + return; + State = ES_DYING; + if (Writer) + ctx.Send(Writer, new TEvents::TEvPoisonPill()); + + if (PipeToBalancer) + NTabletPipe::CloseClient(ctx, PipeToBalancer); + + if (SessionsActive) { + SessionsActive.Dec(); + BytesInflight.Dec(BytesInflight_); + BytesInflightTotal.Dec(BytesInflightTotal_); + } + + LOG_INFO_S(ctx, NKikimrServices::PQ_WRITE_PROXY, "session cookie: " << Cookie << " sessionId: " << OwnerCookie << " is DEAD"); + + if (!Handler->IsShuttingDown()) + Handler->Finish(); + TActorBootstrapped<TWriteSessionActor>::Die(ctx); +} + +void TWriteSessionActor::CheckFinish(const TActorContext& ctx) { + if (!WritesDone) + return; + if (State != ES_INITED) { + CloseSession("out of order Writes done before initialization", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + if (Writes.empty() && FormedWrites.empty() && SentMessages.empty()) { + CloseSession("", NPersQueue::NErrorCode::OK, ctx); + return; + } +} + +void TWriteSessionActor::Handle(TEvPQProxy::TEvDone::TPtr&, const TActorContext& ctx) { + WritesDone = true; + CheckFinish(ctx); +} + +void TWriteSessionActor::CheckACL(const TActorContext& ctx) { + Y_VERIFY(ACLCheckInProgress); + Y_VERIFY(SecurityObject); + NACLib::EAccessRights rights = NACLib::EAccessRights::UpdateRow; + if (!AppData(ctx)->PQConfig.GetCheckACL() || SecurityObject->CheckAccess(rights, *Token)) { + ACLCheckInProgress = false; + if (FirstACLCheck) { + FirstACLCheck = false; + DiscoverPartition(ctx); + } + } else { + TString errorReason = Sprintf("access to topic '%s' denied for '%s' due to 'no WriteTopic rights', Marker# PQ1125", + DiscoveryConverter->GetPrintableString().c_str(), + Token->GetUserSID().c_str()); + CloseSession(errorReason, NPersQueue::NErrorCode::ACCESS_DENIED, ctx); + } +} + +void TWriteSessionActor::Handle(TEvPQProxy::TEvWriteInit::TPtr& ev, const TActorContext& ctx) { + THolder<TEvPQProxy::TEvWriteInit> event(ev->Release()); + + if (State != ES_CREATED) { + //answer error + CloseSession("got second init request", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + const auto& init = event->Request.GetInit(); + + if (init.GetTopic().empty() || init.GetSourceId().empty()) { + CloseSession("no topic or SourceId in init request", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + + if (init.GetProxyCookie() != ctx.SelfID.NodeId() && init.GetProxyCookie() != MAGIC_COOKIE_VALUE) { + CloseSession("you must perform ChooseProxy request at first and go to ProxyName server with ProxyCookie", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + + //1. Database - !(Root or empty) (Need to bring root DB(s) list to PQConfig) - ONLY search modern path /Database/Path + //2. No database. Try parse and resolve account to database. If possible, try search this path. + //3. Fallback from 2 - legacy mode. + + DiscoveryConverter = ConverterFactory->MakeDiscoveryConverter(init.GetTopic(), true, LocalDC, Database); + if (!DiscoveryConverter->IsValid()) { + CloseSession( + TStringBuilder() << "incorrect topic \"" << DiscoveryConverter->GetOriginalTopic() + << "\": " << DiscoveryConverter->GetReason(), + NPersQueue::NErrorCode::BAD_REQUEST, + ctx + ); + } + PeerName = event->PeerName; + if (!event->Database.empty()) { + Database = event->Database; + } + + SourceId = init.GetSourceId(); + //TODO: check that sourceId does not have characters '"\_% - espace them on client may be? + + Auth = event->Request.GetCredentials(); + event->Request.ClearCredentials(); + Y_PROTOBUF_SUPPRESS_NODISCARD Auth.SerializeToString(&AuthStr); + + LOG_INFO_S(ctx, NKikimrServices::PQ_WRITE_PROXY, "session request cookie: " << Cookie << " " << init << " from " << PeerName); + UserAgent = init.GetVersion(); + LogSession(ctx); + + auto* request = new TEvDescribeTopicsRequest({DiscoveryConverter}); + //TODO: GetNode for /Root/PQ then describe from balancer + ctx.Send(SchemeCache, request); + State = ES_WAIT_SCHEME_2; + InitRequest = init; + PreferedPartition = init.GetPartitionGroup() > 0 ? init.GetPartitionGroup() - 1 : Max<ui32>(); +} + +void TWriteSessionActor::InitAfterDiscovery(const TActorContext& ctx) { + try { + EncodedSourceId = NSourceIdEncoding::EncodeSrcId(FullConverter->GetTopicForSrcIdHash(), SourceId); + } catch (yexception& e) { + CloseSession(TStringBuilder() << "incorrect sourceId \"" << SourceId << "\": " << e.what(), NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + + InitMeta = GetInitialDataChunk(InitRequest, FullConverter->GetClientsideName(), PeerName); + + auto subGroup = GetServiceCounters(Counters, "pqproxy|SLI"); + Aggr = {{{{"Account", FullConverter->GetAccount()}}, {"total"}}}; + + SLIErrors = NKikimr::NPQ::TMultiCounter(subGroup, Aggr, {}, {"RequestsError"}, true, "sensor", false); + SLITotal = NKikimr::NPQ::TMultiCounter(subGroup, Aggr, {}, {"RequestsTotal"}, true, "sensor", false); + SLITotal.Inc(); +} + + +void TWriteSessionActor::SetupCounters() +{ + if (SessionsCreated) { + return; + } + + //now topic is checked, can create group for real topic, not garbage + auto subGroup = GetServiceCounters(Counters, "pqproxy|writeSession"); + Y_VERIFY(FullConverter); + auto aggr = GetLabels(FullConverter); + + BytesInflight = NKikimr::NPQ::TMultiCounter(subGroup, aggr, {}, {"BytesInflight"}, false); + SessionsWithoutAuth = NKikimr::NPQ::TMultiCounter(subGroup, aggr, {}, {"WithoutAuth"}, true); + BytesInflightTotal = NKikimr::NPQ::TMultiCounter(subGroup, aggr, {}, {"BytesInflightTotal"}, false); + SessionsCreated = NKikimr::NPQ::TMultiCounter(subGroup, aggr, {}, {"SessionsCreated"}, true); + SessionsActive = NKikimr::NPQ::TMultiCounter(subGroup, aggr, {}, {"SessionsActive"}, false); + Errors = NKikimr::NPQ::TMultiCounter(subGroup, aggr, {}, {"Errors"}, true); + + SessionsCreated.Inc(); + SessionsActive.Inc(); +} + + +void TWriteSessionActor::SetupCounters(const TString& cloudId, const TString& dbId, const TString& folderId) +{ + if (SessionsCreated) { + return; + } + + //now topic is checked, can create group for real topic, not garbage + auto subGroup = GetCountersForStream(Counters); + Y_VERIFY(FullConverter); + auto aggr = GetLabelsForStream(FullConverter, cloudId, dbId, folderId); + + BytesInflight = NKikimr::NPQ::TMultiCounter(subGroup, aggr, {}, {"stream.internal_write.bytes_proceeding"}, false, "name"); + SessionsWithoutAuth = NKikimr::NPQ::TMultiCounter(subGroup, aggr, {}, {"stream.internal_write.sessions_without_auth"}, true, "name"); + BytesInflightTotal = NKikimr::NPQ::TMultiCounter(subGroup, aggr, {}, {"stream.internal_write.bytes_proceeding_total"}, false, "name"); + SessionsCreated = NKikimr::NPQ::TMultiCounter(subGroup, aggr, {}, {"stream.internal_write.sessions_created_per_second"}, true, "name"); + SessionsActive = NKikimr::NPQ::TMultiCounter(subGroup, aggr, {}, {"stream.internal_write.sessions_active"}, false, "name"); + Errors = NKikimr::NPQ::TMultiCounter(subGroup, aggr, {}, {"stream.internal_write.errors_per_second"}, true, "name"); + + SessionsCreated.Inc(); + SessionsActive.Inc(); +} + + +void TWriteSessionActor::Handle(TEvDescribeTopicsResponse::TPtr& ev, const TActorContext& ctx) { + Y_VERIFY(State == ES_WAIT_SCHEME_2); + auto& res = ev->Get()->Result; + Y_VERIFY(res->ResultSet.size() == 1); + + auto& entry = res->ResultSet[0]; + TString errorReason; + + auto& path = entry.Path; + auto& topic = ev->Get()->TopicsRequested[0]; + switch (entry.Status) { + case TSchemeCacheNavigate::EStatus::RootUnknown: { + errorReason = Sprintf("path '%s' has incorrect root prefix, Marker# PQ14", JoinPath(path).c_str()); + CloseSession(errorReason, NPersQueue::NErrorCode::UNKNOWN_TOPIC, ctx); + return; + } + case TSchemeCacheNavigate::EStatus::PathErrorUnknown: { + errorReason = Sprintf("no path '%s', Marker# PQ151", JoinPath(path).c_str()); + CloseSession(errorReason, NPersQueue::NErrorCode::UNKNOWN_TOPIC, ctx); + return; + } + case TSchemeCacheNavigate::EStatus::Ok: + break; + default: { + errorReason = Sprintf("topic '%s' describe error, Status# %s, Marker# PQ1", path.back().c_str(), + ToString(entry.Status).c_str()); + CloseSession(errorReason, NPersQueue::NErrorCode::ERROR, ctx); + break; + } + } + if (!entry.PQGroupInfo) { + + errorReason = Sprintf("topic '%s' describe error, reason - could not retrieve topic metadata, Marker# PQ99", + topic->GetPrintableString().c_str()); + CloseSession(errorReason, NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + PQInfo = entry.PQGroupInfo; + const auto& description = PQInfo->Description; + //const TString topicName = description.GetName(); + + if (entry.Kind != TSchemeCacheNavigate::EKind::KindTopic) { + errorReason = Sprintf("item '%s' is not a topic, Marker# PQ13", DiscoveryConverter->GetPrintableString().c_str()); + CloseSession(errorReason, NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + FullConverter = DiscoveryConverter->UpgradeToFullConverter(description.GetPQTabletConfig()); + InitAfterDiscovery(ctx); + SecurityObject = entry.SecurityObject; + + Y_VERIFY(description.PartitionsSize() > 0); + + for (ui32 i = 0; i < description.PartitionsSize(); ++i) { + const auto& pi = description.GetPartitions(i); + PartitionToTablet[pi.GetPartitionId()] = pi.GetTabletId(); + } + BalancerTabletId = description.GetBalancerTabletID(); + DatabaseId = description.GetPQTabletConfig().GetYdbDatabaseId(); + FolderId = description.GetPQTabletConfig().GetYcFolderId(); + + if (AppData(ctx)->PQConfig.GetTopicsAreFirstClassCitizen()) { + const auto& tabletConfig = description.GetPQTabletConfig(); + SetupCounters(tabletConfig.GetYcCloudId(), tabletConfig.GetYdbDatabaseId(), + tabletConfig.GetYcFolderId()); + } else { + SetupCounters(); + } + + if (!PipeToBalancer) { + NTabletPipe::TClientConfig clientConfig; + clientConfig.RetryPolicy = { + .RetryLimitCount = 6, + .MinRetryTime = TDuration::MilliSeconds(10), + .MaxRetryTime = TDuration::MilliSeconds(100), + .BackoffMultiplier = 2, + .DoFirstRetryInstantly = true + }; + PipeToBalancer = ctx.RegisterWithSameMailbox(NTabletPipe::CreateClient(ctx.SelfID, BalancerTabletId, clientConfig)); + } + + if (Auth.GetCredentialsCase() == NPersQueueCommon::TCredentials::CREDENTIALS_NOT_SET) { + //ACLCheckInProgress is still true - no recheck will be done + LOG_WARN_S(ctx, NKikimrServices::PQ_WRITE_PROXY, "session without AuthInfo : " << DiscoveryConverter->GetPrintableString() + << " sourceId " << SourceId << " from " << PeerName); + SessionsWithoutAuth.Inc(); + if (AppData(ctx)->PQConfig.GetRequireCredentialsInNewProtocol()) { + CloseSession("Unauthenticated access is forbidden, please provide credentials", NPersQueue::NErrorCode::ACCESS_DENIED, ctx); + return; + } + if (FirstACLCheck) { + FirstACLCheck = false; + DiscoverPartition(ctx); + return; + } + } + + InitCheckACL(ctx); +} + +void TWriteSessionActor::InitCheckACL(const TActorContext& ctx) { + + Y_VERIFY(ACLCheckInProgress); + + TString ticket; + switch (Auth.GetCredentialsCase()) { + case NPersQueueCommon::TCredentials::kTvmServiceTicket: + ticket = Auth.GetTvmServiceTicket(); + break; + case NPersQueueCommon::TCredentials::kOauthToken: + ticket = Auth.GetOauthToken(); + break; + default: + CloseSession("Uknown Credentials case", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + + auto entries = GetTicketParserEntries(DatabaseId, FolderId); + ctx.Send(MakeTicketParserID(), new TEvTicketParser::TEvAuthorizeTicket({ + .Database = Database, + .Ticket = ticket, + .PeerName = PeerName, + .Entries = entries + })); +} + +void TWriteSessionActor::Handle(TEvTicketParser::TEvAuthorizeTicketResult::TPtr& ev, const TActorContext& ctx) { + Y_VERIFY(ACLCheckInProgress); + TString ticket = ev->Get()->Ticket; + TString maskedTicket = ticket.size() > 5 ? (ticket.substr(0, 5) + "***" + ticket.substr(ticket.size() - 5)) : "***"; + LOG_INFO_S(ctx, NKikimrServices::PQ_WRITE_PROXY, "CheckACL ticket " << maskedTicket << " got result from TICKET_PARSER response: error: " << ev->Get()->Error << " user: " + << (ev->Get()->Error.empty() ? ev->Get()->Token->GetUserSID() : "")); + + if (!ev->Get()->Error.empty()) { + CloseSession(TStringBuilder() << "Ticket parsing error: " << ev->Get()->Error, NPersQueue::NErrorCode::ACCESS_DENIED, ctx); + return; + } + Token = ev->Get()->Token; + + + Y_VERIFY(ACLCheckInProgress); + LOG_INFO_S(ctx, NKikimrServices::PQ_WRITE_PROXY, "session cookie: " << Cookie << " sessionId: " << OwnerCookie << " describe result for acl check"); + CheckACL(ctx); +} + +void TWriteSessionActor::DiscoverPartition(const NActors::TActorContext& ctx) { + Y_VERIFY(FullConverter); + if (AppData(ctx)->PQConfig.GetTopicsAreFirstClassCitizen()) { + auto partitionId = PreferedPartition; + if (PreferedPartition == Max<ui32>()) { + partitionId = NKikimr::NDataStreams::V1::ShardFromDecimal( + NKikimr::NDataStreams::V1::HexBytesToDecimal(MD5::Calc(SourceId)), PartitionToTablet.size() + ); + } + ProceedPartition(partitionId, ctx); + return; + } + //read from DS + // Hash was always valid here, so new and old are the same + //currently, Topic contains full primary path + SendSelectPartitionRequest(EncodedSourceId.Hash, FullConverter->GetPrimaryPath(), ctx); + //previously topic was like "rt3.dc--account--topic" + SendSelectPartitionRequest(EncodedSourceId.Hash, FullConverter->GetTopicForSrcId(), ctx); + State = ES_WAIT_TABLE_REQUEST_1; +} + +void TWriteSessionActor::SendSelectPartitionRequest(ui32 hash, const TString &topic, + const NActors::TActorContext &ctx +) { + auto ev = MakeHolder<NKqp::TEvKqp::TEvQueryRequest>(); + ev->Record.MutableRequest()->SetAction(NKikimrKqp::QUERY_ACTION_EXECUTE); + ev->Record.MutableRequest()->SetType(NKikimrKqp::QUERY_TYPE_SQL_DML); + ev->Record.MutableRequest()->SetKeepSession(false); + ev->Record.MutableRequest()->SetQuery(SelectSourceIdQuery); + ev->Record.MutableRequest()->SetDatabase(Database); + ev->Record.MutableRequest()->MutableTxControl()->set_commit_tx(true); + ev->Record.MutableRequest()->MutableTxControl()->mutable_begin_tx()->mutable_serializable_read_write(); + ev->Record.MutableRequest()->MutableQueryCachePolicy()->set_keep_in_cache(true); + NClient::TParameters parameters; + parameters["$Hash"] = hash; // 'Valid' hash - short legacy name (account--topic) + parameters["$Topic"] = topic; //currently, Topic contains full primary path + parameters["$SourceId"] = EncodedSourceId.EscapedSourceId; + + ev->Record.MutableRequest()->MutableParameters()->Swap(¶meters); + ctx.Send(NKqp::MakeKqpProxyID(ctx.SelfID.NodeId()), ev.Release()); + SelectReqsInflight++; +} + + +void TWriteSessionActor::UpdatePartition(const TActorContext& ctx) { + Y_VERIFY(State == ES_WAIT_TABLE_REQUEST_1 || State == ES_WAIT_NEXT_PARTITION); + auto ev = MakeUpdateSourceIdMetadataRequest(EncodedSourceId.Hash, FullConverter->GetPrimaryPath()); // Now Topic is a path + ctx.Send(NKqp::MakeKqpProxyID(ctx.SelfID.NodeId()), ev.Release()); + SourceIdUpdatesInflight++; + + //Previously Topic contained legacy name with DC (rt3.dc1--acc--topic) + ev = MakeUpdateSourceIdMetadataRequest(EncodedSourceId.Hash, FullConverter->GetTopicForSrcId()); + ctx.Send(NKqp::MakeKqpProxyID(ctx.SelfID.NodeId()), ev.Release()); + SourceIdUpdatesInflight++; + + State = ES_WAIT_TABLE_REQUEST_2; +} + +void TWriteSessionActor::RequestNextPartition(const TActorContext& ctx) { + Y_VERIFY(State == ES_WAIT_TABLE_REQUEST_1); + State = ES_WAIT_NEXT_PARTITION; + THolder<TEvPersQueue::TEvGetPartitionIdForWrite> x(new TEvPersQueue::TEvGetPartitionIdForWrite); + Y_VERIFY(PipeToBalancer); + + NTabletPipe::SendData(ctx, PipeToBalancer, x.Release()); +} + +void TWriteSessionActor::Handle(TEvPersQueue::TEvGetPartitionIdForWriteResponse::TPtr& ev, const TActorContext& ctx) { + Y_VERIFY(State == ES_WAIT_NEXT_PARTITION); + Partition = ev->Get()->Record.GetPartitionId(); + UpdatePartition(ctx); +} + +void TWriteSessionActor::Handle(NKqp::TEvKqp::TEvQueryResponse::TPtr &ev, const TActorContext &ctx) { + auto& record = ev->Get()->Record.GetRef(); + + if (record.GetYdbStatus() == Ydb::StatusIds::ABORTED) { + LOG_INFO_S(ctx, NKikimrServices::PQ_WRITE_PROXY, "session cookie: " << Cookie << " sessionId: " << OwnerCookie << " sourceID " + << SourceId << " escaped " << EncodedSourceId.EscapedSourceId << " discover partition race, retrying"); + DiscoverPartition(ctx); + return; + } + + if (record.GetYdbStatus() != Ydb::StatusIds::SUCCESS) { + TStringBuilder errorReason; + errorReason << "internal error in kqp Marker# PQ50 : " << record; + if (State == EState::ES_INITED) { + LOG_WARN_S(ctx, NKikimrServices::PQ_WRITE_PROXY, errorReason); + SourceIdUpdatesInflight--; + } else { + CloseSession(errorReason, NPersQueue::NErrorCode::ERROR, ctx); + } + return; + } + + if (State == EState::ES_WAIT_TABLE_REQUEST_1) { + SelectReqsInflight--; + auto& t = record.GetResponse().GetResults(0).GetValue().GetStruct(0); + + if (t.ListSize() != 0) { + auto& tt = t.GetList(0).GetStruct(0); + if (tt.HasOptional() && tt.GetOptional().HasUint32()) { //already got partition + auto accessTime = t.GetList(0).GetStruct(2).GetOptional().GetUint64(); + if (accessTime > MaxSrcIdAccessTime) { // AccessTime + Partition = tt.GetOptional().GetUint32(); + PartitionFound = true; + SourceIdCreateTime = t.GetList(0).GetStruct(1).GetOptional().GetUint64(); + MaxSrcIdAccessTime = accessTime; + } + } + } + if (SelectReqsInflight != 0) { + return; + } + if (SourceIdCreateTime == 0) { + SourceIdCreateTime = TInstant::Now().MilliSeconds(); + } + if (PartitionFound && PreferedPartition < Max<ui32>() && Partition != PreferedPartition) { + CloseSession(TStringBuilder() << "SourceId " << SourceId << " is already bound to PartitionGroup " << (Partition + 1) << ", but client provided " << (PreferedPartition + 1) << ". SourceId->PartitionGroup binding cannot be changed, either use another SourceId, specify PartitionGroup " << (Partition + 1) << ", or do not specify PartitionGroup at all.", + NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + LOG_INFO_S(ctx, NKikimrServices::PQ_WRITE_PROXY, "session cookie: " << Cookie << " sessionId: " << OwnerCookie << " sourceID " + << SourceId << " escaped " << EncodedSourceId.EscapedSourceId << " hash " << EncodedSourceId.Hash << " partition " << Partition << " partitions " + << PartitionToTablet.size() << "(" << EncodedSourceId.Hash % PartitionToTablet.size() << ") create " << SourceIdCreateTime << " result " << t); + + if (!PartitionFound && (PreferedPartition < Max<ui32>() || !AppData(ctx)->PQConfig.GetRoundRobinPartitionMapping())) { + Partition = PreferedPartition < Max<ui32>() ? PreferedPartition : EncodedSourceId.Hash % PartitionToTablet.size(); //choose partition default value + PartitionFound = true; + } + + if (PartitionFound) { + UpdatePartition(ctx); + } else { + RequestNextPartition(ctx); + } + return; + } else if (State == EState::ES_WAIT_TABLE_REQUEST_2) { + Y_VERIFY(SourceIdUpdatesInflight > 0); + SourceIdUpdatesInflight--; + if (SourceIdUpdatesInflight == 0) { + LastSourceIdUpdate = ctx.Now(); + ProceedPartition(Partition, ctx); + } + } else if (State == EState::ES_INITED) { + Y_VERIFY(SourceIdUpdatesInflight > 0); + SourceIdUpdatesInflight--; + if (SourceIdUpdatesInflight == 0) { + LastSourceIdUpdate = ctx.Now(); + } + } else { + Y_FAIL("Wrong state"); + } +} + +THolder<NKqp::TEvKqp::TEvQueryRequest> TWriteSessionActor::MakeUpdateSourceIdMetadataRequest( + ui32 hash, const TString& topic +) { + auto ev = MakeHolder<NKqp::TEvKqp::TEvQueryRequest>(); + + ev->Record.MutableRequest()->SetAction(NKikimrKqp::QUERY_ACTION_EXECUTE); + ev->Record.MutableRequest()->SetType(NKikimrKqp::QUERY_TYPE_SQL_DML); + ev->Record.MutableRequest()->SetQuery(UpdateSourceIdQuery); + ev->Record.MutableRequest()->SetDatabase(Database); + ev->Record.MutableRequest()->SetKeepSession(false); + ev->Record.MutableRequest()->MutableTxControl()->set_commit_tx(true); + ev->Record.MutableRequest()->MutableTxControl()->mutable_begin_tx()->mutable_serializable_read_write(); + ev->Record.MutableRequest()->MutableQueryCachePolicy()->set_keep_in_cache(true); + + NClient::TParameters parameters; + parameters["$Hash"] = hash; + parameters["$Topic"] = topic; //Previously Topic contained legacy name with DC (rt3.dc1--acc--topic) + parameters["$SourceId"] = EncodedSourceId.EscapedSourceId; + parameters["$CreateTime"] = SourceIdCreateTime; + parameters["$AccessTime"] = TInstant::Now().MilliSeconds(); + parameters["$Partition"] = Partition; + ev->Record.MutableRequest()->MutableParameters()->Swap(¶meters); + + return ev; +} + + +void TWriteSessionActor::Handle(NKqp::TEvKqp::TEvProcessResponse::TPtr &ev, const TActorContext &ctx) { + auto& record = ev->Get()->Record; + LOG_INFO_S(ctx, NKikimrServices::PQ_WRITE_PROXY, "session cookie: " << Cookie << " sessionId: " << OwnerCookie << " sourceID " + << SourceId << " escaped " << EncodedSourceId.EscapedSourceId << " discover partition error - " << record); + CloseSession("Internal error on discovering partition", NPersQueue::NErrorCode::ERROR, ctx); +} + + +void TWriteSessionActor::ProceedPartition(const ui32 partition, const TActorContext& ctx) { + Partition = partition; + auto it = PartitionToTablet.find(Partition); + + ui64 tabletId = it != PartitionToTablet.end() ? it->second : 0; + + if (!tabletId) { + CloseSession( + Sprintf("no partition %u in topic '%s', Marker# PQ4", Partition, DiscoveryConverter->GetPrintableString().c_str()), + NPersQueue::NErrorCode::UNKNOWN_TOPIC, ctx + ); + return; + } + + Writer = ctx.RegisterWithSameMailbox(NPQ::CreatePartitionWriter(ctx.SelfID, tabletId, Partition, SourceId)); + State = ES_WAIT_WRITER_INIT; + + ui32 border = AppData(ctx)->PQConfig.GetWriteInitLatencyBigMs(); + auto subGroup = GetServiceCounters(Counters, "pqproxy|SLI"); + + InitLatency = NKikimr::NPQ::CreateSLIDurationCounter(subGroup, Aggr, "WriteInit", border, {100, 200, 500, 1000, 1500, 2000, 5000, 10000, 30000, 99999999}); + SLIBigLatency = NKikimr::NPQ::TMultiCounter(subGroup, Aggr, {}, {"RequestsBigLatency"}, true, "sensor", false); + + ui32 initDurationMs = (ctx.Now() - StartTime).MilliSeconds(); + InitLatency.IncFor(initDurationMs, 1); + if (initDurationMs >= border) { + SLIBigLatency.Inc(); + } +} + +void TWriteSessionActor::CloseSession(const TString& errorReason, const NPersQueue::NErrorCode::EErrorCode errorCode, const NActors::TActorContext& ctx) { + if (errorCode != NPersQueue::NErrorCode::OK) { + if (InternalErrorCode(errorCode)) { + SLIErrors.Inc(); + } + + if (Errors) { + Errors.Inc(); + } else if (!AppData(ctx)->PQConfig.GetTopicsAreFirstClassCitizen()) { + ++(*GetServiceCounters(Counters, "pqproxy|writeSession")->GetCounter("Errors", true)); + } + + TWriteResponse result; + + auto error = result.MutableError(); + error->SetDescription(errorReason); + error->SetCode(errorCode); + + LOG_INFO_S(ctx, NKikimrServices::PQ_WRITE_PROXY, "session error cookie: " << Cookie << " reason: \"" << errorReason << "\" code: " << EErrorCode_Name(errorCode) << " sessionId: " << OwnerCookie); + + Handler->Reply(result); + } else { + LOG_INFO_S(ctx, NKikimrServices::PQ_WRITE_PROXY, "session closed cookie: " << Cookie << " sessionId: " << OwnerCookie); + } + + Die(ctx); +} + +void TWriteSessionActor::Handle(NPQ::TEvPartitionWriter::TEvInitResult::TPtr& ev, const TActorContext& ctx) { + if (State != ES_WAIT_WRITER_INIT) { + return CloseSession("got init result but not wait for it", NPersQueue::NErrorCode::ERROR, ctx); + } + + const auto& result = *ev->Get(); + if (!result.IsSuccess()) { + const auto& error = result.GetError(); + if (error.Response.HasErrorCode()) { + return CloseSession("status is not ok: " + error.Response.GetErrorReason(), error.Response.GetErrorCode(), ctx); + } else { + return CloseSession("error at writer init: " + error.Reason, NPersQueue::NErrorCode::ERROR, ctx); + } + } + + OwnerCookie = result.GetResult().OwnerCookie; + const auto& maxSeqNo = result.GetResult().SourceIdInfo.GetSeqNo(); + + TWriteResponse response; + auto init = response.MutableInit(); + init->SetSessionId(EscapeC(OwnerCookie)); + init->SetMaxSeqNo(maxSeqNo); + init->SetPartition(Partition); + Y_VERIFY(FullConverter); + init->SetTopic(FullConverter->GetClientsideName()); + + LOG_INFO_S(ctx, NKikimrServices::PQ_WRITE_PROXY, "session inited cookie: " << Cookie << " partition: " << Partition + << " MaxSeqNo: " << maxSeqNo << " sessionId: " << OwnerCookie); + + Handler->Reply(response); + + State = ES_INITED; + + ctx.Schedule(CHECK_ACL_DELAY, new TEvents::TEvWakeup()); + + //init completed; wait for first data chunk + NextRequestInited = true; + Handler->ReadyForNextRead(); +} + +void TWriteSessionActor::Handle(NPQ::TEvPartitionWriter::TEvWriteAccepted::TPtr& ev, const TActorContext& ctx) { + if (State != ES_INITED) { + return CloseSession("got write permission but not wait for it", NPersQueue::NErrorCode::ERROR, ctx); + } + + Y_VERIFY(!FormedWrites.empty()); + TWriteRequestBatchInfo::TPtr writeRequest = std::move(FormedWrites.front()); + + if (ev->Get()->Cookie != writeRequest->Cookie) { + return CloseSession("out of order reserve bytes response from server, may be previous is lost", NPersQueue::NErrorCode::ERROR, ctx); + } + + FormedWrites.pop_front(); + + ui64 diff = writeRequest->ByteSize; + + SentMessages.emplace_back(std::move(writeRequest)); + + BytesInflight_ -= diff; + BytesInflight.Dec(diff); + + if (!NextRequestInited && BytesInflight_ < MAX_BYTES_INFLIGHT) { //allow only one big request to be readed but not sended + NextRequestInited = true; + Handler->ReadyForNextRead(); + } + + --NumReserveBytesRequests; + if (!Writes.empty()) + GenerateNextWriteRequest(ctx); +} + +void TWriteSessionActor::Handle(NPQ::TEvPartitionWriter::TEvWriteResponse::TPtr& ev, const TActorContext& ctx) { + if (State != ES_INITED) { + return CloseSession("got write response but not wait for it", NPersQueue::NErrorCode::ERROR, ctx); + } + + const auto& result = *ev->Get(); + if (!result.IsSuccess()) { + const auto& record = result.Record; + if (record.HasErrorCode()) { + return CloseSession("status is not ok: " + record.GetErrorReason(), record.GetErrorCode(), ctx); + } else { + return CloseSession("error at write: " + result.GetError().Reason, NPersQueue::NErrorCode::ERROR, ctx); + } + } + + const auto& resp = result.Record.GetPartitionResponse(); + + if (SentMessages.empty()) { + CloseSession("got too many replies from server, internal error", NPersQueue::NErrorCode::ERROR, ctx); + return; + } + + TWriteRequestBatchInfo::TPtr writeRequest = std::move(SentMessages.front()); + SentMessages.pop_front(); + + if (resp.GetCookie() != writeRequest->Cookie) { + return CloseSession("out of order write response from server, may be previous is lost", NPersQueue::NErrorCode::ERROR, ctx); + } + + auto addAck = [](const TPersQueuePartitionResponse::TCmdWriteResult& res, TWriteResponse::TAck* ack, TWriteResponse::TStat* stat) { + ack->SetSeqNo(res.GetSeqNo()); + ack->SetOffset(res.GetOffset()); + ack->SetAlreadyWritten(res.GetAlreadyWritten()); + + stat->SetTotalTimeInPartitionQueueMs( + Max(res.GetTotalTimeInPartitionQueueMs(), stat->GetTotalTimeInPartitionQueueMs())); + stat->SetPartitionQuotedTimeMs( + Max(res.GetPartitionQuotedTimeMs(), stat->GetPartitionQuotedTimeMs())); + stat->SetTopicQuotedTimeMs( + Max(res.GetTopicQuotedTimeMs(), stat->GetTopicQuotedTimeMs())); + stat->SetWriteTimeMs( + Max(res.GetWriteTimeMs(), stat->GetWriteTimeMs())); + }; + + size_t cmdWriteResultIndex = 0; + for (const auto& userWriteRequest : writeRequest->UserWriteRequests) { + TWriteResponse result; + if (userWriteRequest->Request.HasDataBatch()) { + if (resp.CmdWriteResultSize() - cmdWriteResultIndex < userWriteRequest->Request.GetDataBatch().DataSize()) { + CloseSession("too less responses from server", NPersQueue::NErrorCode::ERROR, ctx); + return; + } + for (size_t endIndex = cmdWriteResultIndex + userWriteRequest->Request.GetDataBatch().DataSize(); cmdWriteResultIndex < endIndex; ++cmdWriteResultIndex) { + addAck(resp.GetCmdWriteResult(cmdWriteResultIndex), + result.MutableAckBatch()->AddAck(), + result.MutableAckBatch()->MutableStat()); + } + } else { + Y_VERIFY(userWriteRequest->Request.HasData()); + if (cmdWriteResultIndex >= resp.CmdWriteResultSize()) { + CloseSession("too less responses from server", NPersQueue::NErrorCode::ERROR, ctx); + return; + } + auto* ack = result.MutableAck(); + addAck(resp.GetCmdWriteResult(cmdWriteResultIndex), ack, ack->MutableStat()); + ++cmdWriteResultIndex; + } + Handler->Reply(result); + } + + ui64 diff = writeRequest->ByteSize; + + BytesInflightTotal_ -= diff; + BytesInflightTotal.Dec(diff); + + CheckFinish(ctx); +} + +void TWriteSessionActor::Handle(NPQ::TEvPartitionWriter::TEvDisconnected::TPtr&, const TActorContext& ctx) { + CloseSession("pipe to partition's tablet is dead", NPersQueue::NErrorCode::ERROR, ctx); +} + +void TWriteSessionActor::Handle(TEvTabletPipe::TEvClientConnected::TPtr& ev, const TActorContext& ctx) { + TEvTabletPipe::TEvClientConnected *msg = ev->Get(); + if (msg->Status != NKikimrProto::OK) { + CloseSession(TStringBuilder() << "pipe to tablet is dead " << msg->TabletId, NPersQueue::NErrorCode::ERROR, ctx); + return; + } +} + +void TWriteSessionActor::Handle(TEvTabletPipe::TEvClientDestroyed::TPtr& ev, const TActorContext& ctx) { + CloseSession(TStringBuilder() << "pipe to tablet is dead " << ev->Get()->TabletId, NPersQueue::NErrorCode::ERROR, ctx); +} + +void TWriteSessionActor::GenerateNextWriteRequest(const TActorContext& ctx) { + TWriteRequestBatchInfo::TPtr writeRequest = new TWriteRequestBatchInfo(); + + auto ev = MakeHolder<NPQ::TEvPartitionWriter::TEvWriteRequest>(++NextRequestCookie); + NKikimrClient::TPersQueueRequest& request = ev->Record; + + writeRequest->UserWriteRequests = std::move(Writes); + Writes.clear(); + + i64 diff = 0; + auto addData = [&](const TWriteRequest::TData& data) { + auto w = request.MutablePartitionRequest()->AddCmdWrite(); + w->SetData(GetSerializedData(InitMeta, data)); + w->SetClientDC(ClientDC); + w->SetSeqNo(data.GetSeqNo()); + w->SetSourceId(NPQ::NSourceIdEncoding::EncodeSimple(SourceId)); // EncodeSimple is needed for compatibility with LB + //TODO: add in SourceID clientId when TVM will be ready + w->SetCreateTimeMS(data.GetCreateTimeMs()); + w->SetUncompressedSize(data.GetUncompressedSize()); + }; + + for (const auto& write : writeRequest->UserWriteRequests) { + diff -= write->Request.ByteSize(); + if (write->Request.HasDataBatch()) { + for (const TWriteRequest::TData& data : write->Request.GetDataBatch().GetData()) { + addData(data); + } + } else { // single data + Y_VERIFY(write->Request.HasData()); + addData(write->Request.GetData()); + } + } + + writeRequest->Cookie = request.GetPartitionRequest().GetCookie(); + + Y_VERIFY(-diff <= (i64)BytesInflight_); + diff += request.ByteSize(); + BytesInflight_ += diff; + BytesInflightTotal_ += diff; + BytesInflight.Inc(diff); + BytesInflightTotal.Inc(diff); + + writeRequest->ByteSize = request.ByteSize(); + FormedWrites.push_back(writeRequest); + + ctx.Send(Writer, std::move(ev)); + ++NumReserveBytesRequests; +} + +TString TWriteSessionActor::CheckSupportedCodec(const ui32 codecId) { + TString err; + const auto& description = PQInfo->Description; + if (!description.GetPQTabletConfig().HasCodecs() || description.GetPQTabletConfig().GetCodecs().IdsSize() == 0) + return ""; + + Y_VERIFY(description.PartitionsSize() > 0); + for (const auto& codec : description.GetPQTabletConfig().GetCodecs().GetIds()) { + if (codecId == codec) { + return ""; + } + } + err = "Unsupported codec provided. Supported codecs for this topic are:"; + bool first = true; + for (const auto& codec : description.GetPQTabletConfig().GetCodecs().GetCodecs()) { + if (first) { + first = false; + } else { + err += ","; + } + err += " " + codec; + } + return err; +} + + +void TWriteSessionActor::Handle(TEvPQProxy::TEvWrite::TPtr& ev, const TActorContext& ctx) { + + RequestNotChecked = true; + + if (State != ES_INITED) { + //answer error + CloseSession("write in not inited session", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return; + } + + auto auth = ev->Get()->Request.GetCredentials(); + ev->Get()->Request.ClearCredentials(); + TString tmp; + Y_PROTOBUF_SUPPRESS_NODISCARD auth.SerializeToString(&tmp); + if (auth.GetCredentialsCase() != NPersQueueCommon::TCredentials::CREDENTIALS_NOT_SET && tmp != AuthStr) { + Auth = auth; + AuthStr = tmp; + ForceACLCheck = true; + } + auto dataCheck = [&](const TWriteRequest::TData& data) -> bool { + if (!data.GetSeqNo()) { + CloseSession("bad write request - SeqNo must be positive", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return false; + } + + if (data.GetData().empty()) { + CloseSession("bad write request - data must be non-empty", NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return false; + } + TString err = CheckSupportedCodec((ui32)data.GetCodec()); + if (!err.empty()) { + CloseSession(err, NPersQueue::NErrorCode::BAD_REQUEST, ctx); + return false; + } + + return true; + }; + if (ev->Get()->Request.HasDataBatch()) { + for (const auto& data : ev->Get()->Request.GetDataBatch().GetData()) { + if (!dataCheck(data)) { + return; + } + } + } else { + Y_VERIFY(ev->Get()->Request.HasData()); + if (!dataCheck(ev->Get()->Request.GetData())) { + return; + } + } + + THolder<TEvPQProxy::TEvWrite> event(ev->Release()); + Writes.push_back(std::move(event)); + + ui64 diff = Writes.back()->Request.ByteSize(); + BytesInflight_ += diff; + BytesInflightTotal_ += diff; + BytesInflight.Inc(diff); + BytesInflightTotal.Inc(diff); + + if (BytesInflight_ < MAX_BYTES_INFLIGHT) { //allow only one big request to be readed but not sended + Y_VERIFY(NextRequestInited); + Handler->ReadyForNextRead(); + } else { + NextRequestInited = false; + } + + if (NumReserveBytesRequests < MAX_RESERVE_REQUESTS_INFLIGHT) { + GenerateNextWriteRequest(ctx); + } +} + + +void TWriteSessionActor::HandlePoison(TEvPQProxy::TEvDieCommand::TPtr& ev, const TActorContext& ctx) { + CloseSession(ev->Get()->Reason, ev->Get()->ErrorCode, ctx); +} + + +void TWriteSessionActor::LogSession(const TActorContext& ctx) { + + LOG_INFO_S(ctx, NKikimrServices::PQ_WRITE_PROXY, "write session: cookie=" << Cookie << " sessionId=" << OwnerCookie + << " userAgent=\"" << UserAgent << "\" ip=" << PeerName << " proto=v0 " + << " topic=" << DiscoveryConverter->GetPrintableString() << " durationSec=" << (ctx.Now() - StartTime).Seconds()); + + LogSessionDeadline = ctx.Now() + TDuration::Hours(1) + TDuration::Seconds(rand() % 60); +} + +void TWriteSessionActor::HandleWakeup(const TActorContext& ctx) { + Y_VERIFY(State == ES_INITED); + ctx.Schedule(CHECK_ACL_DELAY, new TEvents::TEvWakeup()); + if (!ACLCheckInProgress && (ForceACLCheck || (ctx.Now() - LastACLCheckTimestamp > TDuration::Seconds(AppData(ctx)->PQConfig.GetACLRetryTimeoutSec()) && RequestNotChecked))) { + ForceACLCheck = false; + RequestNotChecked = false; + if (Auth.GetCredentialsCase() != NPersQueueCommon::TCredentials::CREDENTIALS_NOT_SET) { + ACLCheckInProgress = true; + auto* request = new TEvDescribeTopicsRequest({DiscoveryConverter}); + ctx.Send(SchemeCache, request); + } + } + if (!SourceIdUpdatesInflight && ctx.Now() - LastSourceIdUpdate > SOURCEID_UPDATE_PERIOD) { + SourceIdUpdatesInflight++; + Y_VERIFY(FullConverter); + auto ev = MakeUpdateSourceIdMetadataRequest(EncodedSourceId.Hash, FullConverter->GetPrimaryPath()); // Now Topic is a path + ctx.Send(NKqp::MakeKqpProxyID(ctx.SelfID.NodeId()), ev.Release()); + // Previously Topic contained legacy name with DC (rt3.dc1--acc--topic) + SourceIdUpdatesInflight++; + ev = MakeUpdateSourceIdMetadataRequest(EncodedSourceId.Hash, FullConverter->GetTopicForSrcId()); + ctx.Send(NKqp::MakeKqpProxyID(ctx.SelfID.NodeId()), ev.Release()); + } + if (ctx.Now() >= LogSessionDeadline) { + LogSession(ctx); + } +} + +} +} diff --git a/kikimr/yndx/grpc_services/persqueue/persqueue.cpp b/kikimr/yndx/grpc_services/persqueue/persqueue.cpp new file mode 100644 index 0000000000..43b406a279 --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/persqueue.cpp @@ -0,0 +1,59 @@ +#include "persqueue.h" +#include "grpc_pq_read.h" +#include "grpc_pq_write.h" + +#include <ydb/core/base/appdata.h> +#include <ydb/core/base/counters.h> + + +namespace NKikimr { +namespace NGRpcService { + +static const ui32 PersQueueWriteSessionsMaxCount = 1000000; +static const ui32 PersQueueReadSessionsMaxCount = 100000; + +TGRpcPersQueueService::TGRpcPersQueueService(NActors::TActorSystem *system, TIntrusivePtr<NMonitoring::TDynamicCounters> counters, const NActors::TActorId& schemeCache) + : ActorSystem(system) + , Counters(counters) + , SchemeCache(schemeCache) +{ } + +void TGRpcPersQueueService::InitService(grpc::ServerCompletionQueue *cq, NGrpc::TLoggerPtr) { + CQ = cq; + if (ActorSystem->AppData<TAppData>()->PQConfig.GetEnabled()) { + WriteService.reset(new NGRpcProxy::TPQWriteService(GetService(), CQ, ActorSystem, SchemeCache, Counters, PersQueueWriteSessionsMaxCount)); + WriteService->InitClustersUpdater(); + ReadService.reset(new NGRpcProxy::TPQReadService(this, CQ, ActorSystem, SchemeCache, Counters, PersQueueReadSessionsMaxCount)); + SetupIncomingRequests(); + } +} + +void TGRpcPersQueueService::SetGlobalLimiterHandle(NGrpc::TGlobalLimiter* limiter) { + Limiter = limiter; +} + +bool TGRpcPersQueueService::IncRequest() { + return Limiter->Inc(); +} + +void TGRpcPersQueueService::DecRequest() { + Limiter->Dec(); +} + +void TGRpcPersQueueService::SetupIncomingRequests() { + WriteService->SetupIncomingRequests(); + ReadService->SetupIncomingRequests(); +} + +void TGRpcPersQueueService::StopService() noexcept { + TGrpcServiceBase::StopService(); + if (WriteService.get() != nullptr) { + WriteService->StopService(); + } + if (ReadService.get() != nullptr) { + ReadService->StopService(); + } +} + +} // namespace NGRpcService +} // namespace NKikimr diff --git a/kikimr/yndx/grpc_services/persqueue/persqueue.h b/kikimr/yndx/grpc_services/persqueue/persqueue.h new file mode 100644 index 0000000000..267efa7a6d --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/persqueue.h @@ -0,0 +1,49 @@ +#pragma once + +#include <library/cpp/actors/core/actorsystem.h> + +#include <kikimr/yndx/api/grpc/persqueue.grpc.pb.h> + +#include <library/cpp/grpc/server/grpc_server.h> + + +namespace NKikimr { + +namespace NGRpcProxy { + class TPQWriteService; + class TPQReadService; +} + +namespace NGRpcService { + +class TGRpcPersQueueService + : public NGrpc::TGrpcServiceBase<NPersQueue::PersQueueService> +{ +public: + TGRpcPersQueueService(NActors::TActorSystem* system, TIntrusivePtr<NMonitoring::TDynamicCounters> counters, const NActors::TActorId& schemeCache); + + void InitService(grpc::ServerCompletionQueue* cq, NGrpc::TLoggerPtr logger) override; + void SetGlobalLimiterHandle(NGrpc::TGlobalLimiter* limiter) override; + void StopService() noexcept override; + + using NGrpc::TGrpcServiceBase<NPersQueue::PersQueueService>::GetService; + + bool IncRequest(); + void DecRequest(); + +private: + void SetupIncomingRequests(); + + NActors::TActorSystem* ActorSystem; + grpc::ServerCompletionQueue* CQ = nullptr; + + TIntrusivePtr<NMonitoring::TDynamicCounters> Counters; + NGrpc::TGlobalLimiter* Limiter = nullptr; + NActors::TActorId SchemeCache; + + std::shared_ptr<NGRpcProxy::TPQWriteService> WriteService; + std::shared_ptr<NGRpcProxy::TPQReadService> ReadService; +}; + +} // namespace NGRpcService +} // namespace NKikimr diff --git a/kikimr/yndx/grpc_services/persqueue/persqueue_compat_ut.cpp b/kikimr/yndx/grpc_services/persqueue/persqueue_compat_ut.cpp new file mode 100644 index 0000000000..7b9b117bcd --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/persqueue_compat_ut.cpp @@ -0,0 +1,122 @@ +#include <kikimr/persqueue/sdk/deprecated/cpp/v2/ut_utils/test_pqlib.h> +#include <kikimr/persqueue/sdk/deprecated/cpp/v2/ut_utils/data_writer.h> +#include <kikimr/persqueue/sdk/deprecated/cpp/v2/types.h> + +namespace NKikimr::NPersQueueTests { + +using namespace NPersQueue::NTests; + +class TPQv0CompatTestBase { +public: + THolder<TTestPQLib> PQLib; + THolder<::NPersQueue::TTestServer> Server; + TString OriginalLegacyName1; + TString OriginalModernName1; + TString MirroredLegacyName1; + TString MirroredModernName1; + TString ShortName1; + + TString OriginalLegacyName2; + TString OriginalModernName2; + TString MirroredLegacyName2; + TString MirroredModernName2; + TString ShortName2; + +public: + TPQv0CompatTestBase() + { + Server = MakeHolder<::NPersQueue::TTestServer>(false); + Server->ServerSettings.PQConfig.MutablePQDiscoveryConfig()->SetLbUserDatabaseRoot("/Root/LB"); + Server->ServerSettings.PQConfig.SetCheckACL(false); + Server->StartServer(); + Server->EnableLogs({ NKikimrServices::KQP_PROXY }, NActors::NLog::PRI_EMERG); + Server->EnableLogs({ NKikimrServices::PERSQUEUE }, NActors::NLog::PRI_INFO); + Server->EnableLogs({ NKikimrServices::PQ_METACACHE }, NActors::NLog::PRI_DEBUG); + OriginalLegacyName1 = "rt3.dc1--account--topic1"; + MirroredLegacyName1 = "rt3.dc2--account--topic1"; + OriginalModernName1 = "/Root/LB/account/topic1"; + MirroredModernName1 = "/Root/LB/account/.topic2/mirrored-from-dc2"; + ShortName1 = "account/topic1"; + + OriginalLegacyName2 = "rt3.dc1--account--topic2"; + MirroredLegacyName2 = "rt3.dc2--account--topic2"; + OriginalModernName2 = "/Root/LB/account/topic2"; + MirroredModernName2 = "/Root/LB/account/.topic2/mirrored-from-dc2"; + ShortName2 = "account/topic2"; + + Server->AnnoyingClient->CreateTopicNoLegacy(OriginalLegacyName1, 1, false); + Server->AnnoyingClient->CreateTopicNoLegacy(MirroredLegacyName1, 1, false); + Server->AnnoyingClient->CreateTopicNoLegacy(OriginalModernName2, 1, true, true, "dc1"); + Server->AnnoyingClient->CreateTopicNoLegacy(MirroredModernName2, 1, true, true, "dc2"); + Server->AnnoyingClient->CreateConsumer("test-consumer"); + InitPQLib(); + } + void InitPQLib() { + PQLib = MakeHolder<TTestPQLib>(*Server); + TPQDataWriter writer{OriginalLegacyName1, ShortName1, "test", *Server}; + writer.WaitWritePQServiceInitialization(); + }; +}; + +Y_UNIT_TEST_SUITE(TPQCompatTest) { + Y_UNIT_TEST(DiscoverTopics) { + TPQv0CompatTestBase testServer; + Cerr << "Create producer\n"; + { + auto [producer, res] = testServer.PQLib->CreateProducer(testServer.ShortName2, "123", {}, ::NPersQueue::ECodec::RAW); + Cerr << "Got response: " << res.Response.ShortDebugString() << Endl; + UNIT_ASSERT(res.Response.HasInit()); + } + Cerr << "Create producer(2)\n"; + { + auto [producer, res] = testServer.PQLib->CreateProducer(testServer.ShortName1, "123", {}, ::NPersQueue::ECodec::RAW); + UNIT_ASSERT(res.Response.HasInit()); + } + } + + Y_UNIT_TEST(SetupLockSession) { + TPQv0CompatTestBase server{}; + auto [consumer, startResult] = server.PQLib->CreateConsumer({server.ShortName1}, "test-consumer", 1, true); + Cerr << startResult.Response << "\n"; + for (ui32 i = 0; i < 2; ++i) { + auto msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << "Response: " << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasLock()); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetTopic() == server.OriginalLegacyName1 + || msg.GetValue().Response.GetLock().GetTopic() == server.MirroredLegacyName1); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetPartition() == 0); + } + auto msg = consumer->GetNextMessage(); + UNIT_ASSERT(!msg.Wait(TDuration::Seconds(1))); + server.Server->AnnoyingClient->AlterTopic(server.MirroredLegacyName1, 2); + msg.Wait(); + UNIT_ASSERT(msg.GetValue().Response.HasLock()); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetTopic() == server.MirroredLegacyName1); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetPartition() == 1); + } + + Y_UNIT_TEST(LegacyRequests) { + TPQv0CompatTestBase server{}; + server.Server->AnnoyingClient->GetPartOffset( + { + {server.OriginalLegacyName1, {0}}, + {server.MirroredLegacyName1, {0}}, + {server.OriginalLegacyName2, {0}}, + {server.MirroredLegacyName2, {0}}, + }, + 4, 0, true + ); + server.Server->AnnoyingClient->SetClientOffsetPQ(server.OriginalLegacyName2, 0, 5); + server.Server->AnnoyingClient->SetClientOffsetPQ(server.MirroredLegacyName2, 0, 5); + + server.Server->AnnoyingClient->GetPartOffset( + { + {server.OriginalLegacyName2, {0}}, + {server.MirroredLegacyName2, {0}}, + }, + 2, 2, true + ); + } +} +} //namespace NKikimr::NPersQueueTests; diff --git a/kikimr/yndx/grpc_services/persqueue/persqueue_ut.cpp b/kikimr/yndx/grpc_services/persqueue/persqueue_ut.cpp new file mode 100644 index 0000000000..6a899a684d --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/persqueue_ut.cpp @@ -0,0 +1,2405 @@ +#include "ut/definitions.h" +#include <ydb/core/base/appdata.h> +#include <ydb/core/testlib/test_pq_client.h> +#include <ydb/core/client/server/grpc_proxy_status.h> +#include <ydb/core/protos/grpc_pq_old.pb.h> +#include <ydb/core/persqueue/writer/source_id_encoding.h> +#include <kikimr/persqueue/sdk/deprecated/cpp/v2/ut_utils/test_pqlib.h> +#include <kikimr/persqueue/sdk/deprecated/cpp/v2/ut_utils/test_server.h> + +#include <ydb/library/aclib/aclib.h> +#include <ydb/library/persqueue/tests/counters.h> + +#include <library/cpp/testing/unittest/tests_data.h> +#include <library/cpp/testing/unittest/registar.h> +#include <library/cpp/http/io/stream.h> +#include <google/protobuf/text_format.h> + +#include <util/string/join.h> +#include <util/string/builder.h> + +#include <grpc++/client_context.h> +#include <grpc++/create_channel.h> + +#include <kikimr/persqueue/sdk/deprecated/cpp/v2/persqueue.h> + +namespace NKikimr { +namespace NPersQueueTests { + +using namespace Tests; +using namespace NKikimrClient; +using namespace NPersQueue; +using namespace NPersQueue::NTests; +using namespace NThreading; +using namespace NNetClassifier; + +static TString FormNetData() { + return "10.99.99.224/32\tSAS\n" + "::1/128\tVLA\n"; +} + +TAutoPtr<IEventHandle> GetClassifierUpdate(TServer& server, const TActorId sender) { + auto& actorSystem = *server.GetRuntime(); + actorSystem.Send( + new IEventHandle(MakeNetClassifierID(), sender, + new TEvNetClassifier::TEvSubscribe() + )); + + TAutoPtr<IEventHandle> handle; + actorSystem.GrabEdgeEvent<NNetClassifier::TEvNetClassifier::TEvClassifierUpdate>(handle); + + UNIT_ASSERT(handle); + UNIT_ASSERT_VALUES_EQUAL(handle->Recipient, sender); + + return handle; +} + +THolder<TTempFileHandle> CreateNetDataFile(const TString& content) { + auto netDataFile = MakeHolder<TTempFileHandle>("data.tsv"); + + netDataFile->Write(content.Data(), content.Size()); + netDataFile->FlushData(); + + return netDataFile; +} + + +Y_UNIT_TEST_SUITE(TPersQueueTest2) { + void PrepareForGrpcNoDC(TFlatMsgBusPQClient& annoyingClient) { + annoyingClient.SetNoConfigMode(); + annoyingClient.FullInit(); + annoyingClient.InitUserRegistry(); + annoyingClient.MkDir("/Root", "account1"); + annoyingClient.MkDir("/Root/PQ", "account1"); + annoyingClient.CreateTopicNoLegacy("/Root/PQ/rt3.db--topic1", 5, false); + annoyingClient.CreateTopicNoLegacy("/Root/PQ/account1/topic1", 5, false, true, Nothing(), {"user1", "user2"}); + annoyingClient.CreateTopicNoLegacy("/Root/account2/topic2", 5); + } + Y_UNIT_TEST(TestGrpcWriteNoDC) { + TTestServer server(false); + server.ServerSettings.PQConfig.SetTopicsAreFirstClassCitizen(true); + server.ServerSettings.PQConfig.SetRoot("/Rt2/PQ"); + server.ServerSettings.PQConfig.SetDatabase("/Root"); + server.StartServer(); + + UNIT_ASSERT_VALUES_EQUAL(NMsgBusProxy::MSTATUS_OK, + server.AnnoyingClient->AlterUserAttributes("/", "Root", {{"folder_id", "somefolder"}, {"cloud_id", "somecloud"}, {"database_id", "root"}})); + + + PrepareForGrpcNoDC(*server.AnnoyingClient); + auto writer = MakeDataWriter(server, "source1"); + + writer.Write("/Root/account2/topic2", {"valuevaluevalue1"}, true, "topic1@" BUILTIN_ACL_DOMAIN); + writer.Write("/Root/PQ/account1/topic1", {"valuevaluevalue1"}, true, "topic1@" BUILTIN_ACL_DOMAIN); + + NACLib::TDiffACL acl; + acl.AddAccess(NACLib::EAccessType::Allow, NACLib::UpdateRow, "topic1@" BUILTIN_ACL_DOMAIN); + acl.AddAccess(NACLib::EAccessType::Allow, NACLib::SelectRow, "topic1@" BUILTIN_ACL_DOMAIN); + + server.AnnoyingClient->ModifyACL("/Root/account2", "topic2", acl.SerializeAsString()); + server.AnnoyingClient->ModifyACL("/Root/PQ/account1", "topic1", acl.SerializeAsString()); + + Sleep(TDuration::Seconds(5)); + + writer.Write("/Root/account2/topic2", {"valuevaluevalue1"}, false, "topic1@" BUILTIN_ACL_DOMAIN); + + writer.Write("/Root/PQ/account1/topic1", {"valuevaluevalue1"}, false, "topic1@" BUILTIN_ACL_DOMAIN); + writer.Write("/Root/PQ/account1/topic1", {"valuevaluevalue2"}, false, "topic1@" BUILTIN_ACL_DOMAIN); + + writer.Read("/Root/PQ/account1/topic1", "user1", "topic1@" BUILTIN_ACL_DOMAIN, false, false, false, true); + + writer.Write("Root/PQ/account1/topic1", {"valuevaluevalue2"}, false, "topic1@" BUILTIN_ACL_DOMAIN); //TODO /Root remove + + writer.Read("Root/PQ/account1/topic1", "user1", "topic1@" BUILTIN_ACL_DOMAIN, false, false, false, true); + } +} +Y_UNIT_TEST_SUITE(TPersQueueTest) { + + Y_UNIT_TEST(SetupLockSession2) { + TTestServer server(false); + server.GrpcServerOptions.SetMaxMessageSize(130_MB); + server.StartServer(); + server.EnableLogs({ NKikimrServices::PQ_READ_PROXY }); + server.EnableLogs({ NKikimrServices::PERSQUEUE }, NActors::NLog::PRI_INFO); + server.AnnoyingClient->CreateTopic("rt3.dc1--acc--topic1", 1); + server.AnnoyingClient->CreateTopic("rt3.dc2--acc--topic1", 1); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1); + server.AnnoyingClient->CreateConsumer("user"); + + auto writer = MakeDataWriter(server); + + TTestPQLib PQLib(server); + auto [consumer, startResult] = PQLib.CreateConsumer({"acc/topic1"}, "user", 1, true); + Cerr << startResult.Response << "\n"; + for (ui32 i = 0; i < 2; ++i) { + auto msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << "Response: " << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasLock()); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetTopic() == "rt3.dc1--acc--topic1" || msg.GetValue().Response.GetLock().GetTopic() == "rt3.dc2--acc--topic1"); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetPartition() == 0); + } + auto msg = consumer->GetNextMessage(); + UNIT_ASSERT(!msg.Wait(TDuration::Seconds(1))); + server.AnnoyingClient->AlterTopic("rt3.dc2--acc--topic1", 2); + msg.Wait(); + UNIT_ASSERT(msg.GetValue().Response.HasLock()); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetTopic() == "rt3.dc2--acc--topic1"); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetPartition() == 1); + } + + + + Y_UNIT_TEST(SetupLockSession) { + TTestServer server; + + server.EnableLogs({ NKikimrServices::PQ_READ_PROXY }); + server.EnableLogs({ NKikimrServices::PERSQUEUE }, NActors::NLog::PRI_INFO); + + server.AnnoyingClient->CreateTopic("rt3.dc1--acc--topic1", 1); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1); + server.AnnoyingClient->CreateConsumer("user"); + + auto writer = MakeDataWriter(server); + + std::shared_ptr<grpc::Channel> Channel_; + std::unique_ptr<NKikimrClient::TGRpcServer::Stub> Stub_; + std::unique_ptr<NPersQueue::PersQueueService::Stub> StubP_; + + Channel_ = grpc::CreateChannel("localhost:" + ToString(server.GrpcPort), grpc::InsecureChannelCredentials()); + Stub_ = NKikimrClient::TGRpcServer::NewStub(Channel_); + + ui64 proxyCookie = 0; + + { + grpc::ClientContext context; + NKikimrClient::TChooseProxyRequest request; + NKikimrClient::TResponse response; + auto status = Stub_->ChooseProxy(&context, request, &response); + UNIT_ASSERT(status.ok()); + Cerr << response << "\n"; + UNIT_ASSERT(response.GetStatus() == NMsgBusProxy::MSTATUS_OK); + proxyCookie = response.GetProxyCookie(); + Channel_ = grpc::CreateChannel("[" + response.GetProxyName() + "]:" + ToString(server.GrpcPort), grpc::InsecureChannelCredentials()); + StubP_ = NPersQueue::PersQueueService::NewStub(Channel_); + } + + + grpc::ClientContext rcontext; + auto readStream = StubP_->ReadSession(&rcontext); + UNIT_ASSERT(readStream); + + // init read session + { + TReadRequest req; + TReadResponse resp; + + req.MutableInit()->AddTopics("acc/topic1"); + + req.MutableInit()->SetClientId("user"); + req.MutableInit()->SetClientsideLocksAllowed(true); + req.MutableInit()->SetProxyCookie(proxyCookie); + req.MutableInit()->SetProtocolVersion(TReadRequest::ReadParamsInInit); + req.MutableInit()->SetMaxReadMessagesCount(3); + + if (!readStream->Write(req)) { + ythrow yexception() << "write fail"; + } + UNIT_ASSERT(readStream->Read(&resp)); + UNIT_ASSERT(resp.HasInit()); + //send some reads + req.Clear(); + req.MutableRead(); + for (ui32 i = 0; i < 10; ++i) { + if (!readStream->Write(req)) { + ythrow yexception() << "write fail"; + } + } + } + + { + TReadRequest req; + TReadResponse resp; + + //lock partition + UNIT_ASSERT(readStream->Read(&resp)); + UNIT_ASSERT(resp.HasLock()); + UNIT_ASSERT_VALUES_EQUAL(resp.GetLock().GetTopic(), "rt3.dc1--acc--topic1"); + UNIT_ASSERT(resp.GetLock().GetPartition() == 0); + + req.Clear(); + req.MutableStartRead()->SetTopic(resp.GetLock().GetTopic()); + req.MutableStartRead()->SetPartition(resp.GetLock().GetPartition()); + req.MutableStartRead()->SetReadOffset(10); + req.MutableStartRead()->SetGeneration(resp.GetLock().GetGeneration()); + if (!readStream->Write(req)) { + ythrow yexception() << "write fail"; + } + + } + + //Write some data + writer.Write("acc/topic1", "valuevaluevalue1"); + Sleep(TDuration::Seconds(15)); //force wait data + writer.Write("acc/topic1", "valuevaluevalue2"); + writer.Write("acc/topic1", "valuevaluevalue3"); + writer.Write("acc/topic1", "valuevaluevalue4"); + + //check read results + TReadResponse resp; + for (ui32 i = 10; i < 16; ++i) { + UNIT_ASSERT(readStream->Read(&resp)); + UNIT_ASSERT_C(resp.HasBatchedData(), resp); + UNIT_ASSERT(resp.GetBatchedData().PartitionDataSize() == 1); + UNIT_ASSERT(resp.GetBatchedData().GetPartitionData(0).BatchSize() == 1); + UNIT_ASSERT(resp.GetBatchedData().GetPartitionData(0).GetBatch(0).MessageDataSize() == 1); + UNIT_ASSERT(resp.GetBatchedData().GetPartitionData(0).GetBatch(0).GetMessageData(0).GetOffset() == i); + } + //TODO: restart here readSession and read from position 10 + { + TReadRequest req; + TReadResponse resp; + + req.MutableCommit()->AddCookie(1); + + if (!readStream->Write(req)) { + ythrow yexception() << "write fail"; + } + UNIT_ASSERT(readStream->Read(&resp)); + UNIT_ASSERT(resp.HasCommit()); + } + } + + + void SetupWriteSessionImpl(bool rr) { + TTestServer server(PQSettings(0, 2, rr)); + + server.EnableLogs({ NKikimrServices::PQ_WRITE_PROXY }); + + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 10); + auto writer = MakeDataWriter(server); + + ui32 p = writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue1"); + + server.AnnoyingClient->AlterTopic(DEFAULT_TOPIC_NAME, 15); + + ui32 pp = writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue2"); + UNIT_ASSERT_VALUES_EQUAL(p, pp); + + writer.WriteBatch(SHORT_TOPIC_NAME, {"1", "2", "3", "4", "5"}); + + writer.Write("topic2", "valuevaluevalue1", true); + + p = writer.InitSession("sid1", 2, true); + pp = writer.InitSession("sid1", 0, true); + + UNIT_ASSERT(p = pp); + UNIT_ASSERT(p == 1); + + { + p = writer.InitSession("sidx", 0, true); + pp = writer.InitSession("sidx", 0, true); + + UNIT_ASSERT(p == pp); + } + + writer.InitSession("sid1", 3, false); + + //check round robin; + TMap<ui32, ui32> ss; + for (ui32 i = 0; i < 15*5; ++i) { + ss[writer.InitSession("sid_rand_" + ToString<ui32>(i), 0, true)]++; + } + for (auto &s : ss) { + Cerr << s.first << " " << s.second << "\n"; + if (rr) { + UNIT_ASSERT(s.second >= 4 && s.second <= 6); + } + } + } + + Y_UNIT_TEST(SetupWriteSession) { + SetupWriteSessionImpl(false); + SetupWriteSessionImpl(true); + } + + Y_UNIT_TEST(SetupWriteSessionOnDisabledCluster) { + TTestServer server; + server.EnableLogs({ NKikimrServices::PERSQUEUE, NKikimrServices::PQ_WRITE_PROXY}); + + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 10); + + auto writer = MakeDataWriter(server); + + server.AnnoyingClient->DisableDC(); + + Sleep(TDuration::Seconds(5)); + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue1", true); + } + + Y_UNIT_TEST(CloseActiveWriteSessionOnClusterDisable) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 10); + server.EnableLogs({ NKikimrServices::PQ_WRITE_PROXY }); + + auto writer = MakeDataWriter(server); + + TTestPQLib PQLib(server); + auto [producer, res] = PQLib.CreateProducer(SHORT_TOPIC_NAME, "123", {}, ECodec::RAW); + + NThreading::TFuture<NPersQueue::TError> isDead = producer->IsDead(); + server.AnnoyingClient->DisableDC(); + isDead.Wait(); + UNIT_ASSERT_EQUAL(isDead.GetValue().GetCode(), NPersQueue::NErrorCode::CLUSTER_DISABLED); + } + + Y_UNIT_TEST(BadSids) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 10); + server.EnableLogs({ NKikimrServices::PQ_WRITE_PROXY }); + + auto writer = MakeDataWriter(server); + TTestPQLib PQLib(server); + auto runSidTest = [&](const TString& srcId, bool shouldFail = true) { + auto[producer, res] = PQLib.CreateProducer(SHORT_TOPIC_NAME, srcId); + if (shouldFail) { + UNIT_ASSERT(res.Response.HasError()); + } else { + UNIT_ASSERT(res.Response.HasInit()); + } + }; + + runSidTest("base64:a***"); + runSidTest("base64:aa=="); + runSidTest("base64:a"); + runSidTest("base64:aa", false); + } + + Y_UNIT_TEST(ReadFromSeveralPartitions) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 10); + auto writer = MakeDataWriter(server, "source1"); + + std::shared_ptr<grpc::Channel> Channel_; + std::unique_ptr<NKikimrClient::TGRpcServer::Stub> Stub_; + std::unique_ptr<NPersQueue::PersQueueService::Stub> StubP_; + + Channel_ = grpc::CreateChannel("localhost:" + ToString(server.GrpcPort), grpc::InsecureChannelCredentials()); + Stub_ = NKikimrClient::TGRpcServer::NewStub(Channel_); + + ui64 proxyCookie = 0; + + { + grpc::ClientContext context; + NKikimrClient::TChooseProxyRequest request; + NKikimrClient::TResponse response; + auto status = Stub_->ChooseProxy(&context, request, &response); + UNIT_ASSERT(status.ok()); + Cerr << response << "\n"; + UNIT_ASSERT(response.GetStatus() == NMsgBusProxy::MSTATUS_OK); + proxyCookie = response.GetProxyCookie(); + Channel_ = grpc::CreateChannel("[" + response.GetProxyName() + "]:" + ToString(server.GrpcPort), grpc::InsecureChannelCredentials()); + StubP_ = NPersQueue::PersQueueService::NewStub(Channel_); + } + + + //Write some data + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue1"); + + auto writer2 = MakeDataWriter(server, "source2"); + writer2.Write(SHORT_TOPIC_NAME, "valuevaluevalue2"); + + grpc::ClientContext rcontext; + auto readStream = StubP_->ReadSession(&rcontext); + UNIT_ASSERT(readStream); + + // init read session + { + TReadRequest req; + TReadResponse resp; + + req.MutableInit()->AddTopics(SHORT_TOPIC_NAME); + + req.MutableInit()->SetClientId("user"); + req.MutableInit()->SetProxyCookie(proxyCookie); + req.MutableInit()->SetProtocolVersion(TReadRequest::ReadParamsInInit); + req.MutableInit()->SetMaxReadMessagesCount(1000); + + if (!readStream->Write(req)) { + ythrow yexception() << "write fail"; + } + UNIT_ASSERT(readStream->Read(&resp)); + UNIT_ASSERT(resp.HasInit()); + + //send some reads + Sleep(TDuration::Seconds(5)); + for (ui32 i = 0; i < 10; ++i) { + req.Clear(); + req.MutableRead(); + + if (!readStream->Write(req)) { + ythrow yexception() << "write fail"; + } + } + } + + //check read results + TReadResponse resp; + for (ui32 i = 0; i < 1; ++i) { + UNIT_ASSERT(readStream->Read(&resp)); + UNIT_ASSERT_C(resp.HasBatchedData(), resp); + UNIT_ASSERT(resp.GetBatchedData().PartitionDataSize() == 2); + } + } + + + void SetupReadSessionTest(bool useBatching) { + TTestServer server; + server.EnableLogs({ NKikimrServices::PQ_READ_PROXY }); + + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 2); + server.AnnoyingClient->CreateTopic("rt3.dc2--topic1", 2); + + auto writer = MakeDataWriter(server, "source1"); + + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue0"); + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue1"); + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue2"); + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue3"); + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue4"); + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue5"); + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue6"); + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue7"); + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue8"); + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue9"); + + writer.Read(SHORT_TOPIC_NAME, "user", "", false, false, useBatching); + } + + Y_UNIT_TEST(SetupReadSession) { + SetupReadSessionTest(false); + } + + Y_UNIT_TEST(SetupReadSessionWithBatching) { + SetupReadSessionTest(true); + } + + void ClosesSessionOnReadSettingsChangeTest(bool initReadSettingsInInitRequest) { + TTestServer server; + server.EnableLogs({ NKikimrServices::PQ_READ_PROXY }); + + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 2); + server.AnnoyingClient->CreateTopic("rt3.dc2--topic1", 2); + + auto writer = MakeDataWriter(server, "source1"); + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue0"); + + // Reading code + std::shared_ptr<grpc::Channel> Channel_; + std::unique_ptr<NKikimrClient::TGRpcServer::Stub> Stub_; + std::unique_ptr<NPersQueue::PersQueueService::Stub> StubP_; + + Channel_ = grpc::CreateChannel("localhost:" + ToString(server.GrpcPort), grpc::InsecureChannelCredentials()); + Stub_ = NKikimrClient::TGRpcServer::NewStub(Channel_); + + ui64 proxyCookie = 0; + + { + grpc::ClientContext context; + NKikimrClient::TChooseProxyRequest request; + NKikimrClient::TResponse response; + auto status = Stub_->ChooseProxy(&context, request, &response); + UNIT_ASSERT(status.ok()); + Cerr << response << "\n"; + UNIT_ASSERT(response.GetStatus() == NMsgBusProxy::MSTATUS_OK); + proxyCookie = response.GetProxyCookie(); + Channel_ = grpc::CreateChannel("[" + response.GetProxyName() + "]:" + ToString(server.GrpcPort), grpc::InsecureChannelCredentials()); + StubP_ = NPersQueue::PersQueueService::NewStub(Channel_); + } + + grpc::ClientContext rcontext; + auto readStream = StubP_->ReadSession(&rcontext); + UNIT_ASSERT(readStream); + + // init read session + { + TReadRequest req; + TReadResponse resp; + + req.MutableInit()->AddTopics(SHORT_TOPIC_NAME); + + req.MutableInit()->SetClientId("user"); + req.MutableInit()->SetProxyCookie(proxyCookie); + if (initReadSettingsInInitRequest) { + req.MutableInit()->SetProtocolVersion(TReadRequest::ReadParamsInInit); + req.MutableInit()->SetMaxReadMessagesCount(1); + } + + if (!readStream->Write(req)) { + ythrow yexception() << "write fail"; + } + UNIT_ASSERT(readStream->Read(&resp)); + UNIT_ASSERT(resp.HasInit()); + + if (!initReadSettingsInInitRequest) { + // send first read + req.Clear(); + req.MutableRead()->SetMaxCount(1); + if (!readStream->Write(req)) { + ythrow yexception() << "write fail"; + } + UNIT_ASSERT(readStream->Read(&resp)); + UNIT_ASSERT_C(resp.HasData(), resp); + } + + // change settings + req.Clear(); + req.MutableRead()->SetMaxCount(42); + if (!readStream->Write(req)) { + ythrow yexception() << "write fail"; + } + UNIT_ASSERT(readStream->Read(&resp)); + UNIT_ASSERT(resp.HasError()); + } + } + + Y_UNIT_TEST(WriteSessionClose) { + + TTestServer server; + server.EnableLogs({ NKikimrServices::PQ_WRITE_PROXY }); + + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 2); + server.AnnoyingClient->CreateTopic("rt3.dc2--topic1", 2); + + auto writer = MakeDataWriter(server, "source1"); + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue0"); + + // Reading code + std::shared_ptr<grpc::Channel> Channel_; + std::unique_ptr<NKikimrClient::TGRpcServer::Stub> Stub_; + std::unique_ptr<NPersQueue::PersQueueService::Stub> StubP_; + + Channel_ = grpc::CreateChannel("localhost:" + ToString(server.GrpcPort), grpc::InsecureChannelCredentials()); + Stub_ = NKikimrClient::TGRpcServer::NewStub(Channel_); + + ui64 proxyCookie = 0; + + { + grpc::ClientContext context; + NKikimrClient::TChooseProxyRequest request; + NKikimrClient::TResponse response; + auto status = Stub_->ChooseProxy(&context, request, &response); + UNIT_ASSERT(status.ok()); + Cerr << response << "\n"; + UNIT_ASSERT(response.GetStatus() == NMsgBusProxy::MSTATUS_OK); + proxyCookie = response.GetProxyCookie(); + Channel_ = grpc::CreateChannel("[" + response.GetProxyName() + "]:" + ToString(server.GrpcPort), grpc::InsecureChannelCredentials()); + StubP_ = NPersQueue::PersQueueService::NewStub(Channel_); + } + + // init write session + for (ui32 i = 0; i < 2; ++i){ + grpc::ClientContext rcontext; + + auto writeStream = StubP_->WriteSession(&rcontext); + UNIT_ASSERT(writeStream); + + TWriteRequest req; + + req.MutableInit()->SetTopic(SHORT_TOPIC_NAME); + + req.MutableInit()->SetSourceId("user"); + req.MutableInit()->SetProxyCookie(proxyCookie); + if (i == 0) + continue; + if (!writeStream->Write(req)) { + ythrow yexception() << "write fail"; + } + } + } + + Y_UNIT_TEST(ClosesSessionOnReadSettingsChange) { + ClosesSessionOnReadSettingsChangeTest(false); + } + + Y_UNIT_TEST(ClosesSessionOnReadSettingsChangeWithInit) { + ClosesSessionOnReadSettingsChangeTest(true); + } + + Y_UNIT_TEST(WriteExisting) { + TTestServer server; + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD }); + + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 2); + + { + THolder<NMsgBusProxy::TBusPersQueue> request = TRequestDescribePQ().GetRequest({}); + + NKikimrClient::TResponse response; + + auto channel = grpc::CreateChannel("localhost:"+ToString(server.GrpcPort), grpc::InsecureChannelCredentials()); + auto stub(NKikimrClient::TGRpcServer::NewStub(channel)); + grpc::ClientContext context; + auto status = stub->PersQueueRequest(&context, request->Record, &response); + + UNIT_ASSERT(status.ok()); + } + + server.AnnoyingClient->WriteToPQ( + DEFAULT_TOPIC_NAME, 1, "abacaba", 1, "valuevaluevalue1", "", ETransport::GRpc + ); + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 1, "abacaba", 2, "valuevaluevalue1", "", ETransport::GRpc); + } + + Y_UNIT_TEST(WriteExistingBigValue) { + TTestServer server(PQSettings(0, 2).SetDomainName("Root")); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 2, 8_MB, 86400, 100000); + + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD }); + + TInstant now(Now()); + + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 1, "abacaba", 1, TString(1000000, 'a')); + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 1, "abacaba", 2, TString(1, 'a')); + UNIT_ASSERT(TInstant::Now() - now > TDuration::MilliSeconds(5990)); //speed limit is 200kb/s and burst is 200kb, so to write 1mb it will take at least 4 seconds + } + + Y_UNIT_TEST(WriteEmptyData) { + TTestServer server(PQSettings(0, 2).SetDomainName("Root")); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 2); + + // empty data and sourecId + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 1, "", 1, "", "", ETransport::MsgBus, NMsgBusProxy::MSTATUS_ERROR); + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 1, "a", 1, "", "", ETransport::MsgBus, NMsgBusProxy::MSTATUS_ERROR); + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 1, "", 1, "a", "", ETransport::MsgBus, NMsgBusProxy::MSTATUS_ERROR); + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 1, "a", 1, "a", "", ETransport::MsgBus, NMsgBusProxy::MSTATUS_OK); + } + + + Y_UNIT_TEST(WriteNonExistingPartition) { + TTestServer server(PQSettings(0, 2).SetDomainName("Root")); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 2); + + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD }); + + server.AnnoyingClient->WriteToPQ( + DEFAULT_TOPIC_NAME, 100500, "abacaba", 1, "valuevaluevalue1", "", + ETransport::MsgBus, NMsgBusProxy::MSTATUS_ERROR, NMsgBusProxy::MSTATUS_ERROR + ); + } + + Y_UNIT_TEST(WriteNonExistingTopic) { + TTestServer server(PQSettings(0, 2).SetDomainName("Root")); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 2); + + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD }); + + server.AnnoyingClient->WriteToPQ("rt3.dc1--topic1000", 1, "abacaba", 1, "valuevaluevalue1", "", ETransport::MsgBus, NMsgBusProxy::MSTATUS_ERROR, NMsgBusProxy::MSTATUS_ERROR); + } + + Y_UNIT_TEST(SchemeshardRestart) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 2); + TString topic2 = "rt3.dc1--topic2"; + server.AnnoyingClient->CreateTopic(topic2, 2); + + // force topic1 into cache and establish pipe from cache to schemeshard + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 1, "abacaba", 1, "valuevaluevalue1"); + + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD, NKikimrServices::PQ_METACACHE }); + + server.AnnoyingClient->RestartSchemeshard(server.CleverServer->GetRuntime()); + server.AnnoyingClient->WriteToPQ(topic2, 1, "abacaba", 1, "valuevaluevalue1"); + } + + Y_UNIT_TEST(WriteAfterAlter) { + TTestServer server(PQSettings(0).SetDomainName("Root").SetNodeCount(2)); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 2); + + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD, NKikimrServices::PQ_METACACHE }); + + + server.AnnoyingClient->WriteToPQ( + DEFAULT_TOPIC_NAME, 5, "abacaba", 1, "valuevaluevalue1", "", + ETransport::MsgBus, NMsgBusProxy::MSTATUS_ERROR, NMsgBusProxy::MSTATUS_ERROR + ); + + server.AnnoyingClient->AlterTopic(DEFAULT_TOPIC_NAME, 10); + Sleep(TDuration::Seconds(1)); + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 5, "abacaba", 1, "valuevaluevalue1"); + server.AnnoyingClient->WriteToPQ( + DEFAULT_TOPIC_NAME, 15, "abacaba", 1, "valuevaluevalue1", "", + ETransport::MsgBus, NMsgBusProxy::MSTATUS_ERROR, NMsgBusProxy::MSTATUS_ERROR + ); + + server.AnnoyingClient->AlterTopic(DEFAULT_TOPIC_NAME, 20); + Sleep(TDuration::Seconds(1)); + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 5, "abacaba", 1, "valuevaluevalue1"); + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 15, "abacaba", 1, "valuevaluevalue1"); + } + + Y_UNIT_TEST(Delete) { + TTestServer server(PQSettings(0).SetDomainName("Root").SetNodeCount(2)); + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD }); + + // Delete non-existing + server.AnnoyingClient->DeleteTopic2(DEFAULT_TOPIC_NAME, NPersQueue::NErrorCode::UNKNOWN_TOPIC); + + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 2); + + // Delete existing + server.AnnoyingClient->DeleteTopic2(DEFAULT_TOPIC_NAME); + + // Double delete - "What Is Dead May Never Die" + server.AnnoyingClient->DeleteTopic2(DEFAULT_TOPIC_NAME, NPersQueue::NErrorCode::UNKNOWN_TOPIC); + + // Resurrect deleted topic + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 2); + server.AnnoyingClient->DeleteTopic2(DEFAULT_TOPIC_NAME); + } + + Y_UNIT_TEST(WriteAfterDelete) { + TTestServer server(PQSettings(0).SetDomainName("Root").SetNodeCount(2)); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 3); + + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 1, "abacaba", 1, "valuevaluevalue1"); + + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD, NKikimrServices::PQ_METACACHE }); + + server.AnnoyingClient->DeleteTopic2(DEFAULT_TOPIC_NAME); + server.AnnoyingClient->WriteToPQ( + DEFAULT_TOPIC_NAME, 1, "abacaba", 2, "valuevaluevalue1", "", + ETransport::MsgBus, NMsgBusProxy::MSTATUS_ERROR, NMsgBusProxy::MSTATUS_ERROR + ); + server.AnnoyingClient->WriteToPQ( + DEFAULT_TOPIC_NAME, 2, "abacaba", 1, "valuevaluevalue1", "", + ETransport::MsgBus, NMsgBusProxy::MSTATUS_ERROR, NMsgBusProxy::MSTATUS_ERROR + ); + } + + Y_UNIT_TEST(WriteAfterCreateDeleteCreate) { + TTestServer server(PQSettings(0).SetDomainName("Root").SetNodeCount(2)); + TString topic = "rt3.dc1--johnsnow"; + server.AnnoyingClient->CreateTopic(topic, 2); + + server.AnnoyingClient->WriteToPQ(topic, 1, "abacaba", 1, "valuevaluevalue1"); + server.AnnoyingClient->WriteToPQ(topic, 3, "abacaba", 1, "valuevaluevalue1", "", ETransport::MsgBus, NMsgBusProxy::MSTATUS_ERROR, NMsgBusProxy::MSTATUS_ERROR); + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD, NKikimrServices::PQ_METACACHE }); + + server.AnnoyingClient->DeleteTopic2(topic); + + server.AnnoyingClient->CreateTopic(topic, 4); + + // Write to topic, cache must be updated by CreateTopic + server.AnnoyingClient->WriteToPQ(topic, 1, "abacaba", 1, "valuevaluevalue1"); + // Write to partition that didn't exist in the old topic + server.AnnoyingClient->WriteToPQ(topic, 3, "abacaba", 1, "valuevaluevalue1"); + } + + Y_UNIT_TEST(GetOffsetsAfterDelete) { + TTestServer server(PQSettings(0).SetDomainName("Root").SetNodeCount(2)); + TString topic2 = "rt3.dc1--topic2"; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 3); + server.AnnoyingClient->CreateTopic(topic2, 3); + + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 1, "abacaba", 1, "valuevaluevalue1"); + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD, NKikimrServices::PQ_METACACHE }); + + server.AnnoyingClient->DeleteTopic2(DEFAULT_TOPIC_NAME); + + // Get offsets from deleted topic + server.AnnoyingClient->GetPartOffset( { + {DEFAULT_TOPIC_NAME, {1,2}} + }, 0, 0, false); + + // Get offsets from multiple topics + server.AnnoyingClient->GetPartOffset( { + {DEFAULT_TOPIC_NAME, {1,2}}, + {topic2, {1,2}}, + }, 0, 0, false); + } + + + Y_UNIT_TEST(GetOffsetsAfterCreateDeleteCreate) { + TTestServer server(PQSettings(0).SetDomainName("Root").SetNodeCount(2)); + TString topic2 = "rt3.dc1--topic2"; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 3); + server.AnnoyingClient->CreateTopic(topic2, 3); + + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 1, "abacaba", 1, "valuevaluevalue1"); + + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD, NKikimrServices::PQ_METACACHE }); + + server.AnnoyingClient->DeleteTopic2(DEFAULT_TOPIC_NAME); + Sleep(TDuration::Seconds(1)); + + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 4); + Sleep(TDuration::Seconds(1)); + + // Get offsets from multiple topics + server.AnnoyingClient->GetPartOffset( { + {DEFAULT_TOPIC_NAME, {1,2}}, + {topic2, {1}}, + }, 3, 0, true); + } + + Y_UNIT_TEST(BigRead) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1, 8_MB, 86400, 20000000, "user", 2000000); + + server.EnableLogs( { NKikimrServices::FLAT_TX_SCHEMESHARD }); + + TString value(1_MB, 'x'); + for (ui32 i = 0; i < 32; ++i) + server.AnnoyingClient->WriteToPQ({DEFAULT_TOPIC_NAME, 0, "source1", i}, value); + + // trying to read small PQ messages in a big messagebus event + auto info = server.AnnoyingClient->ReadFromPQ({DEFAULT_TOPIC_NAME, 0, 0, 32, "user"}, 23, "", NMsgBusProxy::MSTATUS_OK); //will read 21mb + UNIT_ASSERT_VALUES_EQUAL(info.BlobsFromDisk, 0); + UNIT_ASSERT_VALUES_EQUAL(info.BlobsFromCache, 4); + + TInstant now(TInstant::Now()); + info = server.AnnoyingClient->ReadFromPQ({DEFAULT_TOPIC_NAME, 0, 0, 32, "user"}, 23, "", NMsgBusProxy::MSTATUS_OK); //will read 21mb + TDuration dur = TInstant::Now() - now; + UNIT_ASSERT_C(dur > TDuration::Seconds(7) && dur < TDuration::Seconds(20), "dur = " << dur); //speed limit is 2000kb/s and burst is 2000kb, so to read 24mb it will take at least 11 seconds + + server.AnnoyingClient->GetPartStatus({}, 1, true); + + } + + // expects that L2 size is 32Mb + Y_UNIT_TEST(Cache) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1, 8_MB); + + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD }); + + TString value(1_MB, 'x'); + for (ui32 i = 0; i < 32; ++i) + server.AnnoyingClient->WriteToPQ({DEFAULT_TOPIC_NAME, 0, "source1", i}, value); + + auto info0 = server.AnnoyingClient->ReadFromPQ({DEFAULT_TOPIC_NAME, 0, 0, 16, "user"}, 16); + auto info16 = server.AnnoyingClient->ReadFromPQ({DEFAULT_TOPIC_NAME, 0, 16, 16, "user"}, 16); + + UNIT_ASSERT_VALUES_EQUAL(info0.BlobsFromCache, 3); + UNIT_ASSERT_VALUES_EQUAL(info16.BlobsFromCache, 2); + UNIT_ASSERT_VALUES_EQUAL(info0.BlobsFromDisk + info16.BlobsFromDisk, 0); + + for (ui32 i = 0; i < 8; ++i) + server.AnnoyingClient->WriteToPQ({DEFAULT_TOPIC_NAME, 0, "source1", 32+i}, value); + + info0 = server.AnnoyingClient->ReadFromPQ({DEFAULT_TOPIC_NAME, 0, 0, 16, "user"}, 16); + info16 = server.AnnoyingClient->ReadFromPQ({DEFAULT_TOPIC_NAME, 0, 16, 16, "user"}, 16); + + ui32 fromDisk = info0.BlobsFromDisk + info16.BlobsFromDisk; + ui32 fromCache = info0.BlobsFromCache + info16.BlobsFromCache; + UNIT_ASSERT(fromDisk > 0); + UNIT_ASSERT(fromDisk < 5); + UNIT_ASSERT(fromCache > 0); + UNIT_ASSERT(fromCache < 5); + } + + Y_UNIT_TEST(CacheHead) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1, 6_MB); + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD }); + + ui64 seqNo = 0; + for (ui32 blobSizeKB = 256; blobSizeKB < 4096; blobSizeKB *= 2) { + static const ui32 maxEventKB = 24_KB; + ui32 blobSize = blobSizeKB * 1_KB; + ui32 count = maxEventKB / blobSizeKB; + count -= count%2; + ui32 half = count/2; + + ui64 offset = seqNo; + TString value(blobSize, 'a'); + for (ui32 i = 0; i < count; ++i) + server.AnnoyingClient->WriteToPQ({DEFAULT_TOPIC_NAME, 0, "source1", seqNo++}, value); + + auto info_half1 = server.AnnoyingClient->ReadFromPQ({DEFAULT_TOPIC_NAME, 0, offset, half, "user1"}, half); + auto info_half2 = server.AnnoyingClient->ReadFromPQ({DEFAULT_TOPIC_NAME, 0, offset, half, "user1"}, half); + + UNIT_ASSERT(info_half1.BlobsFromCache > 0); + UNIT_ASSERT(info_half2.BlobsFromCache > 0); + UNIT_ASSERT_VALUES_EQUAL(info_half1.BlobsFromDisk, 0); + UNIT_ASSERT_VALUES_EQUAL(info_half2.BlobsFromDisk, 0); + } + } + + Y_UNIT_TEST(SameOffset) { + TTestServer server; + TString topic2 = "rt3.dc1--topic2"; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1, 6_MB); + server.AnnoyingClient->CreateTopic(topic2, 1, 6_MB); + + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD }); + + ui32 valueSize = 128; + TString value1(valueSize, 'a'); + TString value2(valueSize, 'b'); + server.AnnoyingClient->WriteToPQ({DEFAULT_TOPIC_NAME, 0, "source1", 0}, value1); + server.AnnoyingClient->WriteToPQ({topic2, 0, "source1", 0}, value2); + + // avoid reading from head + TString mb(1_MB, 'x'); + for (ui32 i = 1; i < 16; ++i) { + server.AnnoyingClient->WriteToPQ({DEFAULT_TOPIC_NAME, 0, "source1", i}, mb); + server.AnnoyingClient->WriteToPQ({topic2, 0, "source1", i}, mb); + } + + auto info1 = server.AnnoyingClient->ReadFromPQ({DEFAULT_TOPIC_NAME, 0, 0, 1, "user1"}, 1); + auto info2 = server.AnnoyingClient->ReadFromPQ({topic2, 0, 0, 1, "user1"}, 1); + + UNIT_ASSERT_VALUES_EQUAL(info1.BlobsFromCache, 1); + UNIT_ASSERT_VALUES_EQUAL(info2.BlobsFromCache, 1); + UNIT_ASSERT_VALUES_EQUAL(info1.Values.size(), 1); + UNIT_ASSERT_VALUES_EQUAL(info2.Values.size(), 1); + UNIT_ASSERT_VALUES_EQUAL(info1.Values[0].size(), valueSize); + UNIT_ASSERT_VALUES_EQUAL(info2.Values[0].size(), valueSize); + UNIT_ASSERT(info1.Values[0] == value1); + UNIT_ASSERT(info2.Values[0] == value2); + } + + + Y_UNIT_TEST(FetchRequest) { + TTestServer server; + TString topic2 = "rt3.dc1--topic2"; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 10); + server.AnnoyingClient->CreateTopic(topic2, 10); + + ui32 valueSize = 128; + TString value1(valueSize, 'a'); + TString value2(valueSize, 'b'); + server.AnnoyingClient->WriteToPQ({topic2, 5, "source1", 0}, value2); + server.AnnoyingClient->WriteToPQ({DEFAULT_TOPIC_NAME, 1, "source1", 0}, value1); + server.AnnoyingClient->WriteToPQ({DEFAULT_TOPIC_NAME, 1, "source1", 1}, value2); + + server.EnableLogs({ NKikimrServices::FLAT_TX_SCHEMESHARD }); + TInstant tm(TInstant::Now()); + server.AnnoyingClient->FetchRequestPQ( + { {topic2, 5, 0, 400}, {DEFAULT_TOPIC_NAME, 1, 0, 400}, {DEFAULT_TOPIC_NAME, 3, 0, 400} }, + 400, 1000000 + ); + UNIT_ASSERT((TInstant::Now() - tm).Seconds() < 1); + tm = TInstant::Now(); + server.AnnoyingClient->FetchRequestPQ({{topic2, 5, 1, 400}}, 400, 5000); + UNIT_ASSERT((TInstant::Now() - tm).Seconds() > 2); + server.AnnoyingClient->FetchRequestPQ( + { {topic2, 5, 0, 400}, {DEFAULT_TOPIC_NAME, 1, 0, 400}, {DEFAULT_TOPIC_NAME, 3, 0, 400} }, + 1, 1000000 + ); + server.AnnoyingClient->FetchRequestPQ( + { {topic2, 5, 500, 400}, {topic2, 4, 0, 400}, {DEFAULT_TOPIC_NAME, 1, 0, 400} }, + 400, 1000000 + ); + } + + Y_UNIT_TEST(ChooseProxy) { + TTestServer server; + server.AnnoyingClient->ChooseProxy(ETransport::GRpc); + } + + + Y_UNIT_TEST(Init) { + TTestServer server(PQSettings(0).SetDomainName("Root").SetNodeCount(2)); + TString topic2 = "rt3.dc1--topic2"; + TString topic3 = "rt3.dc1--topic3"; + + if (!true) { + server.EnableLogs({ + NKikimrServices::FLAT_TX_SCHEMESHARD, + NKikimrServices::TX_DATASHARD, + NKikimrServices::HIVE, + NKikimrServices::PERSQUEUE, + NKikimrServices::TABLET_MAIN, + NKikimrServices::BS_PROXY_DISCOVER, + NKikimrServices::PIPE_CLIENT, + NKikimrServices::PQ_METACACHE }); + } + + server.AnnoyingClient->DescribeTopic({}); + server.AnnoyingClient->TestCase({}, 0, 0, true); + + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 10); + server.AnnoyingClient->AlterTopic(DEFAULT_TOPIC_NAME, 20); + server.AnnoyingClient->CreateTopic(topic2, 25); + + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 5, "abacaba", 1, "valuevaluevalue1"); + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 5, "abacaba", 2, "valuevaluevalue2"); + server.AnnoyingClient->WriteToPQ(DEFAULT_TOPIC_NAME, 5, "abacabae", 1, "valuevaluevalue3"); + server.AnnoyingClient->ReadFromPQ(DEFAULT_TOPIC_NAME, 5, 0, 10, 3); + + server.AnnoyingClient->SetClientOffsetPQ(DEFAULT_TOPIC_NAME, 5, 2); + + server.AnnoyingClient->TestCase({{DEFAULT_TOPIC_NAME, {5}}}, 1, 1, true); + server.AnnoyingClient->TestCase({{DEFAULT_TOPIC_NAME, {0}}}, 1, 0, true); + server.AnnoyingClient->TestCase({{DEFAULT_TOPIC_NAME, {}}}, 20, 1, true); + server.AnnoyingClient->TestCase({{DEFAULT_TOPIC_NAME, {5, 5}}}, 0, 0, false); + server.AnnoyingClient->TestCase({{DEFAULT_TOPIC_NAME, {111}}}, 0, 0, false); + server.AnnoyingClient->TestCase({}, 45, 1, true); + server.AnnoyingClient->TestCase({{topic3, {}}}, 0, 0, false); + server.AnnoyingClient->TestCase({{DEFAULT_TOPIC_NAME, {}}, {topic3, {}}}, 0, 0, false); + server.AnnoyingClient->TestCase({{DEFAULT_TOPIC_NAME, {}}, {topic2, {}}}, 45, 1, true); + server.AnnoyingClient->TestCase({{DEFAULT_TOPIC_NAME, {0, 3, 5}}, {topic2, {1, 4, 6, 8}}}, 7, 1, true); + + server.AnnoyingClient->DescribeTopic({DEFAULT_TOPIC_NAME}); + server.AnnoyingClient->DescribeTopic({topic2}); + server.AnnoyingClient->DescribeTopic({topic2, DEFAULT_TOPIC_NAME}); + server.AnnoyingClient->DescribeTopic({}); + server.AnnoyingClient->DescribeTopic({topic3}, true); + } + + + Y_UNIT_TEST(DescribeAfterDelete) { + TTestServer server(PQSettings(0).SetDomainName("Root").SetNodeCount(2)); + server.EnableLogs({ NKikimrServices::PQ_METACACHE }); + TString topic2 = "rt3.dc1--topic2"; + + server.AnnoyingClient->DescribeTopic({}); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 10); + server.AnnoyingClient->CreateTopic(topic2, 10); + server.AnnoyingClient->DescribeTopic({}); + + server.AnnoyingClient->DeleteTopic2(DEFAULT_TOPIC_NAME); + + server.AnnoyingClient->DescribeTopic({}); + server.AnnoyingClient->GetClientInfo({}, "user", true); + server.AnnoyingClient->GetClientInfo({topic2}, "user", true); + Sleep(TDuration::Seconds(2)); + server.AnnoyingClient->TestCase({{DEFAULT_TOPIC_NAME, {}}, {topic2, {}}}, 10, 0, false); + server.AnnoyingClient->TestCase({{DEFAULT_TOPIC_NAME, {}}, {topic2, {}}, {"rt3.dc1--topic3", {}}}, 10, 0, false); + } + + Y_UNIT_TEST(DescribeAfterDelete2) { + TTestServer server(PQSettings(0).SetDomainName("Root").SetNodeCount(2)); + server.EnableLogs({ NKikimrServices::PQ_METACACHE }); + TString topic3 = "rt3.dc1--topic3"; + + server.AnnoyingClient->CreateTopic(topic3, 10); + Sleep(TDuration::Seconds(3)); //for invalidation of cache + server.AnnoyingClient->TestCase({{"rt3.dc1--topic4", {}}}, 10, 0, false); //will force caching of topic3 + server.AnnoyingClient->DeleteTopic2(topic3); + server.AnnoyingClient->DescribeTopic({topic3}, true); //describe will fail + server.AnnoyingClient->DescribeTopic({topic3}, true); //describe will fail + } + + + void WaitResolveSuccess(TTestServer& server, TString topic, ui32 numParts) { + const TInstant start = TInstant::Now(); + while (true) { + TAutoPtr<NMsgBusProxy::TBusPersQueue> request(new NMsgBusProxy::TBusPersQueue); + auto req = request->Record.MutableMetaRequest(); + auto partOff = req->MutableCmdGetPartitionLocations(); + auto treq = partOff->AddTopicRequest(); + treq->SetTopic(topic); + for (ui32 i = 0; i < numParts; ++i) + treq->AddPartition(i); + + TAutoPtr<NBus::TBusMessage> reply; + NBus::EMessageStatus status = server.AnnoyingClient->SyncCall(request, reply); + UNIT_ASSERT_VALUES_EQUAL(status, NBus::MESSAGE_OK); + const NMsgBusProxy::TBusResponse* response = dynamic_cast<NMsgBusProxy::TBusResponse*>(reply.Get()); + UNIT_ASSERT(response); + if (response->Record.GetStatus() == NMsgBusProxy::MSTATUS_OK) + break; + UNIT_ASSERT(TInstant::Now() - start < ::DEFAULT_DISPATCH_TIMEOUT); + Sleep(TDuration::MilliSeconds(10)); + } + } + + Y_UNIT_TEST(WhenDisableNodeAndCreateTopic_ThenAllPartitionsAreOnOtherNode) { + // Arrange + TTestServer server(PQSettings(0).SetDomainName("Root").SetNodeCount(2)); + server.EnableLogs({ NKikimrServices::HIVE }); + TString unused = "rt3.dc1--unusedtopic"; + // Just to make sure that HIVE has started + server.AnnoyingClient->CreateTopic(unused, 1); + WaitResolveSuccess(server, unused, 1); + + // Act + // Disable node #0 + server.AnnoyingClient->MarkNodeInHive(server.CleverServer->GetRuntime(), 0, false); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 3); + WaitResolveSuccess(server, DEFAULT_TOPIC_NAME, 3); + + // Assert that all partitions are on node #1 + const ui32 node1Id = server.CleverServer->GetRuntime()->GetNodeId(1); + UNIT_ASSERT_VALUES_EQUAL( + server.AnnoyingClient->GetPartLocation({{DEFAULT_TOPIC_NAME, {0, 1}}}, 2, true), + TVector<ui32>({node1Id, node1Id}) + ); + } + + void PrepareForGrpc(TTestServer& server) { + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 2); + server.AnnoyingClient->InitUserRegistry(); + } + void PrepareForFstClass(TFlatMsgBusPQClient& annoyingClient) { + annoyingClient.SetNoConfigMode(); + annoyingClient.FullInit({}); + annoyingClient.InitUserRegistry(); + annoyingClient.MkDir("/Root", "account1"); + annoyingClient.CreateTopicNoLegacy(FC_TOPIC_PATH, 5); + } + + Y_UNIT_TEST(CheckACLForGrpcWrite) { + TTestServer server; + PrepareForGrpc(server); + + auto writer = MakeDataWriter(server, "source1"); + + writer.Write(SHORT_TOPIC_NAME, "valuevaluevalue1", true, "topic1@" BUILTIN_ACL_DOMAIN); + + NACLib::TDiffACL acl; + acl.AddAccess(NACLib::EAccessType::Allow, NACLib::UpdateRow, "topic1@" BUILTIN_ACL_DOMAIN); + server.AnnoyingClient->ModifyACL("/Root/PQ", DEFAULT_TOPIC_NAME, acl.SerializeAsString()); + + Sleep(TDuration::Seconds(2)); + + auto writer2 = MakeDataWriter(server, "source1"); + writer2.Write(SHORT_TOPIC_NAME, "valuevaluevalue1", false, "topic1@" BUILTIN_ACL_DOMAIN); + } + + void PrepareForGrpcNoDC(TFlatMsgBusPQClient& annoyingClient) { + annoyingClient.SetNoConfigMode(); + annoyingClient.FullInit(); + annoyingClient.InitUserRegistry(); + annoyingClient.MkDir("/Root", "account1"); + annoyingClient.MkDir("/Root/PQ", "account1"); + annoyingClient.CreateTopicNoLegacy("/Root/PQ/rt3.db--topic1", 5, false); + annoyingClient.CreateTopicNoLegacy("/Root/PQ/account1/topic1", 5, false, true, {}, {"user1", "user2"}); + annoyingClient.CreateTopicNoLegacy("/Root/account2/topic2", 5); + } + + Y_UNIT_TEST(TestGrpcWriteNoDC) { + TTestServer server(false); + server.ServerSettings.PQConfig.SetTopicsAreFirstClassCitizen(true); + server.ServerSettings.PQConfig.SetRoot("/Rt2/PQ"); + server.ServerSettings.PQConfig.SetDatabase("/Root"); + server.StartServer(); + + UNIT_ASSERT_VALUES_EQUAL(NMsgBusProxy::MSTATUS_OK, + server.AnnoyingClient->AlterUserAttributes("/", "Root", {{"folder_id", "somefolder"}, {"cloud_id", "somecloud"}, {"database_id", "root"}})); + + + PrepareForGrpcNoDC(*server.AnnoyingClient); + auto writer = MakeDataWriter(server, "source1"); + + writer.Write("/Root/account2/topic2", {"valuevaluevalue1"}, true, "topic1@" BUILTIN_ACL_DOMAIN); + writer.Write("/Root/PQ/account1/topic1", {"valuevaluevalue1"}, true, "topic1@" BUILTIN_ACL_DOMAIN); + + NACLib::TDiffACL acl; + acl.AddAccess(NACLib::EAccessType::Allow, NACLib::UpdateRow, "topic1@" BUILTIN_ACL_DOMAIN); + acl.AddAccess(NACLib::EAccessType::Allow, NACLib::SelectRow, "topic1@" BUILTIN_ACL_DOMAIN); + + server.AnnoyingClient->ModifyACL("/Root/account2", "topic2", acl.SerializeAsString()); + server.AnnoyingClient->ModifyACL("/Root/PQ/account1", "topic1", acl.SerializeAsString()); + + Sleep(TDuration::Seconds(5)); + + writer.Write("/Root/account2/topic2", {"valuevaluevalue1"}, false, "topic1@" BUILTIN_ACL_DOMAIN); + + writer.Write("/Root/PQ/account1/topic1", {"valuevaluevalue1"}, false, "topic1@" BUILTIN_ACL_DOMAIN); + writer.Write("/Root/PQ/account1/topic1", {"valuevaluevalue2"}, false, "topic1@" BUILTIN_ACL_DOMAIN); + + writer.Read("/Root/PQ/account1/topic1", "user1", "topic1@" BUILTIN_ACL_DOMAIN, false, false, false, true); + + writer.Write("Root/PQ/account1/topic1", {"valuevaluevalue2"}, false, "topic1@" BUILTIN_ACL_DOMAIN); //TODO /Root remove + + writer.Read("Root/PQ/account1/topic1", "user1", "topic1@" BUILTIN_ACL_DOMAIN, false, false, false, true); + } + + Y_UNIT_TEST(CheckACLForGrpcRead) { + TTestServer server(PQSettings(0, 1)); + PrepareForGrpc(server); + TString topic2 = "rt3.dc1--topic2"; + TString topic2ShortName = "topic2"; + server.AnnoyingClient->CreateTopic( + topic2, 1, 8_MB, 86400, 20000000, "", 200000000, {"user1", "user2"} + ); + server.EnableLogs({NKikimrServices::PERSQUEUE}, NActors::NLog::PRI_INFO); + + server.AnnoyingClient->CreateConsumer("user1"); + server.AnnoyingClient->CreateConsumer("user2"); + server.AnnoyingClient->CreateConsumer("user5"); + server.AnnoyingClient->GrantConsumerAccess("user1", "user2@" BUILTIN_ACL_DOMAIN); + server.AnnoyingClient->GrantConsumerAccess("user1", "user3@" BUILTIN_ACL_DOMAIN); + + server.AnnoyingClient->GrantConsumerAccess("user1", "1@" BUILTIN_ACL_DOMAIN); + server.AnnoyingClient->GrantConsumerAccess("user2", "2@" BUILTIN_ACL_DOMAIN); + server.AnnoyingClient->GrantConsumerAccess("user5", "1@" BUILTIN_ACL_DOMAIN); + server.AnnoyingClient->GrantConsumerAccess("user5", "2@" BUILTIN_ACL_DOMAIN); + + auto writer = MakeDataWriter(server, "source1"); + + NACLib::TDiffACL acl; + acl.AddAccess(NACLib::EAccessType::Allow, NACLib::SelectRow, "1@" BUILTIN_ACL_DOMAIN); + acl.AddAccess(NACLib::EAccessType::Allow, NACLib::SelectRow, "2@" BUILTIN_ACL_DOMAIN); + acl.AddAccess(NACLib::EAccessType::Allow, NACLib::SelectRow, "user1@" BUILTIN_ACL_DOMAIN); + acl.AddAccess(NACLib::EAccessType::Allow, NACLib::SelectRow, "user2@" BUILTIN_ACL_DOMAIN); + server.AnnoyingClient->ModifyACL("/Root/PQ", topic2, acl.SerializeAsString()); + + Sleep(TDuration::Seconds(2)); + + auto ticket1 = "1@" BUILTIN_ACL_DOMAIN; + auto ticket2 = "2@" BUILTIN_ACL_DOMAIN; + + writer.Read(topic2ShortName, "user1", ticket1, false, false, false, true); + + writer.Read(topic2ShortName, "user1", "user2@" BUILTIN_ACL_DOMAIN, false, false, false, true); + writer.Read(topic2ShortName, "user1", "user3@" BUILTIN_ACL_DOMAIN, true, false, false, true); //for topic + writer.Read(topic2ShortName, "user1", "user1@" BUILTIN_ACL_DOMAIN, true, false, false, true); //for consumer + + writer.Read(topic2ShortName, "user2", ticket1, true, false, false, true); + writer.Read(topic2ShortName, "user2", ticket2, false, false, false, true); + + writer.Read(topic2ShortName, "user5", ticket1, true, false, false, true); + writer.Read(topic2ShortName, "user5", ticket2, true, false, false, true); + + acl.Clear(); + acl.AddAccess(NACLib::EAccessType::Allow, NACLib::SelectRow, "user3@" BUILTIN_ACL_DOMAIN); + server.AnnoyingClient->ModifyACL("/Root/PQ", topic2, acl.SerializeAsString()); + + Sleep(TDuration::Seconds(2)); + writer.Read(topic2ShortName, "user1", "user3@" BUILTIN_ACL_DOMAIN, false, true, false, true); + } + + + + Y_UNIT_TEST(CheckKillBalancer) { + TTestServer server; + server.EnableLogs({ NKikimrServices::PQ_WRITE_PROXY, NKikimrServices::PQ_READ_PROXY}); + server.EnableLogs({ NKikimrServices::PERSQUEUE}, NActors::NLog::PRI_INFO); + + PrepareForGrpc(server); + + auto writer = MakeDataWriter(server, "source1"); + TTestPQLib PQLib(server); + auto [consumer, p] = PQLib.CreateConsumer({SHORT_TOPIC_NAME}, "shared/user", {}, true); + Cerr << p.Response << "\n"; + + auto msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Type == EMT_LOCK); + auto pp = msg.GetValue().ReadyToRead; + pp.SetValue(TLockInfo()); + + msg = consumer->GetNextMessage(); + msg.Wait(); + + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Type == EMT_LOCK); + pp = msg.GetValue().ReadyToRead; + pp.SetValue(TLockInfo()); + + msg = consumer->GetNextMessage(); + + //TODO: make here infly commits - check results + + server.AnnoyingClient->RestartBalancerTablet(server.CleverServer->GetRuntime(), DEFAULT_TOPIC_NAME); + Cerr << "Balancer killed\n"; + + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Type == EMT_RELEASE); + UNIT_ASSERT(!msg.GetValue().Response.GetRelease().GetCanCommit()); + + msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Type == EMT_RELEASE); + UNIT_ASSERT(!msg.GetValue().Response.GetRelease().GetCanCommit()); + + msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Type == EMT_LOCK); + + msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Type == EMT_LOCK); + + + msg = consumer->GetNextMessage(); + bool res = msg.Wait(TDuration::Seconds(1)); + Y_VERIFY(!res); //no read signalled or lock signals + } + + Y_UNIT_TEST(TestWriteStat) { + auto testWriteStat = [](const TString& originallyProvidedConsumerName, + const TString& consumerName, + const TString& consumerPath) { + auto checkCounters = [](auto monPort, const TString& session, + const std::set<std::string>& canonicalSensorNames, + const TString& clientDc, const TString& originDc, + const TString& client, const TString& consumerPath) { + NJson::TJsonValue counters; + if (clientDc.empty() && originDc.empty()) { + counters = GetClientCountersLegacy(monPort, "pqproxy", session, client, consumerPath); + } else { + counters = GetCountersLegacy(monPort, "pqproxy", session, "account/topic1", + clientDc, originDc, client, consumerPath); + } + const auto sensors = counters["sensors"].GetArray(); + std::set<std::string> sensorNames; + std::transform(sensors.begin(), sensors.end(), + std::inserter(sensorNames, sensorNames.begin()), + [](auto& el) { + return el["labels"]["sensor"].GetString(); + }); + auto equal = sensorNames == canonicalSensorNames; + UNIT_ASSERT(equal); + }; + + auto settings = PQSettings(0, 1, true, "10"); + TTestServer server(settings, false); + server.PrepareNetDataFile(FormNetData()); + server.StartServer(); + server.EnableLogs({ NKikimrServices::PQ_WRITE_PROXY, NKikimrServices::NET_CLASSIFIER }); + + const auto monPort = TPortManager().GetPort(); + auto Counters = server.CleverServer->GetGRpcServerRootCounters(); + NActors::TMon Monitoring({monPort, "localhost", 3, "root", "localhost", {}, {}, {}}); + Monitoring.RegisterCountersPage("counters", "Counters", Counters); + Monitoring.Start(); + + auto sender = server.CleverServer->GetRuntime()->AllocateEdgeActor(); + + GetClassifierUpdate(*server.CleverServer, sender); //wait for initializing + + server.AnnoyingClient->CreateTopic("rt3.dc1--account--topic1", 10, 10000, 10000, 2000); + + auto writer = MakeDataWriter(server); + TTestPQLib PQLib(server); + auto [producer, res] = PQLib.CreateProducer("account--topic1", "base64:AAAAaaaa____----12", {}, ECodec::RAW); + + Cerr << res.Response << "\n"; + TInstant st(TInstant::Now()); + for (ui32 i = 1; i <= 5; ++i) { + auto f = producer->Write(i, TString(2000, 'a')); + f.Wait(); + Cerr << f.GetValue().Response << "\n"; + if (i == 5) { + UNIT_ASSERT(TInstant::Now() - st > TDuration::Seconds(3)); + UNIT_ASSERT(f.GetValue().Response.GetAck().GetStat().GetPartitionQuotedTimeMs() <= + f.GetValue().Response.GetAck().GetStat().GetTotalTimeInPartitionQueueMs() + 100); + } + } + + checkCounters(server.CleverServer->GetRuntime()->GetMonPort(), + "writeSession", + { + "BytesWrittenOriginal", + "CompactedBytesWrittenOriginal", + "MessagesWrittenOriginal", + "UncompressedBytesWrittenOriginal" + }, + "", "cluster", "", "" + ); + + checkCounters(monPort, + "writeSession", + { + "BytesInflight", + "BytesInflightTotal", + "Errors", + "SessionsActive", + "SessionsCreated", + "WithoutAuth" + }, + "", "cluster", "", "" + ); + { + auto [consumer, res] = PQLib.CreateConsumer({"account/topic1"}, originallyProvidedConsumerName, + {}, false); + Cerr << res.Response << "\n"; + auto msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Type == EMT_DATA); + + checkCounters(monPort, + "readSession", + { + "Commits", + "PartitionsErrors", + "PartitionsInfly", + "PartitionsLocked", + "PartitionsReleased", + "PartitionsToBeLocked", + "PartitionsToBeReleased", + "WaitsForData" + }, + "", "cluster", "", "" + ); + + checkCounters(monPort, + "readSession", + { + "BytesInflight", + "Errors", + "PipeReconnects", + "SessionsActive", + "SessionsCreated", + "PartsPerSession", + "SessionsWithOldBatchingVersion", + "WithoutAuth" + }, + "", "", consumerName, consumerPath + ); + + checkCounters(server.CleverServer->GetRuntime()->GetMonPort(), + "readSession", + { + "BytesReadFromDC" + }, + "Vla", "", "", "" + ); + + checkCounters(server.CleverServer->GetRuntime()->GetMonPort(), + "readSession", + { + "BytesRead", + "MessagesRead" + }, + "", "Dc1", "", "" + ); + + checkCounters(server.CleverServer->GetRuntime()->GetMonPort(), + "readSession", + { + "BytesRead", + "MessagesRead" + }, + "", "cluster", "", "" + ); + + checkCounters(server.CleverServer->GetRuntime()->GetMonPort(), + "readSession", + { + "BytesRead", + "MessagesRead" + }, + "", "cluster", "", "" + ); + + checkCounters(server.CleverServer->GetRuntime()->GetMonPort(), + "readSession", + { + "BytesRead", + "MessagesRead" + }, + "", "Dc1", consumerName, consumerPath + ); + } + }; + + testWriteStat("some@random@consumer", "some@random@consumer", "some/random/consumer"); + testWriteStat("some@user", "some@user", "some/user"); + testWriteStat("shared@user", "shared@user", "shared/user"); + testWriteStat("shared/user", "user", "shared/user"); + testWriteStat("user", "user", "shared/user"); + testWriteStat("some@random/consumer", "some@random@consumer", "some/random/consumer"); + testWriteStat("/some/user", "some@user", "some/user"); + } + + + Y_UNIT_TEST(TestWriteStat1stClass) { + auto testWriteStat1stClass = [](const TString& consumerPath) { + const auto folderId = "somefolder"; + const auto cloudId = "somecloud"; + const auto databaseId = "root"; + const TString fullTopicName{"/Root/account/topic1"}; + const TString topicName{"account/topic1"}; + + auto checkCounters = + [cloudId, folderId, databaseId](auto monPort, + const std::set<std::string>& canonicalSensorNames, + const TString& stream, const TString& consumer, + const TString& host, const TString& shard) { + auto counters = GetCounters1stClass(monPort, "datastreams", cloudId, databaseId, + folderId, stream, consumer, host, shard); + const auto sensors = counters["sensors"].GetArray(); + std::set<std::string> sensorNames; + std::transform(sensors.begin(), sensors.end(), + std::inserter(sensorNames, sensorNames.begin()), + [](auto& el) { + return el["labels"]["name"].GetString(); + }); + auto equal = sensorNames == canonicalSensorNames; + UNIT_ASSERT(equal); + }; + + auto settings = PQSettings(0, 1, true, "10"); + TTestServer server(settings, false); + server.PrepareNetDataFile(FormNetData()); + server.ServerSettings.PQConfig.SetTopicsAreFirstClassCitizen(true); + server.StartServer(); + server.EnableLogs({ NKikimrServices::PQ_WRITE_PROXY, NKikimrServices::NET_CLASSIFIER }); + + UNIT_ASSERT_VALUES_EQUAL(NMsgBusProxy::MSTATUS_OK, + server.AnnoyingClient->AlterUserAttributes("/", "Root", + {{"folder_id", folderId}, + {"cloud_id", cloudId}, + {"database_id", databaseId}})); + + const auto monPort = TPortManager().GetPort(); + auto Counters = server.CleverServer->GetGRpcServerRootCounters(); + NActors::TMon Monitoring({monPort, "localhost", 3, "root", "localhost", {}, {}, {}}); + Monitoring.RegisterCountersPage("counters", "Counters", Counters); + Monitoring.Start(); + + auto sender = server.CleverServer->GetRuntime()->AllocateEdgeActor(); + + GetClassifierUpdate(*server.CleverServer, sender); //wait for initializing + + server.AnnoyingClient->SetNoConfigMode(); + server.AnnoyingClient->FullInit(); + server.AnnoyingClient->InitUserRegistry(); + server.AnnoyingClient->MkDir("/Root", "account"); + server.AnnoyingClient->CreateTopicNoLegacy(fullTopicName, 5); + + NYdb::TDriverConfig driverCfg; + driverCfg.SetEndpoint(TStringBuilder() << "localhost:" << server.GrpcPort) + .SetLog(CreateLogBackend("cerr", ELogPriority::TLOG_DEBUG)).SetDatabase("/Root"); + + auto ydbDriver = MakeHolder<NYdb::TDriver>(driverCfg); + auto persQueueClient = MakeHolder<NYdb::NPersQueue::TPersQueueClient>(*ydbDriver); + + { + auto res = persQueueClient->AddReadRule(fullTopicName, + NYdb::NPersQueue::TAddReadRuleSettings().ReadRule(NYdb::NPersQueue::TReadRuleSettings().ConsumerName(consumerPath))); + res.Wait(); + UNIT_ASSERT(res.GetValue().IsSuccess()); + } + + auto writer = MakeDataWriter(server); + TTestPQLib PQLib(server); + auto [producer, res] = PQLib.CreateProducer(fullTopicName, "base64:AAAAaaaa____----12", {}, + ECodec::RAW); + + Cerr << res.Response << "\n"; + for (ui32 i = 1; i <= 5; ++i) { + auto f = producer->Write(i, TString(2000, 'a')); + f.Wait(); + Cerr << f.GetValue().Response << "\n"; + + } + + { + auto [consumer, res] = PQLib.CreateConsumer({topicName}, consumerPath, {}, + false); + Cerr << res.Response << "\n"; + auto msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + + checkCounters(monPort, + { + "stream.internal_read.commits_per_second", + "stream.internal_read.partitions_errors_per_second", + "stream.internal_read.partitions_locked", + "stream.internal_read.partitions_locked_per_second", + "stream.internal_read.partitions_released_per_second", + "stream.internal_read.partitions_to_be_locked", + "stream.internal_read.partitions_to_be_released", + "stream.internal_read.waits_for_data", + "stream.internal_write.bytes_proceeding", + "stream.internal_write.bytes_proceeding_total", + "stream.internal_write.errors_per_second", + "stream.internal_write.sessions_active", + "stream.internal_write.sessions_created_per_second", + "stream.internal_write.sessions_without_auth" + }, + topicName, "", "", "" + ); + + checkCounters(monPort, + { + "stream.internal_read.commits_per_second", + "stream.internal_read.partitions_errors_per_second", + "stream.internal_read.partitions_locked", + "stream.internal_read.partitions_locked_per_second", + "stream.internal_read.partitions_released_per_second", + "stream.internal_read.partitions_to_be_locked", + "stream.internal_read.partitions_to_be_released", + "stream.internal_read.waits_for_data", + }, + topicName, consumerPath, "", "" + ); + + checkCounters(server.CleverServer->GetRuntime()->GetMonPort(), + { + "stream.internal_read.time_lags_milliseconds", + "stream.incoming_bytes_per_second", + "stream.incoming_records_per_second", + "stream.internal_write.bytes_per_second", + "stream.internal_write.compacted_bytes_per_second", + "stream.internal_write.partition_write_quota_wait_milliseconds", + "stream.internal_write.record_size_bytes", + "stream.internal_write.records_per_second", + "stream.internal_write.time_lags_milliseconds", + "stream.internal_write.uncompressed_bytes_per_second", + "stream.await_operating_milliseconds", + "stream.internal_write.buffer_brimmed_duration_ms", + "stream.internal_read.bytes_per_second", + "stream.internal_read.records_per_second", + "stream.outgoing_bytes_per_second", + "stream.outgoing_records_per_second", + }, + topicName, "", "", "" + ); + + checkCounters(server.CleverServer->GetRuntime()->GetMonPort(), + { + "stream.internal_read.time_lags_milliseconds", + "stream.await_operating_milliseconds", + "stream.internal_read.bytes_per_second", + "stream.internal_read.records_per_second", + "stream.outgoing_bytes_per_second", + "stream.outgoing_records_per_second", + }, + topicName, consumerPath, "", "" + ); + + checkCounters(server.CleverServer->GetRuntime()->GetMonPort(), + { + "stream.await_operating_milliseconds" + }, + topicName, consumerPath, "1", "" + ); + + checkCounters(server.CleverServer->GetRuntime()->GetMonPort(), + { + "stream.internal_write.buffer_brimmed_duration_ms" + }, + topicName, "", "1", "" + ); + } + }; + testWriteStat1stClass("some@random@consumer"); + testWriteStat1stClass("user1"); + } + + + Y_UNIT_TEST(TestUnorderedCommit) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 3); + server.EnableLogs({ NKikimrServices::PQ_READ_PROXY }); + + auto writer = MakeDataWriter(server); + TTestPQLib PQLib(server); + + for (ui32 i = 1; i <= 3; ++i) { + auto [producer, res] = PQLib.CreateProducer(SHORT_TOPIC_NAME, "123" + ToString<ui32>(i), i, ECodec::RAW); + UNIT_ASSERT(res.Response.HasInit()); + auto f = producer->Write(i, TString(10, 'a')); + f.Wait(); + } + auto [consumer, res] = PQLib.CreateConsumer({SHORT_TOPIC_NAME}, "user", 1, false, false); + Cerr << res.Response << "\n"; + for (ui32 i = 1; i <= 3; ++i) { + auto msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasData()); + UNIT_ASSERT(msg.GetValue().Response.GetData().GetCookie() == i); + } + auto msg = consumer->GetNextMessage(); + consumer->Commit({2}); + UNIT_ASSERT(!msg.Wait(TDuration::Seconds(1))); + consumer->Commit({1}); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasCommit()); + UNIT_ASSERT(msg.GetValue().Response.GetCommit().CookieSize() == 2); + UNIT_ASSERT(msg.GetValue().Response.GetCommit().GetCookie(0) == 1); + UNIT_ASSERT(msg.GetValue().Response.GetCommit().GetCookie(1) == 2); + consumer->Commit({3}); + msg = consumer->GetNextMessage(); + UNIT_ASSERT(!msg.Wait(TDuration::MilliSeconds(500))); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasCommit()); + UNIT_ASSERT(msg.GetValue().Response.GetCommit().CookieSize() == 1); + UNIT_ASSERT(msg.GetValue().Response.GetCommit().GetCookie(0) == 3); + consumer->Commit({4}); //not existed cookie + msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasError()); + } + + Y_UNIT_TEST(TestMaxReadCookies) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1); + server.EnableLogs({ NKikimrServices::PQ_READ_PROXY }); + auto writer = MakeDataWriter(server); + + TTestPQLib PQLib(server); + { + auto [producer, res] = PQLib.CreateProducer(SHORT_TOPIC_NAME, "123"); + for (ui32 i = 1; i <= 11; ++i) { + auto f = producer->Write(i, TString(10, 'a')); + f.Wait(); + } + } + auto [consumer, res] = PQLib.CreateConsumer({SHORT_TOPIC_NAME}, "user", 1, false); + Cerr << res.Response << "\n"; + for (ui32 i = 1; i <= 11; ++i) { + auto msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasData()); + UNIT_ASSERT(msg.GetValue().Response.GetData().GetCookie() == i); + } + for (ui32 i = 11; i >= 1; --i) { + consumer->Commit({i}); + if (i == 5) { + server.AnnoyingClient->GetClientInfo({DEFAULT_TOPIC_NAME}, "user", true); + } + } + Cerr << "cookies committed\n"; + auto msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasError()); + } + + Y_UNIT_TEST(TestWriteSessionsConflicts) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1); + auto writer = MakeDataWriter(server); + server.EnableLogs({ NKikimrServices::PQ_WRITE_PROXY }); + + TTestPQLib PQLib(server); + auto [producer, res] = PQLib.CreateProducer(SHORT_TOPIC_NAME, "123"); + auto dead = producer->IsDead(); + + auto [producer2, res2] = PQLib.CreateProducer(SHORT_TOPIC_NAME, "123"); + + UNIT_ASSERT(dead.Wait(TDuration::Seconds(1))); + UNIT_ASSERT(!producer2->IsDead().Wait(TDuration::Seconds(1))); + } + + Y_UNIT_TEST(TestWriteSessionNaming) { + TTestServer server; + server.EnableLogs({ NKikimrServices::PQ_WRITE_PROXY }); + + + TTestPQLib PQLib(server); + while (true) { + auto [producer, res] = PQLib.CreateProducer("//", "123"); + Cerr << res.Response << "\n"; + UNIT_ASSERT(res.Response.HasError()); + if (res.Response.GetError().GetCode() != NPersQueue::NErrorCode::INITIALIZING) + break; + Sleep(TDuration::Seconds(1)); + } + } + + + Y_UNIT_TEST(TestRelocks) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1); + + server.EnableLogs({ NKikimrServices::PQ_READ_PROXY }); + + auto writer = MakeDataWriter(server); + + TIntrusivePtr<TCerrLogger> logger = new TCerrLogger(DEBUG_LOG_LEVEL); + TPQLib PQLib; + + TConsumerSettings ss; + ss.ClientId = "user"; + ss.Server = TServerSetting{"localhost", server.GrpcPort}; + ss.Topics.push_back(SHORT_TOPIC_NAME); + ss.UseLockSession = true; + ss.MaxCount = 1; + + auto consumer = PQLib.CreateConsumer(ss, logger, false); + auto p = consumer->Start(); + for (ui32 i = 0; i < 30; ++i) { + server.CleverServer->GetRuntime()->ResetScheduledCount(); + server.AnnoyingClient->RestartPartitionTablets(server.CleverServer->GetRuntime(), DEFAULT_TOPIC_NAME); + Sleep(TDuration::MilliSeconds(1)); + } + p.Wait(); + Cerr << p.GetValue().Response << "\n"; + auto msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasLock()); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetTopic() == DEFAULT_TOPIC_NAME); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetPartition() == 0); + } + + Y_UNIT_TEST(TestLockErrors) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1); + server.EnableLogs({ NKikimrServices::PQ_READ_PROXY }); + + auto writer = MakeDataWriter(server); + + TTestPQLib PQLib(server); + + { + auto [producer, res] = PQLib.CreateProducer(SHORT_TOPIC_NAME, "123"); + for (ui32 i = 1; i <= 11; ++i) { + auto f = producer->Write(i, TString(10, 'a')); + f.Wait(); + } + } + auto createConsumer = [&] () { + auto [consumer, res] = PQLib.CreateConsumer({SHORT_TOPIC_NAME}, "user", 1, true); + return std::move(consumer); + }; + auto consumer = createConsumer(); + auto msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasLock()); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetTopic() == DEFAULT_TOPIC_NAME); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetPartition() == 0); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetReadOffset() == 0); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetEndOffset() == 11); + + auto pp = msg.GetValue().ReadyToRead; + pp.SetValue(TLockInfo{0, 5, false}); + auto future = consumer->IsDead(); + future.Wait(); + Cerr << future.GetValue() << "\n"; + + + consumer = createConsumer(); + msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasLock()); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetTopic() == DEFAULT_TOPIC_NAME); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetPartition() == 0); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetReadOffset() == 0); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetEndOffset() == 11); + + pp = msg.GetValue().ReadyToRead; + pp.SetValue(TLockInfo{12, 12, false}); + future = consumer->IsDead(); + future.Wait(); + Cerr << future.GetValue() << "\n"; + + consumer = createConsumer(); + msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasLock()); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetTopic() == DEFAULT_TOPIC_NAME); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetPartition() == 0); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetReadOffset() == 0); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetEndOffset() == 11); + + pp = msg.GetValue().ReadyToRead; + pp.SetValue(TLockInfo{11, 11, false}); + Sleep(TDuration::Seconds(5)); + + + consumer = createConsumer(); + msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasLock()); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetTopic() == DEFAULT_TOPIC_NAME); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetPartition() == 0); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetReadOffset() == 11); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetEndOffset() == 11); + + pp = msg.GetValue().ReadyToRead; + pp.SetValue(TLockInfo{1, 0, true}); + future = consumer->IsDead(); + future.Wait(); + Cerr << future.GetValue() << "\n"; + + consumer = createConsumer(); + msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasLock()); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetTopic() == DEFAULT_TOPIC_NAME); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetPartition() == 0); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetReadOffset() == 11); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetEndOffset() == 11); + + pp = msg.GetValue().ReadyToRead; + pp.SetValue(TLockInfo{0, 0, false}); + future = consumer->IsDead(); + UNIT_ASSERT(!future.Wait(TDuration::Seconds(5))); + } + + + Y_UNIT_TEST(TestLocalChoose) { + TTestServer server(false); + server.ServerSettings.NodeCount = 3; + server.StartServer(); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1); + + server.EnableLogs({ NKikimrServices::CHOOSE_PROXY }); + + auto channel = grpc::CreateChannel("localhost:"+ToString(server.GrpcPort), grpc::InsecureChannelCredentials()); + auto stub(NKikimrClient::TGRpcServer::NewStub(channel)); + + auto sender = server.CleverServer->GetRuntime()->AllocateEdgeActor(); + + auto nodeId = server.CleverServer->GetRuntime()->GetNodeId(0); + server.CleverServer->GetRuntime()->Send(new IEventHandle( + MakeGRpcProxyStatusID(nodeId), sender, new TEvGRpcProxyStatus::TEvSetup(true, 0, 0)), 0, false + ); + nodeId = server.CleverServer->GetRuntime()->GetNodeId(1); + server.CleverServer->GetRuntime()->Send( + new IEventHandle( + MakeGRpcProxyStatusID(nodeId), sender, new TEvGRpcProxyStatus::TEvSetup(true, 10000000, 1000000) + ), 1, false + ); + nodeId = server.CleverServer->GetRuntime()->GetNodeId(2); + server.CleverServer->GetRuntime()->Send( + new IEventHandle( + MakeGRpcProxyStatusID(nodeId), sender, new TEvGRpcProxyStatus::TEvSetup(true, 10000000, 1000000) + ), 2, false + ); + + grpc::ClientContext context; + NKikimrClient::TChooseProxyRequest request; + request.SetPreferLocalProxy(true); + NKikimrClient::TResponse response; + auto status = stub->ChooseProxy(&context, request, &response); + UNIT_ASSERT(status.ok()); + Cerr << response << "\n"; + UNIT_ASSERT(response.GetStatus() == NMsgBusProxy::MSTATUS_OK); + UNIT_ASSERT(response.GetProxyCookie() == server.CleverServer->GetRuntime()->GetNodeId(0)); + + grpc::ClientContext context2; + request.SetPreferLocalProxy(false); + NKikimrClient::TResponse response2; + status = stub->ChooseProxy(&context2, request, &response2); + UNIT_ASSERT(status.ok()); + Cerr << response2 << "\n"; + UNIT_ASSERT(response2.GetStatus() == NMsgBusProxy::MSTATUS_OK); + UNIT_ASSERT(response2.GetProxyCookie() > server.CleverServer->GetRuntime()->GetNodeId(0)); + } + + + Y_UNIT_TEST(TestRestartBalancer) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1); + server.EnableLogs({ NKikimrServices::PQ_READ_PROXY }); + + auto writer = MakeDataWriter(server); + TTestPQLib PQLib(server); + + { + auto [producer, res] = PQLib.CreateProducer(SHORT_TOPIC_NAME, "123"); + for (ui32 i = 1; i < 2; ++i) { + auto f = producer->Write(i, TString(10, 'a')); + f.Wait(); + } + } + auto [consumer, res] = PQLib.CreateConsumer({SHORT_TOPIC_NAME}, "user", 1, true, {}, {}, 100, {}); + Cerr << res.Response << "\n"; + + auto msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasLock()); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetTopic() == DEFAULT_TOPIC_NAME); + UNIT_ASSERT(msg.GetValue().Response.GetLock().GetPartition() == 0); + + + msg.GetValue().ReadyToRead.SetValue(TLockInfo{0,0,false}); + + msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasData()); +// Sleep(TDuration::MilliSeconds(10)); + server.AnnoyingClient->RestartBalancerTablet(server.CleverServer->GetRuntime(), DEFAULT_TOPIC_NAME); + + msg = consumer->GetNextMessage(); + msg.Wait(); + + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT_C(msg.GetValue().Response.HasRelease(), "Response: " << msg.GetValue().Response); + } + + Y_UNIT_TEST(TestBigMessage) { + TTestServer server(false); + server.GrpcServerOptions.SetMaxMessageSize(130_MB); + server.StartServer(); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1); + server.EnableLogs({ NKikimrServices::PQ_WRITE_PROXY }); + + auto writer = MakeDataWriter(server); + TTestPQLib PQLib(server); + auto [producer, res] = PQLib.CreateProducer(SHORT_TOPIC_NAME, "123"); + auto f = producer->Write(1, TString(63_MB, 'a')); + f.Wait(); + Cerr << f.GetValue().Response << "\n"; + UNIT_ASSERT_C(f.GetValue().Response.HasAck(), f.GetValue().Response); + } + + void TestRereadsWhenDataIsEmptyImpl(bool withWait) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1); + server.EnableLogs({ NKikimrServices::PQ_READ_PROXY }); + + auto writer = MakeDataWriter(server); + TTestPQLib PQLib(server); + + // Write nonempty data + NKikimr::NPersQueueTests::TRequestWritePQ writeReq(DEFAULT_TOPIC_NAME, 0, "src", 4); + + auto write = [&](const TString& data, bool empty = false) { + NKikimrPQClient::TDataChunk dataChunk; + dataChunk.SetCreateTime(42); + dataChunk.SetSeqNo(++writeReq.SeqNo); + dataChunk.SetData(data); + if (empty) { + dataChunk.SetChunkType(NKikimrPQClient::TDataChunk::GROW); // this guarantees that data will be threated as empty + } + TString serialized; + UNIT_ASSERT(dataChunk.SerializeToString(&serialized)); + server.AnnoyingClient->WriteToPQ(writeReq, serialized); + }; + write("data1"); + write("data2", true); + if (!withWait) { + write("data3"); + } + + auto [consumer, res] = PQLib.CreateConsumer({SHORT_TOPIC_NAME}, "user", 1, false, {}, {}, 1, 1); + UNIT_ASSERT_C(res.Response.HasInit(), res.Response); + auto msg1 = consumer->GetNextMessage().GetValueSync().Response; + + auto assertHasData = [](const NPersQueue::TReadResponse& msg, const TString& data) { + const auto& d = msg.GetData(); + UNIT_ASSERT_VALUES_EQUAL_C(d.MessageBatchSize(), 1, msg); + UNIT_ASSERT_VALUES_EQUAL_C(d.GetMessageBatch(0).MessageSize(), 1, msg); + UNIT_ASSERT_STRINGS_EQUAL_C(d.GetMessageBatch(0).GetMessage(0).GetData(), data, msg); + }; + UNIT_ASSERT_VALUES_EQUAL_C(msg1.GetData().GetCookie(), 1, msg1); + assertHasData(msg1, "data1"); + + auto resp2Future = consumer->GetNextMessage(); + if (withWait) { + // no data + UNIT_ASSERT(!resp2Future.HasValue()); + UNIT_ASSERT(!resp2Future.HasException()); + + // waits and data doesn't arrive + Sleep(TDuration::MilliSeconds(100)); + UNIT_ASSERT(!resp2Future.HasValue()); + UNIT_ASSERT(!resp2Future.HasException()); + + // write data + write("data3"); + } + const auto& msg2 = resp2Future.GetValueSync().Response; + UNIT_ASSERT_VALUES_EQUAL_C(msg2.GetData().GetCookie(), 2, msg2); + assertHasData(msg2, "data3"); + } + + Y_UNIT_TEST(TestRereadsWhenDataIsEmpty) { + TestRereadsWhenDataIsEmptyImpl(false); + } + + Y_UNIT_TEST(TestRereadsWhenDataIsEmptyWithWait) { + TestRereadsWhenDataIsEmptyImpl(true); + } + + + Y_UNIT_TEST(TestLockAfterDrop) { + TTestServer server; + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1); + server.EnableLogs({ NKikimrServices::PQ_READ_PROXY }); + + auto writer = MakeDataWriter(server); + TTestPQLib PQLib(server); + + auto [producer, res] = PQLib.CreateProducer(SHORT_TOPIC_NAME, "123"); + auto f = producer->Write(1, TString(1_KB, 'a')); + f.Wait(); + + auto [consumer, res2] = PQLib.CreateConsumer({SHORT_TOPIC_NAME}, "user", 1, true); + Cerr << res2.Response << "\n"; + auto msg = consumer->GetNextMessage(); + msg.Wait(); + UNIT_ASSERT_C(msg.GetValue().Response.HasLock(), msg.GetValue().Response); + UNIT_ASSERT_C(msg.GetValue().Response.GetLock().GetTopic() == DEFAULT_TOPIC_NAME, msg.GetValue().Response); + UNIT_ASSERT_C(msg.GetValue().Response.GetLock().GetPartition() == 0, msg.GetValue().Response); + + server.CleverServer->GetRuntime()->ResetScheduledCount(); + server.AnnoyingClient->RestartPartitionTablets(server.CleverServer->GetRuntime(), DEFAULT_TOPIC_NAME); + + msg.GetValue().ReadyToRead.SetValue({0,0,false}); + + msg = consumer->GetNextMessage(); + UNIT_ASSERT(msg.Wait(TDuration::Seconds(10))); + + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasData()); + } + + + Y_UNIT_TEST(TestMaxNewTopicModel) { + TTestServer server; + server.AnnoyingClient->AlterUserAttributes("/", "Root", {{"__extra_path_symbols_allowed", "@"}}); + server.AnnoyingClient->CreateTopic("rt3.dc1--aaa@bbb@ccc--topic", 1); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1); + server.EnableLogs({ NKikimrServices::PQ_READ_PROXY }); + + auto writer = MakeDataWriter(server); + TTestPQLib PQLib(server); + { + auto [producer, res] = PQLib.CreateProducer("aaa/bbb/ccc/topic", "123"); + UNIT_ASSERT_C(res.Response.HasInit(), res.Response); + for (ui32 i = 1; i <= 11; ++i) { + auto f = producer->Write(i, TString(10, 'a')); + f.Wait(); + UNIT_ASSERT_C(f.GetValue().Response.HasAck(), f.GetValue().Response); + } + } + + auto [consumer, res2] = PQLib.CreateConsumer({"aaa/bbb/ccc/topic"}, "user", 1, true); + UNIT_ASSERT_C(res2.Response.HasInit(), res2.Response); + + auto msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Response.HasLock()); + } + + Y_UNIT_TEST(TestPartitionStatus) { + TTestServer server; + server.AnnoyingClient->AlterUserAttributes("/", "Root", {{"__extra_path_symbols_allowed", "@"}}); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1); + server.EnableLogs({ NKikimrServices::PQ_READ_PROXY }); + + auto writer = MakeDataWriter(server); + TTestPQLib PQLib(server); + + auto [consumer, res] = PQLib.CreateConsumer({SHORT_TOPIC_NAME}, "user", 1, true); + UNIT_ASSERT_C(res.Response.HasInit(), res.Response); + + auto msg = consumer->GetNextMessage(); + auto value = msg.ExtractValueSync(); + Cerr << value.Response << "\n"; + UNIT_ASSERT(value.Response.HasLock()); + value.ReadyToRead.SetValue(TLockInfo{}); + auto lock = value.Response.GetLock(); + consumer->RequestPartitionStatus(lock.GetTopic(), lock.GetPartition(), lock.GetGeneration()); + msg = consumer->GetNextMessage(); + value = msg.ExtractValueSync(); + Cerr << value.Response << "\n"; + UNIT_ASSERT(value.Response.HasPartitionStatus()); + UNIT_ASSERT(value.Response.GetPartitionStatus().GetCommittedOffset() == 0); + UNIT_ASSERT(value.Response.GetPartitionStatus().GetEndOffset() == 0); + auto wt = value.Response.GetPartitionStatus().GetWriteWatermarkMs(); + Sleep(TDuration::Seconds(15)); + + consumer->RequestPartitionStatus(lock.GetTopic(), lock.GetPartition(), lock.GetGeneration()); + msg = consumer->GetNextMessage(); + value = msg.ExtractValueSync(); + Cerr << value.Response << "\n"; + UNIT_ASSERT(value.Response.HasPartitionStatus()); + UNIT_ASSERT(value.Response.GetPartitionStatus().GetCommittedOffset() == 0); + UNIT_ASSERT(value.Response.GetPartitionStatus().GetEndOffset() == 0); + UNIT_ASSERT(wt < value.Response.GetPartitionStatus().GetWriteWatermarkMs()); + wt = value.Response.GetPartitionStatus().GetWriteWatermarkMs(); + + { + auto [producer, res] = PQLib.CreateProducer(SHORT_TOPIC_NAME, "123"); + UNIT_ASSERT_C(res.Response.HasInit(), res.Response); + auto f = producer->Write(1, TString(10, 'a')); + UNIT_ASSERT_C(f.GetValueSync().Response.HasAck(), f.GetValueSync().Response); + } + msg = consumer->GetNextMessage(); + value = msg.ExtractValueSync(); + Cerr << value.Response << "\n"; + UNIT_ASSERT(value.Response.HasData()); + auto cookie = value.Response.GetData().GetCookie(); + + consumer->RequestPartitionStatus(lock.GetTopic(), lock.GetPartition(), lock.GetGeneration()); + msg = consumer->GetNextMessage(); + value = msg.ExtractValueSync(); + Cerr << value.Response << "\n"; + UNIT_ASSERT(value.Response.HasPartitionStatus()); + UNIT_ASSERT(value.Response.GetPartitionStatus().GetCommittedOffset() == 0); + UNIT_ASSERT(value.Response.GetPartitionStatus().GetEndOffset() == 1); + UNIT_ASSERT(wt < value.Response.GetPartitionStatus().GetWriteWatermarkMs()); + wt = value.Response.GetPartitionStatus().GetWriteWatermarkMs(); + consumer->Commit({cookie}); + msg = consumer->GetNextMessage(); + Cerr << msg.GetValueSync().Response << "\n"; + UNIT_ASSERT(msg.GetValueSync().Response.HasCommit()); + + consumer->RequestPartitionStatus(lock.GetTopic(), lock.GetPartition(), lock.GetGeneration()); + msg = consumer->GetNextMessage(); + value = msg.ExtractValueSync(); + Cerr << value.Response << "\n"; + UNIT_ASSERT(value.Response.HasPartitionStatus()); + UNIT_ASSERT(value.Response.GetPartitionStatus().GetCommittedOffset() == 1); + UNIT_ASSERT(value.Response.GetPartitionStatus().GetEndOffset() == 1); + } + + Y_UNIT_TEST(TestDeletionOfTopic) { + TTestServer server(false); + server.GrpcServerOptions.SetMaxMessageSize(130_MB); + server.StartServer(); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 1); + server.EnableLogs({ NKikimrServices::PQ_READ_PROXY }); + + auto writer = MakeDataWriter(server); + TTestPQLib PQLib(server); + + server.AnnoyingClient->DescribeTopic({DEFAULT_TOPIC_NAME}); + server.AnnoyingClient->DeleteTopic2(DEFAULT_TOPIC_NAME, NPersQueue::NErrorCode::OK, false); + Sleep(TDuration::Seconds(1)); + auto [consumer, res] = PQLib.CreateConsumer({SHORT_TOPIC_NAME}, "user", 1, {}, {}, false); + Cerr << res.Response << "\n"; + + UNIT_ASSERT_EQUAL_C(res.Response.GetError().GetCode(), NPersQueue::NErrorCode::UNKNOWN_TOPIC, res.Response); + } + + Y_UNIT_TEST(SetupYqlTimeout) { + TTestServer server(PQSettings(0, 1, true, "1")); + server.EnableLogs({ NKikimrServices::PQ_WRITE_PROXY }); + server.AnnoyingClient->CreateTopic(DEFAULT_TOPIC_NAME, 10); + + auto writer = MakeDataWriter(server); + + server.AnnoyingClient->MarkNodeInHive(server.CleverServer->GetRuntime(), 0, false); + server.AnnoyingClient->KickNodeInHive(server.CleverServer->GetRuntime(), 0); + + writer.InitSession("sid1", 2, false); + } + + Y_UNIT_TEST(TestFirstClassWriteAndRead) { + auto settings = PQSettings(0, 1, true, "1"); + settings.PQConfig.SetTopicsAreFirstClassCitizen(true); + TTestServer server(settings, false); + server.StartServer(false); + UNIT_ASSERT_VALUES_EQUAL(NMsgBusProxy::MSTATUS_OK, + server.AnnoyingClient->AlterUserAttributes("/", "Root", {{"folder_id", "somefolder"}, {"cloud_id", "somecloud"}, {"database_id", "root"}})); + + + Cerr << "HERE\n"; + + PrepareForFstClass(*server.AnnoyingClient); + server.EnableLogs({ NKikimrServices::PQ_METACACHE, NKikimrServices::PERSQUEUE }, NActors::NLog::PRI_INFO); + server.EnableLogs({ NKikimrServices::PERSQUEUE_CLUSTER_TRACKER }, NActors::NLog::PRI_EMERG); + server.EnableLogs({ NKikimrServices::PERSQUEUE_READ_BALANCER }, NActors::NLog::PRI_DEBUG); + + TTestPQLib pqLib(server); + + NACLib::TDiffACL acl; +// acl.AddAccess(NACLib::EAccessType::Allow, NACLib::UpdateRow, "topic@" BUILTIN_ACL_DOMAIN); +// server.AnnoyingClient->ModifyACL("/Root/account1", "root-acc-topic", acl.SerializeAsString()); + Sleep(TDuration::Seconds(5)); + { + auto [producer, res] = pqLib.CreateProducer(FC_TOPIC_PATH, "123", {}, {}, {}, true); + UNIT_ASSERT_C(res.Response.HasInit(), res.Response); + Cerr << "Producer 1 start response: " << res.Response << "\n"; + auto f = producer->Write(1, TString(10, 'a')); + UNIT_ASSERT_C(f.GetValueSync().Response.HasAck(), f.GetValueSync().Response); + } + { + auto [producer, res] = pqLib.CreateProducer(FC_TOPIC_PATH, "123", {}, {}, {}, true); + UNIT_ASSERT_C(res.Response.HasInit(), res.Response); + Cerr << "Producer 2 start response: " << res.Response << "\n"; + auto f = producer->Write(2, TString(10, 'b')); + UNIT_ASSERT_C(f.GetValueSync().Response.HasAck(), f.GetValueSync().Response); + } + { + auto [consumer, res] = pqLib.CreateConsumer({FC_TOPIC_PATH}, "user"); + Cerr << "Consumer start response: " << res.Response << "\n"; + auto msg = consumer->GetNextMessage(); + msg.Wait(); + Cerr << "Read response: " << msg.GetValue().Response << "\n"; + UNIT_ASSERT(msg.GetValue().Type == EMT_DATA); + } + { + auto [consumer, res] = pqLib.CreateConsumer({FC_TOPIC_PATH}, "user", {}, true); + Cerr << "Consumer start response: " << res.Response << "\n"; + ui64 commitsDone = 0; + while (true) { + auto msg = consumer->GetNextMessage(); + msg.Wait(); + auto& value = msg.GetValue(); + if (value.Type == EMT_LOCK) { + TStringBuilder message; + Cerr << "Consumer lock response: " << value.Response << "\n"; + UNIT_ASSERT_VALUES_EQUAL(value.Response.GetLock().GetTopic(), "account1/root-acc-topic"); + msg.GetValue().ReadyToRead.SetValue(TLockInfo()); + } else if (value.Type == EMT_DATA) { + auto cookie = msg.GetValue().Response.GetData().GetCookie(); + consumer->Commit({cookie}); + } else { + UNIT_ASSERT(value.Type == EMT_COMMIT); + commitsDone++; + break; + } + } + UNIT_ASSERT(commitsDone > 0); + } + } + + Y_UNIT_TEST(SrcIdCompatibility) { + TString srcId1 = "test-src-id-compat", srcId2 = "test-src-id-compat2"; + TTestServer server{}; + TString topicName = "rt3.dc1--topic100"; + TString fullPath = "Root/PQ/rt3.dc1--topic100"; + TString shortTopicName = "topic100"; + server.AnnoyingClient->CreateTopic(topicName, 100); + server.EnableLogs({ NKikimrServices::PERSQUEUE }, NActors::NLog::PRI_INFO); + + auto runTest = [&] ( + const TString& topicToAdd, const TString& topicForHash, const TString& topicName, + const TString& srcId, ui32 partId, ui64 accessTime = 0 + ) { + TStringBuilder query; + auto encoded = NPQ::NSourceIdEncoding::EncodeSrcId(topicForHash, srcId); + Cerr << "===save partition with time: " << accessTime << Endl; + + if (accessTime == 0) { + accessTime = TInstant::Now().MilliSeconds(); + } + if (!topicToAdd.empty()) { // Empty means don't add anything + query << + "--!syntax_v1\n" + "UPSERT INTO `/Root/PQ/SourceIdMeta2` (Hash, Topic, SourceId, CreateTime, AccessTime, Partition) VALUES (" + << encoded.Hash << ", \"" << topicToAdd << "\", \"" << encoded.EscapedSourceId << "\", " + << TInstant::Now().MilliSeconds() << ", " << accessTime << ", " << partId << "); "; + Cerr << "Run query:\n" << query << Endl; + auto scResult = server.AnnoyingClient->RunYqlDataQuery(query); + //UNIT_ASSERT(scResult.Defined()); + } + TTestPQLib pqLib(server); + auto[producer, response] = pqLib.CreateProducer(topicName, srcId, {}, {}, {}, true); + UNIT_ASSERT_C(response.Response.HasInit(), response.Response); + UNIT_ASSERT_VALUES_EQUAL(response.Response.GetInit().GetPartition(), partId); + }; + + runTest(fullPath, shortTopicName, shortTopicName, srcId1, 5, 100); + runTest(topicName, shortTopicName, shortTopicName, srcId2, 6); + runTest("", "", shortTopicName, srcId1, 5, 100); + // Add another partition to the src mapping with different topic in key. + // Expect newer partition to be discovered. + ui64 time = (TInstant::Now() + TDuration::Hours(4)).MilliSeconds(); + runTest(topicName, shortTopicName, shortTopicName, srcId1, 7, time); + + } + +} // Y_UNIT_TEST_SUITE(TPersQueueTest) + +} // namespace NPersQueueTests +} // namespace NKikimr diff --git a/kikimr/yndx/grpc_services/persqueue/protocol_compatibility_ut.cpp b/kikimr/yndx/grpc_services/persqueue/protocol_compatibility_ut.cpp new file mode 100644 index 0000000000..f69080ea6b --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/protocol_compatibility_ut.cpp @@ -0,0 +1,80 @@ +#include <ydb/public/sdk/cpp/client/ydb_persqueue_core/ut/ut_utils/test_server.h>
+#include <ydb/core/client/server/msgbus_server_pq_metacache.h>
+#include <kikimr/persqueue/sdk/deprecated/cpp/v2/persqueue.h>
+#include <kikimr/yndx/grpc_services/persqueue/persqueue.h>
+#include <kikimr/yndx/persqueue/msgbus_server/read_session_info.h>
+
+
+namespace NKikimr {
+namespace NPersQueueTests {
+
+Y_UNIT_TEST_SUITE(TPersQueueProtocolCompatibility) {
+ Y_UNIT_TEST(GetReadInfoFromV1AboutV0Session) {
+ NKikimr::Tests::TServerSettings serverSettings = PQSettings(0);
+ serverSettings.RegisterGrpcService<NKikimr::NGRpcService::TGRpcPersQueueService>(
+ "pq",
+ NKikimr::NMsgBusProxy::CreatePersQueueMetaCacheV2Id()
+ );
+ serverSettings.SetPersQueueGetReadSessionsInfoWorkerFactory(
+ std::make_shared<NKikimr::NMsgBusProxy::TPersQueueGetReadSessionsInfoWorkerWithPQv0Factory>()
+ );
+
+ NPersQueue::TTestServer server(serverSettings);
+ server.EnableLogs({ NKikimrServices::PERSQUEUE, NKikimrServices::PQ_READ_PROXY });
+ server.AnnoyingClient->CreateTopic("rt3.dc1--topic1", 1);
+
+ NPersQueue::TPQLibSettings pqSettings;
+ pqSettings.DefaultLogger = new NPersQueue::TCerrLogger(NPersQueue::DEBUG_LOG_LEVEL);
+ THolder<NPersQueue::TPQLib> PQLib = MakeHolder<NPersQueue::TPQLib>(pqSettings);
+
+ NPersQueue::TConsumerSettings settings;
+ settings.Server = NPersQueue::TServerSetting{"localhost", server.GrpcPort};
+ settings.ClientId = "user";
+ settings.Topics = {"topic1"};
+ settings.UseLockSession = true;
+ auto consumer = PQLib->CreateConsumer(settings);
+ auto response = consumer->Start().GetValueSync();
+ UNIT_ASSERT_C(response.Response.HasInit(), response.Response);
+
+ auto msg = consumer->GetNextMessage();
+ auto value = msg.ExtractValueSync();
+ Cerr << value.Response << "\n";
+ UNIT_ASSERT(value.Response.HasLock());
+ value.ReadyToRead.SetValue(NPersQueue::TLockInfo{});
+ auto lock = value.Response.GetLock();
+ Cout << lock.DebugString() << Endl;
+ {
+ std::shared_ptr<grpc::Channel> channel;
+ std::unique_ptr<Ydb::PersQueue::V1::PersQueueService::Stub> stub;
+
+ {
+ channel = grpc::CreateChannel(
+ "localhost:" + ToString(server.GrpcPort),
+ grpc::InsecureChannelCredentials()
+ );
+ stub = Ydb::PersQueue::V1::PersQueueService::NewStub(channel);
+ }
+ {
+ Sleep(TDuration::Seconds(10));
+ Ydb::PersQueue::V1::ReadInfoRequest request;
+ Ydb::PersQueue::V1::ReadInfoResponse response;
+ request.mutable_consumer()->set_path("user");
+ request.set_get_only_original(true);
+ request.add_topics()->set_path("topic1");
+ grpc::ClientContext rcontext;
+ auto status = stub->GetReadSessionsInfo(&rcontext, request, &response);
+ UNIT_ASSERT(status.ok());
+ Ydb::PersQueue::V1::ReadInfoResult res;
+ response.operation().result().UnpackTo(&res);
+ Cerr << "Read info response: " << response << Endl << res << Endl;
+ UNIT_ASSERT_VALUES_EQUAL(res.topics_size(), 1);
+ UNIT_ASSERT(res.topics(0).status() == Ydb::StatusIds::SUCCESS);
+ }
+ }
+
+ }
+
+} // Y_UNIT_TEST_SUITE(TPersQueueProtocolCompatibility)
+
+} // namespace NPersQueueTests
+} // namespace NKikimr
diff --git a/kikimr/yndx/grpc_services/persqueue/ut/definitions.h b/kikimr/yndx/grpc_services/persqueue/ut/definitions.h new file mode 100644 index 0000000000..35f9f6bc43 --- /dev/null +++ b/kikimr/yndx/grpc_services/persqueue/ut/definitions.h @@ -0,0 +1,18 @@ +#pragma once +#include <kikimr/persqueue/sdk/deprecated/cpp/v2/ut_utils/data_writer.h> +#include <kikimr/persqueue/sdk/deprecated/cpp/v2/ut_utils/test_server.h> + +namespace NKikimr::NPersQueueTests { + + +namespace { + const TString DEFAULT_TOPIC_NAME = "rt3.dc1--topic1"; + const TString FC_TOPIC_PATH = "/Root/account1/root-acc-topic"; + const TString SHORT_TOPIC_NAME = "topic1"; +} + +NPersQueue::NTests::TPQDataWriter MakeDataWriter(NPersQueue::TTestServer& server, const TString& srcId = "source") { + return NPersQueue::NTests::TPQDataWriter(DEFAULT_TOPIC_NAME, SHORT_TOPIC_NAME, srcId, server); +} + +} // namespace NKikimr::NPersQueueTests diff --git a/kikimr/yndx/persqueue/msgbus_server/CMakeLists.txt b/kikimr/yndx/persqueue/msgbus_server/CMakeLists.txt new file mode 100644 index 0000000000..c923814a2f --- /dev/null +++ b/kikimr/yndx/persqueue/msgbus_server/CMakeLists.txt @@ -0,0 +1,19 @@ + +# This file was gererated by the build system used internally in the Yandex monorepo. +# Only simple modifications are allowed (adding source-files to targets, adding simple properties +# like target_include_directories). These modifications will be ported to original +# ya.make files by maintainers. Any complex modifications which can't be ported back to the +# original buildsystem will not be accepted. + + + +add_library(yndx-persqueue-msgbus_server) +target_link_libraries(yndx-persqueue-msgbus_server PUBLIC + contrib-libs-cxxsupp + yutil + core-client-server + yndx-grpc_services-persqueue +) +target_sources(yndx-persqueue-msgbus_server PRIVATE + ${CMAKE_SOURCE_DIR}/kikimr/yndx/persqueue/msgbus_server/read_session_info.cpp +) diff --git a/kikimr/yndx/persqueue/msgbus_server/read_session_info.cpp b/kikimr/yndx/persqueue/msgbus_server/read_session_info.cpp new file mode 100644 index 0000000000..1ccf3b0539 --- /dev/null +++ b/kikimr/yndx/persqueue/msgbus_server/read_session_info.cpp @@ -0,0 +1,16 @@ +#include "read_session_info.h" + + +namespace NKikimr { +namespace NMsgBusProxy { + +void TPersQueueGetReadSessionsInfoWorkerWithPQv0::SendStatusRequest(const TString& sessionName, TActorId actorId, const TActorContext& ctx) { + if (sessionName.EndsWith("_v1")) { + SendStatusRequest<NGRpcProxy::V1::TEvPQProxy::TEvReadSessionStatus>(actorId, ctx); + } else { + SendStatusRequest<NGRpcProxy::TEvPQProxy::TEvReadSessionStatus>(actorId, ctx); + } +} + +} // namespace NMsgBusProxy +} // namespace NKikimr diff --git a/kikimr/yndx/persqueue/msgbus_server/read_session_info.h b/kikimr/yndx/persqueue/msgbus_server/read_session_info.h new file mode 100644 index 0000000000..f92572677d --- /dev/null +++ b/kikimr/yndx/persqueue/msgbus_server/read_session_info.h @@ -0,0 +1,43 @@ +#pragma once + +#include <ydb/core/client/server/msgbus_server_pq_read_session_info.h> + +#include <kikimr/yndx/grpc_services/persqueue/grpc_pq_actor.h> + + +namespace NKikimr { +namespace NMsgBusProxy { + +class TPersQueueGetReadSessionsInfoWorkerWithPQv0 : public IPersQueueGetReadSessionsInfoWorker { +public: + using TBase = IPersQueueGetReadSessionsInfoWorker; + using TBase::TBase; + using TBase::SendStatusRequest; + + STFUNC(StateFunc) override { + switch (ev->GetTypeRewrite()) { + HFunc(NGRpcProxy::TEvPQProxy::TEvReadSessionStatusResponse, HandleStatusResponse<NGRpcProxy::TEvPQProxy::TEvReadSessionStatusResponse>); + HFunc(NGRpcProxy::V1::TEvPQProxy::TEvReadSessionStatusResponse, HandleStatusResponse<NGRpcProxy::V1::TEvPQProxy::TEvReadSessionStatusResponse>); + HFunc(TEvents::TEvUndelivered, Undelivered); + HFunc(TEvInterconnect::TEvNodeDisconnected, Disconnected); + } + } + +private: + void SendStatusRequest(const TString& sessionName, TActorId actorId, const TActorContext& ctx) override; +}; + +class TPersQueueGetReadSessionsInfoWorkerWithPQv0Factory : public IPersQueueGetReadSessionsInfoWorkerFactory { +public: + THolder<IPersQueueGetReadSessionsInfoWorker> Create( + const TActorId& parentId, + const THashMap<TString, TActorId>& readSessions, + std::shared_ptr<const TPersQueueBaseRequestProcessor::TNodesInfo> nodesInfo + ) const override { + return MakeHolder<TPersQueueGetReadSessionsInfoWorkerWithPQv0>(parentId, readSessions, nodesInfo); + } +}; + +} // namespace NMsgBusProxy +} // namespace NKikimr + diff --git a/kikimr/yndx/persqueue/read_batch_converter/CMakeLists.txt b/kikimr/yndx/persqueue/read_batch_converter/CMakeLists.txt new file mode 100644 index 0000000000..9e8f9b6006 --- /dev/null +++ b/kikimr/yndx/persqueue/read_batch_converter/CMakeLists.txt @@ -0,0 +1,18 @@ + +# This file was gererated by the build system used internally in the Yandex monorepo. +# Only simple modifications are allowed (adding source-files to targets, adding simple properties +# like target_include_directories). These modifications will be ported to original +# ya.make files by maintainers. Any complex modifications which can't be ported back to the +# original buildsystem will not be accepted. + + + +add_library(yndx-persqueue-read_batch_converter) +target_link_libraries(yndx-persqueue-read_batch_converter PUBLIC + contrib-libs-cxxsupp + yutil + api-protos-yndx +) +target_sources(yndx-persqueue-read_batch_converter PRIVATE + ${CMAKE_SOURCE_DIR}/kikimr/yndx/persqueue/read_batch_converter/read_batch_converter.cpp +) diff --git a/kikimr/yndx/persqueue/read_batch_converter/read_batch_converter.cpp b/kikimr/yndx/persqueue/read_batch_converter/read_batch_converter.cpp new file mode 100644 index 0000000000..bca03dc72f --- /dev/null +++ b/kikimr/yndx/persqueue/read_batch_converter/read_batch_converter.cpp @@ -0,0 +1,43 @@ +#include "read_batch_converter.h" + +namespace NPersQueue { + +static void Convert(const ReadResponse::BatchedData::PartitionData& partition, ReadResponse::Data::MessageBatch* dstBatch) { + dstBatch->set_topic(partition.topic()); + dstBatch->set_partition(partition.partition()); + for (const ReadResponse::BatchedData::Batch& batch : partition.batch()) { + for (const ReadResponse::BatchedData::MessageData& message : batch.message_data()) { + ReadResponse::Data::Message* const dstMessage = dstBatch->add_message(); + dstMessage->set_data(message.data()); + dstMessage->set_offset(message.offset()); + + MessageMeta* const meta = dstMessage->mutable_meta(); + meta->set_source_id(batch.source_id()); + meta->set_seq_no(message.seq_no()); + meta->set_create_time_ms(message.create_time_ms()); + meta->set_write_time_ms(batch.write_time_ms()); + meta->set_codec(message.codec()); + meta->set_ip(batch.ip()); + meta->set_uncompressed_size(message.uncompressed_size()); + if (batch.has_extra_fields()) { + *meta->mutable_extra_fields() = batch.extra_fields(); + } + } + } +} + +void ConvertToOldBatch(ReadResponse& response) { + if (!response.has_batched_data()) { + return; + } + ReadResponse::BatchedData data; + data.Swap(response.mutable_batched_data()); + + ReadResponse::Data& dstData = *response.mutable_data(); // this call will clear BatchedData field + dstData.set_cookie(data.cookie()); + for (const ReadResponse::BatchedData::PartitionData& partition : data.partition_data()) { + Convert(partition, dstData.add_message_batch()); + } +} + +} // namespace NPersQueue diff --git a/kikimr/yndx/persqueue/read_batch_converter/read_batch_converter.h b/kikimr/yndx/persqueue/read_batch_converter/read_batch_converter.h new file mode 100644 index 0000000000..ddf32da4ae --- /dev/null +++ b/kikimr/yndx/persqueue/read_batch_converter/read_batch_converter.h @@ -0,0 +1,10 @@ +#pragma once +#include <kikimr/yndx/api/protos/persqueue.pb.h> + +namespace NPersQueue { + +// Converts responses with BatchedData field to responses with Data field. +// Other responses will be leaved unchanged. +void ConvertToOldBatch(ReadResponse& response); + +} // namespace NPersQueue |