...

Text file src/github.com/twmb/franz-go/generate/definitions/08_offset_commit

Documentation: github.com/twmb/franz-go/generate/definitions

     1// OffsetCommitRequest commits offsets for consumed topics / partitions in
     2// a group.
     3OffsetCommitRequest => key 8, max version 8, flexible v8+, group coordinator
     4  // Group is the group this request is committing offsets to.
     5  Group: string
     6  // Generation being -1 and group being empty means the group is being used
     7  // to store offsets only. No generation validation, no rebalancing.
     8  Generation: int32(-1) // v1+
     9  // MemberID is the ID of the client issuing this request in the group.
    10  MemberID: string // v1+
    11  // InstanceID is the instance ID of this member in the group (KIP-345).
    12  InstanceID: nullable-string // v7+
    13  // RetentionTimeMillis is how long this commit will persist in Kafka.
    14  //
    15  // This was introduced in v2, replacing an individual topic/partition's
    16  // Timestamp from v1, and was removed in v5 with Kafka 2.1.0.
    17  //
    18  // This was removed because rarely committing consumers could have their
    19  // offsets expired before committing, even though the consumer was still
    20  // active. After restarting or rebalancing, the consumer would now not know
    21  // the last committed offset and would have to start at the beginning or end,
    22  // leading to duplicates or log loss.
    23  //
    24  // Post 2.1.0, if this field is empty, offsets are only deleted once the
    25  // group is empty. Read KIP-211 for more details.
    26  RetentionTimeMillis: int64(-1) // v2-v4
    27  // Topics is contains topics and partitions for which to commit offsets.
    28  Topics: [=>]
    29    // Topic is a topic to commit offsets for.
    30    Topic: string
    31    // Partitions contains partitions in a topic for which to commit offsets.
    32    Partitions: [=>]
    33      // Partition if a partition to commit offsets for.
    34      Partition: int32
    35      // Offset is an offset to commit.
    36      Offset: int64
    37      // Timestamp is the first iteration of tracking how long offset commits
    38      // should persist in Kafka. This field only existed for v1.
    39      // The expiration would be timestamp + offset.retention.minutes, or, if
    40      // timestamp was zero, current time + offset.retention.minutes.
    41      Timestamp: int64(-1) // v1-v1
    42      // LeaderEpoch, proposed in KIP-320 and introduced in Kafka 2.1.0,
    43      // is the leader epoch of the record this request is committing.
    44      //
    45      // The initial leader epoch can be determined from a MetadataResponse.
    46      // To skip log truncation checking, use -1.
    47      LeaderEpoch: int32(-1) // v6+
    48      // Metadata is optional data to include with committing the offset. This
    49      // can contain information such as which node is doing the committing, etc.
    50      Metadata: nullable-string
    51
    52// OffsetCommitResponse is returned from an OffsetCommitRequest.
    53OffsetCommitResponse =>
    54  ThrottleMillis(4) // v3+
    55  // Topics contains responses for each topic / partition in the commit request.
    56  Topics: [=>]
    57    // Topic is the topic this offset commit response corresponds to.
    58    Topic: string
    59    // Partitions contains responses for each requested partition in
    60    // a topic.
    61    Partitions: [=>]
    62      // Partition is the partition in a topic this array slot corresponds to.
    63      Partition: int32
    64      // ErrorCode is the error for this partition response.
    65      //
    66      // GROUP_AUTHORIZATION_FAILED is returned if the client is not authorized
    67      // for the group.
    68      //
    69      // TOPIC_AUTHORIZATION_FAILED is returned if the client is not authorized
    70      // for the topic / partition.
    71      //
    72      // UNKNOWN_TOPIC_OR_PARTITION is returned if the topic / partition does
    73      // not exist.
    74      //
    75      // OFFSET_METADATA_TOO_LARGE is returned if the request metadata is
    76      // larger than the brokers offset.metadata.max.bytes.
    77      //
    78      // INVALID_GROUP_ID is returned in the requested group ID is invalid.
    79      //
    80      // COORDINATOR_NOT_AVAILABLE is returned if the coordinator is not available
    81      // (due to the requested broker shutting down or it has not completed startup).
    82      //
    83      // COORDINATOR_LOAD_IN_PROGRESS is returned if the group is loading.
    84      //
    85      // NOT_COORDINATOR is returned if the requested broker is not the coordinator
    86      // for the requested group.
    87      //
    88      // ILLEGAL_GENERATION is returned if the request's generation ID is invalid.
    89      //
    90      // UNKNOWN_MEMBER_ID is returned if the group is dead or the group does not
    91      // know of the request's member ID.
    92      //
    93      // REBALANCE_IN_PROGRESS is returned if the group is finishing a rebalance.
    94      //
    95      // INVALID_COMMIT_OFFSET_SIZE is returned if the offset commit results in
    96      // a record batch that is too large (likely due to large metadata).
    97      ErrorCode: int16

View as plain text