mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge pull request #1927 from xiang90/flock
*: lock the in using files; do not purge locked the wal files
This commit is contained in:
commit
0ea8c0929e
@ -67,7 +67,7 @@ func handleBackup(c *cli.Context) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
w, err := wal.OpenAtIndex(srcWAL, index)
|
w, err := wal.OpenNotInUse(srcWAL, index)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Fatal(err)
|
log.Fatal(err)
|
||||||
}
|
}
|
||||||
|
@ -90,21 +90,6 @@ type Response struct {
|
|||||||
err error
|
err error
|
||||||
}
|
}
|
||||||
|
|
||||||
type Storage interface {
|
|
||||||
// Save function saves ents and state to the underlying stable storage.
|
|
||||||
// Save MUST block until st and ents are on stable storage.
|
|
||||||
Save(st raftpb.HardState, ents []raftpb.Entry) error
|
|
||||||
// SaveSnap function saves snapshot to the underlying stable storage.
|
|
||||||
SaveSnap(snap raftpb.Snapshot) error
|
|
||||||
|
|
||||||
// TODO: WAL should be able to control cut itself. After implement self-controlled cut,
|
|
||||||
// remove it in this interface.
|
|
||||||
// Cut cuts out a new wal file for saving new state and entries.
|
|
||||||
Cut() error
|
|
||||||
// Close closes the Storage and performs finalization.
|
|
||||||
Close() error
|
|
||||||
}
|
|
||||||
|
|
||||||
type Server interface {
|
type Server interface {
|
||||||
// Start performs any initialization of the Server necessary for it to
|
// Start performs any initialization of the Server necessary for it to
|
||||||
// begin serving requests. It must be called before Do or Process.
|
// begin serving requests. It must be called before Do or Process.
|
||||||
@ -295,15 +280,12 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
|
|||||||
id: id,
|
id: id,
|
||||||
attributes: Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()},
|
attributes: Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()},
|
||||||
Cluster: cfg.Cluster,
|
Cluster: cfg.Cluster,
|
||||||
storage: struct {
|
storage: NewStorage(w, ss),
|
||||||
*wal.WAL
|
stats: sstats,
|
||||||
*snap.Snapshotter
|
lstats: lstats,
|
||||||
}{w, ss},
|
Ticker: time.Tick(100 * time.Millisecond),
|
||||||
stats: sstats,
|
SyncTicker: time.Tick(500 * time.Millisecond),
|
||||||
lstats: lstats,
|
snapCount: cfg.SnapCount,
|
||||||
Ticker: time.Tick(100 * time.Millisecond),
|
|
||||||
SyncTicker: time.Tick(500 * time.Millisecond),
|
|
||||||
snapCount: cfg.SnapCount,
|
|
||||||
}
|
}
|
||||||
srv.sendhub = newSendHub(cfg.Transport, cfg.Cluster, srv, sstats, lstats)
|
srv.sendhub = newSendHub(cfg.Transport, cfg.Cluster, srv, sstats, lstats)
|
||||||
for _, m := range getOtherMembers(cfg.Cluster, cfg.Name) {
|
for _, m := range getOtherMembers(cfg.Cluster, cfg.Name) {
|
||||||
@ -1005,7 +987,7 @@ func restartNode(cfg *ServerConfig, index uint64, snapshot *raftpb.Snapshot) (ty
|
|||||||
|
|
||||||
func readWAL(waldir string, index uint64) (w *wal.WAL, id, cid types.ID, st raftpb.HardState, ents []raftpb.Entry) {
|
func readWAL(waldir string, index uint64) (w *wal.WAL, id, cid types.ID, st raftpb.HardState, ents []raftpb.Entry) {
|
||||||
var err error
|
var err error
|
||||||
if w, err = wal.OpenAtIndex(waldir, index); err != nil {
|
if w, err = wal.Open(waldir, index); err != nil {
|
||||||
log.Fatalf("etcdserver: open wal error: %v", err)
|
log.Fatalf("etcdserver: open wal error: %v", err)
|
||||||
}
|
}
|
||||||
var wmetadata []byte
|
var wmetadata []byte
|
||||||
|
45
etcdserver/storage.go
Normal file
45
etcdserver/storage.go
Normal file
@ -0,0 +1,45 @@
|
|||||||
|
package etcdserver
|
||||||
|
|
||||||
|
import (
|
||||||
|
"github.com/coreos/etcd/raft/raftpb"
|
||||||
|
"github.com/coreos/etcd/snap"
|
||||||
|
"github.com/coreos/etcd/wal"
|
||||||
|
)
|
||||||
|
|
||||||
|
type Storage interface {
|
||||||
|
// Save function saves ents and state to the underlying stable storage.
|
||||||
|
// Save MUST block until st and ents are on stable storage.
|
||||||
|
Save(st raftpb.HardState, ents []raftpb.Entry) error
|
||||||
|
// SaveSnap function saves snapshot to the underlying stable storage.
|
||||||
|
SaveSnap(snap raftpb.Snapshot) error
|
||||||
|
|
||||||
|
// TODO: WAL should be able to control cut itself. After implement self-controlled cut,
|
||||||
|
// remove it in this interface.
|
||||||
|
// Cut cuts out a new wal file for saving new state and entries.
|
||||||
|
Cut() error
|
||||||
|
// Close closes the Storage and performs finalization.
|
||||||
|
Close() error
|
||||||
|
}
|
||||||
|
|
||||||
|
type storage struct {
|
||||||
|
*wal.WAL
|
||||||
|
*snap.Snapshotter
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewStorage(w *wal.WAL, s *snap.Snapshotter) Storage {
|
||||||
|
return &storage{w, s}
|
||||||
|
}
|
||||||
|
|
||||||
|
// SaveSnap saves the snapshot to disk and release the locked
|
||||||
|
// wal files since they will not be used.
|
||||||
|
func (st *storage) SaveSnap(snap raftpb.Snapshot) error {
|
||||||
|
err := st.Snapshotter.SaveSnap(snap)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
err = st.WAL.ReleaseLockTo(snap.Metadata.Index)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
60
pkg/fileutil/lock.go
Normal file
60
pkg/fileutil/lock.go
Normal file
@ -0,0 +1,60 @@
|
|||||||
|
package fileutil
|
||||||
|
|
||||||
|
import (
|
||||||
|
"errors"
|
||||||
|
"os"
|
||||||
|
"syscall"
|
||||||
|
)
|
||||||
|
|
||||||
|
var (
|
||||||
|
ErrLocked = errors.New("file already locked")
|
||||||
|
)
|
||||||
|
|
||||||
|
type Lock interface {
|
||||||
|
Name() string
|
||||||
|
TryLock() error
|
||||||
|
Lock() error
|
||||||
|
Unlock() error
|
||||||
|
Destroy() error
|
||||||
|
}
|
||||||
|
|
||||||
|
type lock struct {
|
||||||
|
fd int
|
||||||
|
file *os.File
|
||||||
|
}
|
||||||
|
|
||||||
|
func (l *lock) Name() string {
|
||||||
|
return l.file.Name()
|
||||||
|
}
|
||||||
|
|
||||||
|
// TryLock acquires exclusivity on the lock without blocking
|
||||||
|
func (l *lock) TryLock() error {
|
||||||
|
err := syscall.Flock(l.fd, syscall.LOCK_EX|syscall.LOCK_NB)
|
||||||
|
if err != nil && err == syscall.EWOULDBLOCK {
|
||||||
|
return ErrLocked
|
||||||
|
}
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
// Lock acquires exclusivity on the lock without blocking
|
||||||
|
func (l *lock) Lock() error {
|
||||||
|
return syscall.Flock(l.fd, syscall.LOCK_EX)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Unlock unlocks the lock
|
||||||
|
func (l *lock) Unlock() error {
|
||||||
|
return syscall.Flock(l.fd, syscall.LOCK_UN)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (l *lock) Destroy() error {
|
||||||
|
return l.file.Close()
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewLock(file string) (Lock, error) {
|
||||||
|
f, err := os.Open(file)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
l := &lock{int(f.Fd()), f}
|
||||||
|
return l, nil
|
||||||
|
}
|
82
pkg/fileutil/lock_test.go
Normal file
82
pkg/fileutil/lock_test.go
Normal file
@ -0,0 +1,82 @@
|
|||||||
|
package fileutil
|
||||||
|
|
||||||
|
import (
|
||||||
|
"io/ioutil"
|
||||||
|
"os"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestLockAndUnlock(t *testing.T) {
|
||||||
|
f, err := ioutil.TempFile("", "lock")
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
f.Close()
|
||||||
|
defer func() {
|
||||||
|
err := os.Remove(f.Name())
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
}()
|
||||||
|
|
||||||
|
// lock the file
|
||||||
|
l, err := NewLock(f.Name())
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
defer l.Destroy()
|
||||||
|
err = l.Lock()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// try lock a locked file
|
||||||
|
dupl, err := NewLock(f.Name())
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
err = dupl.TryLock()
|
||||||
|
if err != ErrLocked {
|
||||||
|
t.Errorf("err = %v, want %v", err, ErrLocked)
|
||||||
|
}
|
||||||
|
|
||||||
|
// unlock the file
|
||||||
|
err = l.Unlock()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// try lock the unlocked file
|
||||||
|
err = dupl.TryLock()
|
||||||
|
if err != nil {
|
||||||
|
t.Errorf("err = %v, want %v", err, nil)
|
||||||
|
}
|
||||||
|
defer dupl.Destroy()
|
||||||
|
|
||||||
|
// blocking on locked file
|
||||||
|
locked := make(chan struct{}, 1)
|
||||||
|
go func() {
|
||||||
|
l.Lock()
|
||||||
|
locked <- struct{}{}
|
||||||
|
}()
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-locked:
|
||||||
|
t.Error("unexpected unblocking")
|
||||||
|
case <-time.After(10 * time.Millisecond):
|
||||||
|
}
|
||||||
|
|
||||||
|
// unlock
|
||||||
|
err = dupl.Unlock()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
// the previously blocked routine should be unblocked
|
||||||
|
select {
|
||||||
|
case <-locked:
|
||||||
|
case <-time.After(10 * time.Millisecond):
|
||||||
|
t.Error("unexpected blocking")
|
||||||
|
}
|
||||||
|
}
|
@ -27,12 +27,29 @@ func PurgeFile(dirname string, suffix string, max uint, interval time.Duration,
|
|||||||
sort.Strings(newfnames)
|
sort.Strings(newfnames)
|
||||||
for len(newfnames) > int(max) {
|
for len(newfnames) > int(max) {
|
||||||
f := path.Join(dirname, newfnames[0])
|
f := path.Join(dirname, newfnames[0])
|
||||||
err := os.Remove(f)
|
l, err := NewLock(f)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
errC <- err
|
errC <- err
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
log.Printf("filePurge: successfully remvoed file %s", f)
|
err = l.TryLock()
|
||||||
|
if err != nil {
|
||||||
|
break
|
||||||
|
}
|
||||||
|
err = os.Remove(f)
|
||||||
|
if err != nil {
|
||||||
|
errC <- err
|
||||||
|
return
|
||||||
|
}
|
||||||
|
err = l.Unlock()
|
||||||
|
if err != nil {
|
||||||
|
log.Printf("filePurge: unlock %s error %v", l.Name(), err)
|
||||||
|
}
|
||||||
|
err = l.Destroy()
|
||||||
|
if err != nil {
|
||||||
|
log.Printf("filePurge: destroy lock %s error %v", l.Name(), err)
|
||||||
|
}
|
||||||
|
log.Printf("filePurge: successfully removed file %s", f)
|
||||||
newfnames = newfnames[1:]
|
newfnames = newfnames[1:]
|
||||||
}
|
}
|
||||||
select {
|
select {
|
||||||
|
@ -31,7 +31,7 @@ func TestPurgeFile(t *testing.T) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
time.Sleep(time.Millisecond)
|
time.Sleep(2 * time.Millisecond)
|
||||||
}
|
}
|
||||||
fnames, err := ReadDir(dir)
|
fnames, err := ReadDir(dir)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -48,3 +48,71 @@ func TestPurgeFile(t *testing.T) {
|
|||||||
}
|
}
|
||||||
close(stop)
|
close(stop)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestPurgeFileHoldingLock(t *testing.T) {
|
||||||
|
dir, err := ioutil.TempDir("", "purgefile")
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
defer os.RemoveAll(dir)
|
||||||
|
|
||||||
|
for i := 0; i < 10; i++ {
|
||||||
|
_, err := os.Create(path.Join(dir, fmt.Sprintf("%d.test", i)))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// create a purge barrier at 5
|
||||||
|
l, err := NewLock(path.Join(dir, fmt.Sprintf("%d.test", 5)))
|
||||||
|
err = l.Lock()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
stop := make(chan struct{})
|
||||||
|
errch := PurgeFile(dir, "test", 3, time.Millisecond, stop)
|
||||||
|
time.Sleep(5 * time.Millisecond)
|
||||||
|
|
||||||
|
fnames, err := ReadDir(dir)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
wnames := []string{"5.test", "6.test", "7.test", "8.test", "9.test"}
|
||||||
|
if !reflect.DeepEqual(fnames, wnames) {
|
||||||
|
t.Errorf("filenames = %v, want %v", fnames, wnames)
|
||||||
|
}
|
||||||
|
select {
|
||||||
|
case err := <-errch:
|
||||||
|
t.Errorf("unexpected purge error %v", err)
|
||||||
|
case <-time.After(time.Millisecond):
|
||||||
|
}
|
||||||
|
|
||||||
|
// remove the purge barrier
|
||||||
|
err = l.Unlock()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
err = l.Destroy()
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
|
||||||
|
time.Sleep(5 * time.Millisecond)
|
||||||
|
|
||||||
|
fnames, err = ReadDir(dir)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatal(err)
|
||||||
|
}
|
||||||
|
wnames = []string{"7.test", "8.test", "9.test"}
|
||||||
|
if !reflect.DeepEqual(fnames, wnames) {
|
||||||
|
t.Errorf("filenames = %v, want %v", fnames, wnames)
|
||||||
|
}
|
||||||
|
select {
|
||||||
|
case err := <-errch:
|
||||||
|
t.Errorf("unexpected purge error %v", err)
|
||||||
|
case <-time.After(time.Millisecond):
|
||||||
|
}
|
||||||
|
|
||||||
|
close(stop)
|
||||||
|
}
|
||||||
|
2
test
2
test
@ -15,7 +15,7 @@ COVER=${COVER:-"-cover"}
|
|||||||
source ./build
|
source ./build
|
||||||
|
|
||||||
# Hack: gofmt ./ will recursively check the .git directory. So use *.go for gofmt.
|
# Hack: gofmt ./ will recursively check the .git directory. So use *.go for gofmt.
|
||||||
TESTABLE_AND_FORMATTABLE="client discovery error etcdctl/command etcdmain etcdserver etcdserver/etcdhttp etcdserver/etcdhttp/httptypes etcdserver/etcdserverpb integration migrate pkg/flags pkg/ioutils pkg/types pkg/transport pkg/wait proxy raft rafthttp snap store wal"
|
TESTABLE_AND_FORMATTABLE="client discovery error etcdctl/command etcdmain etcdserver etcdserver/etcdhttp etcdserver/etcdhttp/httptypes etcdserver/etcdserverpb integration migrate pkg/fileutil pkg/flags pkg/ioutils pkg/types pkg/transport pkg/wait proxy raft rafthttp snap store wal"
|
||||||
FORMATTABLE="$TESTABLE_AND_FORMATTABLE *.go etcdctl/"
|
FORMATTABLE="$TESTABLE_AND_FORMATTABLE *.go etcdctl/"
|
||||||
|
|
||||||
# user has not provided PKG override
|
# user has not provided PKG override
|
||||||
|
@ -48,7 +48,7 @@ Cut issues 0x10 entries with incremental index later then the file will be calle
|
|||||||
|
|
||||||
At a later time a WAL can be opened at a particular raft index:
|
At a later time a WAL can be opened at a particular raft index:
|
||||||
|
|
||||||
w, err := wal.OpenAtIndex("/var/lib/etcd", 0)
|
w, err := wal.Open("/var/lib/etcd", 0)
|
||||||
...
|
...
|
||||||
|
|
||||||
The raft index must have been written to the WAL. When opening without a
|
The raft index must have been written to the WAL. When opening without a
|
||||||
|
85
wal/wal.go
85
wal/wal.go
@ -21,6 +21,7 @@ import (
|
|||||||
"fmt"
|
"fmt"
|
||||||
"hash/crc32"
|
"hash/crc32"
|
||||||
"io"
|
"io"
|
||||||
|
"log"
|
||||||
"os"
|
"os"
|
||||||
"path"
|
"path"
|
||||||
"reflect"
|
"reflect"
|
||||||
@ -67,6 +68,8 @@ type WAL struct {
|
|||||||
seq uint64 // sequence of the wal file currently used for writes
|
seq uint64 // sequence of the wal file currently used for writes
|
||||||
enti uint64 // index of the last entry saved to the wal
|
enti uint64 // index of the last entry saved to the wal
|
||||||
encoder *encoder // encoder to encode records
|
encoder *encoder // encoder to encode records
|
||||||
|
|
||||||
|
locks []fileutil.Lock // the file locks the WAL is holding (the name is increasing)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Create creates a WAL ready for appending records. The given metadata is
|
// Create creates a WAL ready for appending records. The given metadata is
|
||||||
@ -85,6 +88,15 @@ func Create(dirpath string, metadata []byte) (*WAL, error) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
l, err := fileutil.NewLock(f.Name())
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
err = l.Lock()
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
w := &WAL{
|
w := &WAL{
|
||||||
dir: dirpath,
|
dir: dirpath,
|
||||||
metadata: metadata,
|
metadata: metadata,
|
||||||
@ -92,6 +104,7 @@ func Create(dirpath string, metadata []byte) (*WAL, error) {
|
|||||||
f: f,
|
f: f,
|
||||||
encoder: newEncoder(f, 0),
|
encoder: newEncoder(f, 0),
|
||||||
}
|
}
|
||||||
|
w.locks = append(w.locks, l)
|
||||||
if err := w.saveCrc(0); err != nil {
|
if err := w.saveCrc(0); err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
@ -104,13 +117,23 @@ func Create(dirpath string, metadata []byte) (*WAL, error) {
|
|||||||
return w, nil
|
return w, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// OpenAtIndex opens the WAL at the given index.
|
// Open opens the WAL at the given index.
|
||||||
// The index SHOULD have been previously committed to the WAL, or the following
|
// The index SHOULD have been previously committed to the WAL, or the following
|
||||||
// ReadAll will fail.
|
// ReadAll will fail.
|
||||||
// The returned WAL is ready to read and the first record will be the given
|
// The returned WAL is ready to read and the first record will be the given
|
||||||
// index. The WAL cannot be appended to before reading out all of its
|
// index. The WAL cannot be appended to before reading out all of its
|
||||||
// previous records.
|
// previous records.
|
||||||
func OpenAtIndex(dirpath string, index uint64) (*WAL, error) {
|
func Open(dirpath string, index uint64) (*WAL, error) {
|
||||||
|
return openAtIndex(dirpath, index, true)
|
||||||
|
}
|
||||||
|
|
||||||
|
// OpenNotInUse only opens the wal files that are not in use.
|
||||||
|
// Other than that, it is similar to Open.
|
||||||
|
func OpenNotInUse(dirpath string, index uint64) (*WAL, error) {
|
||||||
|
return openAtIndex(dirpath, index, false)
|
||||||
|
}
|
||||||
|
|
||||||
|
func openAtIndex(dirpath string, index uint64, all bool) (*WAL, error) {
|
||||||
names, err := fileutil.ReadDir(dirpath)
|
names, err := fileutil.ReadDir(dirpath)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
@ -129,12 +152,27 @@ func OpenAtIndex(dirpath string, index uint64) (*WAL, error) {
|
|||||||
|
|
||||||
// open the wal files for reading
|
// open the wal files for reading
|
||||||
rcs := make([]io.ReadCloser, 0)
|
rcs := make([]io.ReadCloser, 0)
|
||||||
|
ls := make([]fileutil.Lock, 0)
|
||||||
for _, name := range names[nameIndex:] {
|
for _, name := range names[nameIndex:] {
|
||||||
f, err := os.Open(path.Join(dirpath, name))
|
f, err := os.Open(path.Join(dirpath, name))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
l, err := fileutil.NewLock(f.Name())
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
err = l.TryLock()
|
||||||
|
if err != nil {
|
||||||
|
if all {
|
||||||
|
return nil, err
|
||||||
|
} else {
|
||||||
|
log.Printf("wal: opened all the files until %s, since it is still in use by an etcd server", name)
|
||||||
|
break
|
||||||
|
}
|
||||||
|
}
|
||||||
rcs = append(rcs, f)
|
rcs = append(rcs, f)
|
||||||
|
ls = append(ls, l)
|
||||||
}
|
}
|
||||||
rc := MultiReadCloser(rcs...)
|
rc := MultiReadCloser(rcs...)
|
||||||
|
|
||||||
@ -157,8 +195,9 @@ func OpenAtIndex(dirpath string, index uint64) (*WAL, error) {
|
|||||||
ri: index,
|
ri: index,
|
||||||
decoder: newDecoder(rc),
|
decoder: newDecoder(rc),
|
||||||
|
|
||||||
f: f,
|
f: f,
|
||||||
seq: seq,
|
seq: seq,
|
||||||
|
locks: ls,
|
||||||
}
|
}
|
||||||
return w, nil
|
return w, nil
|
||||||
}
|
}
|
||||||
@ -224,6 +263,15 @@ func (w *WAL) Cut() error {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
l, err := fileutil.NewLock(f.Name())
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
err = l.Lock()
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
w.locks = append(w.locks, l)
|
||||||
if err = w.sync(); err != nil {
|
if err = w.sync(); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
@ -255,6 +303,30 @@ func (w *WAL) sync() error {
|
|||||||
return w.f.Sync()
|
return w.f.Sync()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// ReleaseLockTo releases the locks w is holding, which
|
||||||
|
// have index smaller or equal to the given index.
|
||||||
|
func (w *WAL) ReleaseLockTo(index uint64) error {
|
||||||
|
for _, l := range w.locks {
|
||||||
|
_, i, err := parseWalName(path.Base(l.Name()))
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
if i > index {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
err = l.Unlock()
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
err = l.Destroy()
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
w.locks = w.locks[1:]
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
func (w *WAL) Close() error {
|
func (w *WAL) Close() error {
|
||||||
if w.f != nil {
|
if w.f != nil {
|
||||||
if err := w.sync(); err != nil {
|
if err := w.sync(); err != nil {
|
||||||
@ -264,6 +336,11 @@ func (w *WAL) Close() error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
for _, l := range w.locks {
|
||||||
|
// TODO: log the error
|
||||||
|
l.Unlock()
|
||||||
|
l.Destroy()
|
||||||
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -90,7 +90,7 @@ func TestOpenAtIndex(t *testing.T) {
|
|||||||
}
|
}
|
||||||
f.Close()
|
f.Close()
|
||||||
|
|
||||||
w, err := OpenAtIndex(dir, 0)
|
w, err := Open(dir, 0)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("err = %v, want nil", err)
|
t.Fatalf("err = %v, want nil", err)
|
||||||
}
|
}
|
||||||
@ -109,7 +109,7 @@ func TestOpenAtIndex(t *testing.T) {
|
|||||||
}
|
}
|
||||||
f.Close()
|
f.Close()
|
||||||
|
|
||||||
w, err = OpenAtIndex(dir, 5)
|
w, err = Open(dir, 5)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("err = %v, want nil", err)
|
t.Fatalf("err = %v, want nil", err)
|
||||||
}
|
}
|
||||||
@ -126,7 +126,7 @@ func TestOpenAtIndex(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
defer os.RemoveAll(emptydir)
|
defer os.RemoveAll(emptydir)
|
||||||
if _, err = OpenAtIndex(emptydir, 0); err != ErrFileNotFound {
|
if _, err = Open(emptydir, 0); err != ErrFileNotFound {
|
||||||
t.Errorf("err = %v, want %v", err, ErrFileNotFound)
|
t.Errorf("err = %v, want %v", err, ErrFileNotFound)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -219,7 +219,7 @@ func TestRecover(t *testing.T) {
|
|||||||
}
|
}
|
||||||
w.Close()
|
w.Close()
|
||||||
|
|
||||||
if w, err = OpenAtIndex(p, 0); err != nil {
|
if w, err = Open(p, 0); err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
metadata, state, entries, err := w.ReadAll()
|
metadata, state, entries, err := w.ReadAll()
|
||||||
@ -238,6 +238,7 @@ func TestRecover(t *testing.T) {
|
|||||||
if !reflect.DeepEqual(state, s) {
|
if !reflect.DeepEqual(state, s) {
|
||||||
t.Errorf("state = %+v, want %+v", state, s)
|
t.Errorf("state = %+v, want %+v", state, s)
|
||||||
}
|
}
|
||||||
|
w.Close()
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestSearchIndex(t *testing.T) {
|
func TestSearchIndex(t *testing.T) {
|
||||||
@ -341,7 +342,7 @@ func TestRecoverAfterCut(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
for i := 0; i < 10; i++ {
|
for i := 0; i < 10; i++ {
|
||||||
w, err := OpenAtIndex(p, uint64(i))
|
w, err := Open(p, uint64(i))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if i <= 4 {
|
if i <= 4 {
|
||||||
if err != ErrFileNotFound {
|
if err != ErrFileNotFound {
|
||||||
@ -365,6 +366,7 @@ func TestRecoverAfterCut(t *testing.T) {
|
|||||||
t.Errorf("#%d: ents[%d].Index = %+v, want %+v", i, j, e.Index, j+i)
|
t.Errorf("#%d: ents[%d].Index = %+v, want %+v", i, j, e.Index, j+i)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
w.Close()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -384,7 +386,7 @@ func TestOpenAtUncommittedIndex(t *testing.T) {
|
|||||||
}
|
}
|
||||||
w.Close()
|
w.Close()
|
||||||
|
|
||||||
w, err = OpenAtIndex(p, 1)
|
w, err = Open(p, 1)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user