summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorIrene Skvortsova <[email protected]>2025-07-10 15:50:53 +0300
committerGitHub <[email protected]>2025-07-10 15:50:53 +0300
commit517d65bc7942b830f2b9808f233913582464ef81 (patch)
tree0080d9ca3a8c7e4c6973bc51c3ff7eb0791b81bc
parent666035fb9411bf8f58bfdba3a24cf82fed7eed37 (diff)
Add describe consumer group for akhq (#19768)
Co-authored-by: Irina Skvortsova <[email protected]>
-rw-r--r--ydb/core/kafka_proxy/actors/actors.h2
-rw-r--r--ydb/core/kafka_proxy/actors/kafka_api_versions_actor.cpp2
-rw-r--r--ydb/core/kafka_proxy/actors/kafka_describe_groups_actor.cpp262
-rw-r--r--ydb/core/kafka_proxy/actors/kafka_describe_groups_actor.h116
-rw-r--r--ydb/core/kafka_proxy/actors/kafka_list_groups_actor.cpp184
-rw-r--r--ydb/core/kafka_proxy/actors/kafka_list_groups_actor.h99
-rw-r--r--ydb/core/kafka_proxy/actors/kafka_state_name_to_int.cpp19
-rw-r--r--ydb/core/kafka_proxy/actors/kafka_state_name_to_int.h12
-rw-r--r--ydb/core/kafka_proxy/kafka_connection.cpp23
-rw-r--r--ydb/core/kafka_proxy/kafka_messages.cpp1604
-rw-r--r--ydb/core/kafka_proxy/kafka_messages.h3852
-rw-r--r--ydb/core/kafka_proxy/ut/kafka_test_client.cpp36
-rw-r--r--ydb/core/kafka_proxy/ut/kafka_test_client.h20
-rw-r--r--ydb/core/kafka_proxy/ut/ut_protocol.cpp297
-rw-r--r--ydb/core/kafka_proxy/ya.make3
15 files changed, 4276 insertions, 2255 deletions
diff --git a/ydb/core/kafka_proxy/actors/actors.h b/ydb/core/kafka_proxy/actors/actors.h
index 643467f7cee..277d180ca63 100644
--- a/ydb/core/kafka_proxy/actors/actors.h
+++ b/ydb/core/kafka_proxy/actors/actors.h
@@ -178,6 +178,8 @@ NActors::IActor* CreateKafkaBalancerActor(const TContext::TPtr context, ui64 coo
NActors::IActor* CreateKafkaSaslHandshakeActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr<TSaslHandshakeRequestData>& message);
NActors::IActor* CreateKafkaSaslAuthActor(const TContext::TPtr context, const ui64 correlationId, const NKikimr::NRawSocket::TSocketDescriptor::TSocketAddressType address, const TMessagePtr<TSaslAuthenticateRequestData>& message);
NActors::IActor* CreateKafkaListOffsetsActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr<TListOffsetsRequestData>& message);
+NActors::IActor* CreateKafkaListGroupsActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr<TListGroupsRequestData>& message);
+NActors::IActor* CreateKafkaDescribeGroupsActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr<TDescribeGroupsRequestData>& message);
NActors::IActor* CreateKafkaFetchActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr<TFetchRequestData>& message);
NActors::IActor* CreateKafkaFindCoordinatorActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr<TFindCoordinatorRequestData>& message);
NActors::IActor* CreateKafkaOffsetCommitActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr<TOffsetCommitRequestData>& message);
diff --git a/ydb/core/kafka_proxy/actors/kafka_api_versions_actor.cpp b/ydb/core/kafka_proxy/actors/kafka_api_versions_actor.cpp
index 600ee6ff50c..1bb42898c47 100644
--- a/ydb/core/kafka_proxy/actors/kafka_api_versions_actor.cpp
+++ b/ydb/core/kafka_proxy/actors/kafka_api_versions_actor.cpp
@@ -41,6 +41,8 @@ TApiVersionsResponseData::TPtr GetApiVersions() {
AddApiKey<TSaslHandshakeRequestData>(response->ApiKeys, SASL_HANDSHAKE, {.MaxVersion=1});
AddApiKey<TSaslAuthenticateRequestData>(response->ApiKeys, SASL_AUTHENTICATE, {.MaxVersion=2});
AddApiKey<TListOffsetsRequestData>(response->ApiKeys, LIST_OFFSETS, {.MinVersion=1, .MaxVersion=1});
+ AddApiKey<TListGroupsRequestData>(response->ApiKeys, LIST_GROUPS, {.MinVersion=0, .MaxVersion=4});
+ AddApiKey<TDescribeGroupsRequestData>(response->ApiKeys, DESCRIBE_GROUPS, {.MinVersion=0, .MaxVersion=5});
AddApiKey<TFetchRequestData>(response->ApiKeys, FETCH, {.MaxVersion=3});
AddApiKey<TJoinGroupRequestData>(response->ApiKeys, JOIN_GROUP, {.MaxVersion=9});
AddApiKey<TSyncGroupRequestData>(response->ApiKeys, SYNC_GROUP, {.MaxVersion=3});
diff --git a/ydb/core/kafka_proxy/actors/kafka_describe_groups_actor.cpp b/ydb/core/kafka_proxy/actors/kafka_describe_groups_actor.cpp
new file mode 100644
index 00000000000..19bc8dadea7
--- /dev/null
+++ b/ydb/core/kafka_proxy/actors/kafka_describe_groups_actor.cpp
@@ -0,0 +1,262 @@
+#include <ydb/library/actors/core/actor.h>
+#include <ydb/core/base/ticket_parser.h>
+#include <ydb/core/grpc_services/local_rpc/local_rpc.h>
+#include <ydb/core/kafka_proxy/kafka_events.h>
+#include <ydb/public/api/grpc/ydb_auth_v1.grpc.pb.h>
+#include <ydb/core/kafka_proxy/kafka_transactions_coordinator.h>
+#include <ydb/services/persqueue_v1/actors/schema_actors.h>
+#include <ydb/core/kafka_proxy/kafka_transactional_producers_initializers.h>
+#include "ydb/core/kafka_proxy/kafka_messages.h"
+#include <ydb/core/kafka_proxy/kafka_consumer_groups_metadata_initializers.h>
+#include <ydb/core/kqp/common/simple/services.h>
+
+#include <util/generic/cast.h>
+#include <regex>
+
+#include "actors.h"
+#include "kafka_describe_groups_actor.h"
+#include "kafka_state_name_to_int.h"
+
+
+namespace NKafka {
+
+NActors::IActor* CreateKafkaDescribeGroupsActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr<TDescribeGroupsRequestData>& message) {
+ return new TKafkaDescribeGroupsActor(context, correlationId, message);
+}
+
+void TKafkaDescribeGroupsActor::Bootstrap(const NActors::TActorContext& ctx) {
+ if (NKikimr::AppData()->FeatureFlags.GetEnableKafkaNativeBalancing()) {
+ Kqp = std::make_unique<TKqpTxHelper>(Context->DatabasePath);
+ Kqp->SendInitTableRequest(ctx, NKikimr::NGRpcProxy::V1::TKafkaConsumerGroupsMetaInitManager::GetInstant());
+ Kqp->SendInitTableRequest(ctx, NKikimr::NGRpcProxy::V1::TKafkaConsumerMembersMetaInitManager::GetInstant());
+ Become(&TKafkaDescribeGroupsActor::StateWork);
+ auto wakeup = std::make_unique<TEvents::TEvWakeup>();
+ ctx.ActorSystem()->Schedule(
+ TDuration::Seconds(WAIT_REQUESTS_SECONDS),
+ new IEventHandle(SelfId(), SelfId(), wakeup.release())
+ );
+ } else {
+ KAFKA_LOG_ERROR("No EnableKafkaNativeBalancing FeatureFlag set.");
+ TDescribeGroupsResponseData groupsDescriptionResponseWithError;
+ Send(Context->ConnectionId,
+ new TEvKafka::TEvResponse(CorrelationId,
+ std::make_shared<TDescribeGroupsResponseData>(std::move(groupsDescriptionResponseWithError)),
+ EKafkaErrors::UNSUPPORTED_VERSION));
+ }
+}
+
+void TKafkaDescribeGroupsActor::Handle(NMetadata::NProvider::TEvManagerPrepared::TPtr&, const TActorContext& ctx) {
+ KAFKA_LOG_D("Received TEvManagerPrepared. Sending create session request to KQP.");
+ InitedTablesCount++;
+ if (InitedTablesCount == TABLES_TO_INIT_COUNT) {
+ StartKqpSession(ctx);
+ }
+}
+
+
+void TKafkaDescribeGroupsActor::Handle(NKqp::TEvKqp::TEvCreateSessionResponse::TPtr& ev, const TActorContext& ctx) {
+ KAFKA_LOG_D("KQP session created");
+ if (!Kqp->HandleCreateSessionResponse(ev, ctx)) {
+ SendFailResponse(EKafkaErrors::BROKER_NOT_AVAILABLE, "Failed to create KQP session");
+ Die(ctx);
+ return;
+ }
+ SendToKqpDescribeGroupsMetadataRequest(ctx);
+}
+
+void TKafkaDescribeGroupsActor::Handle(NKqp::TEvKqp::TEvQueryResponse::TPtr& ev, const TActorContext& ctx) {
+ KAFKA_LOG_D("Received query response from KQP DescribeGroups request");
+ if (auto error = GetErrorFromYdbResponse(ev)) {
+ KAFKA_LOG_W(error);
+ SendFailResponse(EKafkaErrors::BROKER_NOT_AVAILABLE, *error);
+ Die(ctx);
+ return;
+ }
+ HandleSelectResponse(*ev->Get(), ctx);
+}
+
+void TKafkaDescribeGroupsActor::Handle(TEvents::TEvWakeup::TPtr&, const TActorContext& ctx) {
+ KAFKA_LOG_W("Sending fail response because of request timeout " << WAIT_REQUESTS_SECONDS << " sec");
+ Send(Context->ConnectionId,
+ new TEvKafka::TEvResponse(CorrelationId, BuildResponse(), EKafkaErrors::REQUEST_TIMED_OUT));
+ Die(ctx);
+}
+
+void TKafkaDescribeGroupsActor::Die(const TActorContext &ctx) {
+ KAFKA_LOG_D("Dying.");
+ if (Kqp) {
+ Kqp->CloseKqpSession(ctx);
+ }
+ TBase::Die(ctx);
+}
+
+void TKafkaDescribeGroupsActor::StartKqpSession(const TActorContext& ctx) {
+ KAFKA_LOG_D("Sending create session request to KQP for database " << DatabasePath);
+ Kqp->SendCreateSessionRequest(ctx);
+}
+
+void TKafkaDescribeGroupsActor::HandleSelectResponse(const NKqp::TEvKqp::TEvQueryResponse& response, const TActorContext& ctx) {
+ KAFKA_LOG_D("Handling Select Response for DescribeGroups. SELECT result size: " << response.Record.GetResponse().GetYdbResults().size());
+ if (response.Record.GetResponse().GetYdbResults().size() != 2) {
+ TString errorMessage = TStringBuilder() << "KQP returned wrong number of result sets on SELECT query. Expected 2, got " << response.Record.GetResponse().GetYdbResults().size() << ".";
+ KAFKA_LOG_W(errorMessage);
+ return;
+ }
+ ParseGroupDescriptionMetadata(response);
+ auto responseDescribeGroups = BuildResponse();
+ Send(Context->ConnectionId,
+ new TEvKafka::TEvResponse(CorrelationId, responseDescribeGroups,
+ EKafkaErrors::NONE_ERROR));
+ Die(ctx);
+}
+
+void TKafkaDescribeGroupsActor::ParseGroupDescriptionMetadata(const NKqp::TEvKqp::TEvQueryResponse& response) {
+ KAFKA_LOG_D("Parsing Groups metadata");
+ ParseGroupMetadata(response);
+ KAFKA_LOG_D("Parsing Members metadata");
+ ParseMembersMetadata(response);
+}
+
+void TKafkaDescribeGroupsActor::FillInMemberMetadata(NKafka::TDescribeGroupsResponseData::TDescribedGroup& describedGroup,
+ NKafka::TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember& groupMember,
+ const TString& protoStr) {
+ NKafka::TWorkerState workerState;
+ groupMember.MemberMetadataStr = "";
+ if (protoStr.empty() || workerState.ParseFromString(protoStr)) {
+ const auto& protocols = workerState.protocols();
+ TString protocol = *describedGroup.ProtocolData;
+ for (const auto& p : protocols) {
+ if (p.protocol_name() == protocol) {
+ groupMember.MemberMetadataStr = TString(p.metadata());
+ break;
+ }
+ }
+ }
+ groupMember.MemberMetadata = groupMember.MemberMetadataStr;
+}
+
+void TKafkaDescribeGroupsActor::ParseGroupMetadata(const NKqp::TEvKqp::TEvQueryResponse& response) {
+ NYdb::TResultSetParser parserGroup(response.Record.GetResponse().GetYdbResults(REQUEST_GROUPS_QUERY_INDEX));
+ while (parserGroup.TryNextRow()) {
+ TString groupId = parserGroup.ColumnParser("consumer_group").GetUtf8().c_str();
+ ui64 state = parserGroup.ColumnParser("state").GetUint64();
+ TString protocolType = parserGroup.ColumnParser("protocol_type").GetUtf8().c_str();
+ TString protocolData = parserGroup.ColumnParser("protocol").GetOptionalUtf8().value_or("").c_str();
+ auto& describedGroup = GroupIdToDescription[groupId];
+ describedGroup.ProtocolData = protocolData;
+ describedGroup.GroupId = groupId;
+ describedGroup.ProtocolType = protocolType;
+ describedGroup.GroupState = NKafka::NConsumer::NumbersToStatesMapping.at(state);
+ }
+}
+
+void TKafkaDescribeGroupsActor::ParseMembersMetadata(const NKqp::TEvKqp::TEvQueryResponse& response) {
+ NYdb::TResultSetParser parserMembers(response.Record.GetResponse().GetYdbResults(REQUEST_MEMBERS_QUERY_INDEX));
+ while (parserMembers.TryNextRow()) {
+ TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember groupMember;
+ TString groupId = parserMembers.ColumnParser("consumer_group").GetUtf8().c_str();
+ TString memberId = parserMembers.ColumnParser("member_id").GetUtf8().c_str();
+ TString groupInstanceId = parserMembers.ColumnParser("instance_id").GetOptionalUtf8().value_or("").c_str();
+
+ TString protoStr = parserMembers.ColumnParser("worker_state_proto").GetOptionalString().value_or("");
+ groupMember.GroupInstanceId = groupInstanceId;
+ groupMember.MemberId = memberId;
+ groupMember.MemberAssignmentStr = parserMembers.ColumnParser("assignment").GetOptionalString().value_or("");
+ groupMember.MemberAssignment = TString(groupMember.MemberAssignmentStr);
+
+ auto& describedGroup = GroupIdToDescription[groupId];
+ FillInMemberMetadata(describedGroup, groupMember, protoStr);
+
+ describedGroup.Members.push_back(std::move(groupMember));
+ describedGroup.ErrorCode = EKafkaErrors::NONE_ERROR;
+ }
+}
+
+NYdb::TParams TKafkaDescribeGroupsActor::BuildSelectParams() {
+ NYdb::TParamsBuilder params;
+ params.AddParam("$Database").Utf8(DatabasePath).Build();
+ auto& groupIds = params.AddParam("$GroupIds").BeginList();
+
+ KAFKA_LOG_D(TStringBuilder() << "Groups count: " << DescribeGroupsRequestData->Groups.size());
+
+ for (auto& groupId: DescribeGroupsRequestData->Groups) {
+ groupIds.AddListItem().Utf8(*groupId);
+ }
+ groupIds.EndList().Build();
+
+ return params.Build();
+}
+
+TString TKafkaDescribeGroupsActor::GetYqlWithTableNames(const TString& templateStr) {
+ TString templateWithCorrectTableNames = std::regex_replace(
+ templateStr.c_str(),
+ std::regex("<consumer_members_table_name>"),
+ NKikimr::NGRpcProxy::V1::TKafkaConsumerMembersMetaInitManager::GetInstant()->GetStorageTablePath().c_str()
+ );
+
+ templateWithCorrectTableNames = std::regex_replace(
+ templateWithCorrectTableNames.c_str(),
+ std::regex("<consumer_groups_table_name>"),
+ NKikimr::NGRpcProxy::V1::TKafkaConsumerGroupsMetaInitManager::GetInstant()->GetStorageTablePath().c_str()
+ );
+
+ return templateWithCorrectTableNames;
+}
+
+std::shared_ptr<TDescribeGroupsResponseData> TKafkaDescribeGroupsActor::BuildResponse() {
+ TDescribeGroupsResponseData describeGroupsResponse;
+ for (auto& groupId : DescribeGroupsRequestData->Groups) {
+ if (GroupIdToDescription.find(groupId) != GroupIdToDescription.end()) {
+ auto& groupDescription = GroupIdToDescription.at(*groupId);
+ groupDescription.ErrorCode = EKafkaErrors::NONE_ERROR;
+ describeGroupsResponse.Groups.push_back(std::move(groupDescription));
+ } else {
+ TDescribeGroupsResponseData::TDescribedGroup groupDescription;
+ groupDescription.ErrorCode = EKafkaErrors::GROUP_ID_NOT_FOUND;
+ groupDescription.GroupId = groupId;
+ describeGroupsResponse.Groups.push_back(std::move(groupDescription));
+ }
+ }
+ return std::make_shared<TDescribeGroupsResponseData>(std::move(describeGroupsResponse));
+}
+
+void TKafkaDescribeGroupsActor::SendToKqpDescribeGroupsMetadataRequest(const TActorContext& ctx) {
+ if (DescribeGroupsRequestData->Groups.size() == 0) {
+ auto response = BuildResponse();
+ Send(Context->ConnectionId,
+ new TEvKafka::TEvResponse(CorrelationId,
+ response,
+ EKafkaErrors::NONE_ERROR));
+ Die(ctx);
+ }
+ Kqp->SendYqlRequest(GetYqlWithTableNames(SELECT_GROUPS_DESCRIPTION),
+ BuildSelectParams(),
+ KqpCookie,
+ ctx,
+ false);
+}
+
+void TKafkaDescribeGroupsActor::SendFailResponse(EKafkaErrors errorCode, const std::optional<TString>& errorMessage) {
+ for (auto& groupId : DescribeGroupsRequestData->Groups) {
+ GroupIdToDescription[*groupId].ErrorCode = errorCode;
+ }
+ if (errorMessage.has_value()) {
+ KAFKA_LOG_W("Sending fail response with error code: " << errorCode << ". Reason: " << errorMessage);
+ } else {
+ KAFKA_LOG_W("Sending fail response with error code: " << errorCode);
+ }
+
+ Send(Context->ConnectionId,
+ new TEvKafka::TEvResponse(CorrelationId, BuildResponse(), errorCode));
+}
+
+TMaybe<TString> TKafkaDescribeGroupsActor::GetErrorFromYdbResponse(NKqp::TEvKqp::TEvQueryResponse::TPtr& ev) {
+ TStringBuilder builder = TStringBuilder() << "Recieved error on request to KQP. ErrorCode: " << ev->Get()->Record.GetYdbStatus();
+ if (ev->Get()->Record.GetYdbStatus() != Ydb::StatusIds::SUCCESS) {
+ return builder << "Unexpected YDB status in TEvQueryResponse. Expected YDB SUCCESS status, Actual: " << ev->Get()->Record.GetYdbStatus();
+ } else {
+ return {};
+ }
+}
+
+} // namespace NKafka
diff --git a/ydb/core/kafka_proxy/actors/kafka_describe_groups_actor.h b/ydb/core/kafka_proxy/actors/kafka_describe_groups_actor.h
new file mode 100644
index 00000000000..08b93e21223
--- /dev/null
+++ b/ydb/core/kafka_proxy/actors/kafka_describe_groups_actor.h
@@ -0,0 +1,116 @@
+#pragma once
+
+#include <ydb/library/actors/core/actor_bootstrapped.h>
+#include <ydb/core/kafka_proxy/kafka_events.h>
+#include "ydb/library/aclib/aclib.h"
+#include <ydb/core/protos/kafka.pb.h>
+#include <ydb/core/kafka_proxy/kqp_helper.h>
+#include <ydb/services/persqueue_v1/actors/events.h>
+#include "../kafka_consumer_members_metadata_initializers.h"
+
+
+#include "actors.h"
+
+namespace NKafka {
+
+static const TString SELECT_GROUPS_DESCRIPTION = R"sql(
+ DECLARE $Database AS Utf8;
+ DECLARE $GroupIds AS List<Utf8>;
+
+ SELECT * FROM (
+ SELECT
+ `<consumer_groups_table_name>`.*,
+ ROW_NUMBER() OVER (PARTITION BY consumer_group ORDER BY generation DESC) AS row_num
+ FROM `<consumer_groups_table_name>`
+ WHERE database = $Database
+ ) WHERE row_num = 1 AND consumer_group IN $GroupIds;
+
+ SELECT * FROM (
+ SELECT
+ `<consumer_members_table_name>`.*,
+ DENSE_RANK() OVER (PARTITION BY consumer_group ORDER BY generation DESC) AS row_num
+ FROM `<consumer_members_table_name>`
+ WHERE database = $Database
+ ) WHERE row_num = 1 AND consumer_group IN $GroupIds;
+
+
+)sql";
+
+const ui32 TABLES_TO_INIT_COUNT = 2;
+
+class TKafkaDescribeGroupsActor: public NActors::TActorBootstrapped<TKafkaDescribeGroupsActor> {
+
+using TBase = TActorBootstrapped<TKafkaDescribeGroupsActor>;
+
+public:
+ TKafkaDescribeGroupsActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr<TDescribeGroupsRequestData>& message)
+ : Context(context)
+ , CorrelationId(correlationId)
+ , DescribeGroupsRequestData(message)
+ , DatabasePath(context->DatabasePath) {
+ }
+
+
+void Bootstrap(const NActors::TActorContext& ctx);
+
+
+TStringBuilder LogPrefix() const {
+ return TStringBuilder() << "KafkaDescribeGroupsActor{DatabasePath=" << DatabasePath << "}: ";
+}
+
+struct TDescribeGroupsKqpQuery {
+ TString GroupId;
+ ui64 KqpCookie;
+};
+
+ui32 WAIT_REQUESTS_SECONDS = 20;
+ui32 REQUEST_GROUPS_QUERY_INDEX = 0;
+ui32 REQUEST_MEMBERS_QUERY_INDEX = 1;
+
+private:
+ STATEFN(StateWork) {
+ switch (ev->GetTypeRewrite()) {
+ HFunc(NKqp::TEvKqp::TEvCreateSessionResponse, Handle);
+ HFunc(NKqp::TEvKqp::TEvQueryResponse, Handle);
+ HFunc(NMetadata::NProvider::TEvManagerPrepared, Handle);
+ HFunc(TEvents::TEvWakeup, Handle);
+ }
+ }
+
+ void Handle(NMetadata::NProvider::TEvManagerPrepared::TPtr&, const TActorContext& ctx);
+ void Handle(NKqp::TEvKqp::TEvCreateSessionResponse::TPtr& ev, const TActorContext& ctx);
+ void Handle(NKqp::TEvKqp::TEvQueryResponse::TPtr& ev, const TActorContext& ctx);
+ void Handle(TEvents::TEvWakeup::TPtr&, const TActorContext& ctx);
+
+ void Die(const TActorContext &ctx);
+
+ void StartKqpSession(const TActorContext& ctx);
+ void HandleSelectResponse(const NKqp::TEvKqp::TEvQueryResponse& response, const TActorContext& ctx);
+ void ParseGroupDescriptionMetadata(const NKqp::TEvKqp::TEvQueryResponse& response);
+ void ParseGroupMetadata(const NKqp::TEvKqp::TEvQueryResponse& response);
+ void ParseMembersMetadata(const NKqp::TEvKqp::TEvQueryResponse& response);
+
+ void FillInMemberMetadata(TDescribeGroupsResponseData::TDescribedGroup &describedGroup,
+ TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember& groupMember,
+ const TString& protoStr);
+ NYdb::TParams BuildSelectParams();
+ TString GetYqlWithTableNames(const TString& templateStr);
+ std::shared_ptr<TDescribeGroupsResponseData> BuildResponse();
+ void SendToKqpDescribeGroupsMetadataRequest(const TActorContext& ctx);
+ void SendFailResponse(EKafkaErrors errorCode, const std::optional<TString>& errorMessage = std::nullopt);
+ TMaybe<TString> GetErrorFromYdbResponse(NKqp::TEvKqp::TEvQueryResponse::TPtr& ev);
+
+ const TContext::TPtr Context;
+ const ui64 CorrelationId;
+ const TMessagePtr<TDescribeGroupsRequestData> DescribeGroupsRequestData;
+ std::map<std::optional<TString>, TDescribeGroupsResponseData::TDescribedGroup> GroupIdToDescription;
+
+ std::unique_ptr<TKqpTxHelper> Kqp;
+ const TString DatabasePath;
+
+ TString KqpSessionId;
+ ui64 KqpCookie = 0;
+ ui32 InitedTablesCount = 0;
+};
+
+} // namespace NKafka
diff --git a/ydb/core/kafka_proxy/actors/kafka_list_groups_actor.cpp b/ydb/core/kafka_proxy/actors/kafka_list_groups_actor.cpp
new file mode 100644
index 00000000000..910ce75f146
--- /dev/null
+++ b/ydb/core/kafka_proxy/actors/kafka_list_groups_actor.cpp
@@ -0,0 +1,184 @@
+#include <ydb/library/actors/core/actor.h>
+#include <ydb/core/base/ticket_parser.h>
+#include <ydb/core/grpc_services/local_rpc/local_rpc.h>
+#include <ydb/core/kafka_proxy/kafka_events.h>
+#include <ydb/public/api/grpc/ydb_auth_v1.grpc.pb.h>
+#include <ydb/core/kafka_proxy/kafka_transactions_coordinator.h>
+#include <ydb/services/persqueue_v1/actors/schema_actors.h>
+#include <ydb/core/kafka_proxy/kafka_transactional_producers_initializers.h>
+#include "ydb/core/kafka_proxy/kafka_messages.h"
+#include <ydb/core/kafka_proxy/kafka_consumer_groups_metadata_initializers.h>
+#include <ydb/core/kqp/common/simple/services.h>
+#include <util/generic/cast.h>
+#include <regex>
+
+#include "actors.h"
+#include "kafka_list_groups_actor.h"
+#include "kafka_state_name_to_int.h"
+
+
+namespace NKafka {
+
+NActors::IActor* CreateKafkaListGroupsActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr<TListGroupsRequestData>& message) {
+ return new TKafkaListGroupsActor(context, correlationId, message);
+}
+
+void TKafkaListGroupsActor::Bootstrap(const NActors::TActorContext& ctx) {
+ Kqp = std::make_unique<TKqpTxHelper>(Context->DatabasePath);
+ if (NKikimr::AppData()->FeatureFlags.GetEnableKafkaNativeBalancing()) {
+ Kqp->SendInitTableRequest(ctx, NKikimr::NGRpcProxy::V1::TKafkaConsumerGroupsMetaInitManager::GetInstant());
+ Become(&TKafkaListGroupsActor::StateWork);
+ } else {
+ KAFKA_LOG_ERROR("No EnableKafkaNativeBalancing FeatureFlag set.");
+ TListGroupsResponseData consumerGroupsResponseWithError;
+ Send(Context->ConnectionId,
+ new TEvKafka::TEvResponse(CorrelationId,
+ std::make_shared<TListGroupsResponseData>(std::move(consumerGroupsResponseWithError)),
+ EKafkaErrors::UNSUPPORTED_VERSION));
+ }
+}
+
+void TKafkaListGroupsActor::Handle(NMetadata::NProvider::TEvManagerPrepared::TPtr&, const TActorContext& ctx) {
+ KAFKA_LOG_D("Received TEvManagerPrepared. Sending create session request to KQP.");
+ StartKqpSession(ctx);
+}
+
+void TKafkaListGroupsActor::Handle(NKqp::TEvKqp::TEvCreateSessionResponse::TPtr& ev, const TActorContext& ctx) {
+ KAFKA_LOG_D("KQP session created");
+ if (!Kqp->HandleCreateSessionResponse(ev, ctx)) {
+ SendFailResponse(EKafkaErrors::BROKER_NOT_AVAILABLE, "Failed to create KQP session");
+ Die(ctx);
+ return;
+ }
+
+ SendToKqpConsumerGroupsRequest(ctx);
+}
+
+void TKafkaListGroupsActor::Handle(NKqp::TEvKqp::TEvQueryResponse::TPtr& ev, const TActorContext& ctx) {
+ KAFKA_LOG_D("Received query response from KQP ListGroups request");
+ if (auto error = GetErrorFromYdbResponse(ev)) {
+ KAFKA_LOG_W(error);
+ SendFailResponse(EKafkaErrors::BROKER_NOT_AVAILABLE, error->data());
+ Die(ctx);
+ return;
+ }
+
+ HandleSelectResponse(*ev->Get(), ctx);
+}
+
+void TKafkaListGroupsActor::HandleSelectResponse(const NKqp::TEvKqp::TEvQueryResponse& response, const TActorContext& ctx) {
+ KAFKA_LOG_D("Handling Select Response " << response.Record.GetResponse().GetYdbResults().size());
+ if (response.Record.GetResponse().GetYdbResults().size() != 1) {
+ TString errorMessage = TStringBuilder() << "KQP returned wrong number of result sets on SELECT query. Expected 1, got " << response.Record.GetResponse().GetYdbResults().size() << ".";
+ KAFKA_LOG_W(errorMessage);
+ SendFailResponse(EKafkaErrors::BROKER_NOT_AVAILABLE, errorMessage);
+ Die(ctx);
+ return;
+ }
+
+ TListGroupsResponseData consumerGroupsResponse = ParseGroupsMetadata(response);
+
+ auto responseListGroups = BuildResponse(consumerGroupsResponse);
+ Send(Context->ConnectionId,
+ new TEvKafka::TEvResponse(CorrelationId, responseListGroups, EKafkaErrors::NONE_ERROR));
+}
+
+void TKafkaListGroupsActor::Die(const TActorContext &ctx) {
+ KAFKA_LOG_D("Dying.");
+ if (Kqp) {
+ Kqp->CloseKqpSession(ctx);
+ }
+ TBase::Die(ctx);
+}
+
+void TKafkaListGroupsActor::StartKqpSession(const TActorContext& ctx) {
+ KAFKA_LOG_D("Sending create session request to KQP for database " << DatabasePath);
+ Kqp->SendCreateSessionRequest(ctx);
+}
+
+TListGroupsResponseData TKafkaListGroupsActor::ParseGroupsMetadata(const NKqp::TEvKqp::TEvQueryResponse& response) {
+ KAFKA_LOG_D("Parsing KQP response");
+ TListGroupsResponseData listGroupsResponse;
+
+ NYdb::TResultSetParser parser(response.Record.GetResponse().GetYdbResults(0));
+ while (parser.TryNextRow()) {
+ TListGroupsResponseData::TListedGroup groupInfo;
+ TString consumerName = parser.ColumnParser("consumer_group").GetUtf8().c_str();
+ TString protocolType = parser.ColumnParser("protocol_type").GetUtf8().c_str();
+ groupInfo.GroupId = consumerName;
+ groupInfo.ProtocolType = protocolType;
+ ui64 groupStateNumber = parser.ColumnParser("state").GetUint64();
+ groupInfo.GroupState = NKafka::NConsumer::NumbersToStatesMapping.at(groupStateNumber);
+ listGroupsResponse.Groups.push_back(groupInfo);
+ }
+ return listGroupsResponse;
+}
+
+TString TKafkaListGroupsActor::GetYqlWithTableNames(const TString& templateStr) {
+ TString templateWithConsumerStateTable = std::regex_replace(
+ templateStr.c_str(),
+ std::regex("<consumer_state_table_name>"),
+ NKikimr::NGRpcProxy::V1::TKafkaConsumerGroupsMetaInitManager::GetInstant()->GetStorageTablePath().c_str()
+ );
+ return templateWithConsumerStateTable;
+}
+
+TMaybe<TString> TKafkaListGroupsActor::GetErrorFromYdbResponse(NKqp::TEvKqp::TEvQueryResponse::TPtr& ev) {
+ TStringBuilder builder = TStringBuilder() << "Recieved error on request to KQP. Last sent request: " << "SELECT" << ". Reason: ";
+ if (ev->Cookie != KqpCookie) {
+ return builder << "Unexpected cookie in TEvQueryResponse. Expected KQP Cookie: " << KqpCookie << ", Actual: " << ev->Cookie << ".";
+ } else if (ev->Get()->Record.GetYdbStatus() != Ydb::StatusIds::SUCCESS) {
+ return builder << "Unexpected YDB status in TEvQueryResponse. Expected YDB SUCCESS status, Actual: " << ev->Get()->Record.GetYdbStatus() << ".";
+ } else {
+ return {};
+ }
+}
+
+NYdb::TParams TKafkaListGroupsActor::BuildSelectParams() {
+ NYdb::TParamsBuilder params;
+ params.AddParam("$Database").Utf8(DatabasePath).Build();
+ if (ListGroupsRequestData->StatesFilter.size() > 0) {
+ auto& statesFilterParams = params.AddParam("$StatesFilter").BeginList();
+ for (auto& statesNumberFilter : ListGroupsRequestData->StatesFilter) {
+ if (statesNumberFilter.has_value()) {
+ statesFilterParams.AddListItem().Uint32(NKafka::NConsumer::StatesToNumbersMapping.at(*statesNumberFilter));
+ }
+ }
+ statesFilterParams.EndList().Build();
+ }
+ return params.Build();
+}
+
+std::shared_ptr<TListGroupsResponseData> TKafkaListGroupsActor::BuildResponse(TListGroupsResponseData responseData) {
+ auto response = std::make_shared<TListGroupsResponseData>(std::move(responseData));
+ return response;
+};
+
+void TKafkaListGroupsActor::SendToKqpConsumerGroupsRequest(const TActorContext& ctx) {
+ KAFKA_LOG_W("Sending select request to KQP for database " << DatabasePath);
+ Kqp->SendYqlRequest(
+ GetYqlWithTableNames(ListGroupsRequestData->StatesFilter.size() > 0 ?
+ SELECT_GROUPS_WITH_FILTER :
+ SELECT_GROUPS_NO_FILTER),
+ BuildSelectParams(),
+ KqpCookie,
+ ctx,
+ false
+ );
+}
+
+void TKafkaListGroupsActor::SendFailResponse(EKafkaErrors errorCode, const std::optional<TString>& errorMessage = std::nullopt) {
+ if (errorMessage.has_value()) {
+ KAFKA_LOG_W("Sending fail response with error code: " << errorCode << ". Reason: " << errorMessage);
+ } else {
+ KAFKA_LOG_W("Sending fail response with error code: " << errorCode);
+ }
+
+ TListGroupsResponseData consumerGroupsResponseWithError;
+ consumerGroupsResponseWithError.ErrorCode = errorCode;
+
+ Send(Context->ConnectionId,
+ new TEvKafka::TEvResponse(CorrelationId, BuildResponse(consumerGroupsResponseWithError), errorCode));
+}
+
+} // namespace NKafka
diff --git a/ydb/core/kafka_proxy/actors/kafka_list_groups_actor.h b/ydb/core/kafka_proxy/actors/kafka_list_groups_actor.h
new file mode 100644
index 00000000000..db6b2dce4e4
--- /dev/null
+++ b/ydb/core/kafka_proxy/actors/kafka_list_groups_actor.h
@@ -0,0 +1,99 @@
+#pragma once
+
+#include <ydb/library/actors/core/actor_bootstrapped.h>
+#include <ydb/library/actors/core/event.h>
+#include <ydb/core/kafka_proxy/kafka_events.h>
+#include "ydb/library/aclib/aclib.h"
+#include <ydb/core/kafka_proxy/kqp_helper.h>
+#include <ydb/services/persqueue_v1/actors/events.h>
+
+#include "actors.h"
+
+namespace NKafka {
+
+static const TString SELECT_GROUPS_NO_FILTER = R"sql(
+ DECLARE $Database AS Utf8;
+ SELECT * FROM (
+ SELECT
+ `<consumer_state_table_name>`.*,
+ ROW_NUMBER() OVER (PARTITION BY consumer_group ORDER BY generation DESC) AS row_num
+ FROM `<consumer_state_table_name>`
+ WHERE database = $Database
+ )
+ WHERE row_num = 1;
+)sql";
+
+static const TString SELECT_GROUPS_WITH_FILTER = R"sql(
+ DECLARE $Database AS Utf8;
+ DECLARE $StatesFilter AS List<Uint32>;
+
+ SELECT * FROM (
+ SELECT
+ `<consumer_state_table_name>`.*,
+ ROW_NUMBER() OVER (PARTITION BY consumer_group ORDER BY generation DESC) AS row_num
+ FROM `<consumer_state_table_name>`
+ WHERE database = $Database
+ )
+ WHERE row_num = 1 AND state in $StatesFilter;
+)sql";
+
+class TKafkaListGroupsActor: public NActors::TActorBootstrapped<TKafkaListGroupsActor> {
+
+
+using TBase = TActorBootstrapped<TKafkaListGroupsActor>;
+
+public:
+ TKafkaListGroupsActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr<TListGroupsRequestData>& message)
+ : Context(context)
+ , CorrelationId(correlationId)
+ , ListGroupsRequestData(message)
+ , DatabasePath(context->DatabasePath) {
+ }
+
+
+void Bootstrap(const NActors::TActorContext& ctx);
+
+
+TStringBuilder LogPrefix() const {
+ return TStringBuilder() << "KafkaListGroupsActor{DatabasePath=" << DatabasePath << "}: ";
+}
+
+private:
+ STATEFN(StateWork) {
+ switch (ev->GetTypeRewrite()) {
+ HFunc(NKqp::TEvKqp::TEvCreateSessionResponse, Handle);
+ HFunc(NMetadata::NProvider::TEvManagerPrepared, Handle);
+ HFunc(NKqp::TEvKqp::TEvQueryResponse, Handle);
+ }
+ }
+
+ void Handle(NMetadata::NProvider::TEvManagerPrepared::TPtr&, const TActorContext& ctx);
+ void Handle(NKqp::TEvKqp::TEvCreateSessionResponse::TPtr& ev, const TActorContext& ctx);
+ void Handle(NKqp::TEvKqp::TEvQueryResponse::TPtr& ev, const TActorContext& ctx);
+ void HandleSelectResponse(const NKqp::TEvKqp::TEvQueryResponse& response, const TActorContext& ctx);
+
+ void Die(const TActorContext &ctx);
+
+ void StartKqpSession(const TActorContext& ctx);
+ TListGroupsResponseData ParseGroupsMetadata(const NKqp::TEvKqp::TEvQueryResponse& response);
+
+ TString GetYqlWithTableNames(const TString& templateStr);
+ TMaybe<TString> GetErrorFromYdbResponse(NKqp::TEvKqp::TEvQueryResponse::TPtr& ev);
+ NYdb::TParams BuildSelectParams();
+ std::shared_ptr<TListGroupsResponseData> BuildResponse(TListGroupsResponseData responseData);
+ void SendToKqpConsumerGroupsRequest(const TActorContext& ctx);
+ void SendFailResponse(EKafkaErrors errorCode, const std::optional<TString>& errorMessage);
+
+ const TContext::TPtr Context;
+ const ui64 CorrelationId;
+ const TMessagePtr<TListGroupsRequestData> ListGroupsRequestData;
+
+ std::unique_ptr<TKqpTxHelper> Kqp;
+
+ const TString DatabasePath;
+
+ TString KqpSessionId;
+ ui64 KqpCookie = 0;
+};
+
+} // namespace NKafka
diff --git a/ydb/core/kafka_proxy/actors/kafka_state_name_to_int.cpp b/ydb/core/kafka_proxy/actors/kafka_state_name_to_int.cpp
new file mode 100644
index 00000000000..09f0bb348f6
--- /dev/null
+++ b/ydb/core/kafka_proxy/actors/kafka_state_name_to_int.cpp
@@ -0,0 +1,19 @@
+#include "kafka_state_name_to_int.h"
+
+namespace NKafka {
+ namespace NConsumer {
+ const std::map<int, TString> NumbersToStatesMapping = {{0, "Unknown"},
+ {1, "PreparingRebalance"},
+ {2, "CompletingRebalance"},
+ {3, "Stable"},
+ {4, "Dead"},
+ {5, "Empty"}};
+
+ const std::map<TString, int> StatesToNumbersMapping = {{"Unknown", 0},
+ {"PreparingRebalance", 1},
+ {"CompletingRebalance", 2},
+ {"Stable", 3},
+ {"Dead", 4},
+ {"Empty", 5}};
+ }
+}
diff --git a/ydb/core/kafka_proxy/actors/kafka_state_name_to_int.h b/ydb/core/kafka_proxy/actors/kafka_state_name_to_int.h
new file mode 100644
index 00000000000..c03da0af2d2
--- /dev/null
+++ b/ydb/core/kafka_proxy/actors/kafka_state_name_to_int.h
@@ -0,0 +1,12 @@
+#pragma once
+
+#include <util/generic/cast.h>
+#include <map>
+
+namespace NKafka {
+ namespace NConsumer {
+ extern const std::map<int, TString> NumbersToStatesMapping;
+
+ extern const std::map<TString, int> StatesToNumbersMapping;
+ }
+} // namespace NKafka::NConsumer
diff --git a/ydb/core/kafka_proxy/kafka_connection.cpp b/ydb/core/kafka_proxy/kafka_connection.cpp
index 07f245668ff..5add6a27ed3 100644
--- a/ydb/core/kafka_proxy/kafka_connection.cpp
+++ b/ydb/core/kafka_proxy/kafka_connection.cpp
@@ -311,6 +311,14 @@ protected:
Register(CreateKafkaListOffsetsActor(Context, header->CorrelationId, message));
}
+ void HandleMessage(const TRequestHeaderData* header, const TMessagePtr<TDescribeGroupsRequestData>& message) {
+ Register(CreateKafkaDescribeGroupsActor(Context, header->CorrelationId, message));
+ }
+
+ void HandleMessage(const TRequestHeaderData* header, const TMessagePtr<TListGroupsRequestData>& message) {
+ Register(CreateKafkaListGroupsActor(Context, header->CorrelationId, message));
+ }
+
void HandleMessage(const TRequestHeaderData* header, const TMessagePtr<TFetchRequestData>& message) {
Register(CreateKafkaFetchActor(Context, header->CorrelationId, message));
}
@@ -341,7 +349,7 @@ protected:
void HandleMessage(const TRequestHeaderData* header, const TMessagePtr<TAddPartitionsToTxnRequestData>& message) {
Send(MakeTransactionsServiceID(SelfId().NodeId()), new TEvKafka::TEvAddPartitionsToTxnRequest(
- header->CorrelationId,
+ header->CorrelationId,
message,
Context->ConnectionId,
Context->DatabasePath
@@ -350,7 +358,7 @@ protected:
void HandleMessage(const TRequestHeaderData* header, const TMessagePtr<TAddOffsetsToTxnRequestData>& message) {
Send(MakeTransactionsServiceID(SelfId().NodeId()), new TEvKafka::TEvAddOffsetsToTxnRequest(
- header->CorrelationId,
+ header->CorrelationId,
message,
Context->ConnectionId,
Context->DatabasePath
@@ -359,7 +367,7 @@ protected:
void HandleMessage(const TRequestHeaderData* header, const TMessagePtr<TTxnOffsetCommitRequestData>& message) {
Send(MakeTransactionsServiceID(SelfId().NodeId()), new TEvKafka::TEvTxnOffsetCommitRequest(
- header->CorrelationId,
+ header->CorrelationId,
message,
Context->ConnectionId,
Context->DatabasePath
@@ -368,7 +376,7 @@ protected:
void HandleMessage(const TRequestHeaderData* header, const TMessagePtr<TEndTxnRequestData>& message) {
Send(MakeTransactionsServiceID(SelfId().NodeId()), new TEvKafka::TEvEndTxnRequest(
- header->CorrelationId,
+ header->CorrelationId,
message,
Context->ConnectionId,
Context->DatabasePath
@@ -436,6 +444,13 @@ protected:
HandleMessage(&Request->Header, Cast<TListOffsetsRequestData>(Request));
break;
+ case LIST_GROUPS:
+ HandleMessage(&Request->Header, Cast<TListGroupsRequestData>(Request));
+ break;
+ case DESCRIBE_GROUPS:
+ HandleMessage(&Request->Header, Cast<TDescribeGroupsRequestData>(Request));
+ break;
+
case FETCH:
HandleMessage(&Request->Header, Cast<TFetchRequestData>(Request));
break;
diff --git a/ydb/core/kafka_proxy/kafka_messages.cpp b/ydb/core/kafka_proxy/kafka_messages.cpp
index c90be116556..b78b2e1b09c 100644
--- a/ydb/core/kafka_proxy/kafka_messages.cpp
+++ b/ydb/core/kafka_proxy/kafka_messages.cpp
@@ -19,6 +19,8 @@ const std::unordered_map<EApiKey, TString> EApiKeyNames = {
{EApiKey::HEARTBEAT, "HEARTBEAT"},
{EApiKey::LEAVE_GROUP, "LEAVE_GROUP"},
{EApiKey::SYNC_GROUP, "SYNC_GROUP"},
+ {EApiKey::DESCRIBE_GROUPS, "DESCRIBE_GROUPS"},
+ {EApiKey::LIST_GROUPS, "LIST_GROUPS"},
{EApiKey::SASL_HANDSHAKE, "SASL_HANDSHAKE"},
{EApiKey::API_VERSIONS, "API_VERSIONS"},
{EApiKey::CREATE_TOPICS, "CREATE_TOPICS"},
@@ -58,6 +60,10 @@ std::unique_ptr<TApiMessage> CreateRequest(i16 apiKey) {
return std::make_unique<TLeaveGroupRequestData>();
case SYNC_GROUP:
return std::make_unique<TSyncGroupRequestData>();
+ case DESCRIBE_GROUPS:
+ return std::make_unique<TDescribeGroupsRequestData>();
+ case LIST_GROUPS:
+ return std::make_unique<TListGroupsRequestData>();
case SASL_HANDSHAKE:
return std::make_unique<TSaslHandshakeRequestData>();
case API_VERSIONS:
@@ -111,6 +117,10 @@ std::unique_ptr<TApiMessage> CreateResponse(i16 apiKey) {
return std::make_unique<TLeaveGroupResponseData>();
case SYNC_GROUP:
return std::make_unique<TSyncGroupResponseData>();
+ case DESCRIBE_GROUPS:
+ return std::make_unique<TDescribeGroupsResponseData>();
+ case LIST_GROUPS:
+ return std::make_unique<TListGroupsResponseData>();
case SASL_HANDSHAKE:
return std::make_unique<TSaslHandshakeResponseData>();
case API_VERSIONS:
@@ -208,6 +218,18 @@ TKafkaVersion RequestHeaderVersion(i16 apiKey, TKafkaVersion _version) {
} else {
return 1;
}
+ case DESCRIBE_GROUPS:
+ if (_version >= 5) {
+ return 2;
+ } else {
+ return 1;
+ }
+ case LIST_GROUPS:
+ if (_version >= 3) {
+ return 2;
+ } else {
+ return 1;
+ }
case SASL_HANDSHAKE:
return 1;
case API_VERSIONS:
@@ -350,6 +372,18 @@ TKafkaVersion ResponseHeaderVersion(i16 apiKey, TKafkaVersion _version) {
} else {
return 0;
}
+ case DESCRIBE_GROUPS:
+ if (_version >= 5) {
+ return 1;
+ } else {
+ return 0;
+ }
+ case LIST_GROUPS:
+ if (_version >= 3) {
+ return 1;
+ } else {
+ return 0;
+ }
case SASL_HANDSHAKE:
return 0;
case API_VERSIONS:
@@ -434,7 +468,7 @@ const TRequestHeaderData::RequestApiVersionMeta::Type TRequestHeaderData::Reques
const TRequestHeaderData::CorrelationIdMeta::Type TRequestHeaderData::CorrelationIdMeta::Default = 0;
const TRequestHeaderData::ClientIdMeta::Type TRequestHeaderData::ClientIdMeta::Default = {""};
-TRequestHeaderData::TRequestHeaderData()
+TRequestHeaderData::TRequestHeaderData()
: RequestApiKey(RequestApiKeyMeta::Default)
, RequestApiVersion(RequestApiVersionMeta::Default)
, CorrelationId(CorrelationIdMeta::Default)
@@ -449,7 +483,7 @@ void TRequestHeaderData::Read(TKafkaReadable& _readable, TKafkaVersion _version)
NPrivate::Read<RequestApiVersionMeta>(_readable, _version, RequestApiVersion);
NPrivate::Read<CorrelationIdMeta>(_readable, _version, CorrelationId);
NPrivate::Read<ClientIdMeta>(_readable, _version, ClientId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -473,10 +507,10 @@ void TRequestHeaderData::Write(TKafkaWritable& _writable, TKafkaVersion _version
NPrivate::Write<RequestApiVersionMeta>(_collector, _writable, _version, RequestApiVersion);
NPrivate::Write<CorrelationIdMeta>(_collector, _writable, _version, CorrelationId);
NPrivate::Write<ClientIdMeta>(_collector, _writable, _version, ClientId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -486,7 +520,7 @@ i32 TRequestHeaderData::Size(TKafkaVersion _version) const {
NPrivate::Size<RequestApiVersionMeta>(_collector, _version, RequestApiVersion);
NPrivate::Size<CorrelationIdMeta>(_collector, _version, CorrelationId);
NPrivate::Size<ClientIdMeta>(_collector, _version, ClientId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -499,7 +533,7 @@ i32 TRequestHeaderData::Size(TKafkaVersion _version) const {
//
const TResponseHeaderData::CorrelationIdMeta::Type TResponseHeaderData::CorrelationIdMeta::Default = 0;
-TResponseHeaderData::TResponseHeaderData()
+TResponseHeaderData::TResponseHeaderData()
: CorrelationId(CorrelationIdMeta::Default)
{}
@@ -508,7 +542,7 @@ void TResponseHeaderData::Read(TKafkaReadable& _readable, TKafkaVersion _version
ythrow yexception() << "Can't read version " << _version << " of TResponseHeaderData";
}
NPrivate::Read<CorrelationIdMeta>(_readable, _version, CorrelationId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -529,17 +563,17 @@ void TResponseHeaderData::Write(TKafkaWritable& _writable, TKafkaVersion _versio
}
NPrivate::TWriteCollector _collector;
NPrivate::Write<CorrelationIdMeta>(_collector, _writable, _version, CorrelationId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
i32 TResponseHeaderData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<CorrelationIdMeta>(_collector, _version, CorrelationId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -554,7 +588,7 @@ const TProduceRequestData::TransactionalIdMeta::Type TProduceRequestData::Transa
const TProduceRequestData::AcksMeta::Type TProduceRequestData::AcksMeta::Default = 0;
const TProduceRequestData::TimeoutMsMeta::Type TProduceRequestData::TimeoutMsMeta::Default = 0;
-TProduceRequestData::TProduceRequestData()
+TProduceRequestData::TProduceRequestData()
: TransactionalId(TransactionalIdMeta::Default)
, Acks(AcksMeta::Default)
, TimeoutMs(TimeoutMsMeta::Default)
@@ -568,7 +602,7 @@ void TProduceRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _version
NPrivate::Read<AcksMeta>(_readable, _version, Acks);
NPrivate::Read<TimeoutMsMeta>(_readable, _version, TimeoutMs);
NPrivate::Read<TopicDataMeta>(_readable, _version, TopicData);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -592,10 +626,10 @@ void TProduceRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _versio
NPrivate::Write<AcksMeta>(_collector, _writable, _version, Acks);
NPrivate::Write<TimeoutMsMeta>(_collector, _writable, _version, TimeoutMs);
NPrivate::Write<TopicDataMeta>(_collector, _writable, _version, TopicData);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -605,7 +639,7 @@ i32 TProduceRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<AcksMeta>(_collector, _version, Acks);
NPrivate::Size<TimeoutMsMeta>(_collector, _version, TimeoutMs);
NPrivate::Size<TopicDataMeta>(_collector, _version, TopicData);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -618,7 +652,7 @@ i32 TProduceRequestData::Size(TKafkaVersion _version) const {
//
const TProduceRequestData::TTopicProduceData::NameMeta::Type TProduceRequestData::TTopicProduceData::NameMeta::Default = {""};
-TProduceRequestData::TTopicProduceData::TTopicProduceData()
+TProduceRequestData::TTopicProduceData::TTopicProduceData()
: Name(NameMeta::Default)
{}
@@ -628,7 +662,7 @@ void TProduceRequestData::TTopicProduceData::Read(TKafkaReadable& _readable, TKa
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<PartitionDataMeta>(_readable, _version, PartitionData);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -650,10 +684,10 @@ void TProduceRequestData::TTopicProduceData::Write(TKafkaWritable& _writable, TK
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<PartitionDataMeta>(_collector, _writable, _version, PartitionData);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -661,7 +695,7 @@ i32 TProduceRequestData::TTopicProduceData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<PartitionDataMeta>(_collector, _version, PartitionData);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -674,7 +708,7 @@ i32 TProduceRequestData::TTopicProduceData::Size(TKafkaVersion _version) const {
//
const TProduceRequestData::TTopicProduceData::TPartitionProduceData::IndexMeta::Type TProduceRequestData::TTopicProduceData::TPartitionProduceData::IndexMeta::Default = 0;
-TProduceRequestData::TTopicProduceData::TPartitionProduceData::TPartitionProduceData()
+TProduceRequestData::TTopicProduceData::TPartitionProduceData::TPartitionProduceData()
: Index(IndexMeta::Default)
{}
@@ -684,7 +718,7 @@ void TProduceRequestData::TTopicProduceData::TPartitionProduceData::Read(TKafkaR
}
NPrivate::Read<IndexMeta>(_readable, _version, Index);
NPrivate::Read<RecordsMeta>(_readable, _version, Records);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -706,10 +740,10 @@ void TProduceRequestData::TTopicProduceData::TPartitionProduceData::Write(TKafka
NPrivate::TWriteCollector _collector;
NPrivate::Write<IndexMeta>(_collector, _writable, _version, Index);
NPrivate::Write<RecordsMeta>(_collector, _writable, _version, Records);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -717,7 +751,7 @@ i32 TProduceRequestData::TTopicProduceData::TPartitionProduceData::Size(TKafkaVe
NPrivate::TSizeCollector _collector;
NPrivate::Size<IndexMeta>(_collector, _version, Index);
NPrivate::Size<RecordsMeta>(_collector, _version, Records);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -730,7 +764,7 @@ i32 TProduceRequestData::TTopicProduceData::TPartitionProduceData::Size(TKafkaVe
//
const TProduceResponseData::ThrottleTimeMsMeta::Type TProduceResponseData::ThrottleTimeMsMeta::Default = 0;
-TProduceResponseData::TProduceResponseData()
+TProduceResponseData::TProduceResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
{}
@@ -740,7 +774,7 @@ void TProduceResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _versio
}
NPrivate::Read<ResponsesMeta>(_readable, _version, Responses);
NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -762,10 +796,10 @@ void TProduceResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _versi
NPrivate::TWriteCollector _collector;
NPrivate::Write<ResponsesMeta>(_collector, _writable, _version, Responses);
NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -773,7 +807,7 @@ i32 TProduceResponseData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ResponsesMeta>(_collector, _version, Responses);
NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -786,7 +820,7 @@ i32 TProduceResponseData::Size(TKafkaVersion _version) const {
//
const TProduceResponseData::TTopicProduceResponse::NameMeta::Type TProduceResponseData::TTopicProduceResponse::NameMeta::Default = {""};
-TProduceResponseData::TTopicProduceResponse::TTopicProduceResponse()
+TProduceResponseData::TTopicProduceResponse::TTopicProduceResponse()
: Name(NameMeta::Default)
{}
@@ -796,7 +830,7 @@ void TProduceResponseData::TTopicProduceResponse::Read(TKafkaReadable& _readable
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<PartitionResponsesMeta>(_readable, _version, PartitionResponses);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -818,10 +852,10 @@ void TProduceResponseData::TTopicProduceResponse::Write(TKafkaWritable& _writabl
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<PartitionResponsesMeta>(_collector, _writable, _version, PartitionResponses);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -829,7 +863,7 @@ i32 TProduceResponseData::TTopicProduceResponse::Size(TKafkaVersion _version) co
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<PartitionResponsesMeta>(_collector, _version, PartitionResponses);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -847,7 +881,7 @@ const TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::Lo
const TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::LogStartOffsetMeta::Type TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::LogStartOffsetMeta::Default = -1;
const TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::ErrorMessageMeta::Type TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::ErrorMessageMeta::Default = std::nullopt;
-TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::TPartitionProduceResponse()
+TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::TPartitionProduceResponse()
: Index(IndexMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
, BaseOffset(BaseOffsetMeta::Default)
@@ -867,7 +901,7 @@ void TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::Rea
NPrivate::Read<LogStartOffsetMeta>(_readable, _version, LogStartOffset);
NPrivate::Read<RecordErrorsMeta>(_readable, _version, RecordErrors);
NPrivate::Read<ErrorMessageMeta>(_readable, _version, ErrorMessage);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -894,10 +928,10 @@ void TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::Wri
NPrivate::Write<LogStartOffsetMeta>(_collector, _writable, _version, LogStartOffset);
NPrivate::Write<RecordErrorsMeta>(_collector, _writable, _version, RecordErrors);
NPrivate::Write<ErrorMessageMeta>(_collector, _writable, _version, ErrorMessage);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -910,7 +944,7 @@ i32 TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::Size
NPrivate::Size<LogStartOffsetMeta>(_collector, _version, LogStartOffset);
NPrivate::Size<RecordErrorsMeta>(_collector, _version, RecordErrors);
NPrivate::Size<ErrorMessageMeta>(_collector, _version, ErrorMessage);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -924,7 +958,7 @@ i32 TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::Size
const TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::TBatchIndexAndErrorMessage::BatchIndexMeta::Type TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::TBatchIndexAndErrorMessage::BatchIndexMeta::Default = 0;
const TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::TBatchIndexAndErrorMessage::BatchIndexErrorMessageMeta::Type TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::TBatchIndexAndErrorMessage::BatchIndexErrorMessageMeta::Default = std::nullopt;
-TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::TBatchIndexAndErrorMessage::TBatchIndexAndErrorMessage()
+TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::TBatchIndexAndErrorMessage::TBatchIndexAndErrorMessage()
: BatchIndex(BatchIndexMeta::Default)
, BatchIndexErrorMessage(BatchIndexErrorMessageMeta::Default)
{}
@@ -935,7 +969,7 @@ void TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::TBa
}
NPrivate::Read<BatchIndexMeta>(_readable, _version, BatchIndex);
NPrivate::Read<BatchIndexErrorMessageMeta>(_readable, _version, BatchIndexErrorMessage);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -957,10 +991,10 @@ void TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::TBa
NPrivate::TWriteCollector _collector;
NPrivate::Write<BatchIndexMeta>(_collector, _writable, _version, BatchIndex);
NPrivate::Write<BatchIndexErrorMessageMeta>(_collector, _writable, _version, BatchIndexErrorMessage);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -968,7 +1002,7 @@ i32 TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::TBat
NPrivate::TSizeCollector _collector;
NPrivate::Size<BatchIndexMeta>(_collector, _version, BatchIndex);
NPrivate::Size<BatchIndexErrorMessageMeta>(_collector, _version, BatchIndexErrorMessage);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -989,7 +1023,7 @@ const TFetchRequestData::SessionIdMeta::Type TFetchRequestData::SessionIdMeta::D
const TFetchRequestData::SessionEpochMeta::Type TFetchRequestData::SessionEpochMeta::Default = -1;
const TFetchRequestData::RackIdMeta::Type TFetchRequestData::RackIdMeta::Default = {""};
-TFetchRequestData::TFetchRequestData()
+TFetchRequestData::TFetchRequestData()
: ClusterId(ClusterIdMeta::Default)
, ReplicaId(ReplicaIdMeta::Default)
, MaxWaitMs(MaxWaitMsMeta::Default)
@@ -1016,7 +1050,7 @@ void TFetchRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _version)
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
NPrivate::Read<ForgottenTopicsDataMeta>(_readable, _version, ForgottenTopicsData);
NPrivate::Read<RackIdMeta>(_readable, _version, RackId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1050,10 +1084,10 @@ void TFetchRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _version)
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
NPrivate::Write<ForgottenTopicsDataMeta>(_collector, _writable, _version, ForgottenTopicsData);
NPrivate::Write<RackIdMeta>(_collector, _writable, _version, RackId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
NPrivate::WriteTag<ClusterIdMeta>(_writable, _version, ClusterId);
}
}
@@ -1071,7 +1105,7 @@ i32 TFetchRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
NPrivate::Size<ForgottenTopicsDataMeta>(_collector, _version, ForgottenTopicsData);
NPrivate::Size<RackIdMeta>(_collector, _version, RackId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1085,7 +1119,7 @@ i32 TFetchRequestData::Size(TKafkaVersion _version) const {
const TFetchRequestData::TFetchTopic::TopicMeta::Type TFetchRequestData::TFetchTopic::TopicMeta::Default = {""};
const TFetchRequestData::TFetchTopic::TopicIdMeta::Type TFetchRequestData::TFetchTopic::TopicIdMeta::Default = TKafkaUuid(0, 0);
-TFetchRequestData::TFetchTopic::TFetchTopic()
+TFetchRequestData::TFetchTopic::TFetchTopic()
: Topic(TopicMeta::Default)
, TopicId(TopicIdMeta::Default)
{}
@@ -1097,7 +1131,7 @@ void TFetchRequestData::TFetchTopic::Read(TKafkaReadable& _readable, TKafkaVersi
NPrivate::Read<TopicMeta>(_readable, _version, Topic);
NPrivate::Read<TopicIdMeta>(_readable, _version, TopicId);
NPrivate::Read<PartitionsMeta>(_readable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1120,10 +1154,10 @@ void TFetchRequestData::TFetchTopic::Write(TKafkaWritable& _writable, TKafkaVers
NPrivate::Write<TopicMeta>(_collector, _writable, _version, Topic);
NPrivate::Write<TopicIdMeta>(_collector, _writable, _version, TopicId);
NPrivate::Write<PartitionsMeta>(_collector, _writable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -1132,7 +1166,7 @@ i32 TFetchRequestData::TFetchTopic::Size(TKafkaVersion _version) const {
NPrivate::Size<TopicMeta>(_collector, _version, Topic);
NPrivate::Size<TopicIdMeta>(_collector, _version, TopicId);
NPrivate::Size<PartitionsMeta>(_collector, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1150,7 +1184,7 @@ const TFetchRequestData::TFetchTopic::TFetchPartition::LastFetchedEpochMeta::Typ
const TFetchRequestData::TFetchTopic::TFetchPartition::LogStartOffsetMeta::Type TFetchRequestData::TFetchTopic::TFetchPartition::LogStartOffsetMeta::Default = -1;
const TFetchRequestData::TFetchTopic::TFetchPartition::PartitionMaxBytesMeta::Type TFetchRequestData::TFetchTopic::TFetchPartition::PartitionMaxBytesMeta::Default = 0;
-TFetchRequestData::TFetchTopic::TFetchPartition::TFetchPartition()
+TFetchRequestData::TFetchTopic::TFetchPartition::TFetchPartition()
: Partition(PartitionMeta::Default)
, CurrentLeaderEpoch(CurrentLeaderEpochMeta::Default)
, FetchOffset(FetchOffsetMeta::Default)
@@ -1169,7 +1203,7 @@ void TFetchRequestData::TFetchTopic::TFetchPartition::Read(TKafkaReadable& _read
NPrivate::Read<LastFetchedEpochMeta>(_readable, _version, LastFetchedEpoch);
NPrivate::Read<LogStartOffsetMeta>(_readable, _version, LogStartOffset);
NPrivate::Read<PartitionMaxBytesMeta>(_readable, _version, PartitionMaxBytes);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1195,10 +1229,10 @@ void TFetchRequestData::TFetchTopic::TFetchPartition::Write(TKafkaWritable& _wri
NPrivate::Write<LastFetchedEpochMeta>(_collector, _writable, _version, LastFetchedEpoch);
NPrivate::Write<LogStartOffsetMeta>(_collector, _writable, _version, LogStartOffset);
NPrivate::Write<PartitionMaxBytesMeta>(_collector, _writable, _version, PartitionMaxBytes);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -1210,7 +1244,7 @@ i32 TFetchRequestData::TFetchTopic::TFetchPartition::Size(TKafkaVersion _version
NPrivate::Size<LastFetchedEpochMeta>(_collector, _version, LastFetchedEpoch);
NPrivate::Size<LogStartOffsetMeta>(_collector, _version, LogStartOffset);
NPrivate::Size<PartitionMaxBytesMeta>(_collector, _version, PartitionMaxBytes);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1224,7 +1258,7 @@ i32 TFetchRequestData::TFetchTopic::TFetchPartition::Size(TKafkaVersion _version
const TFetchRequestData::TForgottenTopic::TopicMeta::Type TFetchRequestData::TForgottenTopic::TopicMeta::Default = {""};
const TFetchRequestData::TForgottenTopic::TopicIdMeta::Type TFetchRequestData::TForgottenTopic::TopicIdMeta::Default = TKafkaUuid(0, 0);
-TFetchRequestData::TForgottenTopic::TForgottenTopic()
+TFetchRequestData::TForgottenTopic::TForgottenTopic()
: Topic(TopicMeta::Default)
, TopicId(TopicIdMeta::Default)
{}
@@ -1236,7 +1270,7 @@ void TFetchRequestData::TForgottenTopic::Read(TKafkaReadable& _readable, TKafkaV
NPrivate::Read<TopicMeta>(_readable, _version, Topic);
NPrivate::Read<TopicIdMeta>(_readable, _version, TopicId);
NPrivate::Read<PartitionsMeta>(_readable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1259,10 +1293,10 @@ void TFetchRequestData::TForgottenTopic::Write(TKafkaWritable& _writable, TKafka
NPrivate::Write<TopicMeta>(_collector, _writable, _version, Topic);
NPrivate::Write<TopicIdMeta>(_collector, _writable, _version, TopicId);
NPrivate::Write<PartitionsMeta>(_collector, _writable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -1271,7 +1305,7 @@ i32 TFetchRequestData::TForgottenTopic::Size(TKafkaVersion _version) const {
NPrivate::Size<TopicMeta>(_collector, _version, Topic);
NPrivate::Size<TopicIdMeta>(_collector, _version, TopicId);
NPrivate::Size<PartitionsMeta>(_collector, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1286,7 +1320,7 @@ const TFetchResponseData::ThrottleTimeMsMeta::Type TFetchResponseData::ThrottleT
const TFetchResponseData::ErrorCodeMeta::Type TFetchResponseData::ErrorCodeMeta::Default = 0;
const TFetchResponseData::SessionIdMeta::Type TFetchResponseData::SessionIdMeta::Default = 0;
-TFetchResponseData::TFetchResponseData()
+TFetchResponseData::TFetchResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
, SessionId(SessionIdMeta::Default)
@@ -1300,7 +1334,7 @@ void TFetchResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _version)
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
NPrivate::Read<SessionIdMeta>(_readable, _version, SessionId);
NPrivate::Read<ResponsesMeta>(_readable, _version, Responses);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1324,10 +1358,10 @@ void TFetchResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _version
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
NPrivate::Write<SessionIdMeta>(_collector, _writable, _version, SessionId);
NPrivate::Write<ResponsesMeta>(_collector, _writable, _version, Responses);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -1337,7 +1371,7 @@ i32 TFetchResponseData::Size(TKafkaVersion _version) const {
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
NPrivate::Size<SessionIdMeta>(_collector, _version, SessionId);
NPrivate::Size<ResponsesMeta>(_collector, _version, Responses);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1351,7 +1385,7 @@ i32 TFetchResponseData::Size(TKafkaVersion _version) const {
const TFetchResponseData::TFetchableTopicResponse::TopicMeta::Type TFetchResponseData::TFetchableTopicResponse::TopicMeta::Default = {""};
const TFetchResponseData::TFetchableTopicResponse::TopicIdMeta::Type TFetchResponseData::TFetchableTopicResponse::TopicIdMeta::Default = TKafkaUuid(0, 0);
-TFetchResponseData::TFetchableTopicResponse::TFetchableTopicResponse()
+TFetchResponseData::TFetchableTopicResponse::TFetchableTopicResponse()
: Topic(TopicMeta::Default)
, TopicId(TopicIdMeta::Default)
{}
@@ -1363,7 +1397,7 @@ void TFetchResponseData::TFetchableTopicResponse::Read(TKafkaReadable& _readable
NPrivate::Read<TopicMeta>(_readable, _version, Topic);
NPrivate::Read<TopicIdMeta>(_readable, _version, TopicId);
NPrivate::Read<PartitionsMeta>(_readable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1386,10 +1420,10 @@ void TFetchResponseData::TFetchableTopicResponse::Write(TKafkaWritable& _writabl
NPrivate::Write<TopicMeta>(_collector, _writable, _version, Topic);
NPrivate::Write<TopicIdMeta>(_collector, _writable, _version, TopicId);
NPrivate::Write<PartitionsMeta>(_collector, _writable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -1398,7 +1432,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::Size(TKafkaVersion _version) co
NPrivate::Size<TopicMeta>(_collector, _version, Topic);
NPrivate::Size<TopicIdMeta>(_collector, _version, TopicId);
NPrivate::Size<PartitionsMeta>(_collector, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1416,7 +1450,7 @@ const TFetchResponseData::TFetchableTopicResponse::TPartitionData::LastStableOff
const TFetchResponseData::TFetchableTopicResponse::TPartitionData::LogStartOffsetMeta::Type TFetchResponseData::TFetchableTopicResponse::TPartitionData::LogStartOffsetMeta::Default = -1;
const TFetchResponseData::TFetchableTopicResponse::TPartitionData::PreferredReadReplicaMeta::Type TFetchResponseData::TFetchableTopicResponse::TPartitionData::PreferredReadReplicaMeta::Default = -1;
-TFetchResponseData::TFetchableTopicResponse::TPartitionData::TPartitionData()
+TFetchResponseData::TFetchableTopicResponse::TPartitionData::TPartitionData()
: PartitionIndex(PartitionIndexMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
, HighWatermark(HighWatermarkMeta::Default)
@@ -1440,7 +1474,7 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::Read(TKafkaRea
NPrivate::Read<AbortedTransactionsMeta>(_readable, _version, AbortedTransactions);
NPrivate::Read<PreferredReadReplicaMeta>(_readable, _version, PreferredReadReplica);
NPrivate::Read<RecordsMeta>(_readable, _version, Records);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1480,10 +1514,10 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::Write(TKafkaWr
NPrivate::Write<AbortedTransactionsMeta>(_collector, _writable, _version, AbortedTransactions);
NPrivate::Write<PreferredReadReplicaMeta>(_collector, _writable, _version, PreferredReadReplica);
NPrivate::Write<RecordsMeta>(_collector, _writable, _version, Records);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
NPrivate::WriteTag<DivergingEpochMeta>(_writable, _version, DivergingEpoch);
NPrivate::WriteTag<CurrentLeaderMeta>(_writable, _version, CurrentLeader);
NPrivate::WriteTag<SnapshotIdMeta>(_writable, _version, SnapshotId);
@@ -1503,7 +1537,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::Size(TKafkaVers
NPrivate::Size<AbortedTransactionsMeta>(_collector, _version, AbortedTransactions);
NPrivate::Size<PreferredReadReplicaMeta>(_collector, _version, PreferredReadReplica);
NPrivate::Size<RecordsMeta>(_collector, _version, Records);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1517,7 +1551,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::Size(TKafkaVers
const TFetchResponseData::TFetchableTopicResponse::TPartitionData::TEpochEndOffset::EpochMeta::Type TFetchResponseData::TFetchableTopicResponse::TPartitionData::TEpochEndOffset::EpochMeta::Default = -1;
const TFetchResponseData::TFetchableTopicResponse::TPartitionData::TEpochEndOffset::EndOffsetMeta::Type TFetchResponseData::TFetchableTopicResponse::TPartitionData::TEpochEndOffset::EndOffsetMeta::Default = -1;
-TFetchResponseData::TFetchableTopicResponse::TPartitionData::TEpochEndOffset::TEpochEndOffset()
+TFetchResponseData::TFetchableTopicResponse::TPartitionData::TEpochEndOffset::TEpochEndOffset()
: Epoch(EpochMeta::Default)
, EndOffset(EndOffsetMeta::Default)
{}
@@ -1528,7 +1562,7 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TEpochEndOffse
}
NPrivate::Read<EpochMeta>(_readable, _version, Epoch);
NPrivate::Read<EndOffsetMeta>(_readable, _version, EndOffset);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1550,10 +1584,10 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TEpochEndOffse
NPrivate::TWriteCollector _collector;
NPrivate::Write<EpochMeta>(_collector, _writable, _version, Epoch);
NPrivate::Write<EndOffsetMeta>(_collector, _writable, _version, EndOffset);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -1561,7 +1595,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::TEpochEndOffset
NPrivate::TSizeCollector _collector;
NPrivate::Size<EpochMeta>(_collector, _version, Epoch);
NPrivate::Size<EndOffsetMeta>(_collector, _version, EndOffset);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1575,7 +1609,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::TEpochEndOffset
const TFetchResponseData::TFetchableTopicResponse::TPartitionData::TLeaderIdAndEpoch::LeaderIdMeta::Type TFetchResponseData::TFetchableTopicResponse::TPartitionData::TLeaderIdAndEpoch::LeaderIdMeta::Default = -1;
const TFetchResponseData::TFetchableTopicResponse::TPartitionData::TLeaderIdAndEpoch::LeaderEpochMeta::Type TFetchResponseData::TFetchableTopicResponse::TPartitionData::TLeaderIdAndEpoch::LeaderEpochMeta::Default = -1;
-TFetchResponseData::TFetchableTopicResponse::TPartitionData::TLeaderIdAndEpoch::TLeaderIdAndEpoch()
+TFetchResponseData::TFetchableTopicResponse::TPartitionData::TLeaderIdAndEpoch::TLeaderIdAndEpoch()
: LeaderId(LeaderIdMeta::Default)
, LeaderEpoch(LeaderEpochMeta::Default)
{}
@@ -1586,7 +1620,7 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TLeaderIdAndEp
}
NPrivate::Read<LeaderIdMeta>(_readable, _version, LeaderId);
NPrivate::Read<LeaderEpochMeta>(_readable, _version, LeaderEpoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1608,10 +1642,10 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TLeaderIdAndEp
NPrivate::TWriteCollector _collector;
NPrivate::Write<LeaderIdMeta>(_collector, _writable, _version, LeaderId);
NPrivate::Write<LeaderEpochMeta>(_collector, _writable, _version, LeaderEpoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -1619,7 +1653,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::TLeaderIdAndEpo
NPrivate::TSizeCollector _collector;
NPrivate::Size<LeaderIdMeta>(_collector, _version, LeaderId);
NPrivate::Size<LeaderEpochMeta>(_collector, _version, LeaderEpoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1633,7 +1667,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::TLeaderIdAndEpo
const TFetchResponseData::TFetchableTopicResponse::TPartitionData::TSnapshotId::EndOffsetMeta::Type TFetchResponseData::TFetchableTopicResponse::TPartitionData::TSnapshotId::EndOffsetMeta::Default = -1;
const TFetchResponseData::TFetchableTopicResponse::TPartitionData::TSnapshotId::EpochMeta::Type TFetchResponseData::TFetchableTopicResponse::TPartitionData::TSnapshotId::EpochMeta::Default = -1;
-TFetchResponseData::TFetchableTopicResponse::TPartitionData::TSnapshotId::TSnapshotId()
+TFetchResponseData::TFetchableTopicResponse::TPartitionData::TSnapshotId::TSnapshotId()
: EndOffset(EndOffsetMeta::Default)
, Epoch(EpochMeta::Default)
{}
@@ -1644,7 +1678,7 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TSnapshotId::R
}
NPrivate::Read<EndOffsetMeta>(_readable, _version, EndOffset);
NPrivate::Read<EpochMeta>(_readable, _version, Epoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1666,10 +1700,10 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TSnapshotId::W
NPrivate::TWriteCollector _collector;
NPrivate::Write<EndOffsetMeta>(_collector, _writable, _version, EndOffset);
NPrivate::Write<EpochMeta>(_collector, _writable, _version, Epoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -1677,7 +1711,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::TSnapshotId::Si
NPrivate::TSizeCollector _collector;
NPrivate::Size<EndOffsetMeta>(_collector, _version, EndOffset);
NPrivate::Size<EpochMeta>(_collector, _version, Epoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1691,7 +1725,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::TSnapshotId::Si
const TFetchResponseData::TFetchableTopicResponse::TPartitionData::TAbortedTransaction::ProducerIdMeta::Type TFetchResponseData::TFetchableTopicResponse::TPartitionData::TAbortedTransaction::ProducerIdMeta::Default = 0;
const TFetchResponseData::TFetchableTopicResponse::TPartitionData::TAbortedTransaction::FirstOffsetMeta::Type TFetchResponseData::TFetchableTopicResponse::TPartitionData::TAbortedTransaction::FirstOffsetMeta::Default = 0;
-TFetchResponseData::TFetchableTopicResponse::TPartitionData::TAbortedTransaction::TAbortedTransaction()
+TFetchResponseData::TFetchableTopicResponse::TPartitionData::TAbortedTransaction::TAbortedTransaction()
: ProducerId(ProducerIdMeta::Default)
, FirstOffset(FirstOffsetMeta::Default)
{}
@@ -1702,7 +1736,7 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TAbortedTransa
}
NPrivate::Read<ProducerIdMeta>(_readable, _version, ProducerId);
NPrivate::Read<FirstOffsetMeta>(_readable, _version, FirstOffset);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1724,10 +1758,10 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TAbortedTransa
NPrivate::TWriteCollector _collector;
NPrivate::Write<ProducerIdMeta>(_collector, _writable, _version, ProducerId);
NPrivate::Write<FirstOffsetMeta>(_collector, _writable, _version, FirstOffset);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -1735,7 +1769,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::TAbortedTransac
NPrivate::TSizeCollector _collector;
NPrivate::Size<ProducerIdMeta>(_collector, _version, ProducerId);
NPrivate::Size<FirstOffsetMeta>(_collector, _version, FirstOffset);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1749,7 +1783,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::TAbortedTransac
const TListOffsetsRequestData::ReplicaIdMeta::Type TListOffsetsRequestData::ReplicaIdMeta::Default = 0;
const TListOffsetsRequestData::IsolationLevelMeta::Type TListOffsetsRequestData::IsolationLevelMeta::Default = 0;
-TListOffsetsRequestData::TListOffsetsRequestData()
+TListOffsetsRequestData::TListOffsetsRequestData()
: ReplicaId(ReplicaIdMeta::Default)
, IsolationLevel(IsolationLevelMeta::Default)
{}
@@ -1761,7 +1795,7 @@ void TListOffsetsRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _ver
NPrivate::Read<ReplicaIdMeta>(_readable, _version, ReplicaId);
NPrivate::Read<IsolationLevelMeta>(_readable, _version, IsolationLevel);
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1784,10 +1818,10 @@ void TListOffsetsRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _ve
NPrivate::Write<ReplicaIdMeta>(_collector, _writable, _version, ReplicaId);
NPrivate::Write<IsolationLevelMeta>(_collector, _writable, _version, IsolationLevel);
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -1796,7 +1830,7 @@ i32 TListOffsetsRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<ReplicaIdMeta>(_collector, _version, ReplicaId);
NPrivate::Size<IsolationLevelMeta>(_collector, _version, IsolationLevel);
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1809,7 +1843,7 @@ i32 TListOffsetsRequestData::Size(TKafkaVersion _version) const {
//
const TListOffsetsRequestData::TListOffsetsTopic::NameMeta::Type TListOffsetsRequestData::TListOffsetsTopic::NameMeta::Default = {""};
-TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsTopic()
+TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsTopic()
: Name(NameMeta::Default)
{}
@@ -1819,7 +1853,7 @@ void TListOffsetsRequestData::TListOffsetsTopic::Read(TKafkaReadable& _readable,
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<PartitionsMeta>(_readable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1841,10 +1875,10 @@ void TListOffsetsRequestData::TListOffsetsTopic::Write(TKafkaWritable& _writable
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<PartitionsMeta>(_collector, _writable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -1852,7 +1886,7 @@ i32 TListOffsetsRequestData::TListOffsetsTopic::Size(TKafkaVersion _version) con
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<PartitionsMeta>(_collector, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1868,7 +1902,7 @@ const TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::Current
const TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::TimestampMeta::Type TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::TimestampMeta::Default = 0;
const TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::MaxNumOffsetsMeta::Type TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::MaxNumOffsetsMeta::Default = 1;
-TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::TListOffsetsPartition()
+TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::TListOffsetsPartition()
: PartitionIndex(PartitionIndexMeta::Default)
, CurrentLeaderEpoch(CurrentLeaderEpochMeta::Default)
, Timestamp(TimestampMeta::Default)
@@ -1883,7 +1917,7 @@ void TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::Read(TKa
NPrivate::Read<CurrentLeaderEpochMeta>(_readable, _version, CurrentLeaderEpoch);
NPrivate::Read<TimestampMeta>(_readable, _version, Timestamp);
NPrivate::Read<MaxNumOffsetsMeta>(_readable, _version, MaxNumOffsets);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1907,10 +1941,10 @@ void TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::Write(TK
NPrivate::Write<CurrentLeaderEpochMeta>(_collector, _writable, _version, CurrentLeaderEpoch);
NPrivate::Write<TimestampMeta>(_collector, _writable, _version, Timestamp);
NPrivate::Write<MaxNumOffsetsMeta>(_collector, _writable, _version, MaxNumOffsets);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -1920,7 +1954,7 @@ i32 TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::Size(TKaf
NPrivate::Size<CurrentLeaderEpochMeta>(_collector, _version, CurrentLeaderEpoch);
NPrivate::Size<TimestampMeta>(_collector, _version, Timestamp);
NPrivate::Size<MaxNumOffsetsMeta>(_collector, _version, MaxNumOffsets);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1933,7 +1967,7 @@ i32 TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::Size(TKaf
//
const TListOffsetsResponseData::ThrottleTimeMsMeta::Type TListOffsetsResponseData::ThrottleTimeMsMeta::Default = 0;
-TListOffsetsResponseData::TListOffsetsResponseData()
+TListOffsetsResponseData::TListOffsetsResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
{}
@@ -1943,7 +1977,7 @@ void TListOffsetsResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _ve
}
NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -1965,10 +1999,10 @@ void TListOffsetsResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _v
NPrivate::TWriteCollector _collector;
NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -1976,7 +2010,7 @@ i32 TListOffsetsResponseData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -1989,7 +2023,7 @@ i32 TListOffsetsResponseData::Size(TKafkaVersion _version) const {
//
const TListOffsetsResponseData::TListOffsetsTopicResponse::NameMeta::Type TListOffsetsResponseData::TListOffsetsTopicResponse::NameMeta::Default = {""};
-TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsTopicResponse()
+TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsTopicResponse()
: Name(NameMeta::Default)
{}
@@ -1999,7 +2033,7 @@ void TListOffsetsResponseData::TListOffsetsTopicResponse::Read(TKafkaReadable& _
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<PartitionsMeta>(_readable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2021,10 +2055,10 @@ void TListOffsetsResponseData::TListOffsetsTopicResponse::Write(TKafkaWritable&
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<PartitionsMeta>(_collector, _writable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2032,7 +2066,7 @@ i32 TListOffsetsResponseData::TListOffsetsTopicResponse::Size(TKafkaVersion _ver
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<PartitionsMeta>(_collector, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2049,7 +2083,7 @@ const TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsPartition
const TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsPartitionResponse::OffsetMeta::Type TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsPartitionResponse::OffsetMeta::Default = -1;
const TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsPartitionResponse::LeaderEpochMeta::Type TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsPartitionResponse::LeaderEpochMeta::Default = -1;
-TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsPartitionResponse::TListOffsetsPartitionResponse()
+TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsPartitionResponse::TListOffsetsPartitionResponse()
: PartitionIndex(PartitionIndexMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
, Timestamp(TimestampMeta::Default)
@@ -2067,7 +2101,7 @@ void TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsPartitionR
NPrivate::Read<TimestampMeta>(_readable, _version, Timestamp);
NPrivate::Read<OffsetMeta>(_readable, _version, Offset);
NPrivate::Read<LeaderEpochMeta>(_readable, _version, LeaderEpoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2093,10 +2127,10 @@ void TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsPartitionR
NPrivate::Write<TimestampMeta>(_collector, _writable, _version, Timestamp);
NPrivate::Write<OffsetMeta>(_collector, _writable, _version, Offset);
NPrivate::Write<LeaderEpochMeta>(_collector, _writable, _version, LeaderEpoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2108,7 +2142,7 @@ i32 TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsPartitionRe
NPrivate::Size<TimestampMeta>(_collector, _version, Timestamp);
NPrivate::Size<OffsetMeta>(_collector, _version, Offset);
NPrivate::Size<LeaderEpochMeta>(_collector, _version, LeaderEpoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2123,7 +2157,7 @@ const TMetadataRequestData::AllowAutoTopicCreationMeta::Type TMetadataRequestDat
const TMetadataRequestData::IncludeClusterAuthorizedOperationsMeta::Type TMetadataRequestData::IncludeClusterAuthorizedOperationsMeta::Default = false;
const TMetadataRequestData::IncludeTopicAuthorizedOperationsMeta::Type TMetadataRequestData::IncludeTopicAuthorizedOperationsMeta::Default = false;
-TMetadataRequestData::TMetadataRequestData()
+TMetadataRequestData::TMetadataRequestData()
: AllowAutoTopicCreation(AllowAutoTopicCreationMeta::Default)
, IncludeClusterAuthorizedOperations(IncludeClusterAuthorizedOperationsMeta::Default)
, IncludeTopicAuthorizedOperations(IncludeTopicAuthorizedOperationsMeta::Default)
@@ -2137,7 +2171,7 @@ void TMetadataRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _versio
NPrivate::Read<AllowAutoTopicCreationMeta>(_readable, _version, AllowAutoTopicCreation);
NPrivate::Read<IncludeClusterAuthorizedOperationsMeta>(_readable, _version, IncludeClusterAuthorizedOperations);
NPrivate::Read<IncludeTopicAuthorizedOperationsMeta>(_readable, _version, IncludeTopicAuthorizedOperations);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2161,10 +2195,10 @@ void TMetadataRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _versi
NPrivate::Write<AllowAutoTopicCreationMeta>(_collector, _writable, _version, AllowAutoTopicCreation);
NPrivate::Write<IncludeClusterAuthorizedOperationsMeta>(_collector, _writable, _version, IncludeClusterAuthorizedOperations);
NPrivate::Write<IncludeTopicAuthorizedOperationsMeta>(_collector, _writable, _version, IncludeTopicAuthorizedOperations);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2174,7 +2208,7 @@ i32 TMetadataRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<AllowAutoTopicCreationMeta>(_collector, _version, AllowAutoTopicCreation);
NPrivate::Size<IncludeClusterAuthorizedOperationsMeta>(_collector, _version, IncludeClusterAuthorizedOperations);
NPrivate::Size<IncludeTopicAuthorizedOperationsMeta>(_collector, _version, IncludeTopicAuthorizedOperations);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2188,7 +2222,7 @@ i32 TMetadataRequestData::Size(TKafkaVersion _version) const {
const TMetadataRequestData::TMetadataRequestTopic::TopicIdMeta::Type TMetadataRequestData::TMetadataRequestTopic::TopicIdMeta::Default = TKafkaUuid(0, 0);
const TMetadataRequestData::TMetadataRequestTopic::NameMeta::Type TMetadataRequestData::TMetadataRequestTopic::NameMeta::Default = {""};
-TMetadataRequestData::TMetadataRequestTopic::TMetadataRequestTopic()
+TMetadataRequestData::TMetadataRequestTopic::TMetadataRequestTopic()
: TopicId(TopicIdMeta::Default)
, Name(NameMeta::Default)
{}
@@ -2199,7 +2233,7 @@ void TMetadataRequestData::TMetadataRequestTopic::Read(TKafkaReadable& _readable
}
NPrivate::Read<TopicIdMeta>(_readable, _version, TopicId);
NPrivate::Read<NameMeta>(_readable, _version, Name);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2221,10 +2255,10 @@ void TMetadataRequestData::TMetadataRequestTopic::Write(TKafkaWritable& _writabl
NPrivate::TWriteCollector _collector;
NPrivate::Write<TopicIdMeta>(_collector, _writable, _version, TopicId);
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2232,7 +2266,7 @@ i32 TMetadataRequestData::TMetadataRequestTopic::Size(TKafkaVersion _version) co
NPrivate::TSizeCollector _collector;
NPrivate::Size<TopicIdMeta>(_collector, _version, TopicId);
NPrivate::Size<NameMeta>(_collector, _version, Name);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2248,7 +2282,7 @@ const TMetadataResponseData::ClusterIdMeta::Type TMetadataResponseData::ClusterI
const TMetadataResponseData::ControllerIdMeta::Type TMetadataResponseData::ControllerIdMeta::Default = -1;
const TMetadataResponseData::ClusterAuthorizedOperationsMeta::Type TMetadataResponseData::ClusterAuthorizedOperationsMeta::Default = -2147483648;
-TMetadataResponseData::TMetadataResponseData()
+TMetadataResponseData::TMetadataResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
, ClusterId(ClusterIdMeta::Default)
, ControllerId(ControllerIdMeta::Default)
@@ -2265,7 +2299,7 @@ void TMetadataResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _versi
NPrivate::Read<ControllerIdMeta>(_readable, _version, ControllerId);
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
NPrivate::Read<ClusterAuthorizedOperationsMeta>(_readable, _version, ClusterAuthorizedOperations);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2291,10 +2325,10 @@ void TMetadataResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _vers
NPrivate::Write<ControllerIdMeta>(_collector, _writable, _version, ControllerId);
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
NPrivate::Write<ClusterAuthorizedOperationsMeta>(_collector, _writable, _version, ClusterAuthorizedOperations);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2306,7 +2340,7 @@ i32 TMetadataResponseData::Size(TKafkaVersion _version) const {
NPrivate::Size<ControllerIdMeta>(_collector, _version, ControllerId);
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
NPrivate::Size<ClusterAuthorizedOperationsMeta>(_collector, _version, ClusterAuthorizedOperations);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2322,7 +2356,7 @@ const TMetadataResponseData::TMetadataResponseBroker::HostMeta::Type TMetadataRe
const TMetadataResponseData::TMetadataResponseBroker::PortMeta::Type TMetadataResponseData::TMetadataResponseBroker::PortMeta::Default = 0;
const TMetadataResponseData::TMetadataResponseBroker::RackMeta::Type TMetadataResponseData::TMetadataResponseBroker::RackMeta::Default = std::nullopt;
-TMetadataResponseData::TMetadataResponseBroker::TMetadataResponseBroker()
+TMetadataResponseData::TMetadataResponseBroker::TMetadataResponseBroker()
: NodeId(NodeIdMeta::Default)
, Host(HostMeta::Default)
, Port(PortMeta::Default)
@@ -2337,7 +2371,7 @@ void TMetadataResponseData::TMetadataResponseBroker::Read(TKafkaReadable& _reada
NPrivate::Read<HostMeta>(_readable, _version, Host);
NPrivate::Read<PortMeta>(_readable, _version, Port);
NPrivate::Read<RackMeta>(_readable, _version, Rack);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2361,10 +2395,10 @@ void TMetadataResponseData::TMetadataResponseBroker::Write(TKafkaWritable& _writ
NPrivate::Write<HostMeta>(_collector, _writable, _version, Host);
NPrivate::Write<PortMeta>(_collector, _writable, _version, Port);
NPrivate::Write<RackMeta>(_collector, _writable, _version, Rack);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2374,7 +2408,7 @@ i32 TMetadataResponseData::TMetadataResponseBroker::Size(TKafkaVersion _version)
NPrivate::Size<HostMeta>(_collector, _version, Host);
NPrivate::Size<PortMeta>(_collector, _version, Port);
NPrivate::Size<RackMeta>(_collector, _version, Rack);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2391,7 +2425,7 @@ const TMetadataResponseData::TMetadataResponseTopic::TopicIdMeta::Type TMetadata
const TMetadataResponseData::TMetadataResponseTopic::IsInternalMeta::Type TMetadataResponseData::TMetadataResponseTopic::IsInternalMeta::Default = false;
const TMetadataResponseData::TMetadataResponseTopic::TopicAuthorizedOperationsMeta::Type TMetadataResponseData::TMetadataResponseTopic::TopicAuthorizedOperationsMeta::Default = -2147483648;
-TMetadataResponseData::TMetadataResponseTopic::TMetadataResponseTopic()
+TMetadataResponseData::TMetadataResponseTopic::TMetadataResponseTopic()
: ErrorCode(ErrorCodeMeta::Default)
, Name(NameMeta::Default)
, TopicId(TopicIdMeta::Default)
@@ -2409,7 +2443,7 @@ void TMetadataResponseData::TMetadataResponseTopic::Read(TKafkaReadable& _readab
NPrivate::Read<IsInternalMeta>(_readable, _version, IsInternal);
NPrivate::Read<PartitionsMeta>(_readable, _version, Partitions);
NPrivate::Read<TopicAuthorizedOperationsMeta>(_readable, _version, TopicAuthorizedOperations);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2435,10 +2469,10 @@ void TMetadataResponseData::TMetadataResponseTopic::Write(TKafkaWritable& _writa
NPrivate::Write<IsInternalMeta>(_collector, _writable, _version, IsInternal);
NPrivate::Write<PartitionsMeta>(_collector, _writable, _version, Partitions);
NPrivate::Write<TopicAuthorizedOperationsMeta>(_collector, _writable, _version, TopicAuthorizedOperations);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2450,7 +2484,7 @@ i32 TMetadataResponseData::TMetadataResponseTopic::Size(TKafkaVersion _version)
NPrivate::Size<IsInternalMeta>(_collector, _version, IsInternal);
NPrivate::Size<PartitionsMeta>(_collector, _version, Partitions);
NPrivate::Size<TopicAuthorizedOperationsMeta>(_collector, _version, TopicAuthorizedOperations);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2466,7 +2500,7 @@ const TMetadataResponseData::TMetadataResponseTopic::TMetadataResponsePartition:
const TMetadataResponseData::TMetadataResponseTopic::TMetadataResponsePartition::LeaderIdMeta::Type TMetadataResponseData::TMetadataResponseTopic::TMetadataResponsePartition::LeaderIdMeta::Default = 0;
const TMetadataResponseData::TMetadataResponseTopic::TMetadataResponsePartition::LeaderEpochMeta::Type TMetadataResponseData::TMetadataResponseTopic::TMetadataResponsePartition::LeaderEpochMeta::Default = -1;
-TMetadataResponseData::TMetadataResponseTopic::TMetadataResponsePartition::TMetadataResponsePartition()
+TMetadataResponseData::TMetadataResponseTopic::TMetadataResponsePartition::TMetadataResponsePartition()
: ErrorCode(ErrorCodeMeta::Default)
, PartitionIndex(PartitionIndexMeta::Default)
, LeaderId(LeaderIdMeta::Default)
@@ -2484,7 +2518,7 @@ void TMetadataResponseData::TMetadataResponseTopic::TMetadataResponsePartition::
NPrivate::Read<ReplicaNodesMeta>(_readable, _version, ReplicaNodes);
NPrivate::Read<IsrNodesMeta>(_readable, _version, IsrNodes);
NPrivate::Read<OfflineReplicasMeta>(_readable, _version, OfflineReplicas);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2511,10 +2545,10 @@ void TMetadataResponseData::TMetadataResponseTopic::TMetadataResponsePartition::
NPrivate::Write<ReplicaNodesMeta>(_collector, _writable, _version, ReplicaNodes);
NPrivate::Write<IsrNodesMeta>(_collector, _writable, _version, IsrNodes);
NPrivate::Write<OfflineReplicasMeta>(_collector, _writable, _version, OfflineReplicas);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2527,7 +2561,7 @@ i32 TMetadataResponseData::TMetadataResponseTopic::TMetadataResponsePartition::S
NPrivate::Size<ReplicaNodesMeta>(_collector, _version, ReplicaNodes);
NPrivate::Size<IsrNodesMeta>(_collector, _version, IsrNodes);
NPrivate::Size<OfflineReplicasMeta>(_collector, _version, OfflineReplicas);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2544,7 +2578,7 @@ const TOffsetCommitRequestData::MemberIdMeta::Type TOffsetCommitRequestData::Mem
const TOffsetCommitRequestData::GroupInstanceIdMeta::Type TOffsetCommitRequestData::GroupInstanceIdMeta::Default = std::nullopt;
const TOffsetCommitRequestData::RetentionTimeMsMeta::Type TOffsetCommitRequestData::RetentionTimeMsMeta::Default = -1;
-TOffsetCommitRequestData::TOffsetCommitRequestData()
+TOffsetCommitRequestData::TOffsetCommitRequestData()
: GroupId(GroupIdMeta::Default)
, GenerationId(GenerationIdMeta::Default)
, MemberId(MemberIdMeta::Default)
@@ -2562,7 +2596,7 @@ void TOffsetCommitRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _ve
NPrivate::Read<GroupInstanceIdMeta>(_readable, _version, GroupInstanceId);
NPrivate::Read<RetentionTimeMsMeta>(_readable, _version, RetentionTimeMs);
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2588,10 +2622,10 @@ void TOffsetCommitRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _v
NPrivate::Write<GroupInstanceIdMeta>(_collector, _writable, _version, GroupInstanceId);
NPrivate::Write<RetentionTimeMsMeta>(_collector, _writable, _version, RetentionTimeMs);
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2603,7 +2637,7 @@ i32 TOffsetCommitRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<GroupInstanceIdMeta>(_collector, _version, GroupInstanceId);
NPrivate::Size<RetentionTimeMsMeta>(_collector, _version, RetentionTimeMs);
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2616,7 +2650,7 @@ i32 TOffsetCommitRequestData::Size(TKafkaVersion _version) const {
//
const TOffsetCommitRequestData::TOffsetCommitRequestTopic::NameMeta::Type TOffsetCommitRequestData::TOffsetCommitRequestTopic::NameMeta::Default = {""};
-TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestTopic()
+TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestTopic()
: Name(NameMeta::Default)
{}
@@ -2626,7 +2660,7 @@ void TOffsetCommitRequestData::TOffsetCommitRequestTopic::Read(TKafkaReadable& _
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<PartitionsMeta>(_readable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2648,10 +2682,10 @@ void TOffsetCommitRequestData::TOffsetCommitRequestTopic::Write(TKafkaWritable&
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<PartitionsMeta>(_collector, _writable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2659,7 +2693,7 @@ i32 TOffsetCommitRequestData::TOffsetCommitRequestTopic::Size(TKafkaVersion _ver
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<PartitionsMeta>(_collector, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2676,7 +2710,7 @@ const TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestP
const TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestPartition::CommitTimestampMeta::Type TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestPartition::CommitTimestampMeta::Default = -1;
const TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestPartition::CommittedMetadataMeta::Type TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestPartition::CommittedMetadataMeta::Default = {""};
-TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestPartition::TOffsetCommitRequestPartition()
+TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestPartition::TOffsetCommitRequestPartition()
: PartitionIndex(PartitionIndexMeta::Default)
, CommittedOffset(CommittedOffsetMeta::Default)
, CommittedLeaderEpoch(CommittedLeaderEpochMeta::Default)
@@ -2693,7 +2727,7 @@ void TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestPa
NPrivate::Read<CommittedLeaderEpochMeta>(_readable, _version, CommittedLeaderEpoch);
NPrivate::Read<CommitTimestampMeta>(_readable, _version, CommitTimestamp);
NPrivate::Read<CommittedMetadataMeta>(_readable, _version, CommittedMetadata);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2718,10 +2752,10 @@ void TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestPa
NPrivate::Write<CommittedLeaderEpochMeta>(_collector, _writable, _version, CommittedLeaderEpoch);
NPrivate::Write<CommitTimestampMeta>(_collector, _writable, _version, CommitTimestamp);
NPrivate::Write<CommittedMetadataMeta>(_collector, _writable, _version, CommittedMetadata);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2732,7 +2766,7 @@ i32 TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestPar
NPrivate::Size<CommittedLeaderEpochMeta>(_collector, _version, CommittedLeaderEpoch);
NPrivate::Size<CommitTimestampMeta>(_collector, _version, CommitTimestamp);
NPrivate::Size<CommittedMetadataMeta>(_collector, _version, CommittedMetadata);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2745,7 +2779,7 @@ i32 TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestPar
//
const TOffsetCommitResponseData::ThrottleTimeMsMeta::Type TOffsetCommitResponseData::ThrottleTimeMsMeta::Default = 0;
-TOffsetCommitResponseData::TOffsetCommitResponseData()
+TOffsetCommitResponseData::TOffsetCommitResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
{}
@@ -2755,7 +2789,7 @@ void TOffsetCommitResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _v
}
NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2777,10 +2811,10 @@ void TOffsetCommitResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _
NPrivate::TWriteCollector _collector;
NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2788,7 +2822,7 @@ i32 TOffsetCommitResponseData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2801,7 +2835,7 @@ i32 TOffsetCommitResponseData::Size(TKafkaVersion _version) const {
//
const TOffsetCommitResponseData::TOffsetCommitResponseTopic::NameMeta::Type TOffsetCommitResponseData::TOffsetCommitResponseTopic::NameMeta::Default = {""};
-TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitResponseTopic()
+TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitResponseTopic()
: Name(NameMeta::Default)
{}
@@ -2811,7 +2845,7 @@ void TOffsetCommitResponseData::TOffsetCommitResponseTopic::Read(TKafkaReadable&
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<PartitionsMeta>(_readable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2833,10 +2867,10 @@ void TOffsetCommitResponseData::TOffsetCommitResponseTopic::Write(TKafkaWritable
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<PartitionsMeta>(_collector, _writable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2844,7 +2878,7 @@ i32 TOffsetCommitResponseData::TOffsetCommitResponseTopic::Size(TKafkaVersion _v
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<PartitionsMeta>(_collector, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2858,7 +2892,7 @@ i32 TOffsetCommitResponseData::TOffsetCommitResponseTopic::Size(TKafkaVersion _v
const TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitResponsePartition::PartitionIndexMeta::Type TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitResponsePartition::PartitionIndexMeta::Default = 0;
const TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitResponsePartition::ErrorCodeMeta::Type TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitResponsePartition::ErrorCodeMeta::Default = 0;
-TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitResponsePartition::TOffsetCommitResponsePartition()
+TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitResponsePartition::TOffsetCommitResponsePartition()
: PartitionIndex(PartitionIndexMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
{}
@@ -2869,7 +2903,7 @@ void TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitRespons
}
NPrivate::Read<PartitionIndexMeta>(_readable, _version, PartitionIndex);
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2891,10 +2925,10 @@ void TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitRespons
NPrivate::TWriteCollector _collector;
NPrivate::Write<PartitionIndexMeta>(_collector, _writable, _version, PartitionIndex);
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2902,7 +2936,7 @@ i32 TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitResponse
NPrivate::TSizeCollector _collector;
NPrivate::Size<PartitionIndexMeta>(_collector, _version, PartitionIndex);
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2916,7 +2950,7 @@ i32 TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitResponse
const TOffsetFetchRequestData::GroupIdMeta::Type TOffsetFetchRequestData::GroupIdMeta::Default = {""};
const TOffsetFetchRequestData::RequireStableMeta::Type TOffsetFetchRequestData::RequireStableMeta::Default = false;
-TOffsetFetchRequestData::TOffsetFetchRequestData()
+TOffsetFetchRequestData::TOffsetFetchRequestData()
: GroupId(GroupIdMeta::Default)
, RequireStable(RequireStableMeta::Default)
{}
@@ -2929,7 +2963,7 @@ void TOffsetFetchRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _ver
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
NPrivate::Read<GroupsMeta>(_readable, _version, Groups);
NPrivate::Read<RequireStableMeta>(_readable, _version, RequireStable);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -2953,10 +2987,10 @@ void TOffsetFetchRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _ve
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
NPrivate::Write<GroupsMeta>(_collector, _writable, _version, Groups);
NPrivate::Write<RequireStableMeta>(_collector, _writable, _version, RequireStable);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -2966,7 +3000,7 @@ i32 TOffsetFetchRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
NPrivate::Size<GroupsMeta>(_collector, _version, Groups);
NPrivate::Size<RequireStableMeta>(_collector, _version, RequireStable);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -2979,7 +3013,7 @@ i32 TOffsetFetchRequestData::Size(TKafkaVersion _version) const {
//
const TOffsetFetchRequestData::TOffsetFetchRequestTopic::NameMeta::Type TOffsetFetchRequestData::TOffsetFetchRequestTopic::NameMeta::Default = {""};
-TOffsetFetchRequestData::TOffsetFetchRequestTopic::TOffsetFetchRequestTopic()
+TOffsetFetchRequestData::TOffsetFetchRequestTopic::TOffsetFetchRequestTopic()
: Name(NameMeta::Default)
{}
@@ -2989,7 +3023,7 @@ void TOffsetFetchRequestData::TOffsetFetchRequestTopic::Read(TKafkaReadable& _re
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<PartitionIndexesMeta>(_readable, _version, PartitionIndexes);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3011,10 +3045,10 @@ void TOffsetFetchRequestData::TOffsetFetchRequestTopic::Write(TKafkaWritable& _w
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<PartitionIndexesMeta>(_collector, _writable, _version, PartitionIndexes);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3022,7 +3056,7 @@ i32 TOffsetFetchRequestData::TOffsetFetchRequestTopic::Size(TKafkaVersion _versi
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<PartitionIndexesMeta>(_collector, _version, PartitionIndexes);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3035,7 +3069,7 @@ i32 TOffsetFetchRequestData::TOffsetFetchRequestTopic::Size(TKafkaVersion _versi
//
const TOffsetFetchRequestData::TOffsetFetchRequestGroup::GroupIdMeta::Type TOffsetFetchRequestData::TOffsetFetchRequestGroup::GroupIdMeta::Default = {""};
-TOffsetFetchRequestData::TOffsetFetchRequestGroup::TOffsetFetchRequestGroup()
+TOffsetFetchRequestData::TOffsetFetchRequestGroup::TOffsetFetchRequestGroup()
: GroupId(GroupIdMeta::Default)
{}
@@ -3045,7 +3079,7 @@ void TOffsetFetchRequestData::TOffsetFetchRequestGroup::Read(TKafkaReadable& _re
}
NPrivate::Read<GroupIdMeta>(_readable, _version, GroupId);
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3067,10 +3101,10 @@ void TOffsetFetchRequestData::TOffsetFetchRequestGroup::Write(TKafkaWritable& _w
NPrivate::TWriteCollector _collector;
NPrivate::Write<GroupIdMeta>(_collector, _writable, _version, GroupId);
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3078,7 +3112,7 @@ i32 TOffsetFetchRequestData::TOffsetFetchRequestGroup::Size(TKafkaVersion _versi
NPrivate::TSizeCollector _collector;
NPrivate::Size<GroupIdMeta>(_collector, _version, GroupId);
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3091,7 +3125,7 @@ i32 TOffsetFetchRequestData::TOffsetFetchRequestGroup::Size(TKafkaVersion _versi
//
const TOffsetFetchRequestData::TOffsetFetchRequestGroup::TOffsetFetchRequestTopics::NameMeta::Type TOffsetFetchRequestData::TOffsetFetchRequestGroup::TOffsetFetchRequestTopics::NameMeta::Default = {""};
-TOffsetFetchRequestData::TOffsetFetchRequestGroup::TOffsetFetchRequestTopics::TOffsetFetchRequestTopics()
+TOffsetFetchRequestData::TOffsetFetchRequestGroup::TOffsetFetchRequestTopics::TOffsetFetchRequestTopics()
: Name(NameMeta::Default)
{}
@@ -3101,7 +3135,7 @@ void TOffsetFetchRequestData::TOffsetFetchRequestGroup::TOffsetFetchRequestTopic
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<PartitionIndexesMeta>(_readable, _version, PartitionIndexes);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3123,10 +3157,10 @@ void TOffsetFetchRequestData::TOffsetFetchRequestGroup::TOffsetFetchRequestTopic
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<PartitionIndexesMeta>(_collector, _writable, _version, PartitionIndexes);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3134,7 +3168,7 @@ i32 TOffsetFetchRequestData::TOffsetFetchRequestGroup::TOffsetFetchRequestTopics
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<PartitionIndexesMeta>(_collector, _version, PartitionIndexes);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3148,7 +3182,7 @@ i32 TOffsetFetchRequestData::TOffsetFetchRequestGroup::TOffsetFetchRequestTopics
const TOffsetFetchResponseData::ThrottleTimeMsMeta::Type TOffsetFetchResponseData::ThrottleTimeMsMeta::Default = 0;
const TOffsetFetchResponseData::ErrorCodeMeta::Type TOffsetFetchResponseData::ErrorCodeMeta::Default = 0;
-TOffsetFetchResponseData::TOffsetFetchResponseData()
+TOffsetFetchResponseData::TOffsetFetchResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
{}
@@ -3161,7 +3195,7 @@ void TOffsetFetchResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _ve
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
NPrivate::Read<GroupsMeta>(_readable, _version, Groups);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3185,10 +3219,10 @@ void TOffsetFetchResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _v
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
NPrivate::Write<GroupsMeta>(_collector, _writable, _version, Groups);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3198,7 +3232,7 @@ i32 TOffsetFetchResponseData::Size(TKafkaVersion _version) const {
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
NPrivate::Size<GroupsMeta>(_collector, _version, Groups);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3211,7 +3245,7 @@ i32 TOffsetFetchResponseData::Size(TKafkaVersion _version) const {
//
const TOffsetFetchResponseData::TOffsetFetchResponseTopic::NameMeta::Type TOffsetFetchResponseData::TOffsetFetchResponseTopic::NameMeta::Default = {""};
-TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponseTopic()
+TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponseTopic()
: Name(NameMeta::Default)
{}
@@ -3221,7 +3255,7 @@ void TOffsetFetchResponseData::TOffsetFetchResponseTopic::Read(TKafkaReadable& _
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<PartitionsMeta>(_readable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3243,10 +3277,10 @@ void TOffsetFetchResponseData::TOffsetFetchResponseTopic::Write(TKafkaWritable&
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<PartitionsMeta>(_collector, _writable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3254,7 +3288,7 @@ i32 TOffsetFetchResponseData::TOffsetFetchResponseTopic::Size(TKafkaVersion _ver
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<PartitionsMeta>(_collector, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3271,7 +3305,7 @@ const TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponseP
const TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponsePartition::MetadataMeta::Type TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponsePartition::MetadataMeta::Default = {""};
const TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponsePartition::ErrorCodeMeta::Type TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponsePartition::ErrorCodeMeta::Default = 0;
-TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponsePartition::TOffsetFetchResponsePartition()
+TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponsePartition::TOffsetFetchResponsePartition()
: PartitionIndex(PartitionIndexMeta::Default)
, CommittedOffset(CommittedOffsetMeta::Default)
, CommittedLeaderEpoch(CommittedLeaderEpochMeta::Default)
@@ -3288,7 +3322,7 @@ void TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponsePa
NPrivate::Read<CommittedLeaderEpochMeta>(_readable, _version, CommittedLeaderEpoch);
NPrivate::Read<MetadataMeta>(_readable, _version, Metadata);
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3313,10 +3347,10 @@ void TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponsePa
NPrivate::Write<CommittedLeaderEpochMeta>(_collector, _writable, _version, CommittedLeaderEpoch);
NPrivate::Write<MetadataMeta>(_collector, _writable, _version, Metadata);
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3327,7 +3361,7 @@ i32 TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponsePar
NPrivate::Size<CommittedLeaderEpochMeta>(_collector, _version, CommittedLeaderEpoch);
NPrivate::Size<MetadataMeta>(_collector, _version, Metadata);
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3341,7 +3375,7 @@ i32 TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponsePar
const TOffsetFetchResponseData::TOffsetFetchResponseGroup::GroupIdMeta::Type TOffsetFetchResponseData::TOffsetFetchResponseGroup::GroupIdMeta::Default = {""};
const TOffsetFetchResponseData::TOffsetFetchResponseGroup::ErrorCodeMeta::Type TOffsetFetchResponseData::TOffsetFetchResponseGroup::ErrorCodeMeta::Default = 0;
-TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseGroup()
+TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseGroup()
: GroupId(GroupIdMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
{}
@@ -3353,7 +3387,7 @@ void TOffsetFetchResponseData::TOffsetFetchResponseGroup::Read(TKafkaReadable& _
NPrivate::Read<GroupIdMeta>(_readable, _version, GroupId);
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3376,10 +3410,10 @@ void TOffsetFetchResponseData::TOffsetFetchResponseGroup::Write(TKafkaWritable&
NPrivate::Write<GroupIdMeta>(_collector, _writable, _version, GroupId);
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3388,7 +3422,7 @@ i32 TOffsetFetchResponseData::TOffsetFetchResponseGroup::Size(TKafkaVersion _ver
NPrivate::Size<GroupIdMeta>(_collector, _version, GroupId);
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3401,7 +3435,7 @@ i32 TOffsetFetchResponseData::TOffsetFetchResponseGroup::Size(TKafkaVersion _ver
//
const TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTopics::NameMeta::Type TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTopics::NameMeta::Default = {""};
-TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTopics::TOffsetFetchResponseTopics()
+TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTopics::TOffsetFetchResponseTopics()
: Name(NameMeta::Default)
{}
@@ -3411,7 +3445,7 @@ void TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTo
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<PartitionsMeta>(_readable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3433,10 +3467,10 @@ void TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTo
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<PartitionsMeta>(_collector, _writable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3444,7 +3478,7 @@ i32 TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTop
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<PartitionsMeta>(_collector, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3461,7 +3495,7 @@ const TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseT
const TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTopics::TOffsetFetchResponsePartitions::MetadataMeta::Type TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTopics::TOffsetFetchResponsePartitions::MetadataMeta::Default = {""};
const TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTopics::TOffsetFetchResponsePartitions::ErrorCodeMeta::Type TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTopics::TOffsetFetchResponsePartitions::ErrorCodeMeta::Default = 0;
-TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTopics::TOffsetFetchResponsePartitions::TOffsetFetchResponsePartitions()
+TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTopics::TOffsetFetchResponsePartitions::TOffsetFetchResponsePartitions()
: PartitionIndex(PartitionIndexMeta::Default)
, CommittedOffset(CommittedOffsetMeta::Default)
, CommittedLeaderEpoch(CommittedLeaderEpochMeta::Default)
@@ -3478,7 +3512,7 @@ void TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTo
NPrivate::Read<CommittedLeaderEpochMeta>(_readable, _version, CommittedLeaderEpoch);
NPrivate::Read<MetadataMeta>(_readable, _version, Metadata);
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3503,10 +3537,10 @@ void TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTo
NPrivate::Write<CommittedLeaderEpochMeta>(_collector, _writable, _version, CommittedLeaderEpoch);
NPrivate::Write<MetadataMeta>(_collector, _writable, _version, Metadata);
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3517,7 +3551,7 @@ i32 TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTop
NPrivate::Size<CommittedLeaderEpochMeta>(_collector, _version, CommittedLeaderEpoch);
NPrivate::Size<MetadataMeta>(_collector, _version, Metadata);
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3531,7 +3565,7 @@ i32 TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTop
const TFindCoordinatorRequestData::KeyMeta::Type TFindCoordinatorRequestData::KeyMeta::Default = {""};
const TFindCoordinatorRequestData::KeyTypeMeta::Type TFindCoordinatorRequestData::KeyTypeMeta::Default = 0;
-TFindCoordinatorRequestData::TFindCoordinatorRequestData()
+TFindCoordinatorRequestData::TFindCoordinatorRequestData()
: Key(KeyMeta::Default)
, KeyType(KeyTypeMeta::Default)
{}
@@ -3543,7 +3577,7 @@ void TFindCoordinatorRequestData::Read(TKafkaReadable& _readable, TKafkaVersion
NPrivate::Read<KeyMeta>(_readable, _version, Key);
NPrivate::Read<KeyTypeMeta>(_readable, _version, KeyType);
NPrivate::Read<CoordinatorKeysMeta>(_readable, _version, CoordinatorKeys);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3566,10 +3600,10 @@ void TFindCoordinatorRequestData::Write(TKafkaWritable& _writable, TKafkaVersion
NPrivate::Write<KeyMeta>(_collector, _writable, _version, Key);
NPrivate::Write<KeyTypeMeta>(_collector, _writable, _version, KeyType);
NPrivate::Write<CoordinatorKeysMeta>(_collector, _writable, _version, CoordinatorKeys);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3578,7 +3612,7 @@ i32 TFindCoordinatorRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<KeyMeta>(_collector, _version, Key);
NPrivate::Size<KeyTypeMeta>(_collector, _version, KeyType);
NPrivate::Size<CoordinatorKeysMeta>(_collector, _version, CoordinatorKeys);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3596,7 +3630,7 @@ const TFindCoordinatorResponseData::NodeIdMeta::Type TFindCoordinatorResponseDat
const TFindCoordinatorResponseData::HostMeta::Type TFindCoordinatorResponseData::HostMeta::Default = {""};
const TFindCoordinatorResponseData::PortMeta::Type TFindCoordinatorResponseData::PortMeta::Default = 0;
-TFindCoordinatorResponseData::TFindCoordinatorResponseData()
+TFindCoordinatorResponseData::TFindCoordinatorResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
, ErrorMessage(ErrorMessageMeta::Default)
@@ -3616,7 +3650,7 @@ void TFindCoordinatorResponseData::Read(TKafkaReadable& _readable, TKafkaVersion
NPrivate::Read<HostMeta>(_readable, _version, Host);
NPrivate::Read<PortMeta>(_readable, _version, Port);
NPrivate::Read<CoordinatorsMeta>(_readable, _version, Coordinators);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3643,10 +3677,10 @@ void TFindCoordinatorResponseData::Write(TKafkaWritable& _writable, TKafkaVersio
NPrivate::Write<HostMeta>(_collector, _writable, _version, Host);
NPrivate::Write<PortMeta>(_collector, _writable, _version, Port);
NPrivate::Write<CoordinatorsMeta>(_collector, _writable, _version, Coordinators);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3659,7 +3693,7 @@ i32 TFindCoordinatorResponseData::Size(TKafkaVersion _version) const {
NPrivate::Size<HostMeta>(_collector, _version, Host);
NPrivate::Size<PortMeta>(_collector, _version, Port);
NPrivate::Size<CoordinatorsMeta>(_collector, _version, Coordinators);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3677,7 +3711,7 @@ const TFindCoordinatorResponseData::TCoordinator::PortMeta::Type TFindCoordinato
const TFindCoordinatorResponseData::TCoordinator::ErrorCodeMeta::Type TFindCoordinatorResponseData::TCoordinator::ErrorCodeMeta::Default = 0;
const TFindCoordinatorResponseData::TCoordinator::ErrorMessageMeta::Type TFindCoordinatorResponseData::TCoordinator::ErrorMessageMeta::Default = {""};
-TFindCoordinatorResponseData::TCoordinator::TCoordinator()
+TFindCoordinatorResponseData::TCoordinator::TCoordinator()
: Key(KeyMeta::Default)
, NodeId(NodeIdMeta::Default)
, Host(HostMeta::Default)
@@ -3696,7 +3730,7 @@ void TFindCoordinatorResponseData::TCoordinator::Read(TKafkaReadable& _readable,
NPrivate::Read<PortMeta>(_readable, _version, Port);
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
NPrivate::Read<ErrorMessageMeta>(_readable, _version, ErrorMessage);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3722,10 +3756,10 @@ void TFindCoordinatorResponseData::TCoordinator::Write(TKafkaWritable& _writable
NPrivate::Write<PortMeta>(_collector, _writable, _version, Port);
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
NPrivate::Write<ErrorMessageMeta>(_collector, _writable, _version, ErrorMessage);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3737,7 +3771,7 @@ i32 TFindCoordinatorResponseData::TCoordinator::Size(TKafkaVersion _version) con
NPrivate::Size<PortMeta>(_collector, _version, Port);
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
NPrivate::Size<ErrorMessageMeta>(_collector, _version, ErrorMessage);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3756,7 +3790,7 @@ const TJoinGroupRequestData::GroupInstanceIdMeta::Type TJoinGroupRequestData::Gr
const TJoinGroupRequestData::ProtocolTypeMeta::Type TJoinGroupRequestData::ProtocolTypeMeta::Default = {""};
const TJoinGroupRequestData::ReasonMeta::Type TJoinGroupRequestData::ReasonMeta::Default = std::nullopt;
-TJoinGroupRequestData::TJoinGroupRequestData()
+TJoinGroupRequestData::TJoinGroupRequestData()
: GroupId(GroupIdMeta::Default)
, SessionTimeoutMs(SessionTimeoutMsMeta::Default)
, RebalanceTimeoutMs(RebalanceTimeoutMsMeta::Default)
@@ -3778,7 +3812,7 @@ void TJoinGroupRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _versi
NPrivate::Read<ProtocolTypeMeta>(_readable, _version, ProtocolType);
NPrivate::Read<ProtocolsMeta>(_readable, _version, Protocols);
NPrivate::Read<ReasonMeta>(_readable, _version, Reason);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3806,10 +3840,10 @@ void TJoinGroupRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _vers
NPrivate::Write<ProtocolTypeMeta>(_collector, _writable, _version, ProtocolType);
NPrivate::Write<ProtocolsMeta>(_collector, _writable, _version, Protocols);
NPrivate::Write<ReasonMeta>(_collector, _writable, _version, Reason);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3823,7 +3857,7 @@ i32 TJoinGroupRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<ProtocolTypeMeta>(_collector, _version, ProtocolType);
NPrivate::Size<ProtocolsMeta>(_collector, _version, Protocols);
NPrivate::Size<ReasonMeta>(_collector, _version, Reason);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3836,7 +3870,7 @@ i32 TJoinGroupRequestData::Size(TKafkaVersion _version) const {
//
const TJoinGroupRequestData::TJoinGroupRequestProtocol::NameMeta::Type TJoinGroupRequestData::TJoinGroupRequestProtocol::NameMeta::Default = {""};
-TJoinGroupRequestData::TJoinGroupRequestProtocol::TJoinGroupRequestProtocol()
+TJoinGroupRequestData::TJoinGroupRequestProtocol::TJoinGroupRequestProtocol()
: Name(NameMeta::Default)
{}
@@ -3846,7 +3880,7 @@ void TJoinGroupRequestData::TJoinGroupRequestProtocol::Read(TKafkaReadable& _rea
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<MetadataMeta>(_readable, _version, Metadata);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3868,10 +3902,10 @@ void TJoinGroupRequestData::TJoinGroupRequestProtocol::Write(TKafkaWritable& _wr
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<MetadataMeta>(_collector, _writable, _version, Metadata);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3879,7 +3913,7 @@ i32 TJoinGroupRequestData::TJoinGroupRequestProtocol::Size(TKafkaVersion _versio
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<MetadataMeta>(_collector, _version, Metadata);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3899,7 +3933,7 @@ const TJoinGroupResponseData::LeaderMeta::Type TJoinGroupResponseData::LeaderMet
const TJoinGroupResponseData::SkipAssignmentMeta::Type TJoinGroupResponseData::SkipAssignmentMeta::Default = false;
const TJoinGroupResponseData::MemberIdMeta::Type TJoinGroupResponseData::MemberIdMeta::Default = {""};
-TJoinGroupResponseData::TJoinGroupResponseData()
+TJoinGroupResponseData::TJoinGroupResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
, GenerationId(GenerationIdMeta::Default)
@@ -3923,7 +3957,7 @@ void TJoinGroupResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _vers
NPrivate::Read<SkipAssignmentMeta>(_readable, _version, SkipAssignment);
NPrivate::Read<MemberIdMeta>(_readable, _version, MemberId);
NPrivate::Read<MembersMeta>(_readable, _version, Members);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -3952,10 +3986,10 @@ void TJoinGroupResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _ver
NPrivate::Write<SkipAssignmentMeta>(_collector, _writable, _version, SkipAssignment);
NPrivate::Write<MemberIdMeta>(_collector, _writable, _version, MemberId);
NPrivate::Write<MembersMeta>(_collector, _writable, _version, Members);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -3970,7 +4004,7 @@ i32 TJoinGroupResponseData::Size(TKafkaVersion _version) const {
NPrivate::Size<SkipAssignmentMeta>(_collector, _version, SkipAssignment);
NPrivate::Size<MemberIdMeta>(_collector, _version, MemberId);
NPrivate::Size<MembersMeta>(_collector, _version, Members);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -3984,7 +4018,7 @@ i32 TJoinGroupResponseData::Size(TKafkaVersion _version) const {
const TJoinGroupResponseData::TJoinGroupResponseMember::MemberIdMeta::Type TJoinGroupResponseData::TJoinGroupResponseMember::MemberIdMeta::Default = {""};
const TJoinGroupResponseData::TJoinGroupResponseMember::GroupInstanceIdMeta::Type TJoinGroupResponseData::TJoinGroupResponseMember::GroupInstanceIdMeta::Default = std::nullopt;
-TJoinGroupResponseData::TJoinGroupResponseMember::TJoinGroupResponseMember()
+TJoinGroupResponseData::TJoinGroupResponseMember::TJoinGroupResponseMember()
: MemberId(MemberIdMeta::Default)
, GroupInstanceId(GroupInstanceIdMeta::Default)
{}
@@ -3996,7 +4030,7 @@ void TJoinGroupResponseData::TJoinGroupResponseMember::Read(TKafkaReadable& _rea
NPrivate::Read<MemberIdMeta>(_readable, _version, MemberId);
NPrivate::Read<GroupInstanceIdMeta>(_readable, _version, GroupInstanceId);
NPrivate::Read<MetadataMeta>(_readable, _version, Metadata);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4019,10 +4053,10 @@ void TJoinGroupResponseData::TJoinGroupResponseMember::Write(TKafkaWritable& _wr
NPrivate::Write<MemberIdMeta>(_collector, _writable, _version, MemberId);
NPrivate::Write<GroupInstanceIdMeta>(_collector, _writable, _version, GroupInstanceId);
NPrivate::Write<MetadataMeta>(_collector, _writable, _version, Metadata);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -4031,7 +4065,7 @@ i32 TJoinGroupResponseData::TJoinGroupResponseMember::Size(TKafkaVersion _versio
NPrivate::Size<MemberIdMeta>(_collector, _version, MemberId);
NPrivate::Size<GroupInstanceIdMeta>(_collector, _version, GroupInstanceId);
NPrivate::Size<MetadataMeta>(_collector, _version, Metadata);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4047,7 +4081,7 @@ const THeartbeatRequestData::GenerationIdMeta::Type THeartbeatRequestData::Gener
const THeartbeatRequestData::MemberIdMeta::Type THeartbeatRequestData::MemberIdMeta::Default = {""};
const THeartbeatRequestData::GroupInstanceIdMeta::Type THeartbeatRequestData::GroupInstanceIdMeta::Default = std::nullopt;
-THeartbeatRequestData::THeartbeatRequestData()
+THeartbeatRequestData::THeartbeatRequestData()
: GroupId(GroupIdMeta::Default)
, GenerationId(GenerationIdMeta::Default)
, MemberId(MemberIdMeta::Default)
@@ -4062,7 +4096,7 @@ void THeartbeatRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _versi
NPrivate::Read<GenerationIdMeta>(_readable, _version, GenerationId);
NPrivate::Read<MemberIdMeta>(_readable, _version, MemberId);
NPrivate::Read<GroupInstanceIdMeta>(_readable, _version, GroupInstanceId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4086,10 +4120,10 @@ void THeartbeatRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _vers
NPrivate::Write<GenerationIdMeta>(_collector, _writable, _version, GenerationId);
NPrivate::Write<MemberIdMeta>(_collector, _writable, _version, MemberId);
NPrivate::Write<GroupInstanceIdMeta>(_collector, _writable, _version, GroupInstanceId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -4099,7 +4133,7 @@ i32 THeartbeatRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<GenerationIdMeta>(_collector, _version, GenerationId);
NPrivate::Size<MemberIdMeta>(_collector, _version, MemberId);
NPrivate::Size<GroupInstanceIdMeta>(_collector, _version, GroupInstanceId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4113,7 +4147,7 @@ i32 THeartbeatRequestData::Size(TKafkaVersion _version) const {
const THeartbeatResponseData::ThrottleTimeMsMeta::Type THeartbeatResponseData::ThrottleTimeMsMeta::Default = 0;
const THeartbeatResponseData::ErrorCodeMeta::Type THeartbeatResponseData::ErrorCodeMeta::Default = 0;
-THeartbeatResponseData::THeartbeatResponseData()
+THeartbeatResponseData::THeartbeatResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
{}
@@ -4124,7 +4158,7 @@ void THeartbeatResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _vers
}
NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4146,10 +4180,10 @@ void THeartbeatResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _ver
NPrivate::TWriteCollector _collector;
NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -4157,7 +4191,7 @@ i32 THeartbeatResponseData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4171,7 +4205,7 @@ i32 THeartbeatResponseData::Size(TKafkaVersion _version) const {
const TLeaveGroupRequestData::GroupIdMeta::Type TLeaveGroupRequestData::GroupIdMeta::Default = {""};
const TLeaveGroupRequestData::MemberIdMeta::Type TLeaveGroupRequestData::MemberIdMeta::Default = {""};
-TLeaveGroupRequestData::TLeaveGroupRequestData()
+TLeaveGroupRequestData::TLeaveGroupRequestData()
: GroupId(GroupIdMeta::Default)
, MemberId(MemberIdMeta::Default)
{}
@@ -4183,7 +4217,7 @@ void TLeaveGroupRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _vers
NPrivate::Read<GroupIdMeta>(_readable, _version, GroupId);
NPrivate::Read<MemberIdMeta>(_readable, _version, MemberId);
NPrivate::Read<MembersMeta>(_readable, _version, Members);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4206,10 +4240,10 @@ void TLeaveGroupRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _ver
NPrivate::Write<GroupIdMeta>(_collector, _writable, _version, GroupId);
NPrivate::Write<MemberIdMeta>(_collector, _writable, _version, MemberId);
NPrivate::Write<MembersMeta>(_collector, _writable, _version, Members);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -4218,7 +4252,7 @@ i32 TLeaveGroupRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<GroupIdMeta>(_collector, _version, GroupId);
NPrivate::Size<MemberIdMeta>(_collector, _version, MemberId);
NPrivate::Size<MembersMeta>(_collector, _version, Members);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4233,7 +4267,7 @@ const TLeaveGroupRequestData::TMemberIdentity::MemberIdMeta::Type TLeaveGroupReq
const TLeaveGroupRequestData::TMemberIdentity::GroupInstanceIdMeta::Type TLeaveGroupRequestData::TMemberIdentity::GroupInstanceIdMeta::Default = std::nullopt;
const TLeaveGroupRequestData::TMemberIdentity::ReasonMeta::Type TLeaveGroupRequestData::TMemberIdentity::ReasonMeta::Default = std::nullopt;
-TLeaveGroupRequestData::TMemberIdentity::TMemberIdentity()
+TLeaveGroupRequestData::TMemberIdentity::TMemberIdentity()
: MemberId(MemberIdMeta::Default)
, GroupInstanceId(GroupInstanceIdMeta::Default)
, Reason(ReasonMeta::Default)
@@ -4246,7 +4280,7 @@ void TLeaveGroupRequestData::TMemberIdentity::Read(TKafkaReadable& _readable, TK
NPrivate::Read<MemberIdMeta>(_readable, _version, MemberId);
NPrivate::Read<GroupInstanceIdMeta>(_readable, _version, GroupInstanceId);
NPrivate::Read<ReasonMeta>(_readable, _version, Reason);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4269,10 +4303,10 @@ void TLeaveGroupRequestData::TMemberIdentity::Write(TKafkaWritable& _writable, T
NPrivate::Write<MemberIdMeta>(_collector, _writable, _version, MemberId);
NPrivate::Write<GroupInstanceIdMeta>(_collector, _writable, _version, GroupInstanceId);
NPrivate::Write<ReasonMeta>(_collector, _writable, _version, Reason);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -4281,7 +4315,7 @@ i32 TLeaveGroupRequestData::TMemberIdentity::Size(TKafkaVersion _version) const
NPrivate::Size<MemberIdMeta>(_collector, _version, MemberId);
NPrivate::Size<GroupInstanceIdMeta>(_collector, _version, GroupInstanceId);
NPrivate::Size<ReasonMeta>(_collector, _version, Reason);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4295,7 +4329,7 @@ i32 TLeaveGroupRequestData::TMemberIdentity::Size(TKafkaVersion _version) const
const TLeaveGroupResponseData::ThrottleTimeMsMeta::Type TLeaveGroupResponseData::ThrottleTimeMsMeta::Default = 0;
const TLeaveGroupResponseData::ErrorCodeMeta::Type TLeaveGroupResponseData::ErrorCodeMeta::Default = 0;
-TLeaveGroupResponseData::TLeaveGroupResponseData()
+TLeaveGroupResponseData::TLeaveGroupResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
{}
@@ -4307,7 +4341,7 @@ void TLeaveGroupResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _ver
NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
NPrivate::Read<MembersMeta>(_readable, _version, Members);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4330,10 +4364,10 @@ void TLeaveGroupResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _ve
NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
NPrivate::Write<MembersMeta>(_collector, _writable, _version, Members);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -4342,7 +4376,7 @@ i32 TLeaveGroupResponseData::Size(TKafkaVersion _version) const {
NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
NPrivate::Size<MembersMeta>(_collector, _version, Members);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4357,7 +4391,7 @@ const TLeaveGroupResponseData::TMemberResponse::MemberIdMeta::Type TLeaveGroupRe
const TLeaveGroupResponseData::TMemberResponse::GroupInstanceIdMeta::Type TLeaveGroupResponseData::TMemberResponse::GroupInstanceIdMeta::Default = {""};
const TLeaveGroupResponseData::TMemberResponse::ErrorCodeMeta::Type TLeaveGroupResponseData::TMemberResponse::ErrorCodeMeta::Default = 0;
-TLeaveGroupResponseData::TMemberResponse::TMemberResponse()
+TLeaveGroupResponseData::TMemberResponse::TMemberResponse()
: MemberId(MemberIdMeta::Default)
, GroupInstanceId(GroupInstanceIdMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
@@ -4370,7 +4404,7 @@ void TLeaveGroupResponseData::TMemberResponse::Read(TKafkaReadable& _readable, T
NPrivate::Read<MemberIdMeta>(_readable, _version, MemberId);
NPrivate::Read<GroupInstanceIdMeta>(_readable, _version, GroupInstanceId);
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4393,10 +4427,10 @@ void TLeaveGroupResponseData::TMemberResponse::Write(TKafkaWritable& _writable,
NPrivate::Write<MemberIdMeta>(_collector, _writable, _version, MemberId);
NPrivate::Write<GroupInstanceIdMeta>(_collector, _writable, _version, GroupInstanceId);
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -4405,7 +4439,7 @@ i32 TLeaveGroupResponseData::TMemberResponse::Size(TKafkaVersion _version) const
NPrivate::Size<MemberIdMeta>(_collector, _version, MemberId);
NPrivate::Size<GroupInstanceIdMeta>(_collector, _version, GroupInstanceId);
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4423,7 +4457,7 @@ const TSyncGroupRequestData::GroupInstanceIdMeta::Type TSyncGroupRequestData::Gr
const TSyncGroupRequestData::ProtocolTypeMeta::Type TSyncGroupRequestData::ProtocolTypeMeta::Default = std::nullopt;
const TSyncGroupRequestData::ProtocolNameMeta::Type TSyncGroupRequestData::ProtocolNameMeta::Default = std::nullopt;
-TSyncGroupRequestData::TSyncGroupRequestData()
+TSyncGroupRequestData::TSyncGroupRequestData()
: GroupId(GroupIdMeta::Default)
, GenerationId(GenerationIdMeta::Default)
, MemberId(MemberIdMeta::Default)
@@ -4443,7 +4477,7 @@ void TSyncGroupRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _versi
NPrivate::Read<ProtocolTypeMeta>(_readable, _version, ProtocolType);
NPrivate::Read<ProtocolNameMeta>(_readable, _version, ProtocolName);
NPrivate::Read<AssignmentsMeta>(_readable, _version, Assignments);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4470,10 +4504,10 @@ void TSyncGroupRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _vers
NPrivate::Write<ProtocolTypeMeta>(_collector, _writable, _version, ProtocolType);
NPrivate::Write<ProtocolNameMeta>(_collector, _writable, _version, ProtocolName);
NPrivate::Write<AssignmentsMeta>(_collector, _writable, _version, Assignments);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -4486,7 +4520,7 @@ i32 TSyncGroupRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<ProtocolTypeMeta>(_collector, _version, ProtocolType);
NPrivate::Size<ProtocolNameMeta>(_collector, _version, ProtocolName);
NPrivate::Size<AssignmentsMeta>(_collector, _version, Assignments);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4499,7 +4533,7 @@ i32 TSyncGroupRequestData::Size(TKafkaVersion _version) const {
//
const TSyncGroupRequestData::TSyncGroupRequestAssignment::MemberIdMeta::Type TSyncGroupRequestData::TSyncGroupRequestAssignment::MemberIdMeta::Default = {""};
-TSyncGroupRequestData::TSyncGroupRequestAssignment::TSyncGroupRequestAssignment()
+TSyncGroupRequestData::TSyncGroupRequestAssignment::TSyncGroupRequestAssignment()
: MemberId(MemberIdMeta::Default)
{}
@@ -4509,7 +4543,7 @@ void TSyncGroupRequestData::TSyncGroupRequestAssignment::Read(TKafkaReadable& _r
}
NPrivate::Read<MemberIdMeta>(_readable, _version, MemberId);
NPrivate::Read<AssignmentMeta>(_readable, _version, Assignment);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4531,10 +4565,10 @@ void TSyncGroupRequestData::TSyncGroupRequestAssignment::Write(TKafkaWritable& _
NPrivate::TWriteCollector _collector;
NPrivate::Write<MemberIdMeta>(_collector, _writable, _version, MemberId);
NPrivate::Write<AssignmentMeta>(_collector, _writable, _version, Assignment);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -4542,7 +4576,7 @@ i32 TSyncGroupRequestData::TSyncGroupRequestAssignment::Size(TKafkaVersion _vers
NPrivate::TSizeCollector _collector;
NPrivate::Size<MemberIdMeta>(_collector, _version, MemberId);
NPrivate::Size<AssignmentMeta>(_collector, _version, Assignment);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4558,7 +4592,7 @@ const TSyncGroupResponseData::ErrorCodeMeta::Type TSyncGroupResponseData::ErrorC
const TSyncGroupResponseData::ProtocolTypeMeta::Type TSyncGroupResponseData::ProtocolTypeMeta::Default = std::nullopt;
const TSyncGroupResponseData::ProtocolNameMeta::Type TSyncGroupResponseData::ProtocolNameMeta::Default = std::nullopt;
-TSyncGroupResponseData::TSyncGroupResponseData()
+TSyncGroupResponseData::TSyncGroupResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
, ProtocolType(ProtocolTypeMeta::Default)
@@ -4574,7 +4608,7 @@ void TSyncGroupResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _vers
NPrivate::Read<ProtocolTypeMeta>(_readable, _version, ProtocolType);
NPrivate::Read<ProtocolNameMeta>(_readable, _version, ProtocolName);
NPrivate::Read<AssignmentMeta>(_readable, _version, Assignment);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4599,10 +4633,10 @@ void TSyncGroupResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _ver
NPrivate::Write<ProtocolTypeMeta>(_collector, _writable, _version, ProtocolType);
NPrivate::Write<ProtocolNameMeta>(_collector, _writable, _version, ProtocolName);
NPrivate::Write<AssignmentMeta>(_collector, _writable, _version, Assignment);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -4613,7 +4647,449 @@ i32 TSyncGroupResponseData::Size(TKafkaVersion _version) const {
NPrivate::Size<ProtocolTypeMeta>(_collector, _version, ProtocolType);
NPrivate::Size<ProtocolNameMeta>(_collector, _version, ProtocolName);
NPrivate::Size<AssignmentMeta>(_collector, _version, Assignment);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
+ }
+ return _collector.Size;
+}
+
+
+//
+// TDescribeGroupsRequestData
+//
+const TDescribeGroupsRequestData::IncludeAuthorizedOperationsMeta::Type TDescribeGroupsRequestData::IncludeAuthorizedOperationsMeta::Default = false;
+
+TDescribeGroupsRequestData::TDescribeGroupsRequestData()
+ : IncludeAuthorizedOperations(IncludeAuthorizedOperationsMeta::Default)
+{}
+
+void TDescribeGroupsRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _version) {
+ if (!NPrivate::VersionCheck<MessageMeta::PresentVersions.Min, MessageMeta::PresentVersions.Max>(_version)) {
+ ythrow yexception() << "Can't read version " << _version << " of TDescribeGroupsRequestData";
+ }
+ NPrivate::Read<GroupsMeta>(_readable, _version, Groups);
+ NPrivate::Read<IncludeAuthorizedOperationsMeta>(_readable, _version, IncludeAuthorizedOperations);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
+ for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
+ ui32 _tag = _readable.readUnsignedVarint<ui32>();
+ ui32 _size = _readable.readUnsignedVarint<ui32>();
+ switch (_tag) {
+ default:
+ _readable.skip(_size); // skip unknown tag
+ break;
+ }
+ }
+ }
+}
+
+void TDescribeGroupsRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _version) const {
+ if (!NPrivate::VersionCheck<MessageMeta::PresentVersions.Min, MessageMeta::PresentVersions.Max>(_version)) {
+ ythrow yexception() << "Can't write version " << _version << " of TDescribeGroupsRequestData";
+ }
+ NPrivate::TWriteCollector _collector;
+ NPrivate::Write<GroupsMeta>(_collector, _writable, _version, Groups);
+ NPrivate::Write<IncludeAuthorizedOperationsMeta>(_collector, _writable, _version, IncludeAuthorizedOperations);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ _writable.writeUnsignedVarint(_collector.NumTaggedFields);
+
+ }
+}
+
+i32 TDescribeGroupsRequestData::Size(TKafkaVersion _version) const {
+ NPrivate::TSizeCollector _collector;
+ NPrivate::Size<GroupsMeta>(_collector, _version, Groups);
+ NPrivate::Size<IncludeAuthorizedOperationsMeta>(_collector, _version, IncludeAuthorizedOperations);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
+ }
+ return _collector.Size;
+}
+
+
+//
+// TDescribeGroupsResponseData
+//
+const TDescribeGroupsResponseData::ThrottleTimeMsMeta::Type TDescribeGroupsResponseData::ThrottleTimeMsMeta::Default = 0;
+
+TDescribeGroupsResponseData::TDescribeGroupsResponseData()
+ : ThrottleTimeMs(ThrottleTimeMsMeta::Default)
+{}
+
+void TDescribeGroupsResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _version) {
+ if (!NPrivate::VersionCheck<MessageMeta::PresentVersions.Min, MessageMeta::PresentVersions.Max>(_version)) {
+ ythrow yexception() << "Can't read version " << _version << " of TDescribeGroupsResponseData";
+ }
+ NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
+ NPrivate::Read<GroupsMeta>(_readable, _version, Groups);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
+ for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
+ ui32 _tag = _readable.readUnsignedVarint<ui32>();
+ ui32 _size = _readable.readUnsignedVarint<ui32>();
+ switch (_tag) {
+ default:
+ _readable.skip(_size); // skip unknown tag
+ break;
+ }
+ }
+ }
+}
+void TDescribeGroupsResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _version) const {
+ if (!NPrivate::VersionCheck<MessageMeta::PresentVersions.Min, MessageMeta::PresentVersions.Max>(_version)) {
+ ythrow yexception() << "Can't write version " << _version << " of TDescribeGroupsResponseData";
+ }
+ NPrivate::TWriteCollector _collector;
+ NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
+ NPrivate::Write<GroupsMeta>(_collector, _writable, _version, Groups);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ _writable.writeUnsignedVarint(_collector.NumTaggedFields);
+
+ }
+}
+
+i32 TDescribeGroupsResponseData::Size(TKafkaVersion _version) const {
+ NPrivate::TSizeCollector _collector;
+ NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
+ NPrivate::Size<GroupsMeta>(_collector, _version, Groups);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
+ }
+ return _collector.Size;
+}
+
+
+//
+// TDescribeGroupsResponseData::TDescribedGroup
+//
+const TDescribeGroupsResponseData::TDescribedGroup::ErrorCodeMeta::Type TDescribeGroupsResponseData::TDescribedGroup::ErrorCodeMeta::Default = 0;
+const TDescribeGroupsResponseData::TDescribedGroup::GroupIdMeta::Type TDescribeGroupsResponseData::TDescribedGroup::GroupIdMeta::Default = {""};
+const TDescribeGroupsResponseData::TDescribedGroup::GroupStateMeta::Type TDescribeGroupsResponseData::TDescribedGroup::GroupStateMeta::Default = {""};
+const TDescribeGroupsResponseData::TDescribedGroup::ProtocolTypeMeta::Type TDescribeGroupsResponseData::TDescribedGroup::ProtocolTypeMeta::Default = {""};
+const TDescribeGroupsResponseData::TDescribedGroup::ProtocolDataMeta::Type TDescribeGroupsResponseData::TDescribedGroup::ProtocolDataMeta::Default = {""};
+const TDescribeGroupsResponseData::TDescribedGroup::AuthorizedOperationsMeta::Type TDescribeGroupsResponseData::TDescribedGroup::AuthorizedOperationsMeta::Default = -2147483648;
+
+TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroup()
+ : ErrorCode(ErrorCodeMeta::Default)
+ , GroupId(GroupIdMeta::Default)
+ , GroupState(GroupStateMeta::Default)
+ , ProtocolType(ProtocolTypeMeta::Default)
+ , ProtocolData(ProtocolDataMeta::Default)
+ , AuthorizedOperations(AuthorizedOperationsMeta::Default)
+{}
+
+void TDescribeGroupsResponseData::TDescribedGroup::Read(TKafkaReadable& _readable, TKafkaVersion _version) {
+ if (!NPrivate::VersionCheck<MessageMeta::PresentVersions.Min, MessageMeta::PresentVersions.Max>(_version)) {
+ ythrow yexception() << "Can't read version " << _version << " of TDescribeGroupsResponseData::TDescribedGroup";
+ }
+ NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
+ NPrivate::Read<GroupIdMeta>(_readable, _version, GroupId);
+ NPrivate::Read<GroupStateMeta>(_readable, _version, GroupState);
+ NPrivate::Read<ProtocolTypeMeta>(_readable, _version, ProtocolType);
+ NPrivate::Read<ProtocolDataMeta>(_readable, _version, ProtocolData);
+ NPrivate::Read<MembersMeta>(_readable, _version, Members);
+ NPrivate::Read<AuthorizedOperationsMeta>(_readable, _version, AuthorizedOperations);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
+ for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
+ ui32 _tag = _readable.readUnsignedVarint<ui32>();
+ ui32 _size = _readable.readUnsignedVarint<ui32>();
+ switch (_tag) {
+ default:
+ _readable.skip(_size); // skip unknown tag
+ break;
+ }
+ }
+ }
+}
+
+void TDescribeGroupsResponseData::TDescribedGroup::Write(TKafkaWritable& _writable, TKafkaVersion _version) const {
+ if (!NPrivate::VersionCheck<MessageMeta::PresentVersions.Min, MessageMeta::PresentVersions.Max>(_version)) {
+ ythrow yexception() << "Can't write version " << _version << " of TDescribeGroupsResponseData::TDescribedGroup";
+ }
+ NPrivate::TWriteCollector _collector;
+ NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
+ NPrivate::Write<GroupIdMeta>(_collector, _writable, _version, GroupId);
+ NPrivate::Write<GroupStateMeta>(_collector, _writable, _version, GroupState);
+ NPrivate::Write<ProtocolTypeMeta>(_collector, _writable, _version, ProtocolType);
+ NPrivate::Write<ProtocolDataMeta>(_collector, _writable, _version, ProtocolData);
+ NPrivate::Write<MembersMeta>(_collector, _writable, _version, Members);
+ NPrivate::Write<AuthorizedOperationsMeta>(_collector, _writable, _version, AuthorizedOperations);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ _writable.writeUnsignedVarint(_collector.NumTaggedFields);
+
+ }
+}
+
+i32 TDescribeGroupsResponseData::TDescribedGroup::Size(TKafkaVersion _version) const {
+ NPrivate::TSizeCollector _collector;
+ NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
+ NPrivate::Size<GroupIdMeta>(_collector, _version, GroupId);
+ NPrivate::Size<GroupStateMeta>(_collector, _version, GroupState);
+ NPrivate::Size<ProtocolTypeMeta>(_collector, _version, ProtocolType);
+ NPrivate::Size<ProtocolDataMeta>(_collector, _version, ProtocolData);
+ NPrivate::Size<MembersMeta>(_collector, _version, Members);
+ NPrivate::Size<AuthorizedOperationsMeta>(_collector, _version, AuthorizedOperations);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
+ }
+ return _collector.Size;
+}
+
+
+//
+// TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember
+//
+const TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember::MemberIdMeta::Type TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember::MemberIdMeta::Default = {""};
+const TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember::GroupInstanceIdMeta::Type TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember::GroupInstanceIdMeta::Default = std::nullopt;
+const TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember::ClientIdMeta::Type TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember::ClientIdMeta::Default = {""};
+const TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember::ClientHostMeta::Type TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember::ClientHostMeta::Default = {""};
+
+TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember::TDescribedGroupMember()
+ : MemberId(MemberIdMeta::Default)
+ , GroupInstanceId(GroupInstanceIdMeta::Default)
+ , ClientId(ClientIdMeta::Default)
+ , ClientHost(ClientHostMeta::Default)
+{}
+
+void TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember::Read(TKafkaReadable& _readable, TKafkaVersion _version) {
+ if (!NPrivate::VersionCheck<MessageMeta::PresentVersions.Min, MessageMeta::PresentVersions.Max>(_version)) {
+ ythrow yexception() << "Can't read version " << _version << " of TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember";
+ }
+ NPrivate::Read<MemberIdMeta>(_readable, _version, MemberId);
+ NPrivate::Read<GroupInstanceIdMeta>(_readable, _version, GroupInstanceId);
+ NPrivate::Read<ClientIdMeta>(_readable, _version, ClientId);
+ NPrivate::Read<ClientHostMeta>(_readable, _version, ClientHost);
+ NPrivate::Read<MemberMetadataMeta>(_readable, _version, MemberMetadata);
+ NPrivate::Read<MemberAssignmentMeta>(_readable, _version, MemberAssignment);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
+ for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
+ ui32 _tag = _readable.readUnsignedVarint<ui32>();
+ ui32 _size = _readable.readUnsignedVarint<ui32>();
+ switch (_tag) {
+ default:
+ _readable.skip(_size); // skip unknown tag
+ break;
+ }
+ }
+ }
+}
+
+void TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember::Write(TKafkaWritable& _writable, TKafkaVersion _version) const {
+ if (!NPrivate::VersionCheck<MessageMeta::PresentVersions.Min, MessageMeta::PresentVersions.Max>(_version)) {
+ ythrow yexception() << "Can't write version " << _version << " of TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember";
+ }
+ NPrivate::TWriteCollector _collector;
+ NPrivate::Write<MemberIdMeta>(_collector, _writable, _version, MemberId);
+ NPrivate::Write<GroupInstanceIdMeta>(_collector, _writable, _version, GroupInstanceId);
+ NPrivate::Write<ClientIdMeta>(_collector, _writable, _version, ClientId);
+ NPrivate::Write<ClientHostMeta>(_collector, _writable, _version, ClientHost);
+ NPrivate::Write<MemberMetadataMeta>(_collector, _writable, _version, MemberMetadata);
+ NPrivate::Write<MemberAssignmentMeta>(_collector, _writable, _version, MemberAssignment);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ _writable.writeUnsignedVarint(_collector.NumTaggedFields);
+
+ }
+}
+
+i32 TDescribeGroupsResponseData::TDescribedGroup::TDescribedGroupMember::Size(TKafkaVersion _version) const {
+ NPrivate::TSizeCollector _collector;
+ NPrivate::Size<MemberIdMeta>(_collector, _version, MemberId);
+ NPrivate::Size<GroupInstanceIdMeta>(_collector, _version, GroupInstanceId);
+ NPrivate::Size<ClientIdMeta>(_collector, _version, ClientId);
+ NPrivate::Size<ClientHostMeta>(_collector, _version, ClientHost);
+ NPrivate::Size<MemberMetadataMeta>(_collector, _version, MemberMetadata);
+ NPrivate::Size<MemberAssignmentMeta>(_collector, _version, MemberAssignment);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
+ }
+ return _collector.Size;
+}
+
+
+//
+// TListGroupsRequestData
+//
+
+TListGroupsRequestData::TListGroupsRequestData()
+{}
+
+void TListGroupsRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _version) {
+ if (!NPrivate::VersionCheck<MessageMeta::PresentVersions.Min, MessageMeta::PresentVersions.Max>(_version)) {
+ ythrow yexception() << "Can't read version " << _version << " of TListGroupsRequestData";
+ }
+ NPrivate::Read<StatesFilterMeta>(_readable, _version, StatesFilter);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
+ for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
+ ui32 _tag = _readable.readUnsignedVarint<ui32>();
+ ui32 _size = _readable.readUnsignedVarint<ui32>();
+ switch (_tag) {
+ default:
+ _readable.skip(_size); // skip unknown tag
+ break;
+ }
+ }
+ }
+}
+
+void TListGroupsRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _version) const {
+ if (!NPrivate::VersionCheck<MessageMeta::PresentVersions.Min, MessageMeta::PresentVersions.Max>(_version)) {
+ ythrow yexception() << "Can't write version " << _version << " of TListGroupsRequestData";
+ }
+ NPrivate::TWriteCollector _collector;
+ NPrivate::Write<StatesFilterMeta>(_collector, _writable, _version, StatesFilter);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ _writable.writeUnsignedVarint(_collector.NumTaggedFields);
+
+ }
+}
+
+i32 TListGroupsRequestData::Size(TKafkaVersion _version) const {
+ NPrivate::TSizeCollector _collector;
+ NPrivate::Size<StatesFilterMeta>(_collector, _version, StatesFilter);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
+ }
+ return _collector.Size;
+}
+
+
+//
+// TListGroupsResponseData
+//
+const TListGroupsResponseData::ThrottleTimeMsMeta::Type TListGroupsResponseData::ThrottleTimeMsMeta::Default = 0;
+const TListGroupsResponseData::ErrorCodeMeta::Type TListGroupsResponseData::ErrorCodeMeta::Default = 0;
+
+TListGroupsResponseData::TListGroupsResponseData()
+ : ThrottleTimeMs(ThrottleTimeMsMeta::Default)
+ , ErrorCode(ErrorCodeMeta::Default)
+{}
+
+void TListGroupsResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _version) {
+ if (!NPrivate::VersionCheck<MessageMeta::PresentVersions.Min, MessageMeta::PresentVersions.Max>(_version)) {
+ ythrow yexception() << "Can't read version " << _version << " of TListGroupsResponseData";
+ }
+ NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
+ NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
+ NPrivate::Read<GroupsMeta>(_readable, _version, Groups);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
+ for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
+ ui32 _tag = _readable.readUnsignedVarint<ui32>();
+ ui32 _size = _readable.readUnsignedVarint<ui32>();
+ switch (_tag) {
+ default:
+ _readable.skip(_size); // skip unknown tag
+ break;
+ }
+ }
+ }
+}
+
+void TListGroupsResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _version) const {
+ if (!NPrivate::VersionCheck<MessageMeta::PresentVersions.Min, MessageMeta::PresentVersions.Max>(_version)) {
+ ythrow yexception() << "Can't write version " << _version << " of TListGroupsResponseData";
+ }
+ NPrivate::TWriteCollector _collector;
+ NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
+ NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
+ NPrivate::Write<GroupsMeta>(_collector, _writable, _version, Groups);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ _writable.writeUnsignedVarint(_collector.NumTaggedFields);
+
+ }
+}
+
+i32 TListGroupsResponseData::Size(TKafkaVersion _version) const {
+ NPrivate::TSizeCollector _collector;
+ NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
+ NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
+ NPrivate::Size<GroupsMeta>(_collector, _version, Groups);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
+ }
+ return _collector.Size;
+}
+
+
+//
+// TListGroupsResponseData::TListedGroup
+//
+const TListGroupsResponseData::TListedGroup::GroupIdMeta::Type TListGroupsResponseData::TListedGroup::GroupIdMeta::Default = {""};
+const TListGroupsResponseData::TListedGroup::ProtocolTypeMeta::Type TListGroupsResponseData::TListedGroup::ProtocolTypeMeta::Default = {""};
+const TListGroupsResponseData::TListedGroup::GroupStateMeta::Type TListGroupsResponseData::TListedGroup::GroupStateMeta::Default = {""};
+
+TListGroupsResponseData::TListedGroup::TListedGroup()
+ : GroupId(GroupIdMeta::Default)
+ , ProtocolType(ProtocolTypeMeta::Default)
+ , GroupState(GroupStateMeta::Default)
+{}
+
+void TListGroupsResponseData::TListedGroup::Read(TKafkaReadable& _readable, TKafkaVersion _version) {
+ if (!NPrivate::VersionCheck<MessageMeta::PresentVersions.Min, MessageMeta::PresentVersions.Max>(_version)) {
+ ythrow yexception() << "Can't read version " << _version << " of TListGroupsResponseData::TListedGroup";
+ }
+ NPrivate::Read<GroupIdMeta>(_readable, _version, GroupId);
+ NPrivate::Read<ProtocolTypeMeta>(_readable, _version, ProtocolType);
+ NPrivate::Read<GroupStateMeta>(_readable, _version, GroupState);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
+ for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
+ ui32 _tag = _readable.readUnsignedVarint<ui32>();
+ ui32 _size = _readable.readUnsignedVarint<ui32>();
+ switch (_tag) {
+ default:
+ _readable.skip(_size); // skip unknown tag
+ break;
+ }
+ }
+ }
+}
+
+void TListGroupsResponseData::TListedGroup::Write(TKafkaWritable& _writable, TKafkaVersion _version) const {
+ if (!NPrivate::VersionCheck<MessageMeta::PresentVersions.Min, MessageMeta::PresentVersions.Max>(_version)) {
+ ythrow yexception() << "Can't write version " << _version << " of TListGroupsResponseData::TListedGroup";
+ }
+ NPrivate::TWriteCollector _collector;
+ NPrivate::Write<GroupIdMeta>(_collector, _writable, _version, GroupId);
+ NPrivate::Write<ProtocolTypeMeta>(_collector, _writable, _version, ProtocolType);
+ NPrivate::Write<GroupStateMeta>(_collector, _writable, _version, GroupState);
+
+ if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
+ _writable.writeUnsignedVarint(_collector.NumTaggedFields);
+
+ }
+}
+
+i32 TListGroupsResponseData::TListedGroup::Size(TKafkaVersion _version) const {
+ NPrivate::TSizeCollector _collector;
+ NPrivate::Size<GroupIdMeta>(_collector, _version, GroupId);
+ NPrivate::Size<ProtocolTypeMeta>(_collector, _version, ProtocolType);
+ NPrivate::Size<GroupStateMeta>(_collector, _version, GroupState);
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4626,7 +5102,7 @@ i32 TSyncGroupResponseData::Size(TKafkaVersion _version) const {
//
const TSaslHandshakeRequestData::MechanismMeta::Type TSaslHandshakeRequestData::MechanismMeta::Default = {""};
-TSaslHandshakeRequestData::TSaslHandshakeRequestData()
+TSaslHandshakeRequestData::TSaslHandshakeRequestData()
: Mechanism(MechanismMeta::Default)
{}
@@ -4635,7 +5111,7 @@ void TSaslHandshakeRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _v
ythrow yexception() << "Can't read version " << _version << " of TSaslHandshakeRequestData";
}
NPrivate::Read<MechanismMeta>(_readable, _version, Mechanism);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4656,17 +5132,17 @@ void TSaslHandshakeRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _
}
NPrivate::TWriteCollector _collector;
NPrivate::Write<MechanismMeta>(_collector, _writable, _version, Mechanism);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
i32 TSaslHandshakeRequestData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<MechanismMeta>(_collector, _version, Mechanism);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4679,7 +5155,7 @@ i32 TSaslHandshakeRequestData::Size(TKafkaVersion _version) const {
//
const TSaslHandshakeResponseData::ErrorCodeMeta::Type TSaslHandshakeResponseData::ErrorCodeMeta::Default = 0;
-TSaslHandshakeResponseData::TSaslHandshakeResponseData()
+TSaslHandshakeResponseData::TSaslHandshakeResponseData()
: ErrorCode(ErrorCodeMeta::Default)
{}
@@ -4689,7 +5165,7 @@ void TSaslHandshakeResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _
}
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
NPrivate::Read<MechanismsMeta>(_readable, _version, Mechanisms);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4711,10 +5187,10 @@ void TSaslHandshakeResponseData::Write(TKafkaWritable& _writable, TKafkaVersion
NPrivate::TWriteCollector _collector;
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
NPrivate::Write<MechanismsMeta>(_collector, _writable, _version, Mechanisms);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -4722,7 +5198,7 @@ i32 TSaslHandshakeResponseData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
NPrivate::Size<MechanismsMeta>(_collector, _version, Mechanisms);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4736,7 +5212,7 @@ i32 TSaslHandshakeResponseData::Size(TKafkaVersion _version) const {
const TApiVersionsRequestData::ClientSoftwareNameMeta::Type TApiVersionsRequestData::ClientSoftwareNameMeta::Default = {""};
const TApiVersionsRequestData::ClientSoftwareVersionMeta::Type TApiVersionsRequestData::ClientSoftwareVersionMeta::Default = {""};
-TApiVersionsRequestData::TApiVersionsRequestData()
+TApiVersionsRequestData::TApiVersionsRequestData()
: ClientSoftwareName(ClientSoftwareNameMeta::Default)
, ClientSoftwareVersion(ClientSoftwareVersionMeta::Default)
{}
@@ -4747,7 +5223,7 @@ void TApiVersionsRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _ver
}
NPrivate::Read<ClientSoftwareNameMeta>(_readable, _version, ClientSoftwareName);
NPrivate::Read<ClientSoftwareVersionMeta>(_readable, _version, ClientSoftwareVersion);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4769,10 +5245,10 @@ void TApiVersionsRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _ve
NPrivate::TWriteCollector _collector;
NPrivate::Write<ClientSoftwareNameMeta>(_collector, _writable, _version, ClientSoftwareName);
NPrivate::Write<ClientSoftwareVersionMeta>(_collector, _writable, _version, ClientSoftwareVersion);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -4780,7 +5256,7 @@ i32 TApiVersionsRequestData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ClientSoftwareNameMeta>(_collector, _version, ClientSoftwareName);
NPrivate::Size<ClientSoftwareVersionMeta>(_collector, _version, ClientSoftwareVersion);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4796,7 +5272,7 @@ const TApiVersionsResponseData::ThrottleTimeMsMeta::Type TApiVersionsResponseDat
const TApiVersionsResponseData::FinalizedFeaturesEpochMeta::Type TApiVersionsResponseData::FinalizedFeaturesEpochMeta::Default = -1;
const TApiVersionsResponseData::ZkMigrationReadyMeta::Type TApiVersionsResponseData::ZkMigrationReadyMeta::Default = false;
-TApiVersionsResponseData::TApiVersionsResponseData()
+TApiVersionsResponseData::TApiVersionsResponseData()
: ErrorCode(ErrorCodeMeta::Default)
, ThrottleTimeMs(ThrottleTimeMsMeta::Default)
, FinalizedFeaturesEpoch(FinalizedFeaturesEpochMeta::Default)
@@ -4814,7 +5290,7 @@ void TApiVersionsResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _ve
NPrivate::Read<FinalizedFeaturesEpochMeta>(_readable, _version, FinalizedFeaturesEpoch);
NPrivate::Read<FinalizedFeaturesMeta>(_readable, _version, FinalizedFeatures);
NPrivate::Read<ZkMigrationReadyMeta>(_readable, _version, ZkMigrationReady);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4853,10 +5329,10 @@ void TApiVersionsResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _v
NPrivate::Write<FinalizedFeaturesEpochMeta>(_collector, _writable, _version, FinalizedFeaturesEpoch);
NPrivate::Write<FinalizedFeaturesMeta>(_collector, _writable, _version, FinalizedFeatures);
NPrivate::Write<ZkMigrationReadyMeta>(_collector, _writable, _version, ZkMigrationReady);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
NPrivate::WriteTag<SupportedFeaturesMeta>(_writable, _version, SupportedFeatures);
NPrivate::WriteTag<FinalizedFeaturesEpochMeta>(_writable, _version, FinalizedFeaturesEpoch);
NPrivate::WriteTag<FinalizedFeaturesMeta>(_writable, _version, FinalizedFeatures);
@@ -4873,7 +5349,7 @@ i32 TApiVersionsResponseData::Size(TKafkaVersion _version) const {
NPrivate::Size<FinalizedFeaturesEpochMeta>(_collector, _version, FinalizedFeaturesEpoch);
NPrivate::Size<FinalizedFeaturesMeta>(_collector, _version, FinalizedFeatures);
NPrivate::Size<ZkMigrationReadyMeta>(_collector, _version, ZkMigrationReady);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4888,7 +5364,7 @@ const TApiVersionsResponseData::TApiVersion::ApiKeyMeta::Type TApiVersionsRespon
const TApiVersionsResponseData::TApiVersion::MinVersionMeta::Type TApiVersionsResponseData::TApiVersion::MinVersionMeta::Default = 0;
const TApiVersionsResponseData::TApiVersion::MaxVersionMeta::Type TApiVersionsResponseData::TApiVersion::MaxVersionMeta::Default = 0;
-TApiVersionsResponseData::TApiVersion::TApiVersion()
+TApiVersionsResponseData::TApiVersion::TApiVersion()
: ApiKey(ApiKeyMeta::Default)
, MinVersion(MinVersionMeta::Default)
, MaxVersion(MaxVersionMeta::Default)
@@ -4901,7 +5377,7 @@ void TApiVersionsResponseData::TApiVersion::Read(TKafkaReadable& _readable, TKaf
NPrivate::Read<ApiKeyMeta>(_readable, _version, ApiKey);
NPrivate::Read<MinVersionMeta>(_readable, _version, MinVersion);
NPrivate::Read<MaxVersionMeta>(_readable, _version, MaxVersion);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4924,10 +5400,10 @@ void TApiVersionsResponseData::TApiVersion::Write(TKafkaWritable& _writable, TKa
NPrivate::Write<ApiKeyMeta>(_collector, _writable, _version, ApiKey);
NPrivate::Write<MinVersionMeta>(_collector, _writable, _version, MinVersion);
NPrivate::Write<MaxVersionMeta>(_collector, _writable, _version, MaxVersion);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -4936,7 +5412,7 @@ i32 TApiVersionsResponseData::TApiVersion::Size(TKafkaVersion _version) const {
NPrivate::Size<ApiKeyMeta>(_collector, _version, ApiKey);
NPrivate::Size<MinVersionMeta>(_collector, _version, MinVersion);
NPrivate::Size<MaxVersionMeta>(_collector, _version, MaxVersion);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -4951,7 +5427,7 @@ const TApiVersionsResponseData::TSupportedFeatureKey::NameMeta::Type TApiVersion
const TApiVersionsResponseData::TSupportedFeatureKey::MinVersionMeta::Type TApiVersionsResponseData::TSupportedFeatureKey::MinVersionMeta::Default = 0;
const TApiVersionsResponseData::TSupportedFeatureKey::MaxVersionMeta::Type TApiVersionsResponseData::TSupportedFeatureKey::MaxVersionMeta::Default = 0;
-TApiVersionsResponseData::TSupportedFeatureKey::TSupportedFeatureKey()
+TApiVersionsResponseData::TSupportedFeatureKey::TSupportedFeatureKey()
: Name(NameMeta::Default)
, MinVersion(MinVersionMeta::Default)
, MaxVersion(MaxVersionMeta::Default)
@@ -4964,7 +5440,7 @@ void TApiVersionsResponseData::TSupportedFeatureKey::Read(TKafkaReadable& _reada
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<MinVersionMeta>(_readable, _version, MinVersion);
NPrivate::Read<MaxVersionMeta>(_readable, _version, MaxVersion);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -4987,10 +5463,10 @@ void TApiVersionsResponseData::TSupportedFeatureKey::Write(TKafkaWritable& _writ
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<MinVersionMeta>(_collector, _writable, _version, MinVersion);
NPrivate::Write<MaxVersionMeta>(_collector, _writable, _version, MaxVersion);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -4999,7 +5475,7 @@ i32 TApiVersionsResponseData::TSupportedFeatureKey::Size(TKafkaVersion _version)
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<MinVersionMeta>(_collector, _version, MinVersion);
NPrivate::Size<MaxVersionMeta>(_collector, _version, MaxVersion);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5014,7 +5490,7 @@ const TApiVersionsResponseData::TFinalizedFeatureKey::NameMeta::Type TApiVersion
const TApiVersionsResponseData::TFinalizedFeatureKey::MaxVersionLevelMeta::Type TApiVersionsResponseData::TFinalizedFeatureKey::MaxVersionLevelMeta::Default = 0;
const TApiVersionsResponseData::TFinalizedFeatureKey::MinVersionLevelMeta::Type TApiVersionsResponseData::TFinalizedFeatureKey::MinVersionLevelMeta::Default = 0;
-TApiVersionsResponseData::TFinalizedFeatureKey::TFinalizedFeatureKey()
+TApiVersionsResponseData::TFinalizedFeatureKey::TFinalizedFeatureKey()
: Name(NameMeta::Default)
, MaxVersionLevel(MaxVersionLevelMeta::Default)
, MinVersionLevel(MinVersionLevelMeta::Default)
@@ -5027,7 +5503,7 @@ void TApiVersionsResponseData::TFinalizedFeatureKey::Read(TKafkaReadable& _reada
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<MaxVersionLevelMeta>(_readable, _version, MaxVersionLevel);
NPrivate::Read<MinVersionLevelMeta>(_readable, _version, MinVersionLevel);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5050,10 +5526,10 @@ void TApiVersionsResponseData::TFinalizedFeatureKey::Write(TKafkaWritable& _writ
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<MaxVersionLevelMeta>(_collector, _writable, _version, MaxVersionLevel);
NPrivate::Write<MinVersionLevelMeta>(_collector, _writable, _version, MinVersionLevel);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -5062,7 +5538,7 @@ i32 TApiVersionsResponseData::TFinalizedFeatureKey::Size(TKafkaVersion _version)
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<MaxVersionLevelMeta>(_collector, _version, MaxVersionLevel);
NPrivate::Size<MinVersionLevelMeta>(_collector, _version, MinVersionLevel);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5076,7 +5552,7 @@ i32 TApiVersionsResponseData::TFinalizedFeatureKey::Size(TKafkaVersion _version)
const TCreateTopicsRequestData::TimeoutMsMeta::Type TCreateTopicsRequestData::TimeoutMsMeta::Default = 60000;
const TCreateTopicsRequestData::ValidateOnlyMeta::Type TCreateTopicsRequestData::ValidateOnlyMeta::Default = false;
-TCreateTopicsRequestData::TCreateTopicsRequestData()
+TCreateTopicsRequestData::TCreateTopicsRequestData()
: TimeoutMs(TimeoutMsMeta::Default)
, ValidateOnly(ValidateOnlyMeta::Default)
{}
@@ -5088,7 +5564,7 @@ void TCreateTopicsRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _ve
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
NPrivate::Read<TimeoutMsMeta>(_readable, _version, TimeoutMs);
NPrivate::Read<ValidateOnlyMeta>(_readable, _version, ValidateOnly);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5111,10 +5587,10 @@ void TCreateTopicsRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _v
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
NPrivate::Write<TimeoutMsMeta>(_collector, _writable, _version, TimeoutMs);
NPrivate::Write<ValidateOnlyMeta>(_collector, _writable, _version, ValidateOnly);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -5123,7 +5599,7 @@ i32 TCreateTopicsRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
NPrivate::Size<TimeoutMsMeta>(_collector, _version, TimeoutMs);
NPrivate::Size<ValidateOnlyMeta>(_collector, _version, ValidateOnly);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5138,7 +5614,7 @@ const TCreateTopicsRequestData::TCreatableTopic::NameMeta::Type TCreateTopicsReq
const TCreateTopicsRequestData::TCreatableTopic::NumPartitionsMeta::Type TCreateTopicsRequestData::TCreatableTopic::NumPartitionsMeta::Default = 0;
const TCreateTopicsRequestData::TCreatableTopic::ReplicationFactorMeta::Type TCreateTopicsRequestData::TCreatableTopic::ReplicationFactorMeta::Default = 0;
-TCreateTopicsRequestData::TCreatableTopic::TCreatableTopic()
+TCreateTopicsRequestData::TCreatableTopic::TCreatableTopic()
: Name(NameMeta::Default)
, NumPartitions(NumPartitionsMeta::Default)
, ReplicationFactor(ReplicationFactorMeta::Default)
@@ -5153,7 +5629,7 @@ void TCreateTopicsRequestData::TCreatableTopic::Read(TKafkaReadable& _readable,
NPrivate::Read<ReplicationFactorMeta>(_readable, _version, ReplicationFactor);
NPrivate::Read<AssignmentsMeta>(_readable, _version, Assignments);
NPrivate::Read<ConfigsMeta>(_readable, _version, Configs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5178,10 +5654,10 @@ void TCreateTopicsRequestData::TCreatableTopic::Write(TKafkaWritable& _writable,
NPrivate::Write<ReplicationFactorMeta>(_collector, _writable, _version, ReplicationFactor);
NPrivate::Write<AssignmentsMeta>(_collector, _writable, _version, Assignments);
NPrivate::Write<ConfigsMeta>(_collector, _writable, _version, Configs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -5192,7 +5668,7 @@ i32 TCreateTopicsRequestData::TCreatableTopic::Size(TKafkaVersion _version) cons
NPrivate::Size<ReplicationFactorMeta>(_collector, _version, ReplicationFactor);
NPrivate::Size<AssignmentsMeta>(_collector, _version, Assignments);
NPrivate::Size<ConfigsMeta>(_collector, _version, Configs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5205,7 +5681,7 @@ i32 TCreateTopicsRequestData::TCreatableTopic::Size(TKafkaVersion _version) cons
//
const TCreateTopicsRequestData::TCreatableTopic::TCreatableReplicaAssignment::PartitionIndexMeta::Type TCreateTopicsRequestData::TCreatableTopic::TCreatableReplicaAssignment::PartitionIndexMeta::Default = 0;
-TCreateTopicsRequestData::TCreatableTopic::TCreatableReplicaAssignment::TCreatableReplicaAssignment()
+TCreateTopicsRequestData::TCreatableTopic::TCreatableReplicaAssignment::TCreatableReplicaAssignment()
: PartitionIndex(PartitionIndexMeta::Default)
{}
@@ -5215,7 +5691,7 @@ void TCreateTopicsRequestData::TCreatableTopic::TCreatableReplicaAssignment::Rea
}
NPrivate::Read<PartitionIndexMeta>(_readable, _version, PartitionIndex);
NPrivate::Read<BrokerIdsMeta>(_readable, _version, BrokerIds);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5237,10 +5713,10 @@ void TCreateTopicsRequestData::TCreatableTopic::TCreatableReplicaAssignment::Wri
NPrivate::TWriteCollector _collector;
NPrivate::Write<PartitionIndexMeta>(_collector, _writable, _version, PartitionIndex);
NPrivate::Write<BrokerIdsMeta>(_collector, _writable, _version, BrokerIds);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -5248,7 +5724,7 @@ i32 TCreateTopicsRequestData::TCreatableTopic::TCreatableReplicaAssignment::Size
NPrivate::TSizeCollector _collector;
NPrivate::Size<PartitionIndexMeta>(_collector, _version, PartitionIndex);
NPrivate::Size<BrokerIdsMeta>(_collector, _version, BrokerIds);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5262,7 +5738,7 @@ i32 TCreateTopicsRequestData::TCreatableTopic::TCreatableReplicaAssignment::Size
const TCreateTopicsRequestData::TCreatableTopic::TCreateableTopicConfig::NameMeta::Type TCreateTopicsRequestData::TCreatableTopic::TCreateableTopicConfig::NameMeta::Default = {""};
const TCreateTopicsRequestData::TCreatableTopic::TCreateableTopicConfig::ValueMeta::Type TCreateTopicsRequestData::TCreatableTopic::TCreateableTopicConfig::ValueMeta::Default = {""};
-TCreateTopicsRequestData::TCreatableTopic::TCreateableTopicConfig::TCreateableTopicConfig()
+TCreateTopicsRequestData::TCreatableTopic::TCreateableTopicConfig::TCreateableTopicConfig()
: Name(NameMeta::Default)
, Value(ValueMeta::Default)
{}
@@ -5273,7 +5749,7 @@ void TCreateTopicsRequestData::TCreatableTopic::TCreateableTopicConfig::Read(TKa
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<ValueMeta>(_readable, _version, Value);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5295,10 +5771,10 @@ void TCreateTopicsRequestData::TCreatableTopic::TCreateableTopicConfig::Write(TK
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<ValueMeta>(_collector, _writable, _version, Value);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -5306,7 +5782,7 @@ i32 TCreateTopicsRequestData::TCreatableTopic::TCreateableTopicConfig::Size(TKaf
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<ValueMeta>(_collector, _version, Value);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5319,7 +5795,7 @@ i32 TCreateTopicsRequestData::TCreatableTopic::TCreateableTopicConfig::Size(TKaf
//
const TCreateTopicsResponseData::ThrottleTimeMsMeta::Type TCreateTopicsResponseData::ThrottleTimeMsMeta::Default = 0;
-TCreateTopicsResponseData::TCreateTopicsResponseData()
+TCreateTopicsResponseData::TCreateTopicsResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
{}
@@ -5329,7 +5805,7 @@ void TCreateTopicsResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _v
}
NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5351,10 +5827,10 @@ void TCreateTopicsResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _
NPrivate::TWriteCollector _collector;
NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -5362,7 +5838,7 @@ i32 TCreateTopicsResponseData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5381,7 +5857,7 @@ const TCreateTopicsResponseData::TCreatableTopicResult::TopicConfigErrorCodeMeta
const TCreateTopicsResponseData::TCreatableTopicResult::NumPartitionsMeta::Type TCreateTopicsResponseData::TCreatableTopicResult::NumPartitionsMeta::Default = -1;
const TCreateTopicsResponseData::TCreatableTopicResult::ReplicationFactorMeta::Type TCreateTopicsResponseData::TCreatableTopicResult::ReplicationFactorMeta::Default = -1;
-TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicResult()
+TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicResult()
: Name(NameMeta::Default)
, TopicId(TopicIdMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
@@ -5403,7 +5879,7 @@ void TCreateTopicsResponseData::TCreatableTopicResult::Read(TKafkaReadable& _rea
NPrivate::Read<NumPartitionsMeta>(_readable, _version, NumPartitions);
NPrivate::Read<ReplicationFactorMeta>(_readable, _version, ReplicationFactor);
NPrivate::Read<ConfigsMeta>(_readable, _version, Configs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5434,10 +5910,10 @@ void TCreateTopicsResponseData::TCreatableTopicResult::Write(TKafkaWritable& _wr
NPrivate::Write<NumPartitionsMeta>(_collector, _writable, _version, NumPartitions);
NPrivate::Write<ReplicationFactorMeta>(_collector, _writable, _version, ReplicationFactor);
NPrivate::Write<ConfigsMeta>(_collector, _writable, _version, Configs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
NPrivate::WriteTag<TopicConfigErrorCodeMeta>(_writable, _version, TopicConfigErrorCode);
}
}
@@ -5452,7 +5928,7 @@ i32 TCreateTopicsResponseData::TCreatableTopicResult::Size(TKafkaVersion _versio
NPrivate::Size<NumPartitionsMeta>(_collector, _version, NumPartitions);
NPrivate::Size<ReplicationFactorMeta>(_collector, _version, ReplicationFactor);
NPrivate::Size<ConfigsMeta>(_collector, _version, Configs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5469,7 +5945,7 @@ const TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicConfigs::
const TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicConfigs::ConfigSourceMeta::Type TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicConfigs::ConfigSourceMeta::Default = -1;
const TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicConfigs::IsSensitiveMeta::Type TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicConfigs::IsSensitiveMeta::Default = false;
-TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicConfigs::TCreatableTopicConfigs()
+TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicConfigs::TCreatableTopicConfigs()
: Name(NameMeta::Default)
, Value(ValueMeta::Default)
, ReadOnly(ReadOnlyMeta::Default)
@@ -5486,7 +5962,7 @@ void TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicConfigs::R
NPrivate::Read<ReadOnlyMeta>(_readable, _version, ReadOnly);
NPrivate::Read<ConfigSourceMeta>(_readable, _version, ConfigSource);
NPrivate::Read<IsSensitiveMeta>(_readable, _version, IsSensitive);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5511,10 +5987,10 @@ void TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicConfigs::W
NPrivate::Write<ReadOnlyMeta>(_collector, _writable, _version, ReadOnly);
NPrivate::Write<ConfigSourceMeta>(_collector, _writable, _version, ConfigSource);
NPrivate::Write<IsSensitiveMeta>(_collector, _writable, _version, IsSensitive);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -5525,7 +6001,7 @@ i32 TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicConfigs::Si
NPrivate::Size<ReadOnlyMeta>(_collector, _version, ReadOnly);
NPrivate::Size<ConfigSourceMeta>(_collector, _version, ConfigSource);
NPrivate::Size<IsSensitiveMeta>(_collector, _version, IsSensitive);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5541,7 +6017,7 @@ const TInitProducerIdRequestData::TransactionTimeoutMsMeta::Type TInitProducerId
const TInitProducerIdRequestData::ProducerIdMeta::Type TInitProducerIdRequestData::ProducerIdMeta::Default = -1;
const TInitProducerIdRequestData::ProducerEpochMeta::Type TInitProducerIdRequestData::ProducerEpochMeta::Default = -1;
-TInitProducerIdRequestData::TInitProducerIdRequestData()
+TInitProducerIdRequestData::TInitProducerIdRequestData()
: TransactionalId(TransactionalIdMeta::Default)
, TransactionTimeoutMs(TransactionTimeoutMsMeta::Default)
, ProducerId(ProducerIdMeta::Default)
@@ -5556,7 +6032,7 @@ void TInitProducerIdRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _
NPrivate::Read<TransactionTimeoutMsMeta>(_readable, _version, TransactionTimeoutMs);
NPrivate::Read<ProducerIdMeta>(_readable, _version, ProducerId);
NPrivate::Read<ProducerEpochMeta>(_readable, _version, ProducerEpoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5580,10 +6056,10 @@ void TInitProducerIdRequestData::Write(TKafkaWritable& _writable, TKafkaVersion
NPrivate::Write<TransactionTimeoutMsMeta>(_collector, _writable, _version, TransactionTimeoutMs);
NPrivate::Write<ProducerIdMeta>(_collector, _writable, _version, ProducerId);
NPrivate::Write<ProducerEpochMeta>(_collector, _writable, _version, ProducerEpoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -5593,7 +6069,7 @@ i32 TInitProducerIdRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<TransactionTimeoutMsMeta>(_collector, _version, TransactionTimeoutMs);
NPrivate::Size<ProducerIdMeta>(_collector, _version, ProducerId);
NPrivate::Size<ProducerEpochMeta>(_collector, _version, ProducerEpoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5609,7 +6085,7 @@ const TInitProducerIdResponseData::ErrorCodeMeta::Type TInitProducerIdResponseDa
const TInitProducerIdResponseData::ProducerIdMeta::Type TInitProducerIdResponseData::ProducerIdMeta::Default = -1;
const TInitProducerIdResponseData::ProducerEpochMeta::Type TInitProducerIdResponseData::ProducerEpochMeta::Default = 0;
-TInitProducerIdResponseData::TInitProducerIdResponseData()
+TInitProducerIdResponseData::TInitProducerIdResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
, ProducerId(ProducerIdMeta::Default)
@@ -5624,7 +6100,7 @@ void TInitProducerIdResponseData::Read(TKafkaReadable& _readable, TKafkaVersion
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
NPrivate::Read<ProducerIdMeta>(_readable, _version, ProducerId);
NPrivate::Read<ProducerEpochMeta>(_readable, _version, ProducerEpoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5648,10 +6124,10 @@ void TInitProducerIdResponseData::Write(TKafkaWritable& _writable, TKafkaVersion
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
NPrivate::Write<ProducerIdMeta>(_collector, _writable, _version, ProducerId);
NPrivate::Write<ProducerEpochMeta>(_collector, _writable, _version, ProducerEpoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -5661,7 +6137,7 @@ i32 TInitProducerIdResponseData::Size(TKafkaVersion _version) const {
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
NPrivate::Size<ProducerIdMeta>(_collector, _version, ProducerId);
NPrivate::Size<ProducerEpochMeta>(_collector, _version, ProducerEpoch);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5676,7 +6152,7 @@ const TAddPartitionsToTxnRequestData::TransactionalIdMeta::Type TAddPartitionsTo
const TAddPartitionsToTxnRequestData::ProducerIdMeta::Type TAddPartitionsToTxnRequestData::ProducerIdMeta::Default = 0;
const TAddPartitionsToTxnRequestData::ProducerEpochMeta::Type TAddPartitionsToTxnRequestData::ProducerEpochMeta::Default = 0;
-TAddPartitionsToTxnRequestData::TAddPartitionsToTxnRequestData()
+TAddPartitionsToTxnRequestData::TAddPartitionsToTxnRequestData()
: TransactionalId(TransactionalIdMeta::Default)
, ProducerId(ProducerIdMeta::Default)
, ProducerEpoch(ProducerEpochMeta::Default)
@@ -5690,7 +6166,7 @@ void TAddPartitionsToTxnRequestData::Read(TKafkaReadable& _readable, TKafkaVersi
NPrivate::Read<ProducerIdMeta>(_readable, _version, ProducerId);
NPrivate::Read<ProducerEpochMeta>(_readable, _version, ProducerEpoch);
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5714,10 +6190,10 @@ void TAddPartitionsToTxnRequestData::Write(TKafkaWritable& _writable, TKafkaVers
NPrivate::Write<ProducerIdMeta>(_collector, _writable, _version, ProducerId);
NPrivate::Write<ProducerEpochMeta>(_collector, _writable, _version, ProducerEpoch);
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -5727,7 +6203,7 @@ i32 TAddPartitionsToTxnRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<ProducerIdMeta>(_collector, _version, ProducerId);
NPrivate::Size<ProducerEpochMeta>(_collector, _version, ProducerEpoch);
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5740,7 +6216,7 @@ i32 TAddPartitionsToTxnRequestData::Size(TKafkaVersion _version) const {
//
const TAddPartitionsToTxnRequestData::TAddPartitionsToTxnTopic::NameMeta::Type TAddPartitionsToTxnRequestData::TAddPartitionsToTxnTopic::NameMeta::Default = {""};
-TAddPartitionsToTxnRequestData::TAddPartitionsToTxnTopic::TAddPartitionsToTxnTopic()
+TAddPartitionsToTxnRequestData::TAddPartitionsToTxnTopic::TAddPartitionsToTxnTopic()
: Name(NameMeta::Default)
{}
@@ -5750,7 +6226,7 @@ void TAddPartitionsToTxnRequestData::TAddPartitionsToTxnTopic::Read(TKafkaReadab
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<PartitionsMeta>(_readable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5772,10 +6248,10 @@ void TAddPartitionsToTxnRequestData::TAddPartitionsToTxnTopic::Write(TKafkaWrita
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<PartitionsMeta>(_collector, _writable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -5783,7 +6259,7 @@ i32 TAddPartitionsToTxnRequestData::TAddPartitionsToTxnTopic::Size(TKafkaVersion
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<PartitionsMeta>(_collector, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5796,7 +6272,7 @@ i32 TAddPartitionsToTxnRequestData::TAddPartitionsToTxnTopic::Size(TKafkaVersion
//
const TAddPartitionsToTxnResponseData::ThrottleTimeMsMeta::Type TAddPartitionsToTxnResponseData::ThrottleTimeMsMeta::Default = 0;
-TAddPartitionsToTxnResponseData::TAddPartitionsToTxnResponseData()
+TAddPartitionsToTxnResponseData::TAddPartitionsToTxnResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
{}
@@ -5806,7 +6282,7 @@ void TAddPartitionsToTxnResponseData::Read(TKafkaReadable& _readable, TKafkaVers
}
NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
NPrivate::Read<ResultsMeta>(_readable, _version, Results);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5828,10 +6304,10 @@ void TAddPartitionsToTxnResponseData::Write(TKafkaWritable& _writable, TKafkaVer
NPrivate::TWriteCollector _collector;
NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
NPrivate::Write<ResultsMeta>(_collector, _writable, _version, Results);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -5839,7 +6315,7 @@ i32 TAddPartitionsToTxnResponseData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
NPrivate::Size<ResultsMeta>(_collector, _version, Results);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5852,7 +6328,7 @@ i32 TAddPartitionsToTxnResponseData::Size(TKafkaVersion _version) const {
//
const TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::NameMeta::Type TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::NameMeta::Default = {""};
-TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::TAddPartitionsToTxnTopicResult()
+TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::TAddPartitionsToTxnTopicResult()
: Name(NameMeta::Default)
{}
@@ -5862,7 +6338,7 @@ void TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::Read(TKafk
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<ResultsMeta>(_readable, _version, Results);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5884,10 +6360,10 @@ void TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::Write(TKaf
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<ResultsMeta>(_collector, _writable, _version, Results);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -5895,7 +6371,7 @@ i32 TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::Size(TKafka
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<ResultsMeta>(_collector, _version, Results);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5909,7 +6385,7 @@ i32 TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::Size(TKafka
const TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::TAddPartitionsToTxnPartitionResult::PartitionIndexMeta::Type TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::TAddPartitionsToTxnPartitionResult::PartitionIndexMeta::Default = 0;
const TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::TAddPartitionsToTxnPartitionResult::ErrorCodeMeta::Type TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::TAddPartitionsToTxnPartitionResult::ErrorCodeMeta::Default = 0;
-TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::TAddPartitionsToTxnPartitionResult::TAddPartitionsToTxnPartitionResult()
+TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::TAddPartitionsToTxnPartitionResult::TAddPartitionsToTxnPartitionResult()
: PartitionIndex(PartitionIndexMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
{}
@@ -5920,7 +6396,7 @@ void TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::TAddPartit
}
NPrivate::Read<PartitionIndexMeta>(_readable, _version, PartitionIndex);
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -5942,10 +6418,10 @@ void TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::TAddPartit
NPrivate::TWriteCollector _collector;
NPrivate::Write<PartitionIndexMeta>(_collector, _writable, _version, PartitionIndex);
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -5953,7 +6429,7 @@ i32 TAddPartitionsToTxnResponseData::TAddPartitionsToTxnTopicResult::TAddPartiti
NPrivate::TSizeCollector _collector;
NPrivate::Size<PartitionIndexMeta>(_collector, _version, PartitionIndex);
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -5969,7 +6445,7 @@ const TAddOffsetsToTxnRequestData::ProducerIdMeta::Type TAddOffsetsToTxnRequestD
const TAddOffsetsToTxnRequestData::ProducerEpochMeta::Type TAddOffsetsToTxnRequestData::ProducerEpochMeta::Default = 0;
const TAddOffsetsToTxnRequestData::GroupIdMeta::Type TAddOffsetsToTxnRequestData::GroupIdMeta::Default = {""};
-TAddOffsetsToTxnRequestData::TAddOffsetsToTxnRequestData()
+TAddOffsetsToTxnRequestData::TAddOffsetsToTxnRequestData()
: TransactionalId(TransactionalIdMeta::Default)
, ProducerId(ProducerIdMeta::Default)
, ProducerEpoch(ProducerEpochMeta::Default)
@@ -5984,7 +6460,7 @@ void TAddOffsetsToTxnRequestData::Read(TKafkaReadable& _readable, TKafkaVersion
NPrivate::Read<ProducerIdMeta>(_readable, _version, ProducerId);
NPrivate::Read<ProducerEpochMeta>(_readable, _version, ProducerEpoch);
NPrivate::Read<GroupIdMeta>(_readable, _version, GroupId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6008,10 +6484,10 @@ void TAddOffsetsToTxnRequestData::Write(TKafkaWritable& _writable, TKafkaVersion
NPrivate::Write<ProducerIdMeta>(_collector, _writable, _version, ProducerId);
NPrivate::Write<ProducerEpochMeta>(_collector, _writable, _version, ProducerEpoch);
NPrivate::Write<GroupIdMeta>(_collector, _writable, _version, GroupId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6021,7 +6497,7 @@ i32 TAddOffsetsToTxnRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<ProducerIdMeta>(_collector, _version, ProducerId);
NPrivate::Size<ProducerEpochMeta>(_collector, _version, ProducerEpoch);
NPrivate::Size<GroupIdMeta>(_collector, _version, GroupId);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6035,7 +6511,7 @@ i32 TAddOffsetsToTxnRequestData::Size(TKafkaVersion _version) const {
const TAddOffsetsToTxnResponseData::ThrottleTimeMsMeta::Type TAddOffsetsToTxnResponseData::ThrottleTimeMsMeta::Default = 0;
const TAddOffsetsToTxnResponseData::ErrorCodeMeta::Type TAddOffsetsToTxnResponseData::ErrorCodeMeta::Default = 0;
-TAddOffsetsToTxnResponseData::TAddOffsetsToTxnResponseData()
+TAddOffsetsToTxnResponseData::TAddOffsetsToTxnResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
{}
@@ -6046,7 +6522,7 @@ void TAddOffsetsToTxnResponseData::Read(TKafkaReadable& _readable, TKafkaVersion
}
NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6068,10 +6544,10 @@ void TAddOffsetsToTxnResponseData::Write(TKafkaWritable& _writable, TKafkaVersio
NPrivate::TWriteCollector _collector;
NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6079,7 +6555,7 @@ i32 TAddOffsetsToTxnResponseData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6095,7 +6571,7 @@ const TEndTxnRequestData::ProducerIdMeta::Type TEndTxnRequestData::ProducerIdMet
const TEndTxnRequestData::ProducerEpochMeta::Type TEndTxnRequestData::ProducerEpochMeta::Default = 0;
const TEndTxnRequestData::CommittedMeta::Type TEndTxnRequestData::CommittedMeta::Default = false;
-TEndTxnRequestData::TEndTxnRequestData()
+TEndTxnRequestData::TEndTxnRequestData()
: TransactionalId(TransactionalIdMeta::Default)
, ProducerId(ProducerIdMeta::Default)
, ProducerEpoch(ProducerEpochMeta::Default)
@@ -6110,7 +6586,7 @@ void TEndTxnRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _version)
NPrivate::Read<ProducerIdMeta>(_readable, _version, ProducerId);
NPrivate::Read<ProducerEpochMeta>(_readable, _version, ProducerEpoch);
NPrivate::Read<CommittedMeta>(_readable, _version, Committed);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6134,10 +6610,10 @@ void TEndTxnRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _version
NPrivate::Write<ProducerIdMeta>(_collector, _writable, _version, ProducerId);
NPrivate::Write<ProducerEpochMeta>(_collector, _writable, _version, ProducerEpoch);
NPrivate::Write<CommittedMeta>(_collector, _writable, _version, Committed);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6147,7 +6623,7 @@ i32 TEndTxnRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<ProducerIdMeta>(_collector, _version, ProducerId);
NPrivate::Size<ProducerEpochMeta>(_collector, _version, ProducerEpoch);
NPrivate::Size<CommittedMeta>(_collector, _version, Committed);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6161,7 +6637,7 @@ i32 TEndTxnRequestData::Size(TKafkaVersion _version) const {
const TEndTxnResponseData::ThrottleTimeMsMeta::Type TEndTxnResponseData::ThrottleTimeMsMeta::Default = 0;
const TEndTxnResponseData::ErrorCodeMeta::Type TEndTxnResponseData::ErrorCodeMeta::Default = 0;
-TEndTxnResponseData::TEndTxnResponseData()
+TEndTxnResponseData::TEndTxnResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
{}
@@ -6172,7 +6648,7 @@ void TEndTxnResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _version
}
NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6194,10 +6670,10 @@ void TEndTxnResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _versio
NPrivate::TWriteCollector _collector;
NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6205,7 +6681,7 @@ i32 TEndTxnResponseData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6224,7 +6700,7 @@ const TTxnOffsetCommitRequestData::GenerationIdMeta::Type TTxnOffsetCommitReques
const TTxnOffsetCommitRequestData::MemberIdMeta::Type TTxnOffsetCommitRequestData::MemberIdMeta::Default = {""};
const TTxnOffsetCommitRequestData::GroupInstanceIdMeta::Type TTxnOffsetCommitRequestData::GroupInstanceIdMeta::Default = std::nullopt;
-TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestData()
+TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestData()
: TransactionalId(TransactionalIdMeta::Default)
, GroupId(GroupIdMeta::Default)
, ProducerId(ProducerIdMeta::Default)
@@ -6246,7 +6722,7 @@ void TTxnOffsetCommitRequestData::Read(TKafkaReadable& _readable, TKafkaVersion
NPrivate::Read<MemberIdMeta>(_readable, _version, MemberId);
NPrivate::Read<GroupInstanceIdMeta>(_readable, _version, GroupInstanceId);
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6274,10 +6750,10 @@ void TTxnOffsetCommitRequestData::Write(TKafkaWritable& _writable, TKafkaVersion
NPrivate::Write<MemberIdMeta>(_collector, _writable, _version, MemberId);
NPrivate::Write<GroupInstanceIdMeta>(_collector, _writable, _version, GroupInstanceId);
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6291,7 +6767,7 @@ i32 TTxnOffsetCommitRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<MemberIdMeta>(_collector, _version, MemberId);
NPrivate::Size<GroupInstanceIdMeta>(_collector, _version, GroupInstanceId);
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6304,7 +6780,7 @@ i32 TTxnOffsetCommitRequestData::Size(TKafkaVersion _version) const {
//
const TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::NameMeta::Type TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::NameMeta::Default = {""};
-TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::TTxnOffsetCommitRequestTopic()
+TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::TTxnOffsetCommitRequestTopic()
: Name(NameMeta::Default)
{}
@@ -6314,7 +6790,7 @@ void TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::Read(TKafkaReada
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<PartitionsMeta>(_readable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6336,10 +6812,10 @@ void TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::Write(TKafkaWrit
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<PartitionsMeta>(_collector, _writable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6347,7 +6823,7 @@ i32 TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::Size(TKafkaVersio
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<PartitionsMeta>(_collector, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6363,7 +6839,7 @@ const TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::TTxnOffsetCommi
const TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::TTxnOffsetCommitRequestPartition::CommittedLeaderEpochMeta::Type TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::TTxnOffsetCommitRequestPartition::CommittedLeaderEpochMeta::Default = -1;
const TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::TTxnOffsetCommitRequestPartition::CommittedMetadataMeta::Type TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::TTxnOffsetCommitRequestPartition::CommittedMetadataMeta::Default = {""};
-TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::TTxnOffsetCommitRequestPartition::TTxnOffsetCommitRequestPartition()
+TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::TTxnOffsetCommitRequestPartition::TTxnOffsetCommitRequestPartition()
: PartitionIndex(PartitionIndexMeta::Default)
, CommittedOffset(CommittedOffsetMeta::Default)
, CommittedLeaderEpoch(CommittedLeaderEpochMeta::Default)
@@ -6378,7 +6854,7 @@ void TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::TTxnOffsetCommit
NPrivate::Read<CommittedOffsetMeta>(_readable, _version, CommittedOffset);
NPrivate::Read<CommittedLeaderEpochMeta>(_readable, _version, CommittedLeaderEpoch);
NPrivate::Read<CommittedMetadataMeta>(_readable, _version, CommittedMetadata);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6402,10 +6878,10 @@ void TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::TTxnOffsetCommit
NPrivate::Write<CommittedOffsetMeta>(_collector, _writable, _version, CommittedOffset);
NPrivate::Write<CommittedLeaderEpochMeta>(_collector, _writable, _version, CommittedLeaderEpoch);
NPrivate::Write<CommittedMetadataMeta>(_collector, _writable, _version, CommittedMetadata);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6415,7 +6891,7 @@ i32 TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::TTxnOffsetCommitR
NPrivate::Size<CommittedOffsetMeta>(_collector, _version, CommittedOffset);
NPrivate::Size<CommittedLeaderEpochMeta>(_collector, _version, CommittedLeaderEpoch);
NPrivate::Size<CommittedMetadataMeta>(_collector, _version, CommittedMetadata);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6428,7 +6904,7 @@ i32 TTxnOffsetCommitRequestData::TTxnOffsetCommitRequestTopic::TTxnOffsetCommitR
//
const TTxnOffsetCommitResponseData::ThrottleTimeMsMeta::Type TTxnOffsetCommitResponseData::ThrottleTimeMsMeta::Default = 0;
-TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseData()
+TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
{}
@@ -6438,7 +6914,7 @@ void TTxnOffsetCommitResponseData::Read(TKafkaReadable& _readable, TKafkaVersion
}
NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6460,10 +6936,10 @@ void TTxnOffsetCommitResponseData::Write(TKafkaWritable& _writable, TKafkaVersio
NPrivate::TWriteCollector _collector;
NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6471,7 +6947,7 @@ i32 TTxnOffsetCommitResponseData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6484,7 +6960,7 @@ i32 TTxnOffsetCommitResponseData::Size(TKafkaVersion _version) const {
//
const TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::NameMeta::Type TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::NameMeta::Default = {""};
-TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::TTxnOffsetCommitResponseTopic()
+TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::TTxnOffsetCommitResponseTopic()
: Name(NameMeta::Default)
{}
@@ -6494,7 +6970,7 @@ void TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::Read(TKafkaRea
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<PartitionsMeta>(_readable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6516,10 +6992,10 @@ void TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::Write(TKafkaWr
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<PartitionsMeta>(_collector, _writable, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6527,7 +7003,7 @@ i32 TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::Size(TKafkaVers
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<PartitionsMeta>(_collector, _version, Partitions);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6541,7 +7017,7 @@ i32 TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::Size(TKafkaVers
const TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::TTxnOffsetCommitResponsePartition::PartitionIndexMeta::Type TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::TTxnOffsetCommitResponsePartition::PartitionIndexMeta::Default = 0;
const TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::TTxnOffsetCommitResponsePartition::ErrorCodeMeta::Type TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::TTxnOffsetCommitResponsePartition::ErrorCodeMeta::Default = 0;
-TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::TTxnOffsetCommitResponsePartition::TTxnOffsetCommitResponsePartition()
+TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::TTxnOffsetCommitResponsePartition::TTxnOffsetCommitResponsePartition()
: PartitionIndex(PartitionIndexMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
{}
@@ -6552,7 +7028,7 @@ void TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::TTxnOffsetComm
}
NPrivate::Read<PartitionIndexMeta>(_readable, _version, PartitionIndex);
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6574,10 +7050,10 @@ void TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::TTxnOffsetComm
NPrivate::TWriteCollector _collector;
NPrivate::Write<PartitionIndexMeta>(_collector, _writable, _version, PartitionIndex);
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6585,7 +7061,7 @@ i32 TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::TTxnOffsetCommi
NPrivate::TSizeCollector _collector;
NPrivate::Size<PartitionIndexMeta>(_collector, _version, PartitionIndex);
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6599,7 +7075,7 @@ i32 TTxnOffsetCommitResponseData::TTxnOffsetCommitResponseTopic::TTxnOffsetCommi
const TDescribeConfigsRequestData::IncludeSynonymsMeta::Type TDescribeConfigsRequestData::IncludeSynonymsMeta::Default = false;
const TDescribeConfigsRequestData::IncludeDocumentationMeta::Type TDescribeConfigsRequestData::IncludeDocumentationMeta::Default = false;
-TDescribeConfigsRequestData::TDescribeConfigsRequestData()
+TDescribeConfigsRequestData::TDescribeConfigsRequestData()
: IncludeSynonyms(IncludeSynonymsMeta::Default)
, IncludeDocumentation(IncludeDocumentationMeta::Default)
{}
@@ -6611,7 +7087,7 @@ void TDescribeConfigsRequestData::Read(TKafkaReadable& _readable, TKafkaVersion
NPrivate::Read<ResourcesMeta>(_readable, _version, Resources);
NPrivate::Read<IncludeSynonymsMeta>(_readable, _version, IncludeSynonyms);
NPrivate::Read<IncludeDocumentationMeta>(_readable, _version, IncludeDocumentation);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6634,10 +7110,10 @@ void TDescribeConfigsRequestData::Write(TKafkaWritable& _writable, TKafkaVersion
NPrivate::Write<ResourcesMeta>(_collector, _writable, _version, Resources);
NPrivate::Write<IncludeSynonymsMeta>(_collector, _writable, _version, IncludeSynonyms);
NPrivate::Write<IncludeDocumentationMeta>(_collector, _writable, _version, IncludeDocumentation);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6646,7 +7122,7 @@ i32 TDescribeConfigsRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<ResourcesMeta>(_collector, _version, Resources);
NPrivate::Size<IncludeSynonymsMeta>(_collector, _version, IncludeSynonyms);
NPrivate::Size<IncludeDocumentationMeta>(_collector, _version, IncludeDocumentation);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6660,7 +7136,7 @@ i32 TDescribeConfigsRequestData::Size(TKafkaVersion _version) const {
const TDescribeConfigsRequestData::TDescribeConfigsResource::ResourceTypeMeta::Type TDescribeConfigsRequestData::TDescribeConfigsResource::ResourceTypeMeta::Default = 0;
const TDescribeConfigsRequestData::TDescribeConfigsResource::ResourceNameMeta::Type TDescribeConfigsRequestData::TDescribeConfigsResource::ResourceNameMeta::Default = {""};
-TDescribeConfigsRequestData::TDescribeConfigsResource::TDescribeConfigsResource()
+TDescribeConfigsRequestData::TDescribeConfigsResource::TDescribeConfigsResource()
: ResourceType(ResourceTypeMeta::Default)
, ResourceName(ResourceNameMeta::Default)
{}
@@ -6672,7 +7148,7 @@ void TDescribeConfigsRequestData::TDescribeConfigsResource::Read(TKafkaReadable&
NPrivate::Read<ResourceTypeMeta>(_readable, _version, ResourceType);
NPrivate::Read<ResourceNameMeta>(_readable, _version, ResourceName);
NPrivate::Read<ConfigurationKeysMeta>(_readable, _version, ConfigurationKeys);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6695,10 +7171,10 @@ void TDescribeConfigsRequestData::TDescribeConfigsResource::Write(TKafkaWritable
NPrivate::Write<ResourceTypeMeta>(_collector, _writable, _version, ResourceType);
NPrivate::Write<ResourceNameMeta>(_collector, _writable, _version, ResourceName);
NPrivate::Write<ConfigurationKeysMeta>(_collector, _writable, _version, ConfigurationKeys);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6707,7 +7183,7 @@ i32 TDescribeConfigsRequestData::TDescribeConfigsResource::Size(TKafkaVersion _v
NPrivate::Size<ResourceTypeMeta>(_collector, _version, ResourceType);
NPrivate::Size<ResourceNameMeta>(_collector, _version, ResourceName);
NPrivate::Size<ConfigurationKeysMeta>(_collector, _version, ConfigurationKeys);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6720,7 +7196,7 @@ i32 TDescribeConfigsRequestData::TDescribeConfigsResource::Size(TKafkaVersion _v
//
const TDescribeConfigsResponseData::ThrottleTimeMsMeta::Type TDescribeConfigsResponseData::ThrottleTimeMsMeta::Default = 0;
-TDescribeConfigsResponseData::TDescribeConfigsResponseData()
+TDescribeConfigsResponseData::TDescribeConfigsResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
{}
@@ -6730,7 +7206,7 @@ void TDescribeConfigsResponseData::Read(TKafkaReadable& _readable, TKafkaVersion
}
NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
NPrivate::Read<ResultsMeta>(_readable, _version, Results);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6752,10 +7228,10 @@ void TDescribeConfigsResponseData::Write(TKafkaWritable& _writable, TKafkaVersio
NPrivate::TWriteCollector _collector;
NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
NPrivate::Write<ResultsMeta>(_collector, _writable, _version, Results);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6763,7 +7239,7 @@ i32 TDescribeConfigsResponseData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
NPrivate::Size<ResultsMeta>(_collector, _version, Results);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6779,7 +7255,7 @@ const TDescribeConfigsResponseData::TDescribeConfigsResult::ErrorMessageMeta::Ty
const TDescribeConfigsResponseData::TDescribeConfigsResult::ResourceTypeMeta::Type TDescribeConfigsResponseData::TDescribeConfigsResult::ResourceTypeMeta::Default = 0;
const TDescribeConfigsResponseData::TDescribeConfigsResult::ResourceNameMeta::Type TDescribeConfigsResponseData::TDescribeConfigsResult::ResourceNameMeta::Default = {""};
-TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResult()
+TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResult()
: ErrorCode(ErrorCodeMeta::Default)
, ErrorMessage(ErrorMessageMeta::Default)
, ResourceType(ResourceTypeMeta::Default)
@@ -6795,7 +7271,7 @@ void TDescribeConfigsResponseData::TDescribeConfigsResult::Read(TKafkaReadable&
NPrivate::Read<ResourceTypeMeta>(_readable, _version, ResourceType);
NPrivate::Read<ResourceNameMeta>(_readable, _version, ResourceName);
NPrivate::Read<ConfigsMeta>(_readable, _version, Configs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6820,10 +7296,10 @@ void TDescribeConfigsResponseData::TDescribeConfigsResult::Write(TKafkaWritable&
NPrivate::Write<ResourceTypeMeta>(_collector, _writable, _version, ResourceType);
NPrivate::Write<ResourceNameMeta>(_collector, _writable, _version, ResourceName);
NPrivate::Write<ConfigsMeta>(_collector, _writable, _version, Configs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6834,7 +7310,7 @@ i32 TDescribeConfigsResponseData::TDescribeConfigsResult::Size(TKafkaVersion _ve
NPrivate::Size<ResourceTypeMeta>(_collector, _version, ResourceType);
NPrivate::Size<ResourceNameMeta>(_collector, _version, ResourceName);
NPrivate::Size<ConfigsMeta>(_collector, _version, Configs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6854,7 +7330,7 @@ const TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsReso
const TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResourceResult::ConfigTypeMeta::Type TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResourceResult::ConfigTypeMeta::Default = 0;
const TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResourceResult::DocumentationMeta::Type TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResourceResult::DocumentationMeta::Default = {""};
-TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResourceResult::TDescribeConfigsResourceResult()
+TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResourceResult::TDescribeConfigsResourceResult()
: Name(NameMeta::Default)
, Value(ValueMeta::Default)
, ReadOnly(ReadOnlyMeta::Default)
@@ -6878,7 +7354,7 @@ void TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResou
NPrivate::Read<SynonymsMeta>(_readable, _version, Synonyms);
NPrivate::Read<ConfigTypeMeta>(_readable, _version, ConfigType);
NPrivate::Read<DocumentationMeta>(_readable, _version, Documentation);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6907,10 +7383,10 @@ void TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResou
NPrivate::Write<SynonymsMeta>(_collector, _writable, _version, Synonyms);
NPrivate::Write<ConfigTypeMeta>(_collector, _writable, _version, ConfigType);
NPrivate::Write<DocumentationMeta>(_collector, _writable, _version, Documentation);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6925,7 +7401,7 @@ i32 TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResour
NPrivate::Size<SynonymsMeta>(_collector, _version, Synonyms);
NPrivate::Size<ConfigTypeMeta>(_collector, _version, ConfigType);
NPrivate::Size<DocumentationMeta>(_collector, _version, Documentation);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -6940,7 +7416,7 @@ const TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsReso
const TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResourceResult::TDescribeConfigsSynonym::ValueMeta::Type TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResourceResult::TDescribeConfigsSynonym::ValueMeta::Default = {""};
const TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResourceResult::TDescribeConfigsSynonym::SourceMeta::Type TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResourceResult::TDescribeConfigsSynonym::SourceMeta::Default = 0;
-TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResourceResult::TDescribeConfigsSynonym::TDescribeConfigsSynonym()
+TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResourceResult::TDescribeConfigsSynonym::TDescribeConfigsSynonym()
: Name(NameMeta::Default)
, Value(ValueMeta::Default)
, Source(SourceMeta::Default)
@@ -6953,7 +7429,7 @@ void TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResou
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<ValueMeta>(_readable, _version, Value);
NPrivate::Read<SourceMeta>(_readable, _version, Source);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -6976,10 +7452,10 @@ void TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResou
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<ValueMeta>(_collector, _writable, _version, Value);
NPrivate::Write<SourceMeta>(_collector, _writable, _version, Source);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -6988,7 +7464,7 @@ i32 TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResour
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<ValueMeta>(_collector, _version, Value);
NPrivate::Size<SourceMeta>(_collector, _version, Source);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -7001,7 +7477,7 @@ i32 TDescribeConfigsResponseData::TDescribeConfigsResult::TDescribeConfigsResour
//
const TAlterConfigsRequestData::ValidateOnlyMeta::Type TAlterConfigsRequestData::ValidateOnlyMeta::Default = false;
-TAlterConfigsRequestData::TAlterConfigsRequestData()
+TAlterConfigsRequestData::TAlterConfigsRequestData()
: ValidateOnly(ValidateOnlyMeta::Default)
{}
@@ -7011,7 +7487,7 @@ void TAlterConfigsRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _ve
}
NPrivate::Read<ResourcesMeta>(_readable, _version, Resources);
NPrivate::Read<ValidateOnlyMeta>(_readable, _version, ValidateOnly);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -7033,10 +7509,10 @@ void TAlterConfigsRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _v
NPrivate::TWriteCollector _collector;
NPrivate::Write<ResourcesMeta>(_collector, _writable, _version, Resources);
NPrivate::Write<ValidateOnlyMeta>(_collector, _writable, _version, ValidateOnly);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -7044,7 +7520,7 @@ i32 TAlterConfigsRequestData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ResourcesMeta>(_collector, _version, Resources);
NPrivate::Size<ValidateOnlyMeta>(_collector, _version, ValidateOnly);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -7058,7 +7534,7 @@ i32 TAlterConfigsRequestData::Size(TKafkaVersion _version) const {
const TAlterConfigsRequestData::TAlterConfigsResource::ResourceTypeMeta::Type TAlterConfigsRequestData::TAlterConfigsResource::ResourceTypeMeta::Default = 0;
const TAlterConfigsRequestData::TAlterConfigsResource::ResourceNameMeta::Type TAlterConfigsRequestData::TAlterConfigsResource::ResourceNameMeta::Default = {""};
-TAlterConfigsRequestData::TAlterConfigsResource::TAlterConfigsResource()
+TAlterConfigsRequestData::TAlterConfigsResource::TAlterConfigsResource()
: ResourceType(ResourceTypeMeta::Default)
, ResourceName(ResourceNameMeta::Default)
{}
@@ -7070,7 +7546,7 @@ void TAlterConfigsRequestData::TAlterConfigsResource::Read(TKafkaReadable& _read
NPrivate::Read<ResourceTypeMeta>(_readable, _version, ResourceType);
NPrivate::Read<ResourceNameMeta>(_readable, _version, ResourceName);
NPrivate::Read<ConfigsMeta>(_readable, _version, Configs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -7093,10 +7569,10 @@ void TAlterConfigsRequestData::TAlterConfigsResource::Write(TKafkaWritable& _wri
NPrivate::Write<ResourceTypeMeta>(_collector, _writable, _version, ResourceType);
NPrivate::Write<ResourceNameMeta>(_collector, _writable, _version, ResourceName);
NPrivate::Write<ConfigsMeta>(_collector, _writable, _version, Configs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -7105,7 +7581,7 @@ i32 TAlterConfigsRequestData::TAlterConfigsResource::Size(TKafkaVersion _version
NPrivate::Size<ResourceTypeMeta>(_collector, _version, ResourceType);
NPrivate::Size<ResourceNameMeta>(_collector, _version, ResourceName);
NPrivate::Size<ConfigsMeta>(_collector, _version, Configs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -7119,7 +7595,7 @@ i32 TAlterConfigsRequestData::TAlterConfigsResource::Size(TKafkaVersion _version
const TAlterConfigsRequestData::TAlterConfigsResource::TAlterableConfig::NameMeta::Type TAlterConfigsRequestData::TAlterConfigsResource::TAlterableConfig::NameMeta::Default = {""};
const TAlterConfigsRequestData::TAlterConfigsResource::TAlterableConfig::ValueMeta::Type TAlterConfigsRequestData::TAlterConfigsResource::TAlterableConfig::ValueMeta::Default = {""};
-TAlterConfigsRequestData::TAlterConfigsResource::TAlterableConfig::TAlterableConfig()
+TAlterConfigsRequestData::TAlterConfigsResource::TAlterableConfig::TAlterableConfig()
: Name(NameMeta::Default)
, Value(ValueMeta::Default)
{}
@@ -7130,7 +7606,7 @@ void TAlterConfigsRequestData::TAlterConfigsResource::TAlterableConfig::Read(TKa
}
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<ValueMeta>(_readable, _version, Value);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -7152,10 +7628,10 @@ void TAlterConfigsRequestData::TAlterConfigsResource::TAlterableConfig::Write(TK
NPrivate::TWriteCollector _collector;
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<ValueMeta>(_collector, _writable, _version, Value);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -7163,7 +7639,7 @@ i32 TAlterConfigsRequestData::TAlterConfigsResource::TAlterableConfig::Size(TKaf
NPrivate::TSizeCollector _collector;
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<ValueMeta>(_collector, _version, Value);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -7176,7 +7652,7 @@ i32 TAlterConfigsRequestData::TAlterConfigsResource::TAlterableConfig::Size(TKaf
//
const TAlterConfigsResponseData::ThrottleTimeMsMeta::Type TAlterConfigsResponseData::ThrottleTimeMsMeta::Default = 0;
-TAlterConfigsResponseData::TAlterConfigsResponseData()
+TAlterConfigsResponseData::TAlterConfigsResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
{}
@@ -7186,7 +7662,7 @@ void TAlterConfigsResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _v
}
NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
NPrivate::Read<ResponsesMeta>(_readable, _version, Responses);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -7208,10 +7684,10 @@ void TAlterConfigsResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _
NPrivate::TWriteCollector _collector;
NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
NPrivate::Write<ResponsesMeta>(_collector, _writable, _version, Responses);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -7219,7 +7695,7 @@ i32 TAlterConfigsResponseData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
NPrivate::Size<ResponsesMeta>(_collector, _version, Responses);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -7235,7 +7711,7 @@ const TAlterConfigsResponseData::TAlterConfigsResourceResponse::ErrorMessageMeta
const TAlterConfigsResponseData::TAlterConfigsResourceResponse::ResourceTypeMeta::Type TAlterConfigsResponseData::TAlterConfigsResourceResponse::ResourceTypeMeta::Default = 0;
const TAlterConfigsResponseData::TAlterConfigsResourceResponse::ResourceNameMeta::Type TAlterConfigsResponseData::TAlterConfigsResourceResponse::ResourceNameMeta::Default = {""};
-TAlterConfigsResponseData::TAlterConfigsResourceResponse::TAlterConfigsResourceResponse()
+TAlterConfigsResponseData::TAlterConfigsResourceResponse::TAlterConfigsResourceResponse()
: ErrorCode(ErrorCodeMeta::Default)
, ErrorMessage(ErrorMessageMeta::Default)
, ResourceType(ResourceTypeMeta::Default)
@@ -7250,7 +7726,7 @@ void TAlterConfigsResponseData::TAlterConfigsResourceResponse::Read(TKafkaReadab
NPrivate::Read<ErrorMessageMeta>(_readable, _version, ErrorMessage);
NPrivate::Read<ResourceTypeMeta>(_readable, _version, ResourceType);
NPrivate::Read<ResourceNameMeta>(_readable, _version, ResourceName);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -7274,10 +7750,10 @@ void TAlterConfigsResponseData::TAlterConfigsResourceResponse::Write(TKafkaWrita
NPrivate::Write<ErrorMessageMeta>(_collector, _writable, _version, ErrorMessage);
NPrivate::Write<ResourceTypeMeta>(_collector, _writable, _version, ResourceType);
NPrivate::Write<ResourceNameMeta>(_collector, _writable, _version, ResourceName);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -7287,7 +7763,7 @@ i32 TAlterConfigsResponseData::TAlterConfigsResourceResponse::Size(TKafkaVersion
NPrivate::Size<ErrorMessageMeta>(_collector, _version, ErrorMessage);
NPrivate::Size<ResourceTypeMeta>(_collector, _version, ResourceType);
NPrivate::Size<ResourceNameMeta>(_collector, _version, ResourceName);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -7299,7 +7775,7 @@ i32 TAlterConfigsResponseData::TAlterConfigsResourceResponse::Size(TKafkaVersion
// TSaslAuthenticateRequestData
//
-TSaslAuthenticateRequestData::TSaslAuthenticateRequestData()
+TSaslAuthenticateRequestData::TSaslAuthenticateRequestData()
{}
void TSaslAuthenticateRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _version) {
@@ -7307,7 +7783,7 @@ void TSaslAuthenticateRequestData::Read(TKafkaReadable& _readable, TKafkaVersion
ythrow yexception() << "Can't read version " << _version << " of TSaslAuthenticateRequestData";
}
NPrivate::Read<AuthBytesMeta>(_readable, _version, AuthBytes);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -7328,17 +7804,17 @@ void TSaslAuthenticateRequestData::Write(TKafkaWritable& _writable, TKafkaVersio
}
NPrivate::TWriteCollector _collector;
NPrivate::Write<AuthBytesMeta>(_collector, _writable, _version, AuthBytes);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
i32 TSaslAuthenticateRequestData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<AuthBytesMeta>(_collector, _version, AuthBytes);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -7353,7 +7829,7 @@ const TSaslAuthenticateResponseData::ErrorCodeMeta::Type TSaslAuthenticateRespon
const TSaslAuthenticateResponseData::ErrorMessageMeta::Type TSaslAuthenticateResponseData::ErrorMessageMeta::Default = {""};
const TSaslAuthenticateResponseData::SessionLifetimeMsMeta::Type TSaslAuthenticateResponseData::SessionLifetimeMsMeta::Default = 0;
-TSaslAuthenticateResponseData::TSaslAuthenticateResponseData()
+TSaslAuthenticateResponseData::TSaslAuthenticateResponseData()
: ErrorCode(ErrorCodeMeta::Default)
, ErrorMessage(ErrorMessageMeta::Default)
, SessionLifetimeMs(SessionLifetimeMsMeta::Default)
@@ -7367,7 +7843,7 @@ void TSaslAuthenticateResponseData::Read(TKafkaReadable& _readable, TKafkaVersio
NPrivate::Read<ErrorMessageMeta>(_readable, _version, ErrorMessage);
NPrivate::Read<AuthBytesMeta>(_readable, _version, AuthBytes);
NPrivate::Read<SessionLifetimeMsMeta>(_readable, _version, SessionLifetimeMs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -7391,10 +7867,10 @@ void TSaslAuthenticateResponseData::Write(TKafkaWritable& _writable, TKafkaVersi
NPrivate::Write<ErrorMessageMeta>(_collector, _writable, _version, ErrorMessage);
NPrivate::Write<AuthBytesMeta>(_collector, _writable, _version, AuthBytes);
NPrivate::Write<SessionLifetimeMsMeta>(_collector, _writable, _version, SessionLifetimeMs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -7404,7 +7880,7 @@ i32 TSaslAuthenticateResponseData::Size(TKafkaVersion _version) const {
NPrivate::Size<ErrorMessageMeta>(_collector, _version, ErrorMessage);
NPrivate::Size<AuthBytesMeta>(_collector, _version, AuthBytes);
NPrivate::Size<SessionLifetimeMsMeta>(_collector, _version, SessionLifetimeMs);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -7418,7 +7894,7 @@ i32 TSaslAuthenticateResponseData::Size(TKafkaVersion _version) const {
const TCreatePartitionsRequestData::TimeoutMsMeta::Type TCreatePartitionsRequestData::TimeoutMsMeta::Default = 0;
const TCreatePartitionsRequestData::ValidateOnlyMeta::Type TCreatePartitionsRequestData::ValidateOnlyMeta::Default = false;
-TCreatePartitionsRequestData::TCreatePartitionsRequestData()
+TCreatePartitionsRequestData::TCreatePartitionsRequestData()
: TimeoutMs(TimeoutMsMeta::Default)
, ValidateOnly(ValidateOnlyMeta::Default)
{}
@@ -7430,7 +7906,7 @@ void TCreatePartitionsRequestData::Read(TKafkaReadable& _readable, TKafkaVersion
NPrivate::Read<TopicsMeta>(_readable, _version, Topics);
NPrivate::Read<TimeoutMsMeta>(_readable, _version, TimeoutMs);
NPrivate::Read<ValidateOnlyMeta>(_readable, _version, ValidateOnly);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -7453,10 +7929,10 @@ void TCreatePartitionsRequestData::Write(TKafkaWritable& _writable, TKafkaVersio
NPrivate::Write<TopicsMeta>(_collector, _writable, _version, Topics);
NPrivate::Write<TimeoutMsMeta>(_collector, _writable, _version, TimeoutMs);
NPrivate::Write<ValidateOnlyMeta>(_collector, _writable, _version, ValidateOnly);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -7465,7 +7941,7 @@ i32 TCreatePartitionsRequestData::Size(TKafkaVersion _version) const {
NPrivate::Size<TopicsMeta>(_collector, _version, Topics);
NPrivate::Size<TimeoutMsMeta>(_collector, _version, TimeoutMs);
NPrivate::Size<ValidateOnlyMeta>(_collector, _version, ValidateOnly);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -7479,7 +7955,7 @@ i32 TCreatePartitionsRequestData::Size(TKafkaVersion _version) const {
const TCreatePartitionsRequestData::TCreatePartitionsTopic::NameMeta::Type TCreatePartitionsRequestData::TCreatePartitionsTopic::NameMeta::Default = {""};
const TCreatePartitionsRequestData::TCreatePartitionsTopic::CountMeta::Type TCreatePartitionsRequestData::TCreatePartitionsTopic::CountMeta::Default = 0;
-TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsTopic()
+TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsTopic()
: Name(NameMeta::Default)
, Count(CountMeta::Default)
{}
@@ -7491,7 +7967,7 @@ void TCreatePartitionsRequestData::TCreatePartitionsTopic::Read(TKafkaReadable&
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<CountMeta>(_readable, _version, Count);
NPrivate::Read<AssignmentsMeta>(_readable, _version, Assignments);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -7514,10 +7990,10 @@ void TCreatePartitionsRequestData::TCreatePartitionsTopic::Write(TKafkaWritable&
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<CountMeta>(_collector, _writable, _version, Count);
NPrivate::Write<AssignmentsMeta>(_collector, _writable, _version, Assignments);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -7526,7 +8002,7 @@ i32 TCreatePartitionsRequestData::TCreatePartitionsTopic::Size(TKafkaVersion _ve
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<CountMeta>(_collector, _version, Count);
NPrivate::Size<AssignmentsMeta>(_collector, _version, Assignments);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -7538,7 +8014,7 @@ i32 TCreatePartitionsRequestData::TCreatePartitionsTopic::Size(TKafkaVersion _ve
// TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsAssignment
//
-TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsAssignment::TCreatePartitionsAssignment()
+TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsAssignment::TCreatePartitionsAssignment()
{}
void TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsAssignment::Read(TKafkaReadable& _readable, TKafkaVersion _version) {
@@ -7546,7 +8022,7 @@ void TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsAssi
ythrow yexception() << "Can't read version " << _version << " of TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsAssignment";
}
NPrivate::Read<BrokerIdsMeta>(_readable, _version, BrokerIds);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -7567,17 +8043,17 @@ void TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsAssi
}
NPrivate::TWriteCollector _collector;
NPrivate::Write<BrokerIdsMeta>(_collector, _writable, _version, BrokerIds);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
i32 TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsAssignment::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<BrokerIdsMeta>(_collector, _version, BrokerIds);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -7590,7 +8066,7 @@ i32 TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsAssig
//
const TCreatePartitionsResponseData::ThrottleTimeMsMeta::Type TCreatePartitionsResponseData::ThrottleTimeMsMeta::Default = 0;
-TCreatePartitionsResponseData::TCreatePartitionsResponseData()
+TCreatePartitionsResponseData::TCreatePartitionsResponseData()
: ThrottleTimeMs(ThrottleTimeMsMeta::Default)
{}
@@ -7600,7 +8076,7 @@ void TCreatePartitionsResponseData::Read(TKafkaReadable& _readable, TKafkaVersio
}
NPrivate::Read<ThrottleTimeMsMeta>(_readable, _version, ThrottleTimeMs);
NPrivate::Read<ResultsMeta>(_readable, _version, Results);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -7622,10 +8098,10 @@ void TCreatePartitionsResponseData::Write(TKafkaWritable& _writable, TKafkaVersi
NPrivate::TWriteCollector _collector;
NPrivate::Write<ThrottleTimeMsMeta>(_collector, _writable, _version, ThrottleTimeMs);
NPrivate::Write<ResultsMeta>(_collector, _writable, _version, Results);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -7633,7 +8109,7 @@ i32 TCreatePartitionsResponseData::Size(TKafkaVersion _version) const {
NPrivate::TSizeCollector _collector;
NPrivate::Size<ThrottleTimeMsMeta>(_collector, _version, ThrottleTimeMs);
NPrivate::Size<ResultsMeta>(_collector, _version, Results);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
@@ -7648,7 +8124,7 @@ const TCreatePartitionsResponseData::TCreatePartitionsTopicResult::NameMeta::Typ
const TCreatePartitionsResponseData::TCreatePartitionsTopicResult::ErrorCodeMeta::Type TCreatePartitionsResponseData::TCreatePartitionsTopicResult::ErrorCodeMeta::Default = 0;
const TCreatePartitionsResponseData::TCreatePartitionsTopicResult::ErrorMessageMeta::Type TCreatePartitionsResponseData::TCreatePartitionsTopicResult::ErrorMessageMeta::Default = std::nullopt;
-TCreatePartitionsResponseData::TCreatePartitionsTopicResult::TCreatePartitionsTopicResult()
+TCreatePartitionsResponseData::TCreatePartitionsTopicResult::TCreatePartitionsTopicResult()
: Name(NameMeta::Default)
, ErrorCode(ErrorCodeMeta::Default)
, ErrorMessage(ErrorMessageMeta::Default)
@@ -7661,7 +8137,7 @@ void TCreatePartitionsResponseData::TCreatePartitionsTopicResult::Read(TKafkaRea
NPrivate::Read<NameMeta>(_readable, _version, Name);
NPrivate::Read<ErrorCodeMeta>(_readable, _version, ErrorCode);
NPrivate::Read<ErrorMessageMeta>(_readable, _version, ErrorMessage);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
ui32 _numTaggedFields = _readable.readUnsignedVarint<ui32>();
for (ui32 _i = 0; _i < _numTaggedFields; ++_i) {
@@ -7684,10 +8160,10 @@ void TCreatePartitionsResponseData::TCreatePartitionsTopicResult::Write(TKafkaWr
NPrivate::Write<NameMeta>(_collector, _writable, _version, Name);
NPrivate::Write<ErrorCodeMeta>(_collector, _writable, _version, ErrorCode);
NPrivate::Write<ErrorMessageMeta>(_collector, _writable, _version, ErrorMessage);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_writable.writeUnsignedVarint(_collector.NumTaggedFields);
-
+
}
}
@@ -7696,7 +8172,7 @@ i32 TCreatePartitionsResponseData::TCreatePartitionsTopicResult::Size(TKafkaVers
NPrivate::Size<NameMeta>(_collector, _version, Name);
NPrivate::Size<ErrorCodeMeta>(_collector, _version, ErrorCode);
NPrivate::Size<ErrorMessageMeta>(_collector, _version, ErrorMessage);
-
+
if (NPrivate::VersionCheck<MessageMeta::FlexibleVersions.Min, MessageMeta::FlexibleVersions.Max>(_version)) {
_collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields);
}
diff --git a/ydb/core/kafka_proxy/kafka_messages.h b/ydb/core/kafka_proxy/kafka_messages.h
index 2231e95c3c2..94ae90a2ffa 100644
--- a/ydb/core/kafka_proxy/kafka_messages.h
+++ b/ydb/core/kafka_proxy/kafka_messages.h
@@ -15,30 +15,32 @@ enum EListenerType {
};
enum EApiKey {
- HEADER = -1, // []
- PRODUCE = 0, // [ZK_BROKER, BROKER]
- FETCH = 1, // [ZK_BROKER, BROKER, CONTROLLER]
- LIST_OFFSETS = 2, // [ZK_BROKER, BROKER]
- METADATA = 3, // [ZK_BROKER, BROKER]
- OFFSET_COMMIT = 8, // [ZK_BROKER, BROKER]
- OFFSET_FETCH = 9, // [ZK_BROKER, BROKER]
- FIND_COORDINATOR = 10, // [ZK_BROKER, BROKER]
- JOIN_GROUP = 11, // [ZK_BROKER, BROKER]
- HEARTBEAT = 12, // [ZK_BROKER, BROKER]
- LEAVE_GROUP = 13, // [ZK_BROKER, BROKER]
- SYNC_GROUP = 14, // [ZK_BROKER, BROKER]
- SASL_HANDSHAKE = 17, // [ZK_BROKER, BROKER, CONTROLLER]
- API_VERSIONS = 18, // [ZK_BROKER, BROKER, CONTROLLER]
- CREATE_TOPICS = 19, // [ZK_BROKER, BROKER, CONTROLLER]
- INIT_PRODUCER_ID = 22, // [ZK_BROKER, BROKER]
- ADD_PARTITIONS_TO_TXN = 24, // [ZK_BROKER, BROKER]
- ADD_OFFSETS_TO_TXN = 25, // [ZK_BROKER, BROKER]
- END_TXN = 26, // [ZK_BROKER, BROKER]
- TXN_OFFSET_COMMIT = 28, // [ZK_BROKER, BROKER]
- DESCRIBE_CONFIGS = 32, // [ZK_BROKER, BROKER]
- ALTER_CONFIGS = 33, // [ZK_BROKER, BROKER, CONTROLLER]
- SASL_AUTHENTICATE = 36, // [ZK_BROKER, BROKER, CONTROLLER]
- CREATE_PARTITIONS = 37, // [ZK_BROKER, BROKER, CONTROLLER]
+ HEADER = -1, // []
+ PRODUCE = 0, // [ZK_BROKER, BROKER]
+ FETCH = 1, // [ZK_BROKER, BROKER, CONTROLLER]
+ LIST_OFFSETS = 2, // [ZK_BROKER, BROKER]
+ METADATA = 3, // [ZK_BROKER, BROKER]
+ OFFSET_COMMIT = 8, // [ZK_BROKER, BROKER]
+ OFFSET_FETCH = 9, // [ZK_BROKER, BROKER]
+ FIND_COORDINATOR = 10, // [ZK_BROKER, BROKER]
+ JOIN_GROUP = 11, // [ZK_BROKER, BROKER]
+ HEARTBEAT = 12, // [ZK_BROKER, BROKER]
+ LEAVE_GROUP = 13, // [ZK_BROKER, BROKER]
+ SYNC_GROUP = 14, // [ZK_BROKER, BROKER]
+ DESCRIBE_GROUPS = 15, // [ZK_BROKER, BROKER]
+ LIST_GROUPS = 16, // [ZK_BROKER, BROKER]
+ SASL_HANDSHAKE = 17, // [ZK_BROKER, BROKER, CONTROLLER]
+ API_VERSIONS = 18, // [ZK_BROKER, BROKER, CONTROLLER]
+ CREATE_TOPICS = 19, // [ZK_BROKER, BROKER, CONTROLLER]
+ INIT_PRODUCER_ID = 22, // [ZK_BROKER, BROKER]
+ ADD_PARTITIONS_TO_TXN = 24, // [ZK_BROKER, BROKER]
+ ADD_OFFSETS_TO_TXN = 25, // [ZK_BROKER, BROKER]
+ END_TXN = 26, // [ZK_BROKER, BROKER]
+ TXN_OFFSET_COMMIT = 28, // [ZK_BROKER, BROKER]
+ DESCRIBE_CONFIGS = 32, // [ZK_BROKER, BROKER]
+ ALTER_CONFIGS = 33, // [ZK_BROKER, BROKER, CONTROLLER]
+ SASL_AUTHENTICATE = 36, // [ZK_BROKER, BROKER, CONTROLLER]
+ CREATE_PARTITIONS = 37, // [ZK_BROKER, BROKER, CONTROLLER]
};
extern const std::unordered_map<EApiKey, TString> EApiKeyNames;
@@ -49,80 +51,80 @@ extern const std::unordered_map<EApiKey, TString> EApiKeyNames;
class TRequestHeaderData : public TApiMessage {
public:
typedef std::shared_ptr<TRequestHeaderData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 2};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TRequestHeaderData();
~TRequestHeaderData() = default;
-
+
struct RequestApiKeyMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "requestApiKey";
static constexpr const char* About = "The API key of this request.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
RequestApiKeyMeta::Type RequestApiKey;
-
+
struct RequestApiVersionMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "requestApiVersion";
static constexpr const char* About = "The API version of this request.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
RequestApiVersionMeta::Type RequestApiVersion;
-
+
struct CorrelationIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "correlationId";
static constexpr const char* About = "The correlation ID of this request.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
CorrelationIdMeta::Type CorrelationId;
-
+
struct ClientIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "clientId";
static constexpr const char* About = "The client ID string.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsNever;
};
ClientIdMeta::Type ClientId;
-
+
i16 ApiKey() const override { return HEADER; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TRequestHeaderData& other) const = default;
};
@@ -130,35 +132,35 @@ public:
class TResponseHeaderData : public TApiMessage {
public:
typedef std::shared_ptr<TResponseHeaderData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 1};
static constexpr TKafkaVersions FlexibleVersions = {1, Max<TKafkaVersion>()};
};
-
+
TResponseHeaderData();
~TResponseHeaderData() = default;
-
+
struct CorrelationIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "correlationId";
static constexpr const char* About = "The correlation ID of this response.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {1, Max<TKafkaVersion>()};
};
CorrelationIdMeta::Type CorrelationId;
-
+
i16 ApiKey() const override { return HEADER; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TResponseHeaderData& other) const = default;
};
@@ -166,175 +168,175 @@ public:
class TProduceRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TProduceRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 9};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
-
+
TProduceRequestData();
~TProduceRequestData() = default;
-
+
class TTopicProduceData : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 9};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
-
+
TTopicProduceData();
~TTopicProduceData() = default;
-
+
class TPartitionProduceData : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 9};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
-
+
TPartitionProduceData();
~TPartitionProduceData() = default;
-
+
struct IndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "index";
static constexpr const char* About = "The partition index.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
IndexMeta::Type Index;
-
+
struct RecordsMeta {
using Type = TKafkaRecords;
using TypeDesc = NPrivate::TKafkaRecordsDesc;
-
+
static constexpr const char* Name = "records";
static constexpr const char* About = "The record data to be produced.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
RecordsMeta::Type Records;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TPartitionProduceData& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct PartitionDataMeta {
using ItemType = TPartitionProduceData;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TPartitionProduceData>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitionData";
static constexpr const char* About = "Each partition to produce to.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
PartitionDataMeta::Type PartitionData;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TTopicProduceData& other) const = default;
};
-
+
struct TransactionalIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "transactionalId";
static constexpr const char* About = "The transactional ID, or null if the producer is not transactional.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
TransactionalIdMeta::Type TransactionalId;
-
+
struct AcksMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "acks";
static constexpr const char* About = "The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
AcksMeta::Type Acks;
-
+
struct TimeoutMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "timeoutMs";
static constexpr const char* About = "The timeout to await a response in milliseconds.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
TimeoutMsMeta::Type TimeoutMs;
-
+
struct TopicDataMeta {
using ItemType = TTopicProduceData;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TTopicProduceData>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topicData";
static constexpr const char* About = "Each topic to produce to.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
TopicDataMeta::Type TopicData;
-
+
i16 ApiKey() const override { return PRODUCE; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TProduceRequestData& other) const = default;
};
@@ -342,269 +344,269 @@ public:
class TProduceResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TProduceResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 9};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
-
+
TProduceResponseData();
~TProduceResponseData() = default;
-
+
class TTopicProduceResponse : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 9};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
-
+
TTopicProduceResponse();
~TTopicProduceResponse() = default;
-
+
class TPartitionProduceResponse : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 9};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
-
+
TPartitionProduceResponse();
~TPartitionProduceResponse() = default;
-
+
class TBatchIndexAndErrorMessage : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {8, 9};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
-
+
TBatchIndexAndErrorMessage();
~TBatchIndexAndErrorMessage() = default;
-
+
struct BatchIndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "batchIndex";
static constexpr const char* About = "The batch index of the record that cause the batch to be dropped";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
BatchIndexMeta::Type BatchIndex;
-
+
struct BatchIndexErrorMessageMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "batchIndexErrorMessage";
static constexpr const char* About = "The error message of the record that caused the batch to be dropped";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
BatchIndexErrorMessageMeta::Type BatchIndexErrorMessage;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TBatchIndexAndErrorMessage& other) const = default;
};
-
+
struct IndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "index";
static constexpr const char* About = "The partition index.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
IndexMeta::Type Index;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct BaseOffsetMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "baseOffset";
static constexpr const char* About = "The base offset.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
BaseOffsetMeta::Type BaseOffset;
-
+
struct LogAppendTimeMsMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "logAppendTimeMs";
static constexpr const char* About = "The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {2, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
LogAppendTimeMsMeta::Type LogAppendTimeMs;
-
+
struct LogStartOffsetMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "logStartOffset";
static constexpr const char* About = "The log start offset.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
LogStartOffsetMeta::Type LogStartOffset;
-
+
struct RecordErrorsMeta {
using ItemType = TBatchIndexAndErrorMessage;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TBatchIndexAndErrorMessage>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "recordErrors";
static constexpr const char* About = "The batch indices of records that caused the batch to be dropped";
-
+
static constexpr TKafkaVersions PresentVersions = {8, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
RecordErrorsMeta::Type RecordErrors;
-
+
struct ErrorMessageMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "errorMessage";
static constexpr const char* About = "The global error message summarizing the common root cause of the records that caused the batch to be dropped";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {8, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
ErrorMessageMeta::Type ErrorMessage;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TPartitionProduceResponse& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct PartitionResponsesMeta {
using ItemType = TPartitionProduceResponse;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TPartitionProduceResponse>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitionResponses";
static constexpr const char* About = "Each partition that we produced to within the topic.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
PartitionResponsesMeta::Type PartitionResponses;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TTopicProduceResponse& other) const = default;
};
-
+
struct ResponsesMeta {
using ItemType = TTopicProduceResponse;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TTopicProduceResponse>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "responses";
static constexpr const char* About = "Each produce response";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
ResponsesMeta::Type Responses;
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
i16 ApiKey() const override { return PRODUCE; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TProduceResponseData& other) const = default;
};
@@ -612,421 +614,421 @@ public:
class TFetchRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TFetchRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 13};
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
-
+
TFetchRequestData();
~TFetchRequestData() = default;
-
+
class TFetchTopic : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 13};
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
-
+
TFetchTopic();
~TFetchTopic() = default;
-
+
class TFetchPartition : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 13};
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
-
+
TFetchPartition();
~TFetchPartition() = default;
-
+
struct PartitionMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "partition";
static constexpr const char* About = "The partition index.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
PartitionMeta::Type Partition;
-
+
struct CurrentLeaderEpochMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "currentLeaderEpoch";
static constexpr const char* About = "The current leader epoch of the partition.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {9, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
CurrentLeaderEpochMeta::Type CurrentLeaderEpoch;
-
+
struct FetchOffsetMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "fetchOffset";
static constexpr const char* About = "The message offset.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
FetchOffsetMeta::Type FetchOffset;
-
+
struct LastFetchedEpochMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "lastFetchedEpoch";
static constexpr const char* About = "The epoch of the last fetched record or -1 if there is none";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {12, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
LastFetchedEpochMeta::Type LastFetchedEpoch;
-
+
struct LogStartOffsetMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "logStartOffset";
static constexpr const char* About = "The earliest available offset of the follower replica. The field is only used when the request is sent by the follower.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
LogStartOffsetMeta::Type LogStartOffset;
-
+
struct PartitionMaxBytesMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "partitionMaxBytes";
static constexpr const char* About = "The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
PartitionMaxBytesMeta::Type PartitionMaxBytes;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TFetchPartition& other) const = default;
};
-
+
struct TopicMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "topic";
static constexpr const char* About = "The name of the topic to fetch.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {0, 12};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
TopicMeta::Type Topic;
-
+
struct TopicIdMeta {
using Type = TKafkaUuid;
using TypeDesc = NPrivate::TKafkaUuidDesc;
-
+
static constexpr const char* Name = "topicId";
static constexpr const char* About = "The unique topic ID";
static const Type Default; // = TKafkaUuid(0, 0);
-
+
static constexpr TKafkaVersions PresentVersions = {13, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
TopicIdMeta::Type TopicId;
-
+
struct PartitionsMeta {
using ItemType = TFetchPartition;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TFetchPartition>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitions";
static constexpr const char* About = "The partitions to fetch.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
PartitionsMeta::Type Partitions;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TFetchTopic& other) const = default;
};
-
+
class TForgottenTopic : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {7, 13};
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
-
+
TForgottenTopic();
~TForgottenTopic() = default;
-
+
struct TopicMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "topic";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {0, 12};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
TopicMeta::Type Topic;
-
+
struct TopicIdMeta {
using Type = TKafkaUuid;
using TypeDesc = NPrivate::TKafkaUuidDesc;
-
+
static constexpr const char* Name = "topicId";
static constexpr const char* About = "The unique topic ID";
static const Type Default; // = TKafkaUuid(0, 0);
-
+
static constexpr TKafkaVersions PresentVersions = {13, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
TopicIdMeta::Type TopicId;
-
+
struct PartitionsMeta {
using ItemType = TKafkaInt32;
using ItemTypeDesc = NPrivate::TKafkaIntDesc;
using Type = std::vector<TKafkaInt32>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitions";
static constexpr const char* About = "The partitions indexes to forget.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
PartitionsMeta::Type Partitions;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TForgottenTopic& other) const = default;
};
-
+
struct ClusterIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "clusterId";
static constexpr const char* About = "The clusterId if known. This is used to validate metadata fetches prior to broker registration.";
static constexpr const TKafkaInt32 Tag = 0;
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {12, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsAlways;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ClusterIdMeta::Type ClusterId;
-
+
struct ReplicaIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "replicaId";
static constexpr const char* About = "The broker ID of the follower, of -1 if this request is from a consumer.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
ReplicaIdMeta::Type ReplicaId;
-
+
struct MaxWaitMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "maxWaitMs";
static constexpr const char* About = "The maximum time in milliseconds to wait for the response.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
MaxWaitMsMeta::Type MaxWaitMs;
-
+
struct MinBytesMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "minBytes";
static constexpr const char* About = "The minimum bytes to accumulate in the response.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
MinBytesMeta::Type MinBytes;
-
+
struct MaxBytesMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "maxBytes";
static constexpr const char* About = "The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.";
static const Type Default; // = 0x7fffffff;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
MaxBytesMeta::Type MaxBytes;
-
+
struct IsolationLevelMeta {
using Type = TKafkaInt8;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "isolationLevel";
static constexpr const char* About = "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {4, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
IsolationLevelMeta::Type IsolationLevel;
-
+
struct SessionIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "sessionId";
static constexpr const char* About = "The fetch session ID.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {7, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
SessionIdMeta::Type SessionId;
-
+
struct SessionEpochMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "sessionEpoch";
static constexpr const char* About = "The fetch session epoch, which is used for ordering requests in a session.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {7, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
SessionEpochMeta::Type SessionEpoch;
-
+
struct TopicsMeta {
using ItemType = TFetchTopic;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TFetchTopic>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "The topics to fetch.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
struct ForgottenTopicsDataMeta {
using ItemType = TForgottenTopic;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TForgottenTopic>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "forgottenTopicsData";
static constexpr const char* About = "In an incremental fetch request, the partitions to remove.";
-
+
static constexpr TKafkaVersions PresentVersions = {7, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
ForgottenTopicsDataMeta::Type ForgottenTopicsData;
-
+
struct RackIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "rackId";
static constexpr const char* About = "Rack ID of the consumer making this request";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {11, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
RackIdMeta::Type RackId;
-
+
i16 ApiKey() const override { return FETCH; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TFetchRequestData& other) const = default;
};
@@ -1034,514 +1036,514 @@ public:
class TFetchResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TFetchResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 13};
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
-
+
TFetchResponseData();
~TFetchResponseData() = default;
-
+
class TFetchableTopicResponse : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 13};
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
-
+
TFetchableTopicResponse();
~TFetchableTopicResponse() = default;
-
+
class TPartitionData : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 13};
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
-
+
TPartitionData();
~TPartitionData() = default;
-
+
class TEpochEndOffset : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {12, 13};
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
-
+
TEpochEndOffset();
~TEpochEndOffset() = default;
-
+
struct EpochMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "epoch";
static constexpr const char* About = "";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
EpochMeta::Type Epoch;
-
+
struct EndOffsetMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "endOffset";
static constexpr const char* About = "";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
EndOffsetMeta::Type EndOffset;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TEpochEndOffset& other) const = default;
};
-
+
class TLeaderIdAndEpoch : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {12, 13};
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
-
+
TLeaderIdAndEpoch();
~TLeaderIdAndEpoch() = default;
-
+
struct LeaderIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "leaderId";
static constexpr const char* About = "The ID of the current leader or -1 if the leader is unknown.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
LeaderIdMeta::Type LeaderId;
-
+
struct LeaderEpochMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "leaderEpoch";
static constexpr const char* About = "The latest known leader epoch";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
LeaderEpochMeta::Type LeaderEpoch;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TLeaderIdAndEpoch& other) const = default;
};
-
+
class TSnapshotId : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {12, 13};
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
-
+
TSnapshotId();
~TSnapshotId() = default;
-
+
struct EndOffsetMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "endOffset";
static constexpr const char* About = "";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
EndOffsetMeta::Type EndOffset;
-
+
struct EpochMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "epoch";
static constexpr const char* About = "";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
EpochMeta::Type Epoch;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TSnapshotId& other) const = default;
};
-
+
class TAbortedTransaction : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {4, 13};
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
-
+
TAbortedTransaction();
~TAbortedTransaction() = default;
-
+
struct ProducerIdMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "producerId";
static constexpr const char* About = "The producer id associated with the aborted transaction.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
ProducerIdMeta::Type ProducerId;
-
+
struct FirstOffsetMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "firstOffset";
static constexpr const char* About = "The first offset in the aborted transaction.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
FirstOffsetMeta::Type FirstOffset;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TAbortedTransaction& other) const = default;
};
-
+
struct PartitionIndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "partitionIndex";
static constexpr const char* About = "The partition index.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
PartitionIndexMeta::Type PartitionIndex;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no fetch error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct HighWatermarkMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "highWatermark";
static constexpr const char* About = "The current high water mark.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
HighWatermarkMeta::Type HighWatermark;
-
+
struct LastStableOffsetMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "lastStableOffset";
static constexpr const char* About = "The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {4, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
LastStableOffsetMeta::Type LastStableOffset;
-
+
struct LogStartOffsetMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "logStartOffset";
static constexpr const char* About = "The current log start offset.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
LogStartOffsetMeta::Type LogStartOffset;
-
+
struct DivergingEpochMeta {
using Type = TEpochEndOffset;
using TypeDesc = NPrivate::TKafkaStructDesc;
-
+
static constexpr const char* Name = "divergingEpoch";
static constexpr const char* About = "In case divergence is detected based on the `LastFetchedEpoch` and `FetchOffset` in the request, this field indicates the largest epoch and its end offset such that subsequent records are known to diverge";
static constexpr const TKafkaInt32 Tag = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {12, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsAlways;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
DivergingEpochMeta::Type DivergingEpoch;
-
+
struct CurrentLeaderMeta {
using Type = TLeaderIdAndEpoch;
using TypeDesc = NPrivate::TKafkaStructDesc;
-
+
static constexpr const char* Name = "currentLeader";
static constexpr const char* About = "";
static constexpr const TKafkaInt32 Tag = 1;
-
+
static constexpr TKafkaVersions PresentVersions = {12, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsAlways;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
CurrentLeaderMeta::Type CurrentLeader;
-
+
struct SnapshotIdMeta {
using Type = TSnapshotId;
using TypeDesc = NPrivate::TKafkaStructDesc;
-
+
static constexpr const char* Name = "snapshotId";
static constexpr const char* About = "In the case of fetching an offset less than the LogStartOffset, this is the end offset and epoch that should be used in the FetchSnapshot request.";
static constexpr const TKafkaInt32 Tag = 2;
-
+
static constexpr TKafkaVersions PresentVersions = {12, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsAlways;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
SnapshotIdMeta::Type SnapshotId;
-
+
struct AbortedTransactionsMeta {
using ItemType = TAbortedTransaction;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TAbortedTransaction>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "abortedTransactions";
static constexpr const char* About = "The aborted transactions.";
-
+
static constexpr TKafkaVersions PresentVersions = {4, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
AbortedTransactionsMeta::Type AbortedTransactions;
-
+
struct PreferredReadReplicaMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "preferredReadReplica";
static constexpr const char* About = "The preferred read replica for the consumer to use on its next fetch request";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {11, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
PreferredReadReplicaMeta::Type PreferredReadReplica;
-
+
struct RecordsMeta {
using Type = TKafkaRecords;
using TypeDesc = NPrivate::TKafkaRecordsDesc;
-
+
static constexpr const char* Name = "records";
static constexpr const char* About = "The record data.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
RecordsMeta::Type Records;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TPartitionData& other) const = default;
};
-
+
struct TopicMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "topic";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {0, 12};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
TopicMeta::Type Topic;
-
+
struct TopicIdMeta {
using Type = TKafkaUuid;
using TypeDesc = NPrivate::TKafkaUuidDesc;
-
+
static constexpr const char* Name = "topicId";
static constexpr const char* About = "The unique topic ID";
static const Type Default; // = TKafkaUuid(0, 0);
-
+
static constexpr TKafkaVersions PresentVersions = {13, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
TopicIdMeta::Type TopicId;
-
+
struct PartitionsMeta {
using ItemType = TPartitionData;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TPartitionData>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitions";
static constexpr const char* About = "The topic partitions.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
PartitionsMeta::Type Partitions;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TFetchableTopicResponse& other) const = default;
};
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The top level response error code.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {7, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct SessionIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "sessionId";
static constexpr const char* About = "The fetch session ID, or 0 if this is not part of a fetch session.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {7, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
SessionIdMeta::Type SessionId;
-
+
struct ResponsesMeta {
using ItemType = TFetchableTopicResponse;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TFetchableTopicResponse>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "responses";
static constexpr const char* About = "The response topics.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {12, Max<TKafkaVersion>()};
};
ResponsesMeta::Type Responses;
-
+
i16 ApiKey() const override { return FETCH; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TFetchResponseData& other) const = default;
};
@@ -1549,191 +1551,191 @@ public:
class TListOffsetsRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TListOffsetsRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TListOffsetsRequestData();
~TListOffsetsRequestData() = default;
-
+
class TListOffsetsTopic : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TListOffsetsTopic();
~TListOffsetsTopic() = default;
-
+
class TListOffsetsPartition : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TListOffsetsPartition();
~TListOffsetsPartition() = default;
-
+
struct PartitionIndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "partitionIndex";
static constexpr const char* About = "The partition index.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
PartitionIndexMeta::Type PartitionIndex;
-
+
struct CurrentLeaderEpochMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "currentLeaderEpoch";
static constexpr const char* About = "The current leader epoch.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {4, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
CurrentLeaderEpochMeta::Type CurrentLeaderEpoch;
-
+
struct TimestampMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "timestamp";
static constexpr const char* About = "The current timestamp.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
TimestampMeta::Type Timestamp;
-
+
struct MaxNumOffsetsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "maxNumOffsets";
static constexpr const char* About = "The maximum number of offsets to report.";
static const Type Default; // = 1;
-
+
static constexpr TKafkaVersions PresentVersions = {0, 0};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
MaxNumOffsetsMeta::Type MaxNumOffsets;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TListOffsetsPartition& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct PartitionsMeta {
using ItemType = TListOffsetsPartition;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TListOffsetsPartition>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitions";
static constexpr const char* About = "Each partition in the request.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
PartitionsMeta::Type Partitions;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TListOffsetsTopic& other) const = default;
};
-
+
struct ReplicaIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "replicaId";
static constexpr const char* About = "The broker ID of the requestor, or -1 if this request is being made by a normal consumer.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
ReplicaIdMeta::Type ReplicaId;
-
+
struct IsolationLevelMeta {
using Type = TKafkaInt8;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "isolationLevel";
static constexpr const char* About = "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {2, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
IsolationLevelMeta::Type IsolationLevel;
-
+
struct TopicsMeta {
using ItemType = TListOffsetsTopic;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TListOffsetsTopic>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "Each topic in the request.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
i16 ApiKey() const override { return LIST_OFFSETS; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TListOffsetsRequestData& other) const = default;
};
@@ -1741,207 +1743,207 @@ public:
class TListOffsetsResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TListOffsetsResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TListOffsetsResponseData();
~TListOffsetsResponseData() = default;
-
+
class TListOffsetsTopicResponse : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TListOffsetsTopicResponse();
~TListOffsetsTopicResponse() = default;
-
+
class TListOffsetsPartitionResponse : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TListOffsetsPartitionResponse();
~TListOffsetsPartitionResponse() = default;
-
+
struct PartitionIndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "partitionIndex";
static constexpr const char* About = "The partition index.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
PartitionIndexMeta::Type PartitionIndex;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The partition error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct OldStyleOffsetsMeta {
using ItemType = TKafkaInt64;
using ItemTypeDesc = NPrivate::TKafkaIntDesc;
using Type = std::vector<TKafkaInt64>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "oldStyleOffsets";
static constexpr const char* About = "The result offsets.";
-
+
static constexpr TKafkaVersions PresentVersions = {0, 0};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
OldStyleOffsetsMeta::Type OldStyleOffsets;
-
+
struct TimestampMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "timestamp";
static constexpr const char* About = "The timestamp associated with the returned offset.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
TimestampMeta::Type Timestamp;
-
+
struct OffsetMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "offset";
static constexpr const char* About = "The returned offset.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
OffsetMeta::Type Offset;
-
+
struct LeaderEpochMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "leaderEpoch";
static constexpr const char* About = "";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {4, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
LeaderEpochMeta::Type LeaderEpoch;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TListOffsetsPartitionResponse& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct PartitionsMeta {
using ItemType = TListOffsetsPartitionResponse;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TListOffsetsPartitionResponse>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitions";
static constexpr const char* About = "Each partition in the response.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
PartitionsMeta::Type Partitions;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TListOffsetsTopicResponse& other) const = default;
};
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {2, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct TopicsMeta {
using ItemType = TListOffsetsTopicResponse;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TListOffsetsTopicResponse>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "Each topic in the response.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
i16 ApiKey() const override { return LIST_OFFSETS; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TListOffsetsResponseData& other) const = default;
};
@@ -1949,128 +1951,128 @@ public:
class TMetadataRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TMetadataRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 12};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
-
+
TMetadataRequestData();
~TMetadataRequestData() = default;
-
+
class TMetadataRequestTopic : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 12};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
-
+
TMetadataRequestTopic();
~TMetadataRequestTopic() = default;
-
+
struct TopicIdMeta {
using Type = TKafkaUuid;
using TypeDesc = NPrivate::TKafkaUuidDesc;
-
+
static constexpr const char* Name = "topicId";
static constexpr const char* About = "The topic id.";
static const Type Default; // = TKafkaUuid(0, 0);
-
+
static constexpr TKafkaVersions PresentVersions = {10, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
TopicIdMeta::Type TopicId;
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = {10, Max<TKafkaVersion>()};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TMetadataRequestTopic& other) const = default;
};
-
+
struct TopicsMeta {
using ItemType = TMetadataRequestTopic;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TMetadataRequestTopic>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "The topics to fetch metadata for.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
struct AllowAutoTopicCreationMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "allowAutoTopicCreation";
static constexpr const char* About = "If this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.";
static const Type Default; // = true;
-
+
static constexpr TKafkaVersions PresentVersions = {4, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
AllowAutoTopicCreationMeta::Type AllowAutoTopicCreation;
-
+
struct IncludeClusterAuthorizedOperationsMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "includeClusterAuthorizedOperations";
static constexpr const char* About = "Whether to include cluster authorized operations.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = {8, 10};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
IncludeClusterAuthorizedOperationsMeta::Type IncludeClusterAuthorizedOperations;
-
+
struct IncludeTopicAuthorizedOperationsMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "includeTopicAuthorizedOperations";
static constexpr const char* About = "Whether to include topic authorized operations.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = {8, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
IncludeTopicAuthorizedOperationsMeta::Type IncludeTopicAuthorizedOperations;
-
+
i16 ApiKey() const override { return METADATA; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TMetadataRequestData& other) const = default;
};
@@ -2078,422 +2080,422 @@ public:
class TMetadataResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TMetadataResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 12};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
-
+
TMetadataResponseData();
~TMetadataResponseData() = default;
-
+
class TMetadataResponseBroker : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 12};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
-
+
TMetadataResponseBroker();
~TMetadataResponseBroker() = default;
-
+
struct NodeIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "nodeId";
static constexpr const char* About = "The broker ID.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
NodeIdMeta::Type NodeId;
-
+
struct HostMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "host";
static constexpr const char* About = "The broker hostname.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
HostMeta::Type Host;
-
+
struct PortMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "port";
static constexpr const char* About = "The broker port.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
PortMeta::Type Port;
-
+
struct RackMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "rack";
static constexpr const char* About = "The rack of the broker, or null if it has not been assigned to a rack.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
RackMeta::Type Rack;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TMetadataResponseBroker& other) const = default;
};
-
+
class TMetadataResponseTopic : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 12};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
-
+
TMetadataResponseTopic();
~TMetadataResponseTopic() = default;
-
+
class TMetadataResponsePartition : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 12};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
-
+
TMetadataResponsePartition();
~TMetadataResponsePartition() = default;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The partition error, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct PartitionIndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "partitionIndex";
static constexpr const char* About = "The partition index.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
PartitionIndexMeta::Type PartitionIndex;
-
+
struct LeaderIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "leaderId";
static constexpr const char* About = "The ID of the leader broker.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
LeaderIdMeta::Type LeaderId;
-
+
struct LeaderEpochMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "leaderEpoch";
static constexpr const char* About = "The leader epoch of this partition.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {7, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
LeaderEpochMeta::Type LeaderEpoch;
-
+
struct ReplicaNodesMeta {
using ItemType = TKafkaInt32;
using ItemTypeDesc = NPrivate::TKafkaIntDesc;
using Type = std::vector<TKafkaInt32>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "replicaNodes";
static constexpr const char* About = "The set of all nodes that host this partition.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
ReplicaNodesMeta::Type ReplicaNodes;
-
+
struct IsrNodesMeta {
using ItemType = TKafkaInt32;
using ItemTypeDesc = NPrivate::TKafkaIntDesc;
using Type = std::vector<TKafkaInt32>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "isrNodes";
static constexpr const char* About = "The set of nodes that are in sync with the leader for this partition.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
IsrNodesMeta::Type IsrNodes;
-
+
struct OfflineReplicasMeta {
using ItemType = TKafkaInt32;
using ItemTypeDesc = NPrivate::TKafkaIntDesc;
using Type = std::vector<TKafkaInt32>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "offlineReplicas";
static constexpr const char* About = "The set of offline replicas of this partition.";
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
OfflineReplicasMeta::Type OfflineReplicas;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TMetadataResponsePartition& other) const = default;
};
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The topic error, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = {12, Max<TKafkaVersion>()};
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct TopicIdMeta {
using Type = TKafkaUuid;
using TypeDesc = NPrivate::TKafkaUuidDesc;
-
+
static constexpr const char* Name = "topicId";
static constexpr const char* About = "The topic id.";
static const Type Default; // = TKafkaUuid(0, 0);
-
+
static constexpr TKafkaVersions PresentVersions = {10, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
TopicIdMeta::Type TopicId;
-
+
struct IsInternalMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "isInternal";
static constexpr const char* About = "True if the topic is internal.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
IsInternalMeta::Type IsInternal;
-
+
struct PartitionsMeta {
using ItemType = TMetadataResponsePartition;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TMetadataResponsePartition>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitions";
static constexpr const char* About = "Each partition in the topic.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
PartitionsMeta::Type Partitions;
-
+
struct TopicAuthorizedOperationsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "topicAuthorizedOperations";
static constexpr const char* About = "32-bit bitfield to represent authorized operations for this topic.";
static const Type Default; // = -2147483648;
-
+
static constexpr TKafkaVersions PresentVersions = {8, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
TopicAuthorizedOperationsMeta::Type TopicAuthorizedOperations;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TMetadataResponseTopic& other) const = default;
};
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct BrokersMeta {
using ItemType = TMetadataResponseBroker;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TMetadataResponseBroker>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "brokers";
static constexpr const char* About = "Each broker in the response.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
BrokersMeta::Type Brokers;
-
+
struct ClusterIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "clusterId";
static constexpr const char* About = "The cluster ID that responding broker belongs to.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {2, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
ClusterIdMeta::Type ClusterId;
-
+
struct ControllerIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "controllerId";
static constexpr const char* About = "The ID of the controller broker.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
ControllerIdMeta::Type ControllerId;
-
+
struct TopicsMeta {
using ItemType = TMetadataResponseTopic;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TMetadataResponseTopic>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "Each topic in the response.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
struct ClusterAuthorizedOperationsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "clusterAuthorizedOperations";
static constexpr const char* About = "32-bit bitfield to represent authorized operations for this cluster.";
static const Type Default; // = -2147483648;
-
+
static constexpr TKafkaVersions PresentVersions = {8, 10};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {9, Max<TKafkaVersion>()};
};
ClusterAuthorizedOperationsMeta::Type ClusterAuthorizedOperations;
-
+
i16 ApiKey() const override { return METADATA; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TMetadataResponseData& other) const = default;
};
@@ -2501,251 +2503,251 @@ public:
class TOffsetCommitRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TOffsetCommitRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 8};
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
-
+
TOffsetCommitRequestData();
~TOffsetCommitRequestData() = default;
-
+
class TOffsetCommitRequestTopic : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 8};
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
-
+
TOffsetCommitRequestTopic();
~TOffsetCommitRequestTopic() = default;
-
+
class TOffsetCommitRequestPartition : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 8};
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
-
+
TOffsetCommitRequestPartition();
~TOffsetCommitRequestPartition() = default;
-
+
struct PartitionIndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "partitionIndex";
static constexpr const char* About = "The partition index.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
PartitionIndexMeta::Type PartitionIndex;
-
+
struct CommittedOffsetMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "committedOffset";
static constexpr const char* About = "The message offset to be committed.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
CommittedOffsetMeta::Type CommittedOffset;
-
+
struct CommittedLeaderEpochMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "committedLeaderEpoch";
static constexpr const char* About = "The leader epoch of this partition.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {6, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
CommittedLeaderEpochMeta::Type CommittedLeaderEpoch;
-
+
struct CommitTimestampMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "commitTimestamp";
static constexpr const char* About = "The timestamp of the commit.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {1, 1};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
CommitTimestampMeta::Type CommitTimestamp;
-
+
struct CommittedMetadataMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "committedMetadata";
static constexpr const char* About = "Any associated metadata the client wants to keep.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
CommittedMetadataMeta::Type CommittedMetadata;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetCommitRequestPartition& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct PartitionsMeta {
using ItemType = TOffsetCommitRequestPartition;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TOffsetCommitRequestPartition>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitions";
static constexpr const char* About = "Each partition to commit offsets for.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
PartitionsMeta::Type Partitions;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetCommitRequestTopic& other) const = default;
};
-
+
struct GroupIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupId";
static constexpr const char* About = "The unique group identifier.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
GroupIdMeta::Type GroupId;
-
+
struct GenerationIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "generationId";
static constexpr const char* About = "The generation of the group.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
GenerationIdMeta::Type GenerationId;
-
+
struct MemberIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "memberId";
static constexpr const char* About = "The member ID assigned by the group coordinator.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
MemberIdMeta::Type MemberId;
-
+
struct GroupInstanceIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupInstanceId";
static constexpr const char* About = "The unique identifier of the consumer instance provided by end user.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {7, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
GroupInstanceIdMeta::Type GroupInstanceId;
-
+
struct RetentionTimeMsMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "retentionTimeMs";
static constexpr const char* About = "The time period in ms to retain the offset.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {2, 4};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
RetentionTimeMsMeta::Type RetentionTimeMs;
-
+
struct TopicsMeta {
using ItemType = TOffsetCommitRequestTopic;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TOffsetCommitRequestTopic>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "The topics to commit offsets for.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
i16 ApiKey() const override { return OFFSET_COMMIT; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetCommitRequestData& other) const = default;
};
@@ -2753,146 +2755,146 @@ public:
class TOffsetCommitResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TOffsetCommitResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 8};
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
-
+
TOffsetCommitResponseData();
~TOffsetCommitResponseData() = default;
-
+
class TOffsetCommitResponseTopic : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 8};
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
-
+
TOffsetCommitResponseTopic();
~TOffsetCommitResponseTopic() = default;
-
+
class TOffsetCommitResponsePartition : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 8};
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
-
+
TOffsetCommitResponsePartition();
~TOffsetCommitResponsePartition() = default;
-
+
struct PartitionIndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "partitionIndex";
static constexpr const char* About = "The partition index.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
PartitionIndexMeta::Type PartitionIndex;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetCommitResponsePartition& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct PartitionsMeta {
using ItemType = TOffsetCommitResponsePartition;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TOffsetCommitResponsePartition>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitions";
static constexpr const char* About = "The responses for each partition in the topic.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
PartitionsMeta::Type Partitions;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetCommitResponseTopic& other) const = default;
};
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct TopicsMeta {
using ItemType = TOffsetCommitResponseTopic;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TOffsetCommitResponseTopic>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "The responses for each topic.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {8, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
i16 ApiKey() const override { return OFFSET_COMMIT; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetCommitResponseData& other) const = default;
};
@@ -2900,226 +2902,226 @@ public:
class TOffsetFetchRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TOffsetFetchRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 8};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TOffsetFetchRequestData();
~TOffsetFetchRequestData() = default;
-
+
class TOffsetFetchRequestTopic : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TOffsetFetchRequestTopic();
~TOffsetFetchRequestTopic() = default;
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct PartitionIndexesMeta {
using ItemType = TKafkaInt32;
using ItemTypeDesc = NPrivate::TKafkaIntDesc;
using Type = std::vector<TKafkaInt32>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitionIndexes";
static constexpr const char* About = "The partition indexes we would like to fetch offsets for.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
PartitionIndexesMeta::Type PartitionIndexes;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetFetchRequestTopic& other) const = default;
};
-
+
class TOffsetFetchRequestGroup : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {8, 8};
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
-
+
TOffsetFetchRequestGroup();
~TOffsetFetchRequestGroup() = default;
-
+
class TOffsetFetchRequestTopics : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {8, 8};
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
-
+
TOffsetFetchRequestTopics();
~TOffsetFetchRequestTopics() = default;
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
NameMeta::Type Name;
-
+
struct PartitionIndexesMeta {
using ItemType = TKafkaInt32;
using ItemTypeDesc = NPrivate::TKafkaIntDesc;
using Type = std::vector<TKafkaInt32>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitionIndexes";
static constexpr const char* About = "The partition indexes we would like to fetch offsets for.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
PartitionIndexesMeta::Type PartitionIndexes;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetFetchRequestTopics& other) const = default;
};
-
+
struct GroupIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupId";
static constexpr const char* About = "The group ID.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
GroupIdMeta::Type GroupId;
-
+
struct TopicsMeta {
using ItemType = TOffsetFetchRequestTopics;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TOffsetFetchRequestTopics>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "Each topic we would like to fetch offsets for, or null to fetch offsets for all topics.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
TopicsMeta::Type Topics;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetFetchRequestGroup& other) const = default;
};
-
+
struct GroupIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupId";
static constexpr const char* About = "The group to fetch offsets for.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
GroupIdMeta::Type GroupId;
-
+
struct TopicsMeta {
using ItemType = TOffsetFetchRequestTopic;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TOffsetFetchRequestTopic>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "Each topic we would like to fetch offsets for, or null to fetch offsets for all topics.";
-
+
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = {2, Max<TKafkaVersion>()};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
struct GroupsMeta {
using ItemType = TOffsetFetchRequestGroup;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TOffsetFetchRequestGroup>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "groups";
static constexpr const char* About = "Each group we would like to fetch offsets for";
-
+
static constexpr TKafkaVersions PresentVersions = {8, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
GroupsMeta::Type Groups;
-
+
struct RequireStableMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "requireStable";
static constexpr const char* About = "Whether broker should hold on returning unstable offsets but set a retriable error code for the partitions.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = {7, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
RequireStableMeta::Type RequireStable;
-
+
i16 ApiKey() const override { return OFFSET_FETCH; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetFetchRequestData& other) const = default;
};
@@ -3127,425 +3129,425 @@ public:
class TOffsetFetchResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TOffsetFetchResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 8};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TOffsetFetchResponseData();
~TOffsetFetchResponseData() = default;
-
+
class TOffsetFetchResponseTopic : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TOffsetFetchResponseTopic();
~TOffsetFetchResponseTopic() = default;
-
+
class TOffsetFetchResponsePartition : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TOffsetFetchResponsePartition();
~TOffsetFetchResponsePartition() = default;
-
+
struct PartitionIndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "partitionIndex";
static constexpr const char* About = "The partition index.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
PartitionIndexMeta::Type PartitionIndex;
-
+
struct CommittedOffsetMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "committedOffset";
static constexpr const char* About = "The committed message offset.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
CommittedOffsetMeta::Type CommittedOffset;
-
+
struct CommittedLeaderEpochMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "committedLeaderEpoch";
static constexpr const char* About = "The leader epoch.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
CommittedLeaderEpochMeta::Type CommittedLeaderEpoch;
-
+
struct MetadataMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "metadata";
static constexpr const char* About = "The partition metadata.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
MetadataMeta::Type Metadata;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetFetchResponsePartition& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct PartitionsMeta {
using ItemType = TOffsetFetchResponsePartition;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TOffsetFetchResponsePartition>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitions";
static constexpr const char* About = "The responses per partition";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
PartitionsMeta::Type Partitions;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetFetchResponseTopic& other) const = default;
};
-
+
class TOffsetFetchResponseGroup : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {8, 8};
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
-
+
TOffsetFetchResponseGroup();
~TOffsetFetchResponseGroup() = default;
-
+
class TOffsetFetchResponseTopics : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {8, 8};
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
-
+
TOffsetFetchResponseTopics();
~TOffsetFetchResponseTopics() = default;
-
+
class TOffsetFetchResponsePartitions : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {8, 8};
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
-
+
TOffsetFetchResponsePartitions();
~TOffsetFetchResponsePartitions() = default;
-
+
struct PartitionIndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "partitionIndex";
static constexpr const char* About = "The partition index.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
PartitionIndexMeta::Type PartitionIndex;
-
+
struct CommittedOffsetMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "committedOffset";
static constexpr const char* About = "The committed message offset.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
CommittedOffsetMeta::Type CommittedOffset;
-
+
struct CommittedLeaderEpochMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "committedLeaderEpoch";
static constexpr const char* About = "The leader epoch.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
CommittedLeaderEpochMeta::Type CommittedLeaderEpoch;
-
+
struct MetadataMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "metadata";
static constexpr const char* About = "The partition metadata.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
MetadataMeta::Type Metadata;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The partition-level error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ErrorCodeMeta::Type ErrorCode;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetFetchResponsePartitions& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
NameMeta::Type Name;
-
+
struct PartitionsMeta {
using ItemType = TOffsetFetchResponsePartitions;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TOffsetFetchResponsePartitions>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitions";
static constexpr const char* About = "The responses per partition";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
PartitionsMeta::Type Partitions;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetFetchResponseTopics& other) const = default;
};
-
+
struct GroupIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupId";
static constexpr const char* About = "The group ID.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
GroupIdMeta::Type GroupId;
-
+
struct TopicsMeta {
using ItemType = TOffsetFetchResponseTopics;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TOffsetFetchResponseTopics>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "The responses per topic.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
TopicsMeta::Type Topics;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The group-level error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ErrorCodeMeta::Type ErrorCode;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetFetchResponseGroup& other) const = default;
};
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct TopicsMeta {
using ItemType = TOffsetFetchResponseTopic;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TOffsetFetchResponseTopic>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "The responses per topic.";
-
+
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The top-level error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {2, 7};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct GroupsMeta {
using ItemType = TOffsetFetchResponseGroup;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TOffsetFetchResponseGroup>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "groups";
static constexpr const char* About = "The responses per group id.";
-
+
static constexpr TKafkaVersions PresentVersions = {8, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
GroupsMeta::Type Groups;
-
+
i16 ApiKey() const override { return OFFSET_FETCH; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TOffsetFetchResponseData& other) const = default;
};
@@ -3553,66 +3555,66 @@ public:
class TFindCoordinatorRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TFindCoordinatorRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 4};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TFindCoordinatorRequestData();
~TFindCoordinatorRequestData() = default;
-
+
struct KeyMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "key";
static constexpr const char* About = "The coordinator key.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
KeyMeta::Type Key;
-
+
struct KeyTypeMeta {
using Type = TKafkaInt8;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "keyType";
static constexpr const char* About = "The coordinator key type. (Group, transaction, etc.)";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
KeyTypeMeta::Type KeyType;
-
+
struct CoordinatorKeysMeta {
using ItemType = TKafkaString;
using ItemTypeDesc = NPrivate::TKafkaStringDesc;
using Type = std::vector<TKafkaString>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "coordinatorKeys";
static constexpr const char* About = "The coordinator keys.";
-
+
static constexpr TKafkaVersions PresentVersions = {4, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
CoordinatorKeysMeta::Type CoordinatorKeys;
-
+
i16 ApiKey() const override { return FIND_COORDINATOR; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TFindCoordinatorRequestData& other) const = default;
};
@@ -3620,233 +3622,233 @@ public:
class TFindCoordinatorResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TFindCoordinatorResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 4};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TFindCoordinatorResponseData();
~TFindCoordinatorResponseData() = default;
-
+
class TCoordinator : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {4, 4};
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
-
+
TCoordinator();
~TCoordinator() = default;
-
+
struct KeyMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "key";
static constexpr const char* About = "The coordinator key.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
KeyMeta::Type Key;
-
+
struct NodeIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "nodeId";
static constexpr const char* About = "The node id.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
NodeIdMeta::Type NodeId;
-
+
struct HostMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "host";
static constexpr const char* About = "The host name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
HostMeta::Type Host;
-
+
struct PortMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "port";
static constexpr const char* About = "The port.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
PortMeta::Type Port;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct ErrorMessageMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "errorMessage";
static constexpr const char* About = "The error message, or null if there was no error.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ErrorMessageMeta::Type ErrorMessage;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TCoordinator& other) const = default;
};
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct ErrorMessageMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "errorMessage";
static constexpr const char* About = "The error message, or null if there was no error.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {1, 3};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ErrorMessageMeta::Type ErrorMessage;
-
+
struct NodeIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "nodeId";
static constexpr const char* About = "The node id.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
NodeIdMeta::Type NodeId;
-
+
struct HostMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "host";
static constexpr const char* About = "The host name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
HostMeta::Type Host;
-
+
struct PortMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "port";
static constexpr const char* About = "The port.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
PortMeta::Type Port;
-
+
struct CoordinatorsMeta {
using ItemType = TCoordinator;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TCoordinator>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "coordinators";
static constexpr const char* About = "Each coordinator result in the response";
-
+
static constexpr TKafkaVersions PresentVersions = {4, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
CoordinatorsMeta::Type Coordinators;
-
+
i16 ApiKey() const override { return FIND_COORDINATOR; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TFindCoordinatorResponseData& other) const = default;
};
@@ -3854,187 +3856,187 @@ public:
class TJoinGroupRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TJoinGroupRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 9};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TJoinGroupRequestData();
~TJoinGroupRequestData() = default;
-
+
class TJoinGroupRequestProtocol : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 9};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TJoinGroupRequestProtocol();
~TJoinGroupRequestProtocol() = default;
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The protocol name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct MetadataMeta {
using Type = TKafkaBytes;
using TypeDesc = NPrivate::TKafkaBytesDesc;
-
+
static constexpr const char* Name = "metadata";
static constexpr const char* About = "The protocol metadata.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
MetadataMeta::Type Metadata;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TJoinGroupRequestProtocol& other) const = default;
};
-
+
struct GroupIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupId";
static constexpr const char* About = "The group identifier.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
GroupIdMeta::Type GroupId;
-
+
struct SessionTimeoutMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "sessionTimeoutMs";
static constexpr const char* About = "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
SessionTimeoutMsMeta::Type SessionTimeoutMs;
-
+
struct RebalanceTimeoutMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "rebalanceTimeoutMs";
static constexpr const char* About = "The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
RebalanceTimeoutMsMeta::Type RebalanceTimeoutMs;
-
+
struct MemberIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "memberId";
static constexpr const char* About = "The member id assigned by the group coordinator.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
MemberIdMeta::Type MemberId;
-
+
struct GroupInstanceIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupInstanceId";
static constexpr const char* About = "The unique identifier of the consumer instance provided by end user.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
GroupInstanceIdMeta::Type GroupInstanceId;
-
+
struct ProtocolTypeMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "protocolType";
static constexpr const char* About = "The unique name the for class of protocols implemented by the group we want to join.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
ProtocolTypeMeta::Type ProtocolType;
-
+
struct ProtocolsMeta {
using ItemType = TJoinGroupRequestProtocol;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TJoinGroupRequestProtocol>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "protocols";
static constexpr const char* About = "The list of protocols that the member supports.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
ProtocolsMeta::Type Protocols;
-
+
struct ReasonMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "reason";
static constexpr const char* About = "The reason why the member (re-)joins the group.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {8, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ReasonMeta::Type Reason;
-
+
i16 ApiKey() const override { return JOIN_GROUP; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TJoinGroupRequestData& other) const = default;
};
@@ -4042,219 +4044,219 @@ public:
class TJoinGroupResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TJoinGroupResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 9};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TJoinGroupResponseData();
~TJoinGroupResponseData() = default;
-
+
class TJoinGroupResponseMember : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 9};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
-
+
TJoinGroupResponseMember();
~TJoinGroupResponseMember() = default;
-
+
struct MemberIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "memberId";
static constexpr const char* About = "The group member ID.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
MemberIdMeta::Type MemberId;
-
+
struct GroupInstanceIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupInstanceId";
static constexpr const char* About = "The unique identifier of the consumer instance provided by end user.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
GroupInstanceIdMeta::Type GroupInstanceId;
-
+
struct MetadataMeta {
using Type = TKafkaBytes;
using TypeDesc = NPrivate::TKafkaBytesDesc;
-
+
static constexpr const char* Name = "metadata";
static constexpr const char* About = "The group member metadata.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
MetadataMeta::Type Metadata;
-
+
TString MetaStr;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TJoinGroupResponseMember& other) const = default;
};
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {2, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct GenerationIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "generationId";
static constexpr const char* About = "The generation ID of the group.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
GenerationIdMeta::Type GenerationId;
-
+
struct ProtocolTypeMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "protocolType";
static constexpr const char* About = "The group protocol name.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {7, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ProtocolTypeMeta::Type ProtocolType;
-
+
struct ProtocolNameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "protocolName";
static constexpr const char* About = "The group protocol selected by the coordinator.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = {7, Max<TKafkaVersion>()};
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
ProtocolNameMeta::Type ProtocolName;
-
+
struct LeaderMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "leader";
static constexpr const char* About = "The leader of the group.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
LeaderMeta::Type Leader;
-
+
struct SkipAssignmentMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "skipAssignment";
static constexpr const char* About = "True if the leader must skip running the assignment.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = {9, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
SkipAssignmentMeta::Type SkipAssignment;
-
+
struct MemberIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "memberId";
static constexpr const char* About = "The member ID assigned by the group coordinator.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
MemberIdMeta::Type MemberId;
-
+
struct MembersMeta {
using ItemType = TJoinGroupResponseMember;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TJoinGroupResponseMember>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "members";
static constexpr const char* About = "";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {6, Max<TKafkaVersion>()};
};
MembersMeta::Type Members;
-
+
i16 ApiKey() const override { return JOIN_GROUP; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TJoinGroupResponseData& other) const = default;
};
@@ -4262,80 +4264,80 @@ public:
class THeartbeatRequestData : public TApiMessage {
public:
typedef std::shared_ptr<THeartbeatRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 4};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
THeartbeatRequestData();
~THeartbeatRequestData() = default;
-
+
struct GroupIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupId";
static constexpr const char* About = "The group id.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
GroupIdMeta::Type GroupId;
-
+
struct GenerationIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "generationId";
static constexpr const char* About = "The generation of the group.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
GenerationIdMeta::Type GenerationId;
-
+
struct MemberIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "memberId";
static constexpr const char* About = "The member ID.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
MemberIdMeta::Type MemberId;
-
+
struct GroupInstanceIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupInstanceId";
static constexpr const char* About = "The unique identifier of the consumer instance provided by end user.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
GroupInstanceIdMeta::Type GroupInstanceId;
-
+
i16 ApiKey() const override { return HEARTBEAT; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const THeartbeatRequestData& other) const = default;
};
@@ -4343,50 +4345,50 @@ public:
class THeartbeatResponseData : public TApiMessage {
public:
typedef std::shared_ptr<THeartbeatResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 4};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
THeartbeatResponseData();
~THeartbeatResponseData() = default;
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
i16 ApiKey() const override { return HEARTBEAT; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const THeartbeatResponseData& other) const = default;
};
@@ -4394,128 +4396,128 @@ public:
class TLeaveGroupRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TLeaveGroupRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 5};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
TLeaveGroupRequestData();
~TLeaveGroupRequestData() = default;
-
+
class TMemberIdentity : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {3, 5};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
TMemberIdentity();
~TMemberIdentity() = default;
-
+
struct MemberIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "memberId";
static constexpr const char* About = "The member ID to remove from the group.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
MemberIdMeta::Type MemberId;
-
+
struct GroupInstanceIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupInstanceId";
static constexpr const char* About = "The group instance ID to remove from the group.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
GroupInstanceIdMeta::Type GroupInstanceId;
-
+
struct ReasonMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "reason";
static constexpr const char* About = "The reason why the member left the group.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ReasonMeta::Type Reason;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TMemberIdentity& other) const = default;
};
-
+
struct GroupIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupId";
static constexpr const char* About = "The ID of the group to leave.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
GroupIdMeta::Type GroupId;
-
+
struct MemberIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "memberId";
static constexpr const char* About = "The member ID to remove from the group.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {0, 2};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
MemberIdMeta::Type MemberId;
-
+
struct MembersMeta {
using ItemType = TMemberIdentity;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TMemberIdentity>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "members";
static constexpr const char* About = "List of leaving member identities.";
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
MembersMeta::Type Members;
-
+
i16 ApiKey() const override { return LEAVE_GROUP; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TLeaveGroupRequestData& other) const = default;
};
@@ -4523,128 +4525,128 @@ public:
class TLeaveGroupResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TLeaveGroupResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 5};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
TLeaveGroupResponseData();
~TLeaveGroupResponseData() = default;
-
+
class TMemberResponse : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {3, 5};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
TMemberResponse();
~TMemberResponse() = default;
-
+
struct MemberIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "memberId";
static constexpr const char* About = "The member ID to remove from the group.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
MemberIdMeta::Type MemberId;
-
+
struct GroupInstanceIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupInstanceId";
static constexpr const char* About = "The group instance ID to remove from the group.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
GroupInstanceIdMeta::Type GroupInstanceId;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TMemberResponse& other) const = default;
};
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct MembersMeta {
using ItemType = TMemberResponse;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TMemberResponse>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "members";
static constexpr const char* About = "List of leaving member responses.";
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
MembersMeta::Type Members;
-
+
i16 ApiKey() const override { return LEAVE_GROUP; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TLeaveGroupResponseData& other) const = default;
};
@@ -4652,174 +4654,174 @@ public:
class TSyncGroupRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TSyncGroupRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 5};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
TSyncGroupRequestData();
~TSyncGroupRequestData() = default;
-
+
class TSyncGroupRequestAssignment : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 5};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
TSyncGroupRequestAssignment();
~TSyncGroupRequestAssignment() = default;
-
+
struct MemberIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "memberId";
static constexpr const char* About = "The ID of the member to assign.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
MemberIdMeta::Type MemberId;
-
+
struct AssignmentMeta {
using Type = TKafkaBytes;
using TypeDesc = NPrivate::TKafkaBytesDesc;
-
+
static constexpr const char* Name = "assignment";
static constexpr const char* About = "The member assignment.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
AssignmentMeta::Type Assignment;
-
+
TString AssignmentStr;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TSyncGroupRequestAssignment& other) const = default;
};
-
+
struct GroupIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupId";
static constexpr const char* About = "The unique group identifier.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
GroupIdMeta::Type GroupId;
-
+
struct GenerationIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "generationId";
static constexpr const char* About = "The generation of the group.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
GenerationIdMeta::Type GenerationId;
-
+
struct MemberIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "memberId";
static constexpr const char* About = "The member ID assigned by the group.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
MemberIdMeta::Type MemberId;
-
+
struct GroupInstanceIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupInstanceId";
static constexpr const char* About = "The unique identifier of the consumer instance provided by end user.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
GroupInstanceIdMeta::Type GroupInstanceId;
-
+
struct ProtocolTypeMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "protocolType";
static constexpr const char* About = "The group protocol type.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ProtocolTypeMeta::Type ProtocolType;
-
+
struct ProtocolNameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "protocolName";
static constexpr const char* About = "The group protocol name.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ProtocolNameMeta::Type ProtocolName;
-
+
struct AssignmentsMeta {
using ItemType = TSyncGroupRequestAssignment;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TSyncGroupRequestAssignment>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "assignments";
static constexpr const char* About = "Each assignment.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
AssignmentsMeta::Type Assignments;
-
+
i16 ApiKey() const override { return SYNC_GROUP; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TSyncGroupRequestData& other) const = default;
};
@@ -4827,132 +4829,634 @@ public:
class TSyncGroupResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TSyncGroupResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 5};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
TSyncGroupResponseData();
~TSyncGroupResponseData() = default;
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct ProtocolTypeMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "protocolType";
static constexpr const char* About = "The group protocol type.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ProtocolTypeMeta::Type ProtocolType;
-
+
struct ProtocolNameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "protocolName";
static constexpr const char* About = "The group protocol name.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ProtocolNameMeta::Type ProtocolName;
-
+
struct AssignmentMeta {
using Type = TKafkaBytes;
using TypeDesc = NPrivate::TKafkaBytesDesc;
-
+
static constexpr const char* Name = "assignment";
static constexpr const char* About = "The member assignment.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
AssignmentMeta::Type Assignment;
-
+
TString AssignmentStr;
-
+
i16 ApiKey() const override { return SYNC_GROUP; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TSyncGroupResponseData& other) const = default;
};
+class TDescribeGroupsRequestData : public TApiMessage {
+public:
+ typedef std::shared_ptr<TDescribeGroupsRequestData> TPtr;
+
+ struct MessageMeta {
+ static constexpr TKafkaVersions PresentVersions = {0, 5};
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+
+ TDescribeGroupsRequestData();
+ ~TDescribeGroupsRequestData() = default;
+
+ struct GroupsMeta {
+ using ItemType = TKafkaString;
+ using ItemTypeDesc = NPrivate::TKafkaStringDesc;
+ using Type = std::vector<TKafkaString>;
+ using TypeDesc = NPrivate::TKafkaArrayDesc;
+
+ static constexpr const char* Name = "groups";
+ static constexpr const char* About = "The names of the groups to describe";
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ GroupsMeta::Type Groups;
+
+ struct IncludeAuthorizedOperationsMeta {
+ using Type = TKafkaBool;
+ using TypeDesc = NPrivate::TKafkaBoolDesc;
+
+ static constexpr const char* Name = "includeAuthorizedOperations";
+ static constexpr const char* About = "Whether to include authorized operations.";
+ static const Type Default; // = false;
+
+ static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ IncludeAuthorizedOperationsMeta::Type IncludeAuthorizedOperations;
+
+ i16 ApiKey() const override { return DESCRIBE_GROUPS; };
+ i32 Size(TKafkaVersion version) const override;
+ void Read(TKafkaReadable& readable, TKafkaVersion version) override;
+ void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
+
+ bool operator==(const TDescribeGroupsRequestData& other) const = default;
+};
+
+
+class TDescribeGroupsResponseData : public TApiMessage {
+public:
+ typedef std::shared_ptr<TDescribeGroupsResponseData> TPtr;
+
+ struct MessageMeta {
+ static constexpr TKafkaVersions PresentVersions = {0, 5};
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+
+ TDescribeGroupsResponseData();
+ ~TDescribeGroupsResponseData() = default;
+
+ class TDescribedGroup : public TMessage {
+ public:
+ struct MessageMeta {
+ static constexpr TKafkaVersions PresentVersions = {0, 5};
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+
+ TDescribedGroup();
+ ~TDescribedGroup() = default;
+
+ class TDescribedGroupMember : public TMessage {
+ public:
+ struct MessageMeta {
+ static constexpr TKafkaVersions PresentVersions = {0, 5};
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+
+ TDescribedGroupMember();
+ ~TDescribedGroupMember() = default;
+
+ struct MemberIdMeta {
+ using Type = TKafkaString;
+ using TypeDesc = NPrivate::TKafkaStringDesc;
+
+ static constexpr const char* Name = "memberId";
+ static constexpr const char* About = "The member ID assigned by the group coordinator.";
+ static const Type Default; // = {""};
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ MemberIdMeta::Type MemberId;
+
+ struct GroupInstanceIdMeta {
+ using Type = TKafkaString;
+ using TypeDesc = NPrivate::TKafkaStringDesc;
+
+ static constexpr const char* Name = "groupInstanceId";
+ static constexpr const char* About = "The unique identifier of the consumer instance provided by end user.";
+ static const Type Default; // = std::nullopt;
+
+ static constexpr TKafkaVersions PresentVersions = {4, Max<TKafkaVersion>()};
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsAlways;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ GroupInstanceIdMeta::Type GroupInstanceId;
+
+ struct ClientIdMeta {
+ using Type = TKafkaString;
+ using TypeDesc = NPrivate::TKafkaStringDesc;
+
+ static constexpr const char* Name = "clientId";
+ static constexpr const char* About = "The client ID used in the member's latest join group request.";
+ static const Type Default; // = {""};
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ ClientIdMeta::Type ClientId;
+
+ struct ClientHostMeta {
+ using Type = TKafkaString;
+ using TypeDesc = NPrivate::TKafkaStringDesc;
+
+ static constexpr const char* Name = "clientHost";
+ static constexpr const char* About = "The client host.";
+ static const Type Default; // = {""};
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ ClientHostMeta::Type ClientHost;
+
+ struct MemberMetadataMeta {
+ using Type = TKafkaBytes;
+ using TypeDesc = NPrivate::TKafkaBytesDesc;
+
+ static constexpr const char* Name = "memberMetadata";
+ static constexpr const char* About = "The metadata corresponding to the current group protocol in use.";
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ MemberMetadataMeta::Type MemberMetadata;
+
+ struct MemberAssignmentMeta {
+ using Type = TKafkaBytes;
+ using TypeDesc = NPrivate::TKafkaBytesDesc;
+
+ static constexpr const char* Name = "memberAssignment";
+ static constexpr const char* About = "The current assignment provided by the group leader.";
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ MemberAssignmentMeta::Type MemberAssignment;
+
+ TString MemberAssignmentStr;
+
+ TString MemberMetadataStr;
+
+ i32 Size(TKafkaVersion version) const override;
+ void Read(TKafkaReadable& readable, TKafkaVersion version) override;
+ void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
+
+ bool operator==(const TDescribedGroupMember& other) const = default;
+ };
+
+ struct ErrorCodeMeta {
+ using Type = TKafkaInt16;
+ using TypeDesc = NPrivate::TKafkaIntDesc;
+
+ static constexpr const char* Name = "errorCode";
+ static constexpr const char* About = "The describe error, or 0 if there was no error.";
+ static const Type Default; // = 0;
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ ErrorCodeMeta::Type ErrorCode;
+
+ struct GroupIdMeta {
+ using Type = TKafkaString;
+ using TypeDesc = NPrivate::TKafkaStringDesc;
+
+ static constexpr const char* Name = "groupId";
+ static constexpr const char* About = "The group ID string.";
+ static const Type Default; // = {""};
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ GroupIdMeta::Type GroupId;
+
+ struct GroupStateMeta {
+ using Type = TKafkaString;
+ using TypeDesc = NPrivate::TKafkaStringDesc;
+
+ static constexpr const char* Name = "groupState";
+ static constexpr const char* About = "The group state string, or the empty string.";
+ static const Type Default; // = {""};
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ GroupStateMeta::Type GroupState;
+
+ struct ProtocolTypeMeta {
+ using Type = TKafkaString;
+ using TypeDesc = NPrivate::TKafkaStringDesc;
+
+ static constexpr const char* Name = "protocolType";
+ static constexpr const char* About = "The group protocol type, or the empty string.";
+ static const Type Default; // = {""};
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ ProtocolTypeMeta::Type ProtocolType;
+
+ struct ProtocolDataMeta {
+ using Type = TKafkaString;
+ using TypeDesc = NPrivate::TKafkaStringDesc;
+
+ static constexpr const char* Name = "protocolData";
+ static constexpr const char* About = "The group protocol data, or the empty string.";
+ static const Type Default; // = {""};
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ ProtocolDataMeta::Type ProtocolData;
+
+ struct MembersMeta {
+ using ItemType = TDescribedGroupMember;
+ using ItemTypeDesc = NPrivate::TKafkaStructDesc;
+ using Type = std::vector<TDescribedGroupMember>;
+ using TypeDesc = NPrivate::TKafkaArrayDesc;
+
+ static constexpr const char* Name = "members";
+ static constexpr const char* About = "The group members.";
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ MembersMeta::Type Members;
+
+ struct AuthorizedOperationsMeta {
+ using Type = TKafkaInt32;
+ using TypeDesc = NPrivate::TKafkaIntDesc;
+
+ static constexpr const char* Name = "authorizedOperations";
+ static constexpr const char* About = "32-bit bitfield to represent authorized operations for this group.";
+ static const Type Default; // = -2147483648;
+
+ static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ AuthorizedOperationsMeta::Type AuthorizedOperations;
+
+ i32 Size(TKafkaVersion version) const override;
+ void Read(TKafkaReadable& readable, TKafkaVersion version) override;
+ void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
+
+ bool operator==(const TDescribedGroup& other) const = default;
+ };
+
+ struct ThrottleTimeMsMeta {
+ using Type = TKafkaInt32;
+ using TypeDesc = NPrivate::TKafkaIntDesc;
+
+ static constexpr const char* Name = "throttleTimeMs";
+ static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
+ static const Type Default; // = 0;
+
+ static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ ThrottleTimeMsMeta::Type ThrottleTimeMs;
+
+ struct GroupsMeta {
+ using ItemType = TDescribedGroup;
+ using ItemTypeDesc = NPrivate::TKafkaStructDesc;
+ using Type = std::vector<TDescribedGroup>;
+ using TypeDesc = NPrivate::TKafkaArrayDesc;
+
+ static constexpr const char* Name = "groups";
+ static constexpr const char* About = "Each described group.";
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
+ };
+ GroupsMeta::Type Groups;
+
+ i16 ApiKey() const override { return DESCRIBE_GROUPS; };
+ i32 Size(TKafkaVersion version) const override;
+ void Read(TKafkaReadable& readable, TKafkaVersion version) override;
+ void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
+
+ bool operator==(const TDescribeGroupsResponseData& other) const = default;
+};
+
+
+class TListGroupsRequestData : public TApiMessage {
+public:
+ typedef std::shared_ptr<TListGroupsRequestData> TPtr;
+
+ struct MessageMeta {
+ static constexpr TKafkaVersions PresentVersions = {0, 4};
+ static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
+ };
+
+ TListGroupsRequestData();
+ ~TListGroupsRequestData() = default;
+
+ struct StatesFilterMeta {
+ using ItemType = TKafkaString;
+ using ItemTypeDesc = NPrivate::TKafkaStringDesc;
+ using Type = std::vector<TKafkaString>;
+ using TypeDesc = NPrivate::TKafkaArrayDesc;
+
+ static constexpr const char* Name = "statesFilter";
+ static constexpr const char* About = "The states of the groups we want to list. If empty all groups are returned with their state.";
+
+ static constexpr TKafkaVersions PresentVersions = {4, Max<TKafkaVersion>()};
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
+ };
+ StatesFilterMeta::Type StatesFilter;
+
+ i16 ApiKey() const override { return LIST_GROUPS; };
+ i32 Size(TKafkaVersion version) const override;
+ void Read(TKafkaReadable& readable, TKafkaVersion version) override;
+ void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
+
+ bool operator==(const TListGroupsRequestData& other) const = default;
+};
+
+
+class TListGroupsResponseData : public TApiMessage {
+public:
+ typedef std::shared_ptr<TListGroupsResponseData> TPtr;
+
+ struct MessageMeta {
+ static constexpr TKafkaVersions PresentVersions = {0, 4};
+ static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
+ };
+
+ TListGroupsResponseData();
+ ~TListGroupsResponseData() = default;
+
+ class TListedGroup : public TMessage {
+ public:
+ struct MessageMeta {
+ static constexpr TKafkaVersions PresentVersions = {0, 4};
+ static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
+ };
+
+ TListedGroup();
+ ~TListedGroup() = default;
+
+ struct GroupIdMeta {
+ using Type = TKafkaString;
+ using TypeDesc = NPrivate::TKafkaStringDesc;
+
+ static constexpr const char* Name = "groupId";
+ static constexpr const char* About = "The group ID.";
+ static const Type Default; // = {""};
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
+ };
+ GroupIdMeta::Type GroupId;
+
+ struct ProtocolTypeMeta {
+ using Type = TKafkaString;
+ using TypeDesc = NPrivate::TKafkaStringDesc;
+
+ static constexpr const char* Name = "protocolType";
+ static constexpr const char* About = "The group protocol type.";
+ static const Type Default; // = {""};
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
+ };
+ ProtocolTypeMeta::Type ProtocolType;
+
+ struct GroupStateMeta {
+ using Type = TKafkaString;
+ using TypeDesc = NPrivate::TKafkaStringDesc;
+
+ static constexpr const char* Name = "groupState";
+ static constexpr const char* About = "The group state name.";
+ static const Type Default; // = {""};
+
+ static constexpr TKafkaVersions PresentVersions = {4, Max<TKafkaVersion>()};
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
+ };
+ GroupStateMeta::Type GroupState;
+
+ i32 Size(TKafkaVersion version) const override;
+ void Read(TKafkaReadable& readable, TKafkaVersion version) override;
+ void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
+
+ bool operator==(const TListedGroup& other) const = default;
+ };
+
+ struct ThrottleTimeMsMeta {
+ using Type = TKafkaInt32;
+ using TypeDesc = NPrivate::TKafkaIntDesc;
+
+ static constexpr const char* Name = "throttleTimeMs";
+ static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
+ static const Type Default; // = 0;
+
+ static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
+ };
+ ThrottleTimeMsMeta::Type ThrottleTimeMs;
+
+ struct ErrorCodeMeta {
+ using Type = TKafkaInt16;
+ using TypeDesc = NPrivate::TKafkaIntDesc;
+
+ static constexpr const char* Name = "errorCode";
+ static constexpr const char* About = "The error code, or 0 if there was no error.";
+ static const Type Default; // = 0;
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
+ };
+ ErrorCodeMeta::Type ErrorCode;
+
+ struct GroupsMeta {
+ using ItemType = TListedGroup;
+ using ItemTypeDesc = NPrivate::TKafkaStructDesc;
+ using Type = std::vector<TListedGroup>;
+ using TypeDesc = NPrivate::TKafkaArrayDesc;
+
+ static constexpr const char* Name = "groups";
+ static constexpr const char* About = "Each group in the response.";
+
+ static constexpr TKafkaVersions PresentVersions = VersionsAlways;
+ static constexpr TKafkaVersions TaggedVersions = VersionsNever;
+ static constexpr TKafkaVersions NullableVersions = VersionsNever;
+ static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
+ };
+ GroupsMeta::Type Groups;
+
+ i16 ApiKey() const override { return LIST_GROUPS; };
+ i32 Size(TKafkaVersion version) const override;
+ void Read(TKafkaReadable& readable, TKafkaVersion version) override;
+ void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
+
+ bool operator==(const TListGroupsResponseData& other) const = default;
+};
+
+
class TSaslHandshakeRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TSaslHandshakeRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 1};
static constexpr TKafkaVersions FlexibleVersions = VersionsNever;
};
-
+
TSaslHandshakeRequestData();
~TSaslHandshakeRequestData() = default;
-
+
struct MechanismMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "mechanism";
static constexpr const char* About = "The SASL mechanism chosen by the client.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsNever;
};
MechanismMeta::Type Mechanism;
-
+
i16 ApiKey() const override { return SASL_HANDSHAKE; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TSaslHandshakeRequestData& other) const = default;
};
@@ -4960,51 +5464,51 @@ public:
class TSaslHandshakeResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TSaslHandshakeResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 1};
static constexpr TKafkaVersions FlexibleVersions = VersionsNever;
};
-
+
TSaslHandshakeResponseData();
~TSaslHandshakeResponseData() = default;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsNever;
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct MechanismsMeta {
using ItemType = TKafkaString;
using ItemTypeDesc = NPrivate::TKafkaStringDesc;
using Type = std::vector<TKafkaString>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "mechanisms";
static constexpr const char* About = "The mechanisms enabled in the server.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsNever;
};
MechanismsMeta::Type Mechanisms;
-
+
i16 ApiKey() const override { return SASL_HANDSHAKE; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TSaslHandshakeResponseData& other) const = default;
};
@@ -5012,50 +5516,50 @@ public:
class TApiVersionsRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TApiVersionsRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TApiVersionsRequestData();
~TApiVersionsRequestData() = default;
-
+
struct ClientSoftwareNameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "clientSoftwareName";
static constexpr const char* About = "The name of the client.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ClientSoftwareNameMeta::Type ClientSoftwareName;
-
+
struct ClientSoftwareVersionMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "clientSoftwareVersion";
static constexpr const char* About = "The version of the client.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ClientSoftwareVersionMeta::Type ClientSoftwareVersion;
-
+
i16 ApiKey() const override { return API_VERSIONS; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TApiVersionsRequestData& other) const = default;
};
@@ -5063,318 +5567,318 @@ public:
class TApiVersionsResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TApiVersionsResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TApiVersionsResponseData();
~TApiVersionsResponseData() = default;
-
+
class TApiVersion : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TApiVersion();
~TApiVersion() = default;
-
+
struct ApiKeyMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "apiKey";
static constexpr const char* About = "The API index.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ApiKeyMeta::Type ApiKey;
-
+
struct MinVersionMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "minVersion";
static constexpr const char* About = "The minimum supported version, inclusive.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
MinVersionMeta::Type MinVersion;
-
+
struct MaxVersionMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "maxVersion";
static constexpr const char* About = "The maximum supported version, inclusive.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
MaxVersionMeta::Type MaxVersion;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TApiVersion& other) const = default;
};
-
+
class TSupportedFeatureKey : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {3, 3};
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
-
+
TSupportedFeatureKey();
~TSupportedFeatureKey() = default;
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The name of the feature.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
NameMeta::Type Name;
-
+
struct MinVersionMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "minVersion";
static constexpr const char* About = "The minimum supported version for the feature.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
MinVersionMeta::Type MinVersion;
-
+
struct MaxVersionMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "maxVersion";
static constexpr const char* About = "The maximum supported version for the feature.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
MaxVersionMeta::Type MaxVersion;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TSupportedFeatureKey& other) const = default;
};
-
+
class TFinalizedFeatureKey : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {3, 3};
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
-
+
TFinalizedFeatureKey();
~TFinalizedFeatureKey() = default;
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The name of the feature.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
NameMeta::Type Name;
-
+
struct MaxVersionLevelMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "maxVersionLevel";
static constexpr const char* About = "The cluster-wide finalized max version level for the feature.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
MaxVersionLevelMeta::Type MaxVersionLevel;
-
+
struct MinVersionLevelMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "minVersionLevel";
static constexpr const char* About = "The cluster-wide finalized min version level for the feature.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
MinVersionLevelMeta::Type MinVersionLevel;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TFinalizedFeatureKey& other) const = default;
};
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The top-level error code.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct ApiKeysMeta {
using ItemType = TApiVersion;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TApiVersion>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "apiKeys";
static constexpr const char* About = "The APIs supported by the broker.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ApiKeysMeta::Type ApiKeys;
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct SupportedFeaturesMeta {
using ItemType = TSupportedFeatureKey;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TSupportedFeatureKey>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "supportedFeatures";
static constexpr const char* About = "Features supported by the broker.";
static constexpr const TKafkaInt32 Tag = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsAlways;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
SupportedFeaturesMeta::Type SupportedFeatures;
-
+
struct FinalizedFeaturesEpochMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "finalizedFeaturesEpoch";
static constexpr const char* About = "The monotonically increasing epoch for the finalized features information. Valid values are >= 0. A value of -1 is special and represents unknown epoch.";
static constexpr const TKafkaInt32 Tag = 1;
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsAlways;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
FinalizedFeaturesEpochMeta::Type FinalizedFeaturesEpoch;
-
+
struct FinalizedFeaturesMeta {
using ItemType = TFinalizedFeatureKey;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TFinalizedFeatureKey>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "finalizedFeatures";
static constexpr const char* About = "List of cluster-wide finalized features. The information is valid only if FinalizedFeaturesEpoch >= 0.";
static constexpr const TKafkaInt32 Tag = 2;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsAlways;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
FinalizedFeaturesMeta::Type FinalizedFeatures;
-
+
struct ZkMigrationReadyMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "zkMigrationReady";
static constexpr const char* About = "Set by a KRaft controller if the required configurations for ZK migration are present";
static constexpr const TKafkaInt32 Tag = 3;
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsAlways;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ZkMigrationReadyMeta::Type ZkMigrationReady;
-
+
i16 ApiKey() const override { return API_VERSIONS; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TApiVersionsResponseData& other) const = default;
};
@@ -5382,255 +5886,255 @@ public:
class TCreateTopicsRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TCreateTopicsRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
-
+
TCreateTopicsRequestData();
~TCreateTopicsRequestData() = default;
-
+
class TCreatableTopic : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
-
+
TCreatableTopic();
~TCreatableTopic() = default;
-
+
class TCreatableReplicaAssignment : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
-
+
TCreatableReplicaAssignment();
~TCreatableReplicaAssignment() = default;
-
+
struct PartitionIndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "partitionIndex";
static constexpr const char* About = "The partition index.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
PartitionIndexMeta::Type PartitionIndex;
-
+
struct BrokerIdsMeta {
using ItemType = TKafkaInt32;
using ItemTypeDesc = NPrivate::TKafkaIntDesc;
using Type = std::vector<TKafkaInt32>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "brokerIds";
static constexpr const char* About = "The brokers to place the partition on.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
BrokerIdsMeta::Type BrokerIds;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TCreatableReplicaAssignment& other) const = default;
};
-
+
class TCreateableTopicConfig : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
-
+
TCreateableTopicConfig();
~TCreateableTopicConfig() = default;
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The configuration name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct ValueMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "value";
static constexpr const char* About = "The configuration value.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
ValueMeta::Type Value;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TCreateableTopicConfig& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct NumPartitionsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "numPartitions";
static constexpr const char* About = "The number of partitions to create in the topic, or -1 if we are either specifying a manual partition assignment or using the default partitions.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
NumPartitionsMeta::Type NumPartitions;
-
+
struct ReplicationFactorMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "replicationFactor";
static constexpr const char* About = "The number of replicas to create for each partition in the topic, or -1 if we are either specifying a manual partition assignment or using the default replication factor.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
ReplicationFactorMeta::Type ReplicationFactor;
-
+
struct AssignmentsMeta {
using ItemType = TCreatableReplicaAssignment;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TCreatableReplicaAssignment>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "assignments";
static constexpr const char* About = "The manual partition assignment, or the empty array if we are using automatic assignment.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
AssignmentsMeta::Type Assignments;
-
+
struct ConfigsMeta {
using ItemType = TCreateableTopicConfig;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TCreateableTopicConfig>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "configs";
static constexpr const char* About = "The custom topic configurations to set.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
ConfigsMeta::Type Configs;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TCreatableTopic& other) const = default;
};
-
+
struct TopicsMeta {
using ItemType = TCreatableTopic;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TCreatableTopic>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "The topics to create.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
struct TimeoutMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "timeoutMs";
static constexpr const char* About = "How long to wait in milliseconds before timing out the request.";
static const Type Default; // = 60000;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
TimeoutMsMeta::Type TimeoutMs;
-
+
struct ValidateOnlyMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "validateOnly";
static constexpr const char* About = "If true, check that the topics can be created as specified, but don't create anything.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
ValidateOnlyMeta::Type ValidateOnly;
-
+
i16 ApiKey() const override { return CREATE_TOPICS; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TCreateTopicsRequestData& other) const = default;
};
@@ -5638,282 +6142,282 @@ public:
class TCreateTopicsResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TCreateTopicsResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
-
+
TCreateTopicsResponseData();
~TCreateTopicsResponseData() = default;
-
+
class TCreatableTopicResult : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 7};
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
-
+
TCreatableTopicResult();
~TCreatableTopicResult() = default;
-
+
class TCreatableTopicConfigs : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {5, 7};
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
-
+
TCreatableTopicConfigs();
~TCreatableTopicConfigs() = default;
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The configuration name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
NameMeta::Type Name;
-
+
struct ValueMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "value";
static constexpr const char* About = "The configuration value.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ValueMeta::Type Value;
-
+
struct ReadOnlyMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "readOnly";
static constexpr const char* About = "True if the configuration is read-only.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ReadOnlyMeta::Type ReadOnly;
-
+
struct ConfigSourceMeta {
using Type = TKafkaInt8;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "configSource";
static constexpr const char* About = "The configuration source.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ConfigSourceMeta::Type ConfigSource;
-
+
struct IsSensitiveMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "isSensitive";
static constexpr const char* About = "True if this configuration is sensitive.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
IsSensitiveMeta::Type IsSensitive;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TCreatableTopicConfigs& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct TopicIdMeta {
using Type = TKafkaUuid;
using TypeDesc = NPrivate::TKafkaUuidDesc;
-
+
static constexpr const char* Name = "topicId";
static constexpr const char* About = "The unique topic ID";
static const Type Default; // = TKafkaUuid(0, 0);
-
+
static constexpr TKafkaVersions PresentVersions = {7, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
TopicIdMeta::Type TopicId;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct ErrorMessageMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "errorMessage";
static constexpr const char* About = "The error message, or null if there was no error.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
ErrorMessageMeta::Type ErrorMessage;
-
+
struct TopicConfigErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "topicConfigErrorCode";
static constexpr const char* About = "Optional topic config error returned if configs are not returned in the response.";
static constexpr const TKafkaInt32 Tag = 0;
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsAlways;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
TopicConfigErrorCodeMeta::Type TopicConfigErrorCode;
-
+
struct NumPartitionsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "numPartitions";
static constexpr const char* About = "Number of partitions of the topic.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
NumPartitionsMeta::Type NumPartitions;
-
+
struct ReplicationFactorMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "replicationFactor";
static constexpr const char* About = "Replication factor of the topic.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ReplicationFactorMeta::Type ReplicationFactor;
-
+
struct ConfigsMeta {
using ItemType = TCreatableTopicConfigs;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TCreatableTopicConfigs>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "configs";
static constexpr const char* About = "Configuration of the topic.";
-
+
static constexpr TKafkaVersions PresentVersions = {5, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ConfigsMeta::Type Configs;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TCreatableTopicResult& other) const = default;
};
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {2, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct TopicsMeta {
using ItemType = TCreatableTopicResult;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TCreatableTopicResult>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "Results for each topic we tried to create.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {5, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
i16 ApiKey() const override { return CREATE_TOPICS; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TCreateTopicsResponseData& other) const = default;
};
@@ -5921,80 +6425,80 @@ public:
class TInitProducerIdRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TInitProducerIdRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 4};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TInitProducerIdRequestData();
~TInitProducerIdRequestData() = default;
-
+
struct TransactionalIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "transactionalId";
static constexpr const char* About = "The transactional id, or null if the producer is not transactional.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
TransactionalIdMeta::Type TransactionalId;
-
+
struct TransactionTimeoutMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "transactionTimeoutMs";
static constexpr const char* About = "The time in ms to wait before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
TransactionTimeoutMsMeta::Type TransactionTimeoutMs;
-
+
struct ProducerIdMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "producerId";
static constexpr const char* About = "The producer id. This is used to disambiguate requests if a transactional id is reused following its expiration.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ProducerIdMeta::Type ProducerId;
-
+
struct ProducerEpochMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "producerEpoch";
static constexpr const char* About = "The producer's current epoch. This will be checked against the producer epoch on the broker, and the request will return an error if they do not match.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
ProducerEpochMeta::Type ProducerEpoch;
-
+
i16 ApiKey() const override { return INIT_PRODUCER_ID; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TInitProducerIdRequestData& other) const = default;
};
@@ -6002,80 +6506,80 @@ public:
class TInitProducerIdResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TInitProducerIdResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 4};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TInitProducerIdResponseData();
~TInitProducerIdResponseData() = default;
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct ProducerIdMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "producerId";
static constexpr const char* About = "The current producer id.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ProducerIdMeta::Type ProducerId;
-
+
struct ProducerEpochMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "producerEpoch";
static constexpr const char* About = "The current epoch associated with the producer id.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ProducerEpochMeta::Type ProducerEpoch;
-
+
i16 ApiKey() const override { return INIT_PRODUCER_ID; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TInitProducerIdResponseData& other) const = default;
};
@@ -6083,129 +6587,129 @@ public:
class TAddPartitionsToTxnRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TAddPartitionsToTxnRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TAddPartitionsToTxnRequestData();
~TAddPartitionsToTxnRequestData() = default;
-
+
class TAddPartitionsToTxnTopic : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TAddPartitionsToTxnTopic();
~TAddPartitionsToTxnTopic() = default;
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The name of the topic.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct PartitionsMeta {
using ItemType = TKafkaInt32;
using ItemTypeDesc = NPrivate::TKafkaIntDesc;
using Type = std::vector<TKafkaInt32>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitions";
static constexpr const char* About = "The partition indexes to add to the transaction";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
PartitionsMeta::Type Partitions;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TAddPartitionsToTxnTopic& other) const = default;
};
-
+
struct TransactionalIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "transactionalId";
static constexpr const char* About = "The transactional id corresponding to the transaction.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
TransactionalIdMeta::Type TransactionalId;
-
+
struct ProducerIdMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "producerId";
static constexpr const char* About = "Current producer id in use by the transactional id.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ProducerIdMeta::Type ProducerId;
-
+
struct ProducerEpochMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "producerEpoch";
static constexpr const char* About = "Current epoch associated with the producer id.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ProducerEpochMeta::Type ProducerEpoch;
-
+
struct TopicsMeta {
using ItemType = TAddPartitionsToTxnTopic;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TAddPartitionsToTxnTopic>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "The partitions to add to the transaction.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
i16 ApiKey() const override { return ADD_PARTITIONS_TO_TXN; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TAddPartitionsToTxnRequestData& other) const = default;
};
@@ -6213,146 +6717,146 @@ public:
class TAddPartitionsToTxnResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TAddPartitionsToTxnResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TAddPartitionsToTxnResponseData();
~TAddPartitionsToTxnResponseData() = default;
-
+
class TAddPartitionsToTxnTopicResult : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TAddPartitionsToTxnTopicResult();
~TAddPartitionsToTxnTopicResult() = default;
-
+
class TAddPartitionsToTxnPartitionResult : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TAddPartitionsToTxnPartitionResult();
~TAddPartitionsToTxnPartitionResult() = default;
-
+
struct PartitionIndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "partitionIndex";
static constexpr const char* About = "The partition indexes.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
PartitionIndexMeta::Type PartitionIndex;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The response error code.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TAddPartitionsToTxnPartitionResult& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct ResultsMeta {
using ItemType = TAddPartitionsToTxnPartitionResult;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TAddPartitionsToTxnPartitionResult>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "results";
static constexpr const char* About = "The results for each partition";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ResultsMeta::Type Results;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TAddPartitionsToTxnTopicResult& other) const = default;
};
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct ResultsMeta {
using ItemType = TAddPartitionsToTxnTopicResult;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TAddPartitionsToTxnTopicResult>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "results";
static constexpr const char* About = "The results for each topic.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ResultsMeta::Type Results;
-
+
i16 ApiKey() const override { return ADD_PARTITIONS_TO_TXN; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TAddPartitionsToTxnResponseData& other) const = default;
};
@@ -6360,80 +6864,80 @@ public:
class TAddOffsetsToTxnRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TAddOffsetsToTxnRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TAddOffsetsToTxnRequestData();
~TAddOffsetsToTxnRequestData() = default;
-
+
struct TransactionalIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "transactionalId";
static constexpr const char* About = "The transactional id corresponding to the transaction.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
TransactionalIdMeta::Type TransactionalId;
-
+
struct ProducerIdMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "producerId";
static constexpr const char* About = "Current producer id in use by the transactional id.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ProducerIdMeta::Type ProducerId;
-
+
struct ProducerEpochMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "producerEpoch";
static constexpr const char* About = "Current epoch associated with the producer id.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ProducerEpochMeta::Type ProducerEpoch;
-
+
struct GroupIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupId";
static constexpr const char* About = "The unique group identifier.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
GroupIdMeta::Type GroupId;
-
+
i16 ApiKey() const override { return ADD_OFFSETS_TO_TXN; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TAddOffsetsToTxnRequestData& other) const = default;
};
@@ -6441,50 +6945,50 @@ public:
class TAddOffsetsToTxnResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TAddOffsetsToTxnResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TAddOffsetsToTxnResponseData();
~TAddOffsetsToTxnResponseData() = default;
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The response error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
i16 ApiKey() const override { return ADD_OFFSETS_TO_TXN; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TAddOffsetsToTxnResponseData& other) const = default;
};
@@ -6492,80 +6996,80 @@ public:
class TEndTxnRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TEndTxnRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TEndTxnRequestData();
~TEndTxnRequestData() = default;
-
+
struct TransactionalIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "transactionalId";
static constexpr const char* About = "The ID of the transaction to end.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
TransactionalIdMeta::Type TransactionalId;
-
+
struct ProducerIdMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "producerId";
static constexpr const char* About = "The producer ID.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ProducerIdMeta::Type ProducerId;
-
+
struct ProducerEpochMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "producerEpoch";
static constexpr const char* About = "The current epoch associated with the producer.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ProducerEpochMeta::Type ProducerEpoch;
-
+
struct CommittedMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "committed";
static constexpr const char* About = "True if the transaction was committed, false if it was aborted.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
CommittedMeta::Type Committed;
-
+
i16 ApiKey() const override { return END_TXN; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TEndTxnRequestData& other) const = default;
};
@@ -6573,50 +7077,50 @@ public:
class TEndTxnResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TEndTxnResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TEndTxnResponseData();
~TEndTxnResponseData() = default;
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
i16 ApiKey() const override { return END_TXN; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TEndTxnResponseData& other) const = default;
};
@@ -6624,266 +7128,266 @@ public:
class TTxnOffsetCommitRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TTxnOffsetCommitRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TTxnOffsetCommitRequestData();
~TTxnOffsetCommitRequestData() = default;
-
+
class TTxnOffsetCommitRequestTopic : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TTxnOffsetCommitRequestTopic();
~TTxnOffsetCommitRequestTopic() = default;
-
+
class TTxnOffsetCommitRequestPartition : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TTxnOffsetCommitRequestPartition();
~TTxnOffsetCommitRequestPartition() = default;
-
+
struct PartitionIndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "partitionIndex";
static constexpr const char* About = "The index of the partition within the topic.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
PartitionIndexMeta::Type PartitionIndex;
-
+
struct CommittedOffsetMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "committedOffset";
static constexpr const char* About = "The message offset to be committed.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
CommittedOffsetMeta::Type CommittedOffset;
-
+
struct CommittedLeaderEpochMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "committedLeaderEpoch";
static constexpr const char* About = "The leader epoch of the last consumed record.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {2, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
CommittedLeaderEpochMeta::Type CommittedLeaderEpoch;
-
+
struct CommittedMetadataMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "committedMetadata";
static constexpr const char* About = "Any associated metadata the client wants to keep.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
CommittedMetadataMeta::Type CommittedMetadata;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TTxnOffsetCommitRequestPartition& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct PartitionsMeta {
using ItemType = TTxnOffsetCommitRequestPartition;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TTxnOffsetCommitRequestPartition>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitions";
static constexpr const char* About = "The partitions inside the topic that we want to committ offsets for.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
PartitionsMeta::Type Partitions;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TTxnOffsetCommitRequestTopic& other) const = default;
};
-
+
struct TransactionalIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "transactionalId";
static constexpr const char* About = "The ID of the transaction.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
TransactionalIdMeta::Type TransactionalId;
-
+
struct GroupIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupId";
static constexpr const char* About = "The ID of the group.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
GroupIdMeta::Type GroupId;
-
+
struct ProducerIdMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "producerId";
static constexpr const char* About = "The current producer ID in use by the transactional ID.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ProducerIdMeta::Type ProducerId;
-
+
struct ProducerEpochMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "producerEpoch";
static constexpr const char* About = "The current epoch associated with the producer ID.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ProducerEpochMeta::Type ProducerEpoch;
-
+
struct GenerationIdMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "generationId";
static constexpr const char* About = "The generation of the consumer.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
GenerationIdMeta::Type GenerationId;
-
+
struct MemberIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "memberId";
static constexpr const char* About = "The member ID assigned by the group coordinator.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
MemberIdMeta::Type MemberId;
-
+
struct GroupInstanceIdMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "groupInstanceId";
static constexpr const char* About = "The unique identifier of the consumer instance provided by end user.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = VersionsAlways;
};
GroupInstanceIdMeta::Type GroupInstanceId;
-
+
struct TopicsMeta {
using ItemType = TTxnOffsetCommitRequestTopic;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TTxnOffsetCommitRequestTopic>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "Each topic that we want to commit offsets for.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
i16 ApiKey() const override { return TXN_OFFSET_COMMIT; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TTxnOffsetCommitRequestData& other) const = default;
};
@@ -6891,146 +7395,146 @@ public:
class TTxnOffsetCommitResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TTxnOffsetCommitResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TTxnOffsetCommitResponseData();
~TTxnOffsetCommitResponseData() = default;
-
+
class TTxnOffsetCommitResponseTopic : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TTxnOffsetCommitResponseTopic();
~TTxnOffsetCommitResponseTopic() = default;
-
+
class TTxnOffsetCommitResponsePartition : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
-
+
TTxnOffsetCommitResponsePartition();
~TTxnOffsetCommitResponsePartition() = default;
-
+
struct PartitionIndexMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "partitionIndex";
static constexpr const char* About = "The partition index.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
PartitionIndexMeta::Type PartitionIndex;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TTxnOffsetCommitResponsePartition& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct PartitionsMeta {
using ItemType = TTxnOffsetCommitResponsePartition;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TTxnOffsetCommitResponsePartition>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "partitions";
static constexpr const char* About = "The responses for each partition in the topic.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
PartitionsMeta::Type Partitions;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TTxnOffsetCommitResponseTopic& other) const = default;
};
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct TopicsMeta {
using ItemType = TTxnOffsetCommitResponseTopic;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TTxnOffsetCommitResponseTopic>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "The responses for each topic.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {3, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
i16 ApiKey() const override { return TXN_OFFSET_COMMIT; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TTxnOffsetCommitResponseData& other) const = default;
};
@@ -7038,129 +7542,129 @@ public:
class TDescribeConfigsRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TDescribeConfigsRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 4};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
TDescribeConfigsRequestData();
~TDescribeConfigsRequestData() = default;
-
+
class TDescribeConfigsResource : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 4};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
TDescribeConfigsResource();
~TDescribeConfigsResource() = default;
-
+
struct ResourceTypeMeta {
using Type = TKafkaInt8;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "resourceType";
static constexpr const char* About = "The resource type.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ResourceTypeMeta::Type ResourceType;
-
+
struct ResourceNameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "resourceName";
static constexpr const char* About = "The resource name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ResourceNameMeta::Type ResourceName;
-
+
struct ConfigurationKeysMeta {
using ItemType = TKafkaString;
using ItemTypeDesc = NPrivate::TKafkaStringDesc;
using Type = std::vector<TKafkaString>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "configurationKeys";
static constexpr const char* About = "The configuration keys to list, or null to list all configuration keys.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ConfigurationKeysMeta::Type ConfigurationKeys;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TDescribeConfigsResource& other) const = default;
};
-
+
struct ResourcesMeta {
using ItemType = TDescribeConfigsResource;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TDescribeConfigsResource>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "resources";
static constexpr const char* About = "The resources whose configurations we want to describe.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ResourcesMeta::Type Resources;
-
+
struct IncludeSynonymsMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "includeSynonyms";
static constexpr const char* About = "True if we should include all synonyms.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
IncludeSynonymsMeta::Type IncludeSynonyms;
-
+
struct IncludeDocumentationMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "includeDocumentation";
static constexpr const char* About = "True if we should include configuration documentation.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
IncludeDocumentationMeta::Type IncludeDocumentation;
-
+
i16 ApiKey() const override { return DESCRIBE_CONFIGS; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TDescribeConfigsRequestData& other) const = default;
};
@@ -7168,359 +7672,359 @@ public:
class TDescribeConfigsResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TDescribeConfigsResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 4};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
TDescribeConfigsResponseData();
~TDescribeConfigsResponseData() = default;
-
+
class TDescribeConfigsResult : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 4};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
TDescribeConfigsResult();
~TDescribeConfigsResult() = default;
-
+
class TDescribeConfigsResourceResult : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 4};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
TDescribeConfigsResourceResult();
~TDescribeConfigsResourceResult() = default;
-
+
class TDescribeConfigsSynonym : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {1, 4};
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
-
+
TDescribeConfigsSynonym();
~TDescribeConfigsSynonym() = default;
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The synonym name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct ValueMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "value";
static constexpr const char* About = "The synonym value.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ValueMeta::Type Value;
-
+
struct SourceMeta {
using Type = TKafkaInt8;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "source";
static constexpr const char* About = "The synonym source.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
SourceMeta::Type Source;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TDescribeConfigsSynonym& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The configuration name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct ValueMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "value";
static constexpr const char* About = "The configuration value.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ValueMeta::Type Value;
-
+
struct ReadOnlyMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "readOnly";
static constexpr const char* About = "True if the configuration is read-only.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ReadOnlyMeta::Type ReadOnly;
-
+
struct IsDefaultMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "isDefault";
static constexpr const char* About = "True if the configuration is not set.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = {0, 0};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
IsDefaultMeta::Type IsDefault;
-
+
struct ConfigSourceMeta {
using Type = TKafkaInt8;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "configSource";
static constexpr const char* About = "The configuration source.";
static const Type Default; // = -1;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ConfigSourceMeta::Type ConfigSource;
-
+
struct IsSensitiveMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "isSensitive";
static constexpr const char* About = "True if this configuration is sensitive.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
IsSensitiveMeta::Type IsSensitive;
-
+
struct SynonymsMeta {
using ItemType = TDescribeConfigsSynonym;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TDescribeConfigsSynonym>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "synonyms";
static constexpr const char* About = "The synonyms for this configuration key.";
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
SynonymsMeta::Type Synonyms;
-
+
struct ConfigTypeMeta {
using Type = TKafkaInt8;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "configType";
static constexpr const char* About = "The configuration data type. Type can be one of the following values - BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS, PASSWORD";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ConfigTypeMeta::Type ConfigType;
-
+
struct DocumentationMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "documentation";
static constexpr const char* About = "The configuration documentation.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = {3, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
DocumentationMeta::Type Documentation;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TDescribeConfigsResourceResult& other) const = default;
};
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if we were able to successfully describe the configurations.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct ErrorMessageMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "errorMessage";
static constexpr const char* About = "The error message, or null if we were able to successfully describe the configurations.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ErrorMessageMeta::Type ErrorMessage;
-
+
struct ResourceTypeMeta {
using Type = TKafkaInt8;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "resourceType";
static constexpr const char* About = "The resource type.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ResourceTypeMeta::Type ResourceType;
-
+
struct ResourceNameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "resourceName";
static constexpr const char* About = "The resource name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ResourceNameMeta::Type ResourceName;
-
+
struct ConfigsMeta {
using ItemType = TDescribeConfigsResourceResult;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TDescribeConfigsResourceResult>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "configs";
static constexpr const char* About = "Each listed configuration.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ConfigsMeta::Type Configs;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TDescribeConfigsResult& other) const = default;
};
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct ResultsMeta {
using ItemType = TDescribeConfigsResult;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TDescribeConfigsResult>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "results";
static constexpr const char* About = "The results for each resource.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {4, Max<TKafkaVersion>()};
};
ResultsMeta::Type Results;
-
+
i16 ApiKey() const override { return DESCRIBE_CONFIGS; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TDescribeConfigsResponseData& other) const = default;
};
@@ -7528,161 +8032,161 @@ public:
class TAlterConfigsRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TAlterConfigsRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 2};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TAlterConfigsRequestData();
~TAlterConfigsRequestData() = default;
-
+
class TAlterConfigsResource : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 2};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TAlterConfigsResource();
~TAlterConfigsResource() = default;
-
+
class TAlterableConfig : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 2};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TAlterableConfig();
~TAlterableConfig() = default;
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The configuration key name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct ValueMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "value";
static constexpr const char* About = "The value to set for the configuration key.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ValueMeta::Type Value;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TAlterableConfig& other) const = default;
};
-
+
struct ResourceTypeMeta {
using Type = TKafkaInt8;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "resourceType";
static constexpr const char* About = "The resource type.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ResourceTypeMeta::Type ResourceType;
-
+
struct ResourceNameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "resourceName";
static constexpr const char* About = "The resource name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ResourceNameMeta::Type ResourceName;
-
+
struct ConfigsMeta {
using ItemType = TAlterableConfig;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TAlterableConfig>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "configs";
static constexpr const char* About = "The configurations.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ConfigsMeta::Type Configs;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TAlterConfigsResource& other) const = default;
};
-
+
struct ResourcesMeta {
using ItemType = TAlterConfigsResource;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TAlterConfigsResource>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "resources";
static constexpr const char* About = "The updates for each resource.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ResourcesMeta::Type Resources;
-
+
struct ValidateOnlyMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "validateOnly";
static constexpr const char* About = "True if we should validate the request, but not change the configurations.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ValidateOnlyMeta::Type ValidateOnly;
-
+
i16 ApiKey() const override { return ALTER_CONFIGS; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TAlterConfigsRequestData& other) const = default;
};
@@ -7690,128 +8194,128 @@ public:
class TAlterConfigsResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TAlterConfigsResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 2};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TAlterConfigsResponseData();
~TAlterConfigsResponseData() = default;
-
+
class TAlterConfigsResourceResponse : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 2};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TAlterConfigsResourceResponse();
~TAlterConfigsResourceResponse() = default;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The resource error code.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct ErrorMessageMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "errorMessage";
static constexpr const char* About = "The resource error message, or null if there was no error.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ErrorMessageMeta::Type ErrorMessage;
-
+
struct ResourceTypeMeta {
using Type = TKafkaInt8;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "resourceType";
static constexpr const char* About = "The resource type.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ResourceTypeMeta::Type ResourceType;
-
+
struct ResourceNameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "resourceName";
static constexpr const char* About = "The resource name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ResourceNameMeta::Type ResourceName;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TAlterConfigsResourceResponse& other) const = default;
};
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct ResponsesMeta {
using ItemType = TAlterConfigsResourceResponse;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TAlterConfigsResourceResponse>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "responses";
static constexpr const char* About = "The responses for each resource.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ResponsesMeta::Type Responses;
-
+
i16 ApiKey() const override { return ALTER_CONFIGS; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TAlterConfigsResponseData& other) const = default;
};
@@ -7819,34 +8323,34 @@ public:
class TSaslAuthenticateRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TSaslAuthenticateRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 2};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TSaslAuthenticateRequestData();
~TSaslAuthenticateRequestData() = default;
-
+
struct AuthBytesMeta {
using Type = TKafkaBytes;
using TypeDesc = NPrivate::TKafkaBytesDesc;
-
+
static constexpr const char* Name = "authBytes";
static constexpr const char* About = "The SASL authentication bytes from the client, as defined by the SASL mechanism.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
AuthBytesMeta::Type AuthBytes;
-
+
i16 ApiKey() const override { return SASL_AUTHENTICATE; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TSaslAuthenticateRequestData& other) const = default;
};
@@ -7854,79 +8358,79 @@ public:
class TSaslAuthenticateResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TSaslAuthenticateResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 2};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TSaslAuthenticateResponseData();
~TSaslAuthenticateResponseData() = default;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The error code, or 0 if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct ErrorMessageMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "errorMessage";
static constexpr const char* About = "The error message, or null if there was no error.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ErrorMessageMeta::Type ErrorMessage;
-
+
struct AuthBytesMeta {
using Type = TKafkaBytes;
using TypeDesc = NPrivate::TKafkaBytesDesc;
-
+
static constexpr const char* Name = "authBytes";
static constexpr const char* About = "The SASL authentication bytes from the server, as defined by the SASL mechanism.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
AuthBytesMeta::Type AuthBytes;
-
+
struct SessionLifetimeMsMeta {
using Type = TKafkaInt64;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "sessionLifetimeMs";
static constexpr const char* About = "The SASL authentication bytes from the server, as defined by the SASL mechanism.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = {1, Max<TKafkaVersion>()};
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
SessionLifetimeMsMeta::Type SessionLifetimeMs;
-
+
i16 ApiKey() const override { return SASL_AUTHENTICATE; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TSaslAuthenticateResponseData& other) const = default;
};
@@ -7934,162 +8438,162 @@ public:
class TCreatePartitionsRequestData : public TApiMessage {
public:
typedef std::shared_ptr<TCreatePartitionsRequestData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TCreatePartitionsRequestData();
~TCreatePartitionsRequestData() = default;
-
+
class TCreatePartitionsTopic : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TCreatePartitionsTopic();
~TCreatePartitionsTopic() = default;
-
+
class TCreatePartitionsAssignment : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TCreatePartitionsAssignment();
~TCreatePartitionsAssignment() = default;
-
+
struct BrokerIdsMeta {
using ItemType = TKafkaInt32;
using ItemTypeDesc = NPrivate::TKafkaIntDesc;
using Type = std::vector<TKafkaInt32>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "brokerIds";
static constexpr const char* About = "The assigned broker IDs.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
BrokerIdsMeta::Type BrokerIds;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TCreatePartitionsAssignment& other) const = default;
};
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct CountMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "count";
static constexpr const char* About = "The new partition count.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
CountMeta::Type Count;
-
+
struct AssignmentsMeta {
using ItemType = TCreatePartitionsAssignment;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TCreatePartitionsAssignment>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "assignments";
static constexpr const char* About = "The new partition assignments.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
AssignmentsMeta::Type Assignments;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TCreatePartitionsTopic& other) const = default;
};
-
+
struct TopicsMeta {
using ItemType = TCreatePartitionsTopic;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TCreatePartitionsTopic>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "topics";
static constexpr const char* About = "Each topic that we want to create new partitions inside.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
TopicsMeta::Type Topics;
-
+
struct TimeoutMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "timeoutMs";
static constexpr const char* About = "The time in ms to wait for the partitions to be created.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
TimeoutMsMeta::Type TimeoutMs;
-
+
struct ValidateOnlyMeta {
using Type = TKafkaBool;
using TypeDesc = NPrivate::TKafkaBoolDesc;
-
+
static constexpr const char* Name = "validateOnly";
static constexpr const char* About = "If true, then validate the request, but don't actually increase the number of partitions.";
static const Type Default; // = false;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ValidateOnlyMeta::Type ValidateOnly;
-
+
i16 ApiKey() const override { return CREATE_PARTITIONS; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TCreatePartitionsRequestData& other) const = default;
};
@@ -8097,114 +8601,114 @@ public:
class TCreatePartitionsResponseData : public TApiMessage {
public:
typedef std::shared_ptr<TCreatePartitionsResponseData> TPtr;
-
+
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TCreatePartitionsResponseData();
~TCreatePartitionsResponseData() = default;
-
+
class TCreatePartitionsTopicResult : public TMessage {
public:
struct MessageMeta {
static constexpr TKafkaVersions PresentVersions = {0, 3};
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
-
+
TCreatePartitionsTopicResult();
~TCreatePartitionsTopicResult() = default;
-
+
struct NameMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "name";
static constexpr const char* About = "The topic name.";
static const Type Default; // = {""};
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
NameMeta::Type Name;
-
+
struct ErrorCodeMeta {
using Type = TKafkaInt16;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "errorCode";
static constexpr const char* About = "The result error, or zero if there was no error.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ErrorCodeMeta::Type ErrorCode;
-
+
struct ErrorMessageMeta {
using Type = TKafkaString;
using TypeDesc = NPrivate::TKafkaStringDesc;
-
+
static constexpr const char* Name = "errorMessage";
static constexpr const char* About = "The result message, or null if there was no error.";
static const Type Default; // = std::nullopt;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsAlways;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ErrorMessageMeta::Type ErrorMessage;
-
+
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TCreatePartitionsTopicResult& other) const = default;
};
-
+
struct ThrottleTimeMsMeta {
using Type = TKafkaInt32;
using TypeDesc = NPrivate::TKafkaIntDesc;
-
+
static constexpr const char* Name = "throttleTimeMs";
static constexpr const char* About = "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.";
static const Type Default; // = 0;
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ThrottleTimeMsMeta::Type ThrottleTimeMs;
-
+
struct ResultsMeta {
using ItemType = TCreatePartitionsTopicResult;
using ItemTypeDesc = NPrivate::TKafkaStructDesc;
using Type = std::vector<TCreatePartitionsTopicResult>;
using TypeDesc = NPrivate::TKafkaArrayDesc;
-
+
static constexpr const char* Name = "results";
static constexpr const char* About = "The partition creation results for each topic.";
-
+
static constexpr TKafkaVersions PresentVersions = VersionsAlways;
static constexpr TKafkaVersions TaggedVersions = VersionsNever;
static constexpr TKafkaVersions NullableVersions = VersionsNever;
static constexpr TKafkaVersions FlexibleVersions = {2, Max<TKafkaVersion>()};
};
ResultsMeta::Type Results;
-
+
i16 ApiKey() const override { return CREATE_PARTITIONS; };
i32 Size(TKafkaVersion version) const override;
void Read(TKafkaReadable& readable, TKafkaVersion version) override;
void Write(TKafkaWritable& writable, TKafkaVersion version) const override;
-
+
bool operator==(const TCreatePartitionsResponseData& other) const = default;
};
-} // namespace NKafka
+} // namespace NKafka
diff --git a/ydb/core/kafka_proxy/ut/kafka_test_client.cpp b/ydb/core/kafka_proxy/ut/kafka_test_client.cpp
index ce2be7a7236..bfcc4a249de 100644
--- a/ydb/core/kafka_proxy/ut/kafka_test_client.cpp
+++ b/ydb/core/kafka_proxy/ut/kafka_test_client.cpp
@@ -145,10 +145,10 @@ TMessagePtr<TProduceResponseData> TKafkaTestClient::Produce(const TString& topic
return WriteAndRead<TProduceResponseData>(header, request);
}
-TMessagePtr<TProduceResponseData> TKafkaTestClient::Produce(const TTopicPartition& topicPartition,
- const std::vector<std::pair<TString, TString>>& keyValueMessages,
- ui32 baseSequence,
- const std::optional<TProducerInstanceId>& producerInstanceId,
+TMessagePtr<TProduceResponseData> TKafkaTestClient::Produce(const TTopicPartition& topicPartition,
+ const std::vector<std::pair<TString, TString>>& keyValueMessages,
+ ui32 baseSequence,
+ const std::optional<TProducerInstanceId>& producerInstanceId,
const std::optional<TString>& transactionalId) {
TKafkaRecordBatch batch;
batch.BaseSequence = baseSequence;
@@ -430,6 +430,34 @@ TMessagePtr<TOffsetFetchResponseData> TKafkaTestClient::OffsetFetch(TOffsetFetch
return WriteAndRead<TOffsetFetchResponseData>(header, request);
}
+TMessagePtr<TListGroupsResponseData> TKafkaTestClient::ListGroups(TListGroupsRequestData request) {
+ Cerr << ">>>>> TListGroupsResponseData\n";
+ TRequestHeaderData header = Header(NKafka::EApiKey::LIST_GROUPS, 4);
+ return WriteAndRead<TListGroupsResponseData>(header, request);
+}
+
+TMessagePtr<TListGroupsResponseData> TKafkaTestClient::ListGroups(const std::vector<std::optional<TString>>& statesFilter) {
+ Cerr << ">>>>> TListGroupsResponseData\n";
+ TRequestHeaderData header = Header(NKafka::EApiKey::LIST_GROUPS, 4);
+ TListGroupsRequestData request;
+ request.StatesFilter = statesFilter;
+ return WriteAndRead<TListGroupsResponseData>(header, request);
+}
+
+TMessagePtr<TDescribeGroupsResponseData> TKafkaTestClient::DescribeGroups(TDescribeGroupsRequestData& request) {
+ Cerr << ">>>>> TDescribeGroupsResponseData\n";
+ TRequestHeaderData header = Header(NKafka::EApiKey::DESCRIBE_GROUPS, 5);
+ return WriteAndRead<TDescribeGroupsResponseData>(header, request);
+}
+
+TMessagePtr<TDescribeGroupsResponseData> TKafkaTestClient::DescribeGroups(const std::vector<std::optional<TString>>& groups) {
+ Cerr << ">>>>> TDescribeGroupsResponseData\n";
+ TRequestHeaderData header = Header(NKafka::EApiKey::DESCRIBE_GROUPS, 5);
+ TDescribeGroupsRequestData request;
+ request.Groups = groups;
+ return WriteAndRead<TDescribeGroupsResponseData>(header, request);
+}
+
TMessagePtr<TFetchResponseData> TKafkaTestClient::Fetch(const std::vector<std::pair<TString, std::vector<i32>>>& topics, i64 offset) {
Cerr << ">>>>> TFetchRequestData\n";
diff --git a/ydb/core/kafka_proxy/ut/kafka_test_client.h b/ydb/core/kafka_proxy/ut/kafka_test_client.h
index 354dd18d269..db12d6e64f0 100644
--- a/ydb/core/kafka_proxy/ut/kafka_test_client.h
+++ b/ydb/core/kafka_proxy/ut/kafka_test_client.h
@@ -9,7 +9,7 @@
using namespace NKafka;
-static constexpr ui32 EXPECTED_API_KEYS_COUNT = 22u;
+static constexpr ui32 EXPECTED_API_KEYS_COUNT = 24u;
struct TTopicConfig {
inline static const std::map<TString, TString> DummyMap;
@@ -72,11 +72,11 @@ class TKafkaTestClient {
TMessagePtr<TProduceResponseData> Produce(const TString& topicName, ui32 partition, const TKafkaRecordBatch& batch);
TMessagePtr<TProduceResponseData> Produce(const TString& topicName, const std::vector<std::pair<ui32, TKafkaRecordBatch>>& msgs, const std::optional<TString>& transactionalId = {});
-
- TMessagePtr<TProduceResponseData> Produce(const TTopicPartition& topicPartition,
- const std::vector<std::pair<TString, TString>>& keyValueMessages,
- ui32 baseSequence = 0,
- const std::optional<TProducerInstanceId>& producerInstanceId = {},
+
+ TMessagePtr<TProduceResponseData> Produce(const TTopicPartition& topicPartition,
+ const std::vector<std::pair<TString, TString>>& keyValueMessages,
+ ui32 baseSequence = 0,
+ const std::optional<TProducerInstanceId>& producerInstanceId = {},
const std::optional<TString>& transactionalId = {});
TMessagePtr<TListOffsetsResponseData> ListOffsets(std::vector<std::pair<i32,i64>>& partitions, const TString& topic);
@@ -105,6 +105,14 @@ class TKafkaTestClient {
TMessagePtr<TOffsetFetchResponseData> OffsetFetch(TOffsetFetchRequestData request);
+ TMessagePtr<TListGroupsResponseData> ListGroups(TListGroupsRequestData request);
+
+ TMessagePtr<TListGroupsResponseData> ListGroups(const std::vector<std::optional<TString>>& statesFilter);
+
+ TMessagePtr<TDescribeGroupsResponseData> DescribeGroups(TDescribeGroupsRequestData& request);
+
+ TMessagePtr<TDescribeGroupsResponseData> DescribeGroups(const std::vector<std::optional<TString>>& groups);
+
TMessagePtr<TFetchResponseData> Fetch(const std::vector<std::pair<TKafkaUuid, std::vector<i32>>>& topics, i64 offset = 0);
TMessagePtr<TFetchResponseData> Fetch(const std::vector<std::pair<TString, std::vector<i32>>>& topics, i64 offset = 0);
void ValidateNoDataInTopics(const std::vector<std::pair<TString, std::vector<i32>>>& topics, i64 offset = 0);
diff --git a/ydb/core/kafka_proxy/ut/ut_protocol.cpp b/ydb/core/kafka_proxy/ut/ut_protocol.cpp
index d55f81694b8..56f4b7c17f3 100644
--- a/ydb/core/kafka_proxy/ut/ut_protocol.cpp
+++ b/ydb/core/kafka_proxy/ut/ut_protocol.cpp
@@ -2516,6 +2516,298 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) {
UNIT_ASSERT_VALUES_EQUAL(metadataResponse->Brokers[0].Port, FAKE_SERVERLESS_KAFKA_PROXY_PORT);
}
+
+ Y_UNIT_TEST(DescribeGroupsScenario) {
+ TInsecureTestServer testServer("1", false, true);
+
+ TString topicName = "/Root/topic-0";
+ ui64 totalPartitions = 24;
+ TString groupId1 = "consumer-0";
+ TString groupId2 = "consumer-1";
+ TString groupId3 = "consumer-2";
+
+ TString protocolType = "consumer";
+ TString protocolName = "range";
+
+ TKafkaTestClient clientA(testServer.Port, "ClientA");
+ TKafkaTestClient clientB(testServer.Port, "ClientB");
+ TKafkaTestClient clientC(testServer.Port, "ClientC");
+
+ // Checking that DescribeGroups method works correctly if tables have not been inited yet
+
+ std::vector<std::optional<TString>> requestedGroups;
+ requestedGroups.push_back(groupId1);
+ auto response0 = clientA.DescribeGroups(requestedGroups);
+
+ UNIT_ASSERT_VALUES_EQUAL(response0->Groups.size(), 1);
+ UNIT_ASSERT_VALUES_EQUAL(response0->Groups[0].GroupId, groupId1);
+ UNIT_ASSERT_VALUES_EQUAL(response0->Groups[0].Members.size(), 0);
+ UNIT_ASSERT_VALUES_EQUAL(response0->Groups[0].ErrorCode, (TKafkaInt16)EKafkaErrors::GROUP_ID_NOT_FOUND);
+
+ // Creating 3 group members. One member of group "consumer-0" and two members of group "consumer-1"
+
+ {
+ NYdb::NTopic::TTopicClient pqClient(*testServer.Driver);
+ auto result = pqClient
+ .CreateTopic(
+ topicName,
+ NYdb::NTopic::TCreateTopicSettings()
+ .PartitioningSettings(totalPartitions, 100)
+ .BeginAddConsumer(groupId1).EndAddConsumer()
+ .BeginAddConsumer(groupId2).EndAddConsumer()
+ )
+ .ExtractValueSync();
+ UNIT_ASSERT_C(
+ result.IsSuccess(),
+ "CreateTopic failed, issues: " << result.GetIssues().ToString()
+ );
+ }
+
+
+
+ std::vector<TString> topics = {topicName};
+ i32 heartbeatTimeout = 15000;
+ i32 rebalanceTimeout = 5000;
+
+ TRequestHeaderData headerAJoin = clientA.Header(NKafka::EApiKey::JOIN_GROUP, 9);
+ TRequestHeaderData headerBJoin = clientB.Header(NKafka::EApiKey::JOIN_GROUP, 9);
+ TRequestHeaderData headerCJoin = clientC.Header(NKafka::EApiKey::JOIN_GROUP, 9);
+
+ TJoinGroupRequestData joinReq1;
+ joinReq1.GroupId = groupId1;
+ joinReq1.ProtocolType = protocolType;
+ joinReq1.SessionTimeoutMs = heartbeatTimeout;
+ joinReq1.RebalanceTimeoutMs = rebalanceTimeout;
+
+ NKafka::TJoinGroupRequestData::TJoinGroupRequestProtocol protocol;
+ protocol.Name = protocolName;
+
+ TConsumerProtocolSubscription subscribtion;
+ for (auto& topic : topics) {
+ subscribtion.Topics.push_back(topic);
+ }
+ TKafkaVersion version = 3;
+ TWritableBuf buf(nullptr, subscribtion.Size(version) + sizeof(version));
+ TKafkaWritable writable(buf);
+ writable << version;
+ subscribtion.Write(writable, version);
+ protocol.Metadata = TKafkaRawBytes(buf.GetBuffer().data(), buf.GetBuffer().size());
+
+ joinReq1.Protocols.push_back(protocol);
+
+ TJoinGroupRequestData joinReqA = joinReq1;
+ joinReqA.GroupInstanceId = "instanceA";
+
+ TJoinGroupRequestData joinReq2 = joinReq1;
+ joinReq2.GroupId = groupId2;
+
+ TJoinGroupRequestData joinReqB = joinReq2;
+ joinReqB.GroupInstanceId = "instanceB";
+
+ TJoinGroupRequestData joinReqC = joinReq2;
+ joinReqC.GroupInstanceId = "instanceC";
+
+ clientA.WriteToSocket(headerAJoin, joinReqA);
+ clientB.WriteToSocket(headerBJoin, joinReqB);
+ clientC.WriteToSocket(headerCJoin, joinReqC);
+
+ auto joinRespA = clientA.ReadResponse<TJoinGroupResponseData>(headerAJoin);
+ auto joinRespB = clientB.ReadResponse<TJoinGroupResponseData>(headerBJoin);
+ auto joinRespC = clientC.ReadResponse<TJoinGroupResponseData>(headerCJoin);
+
+ UNIT_ASSERT_VALUES_EQUAL(joinRespA->ErrorCode, (TKafkaInt16)EKafkaErrors::NONE_ERROR);
+ UNIT_ASSERT_VALUES_EQUAL(joinRespB->ErrorCode, (TKafkaInt16)EKafkaErrors::NONE_ERROR);
+ UNIT_ASSERT_VALUES_EQUAL(joinRespC->ErrorCode, (TKafkaInt16)EKafkaErrors::NONE_ERROR);
+
+ // check that DescribeGroups information is returned correctly when one group is requested
+
+ auto response1 = clientA.DescribeGroups(requestedGroups);
+ UNIT_ASSERT_VALUES_EQUAL(response1->Groups.size(), 1);
+ auto& groupResponse = response1->Groups[0];
+ UNIT_ASSERT(groupResponse.GroupId.has_value());
+ UNIT_ASSERT_VALUES_EQUAL(*groupResponse.GroupId, groupId1);
+ UNIT_ASSERT_VALUES_EQUAL(groupResponse.Members.size(), 1);
+
+ // check that for two existing requested groups DescribeGroups returns correct member information
+ // and for one unexisting requested group the returned response constains error
+
+ requestedGroups.push_back(groupId2);
+ requestedGroups.push_back(groupId3);
+ auto response2 = clientA.DescribeGroups(requestedGroups);
+ UNIT_ASSERT_VALUES_EQUAL(response2->Groups.size(), 3);
+ UNIT_ASSERT_VALUES_EQUAL(response2->Groups[0].GroupId, groupId1);
+ UNIT_ASSERT_VALUES_EQUAL(response2->Groups[0].Members.size(), 1);
+ UNIT_ASSERT_VALUES_EQUAL(response2->Groups[0].Members[0].MemberId, joinRespA->MemberId);
+ UNIT_ASSERT_VALUES_EQUAL(response2->Groups[0].ErrorCode, (TKafkaInt16)EKafkaErrors::NONE_ERROR);
+ UNIT_ASSERT_VALUES_EQUAL(response2->Groups[1].Members.size(), 2);
+ UNIT_ASSERT_VALUES_EQUAL(response2->Groups[1].GroupId, groupId2);
+ UNIT_ASSERT_VALUES_EQUAL(response2->Groups[1].ErrorCode, (TKafkaInt16)EKafkaErrors::NONE_ERROR);
+ UNIT_ASSERT_VALUES_EQUAL(response2->Groups[2].GroupId, groupId3);
+ UNIT_ASSERT_VALUES_EQUAL(response2->Groups[2].Members.size(), 0);
+ UNIT_ASSERT_VALUES_EQUAL(response2->Groups[2].ErrorCode, (TKafkaInt16)EKafkaErrors::GROUP_ID_NOT_FOUND);
+
+ ui32 memberIdBCount = 0;
+ ui32 memberIdCCount = 0;
+ ui32 wrongMemberIdCount = 0;
+ for (auto& member : response2->Groups[1].Members) {
+ if (member.MemberId == joinRespB->MemberId) {
+ memberIdBCount += 1;
+ } else if (member.MemberId == joinRespC->MemberId) {
+ memberIdCCount += 1;
+ } else {
+ wrongMemberIdCount += 1;
+ }
+ }
+ UNIT_ASSERT_VALUES_EQUAL(memberIdBCount, 1);
+ UNIT_ASSERT_VALUES_EQUAL(memberIdCCount, 1);
+ UNIT_ASSERT_VALUES_EQUAL(wrongMemberIdCount, 0);
+ }
+
+ Y_UNIT_TEST(ListGroupsScenario) {
+ TInsecureTestServer testServer("1", false, true);
+ TString groupId1 = "consumer-0";
+ TString groupId2 = "consumer-1";
+ TString topicName = "/Root/topic-0";
+ ui64 totalPartitions = 24;
+ TString protocolType = "consumer";
+ TString protocolName = "range";
+
+ TKafkaTestClient clientA(testServer.Port, "ClientA");
+ TKafkaTestClient clientB(testServer.Port, "ClientB");
+
+ // check that ListGroups doesn't fail if tables have not been inited yet
+
+ std::vector<std::optional<TString>> statesFilter = {"PreparingRebalance"};
+ auto responseBeforeTablesInit = clientA.ListGroups(statesFilter);
+ UNIT_ASSERT_VALUES_EQUAL(responseBeforeTablesInit->Groups.size(), 0);
+
+ {
+ NYdb::NTopic::TTopicClient pqClient(*testServer.Driver);
+ auto result = pqClient
+ .CreateTopic(
+ topicName,
+ NYdb::NTopic::TCreateTopicSettings()
+ .PartitioningSettings(totalPartitions, 100)
+ .BeginAddConsumer(groupId1).EndAddConsumer()
+ .BeginAddConsumer(groupId2).EndAddConsumer()
+ )
+ .ExtractValueSync();
+ UNIT_ASSERT_C(
+ result.IsSuccess(),
+ "CreateTopic failed, issues: " << result.GetIssues().ToString()
+ );
+ }
+
+
+ // check that before adding any consumers response will contain no groups
+
+ TListGroupsRequestData requestGroups;
+ auto responseEmpty = clientA.ListGroups(requestGroups);
+ Cout << "Recieved TListGroupsRequestData with " << responseEmpty->Groups.size() << Endl;
+ UNIT_ASSERT_VALUES_EQUAL(responseEmpty->Groups.size(), 0);
+
+ std::vector<TString> topics = {topicName};
+ i32 heartbeatTimeout = 15000;
+
+ auto joinRespA = clientA.JoinAndSyncGroupAndWaitPartitions(topics, groupId1, totalPartitions, protocolName, totalPartitions, heartbeatTimeout);
+ auto joinRespB = clientB.JoinAndSyncGroupAndWaitPartitions(topics, groupId2, totalPartitions, protocolName, totalPartitions, heartbeatTimeout);
+
+ // check that after two consumers have joined to two groups, they will be returned with correct status
+
+ auto response = clientA.ListGroups(requestGroups);
+
+ Cout << "Recieved TListGroupsRequestData with " << response->Groups.size() << Endl;
+ UNIT_ASSERT_VALUES_EQUAL(response->Groups.size(), 2);
+ ui32 first_group_count = 0;
+ ui32 second_group_count = 0;
+
+ // check that all metadata is correct and groups are in "preparing rebalance" state
+ for (auto group : response->Groups) {
+ UNIT_ASSERT_C(group.GroupId.has_value(),"Error, no groupId recieved");
+ UNIT_ASSERT_C(group.GroupState.has_value(),"Error, no GroupState recieved");
+ UNIT_ASSERT_C(group.ProtocolType.has_value(),"Error, no ProtocolType recieved");
+ UNIT_ASSERT_C(*group.GroupId == groupId1 || *group.GroupId == groupId2,"Error, wrong GroupId name" << group.GroupId);
+
+ if (*group.GroupId == groupId1) {
+ first_group_count += 1;
+ } else if (*group.GroupId == groupId2) {
+ second_group_count += 1;
+ }
+
+ UNIT_ASSERT_VALUES_EQUAL(*group.GroupState, "CompletingRebalance");
+ UNIT_ASSERT_VALUES_EQUAL(*group.ProtocolType, protocolType);
+
+ Cout << "********" << Endl;
+ Cout << "GroupId: " << *group.GroupId << Endl;
+ Cout << "GroupState: " << *group.GroupState << Endl;
+ Cout << "ProtocolType: " << *group.ProtocolType << Endl;
+
+ }
+ UNIT_ASSERT_VALUES_EQUAL(first_group_count, 1);
+ UNIT_ASSERT_VALUES_EQUAL(second_group_count, 1);
+
+
+ // now we want to check that after calling JoinGroup() Group2 will be in state of "Preparing Rebalance"
+ // because another consumer has joined Group2 recently
+
+ clientA.JoinGroup(topics, groupId2, protocolName, heartbeatTimeout);
+
+ TListGroupsRequestData requestGroups1;
+ auto response1 = clientB.ListGroups(requestGroups1);
+ Cout << "Recieved TListGroupsRequestData with " << response1->Groups.size() << Endl;
+
+ first_group_count = 0;
+ second_group_count = 0;
+ for (auto group : response1->Groups) {
+ UNIT_ASSERT_C(group.GroupId.has_value(),"Error, no groupId recieved");
+ UNIT_ASSERT_C(group.GroupState.has_value(),"Error, no GroupState recieved");
+ UNIT_ASSERT_C(group.ProtocolType.has_value(),"Error, no ProtocolType recieved");
+ UNIT_ASSERT_C(*group.GroupId == groupId1 || *group.GroupId == groupId2, "Error, wrong GroupId name" << group.GroupId);
+
+ if (*group.GroupId == groupId1) {
+ first_group_count += 1;
+ UNIT_ASSERT_VALUES_EQUAL(*group.GroupState, "CompletingRebalance");
+ } else if (*group.GroupId == groupId2) {
+ second_group_count += 1;
+ UNIT_ASSERT_VALUES_EQUAL(*group.GroupState, "PreparingRebalance");
+ }
+ UNIT_ASSERT_VALUES_EQUAL(*group.ProtocolType, protocolType);
+
+ Cout << "********" << Endl;
+ Cout << "GroupId: " << *group.GroupId << Endl;
+ Cout << "GroupState: " << *group.GroupState << Endl;
+ Cout << "ProtocolType: " << *group.ProtocolType << Endl;
+ }
+ UNIT_ASSERT_VALUES_EQUAL(first_group_count, 1);
+ UNIT_ASSERT_VALUES_EQUAL(second_group_count, 1);
+
+
+ // now we want to check that if StatesFilter is filled in TListGroupsRequestData
+ // than only consumers of certain states from StatesFilter are returned
+
+ TListGroupsRequestData requestGroupsStateFilter;
+ requestGroupsStateFilter.StatesFilter.push_back("PreparingRebalance");
+ auto responseStateFilter = clientA.ListGroups(requestGroupsStateFilter);
+
+ first_group_count = 0;
+ second_group_count = 0;
+ UNIT_ASSERT_VALUES_EQUAL(responseStateFilter->Groups.size(), 1);
+ for (auto group : responseStateFilter->Groups) {
+ UNIT_ASSERT_C(group.GroupId.has_value(),"Error, no groupId recieved");
+ UNIT_ASSERT_C(group.GroupState.has_value(),"Error, no GroupState recieved");
+ UNIT_ASSERT_C(group.ProtocolType.has_value(),"Error, no ProtocolType recieved");
+ UNIT_ASSERT_C(*group.GroupId == groupId1 || *group.GroupId == groupId2,"Error, wrong GroupId name" << group.GroupId);
+ UNIT_ASSERT_VALUES_EQUAL(*group.GroupId, groupId2);
+ UNIT_ASSERT_VALUES_EQUAL(*group.GroupState, "PreparingRebalance");
+ UNIT_ASSERT_VALUES_EQUAL(*group.ProtocolType, protocolType);
+
+ Cout << "********" << Endl;
+ Cout << "GroupId: " << *group.GroupId << Endl;
+ Cout << "GroupState: " << *group.GroupState << Endl;
+ Cout << "ProtocolType: " << *group.ProtocolType << Endl;
+ }
+ }
+
Y_UNIT_TEST(NativeKafkaBalanceScenario) {
TInsecureTestServer testServer("1", false, true);
@@ -2842,7 +3134,6 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) {
auto noMasterSyncResponse = clientA.ReadResponse<TSyncGroupResponseData>(syncHeaderNotMaster);
UNIT_ASSERT_VALUES_EQUAL(noMasterSyncResponse->ErrorCode, (TKafkaInt16)EKafkaErrors::REBALANCE_IN_PROGRESS);
}
-
}
Y_UNIT_TEST(InitProducerId_withoutTransactionalIdShouldReturnRandomInt) {
@@ -2965,7 +3256,7 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) {
auto transactionalId = TStringBuilder() << "my-tx-producer-" << TGUID::Create().AsUuidString();
auto resp = kafkaClient.InitProducerId(transactionalId, testServer.KikimrServer->GetRuntime()->GetAppData().KafkaProxyConfig.GetTransactionTimeoutMs() + 1);
-
+
UNIT_ASSERT_VALUES_EQUAL(resp->ErrorCode, EKafkaErrors::INVALID_TRANSACTION_TIMEOUT);
}
@@ -3084,7 +3375,7 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) {
auto consumerInfo = kafkaClient.JoinAndSyncGroupAndWaitPartitions(topicsToSubscribe, consumerName, 3, protocolName, 3, 15000);
kafkaClient.ValidateNoDataInTopics({{outputTopicName, {0}}});
-
+
// move time forward after transaction timeout
Sleep(TDuration::MilliSeconds(txnTimeoutMs));
diff --git a/ydb/core/kafka_proxy/ya.make b/ydb/core/kafka_proxy/ya.make
index 1e361a3799e..3c1b95c5caf 100644
--- a/ydb/core/kafka_proxy/ya.make
+++ b/ydb/core/kafka_proxy/ya.make
@@ -9,6 +9,8 @@ SRCS(
actors/kafka_sasl_handshake_actor.cpp
actors/kafka_metrics_actor.cpp
actors/kafka_list_offsets_actor.cpp
+ actors/kafka_list_groups_actor.cpp
+ actors/kafka_describe_groups_actor.cpp
actors/kafka_topic_offsets_actor.cpp
actors/kafka_fetch_actor.cpp
actors/kafka_find_coordinator_actor.cpp
@@ -24,6 +26,7 @@ SRCS(
actors/txn_actor_response_builder.cpp
actors/kafka_transaction_actor_sql.cpp
actors/kafka_transaction_actor.cpp
+ actors/kafka_state_name_to_int.cpp
kafka_connection.cpp
kafka_connection.h
kafka_constants.h