github.com/confluentinc/confluent-kafka-go@v1.9.2/kafka/consumer_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  	"math/rand"
    22  	"testing"
    23  	"time"
    24  )
    25  
    26  // consumerPerfTest measures the consumer performance using a pre-primed (produced to) topic
    27  func consumerPerfTest(b *testing.B, testname string, msgcnt int, useChannel bool, consumeFunc func(c *Consumer, rd *ratedisp, expCnt int), rebalanceCb func(c *Consumer, event Event) error) {
    28  
    29  	r := testconsumerInit(b)
    30  	if r == -1 {
    31  		b.Skipf("Missing testconf.json")
    32  		return
    33  	}
    34  	if msgcnt == 0 {
    35  		msgcnt = r
    36  	}
    37  
    38  	rand.Seed(int64(time.Now().Unix()))
    39  
    40  	conf := ConfigMap{"bootstrap.servers": testconf.Brokers,
    41  		"go.events.channel.enable": useChannel,
    42  		"group.id":                 fmt.Sprintf("go_cperf_%d", rand.Intn(1000000)),
    43  		"session.timeout.ms":       6000,
    44  		"api.version.request":      "true",
    45  		"enable.auto.commit":       false,
    46  		"debug":                    ",",
    47  		"auto.offset.reset":        "earliest"}
    48  
    49  	conf.updateFromTestconf()
    50  
    51  	c, err := NewConsumer(&conf)
    52  
    53  	if err != nil {
    54  		panic(err)
    55  	}
    56  
    57  	expCnt := msgcnt
    58  	b.Logf("%s, expecting %d messages", testname, expCnt)
    59  
    60  	c.Subscribe(testconf.Topic, rebalanceCb)
    61  
    62  	rd := ratedispStart(b, testname, 10)
    63  
    64  	consumeFunc(c, &rd, expCnt)
    65  
    66  	rd.print("TOTAL: ")
    67  
    68  	c.Close()
    69  
    70  	b.SetBytes(rd.size)
    71  
    72  }
    73  
    74  // handleEvent returns false if processing should stop, else true
    75  func handleEvent(c *Consumer, rd *ratedisp, expCnt int, ev Event) bool {
    76  	switch e := ev.(type) {
    77  	case *Message:
    78  		if e.TopicPartition.Error != nil {
    79  			rd.b.Logf("Error: %v", e.TopicPartition)
    80  		}
    81  
    82  		if rd.cnt == 0 {
    83  			// start measuring time from first message to avoid
    84  			// including rebalancing time.
    85  			rd.b.ResetTimer()
    86  			rd.reset()
    87  		}
    88  
    89  		rd.tick(1, int64(len(e.Value)))
    90  
    91  		if rd.cnt >= int64(expCnt) {
    92  			return false
    93  		}
    94  	case PartitionEOF:
    95  		break // silence
    96  	default:
    97  		rd.b.Fatalf("Consumer error: %v", e)
    98  	}
    99  	return true
   100  
   101  }
   102  
   103  // consume messages through the Events channel
   104  func eventChannelConsumer(c *Consumer, rd *ratedisp, expCnt int) {
   105  	for ev := range c.Events() {
   106  		if !handleEvent(c, rd, expCnt, ev) {
   107  			break
   108  		}
   109  	}
   110  }
   111  
   112  // consume messages through the Poll() interface
   113  func eventPollConsumer(c *Consumer, rd *ratedisp, expCnt int) {
   114  	for true {
   115  		ev := c.Poll(100)
   116  		if ev == nil {
   117  			// timeout
   118  			continue
   119  		}
   120  		if !handleEvent(c, rd, expCnt, ev) {
   121  			break
   122  		}
   123  	}
   124  }
   125  
   126  var testconsumerInited = false
   127  
   128  // Produce messages to consume (if needed)
   129  // Query watermarks of topic to see if we need to prime it at all.
   130  // NOTE: This wont work for compacted topics..
   131  // returns the number of messages to consume
   132  func testconsumerInit(b *testing.B) int {
   133  	if testconsumerInited {
   134  		return testconf.PerfMsgCount
   135  	}
   136  
   137  	if !testconfRead() {
   138  		return -1
   139  	}
   140  
   141  	msgcnt := testconf.PerfMsgCount
   142  
   143  	currcnt, err := getMessageCountInTopic(testconf.Topic)
   144  	if err == nil {
   145  		b.Logf("Topic %s has %d messages, need %d", testconf.Topic, currcnt, msgcnt)
   146  	}
   147  	if currcnt < msgcnt {
   148  		producerPerfTest(b, "Priming producer", msgcnt, false, false,
   149  			true,
   150  			func(p *Producer, m *Message, drChan chan Event) {
   151  				p.ProduceChannel() <- m
   152  			})
   153  	}
   154  
   155  	testconsumerInited = true
   156  	b.ResetTimer()
   157  	return msgcnt
   158  }
   159  
   160  func BenchmarkConsumerChannelPerformance(b *testing.B) {
   161  	consumerPerfTest(b, "Channel Consumer",
   162  		0, true, eventChannelConsumer, nil)
   163  }
   164  
   165  func BenchmarkConsumerPollPerformance(b *testing.B) {
   166  	consumerPerfTest(b, "Poll Consumer",
   167  		0, false, eventPollConsumer, nil)
   168  }
   169  
   170  func BenchmarkConsumerPollRebalancePerformance(b *testing.B) {
   171  	consumerPerfTest(b, "Poll Consumer (rebalance callback)",
   172  		0, false, eventPollConsumer,
   173  		func(c *Consumer, event Event) error {
   174  			b.Logf("Rebalanced: %s", event)
   175  			return nil
   176  		})
   177  }