Skip to content

Commit

Permalink
server: drop read request if found leader is changed
Browse files Browse the repository at this point in the history
  • Loading branch information
nolouch committed Sep 14, 2018
1 parent a7b1306 commit 3b97723
Show file tree
Hide file tree
Showing 5 changed files with 59 additions and 1 deletion.
38 changes: 38 additions & 0 deletions clientv3/integration/network_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package integration
import (
"context"
"errors"
"strings"
"testing"
"time"

Expand Down Expand Up @@ -265,3 +266,40 @@ func testBalancerUnderNetworkPartitionWatch(t *testing.T, isolateLeader bool) {
t.Fatal("took too long to detect leader lost")
}
}

func TestDropReadUnderNetworkPartition(t *testing.T) {
defer testutil.AfterTest(t)

clus := integration.NewClusterV3(t, &integration.ClusterConfig{
Size: 3,
SkipCreatingClient: true,
})
defer clus.Terminate(t)
leaderIndex := clus.WaitLeader(t)
// get a follower endpoint
eps := []string{clus.Members[(leaderIndex+1)%3].GRPCAddr()}
ccfg := clientv3.Config{
Endpoints: eps,
DialTimeout: 10 * time.Second,
DialOptions: []grpc.DialOption{grpc.WithBlock()},
}
cli, err := clientv3.New(ccfg)
if err != nil {
t.Fatal(err)
}
defer cli.Close()

// wait for eps[0] to be pinned
mustWaitPinReady(t, cli)

// add other endpoints for later endpoint switch
cli.SetEndpoints(eps...)
time.Sleep(time.Second * 2)
clus.Members[leaderIndex].InjectPartition(t, clus.Members[(leaderIndex+1)%3], clus.Members[(leaderIndex+2)%3])
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
_, err = cli.Get(ctx, "a")
cancel()
if !strings.Contains(err.Error(), "leader is changed") {
t.Errorf("expect leader is changed error, but get %s", err)
}
}
1 change: 1 addition & 0 deletions etcdserver/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ var (
ErrTimeoutDueToLeaderFail = errors.New("etcdserver: request timed out, possibly due to previous leader failure")
ErrTimeoutDueToConnectionLost = errors.New("etcdserver: request timed out, possibly due to connection lost")
ErrTimeoutLeaderTransfer = errors.New("etcdserver: request timed out, leader transfer took too long")
ErrLeaderChanged = errors.New("etcdserver: leader is changed")
ErrNotEnoughStartedMembers = errors.New("etcdserver: re-configuration failed due to not enough started members")
ErrNoLeader = errors.New("etcdserver: no leader")
ErrNotLeader = errors.New("etcdserver: not leader")
Expand Down
7 changes: 7 additions & 0 deletions etcdserver/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,12 @@ var (
Name: "slow_read_indexes_total",
Help: "The total number of pending read indexes not in sync with leader's or timed out read index requests.",
})
readIndexFailed = prometheus.NewCounter(prometheus.CounterOpts{
Namespace: "etcd",
Subsystem: "server",
Name: "read_indexes_failed_total",
Help: "The total number of failed read indexes seen.",
})
leaseExpired = prometheus.NewCounter(prometheus.CounterOpts{
Namespace: "etcd_debugging",
Subsystem: "server",
Expand Down Expand Up @@ -132,6 +138,7 @@ func init() {
prometheus.MustRegister(proposalsPending)
prometheus.MustRegister(proposalsFailed)
prometheus.MustRegister(slowReadIndex)
prometheus.MustRegister(readIndexFailed)
prometheus.MustRegister(leaseExpired)
prometheus.MustRegister(quotaBackendBytes)
prometheus.MustRegister(currentVersion)
Expand Down
7 changes: 6 additions & 1 deletion etcdserver/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -212,6 +212,8 @@ type EtcdServer struct {
stopping chan struct{}
// done is closed when all goroutines from start() complete.
done chan struct{}
// leaderChanged is used to notify the linearizable read loop to drop the old read requests.
leaderChanged chan struct{}

errorc chan error
id types.ID
Expand Down Expand Up @@ -752,6 +754,7 @@ func (s *EtcdServer) start() {
s.ctx, s.cancel = context.WithCancel(context.Background())
s.readwaitc = make(chan struct{}, 1)
s.readNotifier = newNotifier()
s.leaderChanged = make(chan struct{}, 1)
if s.ClusterVersion() != nil {
if lg != nil {
lg.Info(
Expand Down Expand Up @@ -938,7 +941,9 @@ func (s *EtcdServer) run() {
s.compactor.Resume()
}
}

if newLeader {
s.leaderChanged <- struct{}{}
}
// TODO: remove the nil checking
// current test utility does not provide the stats
if s.stats != nil {
Expand Down
7 changes: 7 additions & 0 deletions etcdserver/v3_server.go
Original file line number Diff line number Diff line change
Expand Up @@ -636,6 +636,8 @@ func (s *EtcdServer) linearizableReadLoop() {
binary.BigEndian.PutUint64(ctxToSend, id1)

select {
case <-s.leaderChanged:
continue
case <-s.readwaitc:
case <-s.stopping:
return
Expand All @@ -660,6 +662,7 @@ func (s *EtcdServer) linearizableReadLoop() {
} else {
plog.Errorf("failed to get read index from raft: %v", err)
}
readIndexFailed.Inc()
nr.notify(err)
continue
}
Expand Down Expand Up @@ -691,6 +694,10 @@ func (s *EtcdServer) linearizableReadLoop() {
}
slowReadIndex.Inc()
}
case <-s.leaderChanged:
timeout = true
readIndexFailed.Inc()
nr.notify(ErrLeaderChanged)
case <-time.After(s.Cfg.ReqTimeout()):
if lg != nil {
lg.Warn("timed out waiting for read index response (local node might have slow network)", zap.Duration("timeout", s.Cfg.ReqTimeout()))
Expand Down

0 comments on commit 3b97723

Please sign in to comment.