message.go 3.7 KB

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