You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "lqjacklee (Jira)" <ji...@apache.org> on 2022/05/11 12:07:00 UTC

[jira] [Commented] (KAFKA-13888) KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

    [ https://issues.apache.org/jira/browse/KAFKA-13888?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17534847#comment-17534847 ] 

lqjacklee commented on KAFKA-13888:
-----------------------------------

{code:java}
// code placeholder
{code}
[~Niket Goel]  I have two solution to resolve it, please help determine which is better ?

 

1, add field Map<Integer, OffsetAndEpoch> in org.apache.kafka.raft.QuorumState, once request comes, will response with the lag (org.apache.kafka.raft.ReplicatedLog#endOffset - OffsetAndEpoch.

offset ? or org.apache.kafka.raft.ReplicatedLog#highWatermark - 

OffsetAndEpoch.

offset)

 

2, change 

DescribeQuorumRequest json definition:

 
{code:java}
{
  "apiKey": 55,
  "type": "request",
  "listeners": ["broker", "controller"],
  "name": "DescribeQuorumRequest",
  "validVersions": "0",
  "flexibleVersions": "0+",
  "fields": [
    { "name": "Topics", "type": "[]TopicData",
      "versions": "0+", "fields": [
      { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
        "about": "The topic name." },
      { "name": "Partitions", "type": "[]PartitionData",
        "versions": "0+", "fields": [
        { "name": "PartitionIndex", "type": "int32", "versions": "0+",
          "about": "The partition index." },
        { "name": "PartitionOffset", "type": "int64", "versions": "0+",
          "about": "The partition offset." }
      ]
      }]
    }
  ]
}
{code}
DescribeQuorumResponse json :

{code:java}

{
  "apiKey": 55,
  "type": "response",
  "name": "DescribeQuorumResponse",
  "validVersions": "0",
  "flexibleVersions": "0+",
  "fields": [
    { "name": "ErrorCode", "type": "int16", "versions": "0+",
      "about": "The top level error code."},
    { "name": "Topics", "type": "[]TopicData",
      "versions": "0+", "fields": [
      { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
        "about": "The topic name." },
      { "name": "Partitions", "type": "[]PartitionData",
        "versions": "0+", "fields": [
        { "name": "PartitionIndex", "type": "int32", "versions": "0+",
          "about": "The partition index." },
        { "name": "ErrorCode", "type": "int16", "versions": "0+"},
        { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId",
          "about": "The ID of the current leader or -1 if the leader is unknown."},
        { "name": "LeaderEpoch", "type": "int32", "versions": "0+",
          "about": "The latest known leader epoch"},
        { "name": "HighWatermark", "type": "int64", "versions": "0+"},
        { "name": "CurrentVoters", "type": "[]ReplicaState", "versions": "0+" },
        { "name": "Observers", "type": "[]ReplicaState", "versions": "0+" },
        { "name": "Lag", "type": "int64", "versions": "0+"}
      ]}
    ]}],
  "commonStructs": [
    { "name": "ReplicaState", "versions": "0+", "fields": [
      { "name": "ReplicaId", "type": "int32", "versions": "0+", "entityType": "brokerId" },
      { "name": "LogEndOffset", "type": "int64", "versions": "0+",
        "about": "The last known log end offset of the follower or -1 if it is unknown"}
    ]}
  ]
}

{code}

> KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag
> --------------------------------------------------------------------------
>
>                 Key: KAFKA-13888
>                 URL: https://issues.apache.org/jira/browse/KAFKA-13888
>             Project: Kafka
>          Issue Type: Improvement
>          Components: kraft
>            Reporter: Niket Goel
>            Priority: Major
>
> Tracking issue for the implementation of KIP:836



--
This message was sent by Atlassian Jira
(v8.20.7#820007)