diff --git a/ydb/core/kafka_proxy/actors/actors.h b/ydb/core/kafka_proxy/actors/actors.h index 5102144fd98b..1fa2a9cd9eb4 100644 --- a/ydb/core/kafka_proxy/actors/actors.h +++ b/ydb/core/kafka_proxy/actors/actors.h @@ -176,6 +176,7 @@ NActors::IActor* CreateKafkaMetadataActor(const TContext::TPtr context, const ui const TActorId& discoveryCacheActor); NActors::IActor* CreateKafkaProduceActor(const TContext::TPtr context); NActors::IActor* CreateKafkaReadSessionActor(const TContext::TPtr context, ui64 cookie); +NActors::IActor* CreateKafkaBalancerActor(const TContext::TPtr context, ui64 cookie); NActors::IActor* CreateKafkaSaslHandshakeActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr& message); NActors::IActor* CreateKafkaSaslAuthActor(const TContext::TPtr context, const ui64 correlationId, const NKikimr::NRawSocket::TSocketDescriptor::TSocketAddressType address, const TMessagePtr& message); NActors::IActor* CreateKafkaListOffsetsActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr& message); diff --git a/ydb/core/kafka_proxy/actors/kafka_balance_actor_sql.cpp b/ydb/core/kafka_proxy/actors/kafka_balance_actor_sql.cpp new file mode 100644 index 000000000000..8155011b351d --- /dev/null +++ b/ydb/core/kafka_proxy/actors/kafka_balance_actor_sql.cpp @@ -0,0 +1,230 @@ +#include "kafka_balancer_actor.h" + +namespace NKafka { + +const TString INSERT_NEW_GROUP = R"sql( + --!syntax_v1 + DECLARE $ConsumerGroup AS Utf8; + DECLARE $Generation AS Uint64; + DECLARE $State AS Uint64; + DECLARE $Database AS Utf8; + DECLARE $Master AS Utf8; + DECLARE $LastHeartbeat AS Datetime; + DECLARE $ProtocolType AS Utf8; + + INSERT INTO `%s` + ( + consumer_group, + generation, + state, + database, + last_heartbeat_time, + master, + protocol_type + ) + VALUES + ( + $ConsumerGroup, + $Generation, + $State, + $Database, + $LastHeartbeat, + $Master, + $ProtocolType + ); +)sql"; + +const TString UPDATE_GROUP = R"sql( + --!syntax_v1 + DECLARE $ConsumerGroup AS Utf8; + DECLARE $State AS Uint64; + DECLARE $Generation AS Uint64; + DECLARE $Database AS Utf8; + DECLARE $Master AS Utf8; + DECLARE $LastHeartbeat AS Datetime; + + UPDATE `%s` + SET + state = $State, + generation = $Generation, + last_heartbeat_time = $LastHeartbeat, + master = $Master + WHERE consumer_group = $ConsumerGroup + AND database = $Database; +)sql"; + +const TString UPDATE_GROUP_STATE_AND_PROTOCOL = R"sql( + --!syntax_v1 + DECLARE $ConsumerGroup AS Utf8; + DECLARE $State AS Uint64; + DECLARE $Database AS Utf8; + DECLARE $Protocol AS Utf8; + DECLARE $LastHeartbeat AS Datetime; + + UPDATE `%s` + SET + state = $State, + last_heartbeat_time = $LastHeartbeat, + protocol = $Protocol + WHERE consumer_group = $ConsumerGroup + AND database = $Database; +)sql"; + +const TString INSERT_MEMBER = R"sql( + --!syntax_v1 + DECLARE $ConsumerGroup AS Utf8; + DECLARE $Generation AS Uint64; + DECLARE $MemberId AS Utf8; + DECLARE $WorkerStateProto AS String; + DECLARE $Database AS Utf8; + DECLARE $LastHeartbeat AS Datetime; + + INSERT INTO `%s` + ( + consumer_group, + generation, + member_id, + last_heartbeat_time, + worker_state_proto, + database + ) + VALUES ( + $ConsumerGroup, + $Generation, + $MemberId, + $LastHeartbeat, + $WorkerStateProto, + $Database + ); +)sql"; + +const TString UPSERT_ASSIGNMENTS_AND_SET_WORKING_STATE = R"sql( + --!syntax_v1 + DECLARE $Assignments AS List>; + DECLARE $ConsumerGroup AS Utf8; + DECLARE $Database AS Utf8; + DECLARE $Generation AS Uint64; + DECLARE $State AS Uint64; + DECLARE $LastHeartbeat AS Datetime; + + UPSERT INTO `%s` + SELECT + item.MemberId AS member_id, + item.Assignment AS assignment, + $ConsumerGroup AS consumer_group, + $Database AS database, + $Generation AS generation + FROM AS_TABLE($Assignments) AS item; + + UPDATE `%s` + SET + state = $State, + last_heartbeat_time = $LastHeartbeat + WHERE consumer_group = $ConsumerGroup + AND database = $Database; +)sql"; + +const TString UPDATE_GROUPS_AND_SELECT_WORKER_STATES = R"sql( + --!syntax_v1 + DECLARE $ConsumerGroup AS Utf8; + DECLARE $State AS Uint64; + DECLARE $Generation AS Uint64; + DECLARE $Database AS Utf8; + DECLARE $LastHeartbeat AS Datetime; + + UPDATE `%s` + SET + state = $State, + last_heartbeat_time = $LastHeartbeat + WHERE consumer_group = $ConsumerGroup + AND database = $Database; + + SELECT worker_state_proto, member_id + FROM `%s` + WHERE consumer_group = $ConsumerGroup + AND generation = $Generation + AND database = $Database; +)sql"; + +const TString CHECK_GROUP_STATE = R"sql( + --!syntax_v1 + DECLARE $ConsumerGroup AS Utf8; + DECLARE $Database AS Utf8; + + SELECT state, generation, master, last_heartbeat_time, consumer_group, database, protocol + FROM `%s` + WHERE consumer_group = $ConsumerGroup + AND database = $Database; +)sql"; + +const TString FETCH_ASSIGNMENTS = R"sql( + --!syntax_v1 + DECLARE $ConsumerGroup AS Utf8; + DECLARE $Generation AS Uint64; + DECLARE $MemberId AS Utf8; + DECLARE $Database AS Utf8; + + SELECT assignment + FROM `%s` + WHERE consumer_group = $ConsumerGroup + AND generation = $Generation + AND member_id = $MemberId + AND database = $Database; +)sql"; + +const TString CHECK_DEAD_MEMBERS = R"sql( + --!syntax_v1 + DECLARE $ConsumerGroup AS Utf8; + DECLARE $Generation AS Uint64; + DECLARE $Database AS Utf8; + DECLARE $Deadline AS Datetime; + + SELECT COUNT(1) as cnt + FROM `%s` + WHERE consumer_group = $ConsumerGroup + AND generation = $Generation + AND database = $Database + AND last_heartbeat_time < $Deadline; +)sql"; + +const TString UPDATE_LASTHEARTBEATS = R"sql( + --!syntax_v1 + DECLARE $ConsumerGroup AS Utf8; + DECLARE $Generation AS Uint64; + DECLARE $MemberId AS Utf8; + DECLARE $Database AS Utf8; + DECLARE $LastHeartbeat AS Datetime; + DECLARE $UpdateGroupHeartbeat AS Bool; + + UPDATE `%s` + SET last_heartbeat_time = $LastHeartbeat + WHERE consumer_group = $ConsumerGroup + AND database = $Database + AND $UpdateGroupHeartbeat = True; + + UPDATE `%s` + SET last_heartbeat_time = $LastHeartbeat + WHERE consumer_group = $ConsumerGroup + AND generation = $Generation + AND member_id = $MemberId + AND database = $Database; +)sql"; + + +const TString UPDATE_LASTHEARTBEAT_TO_LEAVE_GROUP = R"sql( + --!syntax_v1 + DECLARE $ConsumerGroup AS Utf8; + DECLARE $MemberId AS Utf8; + DECLARE $Database AS Utf8; + DECLARE $LastHeartbeat AS Datetime; + + UPDATE `%s` + SET last_heartbeat_time = $LastHeartbeat + WHERE consumer_group = $ConsumerGroup + AND member_id = $MemberId + AND database = $Database; +)sql"; + +} // namespace NKafka + +// savnik check max members count diff --git a/ydb/core/kafka_proxy/actors/kafka_balancer_actor.cpp b/ydb/core/kafka_proxy/actors/kafka_balancer_actor.cpp new file mode 100644 index 000000000000..55fd907c49f7 --- /dev/null +++ b/ydb/core/kafka_proxy/actors/kafka_balancer_actor.cpp @@ -0,0 +1,1139 @@ +#include "kafka_balancer_actor.h" +#include "kqp_balance_transaction.h" + +namespace NKafka { + +using namespace NKikimr; +using namespace NKikimr::NGRpcProxy::V1; + +static constexpr ui8 MASTER_WAIT_JOINS_DELAY_SECONDS = 5; +static constexpr ui8 WAIT_FOR_MASTER_DELAY_SECONDS = 2; +static constexpr ui8 WAIT_MASTER_MAX_RETRY_COUNT = 5; +static constexpr ui8 TX_ABORT_MAX_RETRY_COUNT = 5; +static constexpr ui8 TABLES_TO_INIT_COUNT = 2; + +void TKafkaBalancerActor::Bootstrap(const NActors::TActorContext& ctx) { + Kqp = std::make_unique(Context->DatabasePath); + Kqp->SendInitTablesRequest(ctx); + Become(&TKafkaBalancerActor::StateWork); + if (!MemberId) { + MemberId = SelfId().ToString(); + } +} + +static EKafkaErrors KqpStatusToKafkaError(Ydb::StatusIds::StatusCode status) { + // savnik finish it + if (status == Ydb::StatusIds::SUCCESS) { + return EKafkaErrors::NONE_ERROR; + } + return EKafkaErrors::UNKNOWN_SERVER_ERROR; +} + +TString TKafkaBalancerActor::LogPrefix() { + TStringBuilder sb; + sb << "TKafkaBalancerActor: "; + return sb; +} + +void TKafkaBalancerActor::Handle(NMetadata::NProvider::TEvManagerPrepared::TPtr&, const TActorContext& ctx) { + TablesInited++; + if (TablesInited == TABLES_TO_INIT_COUNT) { + Kqp->SendCreateSessionRequest(ctx); + } +} + +void TKafkaBalancerActor::Handle(NKqp::TEvKqp::TEvCreateSessionResponse::TPtr& ev, const TActorContext& ctx) { + Cookie = 0; // savnik + + if (!Kqp->HandleCreateSessionResponse(ev, ctx)) { + SendResponseFail(ctx, EKafkaErrors::UNKNOWN_SERVER_ERROR, "Failed to create KQP session"); + PassAway(); + return; + } + + HandleResponse(nullptr, ctx); +} + +void TKafkaBalancerActor::Handle(NKqp::TEvKqp::TEvQueryResponse::TPtr& ev, const TActorContext& ctx) { + if (ev->Cookie != KqpReqCookie) { + KAFKA_LOG_ERROR("Unexpected cookie in TEvQueryResponse"); + return; + } + + const auto& record = ev->Get()->Record; + auto status = record.GetYdbStatus(); + if (status == ::Ydb::StatusIds_StatusCode::StatusIds_StatusCode_ABORTED && CurrentRetryNumber < TX_ABORT_MAX_RETRY_COUNT) { + CurrentRetryNumber++; + KAFKA_LOG_I(TStringBuilder() << "Retry after tx aborted. Num of retry# " << CurrentRetryNumber); + switch (RequestType) { + case JOIN_GROUP: + Register(new TKafkaBalancerActor(Context, Cookie, CorrelationId, JoinGroupRequestData, CurrentRetryNumber)); + break; + case SYNC_GROUP: + Register(new TKafkaBalancerActor(Context, Cookie, CorrelationId, SyncGroupRequestData, CurrentRetryNumber)); + break; + case LEAVE_GROUP: + Register(new TKafkaBalancerActor(Context, Cookie, CorrelationId, LeaveGroupRequestData, CurrentRetryNumber)); + break; + case HEARTBEAT: + Register(new TKafkaBalancerActor(Context, Cookie, CorrelationId, HeartbeatGroupRequestData, CurrentRetryNumber)); + break; + default: + break; + } + + PassAway(); + return; + } + + auto kafkaErr = KqpStatusToKafkaError(status); + + if (kafkaErr != EKafkaErrors::NONE_ERROR) { + auto kqpQueryError = TStringBuilder() <<" Kqp error. "; + + NYql::TIssues issues; + NYql::IssuesFromMessage(record.GetResponse().GetQueryIssues(), issues); + NYdb::TStatus status(NYdb::EStatus(record.GetYdbStatus()), std::move(issues)); + kqpQueryError << status; + + SendResponseFail(ctx, kafkaErr, kqpQueryError); + } + + HandleResponse(ev, ctx); +} + +void TKafkaBalancerActor::HandleResponse(NKqp::TEvKqp::TEvQueryResponse::TPtr ev, const TActorContext& ctx) { + KAFKA_LOG_I(TStringBuilder() << "Handle kqp response. CurrentStep# " << (ui8)CurrentStep); + switch (RequestType) { + case JOIN_GROUP: + HandleJoinGroupResponse(ev, ctx); + break; + case SYNC_GROUP: + HandleSyncGroupResponse(ev, ctx); + break; + case LEAVE_GROUP: + HandleLeaveGroupResponse(ev, ctx); + break; + case HEARTBEAT: + HandleHeartbeatResponse(ev, ctx); + break; + default: + KAFKA_LOG_ERROR("Unknown RequestType in TEvCreateSessionResponse"); + PassAway(); + break; + } +} + +void TKafkaBalancerActor::SendResponseFail(const TActorContext& ctx, EKafkaErrors error, const TString& message) { + switch (RequestType) { + case JOIN_GROUP: + SendJoinGroupResponseFail(ctx, CorrelationId, error, message); + break; + case SYNC_GROUP: + SendSyncGroupResponseFail(ctx, CorrelationId, error, message); + break; + case LEAVE_GROUP: + SendLeaveGroupResponseFail(ctx, CorrelationId, error, message); + break; + case HEARTBEAT: + SendHeartbeatResponseFail(ctx, CorrelationId, error, message); + break; + default: + break; + } +} + +std::optional TKafkaBalancerActor::ParseCheckStateAndGeneration( + NKqp::TEvKqp::TEvQueryResponse::TPtr ev +) { + if (!ev) { + return std::nullopt; + } + + TGroupStatus result; + + auto& record = ev->Get()->Record; + auto& resp = record.GetResponse(); + if (resp.GetYdbResults().empty()) { + result.Exists = false; + return result; + } + + NYdb::TResultSetParser parser(resp.GetYdbResults(0)); + if (!parser.TryNextRow()) { + result.Exists = false; + return result; + } + + result.State = parser.ColumnParser("state").GetOptionalUint64().GetOrElse(0); + result.Generation = parser.ColumnParser("generation").GetOptionalUint64().GetOrElse(0); + result.MasterId = parser.ColumnParser("master").GetOptionalUtf8().GetOrElse(""); + result.LastHeartbeat = parser.ColumnParser("last_heartbeat_time").GetOptionalDatetime().GetOrElse(TInstant::Zero()); + result.ProtocolName = parser.ColumnParser("protocol").GetOptionalUtf8().GetOrElse(""); + result.Exists = true; + + if (parser.TryNextRow()) { + return std::nullopt; + } + + return result; +} + +bool TKafkaBalancerActor::ParseAssignments( + NKqp::TEvKqp::TEvQueryResponse::TPtr ev, + TString& assignments) +{ + if (!ev) { + return false; + } + + auto& record = ev->Get()->Record; + if (record.GetResponse().GetYdbResults().empty()) { + return false; + } + + NYdb::TResultSetParser parser(record.GetResponse().GetYdbResults(0)); + assignments.clear(); + + if (!parser.TryNextRow()) { + return false; + } + + assignments = parser.ColumnParser("assignment").GetOptionalString().GetOrElse(""); + + if (parser.TryNextRow()) { + return false; + } + + return !assignments.empty(); +} + + +bool TKafkaBalancerActor::ParseWorkerStatesAndChooseProtocol( + NKqp::TEvKqp::TEvQueryResponse::TPtr ev, + std::unordered_map& workerStates, + TString& chosenProtocol) +{ + if (!ev) { + return false; + } + + auto& record = ev->Get()->Record; + if (record.GetResponse().GetYdbResults().empty()) { + return false; + } + + NYdb::TResultSetParser parser(record.GetResponse().GetYdbResults(0)); + + struct TParsedState { + TString MemberId; + NKafka::TWorkerState WorkerState; + }; + + std::vector states; + while (parser.TryNextRow()) { + TString protoStr = parser.ColumnParser("worker_state_proto").GetOptionalString().GetOrElse(""); + TString memberId = parser.ColumnParser("member_id").GetOptionalUtf8().GetOrElse(""); + + NKafka::TWorkerState workerState; + if (!protoStr.empty() && !workerState.ParseFromString(protoStr)) { + return false; + } + + TParsedState st; + st.MemberId = memberId; + st.WorkerState = std::move(workerState); + states.push_back(std::move(st)); + } + + if (states.empty()) { + return false; + } + + const auto& firstWorkerProtocols = states.front().WorkerState.protocols(); + for (const auto& protocol : firstWorkerProtocols) { + const TString& candidateName = protocol.protocol_name(); + bool allHaveCandidate = true; + + for (size_t i = 1; i < states.size(); ++i) { + const auto& otherWorkerProtocols = states[i].WorkerState.protocols(); + bool foundThisCandidate = false; + for (const auto& otherProtocol : otherWorkerProtocols) { + if (otherProtocol.protocol_name() == candidateName) { + foundThisCandidate = true; + break; + } + } + + if (!foundThisCandidate) { + allHaveCandidate = false; + break; + } + } + + if (allHaveCandidate) { + chosenProtocol = candidateName; + break; + } + } + + if (chosenProtocol.empty()) { + return false; + } + + for (const auto& st : states) { + const auto& protocols = st.WorkerState.protocols(); + for (const auto& pr : protocols) { + if (pr.protocol_name() == chosenProtocol) { + workerStates[st.MemberId] = pr.metadata(); + break; + } + } + } + + return true; +} + +bool TKafkaBalancerActor::ParseDeadCount( + NKqp::TEvKqp::TEvQueryResponse::TPtr ev, + ui64& deadCount +) { + if (!ev) { + return false; + } + + auto& record = ev->Get()->Record; + auto& resp = record.GetResponse(); + if (resp.GetYdbResults().empty()) { + return false; + } + + NYdb::TResultSetParser parser(resp.GetYdbResults(0)); + if (!parser.TryNextRow()) { + return false; + } + + deadCount = parser.ColumnParser(0).GetUint64(); + if (parser.TryNextRow()) { + return false; + } + + return true; +} + +void TKafkaBalancerActor::Die(const TActorContext& ctx) { + KAFKA_LOG_D("Pass away"); + TBase::Die(ctx); +} + +NYdb::TParamsBuilder TKafkaBalancerActor::BuildCheckGroupStateParams() { + NYdb::TParamsBuilder params; + params.AddParam("$ConsumerGroup").Utf8(GroupId).Build(); + params.AddParam("$Database").Utf8(Kqp->DataBase).Build(); + + return params; +} + +NYdb::TParamsBuilder TKafkaBalancerActor::BuildUpdateOrInsertNewGroupParams() { + NYdb::TParamsBuilder params; + params.AddParam("$ConsumerGroup").Utf8(GroupId).Build(); + params.AddParam("$Generation").Uint64(GenerationId).Build(); + params.AddParam("$State").Uint64(GROUP_STATE_JOIN).Build(); + params.AddParam("$Database").Utf8(Kqp->DataBase).Build(); + params.AddParam("$Master").Utf8(MemberId).Build(); + params.AddParam("$LastHeartbeat").Datetime(TInstant::Now()).Build(); + params.AddParam("$ProtocolType").Utf8(ProtocolType).Build(); + + return params; +} + +NYdb::TParamsBuilder TKafkaBalancerActor::BuildInsertMemberParams() { + NYdb::TParamsBuilder params; + params.AddParam("$ConsumerGroup").Utf8(GroupId).Build(); + params.AddParam("$Generation").Uint64(GenerationId).Build(); + params.AddParam("$MemberId").Utf8(MemberId).Build(); + params.AddParam("$Database").Utf8(Kqp->DataBase).Build(); + params.AddParam("$LastHeartbeat").Datetime(TInstant::Now()).Build(); + + NKafka::TWorkerState workerState; + for (const auto& protocol : JoinGroupRequestData->Protocols) { + auto* item = workerState.add_protocols(); + item->set_protocol_name(protocol.Name.value()); + + auto dataRef = protocol.Metadata.value(); + item->mutable_metadata()->assign(dataRef.data(), dataRef.size()); + } + + TString serializedWorkerState = workerState.SerializeAsString(); + params.AddParam("$WorkerStateProto").String(serializedWorkerState).Build(); + + return params; +} + +NYdb::TParamsBuilder TKafkaBalancerActor::BuildAssignmentsParams() { + NYdb::TParamsBuilder params; + params.AddParam("$ConsumerGroup").Utf8(GroupId).Build(); + params.AddParam("$Database").Utf8(Kqp->DataBase).Build(); + params.AddParam("$Generation").Uint64(GenerationId).Build(); + params.AddParam("$State").Uint64(GROUP_STATE_WORKING).Build(); + params.AddParam("$LastHeartbeat").Datetime(TInstant::Now()).Build(); + + auto& assignmentList = params.AddParam("$Assignments").BeginList(); + + KAFKA_LOG_D(TStringBuilder() << "Assignments count: " << SyncGroupRequestData->Assignments.size()); + + for (auto& assignment: SyncGroupRequestData->Assignments) { + assignmentList.AddListItem() + .BeginStruct() + .AddMember("MemberId").Utf8(assignment.MemberId.value()) + .AddMember("Assignment").String(TString(assignment.Assignment.value().data(), + assignment.Assignment.value().size())) + .EndStruct(); + } + assignmentList.EndList().Build(); + + return params; +} + +NYdb::TParamsBuilder TKafkaBalancerActor::BuildUpdatesGroupsAndSelectWorkerStatesParams() { + NYdb::TParamsBuilder params; + params.AddParam("$ConsumerGroup").Utf8(GroupId).Build(); + params.AddParam("$State").Uint64(GROUP_STATE_JOINED).Build(); + params.AddParam("$Database").Utf8(Kqp->DataBase).Build(); + params.AddParam("$Generation").Uint64(GenerationId).Build(); + params.AddParam("$LastHeartbeat").Datetime(TInstant::Now()).Build(); + + return params; +} + +NYdb::TParamsBuilder TKafkaBalancerActor::BuildUpdateGroupStateAndProtocolParams() { + NYdb::TParamsBuilder params; + params.AddParam("$ConsumerGroup").Utf8(GroupId).Build(); + params.AddParam("$State").Uint64(GROUP_STATE_SYNC).Build(); + params.AddParam("$Database").Utf8(Kqp->DataBase).Build(); + params.AddParam("$Protocol").Utf8(Protocol).Build(); + params.AddParam("$LastHeartbeat").Datetime(TInstant::Now()).Build(); + + return params; +} + +NYdb::TParamsBuilder TKafkaBalancerActor::BuildFetchAssignmentsParams() { + NYdb::TParamsBuilder params; + params.AddParam("$ConsumerGroup").Utf8(GroupId).Build(); + params.AddParam("$Generation").Uint64(GenerationId).Build(); + params.AddParam("$MemberId").Utf8(MemberId).Build(); + params.AddParam("$Database").Utf8(Kqp->DataBase).Build(); + + return params; +} + +NYdb::TParamsBuilder TKafkaBalancerActor::BuildLeaveGroupParams() { + NYdb::TParamsBuilder params; + params.AddParam("$ConsumerGroup").Utf8(GroupId).Build(); + params.AddParam("$MemberId").Utf8(MemberId).Build(); + params.AddParam("$Database").Utf8(Kqp->DataBase).Build(); + params.AddParam("$LastHeartbeat").Datetime(TInstant::Now() - TDuration::Hours(1)).Build(); + + return params; +} + +NYdb::TParamsBuilder TKafkaBalancerActor::BuildUpdateLastHeartbeatsParams() { + NYdb::TParamsBuilder params; + params.AddParam("$ConsumerGroup").Utf8(GroupId).Build(); + params.AddParam("$Generation").Uint64(GenerationId).Build(); + params.AddParam("$MemberId").Utf8(MemberId).Build(); + params.AddParam("$Database").Utf8(Kqp->DataBase).Build(); + params.AddParam("$LastHeartbeat").Datetime(TInstant::Now()).Build(); + + if (IsMaster) { + params.AddParam("$UpdateGroupHeartbeat").Bool(true).Build(); + } else { + params.AddParam("$UpdateGroupHeartbeat").Bool(false).Build(); + } + + return params; +} + +NYdb::TParamsBuilder TKafkaBalancerActor::BuildCheckDeadsParams() { + NYdb::TParamsBuilder params; + params.AddParam("$ConsumerGroup").Utf8(GroupId).Build(); + params.AddParam("$Generation").Uint64(GenerationId).Build(); + params.AddParam("$Database").Utf8(Kqp->DataBase).Build(); + params.AddParam("$Deadline").Datetime(TInstant::Now() - TDuration::Seconds(30)).Build(); + + return params; +} + +void TKafkaBalancerActor::Handle(TEvents::TEvWakeup::TPtr&, const TActorContext& ctx) { + if (RequestType == JOIN_GROUP) { + HandleJoinGroupResponse(nullptr, ctx); + } else if (RequestType == SYNC_GROUP) { + HandleSyncGroupResponse(nullptr, ctx); + } +} + +void TKafkaBalancerActor::HandleJoinGroupResponse( + NKqp::TEvKqp::TEvQueryResponse::TPtr ev, + const TActorContext& ctx +) { + + switch (CurrentStep) { + case STEP_NONE: { + CurrentStep = JOIN_TX0_0_BEGIN_TX; + KqpReqCookie++; + Kqp->BeginTransaction(KqpReqCookie, ctx); + break; + } + + case JOIN_TX0_0_BEGIN_TX: { + Kqp->TxId = ev->Get()->Record.GetResponse().GetTxMeta().id(); + CurrentStep = JOIN_TX0_1_CHECK_STATE_AND_GENERATION; + KqpReqCookie++; + + // savnik yql compile? + NYdb::TParamsBuilder params = BuildCheckGroupStateParams(); + + Kqp->SendYqlRequest(Sprintf(CHECK_GROUP_STATE.c_str(), TKafkaConsumerGroupsMetaInitManager::GetInstant()->GetStorageTablePath().c_str()), params.Build(), KqpReqCookie, ctx); + break; + } + + case JOIN_TX0_1_CHECK_STATE_AND_GENERATION: { + auto groupStatus = ParseCheckStateAndGeneration(ev); + if (!groupStatus) { + SendJoinGroupResponseFail(ctx, CorrelationId, + EKafkaErrors::UNKNOWN_SERVER_ERROR, + "Can't get group state"); + PassAway(); + return; + } + + TString yqlRequest; + EBalancerStep nextStep; + + ui64 newGeneration = 0; + if (!groupStatus->Exists) { + nextStep = JOIN_TX0_2_INSERT_NEW_GROUP; + yqlRequest = INSERT_NEW_GROUP; + } else if (groupStatus->State != GROUP_STATE_JOIN) { + newGeneration = groupStatus->Generation + 1; + nextStep = JOIN_TX0_2_UPDATE_GROUP_STATE_AND_GENERATION; + yqlRequest = UPDATE_GROUP; + } else { + IsMaster = false; + CurrentStep = JOIN_TX0_2_SKIP; + GenerationId = groupStatus->Generation; + HandleJoinGroupResponse(ev, ctx); + return; + } + + GenerationId = newGeneration; + CurrentStep = nextStep; + KqpReqCookie++; + + NYdb::TParamsBuilder params = BuildUpdateOrInsertNewGroupParams(); + + Kqp->SendYqlRequest(Sprintf(yqlRequest.c_str(), TKafkaConsumerGroupsMetaInitManager::GetInstant()->GetStorageTablePath().c_str()), params.Build(), KqpReqCookie, ctx); + + break; + } + + case JOIN_TX0_2_SKIP: + case JOIN_TX0_2_INSERT_NEW_GROUP: + case JOIN_TX0_2_UPDATE_GROUP_STATE_AND_GENERATION: { + if (CurrentStep != JOIN_TX0_2_SKIP) { + IsMaster = true; + Master = SelfId().ToString(); + } + + CurrentStep = JOIN_TX0_3_INSERT_MEMBER; + KqpReqCookie++; + + + NYdb::TParamsBuilder params = BuildInsertMemberParams(); + Kqp->SendYqlRequest(Sprintf(INSERT_MEMBER.c_str(), TKafkaConsumerMembersMetaInitManager::GetInstant()->GetStorageTablePath().c_str()), params.Build(), KqpReqCookie, ctx); + break; + } + + + case JOIN_TX0_3_INSERT_MEMBER: { + CurrentStep = JOIN_TX0_4_COMMIT_TX; + KqpReqCookie++; + Kqp->CommitTx(KqpReqCookie, ctx); + break; + } + + case JOIN_TX0_4_COMMIT_TX: { + CurrentStep = JOIN_TX0_5_WAIT; + + if (IsMaster) { + auto wakeup = std::make_unique(0); + ctx.ActorSystem()->Schedule( + TDuration::Seconds(MASTER_WAIT_JOINS_DELAY_SECONDS), + new IEventHandle(SelfId(), SelfId(), wakeup.release()) + ); + } else { + HandleJoinGroupResponse(nullptr, ctx); + } + + break; + } + + case JOIN_TX0_5_WAIT: { + KqpReqCookie++; + CurrentStep = JOIN_TX1_0_BEGIN_TX; + Kqp->BeginTransaction(KqpReqCookie, ctx); + break; + } + + case JOIN_TX1_0_BEGIN_TX: { + CurrentStep = JOIN_TX1_1_CHECK_STATE_AND_GENERATION; + KqpReqCookie++; + Kqp->TxId = ev->Get()->Record.GetResponse().GetTxMeta().id(); + + NYdb::TParamsBuilder params = BuildCheckGroupStateParams(); + + Kqp->SendYqlRequest(Sprintf(CHECK_GROUP_STATE.c_str(), TKafkaConsumerGroupsMetaInitManager::GetInstant()->GetStorageTablePath().c_str()), params.Build(), KqpReqCookie, ctx); + break; + } + + case JOIN_TX1_1_CHECK_STATE_AND_GENERATION: { + auto groupStatus = ParseCheckStateAndGeneration(ev); + + if (!groupStatus) { + SendJoinGroupResponseFail(ctx, CorrelationId, + EKafkaErrors::UNKNOWN_SERVER_ERROR, + "Can't get group state"); + PassAway(); + return; + } + + if (IsMaster) { + if (!groupStatus->Exists || groupStatus->State != GROUP_STATE_JOIN || groupStatus->Generation != GenerationId) { + SendJoinGroupResponseFail(ctx, CorrelationId, + EKafkaErrors::REBALANCE_IN_PROGRESS, + "Rebalance"); + PassAway(); + return; + } + + CurrentStep = JOIN_TX1_2_GET_MEMBERS_AND_SET_STATE_SYNC; + KqpReqCookie++; + + NYdb::TParamsBuilder params = BuildUpdatesGroupsAndSelectWorkerStatesParams(); + + auto sql = Sprintf(UPDATE_GROUPS_AND_SELECT_WORKER_STATES.c_str(), TKafkaConsumerGroupsMetaInitManager::GetInstant()->GetStorageTablePath().c_str(), TKafkaConsumerMembersMetaInitManager::GetInstant()->GetStorageTablePath().c_str()); + + Kqp->SendYqlRequest(sql, params.Build(), KqpReqCookie, ctx); + } else { + if (!groupStatus->Exists || groupStatus->Generation != GenerationId) { + SendJoinGroupResponseFail(ctx, CorrelationId, + EKafkaErrors::REBALANCE_IN_PROGRESS, + "Rebalance"); + PassAway(); + return; + } + + if (groupStatus->State < GROUP_STATE_SYNC) { + if (WaitingWorkingStateRetries == WAIT_MASTER_MAX_RETRY_COUNT) { + SendJoinGroupResponseFail(ctx, CorrelationId, REBALANCE_IN_PROGRESS); //savnik какой статус отдать, чтобы клиент поретраил? + PassAway(); + return; + } + + CurrentStep = JOIN_TX0_5_WAIT; + auto wakeup = std::make_unique(1); + ctx.ActorSystem()->Schedule( + TDuration::Seconds(WAIT_FOR_MASTER_DELAY_SECONDS), + new IEventHandle(SelfId(), SelfId(), wakeup.release()) + ); + WaitingWorkingStateRetries++; + return; + } + + CurrentStep = JOIN_TX1_2_GET_MEMBERS_AND_SET_STATE_SYNC; + HandleJoinGroupResponse(ev, ctx); + return; + } + break; + } + + case JOIN_TX1_2_GET_MEMBERS_AND_SET_STATE_SYNC: { + if (IsMaster) { + if (!ParseWorkerStatesAndChooseProtocol(ev, WorkerStates, Protocol)) { + SendJoinGroupResponseFail(ctx, CorrelationId, + EKafkaErrors::INVALID_REQUEST, + "Can't get workers state"); + PassAway(); + return; + } + + KqpReqCookie++; + NYdb::TParamsBuilder params = BuildUpdateGroupStateAndProtocolParams(); + Kqp->SendYqlRequest(Sprintf(UPDATE_GROUP_STATE_AND_PROTOCOL.c_str(), TKafkaConsumerGroupsMetaInitManager::GetInstant()->GetStorageTablePath().c_str()), params.Build(), KqpReqCookie, ctx, true); + } else { + KqpReqCookie++; + Kqp->CommitTx(KqpReqCookie, ctx); + } + + CurrentStep = JOIN_TX1_3_COMMIT_TX; + + break; + } + + case JOIN_TX1_3_COMMIT_TX: { + SendJoinGroupResponseOk(ctx, CorrelationId); + PassAway(); + break; + } + + default: + KAFKA_LOG_CRIT("JOIN_GROUP: Unexpected step" ); + SendJoinGroupResponseFail(ctx, CorrelationId, EKafkaErrors::UNKNOWN_SERVER_ERROR, "Unexpected step"); + PassAway(); + break; + } // switch (CurrentStep) +} + +void TKafkaBalancerActor::HandleSyncGroupResponse( + NKqp::TEvKqp::TEvQueryResponse::TPtr ev, + const TActorContext& ctx +) { + switch (CurrentStep) { + case STEP_NONE: { + CurrentStep = SYNC_TX0_0_BEGIN_TX; + KqpReqCookie++; + Kqp->BeginTransaction(KqpReqCookie, ctx); + break; + } + + case SYNC_TX0_0_BEGIN_TX: { + CurrentStep = SYNC_TX0_2_CHECK_STATE_AND_GENERATION; + KqpReqCookie++; + Kqp->TxId = ev->Get()->Record.GetResponse().GetTxMeta().id(); + + NYdb::TParamsBuilder params = BuildCheckGroupStateParams(); + Kqp->SendYqlRequest(Sprintf(CHECK_GROUP_STATE.c_str(), TKafkaConsumerGroupsMetaInitManager::GetInstant()->GetStorageTablePath().c_str()), params.Build(), KqpReqCookie, ctx); + break; + } + + case SYNC_TX0_2_CHECK_STATE_AND_GENERATION: { + auto groupStatus = ParseCheckStateAndGeneration(ev); + + if (!groupStatus) { + SendSyncGroupResponseFail(ctx, CorrelationId, + EKafkaErrors::UNKNOWN_SERVER_ERROR, + "Can't get group state"); + PassAway(); + return; + } + + if (!groupStatus->Exists) { + SendSyncGroupResponseFail(ctx, CorrelationId, + EKafkaErrors::GROUP_ID_NOT_FOUND, + "Unknown group# " + GroupId); + PassAway(); + return; + } + + if (groupStatus->Generation != GenerationId) { + SendSyncGroupResponseFail(ctx, CorrelationId, + EKafkaErrors::ILLEGAL_GENERATION, + TStringBuilder() << "Old or unknown group generation# " << GenerationId); + PassAway(); + return; + } + + if (groupStatus->State < GROUP_STATE_SYNC) { + SendSyncGroupResponseFail(ctx, CorrelationId, + EKafkaErrors::INVALID_REQUEST, + TStringBuilder() << "Unexpected group state# " << groupStatus->State); + PassAway(); + return; + } + + Master = groupStatus->MasterId; + + if (MemberId == groupStatus->MasterId) { + IsMaster = true; + } + + CurrentStep = SYNC_TX0_3_SET_ASSIGNMENTS_AND_SET_WORKING_STATE; + + if (IsMaster) { + if (SyncGroupRequestData->Assignments.size() == 0) { + SendSyncGroupResponseFail(ctx, CorrelationId, EKafkaErrors::INVALID_REQUEST); + PassAway(); + return; + } + + KqpReqCookie++; + NYdb::TParamsBuilder params = BuildAssignmentsParams(); + Kqp->SendYqlRequest(Sprintf(UPSERT_ASSIGNMENTS_AND_SET_WORKING_STATE.c_str(), TKafkaConsumerMembersMetaInitManager::GetInstant()->GetStorageTablePath().c_str(), TKafkaConsumerGroupsMetaInitManager::GetInstant()->GetStorageTablePath().c_str()), params.Build(), KqpReqCookie, ctx); + } else { + HandleSyncGroupResponse(nullptr, ctx); + } + break; + } + + case SYNC_TX0_3_SET_ASSIGNMENTS_AND_SET_WORKING_STATE: { + CurrentStep = SYNC_TX0_4_COMMIT_TX; + KqpReqCookie++; + Kqp->CommitTx(KqpReqCookie, ctx); + break; + } + + case SYNC_TX0_4_COMMIT_TX: { + CurrentStep = SYNC_TX1_0_BEGIN_TX; + KqpReqCookie++; + Kqp->BeginTransaction(KqpReqCookie, ctx); + break; + } + + case SYNC_TX1_0_BEGIN_TX: { + CurrentStep = SYNC_TX1_1_CHECK_STATE; + KqpReqCookie++; + + Kqp->TxId = ev->Get()->Record.GetResponse().GetTxMeta().id(); + + NYdb::TParamsBuilder params = BuildCheckGroupStateParams(); + + Kqp->SendYqlRequest(Sprintf(CHECK_GROUP_STATE.c_str(), TKafkaConsumerGroupsMetaInitManager::GetInstant()->GetStorageTablePath().c_str()), params.Build(), KqpReqCookie, ctx); + break; + } + + case SYNC_TX1_1_CHECK_STATE: { + auto groupStatus = ParseCheckStateAndGeneration(ev); + + if (!groupStatus) { + SendSyncGroupResponseFail(ctx, CorrelationId, + EKafkaErrors::UNKNOWN_SERVER_ERROR, + "Can't get group state"); + PassAway(); + return; + } + + if (!groupStatus->Exists) { + SendSyncGroupResponseFail(ctx, CorrelationId, + EKafkaErrors::GROUP_ID_NOT_FOUND, + "Unknown group# " + GroupId); + PassAway(); + return; + } + + if (groupStatus->Generation != GenerationId) { + SendSyncGroupResponseFail(ctx, CorrelationId, + EKafkaErrors::ILLEGAL_GENERATION, + TStringBuilder() << "Old or unknown group generation# " << GenerationId); + PassAway(); + return; + } + + if (groupStatus->State != GROUP_STATE_WORKING) { + if (WaitingWorkingStateRetries == WAIT_MASTER_MAX_RETRY_COUNT) { + SendSyncGroupResponseFail(ctx, CorrelationId, REBALANCE_IN_PROGRESS); + PassAway(); + return; + } + CurrentStep = SYNC_TX0_4_COMMIT_TX; + auto wakeup = std::make_unique(2); + ctx.ActorSystem()->Schedule( + TDuration::Seconds(WAIT_FOR_MASTER_DELAY_SECONDS), + new IEventHandle(SelfId(), SelfId(), wakeup.release()) + ); + WaitingWorkingStateRetries++; + return; + } + + CurrentStep = SYNC_TX1_2_FETCH_ASSIGNMENTS; + KqpReqCookie++; + + NYdb::TParamsBuilder params = BuildFetchAssignmentsParams(); + Kqp->SendYqlRequest(Sprintf(FETCH_ASSIGNMENTS.c_str(), TKafkaConsumerMembersMetaInitManager::GetInstant()->GetStorageTablePath().c_str()), params.Build(), KqpReqCookie, ctx); + break; + } + + case SYNC_TX1_2_FETCH_ASSIGNMENTS: { + if (!ParseAssignments(ev, Assignments)) { + SendSyncGroupResponseFail(ctx, CorrelationId, + EKafkaErrors::INVALID_REQUEST, + "Failed to get assignments from master"); + PassAway(); + return; + } + + CurrentStep = SYNC_TX1_3_COMMIT_TX; + KqpReqCookie++; + Kqp->CommitTx(KqpReqCookie, ctx); + break; + } + + case SYNC_TX1_3_COMMIT_TX: { + SendSyncGroupResponseOk(ctx, CorrelationId); + PassAway(); + break; + } + + default: { + KAFKA_LOG_CRIT("SYNC_GROUP: Unexpected step in HandleSyncGroupResponse"); + SendSyncGroupResponseFail(ctx, CorrelationId, EKafkaErrors::UNKNOWN_SERVER_ERROR, "Failed to get assignments from master"); + PassAway(); + break; + } + } // switch (CurrentStep) +} + +void TKafkaBalancerActor::HandleLeaveGroupResponse( + NKqp::TEvKqp::TEvQueryResponse::TPtr ev, + const TActorContext& ctx +) { + switch (CurrentStep) { + case STEP_NONE: { + CurrentStep = LEAVE_TX0_0_BEGIN_TX; + KqpReqCookie++; + Kqp->BeginTransaction(KqpReqCookie, ctx); + break; + } + + case LEAVE_TX0_0_BEGIN_TX: { + CurrentStep = LEAVE_TX0_1_UPDATE_TTL; + KqpReqCookie++; + Kqp->TxId = ev->Get()->Record.GetResponse().GetTxMeta().id(); + + NYdb::TParamsBuilder params = BuildLeaveGroupParams(); + Kqp->SendYqlRequest(Sprintf(UPDATE_LASTHEARTBEAT_TO_LEAVE_GROUP.c_str(), TKafkaConsumerMembersMetaInitManager::GetInstant()->GetStorageTablePath().c_str()), params.Build(), KqpReqCookie, ctx); + break; + } + + case LEAVE_TX0_1_UPDATE_TTL: { + if (ev->Get()->Record.GetYdbStatus() != Ydb::StatusIds::SUCCESS) { + SendLeaveGroupResponseFail(ctx, CorrelationId, EKafkaErrors::UNKNOWN_SERVER_ERROR, "Leave group update failed"); + PassAway(); + return; + } + + CurrentStep = LEAVE_TX0_2_COMMIT_TX; + KqpReqCookie++; + Kqp->CommitTx(KqpReqCookie, ctx); + break; + } + + case LEAVE_TX0_2_COMMIT_TX: { + if (ev->Get()->Record.GetYdbStatus() != Ydb::StatusIds::SUCCESS) { + SendLeaveGroupResponseFail(ctx, CorrelationId, EKafkaErrors::UNKNOWN_SERVER_ERROR, "Leave group commit failed"); + } else { + SendLeaveGroupResponseOk(ctx, CorrelationId); + } + PassAway(); + break; + } + + default: { + KAFKA_LOG_CRIT("LEAVE_GROUP: Unexpected step"); + SendLeaveGroupResponseFail(ctx, CorrelationId, EKafkaErrors::UNKNOWN_SERVER_ERROR, "Unexpected step"); + PassAway(); + break; + } + } +} + +void TKafkaBalancerActor::HandleHeartbeatResponse( + NKqp::TEvKqp::TEvQueryResponse::TPtr ev, + const TActorContext& ctx +) { + switch (CurrentStep) { + case STEP_NONE: { + CurrentStep = HEARTBEAT_TX0_0_BEGIN_TX; + KqpReqCookie++; + Kqp->BeginTransaction(KqpReqCookie, ctx); + break; + } + + case HEARTBEAT_TX0_0_BEGIN_TX: { + CurrentStep = HEARTBEAT_TX0_1_CHECK_DEAD_MEMBERS; + KqpReqCookie++; + Kqp->TxId = ev->Get()->Record.GetResponse().GetTxMeta().id(); + + NYdb::TParamsBuilder params = BuildCheckDeadsParams(); + Kqp->SendYqlRequest(Sprintf(CHECK_DEAD_MEMBERS.c_str(), TKafkaConsumerMembersMetaInitManager::GetInstant()->GetStorageTablePath().c_str()), params.Build(), KqpReqCookie, ctx); + break; + } + + case HEARTBEAT_TX0_1_CHECK_DEAD_MEMBERS: { + ui64 deadCount = 0; + if (!ParseDeadCount(ev, deadCount)) { + SendHeartbeatResponseFail(ctx, CorrelationId, EKafkaErrors::UNKNOWN_SERVER_ERROR, "Can't parse dead members count"); + PassAway(); + return; + } + + if (deadCount > 0) { + SendHeartbeatResponseFail(ctx, CorrelationId, EKafkaErrors::REBALANCE_IN_PROGRESS, "Rejoin required"); + PassAway(); + return; + } + + CurrentStep = HEARTBEAT_TX0_2_COMMIT_TX; + KqpReqCookie++; + Kqp->CommitTx(KqpReqCookie, ctx); + break; + } + + case HEARTBEAT_TX0_2_COMMIT_TX: { + CurrentStep = HEARTBEAT_TX1_0_BEGIN_TX; + KqpReqCookie++; + Kqp->BeginTransaction(KqpReqCookie, ctx); + break; + } + + case HEARTBEAT_TX1_0_BEGIN_TX: { + CurrentStep = HEARTBEAT_TX1_1_CHECK_GEN_AND_STATE; + KqpReqCookie++; + Kqp->TxId = ev->Get()->Record.GetResponse().GetTxMeta().id(); + + NYdb::TParamsBuilder params = BuildCheckGroupStateParams(); + + Kqp->SendYqlRequest(Sprintf(CHECK_GROUP_STATE.c_str(), TKafkaConsumerGroupsMetaInitManager::GetInstant()->GetStorageTablePath().c_str()), params.Build(), KqpReqCookie, ctx); + break; + } + + case HEARTBEAT_TX1_1_CHECK_GEN_AND_STATE: { + auto groupStatus = ParseCheckStateAndGeneration(ev); + + if (!groupStatus) { + SendHeartbeatResponseFail(ctx, CorrelationId, + EKafkaErrors::UNKNOWN_SERVER_ERROR, + "Can't get group state"); + PassAway(); + return; + } + + if (!groupStatus->Exists || groupStatus->Generation != GenerationId || groupStatus->State != GROUP_STATE_WORKING) { + SendHeartbeatResponseFail(ctx, CorrelationId, EKafkaErrors::REBALANCE_IN_PROGRESS, "Rejoin required"); + PassAway(); + return; + } + + IsMaster = (groupStatus->MasterId == MemberId); + CurrentStep = HEARTBEAT_TX1_2_UPDATE_TTL; + KqpReqCookie++; + + NYdb::TParamsBuilder params = BuildUpdateLastHeartbeatsParams(); + Kqp->SendYqlRequest(Sprintf(UPDATE_LASTHEARTBEATS.c_str(), TKafkaConsumerGroupsMetaInitManager::GetInstant()->GetStorageTablePath().c_str(), TKafkaConsumerMembersMetaInitManager::GetInstant()->GetStorageTablePath().c_str()), params.Build(), KqpReqCookie, ctx); + break; + } + + case HEARTBEAT_TX1_2_UPDATE_TTL: { + CurrentStep = HEARTBEAT_TX1_3_COMMIT_TX; + KqpReqCookie++; + Kqp->CommitTx(KqpReqCookie, ctx); + break; + } + + case HEARTBEAT_TX1_3_COMMIT_TX: { + SendHeartbeatResponseOk(ctx, CorrelationId, EKafkaErrors::NONE_ERROR); + PassAway(); + break; + } + + default: { + KAFKA_LOG_CRIT("HEARTBEAT: Unexpected step"); + SendHeartbeatResponseFail(ctx, CorrelationId, EKafkaErrors::UNKNOWN_SERVER_ERROR, "Unexpected step"); + PassAway(); + break; + } + } +} + +void TKafkaBalancerActor::SendJoinGroupResponseOk(const TActorContext& /*ctx*/, ui64 correlationId) { + auto response = std::make_shared(); + + response->ProtocolType = ProtocolType; + response->ProtocolName = Protocol; + response->ErrorCode = EKafkaErrors::NONE_ERROR; + response->GenerationId = GenerationId; + response->MemberId = MemberId; + + response->Leader = Master; + + if (IsMaster) { + response->Members.reserve(WorkerStates.size()); + for (const auto& [mId, meta] : WorkerStates) { + TJoinGroupResponseData::TJoinGroupResponseMember member; + member.MemberId = mId; + member.MetaStr = meta; + member.Metadata = member.MetaStr; + response->Members.push_back(std::move(member)); + } + } + + Send(Context->ConnectionId, new TEvKafka::TEvResponse(correlationId, response, EKafkaErrors::NONE_ERROR)); +} + +void TKafkaBalancerActor::SendSyncGroupResponseOk(const TActorContext&, ui64 correlationId) { + auto response = std::make_shared(); + response->ProtocolType = ProtocolType; + response->ProtocolName = Protocol; + response->ErrorCode = EKafkaErrors::NONE_ERROR; + response->AssignmentStr = TString(Assignments.data(), Assignments.size()); + response->Assignment = response->AssignmentStr; + Send(Context->ConnectionId, new TEvKafka::TEvResponse(correlationId, response, EKafkaErrors::NONE_ERROR)); +} + +void TKafkaBalancerActor::SendLeaveGroupResponseOk(const TActorContext&, ui64 corellationId) { + auto response = std::make_shared(); + response->ErrorCode = EKafkaErrors::NONE_ERROR; + Send(Context->ConnectionId, new TEvKafka::TEvResponse(corellationId, response, EKafkaErrors::NONE_ERROR)); +} + +void TKafkaBalancerActor::SendHeartbeatResponseOk(const TActorContext&, + ui64 corellationId, + EKafkaErrors error) { + auto response = std::make_shared(); + response->ErrorCode = error; + Send(Context->ConnectionId, new TEvKafka::TEvResponse(corellationId, response, error)); +} + + +void TKafkaBalancerActor::SendJoinGroupResponseFail(const TActorContext&, + ui64 corellationId, + EKafkaErrors error, + TString message) { + + KAFKA_LOG_ERROR("JOIN_GROUP failed. reason# " << message); + auto response = std::make_shared(); + response->ErrorCode = error; + Send(Context->ConnectionId, new TEvKafka::TEvResponse(corellationId, response, error)); +} + +void TKafkaBalancerActor::SendSyncGroupResponseFail(const TActorContext&, + ui64 corellationId, + EKafkaErrors error, + TString message) { + KAFKA_LOG_ERROR("SYNC_GROUP failed. reason# " << message); + auto response = std::make_shared(); + response->ErrorCode = error; + response->Assignment = ""; + Send(Context->ConnectionId, new TEvKafka::TEvResponse(corellationId, response, error)); +} + +void TKafkaBalancerActor::SendLeaveGroupResponseFail(const TActorContext&, + ui64 corellationId, + EKafkaErrors error, + TString message) { + KAFKA_LOG_ERROR("LEAVE_GROUP failed. reason# " << message); + auto response = std::make_shared(); + response->ErrorCode = error; + Send(Context->ConnectionId, new TEvKafka::TEvResponse(corellationId, response, error)); +} + +void TKafkaBalancerActor::SendHeartbeatResponseFail(const TActorContext&, + ui64 corellationId, + EKafkaErrors error, + TString message) { + KAFKA_LOG_ERROR("HEARTBEAT failed. reason# " << message); + auto response = std::make_shared(); + response->ErrorCode = error; + Send(Context->ConnectionId, new TEvKafka::TEvResponse(corellationId, response, error)); +} + +} // namespace NKafka diff --git a/ydb/core/kafka_proxy/actors/kafka_balancer_actor.h b/ydb/core/kafka_proxy/actors/kafka_balancer_actor.h new file mode 100644 index 000000000000..bfbc9da9459d --- /dev/null +++ b/ydb/core/kafka_proxy/actors/kafka_balancer_actor.h @@ -0,0 +1,304 @@ +#pragma once + +#include "actors.h" +#include "kafka_consumer_groups_metadata_initializers.h" +#include "kafka_consumer_members_metadata_initializers.h" +#include "kqp_balance_transaction.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace NKafka { +using namespace NKikimr; + +extern const TString INSERT_NEW_GROUP; +extern const TString UPDATE_GROUP; +extern const TString UPDATE_GROUP_STATE_AND_PROTOCOL; +extern const TString INSERT_MEMBER; +extern const TString UPDATE_GROUPS_AND_SELECT_WORKER_STATES; +extern const TString SELECT_WORKER_STATE_QUERY; +extern const TString SELECT_MASTER; +extern const TString UPSERT_ASSIGNMENTS_AND_SET_WORKING_STATE; +extern const TString CHECK_GROUP_STATE; +extern const TString FETCH_ASSIGNMENTS; +extern const TString CHECK_DEAD_MEMBERS; +extern const TString UPDATE_LASTHEARTBEATS; +extern const TString UPDATE_LASTHEARTBEAT_TO_LEAVE_GROUP; + +struct TGroupStatus { + bool Exists; + ui64 Generation; + ui64 State; + TString MasterId; + TInstant LastHeartbeat; + TString ProtocolName; +}; + +class TKafkaBalancerActor : public NActors::TActorBootstrapped { +public: + using TBase = NActors::TActorBootstrapped; + + enum EBalancerStep : ui8 { + STEP_NONE = 0, + + JOIN_TX0_0_BEGIN_TX, + JOIN_TX0_1_CHECK_STATE_AND_GENERATION, + JOIN_TX0_2_INSERT_NEW_GROUP, + JOIN_TX0_2_UPDATE_GROUP_STATE_AND_GENERATION, + JOIN_TX0_2_SKIP, + JOIN_TX0_3_INSERT_MEMBER, + JOIN_TX0_4_COMMIT_TX, + JOIN_TX0_5_WAIT, + + JOIN_TX1_0_BEGIN_TX, + JOIN_TX1_1_CHECK_STATE_AND_GENERATION, + JOIN_TX1_2_GET_MEMBERS_AND_SET_STATE_SYNC, + JOIN_TX1_3_COMMIT_TX, + + SYNC_TX0_0_BEGIN_TX, + SYNC_TX0_1_SELECT_MASTER, + SYNC_TX0_2_CHECK_STATE_AND_GENERATION, + SYNC_TX0_3_SET_ASSIGNMENTS_AND_SET_WORKING_STATE, + SYNC_TX0_4_COMMIT_TX, + + SYNC_TX1_0_BEGIN_TX, + SYNC_TX1_1_CHECK_STATE, + SYNC_TX1_2_FETCH_ASSIGNMENTS, + SYNC_TX1_3_COMMIT_TX, + + HEARTBEAT_TX0_0_BEGIN_TX, + HEARTBEAT_TX0_1_CHECK_DEAD_MEMBERS, + HEARTBEAT_TX0_2_COMMIT_TX, + + HEARTBEAT_TX1_0_BEGIN_TX, + HEARTBEAT_TX1_1_CHECK_GEN_AND_STATE, + HEARTBEAT_TX1_2_UPDATE_TTL, + HEARTBEAT_TX1_3_COMMIT_TX, + + LEAVE_TX0_0_BEGIN_TX, + LEAVE_TX0_1_UPDATE_TTL, + LEAVE_TX0_2_COMMIT_TX, + }; + + TKafkaBalancerActor(const TContext::TPtr context, ui64 cookie, ui64 corellationId, TMessagePtr message, ui8 retryNum = 0) + : Context(context) + , CorrelationId(corellationId) + , Cookie(cookie) + , CurrentRetryNumber(retryNum) + , JoinGroupRequestData(message) + , SyncGroupRequestData( + std::shared_ptr(), + std::shared_ptr()) + , HeartbeatGroupRequestData( + std::shared_ptr(), + std::shared_ptr()) + , LeaveGroupRequestData( + std::shared_ptr(), + std::shared_ptr()) + { + KAFKA_LOG_D("HandleJoinGroup request"); + + RequestType = JOIN_GROUP; + CurrentStep = STEP_NONE; + + GroupId = JoinGroupRequestData->GroupId.value(); + ProtocolType = JoinGroupRequestData->ProtocolType.value(); + } + + TKafkaBalancerActor(const TContext::TPtr context, ui64 cookie, ui64 corellationId, TMessagePtr message, ui8 retryNum = 0) + : Context(context) + , CorrelationId(corellationId) + , Cookie(cookie) + , CurrentRetryNumber(retryNum) + , JoinGroupRequestData( + std::shared_ptr(), + std::shared_ptr()) + , SyncGroupRequestData(message) + , HeartbeatGroupRequestData( + std::shared_ptr(), + std::shared_ptr()) + , LeaveGroupRequestData( + std::shared_ptr(), + std::shared_ptr()) + { + KAFKA_LOG_D("HandleSyncGroup request"); + + RequestType = SYNC_GROUP; + CurrentStep = STEP_NONE; + + GroupId = SyncGroupRequestData->GroupId.value(); + MemberId = SyncGroupRequestData->MemberId.value(); + GenerationId = SyncGroupRequestData->GenerationId; + ProtocolType = SyncGroupRequestData->ProtocolType.value(); + } + + TKafkaBalancerActor(const TContext::TPtr context, ui64 cookie, ui64 corellationId, TMessagePtr message, ui8 retryNum = 0) + : Context(context) + , CorrelationId(corellationId) + , Cookie(cookie) + , CurrentRetryNumber(retryNum) + , JoinGroupRequestData( + std::shared_ptr(), + std::shared_ptr()) + , SyncGroupRequestData( + std::shared_ptr(), + std::shared_ptr()) + , HeartbeatGroupRequestData(message) + , LeaveGroupRequestData( + std::shared_ptr(), + std::shared_ptr()) + { + KAFKA_LOG_D("HandleHeartbeat request"); + + RequestType = HEARTBEAT; + CurrentStep = STEP_NONE; + + GroupId = HeartbeatGroupRequestData->GroupId.value(); + MemberId = HeartbeatGroupRequestData->MemberId.value(); + GenerationId = HeartbeatGroupRequestData->GenerationId; + } + + TKafkaBalancerActor(const TContext::TPtr context, ui64 cookie, ui64 corellationId, TMessagePtr message, ui8 retryNum = 0) + : Context(context) + , CorrelationId(corellationId) + , Cookie(cookie) + , CurrentRetryNumber(retryNum) + , JoinGroupRequestData( + std::shared_ptr(), + std::shared_ptr()) + , SyncGroupRequestData( + std::shared_ptr(), + std::shared_ptr()) + , HeartbeatGroupRequestData( + std::shared_ptr(), + std::shared_ptr()) + , LeaveGroupRequestData(message) + { + KAFKA_LOG_D("HandleLeaveGroup request"); + RequestType = LEAVE_GROUP; + CurrentStep = STEP_NONE; + + GroupId = LeaveGroupRequestData->GroupId.value(); + MemberId = LeaveGroupRequestData->MemberId.value(); + } + + void Bootstrap(const NActors::TActorContext& ctx); + + static constexpr NKikimrServices::TActivity::EType ActorActivityType() { + return NKikimrServices::TActivity::KAFKA_READ_SESSION_ACTOR; + } + +private: + using TActorContext = NActors::TActorContext; + + STATEFN(StateWork) { + switch (ev->GetTypeRewrite()) { + HFunc(NMetadata::NProvider::TEvManagerPrepared, Handle); + HFunc(NKqp::TEvKqp::TEvCreateSessionResponse, Handle); + HFunc(NKqp::TEvKqp::TEvQueryResponse, Handle); + + HFunc(TEvents::TEvWakeup, Handle); + SFunc(TEvents::TEvPoison, Die); + } + } + + void HandleResponse(NKqp::TEvKqp::TEvQueryResponse::TPtr ev, const TActorContext& ctx); + + 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& ev, const TActorContext& ctx); + + void HandleJoinGroupResponse(NKqp::TEvKqp::TEvQueryResponse::TPtr ev, const TActorContext& ctx); + void HandleSyncGroupResponse(NKqp::TEvKqp::TEvQueryResponse::TPtr ev, const TActorContext& ctx); + void HandleLeaveGroupResponse(NKqp::TEvKqp::TEvQueryResponse::TPtr ev, const TActorContext& ctx); + void HandleHeartbeatResponse(NKqp::TEvKqp::TEvQueryResponse::TPtr ev, const TActorContext& ctx); + + void Die(const TActorContext& ctx) override; + + void SendJoinGroupResponseOk(const TActorContext&, ui64 corellationId); + void SendJoinGroupResponseFail(const TActorContext&, ui64 corellationId, + EKafkaErrors error, TString message = ""); + void SendSyncGroupResponseOk(const TActorContext& ctx, ui64 corellationId); + void SendSyncGroupResponseFail(const TActorContext&, ui64 corellationId, + EKafkaErrors error, TString message = ""); + void SendHeartbeatResponseOk(const TActorContext&, ui64 corellationId, EKafkaErrors error); + void SendHeartbeatResponseFail(const TActorContext&, ui64 corellationId, + EKafkaErrors error, TString message = ""); + void SendLeaveGroupResponseOk(const TActorContext& ctx, ui64 corellationId); + void SendLeaveGroupResponseFail(const TActorContext&, ui64 corellationId, + EKafkaErrors error, TString message = ""); + + TString LogPrefix(); + void SendResponseFail(const TActorContext& ctx, EKafkaErrors error, const TString& message); + + std::optional ParseCheckStateAndGeneration(NKqp::TEvKqp::TEvQueryResponse::TPtr ev); + bool ParseAssignments(NKqp::TEvKqp::TEvQueryResponse::TPtr ev, TString& assignments); + bool ParseWorkerStatesAndChooseProtocol(NKqp::TEvKqp::TEvQueryResponse::TPtr ev, std::unordered_map& workerStates, TString& chosenProtocol); + bool ParseDeadCount(NKqp::TEvKqp::TEvQueryResponse::TPtr ev, ui64& outCount); + + NYdb::TParamsBuilder BuildCheckGroupStateParams(); + NYdb::TParamsBuilder BuildUpdateOrInsertNewGroupParams(); + NYdb::TParamsBuilder BuildInsertMemberParams(); + NYdb::TParamsBuilder BuildAssignmentsParams(); + NYdb::TParamsBuilder BuildUpdatesGroupsAndSelectWorkerStatesParams(); + NYdb::TParamsBuilder BuildUpdateGroupStateAndProtocolParams(); + NYdb::TParamsBuilder BuildFetchAssignmentsParams(); + NYdb::TParamsBuilder BuildLeaveGroupParams(); + NYdb::TParamsBuilder BuildUpdateLastHeartbeatsParams(); + NYdb::TParamsBuilder BuildCheckDeadsParams(); + +private: + enum EGroupState : ui32 { + GROUP_STATE_JOIN = 0, + GROUP_STATE_JOINED, + GROUP_STATE_SYNC, + GROUP_STATE_WORKING + }; + +private: + const TContext::TPtr Context; + NKafka::EApiKey RequestType; + + ui8 TablesInited = 0; + TString GroupId; + TString GroupName; + TString MemberId; + + ui64 GenerationId = 0; + ui64 CorrelationId = 0; + ui64 Cookie = 0; + ui64 KqpReqCookie = 0; + ui8 WaitingWorkingStateRetries = 0; + + TString Assignments; + std::unordered_map WorkerStates; + TString Protocol; + TString ProtocolType; + TString Master; + ui8 CurrentRetryNumber; + + bool IsMaster = false; + + EBalancerStep CurrentStep = STEP_NONE; + + std::unique_ptr Kqp; + + TMessagePtr JoinGroupRequestData; + TMessagePtr SyncGroupRequestData; + TMessagePtr HeartbeatGroupRequestData; + TMessagePtr LeaveGroupRequestData; + +}; + +} // namespace NKafka diff --git a/ydb/core/kafka_proxy/actors/kafka_consumer_groups_metadata_initializers.cpp b/ydb/core/kafka_proxy/actors/kafka_consumer_groups_metadata_initializers.cpp new file mode 100644 index 000000000000..dcdf27ffa4ca --- /dev/null +++ b/ydb/core/kafka_proxy/actors/kafka_consumer_groups_metadata_initializers.cpp @@ -0,0 +1,68 @@ +#include "kafka_consumer_groups_metadata_initializers.h" + +namespace NKikimr::NGRpcProxy::V1 { + +using namespace NMetadata; + +void TKafkaConsumerGroupsMetaInitializer::DoPrepare(NInitializer::IInitializerInput::TPtr controller) const { + TVector result; + auto tablePath = TKafkaConsumerGroupsMetaInitManager::GetInstant()->GetStorageTablePath(); + { + Ydb::Table::CreateTableRequest request; + request.set_session_id(""); + request.set_path(tablePath); + request.add_primary_key("database"); + request.add_primary_key("consumer_group"); + { + auto& column = *request.add_columns(); + column.set_name("database"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::UTF8); + } + { + auto& column = *request.add_columns(); + column.set_name("consumer_group"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::UTF8); + } + { + auto& column = *request.add_columns(); + column.set_name("generation"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::UINT64); + } + { + auto& column = *request.add_columns(); + column.set_name("state"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::UINT64); + } + { + auto& column = *request.add_columns(); + column.set_name("last_heartbeat_time"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::DATETIME); + } + { + auto& column = *request.add_columns(); + column.set_name("master"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::UTF8); + } + { + auto& column = *request.add_columns(); + column.set_name("protocol_type"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::UTF8); + } + { + auto& column = *request.add_columns(); + column.set_name("protocol"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::UTF8); + } + { + auto* ttlSettings = request.mutable_ttl_settings(); + auto* columnTtl = ttlSettings->mutable_date_type_column(); + columnTtl->set_column_name("last_heartbeat_time"); + columnTtl->set_expire_after_seconds(300); + } + result.emplace_back(new NInitializer::TGenericTableModifier(request, "create")); + } + result.emplace_back(NInitializer::TACLModifierConstructor::GetReadOnlyModifier(tablePath, "acl")); + controller->OnPreparationFinished(result); +} + +} diff --git a/ydb/core/kafka_proxy/actors/kafka_consumer_groups_metadata_initializers.h b/ydb/core/kafka_proxy/actors/kafka_consumer_groups_metadata_initializers.h new file mode 100644 index 000000000000..dc6bd6b10f0e --- /dev/null +++ b/ydb/core/kafka_proxy/actors/kafka_consumer_groups_metadata_initializers.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include + +namespace NKikimr::NGRpcProxy::V1 { + +using TInitBehaviourPtr = std::shared_ptr; +using TClassBehaviourPtr = std::shared_ptr; + +class TKafkaConsumerGroupsMetaInitializer : public NMetadata::NInitializer::IInitializationBehaviour { +public: + static TInitBehaviourPtr GetInstant() { + static TInitBehaviourPtr res{new TKafkaConsumerGroupsMetaInitializer()}; + return res; + } + +protected: + virtual void DoPrepare(NMetadata::NInitializer::IInitializerInput::TPtr controller) const override; + +private: + TKafkaConsumerGroupsMetaInitializer() = default; +}; + +class TKafkaConsumerGroupsMetaInitManager : public NMetadata::IClassBehaviour { +protected: + virtual TString GetInternalStorageTablePath() const override { + return "kafka_consumer_groups"; + } + + TInitBehaviourPtr ConstructInitializer() const override { + return TKafkaConsumerGroupsMetaInitializer::GetInstant(); + } + +public: + std::shared_ptr GetOperationsManager() const override { + return nullptr; + } + + static TClassBehaviourPtr GetInstant() { + static TClassBehaviourPtr res{new TKafkaConsumerGroupsMetaInitManager()}; + return res; + } + + virtual TString GetTypeId() const override { + return TypeName(); + } + +private: + TKafkaConsumerGroupsMetaInitManager() = default; +}; + +} diff --git a/ydb/core/kafka_proxy/actors/kafka_consumer_members_metadata_initializers.cpp b/ydb/core/kafka_proxy/actors/kafka_consumer_members_metadata_initializers.cpp new file mode 100644 index 000000000000..ac9328c7ecc4 --- /dev/null +++ b/ydb/core/kafka_proxy/actors/kafka_consumer_members_metadata_initializers.cpp @@ -0,0 +1,75 @@ +#include "kafka_consumer_members_metadata_initializers.h" + +namespace NKikimr::NGRpcProxy::V1 { + +using namespace NMetadata; + +void TKafkaConsumerMembersMetaInitializer::DoPrepare(NInitializer::IInitializerInput::TPtr controller) const { + TVector result; + auto tablePath = TKafkaConsumerMembersMetaInitManager::GetInstant()->GetStorageTablePath(); + { + Ydb::Table::CreateTableRequest request; + request.set_session_id(""); + request.set_path(tablePath); + request.add_primary_key("database"); + request.add_primary_key("consumer_group"); + request.add_primary_key("generation"); + request.add_primary_key("member_id"); + { + auto& column = *request.add_columns(); + column.set_name("database"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::UTF8); + } + { + auto& column = *request.add_columns(); + column.set_name("consumer_group"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::UTF8); + } + { + auto& column = *request.add_columns(); + column.set_name("generation"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::UINT64); + } + { + auto& column = *request.add_columns(); + column.set_name("member_id"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::UTF8); + } + { + auto& column = *request.add_columns(); + column.set_name("assignment"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::STRING); + } + { + auto& column = *request.add_columns(); + column.set_name("worker_state_proto"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::STRING); + } + { + auto& column = *request.add_columns(); + column.set_name("last_heartbeat_time"); + column.mutable_type()->mutable_optional_type()->mutable_item()->set_type_id(Ydb::Type::DATETIME); + } + { + auto* ttlSettings = request.mutable_ttl_settings(); + auto* columnTtl = ttlSettings->mutable_date_type_column(); + columnTtl->set_column_name("last_heartbeat_time"); + columnTtl->set_expire_after_seconds(60); + } + { + auto& index = *request.add_indexes(); + index.set_name("idx_group_generation_db_lht"); + *index.mutable_global_index() = Ydb::Table::GlobalIndex(); + index.add_index_columns("consumer_group"); + index.add_index_columns("generation"); + index.add_index_columns("database"); + index.add_index_columns("last_heartbeat_time"); + } + result.emplace_back(new NInitializer::TGenericTableModifier(request, "create")); + } + + result.emplace_back(NInitializer::TACLModifierConstructor::GetReadOnlyModifier(tablePath, "acl")); + controller->OnPreparationFinished(result); +} + +} diff --git a/ydb/core/kafka_proxy/actors/kafka_consumer_members_metadata_initializers.h b/ydb/core/kafka_proxy/actors/kafka_consumer_members_metadata_initializers.h new file mode 100644 index 000000000000..e9dbe887e410 --- /dev/null +++ b/ydb/core/kafka_proxy/actors/kafka_consumer_members_metadata_initializers.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include + +namespace NKikimr::NGRpcProxy::V1 { + +using TInitBehaviourPtr = std::shared_ptr; +using TClassBehaviourPtr = std::shared_ptr; + +class TKafkaConsumerMembersMetaInitializer : public NMetadata::NInitializer::IInitializationBehaviour { +public: + static TInitBehaviourPtr GetInstant() { + static TInitBehaviourPtr res{new TKafkaConsumerMembersMetaInitializer()}; + return res; + } + +protected: + virtual void DoPrepare(NMetadata::NInitializer::IInitializerInput::TPtr controller) const override; + +private: + TKafkaConsumerMembersMetaInitializer() = default; +}; + +class TKafkaConsumerMembersMetaInitManager : public NMetadata::IClassBehaviour { +protected: + virtual TString GetInternalStorageTablePath() const override { + return "kafka_consumer_members"; + } + + TInitBehaviourPtr ConstructInitializer() const override { + return TKafkaConsumerMembersMetaInitializer::GetInstant(); + } + +public: + std::shared_ptr GetOperationsManager() const override { + return nullptr; + } + + static TClassBehaviourPtr GetInstant() { + static TClassBehaviourPtr res{new TKafkaConsumerMembersMetaInitManager()}; + return res; + } + + virtual TString GetTypeId() const override { + return TypeName(); + } + +private: + TKafkaConsumerMembersMetaInitManager() = default; +}; + +} diff --git a/ydb/core/kafka_proxy/actors/kafka_read_session_actor.cpp b/ydb/core/kafka_proxy/actors/kafka_read_session_actor.cpp index ddaede899141..d357990be9e6 100644 --- a/ydb/core/kafka_proxy/actors/kafka_read_session_actor.cpp +++ b/ydb/core/kafka_proxy/actors/kafka_read_session_actor.cpp @@ -3,8 +3,7 @@ namespace NKafka { static constexpr TDuration WAKEUP_INTERVAL = TDuration::Seconds(1); static constexpr TDuration LOCK_PARTITION_DELAY = TDuration::Seconds(3); -static const TString SUPPORTED_ASSIGN_STRATEGY = "roundrobin"; -static const TString SUPPORTED_JOIN_GROUP_PROTOCOL = "consumer"; +static constexpr TKafkaUint16 ASSIGNMENT_VERSION = 3; NActors::IActor* CreateKafkaReadSessionActor(const TContext::TPtr context, ui64 cookie) { return new TKafkaReadSessionActor(context, cookie); @@ -58,8 +57,8 @@ void TKafkaReadSessionActor::CloseReadSession(const TActorContext& /*ctx*/) { void TKafkaReadSessionActor::HandleJoinGroup(TEvKafka::TEvJoinGroupRequest::TPtr ev, const TActorContext& ctx) { auto joinGroupRequest = ev->Get()->Request; - if (JoinGroupCorellationId != 0) { - JoinGroupCorellationId = 0; + if (CorellationId != 0) { + CorellationId = 0; SendJoinGroupResponseFail(ctx, ev->Get()->CorrelationId, NextRequestError.Code, "JOIN_GROUP request already inflight"); CloseReadSession(ctx); return; @@ -112,7 +111,7 @@ void TKafkaReadSessionActor::HandleJoinGroup(TEvKafka::TEvJoinGroupRequest::TPtr return; } - JoinGroupCorellationId = ev->Get()->CorrelationId; + CorellationId = ev->Get()->CorrelationId; AuthAndFindBalancers(ctx); break; } @@ -255,7 +254,15 @@ void TKafkaReadSessionActor::SendSyncGroupResponseOk(const TActorContext& ctx, u response->ProtocolType = SUPPORTED_JOIN_GROUP_PROTOCOL; response->ProtocolName = SUPPORTED_ASSIGN_STRATEGY; response->ErrorCode = EKafkaErrors::NONE_ERROR; - response->Assignment = BuildAssignmentAndInformBalancerIfRelease(ctx); + + auto assignment = BuildAssignmentAndInformBalancerIfRelease(ctx); + + TWritableBuf buf(nullptr, assignment.Size(ASSIGNMENT_VERSION) + sizeof(ASSIGNMENT_VERSION)); + TKafkaWritable writable(buf); + writable << ASSIGNMENT_VERSION; + assignment.Write(writable, ASSIGNMENT_VERSION); + response->AssignmentStr = TString(buf.GetBuffer().data(), buf.GetBuffer().size()); + response->Assignment = response->AssignmentStr; Send(Context->ConnectionId, new TEvKafka::TEvResponse(corellationId, response, EKafkaErrors::NONE_ERROR)); } @@ -265,6 +272,7 @@ void TKafkaReadSessionActor::SendSyncGroupResponseFail(const TActorContext&, ui6 TSyncGroupResponseData::TPtr response = std::make_shared(); response->ErrorCode = error; + response->Assignment = ""; Send(Context->ConnectionId, new TEvKafka::TEvResponse(corellationId, response, error)); } @@ -442,7 +450,7 @@ void TKafkaReadSessionActor::AuthAndFindBalancers(const TActorContext& ctx) { TopicsToConverter = topicHandler->GetReadTopicsList(TopicsToReadNames, false, Context->DatabasePath); if (!TopicsToConverter.IsValid) { - SendJoinGroupResponseFail(ctx, JoinGroupCorellationId, INVALID_REQUEST, TStringBuilder() << "topicsToConverter is not valid"); + SendJoinGroupResponseFail(ctx, CorellationId, INVALID_REQUEST, TStringBuilder() << "topicsToConverter is not valid"); return; } @@ -452,10 +460,10 @@ void TKafkaReadSessionActor::AuthAndFindBalancers(const TActorContext& ctx) { } void TKafkaReadSessionActor::HandleBalancerError(TEvPersQueue::TEvError::TPtr& ev, const TActorContext& ctx) { - if (JoinGroupCorellationId != 0) { - SendJoinGroupResponseFail(ctx, JoinGroupCorellationId, ConvertErrorCode(ev->Get()->Record.GetCode()), ev->Get()->Record.GetDescription()); + if (CorellationId != 0) { + SendJoinGroupResponseFail(ctx, CorellationId, ConvertErrorCode(ev->Get()->Record.GetCode()), ev->Get()->Record.GetDescription()); CloseReadSession(ctx); - JoinGroupCorellationId = 0; + CorellationId = 0; } else { NextRequestError.Code = ConvertErrorCode(ev->Get()->Record.GetCode()); NextRequestError.Message = ev->Get()->Record.GetDescription(); @@ -479,17 +487,17 @@ void TKafkaReadSessionActor::HandleAuthOk(NGRpcProxy::V1::TEvPQProxy::TEvAuthRes RegisterBalancerSession(topicInfo.FullConverter->GetInternalName(), topicInfo.PipeClient, topicInfo.Groups, ctx); } - if (JoinGroupCorellationId != 0) { - SendJoinGroupResponseOk(ctx, JoinGroupCorellationId); - JoinGroupCorellationId = 0; + if (CorellationId != 0) { + SendJoinGroupResponseOk(ctx, CorellationId); + CorellationId = 0; ReadStep = WAIT_SYNC_GROUP; } } void TKafkaReadSessionActor::HandleAuthCloseSession(NGRpcProxy::V1::TEvPQProxy::TEvCloseSession::TPtr& ev, const TActorContext& ctx) { - if (JoinGroupCorellationId != 0) { - SendJoinGroupResponseFail(ctx, JoinGroupCorellationId, ConvertErrorCode(ev->Get()->ErrorCode), TStringBuilder() << "auth failed. " << ev->Get()->Reason); - JoinGroupCorellationId = 0; + if (CorellationId != 0) { + SendJoinGroupResponseFail(ctx, CorellationId, ConvertErrorCode(ev->Get()->ErrorCode), TStringBuilder() << "auth failed. " << ev->Get()->Reason); + CorellationId = 0; } CloseReadSession(ctx); diff --git a/ydb/core/kafka_proxy/actors/kafka_read_session_actor.h b/ydb/core/kafka_proxy/actors/kafka_read_session_actor.h index e8f29318e78a..ab614f579ff2 100644 --- a/ydb/core/kafka_proxy/actors/kafka_read_session_actor.h +++ b/ydb/core/kafka_proxy/actors/kafka_read_session_actor.h @@ -32,12 +32,12 @@ namespace NKafka { * HEARTBEAT request() * ----------------> * HEARTBEAT response(status = OK) - * <---------------- + * <---------------- * * HEARTBEAT request() * ----------------> * HEARTBEAT response(status = REBALANCE_IN_PROGRESS) //if partitions to read list changes - * <---------------- + * <---------------- * * JOIN_GROUP request(topics) //client send again, because REBALANCE_IN_PROGRESS in heartbeat response * ----------------> @@ -49,9 +49,12 @@ namespace NKafka { * LEAVE_GROUP request() * ----------------> * LEAVE_GROUP response() - * <---------------- + * <---------------- */ +static const TString SUPPORTED_ASSIGN_STRATEGY = "roundrobin"; +static const TString SUPPORTED_JOIN_GROUP_PROTOCOL = "consumer"; + class TKafkaReadSessionActor: public NActors::TActorBootstrapped { enum EReadSessionSteps { @@ -111,11 +114,11 @@ struct TNextRequestError { HFunc(TEvPersQueue::TEvLockPartition, HandleLockPartition); HFunc(TEvPersQueue::TEvReleasePartition, HandleReleasePartition); HFunc(TEvPersQueue::TEvError, HandleBalancerError); - + // from Pipe HFunc(TEvTabletPipe::TEvClientConnected, HandlePipeConnected); HFunc(TEvTabletPipe::TEvClientDestroyed, HandlePipeDestroyed); - + // others HFunc(TEvKafka::TEvWakeup, HandleWakeup); SFunc(TEvents::TEvPoison, Die); @@ -163,7 +166,7 @@ struct TNextRequestError { TString Session; TString AssignProtocolName; i64 GenerationId = 0; - ui64 JoinGroupCorellationId = 0; + ui64 CorellationId = 0; ui64 Cookie; bool NeedRebalance = false; TInstant LastHeartbeatTime = TInstant::Now(); diff --git a/ydb/core/kafka_proxy/actors/kqp_balance_transaction.cpp b/ydb/core/kafka_proxy/actors/kqp_balance_transaction.cpp new file mode 100644 index 000000000000..0309ab199eb9 --- /dev/null +++ b/ydb/core/kafka_proxy/actors/kqp_balance_transaction.cpp @@ -0,0 +1,107 @@ +#include "kqp_balance_transaction.h" +#include "ydb/core/kqp/common/simple/services.h" +#include "kafka_consumer_groups_metadata_initializers.h" +#include "kafka_consumer_members_metadata_initializers.h" + + +namespace NKikimr::NGRpcProxy::V1 { + +TKqpTxHelper::TKqpTxHelper(TString database) + : DataBase(database) +{} + +void TKqpTxHelper::SendCreateSessionRequest(const TActorContext& ctx) { + auto ev = MakeCreateSessionRequest(); + ctx.Send(NKqp::MakeKqpProxyID(ctx.SelfID.NodeId()), ev.Release(), 0, 0); +} + +void TKqpTxHelper::BeginTransaction(ui64 cookie, const NActors::TActorContext& ctx) { + auto begin = MakeHolder(); + + begin->Record.MutableRequest()->SetAction(NKikimrKqp::QUERY_ACTION_BEGIN_TX); + begin->Record.MutableRequest()->MutableTxControl()->mutable_begin_tx()->mutable_serializable_read_write(); + begin->Record.MutableRequest()->SetSessionId(KqpSessionId); + begin->Record.MutableRequest()->SetDatabase(DataBase); + + ctx.Send(NKqp::MakeKqpProxyID(ctx.SelfID.NodeId()), begin.Release(), 0, cookie); +} + +bool TKqpTxHelper::HandleCreateSessionResponse(NKqp::TEvKqp::TEvCreateSessionResponse::TPtr& ev, const TActorContext&) { + const auto& record = ev->Get()->Record; + + if (record.GetYdbStatus() != Ydb::StatusIds::SUCCESS) { + return false; + } + + KqpSessionId = record.GetResponse().GetSessionId(); + Y_ABORT_UNLESS(!KqpSessionId.empty()); + + return true; +} + +void TKqpTxHelper::CloseKqpSession(const TActorContext& ctx) { + if (KqpSessionId) { + auto ev = MakeCloseSessionRequest(); + ctx.Send(NKqp::MakeKqpProxyID(ctx.SelfID.NodeId()), ev.Release(), 0, 0); + KqpSessionId = ""; + } +} + +THolder TKqpTxHelper::MakeCreateSessionRequest() { + auto ev = MakeHolder(); + ev->Record.MutableRequest()->SetDatabase(DataBase); + return ev; +} + +THolder TKqpTxHelper::MakeCloseSessionRequest() { + auto ev = MakeHolder(); + ev->Record.MutableRequest()->SetSessionId(KqpSessionId); + return ev; +} + +void TKqpTxHelper::SendRequest(THolder request, ui64 cookie, const NActors::TActorContext& ctx) { + ctx.Send(NKqp::MakeKqpProxyID(ctx.SelfID.NodeId()), request.Release(), 0, cookie); +} + +void TKqpTxHelper::SendYqlRequest(TString yqlRequest, NYdb::TParams sqlParams, ui64 cookie, const NActors::TActorContext& ctx, bool commit) { + auto ev = MakeHolder(); + + ev->Record.MutableRequest()->SetAction(NKikimrKqp::QUERY_ACTION_EXECUTE); + ev->Record.MutableRequest()->SetType(NKikimrKqp::QUERY_TYPE_SQL_DML); + ev->Record.MutableRequest()->SetQuery(yqlRequest); + ev->Record.MutableRequest()->SetDatabase(DataBase); + ev->Record.MutableRequest()->SetSessionId(KqpSessionId); + ev->Record.MutableRequest()->MutableTxControl()->set_commit_tx(commit); + ev->Record.MutableRequest()->MutableTxControl()->set_tx_id(TxId); + ev->Record.MutableRequest()->SetUsePublicResponseDataFormat(true); + ev->Record.MutableRequest()->MutableQueryCachePolicy()->set_keep_in_cache(true); + + ev->Record.MutableRequest()->MutableYdbParameters()->swap(*(NYdb::TProtoAccessor::GetProtoMapPtr(sqlParams))); + ctx.Send(NKqp::MakeKqpProxyID(ctx.SelfID.NodeId()), ev.Release(), 0, cookie); +} + +void TKqpTxHelper::CommitTx(ui64 cookie, const NActors::TActorContext& ctx) { + auto commit = MakeHolder(); + + commit->Record.MutableRequest()->SetAction(NKikimrKqp::QUERY_ACTION_COMMIT_TX); + commit->Record.MutableRequest()->MutableTxControl()->set_tx_id(TxId); + commit->Record.MutableRequest()->MutableTxControl()->set_commit_tx(true); + commit->Record.MutableRequest()->SetSessionId(KqpSessionId); + commit->Record.MutableRequest()->SetDatabase(DataBase); + + ctx.Send(NKqp::MakeKqpProxyID(ctx.SelfID.NodeId()), commit.Release(), 0, cookie); +} + +void TKqpTxHelper::SendInitTablesRequest(const TActorContext& ctx) { + ctx.Send( + NMetadata::NProvider::MakeServiceId(ctx.SelfID.NodeId()), + new NMetadata::NProvider::TEvPrepareManager(NGRpcProxy::V1::TKafkaConsumerMembersMetaInitManager::GetInstant()) + ); + + ctx.Send( + NMetadata::NProvider::MakeServiceId(ctx.SelfID.NodeId()), + new NMetadata::NProvider::TEvPrepareManager(NGRpcProxy::V1::TKafkaConsumerGroupsMetaInitManager::GetInstant()) + ); +} + +} // namespace NKikimr::NGRpcProxy::V1 diff --git a/ydb/core/kafka_proxy/actors/kqp_balance_transaction.h b/ydb/core/kafka_proxy/actors/kqp_balance_transaction.h new file mode 100644 index 000000000000..72ec296ffa63 --- /dev/null +++ b/ydb/core/kafka_proxy/actors/kqp_balance_transaction.h @@ -0,0 +1,40 @@ +#pragma once + +#include +#include +#include +#include + +namespace NKikimr::NGRpcProxy::V1 { + +using namespace NKikimr::NGRpcService; // savnik change namespace + +class TKqpTxHelper { +public: + TKqpTxHelper(TString database); + void SendCreateSessionRequest(const TActorContext& ctx); + void BeginTransaction(ui64 cookie, const NActors::TActorContext& ctx); + bool HandleCreateSessionResponse(NKqp::TEvKqp::TEvCreateSessionResponse::TPtr& ev, const TActorContext& ctx); + void CloseKqpSession(const TActorContext& ctx); + void SendRequest(THolder request, ui64 cookie, const NActors::TActorContext& ctx); + void CommitTx(ui64 cookie, const NActors::TActorContext& ctx); + void SendYqlRequest(TString yqlRequest, NYdb::TParams sqlParams, ui64 cookie, const NActors::TActorContext& ctx, bool commit = false); + void SendInitTablesRequest(const TActorContext& ctx); + +public: + TString DataBase; + TString TxId; + +private: + THolder MakeCreateSessionRequest(); + THolder MakeCloseSessionRequest(); + + +private: + TString Consumer; + TString Path; + + TString KqpSessionId; +}; + +} // namespace NKikimr::NGRpcProxy::V1 diff --git a/ydb/core/kafka_proxy/kafka_connection.cpp b/ydb/core/kafka_proxy/kafka_connection.cpp index 223cc9013480..13de3d1b8dbc 100644 --- a/ydb/core/kafka_proxy/kafka_connection.cpp +++ b/ydb/core/kafka_proxy/kafka_connection.cpp @@ -2,6 +2,8 @@ #include #include #include +#include + #include "actors/actors.h" #include "kafka_connection.h" @@ -237,36 +239,56 @@ class TKafkaConnection: public TActorBootstrapped, public TNet Send(ProduceActorId, new TEvKafka::TEvProduceRequest(header->CorrelationId, message)); } - void HandleMessage(const TRequestHeaderData* header, const TMessagePtr& message, const TActorContext& ctx) { - if (!ReadSessionActorId) { - ReadSessionActorId = ctx.RegisterWithSameMailbox(CreateKafkaReadSessionActor(Context, 0)); + void HandleMessage(const TRequestHeaderData* header, const TMessagePtr& message, const TActorContext& /*ctx*/) { + if (Context->Config.GetEnableNativeBalancing()) { + Register(new TKafkaBalancerActor(Context, 0, header->CorrelationId, message)); + } else { + if (ReadSessionActorId) { + Send(ReadSessionActorId, new TEvKafka::TEvJoinGroupRequest(header->CorrelationId, message)); + } else { + ReadSessionActorId = RegisterWithSameMailbox(CreateKafkaReadSessionActor(Context, 0)); + Send(ReadSessionActorId, new TEvKafka::TEvJoinGroupRequest(header->CorrelationId, message)); + } } - - Send(ReadSessionActorId, new TEvKafka::TEvSyncGroupRequest(header->CorrelationId, message)); } - void HandleMessage(const TRequestHeaderData* header, const TMessagePtr& message, const TActorContext& ctx) { - if (!ReadSessionActorId) { - ReadSessionActorId = ctx.RegisterWithSameMailbox(CreateKafkaReadSessionActor(Context, 0)); + void HandleMessage(const TRequestHeaderData* header, const TMessagePtr& message, const TActorContext& /*ctx*/) { + if (Context->Config.GetEnableNativeBalancing()) { + Register(new TKafkaBalancerActor(Context, 0, header->CorrelationId, message)); + } else { + if (ReadSessionActorId) { + Send(ReadSessionActorId, new TEvKafka::TEvSyncGroupRequest(header->CorrelationId, message)); + } else { + ReadSessionActorId = RegisterWithSameMailbox(CreateKafkaReadSessionActor(Context, 0)); + Send(ReadSessionActorId, new TEvKafka::TEvSyncGroupRequest(header->CorrelationId, message)); + } } - - Send(ReadSessionActorId, new TEvKafka::TEvHeartbeatRequest(header->CorrelationId, message)); } - void HandleMessage(const TRequestHeaderData* header, const TMessagePtr& message, const TActorContext& ctx) { - if (!ReadSessionActorId) { - ReadSessionActorId = ctx.RegisterWithSameMailbox(CreateKafkaReadSessionActor(Context, 0)); + void HandleMessage(const TRequestHeaderData* header, const TMessagePtr& message, const TActorContext& /*ctx*/) { + if (Context->Config.GetEnableNativeBalancing()) { + Register(new TKafkaBalancerActor(Context, 0, header->CorrelationId, message)); + } else { + if (ReadSessionActorId) { + Send(ReadSessionActorId, new TEvKafka::TEvHeartbeatRequest(header->CorrelationId, message)); + } else { + ReadSessionActorId = RegisterWithSameMailbox(CreateKafkaReadSessionActor(Context, 0)); + Send(ReadSessionActorId, new TEvKafka::TEvHeartbeatRequest(header->CorrelationId, message)); + } } - - Send(ReadSessionActorId, new TEvKafka::TEvJoinGroupRequest(header->CorrelationId, message)); } - void HandleMessage(const TRequestHeaderData* header, const TMessagePtr& message, const TActorContext& ctx) { - if (!ReadSessionActorId) { - ReadSessionActorId = ctx.RegisterWithSameMailbox(CreateKafkaReadSessionActor(Context, 0)); + void HandleMessage(const TRequestHeaderData* header, const TMessagePtr& message, const TActorContext& /*ctx*/) { + if (Context->Config.GetEnableNativeBalancing()) { + Register(new TKafkaBalancerActor(Context, 0, header->CorrelationId, message)); + } else { + if (ReadSessionActorId) { + Send(ReadSessionActorId, new TEvKafka::TEvLeaveGroupRequest(header->CorrelationId, message)); + } else { + ReadSessionActorId = RegisterWithSameMailbox(CreateKafkaReadSessionActor(Context, 0)); + Send(ReadSessionActorId, new TEvKafka::TEvLeaveGroupRequest(header->CorrelationId, message)); + } } - - Send(ReadSessionActorId, new TEvKafka::TEvLeaveGroupRequest(header->CorrelationId, message)); } void HandleMessage(const TRequestHeaderData* header, const TMessagePtr& message) { @@ -718,8 +740,8 @@ class TKafkaConnection: public TActorBootstrapped, public TNet } bool RequireAuthentication(EApiKey apiKey) { - return !(EApiKey::API_VERSIONS == apiKey || - EApiKey::SASL_HANDSHAKE == apiKey || + return !(EApiKey::API_VERSIONS == apiKey || + EApiKey::SASL_HANDSHAKE == apiKey || EApiKey::SASL_AUTHENTICATE == apiKey); } diff --git a/ydb/core/kafka_proxy/kafka_consumer_protocol.cpp b/ydb/core/kafka_proxy/kafka_consumer_protocol.cpp index ce54125ffd56..682cc0bec86d 100644 --- a/ydb/core/kafka_proxy/kafka_consumer_protocol.cpp +++ b/ydb/core/kafka_proxy/kafka_consumer_protocol.cpp @@ -115,29 +115,47 @@ i32 TConsumerProtocolSubscription::TopicPartition::Size(TKafkaVersion _version) // // TConsumerProtocolAssignment // -TConsumerProtocolAssignment::TConsumerProtocolAssignment() +TConsumerProtocolAssignment::TConsumerProtocolAssignment() {} -void TConsumerProtocolAssignment::Read(TKafkaReadable& _readable, TKafkaVersion _version) { - auto size = _readable.readUnsignedVarint(); +void TConsumerProtocolAssignment::Read(TKafkaReadable& r, TKafkaVersion v) { + bool useVarintSize = (v > 3); + + ui32 size; + if (useVarintSize) { + size = r.readUnsignedVarint(); + } else { + TKafkaInt32 s; + r >> s; + size = static_cast(s); + } Y_UNUSED(size); - _readable >> _version; - if (!NPrivate::VersionCheck(_version)) { - ythrow yexception() << "Can't read version " << _version << " of TConsumerProtocolAssignment"; + + TKafkaVersion assignmentVersion; + r >> assignmentVersion; + + if (!NPrivate::VersionCheck(assignmentVersion)) + { + ythrow yexception() << "Can't read version " << assignmentVersion + << " of TConsumerProtocolAssignment"; } - NPrivate::Read(_readable, _version, AssignedPartitions); - NPrivate::Read(_readable, _version, UserData); - if (NPrivate::VersionCheck(_version)) { - ui32 _numTaggedFields = _readable.readUnsignedVarint(); - for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { - ui32 _tag = _readable.readUnsignedVarint(); - ui32 _size = _readable.readUnsignedVarint(); - switch (_tag) { - default: - _readable.skip(_size); // skip unknown tag - break; - } + NPrivate::Read(r, assignmentVersion, AssignedPartitions); + NPrivate::Read(r, assignmentVersion, UserData); + + if (NPrivate::VersionCheck(assignmentVersion)) + { + ui32 numTaggedFields = r.readUnsignedVarint(); + for (ui32 i = 0; i < numTaggedFields; ++i) { + ui32 tag = r.readUnsignedVarint(); + Y_UNUSED(tag); + + ui32 tagSize = r.readUnsignedVarint(); + r.skip(tagSize); + + r.skip(tagSize); } } } @@ -149,19 +167,19 @@ void TConsumerProtocolAssignment::Write(TKafkaWritable& _writable, TKafkaVersion if (!NPrivate::VersionCheck(_version)) { ythrow yexception() << "Can't write version " << _version << " of TConsumerProtocolAssignment"; } - + if (useVarintSize) { _writable.writeUnsignedVarint(Size(_version) + 1); } else { TKafkaInt32 size = Size(_version); _writable << size; } - + _writable << _version; NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, AssignedPartitions); NPrivate::Write(_collector, _writable, _version, UserData); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); } @@ -172,7 +190,7 @@ i32 TConsumerProtocolAssignment::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, AssignedPartitions); NPrivate::Size(_collector, _version, UserData); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -184,7 +202,7 @@ i32 TConsumerProtocolAssignment::Size(TKafkaVersion _version) const { // // TConsumerProtocolAssignment::TopicPartition // -TConsumerProtocolAssignment::TopicPartition::TopicPartition() +TConsumerProtocolAssignment::TopicPartition::TopicPartition() {} const TConsumerProtocolAssignment::TopicPartition::TopicPartition::TopicMeta::Type TConsumerProtocolAssignment::TopicPartition::TopicPartition::TopicMeta::Default = {""}; @@ -195,7 +213,7 @@ void TConsumerProtocolAssignment::TopicPartition::Read(TKafkaReadable& _readable } NPrivate::Read(_readable, _version, Topic); NPrivate::Read(_readable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -217,7 +235,7 @@ void TConsumerProtocolAssignment::TopicPartition::Write(TKafkaWritable& _writabl NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Topic); NPrivate::Write(_collector, _writable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); } @@ -227,7 +245,7 @@ i32 TConsumerProtocolAssignment::TopicPartition::Size(TKafkaVersion _version) co NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Topic); NPrivate::Size(_collector, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } diff --git a/ydb/core/kafka_proxy/kafka_consumer_protocol.h b/ydb/core/kafka_proxy/kafka_consumer_protocol.h index 62f6e13c15ea..37e7528625d7 100644 --- a/ydb/core/kafka_proxy/kafka_consumer_protocol.h +++ b/ydb/core/kafka_proxy/kafka_consumer_protocol.h @@ -9,29 +9,29 @@ namespace NKafka { class TConsumerProtocolSubscription : public TMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 3}; static constexpr TKafkaVersions FlexibleVersions = VersionsNever; }; - + TConsumerProtocolSubscription(); ~TConsumerProtocolSubscription() = default; - + struct TopicPartition : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {1, 3}; static constexpr TKafkaVersions FlexibleVersions = VersionsNever; }; - + TopicPartition(); ~TopicPartition() = default; - + struct TopicMeta { using Type = TKafkaString; using TypeDesc = NPrivate::TKafkaStringDesc; - + static constexpr const char* Name = "topic"; static constexpr const char* About = ""; static const Type Default; // = {""}; @@ -42,13 +42,13 @@ class TConsumerProtocolSubscription : public TMessage { static constexpr TKafkaVersions FlexibleVersions = VersionsNever; }; TopicMeta::Type Topic; - + struct PartitionsMeta { using ItemType = TKafkaInt32; using ItemTypeDesc = NPrivate::TKafkaIntDesc; using Type = std::vector; using TypeDesc = NPrivate::TKafkaArrayDesc; - + static constexpr const char* Name = "partitions"; static constexpr const char* About = ""; @@ -62,34 +62,34 @@ class TConsumerProtocolSubscription : public TMessage { i32 Size(TKafkaVersion version) const override; void Read(TKafkaReadable& readable, TKafkaVersion version) override; void Write(TKafkaWritable& writable, TKafkaVersion version) const override; - + bool operator==(const TopicPartition& other) const = default; }; - + struct TopicsMeta { using ItemType = TKafkaString; using ItemTypeDesc = NPrivate::TKafkaStringDesc; using Type = std::vector; using TypeDesc = NPrivate::TKafkaArrayDesc; - + static constexpr const char* Name = "topics"; static constexpr const char* About = ""; - + static constexpr TKafkaVersions PresentVersions = VersionsAlways; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = VersionsNever; }; TopicsMeta::Type Topics; - + struct UserDataMeta { using Type = TKafkaBytes; using TypeDesc = NPrivate::TKafkaBytesDesc; - + static constexpr const char* Name = "userData"; static constexpr const char* About = ""; static const Type Default; // = std::nullopt; - + static constexpr TKafkaVersions PresentVersions = VersionsAlways; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsAlways; @@ -102,7 +102,7 @@ class TConsumerProtocolSubscription : public TMessage { using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; using TypeDesc = NPrivate::TKafkaArrayDesc; - + static constexpr const char* Name = "ownedPartitions"; static constexpr const char* About = ""; static const Type Default; // = {}; @@ -117,11 +117,11 @@ class TConsumerProtocolSubscription : public TMessage { struct GenerationIdMeta { using Type = TKafkaInt32; using TypeDesc = NPrivate::TKafkaIntDesc; - + static constexpr const char* Name = "generationId"; static constexpr const char* About = ""; static const Type Default; // = -1; - + static constexpr TKafkaVersions PresentVersions = {2, 3}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; @@ -132,22 +132,22 @@ class TConsumerProtocolSubscription : public TMessage { struct RackIdMeta { using Type = TKafkaString; using TypeDesc = NPrivate::TKafkaStringDesc; - + static constexpr const char* Name = "rackId"; static constexpr const char* About = ""; static const Type Default; // = {""}; - + static constexpr TKafkaVersions PresentVersions = {3, 3}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = {3, 3}; static constexpr TKafkaVersions FlexibleVersions = VersionsNever; }; RackIdMeta::Type RackId; - + i32 Size(TKafkaVersion version) const override; void Read(TKafkaReadable& readable, TKafkaVersion version) override; void Write(TKafkaWritable& writable, TKafkaVersion version) const override; - + bool operator==(const TConsumerProtocolSubscription& other) const = default; }; @@ -155,29 +155,29 @@ class TConsumerProtocolSubscription : public TMessage { class TConsumerProtocolAssignment : public TMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { - static constexpr TKafkaVersions PresentVersions = {0, 3}; + static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = VersionsNever; }; - + TConsumerProtocolAssignment(); ~TConsumerProtocolAssignment() = default; - + struct TopicPartition : public TMessage { public: struct MessageMeta { - static constexpr TKafkaVersions PresentVersions = {0, 3}; + static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = VersionsNever; }; - + TopicPartition(); ~TopicPartition() = default; - + struct TopicMeta { using Type = TKafkaString; using TypeDesc = NPrivate::TKafkaStringDesc; - + static constexpr const char* Name = "topic"; static constexpr const char* About = ""; static const Type Default; // = {""}; @@ -188,13 +188,13 @@ class TConsumerProtocolAssignment : public TMessage { static constexpr TKafkaVersions FlexibleVersions = VersionsNever; }; TopicMeta::Type Topic; - + struct PartitionsMeta { using ItemType = TKafkaInt32; using ItemTypeDesc = NPrivate::TKafkaIntDesc; using Type = std::vector; using TypeDesc = NPrivate::TKafkaArrayDesc; - + static constexpr const char* Name = "partitions"; static constexpr const char* About = ""; @@ -208,16 +208,16 @@ class TConsumerProtocolAssignment : public TMessage { i32 Size(TKafkaVersion version) const override; void Read(TKafkaReadable& readable, TKafkaVersion version) override; void Write(TKafkaWritable& writable, TKafkaVersion version) const override; - + bool operator==(const TopicPartition& other) const = default; }; - + struct AssignedPartitionsMeta { using ItemType = TopicPartition; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; using TypeDesc = NPrivate::TKafkaArrayDesc; - + static constexpr const char* Name = "assignedPartitions"; static constexpr const char* About = ""; @@ -227,26 +227,26 @@ class TConsumerProtocolAssignment : public TMessage { static constexpr TKafkaVersions FlexibleVersions = VersionsNever; }; AssignedPartitionsMeta::Type AssignedPartitions; - + struct UserDataMeta { using Type = TKafkaBytes; using TypeDesc = NPrivate::TKafkaBytesDesc; - + static constexpr const char* Name = "userData"; static constexpr const char* About = ""; 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 = VersionsNever; }; UserDataMeta::Type UserData; - + i32 Size(TKafkaVersion version) const override; void Read(TKafkaReadable& readable, TKafkaVersion version) override; void Write(TKafkaWritable& writable, TKafkaVersion version) const override; - + bool operator==(const TConsumerProtocolAssignment& other) const = default; }; diff --git a/ydb/core/kafka_proxy/kafka_messages.cpp b/ydb/core/kafka_proxy/kafka_messages.cpp index e7780da98d82..ea50786a15c7 100644 --- a/ydb/core/kafka_proxy/kafka_messages.cpp +++ b/ydb/core/kafka_proxy/kafka_messages.cpp @@ -349,7 +349,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) @@ -364,7 +364,7 @@ void TRequestHeaderData::Read(TKafkaReadable& _readable, TKafkaVersion _version) NPrivate::Read(_readable, _version, RequestApiVersion); NPrivate::Read(_readable, _version, CorrelationId); NPrivate::Read(_readable, _version, ClientId); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -388,10 +388,10 @@ void TRequestHeaderData::Write(TKafkaWritable& _writable, TKafkaVersion _version NPrivate::Write(_collector, _writable, _version, RequestApiVersion); NPrivate::Write(_collector, _writable, _version, CorrelationId); NPrivate::Write(_collector, _writable, _version, ClientId); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -401,7 +401,7 @@ i32 TRequestHeaderData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, RequestApiVersion); NPrivate::Size(_collector, _version, CorrelationId); NPrivate::Size(_collector, _version, ClientId); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -414,7 +414,7 @@ i32 TRequestHeaderData::Size(TKafkaVersion _version) const { // const TResponseHeaderData::CorrelationIdMeta::Type TResponseHeaderData::CorrelationIdMeta::Default = 0; -TResponseHeaderData::TResponseHeaderData() +TResponseHeaderData::TResponseHeaderData() : CorrelationId(CorrelationIdMeta::Default) {} @@ -423,7 +423,7 @@ void TResponseHeaderData::Read(TKafkaReadable& _readable, TKafkaVersion _version ythrow yexception() << "Can't read version " << _version << " of TResponseHeaderData"; } NPrivate::Read(_readable, _version, CorrelationId); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -444,17 +444,17 @@ void TResponseHeaderData::Write(TKafkaWritable& _writable, TKafkaVersion _versio } NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, CorrelationId); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } i32 TResponseHeaderData::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, CorrelationId); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -469,7 +469,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) @@ -483,7 +483,7 @@ void TProduceRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _version NPrivate::Read(_readable, _version, Acks); NPrivate::Read(_readable, _version, TimeoutMs); NPrivate::Read(_readable, _version, TopicData); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -507,10 +507,10 @@ void TProduceRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _versio NPrivate::Write(_collector, _writable, _version, Acks); NPrivate::Write(_collector, _writable, _version, TimeoutMs); NPrivate::Write(_collector, _writable, _version, TopicData); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -520,7 +520,7 @@ i32 TProduceRequestData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, Acks); NPrivate::Size(_collector, _version, TimeoutMs); NPrivate::Size(_collector, _version, TopicData); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -533,7 +533,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) {} @@ -543,7 +543,7 @@ void TProduceRequestData::TTopicProduceData::Read(TKafkaReadable& _readable, TKa } NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, PartitionData); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -565,10 +565,10 @@ void TProduceRequestData::TTopicProduceData::Write(TKafkaWritable& _writable, TK NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, PartitionData); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -576,7 +576,7 @@ i32 TProduceRequestData::TTopicProduceData::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, PartitionData); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -589,7 +589,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) {} @@ -599,7 +599,7 @@ void TProduceRequestData::TTopicProduceData::TPartitionProduceData::Read(TKafkaR } NPrivate::Read(_readable, _version, Index); NPrivate::Read(_readable, _version, Records); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -621,10 +621,10 @@ void TProduceRequestData::TTopicProduceData::TPartitionProduceData::Write(TKafka NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Index); NPrivate::Write(_collector, _writable, _version, Records); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -632,7 +632,7 @@ i32 TProduceRequestData::TTopicProduceData::TPartitionProduceData::Size(TKafkaVe NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Index); NPrivate::Size(_collector, _version, Records); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -645,7 +645,7 @@ i32 TProduceRequestData::TTopicProduceData::TPartitionProduceData::Size(TKafkaVe // const TProduceResponseData::ThrottleTimeMsMeta::Type TProduceResponseData::ThrottleTimeMsMeta::Default = 0; -TProduceResponseData::TProduceResponseData() +TProduceResponseData::TProduceResponseData() : ThrottleTimeMs(ThrottleTimeMsMeta::Default) {} @@ -655,7 +655,7 @@ void TProduceResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _versio } NPrivate::Read(_readable, _version, Responses); NPrivate::Read(_readable, _version, ThrottleTimeMs); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -677,10 +677,10 @@ void TProduceResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _versi NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Responses); NPrivate::Write(_collector, _writable, _version, ThrottleTimeMs); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -688,7 +688,7 @@ i32 TProduceResponseData::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Responses); NPrivate::Size(_collector, _version, ThrottleTimeMs); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -701,7 +701,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) {} @@ -711,7 +711,7 @@ void TProduceResponseData::TTopicProduceResponse::Read(TKafkaReadable& _readable } NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, PartitionResponses); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -733,10 +733,10 @@ void TProduceResponseData::TTopicProduceResponse::Write(TKafkaWritable& _writabl NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, PartitionResponses); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -744,7 +744,7 @@ i32 TProduceResponseData::TTopicProduceResponse::Size(TKafkaVersion _version) co NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, PartitionResponses); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -762,7 +762,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) @@ -782,7 +782,7 @@ void TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::Rea NPrivate::Read(_readable, _version, LogStartOffset); NPrivate::Read(_readable, _version, RecordErrors); NPrivate::Read(_readable, _version, ErrorMessage); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -809,10 +809,10 @@ void TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::Wri NPrivate::Write(_collector, _writable, _version, LogStartOffset); NPrivate::Write(_collector, _writable, _version, RecordErrors); NPrivate::Write(_collector, _writable, _version, ErrorMessage); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -825,7 +825,7 @@ i32 TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::Size NPrivate::Size(_collector, _version, LogStartOffset); NPrivate::Size(_collector, _version, RecordErrors); NPrivate::Size(_collector, _version, ErrorMessage); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -839,7 +839,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) {} @@ -850,7 +850,7 @@ void TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::TBa } NPrivate::Read(_readable, _version, BatchIndex); NPrivate::Read(_readable, _version, BatchIndexErrorMessage); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -872,10 +872,10 @@ void TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::TBa NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, BatchIndex); NPrivate::Write(_collector, _writable, _version, BatchIndexErrorMessage); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -883,7 +883,7 @@ i32 TProduceResponseData::TTopicProduceResponse::TPartitionProduceResponse::TBat NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, BatchIndex); NPrivate::Size(_collector, _version, BatchIndexErrorMessage); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -904,7 +904,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) @@ -931,7 +931,7 @@ void TFetchRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _version) NPrivate::Read(_readable, _version, Topics); NPrivate::Read(_readable, _version, ForgottenTopicsData); NPrivate::Read(_readable, _version, RackId); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -965,10 +965,10 @@ void TFetchRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _version) NPrivate::Write(_collector, _writable, _version, Topics); NPrivate::Write(_collector, _writable, _version, ForgottenTopicsData); NPrivate::Write(_collector, _writable, _version, RackId); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + NPrivate::WriteTag(_writable, _version, ClusterId); } } @@ -986,7 +986,7 @@ i32 TFetchRequestData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, Topics); NPrivate::Size(_collector, _version, ForgottenTopicsData); NPrivate::Size(_collector, _version, RackId); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1000,7 +1000,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) {} @@ -1012,7 +1012,7 @@ void TFetchRequestData::TFetchTopic::Read(TKafkaReadable& _readable, TKafkaVersi NPrivate::Read(_readable, _version, Topic); NPrivate::Read(_readable, _version, TopicId); NPrivate::Read(_readable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1035,10 +1035,10 @@ void TFetchRequestData::TFetchTopic::Write(TKafkaWritable& _writable, TKafkaVers NPrivate::Write(_collector, _writable, _version, Topic); NPrivate::Write(_collector, _writable, _version, TopicId); NPrivate::Write(_collector, _writable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -1047,7 +1047,7 @@ i32 TFetchRequestData::TFetchTopic::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, Topic); NPrivate::Size(_collector, _version, TopicId); NPrivate::Size(_collector, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1065,7 +1065,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) @@ -1084,7 +1084,7 @@ void TFetchRequestData::TFetchTopic::TFetchPartition::Read(TKafkaReadable& _read NPrivate::Read(_readable, _version, LastFetchedEpoch); NPrivate::Read(_readable, _version, LogStartOffset); NPrivate::Read(_readable, _version, PartitionMaxBytes); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1110,10 +1110,10 @@ void TFetchRequestData::TFetchTopic::TFetchPartition::Write(TKafkaWritable& _wri NPrivate::Write(_collector, _writable, _version, LastFetchedEpoch); NPrivate::Write(_collector, _writable, _version, LogStartOffset); NPrivate::Write(_collector, _writable, _version, PartitionMaxBytes); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -1125,7 +1125,7 @@ i32 TFetchRequestData::TFetchTopic::TFetchPartition::Size(TKafkaVersion _version NPrivate::Size(_collector, _version, LastFetchedEpoch); NPrivate::Size(_collector, _version, LogStartOffset); NPrivate::Size(_collector, _version, PartitionMaxBytes); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1139,7 +1139,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) {} @@ -1151,7 +1151,7 @@ void TFetchRequestData::TForgottenTopic::Read(TKafkaReadable& _readable, TKafkaV NPrivate::Read(_readable, _version, Topic); NPrivate::Read(_readable, _version, TopicId); NPrivate::Read(_readable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1174,10 +1174,10 @@ void TFetchRequestData::TForgottenTopic::Write(TKafkaWritable& _writable, TKafka NPrivate::Write(_collector, _writable, _version, Topic); NPrivate::Write(_collector, _writable, _version, TopicId); NPrivate::Write(_collector, _writable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -1186,7 +1186,7 @@ i32 TFetchRequestData::TForgottenTopic::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, Topic); NPrivate::Size(_collector, _version, TopicId); NPrivate::Size(_collector, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1201,7 +1201,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) @@ -1215,7 +1215,7 @@ void TFetchResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _version) NPrivate::Read(_readable, _version, ErrorCode); NPrivate::Read(_readable, _version, SessionId); NPrivate::Read(_readable, _version, Responses); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1239,10 +1239,10 @@ void TFetchResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _version NPrivate::Write(_collector, _writable, _version, ErrorCode); NPrivate::Write(_collector, _writable, _version, SessionId); NPrivate::Write(_collector, _writable, _version, Responses); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -1252,7 +1252,7 @@ i32 TFetchResponseData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, ErrorCode); NPrivate::Size(_collector, _version, SessionId); NPrivate::Size(_collector, _version, Responses); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1266,7 +1266,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) {} @@ -1278,7 +1278,7 @@ void TFetchResponseData::TFetchableTopicResponse::Read(TKafkaReadable& _readable NPrivate::Read(_readable, _version, Topic); NPrivate::Read(_readable, _version, TopicId); NPrivate::Read(_readable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1301,10 +1301,10 @@ void TFetchResponseData::TFetchableTopicResponse::Write(TKafkaWritable& _writabl NPrivate::Write(_collector, _writable, _version, Topic); NPrivate::Write(_collector, _writable, _version, TopicId); NPrivate::Write(_collector, _writable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -1313,7 +1313,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::Size(TKafkaVersion _version) co NPrivate::Size(_collector, _version, Topic); NPrivate::Size(_collector, _version, TopicId); NPrivate::Size(_collector, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1331,7 +1331,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) @@ -1355,7 +1355,7 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::Read(TKafkaRea NPrivate::Read(_readable, _version, AbortedTransactions); NPrivate::Read(_readable, _version, PreferredReadReplica); NPrivate::Read(_readable, _version, Records); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1395,10 +1395,10 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::Write(TKafkaWr NPrivate::Write(_collector, _writable, _version, AbortedTransactions); NPrivate::Write(_collector, _writable, _version, PreferredReadReplica); NPrivate::Write(_collector, _writable, _version, Records); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + NPrivate::WriteTag(_writable, _version, DivergingEpoch); NPrivate::WriteTag(_writable, _version, CurrentLeader); NPrivate::WriteTag(_writable, _version, SnapshotId); @@ -1418,7 +1418,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::Size(TKafkaVers NPrivate::Size(_collector, _version, AbortedTransactions); NPrivate::Size(_collector, _version, PreferredReadReplica); NPrivate::Size(_collector, _version, Records); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1432,7 +1432,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) {} @@ -1443,7 +1443,7 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TEpochEndOffse } NPrivate::Read(_readable, _version, Epoch); NPrivate::Read(_readable, _version, EndOffset); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1465,10 +1465,10 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TEpochEndOffse NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Epoch); NPrivate::Write(_collector, _writable, _version, EndOffset); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -1476,7 +1476,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::TEpochEndOffset NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Epoch); NPrivate::Size(_collector, _version, EndOffset); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1490,7 +1490,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) {} @@ -1501,7 +1501,7 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TLeaderIdAndEp } NPrivate::Read(_readable, _version, LeaderId); NPrivate::Read(_readable, _version, LeaderEpoch); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1523,10 +1523,10 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TLeaderIdAndEp NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, LeaderId); NPrivate::Write(_collector, _writable, _version, LeaderEpoch); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -1534,7 +1534,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::TLeaderIdAndEpo NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, LeaderId); NPrivate::Size(_collector, _version, LeaderEpoch); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1548,7 +1548,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) {} @@ -1559,7 +1559,7 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TSnapshotId::R } NPrivate::Read(_readable, _version, EndOffset); NPrivate::Read(_readable, _version, Epoch); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1581,10 +1581,10 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TSnapshotId::W NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, EndOffset); NPrivate::Write(_collector, _writable, _version, Epoch); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -1592,7 +1592,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::TSnapshotId::Si NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, EndOffset); NPrivate::Size(_collector, _version, Epoch); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1606,7 +1606,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) {} @@ -1617,7 +1617,7 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TAbortedTransa } NPrivate::Read(_readable, _version, ProducerId); NPrivate::Read(_readable, _version, FirstOffset); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1639,10 +1639,10 @@ void TFetchResponseData::TFetchableTopicResponse::TPartitionData::TAbortedTransa NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, ProducerId); NPrivate::Write(_collector, _writable, _version, FirstOffset); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -1650,7 +1650,7 @@ i32 TFetchResponseData::TFetchableTopicResponse::TPartitionData::TAbortedTransac NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, ProducerId); NPrivate::Size(_collector, _version, FirstOffset); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1664,7 +1664,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) {} @@ -1676,7 +1676,7 @@ void TListOffsetsRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _ver NPrivate::Read(_readable, _version, ReplicaId); NPrivate::Read(_readable, _version, IsolationLevel); NPrivate::Read(_readable, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1699,10 +1699,10 @@ void TListOffsetsRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _ve NPrivate::Write(_collector, _writable, _version, ReplicaId); NPrivate::Write(_collector, _writable, _version, IsolationLevel); NPrivate::Write(_collector, _writable, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -1711,7 +1711,7 @@ i32 TListOffsetsRequestData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, ReplicaId); NPrivate::Size(_collector, _version, IsolationLevel); NPrivate::Size(_collector, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1724,7 +1724,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) {} @@ -1734,7 +1734,7 @@ void TListOffsetsRequestData::TListOffsetsTopic::Read(TKafkaReadable& _readable, } NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1756,10 +1756,10 @@ void TListOffsetsRequestData::TListOffsetsTopic::Write(TKafkaWritable& _writable NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -1767,7 +1767,7 @@ i32 TListOffsetsRequestData::TListOffsetsTopic::Size(TKafkaVersion _version) con NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1783,7 +1783,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) @@ -1798,7 +1798,7 @@ void TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::Read(TKa NPrivate::Read(_readable, _version, CurrentLeaderEpoch); NPrivate::Read(_readable, _version, Timestamp); NPrivate::Read(_readable, _version, MaxNumOffsets); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1822,10 +1822,10 @@ void TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::Write(TK NPrivate::Write(_collector, _writable, _version, CurrentLeaderEpoch); NPrivate::Write(_collector, _writable, _version, Timestamp); NPrivate::Write(_collector, _writable, _version, MaxNumOffsets); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -1835,7 +1835,7 @@ i32 TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::Size(TKaf NPrivate::Size(_collector, _version, CurrentLeaderEpoch); NPrivate::Size(_collector, _version, Timestamp); NPrivate::Size(_collector, _version, MaxNumOffsets); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1848,7 +1848,7 @@ i32 TListOffsetsRequestData::TListOffsetsTopic::TListOffsetsPartition::Size(TKaf // const TListOffsetsResponseData::ThrottleTimeMsMeta::Type TListOffsetsResponseData::ThrottleTimeMsMeta::Default = 0; -TListOffsetsResponseData::TListOffsetsResponseData() +TListOffsetsResponseData::TListOffsetsResponseData() : ThrottleTimeMs(ThrottleTimeMsMeta::Default) {} @@ -1858,7 +1858,7 @@ void TListOffsetsResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _ve } NPrivate::Read(_readable, _version, ThrottleTimeMs); NPrivate::Read(_readable, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1880,10 +1880,10 @@ void TListOffsetsResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _v NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, ThrottleTimeMs); NPrivate::Write(_collector, _writable, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -1891,7 +1891,7 @@ i32 TListOffsetsResponseData::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, ThrottleTimeMs); NPrivate::Size(_collector, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1904,7 +1904,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) {} @@ -1914,7 +1914,7 @@ void TListOffsetsResponseData::TListOffsetsTopicResponse::Read(TKafkaReadable& _ } NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -1936,10 +1936,10 @@ void TListOffsetsResponseData::TListOffsetsTopicResponse::Write(TKafkaWritable& NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -1947,7 +1947,7 @@ i32 TListOffsetsResponseData::TListOffsetsTopicResponse::Size(TKafkaVersion _ver NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -1964,7 +1964,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) @@ -1982,7 +1982,7 @@ void TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsPartitionR NPrivate::Read(_readable, _version, Timestamp); NPrivate::Read(_readable, _version, Offset); NPrivate::Read(_readable, _version, LeaderEpoch); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2008,10 +2008,10 @@ void TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsPartitionR NPrivate::Write(_collector, _writable, _version, Timestamp); NPrivate::Write(_collector, _writable, _version, Offset); NPrivate::Write(_collector, _writable, _version, LeaderEpoch); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2023,7 +2023,7 @@ i32 TListOffsetsResponseData::TListOffsetsTopicResponse::TListOffsetsPartitionRe NPrivate::Size(_collector, _version, Timestamp); NPrivate::Size(_collector, _version, Offset); NPrivate::Size(_collector, _version, LeaderEpoch); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2038,7 +2038,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) @@ -2052,7 +2052,7 @@ void TMetadataRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _versio NPrivate::Read(_readable, _version, AllowAutoTopicCreation); NPrivate::Read(_readable, _version, IncludeClusterAuthorizedOperations); NPrivate::Read(_readable, _version, IncludeTopicAuthorizedOperations); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2076,10 +2076,10 @@ void TMetadataRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _versi NPrivate::Write(_collector, _writable, _version, AllowAutoTopicCreation); NPrivate::Write(_collector, _writable, _version, IncludeClusterAuthorizedOperations); NPrivate::Write(_collector, _writable, _version, IncludeTopicAuthorizedOperations); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2089,7 +2089,7 @@ i32 TMetadataRequestData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, AllowAutoTopicCreation); NPrivate::Size(_collector, _version, IncludeClusterAuthorizedOperations); NPrivate::Size(_collector, _version, IncludeTopicAuthorizedOperations); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2103,7 +2103,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) {} @@ -2114,7 +2114,7 @@ void TMetadataRequestData::TMetadataRequestTopic::Read(TKafkaReadable& _readable } NPrivate::Read(_readable, _version, TopicId); NPrivate::Read(_readable, _version, Name); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2136,10 +2136,10 @@ void TMetadataRequestData::TMetadataRequestTopic::Write(TKafkaWritable& _writabl NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, TopicId); NPrivate::Write(_collector, _writable, _version, Name); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2147,7 +2147,7 @@ i32 TMetadataRequestData::TMetadataRequestTopic::Size(TKafkaVersion _version) co NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, TopicId); NPrivate::Size(_collector, _version, Name); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2163,7 +2163,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) @@ -2180,7 +2180,7 @@ void TMetadataResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _versi NPrivate::Read(_readable, _version, ControllerId); NPrivate::Read(_readable, _version, Topics); NPrivate::Read(_readable, _version, ClusterAuthorizedOperations); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2206,10 +2206,10 @@ void TMetadataResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _vers NPrivate::Write(_collector, _writable, _version, ControllerId); NPrivate::Write(_collector, _writable, _version, Topics); NPrivate::Write(_collector, _writable, _version, ClusterAuthorizedOperations); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2221,7 +2221,7 @@ i32 TMetadataResponseData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, ControllerId); NPrivate::Size(_collector, _version, Topics); NPrivate::Size(_collector, _version, ClusterAuthorizedOperations); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2237,7 +2237,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) @@ -2252,7 +2252,7 @@ void TMetadataResponseData::TMetadataResponseBroker::Read(TKafkaReadable& _reada NPrivate::Read(_readable, _version, Host); NPrivate::Read(_readable, _version, Port); NPrivate::Read(_readable, _version, Rack); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2276,10 +2276,10 @@ void TMetadataResponseData::TMetadataResponseBroker::Write(TKafkaWritable& _writ NPrivate::Write(_collector, _writable, _version, Host); NPrivate::Write(_collector, _writable, _version, Port); NPrivate::Write(_collector, _writable, _version, Rack); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2289,7 +2289,7 @@ i32 TMetadataResponseData::TMetadataResponseBroker::Size(TKafkaVersion _version) NPrivate::Size(_collector, _version, Host); NPrivate::Size(_collector, _version, Port); NPrivate::Size(_collector, _version, Rack); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2306,7 +2306,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) @@ -2324,7 +2324,7 @@ void TMetadataResponseData::TMetadataResponseTopic::Read(TKafkaReadable& _readab NPrivate::Read(_readable, _version, IsInternal); NPrivate::Read(_readable, _version, Partitions); NPrivate::Read(_readable, _version, TopicAuthorizedOperations); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2350,10 +2350,10 @@ void TMetadataResponseData::TMetadataResponseTopic::Write(TKafkaWritable& _writa NPrivate::Write(_collector, _writable, _version, IsInternal); NPrivate::Write(_collector, _writable, _version, Partitions); NPrivate::Write(_collector, _writable, _version, TopicAuthorizedOperations); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2365,7 +2365,7 @@ i32 TMetadataResponseData::TMetadataResponseTopic::Size(TKafkaVersion _version) NPrivate::Size(_collector, _version, IsInternal); NPrivate::Size(_collector, _version, Partitions); NPrivate::Size(_collector, _version, TopicAuthorizedOperations); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2381,7 +2381,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) @@ -2399,7 +2399,7 @@ void TMetadataResponseData::TMetadataResponseTopic::TMetadataResponsePartition:: NPrivate::Read(_readable, _version, ReplicaNodes); NPrivate::Read(_readable, _version, IsrNodes); NPrivate::Read(_readable, _version, OfflineReplicas); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2426,10 +2426,10 @@ void TMetadataResponseData::TMetadataResponseTopic::TMetadataResponsePartition:: NPrivate::Write(_collector, _writable, _version, ReplicaNodes); NPrivate::Write(_collector, _writable, _version, IsrNodes); NPrivate::Write(_collector, _writable, _version, OfflineReplicas); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2442,7 +2442,7 @@ i32 TMetadataResponseData::TMetadataResponseTopic::TMetadataResponsePartition::S NPrivate::Size(_collector, _version, ReplicaNodes); NPrivate::Size(_collector, _version, IsrNodes); NPrivate::Size(_collector, _version, OfflineReplicas); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2459,7 +2459,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) @@ -2477,7 +2477,7 @@ void TOffsetCommitRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _ve NPrivate::Read(_readable, _version, GroupInstanceId); NPrivate::Read(_readable, _version, RetentionTimeMs); NPrivate::Read(_readable, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2503,10 +2503,10 @@ void TOffsetCommitRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _v NPrivate::Write(_collector, _writable, _version, GroupInstanceId); NPrivate::Write(_collector, _writable, _version, RetentionTimeMs); NPrivate::Write(_collector, _writable, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2518,7 +2518,7 @@ i32 TOffsetCommitRequestData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, GroupInstanceId); NPrivate::Size(_collector, _version, RetentionTimeMs); NPrivate::Size(_collector, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2531,7 +2531,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) {} @@ -2541,7 +2541,7 @@ void TOffsetCommitRequestData::TOffsetCommitRequestTopic::Read(TKafkaReadable& _ } NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2563,10 +2563,10 @@ void TOffsetCommitRequestData::TOffsetCommitRequestTopic::Write(TKafkaWritable& NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2574,7 +2574,7 @@ i32 TOffsetCommitRequestData::TOffsetCommitRequestTopic::Size(TKafkaVersion _ver NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2591,7 +2591,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) @@ -2608,7 +2608,7 @@ void TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestPa NPrivate::Read(_readable, _version, CommittedLeaderEpoch); NPrivate::Read(_readable, _version, CommitTimestamp); NPrivate::Read(_readable, _version, CommittedMetadata); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2633,10 +2633,10 @@ void TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestPa NPrivate::Write(_collector, _writable, _version, CommittedLeaderEpoch); NPrivate::Write(_collector, _writable, _version, CommitTimestamp); NPrivate::Write(_collector, _writable, _version, CommittedMetadata); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2647,7 +2647,7 @@ i32 TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestPar NPrivate::Size(_collector, _version, CommittedLeaderEpoch); NPrivate::Size(_collector, _version, CommitTimestamp); NPrivate::Size(_collector, _version, CommittedMetadata); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2660,7 +2660,7 @@ i32 TOffsetCommitRequestData::TOffsetCommitRequestTopic::TOffsetCommitRequestPar // const TOffsetCommitResponseData::ThrottleTimeMsMeta::Type TOffsetCommitResponseData::ThrottleTimeMsMeta::Default = 0; -TOffsetCommitResponseData::TOffsetCommitResponseData() +TOffsetCommitResponseData::TOffsetCommitResponseData() : ThrottleTimeMs(ThrottleTimeMsMeta::Default) {} @@ -2670,7 +2670,7 @@ void TOffsetCommitResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _v } NPrivate::Read(_readable, _version, ThrottleTimeMs); NPrivate::Read(_readable, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2692,10 +2692,10 @@ void TOffsetCommitResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _ NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, ThrottleTimeMs); NPrivate::Write(_collector, _writable, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2703,7 +2703,7 @@ i32 TOffsetCommitResponseData::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, ThrottleTimeMs); NPrivate::Size(_collector, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2716,7 +2716,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) {} @@ -2726,7 +2726,7 @@ void TOffsetCommitResponseData::TOffsetCommitResponseTopic::Read(TKafkaReadable& } NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2748,10 +2748,10 @@ void TOffsetCommitResponseData::TOffsetCommitResponseTopic::Write(TKafkaWritable NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2759,7 +2759,7 @@ i32 TOffsetCommitResponseData::TOffsetCommitResponseTopic::Size(TKafkaVersion _v NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2773,7 +2773,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) {} @@ -2784,7 +2784,7 @@ void TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitRespons } NPrivate::Read(_readable, _version, PartitionIndex); NPrivate::Read(_readable, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2806,10 +2806,10 @@ void TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitRespons NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, PartitionIndex); NPrivate::Write(_collector, _writable, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2817,7 +2817,7 @@ i32 TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitResponse NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, PartitionIndex); NPrivate::Size(_collector, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2831,7 +2831,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) {} @@ -2844,7 +2844,7 @@ void TOffsetFetchRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _ver NPrivate::Read(_readable, _version, Topics); NPrivate::Read(_readable, _version, Groups); NPrivate::Read(_readable, _version, RequireStable); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2868,10 +2868,10 @@ void TOffsetFetchRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _ve NPrivate::Write(_collector, _writable, _version, Topics); NPrivate::Write(_collector, _writable, _version, Groups); NPrivate::Write(_collector, _writable, _version, RequireStable); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2881,7 +2881,7 @@ i32 TOffsetFetchRequestData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, Topics); NPrivate::Size(_collector, _version, Groups); NPrivate::Size(_collector, _version, RequireStable); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2894,7 +2894,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) {} @@ -2904,7 +2904,7 @@ void TOffsetFetchRequestData::TOffsetFetchRequestTopic::Read(TKafkaReadable& _re } NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, PartitionIndexes); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2926,10 +2926,10 @@ void TOffsetFetchRequestData::TOffsetFetchRequestTopic::Write(TKafkaWritable& _w NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, PartitionIndexes); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2937,7 +2937,7 @@ i32 TOffsetFetchRequestData::TOffsetFetchRequestTopic::Size(TKafkaVersion _versi NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, PartitionIndexes); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -2950,7 +2950,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) {} @@ -2960,7 +2960,7 @@ void TOffsetFetchRequestData::TOffsetFetchRequestGroup::Read(TKafkaReadable& _re } NPrivate::Read(_readable, _version, GroupId); NPrivate::Read(_readable, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -2982,10 +2982,10 @@ void TOffsetFetchRequestData::TOffsetFetchRequestGroup::Write(TKafkaWritable& _w NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, GroupId); NPrivate::Write(_collector, _writable, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -2993,7 +2993,7 @@ i32 TOffsetFetchRequestData::TOffsetFetchRequestGroup::Size(TKafkaVersion _versi NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, GroupId); NPrivate::Size(_collector, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3006,7 +3006,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) {} @@ -3016,7 +3016,7 @@ void TOffsetFetchRequestData::TOffsetFetchRequestGroup::TOffsetFetchRequestTopic } NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, PartitionIndexes); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -3038,10 +3038,10 @@ void TOffsetFetchRequestData::TOffsetFetchRequestGroup::TOffsetFetchRequestTopic NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, PartitionIndexes); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -3049,7 +3049,7 @@ i32 TOffsetFetchRequestData::TOffsetFetchRequestGroup::TOffsetFetchRequestTopics NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, PartitionIndexes); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3063,7 +3063,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) {} @@ -3076,7 +3076,7 @@ void TOffsetFetchResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _ve NPrivate::Read(_readable, _version, Topics); NPrivate::Read(_readable, _version, ErrorCode); NPrivate::Read(_readable, _version, Groups); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -3100,10 +3100,10 @@ void TOffsetFetchResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _v NPrivate::Write(_collector, _writable, _version, Topics); NPrivate::Write(_collector, _writable, _version, ErrorCode); NPrivate::Write(_collector, _writable, _version, Groups); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -3113,7 +3113,7 @@ i32 TOffsetFetchResponseData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, Topics); NPrivate::Size(_collector, _version, ErrorCode); NPrivate::Size(_collector, _version, Groups); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3126,7 +3126,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) {} @@ -3136,7 +3136,7 @@ void TOffsetFetchResponseData::TOffsetFetchResponseTopic::Read(TKafkaReadable& _ } NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -3158,10 +3158,10 @@ void TOffsetFetchResponseData::TOffsetFetchResponseTopic::Write(TKafkaWritable& NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -3169,7 +3169,7 @@ i32 TOffsetFetchResponseData::TOffsetFetchResponseTopic::Size(TKafkaVersion _ver NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3186,7 +3186,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) @@ -3203,7 +3203,7 @@ void TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponsePa NPrivate::Read(_readable, _version, CommittedLeaderEpoch); NPrivate::Read(_readable, _version, Metadata); NPrivate::Read(_readable, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -3228,10 +3228,10 @@ void TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponsePa NPrivate::Write(_collector, _writable, _version, CommittedLeaderEpoch); NPrivate::Write(_collector, _writable, _version, Metadata); NPrivate::Write(_collector, _writable, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -3242,7 +3242,7 @@ i32 TOffsetFetchResponseData::TOffsetFetchResponseTopic::TOffsetFetchResponsePar NPrivate::Size(_collector, _version, CommittedLeaderEpoch); NPrivate::Size(_collector, _version, Metadata); NPrivate::Size(_collector, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3256,7 +3256,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) {} @@ -3268,7 +3268,7 @@ void TOffsetFetchResponseData::TOffsetFetchResponseGroup::Read(TKafkaReadable& _ NPrivate::Read(_readable, _version, GroupId); NPrivate::Read(_readable, _version, Topics); NPrivate::Read(_readable, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -3291,10 +3291,10 @@ void TOffsetFetchResponseData::TOffsetFetchResponseGroup::Write(TKafkaWritable& NPrivate::Write(_collector, _writable, _version, GroupId); NPrivate::Write(_collector, _writable, _version, Topics); NPrivate::Write(_collector, _writable, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -3303,7 +3303,7 @@ i32 TOffsetFetchResponseData::TOffsetFetchResponseGroup::Size(TKafkaVersion _ver NPrivate::Size(_collector, _version, GroupId); NPrivate::Size(_collector, _version, Topics); NPrivate::Size(_collector, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3316,7 +3316,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) {} @@ -3326,7 +3326,7 @@ void TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTo } NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -3348,10 +3348,10 @@ void TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTo NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -3359,7 +3359,7 @@ i32 TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTop NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, Partitions); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3376,7 +3376,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) @@ -3393,7 +3393,7 @@ void TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTo NPrivate::Read(_readable, _version, CommittedLeaderEpoch); NPrivate::Read(_readable, _version, Metadata); NPrivate::Read(_readable, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -3418,10 +3418,10 @@ void TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTo NPrivate::Write(_collector, _writable, _version, CommittedLeaderEpoch); NPrivate::Write(_collector, _writable, _version, Metadata); NPrivate::Write(_collector, _writable, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -3432,7 +3432,7 @@ i32 TOffsetFetchResponseData::TOffsetFetchResponseGroup::TOffsetFetchResponseTop NPrivate::Size(_collector, _version, CommittedLeaderEpoch); NPrivate::Size(_collector, _version, Metadata); NPrivate::Size(_collector, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3446,7 +3446,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) {} @@ -3458,7 +3458,7 @@ void TFindCoordinatorRequestData::Read(TKafkaReadable& _readable, TKafkaVersion NPrivate::Read(_readable, _version, Key); NPrivate::Read(_readable, _version, KeyType); NPrivate::Read(_readable, _version, CoordinatorKeys); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -3481,10 +3481,10 @@ void TFindCoordinatorRequestData::Write(TKafkaWritable& _writable, TKafkaVersion NPrivate::Write(_collector, _writable, _version, Key); NPrivate::Write(_collector, _writable, _version, KeyType); NPrivate::Write(_collector, _writable, _version, CoordinatorKeys); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -3493,7 +3493,7 @@ i32 TFindCoordinatorRequestData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, Key); NPrivate::Size(_collector, _version, KeyType); NPrivate::Size(_collector, _version, CoordinatorKeys); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3511,7 +3511,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) @@ -3531,7 +3531,7 @@ void TFindCoordinatorResponseData::Read(TKafkaReadable& _readable, TKafkaVersion NPrivate::Read(_readable, _version, Host); NPrivate::Read(_readable, _version, Port); NPrivate::Read(_readable, _version, Coordinators); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -3558,10 +3558,10 @@ void TFindCoordinatorResponseData::Write(TKafkaWritable& _writable, TKafkaVersio NPrivate::Write(_collector, _writable, _version, Host); NPrivate::Write(_collector, _writable, _version, Port); NPrivate::Write(_collector, _writable, _version, Coordinators); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -3574,7 +3574,7 @@ i32 TFindCoordinatorResponseData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, Host); NPrivate::Size(_collector, _version, Port); NPrivate::Size(_collector, _version, Coordinators); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3592,7 +3592,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) @@ -3611,7 +3611,7 @@ void TFindCoordinatorResponseData::TCoordinator::Read(TKafkaReadable& _readable, NPrivate::Read(_readable, _version, Port); NPrivate::Read(_readable, _version, ErrorCode); NPrivate::Read(_readable, _version, ErrorMessage); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -3637,10 +3637,10 @@ void TFindCoordinatorResponseData::TCoordinator::Write(TKafkaWritable& _writable NPrivate::Write(_collector, _writable, _version, Port); NPrivate::Write(_collector, _writable, _version, ErrorCode); NPrivate::Write(_collector, _writable, _version, ErrorMessage); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -3652,7 +3652,7 @@ i32 TFindCoordinatorResponseData::TCoordinator::Size(TKafkaVersion _version) con NPrivate::Size(_collector, _version, Port); NPrivate::Size(_collector, _version, ErrorCode); NPrivate::Size(_collector, _version, ErrorMessage); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3671,7 +3671,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) @@ -3693,7 +3693,7 @@ void TJoinGroupRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _versi NPrivate::Read(_readable, _version, ProtocolType); NPrivate::Read(_readable, _version, Protocols); NPrivate::Read(_readable, _version, Reason); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -3721,10 +3721,10 @@ void TJoinGroupRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _vers NPrivate::Write(_collector, _writable, _version, ProtocolType); NPrivate::Write(_collector, _writable, _version, Protocols); NPrivate::Write(_collector, _writable, _version, Reason); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -3738,7 +3738,7 @@ i32 TJoinGroupRequestData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, ProtocolType); NPrivate::Size(_collector, _version, Protocols); NPrivate::Size(_collector, _version, Reason); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3751,7 +3751,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) {} @@ -3761,7 +3761,7 @@ void TJoinGroupRequestData::TJoinGroupRequestProtocol::Read(TKafkaReadable& _rea } NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, Metadata); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -3783,10 +3783,10 @@ void TJoinGroupRequestData::TJoinGroupRequestProtocol::Write(TKafkaWritable& _wr NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, Metadata); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -3794,7 +3794,7 @@ i32 TJoinGroupRequestData::TJoinGroupRequestProtocol::Size(TKafkaVersion _versio NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, Metadata); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3814,7 +3814,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) @@ -3838,7 +3838,7 @@ void TJoinGroupResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _vers NPrivate::Read(_readable, _version, SkipAssignment); NPrivate::Read(_readable, _version, MemberId); NPrivate::Read(_readable, _version, Members); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -3867,10 +3867,10 @@ void TJoinGroupResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _ver NPrivate::Write(_collector, _writable, _version, SkipAssignment); NPrivate::Write(_collector, _writable, _version, MemberId); NPrivate::Write(_collector, _writable, _version, Members); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -3885,7 +3885,7 @@ i32 TJoinGroupResponseData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, SkipAssignment); NPrivate::Size(_collector, _version, MemberId); NPrivate::Size(_collector, _version, Members); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3899,7 +3899,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) {} @@ -3911,7 +3911,7 @@ void TJoinGroupResponseData::TJoinGroupResponseMember::Read(TKafkaReadable& _rea NPrivate::Read(_readable, _version, MemberId); NPrivate::Read(_readable, _version, GroupInstanceId); NPrivate::Read(_readable, _version, Metadata); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -3934,10 +3934,10 @@ void TJoinGroupResponseData::TJoinGroupResponseMember::Write(TKafkaWritable& _wr NPrivate::Write(_collector, _writable, _version, MemberId); NPrivate::Write(_collector, _writable, _version, GroupInstanceId); NPrivate::Write(_collector, _writable, _version, Metadata); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -3946,7 +3946,7 @@ i32 TJoinGroupResponseData::TJoinGroupResponseMember::Size(TKafkaVersion _versio NPrivate::Size(_collector, _version, MemberId); NPrivate::Size(_collector, _version, GroupInstanceId); NPrivate::Size(_collector, _version, Metadata); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -3962,7 +3962,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) @@ -3977,7 +3977,7 @@ void THeartbeatRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _versi NPrivate::Read(_readable, _version, GenerationId); NPrivate::Read(_readable, _version, MemberId); NPrivate::Read(_readable, _version, GroupInstanceId); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4001,10 +4001,10 @@ void THeartbeatRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _vers NPrivate::Write(_collector, _writable, _version, GenerationId); NPrivate::Write(_collector, _writable, _version, MemberId); NPrivate::Write(_collector, _writable, _version, GroupInstanceId); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -4014,7 +4014,7 @@ i32 THeartbeatRequestData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, GenerationId); NPrivate::Size(_collector, _version, MemberId); NPrivate::Size(_collector, _version, GroupInstanceId); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4028,7 +4028,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) {} @@ -4039,7 +4039,7 @@ void THeartbeatResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _vers } NPrivate::Read(_readable, _version, ThrottleTimeMs); NPrivate::Read(_readable, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4061,10 +4061,10 @@ void THeartbeatResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _ver NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, ThrottleTimeMs); NPrivate::Write(_collector, _writable, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -4072,7 +4072,7 @@ i32 THeartbeatResponseData::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, ThrottleTimeMs); NPrivate::Size(_collector, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4086,7 +4086,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) {} @@ -4098,7 +4098,7 @@ void TLeaveGroupRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _vers NPrivate::Read(_readable, _version, GroupId); NPrivate::Read(_readable, _version, MemberId); NPrivate::Read(_readable, _version, Members); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4121,10 +4121,10 @@ void TLeaveGroupRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _ver NPrivate::Write(_collector, _writable, _version, GroupId); NPrivate::Write(_collector, _writable, _version, MemberId); NPrivate::Write(_collector, _writable, _version, Members); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -4133,7 +4133,7 @@ i32 TLeaveGroupRequestData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, GroupId); NPrivate::Size(_collector, _version, MemberId); NPrivate::Size(_collector, _version, Members); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4148,7 +4148,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) @@ -4161,7 +4161,7 @@ void TLeaveGroupRequestData::TMemberIdentity::Read(TKafkaReadable& _readable, TK NPrivate::Read(_readable, _version, MemberId); NPrivate::Read(_readable, _version, GroupInstanceId); NPrivate::Read(_readable, _version, Reason); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4184,10 +4184,10 @@ void TLeaveGroupRequestData::TMemberIdentity::Write(TKafkaWritable& _writable, T NPrivate::Write(_collector, _writable, _version, MemberId); NPrivate::Write(_collector, _writable, _version, GroupInstanceId); NPrivate::Write(_collector, _writable, _version, Reason); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -4196,7 +4196,7 @@ i32 TLeaveGroupRequestData::TMemberIdentity::Size(TKafkaVersion _version) const NPrivate::Size(_collector, _version, MemberId); NPrivate::Size(_collector, _version, GroupInstanceId); NPrivate::Size(_collector, _version, Reason); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4210,7 +4210,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) {} @@ -4222,7 +4222,7 @@ void TLeaveGroupResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _ver NPrivate::Read(_readable, _version, ThrottleTimeMs); NPrivate::Read(_readable, _version, ErrorCode); NPrivate::Read(_readable, _version, Members); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4245,10 +4245,10 @@ void TLeaveGroupResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _ve NPrivate::Write(_collector, _writable, _version, ThrottleTimeMs); NPrivate::Write(_collector, _writable, _version, ErrorCode); NPrivate::Write(_collector, _writable, _version, Members); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -4257,7 +4257,7 @@ i32 TLeaveGroupResponseData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, ThrottleTimeMs); NPrivate::Size(_collector, _version, ErrorCode); NPrivate::Size(_collector, _version, Members); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4272,7 +4272,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) @@ -4285,7 +4285,7 @@ void TLeaveGroupResponseData::TMemberResponse::Read(TKafkaReadable& _readable, T NPrivate::Read(_readable, _version, MemberId); NPrivate::Read(_readable, _version, GroupInstanceId); NPrivate::Read(_readable, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4308,10 +4308,10 @@ void TLeaveGroupResponseData::TMemberResponse::Write(TKafkaWritable& _writable, NPrivate::Write(_collector, _writable, _version, MemberId); NPrivate::Write(_collector, _writable, _version, GroupInstanceId); NPrivate::Write(_collector, _writable, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -4320,7 +4320,7 @@ i32 TLeaveGroupResponseData::TMemberResponse::Size(TKafkaVersion _version) const NPrivate::Size(_collector, _version, MemberId); NPrivate::Size(_collector, _version, GroupInstanceId); NPrivate::Size(_collector, _version, ErrorCode); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4338,7 +4338,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) @@ -4358,7 +4358,7 @@ void TSyncGroupRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _versi NPrivate::Read(_readable, _version, ProtocolType); NPrivate::Read(_readable, _version, ProtocolName); NPrivate::Read(_readable, _version, Assignments); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4385,10 +4385,10 @@ void TSyncGroupRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _vers NPrivate::Write(_collector, _writable, _version, ProtocolType); NPrivate::Write(_collector, _writable, _version, ProtocolName); NPrivate::Write(_collector, _writable, _version, Assignments); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -4401,7 +4401,7 @@ i32 TSyncGroupRequestData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, ProtocolType); NPrivate::Size(_collector, _version, ProtocolName); NPrivate::Size(_collector, _version, Assignments); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4414,7 +4414,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) {} @@ -4424,7 +4424,7 @@ void TSyncGroupRequestData::TSyncGroupRequestAssignment::Read(TKafkaReadable& _r } NPrivate::Read(_readable, _version, MemberId); NPrivate::Read(_readable, _version, Assignment); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4446,10 +4446,10 @@ void TSyncGroupRequestData::TSyncGroupRequestAssignment::Write(TKafkaWritable& _ NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, MemberId); NPrivate::Write(_collector, _writable, _version, Assignment); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -4457,7 +4457,7 @@ i32 TSyncGroupRequestData::TSyncGroupRequestAssignment::Size(TKafkaVersion _vers NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, MemberId); NPrivate::Size(_collector, _version, Assignment); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4473,7 +4473,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) @@ -4489,7 +4489,7 @@ void TSyncGroupResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _vers NPrivate::Read(_readable, _version, ProtocolType); NPrivate::Read(_readable, _version, ProtocolName); NPrivate::Read(_readable, _version, Assignment); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4514,10 +4514,10 @@ void TSyncGroupResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _ver NPrivate::Write(_collector, _writable, _version, ProtocolType); NPrivate::Write(_collector, _writable, _version, ProtocolName); NPrivate::Write(_collector, _writable, _version, Assignment); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -4527,28 +4527,20 @@ i32 TSyncGroupResponseData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, ErrorCode); NPrivate::Size(_collector, _version, ProtocolType); NPrivate::Size(_collector, _version, ProtocolName); - NPrivate::TSizeCollector _assignmentCollector; - NPrivate::Size(_assignmentCollector, _version, Assignment); NPrivate::Size(_collector, _version, Assignment); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } - auto useVarintSize = _version > 3; - if (useVarintSize) { - return _collector.Size + NPrivate::SizeOfUnsignedVarint(_assignmentCollector.Size + 1); - } else { - return _collector.Size + sizeof(TKafkaInt32); - } + return _collector.Size; } - // // TSaslHandshakeRequestData // const TSaslHandshakeRequestData::MechanismMeta::Type TSaslHandshakeRequestData::MechanismMeta::Default = {""}; -TSaslHandshakeRequestData::TSaslHandshakeRequestData() +TSaslHandshakeRequestData::TSaslHandshakeRequestData() : Mechanism(MechanismMeta::Default) {} @@ -4557,7 +4549,7 @@ void TSaslHandshakeRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _v ythrow yexception() << "Can't read version " << _version << " of TSaslHandshakeRequestData"; } NPrivate::Read(_readable, _version, Mechanism); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4578,17 +4570,17 @@ void TSaslHandshakeRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _ } NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Mechanism); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } i32 TSaslHandshakeRequestData::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Mechanism); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4601,7 +4593,7 @@ i32 TSaslHandshakeRequestData::Size(TKafkaVersion _version) const { // const TSaslHandshakeResponseData::ErrorCodeMeta::Type TSaslHandshakeResponseData::ErrorCodeMeta::Default = 0; -TSaslHandshakeResponseData::TSaslHandshakeResponseData() +TSaslHandshakeResponseData::TSaslHandshakeResponseData() : ErrorCode(ErrorCodeMeta::Default) {} @@ -4611,7 +4603,7 @@ void TSaslHandshakeResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _ } NPrivate::Read(_readable, _version, ErrorCode); NPrivate::Read(_readable, _version, Mechanisms); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4633,10 +4625,10 @@ void TSaslHandshakeResponseData::Write(TKafkaWritable& _writable, TKafkaVersion NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, ErrorCode); NPrivate::Write(_collector, _writable, _version, Mechanisms); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -4644,7 +4636,7 @@ i32 TSaslHandshakeResponseData::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, ErrorCode); NPrivate::Size(_collector, _version, Mechanisms); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4658,7 +4650,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) {} @@ -4669,7 +4661,7 @@ void TApiVersionsRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _ver } NPrivate::Read(_readable, _version, ClientSoftwareName); NPrivate::Read(_readable, _version, ClientSoftwareVersion); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4691,10 +4683,10 @@ void TApiVersionsRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _ve NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, ClientSoftwareName); NPrivate::Write(_collector, _writable, _version, ClientSoftwareVersion); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -4702,7 +4694,7 @@ i32 TApiVersionsRequestData::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, ClientSoftwareName); NPrivate::Size(_collector, _version, ClientSoftwareVersion); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4718,7 +4710,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) @@ -4736,7 +4728,7 @@ void TApiVersionsResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _ve NPrivate::Read(_readable, _version, FinalizedFeaturesEpoch); NPrivate::Read(_readable, _version, FinalizedFeatures); NPrivate::Read(_readable, _version, ZkMigrationReady); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4775,10 +4767,10 @@ void TApiVersionsResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _v NPrivate::Write(_collector, _writable, _version, FinalizedFeaturesEpoch); NPrivate::Write(_collector, _writable, _version, FinalizedFeatures); NPrivate::Write(_collector, _writable, _version, ZkMigrationReady); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + NPrivate::WriteTag(_writable, _version, SupportedFeatures); NPrivate::WriteTag(_writable, _version, FinalizedFeaturesEpoch); NPrivate::WriteTag(_writable, _version, FinalizedFeatures); @@ -4795,7 +4787,7 @@ i32 TApiVersionsResponseData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, FinalizedFeaturesEpoch); NPrivate::Size(_collector, _version, FinalizedFeatures); NPrivate::Size(_collector, _version, ZkMigrationReady); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4810,7 +4802,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) @@ -4823,7 +4815,7 @@ void TApiVersionsResponseData::TApiVersion::Read(TKafkaReadable& _readable, TKaf NPrivate::Read(_readable, _version, ApiKey); NPrivate::Read(_readable, _version, MinVersion); NPrivate::Read(_readable, _version, MaxVersion); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4846,10 +4838,10 @@ void TApiVersionsResponseData::TApiVersion::Write(TKafkaWritable& _writable, TKa NPrivate::Write(_collector, _writable, _version, ApiKey); NPrivate::Write(_collector, _writable, _version, MinVersion); NPrivate::Write(_collector, _writable, _version, MaxVersion); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -4858,7 +4850,7 @@ i32 TApiVersionsResponseData::TApiVersion::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, ApiKey); NPrivate::Size(_collector, _version, MinVersion); NPrivate::Size(_collector, _version, MaxVersion); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4873,7 +4865,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) @@ -4886,7 +4878,7 @@ void TApiVersionsResponseData::TSupportedFeatureKey::Read(TKafkaReadable& _reada NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, MinVersion); NPrivate::Read(_readable, _version, MaxVersion); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4909,10 +4901,10 @@ void TApiVersionsResponseData::TSupportedFeatureKey::Write(TKafkaWritable& _writ NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, MinVersion); NPrivate::Write(_collector, _writable, _version, MaxVersion); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -4921,7 +4913,7 @@ i32 TApiVersionsResponseData::TSupportedFeatureKey::Size(TKafkaVersion _version) NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, MinVersion); NPrivate::Size(_collector, _version, MaxVersion); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4936,7 +4928,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) @@ -4949,7 +4941,7 @@ void TApiVersionsResponseData::TFinalizedFeatureKey::Read(TKafkaReadable& _reada NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, MaxVersionLevel); NPrivate::Read(_readable, _version, MinVersionLevel); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -4972,10 +4964,10 @@ void TApiVersionsResponseData::TFinalizedFeatureKey::Write(TKafkaWritable& _writ NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, MaxVersionLevel); NPrivate::Write(_collector, _writable, _version, MinVersionLevel); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -4984,7 +4976,7 @@ i32 TApiVersionsResponseData::TFinalizedFeatureKey::Size(TKafkaVersion _version) NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, MaxVersionLevel); NPrivate::Size(_collector, _version, MinVersionLevel); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -4998,7 +4990,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) {} @@ -5010,7 +5002,7 @@ void TCreateTopicsRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _ve NPrivate::Read(_readable, _version, Topics); NPrivate::Read(_readable, _version, TimeoutMs); NPrivate::Read(_readable, _version, ValidateOnly); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5033,10 +5025,10 @@ void TCreateTopicsRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _v NPrivate::Write(_collector, _writable, _version, Topics); NPrivate::Write(_collector, _writable, _version, TimeoutMs); NPrivate::Write(_collector, _writable, _version, ValidateOnly); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -5045,7 +5037,7 @@ i32 TCreateTopicsRequestData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, Topics); NPrivate::Size(_collector, _version, TimeoutMs); NPrivate::Size(_collector, _version, ValidateOnly); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5060,7 +5052,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) @@ -5075,7 +5067,7 @@ void TCreateTopicsRequestData::TCreatableTopic::Read(TKafkaReadable& _readable, NPrivate::Read(_readable, _version, ReplicationFactor); NPrivate::Read(_readable, _version, Assignments); NPrivate::Read(_readable, _version, Configs); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5100,10 +5092,10 @@ void TCreateTopicsRequestData::TCreatableTopic::Write(TKafkaWritable& _writable, NPrivate::Write(_collector, _writable, _version, ReplicationFactor); NPrivate::Write(_collector, _writable, _version, Assignments); NPrivate::Write(_collector, _writable, _version, Configs); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -5114,7 +5106,7 @@ i32 TCreateTopicsRequestData::TCreatableTopic::Size(TKafkaVersion _version) cons NPrivate::Size(_collector, _version, ReplicationFactor); NPrivate::Size(_collector, _version, Assignments); NPrivate::Size(_collector, _version, Configs); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5127,7 +5119,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) {} @@ -5137,7 +5129,7 @@ void TCreateTopicsRequestData::TCreatableTopic::TCreatableReplicaAssignment::Rea } NPrivate::Read(_readable, _version, PartitionIndex); NPrivate::Read(_readable, _version, BrokerIds); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5159,10 +5151,10 @@ void TCreateTopicsRequestData::TCreatableTopic::TCreatableReplicaAssignment::Wri NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, PartitionIndex); NPrivate::Write(_collector, _writable, _version, BrokerIds); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -5170,7 +5162,7 @@ i32 TCreateTopicsRequestData::TCreatableTopic::TCreatableReplicaAssignment::Size NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, PartitionIndex); NPrivate::Size(_collector, _version, BrokerIds); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5184,7 +5176,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) {} @@ -5195,7 +5187,7 @@ void TCreateTopicsRequestData::TCreatableTopic::TCreateableTopicConfig::Read(TKa } NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, Value); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5217,10 +5209,10 @@ void TCreateTopicsRequestData::TCreatableTopic::TCreateableTopicConfig::Write(TK NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, Value); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -5228,7 +5220,7 @@ i32 TCreateTopicsRequestData::TCreatableTopic::TCreateableTopicConfig::Size(TKaf NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, Value); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5241,7 +5233,7 @@ i32 TCreateTopicsRequestData::TCreatableTopic::TCreateableTopicConfig::Size(TKaf // const TCreateTopicsResponseData::ThrottleTimeMsMeta::Type TCreateTopicsResponseData::ThrottleTimeMsMeta::Default = 0; -TCreateTopicsResponseData::TCreateTopicsResponseData() +TCreateTopicsResponseData::TCreateTopicsResponseData() : ThrottleTimeMs(ThrottleTimeMsMeta::Default) {} @@ -5251,7 +5243,7 @@ void TCreateTopicsResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _v } NPrivate::Read(_readable, _version, ThrottleTimeMs); NPrivate::Read(_readable, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5273,10 +5265,10 @@ void TCreateTopicsResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _ NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, ThrottleTimeMs); NPrivate::Write(_collector, _writable, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -5284,7 +5276,7 @@ i32 TCreateTopicsResponseData::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, ThrottleTimeMs); NPrivate::Size(_collector, _version, Topics); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5303,7 +5295,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) @@ -5325,7 +5317,7 @@ void TCreateTopicsResponseData::TCreatableTopicResult::Read(TKafkaReadable& _rea NPrivate::Read(_readable, _version, NumPartitions); NPrivate::Read(_readable, _version, ReplicationFactor); NPrivate::Read(_readable, _version, Configs); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5356,10 +5348,10 @@ void TCreateTopicsResponseData::TCreatableTopicResult::Write(TKafkaWritable& _wr NPrivate::Write(_collector, _writable, _version, NumPartitions); NPrivate::Write(_collector, _writable, _version, ReplicationFactor); NPrivate::Write(_collector, _writable, _version, Configs); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + NPrivate::WriteTag(_writable, _version, TopicConfigErrorCode); } } @@ -5374,7 +5366,7 @@ i32 TCreateTopicsResponseData::TCreatableTopicResult::Size(TKafkaVersion _versio NPrivate::Size(_collector, _version, NumPartitions); NPrivate::Size(_collector, _version, ReplicationFactor); NPrivate::Size(_collector, _version, Configs); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5391,7 +5383,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) @@ -5408,7 +5400,7 @@ void TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicConfigs::R NPrivate::Read(_readable, _version, ReadOnly); NPrivate::Read(_readable, _version, ConfigSource); NPrivate::Read(_readable, _version, IsSensitive); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5433,10 +5425,10 @@ void TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicConfigs::W NPrivate::Write(_collector, _writable, _version, ReadOnly); NPrivate::Write(_collector, _writable, _version, ConfigSource); NPrivate::Write(_collector, _writable, _version, IsSensitive); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -5447,7 +5439,7 @@ i32 TCreateTopicsResponseData::TCreatableTopicResult::TCreatableTopicConfigs::Si NPrivate::Size(_collector, _version, ReadOnly); NPrivate::Size(_collector, _version, ConfigSource); NPrivate::Size(_collector, _version, IsSensitive); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5463,7 +5455,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) @@ -5478,7 +5470,7 @@ void TInitProducerIdRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _ NPrivate::Read(_readable, _version, TransactionTimeoutMs); NPrivate::Read(_readable, _version, ProducerId); NPrivate::Read(_readable, _version, ProducerEpoch); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5502,10 +5494,10 @@ void TInitProducerIdRequestData::Write(TKafkaWritable& _writable, TKafkaVersion NPrivate::Write(_collector, _writable, _version, TransactionTimeoutMs); NPrivate::Write(_collector, _writable, _version, ProducerId); NPrivate::Write(_collector, _writable, _version, ProducerEpoch); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -5515,7 +5507,7 @@ i32 TInitProducerIdRequestData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, TransactionTimeoutMs); NPrivate::Size(_collector, _version, ProducerId); NPrivate::Size(_collector, _version, ProducerEpoch); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5531,7 +5523,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) @@ -5546,7 +5538,7 @@ void TInitProducerIdResponseData::Read(TKafkaReadable& _readable, TKafkaVersion NPrivate::Read(_readable, _version, ErrorCode); NPrivate::Read(_readable, _version, ProducerId); NPrivate::Read(_readable, _version, ProducerEpoch); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5570,10 +5562,10 @@ void TInitProducerIdResponseData::Write(TKafkaWritable& _writable, TKafkaVersion NPrivate::Write(_collector, _writable, _version, ErrorCode); NPrivate::Write(_collector, _writable, _version, ProducerId); NPrivate::Write(_collector, _writable, _version, ProducerEpoch); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -5583,7 +5575,7 @@ i32 TInitProducerIdResponseData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, ErrorCode); NPrivate::Size(_collector, _version, ProducerId); NPrivate::Size(_collector, _version, ProducerEpoch); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5596,7 +5588,7 @@ i32 TInitProducerIdResponseData::Size(TKafkaVersion _version) const { // const TAlterConfigsRequestData::ValidateOnlyMeta::Type TAlterConfigsRequestData::ValidateOnlyMeta::Default = false; -TAlterConfigsRequestData::TAlterConfigsRequestData() +TAlterConfigsRequestData::TAlterConfigsRequestData() : ValidateOnly(ValidateOnlyMeta::Default) {} @@ -5606,7 +5598,7 @@ void TAlterConfigsRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _ve } NPrivate::Read(_readable, _version, Resources); NPrivate::Read(_readable, _version, ValidateOnly); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5628,10 +5620,10 @@ void TAlterConfigsRequestData::Write(TKafkaWritable& _writable, TKafkaVersion _v NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Resources); NPrivate::Write(_collector, _writable, _version, ValidateOnly); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -5639,7 +5631,7 @@ i32 TAlterConfigsRequestData::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Resources); NPrivate::Size(_collector, _version, ValidateOnly); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5653,7 +5645,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) {} @@ -5665,7 +5657,7 @@ void TAlterConfigsRequestData::TAlterConfigsResource::Read(TKafkaReadable& _read NPrivate::Read(_readable, _version, ResourceType); NPrivate::Read(_readable, _version, ResourceName); NPrivate::Read(_readable, _version, Configs); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5688,10 +5680,10 @@ void TAlterConfigsRequestData::TAlterConfigsResource::Write(TKafkaWritable& _wri NPrivate::Write(_collector, _writable, _version, ResourceType); NPrivate::Write(_collector, _writable, _version, ResourceName); NPrivate::Write(_collector, _writable, _version, Configs); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -5700,7 +5692,7 @@ i32 TAlterConfigsRequestData::TAlterConfigsResource::Size(TKafkaVersion _version NPrivate::Size(_collector, _version, ResourceType); NPrivate::Size(_collector, _version, ResourceName); NPrivate::Size(_collector, _version, Configs); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5714,7 +5706,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) {} @@ -5725,7 +5717,7 @@ void TAlterConfigsRequestData::TAlterConfigsResource::TAlterableConfig::Read(TKa } NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, Value); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5747,10 +5739,10 @@ void TAlterConfigsRequestData::TAlterConfigsResource::TAlterableConfig::Write(TK NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, Value); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -5758,7 +5750,7 @@ i32 TAlterConfigsRequestData::TAlterConfigsResource::TAlterableConfig::Size(TKaf NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, Value); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5771,7 +5763,7 @@ i32 TAlterConfigsRequestData::TAlterConfigsResource::TAlterableConfig::Size(TKaf // const TAlterConfigsResponseData::ThrottleTimeMsMeta::Type TAlterConfigsResponseData::ThrottleTimeMsMeta::Default = 0; -TAlterConfigsResponseData::TAlterConfigsResponseData() +TAlterConfigsResponseData::TAlterConfigsResponseData() : ThrottleTimeMs(ThrottleTimeMsMeta::Default) {} @@ -5781,7 +5773,7 @@ void TAlterConfigsResponseData::Read(TKafkaReadable& _readable, TKafkaVersion _v } NPrivate::Read(_readable, _version, ThrottleTimeMs); NPrivate::Read(_readable, _version, Responses); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5803,10 +5795,10 @@ void TAlterConfigsResponseData::Write(TKafkaWritable& _writable, TKafkaVersion _ NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, ThrottleTimeMs); NPrivate::Write(_collector, _writable, _version, Responses); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -5814,7 +5806,7 @@ i32 TAlterConfigsResponseData::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, ThrottleTimeMs); NPrivate::Size(_collector, _version, Responses); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5830,7 +5822,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) @@ -5845,7 +5837,7 @@ void TAlterConfigsResponseData::TAlterConfigsResourceResponse::Read(TKafkaReadab NPrivate::Read(_readable, _version, ErrorMessage); NPrivate::Read(_readable, _version, ResourceType); NPrivate::Read(_readable, _version, ResourceName); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5869,10 +5861,10 @@ void TAlterConfigsResponseData::TAlterConfigsResourceResponse::Write(TKafkaWrita NPrivate::Write(_collector, _writable, _version, ErrorMessage); NPrivate::Write(_collector, _writable, _version, ResourceType); NPrivate::Write(_collector, _writable, _version, ResourceName); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -5882,7 +5874,7 @@ i32 TAlterConfigsResponseData::TAlterConfigsResourceResponse::Size(TKafkaVersion NPrivate::Size(_collector, _version, ErrorMessage); NPrivate::Size(_collector, _version, ResourceType); NPrivate::Size(_collector, _version, ResourceName); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5894,7 +5886,7 @@ i32 TAlterConfigsResponseData::TAlterConfigsResourceResponse::Size(TKafkaVersion // TSaslAuthenticateRequestData // -TSaslAuthenticateRequestData::TSaslAuthenticateRequestData() +TSaslAuthenticateRequestData::TSaslAuthenticateRequestData() {} void TSaslAuthenticateRequestData::Read(TKafkaReadable& _readable, TKafkaVersion _version) { @@ -5902,7 +5894,7 @@ void TSaslAuthenticateRequestData::Read(TKafkaReadable& _readable, TKafkaVersion ythrow yexception() << "Can't read version " << _version << " of TSaslAuthenticateRequestData"; } NPrivate::Read(_readable, _version, AuthBytes); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5923,17 +5915,17 @@ void TSaslAuthenticateRequestData::Write(TKafkaWritable& _writable, TKafkaVersio } NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, AuthBytes); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } i32 TSaslAuthenticateRequestData::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, AuthBytes); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -5948,7 +5940,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) @@ -5962,7 +5954,7 @@ void TSaslAuthenticateResponseData::Read(TKafkaReadable& _readable, TKafkaVersio NPrivate::Read(_readable, _version, ErrorMessage); NPrivate::Read(_readable, _version, AuthBytes); NPrivate::Read(_readable, _version, SessionLifetimeMs); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -5986,10 +5978,10 @@ void TSaslAuthenticateResponseData::Write(TKafkaWritable& _writable, TKafkaVersi NPrivate::Write(_collector, _writable, _version, ErrorMessage); NPrivate::Write(_collector, _writable, _version, AuthBytes); NPrivate::Write(_collector, _writable, _version, SessionLifetimeMs); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -5999,7 +5991,7 @@ i32 TSaslAuthenticateResponseData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, ErrorMessage); NPrivate::Size(_collector, _version, AuthBytes); NPrivate::Size(_collector, _version, SessionLifetimeMs); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -6013,7 +6005,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) {} @@ -6025,7 +6017,7 @@ void TCreatePartitionsRequestData::Read(TKafkaReadable& _readable, TKafkaVersion NPrivate::Read(_readable, _version, Topics); NPrivate::Read(_readable, _version, TimeoutMs); NPrivate::Read(_readable, _version, ValidateOnly); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -6048,10 +6040,10 @@ void TCreatePartitionsRequestData::Write(TKafkaWritable& _writable, TKafkaVersio NPrivate::Write(_collector, _writable, _version, Topics); NPrivate::Write(_collector, _writable, _version, TimeoutMs); NPrivate::Write(_collector, _writable, _version, ValidateOnly); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -6060,7 +6052,7 @@ i32 TCreatePartitionsRequestData::Size(TKafkaVersion _version) const { NPrivate::Size(_collector, _version, Topics); NPrivate::Size(_collector, _version, TimeoutMs); NPrivate::Size(_collector, _version, ValidateOnly); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -6074,7 +6066,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) {} @@ -6086,7 +6078,7 @@ void TCreatePartitionsRequestData::TCreatePartitionsTopic::Read(TKafkaReadable& NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, Count); NPrivate::Read(_readable, _version, Assignments); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -6109,10 +6101,10 @@ void TCreatePartitionsRequestData::TCreatePartitionsTopic::Write(TKafkaWritable& NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, Count); NPrivate::Write(_collector, _writable, _version, Assignments); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -6121,7 +6113,7 @@ i32 TCreatePartitionsRequestData::TCreatePartitionsTopic::Size(TKafkaVersion _ve NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, Count); NPrivate::Size(_collector, _version, Assignments); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -6133,7 +6125,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) { @@ -6141,7 +6133,7 @@ void TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsAssi ythrow yexception() << "Can't read version " << _version << " of TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsAssignment"; } NPrivate::Read(_readable, _version, BrokerIds); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -6162,17 +6154,17 @@ void TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsAssi } NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, BrokerIds); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } i32 TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsAssignment::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, BrokerIds); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -6185,7 +6177,7 @@ i32 TCreatePartitionsRequestData::TCreatePartitionsTopic::TCreatePartitionsAssig // const TCreatePartitionsResponseData::ThrottleTimeMsMeta::Type TCreatePartitionsResponseData::ThrottleTimeMsMeta::Default = 0; -TCreatePartitionsResponseData::TCreatePartitionsResponseData() +TCreatePartitionsResponseData::TCreatePartitionsResponseData() : ThrottleTimeMs(ThrottleTimeMsMeta::Default) {} @@ -6195,7 +6187,7 @@ void TCreatePartitionsResponseData::Read(TKafkaReadable& _readable, TKafkaVersio } NPrivate::Read(_readable, _version, ThrottleTimeMs); NPrivate::Read(_readable, _version, Results); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -6217,10 +6209,10 @@ void TCreatePartitionsResponseData::Write(TKafkaWritable& _writable, TKafkaVersi NPrivate::TWriteCollector _collector; NPrivate::Write(_collector, _writable, _version, ThrottleTimeMs); NPrivate::Write(_collector, _writable, _version, Results); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -6228,7 +6220,7 @@ i32 TCreatePartitionsResponseData::Size(TKafkaVersion _version) const { NPrivate::TSizeCollector _collector; NPrivate::Size(_collector, _version, ThrottleTimeMs); NPrivate::Size(_collector, _version, Results); - + if (NPrivate::VersionCheck(_version)) { _collector.Size += NPrivate::SizeOfUnsignedVarint(_collector.NumTaggedFields); } @@ -6243,7 +6235,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) @@ -6256,7 +6248,7 @@ void TCreatePartitionsResponseData::TCreatePartitionsTopicResult::Read(TKafkaRea NPrivate::Read(_readable, _version, Name); NPrivate::Read(_readable, _version, ErrorCode); NPrivate::Read(_readable, _version, ErrorMessage); - + if (NPrivate::VersionCheck(_version)) { ui32 _numTaggedFields = _readable.readUnsignedVarint(); for (ui32 _i = 0; _i < _numTaggedFields; ++_i) { @@ -6279,10 +6271,10 @@ void TCreatePartitionsResponseData::TCreatePartitionsTopicResult::Write(TKafkaWr NPrivate::Write(_collector, _writable, _version, Name); NPrivate::Write(_collector, _writable, _version, ErrorCode); NPrivate::Write(_collector, _writable, _version, ErrorMessage); - + if (NPrivate::VersionCheck(_version)) { _writable.writeUnsignedVarint(_collector.NumTaggedFields); - + } } @@ -6291,7 +6283,7 @@ i32 TCreatePartitionsResponseData::TCreatePartitionsTopicResult::Size(TKafkaVers NPrivate::Size(_collector, _version, Name); NPrivate::Size(_collector, _version, ErrorCode); NPrivate::Size(_collector, _version, ErrorMessage); - + if (NPrivate::VersionCheck(_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 84dd5625a3b8..72f55d30fc01 100644 --- a/ydb/core/kafka_proxy/kafka_messages.h +++ b/ydb/core/kafka_proxy/kafka_messages.h @@ -14,25 +14,25 @@ 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] - 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] + 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] + 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 EApiKeyNames; @@ -43,80 +43,80 @@ extern const std::unordered_map EApiKeyNames; class TRequestHeaderData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 2}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; 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; }; @@ -124,35 +124,35 @@ class TRequestHeaderData : public TApiMessage { class TResponseHeaderData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 1}; static constexpr TKafkaVersions FlexibleVersions = {1, Max()}; }; - + 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()}; }; 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; }; @@ -160,175 +160,175 @@ class TResponseHeaderData : public TApiMessage { class TProduceRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 9}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; - + TProduceRequestData(); ~TProduceRequestData() = default; - + class TTopicProduceData : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 9}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; - + TTopicProduceData(); ~TTopicProduceData() = default; - + class TPartitionProduceData : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 9}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; NameMeta::Type Name; - + struct PartitionDataMeta { using ItemType = TPartitionProduceData; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsAlways; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; 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()}; }; 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()}; }; TimeoutMsMeta::Type TimeoutMs; - + struct TopicDataMeta { using ItemType = TTopicProduceData; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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; }; @@ -336,269 +336,269 @@ class TProduceRequestData : public TApiMessage { class TProduceResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 9}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; - + TProduceResponseData(); ~TProduceResponseData() = default; - + class TTopicProduceResponse : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 9}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; - + TTopicProduceResponse(); ~TTopicProduceResponse() = default; - + class TPartitionProduceResponse : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 9}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; - + TPartitionProduceResponse(); ~TPartitionProduceResponse() = default; - + class TBatchIndexAndErrorMessage : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {8, 9}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; LogStartOffsetMeta::Type LogStartOffset; - + struct RecordErrorsMeta { using ItemType = TBatchIndexAndErrorMessage; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsAlways; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; 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()}; }; NameMeta::Type Name; - + struct PartitionResponsesMeta { using ItemType = TPartitionProduceResponse; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; 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; }; @@ -606,421 +606,421 @@ class TProduceResponseData : public TApiMessage { class TFetchRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 13}; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; - + TFetchRequestData(); ~TFetchRequestData() = default; - + class TFetchTopic : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 13}; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; - + TFetchTopic(); ~TFetchTopic() = default; - + class TFetchPartition : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 13}; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; - + 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; 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()}; }; 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()}; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; 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()}; }; 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()}; }; 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()}; 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; 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()}; }; 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()}; }; - + 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()}; }; 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()}; 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; 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()}; }; 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()}; 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()}; }; 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; SessionEpochMeta::Type SessionEpoch; - + struct TopicsMeta { using ItemType = TFetchTopic; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; TopicsMeta::Type Topics; - + struct ForgottenTopicsDataMeta { using ItemType = TForgottenTopic; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; 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; }; @@ -1028,514 +1028,514 @@ class TFetchRequestData : public TApiMessage { class TFetchResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 13}; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; - + TFetchResponseData(); ~TFetchResponseData() = default; - + class TFetchableTopicResponse : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 13}; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; - + TFetchableTopicResponse(); ~TFetchableTopicResponse() = default; - + class TPartitionData : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 13}; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; 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()}; 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()}; 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()}; 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; using TypeDesc = NPrivate::TKafkaArrayDesc; - + static constexpr const char* Name = "abortedTransactions"; static constexpr const char* About = "The aborted transactions."; - + static constexpr TKafkaVersions PresentVersions = {4, Max()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsAlways; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; 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()}; }; 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()}; }; 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()}; 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; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {12, Max()}; }; SessionIdMeta::Type SessionId; - + struct ResponsesMeta { using ItemType = TFetchableTopicResponse; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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; }; @@ -1543,191 +1543,191 @@ class TFetchResponseData : public TApiMessage { class TListOffsetsRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + TListOffsetsRequestData(); ~TListOffsetsRequestData() = default; - + class TListOffsetsTopic : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + TListOffsetsTopic(); ~TListOffsetsTopic() = default; - + class TListOffsetsPartition : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; 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()}; }; 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()}; }; 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()}; }; NameMeta::Type Name; - + struct PartitionsMeta { using ItemType = TListOffsetsPartition; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; IsolationLevelMeta::Type IsolationLevel; - + struct TopicsMeta { using ItemType = TListOffsetsTopic; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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; }; @@ -1735,207 +1735,207 @@ class TListOffsetsRequestData : public TApiMessage { class TListOffsetsResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + TListOffsetsResponseData(); ~TListOffsetsResponseData() = default; - + class TListOffsetsTopicResponse : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + TListOffsetsTopicResponse(); ~TListOffsetsTopicResponse() = default; - + class TListOffsetsPartitionResponse : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + 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()}; }; 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()}; }; ErrorCodeMeta::Type ErrorCode; - + struct OldStyleOffsetsMeta { using ItemType = TKafkaInt64; using ItemTypeDesc = NPrivate::TKafkaIntDesc; using Type = std::vector; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; 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()}; }; NameMeta::Type Name; - + struct PartitionsMeta { using ItemType = TListOffsetsPartitionResponse; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; ThrottleTimeMsMeta::Type ThrottleTimeMs; - + struct TopicsMeta { using ItemType = TListOffsetsTopicResponse; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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; }; @@ -1943,128 +1943,128 @@ class TListOffsetsResponseData : public TApiMessage { class TMetadataRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 12}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; - + TMetadataRequestData(); ~TMetadataRequestData() = default; - + class TMetadataRequestTopic : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 12}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; - + 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()}; 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()}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; 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; 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()}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; 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; }; @@ -2072,422 +2072,422 @@ class TMetadataRequestData : public TApiMessage { class TMetadataResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 12}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; - + TMetadataResponseData(); ~TMetadataResponseData() = default; - + class TMetadataResponseBroker : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 12}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsAlways; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; 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()}; }; - + TMetadataResponseTopic(); ~TMetadataResponseTopic() = default; - + class TMetadataResponsePartition : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 12}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; LeaderEpochMeta::Type LeaderEpoch; - + struct ReplicaNodesMeta { using ItemType = TKafkaInt32; using ItemTypeDesc = NPrivate::TKafkaIntDesc; using Type = std::vector; 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()}; }; ReplicaNodesMeta::Type ReplicaNodes; - + struct IsrNodesMeta { using ItemType = TKafkaInt32; using ItemTypeDesc = NPrivate::TKafkaIntDesc; using Type = std::vector; 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()}; }; IsrNodesMeta::Type IsrNodes; - + struct OfflineReplicasMeta { using ItemType = TKafkaInt32; using ItemTypeDesc = NPrivate::TKafkaIntDesc; using Type = std::vector; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; 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()}; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; IsInternalMeta::Type IsInternal; - + struct PartitionsMeta { using ItemType = TMetadataResponsePartition; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; ThrottleTimeMsMeta::Type ThrottleTimeMs; - + struct BrokersMeta { using ItemType = TMetadataResponseBroker; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsAlways; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {9, Max()}; }; ControllerIdMeta::Type ControllerId; - + struct TopicsMeta { using ItemType = TMetadataResponseTopic; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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()}; }; 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; }; @@ -2495,251 +2495,251 @@ class TMetadataResponseData : public TApiMessage { class TOffsetCommitRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 8}; static constexpr TKafkaVersions FlexibleVersions = {8, Max()}; }; - + TOffsetCommitRequestData(); ~TOffsetCommitRequestData() = default; - + class TOffsetCommitRequestTopic : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 8}; static constexpr TKafkaVersions FlexibleVersions = {8, Max()}; }; - + TOffsetCommitRequestTopic(); ~TOffsetCommitRequestTopic() = default; - + class TOffsetCommitRequestPartition : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 8}; static constexpr TKafkaVersions FlexibleVersions = {8, Max()}; }; - + 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {8, Max()}; }; 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()}; }; 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()}; }; 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()}; }; NameMeta::Type Name; - + struct PartitionsMeta { using ItemType = TOffsetCommitRequestPartition; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {8, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {8, Max()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsAlways; static constexpr TKafkaVersions FlexibleVersions = {8, Max()}; }; 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()}; }; RetentionTimeMsMeta::Type RetentionTimeMs; - + struct TopicsMeta { using ItemType = TOffsetCommitRequestTopic; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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; }; @@ -2747,146 +2747,146 @@ class TOffsetCommitRequestData : public TApiMessage { class TOffsetCommitResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 8}; static constexpr TKafkaVersions FlexibleVersions = {8, Max()}; }; - + TOffsetCommitResponseData(); ~TOffsetCommitResponseData() = default; - + class TOffsetCommitResponseTopic : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 8}; static constexpr TKafkaVersions FlexibleVersions = {8, Max()}; }; - + TOffsetCommitResponseTopic(); ~TOffsetCommitResponseTopic() = default; - + class TOffsetCommitResponsePartition : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 8}; static constexpr TKafkaVersions FlexibleVersions = {8, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; NameMeta::Type Name; - + struct PartitionsMeta { using ItemType = TOffsetCommitResponsePartition; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {8, Max()}; }; ThrottleTimeMsMeta::Type ThrottleTimeMs; - + struct TopicsMeta { using ItemType = TOffsetCommitResponseTopic; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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; }; @@ -2894,226 +2894,226 @@ class TOffsetCommitResponseData : public TApiMessage { class TOffsetFetchRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 8}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + TOffsetFetchRequestData(); ~TOffsetFetchRequestData() = default; - + class TOffsetFetchRequestTopic : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + 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()}; }; NameMeta::Type Name; - + struct PartitionIndexesMeta { using ItemType = TKafkaInt32; using ItemTypeDesc = NPrivate::TKafkaIntDesc; using Type = std::vector; 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()}; }; 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; 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; 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()}; }; GroupIdMeta::Type GroupId; - + struct TopicsMeta { using ItemType = TOffsetFetchRequestTopic; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; TopicsMeta::Type Topics; - + struct GroupsMeta { using ItemType = TOffsetFetchRequestGroup; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; 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()}; 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; }; @@ -3121,425 +3121,425 @@ class TOffsetFetchRequestData : public TApiMessage { class TOffsetFetchResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 8}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + TOffsetFetchResponseData(); ~TOffsetFetchResponseData() = default; - + class TOffsetFetchResponseTopic : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + TOffsetFetchResponseTopic(); ~TOffsetFetchResponseTopic() = default; - + class TOffsetFetchResponsePartition : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; 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()}; }; 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()}; }; 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()}; }; NameMeta::Type Name; - + struct PartitionsMeta { using ItemType = TOffsetFetchResponsePartition; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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; 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; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; ThrottleTimeMsMeta::Type ThrottleTimeMs; - + struct TopicsMeta { using ItemType = TOffsetFetchResponseTopic; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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()}; }; ErrorCodeMeta::Type ErrorCode; - + struct GroupsMeta { using ItemType = TOffsetFetchResponseGroup; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; 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; }; @@ -3547,66 +3547,66 @@ class TOffsetFetchResponseData : public TApiMessage { class TFindCoordinatorRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 4}; static constexpr TKafkaVersions FlexibleVersions = {3, Max()}; }; - + 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {3, Max()}; }; KeyTypeMeta::Type KeyType; - + struct CoordinatorKeysMeta { using ItemType = TKafkaString; using ItemTypeDesc = NPrivate::TKafkaStringDesc; using Type = std::vector; using TypeDesc = NPrivate::TKafkaArrayDesc; - + static constexpr const char* Name = "coordinatorKeys"; static constexpr const char* About = "The coordinator keys."; - + static constexpr TKafkaVersions PresentVersions = {4, Max()}; 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; }; @@ -3614,233 +3614,233 @@ class TFindCoordinatorRequestData : public TApiMessage { class TFindCoordinatorResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 4}; static constexpr TKafkaVersions FlexibleVersions = {3, Max()}; }; - + 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {3, Max()}; }; 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()}; }; 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()}; }; 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()}; }; 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()}; }; 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()}; }; PortMeta::Type Port; - + struct CoordinatorsMeta { using ItemType = TCoordinator; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; 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; }; @@ -3848,187 +3848,187 @@ class TFindCoordinatorResponseData : public TApiMessage { class TJoinGroupRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 9}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + TJoinGroupRequestData(); ~TJoinGroupRequestData() = default; - + class TJoinGroupRequestProtocol : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 9}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsAlways; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; 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()}; }; ProtocolTypeMeta::Type ProtocolType; - + struct ProtocolsMeta { using ItemType = TJoinGroupRequestProtocol; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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()}; 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; }; @@ -4036,217 +4036,218 @@ class TJoinGroupRequestData : public TApiMessage { class TJoinGroupResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 9}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + TJoinGroupResponseData(); ~TJoinGroupResponseData() = default; - + class TJoinGroupResponseMember : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 9}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; - + 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsAlways; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; 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()}; }; 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()}; }; 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()}; 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()}; static constexpr TKafkaVersions FlexibleVersions = {6, Max()}; }; 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()}; }; 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()}; 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()}; }; MemberIdMeta::Type MemberId; - + struct MembersMeta { using ItemType = TJoinGroupResponseMember; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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; }; @@ -4254,80 +4255,80 @@ class TJoinGroupResponseData : public TApiMessage { class THeartbeatRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 4}; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsAlways; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; 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; }; @@ -4335,50 +4336,50 @@ class THeartbeatRequestData : public TApiMessage { class THeartbeatResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 4}; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; - + 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; 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()}; }; 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; }; @@ -4386,128 +4387,128 @@ class THeartbeatResponseData : public TApiMessage { class TLeaveGroupRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 5}; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; - + TLeaveGroupRequestData(); ~TLeaveGroupRequestData() = default; - + class TMemberIdentity : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {3, 5}; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; - + 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()}; }; 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()}; }; 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()}; 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()}; }; 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()}; }; MemberIdMeta::Type MemberId; - + struct MembersMeta { using ItemType = TMemberIdentity; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; 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; }; @@ -4515,128 +4516,128 @@ class TLeaveGroupRequestData : public TApiMessage { class TLeaveGroupResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 5}; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; - + TLeaveGroupResponseData(); ~TLeaveGroupResponseData() = default; - + class TMemberResponse : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {3, 5}; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; 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()}; }; ErrorCodeMeta::Type ErrorCode; - + struct MembersMeta { using ItemType = TMemberResponse; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; 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; }; @@ -4644,172 +4645,173 @@ class TLeaveGroupResponseData : public TApiMessage { class TSyncGroupRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 5}; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; - + TSyncGroupRequestData(); ~TSyncGroupRequestData() = default; - + class TSyncGroupRequestAssignment : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 5}; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsAlways; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; 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()}; 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()}; 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; 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()}; }; 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; }; @@ -4817,94 +4819,96 @@ class TSyncGroupRequestData : public TApiMessage { class TSyncGroupResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 5}; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; - + 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; 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()}; }; 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()}; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsAlways; static constexpr TKafkaVersions FlexibleVersions = VersionsAlways; }; ProtocolNameMeta::Type ProtocolName; - + struct AssignmentMeta { - using Type = TConsumerProtocolAssignment; - using TypeDesc = NPrivate::TKafkaStructDesc; - + 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 = VersionsNever; + static constexpr TKafkaVersions FlexibleVersions = {4, Max()}; }; 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; }; @@ -4912,35 +4916,35 @@ class TSyncGroupResponseData : public TApiMessage { class TSaslHandshakeRequestData : public TApiMessage { public: typedef std::shared_ptr 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; }; @@ -4948,51 +4952,51 @@ class TSaslHandshakeRequestData : public TApiMessage { class TSaslHandshakeResponseData : public TApiMessage { public: typedef std::shared_ptr 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; 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; }; @@ -5000,50 +5004,50 @@ class TSaslHandshakeResponseData : public TApiMessage { class TApiVersionsRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 3}; static constexpr TKafkaVersions FlexibleVersions = {3, Max()}; }; - + 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()}; 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()}; 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; }; @@ -5051,318 +5055,318 @@ class TApiVersionsRequestData : public TApiMessage { class TApiVersionsResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 3}; static constexpr TKafkaVersions FlexibleVersions = {3, Max()}; }; - + TApiVersionsResponseData(); ~TApiVersionsResponseData() = default; - + class TApiVersion : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 3}; static constexpr TKafkaVersions FlexibleVersions = {3, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; }; ErrorCodeMeta::Type ErrorCode; - + struct ApiKeysMeta { using ItemType = TApiVersion; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {3, Max()}; }; ThrottleTimeMsMeta::Type ThrottleTimeMs; - + struct SupportedFeaturesMeta { using ItemType = TSupportedFeatureKey; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; 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()}; 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; 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()}; 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()}; 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; }; @@ -5370,255 +5374,255 @@ class TApiVersionsResponseData : public TApiMessage { class TCreateTopicsRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = {5, Max()}; }; - + TCreateTopicsRequestData(); ~TCreateTopicsRequestData() = default; - + class TCreatableTopic : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = {5, Max()}; }; - + TCreatableTopic(); ~TCreatableTopic() = default; - + class TCreatableReplicaAssignment : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = {5, Max()}; }; - + 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()}; }; PartitionIndexMeta::Type PartitionIndex; - + struct BrokerIdsMeta { using ItemType = TKafkaInt32; using ItemTypeDesc = NPrivate::TKafkaIntDesc; using Type = std::vector; 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()}; }; 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()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; }; 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()}; }; ReplicationFactorMeta::Type ReplicationFactor; - + struct AssignmentsMeta { using ItemType = TCreatableReplicaAssignment; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; AssignmentsMeta::Type Assignments; - + struct ConfigsMeta { using ItemType = TCreateableTopicConfig; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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; 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {5, Max()}; }; 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; }; @@ -5626,282 +5630,282 @@ class TCreateTopicsRequestData : public TApiMessage { class TCreateTopicsResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = {5, Max()}; }; - + TCreateTopicsResponseData(); ~TCreateTopicsResponseData() = default; - + class TCreatableTopicResult : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 7}; static constexpr TKafkaVersions FlexibleVersions = {5, Max()}; }; - + 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()}; }; 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()}; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsAlways; static constexpr TKafkaVersions FlexibleVersions = {5, Max()}; }; 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()}; 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()}; 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()}; 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; 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()}; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {5, Max()}; }; ThrottleTimeMsMeta::Type ThrottleTimeMs; - + struct TopicsMeta { using ItemType = TCreatableTopicResult; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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; }; @@ -5909,80 +5913,80 @@ class TCreateTopicsResponseData : public TApiMessage { class TInitProducerIdRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 4}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + 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()}; }; 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()}; }; 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()}; 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()}; 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; }; @@ -5990,80 +5994,80 @@ class TInitProducerIdRequestData : public TApiMessage { class TInitProducerIdResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 4}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; }; 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; }; @@ -6071,161 +6075,161 @@ class TInitProducerIdResponseData : public TApiMessage { class TAlterConfigsRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 2}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + TAlterConfigsRequestData(); ~TAlterConfigsRequestData() = default; - + class TAlterConfigsResource : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 2}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + TAlterConfigsResource(); ~TAlterConfigsResource() = default; - + class TAlterableConfig : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 2}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; }; ResourceNameMeta::Type ResourceName; - + struct ConfigsMeta { using ItemType = TAlterableConfig; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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; 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()}; }; 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()}; }; 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; }; @@ -6233,128 +6237,128 @@ class TAlterConfigsRequestData : public TApiMessage { class TAlterConfigsResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 2}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + TAlterConfigsResponseData(); ~TAlterConfigsResponseData() = default; - + class TAlterConfigsResourceResponse : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 2}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; }; 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()}; }; ThrottleTimeMsMeta::Type ThrottleTimeMs; - + struct ResponsesMeta { using ItemType = TAlterConfigsResourceResponse; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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; }; @@ -6362,34 +6366,34 @@ class TAlterConfigsResponseData : public TApiMessage { class TSaslAuthenticateRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 2}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + 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()}; }; 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; }; @@ -6397,79 +6401,79 @@ class TSaslAuthenticateRequestData : public TApiMessage { class TSaslAuthenticateResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 2}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; static constexpr TKafkaVersions TaggedVersions = VersionsNever; static constexpr TKafkaVersions NullableVersions = VersionsNever; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; 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; }; @@ -6477,162 +6481,162 @@ class TSaslAuthenticateResponseData : public TApiMessage { class TCreatePartitionsRequestData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 3}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + TCreatePartitionsRequestData(); ~TCreatePartitionsRequestData() = default; - + class TCreatePartitionsTopic : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 3}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + TCreatePartitionsTopic(); ~TCreatePartitionsTopic() = default; - + class TCreatePartitionsAssignment : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 3}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + TCreatePartitionsAssignment(); ~TCreatePartitionsAssignment() = default; - + struct BrokerIdsMeta { using ItemType = TKafkaInt32; using ItemTypeDesc = NPrivate::TKafkaIntDesc; using Type = std::vector; 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()}; }; 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()}; }; 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()}; }; CountMeta::Type Count; - + struct AssignmentsMeta { using ItemType = TCreatePartitionsAssignment; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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; 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()}; }; 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()}; }; 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()}; }; 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; }; @@ -6640,114 +6644,114 @@ class TCreatePartitionsRequestData : public TApiMessage { class TCreatePartitionsResponseData : public TApiMessage { public: typedef std::shared_ptr TPtr; - + struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 3}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + TCreatePartitionsResponseData(); ~TCreatePartitionsResponseData() = default; - + class TCreatePartitionsTopicResult : public TMessage { public: struct MessageMeta { static constexpr TKafkaVersions PresentVersions = {0, 3}; static constexpr TKafkaVersions FlexibleVersions = {2, Max()}; }; - + 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()}; }; 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()}; }; 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()}; }; 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()}; }; ThrottleTimeMsMeta::Type ThrottleTimeMs; - + struct ResultsMeta { using ItemType = TCreatePartitionsTopicResult; using ItemTypeDesc = NPrivate::TKafkaStructDesc; using Type = std::vector; 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()}; }; 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/ut_protocol.cpp b/ydb/core/kafka_proxy/ut/ut_protocol.cpp index ed9556688ea0..8593b2e72c57 100644 --- a/ydb/core/kafka_proxy/ut/ut_protocol.cpp +++ b/ydb/core/kafka_proxy/ut/ut_protocol.cpp @@ -19,8 +19,10 @@ #include #include +#include #include +#include using namespace NKafka; using namespace NYdb; @@ -33,6 +35,8 @@ static constexpr const char NON_CHARGEABLE_USER_Y[] = "superuser_y@builtin"; static constexpr const char DEFAULT_CLOUD_ID[] = "somecloud"; static constexpr const char DEFAULT_FOLDER_ID[] = "somefolder"; +static constexpr TKafkaUint16 ASSIGNMENT_VERSION = 3; + static constexpr const ui64 FirstTopicOffset = -2; static constexpr const ui64 LastTopicOffset = -1; @@ -71,7 +75,7 @@ class TTestServer { public: TIpPort Port; - TTestServer(const TString& kafkaApiMode = "1", bool serverless = false) { + TTestServer(const TString& kafkaApiMode = "1", bool serverless = false, bool enableNativeKafkaBalancing = false) { TPortManager portManager; Port = portManager.GetTcpPort(); @@ -107,6 +111,10 @@ class TTestServer { appConfig.MutableKafkaProxyConfig()->MutableProxy()->SetPort(FAKE_SERVERLESS_KAFKA_PROXY_PORT); } + if (enableNativeKafkaBalancing) { + appConfig.MutableKafkaProxyConfig()->SetEnableNativeBalancing(true); + } + appConfig.MutablePQConfig()->MutableQuotingConfig()->SetEnableQuoting(true); appConfig.MutablePQConfig()->MutableQuotingConfig()->SetQuotaWaitDurationMs(300); appConfig.MutablePQConfig()->MutableQuotingConfig()->SetPartitionReadQuotaIsTwiceWriteQuota(true); @@ -215,6 +223,103 @@ class TSecureTestServer : public TTestServer& topics) { + TKafkaVersion version = *(TKafkaVersion*)(metadata.value().data() + sizeof(TKafkaVersion)); + + TBuffer buffer(metadata.value().data() + sizeof(TKafkaVersion), metadata.value().size_bytes() - sizeof(TKafkaVersion)); + TKafkaReadable readable(buffer); + + TConsumerProtocolSubscription result; + result.Read(readable, version); + + for (auto topic: result.Topics) { + if (topic.has_value()) { + topics.emplace(topic.value()); + } + } +} + +std::vector MakeRangeAssignment( + TMessagePtr& joinResponse, + int totalPartitionsCount) +{ + + std::vector assignments; + + std::unordered_map> memberToTopics; + + for (auto& member : joinResponse->Members) { + THashSet memberTopics; + FillTopicsFromJoinGroupMetadata(member.Metadata, memberTopics); + memberToTopics[member.MemberId.value()] = std::move(memberTopics); + } + + THashSet allTopics; + for (auto& kv : memberToTopics) { + for (auto& t : kv.second) { + allTopics.insert(t); + } + } + + std::unordered_map> topicToMembers; + for (auto& t : allTopics) { + for (auto& [mId, topicsSet] : memberToTopics) { + if (topicsSet.contains(t)) { + topicToMembers[t].push_back(mId); + } + } + } + + for (const auto& member : joinResponse->Members) { + TConsumerProtocolAssignment consumerAssignment; + + const auto& requestedTopics = memberToTopics[member.MemberId.value()]; + for (auto& topicName : requestedTopics) { + + auto& interestedMembers = topicToMembers[topicName]; + auto it = std::find(interestedMembers.begin(), interestedMembers.end(), member.MemberId); + if (it == interestedMembers.end()) { + continue; + } + + int idx = static_cast(std::distance(interestedMembers.begin(), it)); + int totalInterested = static_cast(interestedMembers.size()); + + const int totalPartitions = totalPartitionsCount; + + int baseCount = totalPartitions / totalInterested; + int remainder = totalPartitions % totalInterested; + + int start = idx * baseCount + std::min(idx, remainder); + int length = baseCount + (idx < remainder ? 1 : 0); + + + TConsumerProtocolAssignment::TopicPartition topicPartition; + topicPartition.Topic = topicName; + for (int p = start; p < start + length; ++p) { + topicPartition.Partitions.push_back(p); + } + consumerAssignment.AssignedPartitions.push_back(topicPartition); + } + + { + TWritableBuf buf(nullptr, consumerAssignment.Size(ASSIGNMENT_VERSION) + sizeof(ASSIGNMENT_VERSION)); + TKafkaWritable writable(buf); + + writable << ASSIGNMENT_VERSION; + consumerAssignment.Write(writable, ASSIGNMENT_VERSION); + NKafka::TSyncGroupRequestData::TSyncGroupRequestAssignment syncAssignment; + syncAssignment.MemberId = member.MemberId; + syncAssignment.AssignmentStr = TString(buf.GetBuffer().data(), buf.GetBuffer().size()); + syncAssignment.Assignment = syncAssignment.AssignmentStr; + + assignments.push_back(std::move(syncAssignment)); + } + } + + return assignments; +} + void Write(TSocketOutput& so, TApiMessage* request, TKafkaVersion version) { TWritableBuf sb(nullptr, request->Size(version) + 1000); TKafkaWritable writable(sb); @@ -339,6 +444,18 @@ void CreateTopic(NYdb::NTopic::TTopicClient& pqClient, TString& topicName, ui32 UNIT_ASSERT_VALUES_EQUAL(result.IsTransportError(), false); UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + +} + +TConsumerProtocolAssignment GetAssignments(NKafka::TSyncGroupResponseData::AssignmentMeta::Type metadata) { + TKafkaVersion version = *(TKafkaVersion*)(metadata.value().data() + sizeof(TKafkaVersion)); + TBuffer buffer(metadata.value().data() + sizeof(TKafkaVersion), metadata.value().size_bytes() - sizeof(TKafkaVersion)); + TKafkaReadable readable(buffer); + + TConsumerProtocolAssignment result; + result.Read(readable, version); + + return result; } struct TTopicConfig { @@ -376,6 +493,16 @@ class TTestClient { , ClientName(clientName) { } + template T> + void WriteToSocket(TRequestHeaderData& header, T& request) { + Write(So, &header, &request); + } + + template T> + TMessagePtr ReadResponse(TRequestHeaderData& header) { + return ::Read(Si, &header); + } + TMessagePtr ApiVersions() { Cerr << ">>>>> ApiVersionsRequest\n"; @@ -507,7 +634,7 @@ class TTestClient { return WriteAndRead(header, request); } - TMessagePtr JoinGroup(std::vector& topics, TString& groupId, i32 heartbeatTimeout = 1000000) { + TMessagePtr JoinGroup(std::vector& topics, TString& groupId, TString protocolName, i32 heartbeatTimeout = 1000000) { Cerr << ">>>>> TJoinGroupRequestData\n"; TRequestHeaderData header = Header(NKafka::EApiKey::JOIN_GROUP, 9); @@ -518,7 +645,7 @@ class TTestClient { request.SessionTimeoutMs = heartbeatTimeout; NKafka::TJoinGroupRequestData::TJoinGroupRequestProtocol protocol; - protocol.Name = "roundrobin"; + protocol.Name = protocolName; TConsumerProtocolSubscription subscribtion; @@ -539,7 +666,7 @@ class TTestClient { return WriteAndRead(header, request); } - TMessagePtr SyncGroup(TString& memberId, ui64 generationId, TString& groupId) { + TMessagePtr SyncGroup(TString& memberId, ui64 generationId, TString& groupId, std::vector assignments, TString& protocolName) { Cerr << ">>>>> TSyncGroupRequestData\n"; TRequestHeaderData header = Header(NKafka::EApiKey::SYNC_GROUP, 5); @@ -547,30 +674,36 @@ class TTestClient { TSyncGroupRequestData request; request.GroupId = groupId; request.ProtocolType = "consumer"; - request.ProtocolName = "roundrobin"; + request.ProtocolName = protocolName; request.GenerationId = generationId; request.GroupId = groupId; request.MemberId = memberId; + request.Assignments = assignments; + return WriteAndRead(header, request); } - TReadInfo JoinAndSyncGroup(std::vector& topics, TString& groupId, i32 heartbeatTimeout = 1000000) { - auto joinResponse = JoinGroup(topics, groupId, heartbeatTimeout); + TReadInfo JoinAndSyncGroup(std::vector& topics, TString& groupId, TString& protocolName, i32 heartbeatTimeout = 1000000, ui32 totalPartitionsCount = 0) { + auto joinResponse = JoinGroup(topics, groupId, protocolName, heartbeatTimeout); auto memberId = joinResponse->MemberId; - auto generationId = joinResponse->GenerationId; - auto balanceStrategy = joinResponse->ProtocolName; + auto generationId = joinResponse->GenerationId; + auto balanceStrategy = joinResponse->ProtocolName; UNIT_ASSERT_VALUES_EQUAL(joinResponse->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); + const bool isLeader = (joinResponse->Leader == memberId); + std::vector assignments; + if (isLeader) { + assignments = MakeRangeAssignment(joinResponse, totalPartitionsCount); + } - auto syncResponse = SyncGroup(memberId.value(), generationId, groupId); + auto syncResponse = SyncGroup(memberId.value(), generationId, groupId, assignments, protocolName); UNIT_ASSERT_VALUES_EQUAL(syncResponse->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); TReadInfo readInfo; readInfo.GenerationId = generationId; readInfo.MemberId = memberId.value(); - readInfo.Partitions = syncResponse->Assignment.AssignedPartitions; - + readInfo.Partitions = GetAssignments(syncResponse->Assignment).AssignedPartitions; return readInfo; } @@ -596,10 +729,10 @@ class TTestClient { UNIT_ASSERT_VALUES_EQUAL(heartbeatStatus, static_cast(EKafkaErrors::REBALANCE_IN_PROGRESS)); } - TReadInfo JoinAndSyncGroupAndWaitPartitions(std::vector& topics, TString& groupId, ui32 expectedPartitionsCount) { + TReadInfo JoinAndSyncGroupAndWaitPartitions(std::vector& topics, TString& groupId, ui32 expectedPartitionsCount, TString& protocolName, ui32 totalPartitionsCount = 0) { TReadInfo readInfo; for (;;) { - readInfo = JoinAndSyncGroup(topics, groupId); + readInfo = JoinAndSyncGroup(topics, groupId, protocolName, 1000000, totalPartitionsCount); ui32 partitionsCount = 0; for (auto topicPartitions: readInfo.Partitions) { partitionsCount += topicPartitions.Partitions.size(); @@ -616,7 +749,7 @@ class TTestClient { TMessagePtr LeaveGroup(TString& memberId, TString& groupId) { Cerr << ">>>>> TLeaveGroupRequestData\n"; - TRequestHeaderData header = Header(NKafka::EApiKey::LEAVE_GROUP, 5); + TRequestHeaderData header = Header(NKafka::EApiKey::LEAVE_GROUP, 2); TLeaveGroupRequestData request; request.GroupId = groupId; @@ -807,6 +940,16 @@ class TTestClient { } } + + TRequestHeaderData Header(NKafka::EApiKey apiKey, TKafkaVersion version) { + TRequestHeaderData header; + header.RequestApiKey = apiKey; + header.RequestApiVersion = version; + header.CorrelationId = NextCorrelation(); + header.ClientId = ClientName; + return header; + } + protected: ui32 NextCorrelation() { return Correlation++; @@ -818,15 +961,6 @@ class TTestClient { return Read(Si, &header); } - TRequestHeaderData Header(NKafka::EApiKey apiKey, TKafkaVersion version) { - TRequestHeaderData header; - header.RequestApiKey = apiKey; - header.RequestApiVersion = version; - header.CorrelationId = NextCorrelation(); - header.ClientId = ClientName; - return header; - } - private: TNetworkAddress Addr; TSocket Socket; @@ -1303,6 +1437,8 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) { } // Y_UNIT_TEST(FetchScenario) Y_UNIT_TEST(BalanceScenario) { + + TString protocolName = "roundrobin"; TInsecureTestServer testServer("2"); TString topicName = "/Root/topic-0-test"; @@ -1352,25 +1488,25 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) { topics.push_back(topicName); // clientA join group, and get all partitions - auto readInfoA = clientA.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions); + auto readInfoA = clientA.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions, protocolName, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(clientA.Heartbeat(readInfoA.MemberId, readInfoA.GenerationId, group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); // clientB join group, and get 0 partitions, becouse it's all at clientA UNIT_ASSERT_VALUES_EQUAL(clientB.SaslHandshake()->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); UNIT_ASSERT_VALUES_EQUAL(clientB.SaslAuthenticate("ouruser@/Root", "ourUserPassword")->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); - auto readInfoB = clientB.JoinAndSyncGroup(topics, group); + auto readInfoB = clientB.JoinAndSyncGroup(topics, group, protocolName, 1000000, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(readInfoB.Partitions.size(), 0); // clientA gets RABALANCE status, because of new reader. We need to release some partitions for new client clientA.WaitRebalance(readInfoA.MemberId, readInfoA.GenerationId, group); // clientA now gets half of partitions - readInfoA = clientA.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/2); + readInfoA = clientA.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/2, protocolName, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(clientA.Heartbeat(readInfoA.MemberId, readInfoA.GenerationId, group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); // some partitions now released, and we can give them to clientB. clientB now gets half of partitions clientB.WaitRebalance(readInfoB.MemberId, readInfoB.GenerationId, group); - readInfoB = clientB.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/2); + readInfoB = clientB.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/2, protocolName, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(clientB.Heartbeat(readInfoB.MemberId, readInfoB.GenerationId, group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); AssertPartitionsIsUniqueAndCountIsExpected({readInfoA, readInfoB}, minActivePartitions, topicName); @@ -1378,7 +1514,7 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) { // clientC join group, and get 0 partitions, becouse it's all at clientA and clientB UNIT_ASSERT_VALUES_EQUAL(clientC.SaslHandshake()->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); UNIT_ASSERT_VALUES_EQUAL(clientC.SaslAuthenticate("ouruser@/Root", "ourUserPassword")->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); - auto readInfoC = clientC.JoinAndSyncGroup(topics, group); + auto readInfoC = clientC.JoinAndSyncGroup(topics, group, protocolName, 1000000, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(readInfoC.Partitions.size(), 0); // all clients gets RABALANCE status, because of new reader. We need to release some partitions for new client @@ -1386,13 +1522,13 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) { clientB.WaitRebalance(readInfoB.MemberId, readInfoB.GenerationId, group); // all clients now gets minActivePartitions/3 partitions - readInfoA = clientA.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/3); + readInfoA = clientA.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/3, protocolName, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(clientA.Heartbeat(readInfoA.MemberId, readInfoA.GenerationId, group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); - readInfoB = clientB.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/3); + readInfoB = clientB.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/3, protocolName, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(clientB.Heartbeat(readInfoB.MemberId, readInfoB.GenerationId, group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); - readInfoC = clientC.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/3); + readInfoC = clientC.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/3, protocolName, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(clientC.Heartbeat(readInfoC.MemberId, readInfoC.GenerationId, group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); AssertPartitionsIsUniqueAndCountIsExpected({readInfoA, readInfoB, readInfoC}, minActivePartitions, topicName); @@ -1400,7 +1536,7 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) { // clientD join group, and get 0 partitions, becouse it's all at clientA, clientB and clientC UNIT_ASSERT_VALUES_EQUAL(clientD.SaslHandshake()->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); UNIT_ASSERT_VALUES_EQUAL(clientD.SaslAuthenticate("ouruser@/Root", "ourUserPassword")->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); - auto readInfoD = clientD.JoinAndSyncGroup(topics, group); + auto readInfoD = clientD.JoinAndSyncGroup(topics, group, protocolName, 1000000, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(readInfoD.Partitions.size(), 0); // all clients gets RABALANCE status, because of new reader. We need to release some partitions @@ -1409,16 +1545,16 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) { clientC.WaitRebalance(readInfoC.MemberId, readInfoC.GenerationId, group); // all clients now gets minActivePartitions/4 partitions - readInfoA = clientA.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/4); + readInfoA = clientA.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/4, protocolName); UNIT_ASSERT_VALUES_EQUAL(clientA.Heartbeat(readInfoA.MemberId, readInfoA.GenerationId, group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); - readInfoB = clientB.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/4); + readInfoB = clientB.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/4, protocolName, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(clientB.Heartbeat(readInfoB.MemberId, readInfoB.GenerationId, group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); - readInfoC = clientC.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/4); + readInfoC = clientC.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/4, protocolName, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(clientC.Heartbeat(readInfoC.MemberId, readInfoC.GenerationId, group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); - readInfoD = clientD.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/4); + readInfoD = clientD.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/4, protocolName, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(clientD.Heartbeat(readInfoD.MemberId, readInfoD.GenerationId, group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); AssertPartitionsIsUniqueAndCountIsExpected({readInfoA, readInfoB, readInfoC, readInfoD}, minActivePartitions, topicName); @@ -1433,13 +1569,13 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) { clientD.WaitRebalance(readInfoD.MemberId, readInfoD.GenerationId, group); // all other clients now gets minActivePartitions/3 partitions - readInfoB = clientB.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/3); + readInfoB = clientB.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/3, protocolName, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(clientB.Heartbeat(readInfoB.MemberId, readInfoB.GenerationId, group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); - readInfoC = clientC.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/3); + readInfoC = clientC.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/3, protocolName, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(clientC.Heartbeat(readInfoC.MemberId, readInfoC.GenerationId, group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); - readInfoD = clientD.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/3); + readInfoD = clientD.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions/3, protocolName, minActivePartitions); UNIT_ASSERT_VALUES_EQUAL(clientD.Heartbeat(readInfoD.MemberId, readInfoD.GenerationId, group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); AssertPartitionsIsUniqueAndCountIsExpected({readInfoB, readInfoC, readInfoD}, minActivePartitions, topicName); @@ -1456,9 +1592,9 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) { std::vector topics; topics.push_back(topicName); - auto readInfoA = clientA.JoinGroup(topics, group); + auto readInfoA = clientA.JoinGroup(topics, group, protocolName); Sleep(TDuration::MilliSeconds(200)); - auto readInfoB = clientB.JoinGroup(topics, group); + auto readInfoB = clientB.JoinGroup(topics, group, protocolName); Sleep(TDuration::MilliSeconds(200)); UNIT_ASSERT_VALUES_EQUAL(clientA.LeaveGroup(readInfoA->MemberId.value(), group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); @@ -1470,7 +1606,7 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) { std::vector topics; topics.push_back(shortTopicName); - auto joinResponse = clientA.JoinGroup(topics, group); + auto joinResponse = clientA.JoinGroup(topics, group, protocolName); UNIT_ASSERT_VALUES_EQUAL(joinResponse->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); UNIT_ASSERT_VALUES_EQUAL(clientA.LeaveGroup(joinResponse->MemberId.value(), group)->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); } @@ -1480,7 +1616,7 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) { std::vector topics; topics.push_back(topicName); - auto joinResponse = clientA.JoinGroup(topics, notExistsGroup); + auto joinResponse = clientA.JoinGroup(topics, notExistsGroup, protocolName); UNIT_ASSERT_VALUES_EQUAL(joinResponse->ErrorCode, static_cast(EKafkaErrors::GROUP_ID_NOT_FOUND)); } @@ -1489,7 +1625,7 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) { std::vector topics; topics.push_back(notExistsTopicName); - auto joinResponse = clientA.JoinGroup(topics, group); + auto joinResponse = clientA.JoinGroup(topics, group, protocolName); UNIT_ASSERT_VALUES_EQUAL(joinResponse->ErrorCode, static_cast(EKafkaErrors::UNKNOWN_TOPIC_OR_PARTITION)); } @@ -1499,7 +1635,7 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) { topics.push_back(topicName); topics.push_back(secondTopicName); - auto readInfo = clientA.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions * 2); + auto readInfo = clientA.JoinAndSyncGroupAndWaitPartitions(topics, group, minActivePartitions * 2, protocolName, minActivePartitions); std::unordered_set topicsSet; for (auto partition: readInfo.Partitions) { @@ -1510,7 +1646,7 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) { // Check change topics list topics.pop_back(); - auto joinResponse = clientA.JoinGroup(topics, group); + auto joinResponse = clientA.JoinGroup(topics, group, protocolName); UNIT_ASSERT_VALUES_EQUAL(joinResponse->ErrorCode, static_cast(EKafkaErrors::REBALANCE_IN_PROGRESS)); // tell client to rejoin } @@ -2318,4 +2454,324 @@ Y_UNIT_TEST_SUITE(KafkaProtocol) { UNIT_ASSERT_VALUES_EQUAL(metadataResponse->Brokers[0].Host, "localhost"); UNIT_ASSERT_VALUES_EQUAL(metadataResponse->Brokers[0].Port, FAKE_SERVERLESS_KAFKA_PROXY_PORT); } + + Y_UNIT_TEST(NativeKafkaBalanceScenario) { + TInsecureTestServer testServer("1", false, true); + + TString topicName = "/Root/topic-0"; + ui64 totalPartitions = 24; + TString groupId = "consumer-0"; + + TString protocolType = "consumer"; + TString protocolName = "range"; + + { + NYdb::NTopic::TTopicClient pqClient(*testServer.Driver); + auto result = pqClient + .CreateTopic( + topicName, + NYdb::NTopic::TCreateTopicSettings() + .PartitioningSettings(totalPartitions, 100) + .BeginAddConsumer(groupId).EndAddConsumer() + ) + .ExtractValueSync(); + UNIT_ASSERT_C( + result.IsSuccess(), + "CreateTopic failed, issues: " << result.GetIssues().ToString() + ); + } + + TTestClient clientA(testServer.Port, "ClientA"); + TTestClient clientB(testServer.Port, "ClientB"); + TTestClient clientC(testServer.Port, "ClientC"); + + { + auto rA = clientA.ApiVersions(); + auto rB = clientB.ApiVersions(); + auto rC = clientC.ApiVersions(); + UNIT_ASSERT_VALUES_EQUAL(rA->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); + UNIT_ASSERT_VALUES_EQUAL(rB->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); + UNIT_ASSERT_VALUES_EQUAL(rC->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); + } + { + auto rA = clientA.SaslHandshake("PLAIN"); + auto rB = clientB.SaslHandshake("PLAIN"); + auto rC = clientC.SaslHandshake("PLAIN"); + UNIT_ASSERT_VALUES_EQUAL(rA->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); + UNIT_ASSERT_VALUES_EQUAL(rB->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); + UNIT_ASSERT_VALUES_EQUAL(rC->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); + } + { + TString user = "ouruser@/Root"; + TString pass = "ourUserPassword"; + auto rA = clientA.SaslAuthenticate(user, pass); + auto rB = clientB.SaslAuthenticate(user, pass); + auto rC = clientC.SaslAuthenticate(user, pass); + UNIT_ASSERT_VALUES_EQUAL(rA->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); + UNIT_ASSERT_VALUES_EQUAL(rB->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); + UNIT_ASSERT_VALUES_EQUAL(rC->ErrorCode, static_cast(EKafkaErrors::NONE_ERROR)); + } + + std::vector topics = {topicName}; + i32 heartbeatMs = 1000000; // savnik + + // CHECK THREE READERS GETS 1/3 OF PARTITIONS + + 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 joinReq; + joinReq.GroupId = groupId; + joinReq.ProtocolType = protocolType; + joinReq.SessionTimeoutMs = heartbeatMs; + + 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()); + + joinReq.Protocols.push_back(protocol); + + TJoinGroupRequestData joinReqA = joinReq; + TJoinGroupRequestData joinReqB = joinReq; + TJoinGroupRequestData joinReqC = joinReq; + + clientA.WriteToSocket(headerAJoin, joinReqA); + clientB.WriteToSocket(headerBJoin, joinReqB); + clientC.WriteToSocket(headerCJoin, joinReqC); + + auto joinRespA = clientA.ReadResponse(headerAJoin); + auto joinRespB = clientB.ReadResponse(headerBJoin); + auto joinRespC = clientC.ReadResponse(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); + + bool isLeaderA = (joinRespA->Leader == joinRespA->MemberId); + bool isLeaderB = (joinRespB->Leader == joinRespB->MemberId); + + TMessagePtr leaderResp = isLeaderA ? joinRespA + : isLeaderB ? joinRespB + : joinRespC; + + std::vector assignments = MakeRangeAssignment(leaderResp, totalPartitions); + + TRequestHeaderData syncHeaderA = clientA.Header(NKafka::EApiKey::SYNC_GROUP, 5); + TRequestHeaderData syncHeaderB = clientB.Header(NKafka::EApiKey::SYNC_GROUP, 5); + TRequestHeaderData syncHeaderC = clientC.Header(NKafka::EApiKey::SYNC_GROUP, 5); + + TSyncGroupRequestData syncReqA; + syncReqA.GroupId = groupId; + syncReqA.ProtocolType = protocolType; + syncReqA.ProtocolName = protocolName; + syncReqA.GenerationId = joinRespA->GenerationId; + syncReqA.MemberId = joinRespA->MemberId.value(); + + TSyncGroupRequestData syncReqB = syncReqA; + syncReqB.GenerationId = joinRespB->GenerationId; + syncReqB.MemberId = joinRespB->MemberId.value(); + + TSyncGroupRequestData syncReqC = syncReqA; + syncReqC.GenerationId = joinRespC->GenerationId; + syncReqC.MemberId = joinRespC->MemberId.value(); + + if (isLeaderA) { + syncReqA.Assignments = assignments; + } else if (isLeaderB) { + syncReqB.Assignments = assignments; + } else { + syncReqC.Assignments = assignments; + } + + clientA.WriteToSocket(syncHeaderA, syncReqA); + clientB.WriteToSocket(syncHeaderB, syncReqB); + clientC.WriteToSocket(syncHeaderC, syncReqC); + + auto syncRespA = clientA.ReadResponse(syncHeaderA); + auto syncRespB = clientB.ReadResponse(syncHeaderB); + auto syncRespC = clientC.ReadResponse(syncHeaderC); + + UNIT_ASSERT_VALUES_EQUAL(syncRespA->ErrorCode, (TKafkaInt16)EKafkaErrors::NONE_ERROR); + UNIT_ASSERT_VALUES_EQUAL(syncRespB->ErrorCode, (TKafkaInt16)EKafkaErrors::NONE_ERROR); + UNIT_ASSERT_VALUES_EQUAL(syncRespC->ErrorCode, (TKafkaInt16)EKafkaErrors::NONE_ERROR); + + auto countPartitions = [](const TConsumerProtocolAssignment& assignment) { + size_t sum = 0; + for (auto& ta : assignment.AssignedPartitions) { + sum += ta.Partitions.size(); + } + return sum; + }; + + size_t countA = countPartitions(GetAssignments(syncRespA->Assignment)); + size_t countB = countPartitions(GetAssignments(syncRespB->Assignment)); + size_t countC = countPartitions(GetAssignments(syncRespC->Assignment)); + + UNIT_ASSERT_VALUES_EQUAL(countA, size_t(totalPartitions / 3)); + UNIT_ASSERT_VALUES_EQUAL(countB, size_t(totalPartitions / 3)); + UNIT_ASSERT_VALUES_EQUAL(countC, size_t(totalPartitions / 3)); + + UNIT_ASSERT_VALUES_EQUAL( + clientA.Heartbeat(joinRespA->MemberId.value(), joinRespA->GenerationId, groupId)->ErrorCode, + static_cast(EKafkaErrors::NONE_ERROR) + ); + UNIT_ASSERT_VALUES_EQUAL( + clientB.Heartbeat(joinRespB->MemberId.value(), joinRespB->GenerationId, groupId)->ErrorCode, + static_cast(EKafkaErrors::NONE_ERROR) + ); + UNIT_ASSERT_VALUES_EQUAL( + clientC.Heartbeat(joinRespC->MemberId.value(), joinRespC->GenerationId, groupId)->ErrorCode, + static_cast(EKafkaErrors::NONE_ERROR) + ); + + // CHECK ONE CLIENT LEAVE, AND OTHERS GETS 1/2 OF PARTITIONS + + UNIT_ASSERT_VALUES_EQUAL( + clientC.LeaveGroup(joinRespC->MemberId.value(), groupId)->ErrorCode, + static_cast(EKafkaErrors::NONE_ERROR) + ); + + clientA.WaitRebalance(joinRespA->MemberId.value(), joinRespA->GenerationId, groupId); + clientB.WaitRebalance(joinRespB->MemberId.value(), joinRespB->GenerationId, groupId); + + TRequestHeaderData headerAJoin2 = clientA.Header(NKafka::EApiKey::JOIN_GROUP, 9); + TRequestHeaderData headerBJoin2 = clientB.Header(NKafka::EApiKey::JOIN_GROUP, 9); + + TJoinGroupRequestData joinReqA2 = joinReq; + TJoinGroupRequestData joinReqB2 = joinReq; + + clientA.WriteToSocket(headerAJoin2, joinReqA2); + clientB.WriteToSocket(headerBJoin2, joinReqB2); + + auto joinRespA2 = clientA.ReadResponse(headerAJoin2); + auto joinRespB2 = clientB.ReadResponse(headerBJoin2); + + UNIT_ASSERT_VALUES_EQUAL(joinRespA2->ErrorCode, (TKafkaInt16)EKafkaErrors::NONE_ERROR); + UNIT_ASSERT_VALUES_EQUAL(joinRespB2->ErrorCode, (TKafkaInt16)EKafkaErrors::NONE_ERROR); + + bool isLeaderA2 = (joinRespA2->Leader == joinRespA2->MemberId); + + TMessagePtr leaderResp2 = isLeaderA2 ? joinRespA2 : joinRespB2; + + std::vector assignments2 = MakeRangeAssignment(leaderResp2, totalPartitions); + + TRequestHeaderData syncHeaderA2 = clientA.Header(NKafka::EApiKey::SYNC_GROUP, 5); + TRequestHeaderData syncHeaderB2 = clientB.Header(NKafka::EApiKey::SYNC_GROUP, 5); + + TSyncGroupRequestData syncReqA2; + syncReqA2.GroupId = groupId; + syncReqA2.ProtocolType = protocolType; + syncReqA2.ProtocolName = protocolName; + syncReqA2.GenerationId = joinRespA2->GenerationId; + syncReqA2.MemberId = joinRespA2->MemberId.value(); + + TSyncGroupRequestData syncReqB2 = syncReqA2; + syncReqB2.GenerationId = joinRespB2->GenerationId; + syncReqB2.MemberId = joinRespB2->MemberId.value(); + + if (isLeaderA2) { + syncReqA2.Assignments = assignments2; + } else { + syncReqB2.Assignments = assignments2; + } + + clientA.WriteToSocket(syncHeaderA2, syncReqA2); + clientB.WriteToSocket(syncHeaderB2, syncReqB2); + + auto syncRespA2 = clientA.ReadResponse(syncHeaderA2); + auto syncRespB2 = clientB.ReadResponse(syncHeaderB2); + + UNIT_ASSERT_VALUES_EQUAL(syncRespA2->ErrorCode, (TKafkaInt16)EKafkaErrors::NONE_ERROR); + UNIT_ASSERT_VALUES_EQUAL(syncRespB2->ErrorCode, (TKafkaInt16)EKafkaErrors::NONE_ERROR); + + size_t countA2 = countPartitions(GetAssignments(syncRespA2->Assignment)); + size_t countB2 = countPartitions(GetAssignments(syncRespB2->Assignment)); + + UNIT_ASSERT_VALUES_EQUAL(countA2, size_t(totalPartitions / 2)); + UNIT_ASSERT_VALUES_EQUAL(countB2, size_t(totalPartitions / 2)); + + UNIT_ASSERT_VALUES_EQUAL( + clientA.Heartbeat(joinRespA2->MemberId.value(), joinRespA2->GenerationId, groupId)->ErrorCode, + static_cast(EKafkaErrors::NONE_ERROR) + ); + + UNIT_ASSERT_VALUES_EQUAL( + clientB.Heartbeat(joinRespB2->MemberId.value(), joinRespB2->GenerationId, groupId)->ErrorCode, + static_cast(EKafkaErrors::NONE_ERROR) + ); + + // CHECK ONE READER DEAD (NO HEARTBEAT) + + Sleep(TDuration::Seconds(10)); + + UNIT_ASSERT_VALUES_EQUAL( + clientA.Heartbeat(joinRespA2->MemberId.value(), joinRespA2->GenerationId, groupId)->ErrorCode, + static_cast(EKafkaErrors::NONE_ERROR) + ); + + Sleep(TDuration::Seconds(25)); + + UNIT_ASSERT_VALUES_EQUAL( + clientA.Heartbeat(joinRespA2->MemberId.value(), joinRespA2->GenerationId, groupId)->ErrorCode, + static_cast(EKafkaErrors::REBALANCE_IN_PROGRESS) + ); + + // LAST READER GETS ALL PARTITIONS + clientA.JoinAndSyncGroupAndWaitPartitions(topics, groupId, totalPartitions, protocolName, totalPartitions); + + + // CHECK IF MASTER DIE AFTER JOIN + + TRequestHeaderData headerAJoin3 = clientA.Header(NKafka::EApiKey::JOIN_GROUP, 9); + TRequestHeaderData headerBJoin3 = clientB.Header(NKafka::EApiKey::JOIN_GROUP, 9); + + TJoinGroupRequestData joinReqA3 = joinReq; + TJoinGroupRequestData joinReqB3 = joinReq; + + clientA.WriteToSocket(headerAJoin2, joinReqA2); + clientB.WriteToSocket(headerBJoin2, joinReqB2); + + auto joinRespA3 = clientA.ReadResponse(headerAJoin2); + auto joinRespB3 = clientB.ReadResponse(headerBJoin2); + + UNIT_ASSERT_VALUES_EQUAL(joinRespA2->ErrorCode, (TKafkaInt16)EKafkaErrors::NONE_ERROR); + UNIT_ASSERT_VALUES_EQUAL(joinRespB2->ErrorCode, (TKafkaInt16)EKafkaErrors::NONE_ERROR); + + bool isLeaderA3 = (joinRespA3->Leader == joinRespA3->MemberId); + + TSyncGroupRequestData syncReqNotMaster; + syncReqNotMaster.GroupId = groupId; + syncReqNotMaster.ProtocolType = protocolType; + syncReqNotMaster.ProtocolName = protocolName; + + TRequestHeaderData syncHeaderNotMaster; + if (isLeaderA3) { + syncReqNotMaster.GenerationId = joinRespB3->GenerationId; + syncReqNotMaster.MemberId = joinRespB3->MemberId.value(); + syncHeaderNotMaster = clientB.Header(NKafka::EApiKey::SYNC_GROUP, 5); + clientB.WriteToSocket(syncHeaderNotMaster, syncReqNotMaster); + auto noMasterSyncResponse = clientB.ReadResponse(syncHeaderNotMaster); + UNIT_ASSERT_VALUES_EQUAL(noMasterSyncResponse->ErrorCode, (TKafkaInt16)EKafkaErrors::REBALANCE_IN_PROGRESS); + } else { + syncReqNotMaster.GenerationId = joinRespA3->GenerationId; + syncReqNotMaster.MemberId = joinRespA3->MemberId.value(); + syncHeaderNotMaster = clientA.Header(NKafka::EApiKey::SYNC_GROUP, 5); + clientA.WriteToSocket(syncHeaderNotMaster, syncReqNotMaster); + auto noMasterSyncResponse = clientA.ReadResponse(syncHeaderNotMaster); + UNIT_ASSERT_VALUES_EQUAL(noMasterSyncResponse->ErrorCode, (TKafkaInt16)EKafkaErrors::REBALANCE_IN_PROGRESS); + } + + } + } // Y_UNIT_TEST_SUITE(KafkaProtocol) diff --git a/ydb/core/kafka_proxy/ya.make b/ydb/core/kafka_proxy/ya.make index c9c47570e5e4..a53646be3662 100644 --- a/ydb/core/kafka_proxy/ya.make +++ b/ydb/core/kafka_proxy/ya.make @@ -18,6 +18,11 @@ SRCS( actors/kafka_create_topics_actor.cpp actors/kafka_create_partitions_actor.cpp actors/kafka_alter_configs_actor.cpp + actors/kafka_consumer_groups_metadata_initializers.cpp + actors/kafka_consumer_members_metadata_initializers.cpp + actors/kqp_balance_transaction.cpp + actors/kafka_balance_actor_sql.cpp + actors/kafka_balancer_actor.cpp kafka_connection.cpp kafka_connection.h kafka_constants.h diff --git a/ydb/core/protos/config.proto b/ydb/core/protos/config.proto index 3be6ff04988c..bbd2939f8ea7 100644 --- a/ydb/core/protos/config.proto +++ b/ydb/core/protos/config.proto @@ -2126,6 +2126,7 @@ message TKafkaProxyConfig { optional TProxy Proxy = 7; optional bool MeteringV2Enabled = 10 [default = false]; + optional bool EnableNativeBalancing = 11 [default = false]; } message TAwsCompatibilityConfig { diff --git a/ydb/core/protos/kafka.proto b/ydb/core/protos/kafka.proto new file mode 100644 index 000000000000..6e62747ad79b --- /dev/null +++ b/ydb/core/protos/kafka.proto @@ -0,0 +1,12 @@ +syntax = "proto3"; + +package NKafka; + +message TWorkerState { + repeated TProtocol protocols = 1; + + message TProtocol { + string protocol_name = 1; + bytes metadata = 2; + } +} diff --git a/ydb/core/protos/ya.make b/ydb/core/protos/ya.make index d2f4bec42af2..1b24b50b3af5 100644 --- a/ydb/core/protos/ya.make +++ b/ydb/core/protos/ya.make @@ -85,6 +85,7 @@ SRCS( import.proto index_builder.proto kesus.proto + kafka.proto key.proto kqp.proto kqp_physical.proto diff --git a/ydb/core/raw_socket/sock_impl.h b/ydb/core/raw_socket/sock_impl.h index 7f7df797f78f..825f0d9950c8 100644 --- a/ydb/core/raw_socket/sock_impl.h +++ b/ydb/core/raw_socket/sock_impl.h @@ -130,7 +130,7 @@ class TBufferedWriter { if (possible > 0) { Buffer.Append(src, possible); } - if (0 == Buffer.Avail()) { + if (0 == Buffer.Avail() && Socket) { flush(); } size_t left = length - possible; @@ -198,7 +198,7 @@ class TBufferedWriter { private: TSocketDescriptor* Socket; TBuffer Buffer; - size_t BufferSize; + size_t BufferSize; struct Chunk { Chunk(TBuffer&& buffer)