Skip to content

Commit 37b7484

Browse files
committed
lease: Add unit and integration tests for lease checkpointing
1 parent 5e62ae5 commit 37b7484

File tree

3 files changed

+115
-0
lines changed

3 files changed

+115
-0
lines changed

Diff for: integration/cluster.go

+5
Original file line numberDiff line numberDiff line change
@@ -148,6 +148,8 @@ type ClusterConfig struct {
148148

149149
// UseIP is true to use only IP for gRPC requests.
150150
UseIP bool
151+
152+
LeaseCheckpointInterval time.Duration
151153
}
152154

153155
type cluster struct {
@@ -290,6 +292,7 @@ func (c *cluster) mustNewMember(t *testing.T) *member {
290292
clientMaxCallSendMsgSize: c.cfg.ClientMaxCallSendMsgSize,
291293
clientMaxCallRecvMsgSize: c.cfg.ClientMaxCallRecvMsgSize,
292294
useIP: c.cfg.UseIP,
295+
leaseCheckpointInterval: c.cfg.LeaseCheckpointInterval,
293296
})
294297
m.DiscoveryURL = c.cfg.DiscoveryURL
295298
if c.cfg.UseGRPC {
@@ -575,6 +578,7 @@ type memberConfig struct {
575578
clientMaxCallSendMsgSize int
576579
clientMaxCallRecvMsgSize int
577580
useIP bool
581+
leaseCheckpointInterval time.Duration
578582
}
579583

580584
// mustNewMember return an inited member with the given name. If peerTLS is
@@ -665,6 +669,7 @@ func mustNewMember(t *testing.T, mcfg memberConfig) *member {
665669
m.clientMaxCallSendMsgSize = mcfg.clientMaxCallSendMsgSize
666670
m.clientMaxCallRecvMsgSize = mcfg.clientMaxCallRecvMsgSize
667671
m.useIP = mcfg.useIP
672+
m.LeaseCheckpointInterval = mcfg.leaseCheckpointInterval
668673

669674
m.InitialCorruptCheck = true
670675

Diff for: integration/v3_lease_test.go

+52
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,9 @@ import (
2525
"github.com/coreos/etcd/mvcc/mvccpb"
2626
"github.com/coreos/etcd/pkg/testutil"
2727

28+
"google.golang.org/grpc/codes"
2829
"google.golang.org/grpc/metadata"
30+
"google.golang.org/grpc/status"
2931
)
3032

3133
// TestV3LeasePrmote ensures the newly elected leader can promote itself
@@ -222,6 +224,56 @@ func TestV3LeaseKeepAlive(t *testing.T) {
222224
})
223225
}
224226

