offset_commit_request.go 5.3 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214
  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. metadata string
  15. }
  16. func (b *offsetCommitRequestBlock) encode(pe packetEncoder, version int16) error {
  17. pe.putInt64(b.offset)
  18. if version == 1 {
  19. pe.putInt64(b.timestamp)
  20. } else if b.timestamp != 0 {
  21. Logger.Println("Non-zero timestamp specified for OffsetCommitRequest not v1, it will be ignored")
  22. }
  23. return pe.putString(b.metadata)
  24. }
  25. func (b *offsetCommitRequestBlock) decode(pd packetDecoder, version int16) (err error) {
  26. if b.offset, err = pd.getInt64(); err != nil {
  27. return err
  28. }
  29. if version == 1 {
  30. if b.timestamp, err = pd.getInt64(); err != nil {
  31. return err
  32. }
  33. }
  34. b.metadata, err = pd.getString()
  35. return err
  36. }
  37. type OffsetCommitRequest struct {
  38. ConsumerGroup string
  39. ConsumerGroupGeneration int32 // v1 or later
  40. ConsumerID string // v1 or later
  41. RetentionTime int64 // v2 or later
  42. // Version can be:
  43. // - 0 (kafka 0.8.1 and later)
  44. // - 1 (kafka 0.8.2 and later)
  45. // - 2 (kafka 0.9.0 and later)
  46. // - 3 (kafka 0.11.0 and later)
  47. // - 4 (kafka 2.0.0 and later)
  48. Version int16
  49. blocks map[string]map[int32]*offsetCommitRequestBlock
  50. }
  51. func (r *OffsetCommitRequest) encode(pe packetEncoder) error {
  52. if r.Version < 0 || r.Version > 4 {
  53. return PacketEncodingError{"invalid or unsupported OffsetCommitRequest version field"}
  54. }
  55. if err := pe.putString(r.ConsumerGroup); err != nil {
  56. return err
  57. }
  58. if r.Version >= 1 {
  59. pe.putInt32(r.ConsumerGroupGeneration)
  60. if err := pe.putString(r.ConsumerID); err != nil {
  61. return err
  62. }
  63. } else {
  64. if r.ConsumerGroupGeneration != 0 {
  65. Logger.Println("Non-zero ConsumerGroupGeneration specified for OffsetCommitRequest v0, it will be ignored")
  66. }
  67. if r.ConsumerID != "" {
  68. Logger.Println("Non-empty ConsumerID specified for OffsetCommitRequest v0, it will be ignored")
  69. }
  70. }
  71. if r.Version >= 2 {
  72. pe.putInt64(r.RetentionTime)
  73. } else if r.RetentionTime != 0 {
  74. Logger.Println("Non-zero RetentionTime specified for OffsetCommitRequest version <2, it will be ignored")
  75. }
  76. if err := pe.putArrayLength(len(r.blocks)); err != nil {
  77. return err
  78. }
  79. for topic, partitions := range r.blocks {
  80. if err := pe.putString(topic); err != nil {
  81. return err
  82. }
  83. if err := pe.putArrayLength(len(partitions)); err != nil {
  84. return err
  85. }
  86. for partition, block := range partitions {
  87. pe.putInt32(partition)
  88. if err := block.encode(pe, r.Version); err != nil {
  89. return err
  90. }
  91. }
  92. }
  93. return nil
  94. }
  95. func (r *OffsetCommitRequest) decode(pd packetDecoder, version int16) (err error) {
  96. r.Version = version
  97. if r.ConsumerGroup, err = pd.getString(); err != nil {
  98. return err
  99. }
  100. if r.Version >= 1 {
  101. if r.ConsumerGroupGeneration, err = pd.getInt32(); err != nil {
  102. return err
  103. }
  104. if r.ConsumerID, err = pd.getString(); err != nil {
  105. return err
  106. }
  107. }
  108. if r.Version >= 2 {
  109. if r.RetentionTime, err = pd.getInt64(); err != nil {
  110. return err
  111. }
  112. }
  113. topicCount, err := pd.getArrayLength()
  114. if err != nil {
  115. return err
  116. }
  117. if topicCount == 0 {
  118. return nil
  119. }
  120. r.blocks = make(map[string]map[int32]*offsetCommitRequestBlock)
  121. for i := 0; i < topicCount; i++ {
  122. topic, err := pd.getString()
  123. if err != nil {
  124. return err
  125. }
  126. partitionCount, err := pd.getArrayLength()
  127. if err != nil {
  128. return err
  129. }
  130. r.blocks[topic] = make(map[int32]*offsetCommitRequestBlock)
  131. for j := 0; j < partitionCount; j++ {
  132. partition, err := pd.getInt32()
  133. if err != nil {
  134. return err
  135. }
  136. block := &offsetCommitRequestBlock{}
  137. if err := block.decode(pd, r.Version); err != nil {
  138. return err
  139. }
  140. r.blocks[topic][partition] = block
  141. }
  142. }
  143. return nil
  144. }
  145. func (r *OffsetCommitRequest) key() int16 {
  146. return 8
  147. }
  148. func (r *OffsetCommitRequest) version() int16 {
  149. return r.Version
  150. }
  151. func (r *OffsetCommitRequest) headerVersion() int16 {
  152. return 1
  153. }
  154. func (r *OffsetCommitRequest) requiredVersion() KafkaVersion {
  155. switch r.Version {
  156. case 1:
  157. return V0_8_2_0
  158. case 2:
  159. return V0_9_0_0
  160. case 3:
  161. return V0_11_0_0
  162. case 4:
  163. return V2_0_0_0
  164. default:
  165. return MinVersion
  166. }
  167. }
  168. func (r *OffsetCommitRequest) AddBlock(topic string, partitionID int32, offset int64, timestamp int64, metadata string) {
  169. if r.blocks == nil {
  170. r.blocks = make(map[string]map[int32]*offsetCommitRequestBlock)
  171. }
  172. if r.blocks[topic] == nil {
  173. r.blocks[topic] = make(map[int32]*offsetCommitRequestBlock)
  174. }
  175. r.blocks[topic][partitionID] = &offsetCommitRequestBlock{offset, timestamp, metadata}
  176. }
  177. func (r *OffsetCommitRequest) Offset(topic string, partitionID int32) (int64, string, error) {
  178. partitions := r.blocks[topic]
  179. if partitions == nil {
  180. return 0, "", errors.New("no such offset")
  181. }
  182. block := partitions[partitionID]
  183. if block == nil {
  184. return 0, "", errors.New("no such offset")
  185. }
  186. return block.offset, block.metadata, nil
  187. }