// OffsetCommitRequest commits offsets for consumed topics / partitions in // a group. OffsetCommitRequest => key 8, max version 8, flexible v8+, group coordinator // Group is the group this request is committing offsets to. Group: string // Generation being -1 and group being empty means the group is being used // to store offsets only. No generation validation, no rebalancing. Generation: int32(-1) // v1+ // MemberID is the ID of the client issuing this request in the group. MemberID: string // v1+ // InstanceID is the instance ID of this member in the group (KIP-345). InstanceID: nullable-string // v7+ // RetentionTimeMillis is how long this commit will persist in Kafka. // // This was introduced in v2, replacing an individual topic/partition's // Timestamp from v1, and was removed in v5 with Kafka 2.1.0. // // This was removed because rarely committing consumers could have their // offsets expired before committing, even though the consumer was still // active. After restarting or rebalancing, the consumer would now not know // the last committed offset and would have to start at the beginning or end, // leading to duplicates or log loss. // // Post 2.1.0, if this field is empty, offsets are only deleted once the // group is empty. Read KIP-211 for more details. RetentionTimeMillis: int64(-1) // v2-v4 // Topics is contains topics and partitions for which to commit offsets. Topics: [=>] // Topic is a topic to commit offsets for. Topic: string // Partitions contains partitions in a topic for which to commit offsets. Partitions: [=>] // Partition if a partition to commit offsets for. Partition: int32 // Offset is an offset to commit. Offset: int64 // Timestamp is the first iteration of tracking how long offset commits // should persist in Kafka. This field only existed for v1. // The expiration would be timestamp + offset.retention.minutes, or, if // timestamp was zero, current time + offset.retention.minutes. Timestamp: int64(-1) // v1-v1 // LeaderEpoch, proposed in KIP-320 and introduced in Kafka 2.1.0, // is the leader epoch of the record this request is committing. // // The initial leader epoch can be determined from a MetadataResponse. // To skip log truncation checking, use -1. LeaderEpoch: int32(-1) // v6+ // Metadata is optional data to include with committing the offset. This // can contain information such as which node is doing the committing, etc. Metadata: nullable-string // OffsetCommitResponse is returned from an OffsetCommitRequest. OffsetCommitResponse => ThrottleMillis(4) // v3+ // Topics contains responses for each topic / partition in the commit request. Topics: [=>] // Topic is the topic this offset commit response corresponds to. Topic: string // Partitions contains responses for each requested partition in // a topic. Partitions: [=>] // Partition is the partition in a topic this array slot corresponds to. Partition: int32 // ErrorCode is the error for this partition response. // // GROUP_AUTHORIZATION_FAILED is returned if the client is not authorized // for the group. // // TOPIC_AUTHORIZATION_FAILED is returned if the client is not authorized // for the topic / partition. // // UNKNOWN_TOPIC_OR_PARTITION is returned if the topic / partition does // not exist. // // OFFSET_METADATA_TOO_LARGE is returned if the request metadata is // larger than the brokers offset.metadata.max.bytes. // // INVALID_GROUP_ID is returned in the requested group ID is invalid. // // COORDINATOR_NOT_AVAILABLE is returned if the coordinator is not available // (due to the requested broker shutting down or it has not completed startup). // // COORDINATOR_LOAD_IN_PROGRESS is returned if the group is loading. // // NOT_COORDINATOR is returned if the requested broker is not the coordinator // for the requested group. // // ILLEGAL_GENERATION is returned if the request's generation ID is invalid. // // UNKNOWN_MEMBER_ID is returned if the group is dead or the group does not // know of the request's member ID. // // REBALANCE_IN_PROGRESS is returned if the group is finishing a rebalance. // // INVALID_COMMIT_OFFSET_SIZE is returned if the offset commit results in // a record batch that is too large (likely due to large metadata). ErrorCode: int16