|
@@ -20,7 +20,10 @@ import (
|
|
|
"io/ioutil"
|
|
"io/ioutil"
|
|
|
"os"
|
|
"os"
|
|
|
"path"
|
|
"path"
|
|
|
|
|
+ "sync"
|
|
|
|
|
+ "time"
|
|
|
|
|
|
|
|
|
|
+ "github.com/coreos/etcd/Godeps/_workspace/src/github.com/jonboulle/clockwork"
|
|
|
"github.com/coreos/etcd/pkg/fileutil"
|
|
"github.com/coreos/etcd/pkg/fileutil"
|
|
|
"github.com/coreos/etcd/raft"
|
|
"github.com/coreos/etcd/raft"
|
|
|
"github.com/coreos/etcd/raft/raftpb"
|
|
"github.com/coreos/etcd/raft/raftpb"
|
|
@@ -28,6 +31,14 @@ import (
|
|
|
dstorage "github.com/coreos/etcd/storage"
|
|
dstorage "github.com/coreos/etcd/storage"
|
|
|
)
|
|
)
|
|
|
|
|
|
|
|
|
|
+// clearUnusedSnapshotInterval specifies the time interval to wait
|
|
|
|
|
+// before clearing unused snapshot.
|
|
|
|
|
+// The newly created snapshot should be retrieved within one heartbeat
|
|
|
|
|
+// interval because raft state machine retries to send snapshot
|
|
|
|
|
+// to slow follower when receiving MsgHeartbeatResp from the follower.
|
|
|
|
|
+// Set it as 5s to match the upper limit of heartbeat interval.
|
|
|
|
|
+const clearUnusedSnapshotInterval = 5 * time.Second
|
|
|
|
|
+
|
|
|
type snapshot struct {
|
|
type snapshot struct {
|
|
|
r raftpb.Snapshot
|
|
r raftpb.Snapshot
|
|
|
|
|
|
|
@@ -78,7 +89,13 @@ type snapshotStore struct {
|
|
|
// snapshotStore will receive from the chan immediately after it sends empty to reqsnapc
|
|
// snapshotStore will receive from the chan immediately after it sends empty to reqsnapc
|
|
|
raftsnapc chan raftpb.Snapshot
|
|
raftsnapc chan raftpb.Snapshot
|
|
|
|
|
|
|
|
- snap *snapshot
|
|
|
|
|
|
|
+ mu sync.Mutex // protect belowing vars
|
|
|
|
|
+ // snap is nil iff there is no snapshot stored
|
|
|
|
|
+ snap *snapshot
|
|
|
|
|
+ inUse bool
|
|
|
|
|
+ createOnce sync.Once // ensure at most one snapshot is created when no snapshot stored
|
|
|
|
|
+
|
|
|
|
|
+ clock clockwork.Clock
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
func newSnapshotStore(dir string, kv dstorage.KV) *snapshotStore {
|
|
func newSnapshotStore(dir string, kv dstorage.KV) *snapshotStore {
|
|
@@ -87,35 +104,113 @@ func newSnapshotStore(dir string, kv dstorage.KV) *snapshotStore {
|
|
|
kv: kv,
|
|
kv: kv,
|
|
|
reqsnapc: make(chan struct{}),
|
|
reqsnapc: make(chan struct{}),
|
|
|
raftsnapc: make(chan raftpb.Snapshot),
|
|
raftsnapc: make(chan raftpb.Snapshot),
|
|
|
|
|
+ clock: clockwork.NewRealClock(),
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
// getSnap returns a snapshot.
|
|
// getSnap returns a snapshot.
|
|
|
// If there is no available snapshot, ErrSnapshotTemporarilyUnavaliable will be returned.
|
|
// If there is no available snapshot, ErrSnapshotTemporarilyUnavaliable will be returned.
|
|
|
//
|
|
//
|
|
|
-// Internally it creates new snapshot and returns the snapshot. Unless the
|
|
|
|
|
-// returned snapshot is closed, it rejects creating new one and returns
|
|
|
|
|
-// ErrSnapshotTemporarilyUnavailable.
|
|
|
|
|
|
|
+// If the snapshot stored is in use, it returns ErrSnapshotTemporarilyUnavailable.
|
|
|
|
|
+// If there is no snapshot stored, it creates new snapshot
|
|
|
|
|
+// asynchronously and returns ErrSnapshotTemporarilyUnavailable, so
|
|
|
|
|
+// caller could get snapshot later when the snapshot is created.
|
|
|
|
|
+// Otherwise, it returns the snapshot stored.
|
|
|
|
|
+//
|
|
|
|
|
+// The created snapshot is cleared from the snapshot store if it is
|
|
|
|
|
+// either unused after clearUnusedSnapshotInterval, or explicitly cleared
|
|
|
|
|
+// through clearUsedSnap after using.
|
|
|
|
|
+// closeSnapBefore is used to close outdated snapshot,
|
|
|
|
|
+// so the snapshot will be cleared faster when in use.
|
|
|
|
|
+//
|
|
|
|
|
+// snapshot store stores at most one snapshot at a time.
|
|
|
// If raft state machine wants to send two snapshot messages to two followers,
|
|
// If raft state machine wants to send two snapshot messages to two followers,
|
|
|
// the second snapshot message will keep getting snapshot and succeed only after
|
|
// the second snapshot message will keep getting snapshot and succeed only after
|
|
|
// the first message is sent. This increases the time used to send messages,
|
|
// the first message is sent. This increases the time used to send messages,
|
|
|
// but it is acceptable because this should happen seldomly.
|
|
// but it is acceptable because this should happen seldomly.
|
|
|
func (ss *snapshotStore) getSnap() (*snapshot, error) {
|
|
func (ss *snapshotStore) getSnap() (*snapshot, error) {
|
|
|
- // If snapshotStore has some snapshot that has not been closed, it cannot
|
|
|
|
|
- // request new snapshot. So it returns ErrSnapshotTemporarilyUnavailable.
|
|
|
|
|
- if ss.snap != nil && !ss.snap.isClosed() {
|
|
|
|
|
|
|
+ ss.mu.Lock()
|
|
|
|
|
+ defer ss.mu.Unlock()
|
|
|
|
|
+
|
|
|
|
|
+ if ss.inUse {
|
|
|
|
|
+ return nil, raft.ErrSnapshotTemporarilyUnavailable
|
|
|
|
|
+ }
|
|
|
|
|
+
|
|
|
|
|
+ if ss.snap == nil {
|
|
|
|
|
+ // create snapshot asynchronously
|
|
|
|
|
+ ss.createOnce.Do(func() { go ss.createSnap() })
|
|
|
return nil, raft.ErrSnapshotTemporarilyUnavailable
|
|
return nil, raft.ErrSnapshotTemporarilyUnavailable
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
|
|
+ ss.inUse = true
|
|
|
|
|
+ // give transporter the generated snapshot that is ready to send out
|
|
|
|
|
+ ss.tr.SnapshotReady(ss.snap, ss.snap.raft().Metadata.Index)
|
|
|
|
|
+ return ss.snap, nil
|
|
|
|
|
+}
|
|
|
|
|
+
|
|
|
|
|
+// clearUsedSnap clears the snapshot from the snapshot store after it
|
|
|
|
|
+// is used.
|
|
|
|
|
+// After clear, snapshotStore could create new snapshot when getSnap.
|
|
|
|
|
+func (ss *snapshotStore) clearUsedSnap() {
|
|
|
|
|
+ ss.mu.Lock()
|
|
|
|
|
+ defer ss.mu.Unlock()
|
|
|
|
|
+ if !ss.inUse {
|
|
|
|
|
+ plog.Panicf("unexpected clearUsedSnap when snapshot is not in use")
|
|
|
|
|
+ }
|
|
|
|
|
+ ss.clear()
|
|
|
|
|
+}
|
|
|
|
|
+
|
|
|
|
|
+// closeSnapBefore closes the stored snapshot if its index is not greater
|
|
|
|
|
+// than the given compact index.
|
|
|
|
|
+// If it closes the snapshot, it returns true.
|
|
|
|
|
+func (ss *snapshotStore) closeSnapBefore(index uint64) bool {
|
|
|
|
|
+ ss.mu.Lock()
|
|
|
|
|
+ defer ss.mu.Unlock()
|
|
|
|
|
+ if ss.snap != nil && ss.snap.raft().Metadata.Index <= index {
|
|
|
|
|
+ if err := ss.snap.Close(); err != nil {
|
|
|
|
|
+ plog.Errorf("snapshot close error (%v)", err)
|
|
|
|
|
+ }
|
|
|
|
|
+ return true
|
|
|
|
|
+ }
|
|
|
|
|
+ return false
|
|
|
|
|
+}
|
|
|
|
|
+
|
|
|
|
|
+// createSnap creates a new snapshot and stores it into the snapshot store.
|
|
|
|
|
+// It also sets a timer to clear the snapshot if it is not in use after
|
|
|
|
|
+// some time interval.
|
|
|
|
|
+// It should only be called in snapshotStore functions.
|
|
|
|
|
+func (ss *snapshotStore) createSnap() {
|
|
|
// ask to generate v2 snapshot
|
|
// ask to generate v2 snapshot
|
|
|
ss.reqsnapc <- struct{}{}
|
|
ss.reqsnapc <- struct{}{}
|
|
|
// generate KV snapshot
|
|
// generate KV snapshot
|
|
|
kvsnap := ss.kv.Snapshot()
|
|
kvsnap := ss.kv.Snapshot()
|
|
|
raftsnap := <-ss.raftsnapc
|
|
raftsnap := <-ss.raftsnapc
|
|
|
- ss.snap = newSnapshot(raftsnap, kvsnap)
|
|
|
|
|
- // give transporter the generated snapshot that is ready to send out
|
|
|
|
|
- ss.tr.SnapshotReady(ss.snap, raftsnap.Metadata.Index)
|
|
|
|
|
- return ss.snap, nil
|
|
|
|
|
|
|
+ snap := newSnapshot(raftsnap, kvsnap)
|
|
|
|
|
+
|
|
|
|
|
+ ss.mu.Lock()
|
|
|
|
|
+ ss.snap = snap
|
|
|
|
|
+ ss.mu.Unlock()
|
|
|
|
|
+
|
|
|
|
|
+ go func() {
|
|
|
|
|
+ <-ss.clock.After(clearUnusedSnapshotInterval)
|
|
|
|
|
+ ss.mu.Lock()
|
|
|
|
|
+ defer ss.mu.Unlock()
|
|
|
|
|
+ if snap == ss.snap && !ss.inUse {
|
|
|
|
|
+ ss.clear()
|
|
|
|
|
+ }
|
|
|
|
|
+ }()
|
|
|
|
|
+}
|
|
|
|
|
+
|
|
|
|
|
+// clear clears snapshot related variables in snapshotStore. It closes
|
|
|
|
|
+// the snapshot stored and sets the variables to initial values.
|
|
|
|
|
+// It should only be called in snapshotStore functions.
|
|
|
|
|
+func (ss *snapshotStore) clear() {
|
|
|
|
|
+ if err := ss.snap.Close(); err != nil {
|
|
|
|
|
+ plog.Errorf("snapshot close error (%v)", err)
|
|
|
|
|
+ }
|
|
|
|
|
+ ss.snap = nil
|
|
|
|
|
+ ss.inUse = false
|
|
|
|
|
+ ss.createOnce = sync.Once{}
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
// SaveFrom saves snapshot at the given index from the given reader.
|
|
// SaveFrom saves snapshot at the given index from the given reader.
|