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

     1  package kafka
     2  
     3  import (
     4  	"bytes"
     5  	"context"
     6  	"errors"
     7  	"fmt"
     8  	"io"
     9  	"math/rand"
    10  	"net"
    11  	"os"
    12  	"strconv"
    13  	"testing"
    14  	"time"
    15  
    16  	ktesting "github.com/segmentio/kafka-go/testing"
    17  	"golang.org/x/net/nettest"
    18  )
    19  
    20  type timeout struct{}
    21  
    22  func (*timeout) Error() string   { return "timeout" }
    23  func (*timeout) Temporary() bool { return true }
    24  func (*timeout) Timeout() bool   { return true }
    25  
    26  // connPipe is an adapter that implements the net.Conn interface on top of
    27  // two client kafka connections to pass the nettest.TestConn test suite.
    28  type connPipe struct {
    29  	rconn *Conn
    30  	wconn *Conn
    31  }
    32  
    33  func (c *connPipe) Close() error {
    34  	b := [1]byte{} // marker that the connection has been closed
    35  	c.wconn.SetWriteDeadline(time.Time{})
    36  	c.wconn.Write(b[:])
    37  	c.wconn.Close()
    38  	c.rconn.Close()
    39  	return nil
    40  }
    41  
    42  func (c *connPipe) Read(b []byte) (int, error) {
    43  	// See comments in Write.
    44  	time.Sleep(time.Millisecond)
    45  	if t := c.rconn.readDeadline(); !t.IsZero() {
    46  		return 0, &timeout{}
    47  	}
    48  	n, err := c.rconn.Read(b)
    49  	if n == 1 && b[0] == 0 {
    50  		c.rconn.Close()
    51  		n, err = 0, io.EOF
    52  	}
    53  	return n, err
    54  }
    55  
    56  func (c *connPipe) Write(b []byte) (int, error) {
    57  	// The nettest/ConcurrentMethods test spawns a bunch of goroutines that do
    58  	// random stuff on the connection, if a Read or Write was issued before a
    59  	// deadline was set then it could cancel an inflight request to kafka,
    60  	// resulting in the connection being closed.
    61  	// To prevent this from happening we wait a little while to give the other
    62  	// goroutines a chance to start and set the deadline.
    63  	time.Sleep(time.Millisecond)
    64  
    65  	// The nettest code only sets deadlines when it expects the write to time
    66  	// out.  The broker connection is alive and able to accept data, so we need
    67  	// to simulate the timeout in order to get the tests to pass.
    68  	if t := c.wconn.writeDeadline(); !t.IsZero() {
    69  		return 0, &timeout{}
    70  	}
    71  
    72  	return c.wconn.Write(b)
    73  }
    74  
    75  func (c *connPipe) LocalAddr() net.Addr {
    76  	return c.rconn.LocalAddr()
    77  }
    78  
    79  func (c *connPipe) RemoteAddr() net.Addr {
    80  	return c.wconn.LocalAddr()
    81  }
    82  
    83  func (c *connPipe) SetDeadline(t time.Time) error {
    84  	c.rconn.SetDeadline(t)
    85  	c.wconn.SetDeadline(t)
    86  	return nil
    87  }
    88  
    89  func (c *connPipe) SetReadDeadline(t time.Time) error {
    90  	return c.rconn.SetReadDeadline(t)
    91  }
    92  
    93  func (c *connPipe) SetWriteDeadline(t time.Time) error {
    94  	return c.wconn.SetWriteDeadline(t)
    95  }
    96  
    97  func init() {
    98  	rand.Seed(time.Now().UnixNano())
    99  }
   100  
   101  func makeTopic() string {
   102  	return fmt.Sprintf("kafka-go-%016x", rand.Int63())
   103  }
   104  
   105  func makeGroupID() string {
   106  	return fmt.Sprintf("kafka-go-group-%016x", rand.Int63())
   107  }
   108  
   109  func makeTransactionalID() string {
   110  	return fmt.Sprintf("kafka-go-transactional-id-%016x", rand.Int63())
   111  }
   112  
   113  func TestConn(t *testing.T) {
   114  	tests := []struct {
   115  		scenario   string
   116  		function   func(*testing.T, *Conn)
   117  		minVersion string
   118  	}{
   119  		{
   120  			scenario: "close right away",
   121  			function: testConnClose,
   122  		},
   123  
   124  		{
   125  			scenario: "ensure the initial offset of a connection is the first offset",
   126  			function: testConnFirstOffset,
   127  		},
   128  
   129  		{
   130  			scenario: "write a single message to kafka should succeed",
   131  			function: testConnWrite,
   132  		},
   133  
   134  		{
   135  			scenario: "writing a message to a closed kafka connection should fail",
   136  			function: testConnCloseAndWrite,
   137  		},
   138  
   139  		{
   140  			scenario: "ensure the connection can seek to the first offset",
   141  			function: testConnSeekFirstOffset,
   142  		},
   143  
   144  		{
   145  			scenario: "ensure the connection can seek to the last offset",
   146  			function: testConnSeekLastOffset,
   147  		},
   148  
   149  		{
   150  			scenario: "ensure the connection can seek relative to the current offset",
   151  			function: testConnSeekCurrentOffset,
   152  		},
   153  
   154  		{
   155  			scenario: "ensure the connection can seek to a random offset",
   156  			function: testConnSeekRandomOffset,
   157  		},
   158  
   159  		{
   160  			scenario: "unchecked seeks allow the connection to be positioned outside the boundaries of the partition",
   161  			function: testConnSeekDontCheck,
   162  		},
   163  
   164  		{
   165  			scenario: "writing and reading messages sequentially should preserve the order",
   166  			function: testConnWriteReadSequentially,
   167  		},
   168  
   169  		{
   170  			scenario: "writing a batch of messages and reading it sequentially should preserve the order",
   171  			function: testConnWriteBatchReadSequentially,
   172  		},
   173  
   174  		{
   175  			scenario: "writing and reading messages concurrently should preserve the order",
   176  			function: testConnWriteReadConcurrently,
   177  		},
   178  
   179  		{
   180  			scenario: "reading messages with a buffer that is too short should return io.ErrShortBuffer and maintain the connection open",
   181  			function: testConnReadShortBuffer,
   182  		},
   183  
   184  		{
   185  			scenario: "reading messages from an empty partition should timeout after reaching the deadline",
   186  			function: testConnReadEmptyWithDeadline,
   187  		},
   188  
   189  		{
   190  			scenario: "write batch of messages and read the highest offset (watermark)",
   191  			function: testConnReadWatermarkFromBatch,
   192  		},
   193  
   194  		{
   195  			scenario:   "read a batch with no explicit min or max bytes",
   196  			function:   testConnReadBatchWithNoMinMaxBytes,
   197  			minVersion: "0.11.0",
   198  		},
   199  
   200  		{
   201  			scenario: "read a batch using explicit max wait time",
   202  			function: testConnReadBatchWithMaxWait,
   203  		},
   204  
   205  		{
   206  			scenario: "find the group coordinator",
   207  			function: testConnFindCoordinator,
   208  		},
   209  
   210  		{
   211  			scenario: "test join group with an invalid groupID",
   212  			function: testConnJoinGroupInvalidGroupID,
   213  		},
   214  
   215  		{
   216  			scenario: "test join group with an invalid sessionTimeout",
   217  			function: testConnJoinGroupInvalidSessionTimeout,
   218  		},
   219  
   220  		{
   221  			scenario: "test join group with an invalid refreshTimeout",
   222  			function: testConnJoinGroupInvalidRefreshTimeout,
   223  		},
   224  
   225  		{
   226  			scenario: "test heartbeat once group has been created",
   227  			function: testConnHeartbeatErr,
   228  		},
   229  
   230  		{
   231  			scenario: "test leave group returns error when called outside group",
   232  			function: testConnLeaveGroupErr,
   233  		},
   234  
   235  		{
   236  			scenario: "test sync group with bad memberID",
   237  			function: testConnSyncGroupErr,
   238  		},
   239  
   240  		{
   241  			scenario:   "test list groups",
   242  			function:   testConnListGroupsReturnsGroups,
   243  			minVersion: "0.11.0",
   244  		},
   245  
   246  		{
   247  			scenario: "test fetch and commit offset",
   248  			function: testConnFetchAndCommitOffsets,
   249  		},
   250  
   251  		{
   252  			scenario: "test delete topics",
   253  			function: testDeleteTopics,
   254  		},
   255  
   256  		{
   257  			scenario: "test delete topics with an invalid topic",
   258  			function: testDeleteTopicsInvalidTopic,
   259  		},
   260  
   261  		{
   262  			scenario: "test retrieve controller",
   263  			function: testController,
   264  		},
   265  
   266  		{
   267  			scenario: "test list brokers",
   268  			function: testBrokers,
   269  		},
   270  
   271  		{
   272  			scenario: "the connection advertises the broker that it is connected to",
   273  			function: testConnBroker,
   274  		},
   275  	}
   276  
   277  	const (
   278  		tcp   = "tcp"
   279  		kafka = "localhost:9092"
   280  	)
   281  
   282  	for _, test := range tests {
   283  		if !ktesting.KafkaIsAtLeast(test.minVersion) {
   284  			t.Log("skipping " + test.scenario + " because broker is not at least version " + test.minVersion)
   285  			continue
   286  		}
   287  
   288  		testFunc := test.function
   289  		t.Run(test.scenario, func(t *testing.T) {
   290  			t.Parallel()
   291  
   292  			ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
   293  			defer cancel()
   294  
   295  			topic := makeTopic()
   296  
   297  			conn, err := (&Dialer{
   298  				Resolver: &net.Resolver{},
   299  			}).DialLeader(ctx, tcp, kafka, topic, 0)
   300  			if err != nil {
   301  				t.Fatal("failed to open a new kafka connection:", err)
   302  			}
   303  			defer conn.Close()
   304  			testFunc(t, conn)
   305  		})
   306  	}
   307  
   308  	t.Run("nettest", func(t *testing.T) {
   309  		// Need ability to skip nettest on newer Kafka versions to avoid these kinds of errors:
   310  		//  --- FAIL: TestConn/nettest (17.56s)
   311  		//    --- FAIL: TestConn/nettest/PingPong (7.40s)
   312  		//      conntest.go:112: unexpected Read error: [7] Request Timed Out: the request exceeded the user-specified time limit in the request
   313  		//      conntest.go:118: mismatching value: got 77, want 78
   314  		//      conntest.go:118: mismatching value: got 78, want 79
   315  		// ...
   316  		//
   317  		// TODO: Figure out why these are happening and fix them (they don't appear to be new).
   318  		if _, ok := os.LookupEnv("KAFKA_SKIP_NETTEST"); ok {
   319  			t.Log("skipping nettest because KAFKA_SKIP_NETTEST is set")
   320  			t.Skip()
   321  		}
   322  
   323  		t.Parallel()
   324  
   325  		nettest.TestConn(t, func() (c1 net.Conn, c2 net.Conn, stop func(), err error) {
   326  			topic1 := makeTopic()
   327  			topic2 := makeTopic()
   328  			var t1Reader *Conn
   329  			var t2Reader *Conn
   330  			var t1Writer *Conn
   331  			var t2Writer *Conn
   332  			dialer := &Dialer{}
   333  
   334  			ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
   335  			defer cancel()
   336  
   337  			if t1Reader, err = dialer.DialLeader(ctx, tcp, kafka, topic1, 0); err != nil {
   338  				return
   339  			}
   340  			if t2Reader, err = dialer.DialLeader(ctx, tcp, kafka, topic2, 0); err != nil {
   341  				return
   342  			}
   343  			if t1Writer, err = dialer.DialLeader(ctx, tcp, kafka, topic1, 0); err != nil {
   344  				return
   345  			}
   346  			if t2Writer, err = dialer.DialLeader(ctx, tcp, kafka, topic2, 0); err != nil {
   347  				return
   348  			}
   349  
   350  			stop = func() {
   351  				t1Reader.Close()
   352  				t1Writer.Close()
   353  				t2Reader.Close()
   354  				t2Writer.Close()
   355  			}
   356  			c1 = &connPipe{rconn: t1Reader, wconn: t2Writer}
   357  			c2 = &connPipe{rconn: t2Reader, wconn: t1Writer}
   358  			return
   359  		})
   360  	})
   361  }
   362  
   363  func testConnClose(t *testing.T, conn *Conn) {
   364  	if err := conn.Close(); err != nil {
   365  		t.Error(err)
   366  	}
   367  }
   368  
   369  func testConnFirstOffset(t *testing.T, conn *Conn) {
   370  	offset, whence := conn.Offset()
   371  
   372  	if offset != 0 && whence != 0 {
   373  		t.Error("bad first offset:", offset, whence)
   374  	}
   375  }
   376  
   377  func testConnWrite(t *testing.T, conn *Conn) {
   378  	b := []byte("Hello World!")
   379  	n, err := conn.Write(b)
   380  	if err != nil {
   381  		t.Error(err)
   382  	}
   383  
   384  	if n != len(b) {
   385  		t.Error("bad length returned by (*Conn).Write:", n)
   386  	}
   387  }
   388  
   389  func testConnCloseAndWrite(t *testing.T, conn *Conn) {
   390  	conn.Close()
   391  
   392  	_, err := conn.Write([]byte("Hello World!"))
   393  
   394  	// expect a network error
   395  	var netOpError *net.OpError
   396  	if !errors.As(err, &netOpError) {
   397  		t.Error(err)
   398  	}
   399  }
   400  
   401  func testConnSeekFirstOffset(t *testing.T, conn *Conn) {
   402  	for i := 0; i != 10; i++ {
   403  		if _, err := conn.Write([]byte(strconv.Itoa(i))); err != nil {
   404  			t.Fatal(err)
   405  		}
   406  	}
   407  
   408  	offset, err := conn.Seek(0, SeekStart)
   409  	if err != nil {
   410  		t.Error(err)
   411  	}
   412  
   413  	if offset != 0 {
   414  		t.Error("bad offset:", offset)
   415  	}
   416  }
   417  
   418  func testConnSeekLastOffset(t *testing.T, conn *Conn) {
   419  	for i := 0; i != 10; i++ {
   420  		if _, err := conn.Write([]byte(strconv.Itoa(i))); err != nil {
   421  			t.Fatal(err)
   422  		}
   423  	}
   424  
   425  	offset, err := conn.Seek(0, SeekEnd)
   426  	if err != nil {
   427  		t.Error(err)
   428  	}
   429  
   430  	if offset != 10 {
   431  		t.Error("bad offset:", offset)
   432  	}
   433  }
   434  
   435  func testConnSeekCurrentOffset(t *testing.T, conn *Conn) {
   436  	for i := 0; i != 10; i++ {
   437  		if _, err := conn.Write([]byte(strconv.Itoa(i))); err != nil {
   438  			t.Fatal(err)
   439  		}
   440  	}
   441  
   442  	offset, err := conn.Seek(5, SeekStart)
   443  	if err != nil {
   444  		t.Error(err)
   445  	}
   446  
   447  	if offset != 5 {
   448  		t.Error("bad offset:", offset)
   449  	}
   450  
   451  	offset, err = conn.Seek(-2, SeekCurrent)
   452  	if err != nil {
   453  		t.Error(err)
   454  	}
   455  
   456  	if offset != 3 {
   457  		t.Error("bad offset:", offset)
   458  	}
   459  }
   460  
   461  func testConnSeekRandomOffset(t *testing.T, conn *Conn) {
   462  	for i := 0; i != 10; i++ {
   463  		if _, err := conn.Write([]byte(strconv.Itoa(i))); err != nil {
   464  			t.Fatal(err)
   465  		}
   466  	}
   467  
   468  	offset, err := conn.Seek(3, SeekAbsolute)
   469  	if err != nil {
   470  		t.Error(err)
   471  	}
   472  
   473  	if offset != 3 {
   474  		t.Error("bad offset:", offset)
   475  	}
   476  }
   477  
   478  func testConnSeekDontCheck(t *testing.T, conn *Conn) {
   479  	for i := 0; i != 10; i++ {
   480  		if _, err := conn.Write([]byte(strconv.Itoa(i))); err != nil {
   481  			t.Fatal(err)
   482  		}
   483  	}
   484  
   485  	offset, err := conn.Seek(42, SeekAbsolute|SeekDontCheck)
   486  	if err != nil {
   487  		t.Error(err)
   488  	}
   489  
   490  	if offset != 42 {
   491  		t.Error("bad offset:", offset)
   492  	}
   493  
   494  	if _, err := conn.ReadMessage(1024); !errors.Is(err, OffsetOutOfRange) {
   495  		t.Error("unexpected error:", err)
   496  	}
   497  }
   498  
   499  func testConnWriteReadSequentially(t *testing.T, conn *Conn) {
   500  	for i := 0; i != 10; i++ {
   501  		if _, err := conn.Write([]byte(strconv.Itoa(i))); err != nil {
   502  			t.Fatal(err)
   503  		}
   504  	}
   505  
   506  	b := make([]byte, 128)
   507  
   508  	for i := 0; i != 10; i++ {
   509  		n, err := conn.Read(b)
   510  		if err != nil {
   511  			t.Error(err)
   512  			continue
   513  		}
   514  		s := string(b[:n])
   515  		if v, err := strconv.Atoi(s); err != nil {
   516  			t.Error(err)
   517  		} else if v != i {
   518  			t.Errorf("bad message read at offset %d: %s", i, s)
   519  		}
   520  	}
   521  }
   522  
   523  func testConnWriteBatchReadSequentially(t *testing.T, conn *Conn) {
   524  	msgs := makeTestSequence(10)
   525  
   526  	if _, err := conn.WriteMessages(msgs...); err != nil {
   527  		t.Fatal(err)
   528  	}
   529  
   530  	for i := 0; i != 10; i++ {
   531  		msg, err := conn.ReadMessage(128)
   532  		if err != nil {
   533  			t.Error(err)
   534  			continue
   535  		}
   536  		if !bytes.Equal(msg.Key, msgs[i].Key) {
   537  			t.Errorf("bad message key at offset %d: %q != %q", i, msg.Key, msgs[i].Key)
   538  		}
   539  		if !bytes.Equal(msg.Value, msgs[i].Value) {
   540  			t.Errorf("bad message value at offset %d: %q != %q", i, msg.Value, msgs[i].Value)
   541  		}
   542  		if !msg.Time.Equal(msgs[i].Time) {
   543  			t.Errorf("bad message time at offset %d: %s != %s", i, msg.Time, msgs[i].Time)
   544  		}
   545  	}
   546  }
   547  
   548  func testConnReadWatermarkFromBatch(t *testing.T, conn *Conn) {
   549  	if _, err := conn.WriteMessages(makeTestSequence(10)...); err != nil {
   550  		t.Fatal(err)
   551  	}
   552  
   553  	const minBytes = 1
   554  	const maxBytes = 10e6 // 10 MB
   555  
   556  	value := make([]byte, 10e3) // 10 KB
   557  
   558  	batch := conn.ReadBatch(minBytes, maxBytes)
   559  
   560  	for i := 0; i < 10; i++ {
   561  		_, err := batch.Read(value)
   562  		if err != nil {
   563  			if err = batch.Close(); err != nil {
   564  				t.Fatalf("error trying to read batch message: %s", err)
   565  			}
   566  		}
   567  
   568  		if batch.HighWaterMark() != 10 {
   569  			t.Fatal("expected highest offset (watermark) to be 10")
   570  		}
   571  	}
   572  
   573  	batch.Close()
   574  }
   575  
   576  func testConnReadBatchWithNoMinMaxBytes(t *testing.T, conn *Conn) {
   577  	if _, err := conn.WriteMessages(makeTestSequence(10)...); err != nil {
   578  		t.Fatal(err)
   579  	}
   580  
   581  	value := make([]byte, 10e3) // 10 KB
   582  
   583  	batch := conn.ReadBatchWith(ReadBatchConfig{})
   584  
   585  	for i := 0; i < 10; i++ {
   586  		_, err := batch.Read(value)
   587  		if err != nil {
   588  			if err = batch.Close(); err != nil {
   589  				t.Fatalf("error trying to read batch message: %s", err)
   590  			}
   591  		}
   592  
   593  		if batch.HighWaterMark() != 10 {
   594  			t.Fatal("expected highest offset (watermark) to be 10")
   595  		}
   596  	}
   597  
   598  	if err := batch.Close(); err != nil {
   599  		t.Fatalf("error trying to close batch: %s", err)
   600  	}
   601  
   602  	if err := batch.Err(); err != nil {
   603  		t.Fatalf("broken batch: %s", err)
   604  	}
   605  }
   606  
   607  func testConnReadBatchWithMaxWait(t *testing.T, conn *Conn) {
   608  	if _, err := conn.WriteMessages(makeTestSequence(10)...); err != nil {
   609  		t.Fatal(err)
   610  	}
   611  
   612  	const maxBytes = 10e6 // 10 MB
   613  
   614  	value := make([]byte, 10e3) // 10 KB
   615  
   616  	cfg := ReadBatchConfig{
   617  		MinBytes: maxBytes, // use max for both so that we hit max wait time
   618  		MaxBytes: maxBytes,
   619  		MaxWait:  500 * time.Millisecond,
   620  	}
   621  
   622  	// set aa read deadline so the batch will succeed.
   623  	conn.SetDeadline(time.Now().Add(time.Second))
   624  	batch := conn.ReadBatchWith(cfg)
   625  
   626  	for i := 0; i < 10; i++ {
   627  		_, err := batch.Read(value)
   628  		if err != nil {
   629  			if err = batch.Close(); err != nil {
   630  				t.Fatalf("error trying to read batch message: %s", err)
   631  			}
   632  		}
   633  
   634  		if batch.HighWaterMark() != 10 {
   635  			t.Fatal("expected highest offset (watermark) to be 10")
   636  		}
   637  	}
   638  
   639  	batch.Close()
   640  
   641  	// reset the offset and  ensure that the conn deadline takes precedence over
   642  	// the max wait
   643  	conn.Seek(0, SeekAbsolute)
   644  	conn.SetDeadline(time.Now().Add(50 * time.Millisecond))
   645  	batch = conn.ReadBatchWith(cfg)
   646  	var netErr net.Error
   647  	if err := batch.Err(); err == nil {
   648  		t.Fatal("should have timed out, but got no error")
   649  	} else if errors.As(err, &netErr) {
   650  		if !netErr.Timeout() {
   651  			t.Fatalf("should have timed out, but got: %v", err)
   652  		}
   653  	}
   654  }
   655  
   656  func waitForCoordinator(t *testing.T, conn *Conn, groupID string) {
   657  	// ensure that kafka has allocated a group coordinator.  oddly, issue doesn't
   658  	// appear to happen if the kafka been running for a while.
   659  	const maxAttempts = 20
   660  	for attempt := 1; attempt <= maxAttempts; attempt++ {
   661  		_, err := conn.findCoordinator(findCoordinatorRequestV0{
   662  			CoordinatorKey: groupID,
   663  		})
   664  		if err != nil {
   665  			if errors.Is(err, GroupCoordinatorNotAvailable) {
   666  				time.Sleep(250 * time.Millisecond)
   667  				continue
   668  			} else {
   669  				t.Fatalf("unable to find coordinator for group: %v", err)
   670  			}
   671  		} else {
   672  			return
   673  		}
   674  	}
   675  
   676  	t.Fatalf("unable to connect to coordinator after %v attempts", maxAttempts)
   677  }
   678  
   679  func createGroup(t *testing.T, conn *Conn, groupID string) (generationID int32, memberID string, stop func()) {
   680  	waitForCoordinator(t, conn, groupID)
   681  
   682  	join := func() (joinGroup joinGroupResponseV1) {
   683  		var err error
   684  		for attempt := 0; attempt < 10; attempt++ {
   685  			joinGroup, err = conn.joinGroup(joinGroupRequestV1{
   686  				GroupID:          groupID,
   687  				SessionTimeout:   int32(time.Minute / time.Millisecond),
   688  				RebalanceTimeout: int32(time.Second / time.Millisecond),
   689  				ProtocolType:     "roundrobin",
   690  				GroupProtocols: []joinGroupRequestGroupProtocolV1{
   691  					{
   692  						ProtocolName:     "roundrobin",
   693  						ProtocolMetadata: []byte("blah"),
   694  					},
   695  				},
   696  			})
   697  			if err != nil {
   698  				if errors.Is(err, NotCoordinatorForGroup) {
   699  					time.Sleep(250 * time.Millisecond)
   700  					continue
   701  				} else {
   702  					t.Fatalf("bad joinGroup: %s", err)
   703  				}
   704  			} else {
   705  				return
   706  			}
   707  		}
   708  
   709  		return
   710  	}
   711  
   712  	// join the group
   713  	joinGroup := join()
   714  
   715  	// sync the group
   716  	_, err := conn.syncGroup(syncGroupRequestV0{
   717  		GroupID:      groupID,
   718  		GenerationID: joinGroup.GenerationID,
   719  		MemberID:     joinGroup.MemberID,
   720  		GroupAssignments: []syncGroupRequestGroupAssignmentV0{
   721  			{
   722  				MemberID:          joinGroup.MemberID,
   723  				MemberAssignments: []byte("blah"),
   724  			},
   725  		},
   726  	})
   727  	if err != nil {
   728  		t.Fatalf("bad syncGroup: %s", err)
   729  	}
   730  
   731  	generationID = joinGroup.GenerationID
   732  	memberID = joinGroup.MemberID
   733  	stop = func() {
   734  		conn.leaveGroup(leaveGroupRequestV0{
   735  			GroupID:  groupID,
   736  			MemberID: joinGroup.MemberID,
   737  		})
   738  	}
   739  
   740  	return
   741  }
   742  
   743  func testConnFindCoordinator(t *testing.T, conn *Conn) {
   744  	groupID := makeGroupID()
   745  
   746  	for attempt := 0; attempt < 10; attempt++ {
   747  		if attempt != 0 {
   748  			time.Sleep(time.Millisecond * 50)
   749  		}
   750  		response, err := conn.findCoordinator(findCoordinatorRequestV0{CoordinatorKey: groupID})
   751  		if err != nil {
   752  			if errors.Is(err, GroupCoordinatorNotAvailable) {
   753  				continue
   754  			}
   755  
   756  			t.Fatalf("bad findCoordinator: %s", err)
   757  		}
   758  
   759  		if response.Coordinator.NodeID == 0 {
   760  			t.Errorf("bad NodeID")
   761  		}
   762  		if response.Coordinator.Host == "" {
   763  			t.Errorf("bad Host")
   764  		}
   765  		if response.Coordinator.Port == 0 {
   766  			t.Errorf("bad Port")
   767  		}
   768  		return
   769  	}
   770  }
   771  
   772  func testConnJoinGroupInvalidGroupID(t *testing.T, conn *Conn) {
   773  	_, err := conn.joinGroup(joinGroupRequestV1{})
   774  	if !errors.Is(err, InvalidGroupId) && !errors.Is(err, NotCoordinatorForGroup) {
   775  		t.Fatalf("expected %v or %v; got %v", InvalidGroupId, NotCoordinatorForGroup, err)
   776  	}
   777  }
   778  
   779  func testConnJoinGroupInvalidSessionTimeout(t *testing.T, conn *Conn) {
   780  	groupID := makeGroupID()
   781  	waitForCoordinator(t, conn, groupID)
   782  
   783  	_, err := conn.joinGroup(joinGroupRequestV1{
   784  		GroupID: groupID,
   785  	})
   786  	if !errors.Is(err, InvalidSessionTimeout) && !errors.Is(err, NotCoordinatorForGroup) {
   787  		t.Fatalf("expected %v or %v; got %v", InvalidSessionTimeout, NotCoordinatorForGroup, err)
   788  	}
   789  }
   790  
   791  func testConnJoinGroupInvalidRefreshTimeout(t *testing.T, conn *Conn) {
   792  	groupID := makeGroupID()
   793  	waitForCoordinator(t, conn, groupID)
   794  
   795  	_, err := conn.joinGroup(joinGroupRequestV1{
   796  		GroupID:        groupID,
   797  		SessionTimeout: int32(3 * time.Second / time.Millisecond),
   798  	})
   799  	if !errors.Is(err, InvalidSessionTimeout) && !errors.Is(err, NotCoordinatorForGroup) {
   800  		t.Fatalf("expected %v or %v; got %v", InvalidSessionTimeout, NotCoordinatorForGroup, err)
   801  	}
   802  }
   803  
   804  func testConnHeartbeatErr(t *testing.T, conn *Conn) {
   805  	groupID := makeGroupID()
   806  	createGroup(t, conn, groupID)
   807  
   808  	_, err := conn.syncGroup(syncGroupRequestV0{
   809  		GroupID: groupID,
   810  	})
   811  	if !errors.Is(err, UnknownMemberId) && !errors.Is(err, NotCoordinatorForGroup) {
   812  		t.Fatalf("expected %v or %v; got %v", UnknownMemberId, NotCoordinatorForGroup, err)
   813  	}
   814  }
   815  
   816  func testConnLeaveGroupErr(t *testing.T, conn *Conn) {
   817  	groupID := makeGroupID()
   818  	waitForCoordinator(t, conn, groupID)
   819  
   820  	_, err := conn.leaveGroup(leaveGroupRequestV0{
   821  		GroupID: groupID,
   822  	})
   823  	if !errors.Is(err, UnknownMemberId) && !errors.Is(err, NotCoordinatorForGroup) {
   824  		t.Fatalf("expected %v or %v; got %v", UnknownMemberId, NotCoordinatorForGroup, err)
   825  	}
   826  }
   827  
   828  func testConnSyncGroupErr(t *testing.T, conn *Conn) {
   829  	groupID := makeGroupID()
   830  	waitForCoordinator(t, conn, groupID)
   831  
   832  	_, err := conn.syncGroup(syncGroupRequestV0{
   833  		GroupID: groupID,
   834  	})
   835  	if !errors.Is(err, UnknownMemberId) && !errors.Is(err, NotCoordinatorForGroup) {
   836  		t.Fatalf("expected %v or %v; got %v", UnknownMemberId, NotCoordinatorForGroup, err)
   837  	}
   838  }
   839  
   840  func testConnListGroupsReturnsGroups(t *testing.T, conn *Conn) {
   841  	group1 := makeGroupID()
   842  	_, _, stop1 := createGroup(t, conn, group1)
   843  	defer stop1()
   844  
   845  	group2 := makeGroupID()
   846  	_, _, stop2 := createGroup(t, conn, group2)
   847  	defer stop2()
   848  
   849  	out, err := conn.listGroups(listGroupsRequestV1{})
   850  	if err != nil {
   851  		t.Fatalf("bad err: %v", err)
   852  	}
   853  
   854  	containsGroup := func(groupID string) bool {
   855  		for _, group := range out.Groups {
   856  			if group.GroupID == groupID {
   857  				return true
   858  			}
   859  		}
   860  		return false
   861  	}
   862  
   863  	if !containsGroup(group1) {
   864  		t.Errorf("expected groups to contain group1")
   865  	}
   866  
   867  	if !containsGroup(group2) {
   868  		t.Errorf("expected groups to contain group2")
   869  	}
   870  }
   871  
   872  func testConnFetchAndCommitOffsets(t *testing.T, conn *Conn) {
   873  	const N = 10
   874  	if _, err := conn.WriteMessages(makeTestSequence(N)...); err != nil {
   875  		t.Fatal(err)
   876  	}
   877  
   878  	groupID := makeGroupID()
   879  	generationID, memberID, stop := createGroup(t, conn, groupID)
   880  	defer stop()
   881  
   882  	request := offsetFetchRequestV1{
   883  		GroupID: groupID,
   884  		Topics: []offsetFetchRequestV1Topic{
   885  			{
   886  				Topic:      conn.topic,
   887  				Partitions: []int32{0},
   888  			},
   889  		},
   890  	}
   891  	fetch, err := conn.offsetFetch(request)
   892  	if err != nil {
   893  		t.Fatalf("bad err: %v", err)
   894  	}
   895  
   896  	if v := len(fetch.Responses); v != 1 {
   897  		t.Fatalf("expected 1 Response; got %v", v)
   898  	}
   899  
   900  	if v := len(fetch.Responses[0].PartitionResponses); v != 1 {
   901  		t.Fatalf("expected 1 PartitionResponses; got %v", v)
   902  	}
   903  
   904  	if offset := fetch.Responses[0].PartitionResponses[0].Offset; offset != -1 {
   905  		t.Fatalf("expected initial offset of -1; got %v", offset)
   906  	}
   907  
   908  	committedOffset := int64(N - 1)
   909  	_, err = conn.offsetCommit(offsetCommitRequestV2{
   910  		GroupID:       groupID,
   911  		GenerationID:  generationID,
   912  		MemberID:      memberID,
   913  		RetentionTime: int64(time.Hour / time.Millisecond),
   914  		Topics: []offsetCommitRequestV2Topic{
   915  			{
   916  				Topic: conn.topic,
   917  				Partitions: []offsetCommitRequestV2Partition{
   918  					{
   919  						Partition: 0,
   920  						Offset:    committedOffset,
   921  					},
   922  				},
   923  			},
   924  		},
   925  	})
   926  	if err != nil {
   927  		t.Fatalf("bad error: %v", err)
   928  	}
   929  
   930  	fetch, err = conn.offsetFetch(request)
   931  	if err != nil {
   932  		t.Fatalf("bad error: %v", err)
   933  	}
   934  
   935  	fetchedOffset := fetch.Responses[0].PartitionResponses[0].Offset
   936  	if committedOffset != fetchedOffset {
   937  		t.Fatalf("bad offset.  expected %v; got %v", committedOffset, fetchedOffset)
   938  	}
   939  }
   940  
   941  func testConnWriteReadConcurrently(t *testing.T, conn *Conn) {
   942  	const N = 1000
   943  	msgs := make([]string, N)
   944  	done := make(chan struct{})
   945  	written := make(chan struct{}, N/10)
   946  
   947  	for i := 0; i != N; i++ {
   948  		msgs[i] = strconv.Itoa(i)
   949  	}
   950  
   951  	go func() {
   952  		defer close(done)
   953  		for _, msg := range msgs {
   954  			if _, err := conn.Write([]byte(msg)); err != nil {
   955  				t.Error(err)
   956  			}
   957  			written <- struct{}{}
   958  		}
   959  	}()
   960  
   961  	b := make([]byte, 128)
   962  
   963  	for i := 0; i != N; i++ {
   964  		// wait until at least one message has been written.  the reason for
   965  		// this synchronization is that we aren't using deadlines.  as such, if
   966  		// the read happens before a message is available, it will cause a
   967  		// deadlock because the read request will never hit the one byte minimum
   968  		// in order to return and release the lock on the conn.  by ensuring
   969  		// that there's at least one message produced, we don't hit that
   970  		// condition.
   971  		<-written
   972  		n, err := conn.Read(b)
   973  		if err != nil {
   974  			t.Error(err)
   975  		}
   976  		if s := string(b[:n]); s != strconv.Itoa(i) {
   977  			t.Errorf("bad message read at offset %d: %s", i, s)
   978  		}
   979  	}
   980  
   981  	<-done
   982  }
   983  
   984  func testConnReadShortBuffer(t *testing.T, conn *Conn) {
   985  	if _, err := conn.Write([]byte("Hello World!")); err != nil {
   986  		t.Fatal(err)
   987  	}
   988  
   989  	b := make([]byte, 4)
   990  
   991  	for i := 0; i != 10; i++ {
   992  		b[0] = 0
   993  		b[1] = 0
   994  		b[2] = 0
   995  		b[3] = 0
   996  
   997  		n, err := conn.Read(b)
   998  		if !errors.Is(err, io.ErrShortBuffer) {
   999  			t.Error("bad error:", i, err)
  1000  		}
  1001  		if n != 4 {
  1002  			t.Error("bad byte count:", i, n)
  1003  		}
  1004  		if s := string(b); s != "Hell" {
  1005  			t.Error("bad content:", i, s)
  1006  		}
  1007  	}
  1008  }
  1009  
  1010  func testConnReadEmptyWithDeadline(t *testing.T, conn *Conn) {
  1011  	b := make([]byte, 100)
  1012  
  1013  	start := time.Now()
  1014  	deadline := start.Add(time.Second)
  1015  
  1016  	conn.SetReadDeadline(deadline)
  1017  	n, err := conn.Read(b)
  1018  
  1019  	if n != 0 {
  1020  		t.Error("bad byte count:", n)
  1021  	}
  1022  
  1023  	if !isTimeout(err) {
  1024  		t.Error("expected timeout error but got", err)
  1025  	}
  1026  }
  1027  
  1028  func testDeleteTopics(t *testing.T, conn *Conn) {
  1029  	topic1 := makeTopic()
  1030  	topic2 := makeTopic()
  1031  	err := conn.CreateTopics(
  1032  		TopicConfig{
  1033  			Topic:             topic1,
  1034  			NumPartitions:     1,
  1035  			ReplicationFactor: 1,
  1036  		},
  1037  		TopicConfig{
  1038  			Topic:             topic2,
  1039  			NumPartitions:     1,
  1040  			ReplicationFactor: 1,
  1041  		},
  1042  	)
  1043  	if err != nil {
  1044  		t.Fatalf("bad CreateTopics: %v", err)
  1045  	}
  1046  	conn.SetDeadline(time.Now().Add(time.Second))
  1047  	err = conn.DeleteTopics(topic1, topic2)
  1048  	if err != nil {
  1049  		t.Fatalf("bad DeleteTopics: %v", err)
  1050  	}
  1051  	partitions, err := conn.ReadPartitions(topic1, topic2)
  1052  	if err != nil {
  1053  		t.Fatalf("bad ReadPartitions: %v", err)
  1054  	}
  1055  	if len(partitions) != 0 {
  1056  		t.Fatal("exepected partitions to be empty ")
  1057  	}
  1058  }
  1059  
  1060  func testDeleteTopicsInvalidTopic(t *testing.T, conn *Conn) {
  1061  	topic := makeTopic()
  1062  	err := conn.CreateTopics(
  1063  		TopicConfig{
  1064  			Topic:             topic,
  1065  			NumPartitions:     1,
  1066  			ReplicationFactor: 1,
  1067  		},
  1068  	)
  1069  	if err != nil {
  1070  		t.Fatalf("bad CreateTopics: %v", err)
  1071  	}
  1072  	conn.SetDeadline(time.Now().Add(5 * time.Second))
  1073  	err = conn.DeleteTopics("invalid-topic", topic)
  1074  	if !errors.Is(err, UnknownTopicOrPartition) {
  1075  		t.Fatalf("expected UnknownTopicOrPartition error, but got %v", err)
  1076  	}
  1077  	partitions, err := conn.ReadPartitions(topic)
  1078  	if err != nil {
  1079  		t.Fatalf("bad ReadPartitions: %v", err)
  1080  	}
  1081  	if len(partitions) != 0 {
  1082  		t.Fatal("expected partitions to be empty")
  1083  	}
  1084  }
  1085  
  1086  func testController(t *testing.T, conn *Conn) {
  1087  	b, err := conn.Controller()
  1088  	if err != nil {
  1089  		t.Error(err)
  1090  	}
  1091  
  1092  	if b.Host != "localhost" {
  1093  		t.Errorf("expected localhost received %s", b.Host)
  1094  	}
  1095  	if b.Port != 9092 {
  1096  		t.Errorf("expected 9092 received %d", b.Port)
  1097  	}
  1098  	if b.ID != 1 {
  1099  		t.Errorf("expected 1 received %d", b.ID)
  1100  	}
  1101  	if b.Rack != "" {
  1102  		t.Errorf("expected empty string for rack received %s", b.Rack)
  1103  	}
  1104  }
  1105  
  1106  func testBrokers(t *testing.T, conn *Conn) {
  1107  	brokers, err := conn.Brokers()
  1108  	if err != nil {
  1109  		t.Error(err)
  1110  	}
  1111  
  1112  	if len(brokers) != 1 {
  1113  		t.Errorf("expected 1 broker in %+v", brokers)
  1114  	}
  1115  
  1116  	if brokers[0].ID != 1 {
  1117  		t.Errorf("expected ID 1 received %d", brokers[0].ID)
  1118  	}
  1119  }
  1120  
  1121  func testConnBroker(t *testing.T, conn *Conn) {
  1122  	broker := conn.Broker()
  1123  	// Depending on the environment the test is being run, IPv4 or IPv6 may be used.
  1124  	if broker.Host != "::1" && broker.Host != "127.0.0.1" {
  1125  		t.Errorf("invalid broker address: %q", broker.Host)
  1126  	}
  1127  	if broker.Port != 9092 {
  1128  		t.Errorf("invalid broker port: %d", broker.Port)
  1129  	}
  1130  	if broker.ID != 1 {
  1131  		t.Errorf("invalid broker id: %d", broker.ID)
  1132  	}
  1133  	if broker.Rack != "" {
  1134  		t.Errorf("invalid broker rack: %q", broker.Rack)
  1135  	}
  1136  }
  1137  
  1138  func TestReadPartitionsNoTopic(t *testing.T) {
  1139  	conn, err := Dial("tcp", "127.0.0.1:9092")
  1140  	if err != nil {
  1141  		t.Fatal(err)
  1142  	}
  1143  	defer conn.Close()
  1144  
  1145  	parts, err := conn.ReadPartitions()
  1146  	if err != nil {
  1147  		t.Error(err)
  1148  	}
  1149  
  1150  	if len(parts) == 0 {
  1151  		t.Errorf("no partitions were returned")
  1152  	}
  1153  }
  1154  
  1155  func TestUnsupportedSASLMechanism(t *testing.T) {
  1156  	ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
  1157  	defer cancel()
  1158  
  1159  	conn, err := (&Dialer{
  1160  		Resolver: &net.Resolver{},
  1161  	}).DialContext(ctx, "tcp", "127.0.0.1:9093")
  1162  	if err != nil {
  1163  		t.Fatal("failed to open a new kafka connection:", err)
  1164  	}
  1165  	defer conn.Close()
  1166  
  1167  	if err := conn.saslHandshake("FOO"); !errors.Is(err, UnsupportedSASLMechanism) {
  1168  		t.Errorf("Expected UnsupportedSASLMechanism but got %v", err)
  1169  	}
  1170  }
  1171  
  1172  const benchmarkMessageCount = 100
  1173  
  1174  func BenchmarkConn(b *testing.B) {
  1175  	benchmarks := []struct {
  1176  		scenario string
  1177  		function func(*testing.B, *Conn, []byte)
  1178  	}{
  1179  		{
  1180  			scenario: "Seek",
  1181  			function: benchmarkConnSeek,
  1182  		},
  1183  
  1184  		{
  1185  			scenario: "Read",
  1186  			function: benchmarkConnRead,
  1187  		},
  1188  
  1189  		{
  1190  			scenario: "ReadBatch",
  1191  			function: benchmarkConnReadBatch,
  1192  		},
  1193  
  1194  		{
  1195  			scenario: "ReadOffsets",
  1196  			function: benchmarkConnReadOffsets,
  1197  		},
  1198  
  1199  		{
  1200  			scenario: "Write",
  1201  			function: benchmarkConnWrite,
  1202  		},
  1203  	}
  1204  
  1205  	topic := makeTopic()
  1206  	value := make([]byte, 10e3) // 10 KB
  1207  	msgs := make([]Message, benchmarkMessageCount)
  1208  
  1209  	for i := range msgs {
  1210  		msgs[i].Value = value
  1211  	}
  1212  
  1213  	conn, _ := DialLeader(context.Background(), "tcp", "localhost:9092", topic, 0)
  1214  	defer conn.Close()
  1215  
  1216  	if _, err := conn.WriteMessages(msgs...); err != nil {
  1217  		b.Fatal(err)
  1218  	}
  1219  
  1220  	for _, benchmark := range benchmarks {
  1221  		b.Run(benchmark.scenario, func(b *testing.B) {
  1222  			if _, err := conn.Seek(0, SeekStart); err != nil {
  1223  				b.Error(err)
  1224  				return
  1225  			}
  1226  			benchmark.function(b, conn, value)
  1227  		})
  1228  	}
  1229  }
  1230  
  1231  func benchmarkConnSeek(b *testing.B, conn *Conn, _ []byte) {
  1232  	for i := 0; i != b.N; i++ {
  1233  		if _, err := conn.Seek(int64(i%benchmarkMessageCount), SeekAbsolute); err != nil {
  1234  			b.Error(err)
  1235  			return
  1236  		}
  1237  	}
  1238  }
  1239  
  1240  func benchmarkConnRead(b *testing.B, conn *Conn, a []byte) {
  1241  	n := 0
  1242  	i := 0
  1243  
  1244  	for i != b.N {
  1245  		if (i % benchmarkMessageCount) == 0 {
  1246  			if _, err := conn.Seek(0, SeekStart); err != nil {
  1247  				b.Error(err)
  1248  				return
  1249  			}
  1250  		}
  1251  
  1252  		c, err := conn.Read(a)
  1253  		if err != nil {
  1254  			b.Error(err)
  1255  			return
  1256  		}
  1257  
  1258  		n += c
  1259  		i++
  1260  	}
  1261  
  1262  	b.SetBytes(int64(n / i))
  1263  }
  1264  
  1265  func benchmarkConnReadBatch(b *testing.B, conn *Conn, a []byte) {
  1266  	const minBytes = 1
  1267  	const maxBytes = 10e6 // 10 MB
  1268  
  1269  	batch := conn.ReadBatch(minBytes, maxBytes)
  1270  	i := 0
  1271  	n := 0
  1272  
  1273  	for i != b.N {
  1274  		c, err := batch.Read(a)
  1275  		if err != nil {
  1276  			if err = batch.Close(); err != nil {
  1277  				b.Error(err)
  1278  				return
  1279  			}
  1280  			if _, err = conn.Seek(0, SeekStart); err != nil {
  1281  				b.Error(err)
  1282  				return
  1283  			}
  1284  			batch = conn.ReadBatch(minBytes, maxBytes)
  1285  		}
  1286  		n += c
  1287  		i++
  1288  	}
  1289  
  1290  	batch.Close()
  1291  	b.SetBytes(int64(n / i))
  1292  }
  1293  
  1294  func benchmarkConnReadOffsets(b *testing.B, conn *Conn, _ []byte) {
  1295  	for i := 0; i != b.N; i++ {
  1296  		_, _, err := conn.ReadOffsets()
  1297  		if err != nil {
  1298  			b.Error(err)
  1299  			return
  1300  		}
  1301  	}
  1302  }
  1303  
  1304  func benchmarkConnWrite(b *testing.B, conn *Conn, _ []byte) {
  1305  	a := make([]byte, 10e3) // 10 KB
  1306  	n := 0
  1307  	i := 0
  1308  
  1309  	for i != b.N {
  1310  		c, err := conn.Write(a)
  1311  		if err != nil {
  1312  			b.Error(err)
  1313  			return
  1314  		}
  1315  		n += c
  1316  		i++
  1317  	}
  1318  
  1319  	b.SetBytes(int64(n / i))
  1320  }
  1321  
  1322  func TestEmptyToNullableReturnsNil(t *testing.T) {
  1323  	if emptyToNullable("") != nil {
  1324  		t.Error("Empty string is not converted to nil")
  1325  	}
  1326  }
  1327  
  1328  func TestEmptyToNullableLeavesStringsIntact(t *testing.T) {
  1329  	const s = "abc"
  1330  	r := emptyToNullable(s)
  1331  	if *r != s {
  1332  		t.Error("Non empty string is not equal to the original string")
  1333  	}
  1334  }
  1335  
  1336  func TestMakeBrokersAllPresent(t *testing.T) {
  1337  	brokers := make(map[int32]Broker)
  1338  	brokers[1] = Broker{ID: 1, Host: "203.0.113.101", Port: 9092}
  1339  	brokers[2] = Broker{ID: 1, Host: "203.0.113.102", Port: 9092}
  1340  	brokers[3] = Broker{ID: 1, Host: "203.0.113.103", Port: 9092}
  1341  
  1342  	b := makeBrokers(brokers, 1, 2, 3)
  1343  	if len(b) != 3 {
  1344  		t.Errorf("Expected 3 brokers, got %d", len(b))
  1345  	}
  1346  	for _, i := range []int32{1, 2, 3} {
  1347  		if b[i-1] != brokers[i] {
  1348  			t.Errorf("Expected broker %d at index %d, got %d", i, i-1, b[i].ID)
  1349  		}
  1350  	}
  1351  }
  1352  
  1353  func TestMakeBrokersOneMissing(t *testing.T) {
  1354  	brokers := make(map[int32]Broker)
  1355  	brokers[1] = Broker{ID: 1, Host: "203.0.113.101", Port: 9092}
  1356  	brokers[3] = Broker{ID: 3, Host: "203.0.113.103", Port: 9092}
  1357  
  1358  	b := makeBrokers(brokers, 1, 2, 3)
  1359  	if len(b) != 3 {
  1360  		t.Errorf("Expected 3 brokers, got %d", len(b))
  1361  	}
  1362  	if b[0] != brokers[1] {
  1363  		t.Errorf("Expected broker 1 at index 0, got %d", b[0].ID)
  1364  	}
  1365  	if b[1] != (Broker{ID: 2}) {
  1366  		t.Errorf("Expected broker 2 at index 1, got %d", b[1].ID)
  1367  	}
  1368  	if b[2] != brokers[3] {
  1369  		t.Errorf("Expected broker 3 at index 1, got %d", b[2].ID)
  1370  	}
  1371  }