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-15579] Create GetReplicaLogInfo* handlers #18670

Draft
wants to merge 2 commits into
base: trunk
Choose a base branch
from
Draft
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
Original file line number Diff line number Diff line change
Expand Up @@ -133,8 +133,8 @@ public enum ApiKeys {
READ_SHARE_GROUP_STATE_SUMMARY(ApiMessageType.READ_SHARE_GROUP_STATE_SUMMARY, true),
STREAMS_GROUP_HEARTBEAT(ApiMessageType.STREAMS_GROUP_HEARTBEAT),
STREAMS_GROUP_DESCRIBE(ApiMessageType.STREAMS_GROUP_DESCRIBE),
DESCRIBE_SHARE_GROUP_OFFSETS(ApiMessageType.DESCRIBE_SHARE_GROUP_OFFSETS);

DESCRIBE_SHARE_GROUP_OFFSETS(ApiMessageType.DESCRIBE_SHARE_GROUP_OFFSETS),
GET_REPLICA_LOG_INFO(ApiMessageType.GET_REPLICA_LOG_INFO);

private static final Map<ApiMessageType.ListenerType, EnumSet<ApiKeys>> APIS_BY_LISTENER =
new EnumMap<>(ApiMessageType.ListenerType.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -348,6 +348,8 @@ private static AbstractRequest doParseRequest(ApiKeys apiKey, short apiVersion,
return StreamsGroupDescribeRequest.parse(buffer, apiVersion);
case DESCRIBE_SHARE_GROUP_OFFSETS:
return DescribeShareGroupOffsetsRequest.parse(buffer, apiVersion);
case GET_REPLICA_LOG_INFO:
return GetReplicaLogInfoRequest.parse(buffer, apiVersion);
default:
throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseRequest`, the " +
"code should be updated to do so.", apiKey));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -285,6 +285,8 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, ByteBuffer response
return StreamsGroupDescribeResponse.parse(responseBuffer, version);
case DESCRIBE_SHARE_GROUP_OFFSETS:
return DescribeShareGroupOffsetsResponse.parse(responseBuffer, version);
case GET_REPLICA_LOG_INFO:
return GetReplicaLogInfoResponse.parse(responseBuffer, version);
default:
throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " +
"code should be updated to do so.", apiKey));
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.kafka.common.requests;

import org.apache.kafka.common.message.GetReplicaLogInfoRequestData;
import org.apache.kafka.common.message.GetReplicaLogInfoResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors;

import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;

public class GetReplicaLogInfoRequest extends AbstractRequest {
public static class Builder extends AbstractRequest.Builder<GetReplicaLogInfoRequest> {

private final GetReplicaLogInfoRequestData data;

public Builder(GetReplicaLogInfoRequestData data) {
super(ApiKeys.GET_REPLICA_LOG_INFO);
this.data = data;
}

public Builder(List<GetReplicaLogInfoRequestData.TopicPartitions> topicPartitions) {
super(ApiKeys.GET_REPLICA_LOG_INFO, ApiKeys.GET_REPLICA_LOG_INFO.oldestVersion(),
ApiKeys.GET_REPLICA_LOG_INFO.latestVersion());
GetReplicaLogInfoRequestData data = new GetReplicaLogInfoRequestData();
data.setTopicPartitions(topicPartitions);
this.data = data;
}

@Override
public GetReplicaLogInfoRequest build(short version) {
return new GetReplicaLogInfoRequest(data, version);
}

@Override
public String toString() {
return data.toString();
}
}

private final GetReplicaLogInfoRequestData data;

public GetReplicaLogInfoRequest(GetReplicaLogInfoRequestData data) {
super(ApiKeys.GET_REPLICA_LOG_INFO, (short) 0);
this.data = data;
}

public GetReplicaLogInfoRequest(GetReplicaLogInfoRequestData data, short version) {
super(ApiKeys.GET_REPLICA_LOG_INFO, version);
this.data = data;
}

@Override
public GetReplicaLogInfoRequestData data() {
return data;
}

@Override
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
Errors error = Errors.forException(e);
GetReplicaLogInfoResponseData responseData = new GetReplicaLogInfoResponseData();
for (GetReplicaLogInfoRequestData.TopicPartitions topicPartition : data().topicPartitions()) {
ArrayList<GetReplicaLogInfoResponseData.PartitionLogInfo> partitionLogInfos = new ArrayList<>();
for (Integer partition: topicPartition.partitions()) {
partitionLogInfos.add(new GetReplicaLogInfoResponseData.PartitionLogInfo()
.setPartition(partition)
.setErrorCode(error.code())
);
}
responseData.topicPartitionLogInfoList().add(new GetReplicaLogInfoResponseData.TopicPartitionLogInfo()
.setTopicId(topicPartition.topicId())
.setPartitionLogInfo(partitionLogInfos));
}
return new GetReplicaLogInfoResponse(responseData);
}

public static GetReplicaLogInfoRequest parse(ByteBuffer buffer, short version) {
return new GetReplicaLogInfoRequest(new GetReplicaLogInfoRequestData(new ByteBufferAccessor(buffer), version), version);
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.kafka.common.requests;

import org.apache.kafka.common.message.GetReplicaLogInfoResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors;

import java.nio.ByteBuffer;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

public class GetReplicaLogInfoResponse extends AbstractResponse {
private final GetReplicaLogInfoResponseData data;

public GetReplicaLogInfoResponse(GetReplicaLogInfoResponseData data) {
super(ApiKeys.GET_REPLICA_LOG_INFO);
this.data = data;
}

@Override
public GetReplicaLogInfoResponseData data() {
return data;
}

@Override
public int throttleTimeMs() {
return data.throttleTimeMs();
}

@Override
public void maybeSetThrottleTimeMs(int throttleTimeMs) {
data.setThrottleTimeMs(throttleTimeMs);
}

@Override
public Map<Errors, Integer> errorCounts() {
Map<Errors, Integer> errorCounts = new HashMap<>();
data.topicPartitionLogInfoList().forEach(topicPartitionLogInfo -> {
topicPartitionLogInfo.partitionLogInfo().forEach(partitionLogInfo -> {
updateErrorCounts(errorCounts, Errors.forCode(partitionLogInfo.errorCode()));
});
});
return errorCounts;
}


public static GetReplicaLogInfoResponse prepareResponse(List<GetReplicaLogInfoResponseData.TopicPartitionLogInfo> topicPartitionLogInfoList) {
GetReplicaLogInfoResponseData responseData = new GetReplicaLogInfoResponseData();
topicPartitionLogInfoList.forEach(topicPartitionLogInfo -> {
responseData.topicPartitionLogInfoList().add(topicPartitionLogInfo);
});
return new GetReplicaLogInfoResponse(responseData);
}

public static GetReplicaLogInfoResponse parse(ByteBuffer buffer, short version) {
return new GetReplicaLogInfoResponse(new GetReplicaLogInfoResponseData(new ByteBufferAccessor(buffer), version));
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

{
"apiKey":91,
"type": "request",
"listeners": ["broker"],
"name": "GetReplicaLogInfoRequest",
"validVersions": "0",
"flexibleVersions": "0+",
"latestVersionUnstable": true,
"fields": [
{ "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
"about": "The ID of the broker." },
{ "name": "TopicPartitions", "type": "[]TopicPartitions", "versions": "0+", "nullableVersions": "0+",
"about": "The topic partitions to query the log info.",
"fields": [
{ "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The unique topic ID"},
{ "name": "Partitions", "type": "[]int32", "versions": "0+",
"about": "The partitions of this topic whose leader should be elected." }
]}
] }
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

{
"apiKey":91,
"type": "response",
"name": "GetReplicaLogInfoResponse",
"validVersions": "0",
"flexibleVersions": "0+",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
"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." },
{ "name": "BrokerEpoch", "type": "int64", "versions": "0+", "about": "The epoch for the broker." },
{ "name": "TopicPartitionLogInfoList", "type": "[]TopicPartitionLogInfo", "versions": "0+",
"about": "The list of the log info.",
"fields": [
{ "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, "about": "The unique topic ID."},
{ "name": "PartitionLogInfo", "type": "[]PartitionLogInfo", "versions": "0+", "about": "The log info of a partition.",
"fields": [
{ "name": "Partition", "type": "int32", "versions": "0+", "about": "The id for the partition." },
{ "name": "LastWrittenLeaderEpoch", "type": "int32", "versions": "0+", "about": "The last written leader epoch in the log." },
{ "name": "CurrentLeaderEpoch", "type": "int32", "versions": "0+", "about": "The current leader epoch for the partition from the broker point of view." },
{ "name": "LogEndOffset", "type": "int64", "versions": "0+", "about": "The log end offset for the partition." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+", "about": "The result error, or zero if there was no error."},
{ "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "about": "The result message, or null if there was no error."} ]}
]}
] }
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,8 @@
import org.apache.kafka.common.message.FetchSnapshotRequestData;
import org.apache.kafka.common.message.FetchSnapshotResponseData;
import org.apache.kafka.common.message.FindCoordinatorRequestData;
import org.apache.kafka.common.message.GetReplicaLogInfoRequestData;
import org.apache.kafka.common.message.GetReplicaLogInfoResponseData;
import org.apache.kafka.common.message.GetTelemetrySubscriptionsRequestData;
import org.apache.kafka.common.message.GetTelemetrySubscriptionsResponseData;
import org.apache.kafka.common.message.HeartbeatRequestData;
Expand Down Expand Up @@ -1039,6 +1041,7 @@ private AbstractRequest getRequest(ApiKeys apikey, short version) {
case STREAMS_GROUP_HEARTBEAT: return createStreamsGroupHeartbeatRequest(version);
case STREAMS_GROUP_DESCRIBE: return createStreamsGroupDescribeRequest(version);
case DESCRIBE_SHARE_GROUP_OFFSETS: return createDescribeShareGroupOffsetsRequest(version);
case GET_REPLICA_LOG_INFO: return createGetReplicaLogInfoRequest(version);
default: throw new IllegalArgumentException("Unknown API key " + apikey);
}
}
Expand Down Expand Up @@ -1132,10 +1135,26 @@ private AbstractResponse getResponse(ApiKeys apikey, short version) {
case STREAMS_GROUP_HEARTBEAT: return createStreamsGroupHeartbeatResponse();
case STREAMS_GROUP_DESCRIBE: return createStreamsGroupDescribeResponse();
case DESCRIBE_SHARE_GROUP_OFFSETS: return createDescribeShareGroupOffsetsResponse();
case GET_REPLICA_LOG_INFO: return createGetReplicaLogInfoResponse();
default: throw new IllegalArgumentException("Unknown API key " + apikey);
}
}

private GetReplicaLogInfoRequest createGetReplicaLogInfoRequest(short version) {
GetReplicaLogInfoRequestData data = new GetReplicaLogInfoRequestData()
.setBrokerId(0)
.setTopicPartitions(singletonList(new GetReplicaLogInfoRequestData.TopicPartitions()
.setPartitions(singletonList(0))));
return new GetReplicaLogInfoRequest.Builder(data).build(version);
}

private GetReplicaLogInfoResponse createGetReplicaLogInfoResponse() {
GetReplicaLogInfoResponseData data = new GetReplicaLogInfoResponseData();
data.setBrokerEpoch(0);
data.setTopicPartitionLogInfoList(singletonList(new GetReplicaLogInfoResponseData.TopicPartitionLogInfo()));
return new GetReplicaLogInfoResponse(data);
}

private ConsumerGroupDescribeRequest createConsumerGroupDescribeRequest(short version) {
ConsumerGroupDescribeRequestData data = new ConsumerGroupDescribeRequestData()
.setGroupIds(Collections.singletonList("group"))
Expand Down
Loading