message.go 3.4 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163
  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. )
  10. // CompressionCodec represents the various compression codecs recognized by Kafka in messages.
  11. type CompressionCodec int8
  12. // only the last two bits are really used
  13. const compressionCodecMask int8 = 0x03
  14. const (
  15. CompressionNone CompressionCodec = 0
  16. CompressionGZIP CompressionCodec = 1
  17. CompressionSnappy CompressionCodec = 2
  18. )
  19. type Message struct {
  20. Codec CompressionCodec // codec used to compress the message contents
  21. Key []byte // the message key, may be nil
  22. Value []byte // the message contents
  23. Set *MessageSet // the message set a message might wrap
  24. Version int8 // v1 requires Kafka 0.10
  25. Timestamp time.Time // the timestamp of the message (version 1+ only)
  26. compressedCache []byte
  27. }
  28. func (m *Message) encode(pe packetEncoder) error {
  29. pe.push(&crc32Field{})
  30. pe.putInt8(m.Version)
  31. attributes := int8(m.Codec) & compressionCodecMask
  32. pe.putInt8(attributes)
  33. if m.Version >= 1 {
  34. pe.putInt64(m.Timestamp.UnixNano() / int64(time.Millisecond))
  35. }
  36. err := pe.putBytes(m.Key)
  37. if err != nil {
  38. return err
  39. }
  40. var payload []byte
  41. if m.compressedCache != nil {
  42. payload = m.compressedCache
  43. m.compressedCache = nil
  44. } else if m.Value != nil {
  45. switch m.Codec {
  46. case CompressionNone:
  47. payload = m.Value
  48. case CompressionGZIP:
  49. var buf bytes.Buffer
  50. writer := gzip.NewWriter(&buf)
  51. if _, err = writer.Write(m.Value); err != nil {
  52. return err
  53. }
  54. if err = writer.Close(); err != nil {
  55. return err
  56. }
  57. m.compressedCache = buf.Bytes()
  58. payload = m.compressedCache
  59. case CompressionSnappy:
  60. tmp := snappy.Encode(m.Value)
  61. m.compressedCache = tmp
  62. payload = m.compressedCache
  63. default:
  64. return PacketEncodingError{fmt.Sprintf("unsupported compression codec (%d)", m.Codec)}
  65. }
  66. }
  67. if err = pe.putBytes(payload); err != nil {
  68. return err
  69. }
  70. return pe.pop()
  71. }
  72. func (m *Message) decode(pd packetDecoder) (err error) {
  73. err = pd.push(&crc32Field{})
  74. if err != nil {
  75. return err
  76. }
  77. m.Version, err = pd.getInt8()
  78. if err != nil {
  79. return err
  80. }
  81. attribute, err := pd.getInt8()
  82. if err != nil {
  83. return err
  84. }
  85. m.Codec = CompressionCodec(attribute & compressionCodecMask)
  86. if m.Version >= 1 {
  87. millis, err := pd.getInt64()
  88. if err != nil {
  89. return err
  90. }
  91. m.Timestamp = time.Unix(millis/1000, (millis%1000)*int64(time.Millisecond))
  92. }
  93. m.Key, err = pd.getBytes()
  94. if err != nil {
  95. return err
  96. }
  97. m.Value, err = pd.getBytes()
  98. if err != nil {
  99. return err
  100. }
  101. switch m.Codec {
  102. case CompressionNone:
  103. // nothing to do
  104. case CompressionGZIP:
  105. if m.Value == nil {
  106. break
  107. }
  108. reader, err := gzip.NewReader(bytes.NewReader(m.Value))
  109. if err != nil {
  110. return err
  111. }
  112. if m.Value, err = ioutil.ReadAll(reader); err != nil {
  113. return err
  114. }
  115. if err := m.decodeSet(); err != nil {
  116. return err
  117. }
  118. case CompressionSnappy:
  119. if m.Value == nil {
  120. break
  121. }
  122. if m.Value, err = snappy.Decode(m.Value); err != nil {
  123. return err
  124. }
  125. if err := m.decodeSet(); err != nil {
  126. return err
  127. }
  128. default:
  129. return PacketDecodingError{fmt.Sprintf("invalid compression specified (%d)", m.Codec)}
  130. }
  131. return pd.pop()
  132. }
  133. // decodes a message set from a previousy encoded bulk-message
  134. func (m *Message) decodeSet() (err error) {
  135. pd := realDecoder{raw: m.Value}
  136. m.Set = &MessageSet{}
  137. return m.Set.decode(&pd)
  138. }