|
@@ -111,20 +111,9 @@ type Lessor interface {
|
|
|
type lessor struct {
|
|
type lessor struct {
|
|
|
mu sync.Mutex
|
|
mu sync.Mutex
|
|
|
|
|
|
|
|
- // primary indicates if this lessor is the primary lessor. The primary
|
|
|
|
|
- // lessor manages lease expiration and renew.
|
|
|
|
|
- //
|
|
|
|
|
- // in etcd, raft leader is the primary. Thus there might be two primary
|
|
|
|
|
- // leaders at the same time (raft allows concurrent leader but with different term)
|
|
|
|
|
- // for at most a leader election timeout.
|
|
|
|
|
- // The old primary leader cannot affect the correctness since its proposal has a
|
|
|
|
|
- // smaller term and will not be committed.
|
|
|
|
|
- //
|
|
|
|
|
- // TODO: raft follower do not forward lease management proposals. There might be a
|
|
|
|
|
- // very small window (within second normally which depends on go scheduling) that
|
|
|
|
|
- // a raft follow is the primary between the raft leader demotion and lessor demotion.
|
|
|
|
|
- // Usually this should not be a problem. Lease should not be that sensitive to timing.
|
|
|
|
|
- primary bool
|
|
|
|
|
|
|
+ // demotec is set when the lessor is the primary.
|
|
|
|
|
+ // demotec will be closed if the lessor is demoted.
|
|
|
|
|
+ demotec chan struct{}
|
|
|
|
|
|
|
|
// TODO: probably this should be a heap with a secondary
|
|
// TODO: probably this should be a heap with a secondary
|
|
|
// id index.
|
|
// id index.
|
|
@@ -174,6 +163,23 @@ func newLessor(b backend.Backend, minLeaseTTL int64) *lessor {
|
|
|
return l
|
|
return l
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
|
|
+// isPrimary indicates if this lessor is the primary lessor. The primary
|
|
|
|
|
+// lessor manages lease expiration and renew.
|
|
|
|
|
+//
|
|
|
|
|
+// in etcd, raft leader is the primary. Thus there might be two primary
|
|
|
|
|
+// leaders at the same time (raft allows concurrent leader but with different term)
|
|
|
|
|
+// for at most a leader election timeout.
|
|
|
|
|
+// The old primary leader cannot affect the correctness since its proposal has a
|
|
|
|
|
+// smaller term and will not be committed.
|
|
|
|
|
+//
|
|
|
|
|
+// TODO: raft follower do not forward lease management proposals. There might be a
|
|
|
|
|
+// very small window (within second normally which depends on go scheduling) that
|
|
|
|
|
+// a raft follow is the primary between the raft leader demotion and lessor demotion.
|
|
|
|
|
+// Usually this should not be a problem. Lease should not be that sensitive to timing.
|
|
|
|
|
+func (le *lessor) isPrimary() bool {
|
|
|
|
|
+ return le.demotec != nil
|
|
|
|
|
+}
|
|
|
|
|
+
|
|
|
func (le *lessor) SetRangeDeleter(rd RangeDeleter) {
|
|
func (le *lessor) SetRangeDeleter(rd RangeDeleter) {
|
|
|
le.mu.Lock()
|
|
le.mu.Lock()
|
|
|
defer le.mu.Unlock()
|
|
defer le.mu.Unlock()
|
|
@@ -188,7 +194,12 @@ func (le *lessor) Grant(id LeaseID, ttl int64) (*Lease, error) {
|
|
|
|
|
|
|
|
// TODO: when lessor is under high load, it should give out lease
|
|
// TODO: when lessor is under high load, it should give out lease
|
|
|
// with longer TTL to reduce renew load.
|
|
// with longer TTL to reduce renew load.
|
|
|
- l := &Lease{ID: id, TTL: ttl, itemSet: make(map[LeaseItem]struct{})}
|
|
|
|
|
|
|
+ l := &Lease{
|
|
|
|
|
+ ID: id,
|
|
|
|
|
+ TTL: ttl,
|
|
|
|
|
+ itemSet: make(map[LeaseItem]struct{}),
|
|
|
|
|
+ revokec: make(chan struct{}),
|
|
|
|
|
+ }
|
|
|
|
|
|
|
|
le.mu.Lock()
|
|
le.mu.Lock()
|
|
|
defer le.mu.Unlock()
|
|
defer le.mu.Unlock()
|
|
@@ -201,7 +212,7 @@ func (le *lessor) Grant(id LeaseID, ttl int64) (*Lease, error) {
|
|
|
l.TTL = le.minLeaseTTL
|
|
l.TTL = le.minLeaseTTL
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
- if le.primary {
|
|
|
|
|
|
|
+ if le.isPrimary() {
|
|
|
l.refresh(0)
|
|
l.refresh(0)
|
|
|
} else {
|
|
} else {
|
|
|
l.forever()
|
|
l.forever()
|
|
@@ -221,6 +232,7 @@ func (le *lessor) Revoke(id LeaseID) error {
|
|
|
le.mu.Unlock()
|
|
le.mu.Unlock()
|
|
|
return ErrLeaseNotFound
|
|
return ErrLeaseNotFound
|
|
|
}
|
|
}
|
|
|
|
|
+ defer close(l.revokec)
|
|
|
// unlock before doing external work
|
|
// unlock before doing external work
|
|
|
le.mu.Unlock()
|
|
le.mu.Unlock()
|
|
|
|
|
|
|
@@ -264,18 +276,40 @@ func (le *lessor) Revoke(id LeaseID) error {
|
|
|
// has expired, an error will be returned.
|
|
// has expired, an error will be returned.
|
|
|
func (le *lessor) Renew(id LeaseID) (int64, error) {
|
|
func (le *lessor) Renew(id LeaseID) (int64, error) {
|
|
|
le.mu.Lock()
|
|
le.mu.Lock()
|
|
|
- defer le.mu.Unlock()
|
|
|
|
|
|
|
|
|
|
- if !le.primary {
|
|
|
|
|
|
|
+ unlock := func() { le.mu.Unlock() }
|
|
|
|
|
+ defer func() { unlock() }()
|
|
|
|
|
+
|
|
|
|
|
+ if !le.isPrimary() {
|
|
|
// forward renew request to primary instead of returning error.
|
|
// forward renew request to primary instead of returning error.
|
|
|
return -1, ErrNotPrimary
|
|
return -1, ErrNotPrimary
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
|
|
+ demotec := le.demotec
|
|
|
|
|
+
|
|
|
l := le.leaseMap[id]
|
|
l := le.leaseMap[id]
|
|
|
if l == nil {
|
|
if l == nil {
|
|
|
return -1, ErrLeaseNotFound
|
|
return -1, ErrLeaseNotFound
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
|
|
+ if l.expired() {
|
|
|
|
|
+ le.mu.Unlock()
|
|
|
|
|
+ unlock = func() {}
|
|
|
|
|
+ select {
|
|
|
|
|
+ // A expired lease might be pending for revoking or going through
|
|
|
|
|
+ // quorum to be revoked. To be accurate, renew request must wait for the
|
|
|
|
|
+ // deletion to complete.
|
|
|
|
|
+ case <-l.revokec:
|
|
|
|
|
+ return -1, ErrLeaseNotFound
|
|
|
|
|
+ // The expired lease might fail to be revoked if the primary changes.
|
|
|
|
|
+ // The caller will retry on ErrNotPrimary.
|
|
|
|
|
+ case <-demotec:
|
|
|
|
|
+ return -1, ErrNotPrimary
|
|
|
|
|
+ case <-le.stopC:
|
|
|
|
|
+ return -1, ErrNotPrimary
|
|
|
|
|
+ }
|
|
|
|
|
+ }
|
|
|
|
|
+
|
|
|
l.refresh(0)
|
|
l.refresh(0)
|
|
|
return l.TTL, nil
|
|
return l.TTL, nil
|
|
|
}
|
|
}
|
|
@@ -290,7 +324,7 @@ func (le *lessor) Promote(extend time.Duration) {
|
|
|
le.mu.Lock()
|
|
le.mu.Lock()
|
|
|
defer le.mu.Unlock()
|
|
defer le.mu.Unlock()
|
|
|
|
|
|
|
|
- le.primary = true
|
|
|
|
|
|
|
+ le.demotec = make(chan struct{})
|
|
|
|
|
|
|
|
// refresh the expiries of all leases.
|
|
// refresh the expiries of all leases.
|
|
|
for _, l := range le.leaseMap {
|
|
for _, l := range le.leaseMap {
|
|
@@ -307,7 +341,10 @@ func (le *lessor) Demote() {
|
|
|
l.forever()
|
|
l.forever()
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
- le.primary = false
|
|
|
|
|
|
|
+ if le.demotec != nil {
|
|
|
|
|
+ close(le.demotec)
|
|
|
|
|
+ le.demotec = nil
|
|
|
|
|
+ }
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
// Attach attaches items to the lease with given ID. When the lease
|
|
// Attach attaches items to the lease with given ID. When the lease
|
|
@@ -372,7 +409,7 @@ func (le *lessor) runLoop() {
|
|
|
var ls []*Lease
|
|
var ls []*Lease
|
|
|
|
|
|
|
|
le.mu.Lock()
|
|
le.mu.Lock()
|
|
|
- if le.primary {
|
|
|
|
|
|
|
+ if le.isPrimary() {
|
|
|
ls = le.findExpiredLeases()
|
|
ls = le.findExpiredLeases()
|
|
|
}
|
|
}
|
|
|
le.mu.Unlock()
|
|
le.mu.Unlock()
|
|
@@ -401,12 +438,11 @@ func (le *lessor) runLoop() {
|
|
|
// leases that needed to be revoked.
|
|
// leases that needed to be revoked.
|
|
|
func (le *lessor) findExpiredLeases() []*Lease {
|
|
func (le *lessor) findExpiredLeases() []*Lease {
|
|
|
leases := make([]*Lease, 0, 16)
|
|
leases := make([]*Lease, 0, 16)
|
|
|
- now := time.Now()
|
|
|
|
|
|
|
|
|
|
for _, l := range le.leaseMap {
|
|
for _, l := range le.leaseMap {
|
|
|
// TODO: probably should change to <= 100-500 millisecond to
|
|
// TODO: probably should change to <= 100-500 millisecond to
|
|
|
// make up committing latency.
|
|
// make up committing latency.
|
|
|
- if l.expiry.Sub(now) <= 0 {
|
|
|
|
|
|
|
+ if l.expired() {
|
|
|
leases = append(leases, l)
|
|
leases = append(leases, l)
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
@@ -439,6 +475,7 @@ func (le *lessor) initAndRecover() {
|
|
|
// set expiry to forever, refresh when promoted
|
|
// set expiry to forever, refresh when promoted
|
|
|
itemSet: make(map[LeaseItem]struct{}),
|
|
itemSet: make(map[LeaseItem]struct{}),
|
|
|
expiry: forever,
|
|
expiry: forever,
|
|
|
|
|
+ revokec: make(chan struct{}),
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
tx.Unlock()
|
|
tx.Unlock()
|
|
@@ -452,7 +489,12 @@ type Lease struct {
|
|
|
|
|
|
|
|
itemSet map[LeaseItem]struct{}
|
|
itemSet map[LeaseItem]struct{}
|
|
|
// expiry time in unixnano
|
|
// expiry time in unixnano
|
|
|
- expiry time.Time
|
|
|
|
|
|
|
+ expiry time.Time
|
|
|
|
|
+ revokec chan struct{}
|
|
|
|
|
+}
|
|
|
|
|
+
|
|
|
|
|
+func (l Lease) expired() bool {
|
|
|
|
|
+ return l.Remaining() <= 0
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
func (l Lease) persistTo(b backend.Backend) {
|
|
func (l Lease) persistTo(b backend.Backend) {
|