github.com/streamdal/segmentio-kafka-go@v0.4.47-streamdal/txnoffsetcommit_test.go (about)

     1  package kafka
     2  
     3  import (
     4  	"context"
     5  	"log"
     6  	"os"
     7  	"strconv"
     8  	"testing"
     9  	"time"
    10  
    11  	ktesting "github.com/segmentio/kafka-go/testing"
    12  )
    13  
    14  func TestClientTxnOffsetCommit(t *testing.T) {
    15  	if !ktesting.KafkaIsAtLeast("0.11.0") {
    16  		t.Skip("Skipping test because kafka version is not high enough.")
    17  	}
    18  
    19  	transactionalID := makeTransactionalID()
    20  	topic := makeTopic()
    21  
    22  	client, shutdown := newLocalClientWithTopic(topic, 1)
    23  	defer shutdown()
    24  
    25  	now := time.Now()
    26  
    27  	const N = 10
    28  	records := make([]Record, 0, N)
    29  	for i := 0; i < N; i++ {
    30  		records = append(records, Record{
    31  			Time:  now,
    32  			Value: NewBytes([]byte("test-message-" + strconv.Itoa(i))),
    33  		})
    34  	}
    35  	ctx, cancel := context.WithTimeout(context.Background(), time.Second*30)
    36  	defer cancel()
    37  	res, err := client.Produce(ctx, &ProduceRequest{
    38  		Topic:        topic,
    39  		RequiredAcks: RequireAll,
    40  		Records:      NewRecordReader(records...),
    41  	})
    42  	if err != nil {
    43  		t.Fatal(err)
    44  	}
    45  
    46  	if res.Error != nil {
    47  		t.Error(res.Error)
    48  	}
    49  
    50  	for index, err := range res.RecordErrors {
    51  		t.Fatalf("record at index %d produced an error: %v", index, err)
    52  	}
    53  
    54  	ctx, cancel = context.WithTimeout(context.Background(), time.Second*30)
    55  	defer cancel()
    56  	respc, err := waitForCoordinatorIndefinitely(ctx, client, &FindCoordinatorRequest{
    57  		Addr:    client.Addr,
    58  		Key:     transactionalID,
    59  		KeyType: CoordinatorKeyTypeTransaction,
    60  	})
    61  	if err != nil {
    62  		t.Fatal(err)
    63  	}
    64  
    65  	if respc.Error != nil {
    66  		t.Fatal(respc.Error)
    67  	}
    68  
    69  	ctx, cancel = context.WithTimeout(context.Background(), time.Second*30)
    70  	defer cancel()
    71  	respc, err = waitForCoordinatorIndefinitely(ctx, client, &FindCoordinatorRequest{
    72  		Addr:    client.Addr,
    73  		Key:     transactionalID,
    74  		KeyType: CoordinatorKeyTypeConsumer,
    75  	})
    76  	if err != nil {
    77  		t.Fatal(err)
    78  	}
    79  
    80  	if respc.Error != nil {
    81  		t.Fatal(respc.Error)
    82  	}
    83  
    84  	ctx, cancel = context.WithTimeout(context.Background(), time.Second*30)
    85  	defer cancel()
    86  	ipResp, err := client.InitProducerID(ctx, &InitProducerIDRequest{
    87  		TransactionalID:      transactionalID,
    88  		TransactionTimeoutMs: 10000,
    89  	})
    90  	if err != nil {
    91  		t.Fatal(err)
    92  	}
    93  
    94  	if ipResp.Error != nil {
    95  		t.Fatal(ipResp.Error)
    96  	}
    97  
    98  	groupID := makeGroupID()
    99  
   100  	group, err := NewConsumerGroup(ConsumerGroupConfig{
   101  		ID:                groupID,
   102  		Topics:            []string{topic},
   103  		Brokers:           []string{"localhost:9092"},
   104  		HeartbeatInterval: 2 * time.Second,
   105  		RebalanceTimeout:  2 * time.Second,
   106  		RetentionTime:     time.Hour,
   107  		Logger:            log.New(os.Stdout, "cg-test: ", 0),
   108  	})
   109  	if err != nil {
   110  		t.Fatal(err)
   111  	}
   112  	defer group.Close()
   113  
   114  	ctx, cancel = context.WithTimeout(context.Background(), time.Second*30)
   115  	defer cancel()
   116  	gen, err := group.Next(ctx)
   117  	if err != nil {
   118  		t.Fatal(err)
   119  	}
   120  
   121  	apresp, err := client.AddPartitionsToTxn(ctx, &AddPartitionsToTxnRequest{
   122  		TransactionalID: transactionalID,
   123  		ProducerID:      ipResp.Producer.ProducerID,
   124  		ProducerEpoch:   ipResp.Producer.ProducerEpoch,
   125  		Topics: map[string][]AddPartitionToTxn{
   126  			topic: {
   127  				{
   128  					Partition: 0,
   129  				},
   130  			},
   131  		},
   132  	})
   133  	if err != nil {
   134  		t.Fatal(err)
   135  	}
   136  
   137  	appartition := apresp.Topics[topic]
   138  	if len(appartition) != 1 {
   139  		t.Fatalf("unexpected partition count; expected: 1, got: %d", len(appartition))
   140  	}
   141  
   142  	for _, partition := range appartition {
   143  		if partition.Error != nil {
   144  			t.Fatal(partition.Error)
   145  		}
   146  	}
   147  
   148  	client.AddOffsetsToTxn(ctx, &AddOffsetsToTxnRequest{
   149  		TransactionalID: transactionalID,
   150  		ProducerID:      ipResp.Producer.ProducerID,
   151  		ProducerEpoch:   ipResp.Producer.ProducerEpoch,
   152  		GroupID:         groupID,
   153  	})
   154  
   155  	ctx, cancel = context.WithTimeout(context.Background(), time.Second*30)
   156  	defer cancel()
   157  	resp, err := client.TxnOffsetCommit(ctx, &TxnOffsetCommitRequest{
   158  		TransactionalID: transactionalID,
   159  		GroupID:         groupID,
   160  		MemberID:        gen.MemberID,
   161  		ProducerID:      ipResp.Producer.ProducerID,
   162  		ProducerEpoch:   ipResp.Producer.ProducerEpoch,
   163  		GenerationID:    int(gen.ID),
   164  		GroupInstanceID: groupID,
   165  		Topics: map[string][]TxnOffsetCommit{
   166  			topic: {
   167  				{
   168  					Partition: 0,
   169  					Offset:    10,
   170  				},
   171  			},
   172  		},
   173  	})
   174  	if err != nil {
   175  		t.Fatal(err)
   176  	}
   177  
   178  	partitions := resp.Topics[topic]
   179  
   180  	if len(partitions) != 1 {
   181  		t.Fatalf("unexpected partition count; expected: 1, got: %d", len(partitions))
   182  	}
   183  
   184  	for _, partition := range partitions {
   185  		if partition.Error != nil {
   186  			t.Fatal(partition.Error)
   187  		}
   188  	}
   189  
   190  	err = clientEndTxn(client, &EndTxnRequest{
   191  		TransactionalID: transactionalID,
   192  		ProducerID:      ipResp.Producer.ProducerID,
   193  		ProducerEpoch:   ipResp.Producer.ProducerEpoch,
   194  		Committed:       true,
   195  	})
   196  	if err != nil {
   197  		t.Fatal(err)
   198  	}
   199  
   200  	// seems like external visibility of the commit isn't
   201  	// synchronous with the EndTxn request. This seems
   202  	// to give enough time for the commit to become consistently visible.
   203  	<-time.After(time.Second)
   204  
   205  	ofr, err := client.OffsetFetch(ctx, &OffsetFetchRequest{
   206  		GroupID: groupID,
   207  		Topics:  map[string][]int{topic: {0}},
   208  	})
   209  	if err != nil {
   210  		t.Fatal(err)
   211  	}
   212  
   213  	if ofr.Error != nil {
   214  		t.Error(ofr.Error)
   215  	}
   216  
   217  	fetresps := ofr.Topics[topic]
   218  	if len(fetresps) != 1 {
   219  		t.Fatalf("unexpected 1 offsetfetchpartition responses; got %d", len(fetresps))
   220  	}
   221  
   222  	for _, r := range fetresps {
   223  		if r.Error != nil {
   224  			t.Fatal(r.Error)
   225  		}
   226  
   227  		if r.CommittedOffset != 10 {
   228  			t.Fatalf("expected committed offset to be 10; got: %v for partition: %v", r.CommittedOffset, r.Partition)
   229  		}
   230  	}
   231  }