etcd/wal/wal.go

581 lines
14 KiB
Go
Raw Normal View History

// Copyright 2015 CoreOS, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
2014-07-28 07:58:39 +04:00
2014-07-26 01:20:21 +04:00
package wal
import (
2014-09-04 02:10:15 +04:00
"errors"
2014-07-26 01:20:21 +04:00
"fmt"
2014-09-04 02:10:15 +04:00
"hash/crc32"
2014-07-26 11:10:59 +04:00
"io"
2014-07-26 01:20:21 +04:00
"os"
2014-08-22 23:44:18 +04:00
"path"
"reflect"
"sync"
2015-01-30 00:07:23 +03:00
"time"
2014-07-26 01:20:21 +04:00
2014-11-29 08:32:28 +03:00
"github.com/coreos/etcd/pkg/fileutil"
2014-10-11 16:41:00 +04:00
"github.com/coreos/etcd/pkg/pbutil"
2014-09-09 08:45:10 +04:00
"github.com/coreos/etcd/raft"
2014-09-04 02:10:15 +04:00
"github.com/coreos/etcd/raft/raftpb"
2014-09-04 03:46:42 +04:00
"github.com/coreos/etcd/wal/walpb"
2015-06-01 23:38:50 +03:00
2016-03-23 03:10:28 +03:00
"github.com/coreos/pkg/capnslog"
2014-07-26 01:20:21 +04:00
)
2014-08-22 23:44:18 +04:00
const (
metadataType int64 = iota + 1
2014-08-22 23:44:18 +04:00
entryType
stateType
2014-09-04 02:10:15 +04:00
crcType
2015-01-06 10:27:03 +03:00
snapshotType
// the owner can make/remove files inside the directory
privateDirMode = 0700
// the expected size of each wal segment file.
// the actual size might be bigger than it.
segmentSizeBytes = 64 * 1000 * 1000 // 64MB
2014-08-22 23:44:18 +04:00
)
2014-07-26 01:20:21 +04:00
var (
2015-06-03 00:58:24 +03:00
plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "wal")
2015-06-01 23:38:50 +03:00
ErrMetadataConflict = errors.New("wal: conflicting metadata found")
ErrFileNotFound = errors.New("wal: file not found")
ErrCRCMismatch = errors.New("wal: crc mismatch")
2015-01-06 10:27:03 +03:00
ErrSnapshotMismatch = errors.New("wal: snapshot mismatch")
ErrSnapshotNotFound = errors.New("wal: snapshot not found")
crcTable = crc32.MakeTable(crc32.Castagnoli)
2014-07-26 01:20:21 +04:00
)
2016-01-31 05:15:56 +03:00
// WAL is a logical representation of the stable storage.
2014-09-04 02:10:15 +04:00
// WAL is either in read mode or append mode but not both.
// A newly created WAL is in append mode, and ready for appending records.
// A just opened WAL is in read mode, and ready for reading records.
// The WAL will be ready for appending after reading out all the previous records.
2014-07-26 01:20:21 +04:00
type WAL struct {
dir string // the living directory of the underlay files
metadata []byte // metadata recorded at the head of each WAL
state raftpb.HardState // hardstate recorded at the head of WAL
2014-07-26 01:20:21 +04:00
start walpb.Snapshot // snapshot to start reading
decoder *decoder // decoder to decode records
readClose func() error // closer for decode reader
2014-07-28 07:32:23 +04:00
mu sync.Mutex
2014-10-08 15:58:53 +04:00
enti uint64 // index of the last entry saved to the wal
2014-09-04 02:10:15 +04:00
encoder *encoder // encoder to encode records
locks []*fileutil.LockedFile // the locked files the WAL holds (the name is increasing)
fp *filePipeline
2014-08-22 23:44:18 +04:00
}
// Create creates a WAL ready for appending records. The given metadata is
// recorded at the head of each WAL file, and can be retrieved with ReadAll.
func Create(dirpath string, metadata []byte) (*WAL, error) {
2014-08-22 23:44:18 +04:00
if Exist(dirpath) {
2014-07-26 01:54:08 +04:00
return nil, os.ErrExist
}
2014-09-05 08:15:39 +04:00
if err := os.MkdirAll(dirpath, privateDirMode); err != nil {
2014-09-05 08:15:39 +04:00
return nil, err
}
p := path.Join(dirpath, walName(0, 0))
f, err := fileutil.LockFile(p, os.O_WRONLY|os.O_CREATE, 0600)
2014-07-26 01:20:21 +04:00
if err != nil {
return nil, err
}
if _, err := f.Seek(0, os.SEEK_END); err != nil {
return nil, err
}
if err := fileutil.Preallocate(f.File, segmentSizeBytes, true); err != nil {
return nil, err
}
2014-09-04 02:10:15 +04:00
w := &WAL{
2014-10-11 16:41:00 +04:00
dir: dirpath,
metadata: metadata,
encoder: newEncoder(f, 0),
fp: newFilePipeline(dirpath, segmentSizeBytes),
2014-09-04 02:10:15 +04:00
}
w.locks = append(w.locks, f)
2014-09-04 02:10:15 +04:00
if err := w.saveCrc(0); err != nil {
return nil, err
}
if err := w.encoder.encode(&walpb.Record{Type: metadataType, Data: metadata}); err != nil {
return nil, err
}
if err := w.SaveSnapshot(walpb.Snapshot{}); err != nil {
2014-11-08 01:41:34 +03:00
return nil, err
}
2014-09-04 02:10:15 +04:00
return w, nil
2014-07-26 01:20:21 +04:00
}
2015-01-06 10:27:03 +03:00
// Open opens the WAL at the given snap.
// The snap SHOULD have been previously saved to the WAL, or the following
// ReadAll will fail.
2015-01-06 10:27:03 +03:00
// The returned WAL is ready to read and the first record will be the one after
// the given snap. The WAL cannot be appended to before reading out all of its
// previous records.
2015-01-06 10:27:03 +03:00
func Open(dirpath string, snap walpb.Snapshot) (*WAL, error) {
return openAtIndex(dirpath, snap, true)
2014-12-13 01:38:18 +03:00
}
// OpenForRead only opens the wal files for read.
// Write on a read only wal panics.
func OpenForRead(dirpath string, snap walpb.Snapshot) (*WAL, error) {
2015-01-06 10:27:03 +03:00
return openAtIndex(dirpath, snap, false)
2014-12-13 01:38:18 +03:00
}
func openAtIndex(dirpath string, snap walpb.Snapshot, write bool) (*WAL, error) {
2014-11-29 08:32:28 +03:00
names, err := fileutil.ReadDir(dirpath)
2014-08-22 23:44:18 +04:00
if err != nil {
return nil, err
}
names = checkWalNames(names)
if len(names) == 0 {
return nil, ErrFileNotFound
2014-08-22 23:44:18 +04:00
}
2015-01-06 10:27:03 +03:00
nameIndex, ok := searchIndex(names, snap.Index)
2014-09-04 02:10:15 +04:00
if !ok || !isValidSeq(names[nameIndex:]) {
return nil, ErrFileNotFound
2014-07-26 01:54:08 +04:00
}
// open the wal files
2014-09-04 02:10:15 +04:00
rcs := make([]io.ReadCloser, 0)
rs := make([]io.Reader, 0)
ls := make([]*fileutil.LockedFile, 0)
2014-09-04 02:10:15 +04:00
for _, name := range names[nameIndex:] {
p := path.Join(dirpath, name)
if write {
l, err := fileutil.TryLockFile(p, os.O_RDWR, 0600)
if err != nil {
closeAll(rcs...)
2014-12-13 01:38:18 +03:00
return nil, err
}
ls = append(ls, l)
rcs = append(rcs, l)
} else {
rf, err := os.OpenFile(p, os.O_RDONLY, 0600)
if err != nil {
closeAll(rcs...)
return nil, err
}
ls = append(ls, nil)
rcs = append(rcs, rf)
}
rs = append(rs, rcs[len(rcs)-1])
2014-07-26 01:20:21 +04:00
}
closer := func() error { return closeAll(rcs...) }
2014-07-26 01:20:21 +04:00
2014-09-04 02:10:15 +04:00
// create a WAL ready for reading
w := &WAL{
dir: dirpath,
start: snap,
decoder: newDecoder(rs...),
readClose: closer,
locks: ls,
}
if write {
// write reuses the file descriptors from read; don't close so
// WAL can append without dropping the file lock
w.readClose = nil
if _, _, err := parseWalName(path.Base(w.tail().Name())); err != nil {
closer()
return nil, err
}
// don't resize file for preallocation in case tail is corrupted
if err := fileutil.Preallocate(w.tail().File, segmentSizeBytes, false); err != nil {
closer()
2015-08-06 09:46:15 +03:00
plog.Errorf("failed to allocate space when creating new wal file (%v)", err)
return nil, err
}
w.fp = newFilePipeline(w.dir, segmentSizeBytes)
2014-07-26 11:10:59 +04:00
}
2014-09-04 02:10:15 +04:00
return w, nil
2014-07-26 11:10:59 +04:00
}
// ReadAll reads out records of the current WAL.
// If opened in write mode, it must read out all records until EOF. Or an error
// will be returned.
// If opened in read mode, it will try to read all records if possible.
2015-01-06 10:27:03 +03:00
// If it cannot read out the expected snap, it will return ErrSnapshotNotFound.
// If loaded snap doesn't match with the expected one, it will return
// all the records and error ErrSnapshotMismatch.
2015-01-06 10:27:03 +03:00
// TODO: detect not-last-snap error.
// TODO: maybe loose the checking of match.
2014-09-04 02:10:15 +04:00
// After ReadAll, the WAL will be ready for appending new records.
func (w *WAL) ReadAll() (metadata []byte, state raftpb.HardState, ents []raftpb.Entry, err error) {
w.mu.Lock()
defer w.mu.Unlock()
2014-09-04 03:46:42 +04:00
rec := &walpb.Record{}
2014-09-04 02:10:15 +04:00
decoder := w.decoder
2014-09-09 02:36:25 +04:00
2015-01-06 10:27:03 +03:00
var match bool
2014-09-04 02:10:15 +04:00
for err = decoder.decode(rec); err == nil; err = decoder.decode(rec) {
2014-08-05 02:46:12 +04:00
switch rec.Type {
2014-07-26 11:10:59 +04:00
case entryType:
2014-09-04 02:10:15 +04:00
e := mustUnmarshalEntry(rec.Data)
2015-01-06 10:27:03 +03:00
if e.Index > w.start.Index {
ents = append(ents[:e.Index-w.start.Index-1], e)
2014-07-26 11:10:59 +04:00
}
2014-09-15 02:11:50 +04:00
w.enti = e.Index
2014-07-26 11:10:59 +04:00
case stateType:
2014-09-04 02:10:15 +04:00
state = mustUnmarshalState(rec.Data)
case metadataType:
if metadata != nil && !reflect.DeepEqual(metadata, rec.Data) {
2014-09-04 02:10:15 +04:00
state.Reset()
return nil, state, nil, ErrMetadataConflict
2014-07-26 11:10:59 +04:00
}
metadata = rec.Data
2014-09-04 02:10:15 +04:00
case crcType:
crc := decoder.crc.Sum32()
// current crc of decoder must match the crc of the record.
// do no need to match 0 crc, since the decoder is a new one at this case.
2014-09-04 03:46:42 +04:00
if crc != 0 && rec.Validate(crc) != nil {
2014-09-04 02:10:15 +04:00
state.Reset()
return nil, state, nil, ErrCRCMismatch
2014-09-04 02:10:15 +04:00
}
decoder.updateCRC(rec.Crc)
2015-01-06 10:27:03 +03:00
case snapshotType:
var snap walpb.Snapshot
pbutil.MustUnmarshal(&snap, rec.Data)
if snap.Index == w.start.Index {
if snap.Term != w.start.Term {
state.Reset()
return nil, state, nil, ErrSnapshotMismatch
}
match = true
}
2014-07-26 11:10:59 +04:00
default:
2014-09-04 02:10:15 +04:00
state.Reset()
return nil, state, nil, fmt.Errorf("unexpected block type %d", rec.Type)
2014-07-26 11:10:59 +04:00
}
}
switch w.tail() {
case nil:
// We do not have to read out all entries in read mode.
// The last record maybe a partial written one, so
// ErrunexpectedEOF might be returned.
if err != io.EOF && err != io.ErrUnexpectedEOF {
state.Reset()
return nil, state, nil, err
}
default:
// We must read all of the entries if WAL is opened in write mode.
if err != io.EOF {
state.Reset()
return nil, state, nil, err
}
2014-08-22 23:44:18 +04:00
}
err = nil
2015-01-06 10:27:03 +03:00
if !match {
err = ErrSnapshotNotFound
2015-01-06 10:27:03 +03:00
}
2014-08-22 23:44:18 +04:00
2014-09-04 02:10:15 +04:00
// close decoder, disable reading
if w.readClose != nil {
w.readClose()
w.readClose = nil
}
2015-01-06 10:27:03 +03:00
w.start = walpb.Snapshot{}
2014-09-04 02:10:15 +04:00
2014-10-11 16:41:00 +04:00
w.metadata = metadata
if w.tail() != nil {
// create encoder (chain crc with the decoder), enable appending
_, err = w.tail().Seek(w.decoder.lastOffset(), os.SEEK_SET)
w.encoder = newEncoder(w.tail(), w.decoder.lastCRC())
}
w.decoder = nil
return metadata, state, ents, err
2014-08-22 23:44:18 +04:00
}
// cut closes current file written and creates a new one ready to append.
// cut first creates a temp wal file and writes necessary headers into it.
2016-01-31 05:15:56 +03:00
// Then cut atomically rename temp wal file to a wal file.
func (w *WAL) cut() error {
// close old wal file; truncate to avoid wasting space if an early cut
off, serr := w.tail().Seek(0, os.SEEK_CUR)
if serr != nil {
return serr
}
if err := w.tail().Truncate(off); err != nil {
return err
}
if err := w.sync(); err != nil {
2014-09-04 02:10:15 +04:00
return err
2014-08-22 23:44:18 +04:00
}
fpath := path.Join(w.dir, walName(w.seq()+1, w.enti+1))
// create a temp wal file with name sequence + 1, or truncate the existing one
newTail, err := w.fp.Open()
if err != nil {
return err
}
2014-08-22 23:44:18 +04:00
2014-09-04 02:10:15 +04:00
// update writer and save the previous crc
w.locks = append(w.locks, newTail)
2014-09-04 02:10:15 +04:00
prevCrc := w.encoder.crc.Sum32()
w.encoder = newEncoder(w.tail(), prevCrc)
if err = w.saveCrc(prevCrc); err != nil {
return err
}
if err = w.encoder.encode(&walpb.Record{Type: metadataType, Data: w.metadata}); err != nil {
return err
}
if err = w.saveState(&w.state); err != nil {
return err
}
// atomically move temp wal file to wal file
if err = w.sync(); err != nil {
return err
}
off, err = w.tail().Seek(0, os.SEEK_CUR)
if err != nil {
return err
}
if err = os.Rename(newTail.Name(), fpath); err != nil {
return err
}
newTail.Close()
if newTail, err = fileutil.LockFile(fpath, os.O_WRONLY, 0600); err != nil {
2015-08-06 09:46:15 +03:00
return err
}
if _, err = newTail.Seek(off, os.SEEK_SET); err != nil {
return err
}
w.locks[len(w.locks)-1] = newTail
2015-08-06 09:46:15 +03:00
prevCrc = w.encoder.crc.Sum32()
w.encoder = newEncoder(w.tail(), prevCrc)
2015-06-03 00:58:24 +03:00
plog.Infof("segmented wal file %v is created", fpath)
return nil
2014-08-22 23:44:18 +04:00
}
2014-11-08 01:41:34 +03:00
func (w *WAL) sync() error {
2014-09-04 02:10:15 +04:00
if w.encoder != nil {
if err := w.encoder.flush(); err != nil {
return err
2014-08-22 23:44:18 +04:00
}
}
2015-01-30 00:07:23 +03:00
start := time.Now()
err := fileutil.Fdatasync(w.tail().File)
syncDurations.Observe(time.Since(start).Seconds())
2015-01-30 00:07:23 +03:00
return err
2014-08-22 23:44:18 +04:00
}
// ReleaseLockTo releases the locks, which has smaller index than the given index
// except the largest one among them.
// For example, if WAL is holding lock 1,2,3,4,5,6, ReleaseLockTo(4) will release
// lock 1,2 but keep 3. ReleaseLockTo(5) will release 1,2,3 but keep 4.
func (w *WAL) ReleaseLockTo(index uint64) error {
w.mu.Lock()
defer w.mu.Unlock()
var smaller int
found := false
for i, l := range w.locks {
_, lockIndex, err := parseWalName(path.Base(l.Name()))
if err != nil {
return err
}
if lockIndex >= index {
smaller = i - 1
found = true
break
}
}
// if no lock index is greater than the release index, we can
2016-01-31 05:15:56 +03:00
// release lock up to the last one(excluding).
if !found && len(w.locks) != 0 {
smaller = len(w.locks) - 1
}
if smaller <= 0 {
return nil
}
for i := 0; i < smaller; i++ {
if w.locks[i] == nil {
continue
}
w.locks[i].Close()
}
w.locks = w.locks[smaller:]
return nil
}
2014-11-08 01:41:34 +03:00
func (w *WAL) Close() error {
w.mu.Lock()
defer w.mu.Unlock()
if w.fp != nil {
w.fp.Close()
w.fp = nil
}
if w.tail() != nil {
2014-11-08 01:41:34 +03:00
if err := w.sync(); err != nil {
return err
}
2014-08-22 23:44:18 +04:00
}
for _, l := range w.locks {
if l == nil {
continue
}
if err := l.Close(); err != nil {
plog.Errorf("failed to unlock during closing wal: %s", err)
}
}
2014-11-08 01:41:34 +03:00
return nil
2014-07-26 11:10:59 +04:00
}
func (w *WAL) saveEntry(e *raftpb.Entry) error {
// TODO: add MustMarshalTo to reduce one allocation.
2014-10-11 16:41:00 +04:00
b := pbutil.MustMarshal(e)
2014-09-04 03:46:42 +04:00
rec := &walpb.Record{Type: entryType, Data: b}
2014-09-15 02:11:50 +04:00
if err := w.encoder.encode(rec); err != nil {
return err
}
w.enti = e.Index
return nil
2014-07-26 11:10:59 +04:00
}
func (w *WAL) saveState(s *raftpb.HardState) error {
if raft.IsEmptyHardState(*s) {
2014-09-09 08:45:10 +04:00
return nil
}
w.state = *s
2014-10-11 16:41:00 +04:00
b := pbutil.MustMarshal(s)
2014-09-04 03:46:42 +04:00
rec := &walpb.Record{Type: stateType, Data: b}
2014-09-04 02:10:15 +04:00
return w.encoder.encode(rec)
2014-08-22 23:44:18 +04:00
}
func (w *WAL) Save(st raftpb.HardState, ents []raftpb.Entry) error {
w.mu.Lock()
defer w.mu.Unlock()
2015-01-30 00:07:23 +03:00
// short cut, do not call sync
if raft.IsEmptyHardState(st) && len(ents) == 0 {
return nil
}
mustSync := mustSync(st, w.state, len(ents))
2014-09-09 00:57:35 +04:00
// TODO(xiangli): no more reference operator
2014-09-05 08:15:39 +04:00
for i := range ents {
if err := w.saveEntry(&ents[i]); err != nil {
return err
}
2014-09-05 08:15:39 +04:00
}
if err := w.saveState(&st); err != nil {
return err
}
curOff, err := w.tail().Seek(0, os.SEEK_CUR)
if err != nil {
return err
}
if curOff < segmentSizeBytes {
if mustSync {
return w.sync()
}
return nil
}
// TODO: add a test for this code path when refactoring the tests
return w.cut()
2014-09-05 08:15:39 +04:00
}
2015-01-06 10:27:03 +03:00
func (w *WAL) SaveSnapshot(e walpb.Snapshot) error {
w.mu.Lock()
defer w.mu.Unlock()
2015-01-06 10:27:03 +03:00
b := pbutil.MustMarshal(&e)
rec := &walpb.Record{Type: snapshotType, Data: b}
if err := w.encoder.encode(rec); err != nil {
return err
}
// update enti only when snapshot is ahead of last index
if w.enti < e.Index {
w.enti = e.Index
}
return w.sync()
}
2014-09-04 02:10:15 +04:00
func (w *WAL) saveCrc(prevCrc uint32) error {
2014-09-04 03:46:42 +04:00
return w.encoder.encode(&walpb.Record{Type: crcType, Crc: prevCrc})
2014-07-26 02:21:04 +04:00
}
func (w *WAL) tail() *fileutil.LockedFile {
if len(w.locks) > 0 {
return w.locks[len(w.locks)-1]
}
return nil
}
func (w *WAL) seq() uint64 {
t := w.tail()
if t == nil {
return 0
}
seq, _, err := parseWalName(path.Base(t.Name()))
if err != nil {
plog.Fatalf("bad wal name %s (%v)", t.Name(), err)
}
return seq
}
func mustSync(st, prevst raftpb.HardState, entsnum int) bool {
// Persistent state on all servers:
// (Updated on stable storage before responding to RPCs)
// currentTerm
// votedFor
// log entries[]
if entsnum != 0 || st.Vote != prevst.Vote || st.Term != prevst.Term {
return true
}
return false
}
func closeAll(rcs ...io.ReadCloser) error {
for _, f := range rcs {
if err := f.Close(); err != nil {
return err
}
}
return nil
}