message.go 5.0 KB

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