message.go 5.5 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239
  1. package sarama
  2. import (
  3. "bytes"
  4. "compress/gzip"
  5. "fmt"
  6. "io/ioutil"
  7. "time"
  8. "github.com/eapache/go-xerial-snappy"
  9. "github.com/pierrec/lz4"
  10. )
  11. // CompressionCodec represents the various compression codecs recognized by Kafka in messages.
  12. type CompressionCodec int8
  13. // The lowest 3 bits contain the compression codec used for the message
  14. const compressionCodecMask int8 = 0x07
  15. const (
  16. CompressionNone CompressionCodec = 0
  17. CompressionGZIP CompressionCodec = 1
  18. CompressionSnappy CompressionCodec = 2
  19. CompressionLZ4 CompressionCodec = 3
  20. CompressionZSTD CompressionCodec = 4
  21. )
  22. func (cc CompressionCodec) String() string {
  23. return []string{
  24. "none",
  25. "gzip",
  26. "snappy",
  27. "lz4",
  28. }[int(cc)]
  29. }
  30. // CompressionLevelDefault is the constant to use in CompressionLevel
  31. // to have the default compression level for any codec. The value is picked
  32. // that we don't use any existing compression levels.
  33. const CompressionLevelDefault = -1000
  34. type Message struct {
  35. Codec CompressionCodec // codec used to compress the message contents
  36. CompressionLevel int // compression level
  37. Key []byte // the message key, may be nil
  38. Value []byte // the message contents
  39. Set *MessageSet // the message set a message might wrap
  40. Version int8 // v1 requires Kafka 0.10
  41. Timestamp time.Time // the timestamp of the message (version 1+ only)
  42. compressedCache []byte
  43. compressedSize int // used for computing the compression ratio metrics
  44. }
  45. func (m *Message) encode(pe packetEncoder) error {
  46. pe.push(newCRC32Field(crcIEEE))
  47. pe.putInt8(m.Version)
  48. attributes := int8(m.Codec) & compressionCodecMask
  49. pe.putInt8(attributes)
  50. if m.Version >= 1 {
  51. if err := (Timestamp{&m.Timestamp}).encode(pe); err != nil {
  52. return err
  53. }
  54. }
  55. err := pe.putBytes(m.Key)
  56. if err != nil {
  57. return err
  58. }
  59. var payload []byte
  60. if m.compressedCache != nil {
  61. payload = m.compressedCache
  62. m.compressedCache = nil
  63. } else if m.Value != nil {
  64. switch m.Codec {
  65. case CompressionNone:
  66. payload = m.Value
  67. case CompressionGZIP:
  68. var buf bytes.Buffer
  69. var writer *gzip.Writer
  70. if m.CompressionLevel != CompressionLevelDefault {
  71. writer, err = gzip.NewWriterLevel(&buf, m.CompressionLevel)
  72. if err != nil {
  73. return err
  74. }
  75. } else {
  76. writer = gzip.NewWriter(&buf)
  77. }
  78. if _, err = writer.Write(m.Value); err != nil {
  79. return err
  80. }
  81. if err = writer.Close(); err != nil {
  82. return err
  83. }
  84. m.compressedCache = buf.Bytes()
  85. payload = m.compressedCache
  86. case CompressionSnappy:
  87. tmp := snappy.Encode(m.Value)
  88. m.compressedCache = tmp
  89. payload = m.compressedCache
  90. case CompressionLZ4:
  91. var buf bytes.Buffer
  92. writer := lz4.NewWriter(&buf)
  93. if _, err = writer.Write(m.Value); err != nil {
  94. return err
  95. }
  96. if err = writer.Close(); err != nil {
  97. return err
  98. }
  99. m.compressedCache = buf.Bytes()
  100. payload = m.compressedCache
  101. case CompressionZSTD:
  102. c, err := zstdCompressLevel(nil, m.Value, m.CompressionLevel)
  103. if err != nil {
  104. return err
  105. }
  106. m.compressedCache = c
  107. payload = m.compressedCache
  108. default:
  109. return PacketEncodingError{fmt.Sprintf("unsupported compression codec (%d)", m.Codec)}
  110. }
  111. // Keep in mind the compressed payload size for metric gathering
  112. m.compressedSize = len(payload)
  113. }
  114. if err = pe.putBytes(payload); err != nil {
  115. return err
  116. }
  117. return pe.pop()
  118. }
  119. func (m *Message) decode(pd packetDecoder) (err error) {
  120. err = pd.push(newCRC32Field(crcIEEE))
  121. if err != nil {
  122. return err
  123. }
  124. m.Version, err = pd.getInt8()
  125. if err != nil {
  126. return err
  127. }
  128. if m.Version > 1 {
  129. return PacketDecodingError{fmt.Sprintf("unknown magic byte (%v)", m.Version)}
  130. }
  131. attribute, err := pd.getInt8()
  132. if err != nil {
  133. return err
  134. }
  135. m.Codec = CompressionCodec(attribute & compressionCodecMask)
  136. if m.Version == 1 {
  137. if err := (Timestamp{&m.Timestamp}).decode(pd); err != nil {
  138. return err
  139. }
  140. }
  141. m.Key, err = pd.getBytes()
  142. if err != nil {
  143. return err
  144. }
  145. m.Value, err = pd.getBytes()
  146. if err != nil {
  147. return err
  148. }
  149. // Required for deep equal assertion during tests but might be useful
  150. // for future metrics about the compression ratio in fetch requests
  151. m.compressedSize = len(m.Value)
  152. switch m.Codec {
  153. case CompressionNone:
  154. // nothing to do
  155. case CompressionGZIP:
  156. if m.Value == nil {
  157. break
  158. }
  159. reader, err := gzip.NewReader(bytes.NewReader(m.Value))
  160. if err != nil {
  161. return err
  162. }
  163. if m.Value, err = ioutil.ReadAll(reader); err != nil {
  164. return err
  165. }
  166. if err := m.decodeSet(); err != nil {
  167. return err
  168. }
  169. case CompressionSnappy:
  170. if m.Value == nil {
  171. break
  172. }
  173. if m.Value, err = snappy.Decode(m.Value); err != nil {
  174. return err
  175. }
  176. if err := m.decodeSet(); err != nil {
  177. return err
  178. }
  179. case CompressionLZ4:
  180. if m.Value == nil {
  181. break
  182. }
  183. reader := lz4.NewReader(bytes.NewReader(m.Value))
  184. if m.Value, err = ioutil.ReadAll(reader); err != nil {
  185. return err
  186. }
  187. if err := m.decodeSet(); err != nil {
  188. return err
  189. }
  190. case CompressionZSTD:
  191. if m.Value == nil {
  192. break
  193. }
  194. if m.Value, err = zstdDecompress(nil, m.Value); err != nil {
  195. return err
  196. }
  197. if err := m.decodeSet(); err != nil {
  198. return err
  199. }
  200. default:
  201. return PacketDecodingError{fmt.Sprintf("invalid compression specified (%d)", m.Codec)}
  202. }
  203. return pd.pop()
  204. }
  205. // decodes a message set from a previousy encoded bulk-message
  206. func (m *Message) decodeSet() (err error) {
  207. pd := realDecoder{raw: m.Value}
  208. m.Set = &MessageSet{}
  209. return m.Set.decode(&pd)
  210. }