offset_commit_request.go 6.4 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253
  1. package sarama
  2. import "errors"
  3. // ReceiveTime is a special value for the timestamp field of Offset Commit Requests which
  4. // tells the broker to set the timestamp to the time at which the request was received.
  5. // The timestamp is only used if message version 1 is used, which requires kafka 0.8.2.
  6. const ReceiveTime int64 = -1
  7. // GroupGenerationUndefined is a special value for the group generation field of
  8. // Offset Commit Requests that should be used when a consumer group does not rely
  9. // on Kafka for partition management.
  10. const GroupGenerationUndefined = -1
  11. type offsetCommitRequestBlock struct {
  12. offset int64
  13. timestamp int64
  14. committedLeaderEpoch int32
  15. metadata string
  16. }
  17. func (b *offsetCommitRequestBlock) encode(pe packetEncoder, version int16) error {
  18. pe.putInt64(b.offset)
  19. if version == 1 {
  20. pe.putInt64(b.timestamp)
  21. } else if b.timestamp != 0 {
  22. Logger.Println("Non-zero timestamp specified for OffsetCommitRequest not v1, it will be ignored")
  23. }
  24. if version >= 6 {
  25. pe.putInt32(b.committedLeaderEpoch)
  26. }
  27. return pe.putString(b.metadata)
  28. }
  29. func (b *offsetCommitRequestBlock) decode(pd packetDecoder, version int16) (err error) {
  30. if b.offset, err = pd.getInt64(); err != nil {
  31. return err
  32. }
  33. if version == 1 {
  34. if b.timestamp, err = pd.getInt64(); err != nil {
  35. return err
  36. }
  37. }
  38. if version >= 6 {
  39. if b.committedLeaderEpoch, err = pd.getInt32(); err != nil {
  40. return err
  41. }
  42. }
  43. b.metadata, err = pd.getString()
  44. return err
  45. }
  46. type OffsetCommitRequest struct {
  47. ConsumerGroup string
  48. ConsumerGroupGeneration int32 // v1 or later
  49. ConsumerID string // v1 or later
  50. GroupInstanceId *string // v7 or later
  51. RetentionTime int64 // v2 or later
  52. // Version can be:
  53. // - 0 (kafka 0.8.1 and later)
  54. // - 1 (kafka 0.8.2 and later)
  55. // - 2 (kafka 0.9.0 and later)
  56. // - 3 (kafka 0.11.0 and later)
  57. // - 4 (kafka 2.0.0 and later)
  58. // - 5&6 (kafka 2.1.0 and later)
  59. // - 7 (kafka 2.3.0 and later)
  60. Version int16
  61. blocks map[string]map[int32]*offsetCommitRequestBlock
  62. }
  63. func (r *OffsetCommitRequest) encode(pe packetEncoder) error {
  64. if r.Version < 0 || r.Version > 7 {
  65. return PacketEncodingError{"invalid or unsupported OffsetCommitRequest version field"}
  66. }
  67. if err := pe.putString(r.ConsumerGroup); err != nil {
  68. return err
  69. }
  70. if r.Version >= 1 {
  71. pe.putInt32(r.ConsumerGroupGeneration)
  72. if err := pe.putString(r.ConsumerID); err != nil {
  73. return err
  74. }
  75. } else {
  76. if r.ConsumerGroupGeneration != 0 {
  77. Logger.Println("Non-zero ConsumerGroupGeneration specified for OffsetCommitRequest v0, it will be ignored")
  78. }
  79. if r.ConsumerID != "" {
  80. Logger.Println("Non-empty ConsumerID specified for OffsetCommitRequest v0, it will be ignored")
  81. }
  82. }
  83. // Version 5 removes RetentionTime, which is now controlled only by a broker configuration.
  84. if r.Version >= 2 && r.Version <= 4 {
  85. pe.putInt64(r.RetentionTime)
  86. } else if r.RetentionTime != 0 {
  87. Logger.Println("Non-zero RetentionTime specified for OffsetCommitRequest version <2, it will be ignored")
  88. }
  89. if r.Version >= 7 {
  90. if err := pe.putNullableString(r.GroupInstanceId); err != nil {
  91. return err
  92. }
  93. }
  94. if err := pe.putArrayLength(len(r.blocks)); err != nil {
  95. return err
  96. }
  97. for topic, partitions := range r.blocks {
  98. if err := pe.putString(topic); err != nil {
  99. return err
  100. }
  101. if err := pe.putArrayLength(len(partitions)); err != nil {
  102. return err
  103. }
  104. for partition, block := range partitions {
  105. pe.putInt32(partition)
  106. if err := block.encode(pe, r.Version); err != nil {
  107. return err
  108. }
  109. }
  110. }
  111. return nil
  112. }
  113. func (r *OffsetCommitRequest) decode(pd packetDecoder, version int16) (err error) {
  114. r.Version = version
  115. if r.ConsumerGroup, err = pd.getString(); err != nil {
  116. return err
  117. }
  118. if r.Version >= 1 {
  119. if r.ConsumerGroupGeneration, err = pd.getInt32(); err != nil {
  120. return err
  121. }
  122. if r.ConsumerID, err = pd.getString(); err != nil {
  123. return err
  124. }
  125. }
  126. // Version 5 removes RetentionTime, which is now controlled only by a broker configuration.
  127. if r.Version >= 2 && r.Version <= 4 {
  128. if r.RetentionTime, err = pd.getInt64(); err != nil {
  129. return err
  130. }
  131. }
  132. if r.Version >= 7 {
  133. if r.GroupInstanceId, err = pd.getNullableString(); err != nil {
  134. return err
  135. }
  136. }
  137. topicCount, err := pd.getArrayLength()
  138. if err != nil {
  139. return err
  140. }
  141. if topicCount == 0 {
  142. return nil
  143. }
  144. r.blocks = make(map[string]map[int32]*offsetCommitRequestBlock)
  145. for i := 0; i < topicCount; i++ {
  146. topic, err := pd.getString()
  147. if err != nil {
  148. return err
  149. }
  150. partitionCount, err := pd.getArrayLength()
  151. if err != nil {
  152. return err
  153. }
  154. r.blocks[topic] = make(map[int32]*offsetCommitRequestBlock)
  155. for j := 0; j < partitionCount; j++ {
  156. partition, err := pd.getInt32()
  157. if err != nil {
  158. return err
  159. }
  160. block := &offsetCommitRequestBlock{}
  161. if err := block.decode(pd, r.Version); err != nil {
  162. return err
  163. }
  164. r.blocks[topic][partition] = block
  165. }
  166. }
  167. return nil
  168. }
  169. func (r *OffsetCommitRequest) key() int16 {
  170. return 8
  171. }
  172. func (r *OffsetCommitRequest) version() int16 {
  173. return r.Version
  174. }
  175. func (r *OffsetCommitRequest) headerVersion() int16 {
  176. return 1
  177. }
  178. func (r *OffsetCommitRequest) isValidVersion() bool {
  179. return r.Version >= 0 && r.Version <= 7
  180. }
  181. func (r *OffsetCommitRequest) requiredVersion() KafkaVersion {
  182. switch r.Version {
  183. case 7:
  184. return V2_3_0_0
  185. case 5, 6:
  186. return V2_1_0_0
  187. case 4:
  188. return V2_0_0_0
  189. case 3:
  190. return V0_11_0_0
  191. case 2:
  192. return V0_9_0_0
  193. case 0, 1:
  194. return V0_8_2_0
  195. default:
  196. return V2_4_0_0
  197. }
  198. }
  199. func (r *OffsetCommitRequest) AddBlock(topic string, partitionID int32, offset int64, timestamp int64, metadata string) {
  200. r.AddBlockWithLeaderEpoch(topic, partitionID, offset, 0, timestamp, metadata)
  201. }
  202. func (r *OffsetCommitRequest) AddBlockWithLeaderEpoch(topic string, partitionID int32, offset int64, leaderEpoch int32, timestamp int64, metadata string) {
  203. if r.blocks == nil {
  204. r.blocks = make(map[string]map[int32]*offsetCommitRequestBlock)
  205. }
  206. if r.blocks[topic] == nil {
  207. r.blocks[topic] = make(map[int32]*offsetCommitRequestBlock)
  208. }
  209. r.blocks[topic][partitionID] = &offsetCommitRequestBlock{offset, timestamp, leaderEpoch, metadata}
  210. }
  211. func (r *OffsetCommitRequest) Offset(topic string, partitionID int32) (int64, string, error) {
  212. partitions := r.blocks[topic]
  213. if partitions == nil {
  214. return 0, "", errors.New("no such offset")
  215. }
  216. block := partitions[partitionID]
  217. if block == nil {
  218. return 0, "", errors.New("no such offset")
  219. }
  220. return block.offset, block.metadata, nil
  221. }