aboutsummaryrefslogtreecommitdiffstats
path: root/kikimr/yndx
diff options
context:
space:
mode:
authorkomels <komels@yandex-team.ru>2022-04-14 13:10:53 +0300
committerkomels <komels@yandex-team.ru>2022-04-14 13:10:53 +0300
commit21c9b0e6b039e9765eb414c406c2b86e8cea6850 (patch)
treef40ebc18ff8958dfbd189954ad024043ca983ea5 /kikimr/yndx
parent9a4effa852abe489707139c2b260dccc6f4f9aa9 (diff)
downloadydb-21c9b0e6b039e9765eb414c406c2b86e8cea6850.tar.gz
Final part on compatibility layer: LOGBROKER-7215
ref:777c67aadbf705d19034a09a792b2df61ba53697
Diffstat (limited to 'kikimr/yndx')
-rw-r--r--kikimr/yndx/api/grpc/CMakeLists.txt43
-rw-r--r--kikimr/yndx/api/grpc/persqueue.proto68
-rw-r--r--kikimr/yndx/api/grpc/ydb_yndx_keyvalue_v1.proto43
-rw-r--r--kikimr/yndx/api/grpc/ydb_yndx_rate_limiter_v1.proto35
-rw-r--r--kikimr/yndx/api/protos/CMakeLists.txt40
-rw-r--r--kikimr/yndx/api/protos/persqueue.proto335
-rw-r--r--kikimr/yndx/api/protos/ydb_yndx_keyvalue.proto460
-rw-r--r--kikimr/yndx/api/protos/ydb_yndx_rate_limiter.proto273
-rw-r--r--kikimr/yndx/grpc_services/persqueue/CMakeLists.txt38
-rw-r--r--kikimr/yndx/grpc_services/persqueue/grpc_pq_actor.h928
-rw-r--r--kikimr/yndx/grpc_services/persqueue/grpc_pq_clusters_updater_actor.cpp86
-rw-r--r--kikimr/yndx/grpc_services/persqueue/grpc_pq_clusters_updater_actor.h77
-rw-r--r--kikimr/yndx/grpc_services/persqueue/grpc_pq_read.cpp268
-rw-r--r--kikimr/yndx/grpc_services/persqueue/grpc_pq_read.h146
-rw-r--r--kikimr/yndx/grpc_services/persqueue/grpc_pq_read_actor.cpp2585
-rw-r--r--kikimr/yndx/grpc_services/persqueue/grpc_pq_session.h317
-rw-r--r--kikimr/yndx/grpc_services/persqueue/grpc_pq_write.cpp221
-rw-r--r--kikimr/yndx/grpc_services/persqueue/grpc_pq_write.h148
-rw-r--r--kikimr/yndx/grpc_services/persqueue/grpc_pq_write_actor.cpp1055
-rw-r--r--kikimr/yndx/grpc_services/persqueue/persqueue.cpp59
-rw-r--r--kikimr/yndx/grpc_services/persqueue/persqueue.h49
-rw-r--r--kikimr/yndx/grpc_services/persqueue/persqueue_compat_ut.cpp122
-rw-r--r--kikimr/yndx/grpc_services/persqueue/persqueue_ut.cpp2405
-rw-r--r--kikimr/yndx/grpc_services/persqueue/protocol_compatibility_ut.cpp80
-rw-r--r--kikimr/yndx/grpc_services/persqueue/ut/definitions.h18
-rw-r--r--kikimr/yndx/persqueue/msgbus_server/CMakeLists.txt19
-rw-r--r--kikimr/yndx/persqueue/msgbus_server/read_session_info.cpp16
-rw-r--r--kikimr/yndx/persqueue/msgbus_server/read_session_info.h43
-rw-r--r--kikimr/yndx/persqueue/read_batch_converter/CMakeLists.txt18
-rw-r--r--kikimr/yndx/persqueue/read_batch_converter/read_batch_converter.cpp43
-rw-r--r--kikimr/yndx/persqueue/read_batch_converter/read_batch_converter.h10
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(&parameters);
+ 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(&parameters);
+
+ 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