github.com/deanMdreon/kafka-go@v0.4.32/builder_test.go (about)

     1  package kafka
     2  
     3  import (
     4  	"bytes"
     5  	"fmt"
     6  	"io"
     7  	"time"
     8  
     9  	"github.com/deanMdreon/kafka-go/compress"
    10  )
    11  
    12  // This file defines builders to assist in creating kafka payloads for unit testing.
    13  
    14  // fetchResponseBuilder builds v10 fetch responses. The version of the v10 fetch
    15  // responses are not as important as the message sets contained within, as this
    16  // type is ultimately used to unit test the message set reader that consumes the
    17  // rest of the response once the header has been parsed.
    18  type fetchResponseBuilder struct {
    19  	header   fetchResponseHeader
    20  	msgSets  []messageSetBuilder
    21  	rendered []byte
    22  }
    23  
    24  type fetchResponseHeader struct {
    25  	throttle            int32
    26  	errorCode           int16
    27  	sessionID           int32
    28  	topic               string
    29  	partition           int32
    30  	partitionErrorCode  int16
    31  	highWatermarkOffset int64
    32  	lastStableOffset    int64
    33  	logStartOffset      int64
    34  }
    35  
    36  func (b *fetchResponseBuilder) messages() (res []Message) {
    37  	for _, set := range b.msgSets {
    38  		res = append(res, set.messages()...)
    39  	}
    40  	return
    41  }
    42  
    43  func (b *fetchResponseBuilder) bytes() []byte {
    44  	if b.rendered == nil {
    45  		b.rendered = newWB().call(func(wb *kafkaWriteBuffer) {
    46  			wb.writeInt32(b.header.throttle)
    47  			wb.writeInt16(b.header.errorCode)
    48  			wb.writeInt32(b.header.sessionID)
    49  			wb.writeInt32(1) // num topics
    50  			wb.writeString(b.header.topic)
    51  			wb.writeInt32(1) // how many partitions
    52  			wb.writeInt32(b.header.partition)
    53  			wb.writeInt16(b.header.partitionErrorCode)
    54  			wb.writeInt64(b.header.highWatermarkOffset)
    55  			wb.writeInt64(b.header.lastStableOffset)
    56  			wb.writeInt64(b.header.logStartOffset)
    57  			wb.writeInt32(-1) // num aborted tx
    58  			wb.writeBytes(newWB().call(func(wb *kafkaWriteBuffer) {
    59  				for _, msgSet := range b.msgSets {
    60  					wb.Write(msgSet.bytes())
    61  				}
    62  			}))
    63  		})
    64  	}
    65  	return b.rendered
    66  }
    67  
    68  func (b *fetchResponseBuilder) Len() int {
    69  	return len(b.bytes())
    70  }
    71  
    72  type messageSetBuilder interface {
    73  	bytes() []byte
    74  	messages() []Message
    75  }
    76  
    77  type v0MessageSetBuilder struct {
    78  	msgs  []Message
    79  	codec CompressionCodec
    80  }
    81  
    82  func (f v0MessageSetBuilder) messages() []Message {
    83  	return f.msgs
    84  }
    85  
    86  func (f v0MessageSetBuilder) bytes() []byte {
    87  	bs := newWB().call(func(wb *kafkaWriteBuffer) {
    88  		for _, msg := range f.msgs {
    89  			bs := newWB().call(func(wb *kafkaWriteBuffer) {
    90  				wb.writeInt64(msg.Offset) // offset
    91  				wb.writeBytes(newWB().call(func(wb *kafkaWriteBuffer) {
    92  					wb.writeInt32(-1) // crc, unused
    93  					wb.writeInt8(0)   // magic
    94  					wb.writeInt8(0)   // attributes -- zero, no compression for the inner message
    95  					wb.writeBytes(msg.Key)
    96  					wb.writeBytes(msg.Value)
    97  				}))
    98  			})
    99  			wb.Write(bs)
   100  		}
   101  	})
   102  	if f.codec != nil {
   103  		bs = newWB().call(func(wb *kafkaWriteBuffer) {
   104  			wb.writeInt64(f.msgs[0].Offset) // offset
   105  			wb.writeBytes(newWB().call(func(wb *kafkaWriteBuffer) {
   106  				compressed := mustCompress(bs, f.codec)
   107  				wb.writeInt32(-1)            // crc, unused
   108  				wb.writeInt8(0)              // magic
   109  				wb.writeInt8(f.codec.Code()) // attributes
   110  				wb.writeBytes(nil)           // key is always nil for compressed
   111  				wb.writeBytes(compressed)    // the value is the compressed message
   112  			}))
   113  		})
   114  	}
   115  	return bs
   116  }
   117  
   118  type v1MessageSetBuilder struct {
   119  	msgs  []Message
   120  	codec CompressionCodec
   121  }
   122  
   123  func (f v1MessageSetBuilder) messages() []Message {
   124  	return f.msgs
   125  }
   126  
   127  func (f v1MessageSetBuilder) bytes() []byte {
   128  	bs := newWB().call(func(wb *kafkaWriteBuffer) {
   129  		for i, msg := range f.msgs {
   130  			bs := newWB().call(func(wb *kafkaWriteBuffer) {
   131  				if f.codec != nil {
   132  					wb.writeInt64(int64(i)) // compressed inner message offsets are relative
   133  				} else {
   134  					wb.writeInt64(msg.Offset) // offset
   135  				}
   136  				wb.writeBytes(newWB().call(func(wb *kafkaWriteBuffer) {
   137  					wb.writeInt32(-1)                     // crc, unused
   138  					wb.writeInt8(1)                       // magic
   139  					wb.writeInt8(0)                       // attributes -- zero, no compression for the inner message
   140  					wb.writeInt64(1000 * msg.Time.Unix()) // timestamp
   141  					wb.writeBytes(msg.Key)
   142  					wb.writeBytes(msg.Value)
   143  				}))
   144  			})
   145  			wb.Write(bs)
   146  		}
   147  	})
   148  	if f.codec != nil {
   149  		bs = newWB().call(func(wb *kafkaWriteBuffer) {
   150  			wb.writeInt64(f.msgs[len(f.msgs)-1].Offset) // offset of the wrapper message is the last offset of the inner messages
   151  			wb.writeBytes(newWB().call(func(wb *kafkaWriteBuffer) {
   152  				bs := mustCompress(bs, f.codec)
   153  				wb.writeInt32(-1)                           // crc, unused
   154  				wb.writeInt8(1)                             // magic
   155  				wb.writeInt8(f.codec.Code())                // attributes
   156  				wb.writeInt64(1000 * f.msgs[0].Time.Unix()) // timestamp
   157  				wb.writeBytes(nil)                          // key is always nil for compressed
   158  				wb.writeBytes(bs)                           // the value is the compressed message
   159  			}))
   160  		})
   161  	}
   162  	return bs
   163  }
   164  
   165  type v2MessageSetBuilder struct {
   166  	msgs  []Message
   167  	codec CompressionCodec
   168  }
   169  
   170  func (f v2MessageSetBuilder) messages() []Message {
   171  	return f.msgs
   172  }
   173  
   174  func (f v2MessageSetBuilder) bytes() []byte {
   175  	attributes := int16(0)
   176  	if f.codec != nil {
   177  		attributes = int16(f.codec.Code()) // set codec code on attributes
   178  	}
   179  	return newWB().call(func(wb *kafkaWriteBuffer) {
   180  		wb.writeInt64(f.msgs[0].Offset)
   181  		wb.writeBytes(newWB().call(func(wb *kafkaWriteBuffer) {
   182  			wb.writeInt32(0)                            // leader epoch
   183  			wb.writeInt8(2)                             // magic = 2
   184  			wb.writeInt32(0)                            // crc, unused
   185  			wb.writeInt16(attributes)                   // record set attributes
   186  			wb.writeInt32(0)                            // record set last offset delta
   187  			wb.writeInt64(1000 * f.msgs[0].Time.Unix()) // record set first timestamp
   188  			wb.writeInt64(1000 * f.msgs[0].Time.Unix()) // record set last timestamp
   189  			wb.writeInt64(0)                            // record set producer id
   190  			wb.writeInt16(0)                            // record set producer epoch
   191  			wb.writeInt32(0)                            // record set base sequence
   192  			wb.writeInt32(int32(len(f.msgs)))           // record set count
   193  			bs := newWB().call(func(wb *kafkaWriteBuffer) {
   194  				for i, msg := range f.msgs {
   195  					wb.Write(newWB().call(func(wb *kafkaWriteBuffer) {
   196  						bs := newWB().call(func(wb *kafkaWriteBuffer) {
   197  							wb.writeInt8(0)                                              // record attributes, not used here
   198  							wb.writeVarInt(1000 * (time.Now().Unix() - msg.Time.Unix())) // timestamp
   199  							wb.writeVarInt(int64(i))                                     // offset delta
   200  							wb.writeVarInt(int64(len(msg.Key)))                          // key len
   201  							wb.Write(msg.Key)                                            // key bytes
   202  							wb.writeVarInt(int64(len(msg.Value)))                        // value len
   203  							wb.Write(msg.Value)                                          // value bytes
   204  							wb.writeVarInt(int64(len(msg.Headers)))                      // number of headers
   205  							for _, header := range msg.Headers {
   206  								wb.writeVarInt(int64(len(header.Key)))
   207  								wb.Write([]byte(header.Key))
   208  								wb.writeVarInt(int64(len(header.Value)))
   209  								wb.Write(header.Value)
   210  							}
   211  						})
   212  						wb.writeVarInt(int64(len(bs)))
   213  						wb.Write(bs)
   214  					}))
   215  				}
   216  			})
   217  			if f.codec != nil {
   218  				bs = mustCompress(bs, f.codec)
   219  			}
   220  			wb.Write(bs)
   221  		}))
   222  	})
   223  }
   224  
   225  // kafkaWriteBuffer is a write buffer that helps writing fetch responses
   226  type kafkaWriteBuffer struct {
   227  	writeBuffer
   228  	buf bytes.Buffer
   229  }
   230  
   231  func newWB() *kafkaWriteBuffer {
   232  	res := kafkaWriteBuffer{}
   233  	res.writeBuffer.w = &res.buf
   234  	return &res
   235  }
   236  
   237  func (f *kafkaWriteBuffer) Bytes() []byte {
   238  	return f.buf.Bytes()
   239  }
   240  
   241  // call is a convenience method that allows the kafkaWriteBuffer to be used
   242  // in a functional manner. This is helpful when building
   243  // nested structures, as the return value can be fed into
   244  // other fwWB APIs.
   245  func (f *kafkaWriteBuffer) call(cb func(wb *kafkaWriteBuffer)) []byte {
   246  	cb(f)
   247  	bs := f.Bytes()
   248  	if bs == nil {
   249  		bs = []byte{}
   250  	}
   251  	return bs
   252  }
   253  
   254  func mustCompress(bs []byte, codec compress.Codec) (res []byte) {
   255  	buf := bytes.Buffer{}
   256  	codecWriter := codec.NewWriter(&buf)
   257  	_, err := io.Copy(codecWriter, bytes.NewReader(bs))
   258  	if err != nil {
   259  		panic(fmt.Errorf("compress: %w", err))
   260  	}
   261  	err = codecWriter.Close()
   262  	if err != nil {
   263  		panic(fmt.Errorf("close codec writer: %w", err))
   264  	}
   265  	res = buf.Bytes()
   266  	return
   267  }