Explorar o código

Shuffle seed brokers so we don't always connect to the first one provided

Willem van Bergen %!s(int64=10) %!d(string=hai) anos
pai
achega
1874f69e22
Modificáronse 2 ficheiros con 22 adicións e 11 borrados
  1. 5 2
      client.go
  2. 17 9
      client_test.go

+ 5 - 2
client.go

@@ -1,6 +1,7 @@
 package sarama
 
 import (
+	"math/rand"
 	"sort"
 	"sync"
 	"time"
@@ -117,8 +118,10 @@ func NewClient(addrs []string, conf *Config) (Client, error) {
 		cachedPartitionsResults: make(map[string][maxPartitionIndex][]int32),
 		coordinators:            make(map[string]int32),
 	}
-	for _, addr := range addrs {
-		client.seedBrokers = append(client.seedBrokers, NewBroker(addr))
+
+	random := rand.New(rand.NewSource(time.Now().UnixNano()))
+	for _, index := range random.Perm(len(addrs)) {
+		client.seedBrokers = append(client.seedBrokers, NewBroker(addrs[index]))
 	}
 
 	// do an initial fetch of all cluster metadata by specifing an empty list of topics

+ 17 - 9
client_test.go

@@ -375,25 +375,33 @@ func TestClientRefreshBehaviour(t *testing.T) {
 }
 
 func TestClientResurrectDeadSeeds(t *testing.T) {
-	seed1 := newMockBroker(t, 1)
-	seed2 := newMockBroker(t, 2)
-	seed3 := newMockBroker(t, 3)
-	addr1 := seed1.Addr()
-	addr2 := seed2.Addr()
-	addr3 := seed3.Addr()
-
+	initialSeed := newMockBroker(t, 0)
 	emptyMetadata := new(MetadataResponse)
-	seed1.Returns(emptyMetadata)
+	initialSeed.Returns(emptyMetadata)
 
 	conf := NewConfig()
 	conf.Metadata.Retry.Backoff = 0
 	conf.Metadata.RefreshFrequency = 0
-	c, err := NewClient([]string{addr1, addr2, addr3}, conf)
+	c, err := NewClient([]string{initialSeed.Addr()}, conf)
 	if err != nil {
 		t.Fatal(err)
 	}
+	initialSeed.Close()
+
 	client := c.(*client)
 
+	seed1 := newMockBroker(t, 1)
+	seed2 := newMockBroker(t, 2)
+	seed3 := newMockBroker(t, 3)
+	addr1 := seed1.Addr()
+	addr2 := seed2.Addr()
+	addr3 := seed3.Addr()
+
+	// Overwrite the seed brokers with a fixed ordering to make this test deterministic.
+	safeClose(t, client.seedBrokers[0])
+	client.seedBrokers = []*Broker{NewBroker(addr1), NewBroker(addr2), NewBroker(addr3)}
+	client.deadSeeds = []*Broker{}
+
 	wg := sync.WaitGroup{}
 	wg.Add(1)
 	go func() {