From 8cae2578baacb6a97b09e88d15b4a85429b59a4e Mon Sep 17 00:00:00 2001 From: Mustafa Elbehery Date: Tue, 30 May 2023 17:10:15 +0200 Subject: [PATCH 1/5] add kvstore Signed-off-by: Mustafa Elbehery --- raftexample/kvstore.go | 98 +++++++++++++++++++++++++++++++++++++ raftexample/kvstore_test.go | 33 +++++++++++++ raftexample/raft.go | 6 +++ 3 files changed, 137 insertions(+) create mode 100644 raftexample/kvstore.go create mode 100644 raftexample/kvstore_test.go create mode 100644 raftexample/raft.go diff --git a/raftexample/kvstore.go b/raftexample/kvstore.go new file mode 100644 index 00000000..8c3d8b51 --- /dev/null +++ b/raftexample/kvstore.go @@ -0,0 +1,98 @@ +package raftexample + +import ( + "bytes" + "encoding/gob" + "encoding/json" + "log" + "strings" + "sync" +) + +// KVStore is the interface for any key-value store implementation. +type KVStore interface { + // Propose initiate a request to add a key `k` and associated value `v` to the key-value store. + Propose(k, v string) + // Lookup finds the value associated with key `k` in the key-value store. + Lookup(k string) (string, bool) +} + +type kv struct { + Key string + Val string +} + +type kvStore struct { + proposeC chan<- string + mu sync.RWMutex + store map[string]string + // TODO: snapshotter +} + +func newKVStore(proposeC chan<- string, commitC <-chan *commit, errorC <-chan error) KVStore { + s := &kvStore{ + proposeC: proposeC, + mu: sync.RWMutex{}, + store: make(map[string]string), + } + // TODO: load snapshot + + go s.readCommits(commitC, errorC) + return s +} + +func (s *kvStore) Propose(k, v string) { + var buf strings.Builder + if err := gob.NewEncoder(&buf).Encode(kv{k, v}); err != nil { + log.Fatal(err) + } + s.proposeC <- buf.String() +} + +func (s *kvStore) Lookup(k string) (string, bool) { + s.mu.RLock() + defer s.mu.RUnlock() + v, ok := s.store[k] + return v, ok +} + +func (s *kvStore) readCommits(commitC <-chan *commit, errorC <-chan error) { + for c := range commitC { + if c == nil { + continue + } + + for _, data := range c.data { + var dataKv kv + dec := gob.NewDecoder(bytes.NewBufferString(data)) + if err := dec.Decode(&dataKv); err != nil { + log.Fatalf("raftexample: could not decode message (%v)", err) + } + s.mu.Lock() + defer s.mu.Unlock() + s.store[dataKv.Key] = dataKv.Val + } + close(c.applyDoneC) + } + + if err, ok := <-errorC; ok { + log.Fatal(err) + } +} + +func (s *kvStore) getSnapshot() ([]byte, error) { + s.mu.RLock() + defer s.mu.RUnlock() + return json.Marshal(s.store) +} + +func (s *kvStore) recoverFromSnapshot(snapshot []byte) error { + var newStore map[string]string + if err := json.Unmarshal(snapshot, &newStore); err != nil { + return err + } + s.mu.Lock() + defer s.mu.Unlock() + s.store = newStore + return nil +} diff --git a/raftexample/kvstore_test.go b/raftexample/kvstore_test.go new file mode 100644 index 00000000..72b8f6e0 --- /dev/null +++ b/raftexample/kvstore_test.go @@ -0,0 +1,33 @@ +package raftexample + +import ( + "reflect" + "testing" +) + +func Test_kvstore_snapshot(t *testing.T) { + tm := map[string]string{"foo": "bar"} + s := &kvStore{store: tm} + + v, _ := s.Lookup("foo") + if v != "bar" { + t.Fatalf("foo has unexpected value, got %s", v) + } + + data, err := s.getSnapshot() + if err != nil { + t.Fatal(err) + } + s.store = nil + + if err := s.recoverFromSnapshot(data); err != nil { + t.Fatal(err) + } + v, _ = s.Lookup("foo") + if v != "bar" { + t.Fatalf("foo has unexpected value, got %s", v) + } + if !reflect.DeepEqual(s.store, tm) { + t.Fatalf("store expected %+v, got %+v", tm, s.store) + } +} diff --git a/raftexample/raft.go b/raftexample/raft.go new file mode 100644 index 00000000..cb14adf4 --- /dev/null +++ b/raftexample/raft.go @@ -0,0 +1,6 @@ +package raftexample + +type commit struct { + data []string + applyDoneC chan<- struct{} +} From d951570b16e255a044d15ad863ee4381fa132dc9 Mon Sep 17 00:00:00 2001 From: Mustafa Elbehery Date: Wed, 31 May 2023 18:25:00 +0200 Subject: [PATCH 2/5] add dependencies --- raftexample/fileutil/dir_unix.go | 27 ++ raftexample/fileutil/dir_windows.go | 51 +++ raftexample/fileutil/doc.go | 16 + raftexample/fileutil/filereader.go | 60 +++ raftexample/fileutil/filereader_test.go | 44 +++ raftexample/fileutil/fileutil.go | 184 ++++++++++ raftexample/fileutil/fileutil_test.go | 232 ++++++++++++ raftexample/fileutil/lock.go | 26 ++ raftexample/fileutil/lock_flock.go | 49 +++ raftexample/fileutil/lock_linux.go | 92 +++++ raftexample/fileutil/lock_linux_test.go | 29 ++ raftexample/fileutil/lock_plan9.go | 45 +++ raftexample/fileutil/lock_solaris.go | 62 ++++ raftexample/fileutil/lock_test.go | 88 +++++ raftexample/fileutil/lock_unix.go | 29 ++ raftexample/fileutil/lock_windows.go | 97 +++++ raftexample/fileutil/preallocate.go | 54 +++ raftexample/fileutil/preallocate_darwin.go | 66 ++++ raftexample/fileutil/preallocate_test.go | 72 ++++ raftexample/fileutil/preallocate_unix.go | 49 +++ .../fileutil/preallocate_unsupported.go | 25 ++ raftexample/fileutil/purge.go | 101 +++++ raftexample/fileutil/purge_test.go | 170 +++++++++ raftexample/fileutil/read_dir.go | 70 ++++ raftexample/fileutil/read_dir_test.go | 62 ++++ raftexample/fileutil/sync.go | 29 ++ raftexample/fileutil/sync_darwin.go | 38 ++ raftexample/fileutil/sync_linux.go | 34 ++ raftexample/ioutil/pagewriter.go | 115 ++++++ raftexample/ioutil/readcloser.go | 66 ++++ raftexample/ioutil/reader.go | 40 ++ raftexample/ioutil/util.go | 43 +++ raftexample/pbutil/pbutil.go | 56 +++ raftexample/snap/db.go | 99 +++++ raftexample/snap/doc.go | 17 + raftexample/snap/message.go | 64 ++++ raftexample/snap/metrics.go | 82 +++++ raftexample/snap/snappb/snap.pb.go | 344 ++++++++++++++++++ raftexample/snap/snappb/snap.proto | 14 + raftexample/snap/snapshotter.go | 282 ++++++++++++++ raftexample/verify/verify.go | 80 ++++ 41 files changed, 3203 insertions(+) create mode 100644 raftexample/fileutil/dir_unix.go create mode 100644 raftexample/fileutil/dir_windows.go create mode 100644 raftexample/fileutil/doc.go create mode 100644 raftexample/fileutil/filereader.go create mode 100644 raftexample/fileutil/filereader_test.go create mode 100644 raftexample/fileutil/fileutil.go create mode 100644 raftexample/fileutil/fileutil_test.go create mode 100644 raftexample/fileutil/lock.go create mode 100644 raftexample/fileutil/lock_flock.go create mode 100644 raftexample/fileutil/lock_linux.go create mode 100644 raftexample/fileutil/lock_linux_test.go create mode 100644 raftexample/fileutil/lock_plan9.go create mode 100644 raftexample/fileutil/lock_solaris.go create mode 100644 raftexample/fileutil/lock_test.go create mode 100644 raftexample/fileutil/lock_unix.go create mode 100644 raftexample/fileutil/lock_windows.go create mode 100644 raftexample/fileutil/preallocate.go create mode 100644 raftexample/fileutil/preallocate_darwin.go create mode 100644 raftexample/fileutil/preallocate_test.go create mode 100644 raftexample/fileutil/preallocate_unix.go create mode 100644 raftexample/fileutil/preallocate_unsupported.go create mode 100644 raftexample/fileutil/purge.go create mode 100644 raftexample/fileutil/purge_test.go create mode 100644 raftexample/fileutil/read_dir.go create mode 100644 raftexample/fileutil/read_dir_test.go create mode 100644 raftexample/fileutil/sync.go create mode 100644 raftexample/fileutil/sync_darwin.go create mode 100644 raftexample/fileutil/sync_linux.go create mode 100644 raftexample/ioutil/pagewriter.go create mode 100644 raftexample/ioutil/readcloser.go create mode 100644 raftexample/ioutil/reader.go create mode 100644 raftexample/ioutil/util.go create mode 100644 raftexample/pbutil/pbutil.go create mode 100644 raftexample/snap/db.go create mode 100644 raftexample/snap/doc.go create mode 100644 raftexample/snap/message.go create mode 100644 raftexample/snap/metrics.go create mode 100644 raftexample/snap/snappb/snap.pb.go create mode 100644 raftexample/snap/snappb/snap.proto create mode 100644 raftexample/snap/snapshotter.go create mode 100644 raftexample/verify/verify.go diff --git a/raftexample/fileutil/dir_unix.go b/raftexample/fileutil/dir_unix.go new file mode 100644 index 00000000..add54c63 --- /dev/null +++ b/raftexample/fileutil/dir_unix.go @@ -0,0 +1,27 @@ +// Copyright 2016 The etcd Authors +// +// 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. + +//go:build !windows + +package fileutil + +import "os" + +const ( + // PrivateDirMode grants owner to make/remove files inside the directory. + PrivateDirMode = 0700 +) + +// OpenDir opens a directory for syncing. +func OpenDir(path string) (*os.File, error) { return os.Open(path) } diff --git a/raftexample/fileutil/dir_windows.go b/raftexample/fileutil/dir_windows.go new file mode 100644 index 00000000..fd3415d5 --- /dev/null +++ b/raftexample/fileutil/dir_windows.go @@ -0,0 +1,51 @@ +// Copyright 2016 The etcd Authors +// +// 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. + +//go:build windows + +package fileutil + +import ( + "os" + "syscall" +) + +const ( + // PrivateDirMode grants owner to make/remove files inside the directory. + PrivateDirMode = 0777 +) + +// OpenDir opens a directory in windows with write access for syncing. +func OpenDir(path string) (*os.File, error) { + fd, err := openDir(path) + if err != nil { + return nil, err + } + return os.NewFile(uintptr(fd), path), nil +} + +func openDir(path string) (fd syscall.Handle, err error) { + if len(path) == 0 { + return syscall.InvalidHandle, syscall.ERROR_FILE_NOT_FOUND + } + pathp, err := syscall.UTF16PtrFromString(path) + if err != nil { + return syscall.InvalidHandle, err + } + access := uint32(syscall.GENERIC_READ | syscall.GENERIC_WRITE) + sharemode := uint32(syscall.FILE_SHARE_READ | syscall.FILE_SHARE_WRITE) + createmode := uint32(syscall.OPEN_EXISTING) + fl := uint32(syscall.FILE_FLAG_BACKUP_SEMANTICS) + return syscall.CreateFile(pathp, access, sharemode, nil, createmode, fl, 0) +} diff --git a/raftexample/fileutil/doc.go b/raftexample/fileutil/doc.go new file mode 100644 index 00000000..69dde5a7 --- /dev/null +++ b/raftexample/fileutil/doc.go @@ -0,0 +1,16 @@ +// Copyright 2018 The etcd Authors +// +// 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. + +// Package fileutil implements utility functions related to files and paths. +package fileutil diff --git a/raftexample/fileutil/filereader.go b/raftexample/fileutil/filereader.go new file mode 100644 index 00000000..55248888 --- /dev/null +++ b/raftexample/fileutil/filereader.go @@ -0,0 +1,60 @@ +// Copyright 2022 The etcd Authors +// +// 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. + +package fileutil + +import ( + "bufio" + "io" + "io/fs" + "os" +) + +// FileReader is a wrapper of io.Reader. It also provides file info. +type FileReader interface { + io.Reader + FileInfo() (fs.FileInfo, error) +} + +type fileReader struct { + *os.File +} + +func NewFileReader(f *os.File) FileReader { + return &fileReader{f} +} + +func (fr *fileReader) FileInfo() (fs.FileInfo, error) { + return fr.Stat() +} + +// FileBufReader is a wrapper of bufio.Reader. It also provides file info. +type FileBufReader struct { + *bufio.Reader + fi fs.FileInfo +} + +func NewFileBufReader(fr FileReader) *FileBufReader { + bufReader := bufio.NewReader(fr) + fi, err := fr.FileInfo() + if err != nil { + // This should never happen. + panic(err) + } + return &FileBufReader{bufReader, fi} +} + +func (fbr *FileBufReader) FileInfo() fs.FileInfo { + return fbr.fi +} diff --git a/raftexample/fileutil/filereader_test.go b/raftexample/fileutil/filereader_test.go new file mode 100644 index 00000000..2f863cdc --- /dev/null +++ b/raftexample/fileutil/filereader_test.go @@ -0,0 +1,44 @@ +// Copyright 2022 The etcd Authors +// +// 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. + +package fileutil + +import ( + "os" + "strings" + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestFileBufReader(t *testing.T) { + f, err := os.CreateTemp(t.TempDir(), "wal") + if err != nil { + t.Errorf("Unexpected error: %v", err) + } + fi, err := f.Stat() + if err != nil { + t.Errorf("Unexpected error: %v", err) + } + + fbr := NewFileBufReader(NewFileReader(f)) + + if !strings.HasPrefix(fbr.FileInfo().Name(), "wal") { + t.Errorf("Unexpected file name: %s", fbr.FileInfo().Name()) + } + assert.Equal(t, fi.Size(), fbr.FileInfo().Size()) + assert.Equal(t, fi.IsDir(), fbr.FileInfo().IsDir()) + assert.Equal(t, fi.Mode(), fbr.FileInfo().Mode()) + assert.Equal(t, fi.ModTime(), fbr.FileInfo().ModTime()) +} diff --git a/raftexample/fileutil/fileutil.go b/raftexample/fileutil/fileutil.go new file mode 100644 index 00000000..129fc744 --- /dev/null +++ b/raftexample/fileutil/fileutil.go @@ -0,0 +1,184 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package fileutil + +import ( + "fmt" + "io" + "io/fs" + "os" + "path/filepath" + + "go.uber.org/zap" + + "go.etcd.io/raft/v3/raftexample/verify" +) + +const ( + // PrivateFileMode grants owner to read/write a file. + PrivateFileMode = 0600 +) + +// IsDirWriteable checks if dir is writable by writing and removing a file +// to dir. It returns nil if dir is writable. +func IsDirWriteable(dir string) error { + f, err := filepath.Abs(filepath.Join(dir, ".touch")) + if err != nil { + return err + } + if err := os.WriteFile(f, []byte(""), PrivateFileMode); err != nil { + return err + } + return os.Remove(f) +} + +// TouchDirAll is similar to os.MkdirAll. It creates directories with 0700 permission if any directory +// does not exists. TouchDirAll also ensures the given directory is writable. +func TouchDirAll(lg *zap.Logger, dir string) error { + verify.Assert(lg != nil, "nil log isn't allowed") + // If path is already a directory, MkdirAll does nothing and returns nil, so, + // first check if dir exists with an expected permission mode. + if Exist(dir) { + err := CheckDirPermission(dir, PrivateDirMode) + if err != nil { + lg.Warn("check file permission", zap.Error(err)) + } + } else { + err := os.MkdirAll(dir, PrivateDirMode) + if err != nil { + // if mkdirAll("a/text") and "text" is not + // a directory, this will return syscall.ENOTDIR + return err + } + } + + return IsDirWriteable(dir) +} + +// CreateDirAll is similar to TouchDirAll but returns error +// if the deepest directory was not empty. +func CreateDirAll(lg *zap.Logger, dir string) error { + err := TouchDirAll(lg, dir) + if err == nil { + var ns []string + ns, err = ReadDir(dir) + if err != nil { + return err + } + if len(ns) != 0 { + err = fmt.Errorf("expected %q to be empty, got %q", dir, ns) + } + } + return err +} + +// Exist returns true if a file or directory exists. +func Exist(name string) bool { + _, err := os.Stat(name) + return err == nil +} + +// DirEmpty returns true if a directory empty and can access. +func DirEmpty(name string) bool { + ns, err := ReadDir(name) + return len(ns) == 0 && err == nil +} + +// ZeroToEnd zeros a file starting from SEEK_CUR to its SEEK_END. May temporarily +// shorten the length of the file. +func ZeroToEnd(f *os.File) error { + // TODO: support FALLOC_FL_ZERO_RANGE + off, err := f.Seek(0, io.SeekCurrent) + if err != nil { + return err + } + lenf, lerr := f.Seek(0, io.SeekEnd) + if lerr != nil { + return lerr + } + if err = f.Truncate(off); err != nil { + return err + } + // make sure blocks remain allocated + if err = Preallocate(f, lenf, true); err != nil { + return err + } + _, err = f.Seek(off, io.SeekStart) + return err +} + +// CheckDirPermission checks permission on an existing dir. +// Returns error if dir is empty or exist with a different permission than specified. +func CheckDirPermission(dir string, perm os.FileMode) error { + if !Exist(dir) { + return fmt.Errorf("directory %q empty, cannot check permission", dir) + } + //check the existing permission on the directory + dirInfo, err := os.Stat(dir) + if err != nil { + return err + } + dirMode := dirInfo.Mode().Perm() + if dirMode != perm { + err = fmt.Errorf("directory %q exist, but the permission is %q. The recommended permission is %q to prevent possible unprivileged access to the data", dir, dirInfo.Mode(), os.FileMode(PrivateDirMode)) + return err + } + return nil +} + +// RemoveMatchFile deletes file if matchFunc is true on an existing dir +// Returns error if the dir does not exist or remove file fail +func RemoveMatchFile(lg *zap.Logger, dir string, matchFunc func(fileName string) bool) error { + if lg == nil { + lg = zap.NewNop() + } + if !Exist(dir) { + return fmt.Errorf("directory %s does not exist", dir) + } + fileNames, err := ReadDir(dir) + if err != nil { + return err + } + var removeFailedFiles []string + for _, fileName := range fileNames { + if matchFunc(fileName) { + file := filepath.Join(dir, fileName) + if err = os.Remove(file); err != nil { + removeFailedFiles = append(removeFailedFiles, fileName) + lg.Error("remove file failed", + zap.String("file", file), + zap.Error(err)) + continue + } + } + } + if len(removeFailedFiles) != 0 { + return fmt.Errorf("remove file(s) %v error", removeFailedFiles) + } + return nil +} + +// ListFiles lists files if matchFunc is true on an existing dir +// Returns error if the dir does not exist +func ListFiles(dir string, matchFunc func(fileName string) bool) ([]string, error) { + var files []string + err := filepath.Walk(dir, func(path string, info fs.FileInfo, err error) error { + if matchFunc(path) { + files = append(files, path) + } + return nil + }) + return files, err +} diff --git a/raftexample/fileutil/fileutil_test.go b/raftexample/fileutil/fileutil_test.go new file mode 100644 index 00000000..f6b22e55 --- /dev/null +++ b/raftexample/fileutil/fileutil_test.go @@ -0,0 +1,232 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package fileutil + +import ( + "fmt" + "io" + "math/rand" + "os" + "os/user" + "path/filepath" + "runtime" + "strings" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "go.uber.org/zap/zaptest" +) + +func TestIsDirWriteable(t *testing.T) { + tmpdir := t.TempDir() + if err := IsDirWriteable(tmpdir); err != nil { + t.Fatalf("unexpected IsDirWriteable error: %v", err) + } + if err := os.Chmod(tmpdir, 0444); err != nil { + t.Fatalf("unexpected os.Chmod error: %v", err) + } + me, err := user.Current() + if err != nil { + // err can be non-nil when cross compiled + // http://stackoverflow.com/questions/20609415/cross-compiling-user-current-not-implemented-on-linux-amd64 + t.Skipf("failed to get current user: %v", err) + } + if me.Name == "root" || runtime.GOOS == "windows" { + // ideally we should check CAP_DAC_OVERRIDE. + // but it does not matter for tests. + // Chmod is not supported under windows. + t.Skipf("running as a superuser or in windows") + } + if err := IsDirWriteable(tmpdir); err == nil { + t.Fatalf("expected IsDirWriteable to error") + } +} + +func TestCreateDirAll(t *testing.T) { + tmpdir := t.TempDir() + + tmpdir2 := filepath.Join(tmpdir, "testdir") + if err := CreateDirAll(zaptest.NewLogger(t), tmpdir2); err != nil { + t.Fatal(err) + } + + if err := os.WriteFile(filepath.Join(tmpdir2, "text.txt"), []byte("test text"), PrivateFileMode); err != nil { + t.Fatal(err) + } + + if err := CreateDirAll(zaptest.NewLogger(t), tmpdir2); err == nil || !strings.Contains(err.Error(), "to be empty, got") { + t.Fatalf("unexpected error %v", err) + } +} + +func TestExist(t *testing.T) { + fdir := filepath.Join(os.TempDir(), fmt.Sprint(time.Now().UnixNano()+rand.Int63n(1000))) + os.RemoveAll(fdir) + if err := os.Mkdir(fdir, 0666); err != nil { + t.Skip(err) + } + defer os.RemoveAll(fdir) + if !Exist(fdir) { + t.Fatalf("expected Exist true, got %v", Exist(fdir)) + } + + f, err := os.CreateTemp(os.TempDir(), "fileutil") + if err != nil { + t.Fatal(err) + } + f.Close() + + if g := Exist(f.Name()); !g { + t.Errorf("exist = %v, want true", g) + } + + os.Remove(f.Name()) + if g := Exist(f.Name()); g { + t.Errorf("exist = %v, want false", g) + } +} + +func TestDirEmpty(t *testing.T) { + dir := t.TempDir() + + if !DirEmpty(dir) { + t.Fatalf("expected DirEmpty true, got %v", DirEmpty(dir)) + } + + file, err := os.CreateTemp(dir, "new_file") + if err != nil { + t.Fatal(err) + } + file.Close() + + if DirEmpty(dir) { + t.Fatalf("expected DirEmpty false, got %v", DirEmpty(dir)) + } + if DirEmpty(file.Name()) { + t.Fatalf("expected DirEmpty false, got %v", DirEmpty(file.Name())) + } +} + +func TestZeroToEnd(t *testing.T) { + f, err := os.CreateTemp(os.TempDir(), "fileutil") + if err != nil { + t.Fatal(err) + } + defer os.Remove(f.Name()) + defer f.Close() + + // Ensure 0 size is a nop so zero-to-end on an empty file won't give EINVAL. + if err = ZeroToEnd(f); err != nil { + t.Fatal(err) + } + + b := make([]byte, 1024) + for i := range b { + b[i] = 12 + } + if _, err = f.Write(b); err != nil { + t.Fatal(err) + } + if _, err = f.Seek(512, io.SeekStart); err != nil { + t.Fatal(err) + } + if err = ZeroToEnd(f); err != nil { + t.Fatal(err) + } + off, serr := f.Seek(0, io.SeekCurrent) + if serr != nil { + t.Fatal(serr) + } + if off != 512 { + t.Fatalf("expected offset 512, got %d", off) + } + + b = make([]byte, 512) + if _, err = f.Read(b); err != nil { + t.Fatal(err) + } + for i := range b { + if b[i] != 0 { + t.Errorf("expected b[%d] = 0, got %d", i, b[i]) + } + } +} + +func TestDirPermission(t *testing.T) { + tmpdir := t.TempDir() + + tmpdir2 := filepath.Join(tmpdir, "testpermission") + // create a new dir with 0700 + if err := CreateDirAll(zaptest.NewLogger(t), tmpdir2); err != nil { + t.Fatal(err) + } + // check dir permission with mode different than created dir + if err := CheckDirPermission(tmpdir2, 0600); err == nil { + t.Errorf("expected error, got nil") + } +} + +func TestRemoveMatchFile(t *testing.T) { + tmpdir := t.TempDir() + f, err := os.CreateTemp(tmpdir, "tmp") + if err != nil { + t.Fatal(err) + } + f.Close() + f, err = os.CreateTemp(tmpdir, "foo.tmp") + if err != nil { + t.Fatal(err) + } + f.Close() + + err = RemoveMatchFile(zaptest.NewLogger(t), tmpdir, func(fileName string) bool { + return strings.HasPrefix(fileName, "tmp") + }) + if err != nil { + t.Errorf("expected nil, got error") + } + fnames, err := ReadDir(tmpdir) + if err != nil { + t.Fatal(err) + } + if len(fnames) != 1 { + t.Errorf("expected exist 1 files, got %d", len(fnames)) + } + + f, err = os.CreateTemp(tmpdir, "tmp") + if err != nil { + t.Fatal(err) + } + f.Close() + err = RemoveMatchFile(zaptest.NewLogger(t), tmpdir, func(fileName string) bool { + os.Remove(filepath.Join(tmpdir, fileName)) + return strings.HasPrefix(fileName, "tmp") + }) + if err == nil { + t.Errorf("expected error, got nil") + } +} + +func TestTouchDirAll(t *testing.T) { + tmpdir := t.TempDir() + assert.Panics(t, func() { + TouchDirAll(nil, tmpdir) + }, "expected panic with nil log") + + if err := TouchDirAll(zaptest.NewLogger(t), tmpdir); err != nil { + t.Fatal(err) + } +} diff --git a/raftexample/fileutil/lock.go b/raftexample/fileutil/lock.go new file mode 100644 index 00000000..338627f4 --- /dev/null +++ b/raftexample/fileutil/lock.go @@ -0,0 +1,26 @@ +// Copyright 2016 The etcd Authors +// +// 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. + +package fileutil + +import ( + "errors" + "os" +) + +var ( + ErrLocked = errors.New("fileutil: file already locked") +) + +type LockedFile struct{ *os.File } diff --git a/raftexample/fileutil/lock_flock.go b/raftexample/fileutil/lock_flock.go new file mode 100644 index 00000000..a4e5707a --- /dev/null +++ b/raftexample/fileutil/lock_flock.go @@ -0,0 +1,49 @@ +// Copyright 2016 The etcd Authors +// +// 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. + +//go:build !windows && !plan9 && !solaris + +package fileutil + +import ( + "os" + "syscall" +) + +func flockTryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { + f, err := os.OpenFile(path, flag, perm) + if err != nil { + return nil, err + } + if err = syscall.Flock(int(f.Fd()), syscall.LOCK_EX|syscall.LOCK_NB); err != nil { + f.Close() + if err == syscall.EWOULDBLOCK { + err = ErrLocked + } + return nil, err + } + return &LockedFile{f}, nil +} + +func flockLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { + f, err := os.OpenFile(path, flag, perm) + if err != nil { + return nil, err + } + if err = syscall.Flock(int(f.Fd()), syscall.LOCK_EX); err != nil { + f.Close() + return nil, err + } + return &LockedFile{f}, err +} diff --git a/raftexample/fileutil/lock_linux.go b/raftexample/fileutil/lock_linux.go new file mode 100644 index 00000000..c33a2f4a --- /dev/null +++ b/raftexample/fileutil/lock_linux.go @@ -0,0 +1,92 @@ +// Copyright 2016 The etcd Authors +// +// 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. + +//go:build linux + +package fileutil + +import ( + "fmt" + "io" + "os" + "syscall" + + "golang.org/x/sys/unix" +) + +// This used to call syscall.Flock() but that call fails with EBADF on NFS. +// An alternative is lockf() which works on NFS but that call lets a process lock +// the same file twice. Instead, use Linux's non-standard open file descriptor +// locks which will block if the process already holds the file lock. + +var ( + wrlck = syscall.Flock_t{ + Type: syscall.F_WRLCK, + Whence: int16(io.SeekStart), + Start: 0, + Len: 0, + } + + linuxTryLockFile = flockTryLockFile + linuxLockFile = flockLockFile +) + +func init() { + // use open file descriptor locks if the system supports it + getlk := syscall.Flock_t{Type: syscall.F_RDLCK} + if err := syscall.FcntlFlock(0, unix.F_OFD_GETLK, &getlk); err == nil { + linuxTryLockFile = ofdTryLockFile + linuxLockFile = ofdLockFile + } +} + +func TryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { + return linuxTryLockFile(path, flag, perm) +} + +func ofdTryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { + f, err := os.OpenFile(path, flag, perm) + if err != nil { + return nil, fmt.Errorf("ofdTryLockFile failed to open %q (%v)", path, err) + } + + flock := wrlck + if err = syscall.FcntlFlock(f.Fd(), unix.F_OFD_SETLK, &flock); err != nil { + f.Close() + if err == syscall.EWOULDBLOCK { + err = ErrLocked + } + return nil, err + } + return &LockedFile{f}, nil +} + +func LockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { + return linuxLockFile(path, flag, perm) +} + +func ofdLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { + f, err := os.OpenFile(path, flag, perm) + if err != nil { + return nil, fmt.Errorf("ofdLockFile failed to open %q (%v)", path, err) + } + + flock := wrlck + err = syscall.FcntlFlock(f.Fd(), unix.F_OFD_SETLKW, &flock) + if err != nil { + f.Close() + return nil, err + } + return &LockedFile{f}, nil +} diff --git a/raftexample/fileutil/lock_linux_test.go b/raftexample/fileutil/lock_linux_test.go new file mode 100644 index 00000000..65dd96b9 --- /dev/null +++ b/raftexample/fileutil/lock_linux_test.go @@ -0,0 +1,29 @@ +// Copyright 2017 The etcd Authors +// +// 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. + +//go:build linux + +package fileutil + +import "testing" + +// TestLockAndUnlockSyscallFlock tests the fallback flock using the flock syscall. +func TestLockAndUnlockSyscallFlock(t *testing.T) { + oldTryLock, oldLock := linuxTryLockFile, linuxLockFile + defer func() { + linuxTryLockFile, linuxLockFile = oldTryLock, oldLock + }() + linuxTryLockFile, linuxLockFile = flockTryLockFile, flockLockFile + TestLockAndUnlock(t) +} diff --git a/raftexample/fileutil/lock_plan9.go b/raftexample/fileutil/lock_plan9.go new file mode 100644 index 00000000..fee6a7c8 --- /dev/null +++ b/raftexample/fileutil/lock_plan9.go @@ -0,0 +1,45 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package fileutil + +import ( + "os" + "syscall" + "time" +) + +func TryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { + if err := os.Chmod(path, syscall.DMEXCL|PrivateFileMode); err != nil { + return nil, err + } + f, err := os.Open(path, flag, perm) + if err != nil { + return nil, ErrLocked + } + return &LockedFile{f}, nil +} + +func LockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { + if err := os.Chmod(path, syscall.DMEXCL|PrivateFileMode); err != nil { + return nil, err + } + for { + f, err := os.OpenFile(path, flag, perm) + if err == nil { + return &LockedFile{f}, nil + } + time.Sleep(10 * time.Millisecond) + } +} diff --git a/raftexample/fileutil/lock_solaris.go b/raftexample/fileutil/lock_solaris.go new file mode 100644 index 00000000..2e892fec --- /dev/null +++ b/raftexample/fileutil/lock_solaris.go @@ -0,0 +1,62 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +//go:build solaris + +package fileutil + +import ( + "os" + "syscall" +) + +func TryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { + var lock syscall.Flock_t + lock.Start = 0 + lock.Len = 0 + lock.Pid = 0 + lock.Type = syscall.F_WRLCK + lock.Whence = 0 + lock.Pid = 0 + f, err := os.OpenFile(path, flag, perm) + if err != nil { + return nil, err + } + if err := syscall.FcntlFlock(f.Fd(), syscall.F_SETLK, &lock); err != nil { + f.Close() + if err == syscall.EAGAIN { + err = ErrLocked + } + return nil, err + } + return &LockedFile{f}, nil +} + +func LockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { + var lock syscall.Flock_t + lock.Start = 0 + lock.Len = 0 + lock.Pid = 0 + lock.Type = syscall.F_WRLCK + lock.Whence = 0 + f, err := os.OpenFile(path, flag, perm) + if err != nil { + return nil, err + } + if err = syscall.FcntlFlock(f.Fd(), syscall.F_SETLKW, &lock); err != nil { + f.Close() + return nil, err + } + return &LockedFile{f}, nil +} diff --git a/raftexample/fileutil/lock_test.go b/raftexample/fileutil/lock_test.go new file mode 100644 index 00000000..b7f6fd5c --- /dev/null +++ b/raftexample/fileutil/lock_test.go @@ -0,0 +1,88 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package fileutil + +import ( + "os" + "testing" + "time" +) + +func TestLockAndUnlock(t *testing.T) { + f, err := os.CreateTemp("", "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 := LockFile(f.Name(), os.O_WRONLY, PrivateFileMode) + if err != nil { + t.Fatal(err) + } + + // try lock a locked file + if _, err = TryLockFile(f.Name(), os.O_WRONLY, PrivateFileMode); err != ErrLocked { + t.Fatal(err) + } + + // unlock the file + if err = l.Close(); err != nil { + t.Fatal(err) + } + + // try lock the unlocked file + dupl, err := TryLockFile(f.Name(), os.O_WRONLY, PrivateFileMode) + if err != nil { + t.Errorf("err = %v, want %v", err, nil) + } + + // blocking on locked file + locked := make(chan struct{}, 1) + go func() { + bl, blerr := LockFile(f.Name(), os.O_WRONLY, PrivateFileMode) + if blerr != nil { + t.Error(blerr) + } + locked <- struct{}{} + if blerr = bl.Close(); blerr != nil { + t.Error(blerr) + } + }() + + select { + case <-locked: + t.Error("unexpected unblocking") + case <-time.After(100 * time.Millisecond): + } + + // unlock + if err = dupl.Close(); err != nil { + t.Fatal(err) + } + + // the previously blocked routine should be unblocked + select { + case <-locked: + case <-time.After(1 * time.Second): + t.Error("unexpected blocking") + } +} diff --git a/raftexample/fileutil/lock_unix.go b/raftexample/fileutil/lock_unix.go new file mode 100644 index 00000000..05db5367 --- /dev/null +++ b/raftexample/fileutil/lock_unix.go @@ -0,0 +1,29 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +//go:build !windows && !plan9 && !solaris && !linux + +package fileutil + +import ( + "os" +) + +func TryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { + return flockTryLockFile(path, flag, perm) +} + +func LockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { + return flockLockFile(path, flag, perm) +} diff --git a/raftexample/fileutil/lock_windows.go b/raftexample/fileutil/lock_windows.go new file mode 100644 index 00000000..51010bdf --- /dev/null +++ b/raftexample/fileutil/lock_windows.go @@ -0,0 +1,97 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +//go:build windows + +package fileutil + +import ( + "errors" + "fmt" + "os" + "syscall" + + "golang.org/x/sys/windows" +) + +var errLocked = errors.New("the process cannot access the file because another process has locked a portion of the file") + +func TryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { + f, err := open(path, flag, perm) + if err != nil { + return nil, err + } + if err := lockFile(windows.Handle(f.Fd()), windows.LOCKFILE_FAIL_IMMEDIATELY); err != nil { + f.Close() + return nil, err + } + return &LockedFile{f}, nil +} + +func LockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { + f, err := open(path, flag, perm) + if err != nil { + return nil, err + } + if err := lockFile(windows.Handle(f.Fd()), 0); err != nil { + f.Close() + return nil, err + } + return &LockedFile{f}, nil +} + +func open(path string, flag int, perm os.FileMode) (*os.File, error) { + if path == "" { + return nil, errors.New("cannot open empty filename") + } + var access uint32 + switch flag { + case syscall.O_RDONLY: + access = syscall.GENERIC_READ + case syscall.O_WRONLY: + access = syscall.GENERIC_WRITE + case syscall.O_RDWR: + access = syscall.GENERIC_READ | syscall.GENERIC_WRITE + case syscall.O_WRONLY | syscall.O_CREAT: + access = syscall.GENERIC_ALL + default: + panic(fmt.Errorf("flag %v is not supported", flag)) + } + fd, err := syscall.CreateFile(&(syscall.StringToUTF16(path)[0]), + access, + syscall.FILE_SHARE_READ|syscall.FILE_SHARE_WRITE|syscall.FILE_SHARE_DELETE, + nil, + syscall.OPEN_ALWAYS, + syscall.FILE_ATTRIBUTE_NORMAL, + 0) + if err != nil { + return nil, err + } + return os.NewFile(uintptr(fd), path), nil +} + +func lockFile(fd windows.Handle, flags uint32) error { + if fd == windows.InvalidHandle { + return nil + } + err := windows.LockFileEx(fd, flags|windows.LOCKFILE_EXCLUSIVE_LOCK, 0, 1, 0, &windows.Overlapped{}) + if err == nil { + return nil + } else if err.Error() == errLocked.Error() { + return ErrLocked + } else if err != windows.ERROR_LOCK_VIOLATION { + return err + } + return nil +} diff --git a/raftexample/fileutil/preallocate.go b/raftexample/fileutil/preallocate.go new file mode 100644 index 00000000..aadbff71 --- /dev/null +++ b/raftexample/fileutil/preallocate.go @@ -0,0 +1,54 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package fileutil + +import ( + "io" + "os" +) + +// Preallocate tries to allocate the space for given file. This +// operation is only supported on darwin and linux by a few +// filesystems (APFS, btrfs, ext4, etc.). +// If the operation is unsupported, no error will be returned. +// Otherwise, the error encountered will be returned. +func Preallocate(f *os.File, sizeInBytes int64, extendFile bool) error { + if sizeInBytes == 0 { + // fallocate will return EINVAL if length is 0; skip + return nil + } + if extendFile { + return preallocExtend(f, sizeInBytes) + } + return preallocFixed(f, sizeInBytes) +} + +func preallocExtendTrunc(f *os.File, sizeInBytes int64) error { + curOff, err := f.Seek(0, io.SeekCurrent) + if err != nil { + return err + } + size, err := f.Seek(sizeInBytes, io.SeekEnd) + if err != nil { + return err + } + if _, err = f.Seek(curOff, io.SeekStart); err != nil { + return err + } + if sizeInBytes > size { + return nil + } + return f.Truncate(sizeInBytes) +} diff --git a/raftexample/fileutil/preallocate_darwin.go b/raftexample/fileutil/preallocate_darwin.go new file mode 100644 index 00000000..e74968d0 --- /dev/null +++ b/raftexample/fileutil/preallocate_darwin.go @@ -0,0 +1,66 @@ +// Copyright 2016 The etcd Authors +// +// 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. + +//go:build darwin + +package fileutil + +import ( + "os" + "syscall" + + "golang.org/x/sys/unix" +) + +func preallocExtend(f *os.File, sizeInBytes int64) error { + if err := preallocFixed(f, sizeInBytes); err != nil { + return err + } + return preallocExtendTrunc(f, sizeInBytes) +} + +func preallocFixed(f *os.File, sizeInBytes int64) error { + // allocate all requested space or no space at all + // TODO: allocate contiguous space on disk with F_ALLOCATECONTIG flag + fstore := &unix.Fstore_t{ + Flags: unix.F_ALLOCATEALL, + Posmode: unix.F_PEOFPOSMODE, + Length: sizeInBytes, + } + err := unix.FcntlFstore(f.Fd(), unix.F_PREALLOCATE, fstore) + if err == nil || err == unix.ENOTSUP { + return nil + } + + // wrong argument to fallocate syscall + if err == unix.EINVAL { + // filesystem "st_blocks" are allocated in the units of + // "Allocation Block Size" (run "diskutil info /" command) + var stat syscall.Stat_t + syscall.Fstat(int(f.Fd()), &stat) + + // syscall.Statfs_t.Bsize is "optimal transfer block size" + // and contains matching 4096 value when latest OS X kernel + // supports 4,096 KB filesystem block size + var statfs syscall.Statfs_t + syscall.Fstatfs(int(f.Fd()), &statfs) + blockSize := int64(statfs.Bsize) + + if stat.Blocks*blockSize >= sizeInBytes { + // enough blocks are already allocated + return nil + } + } + return err +} diff --git a/raftexample/fileutil/preallocate_test.go b/raftexample/fileutil/preallocate_test.go new file mode 100644 index 00000000..47a00670 --- /dev/null +++ b/raftexample/fileutil/preallocate_test.go @@ -0,0 +1,72 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package fileutil + +import ( + "os" + "testing" +) + +func TestPreallocateExtend(t *testing.T) { + pf := func(f *os.File, sz int64) error { return Preallocate(f, sz, true) } + tf := func(t *testing.T, f *os.File) { testPreallocateExtend(t, f, pf) } + runPreallocTest(t, tf) +} + +func TestPreallocateExtendTrunc(t *testing.T) { + tf := func(t *testing.T, f *os.File) { testPreallocateExtend(t, f, preallocExtendTrunc) } + runPreallocTest(t, tf) +} + +func testPreallocateExtend(t *testing.T, f *os.File, pf func(*os.File, int64) error) { + size := int64(64 * 1000) + if err := pf(f, size); err != nil { + t.Fatal(err) + } + + stat, err := f.Stat() + if err != nil { + t.Fatal(err) + } + if stat.Size() != size { + t.Errorf("size = %d, want %d", stat.Size(), size) + } +} + +func TestPreallocateFixed(t *testing.T) { runPreallocTest(t, testPreallocateFixed) } +func testPreallocateFixed(t *testing.T, f *os.File) { + size := int64(64 * 1000) + if err := Preallocate(f, size, false); err != nil { + t.Fatal(err) + } + + stat, err := f.Stat() + if err != nil { + t.Fatal(err) + } + if stat.Size() != 0 { + t.Errorf("size = %d, want %d", stat.Size(), 0) + } +} + +func runPreallocTest(t *testing.T, test func(*testing.T, *os.File)) { + p := t.TempDir() + + f, err := os.CreateTemp(p, "") + if err != nil { + t.Fatal(err) + } + test(t, f) +} diff --git a/raftexample/fileutil/preallocate_unix.go b/raftexample/fileutil/preallocate_unix.go new file mode 100644 index 00000000..b02070b3 --- /dev/null +++ b/raftexample/fileutil/preallocate_unix.go @@ -0,0 +1,49 @@ +// Copyright 2016 The etcd Authors +// +// 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. + +//go:build linux + +package fileutil + +import ( + "os" + "syscall" +) + +func preallocExtend(f *os.File, sizeInBytes int64) error { + // use mode = 0 to change size + err := syscall.Fallocate(int(f.Fd()), 0, 0, sizeInBytes) + if err != nil { + errno, ok := err.(syscall.Errno) + // not supported; fallback + // fallocate EINTRs frequently in some environments; fallback + if ok && (errno == syscall.ENOTSUP || errno == syscall.EINTR) { + return preallocExtendTrunc(f, sizeInBytes) + } + } + return err +} + +func preallocFixed(f *os.File, sizeInBytes int64) error { + // use mode = 1 to keep size; see FALLOC_FL_KEEP_SIZE + err := syscall.Fallocate(int(f.Fd()), 1, 0, sizeInBytes) + if err != nil { + errno, ok := err.(syscall.Errno) + // treat not supported as nil error + if ok && errno == syscall.ENOTSUP { + return nil + } + } + return err +} diff --git a/raftexample/fileutil/preallocate_unsupported.go b/raftexample/fileutil/preallocate_unsupported.go new file mode 100644 index 00000000..e7fd937a --- /dev/null +++ b/raftexample/fileutil/preallocate_unsupported.go @@ -0,0 +1,25 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +//go:build !linux && !darwin + +package fileutil + +import "os" + +func preallocExtend(f *os.File, sizeInBytes int64) error { + return preallocExtendTrunc(f, sizeInBytes) +} + +func preallocFixed(f *os.File, sizeInBytes int64) error { return nil } diff --git a/raftexample/fileutil/purge.go b/raftexample/fileutil/purge.go new file mode 100644 index 00000000..f4492009 --- /dev/null +++ b/raftexample/fileutil/purge.go @@ -0,0 +1,101 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package fileutil + +import ( + "os" + "path/filepath" + "sort" + "strings" + "time" + + "go.uber.org/zap" +) + +func PurgeFile(lg *zap.Logger, dirname string, suffix string, max uint, interval time.Duration, stop <-chan struct{}) <-chan error { + return purgeFile(lg, dirname, suffix, max, interval, stop, nil, nil) +} + +func PurgeFileWithDoneNotify(lg *zap.Logger, dirname string, suffix string, max uint, interval time.Duration, stop <-chan struct{}) (<-chan struct{}, <-chan error) { + doneC := make(chan struct{}) + errC := purgeFile(lg, dirname, suffix, max, interval, stop, nil, doneC) + return doneC, errC +} + +// purgeFile is the internal implementation for PurgeFile which can post purged files to purgec if non-nil. +// if donec is non-nil, the function closes it to notify its exit. +func purgeFile(lg *zap.Logger, dirname string, suffix string, max uint, interval time.Duration, stop <-chan struct{}, purgec chan<- string, donec chan<- struct{}) <-chan error { + if lg == nil { + lg = zap.NewNop() + } + errC := make(chan error, 1) + lg.Info("started to purge file", + zap.String("dir", dirname), + zap.String("suffix", suffix), + zap.Uint("max", max), + zap.Duration("interval", interval)) + + go func() { + if donec != nil { + defer close(donec) + } + for { + fnames, err := ReadDir(dirname) + if err != nil { + errC <- err + return + } + newfnames := make([]string, 0) + for _, fname := range fnames { + if strings.HasSuffix(fname, suffix) { + newfnames = append(newfnames, fname) + } + } + sort.Strings(newfnames) + fnames = newfnames + for len(newfnames) > int(max) { + f := filepath.Join(dirname, newfnames[0]) + l, err := TryLockFile(f, os.O_WRONLY, PrivateFileMode) + if err != nil { + lg.Warn("failed to lock file", zap.String("path", f), zap.Error(err)) + break + } + if err = os.Remove(f); err != nil { + lg.Error("failed to remove file", zap.String("path", f), zap.Error(err)) + errC <- err + return + } + if err = l.Close(); err != nil { + lg.Error("failed to unlock/close", zap.String("path", l.Name()), zap.Error(err)) + errC <- err + return + } + lg.Info("purged", zap.String("path", f)) + newfnames = newfnames[1:] + } + if purgec != nil { + for i := 0; i < len(fnames)-len(newfnames); i++ { + purgec <- fnames[i] + } + } + select { + case <-time.After(interval): + case <-stop: + return + } + } + }() + return errC +} diff --git a/raftexample/fileutil/purge_test.go b/raftexample/fileutil/purge_test.go new file mode 100644 index 00000000..a10a3283 --- /dev/null +++ b/raftexample/fileutil/purge_test.go @@ -0,0 +1,170 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package fileutil + +import ( + "fmt" + "os" + "path/filepath" + "reflect" + "testing" + "time" + + "go.uber.org/zap/zaptest" +) + +func TestPurgeFile(t *testing.T) { + dir := t.TempDir() + + // minimal file set + for i := 0; i < 3; i++ { + f, ferr := os.Create(filepath.Join(dir, fmt.Sprintf("%d.test", i))) + if ferr != nil { + t.Fatal(ferr) + } + f.Close() + } + + stop, purgec := make(chan struct{}), make(chan string, 10) + + // keep 3 most recent files + errch := purgeFile(zaptest.NewLogger(t), dir, "test", 3, time.Millisecond, stop, purgec, nil) + select { + case f := <-purgec: + t.Errorf("unexpected purge on %q", f) + case <-time.After(10 * time.Millisecond): + } + + // rest of the files + for i := 4; i < 10; i++ { + go func(n int) { + f, ferr := os.Create(filepath.Join(dir, fmt.Sprintf("%d.test", n))) + if ferr != nil { + t.Error(ferr) + } + f.Close() + }(i) + } + + // watch files purge away + for i := 4; i < 10; i++ { + select { + case <-purgec: + case <-time.After(time.Second): + t.Errorf("purge took too long") + } + } + + fnames, rerr := ReadDir(dir) + if rerr != nil { + t.Fatal(rerr) + } + wnames := []string{"7.test", "8.test", "9.test"} + if !reflect.DeepEqual(fnames, wnames) { + t.Errorf("filenames = %v, want %v", fnames, wnames) + } + + // no error should be reported from purge routine + select { + case f := <-purgec: + t.Errorf("unexpected purge on %q", f) + case err := <-errch: + t.Errorf("unexpected purge error %v", err) + case <-time.After(10 * time.Millisecond): + } + close(stop) +} + +func TestPurgeFileHoldingLockFile(t *testing.T) { + dir := t.TempDir() + + for i := 0; i < 10; i++ { + var f *os.File + f, err := os.Create(filepath.Join(dir, fmt.Sprintf("%d.test", i))) + if err != nil { + t.Fatal(err) + } + f.Close() + } + + // create a purge barrier at 5 + p := filepath.Join(dir, fmt.Sprintf("%d.test", 5)) + l, err := LockFile(p, os.O_WRONLY, PrivateFileMode) + if err != nil { + t.Fatal(err) + } + + stop, purgec := make(chan struct{}), make(chan string, 10) + errch := purgeFile(zaptest.NewLogger(t), dir, "test", 3, time.Millisecond, stop, purgec, nil) + + for i := 0; i < 5; i++ { + select { + case <-purgec: + case <-time.After(time.Second): + t.Fatalf("purge took too long") + } + } + + fnames, rerr := ReadDir(dir) + if rerr != nil { + t.Fatal(rerr) + } + + 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 s := <-purgec: + t.Errorf("unexpected purge %q", s) + case err = <-errch: + t.Errorf("unexpected purge error %v", err) + case <-time.After(10 * time.Millisecond): + } + + // remove the purge barrier + if err = l.Close(); err != nil { + t.Fatal(err) + } + + // wait for rest of purges (5, 6) + for i := 0; i < 2; i++ { + select { + case <-purgec: + case <-time.After(time.Second): + t.Fatalf("purge took too long") + } + } + + fnames, rerr = ReadDir(dir) + if rerr != nil { + t.Fatal(rerr) + } + wnames = []string{"7.test", "8.test", "9.test"} + if !reflect.DeepEqual(fnames, wnames) { + t.Errorf("filenames = %v, want %v", fnames, wnames) + } + + select { + case f := <-purgec: + t.Errorf("unexpected purge on %q", f) + case err := <-errch: + t.Errorf("unexpected purge error %v", err) + case <-time.After(10 * time.Millisecond): + } + + close(stop) +} diff --git a/raftexample/fileutil/read_dir.go b/raftexample/fileutil/read_dir.go new file mode 100644 index 00000000..2eeaa89b --- /dev/null +++ b/raftexample/fileutil/read_dir.go @@ -0,0 +1,70 @@ +// Copyright 2018 The etcd Authors +// +// 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. + +package fileutil + +import ( + "os" + "path/filepath" + "sort" +) + +// ReadDirOp represents an read-directory operation. +type ReadDirOp struct { + ext string +} + +// ReadDirOption configures archiver operations. +type ReadDirOption func(*ReadDirOp) + +// WithExt filters file names by their extensions. +// (e.g. WithExt(".wal") to list only WAL files) +func WithExt(ext string) ReadDirOption { + return func(op *ReadDirOp) { op.ext = ext } +} + +func (op *ReadDirOp) applyOpts(opts []ReadDirOption) { + for _, opt := range opts { + opt(op) + } +} + +// ReadDir returns the filenames in the given directory in sorted order. +func ReadDir(d string, opts ...ReadDirOption) ([]string, error) { + op := &ReadDirOp{} + op.applyOpts(opts) + + dir, err := os.Open(d) + if err != nil { + return nil, err + } + defer dir.Close() + + names, err := dir.Readdirnames(-1) + if err != nil { + return nil, err + } + sort.Strings(names) + + if op.ext != "" { + tss := make([]string, 0) + for _, v := range names { + if filepath.Ext(v) == op.ext { + tss = append(tss, v) + } + } + names = tss + } + return names, nil +} diff --git a/raftexample/fileutil/read_dir_test.go b/raftexample/fileutil/read_dir_test.go new file mode 100644 index 00000000..79a37d88 --- /dev/null +++ b/raftexample/fileutil/read_dir_test.go @@ -0,0 +1,62 @@ +// Copyright 2018 The etcd Authors +// +// 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. + +package fileutil + +import ( + "os" + "path/filepath" + "reflect" + "testing" +) + +func TestReadDir(t *testing.T) { + tmpdir := t.TempDir() + + files := []string{"def", "abc", "xyz", "ghi"} + for _, f := range files { + writeFunc(t, filepath.Join(tmpdir, f)) + } + fs, err := ReadDir(tmpdir) + if err != nil { + t.Fatalf("error calling ReadDir: %v", err) + } + wfs := []string{"abc", "def", "ghi", "xyz"} + if !reflect.DeepEqual(fs, wfs) { + t.Fatalf("ReadDir: got %v, want %v", fs, wfs) + } + + files = []string{"def.wal", "abc.wal", "xyz.wal", "ghi.wal"} + for _, f := range files { + writeFunc(t, filepath.Join(tmpdir, f)) + } + fs, err = ReadDir(tmpdir, WithExt(".wal")) + if err != nil { + t.Fatalf("error calling ReadDir: %v", err) + } + wfs = []string{"abc.wal", "def.wal", "ghi.wal", "xyz.wal"} + if !reflect.DeepEqual(fs, wfs) { + t.Fatalf("ReadDir: got %v, want %v", fs, wfs) + } +} + +func writeFunc(t *testing.T, path string) { + fh, err := os.Create(path) + if err != nil { + t.Fatalf("error creating file: %v", err) + } + if err = fh.Close(); err != nil { + t.Fatalf("error closing file: %v", err) + } +} diff --git a/raftexample/fileutil/sync.go b/raftexample/fileutil/sync.go new file mode 100644 index 00000000..670d01fa --- /dev/null +++ b/raftexample/fileutil/sync.go @@ -0,0 +1,29 @@ +// Copyright 2016 The etcd Authors +// +// 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. + +//go:build !linux && !darwin + +package fileutil + +import "os" + +// Fsync is a wrapper around file.Sync(). Special handling is needed on darwin platform. +func Fsync(f *os.File) error { + return f.Sync() +} + +// Fdatasync is a wrapper around file.Sync(). Special handling is needed on linux platform. +func Fdatasync(f *os.File) error { + return f.Sync() +} diff --git a/raftexample/fileutil/sync_darwin.go b/raftexample/fileutil/sync_darwin.go new file mode 100644 index 00000000..7affa78e --- /dev/null +++ b/raftexample/fileutil/sync_darwin.go @@ -0,0 +1,38 @@ +// Copyright 2016 The etcd Authors +// +// 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. + +//go:build darwin + +package fileutil + +import ( + "os" + + "golang.org/x/sys/unix" +) + +// Fsync on HFS/OSX flushes the data on to the physical drive but the drive +// may not write it to the persistent media for quite sometime and it may be +// written in out-of-order sequence. Using F_FULLFSYNC ensures that the +// physical drive's buffer will also get flushed to the media. +func Fsync(f *os.File) error { + _, err := unix.FcntlInt(f.Fd(), unix.F_FULLFSYNC, 0) + return err +} + +// Fdatasync on darwin platform invokes fcntl(F_FULLFSYNC) for actual persistence +// on physical drive media. +func Fdatasync(f *os.File) error { + return Fsync(f) +} diff --git a/raftexample/fileutil/sync_linux.go b/raftexample/fileutil/sync_linux.go new file mode 100644 index 00000000..a3172382 --- /dev/null +++ b/raftexample/fileutil/sync_linux.go @@ -0,0 +1,34 @@ +// Copyright 2016 The etcd Authors +// +// 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. + +//go:build linux + +package fileutil + +import ( + "os" + "syscall" +) + +// Fsync is a wrapper around file.Sync(). Special handling is needed on darwin platform. +func Fsync(f *os.File) error { + return f.Sync() +} + +// Fdatasync is similar to fsync(), but does not flush modified metadata +// unless that metadata is needed in order to allow a subsequent data retrieval +// to be correctly handled. +func Fdatasync(f *os.File) error { + return syscall.Fdatasync(int(f.Fd())) +} diff --git a/raftexample/ioutil/pagewriter.go b/raftexample/ioutil/pagewriter.go new file mode 100644 index 00000000..b8d1a257 --- /dev/null +++ b/raftexample/ioutil/pagewriter.go @@ -0,0 +1,115 @@ +// Copyright 2016 The etcd Authors +// +// 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. + +package ioutil + +import ( + "io" + + "go.etcd.io/raft/v3/raftexample/verify" +) + +var defaultBufferBytes = 128 * 1024 + +// PageWriter implements the io.Writer interface so that writes will +// either be in page chunks or from flushing. +type PageWriter struct { + w io.Writer + // pageOffset tracks the page offset of the base of the buffer + pageOffset int + // pageBytes is the number of bytes per page + pageBytes int + // bufferedBytes counts the number of bytes pending for write in the buffer + bufferedBytes int + // buf holds the write buffer + buf []byte + // bufWatermarkBytes is the number of bytes the buffer can hold before it needs + // to be flushed. It is less than len(buf) so there is space for slack writes + // to bring the writer to page alignment. + bufWatermarkBytes int +} + +// NewPageWriter creates a new PageWriter. pageBytes is the number of bytes +// to write per page. pageOffset is the starting offset of io.Writer. +func NewPageWriter(w io.Writer, pageBytes, pageOffset int) *PageWriter { + verify.Assert(pageBytes > 0, "invalid pageBytes (%d) value, it must be greater than 0", pageBytes) + return &PageWriter{ + w: w, + pageOffset: pageOffset, + pageBytes: pageBytes, + buf: make([]byte, defaultBufferBytes+pageBytes), + bufWatermarkBytes: defaultBufferBytes, + } +} + +func (pw *PageWriter) Write(p []byte) (n int, err error) { + if len(p)+pw.bufferedBytes <= pw.bufWatermarkBytes { + // no overflow + copy(pw.buf[pw.bufferedBytes:], p) + pw.bufferedBytes += len(p) + return len(p), nil + } + // complete the slack page in the buffer if unaligned + slack := pw.pageBytes - ((pw.pageOffset + pw.bufferedBytes) % pw.pageBytes) + if slack != pw.pageBytes { + partial := slack > len(p) + if partial { + // not enough data to complete the slack page + slack = len(p) + } + // special case: writing to slack page in buffer + copy(pw.buf[pw.bufferedBytes:], p[:slack]) + pw.bufferedBytes += slack + n = slack + p = p[slack:] + if partial { + // avoid forcing an unaligned flush + return n, nil + } + } + // buffer contents are now page-aligned; clear out + if err = pw.Flush(); err != nil { + return n, err + } + // directly write all complete pages without copying + if len(p) > pw.pageBytes { + pages := len(p) / pw.pageBytes + c, werr := pw.w.Write(p[:pages*pw.pageBytes]) + n += c + if werr != nil { + return n, werr + } + p = p[pages*pw.pageBytes:] + } + // write remaining tail to buffer + c, werr := pw.Write(p) + n += c + return n, werr +} + +// Flush flushes buffered data. +func (pw *PageWriter) Flush() error { + _, err := pw.flush() + return err +} + +func (pw *PageWriter) flush() (int, error) { + if pw.bufferedBytes == 0 { + return 0, nil + } + n, err := pw.w.Write(pw.buf[:pw.bufferedBytes]) + pw.pageOffset = (pw.pageOffset + pw.bufferedBytes) % pw.pageBytes + pw.bufferedBytes = 0 + return n, err +} diff --git a/raftexample/ioutil/readcloser.go b/raftexample/ioutil/readcloser.go new file mode 100644 index 00000000..d3efcfe3 --- /dev/null +++ b/raftexample/ioutil/readcloser.go @@ -0,0 +1,66 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package ioutil + +import ( + "fmt" + "io" +) + +// ReaderAndCloser implements io.ReadCloser interface by combining +// reader and closer together. +type ReaderAndCloser struct { + io.Reader + io.Closer +} + +var ( + ErrShortRead = fmt.Errorf("ioutil: short read") + ErrExpectEOF = fmt.Errorf("ioutil: expect EOF") +) + +// NewExactReadCloser returns a ReadCloser that returns errors if the underlying +// reader does not read back exactly the requested number of bytes. +func NewExactReadCloser(rc io.ReadCloser, totalBytes int64) io.ReadCloser { + return &exactReadCloser{rc: rc, totalBytes: totalBytes} +} + +type exactReadCloser struct { + rc io.ReadCloser + br int64 + totalBytes int64 +} + +func (e *exactReadCloser) Read(p []byte) (int, error) { + n, err := e.rc.Read(p) + e.br += int64(n) + if e.br > e.totalBytes { + return 0, ErrExpectEOF + } + if e.br < e.totalBytes && n == 0 { + return 0, ErrShortRead + } + return n, err +} + +func (e *exactReadCloser) Close() error { + if err := e.rc.Close(); err != nil { + return err + } + if e.br < e.totalBytes { + return ErrShortRead + } + return nil +} diff --git a/raftexample/ioutil/reader.go b/raftexample/ioutil/reader.go new file mode 100644 index 00000000..0703ed47 --- /dev/null +++ b/raftexample/ioutil/reader.go @@ -0,0 +1,40 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +// Package ioutil implements I/O utility functions. +package ioutil + +import "io" + +// NewLimitedBufferReader returns a reader that reads from the given reader +// but limits the amount of data returned to at most n bytes. +func NewLimitedBufferReader(r io.Reader, n int) io.Reader { + return &limitedBufferReader{ + r: r, + n: n, + } +} + +type limitedBufferReader struct { + r io.Reader + n int +} + +func (r *limitedBufferReader) Read(p []byte) (n int, err error) { + np := p + if len(np) > r.n { + np = np[:r.n] + } + return r.r.Read(np) +} diff --git a/raftexample/ioutil/util.go b/raftexample/ioutil/util.go new file mode 100644 index 00000000..8cd03ca1 --- /dev/null +++ b/raftexample/ioutil/util.go @@ -0,0 +1,43 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package ioutil + +import ( + "io" + "os" + + "go.etcd.io/raft/v3/raftexample/fileutil" +) + +// WriteAndSyncFile behaves just like ioutil.WriteFile in the standard library, +// but calls Sync before closing the file. WriteAndSyncFile guarantees the data +// is synced if there is no error returned. +func WriteAndSyncFile(filename string, data []byte, perm os.FileMode) error { + f, err := os.OpenFile(filename, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, perm) + if err != nil { + return err + } + n, err := f.Write(data) + if err == nil && n < len(data) { + err = io.ErrShortWrite + } + if err == nil { + err = fileutil.Fsync(f) + } + if err1 := f.Close(); err == nil { + err = err1 + } + return err +} diff --git a/raftexample/pbutil/pbutil.go b/raftexample/pbutil/pbutil.go new file mode 100644 index 00000000..821f5970 --- /dev/null +++ b/raftexample/pbutil/pbutil.go @@ -0,0 +1,56 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +// Package pbutil defines interfaces for handling Protocol Buffer objects. +package pbutil + +import "fmt" + +type Marshaler interface { + Marshal() (data []byte, err error) +} + +type Unmarshaler interface { + Unmarshal(data []byte) error +} + +func MustMarshal(m Marshaler) []byte { + d, err := m.Marshal() + if err != nil { + panic(fmt.Sprintf("marshal should never fail (%v)", err)) + } + return d +} + +func MustUnmarshal(um Unmarshaler, data []byte) { + if err := um.Unmarshal(data); err != nil { + panic(fmt.Sprintf("unmarshal should never fail (%v)", err)) + } +} + +func MaybeUnmarshal(um Unmarshaler, data []byte) bool { + if err := um.Unmarshal(data); err != nil { + return false + } + return true +} + +func GetBool(v *bool) (vv bool, set bool) { + if v == nil { + return false, false + } + return *v, true +} + +func Boolp(b bool) *bool { return &b } diff --git a/raftexample/snap/db.go b/raftexample/snap/db.go new file mode 100644 index 00000000..4ac2c24e --- /dev/null +++ b/raftexample/snap/db.go @@ -0,0 +1,99 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package snap + +import ( + "errors" + "fmt" + "io" + "os" + "path/filepath" + "time" + + "go.etcd.io/raft/v3/raftexample/fileutil" + + humanize "github.com/dustin/go-humanize" + "go.uber.org/zap" +) + +var ErrNoDBSnapshot = errors.New("snap: snapshot file doesn't exist") + +// SaveDBFrom saves snapshot of the database from the given reader. It +// guarantees the save operation is atomic. +func (s *Snapshotter) SaveDBFrom(r io.Reader, id uint64) (int64, error) { + start := time.Now() + + f, err := os.CreateTemp(s.dir, "tmp") + if err != nil { + return 0, err + } + var n int64 + n, err = io.Copy(f, r) + if err == nil { + fsyncStart := time.Now() + err = fileutil.Fsync(f) + snapDBFsyncSec.Observe(time.Since(fsyncStart).Seconds()) + } + f.Close() + if err != nil { + os.Remove(f.Name()) + return n, err + } + fn := s.dbFilePath(id) + if fileutil.Exist(fn) { + os.Remove(f.Name()) + return n, nil + } + err = os.Rename(f.Name(), fn) + if err != nil { + os.Remove(f.Name()) + return n, err + } + + s.lg.Info( + "saved database snapshot to disk", + zap.String("path", fn), + zap.Int64("bytes", n), + zap.String("size", humanize.Bytes(uint64(n))), + ) + + snapDBSaveSec.Observe(time.Since(start).Seconds()) + return n, nil +} + +// DBFilePath returns the file path for the snapshot of the database with +// given id. If the snapshot does not exist, it returns error. +func (s *Snapshotter) DBFilePath(id uint64) (string, error) { + if _, err := fileutil.ReadDir(s.dir); err != nil { + return "", err + } + fn := s.dbFilePath(id) + if fileutil.Exist(fn) { + return fn, nil + } + if s.lg != nil { + s.lg.Warn( + "failed to find [SNAPSHOT-INDEX].snap.db", + zap.Uint64("snapshot-index", id), + zap.String("snapshot-file-path", fn), + zap.Error(ErrNoDBSnapshot), + ) + } + return "", ErrNoDBSnapshot +} + +func (s *Snapshotter) dbFilePath(id uint64) string { + return filepath.Join(s.dir, fmt.Sprintf("%016x.snap.db", id)) +} diff --git a/raftexample/snap/doc.go b/raftexample/snap/doc.go new file mode 100644 index 00000000..dcc5db57 --- /dev/null +++ b/raftexample/snap/doc.go @@ -0,0 +1,17 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +// Package snap handles Raft nodes' states with snapshots. +// The snapshot logic is internal to etcd server and raft package. +package snap diff --git a/raftexample/snap/message.go b/raftexample/snap/message.go new file mode 100644 index 00000000..9e7f067f --- /dev/null +++ b/raftexample/snap/message.go @@ -0,0 +1,64 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package snap + +import ( + "io" + + "go.etcd.io/raft/v3/raftexample/ioutil" + "go.etcd.io/raft/v3/raftpb" +) + +// Message is a struct that contains a raft Message and a ReadCloser. The type +// of raft message MUST be MsgSnap, which contains the raft meta-data and an +// additional data []byte field that contains the snapshot of the actual state +// machine. +// Message contains the ReadCloser field for handling large snapshot. This avoid +// copying the entire snapshot into a byte array, which consumes a lot of memory. +// +// User of Message should close the Message after sending it. +type Message struct { + raftpb.Message + ReadCloser io.ReadCloser + TotalSize int64 + closeC chan bool +} + +func NewMessage(rs raftpb.Message, rc io.ReadCloser, rcSize int64) *Message { + return &Message{ + Message: rs, + ReadCloser: ioutil.NewExactReadCloser(rc, rcSize), + TotalSize: int64(rs.Size()) + rcSize, + closeC: make(chan bool, 1), + } +} + +// CloseNotify returns a channel that receives a single value +// when the message sent is finished. true indicates the sent +// is successful. +func (m Message) CloseNotify() <-chan bool { + return m.closeC +} + +func (m Message) CloseWithError(err error) { + if cerr := m.ReadCloser.Close(); cerr != nil { + err = cerr + } + if err == nil { + m.closeC <- true + } else { + m.closeC <- false + } +} diff --git a/raftexample/snap/metrics.go b/raftexample/snap/metrics.go new file mode 100644 index 00000000..2affecf4 --- /dev/null +++ b/raftexample/snap/metrics.go @@ -0,0 +1,82 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package snap + +import "github.com/prometheus/client_golang/prometheus" + +var ( + snapMarshallingSec = prometheus.NewHistogram(prometheus.HistogramOpts{ + Namespace: "etcd_debugging", + Subsystem: "snap", + Name: "save_marshalling_duration_seconds", + Help: "The marshalling cost distributions of save called by snapshot.", + + // lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2 + // highest bucket start of 0.001 sec * 2^13 == 8.192 sec + Buckets: prometheus.ExponentialBuckets(0.001, 2, 14), + }) + + snapSaveSec = prometheus.NewHistogram(prometheus.HistogramOpts{ + Namespace: "etcd_debugging", + Subsystem: "snap", + Name: "save_total_duration_seconds", + Help: "The total latency distributions of save called by snapshot.", + + // lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2 + // highest bucket start of 0.001 sec * 2^13 == 8.192 sec + Buckets: prometheus.ExponentialBuckets(0.001, 2, 14), + }) + + snapFsyncSec = prometheus.NewHistogram(prometheus.HistogramOpts{ + Namespace: "etcd", + Subsystem: "snap", + Name: "fsync_duration_seconds", + Help: "The latency distributions of fsync called by snap.", + + // lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2 + // highest bucket start of 0.001 sec * 2^13 == 8.192 sec + Buckets: prometheus.ExponentialBuckets(0.001, 2, 14), + }) + + snapDBSaveSec = prometheus.NewHistogram(prometheus.HistogramOpts{ + Namespace: "etcd", + Subsystem: "snap_db", + Name: "save_total_duration_seconds", + Help: "The total latency distributions of v3 snapshot save", + + // lowest bucket start of upper bound 0.1 sec (100 ms) with factor 2 + // highest bucket start of 0.1 sec * 2^9 == 51.2 sec + Buckets: prometheus.ExponentialBuckets(0.1, 2, 10), + }) + + snapDBFsyncSec = prometheus.NewHistogram(prometheus.HistogramOpts{ + Namespace: "etcd", + Subsystem: "snap_db", + Name: "fsync_duration_seconds", + Help: "The latency distributions of fsyncing .snap.db file", + + // lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2 + // highest bucket start of 0.001 sec * 2^13 == 8.192 sec + Buckets: prometheus.ExponentialBuckets(0.001, 2, 14), + }) +) + +func init() { + prometheus.MustRegister(snapMarshallingSec) + prometheus.MustRegister(snapSaveSec) + prometheus.MustRegister(snapFsyncSec) + prometheus.MustRegister(snapDBSaveSec) + prometheus.MustRegister(snapDBFsyncSec) +} diff --git a/raftexample/snap/snappb/snap.pb.go b/raftexample/snap/snappb/snap.pb.go new file mode 100644 index 00000000..6fd2b9c0 --- /dev/null +++ b/raftexample/snap/snappb/snap.pb.go @@ -0,0 +1,344 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: snap.proto + +package snappb + +import ( + fmt "fmt" + io "io" + math "math" + math_bits "math/bits" + + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/golang/protobuf/proto" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package + +type Snapshot struct { + Crc uint32 `protobuf:"varint,1,opt,name=crc" json:"crc"` + Data []byte `protobuf:"bytes,2,opt,name=data" json:"data,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Snapshot) Reset() { *m = Snapshot{} } +func (m *Snapshot) String() string { return proto.CompactTextString(m) } +func (*Snapshot) ProtoMessage() {} +func (*Snapshot) Descriptor() ([]byte, []int) { + return fileDescriptor_f2e3c045ebf84d00, []int{0} +} +func (m *Snapshot) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Snapshot) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Snapshot.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Snapshot) XXX_Merge(src proto.Message) { + xxx_messageInfo_Snapshot.Merge(m, src) +} +func (m *Snapshot) XXX_Size() int { + return m.Size() +} +func (m *Snapshot) XXX_DiscardUnknown() { + xxx_messageInfo_Snapshot.DiscardUnknown(m) +} + +var xxx_messageInfo_Snapshot proto.InternalMessageInfo + +func init() { + proto.RegisterType((*Snapshot)(nil), "snappb.snapshot") +} + +func init() { proto.RegisterFile("snap.proto", fileDescriptor_f2e3c045ebf84d00) } + +var fileDescriptor_f2e3c045ebf84d00 = []byte{ + // 126 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xe2, 0x2a, 0xce, 0x4b, 0x2c, + 0xd0, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x62, 0x03, 0xb1, 0x0b, 0x92, 0xa4, 0x44, 0xd2, 0xf3, + 0xd3, 0xf3, 0xc1, 0x42, 0xfa, 0x20, 0x16, 0x44, 0x56, 0xc9, 0x8c, 0x8b, 0x03, 0x24, 0x5f, 0x9c, + 0x91, 0x5f, 0x22, 0x24, 0xc6, 0xc5, 0x9c, 0x5c, 0x94, 0x2c, 0xc1, 0xa8, 0xc0, 0xa8, 0xc1, 0xeb, + 0xc4, 0x72, 0xe2, 0x9e, 0x3c, 0x43, 0x10, 0x48, 0x40, 0x48, 0x88, 0x8b, 0x25, 0x25, 0xb1, 0x24, + 0x51, 0x82, 0x49, 0x81, 0x51, 0x83, 0x27, 0x08, 0xcc, 0x76, 0x12, 0x39, 0xf1, 0x50, 0x8e, 0xe1, + 0xc4, 0x23, 0x39, 0xc6, 0x0b, 0x8f, 0xe4, 0x18, 0x1f, 0x3c, 0x92, 0x63, 0x9c, 0xf1, 0x58, 0x8e, + 0x01, 0x10, 0x00, 0x00, 0xff, 0xff, 0xd8, 0x0f, 0x32, 0xb2, 0x78, 0x00, 0x00, 0x00, +} + +func (m *Snapshot) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Snapshot) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Snapshot) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.Data != nil { + i -= len(m.Data) + copy(dAtA[i:], m.Data) + i = encodeVarintSnap(dAtA, i, uint64(len(m.Data))) + i-- + dAtA[i] = 0x12 + } + i = encodeVarintSnap(dAtA, i, uint64(m.Crc)) + i-- + dAtA[i] = 0x8 + return len(dAtA) - i, nil +} + +func encodeVarintSnap(dAtA []byte, offset int, v uint64) int { + offset -= sovSnap(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *Snapshot) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovSnap(uint64(m.Crc)) + if m.Data != nil { + l = len(m.Data) + n += 1 + l + sovSnap(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func sovSnap(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozSnap(x uint64) (n int) { + return sovSnap(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *Snapshot) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSnap + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: snapshot: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: snapshot: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Crc", wireType) + } + m.Crc = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSnap + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Crc |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSnap + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthSnap + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthSnap + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Data = append(m.Data[:0], dAtA[iNdEx:postIndex]...) + if m.Data == nil { + m.Data = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSnap(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSnap + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipSnap(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + depth := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSnap + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSnap + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + case 1: + iNdEx += 8 + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSnap + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthSnap + } + iNdEx += length + case 3: + depth++ + case 4: + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupSnap + } + depth-- + case 5: + iNdEx += 4 + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + if iNdEx < 0 { + return 0, ErrInvalidLengthSnap + } + if depth == 0 { + return iNdEx, nil + } + } + return 0, io.ErrUnexpectedEOF +} + +var ( + ErrInvalidLengthSnap = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowSnap = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupSnap = fmt.Errorf("proto: unexpected end of group") +) diff --git a/raftexample/snap/snappb/snap.proto b/raftexample/snap/snappb/snap.proto new file mode 100644 index 00000000..cd3d21d0 --- /dev/null +++ b/raftexample/snap/snappb/snap.proto @@ -0,0 +1,14 @@ +syntax = "proto2"; +package snappb; + +import "gogoproto/gogo.proto"; + +option (gogoproto.marshaler_all) = true; +option (gogoproto.sizer_all) = true; +option (gogoproto.unmarshaler_all) = true; +option (gogoproto.goproto_getters_all) = false; + +message snapshot { + optional uint32 crc = 1 [(gogoproto.nullable) = false]; + optional bytes data = 2; +} diff --git a/raftexample/snap/snapshotter.go b/raftexample/snap/snapshotter.go new file mode 100644 index 00000000..df41bd12 --- /dev/null +++ b/raftexample/snap/snapshotter.go @@ -0,0 +1,282 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package snap + +import ( + "errors" + "fmt" + "go.etcd.io/raft/v3/raftexample/ioutil" + "hash/crc32" + "os" + "path/filepath" + "sort" + "strconv" + "strings" + "time" + + "go.etcd.io/etcd/server/v3/storage/wal/walpb" + "go.etcd.io/raft/v3" + pioutil "go.etcd.io/raft/v3/raftexample/pbutil" + "go.etcd.io/raft/v3/raftexample/snap/snappb" + "go.etcd.io/raft/v3/raftexample/verify" + "go.etcd.io/raft/v3/raftpb" + + "go.uber.org/zap" +) + +const snapSuffix = ".snap" + +var ( + ErrNoSnapshot = errors.New("snap: no available snapshot") + ErrEmptySnapshot = errors.New("snap: empty snapshot") + ErrCRCMismatch = errors.New("snap: crc mismatch") + crcTable = crc32.MakeTable(crc32.Castagnoli) + + // A map of valid files that can be present in the snap folder. + validFiles = map[string]bool{ + "db": true, + } +) + +type Snapshotter struct { + lg *zap.Logger + dir string +} + +func New(lg *zap.Logger, dir string) *Snapshotter { + if lg == nil { + lg = zap.NewNop() + } + return &Snapshotter{ + lg: lg, + dir: dir, + } +} + +func (s *Snapshotter) SaveSnap(snapshot raftpb.Snapshot) error { + if raft.IsEmptySnap(snapshot) { + return nil + } + return s.save(&snapshot) +} + +func (s *Snapshotter) save(snapshot *raftpb.Snapshot) error { + start := time.Now() + + fname := fmt.Sprintf("%016x-%016x%s", snapshot.Metadata.Term, snapshot.Metadata.Index, snapSuffix) + b := pioutil.MustMarshal(snapshot) + crc := crc32.Update(0, crcTable, b) + snap := snappb.Snapshot{Crc: crc, Data: b} + d, err := snap.Marshal() + if err != nil { + return err + } + snapMarshallingSec.Observe(time.Since(start).Seconds()) + + spath := filepath.Join(s.dir, fname) + + fsyncStart := time.Now() + err = ioutil.WriteAndSyncFile(spath, d, 0666) + snapFsyncSec.Observe(time.Since(fsyncStart).Seconds()) + + if err != nil { + s.lg.Warn("failed to write a snap file", zap.String("path", spath), zap.Error(err)) + rerr := os.Remove(spath) + if rerr != nil { + s.lg.Warn("failed to remove a broken snap file", zap.String("path", spath), zap.Error(rerr)) + } + return err + } + + snapSaveSec.Observe(time.Since(start).Seconds()) + return nil +} + +// Load returns the newest snapshot. +func (s *Snapshotter) Load() (*raftpb.Snapshot, error) { + return s.loadMatching(func(*raftpb.Snapshot) bool { return true }) +} + +// LoadNewestAvailable loads the newest snapshot available that is in walSnaps. +func (s *Snapshotter) LoadNewestAvailable(walSnaps []walpb.Snapshot) (*raftpb.Snapshot, error) { + return s.loadMatching(func(snapshot *raftpb.Snapshot) bool { + m := snapshot.Metadata + for i := len(walSnaps) - 1; i >= 0; i-- { + if m.Term == walSnaps[i].Term && m.Index == walSnaps[i].Index { + return true + } + } + return false + }) +} + +// loadMatching returns the newest snapshot where matchFn returns true. +func (s *Snapshotter) loadMatching(matchFn func(*raftpb.Snapshot) bool) (*raftpb.Snapshot, error) { + names, err := s.snapNames() + if err != nil { + return nil, err + } + var snap *raftpb.Snapshot + for _, name := range names { + if snap, err = s.loadSnap(name); err == nil && matchFn(snap) { + return snap, nil + } + } + return nil, ErrNoSnapshot +} + +func (s *Snapshotter) loadSnap(name string) (*raftpb.Snapshot, error) { + fpath := filepath.Join(s.dir, name) + snap, err := Read(s.lg, fpath) + if err != nil { + brokenPath := fpath + ".broken" + s.lg.Warn("failed to read a snap file", zap.String("path", fpath), zap.Error(err)) + if rerr := os.Rename(fpath, brokenPath); rerr != nil { + s.lg.Warn("failed to rename a broken snap file", zap.String("path", fpath), zap.String("broken-path", brokenPath), zap.Error(rerr)) + } else { + s.lg.Warn("renamed to a broken snap file", zap.String("path", fpath), zap.String("broken-path", brokenPath)) + } + } + return snap, err +} + +// Read reads the snapshot named by snapname and returns the snapshot. +func Read(lg *zap.Logger, snapname string) (*raftpb.Snapshot, error) { + verify.Assert(lg != nil, "the logger should not be nil") + b, err := os.ReadFile(snapname) + if err != nil { + lg.Warn("failed to read a snap file", zap.String("path", snapname), zap.Error(err)) + return nil, err + } + + if len(b) == 0 { + lg.Warn("failed to read empty snapshot file", zap.String("path", snapname)) + return nil, ErrEmptySnapshot + } + + var serializedSnap snappb.Snapshot + if err = serializedSnap.Unmarshal(b); err != nil { + lg.Warn("failed to unmarshal snappb.Snapshot", zap.String("path", snapname), zap.Error(err)) + return nil, err + } + + if len(serializedSnap.Data) == 0 || serializedSnap.Crc == 0 { + lg.Warn("failed to read empty snapshot data", zap.String("path", snapname)) + return nil, ErrEmptySnapshot + } + + crc := crc32.Update(0, crcTable, serializedSnap.Data) + if crc != serializedSnap.Crc { + lg.Warn("snap file is corrupt", + zap.String("path", snapname), + zap.Uint32("prev-crc", serializedSnap.Crc), + zap.Uint32("new-crc", crc), + ) + return nil, ErrCRCMismatch + } + + var snap raftpb.Snapshot + if err = snap.Unmarshal(serializedSnap.Data); err != nil { + lg.Warn("failed to unmarshal raftpb.Snapshot", zap.String("path", snapname), zap.Error(err)) + return nil, err + } + return &snap, nil +} + +// snapNames returns the filename of the snapshots in logical time order (from newest to oldest). +// If there is no available snapshots, an ErrNoSnapshot will be returned. +func (s *Snapshotter) snapNames() ([]string, error) { + dir, err := os.Open(s.dir) + if err != nil { + return nil, err + } + defer dir.Close() + names, err := dir.Readdirnames(-1) + if err != nil { + return nil, err + } + filenames, err := s.cleanupSnapdir(names) + if err != nil { + return nil, err + } + snaps := s.checkSuffix(filenames) + if len(snaps) == 0 { + return nil, ErrNoSnapshot + } + sort.Sort(sort.Reverse(sort.StringSlice(snaps))) + return snaps, nil +} + +func (s *Snapshotter) checkSuffix(names []string) []string { + var snaps []string + for i := range names { + if strings.HasSuffix(names[i], snapSuffix) { + snaps = append(snaps, names[i]) + } else { + // If we find a file which is not a snapshot then check if it's + // a valid file. If not throw out a warning. + if _, ok := validFiles[names[i]]; !ok { + s.lg.Warn("found unexpected non-snap file; skipping", zap.String("path", names[i])) + } + } + } + return snaps +} + +// cleanupSnapdir removes any files that should not be in the snapshot directory: +// - db.tmp prefixed files that can be orphaned by defragmentation +func (s *Snapshotter) cleanupSnapdir(filenames []string) (names []string, err error) { + names = make([]string, 0, len(filenames)) + for _, filename := range filenames { + if strings.HasPrefix(filename, "db.tmp") { + s.lg.Info("found orphaned defragmentation file; deleting", zap.String("path", filename)) + if rmErr := os.Remove(filepath.Join(s.dir, filename)); rmErr != nil && !os.IsNotExist(rmErr) { + return names, fmt.Errorf("failed to remove orphaned .snap.db file %s: %v", filename, rmErr) + } + } else { + names = append(names, filename) + } + } + return names, nil +} + +func (s *Snapshotter) ReleaseSnapDBs(snap raftpb.Snapshot) error { + dir, err := os.Open(s.dir) + if err != nil { + return err + } + defer dir.Close() + filenames, err := dir.Readdirnames(-1) + if err != nil { + return err + } + for _, filename := range filenames { + if strings.HasSuffix(filename, ".snap.db") { + hexIndex := strings.TrimSuffix(filepath.Base(filename), ".snap.db") + index, err := strconv.ParseUint(hexIndex, 16, 64) + if err != nil { + s.lg.Error("failed to parse index from filename", zap.String("path", filename), zap.String("error", err.Error())) + continue + } + if index < snap.Metadata.Index { + s.lg.Info("found orphaned .snap.db file; deleting", zap.String("path", filename)) + if rmErr := os.Remove(filepath.Join(s.dir, filename)); rmErr != nil && !os.IsNotExist(rmErr) { + s.lg.Error("failed to remove orphaned .snap.db file", zap.String("path", filename), zap.String("error", rmErr.Error())) + } + } + } + } + return nil +} diff --git a/raftexample/verify/verify.go b/raftexample/verify/verify.go new file mode 100644 index 00000000..0cc1b482 --- /dev/null +++ b/raftexample/verify/verify.go @@ -0,0 +1,80 @@ +// Copyright 2022 The etcd Authors +// +// 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. + +package verify + +import ( + "fmt" + "os" + "strings" +) + +const ENV_VERIFY = "ETCD_VERIFY" + +type VerificationType string + +const ( + ENV_VERIFY_VALUE_ALL VerificationType = "all" + ENV_VERIFY_VALUE_ASSERT VerificationType = "assert" +) + +func getEnvVerify() string { + return strings.ToLower(os.Getenv(ENV_VERIFY)) +} + +func IsVerificationEnabled(verification VerificationType) bool { + env := getEnvVerify() + return env == string(ENV_VERIFY_VALUE_ALL) || env == strings.ToLower(string(verification)) +} + +// EnableVerifications sets `ENV_VERIFY` and returns a function that +// can be used to bring the original settings. +func EnableVerifications(verification VerificationType) func() { + previousEnv := getEnvVerify() + os.Setenv(ENV_VERIFY, string(verification)) + return func() { + os.Setenv(ENV_VERIFY, previousEnv) + } +} + +// EnableAllVerifications enables verification and returns a function +// that can be used to bring the original settings. +func EnableAllVerifications() func() { + return EnableVerifications(ENV_VERIFY_VALUE_ALL) +} + +// DisableVerifications unsets `ENV_VERIFY` and returns a function that +// can be used to bring the original settings. +func DisableVerifications() func() { + previousEnv := getEnvVerify() + os.Unsetenv(ENV_VERIFY) + return func() { + os.Setenv(ENV_VERIFY, previousEnv) + } +} + +// Verify performs verification if the assertions are enabled. +// In the default setup running in tests and skipped in the production code. +func Verify(f func()) { + if IsVerificationEnabled(ENV_VERIFY_VALUE_ASSERT) { + f() + } +} + +// Assert will panic with a given formatted message if the given condition is false. +func Assert(condition bool, msg string, v ...interface{}) { + if !condition { + panic(fmt.Sprintf("assertion failed: "+msg, v...)) + } +} From 8da65a050c1a84abe6a9776097f6fbbfd968c110 Mon Sep 17 00:00:00 2001 From: Mustafa Elbehery Date: Wed, 31 May 2023 18:44:27 +0200 Subject: [PATCH 3/5] changes go mod tidy --- go.mod | 19 ++++++++++++++++--- go.sum | 48 +++++++++++++++++++++++++++++++++++++++++------- 2 files changed, 57 insertions(+), 10 deletions(-) diff --git a/go.mod b/go.mod index fe255350..b12ec228 100644 --- a/go.mod +++ b/go.mod @@ -4,15 +4,28 @@ go 1.19 require ( github.com/cockroachdb/datadriven v1.0.2 + github.com/dustin/go-humanize v1.0.1 github.com/gogo/protobuf v1.3.2 - github.com/golang/protobuf v1.5.2 + github.com/golang/protobuf v1.5.3 + github.com/prometheus/client_golang v1.15.1 github.com/stretchr/testify v1.8.3 + go.uber.org/zap v1.24.0 + golang.org/x/sys v0.8.0 ) require ( + github.com/benbjohnson/clock v1.1.0 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/davecgh/go-spew v1.1.1 // indirect - github.com/google/go-cmp v0.5.8 // indirect + github.com/kr/text v0.2.0 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - google.golang.org/protobuf v1.27.1 // indirect + github.com/prometheus/client_model v0.4.0 // indirect + github.com/prometheus/common v0.44.0 // indirect + github.com/prometheus/procfs v0.10.1 // indirect + go.uber.org/atomic v1.11.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + google.golang.org/protobuf v1.30.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go.sum b/go.sum index 777708a6..80a41922 100644 --- a/go.sum +++ b/go.sum @@ -1,23 +1,54 @@ +github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= +github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= +github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= +github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= +github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cockroachdb/datadriven v1.0.2 h1:H9MtNqVoVhvd9nCBwOyDjUEdZCREqbIdCJD93PBm/jA= github.com/cockroachdb/datadriven v1.0.2/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU= +github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= +github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= -github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= +github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.8 h1:e6P7q2lk1O+qJJb4BtCQXlK8vWEO8V1ZeuEdJNOqZyg= -github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= +github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo= +github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= +github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_golang v1.15.1 h1:8tXpTmJbyH5lydzFPoxSIJ0J46jdh3tylbvM1xCv0LI= +github.com/prometheus/client_golang v1.15.1/go.mod h1:e9yaBhRPU2pPNsZwE+JdQl0KEt1N9XgF6zxWmaC0xOk= +github.com/prometheus/client_model v0.4.0 h1:5lQXD3cAg1OXBf4Wq03gTrXHeaV0TQvGfUooCfx1yqY= +github.com/prometheus/client_model v0.4.0/go.mod h1:oMQmHW1/JoDwqLtg57MGgP/Fb1CJEYF2imWWhWtMkYU= +github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdOOfY= +github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= +github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg= +github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= +github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= github.com/stretchr/testify v1.8.3 h1:RP3t2pwF7cMEbC1dqtB6poj3niw/9gnV4Cjg5oW5gtY= github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= +go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= +go.uber.org/goleak v1.1.11 h1:wy28qYRKZgnJTxGxvye5/wgWr1EKjmUDGYox5mGlRlI= +go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= +go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= +go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= +go.uber.org/zap v1.24.0/go.mod h1:2kMP+WWQ8aoFoedH3T2sq6iJ2yDWpHbP0f6MQbS9Gkg= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= @@ -27,12 +58,15 @@ golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.8.0 h1:EBmGv8NaZBZTWvrbjNoL6HVt+IVy3QDQpJs7VRIw3tU= +golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -45,9 +79,9 @@ golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.27.1 h1:SnqbnDw1V7RiZcXPx5MEeqPv2s79L9i7BJUlG/+RurQ= -google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM= +google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng= +google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= From aa642cdbdcc1c20385f4d6b01311cfa17f5c1c57 Mon Sep 17 00:00:00 2001 From: Mustafa Elbehery Date: Wed, 31 May 2023 18:47:46 +0200 Subject: [PATCH 4/5] add walpb --- raftexample/kvstore.go | 2 +- raftexample/snap/snapshotter.go | 2 +- raftexample/walpb/record.go | 43 +++ raftexample/walpb/record.pb.go | 609 +++++++++++++++++++++++++++++++ raftexample/walpb/record.proto | 24 ++ raftexample/walpb/record_test.go | 51 +++ 6 files changed, 729 insertions(+), 2 deletions(-) create mode 100644 raftexample/walpb/record.go create mode 100644 raftexample/walpb/record.pb.go create mode 100644 raftexample/walpb/record.proto create mode 100644 raftexample/walpb/record_test.go diff --git a/raftexample/kvstore.go b/raftexample/kvstore.go index 8c3d8b51..e87c3130 100644 --- a/raftexample/kvstore.go +++ b/raftexample/kvstore.go @@ -69,8 +69,8 @@ func (s *kvStore) readCommits(commitC <-chan *commit, errorC <-chan error) { log.Fatalf("raftexample: could not decode message (%v)", err) } s.mu.Lock() - defer s.mu.Unlock() s.store[dataKv.Key] = dataKv.Val + s.mu.Unlock() } close(c.applyDoneC) } diff --git a/raftexample/snap/snapshotter.go b/raftexample/snap/snapshotter.go index df41bd12..c24e1352 100644 --- a/raftexample/snap/snapshotter.go +++ b/raftexample/snap/snapshotter.go @@ -26,11 +26,11 @@ import ( "strings" "time" - "go.etcd.io/etcd/server/v3/storage/wal/walpb" "go.etcd.io/raft/v3" pioutil "go.etcd.io/raft/v3/raftexample/pbutil" "go.etcd.io/raft/v3/raftexample/snap/snappb" "go.etcd.io/raft/v3/raftexample/verify" + "go.etcd.io/raft/v3/raftexample/walpb" "go.etcd.io/raft/v3/raftpb" "go.uber.org/zap" diff --git a/raftexample/walpb/record.go b/raftexample/walpb/record.go new file mode 100644 index 00000000..693deab1 --- /dev/null +++ b/raftexample/walpb/record.go @@ -0,0 +1,43 @@ +// Copyright 2015 The etcd Authors +// +// 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. + +package walpb + +import ( + "errors" + "fmt" +) + +var ( + ErrCRCMismatch = errors.New("walpb: crc mismatch") +) + +func (rec *Record) Validate(crc uint32) error { + if rec.Crc == crc { + return nil + } + return fmt.Errorf("%w: expected: %x computed: %x", ErrCRCMismatch, rec.Crc, crc) +} + +// ValidateSnapshotForWrite ensures the Snapshot the newly written snapshot is valid. +// +// There might exist log-entries written by old etcd versions that does not conform +// to the requirements. +func ValidateSnapshotForWrite(e *Snapshot) error { + // Since etcd>=3.5.0 + if e.ConfState == nil && e.Index > 0 { + return errors.New("Saved (not-initial) snapshot is missing ConfState: " + e.String()) + } + return nil +} diff --git a/raftexample/walpb/record.pb.go b/raftexample/walpb/record.pb.go new file mode 100644 index 00000000..d0eba734 --- /dev/null +++ b/raftexample/walpb/record.pb.go @@ -0,0 +1,609 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: record.proto + +package walpb + +import ( + fmt "fmt" + io "io" + math "math" + math_bits "math/bits" + + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/golang/protobuf/proto" + raftpb "go.etcd.io/raft/v3/raftpb" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package + +type Record struct { + Type int64 `protobuf:"varint,1,opt,name=type" json:"type"` + Crc uint32 `protobuf:"varint,2,opt,name=crc" json:"crc"` + Data []byte `protobuf:"bytes,3,opt,name=data" json:"data,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Record) Reset() { *m = Record{} } +func (m *Record) String() string { return proto.CompactTextString(m) } +func (*Record) ProtoMessage() {} +func (*Record) Descriptor() ([]byte, []int) { + return fileDescriptor_bf94fd919e302a1d, []int{0} +} +func (m *Record) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Record) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Record.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Record) XXX_Merge(src proto.Message) { + xxx_messageInfo_Record.Merge(m, src) +} +func (m *Record) XXX_Size() int { + return m.Size() +} +func (m *Record) XXX_DiscardUnknown() { + xxx_messageInfo_Record.DiscardUnknown(m) +} + +var xxx_messageInfo_Record proto.InternalMessageInfo + +// Keep in sync with raftpb.SnapshotMetadata. +type Snapshot struct { + Index uint64 `protobuf:"varint,1,opt,name=index" json:"index"` + Term uint64 `protobuf:"varint,2,opt,name=term" json:"term"` + // Field populated since >=etcd-3.5.0. + ConfState *raftpb.ConfState `protobuf:"bytes,3,opt,name=conf_state,json=confState" json:"conf_state,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Snapshot) Reset() { *m = Snapshot{} } +func (m *Snapshot) String() string { return proto.CompactTextString(m) } +func (*Snapshot) ProtoMessage() {} +func (*Snapshot) Descriptor() ([]byte, []int) { + return fileDescriptor_bf94fd919e302a1d, []int{1} +} +func (m *Snapshot) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Snapshot) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Snapshot.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Snapshot) XXX_Merge(src proto.Message) { + xxx_messageInfo_Snapshot.Merge(m, src) +} +func (m *Snapshot) XXX_Size() int { + return m.Size() +} +func (m *Snapshot) XXX_DiscardUnknown() { + xxx_messageInfo_Snapshot.DiscardUnknown(m) +} + +var xxx_messageInfo_Snapshot proto.InternalMessageInfo + +func init() { + proto.RegisterType((*Record)(nil), "walpb.Record") + proto.RegisterType((*Snapshot)(nil), "walpb.Snapshot") +} + +func init() { proto.RegisterFile("record.proto", fileDescriptor_bf94fd919e302a1d) } + +var fileDescriptor_bf94fd919e302a1d = []byte{ + // 233 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x3c, 0x8e, 0x41, 0x4e, 0xc3, 0x30, + 0x10, 0x45, 0x63, 0xe2, 0x22, 0x18, 0xca, 0xa2, 0x56, 0x85, 0xa2, 0x2c, 0x4c, 0xd4, 0x55, 0x56, + 0x29, 0xe2, 0x08, 0x65, 0xcf, 0x22, 0x3d, 0x00, 0x72, 0x1d, 0xa7, 0x20, 0xd1, 0x8c, 0x35, 0xb5, + 0x04, 0xdc, 0x84, 0x23, 0x65, 0xc9, 0x09, 0x10, 0x84, 0x8b, 0xa0, 0x8c, 0x03, 0xab, 0xf9, 0x7a, + 0x5f, 0xff, 0xff, 0x81, 0x39, 0x39, 0x8b, 0xd4, 0x54, 0x9e, 0x30, 0xa0, 0x9a, 0xbd, 0x98, 0x67, + 0xbf, 0xcb, 0x97, 0x7b, 0xdc, 0x23, 0x93, 0xf5, 0xa8, 0xa2, 0x99, 0x2f, 0xc8, 0xb4, 0xc1, 0xef, + 0xd6, 0xe3, 0x89, 0x68, 0x75, 0x0f, 0xa7, 0x35, 0xe7, 0x55, 0x06, 0x32, 0xbc, 0x79, 0x97, 0x89, + 0x42, 0x94, 0xe9, 0x46, 0xf6, 0x9f, 0xd7, 0x49, 0xcd, 0x44, 0x5d, 0x41, 0x6a, 0xc9, 0x66, 0x27, + 0x85, 0x28, 0x2f, 0x27, 0x63, 0x04, 0x4a, 0x81, 0x6c, 0x4c, 0x30, 0x59, 0x5a, 0x88, 0x72, 0x5e, + 0xb3, 0x5e, 0x11, 0x9c, 0x6d, 0x3b, 0xe3, 0x8f, 0x8f, 0x18, 0x54, 0x0e, 0xb3, 0xa7, 0xae, 0x71, + 0xaf, 0x5c, 0x29, 0xa7, 0x64, 0x44, 0xbc, 0xe6, 0xe8, 0xc0, 0xa5, 0xf2, 0x7f, 0xcd, 0xd1, 0x41, + 0xdd, 0x00, 0x58, 0xec, 0xda, 0x87, 0x63, 0x30, 0xc1, 0x71, 0xf7, 0xc5, 0xed, 0xa2, 0x8a, 0x9f, + 0x57, 0x77, 0xd8, 0xb5, 0xdb, 0xd1, 0xa8, 0xcf, 0xed, 0x9f, 0xdc, 0x2c, 0xfb, 0x6f, 0x9d, 0xf4, + 0x83, 0x16, 0x1f, 0x83, 0x16, 0x5f, 0x83, 0x16, 0xef, 0x3f, 0x3a, 0xf9, 0x0d, 0x00, 0x00, 0xff, + 0xff, 0x60, 0x0f, 0x3c, 0x36, 0x18, 0x01, 0x00, 0x00, +} + +func (m *Record) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Record) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Record) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.Data != nil { + i -= len(m.Data) + copy(dAtA[i:], m.Data) + i = encodeVarintRecord(dAtA, i, uint64(len(m.Data))) + i-- + dAtA[i] = 0x1a + } + i = encodeVarintRecord(dAtA, i, uint64(m.Crc)) + i-- + dAtA[i] = 0x10 + i = encodeVarintRecord(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x8 + return len(dAtA) - i, nil +} + +func (m *Snapshot) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Snapshot) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Snapshot) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.ConfState != nil { + { + size, err := m.ConfState.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintRecord(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + i = encodeVarintRecord(dAtA, i, uint64(m.Term)) + i-- + dAtA[i] = 0x10 + i = encodeVarintRecord(dAtA, i, uint64(m.Index)) + i-- + dAtA[i] = 0x8 + return len(dAtA) - i, nil +} + +func encodeVarintRecord(dAtA []byte, offset int, v uint64) int { + offset -= sovRecord(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *Record) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovRecord(uint64(m.Type)) + n += 1 + sovRecord(uint64(m.Crc)) + if m.Data != nil { + l = len(m.Data) + n += 1 + l + sovRecord(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Snapshot) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovRecord(uint64(m.Index)) + n += 1 + sovRecord(uint64(m.Term)) + if m.ConfState != nil { + l = m.ConfState.Size() + n += 1 + l + sovRecord(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func sovRecord(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozRecord(x uint64) (n int) { + return sovRecord(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *Record) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRecord + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Record: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Record: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRecord + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Crc", wireType) + } + m.Crc = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRecord + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Crc |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRecord + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRecord + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthRecord + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Data = append(m.Data[:0], dAtA[iNdEx:postIndex]...) + if m.Data == nil { + m.Data = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRecord(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthRecord + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Snapshot) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRecord + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Snapshot: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Snapshot: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + m.Index = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRecord + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Index |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Term", wireType) + } + m.Term = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRecord + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Term |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ConfState", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRecord + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRecord + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRecord + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ConfState == nil { + m.ConfState = &raftpb.ConfState{} + } + if err := m.ConfState.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRecord(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthRecord + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipRecord(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + depth := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowRecord + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowRecord + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + case 1: + iNdEx += 8 + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowRecord + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthRecord + } + iNdEx += length + case 3: + depth++ + case 4: + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupRecord + } + depth-- + case 5: + iNdEx += 4 + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + if iNdEx < 0 { + return 0, ErrInvalidLengthRecord + } + if depth == 0 { + return iNdEx, nil + } + } + return 0, io.ErrUnexpectedEOF +} + +var ( + ErrInvalidLengthRecord = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowRecord = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupRecord = fmt.Errorf("proto: unexpected end of group") +) diff --git a/raftexample/walpb/record.proto b/raftexample/walpb/record.proto new file mode 100644 index 00000000..aed4351d --- /dev/null +++ b/raftexample/walpb/record.proto @@ -0,0 +1,24 @@ +syntax = "proto2"; +package walpb; + +import "gogoproto/gogo.proto"; +import "raftpb/raft.proto"; + +option (gogoproto.marshaler_all) = true; +option (gogoproto.sizer_all) = true; +option (gogoproto.unmarshaler_all) = true; +option (gogoproto.goproto_getters_all) = false; + +message Record { + optional int64 type = 1 [(gogoproto.nullable) = false]; + optional uint32 crc = 2 [(gogoproto.nullable) = false]; + optional bytes data = 3; +} + +// Keep in sync with raftpb.SnapshotMetadata. +message Snapshot { + optional uint64 index = 1 [(gogoproto.nullable) = false]; + optional uint64 term = 2 [(gogoproto.nullable) = false]; + // Field populated since >=etcd-3.5.0. + optional raftpb.ConfState conf_state = 3; +} diff --git a/raftexample/walpb/record_test.go b/raftexample/walpb/record_test.go new file mode 100644 index 00000000..cdacb3d0 --- /dev/null +++ b/raftexample/walpb/record_test.go @@ -0,0 +1,51 @@ +// Copyright 2022 The etcd Authors +// +// 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. + +package walpb + +import ( + "testing" + + "github.com/golang/protobuf/descriptor" + + "go.etcd.io/raft/v3/raftpb" +) + +func TestSnapshotMetadataCompatibility(t *testing.T) { + _, snapshotMetadataMd := descriptor.ForMessage(&raftpb.SnapshotMetadata{}) + _, snapshotMd := descriptor.ForMessage(&Snapshot{}) + if len(snapshotMetadataMd.GetField()) != len(snapshotMd.GetField()) { + t.Errorf("Different number of fields in raftpb.SnapshotMetadata vs. walpb.Snapshot. " + + "They are supposed to be in sync.") + } +} + +func TestValidateSnapshot(t *testing.T) { + tests := []struct { + name string + snap *Snapshot + wantErr bool + }{ + {name: "empty", snap: &Snapshot{}, wantErr: false}, + {name: "invalid", snap: &Snapshot{Index: 5, Term: 3}, wantErr: true}, + {name: "valid", snap: &Snapshot{Index: 5, Term: 3, ConfState: &raftpb.ConfState{Voters: []uint64{0x00cad1}}}, wantErr: false}, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + if err := ValidateSnapshotForWrite(tt.snap); (err != nil) != tt.wantErr { + t.Errorf("ValidateSnapshotForWrite() error = %v, wantErr %v", err, tt.wantErr) + } + }) + } +} From 6d1391d582f492264516c3b3c89cb56e356fb32b Mon Sep 17 00:00:00 2001 From: Mustafa Elbehery Date: Wed, 31 May 2023 18:54:53 +0200 Subject: [PATCH 5/5] add snapshotter --- raftexample/kvstore.go | 43 +++++++++++++++++++++++++++++++++--------- 1 file changed, 34 insertions(+), 9 deletions(-) diff --git a/raftexample/kvstore.go b/raftexample/kvstore.go index e87c3130..5654e40c 100644 --- a/raftexample/kvstore.go +++ b/raftexample/kvstore.go @@ -4,6 +4,8 @@ import ( "bytes" "encoding/gob" "encoding/json" + "go.etcd.io/raft/v3/raftexample/snap" + "go.etcd.io/raft/v3/raftpb" "log" "strings" "sync" @@ -23,20 +25,31 @@ type kv struct { } type kvStore struct { - proposeC chan<- string - mu sync.RWMutex - store map[string]string - // TODO: snapshotter + proposeC chan<- string + mu sync.RWMutex + store map[string]string + snapshotter *snap.Snapshotter } -func newKVStore(proposeC chan<- string, commitC <-chan *commit, errorC <-chan error) KVStore { +func newKVStore(snapshotter *snap.Snapshotter, proposeC chan<- string, commitC <-chan *commit, errorC <-chan error) KVStore { s := &kvStore{ - proposeC: proposeC, - mu: sync.RWMutex{}, - store: make(map[string]string), + proposeC: proposeC, + mu: sync.RWMutex{}, + store: make(map[string]string), + snapshotter: snapshotter, } - // TODO: load snapshot + snapshot, err := s.loadSnapshot() + if err != nil { + log.Panic(err) + } + if snapshot != nil { + log.Printf("loading snapshot at term %d and index %d", snapshot.Metadata.Term, snapshot.Metadata.Index) + if err := s.recoverFromSnapshot(snapshot.Data); err != nil { + log.Panic(err) + } + } + // read commits from raft into kvStore map until error go s.readCommits(commitC, errorC) return s } @@ -86,6 +99,18 @@ func (s *kvStore) getSnapshot() ([]byte, error) { return json.Marshal(s.store) } +func (s *kvStore) loadSnapshot() (*raftpb.Snapshot, error) { + snapshot, err := s.snapshotter.Load() + if err == snap.ErrNoSnapshot { + return nil, nil + } + if err != nil { + return nil, err + } + + return snapshot, nil +} + func (s *kvStore) recoverFromSnapshot(snapshot []byte) error { var newStore map[string]string if err := json.Unmarshal(snapshot, &newStore); err != nil {