github.com/rbisecke/kafka-go@v0.4.27/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/rbisecke/kafka-go/protocol" 12 "github.com/rbisecke/kafka-go/protocol/createtopics" 13 ) 14 15 func TestIssue477(t *testing.T) { 16 // This test verifies that a connection attempt with a minimal TLS 17 // configuration does not panic. 18 l, err := net.Listen("tcp", "127.0.0.1:0") 19 if err != nil { 20 t.Fatal(err) 21 } 22 defer l.Close() 23 24 cg := connGroup{ 25 addr: l.Addr(), 26 pool: &connPool{ 27 dial: defaultDialer.DialContext, 28 tls: &tls.Config{}, 29 }, 30 } 31 32 if _, err := cg.connect(context.Background(), cg.addr); err != nil { 33 // An error is expected here because we are not actually establishing 34 // a TLS connection to a kafka broker. 35 t.Log(err) 36 } else { 37 t.Error("no error was reported when attempting to establish a TLS connection to a non-TLS endpoint") 38 } 39 } 40 41 func TestIssue672(t *testing.T) { 42 // ensure the test times out if the bug is re-introduced 43 ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) 44 defer cancel() 45 46 // we'll simulate a situation with one good topic and one bad topic (bad configuration) 47 const brokenTopicName = "bad-topic" 48 const okTopicName = "good-topic" 49 50 // make the connection pool think it's immediately ready to send 51 ready := make(chan struct{}) 52 close(ready) 53 54 // allow the system to wake as much as it wants 55 wake := make(chan event) 56 defer close(wake) 57 go func() { 58 for { 59 select { 60 case <-ctx.Done(): 61 return 62 case e := <-wake: 63 if e == nil { 64 return 65 } 66 e.trigger() 67 } 68 } 69 }() 70 71 // handle requests by immediately resolving them with a create topics response, 72 // the "bad topic" will have an error value 73 requests := make(chan connRequest, 1) 74 defer close(requests) 75 go func() { 76 request := <-requests 77 request.res.resolve(&createtopics.Response{ 78 ThrottleTimeMs: 0, 79 Topics: []createtopics.ResponseTopic{ 80 { 81 Name: brokenTopicName, 82 ErrorCode: int16(InvalidPartitionNumber), 83 ErrorMessage: InvalidPartitionNumber.Description(), 84 }, 85 { 86 Name: okTopicName, 87 NumPartitions: 1, 88 ReplicationFactor: 1, 89 }, 90 }, 91 }) 92 }() 93 94 pool := &connPool{ 95 ready: ready, 96 wake: wake, 97 conns: map[int32]*connGroup{}, 98 } 99 100 // configure the state so it can find the good topic, but not the one that fails to create 101 pool.setState(connPoolState{ 102 layout: protocol.Cluster{ 103 Topics: map[string]protocol.Topic{ 104 okTopicName: { 105 Name: okTopicName, 106 Partitions: map[int32]protocol.Partition{ 107 0: {}, 108 }, 109 }, 110 }, 111 }, 112 }) 113 114 // trick the connection pool into thinking it has a valid connection to a broker 115 pool.conns[0] = &connGroup{ 116 pool: pool, 117 broker: Broker{}, 118 idleConns: []*conn{ 119 { 120 reqs: requests, 121 }, 122 }, 123 } 124 125 // perform the round trip: 126 // - if the issue is presenting this will hang waiting for metadata to arrive that will 127 // never arrive, causing a deadline timeout. 128 // - if the issue is fixed this will resolve almost instantaneously 129 r, err := pool.roundTrip(ctx, &createtopics.Request{ 130 Topics: []createtopics.RequestTopic{ 131 { 132 Name: brokenTopicName, 133 NumPartitions: 0, 134 ReplicationFactor: 1, 135 }, 136 { 137 Name: okTopicName, 138 NumPartitions: 1, 139 ReplicationFactor: 1, 140 }, 141 }, 142 }) 143 // detect if the issue is presenting using the context timeout (note that checking the err return value 144 // isn't good enough as the original implementation didn't return the context cancellation error due to 145 // being run in a defer) 146 if errors.Is(ctx.Err(), context.DeadlineExceeded) { 147 t.Fatalf("issue 672 is presenting! roundTrip should not have timed out") 148 } 149 150 // ancillary assertions as general house-keeping, not directly related to the issue: 151 152 // we're not expecting any errors in this test 153 if err != nil { 154 t.Fatalf("unexpected error provoking connection pool roundTrip: %v", err) 155 } 156 157 // we expect a response containing the errors from the broker 158 if r == nil { 159 t.Fatal("expected a non-nil response") 160 } 161 162 // we expect to have the create topic response with created earlier 163 _, ok := r.(*createtopics.Response) 164 if !ok { 165 t.Fatalf("expected a createtopics.Response but got %T", r) 166 } 167 }