github.com/segmentio/kafka-go@v0.4.48-0.20240318174348-3f6244eb34fd/example_consumergroup_test.go (about)

     1  package kafka_test
     2  
     3  import (
     4  	"context"
     5  	"errors"
     6  	"fmt"
     7  	"os"
     8  
     9  	"github.com/segmentio/kafka-go"
    10  )
    11  
    12  func ExampleGeneration_Start_consumerGroupParallelReaders() {
    13  	group, err := kafka.NewConsumerGroup(kafka.ConsumerGroupConfig{
    14  		ID:      "my-group",
    15  		Brokers: []string{"kafka:9092"},
    16  		Topics:  []string{"my-topic"},
    17  	})
    18  	if err != nil {
    19  		fmt.Printf("error creating consumer group: %+v\n", err)
    20  		os.Exit(1)
    21  	}
    22  	defer group.Close()
    23  
    24  	for {
    25  		gen, err := group.Next(context.TODO())
    26  		if err != nil {
    27  			break
    28  		}
    29  
    30  		assignments := gen.Assignments["my-topic"]
    31  		for _, assignment := range assignments {
    32  			partition, offset := assignment.ID, assignment.Offset
    33  			gen.Start(func(ctx context.Context) {
    34  				// create reader for this partition.
    35  				reader := kafka.NewReader(kafka.ReaderConfig{
    36  					Brokers:   []string{"127.0.0.1:9092"},
    37  					Topic:     "my-topic",
    38  					Partition: partition,
    39  				})
    40  				defer reader.Close()
    41  
    42  				// seek to the last committed offset for this partition.
    43  				reader.SetOffset(offset)
    44  				for {
    45  					msg, err := reader.ReadMessage(ctx)
    46  					if err != nil {
    47  						if errors.Is(err, kafka.ErrGenerationEnded) {
    48  							// generation has ended.  commit offsets.  in a real app,
    49  							// offsets would be committed periodically.
    50  							gen.CommitOffsets(map[string]map[int]int64{"my-topic": {partition: offset + 1}})
    51  							return
    52  						}
    53  
    54  						fmt.Printf("error reading message: %+v\n", err)
    55  						return
    56  					}
    57  
    58  					fmt.Printf("received message %s/%d/%d : %s\n", msg.Topic, msg.Partition, msg.Offset, string(msg.Value))
    59  					offset = msg.Offset
    60  				}
    61  			})
    62  		}
    63  	}
    64  }
    65  
    66  func ExampleGeneration_CommitOffsets_overwriteOffsets() {
    67  	group, err := kafka.NewConsumerGroup(kafka.ConsumerGroupConfig{
    68  		ID:      "my-group",
    69  		Brokers: []string{"kafka:9092"},
    70  		Topics:  []string{"my-topic"},
    71  	})
    72  	if err != nil {
    73  		fmt.Printf("error creating consumer group: %+v\n", err)
    74  		os.Exit(1)
    75  	}
    76  	defer group.Close()
    77  
    78  	gen, err := group.Next(context.TODO())
    79  	if err != nil {
    80  		fmt.Printf("error getting next generation: %+v\n", err)
    81  		os.Exit(1)
    82  	}
    83  	err = gen.CommitOffsets(map[string]map[int]int64{
    84  		"my-topic": {
    85  			0: 123,
    86  			1: 456,
    87  			3: 789,
    88  		},
    89  	})
    90  	if err != nil {
    91  		fmt.Printf("error committing offsets next generation: %+v\n", err)
    92  		os.Exit(1)
    93  	}
    94  }