github.com/deanMdreon/kafka-go@v0.4.32/transport_test.go (about)

     1  package kafka
     2  
     3  import (
     4  	"context"
     5  	"crypto/tls"
     6  	"errors"
     7  	"net"
     8  	"testing"
     9  	"time"
    10  
    11  	"github.com/deanMdreon/kafka-go/protocol"
    12  	"github.com/deanMdreon/kafka-go/protocol/createtopics"
    13  	meta "github.com/deanMdreon/kafka-go/protocol/metadata"
    14  )
    15  
    16  func TestIssue477(t *testing.T) {
    17  	// This test verifies that a connection attempt with a minimal TLS
    18  	// configuration does not panic.
    19  	l, err := net.Listen("tcp", "127.0.0.1:0")
    20  	if err != nil {
    21  		t.Fatal(err)
    22  	}
    23  	defer l.Close()
    24  
    25  	cg := connGroup{
    26  		addr: l.Addr(),
    27  		pool: &connPool{
    28  			dial: defaultDialer.DialContext,
    29  			tls:  &tls.Config{},
    30  		},
    31  	}
    32  
    33  	if _, err := cg.connect(context.Background(), cg.addr); err != nil {
    34  		// An error is expected here because we are not actually establishing
    35  		// a TLS connection to a kafka broker.
    36  		t.Log(err)
    37  	} else {
    38  		t.Error("no error was reported when attempting to establish a TLS connection to a non-TLS endpoint")
    39  	}
    40  }
    41  
    42  func TestIssue672(t *testing.T) {
    43  	// ensure the test times out if the bug is re-introduced
    44  	ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
    45  	defer cancel()
    46  
    47  	// we'll simulate a situation with one good topic and one bad topic (bad configuration)
    48  	const brokenTopicName = "bad-topic"
    49  	const okTopicName = "good-topic"
    50  
    51  	// make the connection pool think it's immediately ready to send
    52  	ready := make(chan struct{})
    53  	close(ready)
    54  
    55  	// allow the system to wake as much as it wants
    56  	wake := make(chan event)
    57  	defer close(wake)
    58  	go func() {
    59  		for {
    60  			select {
    61  			case <-ctx.Done():
    62  				return
    63  			case e := <-wake:
    64  				if e == nil {
    65  					return
    66  				}
    67  				e.trigger()
    68  			}
    69  		}
    70  	}()
    71  
    72  	// handle requests by immediately resolving them with a create topics response,
    73  	// the "bad topic" will have an error value
    74  	requests := make(chan connRequest, 1)
    75  	defer close(requests)
    76  	go func() {
    77  		request := <-requests
    78  		request.res.resolve(&createtopics.Response{
    79  			ThrottleTimeMs: 0,
    80  			Topics: []createtopics.ResponseTopic{
    81  				{
    82  					Name:         brokenTopicName,
    83  					ErrorCode:    int16(InvalidPartitionNumber),
    84  					ErrorMessage: InvalidPartitionNumber.Description(),
    85  				},
    86  				{
    87  					Name:              okTopicName,
    88  					NumPartitions:     1,
    89  					ReplicationFactor: 1,
    90  				},
    91  			},
    92  		})
    93  	}()
    94  
    95  	pool := &connPool{
    96  		ready: ready,
    97  		wake:  wake,
    98  		conns: map[int32]*connGroup{},
    99  	}
   100  
   101  	// configure the state so it can find the good topic, but not the one that fails to create
   102  	pool.setState(connPoolState{
   103  		layout: protocol.Cluster{
   104  			Topics: map[string]protocol.Topic{
   105  				okTopicName: {
   106  					Name: okTopicName,
   107  					Partitions: map[int32]protocol.Partition{
   108  						0: {},
   109  					},
   110  				},
   111  			},
   112  		},
   113  	})
   114  
   115  	// trick the connection pool into thinking it has a valid connection to a broker
   116  	pool.conns[0] = &connGroup{
   117  		pool:   pool,
   118  		broker: Broker{},
   119  		idleConns: []*conn{
   120  			{
   121  				reqs: requests,
   122  			},
   123  		},
   124  	}
   125  
   126  	// perform the round trip:
   127  	// - if the issue is presenting this will hang waiting for metadata to arrive that will
   128  	//   never arrive, causing a deadline timeout.
   129  	// - if the issue is fixed this will resolve almost instantaneously
   130  	r, err := pool.roundTrip(ctx, &createtopics.Request{
   131  		Topics: []createtopics.RequestTopic{
   132  			{
   133  				Name:              brokenTopicName,
   134  				NumPartitions:     0,
   135  				ReplicationFactor: 1,
   136  			},
   137  			{
   138  				Name:              okTopicName,
   139  				NumPartitions:     1,
   140  				ReplicationFactor: 1,
   141  			},
   142  		},
   143  	})
   144  	// detect if the issue is presenting using the context timeout (note that checking the err return value
   145  	// isn't good enough as the original implementation didn't return the context cancellation error due to
   146  	// being run in a defer)
   147  	if errors.Is(ctx.Err(), context.DeadlineExceeded) {
   148  		t.Fatalf("issue 672 is presenting! roundTrip should not have timed out")
   149  	}
   150  
   151  	// ancillary assertions as general house-keeping, not directly related to the issue:
   152  
   153  	// we're not expecting any errors in this test
   154  	if err != nil {
   155  		t.Fatalf("unexpected error provoking connection pool roundTrip: %v", err)
   156  	}
   157  
   158  	// we expect a response containing the errors from the broker
   159  	if r == nil {
   160  		t.Fatal("expected a non-nil response")
   161  	}
   162  
   163  	// we expect to have the create topic response with created earlier
   164  	_, ok := r.(*createtopics.Response)
   165  	if !ok {
   166  		t.Fatalf("expected a createtopics.Response but got %T", r)
   167  	}
   168  }
   169  
   170  func TestIssue806(t *testing.T) {
   171  	// ensure the test times out if the bug is re-introduced
   172  	ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
   173  	defer cancel()
   174  
   175  	// simulate unknown topic want auto create with unknownTopicName,
   176  	const unknownTopicName = "unknown-topic"
   177  	const okTopicName = "good-topic"
   178  
   179  	// make the connection pool think it's immediately ready to send
   180  	ready := make(chan struct{})
   181  	close(ready)
   182  
   183  	// allow the system to wake as much as it wants
   184  	wake := make(chan event)
   185  	defer close(wake)
   186  	go func() {
   187  		for {
   188  			select {
   189  			case <-ctx.Done():
   190  				return
   191  			case e := <-wake:
   192  				if e == nil {
   193  					return
   194  				}
   195  				e.trigger()
   196  			}
   197  		}
   198  	}()
   199  
   200  	// handle requests by immediately resolving them with a create topics response,
   201  	// the "unknown topic" will have err UNKNOWN_TOPIC_OR_PARTITION
   202  	requests := make(chan connRequest, 1)
   203  	defer close(requests)
   204  	go func() {
   205  		request := <-requests
   206  		request.res.resolve(&meta.Response{
   207  			Topics: []meta.ResponseTopic{
   208  				{
   209  					Name:      unknownTopicName,
   210  					ErrorCode: int16(UnknownTopicOrPartition),
   211  				},
   212  				{
   213  					Name: okTopicName,
   214  					Partitions: []meta.ResponsePartition{
   215  						{
   216  							PartitionIndex: 0,
   217  						},
   218  					},
   219  				},
   220  			},
   221  		})
   222  	}()
   223  
   224  	pool := &connPool{
   225  		ready: ready,
   226  		wake:  wake,
   227  		conns: map[int32]*connGroup{},
   228  	}
   229  
   230  	// configure the state,
   231  	//
   232  	// set cached metadata only have good topic,
   233  	// so it need to request metadata,
   234  	// caused by unknown topic cannot find in cached metadata
   235  	//
   236  	// set layout only have good topic,
   237  	// so it can find the good topic, but not the one that fails to create
   238  	pool.setState(connPoolState{
   239  		metadata: &meta.Response{
   240  			Topics: []meta.ResponseTopic{
   241  				{
   242  					Name: okTopicName,
   243  					Partitions: []meta.ResponsePartition{
   244  						{
   245  							PartitionIndex: 0,
   246  						},
   247  					},
   248  				},
   249  			},
   250  		},
   251  		layout: protocol.Cluster{
   252  			Topics: map[string]protocol.Topic{
   253  				okTopicName: {
   254  					Name: okTopicName,
   255  					Partitions: map[int32]protocol.Partition{
   256  						0: {},
   257  					},
   258  				},
   259  			},
   260  		},
   261  	})
   262  
   263  	// trick the connection pool into thinking it has a valid connection to request metadata
   264  	pool.ctrl = &connGroup{
   265  		pool:   pool,
   266  		broker: Broker{},
   267  		idleConns: []*conn{
   268  			{
   269  				reqs: requests,
   270  			},
   271  		},
   272  	}
   273  
   274  	// perform the round trip:
   275  	// - if the issue is presenting this will hang waiting for metadata to arrive that will
   276  	//   never arrive, causing a deadline timeout.
   277  	// - if the issue is fixed this will resolve almost instantaneously
   278  	r, err := pool.roundTrip(ctx, &meta.Request{
   279  		TopicNames:             []string{unknownTopicName},
   280  		AllowAutoTopicCreation: true,
   281  	})
   282  	// detect if the issue is presenting using the context timeout (note that checking the err return value
   283  	// isn't good enough as the original implementation didn't return the context cancellation error due to
   284  	// being run in a defer)
   285  	if errors.Is(ctx.Err(), context.DeadlineExceeded) {
   286  		t.Fatalf("issue 806 is presenting! roundTrip should not have timed out")
   287  	}
   288  
   289  	// ancillary assertions as general house-keeping, not directly related to the issue:
   290  
   291  	// we're not expecting any errors in this test
   292  	if err != nil {
   293  		t.Fatalf("unexpected error provoking connection pool roundTrip: %v", err)
   294  	}
   295  
   296  	// we expect a response containing the errors from the broker
   297  	if r == nil {
   298  		t.Fatal("expected a non-nil response")
   299  	}
   300  
   301  	// we expect to have the create topic response with created earlier
   302  	_, ok := r.(*meta.Response)
   303  	if !ok {
   304  		t.Fatalf("expected a meta.Response but got %T", r)
   305  	}
   306  }