Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Kafka protocol offset commit, request units and fixes #831

Merged
merged 8 commits into from
Jan 10, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 5 additions & 1 deletion ydb/core/kafka_proxy/actors/actors.h
Original file line number Diff line number Diff line change
Expand Up @@ -27,12 +27,13 @@ struct TContext {
const NKikimrConfig::TKafkaProxyConfig& Config;

TActorId ConnectionId;
TString ClientId;
TString KafkaClient;


EAuthSteps AuthenticationStep = EAuthSteps::WAIT_HANDSHAKE;
TString SaslMechanism;

TString GroupId;
TString DatabasePath;
TString FolderId;
TString CloudId;
Expand Down Expand Up @@ -128,6 +129,9 @@ inline EKafkaErrors ConvertErrorCode(Ydb::PersQueue::ErrorCode::ErrorCode code)
return EKafkaErrors::UNKNOWN_TOPIC_OR_PARTITION;
case Ydb::PersQueue::ErrorCode::ErrorCode::ACCESS_DENIED:
return EKafkaErrors::TOPIC_AUTHORIZATION_FAILED;
case Ydb::PersQueue::ErrorCode::ErrorCode::SET_OFFSET_ERROR_COMMIT_TO_FUTURE:
case Ydb::PersQueue::ErrorCode::ErrorCode::SET_OFFSET_ERROR_COMMIT_TO_PAST:
return EKafkaErrors::OFFSET_OUT_OF_RANGE;
default:
return EKafkaErrors::UNKNOWN_SERVER_ERROR;
}
Expand Down
2 changes: 1 addition & 1 deletion ydb/core/kafka_proxy/actors/kafka_api_versions_actor.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ TApiVersionsResponseData::TPtr GetApiVersions() {
AddApiKey<TLeaveGroupRequestData>(response->ApiKeys, LEAVE_GROUP);
AddApiKey<THeartbeatRequestData>(response->ApiKeys, HEARTBEAT);
AddApiKey<TFindCoordinatorRequestData>(response->ApiKeys, FIND_COORDINATOR);
AddApiKey<TOffsetCommitRequestData>(response->ApiKeys, OFFSET_COMMIT);
AddApiKey<TOffsetCommitRequestData>(response->ApiKeys, OFFSET_COMMIT, {.MaxVersion=1});
AddApiKey<TOffsetFetchRequestData>(response->ApiKeys, OFFSET_FETCH);

return response;
Expand Down
18 changes: 10 additions & 8 deletions ydb/core/kafka_proxy/actors/kafka_fetch_actor.cpp
Original file line number Diff line number Diff line change
@@ -1,21 +1,22 @@
#include <ydb/library/actors/core/actor.h>
#include <ydb/core/kafka_proxy/kafka_events.h>
#include "ydb/core/kafka_proxy/kafka_metrics.h"
#include <ydb/core/base/ticket_parser.h>
#include "ydb/core/kafka_proxy/kafka_metrics.h"
#include <ydb/core/persqueue/fetch_request_actor.h>
#include <ydb/core/tx/scheme_cache/scheme_cache.h>
#include <ydb/core/persqueue/events/internal.h>
#include <ydb/core/persqueue/user_info.h>
#include <ydb/core/persqueue/write_meta.h>
#include <ydb/core/tx/scheme_cache/scheme_cache.h>
#include <ydb/public/api/grpc/ydb_auth_v1.grpc.pb.h>

#include "kafka_fetch_actor.h"
#include "actors.h"
#include "kafka_fetch_actor.h"


namespace NKafka {

static constexpr size_t SizeOfZeroVarint = 1;
static constexpr size_t BatchFirstTwoFildsSize = 12;
static constexpr size_t BatchFirstTwoFieldsSize = 12;
static constexpr size_t KafkaMagic = 2;

NActors::IActor* CreateKafkaFetchActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr<TFetchRequestData>& message) {
Expand All @@ -34,7 +35,7 @@ void TKafkaFetchActor::SendFetchRequests(const TActorContext& ctx) {
TVector<NKikimr::NPQ::TPartitionFetchRequest> partPQRequests;
PrepareFetchRequestData(topicIndex, partPQRequests);

NKikimr::NPQ::TFetchRequestSettings request(Context->DatabasePath, partPQRequests, FetchRequestData->MaxWaitMs, FetchRequestData->MaxBytes, *Context->UserToken);
NKikimr::NPQ::TFetchRequestSettings request(Context->DatabasePath, partPQRequests, FetchRequestData->MaxWaitMs, FetchRequestData->MaxBytes, Context->RlContext, *Context->UserToken);

auto fetchActor = NKikimr::NPQ::CreatePQFetchRequestActor(request, NKikimr::MakeSchemeCacheID(), ctx.SelfID);
auto actorId = ctx.Register(fetchActor);
Expand All @@ -60,6 +61,7 @@ void TKafkaFetchActor::PrepareFetchRequestData(const size_t topicIndex, TVector<
partPQRequest.Partition = partKafkaRequest.Partition;
partPQRequest.Offset = partKafkaRequest.FetchOffset;
partPQRequest.MaxBytes = partKafkaRequest.PartitionMaxBytes;
partPQRequest.ClientId = Context->GroupId.Empty() ? NKikimr::NPQ::CLIENTID_WITHOUT_CONSUMER : Context->GroupId;
}
}

Expand All @@ -86,7 +88,7 @@ size_t TKafkaFetchActor::CheckTopicIndex(const NKikimr::TEvPQ::TEvFetchResponse:
Y_DEBUG_ABORT_UNLESS(topicIt != TopicIndexes.end());

if (topicIt == TopicIndexes.end()) {
KAFKA_LOG_CRIT("Fetch actor: Received unexpected TEvFetchResponse. Ignoring. Expect malformed/incompled fetch reply.");
KAFKA_LOG_ERROR("Fetch actor: Received unexpected TEvFetchResponse. Ignoring. Expect malformed/incompled fetch reply.");
return std::numeric_limits<size_t>::max();
}

Expand Down Expand Up @@ -172,7 +174,7 @@ void TKafkaFetchActor::FillRecordsBatch(const NKikimrClient::TPersQueueFetchResp
record.TimestampDelta = lastTimestamp - baseTimestamp;

record.Length = record.Size(TKafkaRecord::MessageMeta::PresentVersions.Max) - SizeOfZeroVarint;
KAFKA_LOG_D("Fetch actor: Record info. Value: " << record.DataChunk.GetData() << ", OffsetDelta: " << record.OffsetDelta <<
KAFKA_LOG_D("Fetch actor: Record info. OffsetDelta: " << record.OffsetDelta <<
", TimestampDelta: " << record.TimestampDelta << ", Length: " << record.Length);
}

Expand All @@ -184,7 +186,7 @@ void TKafkaFetchActor::FillRecordsBatch(const NKikimrClient::TPersQueueFetchResp
recordsBatch.BaseSequence = baseSequense;
//recordsBatch.Attributes https://kafka.apache.org/documentation/#recordbatch

recordsBatch.BatchLength = recordsBatch.Size(TKafkaRecordBatch::MessageMeta::PresentVersions.Max) - BatchFirstTwoFildsSize;
recordsBatch.BatchLength = recordsBatch.Size(TKafkaRecordBatch::MessageMeta::PresentVersions.Max) - BatchFirstTwoFieldsSize;
KAFKA_LOG_D("Fetch actor: RecordBatch info. BaseOffset: " << recordsBatch.BaseOffset << ", LastOffsetDelta: " << recordsBatch.LastOffsetDelta <<
", BaseTimestamp: " << recordsBatch.BaseTimestamp << ", MaxTimestamp: " << recordsBatch.MaxTimestamp <<
", BaseSequence: " << recordsBatch.BaseSequence << ", BatchLength: " << recordsBatch.BatchLength);
Expand Down
188 changes: 177 additions & 11 deletions ydb/core/kafka_proxy/actors/kafka_offset_commit_actor.cpp
Original file line number Diff line number Diff line change
@@ -1,38 +1,204 @@
#include "kafka_offset_commit_actor.h"

#include <ydb/core/kafka_proxy/kafka_events.h>

namespace NKafka {


NActors::IActor* CreateKafkaOffsetCommitActor(const TContext::TPtr context, const ui64 correlationId, const TMessagePtr<TOffsetCommitRequestData>& message) {
return new TKafkaOffsetCommitActor(context, correlationId, message);
}

TOffsetCommitResponseData::TPtr TKafkaOffsetCommitActor::GetOffsetCommitResponse() {
TOffsetCommitResponseData::TPtr response = std::make_shared<TOffsetCommitResponseData>();
TString TKafkaOffsetCommitActor::LogPrefix() {
return "TKafkaOffsetCommitActor";
}

void TKafkaOffsetCommitActor::Die(const TActorContext& ctx) {
KAFKA_LOG_D("PassAway");
ctx.Send(AuthInitActor, new TEvents::TEvPoisonPill());
for (const auto& tabletToPipePair: TabletIdToPipe) {
NTabletPipe::CloseClient(ctx, tabletToPipePair.second);
}
TBase::Die(ctx);
}

void TKafkaOffsetCommitActor::Handle(NKikimr::NGRpcProxy::V1::TEvPQProxy::TEvCloseSession::TPtr& ev, const TActorContext& ctx) {
KAFKA_LOG_CRIT("Auth failed. reason# " << ev->Get()->Reason);
Error = ConvertErrorCode(ev->Get()->ErrorCode);
SendFailedForAllPartitions(Error, ctx);
}

void TKafkaOffsetCommitActor::SendFailedForAllPartitions(EKafkaErrors error, const TActorContext& ctx) {
for (auto topicReq: Message->Topics) {
TOffsetCommitResponseData::TOffsetCommitResponseTopic topic;
topic.Name = topicReq.Name;
for (auto partitionRequest: topicReq.Partitions) {
TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitResponsePartition partition;
partition.PartitionIndex = partitionRequest.PartitionIndex;
partition.ErrorCode = NONE_ERROR;
partition.ErrorCode = error;
topic.Partitions.push_back(partition);
}
response->Topics.push_back(topic);
Response->Topics.push_back(topic);
}
Send(Context->ConnectionId, new TEvKafka::TEvResponse(CorrelationId, Response, Error));
Die(ctx);
}

void TKafkaOffsetCommitActor::Handle(TEvTabletPipe::TEvClientConnected::TPtr& ev, const TActorContext& ctx) {
TEvTabletPipe::TEvClientConnected *msg = ev->Get();

if (msg->Status != NKikimrProto::OK) {
KAFKA_LOG_CRIT("Pipe to tablet is dead. status# " << ev->Get()->Status);
ProcessPipeProblem(msg->TabletId, ctx);
}
}

void TKafkaOffsetCommitActor::Handle(TEvTabletPipe::TEvClientDestroyed::TPtr& ev, const TActorContext& ctx) {
KAFKA_LOG_CRIT("Pipe to tablet is destroyed");
ProcessPipeProblem(ev->Get()->TabletId, ctx);
}

void TKafkaOffsetCommitActor::ProcessPipeProblem(ui64 tabletId, const TActorContext& ctx) {
auto cookiesIt = TabletIdToCookies.find(tabletId);
Y_ABORT_UNLESS(cookiesIt != TabletIdToCookies.end());

for (auto cookie: cookiesIt->second) {
auto requestInfoIt = CookieToRequestInfo.find(cookie);
Y_ABORT_UNLESS(requestInfoIt != CookieToRequestInfo.end());

if (!requestInfoIt->second.Done) {
requestInfoIt->second.Done = true;
AddPartitionResponse(EKafkaErrors::UNKNOWN_SERVER_ERROR, requestInfoIt->second.TopicName, requestInfoIt->second.PartitionId, ctx);
}
}
}

void TKafkaOffsetCommitActor::Handle(NGRpcProxy::V1::TEvPQProxy::TEvAuthResultOk::TPtr& ev, const TActorContext& ctx) {
KAFKA_LOG_D("Auth success. Topics count: " << ev->Get()->TopicAndTablets.size());
TopicAndTablets = std::move(ev->Get()->TopicAndTablets);

for (auto topicReq: Message->Topics) {
auto topicIt = TopicAndTablets.find(NormalizePath(Context->DatabasePath, topicReq.Name.value()));
for (auto partitionRequest: topicReq.Partitions) {
if (topicIt == TopicAndTablets.end()) {
AddPartitionResponse(UNKNOWN_TOPIC_OR_PARTITION, topicReq.Name.value(), partitionRequest.PartitionIndex, ctx);
continue;
}

auto tabletIdIt = topicIt->second.PartitionIdToTabletId.find(partitionRequest.PartitionIndex);
if (tabletIdIt == topicIt->second.PartitionIdToTabletId.end()) {
AddPartitionResponse(UNKNOWN_TOPIC_OR_PARTITION, topicReq.Name.value(), partitionRequest.PartitionIndex, ctx);
continue;
}

ui64 tabletId = tabletIdIt->second;

if (!TabletIdToPipe.contains(tabletId)) {
NTabletPipe::TClientConfig clientConfig;
clientConfig.RetryPolicy = RetryPolicyForPipes;
TabletIdToPipe[tabletId] = ctx.Register(NTabletPipe::CreateClient(ctx.SelfID, tabletId, clientConfig));
}

NKikimrClient::TPersQueueRequest request;
request.MutablePartitionRequest()->SetTopic(topicIt->second.TopicNameConverter->GetPrimaryPath());
request.MutablePartitionRequest()->SetPartition(partitionRequest.PartitionIndex);
request.MutablePartitionRequest()->SetCookie(NextCookie);

TRequestInfo info(topicReq.Name.value(), partitionRequest.PartitionIndex);

CookieToRequestInfo.emplace(std::make_pair(NextCookie, info));
TabletIdToCookies[tabletId].push_back(NextCookie);
NextCookie++;

auto commit = request.MutablePartitionRequest()->MutableCmdSetClientOffset();
commit->SetClientId(Message->GroupId.value());
commit->SetOffset(partitionRequest.CommittedOffset);
commit->SetStrict(true);

PendingResponses++;
KAFKA_LOG_D("Send commit request for group# " << Message->GroupId.value() <<
", topic# " << topicIt->second.TopicNameConverter->GetPrimaryPath() <<
", partition# " << partitionRequest.PartitionIndex <<
", offset# " << partitionRequest.CommittedOffset);

TAutoPtr<TEvPersQueue::TEvRequest> req(new TEvPersQueue::TEvRequest);
req->Record.Swap(&request);

NTabletPipe::SendData(ctx, TabletIdToPipe[tabletId], req.Release());
}
}
}

void TKafkaOffsetCommitActor::Handle(TEvPersQueue::TEvResponse::TPtr& ev, const TActorContext& ctx) {
const auto& partitionResult = ev->Get()->Record.GetPartitionResponse();
auto requestInfo = CookieToRequestInfo.find(partitionResult.GetCookie());
requestInfo->second.Done = true;

Y_ABORT_UNLESS(requestInfo != CookieToRequestInfo.end());
if (ev->Get()->Record.GetErrorCode() != NPersQueue::NErrorCode::OK) {
KAFKA_LOG_CRIT("Commit offset error. status# " << EErrorCode_Name(ev->Get()->Record.GetErrorCode()) << ", reason# " << ev->Get()->Record.GetErrorReason());
}

return response;
AddPartitionResponse(ConvertErrorCode(NGRpcProxy::V1::ConvertOldCode(ev->Get()->Record.GetErrorCode())), requestInfo->second.TopicName, requestInfo->second.PartitionId, ctx);
}

void TKafkaOffsetCommitActor::AddPartitionResponse(EKafkaErrors error, const TString& topicName, ui64 partitionId, const TActorContext& ctx) {
if (error != NONE_ERROR) {
Error = error;
}

PendingResponses--;
TOffsetCommitResponseData::TOffsetCommitResponseTopic::TOffsetCommitResponsePartition partitionResponse;
partitionResponse.PartitionIndex = partitionId;
partitionResponse.ErrorCode = error;

auto topicIdIt = ResponseTopicIds.find(topicName);

if (topicIdIt != ResponseTopicIds.end()) {
Response->Topics[topicIdIt->second].Partitions.push_back(partitionResponse);
} else {
ResponseTopicIds[topicName] = Response->Topics.size();

TOffsetCommitResponseData::TOffsetCommitResponseTopic topicResponse;
topicResponse.Name = topicName;
topicResponse.Partitions.push_back(partitionResponse);

Response->Topics.push_back(topicResponse);
}

if (PendingResponses == 0) {
Send(Context->ConnectionId, new TEvKafka::TEvResponse(CorrelationId, Response, Error));
Die(ctx);
}
}

void TKafkaOffsetCommitActor::Bootstrap(const NActors::TActorContext& ctx) {
Y_UNUSED(Message);
auto response = GetOffsetCommitResponse();
THashSet<TString> topicsToResolve;
for (auto topicReq: Message->Topics) {
topicsToResolve.insert(NormalizePath(Context->DatabasePath, topicReq.Name.value()));
}

Send(Context->ConnectionId, new TEvKafka::TEvResponse(CorrelationId, response, EKafkaErrors::NONE_ERROR));
Die(ctx);
auto topicConverterFactory = std::make_shared<NPersQueue::TTopicNamesConverterFactory>(
NKikimr::AppData(ctx)->PQConfig, ""
);

auto topicHandler = std::make_unique<NPersQueue::TTopicsListController>(
topicConverterFactory
);

auto topicsToConverter = topicHandler->GetReadTopicsList(topicsToResolve, false, Context->DatabasePath);
if (!topicsToConverter.IsValid) {
KAFKA_LOG_CRIT("Commit offsets failed. reason# topicsToConverter is not valid");
Error = INVALID_REQUEST;
SendFailedForAllPartitions(Error, ctx);
return;
}

AuthInitActor = ctx.Register(new NKikimr::NGRpcProxy::V1::TReadInitAndAuthActor(
ctx, ctx.SelfID, Message->GroupId.value(), 0, "",
NKikimr::NMsgBusProxy::CreatePersQueueMetaCacheV2Id(), NKikimr::MakeSchemeCacheID(), nullptr, Context->UserToken, topicsToConverter,
topicHandler->GetLocalCluster(), false)
);

Become(&TKafkaOffsetCommitActor::StateWork);
}

} // NKafka
Loading
Loading