You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ch...@apache.org on 2021/03/10 05:28:36 UTC

[kafka] branch trunk updated: MINOR: Add entityType for metadata record definitions (#10116)

This is an automated email from the ASF dual-hosted git repository.

chia7712 pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 8c9bc9c  MINOR: Add entityType for metadata record definitions (#10116)
8c9bc9c is described below

commit 8c9bc9c6401354b04e6f786eb3b8187edb4b3b04
Author: dengziming <sw...@163.com>
AuthorDate: Wed Mar 10 13:27:06 2021 +0800

    MINOR: Add entityType for metadata record definitions (#10116)
    
    Reviewers: Chia-Ping Tsai <ch...@gmail.com>
---
 clients/src/main/resources/common/message/AlterIsrRequest.json    | 2 +-
 clients/src/main/resources/common/message/AlterIsrResponse.json   | 2 +-
 .../common/message/AlterPartitionReassignmentsRequest.json        | 2 +-
 .../main/resources/common/message/BeginQuorumEpochRequest.json    | 2 +-
 .../main/resources/common/message/BeginQuorumEpochResponse.json   | 2 +-
 .../src/main/resources/common/message/BrokerHeartbeatRequest.json | 2 +-
 .../main/resources/common/message/BrokerRegistrationRequest.json  | 2 +-
 .../main/resources/common/message/ConsumerProtocolAssignment.json | 2 +-
 .../resources/common/message/ConsumerProtocolSubscription.json    | 2 +-
 .../src/main/resources/common/message/DeleteTopicsRequest.json    | 2 +-
 .../src/main/resources/common/message/DescribeQuorumResponse.json | 4 ++--
 .../src/main/resources/common/message/EndQuorumEpochRequest.json  | 2 +-
 .../src/main/resources/common/message/EndQuorumEpochResponse.json | 2 +-
 clients/src/main/resources/common/message/FetchRequest.json       | 2 +-
 clients/src/main/resources/common/message/FetchResponse.json      | 4 ++--
 .../src/main/resources/common/message/FetchSnapshotRequest.json   | 2 +-
 .../src/main/resources/common/message/FetchSnapshotResponse.json  | 2 +-
 .../src/main/resources/common/message/InitProducerIdRequest.json  | 2 +-
 .../src/main/resources/common/message/LeaderAndIsrRequest.json    | 8 ++++----
 .../src/main/resources/common/message/LeaderChangeMessage.json    | 2 +-
 .../common/message/ListPartitionReassignmentsResponse.json        | 6 +++---
 .../main/resources/common/message/ListTransactionsRequest.json    | 2 +-
 clients/src/main/resources/common/message/MetadataResponse.json   | 4 ++--
 .../resources/common/message/OffsetForLeaderEpochRequest.json     | 2 +-
 .../main/resources/common/message/UnregisterBrokerRequest.json    | 2 +-
 .../src/main/resources/common/message/UpdateFeaturesResponse.json | 2 +-
 clients/src/main/resources/common/message/VoteRequest.json        | 2 +-
 clients/src/main/resources/common/message/VoteResponse.json       | 2 +-
 .../src/main/resources/common/metadata/FenceBrokerRecord.json     | 2 +-
 .../src/main/resources/common/metadata/PartitionChangeRecord.json | 6 +++---
 metadata/src/main/resources/common/metadata/PartitionRecord.json  | 8 ++++----
 .../src/main/resources/common/metadata/RegisterBrokerRecord.json  | 2 +-
 metadata/src/main/resources/common/metadata/TopicRecord.json      | 2 +-
 .../src/main/resources/common/metadata/UnfenceBrokerRecord.json   | 2 +-
 .../main/resources/common/metadata/UnregisterBrokerRecord.json    | 2 +-
 35 files changed, 48 insertions(+), 48 deletions(-)

diff --git a/clients/src/main/resources/common/message/AlterIsrRequest.json b/clients/src/main/resources/common/message/AlterIsrRequest.json
index f950cd7..70736db 100644
--- a/clients/src/main/resources/common/message/AlterIsrRequest.json
+++ b/clients/src/main/resources/common/message/AlterIsrRequest.json
@@ -33,7 +33,7 @@
           "about": "The partition index" },
         { "name": "LeaderEpoch", "type": "int32", "versions": "0+",
           "about": "The leader epoch of this partition" },
-        { "name": "NewIsr", "type": "[]int32", "versions": "0+",
+        { "name": "NewIsr", "type": "[]int32", "versions": "0+", "entityType": "brokerId",
           "about": "The ISR for this partition"},
         { "name": "CurrentIsrVersion", "type": "int32", "versions": "0+",
           "about": "The expected version of ISR which is being updated"}
diff --git a/clients/src/main/resources/common/message/AlterIsrResponse.json b/clients/src/main/resources/common/message/AlterIsrResponse.json
index 7a81339..3383799 100644
--- a/clients/src/main/resources/common/message/AlterIsrResponse.json
+++ b/clients/src/main/resources/common/message/AlterIsrResponse.json
@@ -36,7 +36,7 @@
           "about": "The broker ID of the leader." },
         { "name": "LeaderEpoch", "type": "int32", "versions": "0+",
           "about": "The leader epoch." },
-        { "name": "Isr", "type": "[]int32", "versions": "0+",
+        { "name": "Isr", "type": "[]int32", "versions": "0+", "entityType": "brokerId",
           "about": "The in-sync replica IDs." },
         { "name": "CurrentIsrVersion", "type": "int32", "versions": "0+",
           "about": "The current ISR version." }
diff --git a/clients/src/main/resources/common/message/AlterPartitionReassignmentsRequest.json b/clients/src/main/resources/common/message/AlterPartitionReassignmentsRequest.json
index ee05b42..7a96647 100644
--- a/clients/src/main/resources/common/message/AlterPartitionReassignmentsRequest.json
+++ b/clients/src/main/resources/common/message/AlterPartitionReassignmentsRequest.json
@@ -31,7 +31,7 @@
         "about": "The partitions to reassign.", "fields": [
         { "name": "PartitionIndex", "type": "int32", "versions": "0+",
           "about": "The partition index." },
-        { "name": "Replicas", "type": "[]int32", "versions": "0+", "nullableVersions": "0+", "default": "null",
+        { "name": "Replicas", "type": "[]int32", "versions": "0+", "nullableVersions": "0+", "default": "null", "entityType": "brokerId",
           "about": "The replicas to place the partitions on, or null to cancel a pending reassignment for this partition." }
       ]}
     ]}
diff --git a/clients/src/main/resources/common/message/BeginQuorumEpochRequest.json b/clients/src/main/resources/common/message/BeginQuorumEpochRequest.json
index 9f7969f..f419589 100644
--- a/clients/src/main/resources/common/message/BeginQuorumEpochRequest.json
+++ b/clients/src/main/resources/common/message/BeginQuorumEpochRequest.json
@@ -30,7 +30,7 @@
         "versions": "0+", "fields": [
         { "name": "PartitionIndex", "type": "int32", "versions": "0+",
           "about": "The partition index." },
-        { "name": "LeaderId", "type": "int32", "versions": "0+",
+        { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId",
           "about": "The ID of the newly elected leader"},
         { "name": "LeaderEpoch", "type": "int32", "versions": "0+",
           "about": "The epoch of the newly elected leader"}
diff --git a/clients/src/main/resources/common/message/BeginQuorumEpochResponse.json b/clients/src/main/resources/common/message/BeginQuorumEpochResponse.json
index 41e2292..923cd1d 100644
--- a/clients/src/main/resources/common/message/BeginQuorumEpochResponse.json
+++ b/clients/src/main/resources/common/message/BeginQuorumEpochResponse.json
@@ -30,7 +30,7 @@
         { "name": "PartitionIndex", "type": "int32", "versions": "0+",
           "about": "The partition index." },
         { "name": "ErrorCode", "type": "int16", "versions": "0+"},
-        { "name": "LeaderId", "type": "int32", "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"}
diff --git a/clients/src/main/resources/common/message/BrokerHeartbeatRequest.json b/clients/src/main/resources/common/message/BrokerHeartbeatRequest.json
index ce08d11..2cf2577 100644
--- a/clients/src/main/resources/common/message/BrokerHeartbeatRequest.json
+++ b/clients/src/main/resources/common/message/BrokerHeartbeatRequest.json
@@ -21,7 +21,7 @@
   "validVersions": "0",
   "flexibleVersions": "0+",
   "fields": [
-    { "name": "BrokerId", "type": "int32", "versions": "0+",
+    { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
       "about": "The broker ID." },
     { "name": "BrokerEpoch", "type": "int64", "versions": "0+", "default": "-1",
       "about": "The broker epoch." },
diff --git a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
index 3e27cf1..b17415d 100644
--- a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
+++ b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
@@ -21,7 +21,7 @@
   "validVersions": "0",
   "flexibleVersions": "0+",
   "fields": [
-    { "name": "BrokerId", "type": "int32", "versions": "0+",
+    { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
       "about": "The broker ID." },
     { "name": "ClusterId", "type": "uuid", "versions": "0+",
       "about": "The cluster id of the broker process." },
diff --git a/clients/src/main/resources/common/message/ConsumerProtocolAssignment.json b/clients/src/main/resources/common/message/ConsumerProtocolAssignment.json
index 544db20..d148085 100644
--- a/clients/src/main/resources/common/message/ConsumerProtocolAssignment.json
+++ b/clients/src/main/resources/common/message/ConsumerProtocolAssignment.json
@@ -25,7 +25,7 @@
   "fields": [
     { "name": "AssignedPartitions", "type": "[]TopicPartition", "versions": "0+",
       "fields": [
-        { "name": "Topic", "type": "string", "mapKey": true, "versions": "0+" },
+        { "name": "Topic", "type": "string", "mapKey": true, "versions": "0+", "entityType": "topicName" },
         { "name": "Partitions", "type": "[]int32", "versions": "0+" }
       ]
     },
diff --git a/clients/src/main/resources/common/message/ConsumerProtocolSubscription.json b/clients/src/main/resources/common/message/ConsumerProtocolSubscription.json
index 207dac7..81ed7f7 100644
--- a/clients/src/main/resources/common/message/ConsumerProtocolSubscription.json
+++ b/clients/src/main/resources/common/message/ConsumerProtocolSubscription.json
@@ -28,7 +28,7 @@
       "default": "null", "zeroCopy": true },
     { "name": "OwnedPartitions", "type": "[]TopicPartition", "versions": "1+", "ignorable": true,
       "fields": [
-        { "name": "Topic", "type": "string", "mapKey": true, "versions": "1+" },
+        { "name": "Topic", "type": "string", "mapKey": true, "versions": "1+", "entityType": "topicName" },
         { "name": "Partitions", "type": "[]int32", "versions": "1+"}
       ]
     }
diff --git a/clients/src/main/resources/common/message/DeleteTopicsRequest.json b/clients/src/main/resources/common/message/DeleteTopicsRequest.json
index f757ff7..2e020e980 100644
--- a/clients/src/main/resources/common/message/DeleteTopicsRequest.json
+++ b/clients/src/main/resources/common/message/DeleteTopicsRequest.json
@@ -31,7 +31,7 @@
   "fields": [
     { "name": "Topics", "type": "[]DeleteTopicState", "versions": "6+", "about": "The name or topic ID of the topic",
       "fields": [
-        {"name": "Name", "type": "string", "versions": "6+", "nullableVersions": "6+", "about": "The topic name"},
+        {"name": "Name", "type": "string", "versions": "6+", "nullableVersions": "6+", "entityType": "topicName", "about": "The topic name"},
         {"name": "TopicId", "type": "uuid", "versions": "6+", "about": "The unique topic ID"}
     ]},
     { "name": "TopicNames", "type": "[]string", "versions": "0-5", "entityType": "topicName", "ignorable": true,
diff --git a/clients/src/main/resources/common/message/DescribeQuorumResponse.json b/clients/src/main/resources/common/message/DescribeQuorumResponse.json
index 98fbb43..444fee3 100644
--- a/clients/src/main/resources/common/message/DescribeQuorumResponse.json
+++ b/clients/src/main/resources/common/message/DescribeQuorumResponse.json
@@ -31,7 +31,7 @@
         { "name": "PartitionIndex", "type": "int32", "versions": "0+",
           "about": "The partition index." },
         { "name": "ErrorCode", "type": "int16", "versions": "0+"},
-        { "name": "LeaderId", "type": "int32", "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"},
@@ -42,7 +42,7 @@
     ]}],
   "commonStructs": [
     { "name": "ReplicaState", "versions": "0+", "fields": [
-      { "name": "ReplicaId", "type": "int32", "versions": "0+"},
+      { "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"}
     ]}
diff --git a/clients/src/main/resources/common/message/EndQuorumEpochRequest.json b/clients/src/main/resources/common/message/EndQuorumEpochRequest.json
index 3ef7f63..25bb57a 100644
--- a/clients/src/main/resources/common/message/EndQuorumEpochRequest.json
+++ b/clients/src/main/resources/common/message/EndQuorumEpochRequest.json
@@ -30,7 +30,7 @@
         "versions": "0+", "fields": [
         { "name": "PartitionIndex", "type": "int32", "versions": "0+",
           "about": "The partition index." },
-        { "name": "LeaderId", "type": "int32", "versions": "0+",
+        { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId",
           "about": "The current leader ID that is resigning"},
         { "name": "LeaderEpoch", "type": "int32", "versions": "0+",
           "about": "The current epoch"},
diff --git a/clients/src/main/resources/common/message/EndQuorumEpochResponse.json b/clients/src/main/resources/common/message/EndQuorumEpochResponse.json
index 29d24d0..62f67bd 100644
--- a/clients/src/main/resources/common/message/EndQuorumEpochResponse.json
+++ b/clients/src/main/resources/common/message/EndQuorumEpochResponse.json
@@ -30,7 +30,7 @@
         { "name": "PartitionIndex", "type": "int32", "versions": "0+",
           "about": "The partition index." },
         { "name": "ErrorCode", "type": "int16", "versions": "0+"},
-        { "name": "LeaderId", "type": "int32", "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"}
diff --git a/clients/src/main/resources/common/message/FetchRequest.json b/clients/src/main/resources/common/message/FetchRequest.json
index 6594773..48e31a0 100644
--- a/clients/src/main/resources/common/message/FetchRequest.json
+++ b/clients/src/main/resources/common/message/FetchRequest.json
@@ -53,7 +53,7 @@
     { "name": "ClusterId", "type": "string", "versions": "12+", "nullableVersions": "12+", "default": "null",
       "taggedVersions": "12+", "tag": 0, "ignorable": true,
       "about": "The clusterId if known. This is used to validate metadata fetches prior to broker registration." },
-    { "name": "ReplicaId", "type": "int32", "versions": "0+",
+    { "name": "ReplicaId", "type": "int32", "versions": "0+", "entityType": "brokerId",
       "about": "The broker ID of the follower, of -1 if this request is from a consumer." },
     { "name": "MaxWaitMs", "type": "int32", "versions": "0+",
       "about": "The maximum time in milliseconds to wait for the response." },
diff --git a/clients/src/main/resources/common/message/FetchResponse.json b/clients/src/main/resources/common/message/FetchResponse.json
index 2280725..67f7527 100644
--- a/clients/src/main/resources/common/message/FetchResponse.json
+++ b/clients/src/main/resources/common/message/FetchResponse.json
@@ -73,7 +73,7 @@
         ]},
         { "name": "CurrentLeader", "type": "LeaderIdAndEpoch",
           "versions": "12+", "taggedVersions": "12+", "tag": 1, "fields": [
-          { "name": "LeaderId", "type": "int32", "versions": "12+", "default": "-1",
+          { "name": "LeaderId", "type": "int32", "versions": "12+", "default": "-1", "entityType": "brokerId",
             "about": "The ID of the current leader or -1 if the leader is unknown."},
           { "name": "LeaderEpoch", "type": "int32", "versions": "12+", "default": "-1",
             "about": "The latest known leader epoch"}
@@ -92,7 +92,7 @@
           { "name": "FirstOffset", "type": "int64", "versions": "4+",
             "about": "The first offset in the aborted transaction." }
         ]},
-        { "name": "PreferredReadReplica", "type": "int32", "versions": "11+", "default": "-1", "ignorable": false,
+        { "name": "PreferredReadReplica", "type": "int32", "versions": "11+", "default": "-1", "ignorable": false, "entityType": "brokerId",
           "about": "The preferred read replica for the consumer to use on its next fetch request"},
         { "name": "Records", "type": "records", "versions": "0+", "nullableVersions": "0+", "about": "The record data."}
       ]}
diff --git a/clients/src/main/resources/common/message/FetchSnapshotRequest.json b/clients/src/main/resources/common/message/FetchSnapshotRequest.json
index accc227..358ef2e 100644
--- a/clients/src/main/resources/common/message/FetchSnapshotRequest.json
+++ b/clients/src/main/resources/common/message/FetchSnapshotRequest.json
@@ -23,7 +23,7 @@
   "fields": [
     { "name": "ClusterId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "taggedVersions": "0+", "tag": 0,
       "about": "The clusterId if known, this is used to validate metadata fetches prior to broker registration" },
-    { "name": "ReplicaId", "type": "int32", "versions": "0+", "default": "-1",
+    { "name": "ReplicaId", "type": "int32", "versions": "0+", "default": "-1", "entityType": "brokerId",
       "about": "The broker ID of the follower" },
     { "name": "MaxBytes", "type": "int32", "versions": "0+", "default": "0x7fffffff",
       "about": "The maximum bytes to fetch from all of the snapshots" },
diff --git a/clients/src/main/resources/common/message/FetchSnapshotResponse.json b/clients/src/main/resources/common/message/FetchSnapshotResponse.json
index 4e6ff85..887a5e4 100644
--- a/clients/src/main/resources/common/message/FetchSnapshotResponse.json
+++ b/clients/src/main/resources/common/message/FetchSnapshotResponse.json
@@ -42,7 +42,7 @@
         ]},
         { "name": "CurrentLeader", "type": "LeaderIdAndEpoch",
           "versions": "0+", "taggedVersions": "0+", "tag": 0, "fields": [
-          { "name": "LeaderId", "type": "int32", "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"}
diff --git a/clients/src/main/resources/common/message/InitProducerIdRequest.json b/clients/src/main/resources/common/message/InitProducerIdRequest.json
index 9e34505..5537aa9 100644
--- a/clients/src/main/resources/common/message/InitProducerIdRequest.json
+++ b/clients/src/main/resources/common/message/InitProducerIdRequest.json
@@ -32,7 +32,7 @@
       "about": "The transactional id, or null if the producer is not transactional." },
     { "name": "TransactionTimeoutMs", "type": "int32", "versions": "0+",
       "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." },
-    { "name": "ProducerId", "type": "int64", "versions": "3+", "default": "-1",
+    { "name": "ProducerId", "type": "int64", "versions": "3+", "default": "-1", "entityType": "producerId",
       "about": "The producer id. This is used to disambiguate requests if a transactional id is reused following its expiration." },
     { "name": "ProducerEpoch", "type": "int16", "versions": "3+", "default": "-1",
       "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." }
diff --git a/clients/src/main/resources/common/message/LeaderAndIsrRequest.json b/clients/src/main/resources/common/message/LeaderAndIsrRequest.json
index 57e6f21..c38f21e 100644
--- a/clients/src/main/resources/common/message/LeaderAndIsrRequest.json
+++ b/clients/src/main/resources/common/message/LeaderAndIsrRequest.json
@@ -74,15 +74,15 @@
         "about": "The broker ID of the leader." },
       { "name": "LeaderEpoch", "type": "int32", "versions": "0+",
         "about": "The leader epoch." },
-      { "name": "Isr", "type": "[]int32", "versions": "0+",
+      { "name": "Isr", "type": "[]int32", "versions": "0+", "entityType": "brokerId",
         "about": "The in-sync replica IDs." },
       { "name": "ZkVersion", "type": "int32", "versions": "0+",
         "about": "The ZooKeeper version." },
-      { "name": "Replicas", "type": "[]int32", "versions": "0+",
+      { "name": "Replicas", "type": "[]int32", "versions": "0+", "entityType": "brokerId",
         "about": "The replica IDs." },
-      { "name": "AddingReplicas", "type": "[]int32", "versions": "3+", "ignorable": true,
+      { "name": "AddingReplicas", "type": "[]int32", "versions": "3+", "ignorable": true, "entityType": "brokerId",
         "about": "The replica IDs that we are adding this partition to, or null if no replicas are being added." },
-      { "name": "RemovingReplicas", "type": "[]int32", "versions": "3+", "ignorable": true,
+      { "name": "RemovingReplicas", "type": "[]int32", "versions": "3+", "ignorable": true, "entityType": "brokerId",
         "about": "The replica IDs that we are removing this partition from, or null if no replicas are being removed." },
       { "name": "IsNew", "type": "bool", "versions": "1+", "default": "false", "ignorable": true,
         "about": "Whether the replica should have existed on the broker or not." }
diff --git a/clients/src/main/resources/common/message/LeaderChangeMessage.json b/clients/src/main/resources/common/message/LeaderChangeMessage.json
index cb4d0fd..925df6a 100644
--- a/clients/src/main/resources/common/message/LeaderChangeMessage.json
+++ b/clients/src/main/resources/common/message/LeaderChangeMessage.json
@@ -19,7 +19,7 @@
   "validVersions": "0",
   "flexibleVersions": "0+",
   "fields": [
-    {"name": "LeaderId", "type": "int32", "versions": "0+",
+    {"name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId",
       "about": "The ID of the newly elected leader"},
     {"name": "Voters", "type": "[]Voter", "versions": "0+",
       "about": "The set of voters in the quorum for this epoch"},
diff --git a/clients/src/main/resources/common/message/ListPartitionReassignmentsResponse.json b/clients/src/main/resources/common/message/ListPartitionReassignmentsResponse.json
index 36cf70c..753d9bf 100644
--- a/clients/src/main/resources/common/message/ListPartitionReassignmentsResponse.json
+++ b/clients/src/main/resources/common/message/ListPartitionReassignmentsResponse.json
@@ -34,11 +34,11 @@
         "about": "The ongoing reassignments for each partition.", "fields": [
         { "name": "PartitionIndex", "type": "int32", "versions": "0+",
           "about": "The index of the partition." },
-        { "name": "Replicas", "type": "[]int32", "versions": "0+",
+        { "name": "Replicas", "type": "[]int32", "versions": "0+", "entityType": "brokerId",
           "about": "The current replica set." },
-        { "name": "AddingReplicas", "type": "[]int32", "versions": "0+",
+        { "name": "AddingReplicas", "type": "[]int32", "versions": "0+", "entityType": "brokerId",
           "about": "The set of replicas we are currently adding." },
-        { "name": "RemovingReplicas", "type": "[]int32", "versions": "0+",
+        { "name": "RemovingReplicas", "type": "[]int32", "versions": "0+", "entityType": "brokerId",
           "about": "The set of replicas we are currently removing." }
       ]}
     ]}
diff --git a/clients/src/main/resources/common/message/ListTransactionsRequest.json b/clients/src/main/resources/common/message/ListTransactionsRequest.json
index 716b753..21f4552 100644
--- a/clients/src/main/resources/common/message/ListTransactionsRequest.json
+++ b/clients/src/main/resources/common/message/ListTransactionsRequest.json
@@ -24,7 +24,7 @@
     { "name": "StateFilters", "type": "[]string", "versions": "0+",
       "about": "The transaction states to filter by: if empty, all transactions are returned; if non-empty, then only transactions matching one of the filtered states will be returned"
     },
-    { "name": "ProducerIdFilters", "type": "[]int64", "versions": "0+",
+    { "name": "ProducerIdFilters", "type": "[]int64", "versions": "0+", "entityType": "producerId",
       "about": "The producerIds to filter by: if empty, all transactions will be returned; if non-empty, only transactions which match one of the filtered producerIds will be returned"
     }
   ]
diff --git a/clients/src/main/resources/common/message/MetadataResponse.json b/clients/src/main/resources/common/message/MetadataResponse.json
index 70638d2..d005294 100644
--- a/clients/src/main/resources/common/message/MetadataResponse.json
+++ b/clients/src/main/resources/common/message/MetadataResponse.json
@@ -82,9 +82,9 @@
           "about": "The leader epoch of this partition." },
         { "name": "ReplicaNodes", "type": "[]int32", "versions": "0+", "entityType": "brokerId",
           "about": "The set of all nodes that host this partition." },
-        { "name": "IsrNodes", "type": "[]int32", "versions": "0+",
+        { "name": "IsrNodes", "type": "[]int32", "versions": "0+", "entityType": "brokerId",
           "about": "The set of nodes that are in sync with the leader for this partition." },
-        { "name": "OfflineReplicas", "type": "[]int32", "versions": "5+", "ignorable": true,
+        { "name": "OfflineReplicas", "type": "[]int32", "versions": "5+", "ignorable": true, "entityType": "brokerId",
           "about": "The set of offline replicas of this partition." }
       ]},
       { "name": "TopicAuthorizedOperations", "type": "int32", "versions": "8+", "default": "-2147483648",
diff --git a/clients/src/main/resources/common/message/OffsetForLeaderEpochRequest.json b/clients/src/main/resources/common/message/OffsetForLeaderEpochRequest.json
index 2440bec..6645ad2 100644
--- a/clients/src/main/resources/common/message/OffsetForLeaderEpochRequest.json
+++ b/clients/src/main/resources/common/message/OffsetForLeaderEpochRequest.json
@@ -30,7 +30,7 @@
   "validVersions": "0-4",
   "flexibleVersions": "4+",
   "fields": [
-    { "name": "ReplicaId", "type": "int32", "versions": "3+", "default": -2, "ignorable": true,
+    { "name": "ReplicaId", "type": "int32", "versions": "3+", "default": -2, "ignorable": true, "entityType": "brokerId",
       "about": "The broker ID of the follower, of -1 if this request is from a consumer." },
     { "name": "Topics", "type": "[]OffsetForLeaderTopic", "versions": "0+",
       "about": "Each topic to get offsets for.", "fields": [
diff --git a/clients/src/main/resources/common/message/UnregisterBrokerRequest.json b/clients/src/main/resources/common/message/UnregisterBrokerRequest.json
index ef72bfe..05fd315 100644
--- a/clients/src/main/resources/common/message/UnregisterBrokerRequest.json
+++ b/clients/src/main/resources/common/message/UnregisterBrokerRequest.json
@@ -21,7 +21,7 @@
   "validVersions": "0",
   "flexibleVersions": "0+",
   "fields": [
-    { "name": "BrokerId", "type": "int32", "versions": "0+",
+    { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
       "about": "The broker ID to unregister." }
   ]
 }
diff --git a/clients/src/main/resources/common/message/UpdateFeaturesResponse.json b/clients/src/main/resources/common/message/UpdateFeaturesResponse.json
index 615f617..63e84ff 100644
--- a/clients/src/main/resources/common/message/UpdateFeaturesResponse.json
+++ b/clients/src/main/resources/common/message/UpdateFeaturesResponse.json
@@ -28,7 +28,7 @@
       "about": "The top-level error message, or `null` if there was no top-level error." },
     { "name": "Results", "type": "[]UpdatableFeatureResult", "versions": "0+",
       "about": "Results for each feature update.", "fields": [
-      {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true,
+      { "name": "Feature", "type": "string", "versions": "0+", "mapKey": true,
         "about": "The name of the finalized feature."},
       { "name": "ErrorCode", "type": "int16", "versions": "0+",
         "about": "The feature update error code or `0` if the feature update succeeded." },
diff --git a/clients/src/main/resources/common/message/VoteRequest.json b/clients/src/main/resources/common/message/VoteRequest.json
index fcc0017..35583a7 100644
--- a/clients/src/main/resources/common/message/VoteRequest.json
+++ b/clients/src/main/resources/common/message/VoteRequest.json
@@ -33,7 +33,7 @@
           "about": "The partition index." },
         { "name": "CandidateEpoch", "type": "int32", "versions": "0+",
           "about": "The bumped epoch of the candidate sending the request"},
-        { "name": "CandidateId", "type": "int32", "versions": "0+",
+        { "name": "CandidateId", "type": "int32", "versions": "0+", "entityType": "brokerId",
           "about": "The ID of the voter sending the request"},
         { "name": "LastOffsetEpoch", "type": "int32", "versions": "0+",
           "about": "The epoch of the last record written to the metadata log"},
diff --git a/clients/src/main/resources/common/message/VoteResponse.json b/clients/src/main/resources/common/message/VoteResponse.json
index 83d1a8d..b92d007 100644
--- a/clients/src/main/resources/common/message/VoteResponse.json
+++ b/clients/src/main/resources/common/message/VoteResponse.json
@@ -31,7 +31,7 @@
         { "name": "PartitionIndex", "type": "int32", "versions": "0+",
           "about": "The partition index." },
         { "name": "ErrorCode", "type": "int16", "versions": "0+"},
-        { "name": "LeaderId", "type": "int32", "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"},
diff --git a/metadata/src/main/resources/common/metadata/FenceBrokerRecord.json b/metadata/src/main/resources/common/metadata/FenceBrokerRecord.json
index d45ba4e..e8be19e 100644
--- a/metadata/src/main/resources/common/metadata/FenceBrokerRecord.json
+++ b/metadata/src/main/resources/common/metadata/FenceBrokerRecord.json
@@ -19,7 +19,7 @@
   "name": "FenceBrokerRecord",
   "validVersions": "0",
   "fields": [
-    { "name": "Id", "type": "int32", "versions": "0+",
+    { "name": "Id", "type": "int32", "versions": "0+", "entityType": "brokerId",
       "about": "The broker ID to fence. It will be removed from all ISRs." },
 	{ "name": "Epoch", "type": "int64", "versions": "0+",
       "about": "The epoch of the broker to fence." }
diff --git a/metadata/src/main/resources/common/metadata/PartitionChangeRecord.json b/metadata/src/main/resources/common/metadata/PartitionChangeRecord.json
index 070e5c9..ee9809f 100644
--- a/metadata/src/main/resources/common/metadata/PartitionChangeRecord.json
+++ b/metadata/src/main/resources/common/metadata/PartitionChangeRecord.json
@@ -24,10 +24,10 @@
       "about": "The partition id." },
     { "name": "TopicId", "type": "uuid", "versions": "0+",
       "about": "The unique ID of this topic." },
-    { "name": "Isr", "type":  "[]int32", "default": "null",
-      "versions": "0+", "nullableVersions":  "0+", "taggedVersions": "0+", "tag": 0,
+    { "name": "Isr", "type":  "[]int32", "default": "null", "entityType": "brokerId",
+      "versions": "0+", "nullableVersions": "0+", "taggedVersions": "0+", "tag": 0,
       "about": "null if the ISR didn't change; the new in-sync replicas otherwise." },
-    { "name": "Leader", "type": "int32", "default": "-2",
+    { "name": "Leader", "type": "int32", "default": "-2", "entityType": "brokerId",
       "versions": "0+", "taggedVersions": "0+", "tag": 1,
       "about": "-1 if there is now no leader; -2 if the leader didn't change; the new leader otherwise." }
   ]
diff --git a/metadata/src/main/resources/common/metadata/PartitionRecord.json b/metadata/src/main/resources/common/metadata/PartitionRecord.json
index 2a92c21..4dc924a 100644
--- a/metadata/src/main/resources/common/metadata/PartitionRecord.json
+++ b/metadata/src/main/resources/common/metadata/PartitionRecord.json
@@ -23,15 +23,15 @@
       "about": "The partition id." },
     { "name": "TopicId", "type": "uuid", "versions": "0+",
       "about": "The unique ID of this topic." },
-    { "name": "Replicas", "type":  "[]int32", "versions":  "0+",
+    { "name": "Replicas", "type":  "[]int32", "versions":  "0+", "entityType": "brokerId",
       "about": "The replicas of this partition, sorted by preferred order." },
     { "name": "Isr", "type":  "[]int32", "versions":  "0+",
       "about": "The in-sync replicas of this partition" },
-    { "name": "RemovingReplicas", "type":  "[]int32", "versions":  "0+", "nullableVersions": "0+",
+    { "name": "RemovingReplicas", "type":  "[]int32", "versions":  "0+", "nullableVersions": "0+", "entityType": "brokerId",
       "about": "The replicas that we are in the process of removing." },
-    { "name": "AddingReplicas", "type":  "[]int32", "versions":  "0+", "nullableVersions": "0+",
+    { "name": "AddingReplicas", "type":  "[]int32", "versions":  "0+", "nullableVersions": "0+", "entityType": "brokerId",
       "about": "The replicas that we are in the process of adding." },
-    { "name": "Leader", "type": "int32", "versions": "0+", "default": "-1",
+    { "name": "Leader", "type": "int32", "versions": "0+", "default": "-1", "entityType": "brokerId",
       "about": "The lead replica, or -1 if there is no leader." },
     { "name": "LeaderEpoch", "type": "int32", "versions": "0+", "default": "-1",
       "about": "The epoch of the partition leader." },
diff --git a/metadata/src/main/resources/common/metadata/RegisterBrokerRecord.json b/metadata/src/main/resources/common/metadata/RegisterBrokerRecord.json
index 16e724b..b457093 100644
--- a/metadata/src/main/resources/common/metadata/RegisterBrokerRecord.json
+++ b/metadata/src/main/resources/common/metadata/RegisterBrokerRecord.json
@@ -19,7 +19,7 @@
   "name": "RegisterBrokerRecord",
   "validVersions": "0",
   "fields": [
-    { "name": "BrokerId", "type": "int32", "versions": "0+",
+    { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
       "about": "The broker id." },
     { "name": "IncarnationId", "type": "uuid", "versions": "0+",
       "about": "The incarnation ID of the broker process" },
diff --git a/metadata/src/main/resources/common/metadata/TopicRecord.json b/metadata/src/main/resources/common/metadata/TopicRecord.json
index 64538fb..dae550f 100644
--- a/metadata/src/main/resources/common/metadata/TopicRecord.json
+++ b/metadata/src/main/resources/common/metadata/TopicRecord.json
@@ -19,7 +19,7 @@
   "name": "TopicRecord",
   "validVersions": "0",
   "fields": [
-    { "name": "Name", "type": "string", "versions": "0+",
+    { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName",
       "about": "The topic name." },
     { "name": "TopicId", "type": "uuid", "versions": "0+",
       "about": "The unique ID of this topic." }
diff --git a/metadata/src/main/resources/common/metadata/UnfenceBrokerRecord.json b/metadata/src/main/resources/common/metadata/UnfenceBrokerRecord.json
index 1e41f7f..23ed22a 100644
--- a/metadata/src/main/resources/common/metadata/UnfenceBrokerRecord.json
+++ b/metadata/src/main/resources/common/metadata/UnfenceBrokerRecord.json
@@ -19,7 +19,7 @@
   "name": "UnfenceBrokerRecord",
   "validVersions": "0",
   "fields": [
-    { "name": "Id", "type": "int32", "versions": "0+",
+    { "name": "Id", "type": "int32", "versions": "0+", "entityType": "brokerId",
       "about": "The broker ID to unfence." },
     { "name": "Epoch", "type": "int64", "versions": "0+",
       "about": "The epoch of the broker to unfence." }
diff --git a/metadata/src/main/resources/common/metadata/UnregisterBrokerRecord.json b/metadata/src/main/resources/common/metadata/UnregisterBrokerRecord.json
index e2cbf57..5f5989af 100644
--- a/metadata/src/main/resources/common/metadata/UnregisterBrokerRecord.json
+++ b/metadata/src/main/resources/common/metadata/UnregisterBrokerRecord.json
@@ -19,7 +19,7 @@
   "name": "UnregisterBrokerRecord",
   "validVersions": "0",
   "fields": [
-    { "name": "BrokerId", "type": "int32", "versions": "0+",
+    { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
       "about": "The broker id." },
     { "name": "BrokerEpoch", "type": "int64", "versions": "0+",
       "about": "The broker epoch." }