github.com/confluentinc/confluent-kafka-go@v1.9.2/kafka/producer_performance_test.go (about)

     1  /**
     2   * Copyright 2016 Confluent Inc.
     3   *
     4   * Licensed under the Apache License, Version 2.0 (the "License");
     5   * you may not use this file except in compliance with the License.
     6   * You may obtain a copy of the License at
     7   *
     8   * http://www.apache.org/licenses/LICENSE-2.0
     9   *
    10   * Unless required by applicable law or agreed to in writing, software
    11   * distributed under the License is distributed on an "AS IS" BASIS,
    12   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    13   * See the License for the specific language governing permissions and
    14   * limitations under the License.
    15   */
    16  
    17  package kafka
    18  
    19  import (
    20  	"fmt"
    21  	"strings"
    22  	"testing"
    23  )
    24  
    25  func deliveryHandler(b *testing.B, expCnt int64, deliveryChan chan Event, doneChan chan int64) {
    26  
    27  	var cnt, size int64
    28  
    29  	for ev := range deliveryChan {
    30  		m, ok := ev.(*Message)
    31  		if !ok {
    32  			continue
    33  		}
    34  
    35  		if m.TopicPartition.Error != nil {
    36  			b.Errorf("Message delivery error: %v", m.TopicPartition)
    37  			break
    38  		}
    39  
    40  		cnt++
    41  		// b.Logf("Delivered %d/%d to %s", cnt, expCnt, m.TopicPartition)
    42  
    43  		if m.Value != nil {
    44  			size += int64(len(m.Value))
    45  		}
    46  		if cnt >= expCnt {
    47  			break
    48  		}
    49  
    50  	}
    51  
    52  	doneChan <- cnt
    53  	doneChan <- size
    54  	close(doneChan)
    55  }
    56  
    57  func producerPerfTest(b *testing.B, testname string, msgcnt int, withDr bool, batchProducer bool, silent bool, produceFunc func(p *Producer, m *Message, drChan chan Event)) {
    58  
    59  	if !testconfRead() {
    60  		b.Skipf("Missing testconf.json")
    61  	}
    62  
    63  	if msgcnt == 0 {
    64  		msgcnt = testconf.PerfMsgCount
    65  	}
    66  
    67  	conf := ConfigMap{"bootstrap.servers": testconf.Brokers,
    68  		"go.batch.producer":            batchProducer,
    69  		"go.delivery.reports":          withDr,
    70  		"queue.buffering.max.messages": msgcnt,
    71  		"api.version.request":          "true",
    72  		"broker.version.fallback":      "0.9.0.1",
    73  		"acks":                         1}
    74  
    75  	conf.updateFromTestconf()
    76  
    77  	p, err := NewProducer(&conf)
    78  	if err != nil {
    79  		panic(err)
    80  	}
    81  
    82  	topic := testconf.Topic
    83  	partition := int32(-1)
    84  	size := testconf.PerfMsgSize
    85  	pattern := "Hello"
    86  	buf := []byte(strings.Repeat(pattern, size/len(pattern)))
    87  
    88  	var doneChan chan int64
    89  	var drChan chan Event
    90  
    91  	if withDr {
    92  		doneChan = make(chan int64)
    93  		drChan = p.Events()
    94  		go deliveryHandler(b, int64(msgcnt), p.Events(), doneChan)
    95  	}
    96  
    97  	if !silent {
    98  		b.Logf("%s: produce %d messages", testname, msgcnt)
    99  	}
   100  
   101  	displayInterval := 5.0
   102  	if !silent {
   103  		displayInterval = 1000.0
   104  	}
   105  	rd := ratedispStart(b, fmt.Sprintf("%s: produce", testname), displayInterval)
   106  	rdDelivery := ratedispStart(b, fmt.Sprintf("%s: delivery", testname), displayInterval)
   107  
   108  	for i := 0; i < msgcnt; i++ {
   109  		m := Message{TopicPartition: TopicPartition{Topic: &topic, Partition: partition}, Value: buf}
   110  
   111  		produceFunc(p, &m, drChan)
   112  
   113  		rd.tick(1, int64(size))
   114  	}
   115  
   116  	if !silent {
   117  		rd.print("produce done: ")
   118  	}
   119  
   120  	// Wait for messages in-flight and in-queue to get delivered.
   121  	if !silent {
   122  		b.Logf("%s: %d messages in queue", testname, p.Len())
   123  	}
   124  	r := p.Flush(10000)
   125  	if r > 0 {
   126  		b.Errorf("%s: %d messages remains in queue after Flush()", testname, r)
   127  	}
   128  
   129  	// Close producer
   130  	p.Close()
   131  
   132  	var deliveryCnt, deliverySize int64
   133  
   134  	if withDr {
   135  		deliveryCnt = <-doneChan
   136  		deliverySize = <-doneChan
   137  	} else {
   138  		deliveryCnt = int64(msgcnt)
   139  		deliverySize = deliveryCnt * int64(size)
   140  	}
   141  	rdDelivery.tick(deliveryCnt, deliverySize)
   142  
   143  	rd.print("TOTAL: ")
   144  
   145  	b.SetBytes(deliverySize)
   146  }
   147  
   148  func BenchmarkProducerFunc(b *testing.B) {
   149  	producerPerfTest(b, "Function producer (without DR)",
   150  		0, false, false, false,
   151  		func(p *Producer, m *Message, drChan chan Event) {
   152  			err := p.Produce(m, drChan)
   153  			if err != nil {
   154  				b.Errorf("Produce() failed: %v", err)
   155  			}
   156  		})
   157  }
   158  
   159  func BenchmarkProducerFuncDR(b *testing.B) {
   160  	producerPerfTest(b, "Function producer (with DR)",
   161  		0, true, false, false,
   162  		func(p *Producer, m *Message, drChan chan Event) {
   163  			err := p.Produce(m, drChan)
   164  			if err != nil {
   165  				b.Errorf("Produce() failed: %v", err)
   166  			}
   167  		})
   168  }
   169  
   170  func BenchmarkProducerChannel(b *testing.B) {
   171  	producerPerfTest(b, "Channel producer (without DR)",
   172  		0, false, false, false,
   173  		func(p *Producer, m *Message, drChan chan Event) {
   174  			p.ProduceChannel() <- m
   175  		})
   176  }
   177  
   178  func BenchmarkProducerChannelDR(b *testing.B) {
   179  	producerPerfTest(b, "Channel producer (with DR)",
   180  		testconf.PerfMsgCount, true, false, false,
   181  		func(p *Producer, m *Message, drChan chan Event) {
   182  			p.ProduceChannel() <- m
   183  		})
   184  
   185  }
   186  
   187  func BenchmarkProducerBatchChannel(b *testing.B) {
   188  	producerPerfTest(b, "Channel producer (without DR, batch channel)",
   189  		0, false, true, false,
   190  		func(p *Producer, m *Message, drChan chan Event) {
   191  			p.ProduceChannel() <- m
   192  		})
   193  }
   194  
   195  func BenchmarkProducerBatchChannelDR(b *testing.B) {
   196  	producerPerfTest(b, "Channel producer (DR, batch channel)",
   197  		0, true, true, false,
   198  		func(p *Producer, m *Message, drChan chan Event) {
   199  			p.ProduceChannel() <- m
   200  		})
   201  }
   202  
   203  func BenchmarkProducerInternalMessageInstantiation(b *testing.B) {
   204  	topic := "test"
   205  	buf := []byte(strings.Repeat("Ten bytes!", 10))
   206  	v := 0
   207  	for i := 0; i < b.N; i++ {
   208  		msg := Message{TopicPartition: TopicPartition{Topic: &topic, Partition: 0}, Value: buf}
   209  		v += int(msg.TopicPartition.Partition) // avoid msg unused error
   210  	}
   211  }
   212  
   213  func BenchmarkProducerInternalMessageToC(b *testing.B) {
   214  	p, err := NewProducer(&ConfigMap{})
   215  	if err != nil {
   216  		b.Fatalf("NewProducer failed: %s", err)
   217  	}
   218  	b.ResetTimer()
   219  	topic := "test"
   220  	buf := []byte(strings.Repeat("Ten bytes!", 10))
   221  	for i := 0; i < b.N; i++ {
   222  		msg := Message{TopicPartition: TopicPartition{Topic: &topic, Partition: 0}, Value: buf}
   223  		p.handle.messageToCDummy(&msg)
   224  	}
   225  }