message.go 4.5 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190
  1. package sarama
  2. import (
  3. "fmt"
  4. "time"
  5. )
  6. const (
  7. // CompressionNone no compression
  8. CompressionNone CompressionCodec = iota
  9. // CompressionGZIP compression using GZIP
  10. CompressionGZIP
  11. // CompressionSnappy compression using snappy
  12. CompressionSnappy
  13. // CompressionLZ4 compression using LZ4
  14. CompressionLZ4
  15. // CompressionZSTD compression using ZSTD
  16. CompressionZSTD
  17. // The lowest 3 bits contain the compression codec used for the message
  18. compressionCodecMask int8 = 0x07
  19. // Bit 3 set for "LogAppend" timestamps
  20. timestampTypeMask = 0x08
  21. // CompressionLevelDefault is the constant to use in CompressionLevel
  22. // to have the default compression level for any codec. The value is picked
  23. // that we don't use any existing compression levels.
  24. CompressionLevelDefault = -1000
  25. )
  26. // CompressionCodec represents the various compression codecs recognized by Kafka in messages.
  27. type CompressionCodec int8
  28. func (cc CompressionCodec) String() string {
  29. return []string{
  30. "none",
  31. "gzip",
  32. "snappy",
  33. "lz4",
  34. "zstd",
  35. }[int(cc)]
  36. }
  37. // UnmarshalText returns a CompressionCodec from its string representation.
  38. func (cc *CompressionCodec) UnmarshalText(text []byte) error {
  39. codecs := map[string]CompressionCodec{
  40. "none": CompressionNone,
  41. "gzip": CompressionGZIP,
  42. "snappy": CompressionSnappy,
  43. "lz4": CompressionLZ4,
  44. "zstd": CompressionZSTD,
  45. }
  46. codec, ok := codecs[string(text)]
  47. if !ok {
  48. return fmt.Errorf("cannot parse %q as a compression codec", string(text))
  49. }
  50. *cc = codec
  51. return nil
  52. }
  53. // MarshalText transforms a CompressionCodec into its string representation.
  54. func (cc CompressionCodec) MarshalText() ([]byte, error) {
  55. return []byte(cc.String()), nil
  56. }
  57. // Message is a kafka message type
  58. type Message struct {
  59. Codec CompressionCodec // codec used to compress the message contents
  60. CompressionLevel int // compression level
  61. LogAppendTime bool // the used timestamp is LogAppendTime
  62. Key []byte // the message key, may be nil
  63. Value []byte // the message contents
  64. Set *MessageSet // the message set a message might wrap
  65. Version int8 // v1 requires Kafka 0.10
  66. Timestamp time.Time // the timestamp of the message (version 1+ only)
  67. compressedCache []byte
  68. compressedSize int // used for computing the compression ratio metrics
  69. }
  70. func (m *Message) encode(pe packetEncoder) error {
  71. pe.push(newCRC32Field(crcIEEE))
  72. pe.putInt8(m.Version)
  73. attributes := int8(m.Codec) & compressionCodecMask
  74. if m.LogAppendTime {
  75. attributes |= timestampTypeMask
  76. }
  77. pe.putInt8(attributes)
  78. if m.Version >= 1 {
  79. if err := (Timestamp{&m.Timestamp}).encode(pe); err != nil {
  80. return err
  81. }
  82. }
  83. err := pe.putBytes(m.Key)
  84. if err != nil {
  85. return err
  86. }
  87. var payload []byte
  88. if m.compressedCache != nil {
  89. payload = m.compressedCache
  90. m.compressedCache = nil
  91. } else if m.Value != nil {
  92. payload, err = compress(m.Codec, m.CompressionLevel, m.Value)
  93. if err != nil {
  94. return err
  95. }
  96. m.compressedCache = payload
  97. // Keep in mind the compressed payload size for metric gathering
  98. m.compressedSize = len(payload)
  99. }
  100. if err = pe.putBytes(payload); err != nil {
  101. return err
  102. }
  103. return pe.pop()
  104. }
  105. func (m *Message) decode(pd packetDecoder) (err error) {
  106. crc32Decoder := acquireCrc32Field(crcIEEE)
  107. defer releaseCrc32Field(crc32Decoder)
  108. err = pd.push(crc32Decoder)
  109. if err != nil {
  110. return err
  111. }
  112. m.Version, err = pd.getInt8()
  113. if err != nil {
  114. return err
  115. }
  116. if m.Version > 1 {
  117. return PacketDecodingError{fmt.Sprintf("unknown magic byte (%v)", m.Version)}
  118. }
  119. attribute, err := pd.getInt8()
  120. if err != nil {
  121. return err
  122. }
  123. m.Codec = CompressionCodec(attribute & compressionCodecMask)
  124. m.LogAppendTime = attribute&timestampTypeMask == timestampTypeMask
  125. if m.Version == 1 {
  126. if err := (Timestamp{&m.Timestamp}).decode(pd); err != nil {
  127. return err
  128. }
  129. }
  130. m.Key, err = pd.getBytes()
  131. if err != nil {
  132. return err
  133. }
  134. m.Value, err = pd.getBytes()
  135. if err != nil {
  136. return err
  137. }
  138. // Required for deep equal assertion during tests but might be useful
  139. // for future metrics about the compression ratio in fetch requests
  140. m.compressedSize = len(m.Value)
  141. if m.Value != nil && m.Codec != CompressionNone {
  142. m.Value, err = decompress(m.Codec, m.Value)
  143. if err != nil {
  144. return err
  145. }
  146. if err := m.decodeSet(); err != nil {
  147. return err
  148. }
  149. }
  150. return pd.pop()
  151. }
  152. // decodes a message set from a previously encoded bulk-message
  153. func (m *Message) decodeSet() (err error) {
  154. pd := realDecoder{raw: m.Value}
  155. m.Set = &MessageSet{}
  156. return m.Set.decode(&pd)
  157. }