github.com/hack0072008/kafka-go@v1.0.1/write.go (about)

     1  package kafka
     2  
     3  import (
     4  	"bytes"
     5  	"encoding/binary"
     6  	"fmt"
     7  	"hash/crc32"
     8  	"io"
     9  	"time"
    10  )
    11  
    12  type writeBuffer struct {
    13  	w io.Writer
    14  	b [16]byte
    15  }
    16  
    17  func (wb *writeBuffer) writeInt8(i int8) {
    18  	wb.b[0] = byte(i)
    19  	wb.Write(wb.b[:1])
    20  }
    21  
    22  func (wb *writeBuffer) writeInt16(i int16) {
    23  	binary.BigEndian.PutUint16(wb.b[:2], uint16(i))
    24  	wb.Write(wb.b[:2])
    25  }
    26  
    27  func (wb *writeBuffer) writeInt32(i int32) {
    28  	binary.BigEndian.PutUint32(wb.b[:4], uint32(i))
    29  	wb.Write(wb.b[:4])
    30  }
    31  
    32  func (wb *writeBuffer) writeInt64(i int64) {
    33  	binary.BigEndian.PutUint64(wb.b[:8], uint64(i))
    34  	wb.Write(wb.b[:8])
    35  }
    36  
    37  func (wb *writeBuffer) writeVarInt(i int64) {
    38  	u := uint64((i << 1) ^ (i >> 63))
    39  	n := 0
    40  
    41  	for u >= 0x80 && n < len(wb.b) {
    42  		wb.b[n] = byte(u) | 0x80
    43  		u >>= 7
    44  		n++
    45  	}
    46  
    47  	if n < len(wb.b) {
    48  		wb.b[n] = byte(u)
    49  		n++
    50  	}
    51  
    52  	wb.Write(wb.b[:n])
    53  }
    54  
    55  func (wb *writeBuffer) writeString(s string) {
    56  	wb.writeInt16(int16(len(s)))
    57  	wb.WriteString(s)
    58  }
    59  
    60  func (wb *writeBuffer) writeVarString(s string) {
    61  	wb.writeVarInt(int64(len(s)))
    62  	wb.WriteString(s)
    63  }
    64  
    65  func (wb *writeBuffer) writeNullableString(s *string) {
    66  	if s == nil {
    67  		wb.writeInt16(-1)
    68  	} else {
    69  		wb.writeString(*s)
    70  	}
    71  }
    72  
    73  func (wb *writeBuffer) writeBytes(b []byte) {
    74  	n := len(b)
    75  	if b == nil {
    76  		n = -1
    77  	}
    78  	wb.writeInt32(int32(n))
    79  	wb.Write(b)
    80  }
    81  
    82  func (wb *writeBuffer) writeVarBytes(b []byte) {
    83  	if b != nil {
    84  		wb.writeVarInt(int64(len(b)))
    85  		wb.Write(b)
    86  	} else {
    87  		//-1 is used to indicate nil key
    88  		wb.writeVarInt(-1)
    89  	}
    90  }
    91  
    92  func (wb *writeBuffer) writeBool(b bool) {
    93  	v := int8(0)
    94  	if b {
    95  		v = 1
    96  	}
    97  	wb.writeInt8(v)
    98  }
    99  
   100  func (wb *writeBuffer) writeArrayLen(n int) {
   101  	wb.writeInt32(int32(n))
   102  }
   103  
   104  func (wb *writeBuffer) writeArray(n int, f func(int)) {
   105  	wb.writeArrayLen(n)
   106  	for i := 0; i < n; i++ {
   107  		f(i)
   108  	}
   109  }
   110  
   111  func (wb *writeBuffer) writeVarArray(n int, f func(int)) {
   112  	wb.writeVarInt(int64(n))
   113  	for i := 0; i < n; i++ {
   114  		f(i)
   115  	}
   116  }
   117  
   118  func (wb *writeBuffer) writeStringArray(a []string) {
   119  	wb.writeArray(len(a), func(i int) { wb.writeString(a[i]) })
   120  }
   121  
   122  func (wb *writeBuffer) writeInt32Array(a []int32) {
   123  	wb.writeArray(len(a), func(i int) { wb.writeInt32(a[i]) })
   124  }
   125  
   126  func (wb *writeBuffer) write(a interface{}) {
   127  	switch v := a.(type) {
   128  	case int8:
   129  		wb.writeInt8(v)
   130  	case int16:
   131  		wb.writeInt16(v)
   132  	case int32:
   133  		wb.writeInt32(v)
   134  	case int64:
   135  		wb.writeInt64(v)
   136  	case string:
   137  		wb.writeString(v)
   138  	case []byte:
   139  		wb.writeBytes(v)
   140  	case bool:
   141  		wb.writeBool(v)
   142  	case writable:
   143  		v.writeTo(wb)
   144  	default:
   145  		panic(fmt.Sprintf("unsupported type: %T", a))
   146  	}
   147  }
   148  
   149  func (wb *writeBuffer) Write(b []byte) (int, error) {
   150  	return wb.w.Write(b)
   151  }
   152  
   153  func (wb *writeBuffer) WriteString(s string) (int, error) {
   154  	return io.WriteString(wb.w, s)
   155  }
   156  
   157  func (wb *writeBuffer) Flush() error {
   158  	if x, ok := wb.w.(interface{ Flush() error }); ok {
   159  		return x.Flush()
   160  	}
   161  	return nil
   162  }
   163  
   164  type writable interface {
   165  	writeTo(*writeBuffer)
   166  }
   167  
   168  func (wb *writeBuffer) writeFetchRequestV2(correlationID int32, clientID, topic string, partition int32, offset int64, minBytes, maxBytes int, maxWait time.Duration) error {
   169  	h := requestHeader{
   170  		ApiKey:        int16(fetch),
   171  		ApiVersion:    int16(v2),
   172  		CorrelationID: correlationID,
   173  		ClientID:      clientID,
   174  	}
   175  	h.Size = (h.size() - 4) +
   176  		4 + // replica ID
   177  		4 + // max wait time
   178  		4 + // min bytes
   179  		4 + // topic array length
   180  		sizeofString(topic) +
   181  		4 + // partition array length
   182  		4 + // partition
   183  		8 + // offset
   184  		4 // max bytes
   185  
   186  	h.writeTo(wb)
   187  	wb.writeInt32(-1) // replica ID
   188  	wb.writeInt32(milliseconds(maxWait))
   189  	wb.writeInt32(int32(minBytes))
   190  
   191  	// topic array
   192  	wb.writeArrayLen(1)
   193  	wb.writeString(topic)
   194  
   195  	// partition array
   196  	wb.writeArrayLen(1)
   197  	wb.writeInt32(partition)
   198  	wb.writeInt64(offset)
   199  	wb.writeInt32(int32(maxBytes))
   200  
   201  	return wb.Flush()
   202  }
   203  
   204  func (wb *writeBuffer) writeFetchRequestV5(correlationID int32, clientID, topic string, partition int32, offset int64, minBytes, maxBytes int, maxWait time.Duration, isolationLevel int8) error {
   205  	h := requestHeader{
   206  		ApiKey:        int16(fetch),
   207  		ApiVersion:    int16(v5),
   208  		CorrelationID: correlationID,
   209  		ClientID:      clientID,
   210  	}
   211  	h.Size = (h.size() - 4) +
   212  		4 + // replica ID
   213  		4 + // max wait time
   214  		4 + // min bytes
   215  		4 + // max bytes
   216  		1 + // isolation level
   217  		4 + // topic array length
   218  		sizeofString(topic) +
   219  		4 + // partition array length
   220  		4 + // partition
   221  		8 + // offset
   222  		8 + // log start offset
   223  		4 // max bytes
   224  
   225  	h.writeTo(wb)
   226  	wb.writeInt32(-1) // replica ID
   227  	wb.writeInt32(milliseconds(maxWait))
   228  	wb.writeInt32(int32(minBytes))
   229  	wb.writeInt32(int32(maxBytes))
   230  	wb.writeInt8(isolationLevel) // isolation level 0 - read uncommitted
   231  
   232  	// topic array
   233  	wb.writeArrayLen(1)
   234  	wb.writeString(topic)
   235  
   236  	// partition array
   237  	wb.writeArrayLen(1)
   238  	wb.writeInt32(partition)
   239  	wb.writeInt64(offset)
   240  	wb.writeInt64(int64(0)) // log start offset only used when is sent by follower
   241  	wb.writeInt32(int32(maxBytes))
   242  
   243  	return wb.Flush()
   244  }
   245  
   246  func (wb *writeBuffer) writeFetchRequestV10(correlationID int32, clientID, topic string, partition int32, offset int64, minBytes, maxBytes int, maxWait time.Duration, isolationLevel int8) error {
   247  	h := requestHeader{
   248  		ApiKey:        int16(fetch),
   249  		ApiVersion:    int16(v10),
   250  		CorrelationID: correlationID,
   251  		ClientID:      clientID,
   252  	}
   253  	h.Size = (h.size() - 4) +
   254  		4 + // replica ID
   255  		4 + // max wait time
   256  		4 + // min bytes
   257  		4 + // max bytes
   258  		1 + // isolation level
   259  		4 + // session ID
   260  		4 + // session epoch
   261  		4 + // topic array length
   262  		sizeofString(topic) +
   263  		4 + // partition array length
   264  		4 + // partition
   265  		4 + // current leader epoch
   266  		8 + // fetch offset
   267  		8 + // log start offset
   268  		4 + // partition max bytes
   269  		4 // forgotten topics data
   270  
   271  	h.writeTo(wb)
   272  	wb.writeInt32(-1) // replica ID
   273  	wb.writeInt32(milliseconds(maxWait))
   274  	wb.writeInt32(int32(minBytes))
   275  	wb.writeInt32(int32(maxBytes))
   276  	wb.writeInt8(isolationLevel) // isolation level 0 - read uncommitted
   277  	wb.writeInt32(0)             //FIXME
   278  	wb.writeInt32(-1)            //FIXME
   279  
   280  	// topic array
   281  	wb.writeArrayLen(1)
   282  	wb.writeString(topic)
   283  
   284  	// partition array
   285  	wb.writeArrayLen(1)
   286  	wb.writeInt32(partition)
   287  	wb.writeInt32(-1) //FIXME
   288  	wb.writeInt64(offset)
   289  	wb.writeInt64(int64(0)) // log start offset only used when is sent by follower
   290  	wb.writeInt32(int32(maxBytes))
   291  
   292  	// forgotten topics array
   293  	wb.writeArrayLen(0) // forgotten topics not supported yet
   294  
   295  	return wb.Flush()
   296  }
   297  
   298  func (wb *writeBuffer) writeListOffsetRequestV1(correlationID int32, clientID, topic string, partition int32, time int64) error {
   299  	h := requestHeader{
   300  		ApiKey:        int16(listOffsets),
   301  		ApiVersion:    int16(v1),
   302  		CorrelationID: correlationID,
   303  		ClientID:      clientID,
   304  	}
   305  	h.Size = (h.size() - 4) +
   306  		4 + // replica ID
   307  		4 + // topic array length
   308  		sizeofString(topic) + // topic
   309  		4 + // partition array length
   310  		4 + // partition
   311  		8 // time
   312  
   313  	h.writeTo(wb)
   314  	wb.writeInt32(-1) // replica ID
   315  
   316  	// topic array
   317  	wb.writeArrayLen(1)
   318  	wb.writeString(topic)
   319  
   320  	// partition array
   321  	wb.writeArrayLen(1)
   322  	wb.writeInt32(partition)
   323  	wb.writeInt64(time)
   324  
   325  	return wb.Flush()
   326  }
   327  
   328  func (wb *writeBuffer) writeProduceRequestV2(codec CompressionCodec, correlationID int32, clientID, topic string, partition int32, timeout time.Duration, requiredAcks int16, msgs ...Message) (err error) {
   329  	var size int32
   330  	var attributes int8
   331  	var compressed *bytes.Buffer
   332  
   333  	if codec == nil {
   334  		size = messageSetSize(msgs...)
   335  	} else {
   336  		compressed, attributes, size, err = compressMessageSet(codec, msgs...)
   337  		if err != nil {
   338  			return
   339  		}
   340  		msgs = []Message{{Value: compressed.Bytes()}}
   341  	}
   342  
   343  	h := requestHeader{
   344  		ApiKey:        int16(produce),
   345  		ApiVersion:    int16(v2),
   346  		CorrelationID: correlationID,
   347  		ClientID:      clientID,
   348  	}
   349  	h.Size = (h.size() - 4) +
   350  		2 + // required acks
   351  		4 + // timeout
   352  		4 + // topic array length
   353  		sizeofString(topic) + // topic
   354  		4 + // partition array length
   355  		4 + // partition
   356  		4 + // message set size
   357  		size
   358  
   359  	h.writeTo(wb)
   360  	wb.writeInt16(requiredAcks) // required acks
   361  	wb.writeInt32(milliseconds(timeout))
   362  
   363  	// topic array
   364  	wb.writeArrayLen(1)
   365  	wb.writeString(topic)
   366  
   367  	// partition array
   368  	wb.writeArrayLen(1)
   369  	wb.writeInt32(partition)
   370  
   371  	wb.writeInt32(size)
   372  	cw := &crc32Writer{table: crc32.IEEETable}
   373  
   374  	for _, msg := range msgs {
   375  		wb.writeMessage(msg.Offset, attributes, msg.Time, msg.Key, msg.Value, cw)
   376  	}
   377  
   378  	releaseBuffer(compressed)
   379  	return wb.Flush()
   380  }
   381  
   382  func (wb *writeBuffer) writeProduceRequestV3(correlationID int32, clientID, topic string, partition int32, timeout time.Duration, requiredAcks int16, transactionalID *string, recordBatch *recordBatch) (err error) {
   383  
   384  	h := requestHeader{
   385  		ApiKey:        int16(produce),
   386  		ApiVersion:    int16(v3),
   387  		CorrelationID: correlationID,
   388  		ClientID:      clientID,
   389  	}
   390  
   391  	h.Size = (h.size() - 4) +
   392  		sizeofNullableString(transactionalID) +
   393  		2 + // required acks
   394  		4 + // timeout
   395  		4 + // topic array length
   396  		sizeofString(topic) + // topic
   397  		4 + // partition array length
   398  		4 + // partition
   399  		4 + // message set size
   400  		recordBatch.size
   401  
   402  	h.writeTo(wb)
   403  	wb.writeNullableString(transactionalID)
   404  	wb.writeInt16(requiredAcks) // required acks
   405  	wb.writeInt32(milliseconds(timeout))
   406  
   407  	// topic array
   408  	wb.writeArrayLen(1)
   409  	wb.writeString(topic)
   410  
   411  	// partition array
   412  	wb.writeArrayLen(1)
   413  	wb.writeInt32(partition)
   414  
   415  	recordBatch.writeTo(wb)
   416  
   417  	return wb.Flush()
   418  }
   419  
   420  func (wb *writeBuffer) writeProduceRequestV7(correlationID int32, clientID, topic string, partition int32, timeout time.Duration, requiredAcks int16, transactionalID *string, recordBatch *recordBatch) (err error) {
   421  
   422  	h := requestHeader{
   423  		ApiKey:        int16(produce),
   424  		ApiVersion:    int16(v7),
   425  		CorrelationID: correlationID,
   426  		ClientID:      clientID,
   427  	}
   428  	h.Size = (h.size() - 4) +
   429  		sizeofNullableString(transactionalID) +
   430  		2 + // required acks
   431  		4 + // timeout
   432  		4 + // topic array length
   433  		sizeofString(topic) + // topic
   434  		4 + // partition array length
   435  		4 + // partition
   436  		4 + // message set size
   437  		recordBatch.size
   438  
   439  	h.writeTo(wb)
   440  	wb.writeNullableString(transactionalID)
   441  	wb.writeInt16(requiredAcks) // required acks
   442  	wb.writeInt32(milliseconds(timeout))
   443  
   444  	// topic array
   445  	wb.writeArrayLen(1)
   446  	wb.writeString(topic)
   447  
   448  	// partition array
   449  	wb.writeArrayLen(1)
   450  	wb.writeInt32(partition)
   451  
   452  	recordBatch.writeTo(wb)
   453  
   454  	return wb.Flush()
   455  }
   456  
   457  func (wb *writeBuffer) writeRecordBatch(attributes int16, size int32, count int, baseTime, lastTime time.Time, write func(*writeBuffer)) {
   458  	var (
   459  		baseTimestamp   = timestamp(baseTime)
   460  		lastTimestamp   = timestamp(lastTime)
   461  		lastOffsetDelta = int32(count - 1)
   462  		producerID      = int64(-1)    // default producer id for now
   463  		producerEpoch   = int16(-1)    // default producer epoch for now
   464  		baseSequence    = int32(-1)    // default base sequence
   465  		recordCount     = int32(count) // record count
   466  		writerBackup    = wb.w
   467  	)
   468  
   469  	// dry run to compute the checksum
   470  	cw := &crc32Writer{table: crc32.MakeTable(crc32.Castagnoli)}
   471  	wb.w = cw
   472  	cw.writeInt16(attributes) // attributes, timestamp type 0 - create time, not part of a transaction, no control messages
   473  	cw.writeInt32(lastOffsetDelta)
   474  	cw.writeInt64(baseTimestamp)
   475  	cw.writeInt64(lastTimestamp)
   476  	cw.writeInt64(producerID)
   477  	cw.writeInt16(producerEpoch)
   478  	cw.writeInt32(baseSequence)
   479  	cw.writeInt32(recordCount)
   480  	write(wb)
   481  	wb.w = writerBackup
   482  
   483  	// actual write to the output buffer
   484  	wb.writeInt64(int64(0))
   485  	wb.writeInt32(int32(size - 12)) // 12 = batch length + base offset sizes
   486  	wb.writeInt32(-1)               // partition leader epoch
   487  	wb.writeInt8(2)                 // magic byte
   488  	wb.writeInt32(int32(cw.crc32))
   489  
   490  	wb.writeInt16(attributes)
   491  	wb.writeInt32(lastOffsetDelta)
   492  	wb.writeInt64(baseTimestamp)
   493  	wb.writeInt64(lastTimestamp)
   494  	wb.writeInt64(producerID)
   495  	wb.writeInt16(producerEpoch)
   496  	wb.writeInt32(baseSequence)
   497  	wb.writeInt32(recordCount)
   498  	write(wb)
   499  }
   500  
   501  func compressMessageSet(codec CompressionCodec, msgs ...Message) (compressed *bytes.Buffer, attributes int8, size int32, err error) {
   502  	compressed = acquireBuffer()
   503  	compressor := codec.NewWriter(compressed)
   504  	wb := &writeBuffer{w: compressor}
   505  	cw := &crc32Writer{table: crc32.IEEETable}
   506  
   507  	for offset, msg := range msgs {
   508  		wb.writeMessage(int64(offset), 0, msg.Time, msg.Key, msg.Value, cw)
   509  	}
   510  
   511  	if err = compressor.Close(); err != nil {
   512  		releaseBuffer(compressed)
   513  		return
   514  	}
   515  
   516  	attributes = codec.Code()
   517  	size = messageSetSize(Message{Value: compressed.Bytes()})
   518  	return
   519  }
   520  
   521  func (wb *writeBuffer) writeMessage(offset int64, attributes int8, time time.Time, key, value []byte, cw *crc32Writer) {
   522  	const magicByte = 1 // compatible with kafka 0.10.0.0+
   523  
   524  	timestamp := timestamp(time)
   525  	size := messageSize(key, value)
   526  
   527  	// dry run to compute the checksum
   528  	cw.crc32 = 0
   529  	cw.writeInt8(magicByte)
   530  	cw.writeInt8(attributes)
   531  	cw.writeInt64(timestamp)
   532  	cw.writeBytes(key)
   533  	cw.writeBytes(value)
   534  
   535  	// actual write to the output buffer
   536  	wb.writeInt64(offset)
   537  	wb.writeInt32(size)
   538  	wb.writeInt32(int32(cw.crc32))
   539  	wb.writeInt8(magicByte)
   540  	wb.writeInt8(attributes)
   541  	wb.writeInt64(timestamp)
   542  	wb.writeBytes(key)
   543  	wb.writeBytes(value)
   544  }
   545  
   546  // Messages with magic >2 are called records. This method writes messages using message format 2.
   547  func (wb *writeBuffer) writeRecord(attributes int8, baseTime time.Time, offset int64, msg Message) {
   548  	timestampDelta := msg.Time.Sub(baseTime)
   549  	offsetDelta := int64(offset)
   550  
   551  	wb.writeVarInt(int64(recordSize(&msg, timestampDelta, offsetDelta)))
   552  	wb.writeInt8(attributes)
   553  	wb.writeVarInt(int64(milliseconds(timestampDelta)))
   554  	wb.writeVarInt(offsetDelta)
   555  
   556  	wb.writeVarBytes(msg.Key)
   557  	wb.writeVarBytes(msg.Value)
   558  	wb.writeVarArray(len(msg.Headers), func(i int) {
   559  		h := &msg.Headers[i]
   560  		wb.writeVarString(h.Key)
   561  		wb.writeVarBytes(h.Value)
   562  	})
   563  }
   564  
   565  func varIntLen(i int64) int {
   566  	u := uint64((i << 1) ^ (i >> 63)) // zig-zag encoding
   567  	n := 0
   568  
   569  	for u >= 0x80 {
   570  		u >>= 7
   571  		n++
   572  	}
   573  
   574  	return n + 1
   575  }
   576  
   577  func varBytesLen(b []byte) int {
   578  	return varIntLen(int64(len(b))) + len(b)
   579  }
   580  
   581  func varStringLen(s string) int {
   582  	return varIntLen(int64(len(s))) + len(s)
   583  }
   584  
   585  func varArrayLen(n int, f func(int) int) int {
   586  	size := varIntLen(int64(n))
   587  	for i := 0; i < n; i++ {
   588  		size += f(i)
   589  	}
   590  	return size
   591  }
   592  
   593  func messageSize(key, value []byte) int32 {
   594  	return 4 + // crc
   595  		1 + // magic byte
   596  		1 + // attributes
   597  		8 + // timestamp
   598  		sizeofBytes(key) +
   599  		sizeofBytes(value)
   600  }
   601  
   602  func messageSetSize(msgs ...Message) (size int32) {
   603  	for _, msg := range msgs {
   604  		size += 8 + // offset
   605  			4 + // message size
   606  			4 + // crc
   607  			1 + // magic byte
   608  			1 + // attributes
   609  			8 + // timestamp
   610  			sizeofBytes(msg.Key) +
   611  			sizeofBytes(msg.Value)
   612  	}
   613  	return
   614  }