227+
// TestV3LeaseCheckpoint ensures a lease checkpoint results in a remaining TTL being persisted
228+
// across leader elections.
229+
func TestV3LeaseCheckpoint(t *testing.T) {
230+
var ttl int64 = 300
231+
leaseInterval := 2 * time.Second
232+
defer testutil.AfterTest(t)
233+
clus := NewClusterV3(t, &ClusterConfig{Size: 3, LeaseCheckpointInterval: leaseInterval})
234+
defer clus.Terminate(t)
235+
236+
// create lease
237+
ctx, cancel := context.WithCancel(context.Background())
238+
defer cancel()
239+
c := toGRPC(clus.RandClient())
240+
lresp, err := c.Lease.LeaseGrant(ctx, &pb.LeaseGrantRequest{TTL: ttl})
241+
if err != nil {
242+
t.Fatal(err)
243+
}
244+
245+
// wait for a checkpoint to occur
246+
time.Sleep(leaseInterval + 1*time.Second)
247+
248+
// Force a leader election
249+
leaderId := clus.WaitLeader(t)
250+
leader := clus.Members[leaderId]
251+
leader.Stop(t)
252+
time.Sleep(time.Duration(3*electionTicks) * tickDuration)
253+
leader.Restart(t)
254+
newLeaderId := clus.WaitLeader(t)
255+
c2 := toGRPC(clus.Client(newLeaderId))
256+
257+
time.Sleep(250 * time.Millisecond)
258+
259+
// Check the TTL of the new leader
260+
var ttlresp *pb.LeaseTimeToLiveResponse
261+
for i := 0; i < 10; i++ {
262+
if ttlresp, err = c2.Lease.LeaseTimeToLive(ctx, &pb.LeaseTimeToLiveRequest{ID: lresp.ID}); err != nil {
263+
if status, ok := status.FromError(err); ok && status.Code() == codes.Unavailable {
264+
time.Sleep(time.Millisecond * 250)
265+
} else {
266+
t.Fatal(err)
267+
}
268+
}
269+
}
270+
271+
expectedTTL := ttl - int64(leaseInterval.Seconds())
272+
if ttlresp.TTL < expectedTTL-1 || ttlresp.TTL > expectedTTL {
273+
t.Fatalf("expected lease to be checkpointed after restart such that %d < TTL <%d, but got TTL=%d", expectedTTL-1, expectedTTL, ttlresp.TTL)
274+
}
275+
}
276+
225277
// TestV3LeaseExists creates a lease on a random client and confirms it exists in the cluster.
226278
func TestV3LeaseExists(t *testing.T) {
227279
defer testutil.AfterTest(t)

Diff for: lease/lessor_test.go

+58
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@
1515
package lease
1616

1717
import (
18+
"context"
1819
"fmt"
1920
"io/ioutil"
2021
"os"
@@ -25,6 +26,7 @@ import (
2526
"testing"
2627
"time"
2728

29+
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
2830
"github.com/coreos/etcd/mvcc/backend"
2931
"go.uber.org/zap"
3032
)
@@ -476,6 +478,62 @@ func TestLessorMaxTTL(t *testing.T) {
476478
}
477479
}
478480

481+
func TestLessorCheckpointScheduling(t *testing.T) {
482+
lg := zap.NewNop()
483+
484+
dir, be := NewTestBackend(t)
485+
defer os.RemoveAll(dir)
486+
defer be.Close()
487+
488+
le := newLessor(lg, be, LessorConfig{MinLeaseTTL: minLeaseTTL, CheckpointInterval: 1 * time.Second})
489+
le.minLeaseTTL = 1
490+
checkpointedC := make(chan struct{})
491+
le.SetCheckpointer(func(ctx context.Context, lc *pb.LeaseCheckpointRequest) {
492+
close(checkpointedC)
493+
if len(lc.Checkpoints) != 1 {
494+
t.Errorf("expected 1 checkpoint but got %d", len(lc.Checkpoints))
495+
}
496+
c := lc.Checkpoints[0]
497+
if c.Remaining_TTL != 1 {
498+
t.Errorf("expected checkpoint to be called with Remaining_TTL=%d but got %d", 1, c.Remaining_TTL)
499+
}
500+
})
501+
defer le.Stop()
502+
le.Promote(0)
503+
504+
_, err := le.Grant(1, 2)
505+
if err != nil {
506+
t.Fatal(err)
507+
}
508+
509+
// TODO: Is there any way to avoid doing this wait? Lease TTL granularity is in seconds.
510+
select {
511+
case <-checkpointedC:
512+
case <-time.After(2 * time.Second):
513+
t.Fatal("expected checkpointer to be called, but it was not")
514+
}
515+
}
516+
517+
func TestLessorCheckpointsRestoredOnPromote(t *testing.T) {
518+
lg := zap.NewNop()
519+
dir, be := NewTestBackend(t)
520+
defer os.RemoveAll(dir)
521+
defer be.Close()
522+
523+
le := newLessor(lg, be, LessorConfig{MinLeaseTTL: minLeaseTTL})
524+
defer le.Stop()
525+
l, err := le.Grant(1, 10)
526+
if err != nil {
527+
t.Fatal(err)
528+
}
529+
le.Checkpoint(l.ID, 5)
530+
le.Promote(0)
531+
remaining := l.Remaining().Seconds()
532+
if !(remaining > 4 && remaining < 5) {
533+
t.Fatalf("expected expiry to be less than 1s in the future, but got %f seconds", remaining)
534+
}
535+
}
536+
479537
type fakeDeleter struct {
480538
deleted []string
481539
tx backend.BatchTx

0 commit comments

Comments
 (0)