message.go 4.0 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175
  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. //Message is a kafka message type
  38. type Message struct {
  39. Codec CompressionCodec // codec used to compress the message contents
  40. CompressionLevel int // compression level
  41. LogAppendTime bool // the used timestamp is LogAppendTime
  42. Key []byte // the message key, may be nil
  43. Value []byte // the message contents
  44. Set *MessageSet // the message set a message might wrap
  45. Version int8 // v1 requires Kafka 0.10
  46. Timestamp time.Time // the timestamp of the message (version 1+ only)
  47. compressedCache []byte
  48. compressedSize int // used for computing the compression ratio metrics
  49. }
  50. func (m *Message) encode(pe packetEncoder) error {
  51. pe.push(newCRC32Field(crcIEEE))
  52. pe.putInt8(m.Version)
  53. attributes := int8(m.Codec) & compressionCodecMask
  54. if m.LogAppendTime {
  55. attributes |= timestampTypeMask
  56. }
  57. pe.putInt8(attributes)
  58. if m.Version >= 1 {
  59. if err := (Timestamp{&m.Timestamp}).encode(pe); err != nil {
  60. return err
  61. }
  62. }
  63. err := pe.putBytes(m.Key)
  64. if err != nil {
  65. return err
  66. }
  67. var payload []byte
  68. if m.compressedCache != nil {
  69. payload = m.compressedCache
  70. m.compressedCache = nil
  71. } else if m.Value != nil {
  72. payload, err = compress(m.Codec, m.CompressionLevel, m.Value)
  73. if err != nil {
  74. return err
  75. }
  76. m.compressedCache = payload
  77. // Keep in mind the compressed payload size for metric gathering
  78. m.compressedSize = len(payload)
  79. }
  80. if err = pe.putBytes(payload); err != nil {
  81. return err
  82. }
  83. return pe.pop()
  84. }
  85. func (m *Message) decode(pd packetDecoder) (err error) {
  86. crc32Decoder := acquireCrc32Field(crcIEEE)
  87. defer releaseCrc32Field(crc32Decoder)
  88. err = pd.push(crc32Decoder)
  89. if err != nil {
  90. return err
  91. }
  92. m.Version, err = pd.getInt8()
  93. if err != nil {
  94. return err
  95. }
  96. if m.Version > 1 {
  97. return PacketDecodingError{fmt.Sprintf("unknown magic byte (%v)", m.Version)}
  98. }
  99. attribute, err := pd.getInt8()
  100. if err != nil {
  101. return err
  102. }
  103. m.Codec = CompressionCodec(attribute & compressionCodecMask)
  104. m.LogAppendTime = attribute&timestampTypeMask == timestampTypeMask
  105. if m.Version == 1 {
  106. if err := (Timestamp{&m.Timestamp}).decode(pd); err != nil {
  107. return err
  108. }
  109. }
  110. m.Key, err = pd.getBytes()
  111. if err != nil {
  112. return err
  113. }
  114. m.Value, err = pd.getBytes()
  115. if err != nil {
  116. return err
  117. }
  118. // Required for deep equal assertion during tests but might be useful
  119. // for future metrics about the compression ratio in fetch requests
  120. m.compressedSize = len(m.Value)
  121. switch m.Codec {
  122. case CompressionNone:
  123. // nothing to do
  124. default:
  125. if m.Value == nil {
  126. break
  127. }
  128. m.Value, err = decompress(m.Codec, m.Value)
  129. if err != nil {
  130. return err
  131. }
  132. if err := m.decodeSet(); err != nil {
  133. return err
  134. }
  135. }
  136. return pd.pop()
  137. }
  138. // decodes a message set from a previously encoded bulk-message
  139. func (m *Message) decodeSet() (err error) {
  140. pd := realDecoder{raw: m.Value}
  141. m.Set = &MessageSet{}
  142. return m.Set.decode(&pd)
  143. }