From 0918ef681540d7aec8d141f79dee1e9ac06af752 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 2 May 2018 21:26:11 +0200 Subject: [PATCH 001/167] WIP: diffing and replication algorithm --- cmd/replication/diff.go | 116 ++++++++++++ cmd/replication/diff_test.go | 269 ++++++++++++++++++++++++++++ cmd/replication/replication.go | 209 +++++++++++++++++++++ cmd/replication/replication_test.go | 152 ++++++++++++++++ 4 files changed, 746 insertions(+) create mode 100644 cmd/replication/diff.go create mode 100644 cmd/replication/diff_test.go create mode 100644 cmd/replication/replication.go create mode 100644 cmd/replication/replication_test.go diff --git a/cmd/replication/diff.go b/cmd/replication/diff.go new file mode 100644 index 0000000..3494ba6 --- /dev/null +++ b/cmd/replication/diff.go @@ -0,0 +1,116 @@ +package replication + +import ( + "github.com/zrepl/zrepl/zfs" + "sort" +) + +type ConflictNoCommonAncestor struct { + SortedSenderVersions, SortedReceiverVersions []zfs.FilesystemVersion +} + +func (c *ConflictNoCommonAncestor) Error() string { + return "no common snapshot or suitable bookmark between sender and receiver" +} + +type ConflictDiverged struct { + SortedSenderVersions, SortedReceiverVersions []zfs.FilesystemVersion + CommonAncestor zfs.FilesystemVersion + SenderOnly, ReceiverOnly []zfs.FilesystemVersion +} + +func (c *ConflictDiverged) Error() string { + return "the receiver's latest snapshot is not present on sender" +} + +func SortVersionListByCreateTXGThenBookmarkLTSnapshot(fsvslice []zfs.FilesystemVersion) []zfs.FilesystemVersion { + lesser := func(s []zfs.FilesystemVersion) func(i, j int) bool { + return func(i, j int) bool { + if s[i].CreateTXG < s[j].CreateTXG { + return true + } + if s[i].CreateTXG == s[j].CreateTXG { + // Bookmark < Snapshot + return s[i].Type == zfs.Bookmark && s[j].Type == zfs.Snapshot + } + return false + } + } + if sort.SliceIsSorted(fsvslice, lesser(fsvslice)) { + return fsvslice + } + sorted := make([]zfs.FilesystemVersion, len(fsvslice)) + copy(sorted, fsvslice) + sort.Slice(sorted, lesser(sorted)) + return sorted +} + +// conflict may be a *ConflictDiverged or a *ConflictNoCommonAncestor +func IncrementalPath(receiver, sender []zfs.FilesystemVersion) (incPath []zfs.FilesystemVersion, conflict error) { + + if receiver == nil { + panic("receiver must not be nil") + } + if sender == nil { + panic("sender must not be nil") + } + + receiver = SortVersionListByCreateTXGThenBookmarkLTSnapshot(receiver) + sender = SortVersionListByCreateTXGThenBookmarkLTSnapshot(sender) + + if len(sender) == 0 { + return []zfs.FilesystemVersion{}, nil + } + + // Find most recent common ancestor by name, preferring snapshots over bookmarks + + mrcaRcv := len(receiver) - 1 + mrcaSnd := len(sender) - 1 + + for mrcaRcv >= 0 && mrcaSnd >= 0 { + if receiver[mrcaRcv].Guid == sender[mrcaSnd].Guid { + if mrcaSnd-1 >= 0 && sender[mrcaSnd-1].Guid == sender[mrcaSnd].Guid && sender[mrcaSnd-1].Type == zfs.Bookmark { + // prefer bookmarks over snapshots as the snapshot might go away sooner + mrcaSnd -= 1 + } + break + } + if receiver[mrcaRcv].CreateTXG < sender[mrcaSnd].CreateTXG { + mrcaSnd-- + } else { + mrcaRcv-- + } + } + + if mrcaRcv == -1 || mrcaSnd == -1 { + return nil, &ConflictNoCommonAncestor{ + SortedSenderVersions: sender, + SortedReceiverVersions: receiver, + } + } + + if mrcaRcv != len(receiver)-1 { + return nil, &ConflictDiverged{ + SortedSenderVersions: sender, + SortedReceiverVersions: receiver, + CommonAncestor: sender[mrcaSnd], + SenderOnly: sender[mrcaSnd+1:], + ReceiverOnly: receiver[mrcaRcv+1:], + } + } + + // incPath must not contain bookmarks except initial one, + incPath = make([]zfs.FilesystemVersion, 0, len(sender)) + incPath = append(incPath, sender[mrcaSnd]) + // it's ok if incPath[0] is a bookmark, but not the subsequent ones in the incPath + for i := mrcaSnd + 1; i < len(sender); i++ { + if sender[i].Type == zfs.Snapshot && incPath[len(incPath)-1].Guid != sender[i].Guid { + incPath = append(incPath, sender[i]) + } + } + if len(incPath) == 1 { + // nothing to do + incPath = incPath[1:] + } + return incPath, nil +} diff --git a/cmd/replication/diff_test.go b/cmd/replication/diff_test.go new file mode 100644 index 0000000..916b8b0 --- /dev/null +++ b/cmd/replication/diff_test.go @@ -0,0 +1,269 @@ +package replication_test + +import ( + "github.com/stretchr/testify/assert" + "github.com/zrepl/zrepl/cmd/replication" + "github.com/zrepl/zrepl/zfs" + "strconv" + "strings" + "testing" + "time" +) + +func fsvlist(fsv ...string) (r []zfs.FilesystemVersion) { + + r = make([]zfs.FilesystemVersion, len(fsv)) + for i, f := range fsv { + + // parse the id from fsvlist. it is used to derivce Guid,CreateTXG and Creation attrs + split := strings.Split(f, ",") + if len(split) != 2 { + panic("invalid fsv spec") + } + id, err := strconv.Atoi(split[1]) + if err != nil { + panic(err) + } + + if strings.HasPrefix(f, "#") { + r[i] = zfs.FilesystemVersion{ + Name: strings.TrimPrefix(f, "#"), + Type: zfs.Bookmark, + Guid: uint64(id), + CreateTXG: uint64(id), + Creation: time.Unix(0, 0).Add(time.Duration(id) * time.Second), + } + } else if strings.HasPrefix(f, "@") { + r[i] = zfs.FilesystemVersion{ + Name: strings.TrimPrefix(f, "@"), + Type: zfs.Snapshot, + Guid: uint64(id), + CreateTXG: uint64(id), + Creation: time.Unix(0, 0).Add(time.Duration(id) * time.Second), + } + } else { + panic("invalid character") + } + } + return +} + +type incPathResult struct { + incPath []zfs.FilesystemVersion + conflict error +} + +type IncrementalPathTest struct { + Msg string + Receiver, Sender []zfs.FilesystemVersion + ExpectIncPath []zfs.FilesystemVersion + ExpectNoCommonAncestor bool + ExpectDiverged *replication.ConflictDiverged + ExpectPanic bool +} + +func (tt *IncrementalPathTest) Test(t *testing.T) { + + t.Logf("test: %s", tt.Msg) + + if tt.ExpectPanic { + assert.Panics(t, func() { + replication.IncrementalPath(tt.Receiver, tt.Sender) + }) + return + } + + incPath, conflict := replication.IncrementalPath(tt.Receiver, tt.Sender) + + if tt.ExpectIncPath != nil { + assert.Nil(t, conflict) + assert.True(t, len(incPath) == 0 || len(incPath) >= 2) + assert.Equal(t, tt.ExpectIncPath, incPath) + return + } + if conflict == nil { + t.Logf("conflict is (unexpectly) \nincPath: %#v", incPath) + } + if tt.ExpectNoCommonAncestor { + assert.IsType(t, &replication.ConflictNoCommonAncestor{}, conflict) + // TODO check sorting + return + } + if tt.ExpectDiverged != nil { + if !assert.IsType(t, &replication.ConflictDiverged{}, conflict) { + return + } + c := conflict.(*replication.ConflictDiverged) + // TODO check sorting + assert.NotZero(t, c.CommonAncestor) + assert.NotEmpty(t, c.ReceiverOnly) + assert.Equal(t, tt.ExpectDiverged.ReceiverOnly, c.ReceiverOnly) + assert.Equal(t, tt.ExpectDiverged.SenderOnly, c.SenderOnly) + return + } + +} + +func TestIncrementalPlan_IncrementalSnapshots(t *testing.T) { + l := fsvlist + + tbl := []IncrementalPathTest{ + { + Msg: "basic functionality", + Receiver: l("@a,1", "@b,2"), + Sender: l("@a,1", "@b,2", "@c,3", "@d,4"), + ExpectIncPath: l("@b,2", "@c,3", "@d,4"), + }, + { + Msg: "no snaps on receiver yields no common ancestor", + Receiver: l(), + Sender: l("@a,1"), + ExpectNoCommonAncestor: true, + }, + { + Msg: "no snapshots on sender yields empty incremental path", + Receiver: l(), + Sender: l(), + ExpectIncPath: l(), + }, + { + Msg: "nothing to do yields empty incremental path", + Receiver: l("@a,1"), + Sender: l("@a,1"), + ExpectIncPath: l(), + }, + { + Msg: "drifting apart", + Receiver: l("@a,1", "@b,2"), + Sender: l("@c,3", "@d,4"), + ExpectNoCommonAncestor: true, + }, + { + Msg: "different snapshots on sender and receiver", + Receiver: l("@a,1", "@c,2"), + Sender: l("@a,1", "@b,3"), + ExpectDiverged: &replication.ConflictDiverged{ + CommonAncestor: l("@a,1")[0], + SenderOnly: l("@b,3"), + ReceiverOnly: l("@c,2"), + }, + }, + { + Msg: "snapshot on receiver not present on sender", + Receiver: l("@a,1", "@b,2"), + Sender: l("@a,1"), + ExpectDiverged: &replication.ConflictDiverged{ + CommonAncestor: l("@a,1")[0], + SenderOnly: l(), + ReceiverOnly: l("@b,2"), + }, + }, + { + Msg: "gaps before most recent common ancestor do not matter", + Receiver: l("@a,1", "@b,2", "@c,3"), + Sender: l("@a,1", "@c,3", "@d,4"), + ExpectIncPath: l("@c,3", "@d,4"), + }, + } + + for _, test := range tbl { + test.Test(t) + } + +} + +func TestIncrementalPlan_BookmarksSupport(t *testing.T) { + l := fsvlist + + tbl := []IncrementalPathTest{ + { + Msg: "bookmarks are used", + Receiver: l("@a,1"), + Sender: l("#a,1", "@b,2"), + ExpectIncPath: l("#a,1", "@b,2"), + }, + { + Msg: "boomarks are stripped from incPath (cannot send incrementally)", + Receiver: l("@a,1"), + Sender: l("#a,1", "#b,2", "@c,3"), + ExpectIncPath: l("#a,1", "@c,3"), + }, + { + Msg: "bookmarks are preferred over snapshots for start of incPath", + Receiver: l("@a,1"), + Sender: l("#a,1", "@a,1", "@b,2"), + ExpectIncPath: l("#a,1", "@b,2"), + }, + { + Msg: "bookmarks are preferred over snapshots for start of incPath (regardless of order)", + Receiver: l("@a,1"), + Sender: l("@a,1", "#a,1", "@b,2"), + ExpectIncPath: l("#a,1", "@b,2"), + }, + } + + for _, test := range tbl { + test.Test(t) + } + +} + +func TestSortVersionListByCreateTXGThenBookmarkLTSnapshot(t *testing.T) { + + type Test struct { + Msg string + Input, Output []zfs.FilesystemVersion + } + + l := fsvlist + + tbl := []Test{ + { + "snapshot sorting already sorted", + l("@a,1", "@b,2"), + l("@a,1", "@b,2"), + }, + { + "bookmark sorting already sorted", + l("#a,1", "#b,2"), + l("#a,1", "#b,2"), + }, + { + "snapshot sorting", + l("@b,2", "@a,1"), + l("@a,1", "@b,2"), + }, + { + "bookmark sorting", + l("#b,2", "#a,1"), + l("#a,1", "#b,2"), + }, + } + + for _, test := range tbl { + t.Logf("test: %s", test.Msg) + inputlen := len(test.Input) + sorted := replication.SortVersionListByCreateTXGThenBookmarkLTSnapshot(test.Input) + if len(sorted) != inputlen { + t.Errorf("lenghts of input and output do not match: %d vs %d", inputlen, len(sorted)) + continue + } + if !assert.Equal(t, test.Output, sorted) { + continue + } + last := sorted[0] + for _, s := range sorted[1:] { + if s.CreateTXG < last.CreateTXG { + t.Errorf("must be sorted ascending, got:\n\t%#v", sorted) + break + } + if s.CreateTXG == last.CreateTXG { + if last.Type == zfs.Bookmark && s.Type != zfs.Snapshot { + t.Errorf("snapshots must come after bookmarks") + } + } + last = s + } + } + +} diff --git a/cmd/replication/replication.go b/cmd/replication/replication.go new file mode 100644 index 0000000..1759560 --- /dev/null +++ b/cmd/replication/replication.go @@ -0,0 +1,209 @@ +package replication + +import ( + "context" + "github.com/zrepl/zrepl/zfs" + "io" +) + +type ReplicationEndpoint interface { + // Does not include placeholder filesystems + ListFilesystems() ([]Filesystem, error) + ListFilesystemVersions(fs string) ([]zfs.FilesystemVersion, error) // fix depS + Sender + Receiver +} + +type Filesystem struct { + Path string + ResumeToken string +} + +type FilteredError struct{ fs string } + +func (f FilteredError) Error() string { return "endpoint does not allow access to filesystem " + f.fs } + +type SendRequest struct { + Filesystem string + From, To string + // If ResumeToken is not empty, the resume token that CAN be tried for 'zfs send' by the sender + // If it does not work, the sender SHOULD clear the resume token on their side + // and use From and To instead + // If ResumeToken is not empty, the GUIDs of From and To + // MUST correspond to those encoded in the ResumeToken. + // Otherwise, the Sender MUST return an error. + ResumeToken string + Compress bool + Dedup bool +} + +type SendResponse struct { + Properties zfs.ZFSProperties // fix dep + Stream io.Reader +} + +type ReceiveRequest struct { + Filesystem string + // The resume token used by the sending side. + // The receiver MUST discard the saved state on their side if ResumeToken + // does not match the zfs property of Filesystem on their side. + ResumeToken string +} + +type ReplicationMode int + +const ( + ReplicationModePull ReplicationMode = iota + ReplicationModePush +) + +type EndpointPair struct { + a, b ReplicationEndpoint + m ReplicationMode +} + +func NewEndpointPairPull(sender, receiver ReplicationEndpoint) EndpointPair { + return EndpointPair{sender, receiver, ReplicationModePull} +} + +func NewEndpointPairPush(sender, receiver ReplicationEndpoint) EndpointPair { + return EndpointPair{receiver, sender, ReplicationModePush} +} + +func (p EndpointPair) Sender() ReplicationEndpoint { + switch p.m { + case ReplicationModePull: + return p.a + case ReplicationModePush: + return p.b + } + panic("should not be reached") + return nil +} + +func (p EndpointPair) Receiver() ReplicationEndpoint { + switch p.m { + case ReplicationModePull: + return p.b + case ReplicationModePush: + return p.a + } + panic("should not be reached") + return nil +} + +func (p EndpointPair) Mode() ReplicationMode { + return p.m +} + +func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicator) { + + sfss, err := ep.Sender().ListFilesystems() + if err != nil { + // log error + return + } + + for _, fs := range sfss { + sfsvs, err := ep.Sender().ListFilesystemVersions(fs.Path) + rfsvs, err := ep.Receiver().ListFilesystemVersions(fs.Path) + if err != nil { + if _, ok := err.(FilteredError); ok { + // Remote does not map filesystem, don't try to tx it + continue + } + // log and ignore + continue + } + + path, conflict := IncrementalPath(rfsvs, sfsvs) + if conflict != nil { + // handle or ignore for now + continue + } + + ipr.Replicate(ctx, ep.Sender(), ep.Receiver(), NewCopier(), fs, path) + + } + +} + +type Sender interface { + Send(r SendRequest) (SendResponse, error) +} + +type Receiver interface { + Receive(r ReceiveRequest) (io.Writer, error) +} + +type Copier interface { + Copy(writer io.Writer, reader io.Reader) (int64, error) +} + +type copier struct{} + +func (copier) Copy(writer io.Writer, reader io.Reader) (int64, error) { + return io.Copy(writer, reader) +} + +func NewCopier() Copier { + return copier{} +} + +type IncrementalPathReplicator interface { + Replicate(ctx context.Context, sender Sender, receiver Receiver, copier Copier, fs Filesystem, path []zfs.FilesystemVersion) +} + +type incrementalPathReplicator struct{} + +func NewIncrementalPathReplicator() IncrementalPathReplicator { + return incrementalPathReplicator{} +} + +func (incrementalPathReplicator) Replicate(ctx context.Context, sender Sender, receiver Receiver, copier Copier, fs Filesystem, path []zfs.FilesystemVersion) { + + if len(path) == 0 { + // nothing to do + return + } + + usedResumeToken := false + +incrementalLoop: + for j := 0; j < len(path)-1; j++ { + rt := "" + if !usedResumeToken { + rt = fs.ResumeToken + usedResumeToken = true + } + sr := SendRequest{ + Filesystem: fs.Path, + From: path[j].String(), + To: path[j+1].String(), + ResumeToken: rt, + } + sres, err := sender.Send(sr) + if err != nil { + // handle and ignore + break incrementalLoop + } + // try to consume stream + + rr := ReceiveRequest{ + Filesystem: fs.Path, + ResumeToken: rt, + } + recvWriter, err := receiver.Receive(rr) + if err != nil { + // handle and ignore + break incrementalLoop + } + _, err = copier.Copy(recvWriter, sres.Stream) + if err != nil { + // handle and ignore + break incrementalLoop + } + + // handle properties from sres + } +} diff --git a/cmd/replication/replication_test.go b/cmd/replication/replication_test.go new file mode 100644 index 0000000..424c554 --- /dev/null +++ b/cmd/replication/replication_test.go @@ -0,0 +1,152 @@ +package replication_test + +import ( + "context" + "github.com/stretchr/testify/assert" + "github.com/zrepl/zrepl/cmd/replication" + "github.com/zrepl/zrepl/zfs" + "io" + "testing" +) + +type IncrementalPathSequenceStep struct { + SendRequest replication.SendRequest + SendResponse replication.SendResponse + SendError error + ReceiveRequest replication.ReceiveRequest + ReceiveWriter io.Writer + ReceiveError error +} + +type MockIncrementalPathRecorder struct { + T *testing.T + Sequence []IncrementalPathSequenceStep + Pos int +} + +func (m *MockIncrementalPathRecorder) Receive(r replication.ReceiveRequest) (io.Writer, error) { + if m.Pos >= len(m.Sequence) { + m.T.Fatal("unexpected Receive") + } + i := m.Sequence[m.Pos] + m.Pos++ + if !assert.Equal(m.T, i.ReceiveRequest, r) { + m.T.FailNow() + } + return i.ReceiveWriter, i.ReceiveError +} + +func (m *MockIncrementalPathRecorder) Send(r replication.SendRequest) (replication.SendResponse, error) { + if m.Pos >= len(m.Sequence) { + m.T.Fatal("unexpected Send") + } + i := m.Sequence[m.Pos] + m.Pos++ + if !assert.Equal(m.T, i.SendRequest, r) { + m.T.FailNow() + } + return i.SendResponse, i.SendError +} + +func (m *MockIncrementalPathRecorder) Finished() bool { + return m.Pos == len(m.Sequence) +} + +type DiscardCopier struct{} + +func (DiscardCopier) Copy(writer io.Writer, reader io.Reader) (int64, error) { + return 0, nil +} + +type IncrementalPathReplicatorTest struct { + Msg string + Filesystem replication.Filesystem + Path []zfs.FilesystemVersion + Steps []IncrementalPathSequenceStep +} + +func (test *IncrementalPathReplicatorTest) Test(t *testing.T) { + + t.Log(test.Msg) + + rec := &MockIncrementalPathRecorder{ + T: t, + Sequence: test.Steps, + } + + ipr := replication.NewIncrementalPathReplicator() + ipr.Replicate( + context.TODO(), + rec, + rec, + DiscardCopier{}, + test.Filesystem, + test.Path, + ) + + assert.True(t, rec.Finished()) + +} + +func TestIncrementalPathReplicator_Replicate(t *testing.T) { + + tbl := []IncrementalPathReplicatorTest{ + { + Msg: "generic happy place with resume token", + Filesystem: replication.Filesystem{ + Path: "foo/bar", + ResumeToken: "blafoo", + }, + Path: fsvlist("@a,1", "@b,2", "@c,3"), + Steps: []IncrementalPathSequenceStep{ + { + SendRequest: replication.SendRequest{ + Filesystem: "foo/bar", + From: "@a,1", + To: "@b,2", + ResumeToken: "blafoo", + }, + }, + { + ReceiveRequest: replication.ReceiveRequest{ + Filesystem: "foo/bar", + ResumeToken: "blafoo", + }, + }, + { + SendRequest: replication.SendRequest{ + Filesystem: "foo/bar", + From: "@b,2", + To: "@c,3", + }, + }, + { + ReceiveRequest: replication.ReceiveRequest{ + Filesystem: "foo/bar", + }, + }, + }, + }, + { + Msg: "no action on empty sequence", + Filesystem: replication.Filesystem{ + Path: "foo/bar", + }, + Path: fsvlist(), + Steps: []IncrementalPathSequenceStep{}, + }, + { + Msg: "no action on invalid path", + Filesystem: replication.Filesystem{ + Path: "foo/bar", + }, + Path: fsvlist("@justone,1"), + Steps: []IncrementalPathSequenceStep{}, + }, + } + + for _, test := range tbl { + test.Test(t) + } + +} From fa6426f803b8a4f8559519db82596613d36e8575 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 2 May 2018 21:26:56 +0200 Subject: [PATCH 002/167] WIP: zfs: hacky resume token parsing --- zfs/resume_token.go | 105 +++++++++++++++++++++++++++++++++++++++ zfs/resume_token_test.go | 64 ++++++++++++++++++++++++ 2 files changed, 169 insertions(+) create mode 100644 zfs/resume_token.go create mode 100644 zfs/resume_token_test.go diff --git a/zfs/resume_token.go b/zfs/resume_token.go new file mode 100644 index 0000000..4423626 --- /dev/null +++ b/zfs/resume_token.go @@ -0,0 +1,105 @@ +package zfs + +import ( + "context" + "errors" + "os/exec" + "regexp" + "strconv" + "time" +) + +type ResumeToken struct { + HasFromGUID, HasToGUID bool + FromGUID, ToGUID uint64 + // no support for other fields +} + +var resumeTokenNVListRE = regexp.MustCompile(`\t(\S+) = (.*)`) +var resumeTokenContentsRE = regexp.MustCompile(`resume token contents:\nnvlist version: 0`) +var resumeTokenIsCorruptRE = regexp.MustCompile(`resume token is corrupt`) + +var ResumeTokenCorruptError = errors.New("resume token is corrupt") +var ResumeTokenDecodingNotSupported = errors.New("zfs binary does not allow decoding resume token or zrepl cannot scrape zfs output") +var ResumeTokenParsingError = errors.New("zrepl cannot parse resume token values") + +// Abuse 'zfs send' to decode the resume token +// +// FIXME: implement nvlist unpacking in Go and read through libzfs_sendrecv.c +func ParseResumeToken(ctx context.Context, token string) (*ResumeToken, error) { + + // Example resume tokens: + // + // From a non-incremental send + // 1-bf31b879a-b8-789c636064000310a500c4ec50360710e72765a5269740f80cd8e4d3d28a534b18e00024cf86249f5459925acc802a8facbf243fbd3433858161f5ddb9ab1ae7c7466a20c97382e5f312735319180af2f3730cf58166953824c2cc0200cde81651 + + // From an incremental send + // 1-c49b979a2-e0-789c636064000310a501c49c50360710a715e5e7a69766a63040c1eabb735735ce8f8d5400b2d991d4e52765a5269740f82080219f96569c5ac2000720793624f9a4ca92d46206547964fd25f91057f09e37babb88c9bf5503499e132c9f97989bcac050909f9f63a80f34abc421096616007c881d4c + + // Resulting output of zfs send -nvt + // + //resume token contents: + //nvlist version: 0 + // fromguid = 0x595d9f81aa9dddab + // object = 0x1 + // offset = 0x0 + // bytes = 0x0 + // toguid = 0x854f02a2dd32cf0d + // toname = pool1/test@b + //cannot resume send: 'pool1/test@b' used in the initial send no longer exists + + ctx, _ = context.WithTimeout(ctx, 500*time.Millisecond) + cmd := exec.CommandContext(ctx, ZFS_BINARY, "send", "-nvt", string(token)) + output, err := cmd.CombinedOutput() + if err != nil { + if exitErr, ok := err.(*exec.ExitError); ok { + if !exitErr.Exited() { + return nil, err + } + // we abuse zfs send for decoding, the exit error may be due to + // a) the token being from a third machine + // b) it no longer exists on the machine where + } else { + return nil, err + } + } + + if !resumeTokenContentsRE.Match(output) { + if resumeTokenIsCorruptRE.Match(output) { + return nil, ResumeTokenCorruptError + } + return nil, ResumeTokenDecodingNotSupported + } + + matches := resumeTokenNVListRE.FindAllStringSubmatch(string(output), -1) + if matches == nil { + return nil, ResumeTokenDecodingNotSupported + } + + rt := &ResumeToken{} + + for _, m := range matches { + attr, val := m[1], m[2] + switch attr { + case "fromguid": + rt.FromGUID, err = strconv.ParseUint(val, 0, 64) + if err != nil { + return nil, ResumeTokenParsingError + } + rt.HasFromGUID = true + case "toguid": + rt.ToGUID, err = strconv.ParseUint(val, 0, 64) + if err != nil { + return nil, ResumeTokenParsingError + } + rt.HasToGUID = true + } + } + + if !rt.HasToGUID { + return nil, ResumeTokenDecodingNotSupported + } + + return rt, nil + +} diff --git a/zfs/resume_token_test.go b/zfs/resume_token_test.go new file mode 100644 index 0000000..33f560a --- /dev/null +++ b/zfs/resume_token_test.go @@ -0,0 +1,64 @@ +package zfs_test + +import ( + "context" + "github.com/stretchr/testify/assert" + "github.com/zrepl/zrepl/zfs" + "testing" +) + +type ResumeTokenTest struct { + Msg string + Token string + ExpectToken *zfs.ResumeToken + ExpectError error +} + +func (rtt *ResumeTokenTest) Test(t *testing.T) { + t.Log(rtt.Msg) + res, err := zfs.ParseResumeToken(context.TODO(), rtt.Token) + + if rtt.ExpectError != nil { + assert.EqualValues(t, rtt.ExpectError, err) + return + } + if rtt.ExpectToken != nil { + assert.Nil(t, err) + assert.EqualValues(t, rtt.ExpectToken, res) + return + } +} + +func TestParseResumeToken(t *testing.T) { + + tbl := []ResumeTokenTest{ + { + Msg: "normal send (non-incremental)", + Token: `1-bf31b879a-b8-789c636064000310a500c4ec50360710e72765a5269740f80cd8e4d3d28a534b18e00024cf86249f5459925acc802a8facbf243fbd3433858161f5ddb9ab1ae7c7466a20c97382e5f312735319180af2f3730cf58166953824c2cc0200cde81651`, + ExpectToken: &zfs.ResumeToken{ + HasToGUID: true, + ToGUID: 0x595d9f81aa9dddab, + }, + }, + { + Msg: "normal send (incremental)", + Token: `1-c49b979a2-e0-789c636064000310a501c49c50360710a715e5e7a69766a63040c1eabb735735ce8f8d5400b2d991d4e52765a5269740f82080219f96569c5ac2000720793624f9a4ca92d46206547964fd25f91057f09e37babb88c9bf5503499e132c9f97989bcac050909f9f63a80f34abc421096616007c881d4c`, + ExpectToken: &zfs.ResumeToken{ + HasToGUID: true, + ToGUID: 0x854f02a2dd32cf0d, + HasFromGUID: true, + FromGUID: 0x595d9f81aa9dddab, + }, + }, + { + Msg: "corrupted token", + Token: `1-bf31b879a-b8-789c636064000310a500c4ec50360710e72765a5269740f80cd8e4d3d28a534b18e00024cf86249f5459925acc802a8facbf243fbd3433858161f5ddb9ab1ae7c7466a20c97382e5f312735319180af2f3730cf58166953824c2cc0200cd12345`, + ExpectError: zfs.ResumeTokenCorruptError, + }, + } + + for _, test := range tbl { + test.Test(t) + } + +} From 8cca0a8547fa81956253325bb02d70f8861a7d19 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 20 Jun 2018 20:20:37 +0200 Subject: [PATCH 003/167] Initial working version Summary: * Logging is still bad * test output in a lot of placed * FIXMEs every where Test Plan: None, just review Differential Revision: https://phabricator.cschwarz.com/D2 --- Makefile | 3 + cmd/config_fsvfilter.go | 6 +- cmd/config_job_local.go | 22 +- cmd/config_job_pull.go | 34 +- cmd/config_job_source.go | 23 +- cmd/config_mapfilter.go | 47 +- cmd/config_test.go | 59 +++ cmd/handler.go | 181 -------- cmd/replication.go | 658 ++++++++++++++++------------ cmd/replication/diff.go | 27 +- cmd/replication/diff_test.go | 27 +- cmd/replication/pdu.pb.go | 384 ++++++++++++++++ cmd/replication/pdu.proto | 78 ++++ cmd/replication/pdu_extras.go | 60 +++ cmd/replication/pdu_test.go | 64 +++ cmd/replication/replication.go | 187 +++++--- cmd/replication/replication_test.go | 79 ++-- rpc/client.go | 136 ------ rpc/datatype_string.go | 17 - rpc/frame_layer.go | 302 ------------- rpc/frametype_string.go | 26 -- rpc/local.go | 63 --- rpc/server.go | 259 ----------- rpc/shared.go | 111 ----- rpc/status_string.go | 17 - util/iocommand.go | 4 +- zfs/diff.go | 14 + zfs/mapping.go | 47 +- zfs/resume_token.go | 15 + zfs/versions.go | 57 ++- zfs/zfs.go | 115 ++++- 31 files changed, 1536 insertions(+), 1586 deletions(-) delete mode 100644 cmd/handler.go create mode 100644 cmd/replication/pdu.pb.go create mode 100644 cmd/replication/pdu.proto create mode 100644 cmd/replication/pdu_extras.go create mode 100644 cmd/replication/pdu_test.go delete mode 100644 rpc/client.go delete mode 100644 rpc/datatype_string.go delete mode 100644 rpc/frame_layer.go delete mode 100644 rpc/frametype_string.go delete mode 100644 rpc/local.go delete mode 100644 rpc/server.go delete mode 100644 rpc/shared.go delete mode 100644 rpc/status_string.go diff --git a/Makefile b/Makefile index 6d26c7d..3970783 100644 --- a/Makefile +++ b/Makefile @@ -29,6 +29,9 @@ generate: #not part of the build, must do that manually @for pkg in $(_TESTPKGS); do\ go generate "$$pkg" || exit 1; \ done; + protoc -I=cmd/replication --go_out=cmd/replication cmd/replication/pdu.proto + # FIXME fix docker build! + build: @echo "INFO: In case of missing dependencies, run 'make vendordeps'" diff --git a/cmd/config_fsvfilter.go b/cmd/config_fsvfilter.go index 8e13dd5..3391780 100644 --- a/cmd/config_fsvfilter.go +++ b/cmd/config_fsvfilter.go @@ -29,8 +29,8 @@ func parseSnapshotPrefix(i string) (p string, err error) { return } -func (f *PrefixFilter) Filter(fsv zfs.FilesystemVersion) (accept bool, err error) { - fstypeMatches := (!f.fstypeSet || fsv.Type == f.fstype) - prefixMatches := strings.HasPrefix(fsv.Name, f.prefix) +func (f *PrefixFilter) Filter(t zfs.VersionType, name string) (accept bool, err error) { + fstypeMatches := (!f.fstypeSet || t == f.fstype) + prefixMatches := strings.HasPrefix(name, f.prefix) return fstypeMatches && prefixMatches, nil } diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index 7bbe517..82b72fc 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -6,9 +6,9 @@ import ( "context" "github.com/mitchellh/mapstructure" "github.com/pkg/errors" - "github.com/zrepl/zrepl/rpc" "github.com/zrepl/zrepl/zfs" "sync" + "github.com/zrepl/zrepl/cmd/replication" ) type LocalJob struct { @@ -96,15 +96,19 @@ func (j *LocalJob) JobStart(ctx context.Context) { j.pruneRHSTask = NewTask("prune_rhs", j, rootLog) j.pruneLHSTask = NewTask("prune_lhs", j, rootLog) - local := rpc.NewLocalRPC() // Allow access to any dataset since we control what mapping // is passed to the pull routine. // All local datasets will be passed to its Map() function, // but only those for which a mapping exists will actually be pulled. // We can pay this small performance penalty for now. - handler := NewHandler(j.handlerTask.Log(), localPullACL{}, NewPrefixFilter(j.SnapshotPrefix)) + wildcardMapFilter := NewDatasetMapFilter(1, false) + wildcardMapFilter.Add("<", "<") + sender := &SenderEndpoint{wildcardMapFilter, NewPrefixFilter(j.SnapshotPrefix)} - registerEndpoints(local, handler) + receiver, err := NewReceiverEndpoint(j.Mapping, NewPrefixFilter(j.SnapshotPrefix)) + if err != nil { + rootLog.WithError(err).Error("unexpected error setting up local handler") + } snapper := IntervalAutosnap{ task: j.snapperTask, @@ -141,8 +145,14 @@ outer: j.mainTask.Log().Debug("replicating from lhs to rhs") j.mainTask.Enter("replicate") - puller := Puller{j.mainTask, local, j.Mapping, j.InitialReplPolicy} - puller.Pull() + + + replication.Replicate( + ctx, + replication.NewEndpointPairPull(sender, receiver), + replication.NewIncrementalPathReplicator(), + ) + j.mainTask.Finish() // use a ctx as soon as Pull gains ctx support diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index c3e2413..9d78e7a 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -7,8 +7,9 @@ import ( "fmt" "github.com/mitchellh/mapstructure" "github.com/pkg/errors" - "github.com/zrepl/zrepl/rpc" "github.com/zrepl/zrepl/util" + "github.com/zrepl/zrepl/cmd/replication" + "github.com/problame/go-streamrpc" ) type PullJob struct { @@ -116,6 +117,13 @@ func (j *PullJob) JobStart(ctx context.Context) { } } +var STREAMRPC_CONFIG = &streamrpc.ConnConfig{ // FIXME oversight and configurability + RxHeaderMaxLen: 4096, + RxStructuredMaxLen: 4096 * 4096, + RxStreamMaxChunkSize: 4096 * 4096, + TxChunkSize: 4096 * 4096, +} + func (j *PullJob) doRun(ctx context.Context) { j.task.Enter("run") @@ -133,14 +141,28 @@ func (j *PullJob) doRun(ctx context.Context) { return } - client := rpc.NewClient(rwc) + + client := RemoteEndpoint{streamrpc.NewClientOnConn(rwc, STREAMRPC_CONFIG)} if j.Debug.RPC.Log { - client.SetLogger(j.task.Log(), true) + // FIXME implement support + // client.SetLogger(j.task.Log(), true) } j.task.Enter("pull") - puller := Puller{j.task, client, j.Mapping, j.InitialReplPolicy} - puller.Pull() + + puller, err := NewReceiverEndpoint( + j.Mapping, + NewPrefixFilter(j.SnapshotPrefix), + ) + if err != nil { + j.task.Log().WithError(err).Error("error creating receiver endpoint") + j.task.Finish() + return + } + + replicator := replication.NewIncrementalPathReplicator() + replication.Replicate(context.WithValue(ctx, replication.ContextKeyLog, j.task.Log()), replication.NewEndpointPairPull(client, puller), replicator) + closeRPCWithTimeout(j.task, client, time.Second*1, "") rwc.Close() j.task.Finish() @@ -172,7 +194,7 @@ func (j *PullJob) Pruner(task *Task, side PrunePolicySide, dryRun bool) (p Prune return } -func closeRPCWithTimeout(task *Task, remote rpc.RPCClient, timeout time.Duration, goodbye string) { +func closeRPCWithTimeout(task *Task, remote RemoteEndpoint, timeout time.Duration, goodbye string) { task.Log().Info("closing rpc connection") diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index 79b8559..4ebb385 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -5,10 +5,10 @@ import ( "io" "time" - mapstructure "github.com/mitchellh/mapstructure" + "github.com/mitchellh/mapstructure" "github.com/pkg/errors" - "github.com/zrepl/zrepl/rpc" "github.com/zrepl/zrepl/util" + "github.com/problame/go-streamrpc" ) type SourceJob struct { @@ -206,17 +206,16 @@ func (j *SourceJob) handleConnection(rwc io.ReadWriteCloser, task *Task) { panic(err) } - // construct connection handler - handler := NewHandler(task.Log(), j.Filesystems, NewPrefixFilter(j.SnapshotPrefix)) + senderEP := NewSenderEndpoint(j.Filesystems, NewPrefixFilter(j.SnapshotPrefix)) - // handle connection - rpcServer := rpc.NewServer(rwc) - if j.Debug.RPC.Log { - rpclog := task.Log().WithField("subsystem", "rpc") - rpcServer.SetLogger(rpclog, true) - } - registerEndpoints(rpcServer, handler) - if err = rpcServer.Serve(); err != nil { + handler := HandlerAdaptor{senderEP} + // FIXME logging support or erase config + //if j.Debug.RPC.Log { + // rpclog := task.Log().WithField("subsystem", "rpc") + // rpcServer.SetLogger(rpclog, true) + //} + + if err := streamrpc.ServeConn(rwc, STREAMRPC_CONFIG, handler.Handle); err != nil { task.Log().WithError(err).Error("error serving connection") } diff --git a/cmd/config_mapfilter.go b/cmd/config_mapfilter.go index c625fc8..04d7076 100644 --- a/cmd/config_mapfilter.go +++ b/cmd/config_mapfilter.go @@ -101,6 +101,7 @@ func (m DatasetMapFilter) mostSpecificPrefixMapping(path *zfs.DatasetPath) (idx return } +// Returns target == nil if there is no mapping func (m DatasetMapFilter) Map(source *zfs.DatasetPath) (target *zfs.DatasetPath, err error) { if m.filterMode { @@ -114,9 +115,17 @@ func (m DatasetMapFilter) Map(source *zfs.DatasetPath) (target *zfs.DatasetPath, } me := m.entries[mi] - if strings.HasPrefix("!", me.mapping) { - // reject mapping - return nil, nil + if me.mapping == "" { + // Special case treatment: 'foo/bar<' => '' + if !me.subtreeMatch { + return nil, fmt.Errorf("mapping to '' must be a subtree match") + } + // ok... + } else { + if strings.HasPrefix("!", me.mapping) { + // reject mapping + return nil, nil + } } target, err = zfs.NewDatasetPath(me.mapping) @@ -177,6 +186,38 @@ func (m DatasetMapFilter) InvertedFilter() (inv *DatasetMapFilter, err error) { return inv, nil } +// FIXME investigate whether we can support more... +func (m DatasetMapFilter) Invert() (inv *DatasetMapFilter, err error) { + + if m.filterMode { + err = errors.Errorf("can only invert mappings") + return + } + + if len(m.entries) != 1 { + return nil, errors.Errorf("inversion of complicated mappings is not implemented") // FIXME + } + + e := m.entries[0] + + inv = &DatasetMapFilter{ + make([]datasetMapFilterEntry, len(m.entries)), + false, + } + mp, err := zfs.NewDatasetPath(e.mapping) + if err != nil { + return nil, err + } + + inv.entries[0] = datasetMapFilterEntry{ + path: mp, + mapping: e.path.ToString(), + subtreeMatch: e.subtreeMatch, + } + + return inv, nil +} + // Creates a new DatasetMapFilter in filter mode from a mapping // All accepting mapping results are mapped to accepting filter results // All rejecting mapping results are mapped to rejecting filter results diff --git a/cmd/config_test.go b/cmd/config_test.go index 5f666a1..5bfcb2b 100644 --- a/cmd/config_test.go +++ b/cmd/config_test.go @@ -135,6 +135,25 @@ func TestDatasetMapFilter(t *testing.T) { expectMapping(map1, "b", "") expectMapping(map1, "q/r", "root4/1/2/r") + map2 := map[string]string{ // identity mapping + "<":"", + } + expectMapping(map2, "foo/bar", "foo/bar") + + map3 := map[string]string{ // subtree to local mapping, need that for Invert() + "foo/bar<": "", + } + { + m, _ := parseDatasetMapFilter(map3, false) + p, _ := zfs.NewDatasetPath("foo/bar") + tp, err := m.Map(p) + assert.Nil(t, err) + assert.True(t, tp.Empty()) + + expectMapping(map3, "foo/bar/x", "x") + expectMapping(map3, "x", "") + } + filter1 := map[string]string{ "<": "!", "a<": "ok", @@ -220,3 +239,43 @@ func TestDatasetMapFilter_InvertedFilter(t *testing.T) { expectMapping(inv, "1/2/a/b", true) } + +func TestDatasetMapFilter_Invert(t *testing.T) { + + mapspec := map[string]string{ + "<": "foo/bar", + } + + m, err := parseDatasetMapFilter(mapspec, false) + assert.NoError(t, err) + + inv, err := m.Invert() + assert.NoError(t, err) + + expectMapping := func(m *DatasetMapFilter, input, expect string, expErr bool, expEmpty bool) { + p, err := zfs.NewDatasetPath(input) + assert.Nil(t, err) + r, err := m.Map(p) + if expErr { + assert.Nil(t, r) + assert.Error(t, err) + return + } + if expEmpty { + assert.Nil(t, err) + assert.True(t, r.Empty()) + } else if expect == "" { + assert.Nil(t, r) + assert.Nil(t, err) + } else { + assert.Nil(t, err) + assert.NotNil(t, r) + assert.Equal(t, expect, r.ToString()) + } + } + + expectMapping(inv, "x", "", false, false) + expectMapping(inv, "foo/bar", "", false, true) + expectMapping(inv, "foo/bar/bee", "bee", false, false) + +} \ No newline at end of file diff --git a/cmd/handler.go b/cmd/handler.go deleted file mode 100644 index 5e9a6bb..0000000 --- a/cmd/handler.go +++ /dev/null @@ -1,181 +0,0 @@ -package cmd - -import ( - "fmt" - "io" - - "github.com/pkg/errors" - "github.com/zrepl/zrepl/rpc" - "github.com/zrepl/zrepl/zfs" -) - -type DatasetMapping interface { - Map(source *zfs.DatasetPath) (target *zfs.DatasetPath, err error) -} - -type FilesystemRequest struct { - Roots []string // may be nil, indicating interest in all filesystems -} - -type FilesystemVersionsRequest struct { - Filesystem *zfs.DatasetPath -} - -type InitialTransferRequest struct { - Filesystem *zfs.DatasetPath - FilesystemVersion zfs.FilesystemVersion -} - -type IncrementalTransferRequest struct { - Filesystem *zfs.DatasetPath - From zfs.FilesystemVersion - To zfs.FilesystemVersion -} - -type Handler struct { - logger Logger - dsf zfs.DatasetFilter - fsvf zfs.FilesystemVersionFilter -} - -func NewHandler(logger Logger, dsfilter zfs.DatasetFilter, snapfilter zfs.FilesystemVersionFilter) (h Handler) { - return Handler{logger, dsfilter, snapfilter} -} - -func registerEndpoints(server rpc.RPCServer, handler Handler) (err error) { - err = server.RegisterEndpoint("FilesystemRequest", handler.HandleFilesystemRequest) - if err != nil { - panic(err) - } - err = server.RegisterEndpoint("FilesystemVersionsRequest", handler.HandleFilesystemVersionsRequest) - if err != nil { - panic(err) - } - err = server.RegisterEndpoint("InitialTransferRequest", handler.HandleInitialTransferRequest) - if err != nil { - panic(err) - } - err = server.RegisterEndpoint("IncrementalTransferRequest", handler.HandleIncrementalTransferRequest) - if err != nil { - panic(err) - } - return nil -} - -func (h Handler) HandleFilesystemRequest(r *FilesystemRequest, roots *[]*zfs.DatasetPath) (err error) { - - log := h.logger.WithField("endpoint", "FilesystemRequest") - - log.WithField("request", r).Debug("request") - log.WithField("dataset_filter", h.dsf).Debug("dsf") - - allowed, err := zfs.ZFSListMapping(h.dsf) - if err != nil { - log.WithError(err).Error("error listing filesystems") - return - } - - log.WithField("response", allowed).Debug("response") - *roots = allowed - return -} - -func (h Handler) HandleFilesystemVersionsRequest(r *FilesystemVersionsRequest, versions *[]zfs.FilesystemVersion) (err error) { - - log := h.logger.WithField("endpoint", "FilesystemVersionsRequest") - - log.WithField("request", r).Debug("request") - - // allowed to request that? - if h.pullACLCheck(r.Filesystem, nil); err != nil { - log.WithError(err).Warn("pull ACL check failed") - return - } - - // find our versions - vs, err := zfs.ZFSListFilesystemVersions(r.Filesystem, h.fsvf) - if err != nil { - log.WithError(err).Error("cannot list filesystem versions") - return - } - - log.WithField("response", vs).Debug("response") - - *versions = vs - return - -} - -func (h Handler) HandleInitialTransferRequest(r *InitialTransferRequest, stream *io.Reader) (err error) { - - log := h.logger.WithField("endpoint", "InitialTransferRequest") - - log.WithField("request", r).Debug("request") - if err = h.pullACLCheck(r.Filesystem, &r.FilesystemVersion); err != nil { - log.WithError(err).Warn("pull ACL check failed") - return - } - - log.Debug("invoking zfs send") - - s, err := zfs.ZFSSend(r.Filesystem, &r.FilesystemVersion, nil) - if err != nil { - log.WithError(err).Error("cannot send filesystem") - } - *stream = s - - return - -} - -func (h Handler) HandleIncrementalTransferRequest(r *IncrementalTransferRequest, stream *io.Reader) (err error) { - - log := h.logger.WithField("endpoint", "IncrementalTransferRequest") - log.WithField("request", r).Debug("request") - if err = h.pullACLCheck(r.Filesystem, &r.From); err != nil { - log.WithError(err).Warn("pull ACL check failed") - return - } - if err = h.pullACLCheck(r.Filesystem, &r.To); err != nil { - log.WithError(err).Warn("pull ACL check failed") - return - } - - log.Debug("invoking zfs send") - - s, err := zfs.ZFSSend(r.Filesystem, &r.From, &r.To) - if err != nil { - log.WithError(err).Error("cannot send filesystem") - } - - *stream = s - return - -} - -func (h Handler) pullACLCheck(p *zfs.DatasetPath, v *zfs.FilesystemVersion) (err error) { - var fsAllowed, vAllowed bool - fsAllowed, err = h.dsf.Filter(p) - if err != nil { - err = fmt.Errorf("error evaluating ACL: %s", err) - return - } - if !fsAllowed { - err = fmt.Errorf("ACL prohibits access to %s", p.ToString()) - return - } - if v == nil { - return - } - - vAllowed, err = h.fsvf.Filter(*v) - if err != nil { - err = errors.Wrap(err, "error evaluating version filter") - return - } - if !vAllowed { - err = fmt.Errorf("ACL prohibits access to %s", v.ToAbsPath(p)) - return - } - return -} diff --git a/cmd/replication.go b/cmd/replication.go index 2cb5626..ee0e47a 100644 --- a/cmd/replication.go +++ b/cmd/replication.go @@ -2,22 +2,16 @@ package cmd import ( "fmt" - "io" - - "bytes" - "encoding/json" - "github.com/zrepl/zrepl/rpc" + "github.com/zrepl/zrepl/cmd/replication" + "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/zfs" + "io" + "github.com/pkg/errors" + "github.com/golang/protobuf/proto" + "bytes" + "os" ) -type localPullACL struct{} - -func (a localPullACL) Filter(p *zfs.DatasetPath) (pass bool, err error) { - return true, nil -} - -const DEFAULT_INITIAL_REPL_POLICY = InitialReplPolicyMostRecent - type InitialReplPolicy string const ( @@ -25,299 +19,383 @@ const ( InitialReplPolicyAll InitialReplPolicy = "all" ) -type Puller struct { - task *Task - Remote rpc.RPCClient - Mapping DatasetMapping - InitialReplPolicy InitialReplPolicy +const DEFAULT_INITIAL_REPL_POLICY = InitialReplPolicyMostRecent + +// SenderEndpoint implements replication.ReplicationEndpoint for a sending side +type SenderEndpoint struct { + FSFilter zfs.DatasetFilter + FilesystemVersionFilter zfs.FilesystemVersionFilter } -type remoteLocalMapping struct { - Remote *zfs.DatasetPath - Local *zfs.DatasetPath +func NewSenderEndpoint(fsf zfs.DatasetFilter, fsvf zfs.FilesystemVersionFilter) *SenderEndpoint { + return &SenderEndpoint{fsf, fsvf} } -func (p *Puller) getRemoteFilesystems() (rfs []*zfs.DatasetPath, ok bool) { - p.task.Enter("fetch_remote_fs_list") - defer p.task.Finish() - - fsr := FilesystemRequest{} - if err := p.Remote.Call("FilesystemRequest", &fsr, &rfs); err != nil { - p.task.Log().WithError(err).Error("cannot fetch remote filesystem list") - return nil, false - } - return rfs, true -} - -func (p *Puller) buildReplMapping(remoteFilesystems []*zfs.DatasetPath) (replMapping map[string]remoteLocalMapping, ok bool) { - p.task.Enter("build_repl_mapping") - defer p.task.Finish() - - replMapping = make(map[string]remoteLocalMapping, len(remoteFilesystems)) - for fs := range remoteFilesystems { - var err error - var localFs *zfs.DatasetPath - localFs, err = p.Mapping.Map(remoteFilesystems[fs]) - if err != nil { - err := fmt.Errorf("error mapping %s: %s", remoteFilesystems[fs], err) - p.task.Log().WithError(err).WithField(logMapFromField, remoteFilesystems[fs]).Error("cannot map") - return nil, false - } - if localFs == nil { - continue - } - p.task.Log().WithField(logMapFromField, remoteFilesystems[fs].ToString()). - WithField(logMapToField, localFs.ToString()).Debug("mapping") - m := remoteLocalMapping{remoteFilesystems[fs], localFs} - replMapping[m.Local.ToString()] = m - } - return replMapping, true -} - -// returns true if the receiving filesystem (local side) exists and can have child filesystems -func (p *Puller) replFilesystem(m remoteLocalMapping, localFilesystemState map[string]zfs.FilesystemState) (localExists bool) { - - p.task.Enter("repl_fs") - defer p.task.Finish() - var err error - remote := p.Remote - - log := p.task.Log(). - WithField(logMapFromField, m.Remote.ToString()). - WithField(logMapToField, m.Local.ToString()) - - log.Debug("examining local filesystem state") - localState, localExists := localFilesystemState[m.Local.ToString()] - var versions []zfs.FilesystemVersion - switch { - case !localExists: - log.Info("local filesystem does not exist") - case localState.Placeholder: - log.Info("local filesystem is marked as placeholder") - default: - log.Debug("local filesystem exists") - log.Debug("requesting local filesystem versions") - if versions, err = zfs.ZFSListFilesystemVersions(m.Local, nil); err != nil { - log.WithError(err).Error("cannot get local filesystem versions") - return false - } - } - - log.Info("requesting remote filesystem versions") - r := FilesystemVersionsRequest{ - Filesystem: m.Remote, - } - var theirVersions []zfs.FilesystemVersion - if err = remote.Call("FilesystemVersionsRequest", &r, &theirVersions); err != nil { - log.WithError(err).Error("cannot get remote filesystem versions") - log.Warn("stopping replication for all filesystems mapped as children of receiving filesystem") - return false - } - - log.Debug("computing diff between remote and local filesystem versions") - diff := zfs.MakeFilesystemDiff(versions, theirVersions) - log.WithField("diff", diff).Debug("diff between local and remote filesystem") - - if localState.Placeholder && diff.Conflict != zfs.ConflictAllRight { - panic("internal inconsistency: local placeholder implies ConflictAllRight") - } - - switch diff.Conflict { - case zfs.ConflictAllRight: - - log.WithField("replication_policy", p.InitialReplPolicy).Info("performing initial sync, following policy") - - if p.InitialReplPolicy != InitialReplPolicyMostRecent { - panic(fmt.Sprintf("policy '%s' not implemented", p.InitialReplPolicy)) - } - - snapsOnly := make([]zfs.FilesystemVersion, 0, len(diff.MRCAPathRight)) - for s := range diff.MRCAPathRight { - if diff.MRCAPathRight[s].Type == zfs.Snapshot { - snapsOnly = append(snapsOnly, diff.MRCAPathRight[s]) - } - } - - if len(snapsOnly) < 1 { - log.Warn("cannot perform initial sync: no remote snapshots") - return false - } - - r := InitialTransferRequest{ - Filesystem: m.Remote, - FilesystemVersion: snapsOnly[len(snapsOnly)-1], - } - - log.WithField("version", r.FilesystemVersion).Debug("requesting snapshot stream") - - var stream io.Reader - - if err = remote.Call("InitialTransferRequest", &r, &stream); err != nil { - log.WithError(err).Error("cannot request initial transfer") - return false - } - log.Debug("received initial transfer request response") - - log.Debug("invoke zfs receive") - recvArgs := []string{"-u"} - if localState.Placeholder { - log.Info("receive with forced rollback to replace placeholder filesystem") - recvArgs = append(recvArgs, "-F") - } - progressStream := p.task.ProgressUpdater(stream) - if err = zfs.ZFSRecv(m.Local, progressStream, recvArgs...); err != nil { - log.WithError(err).Error("cannot receive stream") - return false - } - log.Info("finished receiving stream") // TODO rx delta - - // TODO unify with recv path of ConflictIncremental - log.Debug("configuring properties of received filesystem") - props := zfs.NewZFSProperties() - props.Set("readonly", "on") - if err = zfs.ZFSSet(m.Local, props); err != nil { - log.WithError(err).Error("cannot set readonly property") - } - - log.Info("finished initial transfer") - return true - - case zfs.ConflictIncremental: - - if len(diff.IncrementalPath) < 2 { - log.Info("remote and local are in sync") - return true - } - - log.Info("following incremental path from diff") - for i := 0; i < len(diff.IncrementalPath)-1; i++ { - - from, to := diff.IncrementalPath[i], diff.IncrementalPath[i+1] - - log, _ := log.WithField(logIncFromField, from.Name).WithField(logIncToField, to.Name), 0 - - log.Debug("requesting incremental snapshot stream") - r := IncrementalTransferRequest{ - Filesystem: m.Remote, - From: from, - To: to, - } - var stream io.Reader - if err = remote.Call("IncrementalTransferRequest", &r, &stream); err != nil { - log.WithError(err).Error("cannot request incremental snapshot stream") - return false - } - - log.Debug("invoking zfs receive") - progressStream := p.task.ProgressUpdater(stream) - // TODO protect against malicious incremental stream - if err = zfs.ZFSRecv(m.Local, progressStream); err != nil { - log.WithError(err).Error("cannot receive stream") - return false - } - log.Info("finished incremental transfer") // TODO increment rx - - } - log.Info("finished following incremental path") // TODO path rx - return true - - case zfs.ConflictNoCommonAncestor: - fallthrough - case zfs.ConflictDiverged: - - var jsonDiff bytes.Buffer - if err := json.NewEncoder(&jsonDiff).Encode(diff); err != nil { - log.WithError(err).Error("cannot JSON-encode diff") - return false - } - - var problem, resolution string - - switch diff.Conflict { - case zfs.ConflictNoCommonAncestor: - problem = "remote and local filesystem have snapshots, but no common one" - resolution = "perform manual establish a common snapshot history" - case zfs.ConflictDiverged: - problem = "remote and local filesystem share a history but have diverged" - resolution = "perform manual replication or delete snapshots on the receiving" + - "side to establish an incremental replication parse" - } - - log.WithField("diff", jsonDiff.String()). - WithField("problem", problem). - WithField("resolution", resolution). - Error("manual conflict resolution required") - - return false - - } - - panic("should not be reached") -} - -func (p *Puller) Pull() { - p.task.Enter("run") - defer p.task.Finish() - - p.task.Log().Info("request remote filesystem list") - remoteFilesystems, ok := p.getRemoteFilesystems() - if !ok { - return - } - - p.task.Log().Debug("map remote filesystems to local paths and determine order for per-filesystem sync") - replMapping, ok := p.buildReplMapping(remoteFilesystems) - if !ok { - - } - - p.task.Log().Debug("build cache for already present local filesystem state") - p.task.Enter("cache_local_fs_state") - localFilesystemState, err := zfs.ZFSListFilesystemState() - p.task.Finish() +func (p *SenderEndpoint) ListFilesystems() ([]*replication.Filesystem, error) { + fss, err := zfs.ZFSListMapping(p.FSFilter) if err != nil { - p.task.Log().WithError(err).Error("cannot request local filesystem state") - return + return nil, err + } + rfss := make([]*replication.Filesystem, len(fss)) + for i := range fss { + rfss[i] = &replication.Filesystem{ + Path: fss[i].ToString(), + // FIXME: not supporting ResumeToken yet + } + } + return rfss, nil +} + +func (p *SenderEndpoint) ListFilesystemVersions(fs string) ([]*replication.FilesystemVersion, error) { + dp, err := zfs.NewDatasetPath(fs) + if err != nil { + return nil, err + } + pass, err := p.FSFilter.Filter(dp) + if err != nil { + return nil, err + } + if !pass { + return nil, replication.NewFilteredError(fs) + } + fsvs, err := zfs.ZFSListFilesystemVersions(dp, p.FilesystemVersionFilter) + if err != nil { + return nil, err + } + rfsvs := make([]*replication.FilesystemVersion, len(fsvs)) + for i := range fsvs { + rfsvs[i] = replication.FilesystemVersionFromZFS(fsvs[i]) + } + return rfsvs, nil +} + +func (p *SenderEndpoint) Send(r *replication.SendReq) (*replication.SendRes, io.Reader, error) { + os.Stderr.WriteString("sending " + r.String() + "\n") + dp, err := zfs.NewDatasetPath(r.Filesystem) + if err != nil { + return nil, nil, err + } + pass, err := p.FSFilter.Filter(dp) + if err != nil { + return nil, nil, err + } + if !pass { + return nil, nil, replication.NewFilteredError(r.Filesystem) + } + stream, err := zfs.ZFSSend(r.Filesystem, r.From, r.To) + if err != nil { + return nil, nil, err + } + return &replication.SendRes{}, stream, nil +} + +func (p *SenderEndpoint) Receive(r *replication.ReceiveReq, sendStream io.Reader) (error) { + return fmt.Errorf("sender endpoint does not receive") +} + + +// ReceiverEndpoint implements replication.ReplicationEndpoint for a receiving side +type ReceiverEndpoint struct { + fsmapInv *DatasetMapFilter + fsmap *DatasetMapFilter + fsvf zfs.FilesystemVersionFilter +} + +func NewReceiverEndpoint(fsmap *DatasetMapFilter, fsvf zfs.FilesystemVersionFilter) (*ReceiverEndpoint, error) { + fsmapInv, err := fsmap.Invert() + if err != nil { + return nil, err + } + return &ReceiverEndpoint{fsmapInv, fsmap, fsvf}, nil +} + +func (e *ReceiverEndpoint) ListFilesystems() ([]*replication.Filesystem, error) { + filtered, err := zfs.ZFSListMapping(e.fsmapInv.AsFilter()) + if err != nil { + return nil, errors.Wrap(err, "error checking client permission") + } + fss := make([]*replication.Filesystem, len(filtered)) + for i, a := range filtered { + mapped, err := e.fsmapInv.Map(a) + if err != nil { + return nil, err + } + fss[i] = &replication.Filesystem{Path: mapped.ToString()} + } + return fss, nil +} + +func (e *ReceiverEndpoint) ListFilesystemVersions(fs string) ([]*replication.FilesystemVersion, error) { + p, err := zfs.NewDatasetPath(fs) + if err != nil { + return nil, err + } + lp, err := e.fsmap.Map(p) + if err != nil { + return nil, err + } + if lp == nil { + return nil, errors.New("access to filesystem denied") } - localTraversal := zfs.NewDatasetPathForest() - for _, m := range replMapping { - localTraversal.Add(m.Local) + fsvs, err := zfs.ZFSListFilesystemVersions(lp, e.fsvf) + if err != nil { + return nil, err } - p.task.Log().Info("start per-filesystem sync") - localTraversal.WalkTopDown(func(v zfs.DatasetPathVisit) bool { + rfsvs := make([]*replication.FilesystemVersion, len(fsvs)) + for i := range fsvs { + rfsvs[i] = replication.FilesystemVersionFromZFS(fsvs[i]) + } - p.task.Enter("tree_walk") - defer p.task.Finish() + return rfsvs, nil +} - log := p.task.Log().WithField(logFSField, v.Path.ToString()) +func (e *ReceiverEndpoint) Send(req *replication.SendReq) (*replication.SendRes, io.Reader, error) { + return nil, nil, errors.New("receiver endpoint does not send") +} - if v.FilledIn { - if _, exists := localFilesystemState[v.Path.ToString()]; exists { - // No need to verify if this is a placeholder or not. It is sufficient - // to know we can add child filesystems to it - return true - } - log.Debug("create placeholder filesystem") - p.task.Enter("create_placeholder") - err = zfs.ZFSCreatePlaceholderFilesystem(v.Path) - p.task.Finish() - if err != nil { - log.Error("cannot create placeholder filesystem") +func (e *ReceiverEndpoint) Receive(req *replication.ReceiveReq, sendStream io.Reader) error { + p, err := zfs.NewDatasetPath(req.Filesystem) + if err != nil { + return err + } + lp, err := e.fsmap.Map(p) + if err != nil { + return err + } + if lp == nil { + return errors.New("receive to filesystem denied") + } + + // create placeholder parent filesystems as appropriate + var visitErr error + f := zfs.NewDatasetPathForest() + f.Add(lp) + f.WalkTopDown(func(v zfs.DatasetPathVisit) (visitChildTree bool) { + if v.Path.Equal(lp) { + return false + } + _, err := zfs.ZFSGet(v.Path, []string{zfs.ZREPL_PLACEHOLDER_PROPERTY_NAME}) + if err != nil { + os.Stderr.WriteString("error zfsget " + err.Error() + "\n") + // interpret this as an early exit of the zfs binary due to the fs not existing + if err := zfs.ZFSCreatePlaceholderFilesystem(v.Path); err != nil { + os.Stderr.WriteString("error creating placeholder " + v.Path.ToString() + "\n") + visitErr = err return false } - return true } - - m, ok := replMapping[v.Path.ToString()] - if !ok { - panic("internal inconsistency: replMapping should contain mapping for any path that was not filled in by WalkTopDown()") - } - - return p.replFilesystem(m, localFilesystemState) + os.Stderr.WriteString(v.Path.ToString() + " exists\n") + return true // leave this fs as is }) - return + if visitErr != nil { + return visitErr + } + needForceRecv := false + props, err := zfs.ZFSGet(lp, []string{zfs.ZREPL_PLACEHOLDER_PROPERTY_NAME}) + if err == nil { + if isPlaceholder, _ := zfs.IsPlaceholder(lp, props.Get(zfs.ZREPL_PLACEHOLDER_PROPERTY_NAME)); isPlaceholder { + needForceRecv = true + } + } + + args := make([]string, 0, 1) + if needForceRecv { + args = append(args, "-F") + } + + os.Stderr.WriteString("receiving...\n") + + if err := zfs.ZFSRecv(lp.ToString(), sendStream, args...); err != nil { + // FIXME sendStream is on the wire and contains data, if we don't consume it, wire must be closed + return err + } + return nil +} + +// =-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= +// RPC STUBS +// =-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= + + +const ( + RPCListFilesystems = "ListFilesystems" + RPCListFilesystemVersions = "ListFilesystemVersions" + RPCReceive = "Receive" + RPCSend = "Send" +) + +type RemoteEndpoint struct { + *streamrpc.Client +} + +func (s RemoteEndpoint) ListFilesystems() ([]*replication.Filesystem, error) { + req := replication.ListFilesystemReq{} + b, err := proto.Marshal(&req) + if err != nil { + return nil, err + } + rb, rs, err := s.RequestReply(RPCListFilesystems, bytes.NewBuffer(b), nil) + if err != nil { + return nil, err + } + if rs != nil { + os.Stderr.WriteString(fmt.Sprintf("%#v\n", rs)) + s.Close() // FIXME + return nil, errors.New("response contains unexpected stream") + } + var res replication.ListFilesystemRes + if err := proto.Unmarshal(rb.Bytes(), &res); err != nil { + return nil, err + } + return res.Filesystems, nil +} + +func (s RemoteEndpoint) ListFilesystemVersions(fs string) ([]*replication.FilesystemVersion, error) { + req := replication.ListFilesystemVersionsReq{ + Filesystem: fs, + } + b, err := proto.Marshal(&req) + if err != nil { + return nil, err + } + rb, rs, err := s.RequestReply(RPCListFilesystemVersions, bytes.NewBuffer(b), nil) + if err != nil { + return nil, err + } + if rs != nil { + s.Close() // FIXME + return nil, errors.New("response contains unexpected stream") + } + var res replication.ListFilesystemVersionsRes + if err := proto.Unmarshal(rb.Bytes(), &res); err != nil { + return nil, err + } + return res.Versions, nil +} + +func (s RemoteEndpoint) Send(r *replication.SendReq) (*replication.SendRes, io.Reader, error) { + b, err := proto.Marshal(r) + if err != nil { + return nil, nil, err + } + rb, rs, err := s.RequestReply(RPCSend, bytes.NewBuffer(b), nil) + if err != nil { + return nil, nil, err + } + if rs == nil { + return nil, nil, errors.New("response does not contain a stream") + } + var res replication.SendRes + if err := proto.Unmarshal(rb.Bytes(), &res); err != nil { + s.Close() // FIXME + return nil, nil, err + } + // FIXME make sure the consumer will read the reader until the end... + return &res, rs, nil +} + +func (s RemoteEndpoint) Receive(r *replication.ReceiveReq, sendStream io.Reader) (error) { + b, err := proto.Marshal(r) + if err != nil { + return err + } + rb, rs, err := s.RequestReply(RPCReceive, bytes.NewBuffer(b), sendStream) + if err != nil { + s.Close() // FIXME + return err + } + if rs != nil { + return errors.New("response contains unexpected stream") + } + var res replication.ReceiveRes + if err := proto.Unmarshal(rb.Bytes(), &res); err != nil { + return err + } + return nil +} + +type HandlerAdaptor struct { + ep replication.ReplicationEndpoint +} + +func (a *HandlerAdaptor) Handle(endpoint string, reqStructured *bytes.Buffer, reqStream io.Reader) (resStructured *bytes.Buffer, resStream io.Reader, err error) { + + switch endpoint { + case RPCListFilesystems: + var req replication.ListFilesystemReq + if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { + return nil, nil, err + } + fsses, err := a.ep.ListFilesystems() + if err != nil { + return nil, nil, err + } + res := &replication.ListFilesystemRes{ + Filesystems: fsses, + } + b, err := proto.Marshal(res) + if err != nil { + return nil, nil, err + } + return bytes.NewBuffer(b), nil, nil + + case RPCListFilesystemVersions: + + var req replication.ListFilesystemVersionsReq + if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { + return nil, nil, err + } + fsvs, err := a.ep.ListFilesystemVersions(req.Filesystem) + if err != nil { + return nil, nil, err + } + res := &replication.ListFilesystemVersionsRes{ + Versions: fsvs, + } + b, err := proto.Marshal(res) + if err != nil { + return nil, nil, err + } + return bytes.NewBuffer(b), nil, nil + + case RPCSend: + + var req replication.SendReq + if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { + return nil, nil, err + } + res, sendStream, err := a.ep.Send(&req) + if err != nil { + return nil, nil, err + } + b, err := proto.Marshal(res) + if err != nil { + return nil, nil, err + } + return bytes.NewBuffer(b), sendStream, err + + case RPCReceive: + + var req replication.ReceiveReq + if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { + return nil, nil, err + } + err := a.ep.Receive(&req, reqStream) + if err != nil { + return nil, nil, err + } + b, err := proto.Marshal(&replication.ReceiveRes{}) + if err != nil { + return nil, nil, err + } + return bytes.NewBuffer(b), nil, err + + + default: + return nil, nil, errors.New("no handler for given endpoint") + } } diff --git a/cmd/replication/diff.go b/cmd/replication/diff.go index 3494ba6..bb6cb17 100644 --- a/cmd/replication/diff.go +++ b/cmd/replication/diff.go @@ -1,12 +1,11 @@ package replication import ( - "github.com/zrepl/zrepl/zfs" "sort" ) type ConflictNoCommonAncestor struct { - SortedSenderVersions, SortedReceiverVersions []zfs.FilesystemVersion + SortedSenderVersions, SortedReceiverVersions []*FilesystemVersion } func (c *ConflictNoCommonAncestor) Error() string { @@ -14,24 +13,24 @@ func (c *ConflictNoCommonAncestor) Error() string { } type ConflictDiverged struct { - SortedSenderVersions, SortedReceiverVersions []zfs.FilesystemVersion - CommonAncestor zfs.FilesystemVersion - SenderOnly, ReceiverOnly []zfs.FilesystemVersion + SortedSenderVersions, SortedReceiverVersions []*FilesystemVersion + CommonAncestor *FilesystemVersion + SenderOnly, ReceiverOnly []*FilesystemVersion } func (c *ConflictDiverged) Error() string { return "the receiver's latest snapshot is not present on sender" } -func SortVersionListByCreateTXGThenBookmarkLTSnapshot(fsvslice []zfs.FilesystemVersion) []zfs.FilesystemVersion { - lesser := func(s []zfs.FilesystemVersion) func(i, j int) bool { +func SortVersionListByCreateTXGThenBookmarkLTSnapshot(fsvslice []*FilesystemVersion) []*FilesystemVersion { + lesser := func(s []*FilesystemVersion) func(i, j int) bool { return func(i, j int) bool { if s[i].CreateTXG < s[j].CreateTXG { return true } if s[i].CreateTXG == s[j].CreateTXG { // Bookmark < Snapshot - return s[i].Type == zfs.Bookmark && s[j].Type == zfs.Snapshot + return s[i].Type == FilesystemVersion_Bookmark && s[j].Type == FilesystemVersion_Snapshot } return false } @@ -39,14 +38,14 @@ func SortVersionListByCreateTXGThenBookmarkLTSnapshot(fsvslice []zfs.FilesystemV if sort.SliceIsSorted(fsvslice, lesser(fsvslice)) { return fsvslice } - sorted := make([]zfs.FilesystemVersion, len(fsvslice)) + sorted := make([]*FilesystemVersion, len(fsvslice)) copy(sorted, fsvslice) sort.Slice(sorted, lesser(sorted)) return sorted } // conflict may be a *ConflictDiverged or a *ConflictNoCommonAncestor -func IncrementalPath(receiver, sender []zfs.FilesystemVersion) (incPath []zfs.FilesystemVersion, conflict error) { +func IncrementalPath(receiver, sender []*FilesystemVersion) (incPath []*FilesystemVersion, conflict error) { if receiver == nil { panic("receiver must not be nil") @@ -59,7 +58,7 @@ func IncrementalPath(receiver, sender []zfs.FilesystemVersion) (incPath []zfs.Fi sender = SortVersionListByCreateTXGThenBookmarkLTSnapshot(sender) if len(sender) == 0 { - return []zfs.FilesystemVersion{}, nil + return []*FilesystemVersion{}, nil } // Find most recent common ancestor by name, preferring snapshots over bookmarks @@ -69,7 +68,7 @@ func IncrementalPath(receiver, sender []zfs.FilesystemVersion) (incPath []zfs.Fi for mrcaRcv >= 0 && mrcaSnd >= 0 { if receiver[mrcaRcv].Guid == sender[mrcaSnd].Guid { - if mrcaSnd-1 >= 0 && sender[mrcaSnd-1].Guid == sender[mrcaSnd].Guid && sender[mrcaSnd-1].Type == zfs.Bookmark { + if mrcaSnd-1 >= 0 && sender[mrcaSnd-1].Guid == sender[mrcaSnd].Guid && sender[mrcaSnd-1].Type == FilesystemVersion_Bookmark { // prefer bookmarks over snapshots as the snapshot might go away sooner mrcaSnd -= 1 } @@ -100,11 +99,11 @@ func IncrementalPath(receiver, sender []zfs.FilesystemVersion) (incPath []zfs.Fi } // incPath must not contain bookmarks except initial one, - incPath = make([]zfs.FilesystemVersion, 0, len(sender)) + incPath = make([]*FilesystemVersion, 0, len(sender)) incPath = append(incPath, sender[mrcaSnd]) // it's ok if incPath[0] is a bookmark, but not the subsequent ones in the incPath for i := mrcaSnd + 1; i < len(sender); i++ { - if sender[i].Type == zfs.Snapshot && incPath[len(incPath)-1].Guid != sender[i].Guid { + if sender[i].Type == FilesystemVersion_Snapshot && incPath[len(incPath)-1].Guid != sender[i].Guid { incPath = append(incPath, sender[i]) } } diff --git a/cmd/replication/diff_test.go b/cmd/replication/diff_test.go index 916b8b0..cf9b771 100644 --- a/cmd/replication/diff_test.go +++ b/cmd/replication/diff_test.go @@ -3,16 +3,15 @@ package replication_test import ( "github.com/stretchr/testify/assert" "github.com/zrepl/zrepl/cmd/replication" - "github.com/zrepl/zrepl/zfs" "strconv" "strings" "testing" "time" ) -func fsvlist(fsv ...string) (r []zfs.FilesystemVersion) { +func fsvlist(fsv ...string) (r []*replication.FilesystemVersion) { - r = make([]zfs.FilesystemVersion, len(fsv)) + r = make([]*replication.FilesystemVersion, len(fsv)) for i, f := range fsv { // parse the id from fsvlist. it is used to derivce Guid,CreateTXG and Creation attrs @@ -26,20 +25,20 @@ func fsvlist(fsv ...string) (r []zfs.FilesystemVersion) { } if strings.HasPrefix(f, "#") { - r[i] = zfs.FilesystemVersion{ + r[i] = &replication.FilesystemVersion{ Name: strings.TrimPrefix(f, "#"), - Type: zfs.Bookmark, + Type: replication.FilesystemVersion_Bookmark, Guid: uint64(id), CreateTXG: uint64(id), - Creation: time.Unix(0, 0).Add(time.Duration(id) * time.Second), + Creation: time.Unix(0, 0).Add(time.Duration(id) * time.Second).Format(time.RFC3339), } } else if strings.HasPrefix(f, "@") { - r[i] = zfs.FilesystemVersion{ + r[i] = &replication.FilesystemVersion{ Name: strings.TrimPrefix(f, "@"), - Type: zfs.Snapshot, + Type: replication.FilesystemVersion_Snapshot, Guid: uint64(id), CreateTXG: uint64(id), - Creation: time.Unix(0, 0).Add(time.Duration(id) * time.Second), + Creation: time.Unix(0, 0).Add(time.Duration(id) * time.Second).Format(time.RFC3339), } } else { panic("invalid character") @@ -49,14 +48,14 @@ func fsvlist(fsv ...string) (r []zfs.FilesystemVersion) { } type incPathResult struct { - incPath []zfs.FilesystemVersion + incPath []*replication.FilesystemVersion conflict error } type IncrementalPathTest struct { Msg string - Receiver, Sender []zfs.FilesystemVersion - ExpectIncPath []zfs.FilesystemVersion + Receiver, Sender []*replication.FilesystemVersion + ExpectIncPath []*replication.FilesystemVersion ExpectNoCommonAncestor bool ExpectDiverged *replication.ConflictDiverged ExpectPanic bool @@ -212,7 +211,7 @@ func TestSortVersionListByCreateTXGThenBookmarkLTSnapshot(t *testing.T) { type Test struct { Msg string - Input, Output []zfs.FilesystemVersion + Input, Output []*replication.FilesystemVersion } l := fsvlist @@ -258,7 +257,7 @@ func TestSortVersionListByCreateTXGThenBookmarkLTSnapshot(t *testing.T) { break } if s.CreateTXG == last.CreateTXG { - if last.Type == zfs.Bookmark && s.Type != zfs.Snapshot { + if last.Type == replication.FilesystemVersion_Bookmark && s.Type != replication.FilesystemVersion_Snapshot { t.Errorf("snapshots must come after bookmarks") } } diff --git a/cmd/replication/pdu.pb.go b/cmd/replication/pdu.pb.go new file mode 100644 index 0000000..1d382bd --- /dev/null +++ b/cmd/replication/pdu.pb.go @@ -0,0 +1,384 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// source: pdu.proto + +/* +Package replication is a generated protocol buffer package. + +It is generated from these files: + pdu.proto + +It has these top-level messages: + ListFilesystemReq + ListFilesystemRes + Filesystem + ListFilesystemVersionsReq + ListFilesystemVersionsRes + FilesystemVersion + SendReq + Property + SendRes + ReceiveReq + ReceiveRes +*/ +package replication + +import proto "github.com/golang/protobuf/proto" +import fmt "fmt" +import math "math" + +// 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.ProtoPackageIsVersion2 // please upgrade the proto package + +type FilesystemVersion_VersionType int32 + +const ( + FilesystemVersion_Snapshot FilesystemVersion_VersionType = 0 + FilesystemVersion_Bookmark FilesystemVersion_VersionType = 1 +) + +var FilesystemVersion_VersionType_name = map[int32]string{ + 0: "Snapshot", + 1: "Bookmark", +} +var FilesystemVersion_VersionType_value = map[string]int32{ + "Snapshot": 0, + "Bookmark": 1, +} + +func (x FilesystemVersion_VersionType) String() string { + return proto.EnumName(FilesystemVersion_VersionType_name, int32(x)) +} +func (FilesystemVersion_VersionType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor0, []int{5, 0} +} + +type ListFilesystemReq struct { +} + +func (m *ListFilesystemReq) Reset() { *m = ListFilesystemReq{} } +func (m *ListFilesystemReq) String() string { return proto.CompactTextString(m) } +func (*ListFilesystemReq) ProtoMessage() {} +func (*ListFilesystemReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} } + +type ListFilesystemRes struct { + Filesystems []*Filesystem `protobuf:"bytes,1,rep,name=Filesystems" json:"Filesystems,omitempty"` +} + +func (m *ListFilesystemRes) Reset() { *m = ListFilesystemRes{} } +func (m *ListFilesystemRes) String() string { return proto.CompactTextString(m) } +func (*ListFilesystemRes) ProtoMessage() {} +func (*ListFilesystemRes) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } + +func (m *ListFilesystemRes) GetFilesystems() []*Filesystem { + if m != nil { + return m.Filesystems + } + return nil +} + +type Filesystem struct { + Path string `protobuf:"bytes,1,opt,name=Path" json:"Path,omitempty"` + ResumeToken string `protobuf:"bytes,2,opt,name=ResumeToken" json:"ResumeToken,omitempty"` +} + +func (m *Filesystem) Reset() { *m = Filesystem{} } +func (m *Filesystem) String() string { return proto.CompactTextString(m) } +func (*Filesystem) ProtoMessage() {} +func (*Filesystem) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} } + +func (m *Filesystem) GetPath() string { + if m != nil { + return m.Path + } + return "" +} + +func (m *Filesystem) GetResumeToken() string { + if m != nil { + return m.ResumeToken + } + return "" +} + +type ListFilesystemVersionsReq struct { + Filesystem string `protobuf:"bytes,1,opt,name=Filesystem" json:"Filesystem,omitempty"` +} + +func (m *ListFilesystemVersionsReq) Reset() { *m = ListFilesystemVersionsReq{} } +func (m *ListFilesystemVersionsReq) String() string { return proto.CompactTextString(m) } +func (*ListFilesystemVersionsReq) ProtoMessage() {} +func (*ListFilesystemVersionsReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} } + +func (m *ListFilesystemVersionsReq) GetFilesystem() string { + if m != nil { + return m.Filesystem + } + return "" +} + +type ListFilesystemVersionsRes struct { + Versions []*FilesystemVersion `protobuf:"bytes,1,rep,name=Versions" json:"Versions,omitempty"` +} + +func (m *ListFilesystemVersionsRes) Reset() { *m = ListFilesystemVersionsRes{} } +func (m *ListFilesystemVersionsRes) String() string { return proto.CompactTextString(m) } +func (*ListFilesystemVersionsRes) ProtoMessage() {} +func (*ListFilesystemVersionsRes) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} } + +func (m *ListFilesystemVersionsRes) GetVersions() []*FilesystemVersion { + if m != nil { + return m.Versions + } + return nil +} + +type FilesystemVersion struct { + Type FilesystemVersion_VersionType `protobuf:"varint,1,opt,name=Type,enum=replication.FilesystemVersion_VersionType" json:"Type,omitempty"` + Name string `protobuf:"bytes,2,opt,name=Name" json:"Name,omitempty"` + Guid uint64 `protobuf:"varint,3,opt,name=Guid" json:"Guid,omitempty"` + CreateTXG uint64 `protobuf:"varint,4,opt,name=CreateTXG" json:"CreateTXG,omitempty"` + Creation string `protobuf:"bytes,5,opt,name=Creation" json:"Creation,omitempty"` +} + +func (m *FilesystemVersion) Reset() { *m = FilesystemVersion{} } +func (m *FilesystemVersion) String() string { return proto.CompactTextString(m) } +func (*FilesystemVersion) ProtoMessage() {} +func (*FilesystemVersion) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} } + +func (m *FilesystemVersion) GetType() FilesystemVersion_VersionType { + if m != nil { + return m.Type + } + return FilesystemVersion_Snapshot +} + +func (m *FilesystemVersion) GetName() string { + if m != nil { + return m.Name + } + return "" +} + +func (m *FilesystemVersion) GetGuid() uint64 { + if m != nil { + return m.Guid + } + return 0 +} + +func (m *FilesystemVersion) GetCreateTXG() uint64 { + if m != nil { + return m.CreateTXG + } + return 0 +} + +func (m *FilesystemVersion) GetCreation() string { + if m != nil { + return m.Creation + } + return "" +} + +type SendReq struct { + Filesystem string `protobuf:"bytes,1,opt,name=Filesystem" json:"Filesystem,omitempty"` + From string `protobuf:"bytes,2,opt,name=From" json:"From,omitempty"` + To string `protobuf:"bytes,3,opt,name=To" json:"To,omitempty"` + // If ResumeToken is not empty, the resume token that CAN be tried for 'zfs send' by the sender. + // The sender MUST indicate in SendRes.UsedResumeToken + // If it does not work, the sender SHOULD clear the resume token on their side + // and use From and To instead + // If ResumeToken is not empty, the GUIDs of From and To + // MUST correspond to those encoded in the ResumeToken. + // Otherwise, the Sender MUST return an error. + ResumeToken string `protobuf:"bytes,4,opt,name=ResumeToken" json:"ResumeToken,omitempty"` + Compress bool `protobuf:"varint,5,opt,name=Compress" json:"Compress,omitempty"` + Dedup bool `protobuf:"varint,6,opt,name=Dedup" json:"Dedup,omitempty"` +} + +func (m *SendReq) Reset() { *m = SendReq{} } +func (m *SendReq) String() string { return proto.CompactTextString(m) } +func (*SendReq) ProtoMessage() {} +func (*SendReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} } + +func (m *SendReq) GetFilesystem() string { + if m != nil { + return m.Filesystem + } + return "" +} + +func (m *SendReq) GetFrom() string { + if m != nil { + return m.From + } + return "" +} + +func (m *SendReq) GetTo() string { + if m != nil { + return m.To + } + return "" +} + +func (m *SendReq) GetResumeToken() string { + if m != nil { + return m.ResumeToken + } + return "" +} + +func (m *SendReq) GetCompress() bool { + if m != nil { + return m.Compress + } + return false +} + +func (m *SendReq) GetDedup() bool { + if m != nil { + return m.Dedup + } + return false +} + +type Property struct { + Name string `protobuf:"bytes,1,opt,name=Name" json:"Name,omitempty"` + Value string `protobuf:"bytes,2,opt,name=Value" json:"Value,omitempty"` +} + +func (m *Property) Reset() { *m = Property{} } +func (m *Property) String() string { return proto.CompactTextString(m) } +func (*Property) ProtoMessage() {} +func (*Property) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} } + +func (m *Property) GetName() string { + if m != nil { + return m.Name + } + return "" +} + +func (m *Property) GetValue() string { + if m != nil { + return m.Value + } + return "" +} + +type SendRes struct { + // Whether the resume token provided in the request has been used or not. + UsedResumeToken bool `protobuf:"varint,1,opt,name=UsedResumeToken" json:"UsedResumeToken,omitempty"` + Properties []*Property `protobuf:"bytes,2,rep,name=Properties" json:"Properties,omitempty"` +} + +func (m *SendRes) Reset() { *m = SendRes{} } +func (m *SendRes) String() string { return proto.CompactTextString(m) } +func (*SendRes) ProtoMessage() {} +func (*SendRes) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} } + +func (m *SendRes) GetUsedResumeToken() bool { + if m != nil { + return m.UsedResumeToken + } + return false +} + +func (m *SendRes) GetProperties() []*Property { + if m != nil { + return m.Properties + } + return nil +} + +type ReceiveReq struct { + Filesystem string `protobuf:"bytes,1,opt,name=Filesystem" json:"Filesystem,omitempty"` + // If true, the receiver should clear the resume token before perfoming the zfs recv of the stream in the request + ClearResumeToken bool `protobuf:"varint,2,opt,name=ClearResumeToken" json:"ClearResumeToken,omitempty"` +} + +func (m *ReceiveReq) Reset() { *m = ReceiveReq{} } +func (m *ReceiveReq) String() string { return proto.CompactTextString(m) } +func (*ReceiveReq) ProtoMessage() {} +func (*ReceiveReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} } + +func (m *ReceiveReq) GetFilesystem() string { + if m != nil { + return m.Filesystem + } + return "" +} + +func (m *ReceiveReq) GetClearResumeToken() bool { + if m != nil { + return m.ClearResumeToken + } + return false +} + +type ReceiveRes struct { +} + +func (m *ReceiveRes) Reset() { *m = ReceiveRes{} } +func (m *ReceiveRes) String() string { return proto.CompactTextString(m) } +func (*ReceiveRes) ProtoMessage() {} +func (*ReceiveRes) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} } + +func init() { + proto.RegisterType((*ListFilesystemReq)(nil), "replication.ListFilesystemReq") + proto.RegisterType((*ListFilesystemRes)(nil), "replication.ListFilesystemRes") + proto.RegisterType((*Filesystem)(nil), "replication.Filesystem") + proto.RegisterType((*ListFilesystemVersionsReq)(nil), "replication.ListFilesystemVersionsReq") + proto.RegisterType((*ListFilesystemVersionsRes)(nil), "replication.ListFilesystemVersionsRes") + proto.RegisterType((*FilesystemVersion)(nil), "replication.FilesystemVersion") + proto.RegisterType((*SendReq)(nil), "replication.SendReq") + proto.RegisterType((*Property)(nil), "replication.Property") + proto.RegisterType((*SendRes)(nil), "replication.SendRes") + proto.RegisterType((*ReceiveReq)(nil), "replication.ReceiveReq") + proto.RegisterType((*ReceiveRes)(nil), "replication.ReceiveRes") + proto.RegisterEnum("replication.FilesystemVersion_VersionType", FilesystemVersion_VersionType_name, FilesystemVersion_VersionType_value) +} + +func init() { proto.RegisterFile("pdu.proto", fileDescriptor0) } + +var fileDescriptor0 = []byte{ + // 454 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x53, 0x4d, 0x6f, 0xd3, 0x40, + 0x10, 0x65, 0x53, 0xa7, 0x38, 0xe3, 0xaa, 0xa4, 0x4b, 0x11, 0x06, 0xa1, 0x2a, 0xda, 0x53, 0xe8, + 0x21, 0x87, 0x02, 0x07, 0x40, 0xe2, 0xd0, 0xa2, 0xf6, 0x82, 0xaa, 0x6a, 0x6b, 0x4a, 0xaf, 0xa6, + 0x1e, 0xa9, 0x4b, 0x62, 0xaf, 0xbb, 0x63, 0x23, 0xe5, 0xe7, 0xf0, 0xcf, 0xf8, 0x29, 0xc8, 0x53, + 0x3b, 0xd9, 0x26, 0x2a, 0xca, 0xc9, 0xf3, 0xde, 0x7c, 0xbd, 0x7d, 0xeb, 0x85, 0x41, 0x99, 0xd5, + 0x93, 0xd2, 0xd9, 0xca, 0xca, 0xc8, 0x61, 0x39, 0x33, 0x37, 0x69, 0x65, 0x6c, 0xa1, 0x9e, 0xc3, + 0xde, 0x37, 0x43, 0xd5, 0xa9, 0x99, 0x21, 0xcd, 0xa9, 0xc2, 0x5c, 0xe3, 0x9d, 0x3a, 0x5f, 0x27, + 0x49, 0x7e, 0x84, 0x68, 0x49, 0x50, 0x2c, 0x46, 0x5b, 0xe3, 0xe8, 0xe8, 0xe5, 0xc4, 0x1b, 0x36, + 0xf1, 0x1a, 0xfc, 0x5a, 0x75, 0x0c, 0xb0, 0x84, 0x52, 0x42, 0x70, 0x91, 0x56, 0xb7, 0xb1, 0x18, + 0x89, 0xf1, 0x40, 0x73, 0x2c, 0x47, 0x10, 0x69, 0xa4, 0x3a, 0xc7, 0xc4, 0x4e, 0xb1, 0x88, 0x7b, + 0x9c, 0xf2, 0x29, 0xf5, 0x19, 0x5e, 0x3d, 0xd4, 0x74, 0x85, 0x8e, 0x8c, 0x2d, 0x48, 0xe3, 0x9d, + 0x3c, 0xf0, 0x17, 0xb4, 0x83, 0x3d, 0x46, 0xfd, 0x78, 0xbc, 0x99, 0xe4, 0x27, 0x08, 0x3b, 0xd8, + 0x9e, 0xea, 0xe0, 0x91, 0x53, 0xb5, 0x65, 0x7a, 0x51, 0xaf, 0xfe, 0x0a, 0xd8, 0x5b, 0xcb, 0xcb, + 0x2f, 0x10, 0x24, 0xf3, 0x12, 0x59, 0xc8, 0xee, 0xd1, 0xe1, 0xff, 0xa7, 0x4d, 0xda, 0x6f, 0xd3, + 0xa1, 0xb9, 0xaf, 0x71, 0xe8, 0x3c, 0xcd, 0xb1, 0xb5, 0x81, 0xe3, 0x86, 0x3b, 0xab, 0x4d, 0x16, + 0x6f, 0x8d, 0xc4, 0x38, 0xd0, 0x1c, 0xcb, 0x37, 0x30, 0x38, 0x71, 0x98, 0x56, 0x98, 0x5c, 0x9f, + 0xc5, 0x01, 0x27, 0x96, 0x84, 0x7c, 0x0d, 0x21, 0x03, 0x63, 0x8b, 0xb8, 0xcf, 0x93, 0x16, 0x58, + 0xbd, 0x85, 0xc8, 0x5b, 0x2b, 0x77, 0x20, 0xbc, 0x2c, 0xd2, 0x92, 0x6e, 0x6d, 0x35, 0x7c, 0xd2, + 0xa0, 0x63, 0x6b, 0xa7, 0x79, 0xea, 0xa6, 0x43, 0xa1, 0xfe, 0x08, 0x78, 0x7a, 0x89, 0x45, 0xb6, + 0x81, 0xcf, 0x8d, 0xc8, 0x53, 0x67, 0xf3, 0x4e, 0x78, 0x13, 0xcb, 0x5d, 0xe8, 0x25, 0x96, 0x65, + 0x0f, 0x74, 0x2f, 0xb1, 0xab, 0x57, 0x1d, 0xac, 0x5d, 0x35, 0x0b, 0xb7, 0x79, 0xe9, 0x90, 0x88, + 0x85, 0x87, 0x7a, 0x81, 0xe5, 0x3e, 0xf4, 0xbf, 0x62, 0x56, 0x97, 0xf1, 0x36, 0x27, 0xee, 0x81, + 0x7a, 0x0f, 0xe1, 0x85, 0xb3, 0x25, 0xba, 0x6a, 0xbe, 0x30, 0x4f, 0x78, 0xe6, 0xed, 0x43, 0xff, + 0x2a, 0x9d, 0xd5, 0x9d, 0xa3, 0xf7, 0x40, 0xfd, 0xea, 0x0e, 0x46, 0x72, 0x0c, 0xcf, 0xbe, 0x13, + 0x66, 0xbe, 0x30, 0xc1, 0x0b, 0x56, 0x69, 0xf9, 0x01, 0xa0, 0x5d, 0x65, 0x90, 0xe2, 0x1e, 0xff, + 0x2f, 0x2f, 0x1e, 0xdc, 0x70, 0xa7, 0x44, 0x7b, 0x85, 0xea, 0x1a, 0x40, 0xe3, 0x0d, 0x9a, 0xdf, + 0xb8, 0x89, 0x8f, 0x87, 0x30, 0x3c, 0x99, 0x61, 0xea, 0x56, 0xdf, 0x44, 0xa8, 0xd7, 0x78, 0xb5, + 0xe3, 0x4d, 0xa6, 0x9f, 0xdb, 0xfc, 0xc6, 0xdf, 0xfd, 0x0b, 0x00, 0x00, 0xff, 0xff, 0xa4, 0x5a, + 0xf6, 0xa7, 0xf0, 0x03, 0x00, 0x00, +} diff --git a/cmd/replication/pdu.proto b/cmd/replication/pdu.proto new file mode 100644 index 0000000..72fc56c --- /dev/null +++ b/cmd/replication/pdu.proto @@ -0,0 +1,78 @@ +syntax = "proto3"; + +package replication; + +message ListFilesystemReq {} + +message ListFilesystemRes { + repeated Filesystem Filesystems = 1; +} + +message Filesystem { + string Path = 1; + string ResumeToken = 2; +} + +message ListFilesystemVersionsReq { + string Filesystem = 1; +} + +message ListFilesystemVersionsRes { + repeated FilesystemVersion Versions = 1; +} + +message FilesystemVersion { + enum VersionType { + Snapshot = 0; + Bookmark = 1; + } + VersionType Type = 1; + string Name = 2; + uint64 Guid = 3; + uint64 CreateTXG = 4; + string Creation = 5; // RFC 3339 +} + + +message SendReq { + string Filesystem = 1; + string From = 2; + // May be empty / null to request a full transfer of From + string To = 3; + + // If ResumeToken is not empty, the resume token that CAN be tried for 'zfs send' by the sender. + // The sender MUST indicate in SendRes.UsedResumeToken + // If it does not work, the sender SHOULD clear the resume token on their side + // and use From and To instead + // If ResumeToken is not empty, the GUIDs of From and To + // MUST correspond to those encoded in the ResumeToken. + // Otherwise, the Sender MUST return an error. + string ResumeToken = 4; + bool Compress = 5; + bool Dedup = 6; +} + +message Property { + string Name = 1; + string Value = 2; +} + +message SendRes { + // The actual stream is in the stream part of the streamrpc response + + // Whether the resume token provided in the request has been used or not. + bool UsedResumeToken = 1; + + repeated Property Properties = 2; +} + +message ReceiveReq { + // The stream part of the streamrpc request contains the zfs send stream + + string Filesystem = 1; + + // If true, the receiver should clear the resume token before perfoming the zfs recv of the stream in the request + bool ClearResumeToken = 2; +} + +message ReceiveRes {} diff --git a/cmd/replication/pdu_extras.go b/cmd/replication/pdu_extras.go new file mode 100644 index 0000000..b962059 --- /dev/null +++ b/cmd/replication/pdu_extras.go @@ -0,0 +1,60 @@ +package replication + +import ( + "fmt" + "github.com/zrepl/zrepl/zfs" + "time" +) + +func (v *FilesystemVersion) RelName() string { + zv := v.ZFSFilesystemVersion() + return zv.String() +} + +func (v FilesystemVersion_VersionType) ZFSVersionType() zfs.VersionType { + switch v { + case FilesystemVersion_Snapshot: + return zfs.Snapshot + case FilesystemVersion_Bookmark: + return zfs.Bookmark + default: + panic(fmt.Sprintf("unexpected v.Type %#v", v)) + } +} + +func FilesystemVersionFromZFS(fsv zfs.FilesystemVersion) *FilesystemVersion { + var t FilesystemVersion_VersionType + switch fsv.Type { + case zfs.Bookmark: + t = FilesystemVersion_Bookmark + case zfs.Snapshot: + t = FilesystemVersion_Snapshot + default: + panic("unknown fsv.Type: " + fsv.Type) + } + return &FilesystemVersion{ + Type: t, + Name: fsv.Name, + Guid: fsv.Guid, + CreateTXG: fsv.CreateTXG, + Creation: fsv.Creation.Format(time.RFC3339), + } +} + +func (v *FilesystemVersion) ZFSFilesystemVersion() *zfs.FilesystemVersion { + ct := time.Time{} + if v.Creation != "" { + var err error + ct, err = time.Parse(time.RFC3339, v.Creation) + if err != nil { + panic(err) + } + } + return &zfs.FilesystemVersion{ + Type: v.Type.ZFSVersionType(), + Name: v.Name, + Guid: v.Guid, + CreateTXG: v.CreateTXG, + Creation: ct, + } +} \ No newline at end of file diff --git a/cmd/replication/pdu_test.go b/cmd/replication/pdu_test.go new file mode 100644 index 0000000..3b26572 --- /dev/null +++ b/cmd/replication/pdu_test.go @@ -0,0 +1,64 @@ +package replication + +import ( + "testing" + "github.com/stretchr/testify/assert" +) + +func TestFilesystemVersion_RelName(t *testing.T) { + + type TestCase struct { + In FilesystemVersion + Out string + Panic bool + } + + tcs := []TestCase{ + { + In: FilesystemVersion{ + Type: FilesystemVersion_Snapshot, + Name: "foobar", + }, + Out: "@foobar", + }, + { + In: FilesystemVersion{ + Type: FilesystemVersion_Bookmark, + Name: "foobar", + }, + Out: "#foobar", + }, + { + In: FilesystemVersion{ + Type: 2342, + Name: "foobar", + }, + Panic: true, + }, + } + + for _, tc := range tcs { + if tc.Panic { + assert.Panics(t, func() { + tc.In.RelName() + }) + } else { + o := tc.In.RelName() + assert.Equal(t, tc.Out, o) + } + } + +} + +func TestFilesystemVersion_ZFSFilesystemVersion(t *testing.T) { + + empty := &FilesystemVersion{} + emptyZFS := empty.ZFSFilesystemVersion() + assert.Zero(t, emptyZFS.Creation) + + dateInvalid := &FilesystemVersion{Creation:"foobar"} + assert.Panics(t, func() { + dateInvalid.ZFSFilesystemVersion() + }) + +} diff --git a/cmd/replication/replication.go b/cmd/replication/replication.go index 1759560..6b3f57b 100644 --- a/cmd/replication/replication.go +++ b/cmd/replication/replication.go @@ -2,54 +2,25 @@ package replication import ( "context" - "github.com/zrepl/zrepl/zfs" "io" ) type ReplicationEndpoint interface { // Does not include placeholder filesystems - ListFilesystems() ([]Filesystem, error) - ListFilesystemVersions(fs string) ([]zfs.FilesystemVersion, error) // fix depS + ListFilesystems() ([]*Filesystem, error) + ListFilesystemVersions(fs string) ([]*FilesystemVersion, error) // fix depS Sender Receiver } -type Filesystem struct { - Path string - ResumeToken string -} - type FilteredError struct{ fs string } +func NewFilteredError(fs string) FilteredError { + return FilteredError{fs} +} + func (f FilteredError) Error() string { return "endpoint does not allow access to filesystem " + f.fs } -type SendRequest struct { - Filesystem string - From, To string - // If ResumeToken is not empty, the resume token that CAN be tried for 'zfs send' by the sender - // If it does not work, the sender SHOULD clear the resume token on their side - // and use From and To instead - // If ResumeToken is not empty, the GUIDs of From and To - // MUST correspond to those encoded in the ResumeToken. - // Otherwise, the Sender MUST return an error. - ResumeToken string - Compress bool - Dedup bool -} - -type SendResponse struct { - Properties zfs.ZFSProperties // fix dep - Stream io.Reader -} - -type ReceiveRequest struct { - Filesystem string - // The resume token used by the sending side. - // The receiver MUST discard the saved state on their side if ResumeToken - // does not match the zfs property of Filesystem on their side. - ResumeToken string -} - type ReplicationMode int const ( @@ -96,28 +67,90 @@ func (p EndpointPair) Mode() ReplicationMode { return p.m } +type contextKey int + +const ( + ContextKeyLog contextKey = iota +) + +type Logger interface{ + Printf(fmt string, args ... interface{}) +} + func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicator) { + log := ctx.Value(ContextKeyLog).(Logger) + sfss, err := ep.Sender().ListFilesystems() if err != nil { - // log error + log.Printf("error listing sender filesystems: %s", err) + return + } + + rfss, err := ep.Receiver().ListFilesystems() + if err != nil { + log.Printf("error listing receiver filesystems: %s", err) return } for _, fs := range sfss { + log.Printf("replication fs %s", fs.Path) sfsvs, err := ep.Sender().ListFilesystemVersions(fs.Path) - rfsvs, err := ep.Receiver().ListFilesystemVersions(fs.Path) if err != nil { - if _, ok := err.(FilteredError); ok { - // Remote does not map filesystem, don't try to tx it - continue - } - // log and ignore + log.Printf("sender error %s", err) continue } + if len(sfsvs) <= 1 { + log.Printf("sender does not have any versions") + continue + } + + receiverFSExists := false + for _, rfs := range rfss { + if rfs.Path == fs.Path { + receiverFSExists = true + } + } + + var rfsvs []*FilesystemVersion + if receiverFSExists { + rfsvs, err = ep.Receiver().ListFilesystemVersions(fs.Path) + if err != nil { + log.Printf("receiver error %s", err) + if _, ok := err.(FilteredError); ok { + // Remote does not map filesystem, don't try to tx it + continue + } + // log and ignore + continue + } + } else { + rfsvs = []*FilesystemVersion{} + } + path, conflict := IncrementalPath(rfsvs, sfsvs) - if conflict != nil { + if noCommonAncestor, ok := conflict.(*ConflictNoCommonAncestor); ok { + if len(noCommonAncestor.SortedReceiverVersions) == 0 { + log.Printf("initial replication") + // FIXME hard-coded replication policy: most recent + // snapshot as source + var mostRecentSnap *FilesystemVersion + for n := len(sfsvs) -1; n >= 0; n-- { + if sfsvs[n].Type == FilesystemVersion_Snapshot { + mostRecentSnap = sfsvs[n] + break + } + } + if mostRecentSnap == nil { + log.Printf("no snapshot on sender side") + continue + } + log.Printf("starting at most recent snapshot %s", mostRecentSnap) + path = []*FilesystemVersion{mostRecentSnap} + } + } else if conflict != nil { + log.Printf("unresolvable conflict: %s", conflict) // handle or ignore for now continue } @@ -129,11 +162,11 @@ func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicat } type Sender interface { - Send(r SendRequest) (SendResponse, error) + Send(r *SendReq) (*SendRes, io.Reader, error) } type Receiver interface { - Receive(r ReceiveRequest) (io.Writer, error) + Receive(r *ReceiveReq, sendStream io.Reader) (error) } type Copier interface { @@ -151,7 +184,7 @@ func NewCopier() Copier { } type IncrementalPathReplicator interface { - Replicate(ctx context.Context, sender Sender, receiver Receiver, copier Copier, fs Filesystem, path []zfs.FilesystemVersion) + Replicate(ctx context.Context, sender Sender, receiver Receiver, copier Copier, fs *Filesystem, path []*FilesystemVersion) } type incrementalPathReplicator struct{} @@ -160,50 +193,82 @@ func NewIncrementalPathReplicator() IncrementalPathReplicator { return incrementalPathReplicator{} } -func (incrementalPathReplicator) Replicate(ctx context.Context, sender Sender, receiver Receiver, copier Copier, fs Filesystem, path []zfs.FilesystemVersion) { +func (incrementalPathReplicator) Replicate(ctx context.Context, sender Sender, receiver Receiver, copier Copier, fs *Filesystem, path []*FilesystemVersion) { + + log := ctx.Value(ContextKeyLog).(Logger) if len(path) == 0 { + log.Printf("nothing to do") // nothing to do return } + if len(path) == 1 { + log.Printf("full send of version %s", path[0]) + + sr := &SendReq{ + Filesystem: fs.Path, + From: path[0].RelName(), + ResumeToken: fs.ResumeToken, + } + sres, sstream, err := sender.Send(sr) + if err != nil { + log.Printf("send request failed: %s", err) + // FIXME must close connection... + return + } + + rr := &ReceiveReq{ + Filesystem: fs.Path, + ClearResumeToken: fs.ResumeToken != "" && !sres.UsedResumeToken, + } + err = receiver.Receive(rr, sstream) + if err != nil { + // FIXME this failure could be due to an unexpected exit of ZFS on the sending side + // FIXME which is transported through the streamrpc protocol, and known to the sendStream.(*streamrpc.streamReader), + // FIXME but the io.Reader interface design doesn not allow us to infer that it is a *streamrpc.streamReader right now + log.Printf("receive request failed (might also be error on sender...): %s", err) + // FIXME must close connection + return + } + + return + } + usedResumeToken := false incrementalLoop: for j := 0; j < len(path)-1; j++ { rt := "" - if !usedResumeToken { + if !usedResumeToken { // only send resume token for first increment rt = fs.ResumeToken usedResumeToken = true } - sr := SendRequest{ + sr := &SendReq{ Filesystem: fs.Path, - From: path[j].String(), - To: path[j+1].String(), + From: path[j].RelName(), + To: path[j+1].RelName(), ResumeToken: rt, } - sres, err := sender.Send(sr) + sres, sstream, err := sender.Send(sr) if err != nil { + log.Printf("send request failed: %s", err) // handle and ignore break incrementalLoop } // try to consume stream - rr := ReceiveRequest{ + rr := &ReceiveReq{ Filesystem: fs.Path, - ResumeToken: rt, + ClearResumeToken: rt != "" && !sres.UsedResumeToken, } - recvWriter, err := receiver.Receive(rr) - if err != nil { - // handle and ignore - break incrementalLoop - } - _, err = copier.Copy(recvWriter, sres.Stream) + err = receiver.Receive(rr, sstream) if err != nil { + log.Printf("receive request failed: %s", err) // handle and ignore break incrementalLoop } - // handle properties from sres + // FIXME handle properties from sres } } diff --git a/cmd/replication/replication_test.go b/cmd/replication/replication_test.go index 424c554..0334d02 100644 --- a/cmd/replication/replication_test.go +++ b/cmd/replication/replication_test.go @@ -4,17 +4,16 @@ import ( "context" "github.com/stretchr/testify/assert" "github.com/zrepl/zrepl/cmd/replication" - "github.com/zrepl/zrepl/zfs" "io" "testing" ) type IncrementalPathSequenceStep struct { - SendRequest replication.SendRequest - SendResponse replication.SendResponse + SendRequest *replication.SendReq + SendResponse *replication.SendRes + SendReader io.Reader SendError error - ReceiveRequest replication.ReceiveRequest - ReceiveWriter io.Writer + ReceiveRequest *replication.ReceiveReq ReceiveError error } @@ -24,7 +23,7 @@ type MockIncrementalPathRecorder struct { Pos int } -func (m *MockIncrementalPathRecorder) Receive(r replication.ReceiveRequest) (io.Writer, error) { +func (m *MockIncrementalPathRecorder) Receive(r *replication.ReceiveReq, rs io.Reader) (error) { if m.Pos >= len(m.Sequence) { m.T.Fatal("unexpected Receive") } @@ -33,10 +32,10 @@ func (m *MockIncrementalPathRecorder) Receive(r replication.ReceiveRequest) (io. if !assert.Equal(m.T, i.ReceiveRequest, r) { m.T.FailNow() } - return i.ReceiveWriter, i.ReceiveError + return i.ReceiveError } -func (m *MockIncrementalPathRecorder) Send(r replication.SendRequest) (replication.SendResponse, error) { +func (m *MockIncrementalPathRecorder) Send(r *replication.SendReq) (*replication.SendRes, io.Reader, error) { if m.Pos >= len(m.Sequence) { m.T.Fatal("unexpected Send") } @@ -45,7 +44,7 @@ func (m *MockIncrementalPathRecorder) Send(r replication.SendRequest) (replicati if !assert.Equal(m.T, i.SendRequest, r) { m.T.FailNow() } - return i.SendResponse, i.SendError + return i.SendResponse, i.SendReader, i.SendError } func (m *MockIncrementalPathRecorder) Finished() bool { @@ -60,8 +59,8 @@ func (DiscardCopier) Copy(writer io.Writer, reader io.Reader) (int64, error) { type IncrementalPathReplicatorTest struct { Msg string - Filesystem replication.Filesystem - Path []zfs.FilesystemVersion + Filesystem *replication.Filesystem + Path []*replication.FilesystemVersion Steps []IncrementalPathSequenceStep } @@ -74,9 +73,11 @@ func (test *IncrementalPathReplicatorTest) Test(t *testing.T) { Sequence: test.Steps, } + ctx := context.WithValue(context.Background(), replication.ContextKeyLog, testLog{t}) + ipr := replication.NewIncrementalPathReplicator() ipr.Replicate( - context.TODO(), + ctx, rec, rec, DiscardCopier{}, @@ -88,40 +89,51 @@ func (test *IncrementalPathReplicatorTest) Test(t *testing.T) { } +type testLog struct { + t *testing.T +} + +func (t testLog) Printf(fmt string, args ...interface{}) { + t.t.Logf(fmt, args) +} + func TestIncrementalPathReplicator_Replicate(t *testing.T) { tbl := []IncrementalPathReplicatorTest{ { Msg: "generic happy place with resume token", - Filesystem: replication.Filesystem{ + Filesystem: &replication.Filesystem{ Path: "foo/bar", ResumeToken: "blafoo", }, Path: fsvlist("@a,1", "@b,2", "@c,3"), Steps: []IncrementalPathSequenceStep{ { - SendRequest: replication.SendRequest{ + SendRequest: &replication.SendReq{ Filesystem: "foo/bar", From: "@a,1", To: "@b,2", ResumeToken: "blafoo", }, - }, - { - ReceiveRequest: replication.ReceiveRequest{ - Filesystem: "foo/bar", - ResumeToken: "blafoo", + SendResponse: &replication.SendRes{ + UsedResumeToken: true, }, }, { - SendRequest: replication.SendRequest{ + ReceiveRequest: &replication.ReceiveReq{ + Filesystem: "foo/bar", + ClearResumeToken: false, + }, + }, + { + SendRequest: &replication.SendReq{ Filesystem: "foo/bar", From: "@b,2", To: "@c,3", }, }, { - ReceiveRequest: replication.ReceiveRequest{ + ReceiveRequest: &replication.ReceiveReq{ Filesystem: "foo/bar", }, }, @@ -129,19 +141,36 @@ func TestIncrementalPathReplicator_Replicate(t *testing.T) { }, { Msg: "no action on empty sequence", - Filesystem: replication.Filesystem{ + Filesystem: &replication.Filesystem{ Path: "foo/bar", }, Path: fsvlist(), Steps: []IncrementalPathSequenceStep{}, }, { - Msg: "no action on invalid path", - Filesystem: replication.Filesystem{ + Msg: "full send on single entry path", + Filesystem: &replication.Filesystem{ Path: "foo/bar", }, Path: fsvlist("@justone,1"), - Steps: []IncrementalPathSequenceStep{}, + Steps: []IncrementalPathSequenceStep{ + { + SendRequest: &replication.SendReq{ + Filesystem: "foo/bar", + From: "@justone,1", + To: "", // empty means full send + }, + SendResponse: &replication.SendRes{ + UsedResumeToken: false, + }, + }, + { + ReceiveRequest: &replication.ReceiveReq{ + Filesystem: "foo/bar", + ClearResumeToken: false, + }, + }, + }, }, } diff --git a/rpc/client.go b/rpc/client.go deleted file mode 100644 index 262b9d1..0000000 --- a/rpc/client.go +++ /dev/null @@ -1,136 +0,0 @@ -package rpc - -import ( - "bytes" - "encoding/json" - "io" - "reflect" - - "github.com/pkg/errors" -) - -type Client struct { - ml *MessageLayer - logger Logger -} - -func NewClient(rwc io.ReadWriteCloser) *Client { - return &Client{NewMessageLayer(rwc), noLogger{}} -} - -func (c *Client) SetLogger(logger Logger, logMessageLayer bool) { - c.logger = logger - if logMessageLayer { - c.ml.logger = logger - } else { - c.ml.logger = noLogger{} - } -} - -func (c *Client) Close() (err error) { - - c.logger.Printf("sending Close request") - header := Header{ - DataType: DataTypeControl, - Endpoint: ControlEndpointClose, - Accept: DataTypeControl, - } - err = c.ml.WriteHeader(&header) - if err != nil { - return - } - - c.logger.Printf("reading Close ACK") - ack, err := c.ml.ReadHeader() - if err != nil { - return err - } - c.logger.Printf("received Close ACK: %#v", ack) - if ack.Error != StatusOK { - err = errors.Errorf("error hanging up: remote error (%s) %s", ack.Error, ack.ErrorMessage) - return - } - - c.logger.Printf("closing MessageLayer") - if err = c.ml.Close(); err != nil { - c.logger.Printf("error closing RWC: %+v", err) - return - } - - return err -} - -func (c *Client) recvResponse() (h *Header, err error) { - h, err = c.ml.ReadHeader() - if err != nil { - return nil, errors.Wrap(err, "cannot read header") - } - // TODO validate - return -} - -func (c *Client) writeRequest(h *Header) (err error) { - // TODO validate - err = c.ml.WriteHeader(h) - if err != nil { - return errors.Wrap(err, "cannot write header") - } - return -} - -func (c *Client) Call(endpoint string, in, out interface{}) (err error) { - - var accept DataType - { - outType := reflect.TypeOf(out) - if typeIsIOReaderPtr(outType) { - accept = DataTypeOctets - } else { - accept = DataTypeMarshaledJSON - } - } - - h := Header{ - Endpoint: endpoint, - DataType: DataTypeMarshaledJSON, - Accept: accept, - } - - if err = c.writeRequest(&h); err != nil { - return err - } - - var buf bytes.Buffer - if err = json.NewEncoder(&buf).Encode(in); err != nil { - panic("cannot encode 'in' parameter") - } - if err = c.ml.WriteData(&buf); err != nil { - return err - } - - rh, err := c.recvResponse() - if err != nil { - return err - } - if rh.Error != StatusOK { - return &RPCError{rh} - } - - rd := c.ml.ReadData() - - switch accept { - case DataTypeOctets: - c.logger.Printf("setting out to ML data reader") - outPtr := out.(*io.Reader) // we checked that above - *outPtr = rd - case DataTypeMarshaledJSON: - c.logger.Printf("decoding marshaled json") - if err = json.NewDecoder(c.ml.ReadData()).Decode(out); err != nil { - return errors.Wrap(err, "cannot decode marshaled reply") - } - default: - panic("implementation error") // accept is controlled by us - } - - return -} diff --git a/rpc/datatype_string.go b/rpc/datatype_string.go deleted file mode 100644 index e7293b7..0000000 --- a/rpc/datatype_string.go +++ /dev/null @@ -1,17 +0,0 @@ -// Code generated by "stringer -type=DataType"; DO NOT EDIT. - -package rpc - -import "strconv" - -const _DataType_name = "DataTypeNoneDataTypeControlDataTypeMarshaledJSONDataTypeOctets" - -var _DataType_index = [...]uint8{0, 12, 27, 48, 62} - -func (i DataType) String() string { - i -= 1 - if i >= DataType(len(_DataType_index)-1) { - return "DataType(" + strconv.FormatInt(int64(i+1), 10) + ")" - } - return _DataType_name[_DataType_index[i]:_DataType_index[i+1]] -} diff --git a/rpc/frame_layer.go b/rpc/frame_layer.go deleted file mode 100644 index 9d6327a..0000000 --- a/rpc/frame_layer.go +++ /dev/null @@ -1,302 +0,0 @@ -package rpc - -import ( - "bytes" - "encoding/binary" - "encoding/json" - "fmt" - "io" - - "github.com/pkg/errors" -) - -type Frame struct { - Type FrameType - NoMoreFrames bool - PayloadLength uint32 -} - -//go:generate stringer -type=FrameType -type FrameType uint8 - -const ( - FrameTypeHeader FrameType = 0x01 - FrameTypeData FrameType = 0x02 - FrameTypeTrailer FrameType = 0x03 - FrameTypeRST FrameType = 0xff -) - -//go:generate stringer -type=Status -type Status uint64 - -const ( - StatusOK Status = 1 + iota - StatusRequestError - StatusServerError - // Returned when an error occurred but the side at fault cannot be determined - StatusError -) - -type Header struct { - // Request-only - Endpoint string - // Data type of body (request & reply) - DataType DataType - // Request-only - Accept DataType - // Reply-only - Error Status - // Reply-only - ErrorMessage string -} - -func NewErrorHeader(status Status, format string, args ...interface{}) (h *Header) { - h = &Header{} - h.Error = status - h.ErrorMessage = fmt.Sprintf(format, args...) - return -} - -//go:generate stringer -type=DataType -type DataType uint8 - -const ( - DataTypeNone DataType = 1 + iota - DataTypeControl - DataTypeMarshaledJSON - DataTypeOctets -) - -const ( - MAX_PAYLOAD_LENGTH = 4 * 1024 * 1024 - MAX_HEADER_LENGTH = 4 * 1024 -) - -type frameBridgingReader struct { - l *MessageLayer - frameType FrameType - // < 0 means no limit - bytesLeftToLimit int - f Frame -} - -func NewFrameBridgingReader(l *MessageLayer, frameType FrameType, totalLimit int) *frameBridgingReader { - return &frameBridgingReader{l, frameType, totalLimit, Frame{}} -} - -func (r *frameBridgingReader) Read(b []byte) (n int, err error) { - if r.bytesLeftToLimit == 0 { - r.l.logger.Printf("limit reached, returning EOF") - return 0, io.EOF - } - log := r.l.logger - if r.f.PayloadLength == 0 { - - if r.f.NoMoreFrames { - r.l.logger.Printf("no more frames flag set, returning EOF") - err = io.EOF - return - } - - log.Printf("reading frame") - r.f, err = r.l.readFrame() - if err != nil { - log.Printf("error reading frame: %+v", err) - return 0, err - } - log.Printf("read frame: %#v", r.f) - if r.f.Type != r.frameType { - err = errors.Wrapf(err, "expected frame of type %s", r.frameType) - return 0, err - } - } - maxread := len(b) - if maxread > int(r.f.PayloadLength) { - maxread = int(r.f.PayloadLength) - } - if r.bytesLeftToLimit > 0 && maxread > r.bytesLeftToLimit { - maxread = r.bytesLeftToLimit - } - nb, err := r.l.rwc.Read(b[:maxread]) - log.Printf("read %v from rwc\n", nb) - if nb < 0 { - panic("should not return negative number of bytes") - } - r.f.PayloadLength -= uint32(nb) - r.bytesLeftToLimit -= nb - return nb, err // TODO io.EOF for maxread = r.f.PayloadLength ? -} - -type frameBridgingWriter struct { - l *MessageLayer - frameType FrameType - // < 0 means no limit - bytesLeftToLimit int - payloadLength int - buffer *bytes.Buffer -} - -func NewFrameBridgingWriter(l *MessageLayer, frameType FrameType, totalLimit int) *frameBridgingWriter { - return &frameBridgingWriter{l, frameType, totalLimit, MAX_PAYLOAD_LENGTH, bytes.NewBuffer(make([]byte, 0, MAX_PAYLOAD_LENGTH))} -} - -func (w *frameBridgingWriter) Write(b []byte) (n int, err error) { - for n = 0; n < len(b); { - i, err := w.writeUntilFrameFull(b[n:]) - n += i - if err != nil { - return n, errors.WithStack(err) - } - } - return -} - -func (w *frameBridgingWriter) writeUntilFrameFull(b []byte) (n int, err error) { - if len(b) <= 0 { - return - } - if w.bytesLeftToLimit == 0 { - err = errors.Errorf("message exceeds max number of allowed bytes") - return - } - maxwrite := len(b) - remainingInFrame := w.payloadLength - w.buffer.Len() - - if maxwrite > remainingInFrame { - maxwrite = remainingInFrame - } - if w.bytesLeftToLimit > 0 && maxwrite > w.bytesLeftToLimit { - maxwrite = w.bytesLeftToLimit - } - w.buffer.Write(b[:maxwrite]) - w.bytesLeftToLimit -= maxwrite - n = maxwrite - if w.bytesLeftToLimit == 0 { - err = w.flush(true) - } else if w.buffer.Len() == w.payloadLength { - err = w.flush(false) - } - return -} - -func (w *frameBridgingWriter) flush(nomore bool) (err error) { - - f := Frame{w.frameType, nomore, uint32(w.buffer.Len())} - err = w.l.writeFrame(f) - if err != nil { - errors.WithStack(err) - } - _, err = w.buffer.WriteTo(w.l.rwc) - return -} - -func (w *frameBridgingWriter) Close() (err error) { - return w.flush(true) -} - -type MessageLayer struct { - rwc io.ReadWriteCloser - logger Logger -} - -func NewMessageLayer(rwc io.ReadWriteCloser) *MessageLayer { - return &MessageLayer{rwc, noLogger{}} -} - -func (l *MessageLayer) Close() (err error) { - f := Frame{ - Type: FrameTypeRST, - NoMoreFrames: true, - } - if err = l.writeFrame(f); err != nil { - l.logger.Printf("error sending RST frame: %s", err) - return errors.WithStack(err) - } - return nil -} - -var RST error = fmt.Errorf("reset frame observed on connection") - -func (l *MessageLayer) readFrame() (f Frame, err error) { - err = binary.Read(l.rwc, binary.LittleEndian, &f.Type) - if err != nil { - err = errors.WithStack(err) - return - } - err = binary.Read(l.rwc, binary.LittleEndian, &f.NoMoreFrames) - if err != nil { - err = errors.WithStack(err) - return - } - err = binary.Read(l.rwc, binary.LittleEndian, &f.PayloadLength) - if err != nil { - err = errors.WithStack(err) - return - } - if f.Type == FrameTypeRST { - l.logger.Printf("read RST frame") - err = RST - return - } - if f.PayloadLength > MAX_PAYLOAD_LENGTH { - err = errors.Errorf("frame exceeds max payload length") - return - } - return -} - -func (l *MessageLayer) writeFrame(f Frame) (err error) { - err = binary.Write(l.rwc, binary.LittleEndian, &f.Type) - if err != nil { - return errors.WithStack(err) - } - err = binary.Write(l.rwc, binary.LittleEndian, &f.NoMoreFrames) - if err != nil { - return errors.WithStack(err) - } - err = binary.Write(l.rwc, binary.LittleEndian, &f.PayloadLength) - if err != nil { - return errors.WithStack(err) - } - if f.PayloadLength > MAX_PAYLOAD_LENGTH { - err = errors.Errorf("frame exceeds max payload length") - return - } - return -} - -func (l *MessageLayer) ReadHeader() (h *Header, err error) { - - r := NewFrameBridgingReader(l, FrameTypeHeader, MAX_HEADER_LENGTH) - h = &Header{} - if err = json.NewDecoder(r).Decode(&h); err != nil { - l.logger.Printf("cannot decode marshaled header: %s", err) - return nil, err - } - return h, nil -} - -func (l *MessageLayer) WriteHeader(h *Header) (err error) { - w := NewFrameBridgingWriter(l, FrameTypeHeader, MAX_HEADER_LENGTH) - err = json.NewEncoder(w).Encode(h) - if err != nil { - return errors.Wrap(err, "cannot encode header, probably fatal") - } - w.Close() - return -} - -func (l *MessageLayer) ReadData() (reader io.Reader) { - r := NewFrameBridgingReader(l, FrameTypeData, -1) - return r -} - -func (l *MessageLayer) WriteData(source io.Reader) (err error) { - w := NewFrameBridgingWriter(l, FrameTypeData, -1) - _, err = io.Copy(w, source) - if err != nil { - return errors.WithStack(err) - } - err = w.Close() - return -} diff --git a/rpc/frametype_string.go b/rpc/frametype_string.go deleted file mode 100644 index 9e3ced3..0000000 --- a/rpc/frametype_string.go +++ /dev/null @@ -1,26 +0,0 @@ -// Code generated by "stringer -type=FrameType"; DO NOT EDIT. - -package rpc - -import "strconv" - -const ( - _FrameType_name_0 = "FrameTypeHeaderFrameTypeDataFrameTypeTrailer" - _FrameType_name_1 = "FrameTypeRST" -) - -var ( - _FrameType_index_0 = [...]uint8{0, 15, 28, 44} -) - -func (i FrameType) String() string { - switch { - case 1 <= i && i <= 3: - i -= 1 - return _FrameType_name_0[_FrameType_index_0[i]:_FrameType_index_0[i+1]] - case i == 255: - return _FrameType_name_1 - default: - return "FrameType(" + strconv.FormatInt(int64(i), 10) + ")" - } -} diff --git a/rpc/local.go b/rpc/local.go deleted file mode 100644 index 4f729d7..0000000 --- a/rpc/local.go +++ /dev/null @@ -1,63 +0,0 @@ -package rpc - -import ( - "github.com/pkg/errors" - "reflect" -) - -type LocalRPC struct { - endpoints map[string]reflect.Value -} - -func NewLocalRPC() *LocalRPC { - return &LocalRPC{make(map[string]reflect.Value, 0)} -} - -func (s *LocalRPC) RegisterEndpoint(name string, handler interface{}) (err error) { - _, ok := s.endpoints[name] - if ok { - return errors.Errorf("already set up an endpoint for '%s'", name) - } - ep, err := makeEndpointDescr(handler) - if err != nil { - return err - } - s.endpoints[name] = ep.handler - return nil -} - -func (s *LocalRPC) Serve() (err error) { - panic("local cannot serve") -} - -func (c *LocalRPC) Call(endpoint string, in, out interface{}) (err error) { - ep, ok := c.endpoints[endpoint] - if !ok { - panic("implementation error: implementation should not call local RPC without knowing which endpoints exist") - } - - args := []reflect.Value{reflect.ValueOf(in), reflect.ValueOf(out)} - - if err = checkRPCParamTypes(args[0].Type(), args[1].Type()); err != nil { - return - } - - rets := ep.Call(args) - - if len(rets) != 1 { - panic("implementation error: endpoints must have one error ") - } - if err = checkRPCReturnType(rets[0].Type()); err != nil { - panic(err) - } - - err = nil - if !rets[0].IsNil() { - err = rets[0].Interface().(error) // we checked that above - } - return -} - -func (c *LocalRPC) Close() (err error) { - return nil -} diff --git a/rpc/server.go b/rpc/server.go deleted file mode 100644 index 7d7ee16..0000000 --- a/rpc/server.go +++ /dev/null @@ -1,259 +0,0 @@ -package rpc - -import ( - "bytes" - "encoding/json" - "io" - "reflect" - - "github.com/pkg/errors" -) - -type Server struct { - ml *MessageLayer - logger Logger - endpoints map[string]endpointDescr -} - -type typeMap struct { - local reflect.Type - proto DataType -} -type endpointDescr struct { - inType typeMap - outType typeMap - handler reflect.Value -} - -type MarshaledJSONEndpoint func(bodyJSON interface{}) - -func NewServer(rwc io.ReadWriteCloser) *Server { - ml := NewMessageLayer(rwc) - return &Server{ - ml, noLogger{}, make(map[string]endpointDescr), - } -} - -func (s *Server) SetLogger(logger Logger, logMessageLayer bool) { - s.logger = logger - if logMessageLayer { - s.ml.logger = logger - } else { - s.ml.logger = noLogger{} - } -} - -func (s *Server) RegisterEndpoint(name string, handler interface{}) (err error) { - _, ok := s.endpoints[name] - if ok { - return errors.Errorf("already set up an endpoint for '%s'", name) - } - s.endpoints[name], err = makeEndpointDescr(handler) - return -} - -func checkResponseHeader(h *Header) (err error) { - var statusNotSet Status - if h.Error == statusNotSet { - return errors.Errorf("status has zero-value") - } - return nil -} - -func (s *Server) writeResponse(h *Header) (err error) { - // TODO validate - return s.ml.WriteHeader(h) -} - -func (s *Server) recvRequest() (h *Header, err error) { - h, err = s.ml.ReadHeader() - if err != nil { - s.logger.Printf("error reading header: %s", err) - return nil, err - } - - s.logger.Printf("validating request") - err = nil // TODO validate - if err == nil { - return h, nil - } - s.logger.Printf("request validation error: %s", err) - - r := NewErrorHeader(StatusRequestError, "%s", err) - return nil, s.writeResponse(r) -} - -var doneServeNext error = errors.New("this should not cause a HangUp() in the server") -var doneStopServing error = errors.New("this should cause the server to close the connection") - -var ProtocolError error = errors.New("protocol error, server should hang up") - -const ControlEndpointClose string = "Close" - -// Serve the connection until failure or the client hangs up -func (s *Server) Serve() (err error) { - for { - - err = s.ServeRequest() - - if err == nil { - continue - } - if err == doneServeNext { - s.logger.Printf("subroutine returned pseudo-error indicating early-exit") - err = nil - continue - } - - if err == doneStopServing { - s.logger.Printf("subroutine returned pseudo-error indicating close request") - err = nil - break - } - - break - } - - if err != nil { - s.logger.Printf("an error occurred that could not be handled on PRC protocol level: %+v", err) - } - - s.logger.Printf("cloing MessageLayer") - if mlErr := s.ml.Close(); mlErr != nil { - s.logger.Printf("error closing MessageLayer: %+v", mlErr) - } - - return err -} - -// Serve a single request -// * wait for request to come in -// * call handler -// * reply -// -// The connection is left open, the next bytes on the conn should be -// the next request header. -// -// Returns an err != nil if the error is bad enough to hang up on the client. -// Examples: protocol version mismatches, protocol errors in general, ... -// Non-Examples: a handler error -func (s *Server) ServeRequest() (err error) { - - ml := s.ml - - s.logger.Printf("reading header") - h, err := s.recvRequest() - if err != nil { - return err - } - - if h.DataType == DataTypeControl { - switch h.Endpoint { - case ControlEndpointClose: - ack := Header{Error: StatusOK, DataType: DataTypeControl} - err = s.writeResponse(&ack) - if err != nil { - return err - } - return doneStopServing - default: - r := NewErrorHeader(StatusRequestError, "unregistered control endpoint %s", h.Endpoint) - return s.writeResponse(r) - } - } - - ep, ok := s.endpoints[h.Endpoint] - if !ok { - r := NewErrorHeader(StatusRequestError, "unregistered endpoint %s", h.Endpoint) - return s.writeResponse(r) - } - - if ep.inType.proto != h.DataType { - r := NewErrorHeader(StatusRequestError, "wrong DataType for endpoint %s (has %s, you provided %s)", h.Endpoint, ep.inType.proto, h.DataType) - return s.writeResponse(r) - } - - if ep.outType.proto != h.Accept { - r := NewErrorHeader(StatusRequestError, "wrong Accept for endpoint %s (has %s, you provided %s)", h.Endpoint, ep.outType.proto, h.Accept) - return s.writeResponse(r) - } - - dr := ml.ReadData() - - // Determine inval - var inval reflect.Value - switch ep.inType.proto { - case DataTypeMarshaledJSON: - // Unmarshal input - inval = reflect.New(ep.inType.local.Elem()) - invalIface := inval.Interface() - err = json.NewDecoder(dr).Decode(invalIface) - if err != nil { - r := NewErrorHeader(StatusRequestError, "cannot decode marshaled JSON: %s", err) - return s.writeResponse(r) - } - case DataTypeOctets: - // Take data as is - inval = reflect.ValueOf(dr) - default: - panic("not implemented") - } - - outval := reflect.New(ep.outType.local.Elem()) // outval is a double pointer - - s.logger.Printf("before handler, inval=%v outval=%v", inval, outval) - - // Call the handler - errs := ep.handler.Call([]reflect.Value{inval, outval}) - - if !errs[0].IsNil() { - he := errs[0].Interface().(error) // we checked that before... - s.logger.Printf("handler returned error: %s", err) - r := NewErrorHeader(StatusError, "%s", he.Error()) - return s.writeResponse(r) - } - - switch ep.outType.proto { - - case DataTypeMarshaledJSON: - - var dataBuf bytes.Buffer - // Marshal output - err = json.NewEncoder(&dataBuf).Encode(outval.Interface()) - if err != nil { - r := NewErrorHeader(StatusServerError, "cannot marshal response: %s", err) - return s.writeResponse(r) - } - - replyHeader := Header{ - Error: StatusOK, - DataType: ep.outType.proto, - } - if err = s.writeResponse(&replyHeader); err != nil { - return err - } - - if err = ml.WriteData(&dataBuf); err != nil { - return - } - - case DataTypeOctets: - - h := Header{ - Error: StatusOK, - DataType: DataTypeOctets, - } - if err = s.writeResponse(&h); err != nil { - return - } - - reader := outval.Interface().(*io.Reader) // we checked that when adding the endpoint - err = ml.WriteData(*reader) - if err != nil { - return err - } - - } - - return nil -} diff --git a/rpc/shared.go b/rpc/shared.go deleted file mode 100644 index 052f441..0000000 --- a/rpc/shared.go +++ /dev/null @@ -1,111 +0,0 @@ -package rpc - -import ( - "fmt" - "github.com/pkg/errors" - "io" - "reflect" -) - -type RPCServer interface { - Serve() (err error) - RegisterEndpoint(name string, handler interface{}) (err error) -} - -type RPCClient interface { - Call(endpoint string, in, out interface{}) (err error) - Close() (err error) -} - -type Logger interface { - Printf(format string, args ...interface{}) -} - -type noLogger struct{} - -func (l noLogger) Printf(format string, args ...interface{}) {} -func typeIsIOReader(t reflect.Type) bool { - return t == reflect.TypeOf((*io.Reader)(nil)).Elem() -} - -func typeIsIOReaderPtr(t reflect.Type) bool { - return t == reflect.TypeOf((*io.Reader)(nil)) -} - -// An error returned by the Client if the response indicated a status code other than StatusOK -type RPCError struct { - ResponseHeader *Header -} - -func (e *RPCError) Error() string { - return fmt.Sprintf("%s: %s", e.ResponseHeader.Error, e.ResponseHeader.ErrorMessage) -} - -type RPCProtoError struct { - Message string - UnderlyingError error -} - -func (e *RPCProtoError) Error() string { - return e.Message -} - -func checkRPCParamTypes(in, out reflect.Type) (err error) { - if !(in.Kind() == reflect.Ptr || typeIsIOReader(in)) { - err = errors.Errorf("input parameter must be a pointer or an io.Reader, is of kind %s, type %s", in.Kind(), in) - return - } - if !(out.Kind() == reflect.Ptr) { - err = errors.Errorf("second input parameter (the non-error output parameter) must be a pointer or an *io.Reader") - return - } - return nil -} - -func checkRPCReturnType(rt reflect.Type) (err error) { - errInterfaceType := reflect.TypeOf((*error)(nil)).Elem() - if !rt.Implements(errInterfaceType) { - err = errors.Errorf("handler must return an error") - return - } - return nil -} - -func makeEndpointDescr(handler interface{}) (descr endpointDescr, err error) { - - ht := reflect.TypeOf(handler) - - if ht.Kind() != reflect.Func { - err = errors.Errorf("handler must be of kind reflect.Func") - return - } - - if ht.NumIn() != 2 || ht.NumOut() != 1 { - err = errors.Errorf("handler must have exactly two input parameters and one output parameter") - return - } - if err = checkRPCParamTypes(ht.In(0), ht.In(1)); err != nil { - return - } - if err = checkRPCReturnType(ht.Out(0)); err != nil { - return - } - - descr.handler = reflect.ValueOf(handler) - descr.inType.local = ht.In(0) - descr.outType.local = ht.In(1) - - if typeIsIOReader(ht.In(0)) { - descr.inType.proto = DataTypeOctets - } else { - descr.inType.proto = DataTypeMarshaledJSON - } - - if typeIsIOReaderPtr(ht.In(1)) { - descr.outType.proto = DataTypeOctets - } else { - descr.outType.proto = DataTypeMarshaledJSON - } - - return -} diff --git a/rpc/status_string.go b/rpc/status_string.go deleted file mode 100644 index 10e801d..0000000 --- a/rpc/status_string.go +++ /dev/null @@ -1,17 +0,0 @@ -// Code generated by "stringer -type=Status"; DO NOT EDIT. - -package rpc - -import "strconv" - -const _Status_name = "StatusOKStatusRequestErrorStatusServerErrorStatusError" - -var _Status_index = [...]uint8{0, 8, 26, 43, 54} - -func (i Status) String() string { - i -= 1 - if i >= Status(len(_Status_index)-1) { - return "Status(" + strconv.FormatInt(int64(i+1), 10) + ")" - } - return _Status_name[_Status_index[i]:_Status_index[i+1]] -} diff --git a/util/iocommand.go b/util/iocommand.go index 44671da..406e98f 100644 --- a/util/iocommand.go +++ b/util/iocommand.go @@ -11,8 +11,8 @@ import ( // An IOCommand exposes a forked process's std(in|out|err) through the io.ReadWriteCloser interface. type IOCommand struct { Cmd *exec.Cmd - Stdin io.Writer - Stdout io.Reader + Stdin io.WriteCloser + Stdout io.ReadCloser StderrBuf *bytes.Buffer ExitResult *IOCommandExitResult } diff --git a/zfs/diff.go b/zfs/diff.go index 12f9174..c3564a1 100644 --- a/zfs/diff.go +++ b/zfs/diff.go @@ -7,6 +7,7 @@ import ( "fmt" "os/exec" "sort" + "io" ) type fsbyCreateTXG []FilesystemVersion @@ -245,6 +246,19 @@ func IsPlaceholder(p *DatasetPath, placeholderPropertyValue string) (isPlacehold return } +// for nonexistent FS, isPlaceholder == false && err == nil +func ZFSIsPlaceholderFilesystem(p *DatasetPath) (isPlaceholder bool, err error) { + props, err := ZFSGet(p, []string{ZREPL_PLACEHOLDER_PROPERTY_NAME}) + if err == io.ErrUnexpectedEOF { + // interpret this as an early exit of the zfs binary due to the fs not existing + return false, nil + } else if err != nil { + return false, err + } + isPlaceholder, _ = IsPlaceholder(p, props.Get(ZREPL_PLACEHOLDER_PROPERTY_NAME)) + return +} + func ZFSCreatePlaceholderFilesystem(p *DatasetPath) (err error) { v := PlaceholderPropertyValue(p) cmd := exec.Command(ZFS_BINARY, "create", diff --git a/zfs/mapping.go b/zfs/mapping.go index 4ccfc70..8fcbd9a 100644 --- a/zfs/mapping.go +++ b/zfs/mapping.go @@ -10,26 +10,56 @@ type DatasetFilter interface { } func ZFSListMapping(filter DatasetFilter) (datasets []*DatasetPath, err error) { + res, err := ZFSListMappingProperties(filter, nil) + if err != nil { + return nil, err + } + datasets = make([]*DatasetPath, len(res)) + for i, r := range res { + datasets[i] = r.Path + } + return datasets, nil +} + +type ZFSListMappingPropertiesResult struct { + Path *DatasetPath + // Guaranteed to have the same length as properties in the originating call + Fields []string +} + +// properties must not contain 'name' +func ZFSListMappingProperties(filter DatasetFilter, properties []string) (datasets []ZFSListMappingPropertiesResult, err error) { if filter == nil { panic("filter must not be nil") } + for _, p := range properties { + if p == "name" { + panic("properties must not contain 'name'") + } + } + newProps := make([]string, len(properties) + 1) + newProps[0] = "name" + copy(newProps[1:], properties) + properties = newProps + ctx, cancel := context.WithCancel(context.Background()) defer cancel() rchan := make(chan ZFSListResult) - go ZFSListChan(ctx, rchan, []string{"name"}, "-r", "-t", "filesystem,volume") - datasets = make([]*DatasetPath, 0) + go ZFSListChan(ctx, rchan, properties, "-r", "-t", "filesystem,volume") + + datasets = make([]ZFSListMappingPropertiesResult, 0) for r := range rchan { - if r.err != nil { - err = r.err + if r.Err != nil { + err = r.Err return } var path *DatasetPath - if path, err = NewDatasetPath(r.fields[0]); err != nil { + if path, err = NewDatasetPath(r.Fields[0]); err != nil { return } @@ -38,10 +68,15 @@ func ZFSListMapping(filter DatasetFilter) (datasets []*DatasetPath, err error) { return nil, fmt.Errorf("error calling filter: %s", filterErr) } if pass { - datasets = append(datasets, path) + datasets = append(datasets, ZFSListMappingPropertiesResult{ + Path: path, + Fields: r.Fields[1:], + }) } } return } + + diff --git a/zfs/resume_token.go b/zfs/resume_token.go index 4423626..5bf1632 100644 --- a/zfs/resume_token.go +++ b/zfs/resume_token.go @@ -103,3 +103,18 @@ func ParseResumeToken(ctx context.Context, token string) (*ResumeToken, error) { return rt, nil } + +func ZFSGetReceiveResumeToken(fs *DatasetPath) (string, error) { + const prop_receive_resume_token = "receive_resume_token" + props, err := ZFSGet(fs, []string{prop_receive_resume_token}) + if err != nil { + return "", err + } + res := props.m[prop_receive_resume_token] + if res == "-" { + return "", nil + } else { + return res, nil + } + +} diff --git a/zfs/versions.go b/zfs/versions.go index 1ef9e20..552af35 100644 --- a/zfs/versions.go +++ b/zfs/versions.go @@ -9,6 +9,7 @@ import ( "strconv" "strings" "time" + "io" ) type VersionType string @@ -33,6 +34,26 @@ func (t VersionType) String() string { return string(t) } +func DecomposeVersionString(v string) (fs string, versionType VersionType, name string, err error) { + if len(v) < 3 { + err = errors.New(fmt.Sprintf("snapshot or bookmark name implausibly short: %s", v)) + return + } + + snapSplit := strings.SplitN(v, "@", 2) + bookmarkSplit := strings.SplitN(v, "#", 2) + if len(snapSplit)*len(bookmarkSplit) != 2 { + err = errors.New(fmt.Sprintf("dataset cannot be snapshot and bookmark at the same time: %s", v)) + return + } + + if len(snapSplit) == 2 { + return snapSplit[0], Snapshot, snapSplit[1], nil + } else { + return bookmarkSplit[0], Bookmark, bookmarkSplit[1], nil + } +} + type FilesystemVersion struct { Type VersionType @@ -63,7 +84,7 @@ func (v FilesystemVersion) ToAbsPath(p *DatasetPath) string { } type FilesystemVersionFilter interface { - Filter(fsv FilesystemVersion) (accept bool, err error) + Filter(t VersionType, name string) (accept bool, err error) } func ZFSListFilesystemVersions(fs *DatasetPath, filter FilesystemVersionFilter) (res []FilesystemVersion, err error) { @@ -82,31 +103,21 @@ func ZFSListFilesystemVersions(fs *DatasetPath, filter FilesystemVersionFilter) res = make([]FilesystemVersion, 0) for listResult := range listResults { - if listResult.err != nil { - return nil, listResult.err + if listResult.Err != nil { + if listResult.Err == io.ErrUnexpectedEOF { + // Since we specified the fs on the command line, we'll treat this like the filesystem doesn't exist + return []FilesystemVersion{}, nil + } + return nil, listResult.Err } - line := listResult.fields - - if len(line[0]) < 3 { - err = errors.New(fmt.Sprintf("snapshot or bookmark name implausibly short: %s", line[0])) - return - } - - snapSplit := strings.SplitN(line[0], "@", 2) - bookmarkSplit := strings.SplitN(line[0], "#", 2) - if len(snapSplit)*len(bookmarkSplit) != 2 { - err = errors.New(fmt.Sprintf("dataset cannot be snapshot and bookmark at the same time: %s", line[0])) - return - } + line := listResult.Fields var v FilesystemVersion - if len(snapSplit) == 2 { - v.Name = snapSplit[1] - v.Type = Snapshot - } else { - v.Name = bookmarkSplit[1] - v.Type = Bookmark + + _, v.Type, v.Name, err = DecomposeVersionString(line[0]) + if err != nil { + return nil, err } if v.Guid, err = strconv.ParseUint(line[1], 10, 64); err != nil { @@ -129,7 +140,7 @@ func ZFSListFilesystemVersions(fs *DatasetPath, filter FilesystemVersionFilter) accept := true if filter != nil { - accept, err = filter.Filter(v) + accept, err = filter.Filter(v.Type, v.Name) if err != nil { err = fmt.Errorf("error executing filter: %s", err) return nil, err diff --git a/zfs/zfs.go b/zfs/zfs.go index 29da261..b9557fe 100644 --- a/zfs/zfs.go +++ b/zfs/zfs.go @@ -189,13 +189,13 @@ func ZFSList(properties []string, zfsArgs ...string) (res [][]string, err error) } type ZFSListResult struct { - fields []string - err error + Fields []string + Err error } // ZFSListChan executes `zfs list` and sends the results to the `out` channel. // The `out` channel is always closed by ZFSListChan: -// If an error occurs, it is closed after sending a result with the err field set. +// If an error occurs, it is closed after sending a result with the Err field set. // If no error occurs, it is just closed. // If the operation is cancelled via context, the channel is just closed. // @@ -250,15 +250,56 @@ func ZFSListChan(ctx context.Context, out chan ZFSListResult, properties []strin return } -func ZFSSend(fs *DatasetPath, from, to *FilesystemVersion) (stream io.Reader, err error) { +func validateRelativeZFSVersion(s string) error { + if len(s) <= 1 { + return errors.New("version must start with a delimiter char followed by at least one character") + } + if !(s[0] == '#' || s[0] == '@') { + return errors.New("version name starts with invalid delimiter char") + } + // FIXME whitespace check... + return nil +} + +func validateZFSFilesystem(fs string) error { + if len(fs) < 1 { + return errors.New("filesystem path must have length > 0") + } + return nil +} + +func absVersion(fs, v string) (full string, err error) { + if err := validateZFSFilesystem(fs); err != nil { + return "", err + } + if err := validateRelativeZFSVersion(v); err != nil { + return "", err + } + return fmt.Sprintf("%s%s", fs, v), nil +} + +func ZFSSend(fs string, from, to string) (stream io.Reader, err error) { + + fromV, err := absVersion(fs, from) + if err != nil { + return nil, err + } + + toV := "" + if to != "" { + toV, err = absVersion(fs, to) + if err != nil { + return nil, err + } + } args := make([]string, 0) args = append(args, "send") - if to == nil { // Initial - args = append(args, from.ToAbsPath(fs)) + if toV == "" { // Initial + args = append(args, fromV) } else { - args = append(args, "-i", from.ToAbsPath(fs), to.ToAbsPath(fs)) + args = append(args, "-i", fromV, toV) } stream, err = util.RunIOCommand(ZFS_BINARY, args...) @@ -266,14 +307,18 @@ func ZFSSend(fs *DatasetPath, from, to *FilesystemVersion) (stream io.Reader, er return } -func ZFSRecv(fs *DatasetPath, stream io.Reader, additionalArgs ...string) (err error) { +func ZFSRecv(fs string, stream io.Reader, additionalArgs ...string) (err error) { + + if err := validateZFSFilesystem(fs); err != nil { + return err + } args := make([]string, 0) args = append(args, "recv") if len(args) > 0 { args = append(args, additionalArgs...) } - args = append(args, fs.ToString()) + args = append(args, fs) cmd := exec.Command(ZFS_BINARY, args...) @@ -304,6 +349,27 @@ func ZFSRecv(fs *DatasetPath, stream io.Reader, additionalArgs ...string) (err e return nil } +func ZFSRecvWriter(fs *DatasetPath, additionalArgs ...string) (io.WriteCloser, error) { + + args := make([]string, 0) + args = append(args, "recv") + if len(args) > 0 { + args = append(args, additionalArgs...) + } + args = append(args, fs.ToString()) + + cmd, err := util.NewIOCommand(ZFS_BINARY, args, 1024) + if err != nil { + return nil, err + } + + if err = cmd.Start(); err != nil { + return nil, err + } + + return cmd.Stdin, nil +} + type ZFSProperties struct { m map[string]string } @@ -316,6 +382,10 @@ func (p *ZFSProperties) Set(key, val string) { p.m[key] = val } +func (p *ZFSProperties) Get(key string) string { + return p.m[key] +} + func (p *ZFSProperties) appendArgs(args *[]string) (err error) { for prop, val := range p.m { if strings.Contains(prop, "=") { @@ -355,6 +425,33 @@ func ZFSSet(fs *DatasetPath, props *ZFSProperties) (err error) { return } +func ZFSGet(fs *DatasetPath, props []string) (*ZFSProperties, error) { + args := []string{"get", "-Hp", "-o", "property,value", strings.Join(props, ","), fs.ToString()} + + cmd := exec.Command(ZFS_BINARY, args...) + output, err := cmd.CombinedOutput() + if err != nil { + return nil, err + } + o := string(output) + lines := strings.Split(o, "\n") + if len(lines) < 1 || // account for newlines + len(lines)-1 != len(props) { + return nil, fmt.Errorf("zfs get did not return the number of expected property values") + } + res := &ZFSProperties{ + make(map[string]string, len(lines)), + } + for _, line := range lines[:len(lines)-1] { + fields := strings.Fields(line) + if len(fields) != 2 { + return nil, fmt.Errorf("zfs get did not return property value pairs") + } + res.m[fields[0]] = fields[1] + } + return res, nil +} + func ZFSDestroy(dataset string) (err error) { cmd := exec.Command(ZFS_BINARY, "destroy", dataset) From 1a8d2c5ebe9acef8a12b89032446d445b030eed4 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 8 Jul 2018 23:31:46 +0200 Subject: [PATCH 004/167] replication: context support and propert closing of stale readers --- cmd/replication.go | 63 ++++++++++++++++------------- cmd/replication/replication.go | 27 +++++++------ cmd/replication/replication_test.go | 6 +-- zfs/zfs.go | 2 +- 4 files changed, 54 insertions(+), 44 deletions(-) diff --git a/cmd/replication.go b/cmd/replication.go index ee0e47a..d333056 100644 --- a/cmd/replication.go +++ b/cmd/replication.go @@ -10,6 +10,7 @@ import ( "github.com/golang/protobuf/proto" "bytes" "os" + "context" ) type InitialReplPolicy string @@ -31,7 +32,7 @@ func NewSenderEndpoint(fsf zfs.DatasetFilter, fsvf zfs.FilesystemVersionFilter) return &SenderEndpoint{fsf, fsvf} } -func (p *SenderEndpoint) ListFilesystems() ([]*replication.Filesystem, error) { +func (p *SenderEndpoint) ListFilesystems(ctx context.Context) ([]*replication.Filesystem, error) { fss, err := zfs.ZFSListMapping(p.FSFilter) if err != nil { return nil, err @@ -46,7 +47,7 @@ func (p *SenderEndpoint) ListFilesystems() ([]*replication.Filesystem, error) { return rfss, nil } -func (p *SenderEndpoint) ListFilesystemVersions(fs string) ([]*replication.FilesystemVersion, error) { +func (p *SenderEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ([]*replication.FilesystemVersion, error) { dp, err := zfs.NewDatasetPath(fs) if err != nil { return nil, err @@ -69,7 +70,7 @@ func (p *SenderEndpoint) ListFilesystemVersions(fs string) ([]*replication.Files return rfsvs, nil } -func (p *SenderEndpoint) Send(r *replication.SendReq) (*replication.SendRes, io.Reader, error) { +func (p *SenderEndpoint) Send(ctx context.Context, r *replication.SendReq) (*replication.SendRes, io.ReadCloser, error) { os.Stderr.WriteString("sending " + r.String() + "\n") dp, err := zfs.NewDatasetPath(r.Filesystem) if err != nil { @@ -89,7 +90,7 @@ func (p *SenderEndpoint) Send(r *replication.SendReq) (*replication.SendRes, io. return &replication.SendRes{}, stream, nil } -func (p *SenderEndpoint) Receive(r *replication.ReceiveReq, sendStream io.Reader) (error) { +func (p *SenderEndpoint) Receive(ctx context.Context, r *replication.ReceiveReq, sendStream io.ReadCloser) (error) { return fmt.Errorf("sender endpoint does not receive") } @@ -109,7 +110,7 @@ func NewReceiverEndpoint(fsmap *DatasetMapFilter, fsvf zfs.FilesystemVersionFilt return &ReceiverEndpoint{fsmapInv, fsmap, fsvf}, nil } -func (e *ReceiverEndpoint) ListFilesystems() ([]*replication.Filesystem, error) { +func (e *ReceiverEndpoint) ListFilesystems(ctx context.Context) ([]*replication.Filesystem, error) { filtered, err := zfs.ZFSListMapping(e.fsmapInv.AsFilter()) if err != nil { return nil, errors.Wrap(err, "error checking client permission") @@ -125,7 +126,7 @@ func (e *ReceiverEndpoint) ListFilesystems() ([]*replication.Filesystem, error) return fss, nil } -func (e *ReceiverEndpoint) ListFilesystemVersions(fs string) ([]*replication.FilesystemVersion, error) { +func (e *ReceiverEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ([]*replication.FilesystemVersion, error) { p, err := zfs.NewDatasetPath(fs) if err != nil { return nil, err @@ -151,11 +152,13 @@ func (e *ReceiverEndpoint) ListFilesystemVersions(fs string) ([]*replication.Fil return rfsvs, nil } -func (e *ReceiverEndpoint) Send(req *replication.SendReq) (*replication.SendRes, io.Reader, error) { +func (e *ReceiverEndpoint) Send(ctx context.Context, req *replication.SendReq) (*replication.SendRes, io.ReadCloser, error) { return nil, nil, errors.New("receiver endpoint does not send") } -func (e *ReceiverEndpoint) Receive(req *replication.ReceiveReq, sendStream io.Reader) error { +func (e *ReceiverEndpoint) Receive(ctx context.Context, req *replication.ReceiveReq, sendStream io.ReadCloser) error { + defer sendStream.Close() + p, err := zfs.NewDatasetPath(req.Filesystem) if err != nil { return err @@ -210,7 +213,6 @@ func (e *ReceiverEndpoint) Receive(req *replication.ReceiveReq, sendStream io.Re os.Stderr.WriteString("receiving...\n") if err := zfs.ZFSRecv(lp.ToString(), sendStream, args...); err != nil { - // FIXME sendStream is on the wire and contains data, if we don't consume it, wire must be closed return err } return nil @@ -232,19 +234,18 @@ type RemoteEndpoint struct { *streamrpc.Client } -func (s RemoteEndpoint) ListFilesystems() ([]*replication.Filesystem, error) { +func (s RemoteEndpoint) ListFilesystems(ctx context.Context) ([]*replication.Filesystem, error) { req := replication.ListFilesystemReq{} b, err := proto.Marshal(&req) if err != nil { return nil, err } - rb, rs, err := s.RequestReply(RPCListFilesystems, bytes.NewBuffer(b), nil) + rb, rs, err := s.RequestReply(ctx, RPCListFilesystems, bytes.NewBuffer(b), nil) if err != nil { return nil, err } if rs != nil { - os.Stderr.WriteString(fmt.Sprintf("%#v\n", rs)) - s.Close() // FIXME + rs.Close() return nil, errors.New("response contains unexpected stream") } var res replication.ListFilesystemRes @@ -254,7 +255,7 @@ func (s RemoteEndpoint) ListFilesystems() ([]*replication.Filesystem, error) { return res.Filesystems, nil } -func (s RemoteEndpoint) ListFilesystemVersions(fs string) ([]*replication.FilesystemVersion, error) { +func (s RemoteEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ([]*replication.FilesystemVersion, error) { req := replication.ListFilesystemVersionsReq{ Filesystem: fs, } @@ -262,12 +263,12 @@ func (s RemoteEndpoint) ListFilesystemVersions(fs string) ([]*replication.Filesy if err != nil { return nil, err } - rb, rs, err := s.RequestReply(RPCListFilesystemVersions, bytes.NewBuffer(b), nil) + rb, rs, err := s.RequestReply(ctx, RPCListFilesystemVersions, bytes.NewBuffer(b), nil) if err != nil { return nil, err } if rs != nil { - s.Close() // FIXME + rs.Close() return nil, errors.New("response contains unexpected stream") } var res replication.ListFilesystemVersionsRes @@ -277,12 +278,12 @@ func (s RemoteEndpoint) ListFilesystemVersions(fs string) ([]*replication.Filesy return res.Versions, nil } -func (s RemoteEndpoint) Send(r *replication.SendReq) (*replication.SendRes, io.Reader, error) { +func (s RemoteEndpoint) Send(ctx context.Context, r *replication.SendReq) (*replication.SendRes, io.ReadCloser, error) { b, err := proto.Marshal(r) if err != nil { return nil, nil, err } - rb, rs, err := s.RequestReply(RPCSend, bytes.NewBuffer(b), nil) + rb, rs, err := s.RequestReply(ctx, RPCSend, bytes.NewBuffer(b), nil) if err != nil { return nil, nil, err } @@ -291,24 +292,25 @@ func (s RemoteEndpoint) Send(r *replication.SendReq) (*replication.SendRes, io.R } var res replication.SendRes if err := proto.Unmarshal(rb.Bytes(), &res); err != nil { - s.Close() // FIXME + rs.Close() return nil, nil, err } // FIXME make sure the consumer will read the reader until the end... return &res, rs, nil } -func (s RemoteEndpoint) Receive(r *replication.ReceiveReq, sendStream io.Reader) (error) { +func (s RemoteEndpoint) Receive(ctx context.Context, r *replication.ReceiveReq, sendStream io.ReadCloser) (error) { + defer sendStream.Close() b, err := proto.Marshal(r) if err != nil { return err } - rb, rs, err := s.RequestReply(RPCReceive, bytes.NewBuffer(b), sendStream) + rb, rs, err := s.RequestReply(ctx, RPCReceive, bytes.NewBuffer(b), sendStream) if err != nil { - s.Close() // FIXME return err } if rs != nil { + rs.Close() return errors.New("response contains unexpected stream") } var res replication.ReceiveRes @@ -320,9 +322,16 @@ func (s RemoteEndpoint) Receive(r *replication.ReceiveReq, sendStream io.Reader) type HandlerAdaptor struct { ep replication.ReplicationEndpoint + log Logger } -func (a *HandlerAdaptor) Handle(endpoint string, reqStructured *bytes.Buffer, reqStream io.Reader) (resStructured *bytes.Buffer, resStream io.Reader, err error) { +func (a *HandlerAdaptor) Handle(endpoint string, reqStructured *bytes.Buffer, reqStream io.ReadCloser) (resStructured *bytes.Buffer, resStream io.ReadCloser, err error) { + + ctx := context.Background() + if a.log != nil { + // FIXME validate type conversion here? + ctx = context.WithValue(ctx, streamrpc.ContextKeyLogger, a.log) + } switch endpoint { case RPCListFilesystems: @@ -330,7 +339,7 @@ func (a *HandlerAdaptor) Handle(endpoint string, reqStructured *bytes.Buffer, re if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { return nil, nil, err } - fsses, err := a.ep.ListFilesystems() + fsses, err := a.ep.ListFilesystems(ctx) if err != nil { return nil, nil, err } @@ -349,7 +358,7 @@ func (a *HandlerAdaptor) Handle(endpoint string, reqStructured *bytes.Buffer, re if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { return nil, nil, err } - fsvs, err := a.ep.ListFilesystemVersions(req.Filesystem) + fsvs, err := a.ep.ListFilesystemVersions(ctx, req.Filesystem) if err != nil { return nil, nil, err } @@ -368,7 +377,7 @@ func (a *HandlerAdaptor) Handle(endpoint string, reqStructured *bytes.Buffer, re if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { return nil, nil, err } - res, sendStream, err := a.ep.Send(&req) + res, sendStream, err := a.ep.Send(ctx, &req) if err != nil { return nil, nil, err } @@ -384,7 +393,7 @@ func (a *HandlerAdaptor) Handle(endpoint string, reqStructured *bytes.Buffer, re if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { return nil, nil, err } - err := a.ep.Receive(&req, reqStream) + err := a.ep.Receive(ctx, &req, reqStream) if err != nil { return nil, nil, err } diff --git a/cmd/replication/replication.go b/cmd/replication/replication.go index 6b3f57b..fb3b154 100644 --- a/cmd/replication/replication.go +++ b/cmd/replication/replication.go @@ -7,8 +7,8 @@ import ( type ReplicationEndpoint interface { // Does not include placeholder filesystems - ListFilesystems() ([]*Filesystem, error) - ListFilesystemVersions(fs string) ([]*FilesystemVersion, error) // fix depS + ListFilesystems(ctx context.Context) ([]*Filesystem, error) + ListFilesystemVersions(ctx context.Context, fs string) ([]*FilesystemVersion, error) // fix depS Sender Receiver } @@ -81,21 +81,22 @@ func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicat log := ctx.Value(ContextKeyLog).(Logger) - sfss, err := ep.Sender().ListFilesystems() + sfss, err := ep.Sender().ListFilesystems(ctx) if err != nil { log.Printf("error listing sender filesystems: %s", err) return } - rfss, err := ep.Receiver().ListFilesystems() + rfss, err := ep.Receiver().ListFilesystems(ctx) if err != nil { log.Printf("error listing receiver filesystems: %s", err) return } for _, fs := range sfss { - log.Printf("replication fs %s", fs.Path) - sfsvs, err := ep.Sender().ListFilesystemVersions(fs.Path) + log.Printf("replicating %s", fs.Path) + + sfsvs, err := ep.Sender().ListFilesystemVersions(ctx, fs.Path) if err != nil { log.Printf("sender error %s", err) continue @@ -115,7 +116,7 @@ func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicat var rfsvs []*FilesystemVersion if receiverFSExists { - rfsvs, err = ep.Receiver().ListFilesystemVersions(fs.Path) + rfsvs, err = ep.Receiver().ListFilesystemVersions(ctx, fs.Path) if err != nil { log.Printf("receiver error %s", err) if _, ok := err.(FilteredError); ok { @@ -162,11 +163,11 @@ func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicat } type Sender interface { - Send(r *SendReq) (*SendRes, io.Reader, error) + Send(ctx context.Context, r *SendReq) (*SendRes, io.ReadCloser, error) } type Receiver interface { - Receive(r *ReceiveReq, sendStream io.Reader) (error) + Receive(ctx context.Context, r *ReceiveReq, sendStream io.ReadCloser) (error) } type Copier interface { @@ -211,7 +212,7 @@ func (incrementalPathReplicator) Replicate(ctx context.Context, sender Sender, r From: path[0].RelName(), ResumeToken: fs.ResumeToken, } - sres, sstream, err := sender.Send(sr) + sres, sstream, err := sender.Send(ctx, sr) if err != nil { log.Printf("send request failed: %s", err) // FIXME must close connection... @@ -222,7 +223,7 @@ func (incrementalPathReplicator) Replicate(ctx context.Context, sender Sender, r Filesystem: fs.Path, ClearResumeToken: fs.ResumeToken != "" && !sres.UsedResumeToken, } - err = receiver.Receive(rr, sstream) + err = receiver.Receive(ctx, rr, sstream) if err != nil { // FIXME this failure could be due to an unexpected exit of ZFS on the sending side // FIXME which is transported through the streamrpc protocol, and known to the sendStream.(*streamrpc.streamReader), @@ -250,7 +251,7 @@ incrementalLoop: To: path[j+1].RelName(), ResumeToken: rt, } - sres, sstream, err := sender.Send(sr) + sres, sstream, err := sender.Send(ctx, sr) if err != nil { log.Printf("send request failed: %s", err) // handle and ignore @@ -262,7 +263,7 @@ incrementalLoop: Filesystem: fs.Path, ClearResumeToken: rt != "" && !sres.UsedResumeToken, } - err = receiver.Receive(rr, sstream) + err = receiver.Receive(ctx, rr, sstream) if err != nil { log.Printf("receive request failed: %s", err) // handle and ignore diff --git a/cmd/replication/replication_test.go b/cmd/replication/replication_test.go index 0334d02..2e6072b 100644 --- a/cmd/replication/replication_test.go +++ b/cmd/replication/replication_test.go @@ -11,7 +11,7 @@ import ( type IncrementalPathSequenceStep struct { SendRequest *replication.SendReq SendResponse *replication.SendRes - SendReader io.Reader + SendReader io.ReadCloser SendError error ReceiveRequest *replication.ReceiveReq ReceiveError error @@ -23,7 +23,7 @@ type MockIncrementalPathRecorder struct { Pos int } -func (m *MockIncrementalPathRecorder) Receive(r *replication.ReceiveReq, rs io.Reader) (error) { +func (m *MockIncrementalPathRecorder) Receive(ctx context.Context, r *replication.ReceiveReq, rs io.ReadCloser) (error) { if m.Pos >= len(m.Sequence) { m.T.Fatal("unexpected Receive") } @@ -35,7 +35,7 @@ func (m *MockIncrementalPathRecorder) Receive(r *replication.ReceiveReq, rs io.R return i.ReceiveError } -func (m *MockIncrementalPathRecorder) Send(r *replication.SendReq) (*replication.SendRes, io.Reader, error) { +func (m *MockIncrementalPathRecorder) Send(ctx context.Context, r *replication.SendReq) (*replication.SendRes, io.ReadCloser, error) { if m.Pos >= len(m.Sequence) { m.T.Fatal("unexpected Send") } diff --git a/zfs/zfs.go b/zfs/zfs.go index b9557fe..2b11da3 100644 --- a/zfs/zfs.go +++ b/zfs/zfs.go @@ -278,7 +278,7 @@ func absVersion(fs, v string) (full string, err error) { return fmt.Sprintf("%s%s", fs, v), nil } -func ZFSSend(fs string, from, to string) (stream io.Reader, err error) { +func ZFSSend(fs string, from, to string) (stream io.ReadCloser, err error) { fromV, err := absVersion(fs, from) if err != nil { From 1826535e6faa4967f789e28b41309fe7204b1c76 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 15 Jul 2018 17:36:53 +0200 Subject: [PATCH 005/167] WIP --- cmd/config_job_local.go | 2 +- cmd/config_job_pull.go | 122 ++++++++++++++++--- cmd/config_job_source.go | 38 ++---- cmd/replication/replication.go | 181 ++++++++++++++++++++-------- util/contextflexibletimeout.go | 83 +++++++++++++ util/contextflexibletimeout_test.go | 84 +++++++++++++ 6 files changed, 406 insertions(+), 104 deletions(-) create mode 100644 util/contextflexibletimeout.go create mode 100644 util/contextflexibletimeout_test.go diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index 82b72fc..88316d4 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -146,11 +146,11 @@ outer: j.mainTask.Log().Debug("replicating from lhs to rhs") j.mainTask.Enter("replicate") - replication.Replicate( ctx, replication.NewEndpointPairPull(sender, receiver), replication.NewIncrementalPathReplicator(), + nil, //FIXME ) j.mainTask.Finish() diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index 9d78e7a..cb56ee5 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -10,6 +10,8 @@ import ( "github.com/zrepl/zrepl/util" "github.com/zrepl/zrepl/cmd/replication" "github.com/problame/go-streamrpc" + "io" + "net" ) type PullJob struct { @@ -107,7 +109,15 @@ func (j *PullJob) JobStart(ctx context.Context) { ticker := time.NewTicker(j.Interval) for { + begin := time.Now() j.doRun(ctx) + duration := time.Now().Sub(begin) + if duration > j.Interval { + j.task.Log(). + WithField("actual_duration", duration). + WithField("configured_interval", j.Interval). + Warn("pull run took longer than configured interval") + } select { case <-ctx.Done(): j.task.Log().WithError(ctx.Err()).Info("context") @@ -124,32 +134,86 @@ var STREAMRPC_CONFIG = &streamrpc.ConnConfig{ // FIXME oversight and configurabi TxChunkSize: 4096 * 4096, } +type streamrpcRWCToNetConnAdatper struct { + io.ReadWriteCloser +} + +func (streamrpcRWCToNetConnAdatper) LocalAddr() net.Addr { + panic("implement me") +} + +func (streamrpcRWCToNetConnAdatper) RemoteAddr() net.Addr { + panic("implement me") +} + +func (streamrpcRWCToNetConnAdatper) SetDeadline(t time.Time) error { + panic("implement me") +} + +func (streamrpcRWCToNetConnAdatper) SetReadDeadline(t time.Time) error { + panic("implement me") +} + +func (streamrpcRWCToNetConnAdatper) SetWriteDeadline(t time.Time) error { + panic("implement me") +} + +type streamrpcRWCConnecterToNetConnAdapter struct { + RWCConnecter + ReadDump, WriteDump string +} + +func (s streamrpcRWCConnecterToNetConnAdapter) Connect(ctx context.Context) (net.Conn, error) { + rwc, err := s.RWCConnecter.Connect() + if err != nil { + return nil, err + } + rwc, err = util.NewReadWriteCloserLogger(rwc, s.ReadDump, s.WriteDump) + if err != nil { + rwc.Close() + return nil, err + } + return streamrpcRWCToNetConnAdatper{rwc}, nil +} + +type tcpConnecter struct { + d net.Dialer +} + +func (t *tcpConnecter) Connect(ctx context.Context) (net.Conn, error) { + return t.d.DialContext(ctx, "tcp", "192.168.122.128:8888") +} + func (j *PullJob) doRun(ctx context.Context) { j.task.Enter("run") defer j.task.Finish() - j.task.Log().Info("connecting") - rwc, err := j.Connect.Connect() - if err != nil { - j.task.Log().WithError(err).Error("error connecting") - return + //connecter := streamrpcRWCConnecterToNetConnAdapter{ + // RWCConnecter: j.Connect, + // ReadDump: j.Debug.Conn.ReadDump, + // WriteDump: j.Debug.Conn.WriteDump, + //} + + // FIXME + connecter := &tcpConnecter{net.Dialer{ + Timeout: 2*time.Second, + }} + + clientConf := &streamrpc.ClientConfig{ + MaxConnectAttempts: 5, // FIXME + ReconnectBackoffBase: 1*time.Second, + ReconnectBackoffFactor: 2, + ConnConfig: STREAMRPC_CONFIG, } - rwc, err = util.NewReadWriteCloserLogger(rwc, j.Debug.Conn.ReadDump, j.Debug.Conn.WriteDump) - if err != nil { - return - } - - - client := RemoteEndpoint{streamrpc.NewClientOnConn(rwc, STREAMRPC_CONFIG)} - if j.Debug.RPC.Log { - // FIXME implement support - // client.SetLogger(j.task.Log(), true) - } + client, err := streamrpc.NewClient(connecter, clientConf) + defer client.Close() j.task.Enter("pull") + sender := RemoteEndpoint{client} + puller, err := NewReceiverEndpoint( j.Mapping, NewPrefixFilter(j.SnapshotPrefix), @@ -161,10 +225,27 @@ func (j *PullJob) doRun(ctx context.Context) { } replicator := replication.NewIncrementalPathReplicator() - replication.Replicate(context.WithValue(ctx, replication.ContextKeyLog, j.task.Log()), replication.NewEndpointPairPull(client, puller), replicator) + ctx = context.WithValue(ctx, replication.ContextKeyLog, j.task.Log()) + ctx = context.WithValue(ctx, streamrpc.ContextKeyLogger, j.task.Log()) + ctx, enforceDeadline := util.ContextWithOptionalDeadline(ctx) - closeRPCWithTimeout(j.task, client, time.Second*1, "") - rwc.Close() + // Try replicating each file system regardless of j.Interval + // (this does not solve the underlying problem that j.Interval is too short, + // but it covers the case of initial replication taking longer than all + // incremental replications afterwards) + allTriedOnce := make(chan struct{}) + replicationBegin := time.Now() + go func() { + select { + case <-allTriedOnce: + enforceDeadline(replicationBegin.Add(j.Interval)) + case <-ctx.Done(): + } + }() + replication.Replicate(ctx, replication.NewEndpointPairPull(sender, puller), replicator, allTriedOnce) + + + client.Close() j.task.Finish() j.task.Enter("prune") @@ -200,7 +281,8 @@ func closeRPCWithTimeout(task *Task, remote RemoteEndpoint, timeout time.Duratio ch := make(chan error) go func() { - ch <- remote.Close() + remote.Close() + ch <- nil close(ch) }() diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index 4ebb385..6c98cab 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -9,6 +9,7 @@ import ( "github.com/pkg/errors" "github.com/zrepl/zrepl/util" "github.com/problame/go-streamrpc" + "net" ) type SourceJob struct { @@ -138,7 +139,9 @@ func (j *SourceJob) Pruner(task *Task, side PrunePolicySide, dryRun bool) (p Pru func (j *SourceJob) serve(ctx context.Context, task *Task) { - listener, err := j.Serve.Listen() + //listener, err := j.Serve.Listen() + // FIXME + listener, err := net.Listen("tcp", "192.168.122.128:8888") if err != nil { task.Log().WithError(err).Error("error listening") return @@ -208,7 +211,7 @@ func (j *SourceJob) handleConnection(rwc io.ReadWriteCloser, task *Task) { senderEP := NewSenderEndpoint(j.Filesystems, NewPrefixFilter(j.SnapshotPrefix)) - handler := HandlerAdaptor{senderEP} + handler := HandlerAdaptor{senderEP, task.Log()} // FIXME logging support or erase config //if j.Debug.RPC.Log { // rpclog := task.Log().WithField("subsystem", "rpc") @@ -217,35 +220,8 @@ func (j *SourceJob) handleConnection(rwc io.ReadWriteCloser, task *Task) { if err := streamrpc.ServeConn(rwc, STREAMRPC_CONFIG, handler.Handle); err != nil { task.Log().WithError(err).Error("error serving connection") + } else { + task.Log().Info("client closed connection") } - // wait for client to close connection - // FIXME: we cannot just close it like we would to with a TCP socket because - // FIXME: go-nettsh's Close() may overtake the remaining data in the pipe - const CLIENT_HANGUP_TIMEOUT = 1 * time.Second - task.Log(). - WithField("timeout", CLIENT_HANGUP_TIMEOUT). - Debug("waiting for client to hang up") - - wchan := make(chan error) - go func() { - var pseudo [1]byte - _, err := io.ReadFull(rwc, pseudo[:]) - wchan <- err - }() - var werr error - select { - case werr = <-wchan: - // all right - case <-time.After(CLIENT_HANGUP_TIMEOUT): - werr = errors.New("client did not close connection within timeout") - } - if werr != nil && werr != io.EOF { - task.Log().WithError(werr). - Error("error waiting for client to hang up") - } - task.Log().Info("closing client connection") - if err = rwc.Close(); err != nil { - task.Log().WithError(err).Error("error force-closing connection") - } } diff --git a/cmd/replication/replication.go b/cmd/replication/replication.go index fb3b154..1b6e37f 100644 --- a/cmd/replication/replication.go +++ b/cmd/replication/replication.go @@ -3,6 +3,9 @@ package replication import ( "context" "io" + "container/list" + "fmt" + "net" ) type ReplicationEndpoint interface { @@ -77,7 +80,87 @@ type Logger interface{ Printf(fmt string, args ... interface{}) } -func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicator) { +type replicationWork struct { + fs *Filesystem +} + +type FilesystemReplicationResult struct { + Done bool + Retry bool + Unfixable bool +} + +func handleGenericEndpointError(err error) FilesystemReplicationResult { + if _, ok := err.(net.Error); ok { + return FilesystemReplicationResult{Retry: true} + } + return FilesystemReplicationResult{Unfixable: true} +} + +func driveFSReplication(ctx context.Context, ws *list.List, allTriedOnce chan struct{}, log Logger, f func(w *replicationWork) FilesystemReplicationResult) { + initialLen, fCalls := ws.Len(), 0 + for ws.Len() > 0 { + select { + case <-ctx.Done(): + log.Printf("aborting replication due to context error: %s", ctx.Err()) + return + default: + } + + w := ws.Remove(ws.Front()).(*replicationWork) + res := f(w) + fCalls++ + if fCalls == initialLen { + select { + case allTriedOnce <- struct{}{}: + default: + } + } + if res.Done { + log.Printf("finished replication of %s", w.fs.Path) + continue + } + + if res.Unfixable { + log.Printf("aborting replication of %s after unfixable error", w.fs.Path) + continue + } + + log.Printf("queuing replication of %s for retry", w.fs.Path) + ws.PushBack(w) + } +} + +func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) { + if noCommonAncestor, ok := conflict.(*ConflictNoCommonAncestor); ok { + if len(noCommonAncestor.SortedReceiverVersions) == 0 { + // FIXME hard-coded replication policy: most recent + // snapshot as source + var mostRecentSnap *FilesystemVersion + for n := len(noCommonAncestor.SortedSenderVersions) -1; n >= 0; n-- { + if noCommonAncestor.SortedSenderVersions[n].Type == FilesystemVersion_Snapshot { + mostRecentSnap = noCommonAncestor.SortedSenderVersions[n] + break + } + } + if mostRecentSnap == nil { + return nil, "no snapshots available on sender side" + } + return []*FilesystemVersion{mostRecentSnap}, fmt.Sprintf("start replication at most recent snapshot %s", mostRecentSnap) + } + } + return nil, "no automated way to handle conflict type" +} + +// Replicate replicates filesystems from ep.Sender() to ep.Receiver(). +// +// All filesystems presented by the sending side are replicated, +// unless the receiver rejects a Receive request with a *FilteredError. +// +// If an error occurs when replicating a filesystem, that error is logged to the logger in ctx. +// Replicate continues with the replication of the remaining file systems. +// Depending on the type of error, failed replications are retried in an unspecified order (currently FIFO). +func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicator, allTriedOnce chan struct{}) { log := ctx.Value(ContextKeyLog).(Logger) @@ -93,18 +176,27 @@ func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicat return } + wq := list.New() for _, fs := range sfss { + wq.PushBack(&replicationWork{ + fs: fs, + }) + } + + driveFSReplication(ctx, wq, allTriedOnce, log, func(w *replicationWork) FilesystemReplicationResult { + fs := w.fs + log.Printf("replicating %s", fs.Path) sfsvs, err := ep.Sender().ListFilesystemVersions(ctx, fs.Path) if err != nil { - log.Printf("sender error %s", err) - continue + log.Printf("cannot get remote filesystem versions: %s", err) + return handleGenericEndpointError(err) } if len(sfsvs) <= 1 { log.Printf("sender does not have any versions") - continue + return FilesystemReplicationResult{Unfixable: true} } receiverFSExists := false @@ -118,47 +210,35 @@ func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicat if receiverFSExists { rfsvs, err = ep.Receiver().ListFilesystemVersions(ctx, fs.Path) if err != nil { - log.Printf("receiver error %s", err) if _, ok := err.(FilteredError); ok { - // Remote does not map filesystem, don't try to tx it - continue + log.Printf("receiver does not map %s", fs.Path) + return FilesystemReplicationResult{Done: true} } - // log and ignore - continue + log.Printf("receiver error %s", err) + return handleGenericEndpointError(err) } } else { rfsvs = []*FilesystemVersion{} } path, conflict := IncrementalPath(rfsvs, sfsvs) - if noCommonAncestor, ok := conflict.(*ConflictNoCommonAncestor); ok { - if len(noCommonAncestor.SortedReceiverVersions) == 0 { - log.Printf("initial replication") - // FIXME hard-coded replication policy: most recent - // snapshot as source - var mostRecentSnap *FilesystemVersion - for n := len(sfsvs) -1; n >= 0; n-- { - if sfsvs[n].Type == FilesystemVersion_Snapshot { - mostRecentSnap = sfsvs[n] - break - } - } - if mostRecentSnap == nil { - log.Printf("no snapshot on sender side") - continue - } - log.Printf("starting at most recent snapshot %s", mostRecentSnap) - path = []*FilesystemVersion{mostRecentSnap} + if conflict != nil { + log.Printf("conflict: %s", conflict) + var msg string + path, msg = resolveConflict(conflict) + if path != nil { + log.Printf("conflict resolved: %s", msg) + } else { + log.Printf("%s", msg) } - } else if conflict != nil { - log.Printf("unresolvable conflict: %s", conflict) - // handle or ignore for now - continue + } + if path == nil { + return FilesystemReplicationResult{Unfixable: true} } - ipr.Replicate(ctx, ep.Sender(), ep.Receiver(), NewCopier(), fs, path) + return ipr.Replicate(ctx, ep.Sender(), ep.Receiver(), NewCopier(), fs, path) - } + }) } @@ -185,7 +265,7 @@ func NewCopier() Copier { } type IncrementalPathReplicator interface { - Replicate(ctx context.Context, sender Sender, receiver Receiver, copier Copier, fs *Filesystem, path []*FilesystemVersion) + Replicate(ctx context.Context, sender Sender, receiver Receiver, copier Copier, fs *Filesystem, path []*FilesystemVersion) FilesystemReplicationResult } type incrementalPathReplicator struct{} @@ -194,14 +274,13 @@ func NewIncrementalPathReplicator() IncrementalPathReplicator { return incrementalPathReplicator{} } -func (incrementalPathReplicator) Replicate(ctx context.Context, sender Sender, receiver Receiver, copier Copier, fs *Filesystem, path []*FilesystemVersion) { +func (incrementalPathReplicator) Replicate(ctx context.Context, sender Sender, receiver Receiver, copier Copier, fs *Filesystem, path []*FilesystemVersion) FilesystemReplicationResult { log := ctx.Value(ContextKeyLog).(Logger) if len(path) == 0 { log.Printf("nothing to do") - // nothing to do - return + return FilesystemReplicationResult{Done: true} } if len(path) == 1 { @@ -215,8 +294,7 @@ func (incrementalPathReplicator) Replicate(ctx context.Context, sender Sender, r sres, sstream, err := sender.Send(ctx, sr) if err != nil { log.Printf("send request failed: %s", err) - // FIXME must close connection... - return + return handleGenericEndpointError(err) } rr := &ReceiveReq{ @@ -225,20 +303,19 @@ func (incrementalPathReplicator) Replicate(ctx context.Context, sender Sender, r } err = receiver.Receive(ctx, rr, sstream) if err != nil { - // FIXME this failure could be due to an unexpected exit of ZFS on the sending side - // FIXME which is transported through the streamrpc protocol, and known to the sendStream.(*streamrpc.streamReader), - // FIXME but the io.Reader interface design doesn not allow us to infer that it is a *streamrpc.streamReader right now - log.Printf("receive request failed (might also be error on sender...): %s", err) - // FIXME must close connection - return + log.Printf("receive request failed (might also be error on sender): %s", err) + sstream.Close() + // This failure could be due to + // - an unexpected exit of ZFS on the sending side + // - an unexpected exit of ZFS on the receiving side + // - a connectivity issue + return handleGenericEndpointError(err) } - - return + return FilesystemReplicationResult{Done: true} } usedResumeToken := false -incrementalLoop: for j := 0; j < len(path)-1; j++ { rt := "" if !usedResumeToken { // only send resume token for first increment @@ -254,8 +331,7 @@ incrementalLoop: sres, sstream, err := sender.Send(ctx, sr) if err != nil { log.Printf("send request failed: %s", err) - // handle and ignore - break incrementalLoop + return handleGenericEndpointError(err) } // try to consume stream @@ -266,10 +342,11 @@ incrementalLoop: err = receiver.Receive(ctx, rr, sstream) if err != nil { log.Printf("receive request failed: %s", err) - // handle and ignore - break incrementalLoop + return handleGenericEndpointError(err) // FIXME resume state on receiver -> update ResumeToken } // FIXME handle properties from sres } + + return FilesystemReplicationResult{Done: true} } diff --git a/util/contextflexibletimeout.go b/util/contextflexibletimeout.go new file mode 100644 index 0000000..dc84549 --- /dev/null +++ b/util/contextflexibletimeout.go @@ -0,0 +1,83 @@ +package util + +import ( + "context" + "time" + "sync" +) + +type contextWithOptionalDeadline struct { + context.Context + + m sync.Mutex + deadline time.Time + + done chan struct{} + err error +} + +func (c *contextWithOptionalDeadline) Deadline() (deadline time.Time, ok bool) { + c.m.Lock() + defer c.m.Unlock() + return c.deadline, !c.deadline.IsZero() +} + +func (c *contextWithOptionalDeadline) Err() error { + c.m.Lock() + defer c.m.Unlock() + return c.err +} + +func (c *contextWithOptionalDeadline) Done() (<-chan struct{}) { + return c.done +} + +func ContextWithOptionalDeadline(pctx context.Context) (ctx context.Context, enforceDeadline func(deadline time.Time)) { + + // mctx can only be cancelled by cancelMctx, not by a potential cancel of pctx + rctx := &contextWithOptionalDeadline{ + Context: pctx, + done: make(chan struct{}), + err: nil, + } + enforceDeadline = func(deadline time.Time) { + + // Set deadline and prohibit multiple calls + rctx.m.Lock() + alreadyCalled := !rctx.deadline.IsZero() + if !alreadyCalled { + rctx.deadline = deadline + } + rctx.m.Unlock() + if alreadyCalled { + return + } + + // Deadline in past? + sleepTime := deadline.Sub(time.Now()) + if sleepTime <= 0 { + rctx.m.Lock() + rctx.err = context.DeadlineExceeded + rctx.m.Unlock() + close(rctx.done) + return + } + go func() { + // Set a timer and wait for timer or parent context to be cancelled + timer := time.NewTimer(sleepTime) + var setErr error + select { + case <-pctx.Done(): + timer.Stop() + setErr = pctx.Err() + case <-timer.C: + setErr = context.DeadlineExceeded + } + rctx.m.Lock() + rctx.err = setErr + rctx.m.Unlock() + close(rctx.done) + }() + } + return rctx, enforceDeadline +} diff --git a/util/contextflexibletimeout_test.go b/util/contextflexibletimeout_test.go new file mode 100644 index 0000000..77031d5 --- /dev/null +++ b/util/contextflexibletimeout_test.go @@ -0,0 +1,84 @@ +package util + +import ( + "testing" + "context" + "time" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/assert" +) + +func TestContextWithOptionalDeadline(t *testing.T) { + + ctx := context.Background() + cctx, enforceDeadline := ContextWithOptionalDeadline(ctx) + + begin := time.Now() + var receivedCancellation time.Time + var cancellationError error + go func() { + select { + case <- cctx.Done(): + receivedCancellation = time.Now() + cancellationError = cctx.Err() + case <- time.After(600*time.Millisecond): + t.Fatalf("should have been cancelled by deadline") + } + }() + time.Sleep(100*time.Millisecond) + if !receivedCancellation.IsZero() { + t.Fatalf("no enforcement means no cancellation") + } + require.Nil(t, cctx.Err(), "no error while not cancelled") + dl, ok := cctx.Deadline() + require.False(t, ok) + require.Zero(t, dl) + enforceDeadline(begin.Add(200*time.Millisecond)) + // second call must be ignored, i.e. we expect the deadline to be at begin+200ms, not begin+400ms + enforceDeadline(begin.Add(400*time.Millisecond)) + + time.Sleep(300*time.Millisecond) // 100ms margin for scheduler + if receivedCancellation.Sub(begin) > 250*time.Millisecond { + t.Fatalf("cancellation is beyond acceptable scheduler latency") + } + require.Equal(t, context.DeadlineExceeded, cancellationError) +} + +func TestContextWithOptionalDeadlineNegativeDeadline(t *testing.T) { + ctx := context.Background() + cctx, enforceDeadline := ContextWithOptionalDeadline(ctx) + enforceDeadline(time.Now().Add(-10*time.Second)) + select { + case <-cctx.Done(): + default: + t.FailNow() + } +} + +func TestContextWithOptionalDeadlineParentCancellation(t *testing.T) { + + pctx, cancel := context.WithCancel(context.Background()) + cctx, enforceDeadline := ContextWithOptionalDeadline(pctx) + + // 0 ms + start := time.Now() + enforceDeadline(start.Add(400*time.Millisecond)) + time.Sleep(100*time.Millisecond) + cancel() // cancel @ ~100ms + time.Sleep(100*time.Millisecond) // give 100ms time to propagate cancel + // @ ~200ms + select { + case <-cctx.Done(): + assert.True(t, time.Now().Before(start.Add(300*time.Millisecond))) + assert.Equal(t, context.Canceled, cctx.Err()) + default: + t.FailNow() + } + +} + +func TestContextWithOptionalDeadlineValue(t *testing.T) { + pctx := context.WithValue(context.Background(), "key", "value") + cctx, _ := ContextWithOptionalDeadline(pctx) + assert.Equal(t, "value", cctx.Value("key")) +} \ No newline at end of file From a0b320bfeba81bbe02d31e50d971c6e182786183 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 8 Aug 2018 13:09:51 +0200 Subject: [PATCH 006/167] streamrpc now requires net.Conn => use it instead of rwc everywhere --- cmd/adaptors.go | 76 ++++++++++++++++++++++++++++++ cmd/config.go | 14 ++---- cmd/config_connect.go | 28 +++++++---- cmd/config_job_pull.go | 82 +++++++-------------------------- cmd/config_job_source.go | 41 +++++++++-------- cmd/config_parse.go | 5 +- cmd/config_serve_stdinserver.go | 18 ++++++-- util/io.go | 23 ++++----- 8 files changed, 164 insertions(+), 123 deletions(-) create mode 100644 cmd/adaptors.go diff --git a/cmd/adaptors.go b/cmd/adaptors.go new file mode 100644 index 0000000..6423518 --- /dev/null +++ b/cmd/adaptors.go @@ -0,0 +1,76 @@ +package cmd + +import ( + "context" + "io" + "net" + "time" + + "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/util" +) + +type logNetConnConnecter struct { + streamrpc.Connecter + ReadDump, WriteDump string +} + +var _ streamrpc.Connecter = logNetConnConnecter{} + +func (l logNetConnConnecter) Connect(ctx context.Context) (net.Conn, error) { + conn, err := l.Connecter.Connect(ctx) + if err != nil { + return nil, err + } + return util.NewNetConnLogger(conn, l.ReadDump, l.WriteDump) +} + +type logListenerFactory struct { + ListenerFactory + ReadDump, WriteDump string +} + +var _ ListenerFactory = logListenerFactory{} + +type logListener struct { + net.Listener + ReadDump, WriteDump string +} + +var _ net.Listener = logListener{} + +func (m logListenerFactory) Listen() (net.Listener, error) { + l, err := m.ListenerFactory.Listen() + if err != nil { + return nil, err + } + return logListener{l, m.ReadDump, m.WriteDump}, nil +} + +func (l logListener) Accept() (net.Conn, error) { + conn, err := l.Listener.Accept() + if err != nil { + return nil, err + } + return util.NewNetConnLogger(conn, l.ReadDump, l.WriteDump) +} + + +type netsshAddr struct{} + +func (netsshAddr) Network() string { return "netssh" } +func (netsshAddr) String() string { return "???" } + +type netsshConnToNetConnAdatper struct { + io.ReadWriteCloser // works for both netssh.SSHConn and netssh.ServeConn +} + +func (netsshConnToNetConnAdatper) LocalAddr() net.Addr { return netsshAddr{} } + +func (netsshConnToNetConnAdatper) RemoteAddr() net.Addr { return netsshAddr{} } + +func (netsshConnToNetConnAdatper) SetDeadline(t time.Time) error { return nil } + +func (netsshConnToNetConnAdatper) SetReadDeadline(t time.Time) error { return nil } + +func (netsshConnToNetConnAdatper) SetWriteDeadline(t time.Time) error { return nil } diff --git a/cmd/config.go b/cmd/config.go index 38a95be..1361d75 100644 --- a/cmd/config.go +++ b/cmd/config.go @@ -1,7 +1,7 @@ package cmd import ( - "io" + "net" "fmt" "github.com/pkg/errors" @@ -43,16 +43,8 @@ type JobDebugSettings struct { } } -type RWCConnecter interface { - Connect() (io.ReadWriteCloser, error) -} -type AuthenticatedChannelListenerFactory interface { - Listen() (AuthenticatedChannelListener, error) -} - -type AuthenticatedChannelListener interface { - Accept() (ch io.ReadWriteCloser, err error) - Close() (err error) +type ListenerFactory interface { + Listen() (net.Listener, error) } type SSHStdinServerConnectDescr struct { diff --git a/cmd/config_connect.go b/cmd/config_connect.go index b6d8435..d94f065 100644 --- a/cmd/config_connect.go +++ b/cmd/config_connect.go @@ -2,13 +2,14 @@ package cmd import ( "fmt" - "io" + "net" "context" "github.com/jinzhu/copier" "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-netssh" + "github.com/problame/go-streamrpc" "time" ) @@ -24,6 +25,8 @@ type SSHStdinserverConnecter struct { dialTimeout time.Duration } +var _ streamrpc.Connecter = &SSHStdinserverConnecter{} + func parseSSHStdinserverConnecter(i map[string]interface{}) (c *SSHStdinserverConnecter, err error) { c = &SSHStdinserverConnecter{} @@ -46,21 +49,28 @@ func parseSSHStdinserverConnecter(i map[string]interface{}) (c *SSHStdinserverCo } -func (c *SSHStdinserverConnecter) Connect() (rwc io.ReadWriteCloser, err error) { +type netsshConnToConn struct { *netssh.SSHConn } + +var _ net.Conn = netsshConnToConn{} + +func (netsshConnToConn) SetDeadline(dl time.Time) error { return nil } +func (netsshConnToConn) SetReadDeadline(dl time.Time) error { return nil } +func (netsshConnToConn) SetWriteDeadline(dl time.Time) error { return nil } + +func (c *SSHStdinserverConnecter) Connect(dialCtx context.Context) (net.Conn, error) { var endpoint netssh.Endpoint - if err = copier.Copy(&endpoint, c); err != nil { + if err := copier.Copy(&endpoint, c); err != nil { return nil, errors.WithStack(err) } - var dialCtx context.Context - dialCtx, dialCancel := context.WithTimeout(context.TODO(), c.dialTimeout) // context.TODO tied to error handling below + dialCtx, dialCancel := context.WithTimeout(dialCtx, c.dialTimeout) // context.TODO tied to error handling below defer dialCancel() - if rwc, err = netssh.Dial(dialCtx, endpoint); err != nil { + nconn, err := netssh.Dial(dialCtx, endpoint) + if err != nil { if err == context.DeadlineExceeded { err = errors.Errorf("dial_timeout of %s exceeded", c.dialTimeout) } - err = errors.WithStack(err) - return + return nil, err } - return + return netsshConnToConn{nconn}, nil } diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index cb56ee5..5b4c112 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -16,7 +16,7 @@ import ( type PullJob struct { Name string - Connect RWCConnecter + Connect streamrpc.Connecter Interval time.Duration Mapping *DatasetMapFilter // constructed from mapping during parsing @@ -90,6 +90,15 @@ func parsePullJob(c JobParsingContext, name string, i map[string]interface{}) (j return } + if j.Debug.Conn.ReadDump != "" || j.Debug.Conn.WriteDump != "" { + logConnecter := logNetConnConnecter{ + Connecter: j.Connect, + ReadDump: j.Debug.Conn.ReadDump, + WriteDump: j.Debug.Conn.WriteDump, + } + j.Connect = logConnecter + } + return } @@ -132,56 +141,12 @@ var STREAMRPC_CONFIG = &streamrpc.ConnConfig{ // FIXME oversight and configurabi RxStructuredMaxLen: 4096 * 4096, RxStreamMaxChunkSize: 4096 * 4096, TxChunkSize: 4096 * 4096, -} - -type streamrpcRWCToNetConnAdatper struct { - io.ReadWriteCloser -} - -func (streamrpcRWCToNetConnAdatper) LocalAddr() net.Addr { - panic("implement me") -} - -func (streamrpcRWCToNetConnAdatper) RemoteAddr() net.Addr { - panic("implement me") -} - -func (streamrpcRWCToNetConnAdatper) SetDeadline(t time.Time) error { - panic("implement me") -} - -func (streamrpcRWCToNetConnAdatper) SetReadDeadline(t time.Time) error { - panic("implement me") -} - -func (streamrpcRWCToNetConnAdatper) SetWriteDeadline(t time.Time) error { - panic("implement me") -} - -type streamrpcRWCConnecterToNetConnAdapter struct { - RWCConnecter - ReadDump, WriteDump string -} - -func (s streamrpcRWCConnecterToNetConnAdapter) Connect(ctx context.Context) (net.Conn, error) { - rwc, err := s.RWCConnecter.Connect() - if err != nil { - return nil, err - } - rwc, err = util.NewReadWriteCloserLogger(rwc, s.ReadDump, s.WriteDump) - if err != nil { - rwc.Close() - return nil, err - } - return streamrpcRWCToNetConnAdatper{rwc}, nil -} - -type tcpConnecter struct { - d net.Dialer -} - -func (t *tcpConnecter) Connect(ctx context.Context) (net.Conn, error) { - return t.d.DialContext(ctx, "tcp", "192.168.122.128:8888") + RxTimeout: streamrpc.Timeout{ + Progress: 10*time.Second, + }, + TxTimeout: streamrpc.Timeout{ + Progress: 10*time.Second, + }, } func (j *PullJob) doRun(ctx context.Context) { @@ -189,25 +154,12 @@ func (j *PullJob) doRun(ctx context.Context) { j.task.Enter("run") defer j.task.Finish() - //connecter := streamrpcRWCConnecterToNetConnAdapter{ - // RWCConnecter: j.Connect, - // ReadDump: j.Debug.Conn.ReadDump, - // WriteDump: j.Debug.Conn.WriteDump, - //} - // FIXME - connecter := &tcpConnecter{net.Dialer{ - Timeout: 2*time.Second, - }} - clientConf := &streamrpc.ClientConfig{ - MaxConnectAttempts: 5, // FIXME - ReconnectBackoffBase: 1*time.Second, - ReconnectBackoffFactor: 2, ConnConfig: STREAMRPC_CONFIG, } - client, err := streamrpc.NewClient(connecter, clientConf) + client, err := streamrpc.NewClient(j.Connect, clientConf) defer client.Close() j.task.Enter("pull") diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index 6c98cab..d212fa5 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -2,19 +2,17 @@ package cmd import ( "context" - "io" "time" "github.com/mitchellh/mapstructure" "github.com/pkg/errors" - "github.com/zrepl/zrepl/util" "github.com/problame/go-streamrpc" "net" ) type SourceJob struct { Name string - Serve AuthenticatedChannelListenerFactory + Serve ListenerFactory Filesystems *DatasetMapFilter SnapshotPrefix string Interval time.Duration @@ -70,6 +68,15 @@ func parseSourceJob(c JobParsingContext, name string, i map[string]interface{}) return } + if j.Debug.Conn.ReadDump != "" || j.Debug.Conn.WriteDump != "" { + logServe := logListenerFactory{ + ListenerFactory: j.Serve, + ReadDump: j.Debug.Conn.ReadDump, + WriteDump: j.Debug.Conn.WriteDump, + } + j.Serve = logServe + } + return } @@ -139,19 +146,17 @@ func (j *SourceJob) Pruner(task *Task, side PrunePolicySide, dryRun bool) (p Pru func (j *SourceJob) serve(ctx context.Context, task *Task) { - //listener, err := j.Serve.Listen() - // FIXME - listener, err := net.Listen("tcp", "192.168.122.128:8888") + listener, err := j.Serve.Listen() if err != nil { task.Log().WithError(err).Error("error listening") return } - type rwcChanMsg struct { - rwc io.ReadWriteCloser - err error + type connChanMsg struct { + conn net.Conn + err error } - rwcChan := make(chan rwcChanMsg) + connChan := make(chan connChanMsg) // Serve connections until interrupted or error outer: @@ -160,23 +165,23 @@ outer: go func() { rwc, err := listener.Accept() if err != nil { - rwcChan <- rwcChanMsg{rwc, err} - close(rwcChan) + connChan <- connChanMsg{rwc, err} + close(connChan) return } - rwcChan <- rwcChanMsg{rwc, err} + connChan <- connChanMsg{rwc, err} }() select { - case rwcMsg := <-rwcChan: + case rwcMsg := <-connChan: if rwcMsg.err != nil { task.Log().WithError(err).Error("error accepting connection") break outer } - j.handleConnection(rwcMsg.rwc, task) + j.handleConnection(rwcMsg.conn, task) case <-ctx.Done(): task.Log().WithError(ctx.Err()).Info("context") @@ -197,17 +202,13 @@ outer: } -func (j *SourceJob) handleConnection(rwc io.ReadWriteCloser, task *Task) { +func (j *SourceJob) handleConnection(conn net.Conn, task *Task) { task.Enter("handle_connection") defer task.Finish() task.Log().Info("handling client connection") - rwc, err := util.NewReadWriteCloserLogger(rwc, j.Debug.Conn.ReadDump, j.Debug.Conn.WriteDump) - if err != nil { - panic(err) - } senderEP := NewSenderEndpoint(j.Filesystems, NewPrefixFilter(j.SnapshotPrefix)) diff --git a/cmd/config_parse.go b/cmd/config_parse.go index cd733eb..6f6784c 100644 --- a/cmd/config_parse.go +++ b/cmd/config_parse.go @@ -11,6 +11,7 @@ import ( "regexp" "strconv" "time" + "github.com/problame/go-streamrpc" ) var ConfigFileDefaultLocations []string = []string{ @@ -208,7 +209,7 @@ func parseJob(c JobParsingContext, i map[string]interface{}) (j Job, err error) } -func parseConnect(i map[string]interface{}) (c RWCConnecter, err error) { +func parseConnect(i map[string]interface{}) (c streamrpc.Connecter, err error) { t, err := extractStringField(i, "type", true) if err != nil { @@ -266,7 +267,7 @@ func parsePrunePolicy(v map[string]interface{}, willSeeBookmarks bool) (p PruneP } } -func parseAuthenticatedChannelListenerFactory(c JobParsingContext, v map[string]interface{}) (p AuthenticatedChannelListenerFactory, err error) { +func parseAuthenticatedChannelListenerFactory(c JobParsingContext, v map[string]interface{}) (p ListenerFactory, err error) { t, err := extractStringField(v, "type", true) if err != nil { diff --git a/cmd/config_serve_stdinserver.go b/cmd/config_serve_stdinserver.go index ed7b68d..2380cc8 100644 --- a/cmd/config_serve_stdinserver.go +++ b/cmd/config_serve_stdinserver.go @@ -4,7 +4,7 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-netssh" - "io" + "net" "path" ) @@ -30,9 +30,9 @@ func parseStdinserverListenerFactory(c JobParsingContext, i map[string]interface return } -func (f *StdinserverListenerFactory) Listen() (al AuthenticatedChannelListener, err error) { +func (f *StdinserverListenerFactory) Listen() (net.Listener, error) { - if err = PreparePrivateSockpath(f.sockpath); err != nil { + if err := PreparePrivateSockpath(f.sockpath); err != nil { return nil, err } @@ -47,8 +47,16 @@ type StdinserverListener struct { l *netssh.Listener } -func (l StdinserverListener) Accept() (ch io.ReadWriteCloser, err error) { - return l.l.Accept() +func (l StdinserverListener) Addr() net.Addr { + return netsshAddr{} +} + +func (l StdinserverListener) Accept() (net.Conn, error) { + c, err := l.l.Accept() + if err != nil { + return nil, err + } + return netsshConnToNetConnAdatper{c}, nil } func (l StdinserverListener) Close() (err error) { diff --git a/util/io.go b/util/io.go index 68ae286..857df26 100644 --- a/util/io.go +++ b/util/io.go @@ -2,18 +2,19 @@ package util import ( "io" + "net" "os" ) -type ReadWriteCloserLogger struct { - RWC io.ReadWriteCloser +type NetConnLogger struct { + net.Conn ReadFile *os.File WriteFile *os.File } -func NewReadWriteCloserLogger(rwc io.ReadWriteCloser, readlog, writelog string) (l *ReadWriteCloserLogger, err error) { - l = &ReadWriteCloserLogger{ - RWC: rwc, +func NewNetConnLogger(conn net.Conn, readlog, writelog string) (l *NetConnLogger, err error) { + l = &NetConnLogger{ + Conn: conn, } flags := os.O_CREATE | os.O_WRONLY if readlog != "" { @@ -29,8 +30,8 @@ func NewReadWriteCloserLogger(rwc io.ReadWriteCloser, readlog, writelog string) return } -func (c *ReadWriteCloserLogger) Read(buf []byte) (n int, err error) { - n, err = c.RWC.Read(buf) +func (c *NetConnLogger) Read(buf []byte) (n int, err error) { + n, err = c.Conn.Read(buf) if c.WriteFile != nil { if _, writeErr := c.ReadFile.Write(buf[0:n]); writeErr != nil { panic(writeErr) @@ -39,8 +40,8 @@ func (c *ReadWriteCloserLogger) Read(buf []byte) (n int, err error) { return } -func (c *ReadWriteCloserLogger) Write(buf []byte) (n int, err error) { - n, err = c.RWC.Write(buf) +func (c *NetConnLogger) Write(buf []byte) (n int, err error) { + n, err = c.Conn.Write(buf) if c.ReadFile != nil { if _, writeErr := c.WriteFile.Write(buf[0:n]); writeErr != nil { panic(writeErr) @@ -48,8 +49,8 @@ func (c *ReadWriteCloserLogger) Write(buf []byte) (n int, err error) { } return } -func (c *ReadWriteCloserLogger) Close() (err error) { - err = c.RWC.Close() +func (c *NetConnLogger) Close() (err error) { + err = c.Conn.Close() if err != nil { return } From 74445a0017bb214d34d08971b5d47bf74de7eb38 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 8 Aug 2018 13:12:50 +0200 Subject: [PATCH 007/167] fixup --- cmd/config_job_pull.go | 2 -- cmd/config_job_source.go | 2 +- cmd/replication.go | 3 +-- 3 files changed, 2 insertions(+), 5 deletions(-) diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index 5b4c112..ab65c5b 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -10,8 +10,6 @@ import ( "github.com/zrepl/zrepl/util" "github.com/zrepl/zrepl/cmd/replication" "github.com/problame/go-streamrpc" - "io" - "net" ) type PullJob struct { diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index d212fa5..ef6e991 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -219,7 +219,7 @@ func (j *SourceJob) handleConnection(conn net.Conn, task *Task) { // rpcServer.SetLogger(rpclog, true) //} - if err := streamrpc.ServeConn(rwc, STREAMRPC_CONFIG, handler.Handle); err != nil { + if err := streamrpc.ServeConn(context.TODO(), conn, STREAMRPC_CONFIG, handler.Handle); err != nil { task.Log().WithError(err).Error("error serving connection") } else { task.Log().Info("client closed connection") diff --git a/cmd/replication.go b/cmd/replication.go index d333056..d5ce665 100644 --- a/cmd/replication.go +++ b/cmd/replication.go @@ -325,9 +325,8 @@ type HandlerAdaptor struct { log Logger } -func (a *HandlerAdaptor) Handle(endpoint string, reqStructured *bytes.Buffer, reqStream io.ReadCloser) (resStructured *bytes.Buffer, resStream io.ReadCloser, err error) { +func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructured *bytes.Buffer, reqStream io.ReadCloser) (resStructured *bytes.Buffer, resStream io.ReadCloser, err error) { - ctx := context.Background() if a.log != nil { // FIXME validate type conversion here? ctx = context.WithValue(ctx, streamrpc.ContextKeyLogger, a.log) From c1f3076eb3e2539e88bb31ece950bb69fc5ea8bd Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 10 Aug 2018 17:06:00 +0200 Subject: [PATCH 008/167] WIP2 logging done somewhat --- cmd/adaptors.go | 29 ++ cmd/config_job_control.go | 3 +- cmd/config_job_local.go | 7 +- cmd/config_job_pull.go | 48 ++- cmd/config_job_source.go | 18 +- cmd/daemon.go | 19 +- cmd/logging_formatters.go | 29 +- cmd/main.go | 2 +- cmd/replication.go | 20 +- cmd/replication/pdu_extras.go | 4 + cmd/replication/replication.go | 478 +++++++++++++++++----------- cmd/replication/replication_test.go | 238 +++++++------- logger/logger.go | 53 +-- logger/nulllogger.go | 22 ++ 14 files changed, 573 insertions(+), 397 deletions(-) create mode 100644 logger/nulllogger.go diff --git a/cmd/adaptors.go b/cmd/adaptors.go index 6423518..86e7c37 100644 --- a/cmd/adaptors.go +++ b/cmd/adaptors.go @@ -2,12 +2,15 @@ package cmd import ( "context" + "fmt" "io" "net" + "strings" "time" "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/util" + "github.com/zrepl/zrepl/logger" ) type logNetConnConnecter struct { @@ -74,3 +77,29 @@ func (netsshConnToNetConnAdatper) SetDeadline(t time.Time) error { return nil } func (netsshConnToNetConnAdatper) SetReadDeadline(t time.Time) error { return nil } func (netsshConnToNetConnAdatper) SetWriteDeadline(t time.Time) error { return nil } + +type streamrpcLogAdaptor = twoClassLogAdaptor +type replicationLogAdaptor = twoClassLogAdaptor + +type twoClassLogAdaptor struct { + logger.Logger +} + +var _ streamrpc.Logger = twoClassLogAdaptor{} + +func (a twoClassLogAdaptor) Errorf(fmtStr string, args... interface{}) { + const errorSuffix = ": %s" + if len(args) == 1 { + if err, ok := args[0].(error); ok && strings.HasSuffix(fmtStr, errorSuffix) { + msg := strings.TrimSuffix(fmtStr, errorSuffix) + a.WithError(err).Error(msg) + return + } + } + a.Logger.Error(fmt.Sprintf(fmtStr, args...)) +} + +func (a twoClassLogAdaptor) Infof(fmtStr string, args... interface{}) { + a.Logger.Info(fmt.Sprintf(fmtStr, args...)) +} + diff --git a/cmd/config_job_control.go b/cmd/config_job_control.go index 3baf715..1e7baae 100644 --- a/cmd/config_job_control.go +++ b/cmd/config_job_control.go @@ -8,6 +8,7 @@ import ( "io" "net" "net/http" + "github.com/zrepl/zrepl/logger" ) type ControlJob struct { @@ -126,7 +127,7 @@ func (j jsonResponder) ServeHTTP(w http.ResponseWriter, r *http.Request) { } type requestLogger struct { - log Logger + log logger.Logger handler http.Handler handlerFunc http.HandlerFunc } diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index 88316d4..d66bdce 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -146,12 +146,7 @@ outer: j.mainTask.Log().Debug("replicating from lhs to rhs") j.mainTask.Enter("replicate") - replication.Replicate( - ctx, - replication.NewEndpointPairPull(sender, receiver), - replication.NewIncrementalPathReplicator(), - nil, //FIXME - ) + replication.Replicate(ctx, replication.NewEndpointPairPull(sender, receiver)) j.mainTask.Finish() diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index ab65c5b..85685e4 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -1,15 +1,18 @@ package cmd import ( + "os" + "os/signal" + "syscall" "time" "context" "fmt" + "github.com/mitchellh/mapstructure" "github.com/pkg/errors" - "github.com/zrepl/zrepl/util" - "github.com/zrepl/zrepl/cmd/replication" "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/cmd/replication" ) type PullJob struct { @@ -91,7 +94,7 @@ func parsePullJob(c JobParsingContext, name string, i map[string]interface{}) (j if j.Debug.Conn.ReadDump != "" || j.Debug.Conn.WriteDump != "" { logConnecter := logNetConnConnecter{ Connecter: j.Connect, - ReadDump: j.Debug.Conn.ReadDump, + ReadDump: j.Debug.Conn.ReadDump, WriteDump: j.Debug.Conn.WriteDump, } j.Connect = logConnecter @@ -113,6 +116,9 @@ func (j *PullJob) JobStart(ctx context.Context) { j.task = NewTask("main", j, log) // j.task is idle here idle here + usr1 := make(chan os.Signal) + signal.Notify(usr1, syscall.SIGUSR1) + defer signal.Stop(usr1) ticker := time.NewTicker(j.Interval) for { @@ -130,23 +136,25 @@ func (j *PullJob) JobStart(ctx context.Context) { j.task.Log().WithError(ctx.Err()).Info("context") return case <-ticker.C: + case <-usr1: } } } var STREAMRPC_CONFIG = &streamrpc.ConnConfig{ // FIXME oversight and configurability - RxHeaderMaxLen: 4096, - RxStructuredMaxLen: 4096 * 4096, + RxHeaderMaxLen: 4096, + RxStructuredMaxLen: 4096 * 4096, RxStreamMaxChunkSize: 4096 * 4096, - TxChunkSize: 4096 * 4096, + TxChunkSize: 4096 * 4096, RxTimeout: streamrpc.Timeout{ - Progress: 10*time.Second, + Progress: 10 * time.Second, }, TxTimeout: streamrpc.Timeout{ - Progress: 10*time.Second, + Progress: 10 * time.Second, }, } + func (j *PullJob) doRun(ctx context.Context) { j.task.Enter("run") @@ -174,26 +182,10 @@ func (j *PullJob) doRun(ctx context.Context) { return } - replicator := replication.NewIncrementalPathReplicator() - ctx = context.WithValue(ctx, replication.ContextKeyLog, j.task.Log()) - ctx = context.WithValue(ctx, streamrpc.ContextKeyLogger, j.task.Log()) - ctx, enforceDeadline := util.ContextWithOptionalDeadline(ctx) - - // Try replicating each file system regardless of j.Interval - // (this does not solve the underlying problem that j.Interval is too short, - // but it covers the case of initial replication taking longer than all - // incremental replications afterwards) - allTriedOnce := make(chan struct{}) - replicationBegin := time.Now() - go func() { - select { - case <-allTriedOnce: - enforceDeadline(replicationBegin.Add(j.Interval)) - case <-ctx.Done(): - } - }() - replication.Replicate(ctx, replication.NewEndpointPairPull(sender, puller), replicator, allTriedOnce) - + ctx = replication.ContextWithLogger(ctx, replicationLogAdaptor{j.task.Log().WithField("subsystem", "replication")}) + ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{j.task.Log().WithField("subsystem", "rpc.protocol")}) + ctx = context.WithValue(ctx, contextKeyLog, j.task.Log().WithField("subsystem", "rpc.endpoint")) + replication.Replicate(ctx, replication.NewEndpointPairPull(sender, puller)) client.Close() j.task.Finish() diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index ef6e991..f6b7851 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -71,8 +71,8 @@ func parseSourceJob(c JobParsingContext, name string, i map[string]interface{}) if j.Debug.Conn.ReadDump != "" || j.Debug.Conn.WriteDump != "" { logServe := logListenerFactory{ ListenerFactory: j.Serve, - ReadDump: j.Debug.Conn.ReadDump, - WriteDump: j.Debug.Conn.WriteDump, + ReadDump: j.Debug.Conn.ReadDump, + WriteDump: j.Debug.Conn.WriteDump, } j.Serve = logServe } @@ -209,17 +209,13 @@ func (j *SourceJob) handleConnection(conn net.Conn, task *Task) { task.Log().Info("handling client connection") - senderEP := NewSenderEndpoint(j.Filesystems, NewPrefixFilter(j.SnapshotPrefix)) - handler := HandlerAdaptor{senderEP, task.Log()} - // FIXME logging support or erase config - //if j.Debug.RPC.Log { - // rpclog := task.Log().WithField("subsystem", "rpc") - // rpcServer.SetLogger(rpclog, true) - //} - - if err := streamrpc.ServeConn(context.TODO(), conn, STREAMRPC_CONFIG, handler.Handle); err != nil { + ctx := context.Background() + ctx = context.WithValue(ctx, contextKeyLog, task.Log().WithField("subsystem", "rpc.endpoint")) + ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{task.Log().WithField("subsystem", "rpc.protocol")}) + handler := HandlerAdaptor{senderEP} + if err := streamrpc.ServeConn(ctx, conn, STREAMRPC_CONFIG, handler.Handle); err != nil { task.Log().WithError(err).Error("error serving connection") } else { task.Log().Info("client closed connection") diff --git a/cmd/daemon.go b/cmd/daemon.go index 46746a2..81a718a 100644 --- a/cmd/daemon.go +++ b/cmd/daemon.go @@ -9,7 +9,7 @@ import ( "io" "os" "os/signal" - "strings" + //"strings" "sync" "syscall" "time" @@ -88,6 +88,10 @@ const ( contextKeyDaemon contextKey = contextKey("daemon") ) +func getLogger(ctx context.Context) Logger { + return ctx.Value(contextKeyLog).(Logger) +} + type Daemon struct { conf *Config startedAt time.Time @@ -276,7 +280,7 @@ func (p *taskProgress) Read() (out taskProgress) { type taskActivity struct { name string idle bool - logger *logger.Logger + logger logger.Logger // The progress of the task that is updated by UpdateIO() and UpdateLogEntry() // // Progress happens on a task-level and is thus global to the task. @@ -285,7 +289,7 @@ type taskActivity struct { progress *taskProgress } -func NewTask(name string, parent Job, lg *logger.Logger) *Task { +func NewTask(name string, parent Job, lg logger.Logger) *Task { t := &Task{ name: name, parent: parent, @@ -336,9 +340,10 @@ func (t *Task) Enter(activity string) { } act := &taskActivity{activity, false, nil, prev.progress} t.activities.PushFront(act) - stack := t.buildActivityStack() - activityField := strings.Join(stack, ".") - act.logger = prev.logger.ReplaceField(logTaskField, activityField) + //stack := t.buildActivityStack() + //activityField := strings.Join(stack, ".") + act.logger = prev.logger + // act.logger = prev.logger.ReplaceField(logTaskField, activityField) t.activitiesLastUpdate = time.Now() } @@ -425,7 +430,7 @@ func (t *Task) Finish() { // Returns a logger derived from the logger passed to the constructor function. // The logger's task field contains the current activity stack joined by '.'. -func (t *Task) Log() *logger.Logger { +func (t *Task) Log() logger.Logger { t.rwl.RLock() defer t.rwl.RUnlock() // FIXME should influence TaskStatus's LastUpdate field diff --git a/cmd/logging_formatters.go b/cmd/logging_formatters.go index cf0f985..02795e8 100644 --- a/cmd/logging_formatters.go +++ b/cmd/logging_formatters.go @@ -25,6 +25,7 @@ const ( logJobField string = "job" logTaskField string = "task" logFSField string = "filesystem" + logSubsysField string = "subsystem" logMapFromField string = "map_from" logMapToField string = "map_to" logIncFromField string = "inc_from" @@ -77,7 +78,7 @@ func (f *HumanFormatter) Format(e *logger.Entry) (out []byte, err error) { fmt.Fprintf(&line, "[%s]", e.Level.Short()) } - prefixFields := []string{logJobField, logTaskField, logFSField} + prefixFields := []string{logJobField, logTaskField, logSubsysField} prefixed := make(map[string]bool, len(prefixFields)+2) for _, field := range prefixFields { val, ok := e.Fields[field].(string) @@ -91,18 +92,18 @@ func (f *HumanFormatter) Format(e *logger.Entry) (out []byte, err error) { } } // even more prefix fields - mapFrom, mapFromOk := e.Fields[logMapFromField].(string) - mapTo, mapToOk := e.Fields[logMapToField].(string) - if mapFromOk && mapToOk && !f.ignored(logMapFromField) && !f.ignored(logMapToField) { - fmt.Fprintf(&line, "[%s => %s]", mapFrom, mapTo) - prefixed[logMapFromField], prefixed[logMapToField] = true, true - } - incFrom, incFromOk := e.Fields[logIncFromField].(string) - incTo, incToOk := e.Fields[logIncToField].(string) - if incFromOk && incToOk && !f.ignored(logIncFromField) && !f.ignored(logMapToField) { - fmt.Fprintf(&line, "[%s => %s]", incFrom, incTo) - prefixed[logIncFromField], prefixed[logIncToField] = true, true - } + //mapFrom, mapFromOk := e.Fields[logMapFromField].(string) + //mapTo, mapToOk := e.Fields[logMapToField].(string) + //if mapFromOk && mapToOk && !f.ignored(logMapFromField) && !f.ignored(logMapToField) { + // fmt.Fprintf(&line, "[%s => %s]", mapFrom, mapTo) + // prefixed[logMapFromField], prefixed[logMapToField] = true, true + //} + //incFrom, incFromOk := e.Fields[logIncFromField].(string) + //incTo, incToOk := e.Fields[logIncToField].(string) + //if incFromOk && incToOk && !f.ignored(logIncFromField) && !f.ignored(logMapToField) { + // fmt.Fprintf(&line, "[%s => %s]", incFrom, incTo) + // prefixed[logIncFromField], prefixed[logIncToField] = true, true + //} if line.Len() > 0 { fmt.Fprint(&line, ": ") @@ -179,7 +180,7 @@ func (f *LogfmtFormatter) Format(e *logger.Entry) ([]byte, error) { // at least try and put job and task in front prefixed := make(map[string]bool, 2) - prefix := []string{logJobField, logTaskField} + prefix := []string{logJobField, logTaskField, logSubsysField} for _, pf := range prefix { v, ok := e.Fields[pf] if !ok { diff --git a/cmd/main.go b/cmd/main.go index 4a5c626..242367b 100644 --- a/cmd/main.go +++ b/cmd/main.go @@ -24,7 +24,7 @@ var ( zreplVersion string // set by build infrastructure ) -type Logger = *logger.Logger +type Logger = logger.Logger var RootCmd = &cobra.Command{ Use: "zrepl", diff --git a/cmd/replication.go b/cmd/replication.go index d5ce665..ff18bb0 100644 --- a/cmd/replication.go +++ b/cmd/replication.go @@ -9,7 +9,6 @@ import ( "github.com/pkg/errors" "github.com/golang/protobuf/proto" "bytes" - "os" "context" ) @@ -71,7 +70,6 @@ func (p *SenderEndpoint) ListFilesystemVersions(ctx context.Context, fs string) } func (p *SenderEndpoint) Send(ctx context.Context, r *replication.SendReq) (*replication.SendRes, io.ReadCloser, error) { - os.Stderr.WriteString("sending " + r.String() + "\n") dp, err := zfs.NewDatasetPath(r.Filesystem) if err != nil { return nil, nil, err @@ -175,23 +173,27 @@ func (e *ReceiverEndpoint) Receive(ctx context.Context, req *replication.Receive var visitErr error f := zfs.NewDatasetPathForest() f.Add(lp) + getLogger(ctx).Debug("begin tree-walk") f.WalkTopDown(func(v zfs.DatasetPathVisit) (visitChildTree bool) { if v.Path.Equal(lp) { return false } _, err := zfs.ZFSGet(v.Path, []string{zfs.ZREPL_PLACEHOLDER_PROPERTY_NAME}) if err != nil { - os.Stderr.WriteString("error zfsget " + err.Error() + "\n") // interpret this as an early exit of the zfs binary due to the fs not existing if err := zfs.ZFSCreatePlaceholderFilesystem(v.Path); err != nil { - os.Stderr.WriteString("error creating placeholder " + v.Path.ToString() + "\n") + getLogger(ctx). + WithError(err). + WithField("placeholder_fs", v.Path). + Error("cannot create placeholder filesystem") visitErr = err return false } } - os.Stderr.WriteString(v.Path.ToString() + " exists\n") + getLogger(ctx).WithField("filesystem", v.Path.ToString()).Debug("exists") return true // leave this fs as is }) + getLogger(ctx).WithField("visitErr", visitErr).Debug("complete tree-walk") if visitErr != nil { return visitErr @@ -210,7 +212,7 @@ func (e *ReceiverEndpoint) Receive(ctx context.Context, req *replication.Receive args = append(args, "-F") } - os.Stderr.WriteString("receiving...\n") + getLogger(ctx).Debug("start receive command") if err := zfs.ZFSRecv(lp.ToString(), sendStream, args...); err != nil { return err @@ -322,16 +324,10 @@ func (s RemoteEndpoint) Receive(ctx context.Context, r *replication.ReceiveReq, type HandlerAdaptor struct { ep replication.ReplicationEndpoint - log Logger } func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructured *bytes.Buffer, reqStream io.ReadCloser) (resStructured *bytes.Buffer, resStream io.ReadCloser, err error) { - if a.log != nil { - // FIXME validate type conversion here? - ctx = context.WithValue(ctx, streamrpc.ContextKeyLogger, a.log) - } - switch endpoint { case RPCListFilesystems: var req replication.ListFilesystemReq diff --git a/cmd/replication/pdu_extras.go b/cmd/replication/pdu_extras.go index b962059..0848bd6 100644 --- a/cmd/replication/pdu_extras.go +++ b/cmd/replication/pdu_extras.go @@ -41,6 +41,10 @@ func FilesystemVersionFromZFS(fsv zfs.FilesystemVersion) *FilesystemVersion { } } +func (v *FilesystemVersion) CreationAsTime() (time.Time, error) { + return time.Parse(time.RFC3339, v.Creation) +} + func (v *FilesystemVersion) ZFSFilesystemVersion() *zfs.FilesystemVersion { ct := time.Time{} if v.Creation != "" { diff --git a/cmd/replication/replication.go b/cmd/replication/replication.go index 1b6e37f..8f9ea4d 100644 --- a/cmd/replication/replication.go +++ b/cmd/replication/replication.go @@ -2,18 +2,20 @@ package replication import ( "context" - "io" - "container/list" "fmt" + "github.com/zrepl/zrepl/logger" + "io" "net" + "sort" + "time" ) type ReplicationEndpoint interface { // Does not include placeholder filesystems ListFilesystems(ctx context.Context) ([]*Filesystem, error) ListFilesystemVersions(ctx context.Context, fs string) ([]*FilesystemVersion, error) // fix depS - Sender - Receiver + Send(ctx context.Context, r *SendReq) (*SendRes, io.ReadCloser, error) + Receive(ctx context.Context, r *ReceiveReq, sendStream io.ReadCloser) error } type FilteredError struct{ fs string } @@ -73,62 +75,175 @@ func (p EndpointPair) Mode() ReplicationMode { type contextKey int const ( - ContextKeyLog contextKey = iota + contextKeyLog contextKey = iota ) -type Logger interface{ - Printf(fmt string, args ... interface{}) +//type Logger interface { +// Infof(fmt string, args ...interface{}) +// Errorf(fmt string, args ...interface{}) +//} + +//var _ Logger = nullLogger{} + +//type nullLogger struct{} +// +//func (nullLogger) Infof(fmt string, args ...interface{}) {} +//func (nullLogger) Errorf(fmt string, args ...interface{}) {} + +type Logger = logger.Logger + +func ContextWithLogger(ctx context.Context, l Logger) context.Context { + return context.WithValue(ctx, contextKeyLog, l) } -type replicationWork struct { - fs *Filesystem -} - -type FilesystemReplicationResult struct { - Done bool - Retry bool - Unfixable bool -} - -func handleGenericEndpointError(err error) FilesystemReplicationResult { - if _, ok := err.(net.Error); ok { - return FilesystemReplicationResult{Retry: true} +func getLogger(ctx context.Context) Logger { + l, ok := ctx.Value(contextKeyLog).(Logger) + if !ok { + l = logger.NewNullLogger() } - return FilesystemReplicationResult{Unfixable: true} + return l } -func driveFSReplication(ctx context.Context, ws *list.List, allTriedOnce chan struct{}, log Logger, f func(w *replicationWork) FilesystemReplicationResult) { - initialLen, fCalls := ws.Len(), 0 - for ws.Len() > 0 { +type replicationStep struct { + from, to *FilesystemVersion + fswork *replicateFSWork +} + +func (s *replicationStep) String() string { + if s.from == nil { // FIXME: ZFS semantics are that to is nil on non-incremental send + return fmt.Sprintf("%s%s (full)", s.fswork.fs.Path, s.to.RelName()) + } else { + return fmt.Sprintf("%s(%s => %s)", s.fswork.fs.Path, s.from.RelName(), s.to.RelName()) + } +} + +func newReplicationStep(from, to *FilesystemVersion) *replicationStep { + return &replicationStep{from: from, to: to} +} + +type replicateFSWork struct { + fs *Filesystem + steps []*replicationStep + currentStep int + errorCount int +} + +func newReplicateFSWork(fs *Filesystem) *replicateFSWork { + if fs == nil { + panic("implementation error") + } + return &replicateFSWork{ + fs: fs, + steps: make([]*replicationStep, 0), + } +} + +func newReplicateFSWorkWithConflict(fs *Filesystem, conflict error) *replicateFSWork { + // FIXME ignore conflict for now, but will be useful later when we make the replicationPlan exportable + return &replicateFSWork{ + fs: fs, + steps: make([]*replicationStep, 0), + } +} + +func (r *replicateFSWork) AddStep(step *replicationStep) { + if step == nil { + panic("implementation error") + } + if step.fswork != nil { + panic("implementation error") + } + step.fswork = r + r.steps = append(r.steps, step) +} + +func (w *replicateFSWork) CurrentStepDate() time.Time { + if len(w.steps) == 0 { + return time.Time{} + } + toTime, err := w.steps[w.currentStep].to.CreationAsTime() + if err != nil { + panic(err) // implementation inconsistent: should not admit invalid FilesystemVersion objects + } + return toTime +} + +func (w *replicateFSWork) CurrentStep() *replicationStep { + if w.currentStep >= len(w.steps) { + return nil + } + return w.steps[w.currentStep] +} + +func (w *replicateFSWork) CompleteStep() { + w.currentStep++ +} + +type replicationPlan struct { + fsws []*replicateFSWork +} + +func newReplicationPlan() *replicationPlan { + return &replicationPlan{ + fsws: make([]*replicateFSWork, 0), + } +} + +func (p *replicationPlan) addWork(work *replicateFSWork) { + p.fsws = append(p.fsws, work) +} + +func (p *replicationPlan) executeOldestFirst(ctx context.Context, doStep func(fs *Filesystem, from, to *FilesystemVersion) tryRes) { + log := getLogger(ctx) + + for { select { case <-ctx.Done(): - log.Printf("aborting replication due to context error: %s", ctx.Err()) + log.WithError(ctx.Err()).Info("aborting replication due to context error") return default: } - w := ws.Remove(ws.Front()).(*replicationWork) - res := f(w) - fCalls++ - if fCalls == initialLen { - select { - case allTriedOnce <- struct{}{}: - default: + // FIXME poor man's nested priority queue + pending := make([]*replicateFSWork, 0, len(p.fsws)) + for _, fsw := range p.fsws { + if fsw.CurrentStep() != nil { + pending = append(pending, fsw) } } - if res.Done { - log.Printf("finished replication of %s", w.fs.Path) - continue + sort.Slice(pending, func(i, j int) bool { + if pending[i].errorCount == pending[j].errorCount { + return pending[i].CurrentStepDate().Before(pending[j].CurrentStepDate()) + } + return pending[i].errorCount < pending[j].errorCount + }) + // pending is now sorted ascending by errorCount,CurrentStep().Creation + + if len(pending) == 0 { + log.Info("replication complete") + return } - if res.Unfixable { - log.Printf("aborting replication of %s after unfixable error", w.fs.Path) - continue + fsw := pending[0] + step := fsw.CurrentStep() + if step == nil { + panic("implementation error") + } + + log.WithField("step", step).Info("begin replication step") + res := doStep(step.fswork.fs, step.from, step.to) + + if res.done { + log.Info("replication step successful") + fsw.errorCount = 0 + fsw.CompleteStep() + } else { + log.Error("replication step failed, queuing for retry result") + fsw.errorCount++ } - log.Printf("queuing replication of %s for retry", w.fs.Path) - ws.PushBack(w) } + } func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) { @@ -137,7 +252,7 @@ func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) { // FIXME hard-coded replication policy: most recent // snapshot as source var mostRecentSnap *FilesystemVersion - for n := len(noCommonAncestor.SortedSenderVersions) -1; n >= 0; n-- { + for n := len(noCommonAncestor.SortedSenderVersions) - 1; n >= 0; n-- { if noCommonAncestor.SortedSenderVersions[n].Type == FilesystemVersion_Snapshot { mostRecentSnap = noCommonAncestor.SortedSenderVersions[n] break @@ -146,7 +261,7 @@ func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) { if mostRecentSnap == nil { return nil, "no snapshots available on sender side" } - return []*FilesystemVersion{mostRecentSnap}, fmt.Sprintf("start replication at most recent snapshot %s", mostRecentSnap) + return []*FilesystemVersion{mostRecentSnap}, fmt.Sprintf("start replication at most recent snapshot %s", mostRecentSnap.RelName()) } } return nil, "no automated way to handle conflict type" @@ -160,43 +275,144 @@ func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) { // If an error occurs when replicating a filesystem, that error is logged to the logger in ctx. // Replicate continues with the replication of the remaining file systems. // Depending on the type of error, failed replications are retried in an unspecified order (currently FIFO). -func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicator, allTriedOnce chan struct{}) { +func Replicate(ctx context.Context, ep EndpointPair) { - log := ctx.Value(ContextKeyLog).(Logger) + log := getLogger(ctx) + + retryPlanTicker := time.NewTicker(15 * time.Second) // FIXME make configurable + defer retryPlanTicker.Stop() + + var ( + plan *replicationPlan + res tryRes + ) + for { + log.Info("build replication plan") + plan, res = tryBuildReplicationPlan(ctx, ep) + if plan != nil { + break + } + log.WithField("result", res).Error("building replication plan failed, wait for retry timer result") + select { + case <-ctx.Done(): + log.WithError(ctx.Err()).Info("aborting replication because context is done") + return + case <-retryPlanTicker.C: + // TODO also accept an external channel that allows us to tick + } + } + retryPlanTicker.Stop() + + mainlog := log + plan.executeOldestFirst(ctx, func(fs *Filesystem, from, to *FilesystemVersion) tryRes { + + log := mainlog.WithField("filesystem", fs.Path) + + // FIXME refresh fs resume token + fs.ResumeToken = "" + + var sr *SendReq + if fs.ResumeToken != "" { + sr = &SendReq{ + Filesystem: fs.Path, + ResumeToken: fs.ResumeToken, + } + } else if from == nil { + sr = &SendReq{ + Filesystem: fs.Path, + From: to.RelName(), // FIXME fix protocol to use To, like zfs does internally + } + } else { + sr = &SendReq{ + Filesystem: fs.Path, + From: from.RelName(), + To: to.RelName(), + } + } + + log.WithField("request", sr).Debug("initiate send request") + sres, sstream, err := ep.Sender().Send(ctx, sr) + if err != nil { + log.WithError(err).Error("send request failed") + return tryResFromEndpointError(err) + } + if sstream == nil { + log.Error("send request did not return a stream, broken endpoint implementation") + return tryRes{unfixable: true} + } + + rr := &ReceiveReq{ + Filesystem: fs.Path, + ClearResumeToken: !sres.UsedResumeToken, + } + log.WithField("request", rr).Debug("initiate receive request") + err = ep.Receiver().Receive(ctx, rr, sstream) + if err != nil { + log.WithError(err).Error("receive request failed (might also be error on sender)") + sstream.Close() + // This failure could be due to + // - an unexpected exit of ZFS on the sending side + // - an unexpected exit of ZFS on the receiving side + // - a connectivity issue + return tryResFromEndpointError(err) + } + log.Info("receive finished") + return tryRes{done: true} + + }) + +} + +type tryRes struct { + done bool + retry bool + unfixable bool +} + +func tryResFromEndpointError(err error) tryRes { + if _, ok := err.(net.Error); ok { + return tryRes{retry: true} + } + return tryRes{unfixable: true} +} + +func tryBuildReplicationPlan(ctx context.Context, ep EndpointPair) (*replicationPlan, tryRes) { + + log := getLogger(ctx) + + early := func(err error) (*replicationPlan, tryRes) { + return nil, tryResFromEndpointError(err) + } sfss, err := ep.Sender().ListFilesystems(ctx) if err != nil { - log.Printf("error listing sender filesystems: %s", err) - return + log.WithError(err).Error("error listing sender filesystems") + return early(err) } rfss, err := ep.Receiver().ListFilesystems(ctx) if err != nil { - log.Printf("error listing receiver filesystems: %s", err) - return + log.WithError(err).Error("error listing receiver filesystems") + return early(err) } - wq := list.New() + plan := newReplicationPlan() + mainlog := log for _, fs := range sfss { - wq.PushBack(&replicationWork{ - fs: fs, - }) - } - driveFSReplication(ctx, wq, allTriedOnce, log, func(w *replicationWork) FilesystemReplicationResult { - fs := w.fs + log := mainlog.WithField("filesystem", fs.Path) - log.Printf("replicating %s", fs.Path) + log.Info("assessing filesystem") sfsvs, err := ep.Sender().ListFilesystemVersions(ctx, fs.Path) if err != nil { - log.Printf("cannot get remote filesystem versions: %s", err) - return handleGenericEndpointError(err) + log.WithError(err).Error("cannot get remote filesystem versions") + return early(err) } if len(sfsvs) <= 1 { - log.Printf("sender does not have any versions") - return FilesystemReplicationResult{Unfixable: true} + log.Error("sender does not have any versions") + return nil, tryRes{unfixable: true} } receiverFSExists := false @@ -211,11 +427,11 @@ func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicat rfsvs, err = ep.Receiver().ListFilesystemVersions(ctx, fs.Path) if err != nil { if _, ok := err.(FilteredError); ok { - log.Printf("receiver does not map %s", fs.Path) - return FilesystemReplicationResult{Done: true} + log.Info("receiver ignores filesystem") + continue } - log.Printf("receiver error %s", err) - return handleGenericEndpointError(err) + log.WithError(err).Error("receiver error") + return early(err) } } else { rfsvs = []*FilesystemVersion{} @@ -223,130 +439,34 @@ func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicat path, conflict := IncrementalPath(rfsvs, sfsvs) if conflict != nil { - log.Printf("conflict: %s", conflict) var msg string - path, msg = resolveConflict(conflict) + path, msg = resolveConflict(conflict) // no shadowing allowed! if path != nil { - log.Printf("conflict resolved: %s", msg) + log.WithField("conflict", conflict).Info("conflict") + log.WithField("resolution", msg).Info("automatically resolved") } else { - log.Printf("%s", msg) + log.WithField("conflict", conflict).Error("conflict") + log.WithField("problem", msg).Error("cannot resolve conflict") } } if path == nil { - return FilesystemReplicationResult{Unfixable: true} + plan.addWork(newReplicateFSWorkWithConflict(fs, conflict)) + continue } - return ipr.Replicate(ctx, ep.Sender(), ep.Receiver(), NewCopier(), fs, path) + w := newReplicateFSWork(fs) + if len(path) == 1 { + step := newReplicationStep(nil, path[0]) + w.AddStep(step) + } else { + for i := 0; i < len(path)-1; i++ { + step := newReplicationStep(path[i], path[i+1]) + w.AddStep(step) + } + } + plan.addWork(w) - }) - -} - -type Sender interface { - Send(ctx context.Context, r *SendReq) (*SendRes, io.ReadCloser, error) -} - -type Receiver interface { - Receive(ctx context.Context, r *ReceiveReq, sendStream io.ReadCloser) (error) -} - -type Copier interface { - Copy(writer io.Writer, reader io.Reader) (int64, error) -} - -type copier struct{} - -func (copier) Copy(writer io.Writer, reader io.Reader) (int64, error) { - return io.Copy(writer, reader) -} - -func NewCopier() Copier { - return copier{} -} - -type IncrementalPathReplicator interface { - Replicate(ctx context.Context, sender Sender, receiver Receiver, copier Copier, fs *Filesystem, path []*FilesystemVersion) FilesystemReplicationResult -} - -type incrementalPathReplicator struct{} - -func NewIncrementalPathReplicator() IncrementalPathReplicator { - return incrementalPathReplicator{} -} - -func (incrementalPathReplicator) Replicate(ctx context.Context, sender Sender, receiver Receiver, copier Copier, fs *Filesystem, path []*FilesystemVersion) FilesystemReplicationResult { - - log := ctx.Value(ContextKeyLog).(Logger) - - if len(path) == 0 { - log.Printf("nothing to do") - return FilesystemReplicationResult{Done: true} } - if len(path) == 1 { - log.Printf("full send of version %s", path[0]) - - sr := &SendReq{ - Filesystem: fs.Path, - From: path[0].RelName(), - ResumeToken: fs.ResumeToken, - } - sres, sstream, err := sender.Send(ctx, sr) - if err != nil { - log.Printf("send request failed: %s", err) - return handleGenericEndpointError(err) - } - - rr := &ReceiveReq{ - Filesystem: fs.Path, - ClearResumeToken: fs.ResumeToken != "" && !sres.UsedResumeToken, - } - err = receiver.Receive(ctx, rr, sstream) - if err != nil { - log.Printf("receive request failed (might also be error on sender): %s", err) - sstream.Close() - // This failure could be due to - // - an unexpected exit of ZFS on the sending side - // - an unexpected exit of ZFS on the receiving side - // - a connectivity issue - return handleGenericEndpointError(err) - } - return FilesystemReplicationResult{Done: true} - } - - usedResumeToken := false - - for j := 0; j < len(path)-1; j++ { - rt := "" - if !usedResumeToken { // only send resume token for first increment - rt = fs.ResumeToken - usedResumeToken = true - } - sr := &SendReq{ - Filesystem: fs.Path, - From: path[j].RelName(), - To: path[j+1].RelName(), - ResumeToken: rt, - } - sres, sstream, err := sender.Send(ctx, sr) - if err != nil { - log.Printf("send request failed: %s", err) - return handleGenericEndpointError(err) - } - // try to consume stream - - rr := &ReceiveReq{ - Filesystem: fs.Path, - ClearResumeToken: rt != "" && !sres.UsedResumeToken, - } - err = receiver.Receive(ctx, rr, sstream) - if err != nil { - log.Printf("receive request failed: %s", err) - return handleGenericEndpointError(err) // FIXME resume state on receiver -> update ResumeToken - } - - // FIXME handle properties from sres - } - - return FilesystemReplicationResult{Done: true} + return plan, tryRes{done: true} } diff --git a/cmd/replication/replication_test.go b/cmd/replication/replication_test.go index 2e6072b..36faf55 100644 --- a/cmd/replication/replication_test.go +++ b/cmd/replication/replication_test.go @@ -51,131 +51,131 @@ func (m *MockIncrementalPathRecorder) Finished() bool { return m.Pos == len(m.Sequence) } -type DiscardCopier struct{} - -func (DiscardCopier) Copy(writer io.Writer, reader io.Reader) (int64, error) { - return 0, nil -} - -type IncrementalPathReplicatorTest struct { - Msg string - Filesystem *replication.Filesystem - Path []*replication.FilesystemVersion - Steps []IncrementalPathSequenceStep -} - -func (test *IncrementalPathReplicatorTest) Test(t *testing.T) { - - t.Log(test.Msg) - - rec := &MockIncrementalPathRecorder{ - T: t, - Sequence: test.Steps, - } - - ctx := context.WithValue(context.Background(), replication.ContextKeyLog, testLog{t}) - - ipr := replication.NewIncrementalPathReplicator() - ipr.Replicate( - ctx, - rec, - rec, - DiscardCopier{}, - test.Filesystem, - test.Path, - ) - - assert.True(t, rec.Finished()) - -} +//type IncrementalPathReplicatorTest struct { +// Msg string +// Filesystem *replication.Filesystem +// Path []*replication.FilesystemVersion +// Steps []IncrementalPathSequenceStep +//} +// +//func (test *IncrementalPathReplicatorTest) Test(t *testing.T) { +// +// t.Log(test.Msg) +// +// rec := &MockIncrementalPathRecorder{ +// T: t, +// Sequence: test.Steps, +// } +// +// ctx := replication.ContextWithLogger(context.Background(), testLog{t}) +// +// ipr := replication.NewIncrementalPathReplicator() +// ipr.Replicate( +// ctx, +// rec, +// rec, +// DiscardCopier{}, +// test.Filesystem, +// test.Path, +// ) +// +// assert.True(t, rec.Finished()) +// +//} type testLog struct { t *testing.T } -func (t testLog) Printf(fmt string, args ...interface{}) { +var _ replication.Logger = testLog{} + +func (t testLog) Infof(fmt string, args ...interface{}) { + t.t.Logf(fmt, args) +} +func (t testLog) Errorf(fmt string, args ...interface{}) { t.t.Logf(fmt, args) } -func TestIncrementalPathReplicator_Replicate(t *testing.T) { - tbl := []IncrementalPathReplicatorTest{ - { - Msg: "generic happy place with resume token", - Filesystem: &replication.Filesystem{ - Path: "foo/bar", - ResumeToken: "blafoo", - }, - Path: fsvlist("@a,1", "@b,2", "@c,3"), - Steps: []IncrementalPathSequenceStep{ - { - SendRequest: &replication.SendReq{ - Filesystem: "foo/bar", - From: "@a,1", - To: "@b,2", - ResumeToken: "blafoo", - }, - SendResponse: &replication.SendRes{ - UsedResumeToken: true, - }, - }, - { - ReceiveRequest: &replication.ReceiveReq{ - Filesystem: "foo/bar", - ClearResumeToken: false, - }, - }, - { - SendRequest: &replication.SendReq{ - Filesystem: "foo/bar", - From: "@b,2", - To: "@c,3", - }, - }, - { - ReceiveRequest: &replication.ReceiveReq{ - Filesystem: "foo/bar", - }, - }, - }, - }, - { - Msg: "no action on empty sequence", - Filesystem: &replication.Filesystem{ - Path: "foo/bar", - }, - Path: fsvlist(), - Steps: []IncrementalPathSequenceStep{}, - }, - { - Msg: "full send on single entry path", - Filesystem: &replication.Filesystem{ - Path: "foo/bar", - }, - Path: fsvlist("@justone,1"), - Steps: []IncrementalPathSequenceStep{ - { - SendRequest: &replication.SendReq{ - Filesystem: "foo/bar", - From: "@justone,1", - To: "", // empty means full send - }, - SendResponse: &replication.SendRes{ - UsedResumeToken: false, - }, - }, - { - ReceiveRequest: &replication.ReceiveReq{ - Filesystem: "foo/bar", - ClearResumeToken: false, - }, - }, - }, - }, - } - - for _, test := range tbl { - test.Test(t) - } - -} +//func TestIncrementalPathReplicator_Replicate(t *testing.T) { +// +// tbl := []IncrementalPathReplicatorTest{ +// { +// Msg: "generic happy place with resume token", +// Filesystem: &replication.Filesystem{ +// Path: "foo/bar", +// ResumeToken: "blafoo", +// }, +// Path: fsvlist("@a,1", "@b,2", "@c,3"), +// Steps: []IncrementalPathSequenceStep{ +// { +// SendRequest: &replication.SendReq{ +// Filesystem: "foo/bar", +// From: "@a,1", +// To: "@b,2", +// ResumeToken: "blafoo", +// }, +// SendResponse: &replication.SendRes{ +// UsedResumeToken: true, +// }, +// }, +// { +// ReceiveRequest: &replication.ReceiveReq{ +// Filesystem: "foo/bar", +// ClearResumeToken: false, +// }, +// }, +// { +// SendRequest: &replication.SendReq{ +// Filesystem: "foo/bar", +// From: "@b,2", +// To: "@c,3", +// }, +// }, +// { +// ReceiveRequest: &replication.ReceiveReq{ +// Filesystem: "foo/bar", +// }, +// }, +// }, +// }, +// { +// Msg: "no action on empty sequence", +// Filesystem: &replication.Filesystem{ +// Path: "foo/bar", +// }, +// Path: fsvlist(), +// Steps: []IncrementalPathSequenceStep{}, +// }, +// { +// Msg: "full send on single entry path", +// Filesystem: &replication.Filesystem{ +// Path: "foo/bar", +// }, +// Path: fsvlist("@justone,1"), +// Steps: []IncrementalPathSequenceStep{ +// { +// SendRequest: &replication.SendReq{ +// Filesystem: "foo/bar", +// From: "@justone,1", +// To: "", // empty means full send +// }, +// SendResponse: &replication.SendRes{ +// UsedResumeToken: false, +// }, +// }, +// { +// ReceiveRequest: &replication.ReceiveReq{ +// Filesystem: "foo/bar", +// ClearResumeToken: false, +// }, +// }, +// }, +// }, +// } +// +// for _, test := range tbl { +// test.Test(t) +// } +// +//} diff --git a/logger/logger.go b/logger/logger.go index fdfe103..b007267 100644 --- a/logger/logger.go +++ b/logger/logger.go @@ -14,7 +14,20 @@ const ( const DefaultUserFieldCapacity = 5 -type Logger struct { +type Logger interface { + WithOutlet(outlet Outlet, level Level) Logger + ReplaceField(field string, val interface{}) Logger + WithField(field string, val interface{}) Logger + WithFields(fields Fields) Logger + WithError(err error) Logger + Debug(msg string) + Info(msg string) + Warn(msg string) + Error(msg string) + Printf(format string, args ...interface{}) +} + +type loggerImpl struct { fields Fields outlets *Outlets outletTimeout time.Duration @@ -22,8 +35,10 @@ type Logger struct { mtx *sync.Mutex } -func NewLogger(outlets *Outlets, outletTimeout time.Duration) *Logger { - return &Logger{ +var _ Logger = &loggerImpl{} + +func NewLogger(outlets *Outlets, outletTimeout time.Duration) Logger { + return &loggerImpl{ make(Fields, DefaultUserFieldCapacity), outlets, outletTimeout, @@ -36,7 +51,7 @@ type outletResult struct { Error error } -func (l *Logger) logInternalError(outlet Outlet, err string) { +func (l *loggerImpl) logInternalError(outlet Outlet, err string) { fields := Fields{} if outlet != nil { if _, ok := outlet.(fmt.Stringer); ok { @@ -54,7 +69,7 @@ func (l *Logger) logInternalError(outlet Outlet, err string) { l.outlets.GetLoggerErrorOutlet().WriteEntry(entry) } -func (l *Logger) log(level Level, msg string) { +func (l *loggerImpl) log(level Level, msg string) { l.mtx.Lock() defer l.mtx.Unlock() @@ -78,12 +93,12 @@ func (l *Logger) log(level Level, msg string) { } -func (l *Logger) WithOutlet(outlet Outlet, level Level) *Logger { +func (l *loggerImpl) WithOutlet(outlet Outlet, level Level) Logger { l.mtx.Lock() defer l.mtx.Unlock() newOutlets := l.outlets.DeepCopy() newOutlets.Add(outlet, level) - child := &Logger{ + child := &loggerImpl{ fields: l.fields, outlets: newOutlets, outletTimeout: l.outletTimeout, @@ -93,9 +108,9 @@ func (l *Logger) WithOutlet(outlet Outlet, level Level) *Logger { } // callers must hold l.mtx -func (l *Logger) forkLogger(field string, val interface{}) *Logger { +func (l *loggerImpl) forkLogger(field string, val interface{}) *loggerImpl { - child := &Logger{ + child := &loggerImpl{ fields: make(Fields, len(l.fields)+1), outlets: l.outlets, outletTimeout: l.outletTimeout, @@ -109,13 +124,13 @@ func (l *Logger) forkLogger(field string, val interface{}) *Logger { return child } -func (l *Logger) ReplaceField(field string, val interface{}) *Logger { +func (l *loggerImpl) ReplaceField(field string, val interface{}) Logger { l.mtx.Lock() defer l.mtx.Unlock() return l.forkLogger(field, val) } -func (l *Logger) WithField(field string, val interface{}) *Logger { +func (l *loggerImpl) WithField(field string, val interface{}) Logger { l.mtx.Lock() defer l.mtx.Unlock() if val, ok := l.fields[field]; ok && val != nil { @@ -125,16 +140,16 @@ func (l *Logger) WithField(field string, val interface{}) *Logger { return l.forkLogger(field, val) } -func (l *Logger) WithFields(fields Fields) (ret *Logger) { +func (l *loggerImpl) WithFields(fields Fields) Logger { // TODO optimize - ret = l + var ret Logger = l for field, value := range fields { ret = ret.WithField(field, value) } return ret } -func (l *Logger) WithError(err error) *Logger { +func (l *loggerImpl) WithError(err error) Logger { val := interface{}(nil) if err != nil { val = err.Error() @@ -142,22 +157,22 @@ func (l *Logger) WithError(err error) *Logger { return l.WithField(FieldError, val) } -func (l *Logger) Debug(msg string) { +func (l *loggerImpl) Debug(msg string) { l.log(Debug, msg) } -func (l *Logger) Info(msg string) { +func (l *loggerImpl) Info(msg string) { l.log(Info, msg) } -func (l *Logger) Warn(msg string) { +func (l *loggerImpl) Warn(msg string) { l.log(Warn, msg) } -func (l *Logger) Error(msg string) { +func (l *loggerImpl) Error(msg string) { l.log(Error, msg) } -func (l *Logger) Printf(format string, args ...interface{}) { +func (l *loggerImpl) Printf(format string, args ...interface{}) { l.log(Error, fmt.Sprintf(format, args...)) } diff --git a/logger/nulllogger.go b/logger/nulllogger.go new file mode 100644 index 0000000..3233dfe --- /dev/null +++ b/logger/nulllogger.go @@ -0,0 +1,22 @@ +package logger + + +type nullLogger struct {} + +var _ Logger = nullLogger{} + +func NewNullLogger() Logger { + return nullLogger{} +} + +func (n nullLogger) WithOutlet(outlet Outlet, level Level) Logger { return n } +func (n nullLogger) ReplaceField(field string, val interface{}) Logger { return n } +func (n nullLogger) WithField(field string, val interface{}) Logger { return n } +func (n nullLogger) WithFields(fields Fields) Logger { return n } +func (n nullLogger) WithError(err error) Logger { return n } +func (nullLogger) Debug(msg string) {} +func (nullLogger) Info(msg string) {} +func (nullLogger) Warn(msg string) {} +func (nullLogger) Error(msg string) {} +func (nullLogger) Printf(format string, args ...interface{}) {} + From 7303d91abf834ee188c589da5731a61dda0906fa Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 11 Aug 2018 12:19:10 +0200 Subject: [PATCH 009/167] WIP state-machine based replication --- cmd/config_job_local.go | 4 +- cmd/config_job_pull.go | 26 +- cmd/config_job_source.go | 4 +- cmd/replication.go | 2 +- cmd/{replication => replication.v2}/diff.go | 0 .../diff_test.go | 0 .../fsreplicationstate_string.go | 32 ++ .../fsreplicationstepstate_string.go | 16 + cmd/{replication => replication.v2}/pdu.pb.go | 0 cmd/{replication => replication.v2}/pdu.proto | 0 .../pdu_extras.go | 0 .../pdu_test.go | 0 cmd/replication.v2/plan.go | 474 ++++++++++++++++++ cmd/replication.v2/replication.go | 137 +++++ .../replication_test.go | 24 +- cmd/replication.v2/replicationstate_string.go | 16 + cmd/replication/replication.go | 472 ----------------- 17 files changed, 716 insertions(+), 491 deletions(-) rename cmd/{replication => replication.v2}/diff.go (100%) rename cmd/{replication => replication.v2}/diff_test.go (100%) create mode 100644 cmd/replication.v2/fsreplicationstate_string.go create mode 100644 cmd/replication.v2/fsreplicationstepstate_string.go rename cmd/{replication => replication.v2}/pdu.pb.go (100%) rename cmd/{replication => replication.v2}/pdu.proto (100%) rename cmd/{replication => replication.v2}/pdu_extras.go (100%) rename cmd/{replication => replication.v2}/pdu_test.go (100%) create mode 100644 cmd/replication.v2/plan.go create mode 100644 cmd/replication.v2/replication.go rename cmd/{replication => replication.v2}/replication_test.go (93%) create mode 100644 cmd/replication.v2/replicationstate_string.go delete mode 100644 cmd/replication/replication.go diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index d66bdce..f1902e1 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -8,7 +8,7 @@ import ( "github.com/pkg/errors" "github.com/zrepl/zrepl/zfs" "sync" - "github.com/zrepl/zrepl/cmd/replication" + "github.com/zrepl/zrepl/cmd/replication.v2" ) type LocalJob struct { @@ -146,7 +146,7 @@ outer: j.mainTask.Log().Debug("replicating from lhs to rhs") j.mainTask.Enter("replicate") - replication.Replicate(ctx, replication.NewEndpointPairPull(sender, receiver)) + replication.Replicate(ctx, replication.NewEndpointPairPull(sender, receiver), nil) // FIXME j.mainTask.Finish() diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index 85685e4..08d5f1e 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -1,6 +1,7 @@ package cmd import ( + "net" "os" "os/signal" "syscall" @@ -12,7 +13,7 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/cmd/replication" + "github.com/zrepl/zrepl/cmd/replication.v2" ) type PullJob struct { @@ -165,7 +166,10 @@ func (j *PullJob) doRun(ctx context.Context) { ConnConfig: STREAMRPC_CONFIG, } - client, err := streamrpc.NewClient(j.Connect, clientConf) + //client, err := streamrpc.NewClient(j.Connect, clientConf) + client, err := streamrpc.NewClient(&tcpConnecter{net.Dialer{ + Timeout: 10*time.Second, + }}, clientConf) defer client.Close() j.task.Enter("pull") @@ -182,10 +186,26 @@ func (j *PullJob) doRun(ctx context.Context) { return } + usr2 := make(chan os.Signal) + defer close(usr2) + signal.Notify(usr2, syscall.SIGUSR2) + defer signal.Stop(usr2) + retryNow := make(chan struct{}, 1) // buffered so we don't leak the goroutine + go func() { + for { + sig := <-usr2 + if sig != nil { + retryNow <- struct{}{} + } else { + break + } + } + }() + ctx = replication.ContextWithLogger(ctx, replicationLogAdaptor{j.task.Log().WithField("subsystem", "replication")}) ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{j.task.Log().WithField("subsystem", "rpc.protocol")}) ctx = context.WithValue(ctx, contextKeyLog, j.task.Log().WithField("subsystem", "rpc.endpoint")) - replication.Replicate(ctx, replication.NewEndpointPairPull(sender, puller)) + replication.Replicate(ctx, replication.NewEndpointPairPull(sender, puller), retryNow) client.Close() j.task.Finish() diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index f6b7851..a3ff349 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -146,7 +146,9 @@ func (j *SourceJob) Pruner(task *Task, side PrunePolicySide, dryRun bool) (p Pru func (j *SourceJob) serve(ctx context.Context, task *Task) { - listener, err := j.Serve.Listen() + //listener, err := j.Serve.Listen() + + listener, err := net.Listen("tcp", ":8888") if err != nil { task.Log().WithError(err).Error("error listening") return diff --git a/cmd/replication.go b/cmd/replication.go index ff18bb0..b7d1c56 100644 --- a/cmd/replication.go +++ b/cmd/replication.go @@ -2,7 +2,7 @@ package cmd import ( "fmt" - "github.com/zrepl/zrepl/cmd/replication" + "github.com/zrepl/zrepl/cmd/replication.v2" "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/zfs" "io" diff --git a/cmd/replication/diff.go b/cmd/replication.v2/diff.go similarity index 100% rename from cmd/replication/diff.go rename to cmd/replication.v2/diff.go diff --git a/cmd/replication/diff_test.go b/cmd/replication.v2/diff_test.go similarity index 100% rename from cmd/replication/diff_test.go rename to cmd/replication.v2/diff_test.go diff --git a/cmd/replication.v2/fsreplicationstate_string.go b/cmd/replication.v2/fsreplicationstate_string.go new file mode 100644 index 0000000..4859c2a --- /dev/null +++ b/cmd/replication.v2/fsreplicationstate_string.go @@ -0,0 +1,32 @@ +// Code generated by "stringer -type=FSReplicationState"; DO NOT EDIT. + +package replication + +import "strconv" + +const ( + _FSReplicationState_name_0 = "FSQueuedFSActive" + _FSReplicationState_name_1 = "FSRetry" + _FSReplicationState_name_2 = "FSPermanentError" + _FSReplicationState_name_3 = "FSCompleted" +) + +var ( + _FSReplicationState_index_0 = [...]uint8{0, 8, 16} +) + +func (i FSReplicationState) String() string { + switch { + case 1 <= i && i <= 2: + i -= 1 + return _FSReplicationState_name_0[_FSReplicationState_index_0[i]:_FSReplicationState_index_0[i+1]] + case i == 4: + return _FSReplicationState_name_1 + case i == 8: + return _FSReplicationState_name_2 + case i == 16: + return _FSReplicationState_name_3 + default: + return "FSReplicationState(" + strconv.FormatInt(int64(i), 10) + ")" + } +} diff --git a/cmd/replication.v2/fsreplicationstepstate_string.go b/cmd/replication.v2/fsreplicationstepstate_string.go new file mode 100644 index 0000000..6f45363 --- /dev/null +++ b/cmd/replication.v2/fsreplicationstepstate_string.go @@ -0,0 +1,16 @@ +// Code generated by "stringer -type=FSReplicationStepState"; DO NOT EDIT. + +package replication + +import "strconv" + +const _FSReplicationStepState_name = "StepPendingStepActiveStepRetryStepPermanentErrorStepCompleted" + +var _FSReplicationStepState_index = [...]uint8{0, 11, 21, 30, 48, 61} + +func (i FSReplicationStepState) String() string { + if i < 0 || i >= FSReplicationStepState(len(_FSReplicationStepState_index)-1) { + return "FSReplicationStepState(" + strconv.FormatInt(int64(i), 10) + ")" + } + return _FSReplicationStepState_name[_FSReplicationStepState_index[i]:_FSReplicationStepState_index[i+1]] +} diff --git a/cmd/replication/pdu.pb.go b/cmd/replication.v2/pdu.pb.go similarity index 100% rename from cmd/replication/pdu.pb.go rename to cmd/replication.v2/pdu.pb.go diff --git a/cmd/replication/pdu.proto b/cmd/replication.v2/pdu.proto similarity index 100% rename from cmd/replication/pdu.proto rename to cmd/replication.v2/pdu.proto diff --git a/cmd/replication/pdu_extras.go b/cmd/replication.v2/pdu_extras.go similarity index 100% rename from cmd/replication/pdu_extras.go rename to cmd/replication.v2/pdu_extras.go diff --git a/cmd/replication/pdu_test.go b/cmd/replication.v2/pdu_test.go similarity index 100% rename from cmd/replication/pdu_test.go rename to cmd/replication.v2/pdu_test.go diff --git a/cmd/replication.v2/plan.go b/cmd/replication.v2/plan.go new file mode 100644 index 0000000..d0833e2 --- /dev/null +++ b/cmd/replication.v2/plan.go @@ -0,0 +1,474 @@ +package replication + +import ( + "context" + "errors" + "fmt" + "io" + "net" + "sort" + "time" +) + +//go:generate stringer -type=ReplicationState +type ReplicationState int + +const ( + Planning ReplicationState = iota + PlanningError + Working + WorkingWait + Completed + ContextDone +) + +type Replication struct { + state ReplicationState + + // Working / WorkingWait + + pending, completed []*FSReplication + + // PlanningError + planningError error + + // ContextDone + contextError error +} + +type FSReplicationState int + +//go:generate stringer -type=FSReplicationState +const ( + FSQueued FSReplicationState = 1 << iota + FSActive + FSRetry + FSPermanentError + FSCompleted +) + +type FSReplication struct { + state FSReplicationState + fs *Filesystem + permanentError error + retryAt time.Time + completed, pending []*FSReplicationStep +} + +func newFSReplicationPermanentError(fs *Filesystem, err error) *FSReplication { + return &FSReplication{ + state: FSPermanentError, + fs: fs, + permanentError: err, + } +} + +type FSReplicationBuilder struct { + r *FSReplication + steps []*FSReplicationStep +} + +func buildNewFSReplication(fs *Filesystem) *FSReplicationBuilder { + return &FSReplicationBuilder{ + r: &FSReplication{ + fs: fs, + pending: make([]*FSReplicationStep, 0), + }, + } +} + +func (b *FSReplicationBuilder) AddStep(from, to *FilesystemVersion) *FSReplication { + step := &FSReplicationStep{ + state: StepPending, + fsrep: b.r, + from: from, + to: to, + } + b.r.pending = append(b.r.pending, step) + return b.r +} + +func (b *FSReplicationBuilder) Complete() *FSReplication { + if len(b.r.pending) > 0 { + b.r.state = FSQueued + } else { + b.r.state = FSCompleted + } + r := b.r + return r +} + +//go:generate stringer -type=FSReplicationStepState +type FSReplicationStepState int + +const ( + StepPending FSReplicationStepState = iota + StepActive + StepRetry + StepPermanentError + StepCompleted +) + +type FSReplicationStep struct { + state FSReplicationStepState + from, to *FilesystemVersion + fsrep *FSReplication + + // both retry and permanent error + err error +} + +func (r *Replication) Drive(ctx context.Context, ep EndpointPair, retryNow chan struct{}) { + for !(r.state == Completed || r.state == ContextDone) { + pre := r.state + preTime := time.Now() + r.doDrive(ctx, ep, retryNow) + delta := time.Now().Sub(preTime) + post := r.state + getLogger(ctx). + WithField("transition", fmt.Sprintf("%s => %s", pre, post)). + WithField("duration", delta). + Debug("state transition") + } +} + +func (r *Replication) doDrive(ctx context.Context, ep EndpointPair, retryNow chan struct{}) { + + switch r.state { + + case Planning: + r.tryBuildPlan(ctx, ep) + + case PlanningError: + w := time.NewTimer(10 * time.Second) // FIXME constant make configurable + defer w.Stop() + select { + case <-ctx.Done(): + r.state = ContextDone + r.contextError = ctx.Err() + case <-retryNow: + r.state = Planning + r.planningError = nil + case <-w.C: + r.state = Planning + r.planningError = nil + } + + case Working: + + if len(r.pending) == 0 { + r.state = Completed + return + } + + sort.Slice(r.pending, func(i, j int) bool { + a, b := r.pending[i], r.pending[j] + statePrio := func(x *FSReplication) int { + if !(x.state == FSQueued || x.state == FSRetry) { + panic(x) + } + if x.state == FSQueued { + return 0 + } else { + return 1 + } + } + aprio, bprio := statePrio(a), statePrio(b) + if aprio != bprio { + return aprio < bprio + } + // now we know they are the same state + if a.state == FSQueued { + return a.nextStepDate().Before(b.nextStepDate()) + } + if a.state == FSRetry { + return a.retryAt.Before(b.retryAt) + } + panic("should not be reached") + }) + + fsrep := r.pending[0] + + if fsrep.state == FSRetry { + r.state = WorkingWait + return + } + if fsrep.state != FSQueued { + panic(fsrep) + } + + fsState := fsrep.takeStep(ctx, ep) + if fsState&(FSPermanentError|FSCompleted) != 0 { + r.pending = r.pending[1:] + r.completed = append(r.completed, fsrep) + } + + case WorkingWait: + fsrep := r.pending[0] + w := time.NewTimer(fsrep.retryAt.Sub(time.Now())) + defer w.Stop() + select { + case <-ctx.Done(): + r.state = ContextDone + r.contextError = ctx.Err() + case <-retryNow: + for _, fsr := range r.pending { + fsr.retryNow() + } + r.state = Working + case <-w.C: + fsrep.retryNow() // avoid timer jitter + r.state = Working + } + } +} + +func (r *Replication) tryBuildPlan(ctx context.Context, ep EndpointPair) ReplicationState { + + log := getLogger(ctx) + + planningError := func(err error) ReplicationState { + r.state = PlanningError + r.planningError = err + return r.state + } + done := func() ReplicationState { + r.state = Working + r.planningError = nil + return r.state + } + + sfss, err := ep.Sender().ListFilesystems(ctx) + if err != nil { + log.WithError(err).Error("error listing sender filesystems") + return planningError(err) + } + + rfss, err := ep.Receiver().ListFilesystems(ctx) + if err != nil { + log.WithError(err).Error("error listing receiver filesystems") + return planningError(err) + } + + r.pending = make([]*FSReplication, 0, len(sfss)) + r.completed = make([]*FSReplication, 0, len(sfss)) + mainlog := log + for _, fs := range sfss { + + log := mainlog.WithField("filesystem", fs.Path) + + log.Info("assessing filesystem") + + sfsvs, err := ep.Sender().ListFilesystemVersions(ctx, fs.Path) + if err != nil { + log.WithError(err).Error("cannot get remote filesystem versions") + return planningError(err) + } + + if len(sfsvs) <= 1 { + err := errors.New("sender does not have any versions") + log.Error(err.Error()) + r.completed = append(r.completed, newFSReplicationPermanentError(fs, err)) + continue + } + + receiverFSExists := false + for _, rfs := range rfss { + if rfs.Path == fs.Path { + receiverFSExists = true + } + } + + var rfsvs []*FilesystemVersion + if receiverFSExists { + rfsvs, err = ep.Receiver().ListFilesystemVersions(ctx, fs.Path) + if err != nil { + if _, ok := err.(FilteredError); ok { + log.Info("receiver ignores filesystem") + continue + } + log.WithError(err).Error("receiver error") + return planningError(err) + } + } else { + rfsvs = []*FilesystemVersion{} + } + + path, conflict := IncrementalPath(rfsvs, sfsvs) + if conflict != nil { + var msg string + path, msg = resolveConflict(conflict) // no shadowing allowed! + if path != nil { + log.WithField("conflict", conflict).Info("conflict") + log.WithField("resolution", msg).Info("automatically resolved") + } else { + log.WithField("conflict", conflict).Error("conflict") + log.WithField("problem", msg).Error("cannot resolve conflict") + } + } + if path == nil { + r.completed = append(r.completed, newFSReplicationPermanentError(fs, conflict)) + continue + } + + fsreplbuilder := buildNewFSReplication(fs) + if len(path) == 1 { + fsreplbuilder.AddStep(nil, path[0]) + } else { + for i := 0; i < len(path)-1; i++ { + fsreplbuilder.AddStep(path[i], path[i+1]) + } + } + fsrepl := fsreplbuilder.Complete() + switch fsrepl.state { + case FSCompleted: + r.completed = append(r.completed, fsreplbuilder.Complete()) + case FSQueued: + r.pending = append(r.pending, fsreplbuilder.Complete()) + default: + panic(fsrepl) + } + + } + + return done() +} + +func (f *FSReplication) nextStepDate() time.Time { + if f.state != FSQueued { + panic(f) + } + ct, err := f.pending[0].to.CreationAsTime() + if err != nil { + panic(err) // FIXME + } + return ct +} + +func (f *FSReplication) takeStep(ctx context.Context, ep EndpointPair) FSReplicationState { + if f.state != FSQueued { + panic(f) + } + + f.state = FSActive + step := f.pending[0] + stepState := step.do(ctx, ep) + + switch stepState { + case StepCompleted: + f.pending = f.pending[1:] + f.completed = append(f.completed, step) + if len(f.pending) > 0 { + f.state = FSQueued + } else { + f.state = FSCompleted + } + + case StepRetry: + f.state = FSRetry + f.retryAt = time.Now().Add(10 * time.Second) // FIXME hardcoded constant + + case StepPermanentError: + f.state = FSPermanentError + + } + return f.state +} + +func (f *FSReplication) retryNow() { + if f.state != FSRetry { + panic(f) + } + f.retryAt = time.Time{} + f.state = FSQueued +} + +func (s *FSReplicationStep) do(ctx context.Context, ep EndpointPair) FSReplicationStepState { + + fs := s.fsrep.fs + + log := getLogger(ctx). + WithField("filesystem", fs.Path). + WithField("step", s.String()) + + updateStateError := func(err error) FSReplicationStepState { + s.err = err + switch err { + case io.EOF: fallthrough + case io.ErrUnexpectedEOF: fallthrough + case io.ErrClosedPipe: + return StepRetry + } + if _, ok := err.(net.Error); ok { + return StepRetry + } + return StepPermanentError + } + + updateStateCompleted := func() FSReplicationStepState { + s.err = nil + s.state = StepCompleted + return s.state + } + + // FIXME refresh fs resume token + fs.ResumeToken = "" + + var sr *SendReq + if fs.ResumeToken != "" { + sr = &SendReq{ + Filesystem: fs.Path, + ResumeToken: fs.ResumeToken, + } + } else if s.from == nil { + sr = &SendReq{ + Filesystem: fs.Path, + From: s.to.RelName(), // FIXME fix protocol to use To, like zfs does internally + } + } else { + sr = &SendReq{ + Filesystem: fs.Path, + From: s.from.RelName(), + To: s.to.RelName(), + } + } + + log.WithField("request", sr).Debug("initiate send request") + sres, sstream, err := ep.Sender().Send(ctx, sr) + if err != nil { + log.WithError(err).Error("send request failed") + return updateStateError(err) + } + if sstream == nil { + err := errors.New("send request did not return a stream, broken endpoint implementation") + return updateStateError(err) + } + + rr := &ReceiveReq{ + Filesystem: fs.Path, + ClearResumeToken: !sres.UsedResumeToken, + } + log.WithField("request", rr).Debug("initiate receive request") + err = ep.Receiver().Receive(ctx, rr, sstream) + if err != nil { + log.WithError(err).Error("receive request failed (might also be error on sender)") + sstream.Close() + // This failure could be due to + // - an unexpected exit of ZFS on the sending side + // - an unexpected exit of ZFS on the receiving side + // - a connectivity issue + return updateStateError(err) + } + log.Info("receive finished") + return updateStateCompleted() + +} + +func (s *FSReplicationStep) String() string { + if s.from == nil { // FIXME: ZFS semantics are that to is nil on non-incremental send + return fmt.Sprintf("%s%s (full)", s.fsrep.fs.Path, s.to.RelName()) + } else { + return fmt.Sprintf("%s(%s => %s)", s.fsrep.fs.Path, s.from.RelName(), s.to.RelName()) + } +} + diff --git a/cmd/replication.v2/replication.go b/cmd/replication.v2/replication.go new file mode 100644 index 0000000..8d567fc --- /dev/null +++ b/cmd/replication.v2/replication.go @@ -0,0 +1,137 @@ +package replication + +import ( + "context" + "fmt" + "github.com/zrepl/zrepl/logger" + "io" +) + +type ReplicationEndpoint interface { + // Does not include placeholder filesystems + ListFilesystems(ctx context.Context) ([]*Filesystem, error) + ListFilesystemVersions(ctx context.Context, fs string) ([]*FilesystemVersion, error) // fix depS + Send(ctx context.Context, r *SendReq) (*SendRes, io.ReadCloser, error) + Receive(ctx context.Context, r *ReceiveReq, sendStream io.ReadCloser) error +} + +type FilteredError struct{ fs string } + +func NewFilteredError(fs string) FilteredError { + return FilteredError{fs} +} + +func (f FilteredError) Error() string { return "endpoint does not allow access to filesystem " + f.fs } + +type ReplicationMode int + +const ( + ReplicationModePull ReplicationMode = iota + ReplicationModePush +) + +type EndpointPair struct { + a, b ReplicationEndpoint + m ReplicationMode +} + +func NewEndpointPairPull(sender, receiver ReplicationEndpoint) EndpointPair { + return EndpointPair{sender, receiver, ReplicationModePull} +} + +func NewEndpointPairPush(sender, receiver ReplicationEndpoint) EndpointPair { + return EndpointPair{receiver, sender, ReplicationModePush} +} + +func (p EndpointPair) Sender() ReplicationEndpoint { + switch p.m { + case ReplicationModePull: + return p.a + case ReplicationModePush: + return p.b + } + panic("should not be reached") + return nil +} + +func (p EndpointPair) Receiver() ReplicationEndpoint { + switch p.m { + case ReplicationModePull: + return p.b + case ReplicationModePush: + return p.a + } + panic("should not be reached") + return nil +} + +func (p EndpointPair) Mode() ReplicationMode { + return p.m +} + +type contextKey int + +const ( + contextKeyLog contextKey = iota +) + +//type Logger interface { +// Infof(fmt string, args ...interface{}) +// Errorf(fmt string, args ...interface{}) +//} + +//var _ Logger = nullLogger{} + +//type nullLogger struct{} +// +//func (nullLogger) Infof(fmt string, args ...interface{}) {} +//func (nullLogger) Errorf(fmt string, args ...interface{}) {} + +type Logger = logger.Logger + +func ContextWithLogger(ctx context.Context, l Logger) context.Context { + return context.WithValue(ctx, contextKeyLog, l) +} + +func getLogger(ctx context.Context) Logger { + l, ok := ctx.Value(contextKeyLog).(Logger) + if !ok { + l = logger.NewNullLogger() + } + return l +} + +func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) { + if noCommonAncestor, ok := conflict.(*ConflictNoCommonAncestor); ok { + if len(noCommonAncestor.SortedReceiverVersions) == 0 { + // FIXME hard-coded replication policy: most recent + // snapshot as source + var mostRecentSnap *FilesystemVersion + for n := len(noCommonAncestor.SortedSenderVersions) - 1; n >= 0; n-- { + if noCommonAncestor.SortedSenderVersions[n].Type == FilesystemVersion_Snapshot { + mostRecentSnap = noCommonAncestor.SortedSenderVersions[n] + break + } + } + if mostRecentSnap == nil { + return nil, "no snapshots available on sender side" + } + return []*FilesystemVersion{mostRecentSnap}, fmt.Sprintf("start replication at most recent snapshot %s", mostRecentSnap.RelName()) + } + } + return nil, "no automated way to handle conflict type" +} + +// Replicate replicates filesystems from ep.Sender() to ep.Receiver(). +// +// All filesystems presented by the sending side are replicated, +// unless the receiver rejects a Receive request with a *FilteredError. +// +// If an error occurs when replicating a filesystem, that error is logged to the logger in ctx. +// Replicate continues with the replication of the remaining file systems. +// Depending on the type of error, failed replications are retried in an unspecified order (currently FIFO). +func Replicate(ctx context.Context, ep EndpointPair, retryNow chan struct{}) { + r := Replication{} + r.Drive(ctx, ep, retryNow) +} + diff --git a/cmd/replication/replication_test.go b/cmd/replication.v2/replication_test.go similarity index 93% rename from cmd/replication/replication_test.go rename to cmd/replication.v2/replication_test.go index 36faf55..00b3868 100644 --- a/cmd/replication/replication_test.go +++ b/cmd/replication.v2/replication_test.go @@ -83,18 +83,18 @@ func (m *MockIncrementalPathRecorder) Finished() bool { // //} -type testLog struct { - t *testing.T -} - -var _ replication.Logger = testLog{} - -func (t testLog) Infof(fmt string, args ...interface{}) { - t.t.Logf(fmt, args) -} -func (t testLog) Errorf(fmt string, args ...interface{}) { - t.t.Logf(fmt, args) -} +//type testLog struct { +// t *testing.T +//} +// +//var _ replication.Logger = testLog{} +// +//func (t testLog) Infof(fmt string, args ...interface{}) { +// t.t.Logf(fmt, args) +//} +//func (t testLog) Errorf(fmt string, args ...interface{}) { +// t.t.Logf(fmt, args) +//} //func TestIncrementalPathReplicator_Replicate(t *testing.T) { diff --git a/cmd/replication.v2/replicationstate_string.go b/cmd/replication.v2/replicationstate_string.go new file mode 100644 index 0000000..7e4c0ac --- /dev/null +++ b/cmd/replication.v2/replicationstate_string.go @@ -0,0 +1,16 @@ +// Code generated by "stringer -type=ReplicationState"; DO NOT EDIT. + +package replication + +import "strconv" + +const _ReplicationState_name = "PlanningPlanningErrorWorkingWorkingWaitCompletedContextDone" + +var _ReplicationState_index = [...]uint8{0, 8, 21, 28, 39, 48, 59} + +func (i ReplicationState) String() string { + if i < 0 || i >= ReplicationState(len(_ReplicationState_index)-1) { + return "ReplicationState(" + strconv.FormatInt(int64(i), 10) + ")" + } + return _ReplicationState_name[_ReplicationState_index[i]:_ReplicationState_index[i+1]] +} diff --git a/cmd/replication/replication.go b/cmd/replication/replication.go deleted file mode 100644 index 8f9ea4d..0000000 --- a/cmd/replication/replication.go +++ /dev/null @@ -1,472 +0,0 @@ -package replication - -import ( - "context" - "fmt" - "github.com/zrepl/zrepl/logger" - "io" - "net" - "sort" - "time" -) - -type ReplicationEndpoint interface { - // Does not include placeholder filesystems - ListFilesystems(ctx context.Context) ([]*Filesystem, error) - ListFilesystemVersions(ctx context.Context, fs string) ([]*FilesystemVersion, error) // fix depS - Send(ctx context.Context, r *SendReq) (*SendRes, io.ReadCloser, error) - Receive(ctx context.Context, r *ReceiveReq, sendStream io.ReadCloser) error -} - -type FilteredError struct{ fs string } - -func NewFilteredError(fs string) FilteredError { - return FilteredError{fs} -} - -func (f FilteredError) Error() string { return "endpoint does not allow access to filesystem " + f.fs } - -type ReplicationMode int - -const ( - ReplicationModePull ReplicationMode = iota - ReplicationModePush -) - -type EndpointPair struct { - a, b ReplicationEndpoint - m ReplicationMode -} - -func NewEndpointPairPull(sender, receiver ReplicationEndpoint) EndpointPair { - return EndpointPair{sender, receiver, ReplicationModePull} -} - -func NewEndpointPairPush(sender, receiver ReplicationEndpoint) EndpointPair { - return EndpointPair{receiver, sender, ReplicationModePush} -} - -func (p EndpointPair) Sender() ReplicationEndpoint { - switch p.m { - case ReplicationModePull: - return p.a - case ReplicationModePush: - return p.b - } - panic("should not be reached") - return nil -} - -func (p EndpointPair) Receiver() ReplicationEndpoint { - switch p.m { - case ReplicationModePull: - return p.b - case ReplicationModePush: - return p.a - } - panic("should not be reached") - return nil -} - -func (p EndpointPair) Mode() ReplicationMode { - return p.m -} - -type contextKey int - -const ( - contextKeyLog contextKey = iota -) - -//type Logger interface { -// Infof(fmt string, args ...interface{}) -// Errorf(fmt string, args ...interface{}) -//} - -//var _ Logger = nullLogger{} - -//type nullLogger struct{} -// -//func (nullLogger) Infof(fmt string, args ...interface{}) {} -//func (nullLogger) Errorf(fmt string, args ...interface{}) {} - -type Logger = logger.Logger - -func ContextWithLogger(ctx context.Context, l Logger) context.Context { - return context.WithValue(ctx, contextKeyLog, l) -} - -func getLogger(ctx context.Context) Logger { - l, ok := ctx.Value(contextKeyLog).(Logger) - if !ok { - l = logger.NewNullLogger() - } - return l -} - -type replicationStep struct { - from, to *FilesystemVersion - fswork *replicateFSWork -} - -func (s *replicationStep) String() string { - if s.from == nil { // FIXME: ZFS semantics are that to is nil on non-incremental send - return fmt.Sprintf("%s%s (full)", s.fswork.fs.Path, s.to.RelName()) - } else { - return fmt.Sprintf("%s(%s => %s)", s.fswork.fs.Path, s.from.RelName(), s.to.RelName()) - } -} - -func newReplicationStep(from, to *FilesystemVersion) *replicationStep { - return &replicationStep{from: from, to: to} -} - -type replicateFSWork struct { - fs *Filesystem - steps []*replicationStep - currentStep int - errorCount int -} - -func newReplicateFSWork(fs *Filesystem) *replicateFSWork { - if fs == nil { - panic("implementation error") - } - return &replicateFSWork{ - fs: fs, - steps: make([]*replicationStep, 0), - } -} - -func newReplicateFSWorkWithConflict(fs *Filesystem, conflict error) *replicateFSWork { - // FIXME ignore conflict for now, but will be useful later when we make the replicationPlan exportable - return &replicateFSWork{ - fs: fs, - steps: make([]*replicationStep, 0), - } -} - -func (r *replicateFSWork) AddStep(step *replicationStep) { - if step == nil { - panic("implementation error") - } - if step.fswork != nil { - panic("implementation error") - } - step.fswork = r - r.steps = append(r.steps, step) -} - -func (w *replicateFSWork) CurrentStepDate() time.Time { - if len(w.steps) == 0 { - return time.Time{} - } - toTime, err := w.steps[w.currentStep].to.CreationAsTime() - if err != nil { - panic(err) // implementation inconsistent: should not admit invalid FilesystemVersion objects - } - return toTime -} - -func (w *replicateFSWork) CurrentStep() *replicationStep { - if w.currentStep >= len(w.steps) { - return nil - } - return w.steps[w.currentStep] -} - -func (w *replicateFSWork) CompleteStep() { - w.currentStep++ -} - -type replicationPlan struct { - fsws []*replicateFSWork -} - -func newReplicationPlan() *replicationPlan { - return &replicationPlan{ - fsws: make([]*replicateFSWork, 0), - } -} - -func (p *replicationPlan) addWork(work *replicateFSWork) { - p.fsws = append(p.fsws, work) -} - -func (p *replicationPlan) executeOldestFirst(ctx context.Context, doStep func(fs *Filesystem, from, to *FilesystemVersion) tryRes) { - log := getLogger(ctx) - - for { - select { - case <-ctx.Done(): - log.WithError(ctx.Err()).Info("aborting replication due to context error") - return - default: - } - - // FIXME poor man's nested priority queue - pending := make([]*replicateFSWork, 0, len(p.fsws)) - for _, fsw := range p.fsws { - if fsw.CurrentStep() != nil { - pending = append(pending, fsw) - } - } - sort.Slice(pending, func(i, j int) bool { - if pending[i].errorCount == pending[j].errorCount { - return pending[i].CurrentStepDate().Before(pending[j].CurrentStepDate()) - } - return pending[i].errorCount < pending[j].errorCount - }) - // pending is now sorted ascending by errorCount,CurrentStep().Creation - - if len(pending) == 0 { - log.Info("replication complete") - return - } - - fsw := pending[0] - step := fsw.CurrentStep() - if step == nil { - panic("implementation error") - } - - log.WithField("step", step).Info("begin replication step") - res := doStep(step.fswork.fs, step.from, step.to) - - if res.done { - log.Info("replication step successful") - fsw.errorCount = 0 - fsw.CompleteStep() - } else { - log.Error("replication step failed, queuing for retry result") - fsw.errorCount++ - } - - } - -} - -func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) { - if noCommonAncestor, ok := conflict.(*ConflictNoCommonAncestor); ok { - if len(noCommonAncestor.SortedReceiverVersions) == 0 { - // FIXME hard-coded replication policy: most recent - // snapshot as source - var mostRecentSnap *FilesystemVersion - for n := len(noCommonAncestor.SortedSenderVersions) - 1; n >= 0; n-- { - if noCommonAncestor.SortedSenderVersions[n].Type == FilesystemVersion_Snapshot { - mostRecentSnap = noCommonAncestor.SortedSenderVersions[n] - break - } - } - if mostRecentSnap == nil { - return nil, "no snapshots available on sender side" - } - return []*FilesystemVersion{mostRecentSnap}, fmt.Sprintf("start replication at most recent snapshot %s", mostRecentSnap.RelName()) - } - } - return nil, "no automated way to handle conflict type" -} - -// Replicate replicates filesystems from ep.Sender() to ep.Receiver(). -// -// All filesystems presented by the sending side are replicated, -// unless the receiver rejects a Receive request with a *FilteredError. -// -// If an error occurs when replicating a filesystem, that error is logged to the logger in ctx. -// Replicate continues with the replication of the remaining file systems. -// Depending on the type of error, failed replications are retried in an unspecified order (currently FIFO). -func Replicate(ctx context.Context, ep EndpointPair) { - - log := getLogger(ctx) - - retryPlanTicker := time.NewTicker(15 * time.Second) // FIXME make configurable - defer retryPlanTicker.Stop() - - var ( - plan *replicationPlan - res tryRes - ) - for { - log.Info("build replication plan") - plan, res = tryBuildReplicationPlan(ctx, ep) - if plan != nil { - break - } - log.WithField("result", res).Error("building replication plan failed, wait for retry timer result") - select { - case <-ctx.Done(): - log.WithError(ctx.Err()).Info("aborting replication because context is done") - return - case <-retryPlanTicker.C: - // TODO also accept an external channel that allows us to tick - } - } - retryPlanTicker.Stop() - - mainlog := log - plan.executeOldestFirst(ctx, func(fs *Filesystem, from, to *FilesystemVersion) tryRes { - - log := mainlog.WithField("filesystem", fs.Path) - - // FIXME refresh fs resume token - fs.ResumeToken = "" - - var sr *SendReq - if fs.ResumeToken != "" { - sr = &SendReq{ - Filesystem: fs.Path, - ResumeToken: fs.ResumeToken, - } - } else if from == nil { - sr = &SendReq{ - Filesystem: fs.Path, - From: to.RelName(), // FIXME fix protocol to use To, like zfs does internally - } - } else { - sr = &SendReq{ - Filesystem: fs.Path, - From: from.RelName(), - To: to.RelName(), - } - } - - log.WithField("request", sr).Debug("initiate send request") - sres, sstream, err := ep.Sender().Send(ctx, sr) - if err != nil { - log.WithError(err).Error("send request failed") - return tryResFromEndpointError(err) - } - if sstream == nil { - log.Error("send request did not return a stream, broken endpoint implementation") - return tryRes{unfixable: true} - } - - rr := &ReceiveReq{ - Filesystem: fs.Path, - ClearResumeToken: !sres.UsedResumeToken, - } - log.WithField("request", rr).Debug("initiate receive request") - err = ep.Receiver().Receive(ctx, rr, sstream) - if err != nil { - log.WithError(err).Error("receive request failed (might also be error on sender)") - sstream.Close() - // This failure could be due to - // - an unexpected exit of ZFS on the sending side - // - an unexpected exit of ZFS on the receiving side - // - a connectivity issue - return tryResFromEndpointError(err) - } - log.Info("receive finished") - return tryRes{done: true} - - }) - -} - -type tryRes struct { - done bool - retry bool - unfixable bool -} - -func tryResFromEndpointError(err error) tryRes { - if _, ok := err.(net.Error); ok { - return tryRes{retry: true} - } - return tryRes{unfixable: true} -} - -func tryBuildReplicationPlan(ctx context.Context, ep EndpointPair) (*replicationPlan, tryRes) { - - log := getLogger(ctx) - - early := func(err error) (*replicationPlan, tryRes) { - return nil, tryResFromEndpointError(err) - } - - sfss, err := ep.Sender().ListFilesystems(ctx) - if err != nil { - log.WithError(err).Error("error listing sender filesystems") - return early(err) - } - - rfss, err := ep.Receiver().ListFilesystems(ctx) - if err != nil { - log.WithError(err).Error("error listing receiver filesystems") - return early(err) - } - - plan := newReplicationPlan() - mainlog := log - for _, fs := range sfss { - - log := mainlog.WithField("filesystem", fs.Path) - - log.Info("assessing filesystem") - - sfsvs, err := ep.Sender().ListFilesystemVersions(ctx, fs.Path) - if err != nil { - log.WithError(err).Error("cannot get remote filesystem versions") - return early(err) - } - - if len(sfsvs) <= 1 { - log.Error("sender does not have any versions") - return nil, tryRes{unfixable: true} - } - - receiverFSExists := false - for _, rfs := range rfss { - if rfs.Path == fs.Path { - receiverFSExists = true - } - } - - var rfsvs []*FilesystemVersion - if receiverFSExists { - rfsvs, err = ep.Receiver().ListFilesystemVersions(ctx, fs.Path) - if err != nil { - if _, ok := err.(FilteredError); ok { - log.Info("receiver ignores filesystem") - continue - } - log.WithError(err).Error("receiver error") - return early(err) - } - } else { - rfsvs = []*FilesystemVersion{} - } - - path, conflict := IncrementalPath(rfsvs, sfsvs) - if conflict != nil { - var msg string - path, msg = resolveConflict(conflict) // no shadowing allowed! - if path != nil { - log.WithField("conflict", conflict).Info("conflict") - log.WithField("resolution", msg).Info("automatically resolved") - } else { - log.WithField("conflict", conflict).Error("conflict") - log.WithField("problem", msg).Error("cannot resolve conflict") - } - } - if path == nil { - plan.addWork(newReplicateFSWorkWithConflict(fs, conflict)) - continue - } - - w := newReplicateFSWork(fs) - if len(path) == 1 { - step := newReplicationStep(nil, path[0]) - w.AddStep(step) - } else { - for i := 0; i < len(path)-1; i++ { - step := newReplicationStep(path[i], path[i+1]) - w.AddStep(step) - } - } - plan.addWork(w) - - } - - return plan, tryRes{done: true} -} From 991f13a3dad24045edd410005bab06e4be5344d9 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 15 Aug 2018 20:29:34 +0200 Subject: [PATCH 010/167] Reporting --- .../fsreplicationstate_string.go | 2 +- .../fsreplicationstepstate_string.go | 4 +- cmd/replication.v2/plan.go | 371 +++++++++++------- cmd/replication.v2/replication.go | 40 +- cmd/replication.v2/replicationstate_string.go | 27 +- cmd/replication.v2/report.go | 97 +++++ 6 files changed, 387 insertions(+), 154 deletions(-) create mode 100644 cmd/replication.v2/report.go diff --git a/cmd/replication.v2/fsreplicationstate_string.go b/cmd/replication.v2/fsreplicationstate_string.go index 4859c2a..ccff931 100644 --- a/cmd/replication.v2/fsreplicationstate_string.go +++ b/cmd/replication.v2/fsreplicationstate_string.go @@ -6,7 +6,7 @@ import "strconv" const ( _FSReplicationState_name_0 = "FSQueuedFSActive" - _FSReplicationState_name_1 = "FSRetry" + _FSReplicationState_name_1 = "FSRetryWait" _FSReplicationState_name_2 = "FSPermanentError" _FSReplicationState_name_3 = "FSCompleted" ) diff --git a/cmd/replication.v2/fsreplicationstepstate_string.go b/cmd/replication.v2/fsreplicationstepstate_string.go index 6f45363..3af34f0 100644 --- a/cmd/replication.v2/fsreplicationstepstate_string.go +++ b/cmd/replication.v2/fsreplicationstepstate_string.go @@ -4,9 +4,9 @@ package replication import "strconv" -const _FSReplicationStepState_name = "StepPendingStepActiveStepRetryStepPermanentErrorStepCompleted" +const _FSReplicationStepState_name = "StepPendingStepRetryStepPermanentErrorStepCompleted" -var _FSReplicationStepState_index = [...]uint8{0, 11, 21, 30, 48, 61} +var _FSReplicationStepState_index = [...]uint8{0, 11, 20, 38, 51} func (i FSReplicationStepState) String() string { if i < 0 || i >= FSReplicationStepState(len(_FSReplicationStepState_index)-1) { diff --git a/cmd/replication.v2/plan.go b/cmd/replication.v2/plan.go index d0833e2..0e1f9b7 100644 --- a/cmd/replication.v2/plan.go +++ b/cmd/replication.v2/plan.go @@ -7,6 +7,7 @@ import ( "io" "net" "sort" + "sync" "time" ) @@ -14,7 +15,7 @@ import ( type ReplicationState int const ( - Planning ReplicationState = iota + Planning ReplicationState = 1 << iota PlanningError Working WorkingWait @@ -22,80 +23,93 @@ const ( ContextDone ) +type replicationQueueItem struct { + retriesSinceLastError int + fsr *FSReplication +} + type Replication struct { + + // lock protects all fields of this struct (but not the fields behind pointers!) + lock sync.Mutex + state ReplicationState // Working / WorkingWait - pending, completed []*FSReplication + pending, completed []*replicationQueueItem + active *replicationQueueItem // PlanningError planningError error // ContextDone contextError error + + sleepUntil time.Time } +//go:generate stringer -type=FSReplicationState type FSReplicationState int -//go:generate stringer -type=FSReplicationState const ( FSQueued FSReplicationState = 1 << iota FSActive - FSRetry + FSRetryWait FSPermanentError FSCompleted ) type FSReplication struct { + lock sync.Mutex state FSReplicationState fs *Filesystem - permanentError error - retryAt time.Time + permanentError error completed, pending []*FSReplicationStep + active *FSReplicationStep } -func newFSReplicationPermanentError(fs *Filesystem, err error) *FSReplication { - return &FSReplication{ - state: FSPermanentError, - fs: fs, +func newReplicationQueueItemPermanentError(fs *Filesystem, err error) *replicationQueueItem { + return &replicationQueueItem{0, &FSReplication{ + state: FSPermanentError, + fs: fs, permanentError: err, - } + }} } -type FSReplicationBuilder struct { - r *FSReplication +type replicationQueueItemBuilder struct { + r *FSReplication steps []*FSReplicationStep } -func buildNewFSReplication(fs *Filesystem) *FSReplicationBuilder { - return &FSReplicationBuilder{ +func buildNewFSReplication(fs *Filesystem) *replicationQueueItemBuilder { + return &replicationQueueItemBuilder{ r: &FSReplication{ - fs: fs, + fs: fs, pending: make([]*FSReplicationStep, 0), }, } } -func (b *FSReplicationBuilder) AddStep(from, to *FilesystemVersion) *FSReplication { +func (b *replicationQueueItemBuilder) AddStep(from, to *FilesystemVersion) *replicationQueueItemBuilder { step := &FSReplicationStep{ state: StepPending, fsrep: b.r, - from: from, - to: to, + from: from, + to: to, } b.r.pending = append(b.r.pending, step) - return b.r + return b } -func (b *FSReplicationBuilder) Complete() *FSReplication { +func (b *replicationQueueItemBuilder) Complete() *replicationQueueItem { if len(b.r.pending) > 0 { b.r.state = FSQueued } else { b.r.state = FSCompleted } r := b.r - return r + return &replicationQueueItem{0, r} } //go:generate stringer -type=FSReplicationStepState @@ -103,13 +117,16 @@ type FSReplicationStepState int const ( StepPending FSReplicationStepState = iota - StepActive StepRetry StepPermanentError StepCompleted ) type FSReplicationStep struct { + // only protects state, err + // from, to and fsrep are assumed to be immutable + lock sync.Mutex + state FSReplicationStepState from, to *FilesystemVersion fsrep *FSReplication @@ -119,7 +136,7 @@ type FSReplicationStep struct { } func (r *Replication) Drive(ctx context.Context, ep EndpointPair, retryNow chan struct{}) { - for !(r.state == Completed || r.state == ContextDone) { + for r.state&(Completed|ContextDone) == 0 { pre := r.state preTime := time.Now() r.doDrive(ctx, ep, retryNow) @@ -128,7 +145,22 @@ func (r *Replication) Drive(ctx context.Context, ep EndpointPair, retryNow chan getLogger(ctx). WithField("transition", fmt.Sprintf("%s => %s", pre, post)). WithField("duration", delta). - Debug("state transition") + Debug("main state transition") + now := time.Now() + sleepDuration := r.sleepUntil.Sub(now) + if sleepDuration > 100*time.Millisecond { + getLogger(ctx). + WithField("duration", sleepDuration). + WithField("wakeup_at", r.sleepUntil). + Error("sleeping until next attempt") + timer := time.NewTimer(sleepDuration) + select { + case <-timer.C: + case <-ctx.Done(): + case <-retryNow: + } + timer.Stop() + } } } @@ -140,86 +172,86 @@ func (r *Replication) doDrive(ctx context.Context, ep EndpointPair, retryNow cha r.tryBuildPlan(ctx, ep) case PlanningError: - w := time.NewTimer(10 * time.Second) // FIXME constant make configurable - defer w.Stop() - select { - case <-ctx.Done(): - r.state = ContextDone - r.contextError = ctx.Err() - case <-retryNow: - r.state = Planning - r.planningError = nil - case <-w.C: - r.state = Planning - r.planningError = nil - } + r.sleepUntil = time.Now().Add(10 * time.Second) // FIXME constant make configurable case Working: - - if len(r.pending) == 0 { - r.state = Completed - return + withLocks := func(f func()) { + r.lock.Lock() + defer r.lock.Unlock() + f() } + withLocks(func() { + if r.active == nil { - sort.Slice(r.pending, func(i, j int) bool { - a, b := r.pending[i], r.pending[j] - statePrio := func(x *FSReplication) int { - if !(x.state == FSQueued || x.state == FSRetry) { - panic(x) - } - if x.state == FSQueued { - return 0 - } else { - return 1 + if len(r.pending) == 0 { + r.state = Completed + return } + + sort.Slice(r.pending, func(i, j int) bool { + a, b := r.pending[i], r.pending[j] + statePrio := func(x *replicationQueueItem) int { + if x.fsr.state&(FSQueued|FSRetryWait) == 0 { + panic(x) + } + if x.fsr.state == FSQueued { + return 0 + } else { + return 1 + } + } + aprio, bprio := statePrio(a), statePrio(b) + if aprio != bprio { + return aprio < bprio + } + // now we know they are the same state + if a.fsr.state == FSQueued { + return a.fsr.nextStepDate().Before(b.fsr.nextStepDate()) + } + if a.fsr.state == FSRetryWait { + return a.retriesSinceLastError < b.retriesSinceLastError + } + panic("should not be reached") + }) + + r.active = r.pending[0] + r.pending = r.pending[1:] } - aprio, bprio := statePrio(a), statePrio(b) - if aprio != bprio { - return aprio < bprio + + if r.active.fsr.state == FSRetryWait { + r.state = WorkingWait + return } - // now we know they are the same state - if a.state == FSQueued { - return a.nextStepDate().Before(b.nextStepDate()) + if r.active.fsr.state != FSQueued { + panic(r.active) } - if a.state == FSRetry { - return a.retryAt.Before(b.retryAt) - } - panic("should not be reached") }) - fsrep := r.pending[0] - - if fsrep.state == FSRetry { - r.state = WorkingWait + if r.active == nil { return } - if fsrep.state != FSQueued { - panic(fsrep) - } - fsState := fsrep.takeStep(ctx, ep) - if fsState&(FSPermanentError|FSCompleted) != 0 { - r.pending = r.pending[1:] - r.completed = append(r.completed, fsrep) - } + fsState := r.active.fsr.drive(ctx, ep) + + withLocks(func() { + + if fsState&FSQueued != 0 { + r.active.retriesSinceLastError = 0 + } else if fsState&FSRetryWait != 0 { + r.active.retriesSinceLastError++ + } else if fsState&(FSPermanentError|FSCompleted) != 0 { + r.completed = append(r.completed, r.active) + r.active = nil + } else { + panic(r.active) + } + }) case WorkingWait: - fsrep := r.pending[0] - w := time.NewTimer(fsrep.retryAt.Sub(time.Now())) - defer w.Stop() - select { - case <-ctx.Done(): - r.state = ContextDone - r.contextError = ctx.Err() - case <-retryNow: - for _, fsr := range r.pending { - fsr.retryNow() - } - r.state = Working - case <-w.C: - fsrep.retryNow() // avoid timer jitter - r.state = Working - } + r.sleepUntil = time.Now().Add(10 * time.Second) // FIXME make configurable + + default: + panic(r.state) } } @@ -227,16 +259,19 @@ func (r *Replication) tryBuildPlan(ctx context.Context, ep EndpointPair) Replica log := getLogger(ctx) + updateLock := func() func() { + r.lock.Lock() + return func() { + r.lock.Unlock() + } + } + planningError := func(err error) ReplicationState { + defer updateLock()() r.state = PlanningError r.planningError = err return r.state } - done := func() ReplicationState { - r.state = Working - r.planningError = nil - return r.state - } sfss, err := ep.Sender().ListFilesystems(ctx) if err != nil { @@ -250,8 +285,8 @@ func (r *Replication) tryBuildPlan(ctx context.Context, ep EndpointPair) Replica return planningError(err) } - r.pending = make([]*FSReplication, 0, len(sfss)) - r.completed = make([]*FSReplication, 0, len(sfss)) + pending := make([]*replicationQueueItem, 0, len(sfss)) + completed := make([]*replicationQueueItem, 0, len(sfss)) mainlog := log for _, fs := range sfss { @@ -268,7 +303,7 @@ func (r *Replication) tryBuildPlan(ctx context.Context, ep EndpointPair) Replica if len(sfsvs) <= 1 { err := errors.New("sender does not have any versions") log.Error(err.Error()) - r.completed = append(r.completed, newFSReplicationPermanentError(fs, err)) + completed = append(completed, newReplicationQueueItemPermanentError(fs, err)) continue } @@ -307,33 +342,40 @@ func (r *Replication) tryBuildPlan(ctx context.Context, ep EndpointPair) Replica } } if path == nil { - r.completed = append(r.completed, newFSReplicationPermanentError(fs, conflict)) + completed = append(completed, newReplicationQueueItemPermanentError(fs, conflict)) continue } - fsreplbuilder := buildNewFSReplication(fs) + builder := buildNewFSReplication(fs) if len(path) == 1 { - fsreplbuilder.AddStep(nil, path[0]) + builder.AddStep(nil, path[0]) } else { for i := 0; i < len(path)-1; i++ { - fsreplbuilder.AddStep(path[i], path[i+1]) + builder.AddStep(path[i], path[i+1]) } } - fsrepl := fsreplbuilder.Complete() - switch fsrepl.state { + qitem := builder.Complete() + switch qitem.fsr.state { case FSCompleted: - r.completed = append(r.completed, fsreplbuilder.Complete()) + completed = append(completed, qitem) case FSQueued: - r.pending = append(r.pending, fsreplbuilder.Complete()) + pending = append(pending, qitem) default: - panic(fsrepl) + panic(qitem) } } - return done() + + defer updateLock()() + r.completed = completed + r.pending = pending + r.state = Working + r.planningError = nil + return r.state } +// caller must have exclusive access to f func (f *FSReplication) nextStepDate() time.Time { if f.state != FSQueued { panic(f) @@ -345,42 +387,70 @@ func (f *FSReplication) nextStepDate() time.Time { return ct } -func (f *FSReplication) takeStep(ctx context.Context, ep EndpointPair) FSReplicationState { - if f.state != FSQueued { - panic(f) - } - - f.state = FSActive - step := f.pending[0] - stepState := step.do(ctx, ep) - - switch stepState { - case StepCompleted: - f.pending = f.pending[1:] - f.completed = append(f.completed, step) - if len(f.pending) > 0 { - f.state = FSQueued - } else { - f.state = FSCompleted - } - - case StepRetry: - f.state = FSRetry - f.retryAt = time.Now().Add(10 * time.Second) // FIXME hardcoded constant - - case StepPermanentError: - f.state = FSPermanentError - +func (f *FSReplication) drive(ctx context.Context, ep EndpointPair) FSReplicationState { + f.lock.Lock() + defer f.lock.Unlock() + for f.state&(FSRetryWait|FSPermanentError|FSCompleted) == 0 { + pre := f.state + preTime := time.Now() + f.doDrive(ctx, ep) + delta := time.Now().Sub(preTime) + post := f.state + getLogger(ctx). + WithField("transition", fmt.Sprintf("%s => %s", pre, post)). + WithField("duration", delta). + Debug("fsr state transition") } return f.state } -func (f *FSReplication) retryNow() { - if f.state != FSRetry { - panic(f) +// caller must hold f.lock +func (f *FSReplication) doDrive(ctx context.Context, ep EndpointPair) FSReplicationState { + switch f.state { + case FSPermanentError: + fallthrough + case FSCompleted: + return f.state + case FSRetryWait: + f.state = FSQueued + return f.state + case FSQueued: + if f.active == nil { + if len(f.pending) == 0 { + f.state = FSCompleted + return f.state + } + f.active = f.pending[0] + f.pending = f.pending[1:] + } + f.state = FSActive + return f.state + + case FSActive: + var stepState FSReplicationStepState + func() { // drop lock during long call + f.lock.Unlock() + defer f.lock.Lock() + stepState = f.active.do(ctx, ep) + }() + switch stepState { + case StepCompleted: + f.completed = append(f.completed, f.active) + f.active = nil + if len(f.pending) > 0 { + f.state = FSQueued + } else { + f.state = FSCompleted + } + case StepRetry: + f.state = FSRetryWait + case StepPermanentError: + f.state = FSPermanentError + } + return f.state } - f.retryAt = time.Time{} - f.state = FSQueued + + panic(f) } func (s *FSReplicationStep) do(ctx context.Context, ep EndpointPair) FSReplicationStepState { @@ -392,20 +462,30 @@ func (s *FSReplicationStep) do(ctx context.Context, ep EndpointPair) FSReplicati WithField("step", s.String()) updateStateError := func(err error) FSReplicationStepState { + s.lock.Lock() + defer s.lock.Unlock() + s.err = err switch err { - case io.EOF: fallthrough - case io.ErrUnexpectedEOF: fallthrough - case io.ErrClosedPipe: - return StepRetry + case io.EOF: + fallthrough + case io.ErrUnexpectedEOF: + fallthrough + case io.ErrClosedPipe: + s.state = StepRetry + return s.state } if _, ok := err.(net.Error); ok { - return StepRetry + s.state = StepRetry + return s.state } - return StepPermanentError + s.state = StepPermanentError + return s.state } updateStateCompleted := func() FSReplicationStepState { + s.lock.Lock() + defer s.lock.Unlock() s.err = nil s.state = StepCompleted return s.state @@ -471,4 +551,3 @@ func (s *FSReplicationStep) String() string { return fmt.Sprintf("%s(%s => %s)", s.fsrep.fs.Path, s.from.RelName(), s.to.RelName()) } } - diff --git a/cmd/replication.v2/replication.go b/cmd/replication.v2/replication.go index 8d567fc..3d4d34c 100644 --- a/cmd/replication.v2/replication.go +++ b/cmd/replication.v2/replication.go @@ -1,10 +1,14 @@ package replication import ( + "os" + "syscall" + "encoding/json" "context" "fmt" "github.com/zrepl/zrepl/logger" "io" + "os/signal" ) type ReplicationEndpoint interface { @@ -131,7 +135,41 @@ func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) { // Replicate continues with the replication of the remaining file systems. // Depending on the type of error, failed replications are retried in an unspecified order (currently FIFO). func Replicate(ctx context.Context, ep EndpointPair, retryNow chan struct{}) { - r := Replication{} + r := Replication{ + state: Planning, + } + + c := make(chan os.Signal) + defer close(c) + signal.Notify(c, syscall.SIGHUP) + go func() { + f, err := os.OpenFile("/tmp/report", os.O_CREATE|os.O_APPEND|os.O_WRONLY, 0600) + if err != nil { + getLogger(ctx).WithError(err).Error("cannot open report file") + panic(err) + } + defer f.Close() + for { + select { + case <-ctx.Done(): + return + case sig := <-c: + if sig == nil { + return + } + report := r.Report() + enc := json.NewEncoder(f) + enc.SetIndent(" ", " ") + if err := enc.Encode(report); err != nil { + getLogger(ctx).WithError(err).Error("cannot encode report") + panic(err) + } + f.Write([]byte("\n")) + f.Sync() + } + } + }() + r.Drive(ctx, ep, retryNow) } diff --git a/cmd/replication.v2/replicationstate_string.go b/cmd/replication.v2/replicationstate_string.go index 7e4c0ac..e4381ba 100644 --- a/cmd/replication.v2/replicationstate_string.go +++ b/cmd/replication.v2/replicationstate_string.go @@ -4,13 +4,32 @@ package replication import "strconv" -const _ReplicationState_name = "PlanningPlanningErrorWorkingWorkingWaitCompletedContextDone" +const ( + _ReplicationState_name_0 = "PlanningPlanningError" + _ReplicationState_name_1 = "Working" + _ReplicationState_name_2 = "WorkingWait" + _ReplicationState_name_3 = "Completed" + _ReplicationState_name_4 = "ContextDone" +) -var _ReplicationState_index = [...]uint8{0, 8, 21, 28, 39, 48, 59} +var ( + _ReplicationState_index_0 = [...]uint8{0, 8, 21} +) func (i ReplicationState) String() string { - if i < 0 || i >= ReplicationState(len(_ReplicationState_index)-1) { + switch { + case 1 <= i && i <= 2: + i -= 1 + return _ReplicationState_name_0[_ReplicationState_index_0[i]:_ReplicationState_index_0[i+1]] + case i == 4: + return _ReplicationState_name_1 + case i == 8: + return _ReplicationState_name_2 + case i == 16: + return _ReplicationState_name_3 + case i == 32: + return _ReplicationState_name_4 + default: return "ReplicationState(" + strconv.FormatInt(int64(i), 10) + ")" } - return _ReplicationState_name[_ReplicationState_index[i]:_ReplicationState_index[i+1]] } diff --git a/cmd/replication.v2/report.go b/cmd/replication.v2/report.go new file mode 100644 index 0000000..7e4ccaa --- /dev/null +++ b/cmd/replication.v2/report.go @@ -0,0 +1,97 @@ +package replication + +type Report struct { + Status string + Problem string + Completed []*FilesystemReplicationReport + Pending []*FilesystemReplicationReport + Active *FilesystemReplicationReport +} + +type StepReport struct { + From, To string + Status string + Problem string +} + +type FilesystemReplicationReport struct { + Filesystem string + Status string + Problem string + Steps []*StepReport +} + +func stepReportFromStep(step *FSReplicationStep) *StepReport { + var from string // FIXME follow same convention as ZFS: to should be nil on full send + if step.from != nil { + from = step.from.RelName() + } + rep := StepReport{ + From: from, + To: step.to.RelName(), + Status: step.state.String(), + } + return &rep +} + +// access to fsr's members must be exclusive +func filesystemReplicationReportFromQueueItem(qitem *replicationQueueItem) *FilesystemReplicationReport { + fsr := qitem.fsr + fsr.lock.Lock() + defer fsr.lock.Unlock() + + rep := FilesystemReplicationReport{ + Filesystem: fsr.fs.Path, + Status: fsr.state.String(), + } + + if fsr.state&FSPermanentError != 0 { + rep.Problem = fsr.permanentError.Error() + return &rep + } + + rep.Steps = make([]*StepReport, 0, len(fsr.completed)+len(fsr.pending) + 1) + for _, step := range fsr.completed { + rep.Steps = append(rep.Steps, stepReportFromStep(step)) + } + if fsr.active != nil { + rep.Steps = append(rep.Steps, stepReportFromStep(fsr.active)) + } + for _, step := range fsr.pending { + rep.Steps = append(rep.Steps, stepReportFromStep(step)) + } + return &rep +} + +func (r *Replication) Report() *Report { + r.lock.Lock() + defer r.lock.Unlock() + + rep := Report{ + Status: r.state.String(), + } + + if r.state&(Planning|PlanningError|ContextDone) != 0 { + switch r.state { + case PlanningError: + rep.Problem = r.planningError.Error() + case ContextDone: + rep.Problem = r.contextError.Error() + } + return &rep + } + + rep.Pending = make([]*FilesystemReplicationReport, 0, len(r.pending)) + rep.Completed = make([]*FilesystemReplicationReport, 0, len(r.completed)) // room for active (potentially) + + for _, qitem := range r.pending { + rep.Pending = append(rep.Pending, filesystemReplicationReportFromQueueItem(qitem)) + } + for _, qitem := range r.completed { + rep.Completed = append(rep.Completed, filesystemReplicationReportFromQueueItem(qitem)) + } + + rep.Active = filesystemReplicationReportFromQueueItem(r.active) + + return &rep +} From 094eced2c7a17759167701517577ecc45a0e5fc9 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 16 Aug 2018 01:26:09 +0200 Subject: [PATCH 011/167] WIP: states with updater func instead of direct locking --- cmd/config_job_control.go | 5 + cmd/config_job_pull.go | 25 +-- cmd/replication.v2/plan.go | 307 +++++++++++++++++++---------------- cmd/replication.v2/report.go | 5 +- 4 files changed, 187 insertions(+), 155 deletions(-) diff --git a/cmd/config_job_control.go b/cmd/config_job_control.go index 1e7baae..300e4b0 100644 --- a/cmd/config_job_control.go +++ b/cmd/config_job_control.go @@ -78,6 +78,11 @@ func (j *ControlJob) JobStart(ctx context.Context) { requestLogger{log: log, handler: jsonResponder{func() (interface{}, error) { return daemon.Status(), nil }}}) + mux.Handle("/pulljobreport", + requestLogger{log: log, handler: jsonResponder{func() (interface{}, error) { + j := daemon.conf.Jobs["debian"] + return j.(*PullJob).Report(), nil + }}}) server := http.Server{Handler: mux} outer: diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index 08d5f1e..a5766c8 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -29,6 +29,7 @@ type PullJob struct { Debug JobDebugSettings task *Task + rep *replication.Replication } func parsePullJob(c JobParsingContext, name string, i map[string]interface{}) (j *PullJob, err error) { @@ -186,26 +187,14 @@ func (j *PullJob) doRun(ctx context.Context) { return } - usr2 := make(chan os.Signal) - defer close(usr2) - signal.Notify(usr2, syscall.SIGUSR2) - defer signal.Stop(usr2) - retryNow := make(chan struct{}, 1) // buffered so we don't leak the goroutine - go func() { - for { - sig := <-usr2 - if sig != nil { - retryNow <- struct{}{} - } else { - break - } - } - }() ctx = replication.ContextWithLogger(ctx, replicationLogAdaptor{j.task.Log().WithField("subsystem", "replication")}) ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{j.task.Log().WithField("subsystem", "rpc.protocol")}) ctx = context.WithValue(ctx, contextKeyLog, j.task.Log().WithField("subsystem", "rpc.endpoint")) - replication.Replicate(ctx, replication.NewEndpointPairPull(sender, puller), retryNow) + + j.rep = &replication.Replication{} + retryNow := make(chan struct{}) + j.rep.Drive(ctx, replication.NewEndpointPairPull(sender, puller), retryNow) client.Close() j.task.Finish() @@ -221,6 +210,10 @@ func (j *PullJob) doRun(ctx context.Context) { } +func (j *PullJob) Report() *replication.Report { + return j.rep.Report() +} + func (j *PullJob) JobStatus(ctxt context.Context) (*JobStatus, error) { return &JobStatus{Tasks: []*TaskStatus{j.task.Status()}}, nil } diff --git a/cmd/replication.v2/plan.go b/cmd/replication.v2/plan.go index 0e1f9b7..be275f0 100644 --- a/cmd/replication.v2/plan.go +++ b/cmd/replication.v2/plan.go @@ -29,14 +29,12 @@ type replicationQueueItem struct { } type Replication struct { - // lock protects all fields of this struct (but not the fields behind pointers!) lock sync.Mutex state ReplicationState - // Working / WorkingWait - + // Working, WorkingWait, Completed, ContextDone pending, completed []*replicationQueueItem active *replicationQueueItem @@ -46,9 +44,13 @@ type Replication struct { // ContextDone contextError error + // PlanningError, WorkingWait sleepUntil time.Time } +type replicationUpdater func(func(*Replication)) +type replicationStateFunc func(context.Context, EndpointPair, replicationUpdater) replicationStateFunc + //go:generate stringer -type=FSReplicationState type FSReplicationState int @@ -136,153 +138,58 @@ type FSReplicationStep struct { } func (r *Replication) Drive(ctx context.Context, ep EndpointPair, retryNow chan struct{}) { - for r.state&(Completed|ContextDone) == 0 { - pre := r.state + + var u replicationUpdater = func(f func(*Replication)) { + r.lock.Lock() + defer r.lock.Unlock() + f(r) + } + + var s replicationStateFunc = rsfPlanning + var pre, post ReplicationState + for s != nil { preTime := time.Now() - r.doDrive(ctx, ep, retryNow) + u(func(r *Replication){ + pre = r.state + }) + s = s(ctx, ep, u) delta := time.Now().Sub(preTime) - post := r.state + u(func(r *Replication){ + post = r.state + }) getLogger(ctx). WithField("transition", fmt.Sprintf("%s => %s", pre, post)). WithField("duration", delta). Debug("main state transition") - now := time.Now() - sleepDuration := r.sleepUntil.Sub(now) - if sleepDuration > 100*time.Millisecond { - getLogger(ctx). - WithField("duration", sleepDuration). - WithField("wakeup_at", r.sleepUntil). - Error("sleeping until next attempt") - timer := time.NewTimer(sleepDuration) - select { - case <-timer.C: - case <-ctx.Done(): - case <-retryNow: - } - timer.Stop() - } } + + getLogger(ctx). + WithField("final_state", post). + Debug("main final state") } -func (r *Replication) doDrive(ctx context.Context, ep EndpointPair, retryNow chan struct{}) { - - switch r.state { - - case Planning: - r.tryBuildPlan(ctx, ep) - - case PlanningError: - r.sleepUntil = time.Now().Add(10 * time.Second) // FIXME constant make configurable - - case Working: - withLocks := func(f func()) { - r.lock.Lock() - defer r.lock.Unlock() - f() - } - withLocks(func() { - if r.active == nil { - - if len(r.pending) == 0 { - r.state = Completed - return - } - - sort.Slice(r.pending, func(i, j int) bool { - a, b := r.pending[i], r.pending[j] - statePrio := func(x *replicationQueueItem) int { - if x.fsr.state&(FSQueued|FSRetryWait) == 0 { - panic(x) - } - if x.fsr.state == FSQueued { - return 0 - } else { - return 1 - } - } - aprio, bprio := statePrio(a), statePrio(b) - if aprio != bprio { - return aprio < bprio - } - // now we know they are the same state - if a.fsr.state == FSQueued { - return a.fsr.nextStepDate().Before(b.fsr.nextStepDate()) - } - if a.fsr.state == FSRetryWait { - return a.retriesSinceLastError < b.retriesSinceLastError - } - panic("should not be reached") - }) - - r.active = r.pending[0] - r.pending = r.pending[1:] - } - - if r.active.fsr.state == FSRetryWait { - r.state = WorkingWait - return - } - if r.active.fsr.state != FSQueued { - panic(r.active) - } - }) - - if r.active == nil { - return - } - - fsState := r.active.fsr.drive(ctx, ep) - - withLocks(func() { - - if fsState&FSQueued != 0 { - r.active.retriesSinceLastError = 0 - } else if fsState&FSRetryWait != 0 { - r.active.retriesSinceLastError++ - } else if fsState&(FSPermanentError|FSCompleted) != 0 { - r.completed = append(r.completed, r.active) - r.active = nil - } else { - panic(r.active) - } - }) - - case WorkingWait: - r.sleepUntil = time.Now().Add(10 * time.Second) // FIXME make configurable - - default: - panic(r.state) - } -} - -func (r *Replication) tryBuildPlan(ctx context.Context, ep EndpointPair) ReplicationState { - +func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { + log := getLogger(ctx) - updateLock := func() func() { - r.lock.Lock() - return func() { - r.lock.Unlock() - } - } - - planningError := func(err error) ReplicationState { - defer updateLock()() - r.state = PlanningError - r.planningError = err - return r.state + handlePlanningError := func(err error) replicationStateFunc { + u(func(r *Replication){ + r.state = PlanningError + r.planningError = err + }) + return rsfPlanningError } sfss, err := ep.Sender().ListFilesystems(ctx) if err != nil { log.WithError(err).Error("error listing sender filesystems") - return planningError(err) + return handlePlanningError(err) } rfss, err := ep.Receiver().ListFilesystems(ctx) if err != nil { log.WithError(err).Error("error listing receiver filesystems") - return planningError(err) + return handlePlanningError(err) } pending := make([]*replicationQueueItem, 0, len(sfss)) @@ -297,7 +204,7 @@ func (r *Replication) tryBuildPlan(ctx context.Context, ep EndpointPair) Replica sfsvs, err := ep.Sender().ListFilesystemVersions(ctx, fs.Path) if err != nil { log.WithError(err).Error("cannot get remote filesystem versions") - return planningError(err) + return handlePlanningError(err) } if len(sfsvs) <= 1 { @@ -323,7 +230,7 @@ func (r *Replication) tryBuildPlan(ctx context.Context, ep EndpointPair) Replica continue } log.WithError(err).Error("receiver error") - return planningError(err) + return handlePlanningError(err) } } else { rfsvs = []*FilesystemVersion{} @@ -366,13 +273,139 @@ func (r *Replication) tryBuildPlan(ctx context.Context, ep EndpointPair) Replica } + u(func(r *Replication){ + r.completed = completed + r.pending = pending + r.state = Working + r.planningError = nil + }) + return rsfWorking +} - defer updateLock()() - r.completed = completed - r.pending = pending - r.state = Working - r.planningError = nil - return r.state +func rsfPlanningError(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { + sleepTime := 10*time.Second + u(func(r *Replication){ + r.sleepUntil = time.Now().Add(sleepTime) + }) + t := time.NewTimer(sleepTime) // FIXME make constant onfigurable + defer t.Stop() + select { + case <- ctx.Done(): + u(func(r *Replication){ + r.state = ContextDone + r.contextError = ctx.Err() + }) + return nil + case <- t.C: + u(func(r *Replication){ + r.state = Planning + }) + return rsfPlanning + } +} + +func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) (rsfNext replicationStateFunc) { + + var active *replicationQueueItem + + u(func(r *Replication) { + if r.active != nil { + active = r.active + return + } + + if len(r.pending) == 0 { + r.state = Completed + return + } + + sort.Slice(r.pending, func(i, j int) bool { + a, b := r.pending[i], r.pending[j] + statePrio := func(x *replicationQueueItem) int { + if x.fsr.state&(FSQueued|FSRetryWait) == 0 { + panic(x) + } + if x.fsr.state == FSQueued { + return 0 + } else { + return 1 + } + } + aprio, bprio := statePrio(a), statePrio(b) + if aprio != bprio { + return aprio < bprio + } + // now we know they are the same state + if a.fsr.state == FSQueued { + return a.fsr.nextStepDate().Before(b.fsr.nextStepDate()) + } + if a.fsr.state == FSRetryWait { + return a.retriesSinceLastError < b.retriesSinceLastError + } + panic("should not be reached") + }) + + r.active = r.pending[0] + active = r.active + r.pending = r.pending[1:] + + }) + + if active == nil { + return rsfNext + } + + if active.fsr.state == FSRetryWait { + u(func(r *Replication) { + r.state = WorkingWait + }) + return rsfWorkingWait + } + if active.fsr.state != FSQueued { + panic(active) + } + + fsState := active.fsr.drive(ctx, ep) + + u(func(r *Replication) { + + if fsState&FSQueued != 0 { + r.active.retriesSinceLastError = 0 + } else if fsState&FSRetryWait != 0 { + r.active.retriesSinceLastError++ + } else if fsState&(FSPermanentError|FSCompleted) != 0 { + r.completed = append(r.completed, r.active) + r.active = nil + } else { + panic(r.active) + } + + }) + + return rsfWorking + +} + +func rsfWorkingWait(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { + sleepTime := 10*time.Second + u(func(r* Replication){ + r.sleepUntil = time.Now().Add(sleepTime) + }) + t := time.NewTimer(sleepTime) + defer t.Stop() + select { + case <- ctx.Done(): + u(func(r *Replication){ + r.state = ContextDone + r.contextError = ctx.Err() + }) + return nil + case <- t.C: + u(func(r *Replication){ + r.state = Working + }) + return rsfWorking + } } // caller must have exclusive access to f diff --git a/cmd/replication.v2/report.go b/cmd/replication.v2/report.go index 7e4ccaa..7fe0ccb 100644 --- a/cmd/replication.v2/report.go +++ b/cmd/replication.v2/report.go @@ -91,7 +91,8 @@ func (r *Replication) Report() *Report { rep.Completed = append(rep.Completed, filesystemReplicationReportFromQueueItem(qitem)) } - rep.Active = filesystemReplicationReportFromQueueItem(r.active) - + if r.active != nil { + rep.Active = filesystemReplicationReportFromQueueItem(r.active) + } return &rep } From 54794637837f1465142069c564142cf12c325352 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 16 Aug 2018 11:02:34 +0200 Subject: [PATCH 012/167] always use ReplicationState, and have a map from that to the rsfs --- cmd/replication.v2/plan.go | 114 +++++++++++++++++++------------------ 1 file changed, 60 insertions(+), 54 deletions(-) diff --git a/cmd/replication.v2/plan.go b/cmd/replication.v2/plan.go index be275f0..d0380ca 100644 --- a/cmd/replication.v2/plan.go +++ b/cmd/replication.v2/plan.go @@ -5,6 +5,7 @@ import ( "errors" "fmt" "io" + "math/bits" "net" "sort" "sync" @@ -12,7 +13,7 @@ import ( ) //go:generate stringer -type=ReplicationState -type ReplicationState int +type ReplicationState uint const ( Planning ReplicationState = 1 << iota @@ -23,6 +24,22 @@ const ( ContextDone ) +func (s ReplicationState) rsf() replicationStateFunc { + idx := bits.TrailingZeros(uint(s)) + if idx == bits.UintSize { + panic(s) // invalid value + } + m := []replicationStateFunc{ + rsfPlanning, + rsfPlanningError, + rsfWorking, + rsfWorkingWait, + nil, + nil, + } + return m[idx] +} + type replicationQueueItem struct { retriesSinceLastError int fsr *FSReplication @@ -48,7 +65,7 @@ type Replication struct { sleepUntil time.Time } -type replicationUpdater func(func(*Replication)) +type replicationUpdater func(func(*Replication)) (newState ReplicationState) type replicationStateFunc func(context.Context, EndpointPair, replicationUpdater) replicationStateFunc //go:generate stringer -type=FSReplicationState @@ -138,25 +155,24 @@ type FSReplicationStep struct { } func (r *Replication) Drive(ctx context.Context, ep EndpointPair, retryNow chan struct{}) { - - var u replicationUpdater = func(f func(*Replication)) { + + var u replicationUpdater = func(f func(*Replication)) ReplicationState { r.lock.Lock() defer r.lock.Unlock() - f(r) + if f != nil { + f(r) + } + return r.state } var s replicationStateFunc = rsfPlanning var pre, post ReplicationState for s != nil { preTime := time.Now() - u(func(r *Replication){ - pre = r.state - }) + pre = u(nil) s = s(ctx, ep, u) delta := time.Now().Sub(preTime) - u(func(r *Replication){ - post = r.state - }) + post = u(nil) getLogger(ctx). WithField("transition", fmt.Sprintf("%s => %s", pre, post)). WithField("duration", delta). @@ -173,11 +189,10 @@ func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) rep log := getLogger(ctx) handlePlanningError := func(err error) replicationStateFunc { - u(func(r *Replication){ - r.state = PlanningError + return u(func(r *Replication) { r.planningError = err - }) - return rsfPlanningError + r.state = PlanningError + }).rsf() } sfss, err := ep.Sender().ListFilesystems(ctx) @@ -273,13 +288,12 @@ func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) rep } - u(func(r *Replication){ + return u(func(r *Replication) { r.completed = completed r.pending = pending - r.state = Working r.planningError = nil - }) - return rsfWorking + r.state = Working + }).rsf() } func rsfPlanningError(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { @@ -290,25 +304,23 @@ func rsfPlanningError(ctx context.Context, ep EndpointPair, u replicationUpdater t := time.NewTimer(sleepTime) // FIXME make constant onfigurable defer t.Stop() select { - case <- ctx.Done(): - u(func(r *Replication){ - r.state = ContextDone - r.contextError = ctx.Err() - }) - return nil - case <- t.C: - u(func(r *Replication){ - r.state = Planning - }) - return rsfPlanning + case <-ctx.Done(): + return u(func(r *Replication) { + r.state = ContextDone + r.contextError = ctx.Err() + }).rsf() + case <-t.C: + return u(func(r *Replication) { + r.state = Planning + }).rsf() } } -func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) (rsfNext replicationStateFunc) { +func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { var active *replicationQueueItem - u(func(r *Replication) { + rsfNext := u(func(r *Replication) { if r.active != nil { active = r.active return @@ -349,17 +361,16 @@ func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) (rsf active = r.active r.pending = r.pending[1:] - }) + }).rsf() if active == nil { return rsfNext } if active.fsr.state == FSRetryWait { - u(func(r *Replication) { + return u(func(r *Replication) { r.state = WorkingWait - }) - return rsfWorkingWait + }).rsf() } if active.fsr.state != FSQueued { panic(active) @@ -367,7 +378,7 @@ func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) (rsf fsState := active.fsr.drive(ctx, ep) - u(func(r *Replication) { + return u(func(r *Replication) { if fsState&FSQueued != 0 { r.active.retriesSinceLastError = 0 @@ -380,31 +391,26 @@ func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) (rsf panic(r.active) } - }) - - return rsfWorking - + }).rsf() } func rsfWorkingWait(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { - sleepTime := 10*time.Second - u(func(r* Replication){ + sleepTime := 10 * time.Second + u(func(r *Replication) { r.sleepUntil = time.Now().Add(sleepTime) }) t := time.NewTimer(sleepTime) defer t.Stop() select { - case <- ctx.Done(): - u(func(r *Replication){ - r.state = ContextDone - r.contextError = ctx.Err() - }) - return nil - case <- t.C: - u(func(r *Replication){ - r.state = Working - }) - return rsfWorking + case <-ctx.Done(): + return u(func(r *Replication) { + r.state = ContextDone + r.contextError = ctx.Err() + }).rsf() + case <-t.C: + return u(func(r *Replication) { + r.state = Working + }).rsf() } } From 93929b61e49e530ba697bc68bce41b53aeff0f01 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 16 Aug 2018 12:01:51 +0200 Subject: [PATCH 013/167] propert locking on FSReplication --- .../fsreplicationstate_string.go | 11 +- cmd/replication.v2/plan.go | 162 +++++++++++------- cmd/replication.v2/report.go | 6 +- 3 files changed, 103 insertions(+), 76 deletions(-) diff --git a/cmd/replication.v2/fsreplicationstate_string.go b/cmd/replication.v2/fsreplicationstate_string.go index ccff931..6002737 100644 --- a/cmd/replication.v2/fsreplicationstate_string.go +++ b/cmd/replication.v2/fsreplicationstate_string.go @@ -5,14 +5,13 @@ package replication import "strconv" const ( - _FSReplicationState_name_0 = "FSQueuedFSActive" - _FSReplicationState_name_1 = "FSRetryWait" - _FSReplicationState_name_2 = "FSPermanentError" - _FSReplicationState_name_3 = "FSCompleted" + _FSReplicationState_name_0 = "FSReadyFSRetryWait" + _FSReplicationState_name_1 = "FSPermanentError" + _FSReplicationState_name_2 = "FSCompleted" ) var ( - _FSReplicationState_index_0 = [...]uint8{0, 8, 16} + _FSReplicationState_index_0 = [...]uint8{0, 7, 18} ) func (i FSReplicationState) String() string { @@ -24,8 +23,6 @@ func (i FSReplicationState) String() string { return _FSReplicationState_name_1 case i == 8: return _FSReplicationState_name_2 - case i == 16: - return _FSReplicationState_name_3 default: return "FSReplicationState(" + strconv.FormatInt(int64(i), 10) + ")" } diff --git a/cmd/replication.v2/plan.go b/cmd/replication.v2/plan.go index d0380ca..c4d534a 100644 --- a/cmd/replication.v2/plan.go +++ b/cmd/replication.v2/plan.go @@ -69,30 +69,45 @@ type replicationUpdater func(func(*Replication)) (newState ReplicationState) type replicationStateFunc func(context.Context, EndpointPair, replicationUpdater) replicationStateFunc //go:generate stringer -type=FSReplicationState -type FSReplicationState int +type FSReplicationState uint const ( - FSQueued FSReplicationState = 1 << iota - FSActive + FSReady FSReplicationState = 1 << iota FSRetryWait FSPermanentError FSCompleted ) +func (s FSReplicationState) fsrsf() fsrsf { + idx := bits.TrailingZeros(uint(s)) + if idx == bits.UintSize { + panic(s) + } + m := []fsrsf{ + fsrsfReady, + fsrsfRetryWait, + nil, + nil, + } + return m[idx] +} + type FSReplication struct { - lock sync.Mutex + // lock protects all fields in this struct, but not the data behind pointers + lock sync.Mutex state FSReplicationState fs *Filesystem - permanentError error + err error + retryWaitUntil time.Time completed, pending []*FSReplicationStep - active *FSReplicationStep + current *FSReplicationStep } func newReplicationQueueItemPermanentError(fs *Filesystem, err error) *replicationQueueItem { return &replicationQueueItem{0, &FSReplication{ - state: FSPermanentError, - fs: fs, - permanentError: err, + state: FSPermanentError, + fs: fs, + err: err, }} } @@ -123,7 +138,7 @@ func (b *replicationQueueItemBuilder) AddStep(from, to *FilesystemVersion) *repl func (b *replicationQueueItemBuilder) Complete() *replicationQueueItem { if len(b.r.pending) > 0 { - b.r.state = FSQueued + b.r.state = FSReady } else { b.r.state = FSCompleted } @@ -185,7 +200,7 @@ func (r *Replication) Drive(ctx context.Context, ep EndpointPair, retryNow chan } func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { - + log := getLogger(ctx) handlePlanningError := func(err error) replicationStateFunc { @@ -280,7 +295,7 @@ func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) rep switch qitem.fsr.state { case FSCompleted: completed = append(completed, qitem) - case FSQueued: + case FSReady: pending = append(pending, qitem) default: panic(qitem) @@ -297,8 +312,8 @@ func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) rep } func rsfPlanningError(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { - sleepTime := 10*time.Second - u(func(r *Replication){ + sleepTime := 10 * time.Second + u(func(r *Replication) { r.sleepUntil = time.Now().Add(sleepTime) }) t := time.NewTimer(sleepTime) // FIXME make constant onfigurable @@ -334,10 +349,10 @@ func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) repl sort.Slice(r.pending, func(i, j int) bool { a, b := r.pending[i], r.pending[j] statePrio := func(x *replicationQueueItem) int { - if x.fsr.state&(FSQueued|FSRetryWait) == 0 { + if x.fsr.state&(FSReady|FSRetryWait) == 0 { panic(x) } - if x.fsr.state == FSQueued { + if x.fsr.state == FSReady { return 0 } else { return 1 @@ -348,7 +363,7 @@ func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) repl return aprio < bprio } // now we know they are the same state - if a.fsr.state == FSQueued { + if a.fsr.state == FSReady { return a.fsr.nextStepDate().Before(b.fsr.nextStepDate()) } if a.fsr.state == FSRetryWait { @@ -367,20 +382,11 @@ func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) repl return rsfNext } - if active.fsr.state == FSRetryWait { - return u(func(r *Replication) { - r.state = WorkingWait - }).rsf() - } - if active.fsr.state != FSQueued { - panic(active) - } - - fsState := active.fsr.drive(ctx, ep) + fsState := active.fsr.takeStep(ctx, ep) return u(func(r *Replication) { - if fsState&FSQueued != 0 { + if fsState&FSReady != 0 { r.active.retriesSinceLastError = 0 } else if fsState&FSRetryWait != 0 { r.active.retriesSinceLastError++ @@ -416,7 +422,7 @@ func rsfWorkingWait(ctx context.Context, ep EndpointPair, u replicationUpdater) // caller must have exclusive access to f func (f *FSReplication) nextStepDate() time.Time { - if f.state != FSQueued { + if f.state != FSReady { panic(f) } ct, err := f.pending[0].to.CreationAsTime() @@ -426,70 +432,94 @@ func (f *FSReplication) nextStepDate() time.Time { return ct } -func (f *FSReplication) drive(ctx context.Context, ep EndpointPair) FSReplicationState { - f.lock.Lock() - defer f.lock.Unlock() - for f.state&(FSRetryWait|FSPermanentError|FSCompleted) == 0 { - pre := f.state +func (f *FSReplication) takeStep(ctx context.Context, ep EndpointPair) FSReplicationState { + + var u fsrUpdater = func(fu func(*FSReplication)) FSReplicationState { + f.lock.Lock() + defer f.lock.Unlock() + if fu != nil { + fu(f) + } + return f.state + } + var s fsrsf = u(nil).fsrsf() + for s != nil { + pre := u(nil) preTime := time.Now() - f.doDrive(ctx, ep) + s = s(ctx, ep, u) delta := time.Now().Sub(preTime) - post := f.state + post := u(nil) getLogger(ctx). + WithField("fs", f.fs.Path). WithField("transition", fmt.Sprintf("%s => %s", pre, post)). WithField("duration", delta). Debug("fsr state transition") } - return f.state + return u(nil) } -// caller must hold f.lock -func (f *FSReplication) doDrive(ctx context.Context, ep EndpointPair) FSReplicationState { - switch f.state { - case FSPermanentError: - fallthrough - case FSCompleted: - return f.state - case FSRetryWait: - f.state = FSQueued - return f.state - case FSQueued: - if f.active == nil { +type fsrUpdater func(func(fsr *FSReplication)) FSReplicationState +type fsrsf func(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf + +func fsrsfReady(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf { + + var current *FSReplicationStep + s := u(func(f *FSReplication) { + if f.current == nil { if len(f.pending) == 0 { f.state = FSCompleted - return f.state + return } - f.active = f.pending[0] + f.current = f.pending[0] f.pending = f.pending[1:] } - f.state = FSActive - return f.state + current = f.current + }) + if s != FSReady { + return s.fsrsf() + } - case FSActive: - var stepState FSReplicationStepState - func() { // drop lock during long call - f.lock.Unlock() - defer f.lock.Lock() - stepState = f.active.do(ctx, ep) - }() + stepState := current.do(ctx, ep) + + return u(func(f *FSReplication) { switch stepState { case StepCompleted: - f.completed = append(f.completed, f.active) - f.active = nil + f.completed = append(f.completed, f.current) + f.current = nil if len(f.pending) > 0 { - f.state = FSQueued + f.state = FSReady } else { f.state = FSCompleted } case StepRetry: + f.retryWaitUntil = time.Now().Add(10 * time.Second) // FIXME make configurable f.state = FSRetryWait case StepPermanentError: f.state = FSPermanentError + default: + panic(f) } - return f.state - } + }).fsrsf() +} - panic(f) +func fsrsfRetryWait(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf { + var sleepUntil time.Time + u(func(f *FSReplication) { + sleepUntil = f.retryWaitUntil + }) + t := time.NewTimer(sleepUntil.Sub(time.Now())) + defer t.Stop() + select { + case <-ctx.Done(): + return u(func(f *FSReplication) { + f.state = FSPermanentError + f.err = ctx.Err() + }).fsrsf() + case <-t.C: + } + return u(func(f *FSReplication) { + f.state = FSReady + }).fsrsf() } func (s *FSReplicationStep) do(ctx context.Context, ep EndpointPair) FSReplicationStepState { diff --git a/cmd/replication.v2/report.go b/cmd/replication.v2/report.go index 7fe0ccb..06310db 100644 --- a/cmd/replication.v2/report.go +++ b/cmd/replication.v2/report.go @@ -46,7 +46,7 @@ func filesystemReplicationReportFromQueueItem(qitem *replicationQueueItem) *File } if fsr.state&FSPermanentError != 0 { - rep.Problem = fsr.permanentError.Error() + rep.Problem = fsr.err.Error() return &rep } @@ -54,8 +54,8 @@ func filesystemReplicationReportFromQueueItem(qitem *replicationQueueItem) *File for _, step := range fsr.completed { rep.Steps = append(rep.Steps, stepReportFromStep(step)) } - if fsr.active != nil { - rep.Steps = append(rep.Steps, stepReportFromStep(fsr.active)) + if fsr.current != nil { + rep.Steps = append(rep.Steps, stepReportFromStep(fsr.current)) } for _, step := range fsr.pending { rep.Steps = append(rep.Steps, stepReportFromStep(step)) From bf1e626b9ab5c7713cee5b970917d36dd87110af Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 16 Aug 2018 14:02:16 +0200 Subject: [PATCH 014/167] proper queue abstraction --- cmd/config_job_pull.go | 2 +- .../fsreplicationstepstate_string.go | 21 +- cmd/replication.v2/plan.go | 292 ++++++++++-------- cmd/replication.v2/replication.go | 6 + cmd/replication.v2/report.go | 18 +- 5 files changed, 199 insertions(+), 140 deletions(-) diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index a5766c8..f3e6c92 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -192,7 +192,7 @@ func (j *PullJob) doRun(ctx context.Context) { ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{j.task.Log().WithField("subsystem", "rpc.protocol")}) ctx = context.WithValue(ctx, contextKeyLog, j.task.Log().WithField("subsystem", "rpc.endpoint")) - j.rep = &replication.Replication{} + j.rep = replication.NewReplication() retryNow := make(chan struct{}) j.rep.Drive(ctx, replication.NewEndpointPairPull(sender, puller), retryNow) diff --git a/cmd/replication.v2/fsreplicationstepstate_string.go b/cmd/replication.v2/fsreplicationstepstate_string.go index 3af34f0..3d075c1 100644 --- a/cmd/replication.v2/fsreplicationstepstate_string.go +++ b/cmd/replication.v2/fsreplicationstepstate_string.go @@ -4,13 +4,26 @@ package replication import "strconv" -const _FSReplicationStepState_name = "StepPendingStepRetryStepPermanentErrorStepCompleted" +const ( + _FSReplicationStepState_name_0 = "StepReadyStepRetry" + _FSReplicationStepState_name_1 = "StepPermanentError" + _FSReplicationStepState_name_2 = "StepCompleted" +) -var _FSReplicationStepState_index = [...]uint8{0, 11, 20, 38, 51} +var ( + _FSReplicationStepState_index_0 = [...]uint8{0, 9, 18} +) func (i FSReplicationStepState) String() string { - if i < 0 || i >= FSReplicationStepState(len(_FSReplicationStepState_index)-1) { + switch { + case 1 <= i && i <= 2: + i -= 1 + return _FSReplicationStepState_name_0[_FSReplicationStepState_index_0[i]:_FSReplicationStepState_index_0[i+1]] + case i == 4: + return _FSReplicationStepState_name_1 + case i == 8: + return _FSReplicationStepState_name_2 + default: return "FSReplicationStepState(" + strconv.FormatInt(int64(i), 10) + ")" } - return _FSReplicationStepState_name[_FSReplicationStepState_index[i]:_FSReplicationStepState_index[i+1]] } diff --git a/cmd/replication.v2/plan.go b/cmd/replication.v2/plan.go index c4d534a..cd62cf1 100644 --- a/cmd/replication.v2/plan.go +++ b/cmd/replication.v2/plan.go @@ -42,6 +42,11 @@ func (s ReplicationState) rsf() replicationStateFunc { type replicationQueueItem struct { retriesSinceLastError int + // duplicates fsr.state to avoid accessing and locking fsr + state FSReplicationState + // duplicates fsr.current.nextStepDate to avoid accessing & locking fsr + nextStepDate time.Time + fsr *FSReplication } @@ -52,8 +57,9 @@ type Replication struct { state ReplicationState // Working, WorkingWait, Completed, ContextDone - pending, completed []*replicationQueueItem - active *replicationQueueItem + queue *replicationQueue + completed []*FSReplication + active *FSReplication // PlanningError planningError error @@ -104,38 +110,17 @@ type FSReplication struct { } func newReplicationQueueItemPermanentError(fs *Filesystem, err error) *replicationQueueItem { - return &replicationQueueItem{0, &FSReplication{ + return &replicationQueueItem{ + retriesSinceLastError: 0, state: FSPermanentError, - fs: fs, - err: err, - }} -} - -type replicationQueueItemBuilder struct { - r *FSReplication - steps []*FSReplicationStep -} - -func buildNewFSReplication(fs *Filesystem) *replicationQueueItemBuilder { - return &replicationQueueItemBuilder{ - r: &FSReplication{ - fs: fs, - pending: make([]*FSReplicationStep, 0), + fsr: &FSReplication{ + state: FSPermanentError, + fs: fs, + err: err, }, } } -func (b *replicationQueueItemBuilder) AddStep(from, to *FilesystemVersion) *replicationQueueItemBuilder { - step := &FSReplicationStep{ - state: StepPending, - fsrep: b.r, - from: from, - to: to, - } - b.r.pending = append(b.r.pending, step) - return b -} - func (b *replicationQueueItemBuilder) Complete() *replicationQueueItem { if len(b.r.pending) > 0 { b.r.state = FSReady @@ -143,14 +128,14 @@ func (b *replicationQueueItemBuilder) Complete() *replicationQueueItem { b.r.state = FSCompleted } r := b.r - return &replicationQueueItem{0, r} + return &replicationQueueItem{0, b.r.state, time.Time{}, r} } //go:generate stringer -type=FSReplicationStepState -type FSReplicationStepState int +type FSReplicationStepState uint const ( - StepPending FSReplicationStepState = iota + StepReady FSReplicationStepState = 1 << iota StepRetry StepPermanentError StepCompleted @@ -222,8 +207,7 @@ func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) rep return handlePlanningError(err) } - pending := make([]*replicationQueueItem, 0, len(sfss)) - completed := make([]*replicationQueueItem, 0, len(sfss)) + q := newReplicationQueue() mainlog := log for _, fs := range sfss { @@ -240,7 +224,7 @@ func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) rep if len(sfsvs) <= 1 { err := errors.New("sender does not have any versions") log.Error(err.Error()) - completed = append(completed, newReplicationQueueItemPermanentError(fs, err)) + q.Add(newReplicationQueueItemPermanentError(fs, err)) continue } @@ -279,11 +263,11 @@ func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) rep } } if path == nil { - completed = append(completed, newReplicationQueueItemPermanentError(fs, conflict)) + q.Add(newReplicationQueueItemPermanentError(fs, conflict)) continue } - builder := buildNewFSReplication(fs) + builder := buildReplicationQueueItem(fs) if len(path) == 1 { builder.AddStep(nil, path[0]) } else { @@ -292,20 +276,12 @@ func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) rep } } qitem := builder.Complete() - switch qitem.fsr.state { - case FSCompleted: - completed = append(completed, qitem) - case FSReady: - pending = append(pending, qitem) - default: - panic(qitem) - } - + q.Add(qitem) } return u(func(r *Replication) { - r.completed = completed - r.pending = pending + r.completed = nil + r.queue = q r.planningError = nil r.state = Working }).rsf() @@ -333,70 +309,25 @@ func rsfPlanningError(ctx context.Context, ep EndpointPair, u replicationUpdater func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { - var active *replicationQueueItem + var active *replicationQueueItemHandle rsfNext := u(func(r *Replication) { - if r.active != nil { - active = r.active - return - } - - if len(r.pending) == 0 { + done, next := r.queue.GetNext() + r.completed = append(r.completed, done...) + if next == nil { r.state = Completed - return } - - sort.Slice(r.pending, func(i, j int) bool { - a, b := r.pending[i], r.pending[j] - statePrio := func(x *replicationQueueItem) int { - if x.fsr.state&(FSReady|FSRetryWait) == 0 { - panic(x) - } - if x.fsr.state == FSReady { - return 0 - } else { - return 1 - } - } - aprio, bprio := statePrio(a), statePrio(b) - if aprio != bprio { - return aprio < bprio - } - // now we know they are the same state - if a.fsr.state == FSReady { - return a.fsr.nextStepDate().Before(b.fsr.nextStepDate()) - } - if a.fsr.state == FSRetryWait { - return a.retriesSinceLastError < b.retriesSinceLastError - } - panic("should not be reached") - }) - - r.active = r.pending[0] - active = r.active - r.pending = r.pending[1:] - + active = next }).rsf() if active == nil { return rsfNext } - fsState := active.fsr.takeStep(ctx, ep) + state, nextStepDate := active.GetFSReplication().takeStep(ctx, ep) return u(func(r *Replication) { - - if fsState&FSReady != 0 { - r.active.retriesSinceLastError = 0 - } else if fsState&FSRetryWait != 0 { - r.active.retriesSinceLastError++ - } else if fsState&(FSPermanentError|FSCompleted) != 0 { - r.completed = append(r.completed, r.active) - r.active = nil - } else { - panic(r.active) - } - + active.Update(state, nextStepDate) }).rsf() } @@ -420,19 +351,121 @@ func rsfWorkingWait(ctx context.Context, ep EndpointPair, u replicationUpdater) } } -// caller must have exclusive access to f -func (f *FSReplication) nextStepDate() time.Time { - if f.state != FSReady { - panic(f) - } - ct, err := f.pending[0].to.CreationAsTime() - if err != nil { - panic(err) // FIXME - } - return ct +type replicationQueueItemBuilder struct { + r *FSReplication + steps []*FSReplicationStep } -func (f *FSReplication) takeStep(ctx context.Context, ep EndpointPair) FSReplicationState { +func buildReplicationQueueItem(fs *Filesystem) *replicationQueueItemBuilder { + return &replicationQueueItemBuilder{ + r: &FSReplication{ + fs: fs, + pending: make([]*FSReplicationStep, 0), + }, + } +} + +func (b *replicationQueueItemBuilder) AddStep(from, to *FilesystemVersion) *replicationQueueItemBuilder { + step := &FSReplicationStep{ + state: StepReady, + fsrep: b.r, + from: from, + to: to, + } + b.r.pending = append(b.r.pending, step) + return b +} + +type replicationQueue []*replicationQueueItem + +func newReplicationQueue() *replicationQueue { + q := make(replicationQueue, 0) + return &q +} + +func (q replicationQueue) Len() int { return len(q) } +func (q replicationQueue) Swap(i,j int) { q[i], q[j] = q[j], q[i]} +func (q replicationQueue) Less(i,j int) bool { + a, b := q[i], q[j] + statePrio := func(x *replicationQueueItem) int { + if x.state&(FSReady|FSRetryWait) == 0 { + panic(x) + } + if x.state== FSReady { + return 0 + } + return 1 + } + + aprio, bprio := statePrio(a), statePrio(b) + if aprio != bprio { + return aprio < bprio + } + // now we know they are the same state + if a.state == FSReady { + return a.nextStepDate.Before(b.nextStepDate) + } + if a.state == FSRetryWait { + return a.retriesSinceLastError < b.retriesSinceLastError + } + panic("should not be reached") +} + +func (q *replicationQueue) sort() (done []*FSReplication) { + // pre-scan for everything that is not ready + newq := make(replicationQueue, 0, len(*q)) + done = make([]*FSReplication, 0, len(*q)) + for _, qitem := range *q { + if qitem.state&(FSReady|FSRetryWait) == 0 { + done = append(done, qitem.fsr) + } else { + newq = append(newq, qitem) + } + } + sort.SortStable(newq) // stable to avoid flickering in reports + *q = newq + return done +} + +// next remains valid until the next call to GetNext() +func (q *replicationQueue) GetNext() (done []*FSReplication, next *replicationQueueItemHandle) { + done = q.sort() + if len(*q) == 0 { + return done, nil + } + next = &replicationQueueItemHandle{(*q)[0]} + return done, next +} + +func (q *replicationQueue) Add(qitem *replicationQueueItem) { + *q = append(*q, qitem) +} + +func (q *replicationQueue) Foreach(fu func(*replicationQueueItemHandle)) { + for _, qitem := range *q { + fu(&replicationQueueItemHandle{qitem}) + } +} + +type replicationQueueItemHandle struct { + i *replicationQueueItem +} + +func (h replicationQueueItemHandle) GetFSReplication() *FSReplication { + return h.i.fsr +} + +func (h replicationQueueItemHandle) Update(newState FSReplicationState, nextStepDate time.Time) { + h.i.state = newState + h.i.nextStepDate = nextStepDate + if h.i.state&FSReady != 0 { + h.i.retriesSinceLastError = 0 + } else if h.i.state&FSRetryWait != 0 { + h.i.retriesSinceLastError++ + } +} + +func (f *FSReplication) takeStep(ctx context.Context, ep EndpointPair) (post FSReplicationState, nextStepDate time.Time) { var u fsrUpdater = func(fu func(*FSReplication)) FSReplicationState { f.lock.Lock() @@ -443,19 +476,29 @@ func (f *FSReplication) takeStep(ctx context.Context, ep EndpointPair) FSReplica return f.state } var s fsrsf = u(nil).fsrsf() - for s != nil { - pre := u(nil) - preTime := time.Now() - s = s(ctx, ep, u) - delta := time.Now().Sub(preTime) - post := u(nil) - getLogger(ctx). - WithField("fs", f.fs.Path). - WithField("transition", fmt.Sprintf("%s => %s", pre, post)). - WithField("duration", delta). - Debug("fsr state transition") - } - return u(nil) + + pre := u(nil) + preTime := time.Now() + s = s(ctx, ep, u) + delta := time.Now().Sub(preTime) + post = u(func(f *FSReplication) { + if f.state != FSReady { + return + } + ct, err := f.current.to.CreationAsTime() + if err != nil { + panic(err) // FIXME + } + nextStepDate = ct + }) + + getLogger(ctx). + WithField("fs", f.fs.Path). + WithField("transition", fmt.Sprintf("%s => %s", pre, post)). + WithField("duration", delta). + Debug("fsr step taken") + + return post, nextStepDate } type fsrUpdater func(func(fsr *FSReplication)) FSReplicationState @@ -496,6 +539,7 @@ func fsrsfReady(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf { f.state = FSRetryWait case StepPermanentError: f.state = FSPermanentError + f.err = errors.New("a replication step failed with a permanent error") default: panic(f) } diff --git a/cmd/replication.v2/replication.go b/cmd/replication.v2/replication.go index 3d4d34c..83fd38b 100644 --- a/cmd/replication.v2/replication.go +++ b/cmd/replication.v2/replication.go @@ -126,6 +126,12 @@ func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) { return nil, "no automated way to handle conflict type" } +func NewReplication() *Replication { + r := Replication{ + state: Planning, + } + return &r +} // Replicate replicates filesystems from ep.Sender() to ep.Receiver(). // // All filesystems presented by the sending side are replicated, diff --git a/cmd/replication.v2/report.go b/cmd/replication.v2/report.go index 06310db..6df5b68 100644 --- a/cmd/replication.v2/report.go +++ b/cmd/replication.v2/report.go @@ -35,8 +35,7 @@ func stepReportFromStep(step *FSReplicationStep) *StepReport { } // access to fsr's members must be exclusive -func filesystemReplicationReportFromQueueItem(qitem *replicationQueueItem) *FilesystemReplicationReport { - fsr := qitem.fsr +func filesystemReplicationReport(fsr *FSReplication) *FilesystemReplicationReport { fsr.lock.Lock() defer fsr.lock.Unlock() @@ -81,18 +80,15 @@ func (r *Replication) Report() *Report { return &rep } - rep.Pending = make([]*FilesystemReplicationReport, 0, len(r.pending)) + rep.Pending = make([]*FilesystemReplicationReport, 0, r.queue.Len()) rep.Completed = make([]*FilesystemReplicationReport, 0, len(r.completed)) // room for active (potentially) - for _, qitem := range r.pending { - rep.Pending = append(rep.Pending, filesystemReplicationReportFromQueueItem(qitem)) - } - for _, qitem := range r.completed { - rep.Completed = append(rep.Completed, filesystemReplicationReportFromQueueItem(qitem)) + r.queue.Foreach(func (h *replicationQueueItemHandle){ + rep.Pending = append(rep.Pending, filesystemReplicationReport(h.GetFSReplication())) + }) + for _, fsr := range r.completed { + rep.Completed = append(rep.Completed, filesystemReplicationReport(fsr)) } - if r.active != nil { - rep.Active = filesystemReplicationReportFromQueueItem(r.active) - } return &rep } From c7d28fee8f3cbb73de23c4bae7f6ace567f776cf Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 16 Aug 2018 14:02:33 +0200 Subject: [PATCH 015/167] gofmt --- cmd/replication.v2/plan.go | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/cmd/replication.v2/plan.go b/cmd/replication.v2/plan.go index cd62cf1..b0045ec 100644 --- a/cmd/replication.v2/plan.go +++ b/cmd/replication.v2/plan.go @@ -45,9 +45,9 @@ type replicationQueueItem struct { // duplicates fsr.state to avoid accessing and locking fsr state FSReplicationState // duplicates fsr.current.nextStepDate to avoid accessing & locking fsr - nextStepDate time.Time + nextStepDate time.Time - fsr *FSReplication + fsr *FSReplication } type Replication struct { @@ -57,9 +57,9 @@ type Replication struct { state ReplicationState // Working, WorkingWait, Completed, ContextDone - queue *replicationQueue + queue *replicationQueue completed []*FSReplication - active *FSReplication + active *FSReplication // PlanningError planningError error @@ -379,19 +379,19 @@ func (b *replicationQueueItemBuilder) AddStep(from, to *FilesystemVersion) *repl type replicationQueue []*replicationQueueItem func newReplicationQueue() *replicationQueue { - q := make(replicationQueue, 0) - return &q + q := make(replicationQueue, 0) + return &q } -func (q replicationQueue) Len() int { return len(q) } -func (q replicationQueue) Swap(i,j int) { q[i], q[j] = q[j], q[i]} -func (q replicationQueue) Less(i,j int) bool { +func (q replicationQueue) Len() int { return len(q) } +func (q replicationQueue) Swap(i, j int) { q[i], q[j] = q[j], q[i] } +func (q replicationQueue) Less(i, j int) bool { a, b := q[i], q[j] statePrio := func(x *replicationQueueItem) int { if x.state&(FSReady|FSRetryWait) == 0 { panic(x) } - if x.state== FSReady { + if x.state == FSReady { return 0 } return 1 From 38532abf452699cca2efcc32eaaea203bd6b80dd Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 16 Aug 2018 21:05:21 +0200 Subject: [PATCH 016/167] enforce encapsulation by breaking up replication package into packages not perfect yet, public shouldn't be required to use 'common' package to use replication package --- Makefile | 9 +- cmd/config_job_local.go | 6 +- cmd/config_job_pull.go | 10 +- cmd/replication.go | 77 +- cmd/replication.v2/diff_test.go | 268 ------- cmd/replication.v2/plan.go | 666 ------------------ cmd/replication.v2/replication.go | 181 ----- cmd/replication.v2/replication_test.go | 181 ----- cmd/replication.v2/report.go | 94 --- cmd/replication/common/common.go | 91 +++ cmd/replication/internal/fsfsm/fsfsm.go | 361 ++++++++++ .../fsfsm}/fsreplicationstate_string.go | 2 +- .../fsfsm}/fsreplicationstepstate_string.go | 2 +- .../internal/mainfsm}/diff.go | 4 +- cmd/replication/internal/mainfsm/mainfsm.go | 342 +++++++++ .../internal/mainfsm/queue/queue.go | 124 ++++ .../mainfsm}/replicationstate_string.go | 2 +- .../pdu}/pdu.pb.go | 92 +-- .../pdu}/pdu.proto | 2 +- .../pdu}/pdu_extras.go | 13 +- .../pdu}/pdu_test.go | 2 +- cmd/replication/replication.go | 19 + 22 files changed, 1057 insertions(+), 1491 deletions(-) delete mode 100644 cmd/replication.v2/diff_test.go delete mode 100644 cmd/replication.v2/plan.go delete mode 100644 cmd/replication.v2/replication.go delete mode 100644 cmd/replication.v2/replication_test.go delete mode 100644 cmd/replication.v2/report.go create mode 100644 cmd/replication/common/common.go create mode 100644 cmd/replication/internal/fsfsm/fsfsm.go rename cmd/{replication.v2 => replication/internal/fsfsm}/fsreplicationstate_string.go (97%) rename cmd/{replication.v2 => replication/internal/fsfsm}/fsreplicationstepstate_string.go (97%) rename cmd/{replication.v2 => replication/internal/mainfsm}/diff.go (98%) create mode 100644 cmd/replication/internal/mainfsm/mainfsm.go create mode 100644 cmd/replication/internal/mainfsm/queue/queue.go rename cmd/{replication.v2 => replication/internal/mainfsm}/replicationstate_string.go (97%) rename cmd/{replication.v2 => replication/pdu}/pdu.pb.go (72%) rename cmd/{replication.v2 => replication/pdu}/pdu.proto (98%) rename cmd/{replication.v2 => replication/pdu}/pdu_extras.go (87%) rename cmd/{replication.v2 => replication/pdu}/pdu_test.go (98%) create mode 100644 cmd/replication/replication.go diff --git a/Makefile b/Makefile index 3970783..5944a64 100644 --- a/Makefile +++ b/Makefile @@ -2,7 +2,12 @@ .DEFAULT_GOAL := build ROOT := github.com/zrepl/zrepl -SUBPKGS := cmd logger rpc util zfs +SUBPKGS := cmd +SUBPKGS += cmd/replication +SUBPKGS += cmd/replication/internal/common +SUBPKGS += cmd/replication/internal/mainfsm +SUBPKGS += cmd/replication/internal/fsfsm +SUBPKGS += logger util zfs _TESTPKGS := $(ROOT) $(foreach p,$(SUBPKGS),$(ROOT)/$(p)) @@ -26,10 +31,10 @@ vendordeps: dep ensure -v -vendor-only generate: #not part of the build, must do that manually + protoc -I=cmd/replication/pdu --go_out=cmd/replication/pdu cmd/replication/pdu/pdu.proto @for pkg in $(_TESTPKGS); do\ go generate "$$pkg" || exit 1; \ done; - protoc -I=cmd/replication --go_out=cmd/replication cmd/replication/pdu.proto # FIXME fix docker build! diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index f1902e1..347a9d8 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -8,7 +8,8 @@ import ( "github.com/pkg/errors" "github.com/zrepl/zrepl/zfs" "sync" - "github.com/zrepl/zrepl/cmd/replication.v2" + "github.com/zrepl/zrepl/cmd/replication" + "github.com/zrepl/zrepl/cmd/replication/common" ) type LocalJob struct { @@ -146,7 +147,8 @@ outer: j.mainTask.Log().Debug("replicating from lhs to rhs") j.mainTask.Enter("replicate") - replication.Replicate(ctx, replication.NewEndpointPairPull(sender, receiver), nil) // FIXME + rep := replication.NewReplication() + rep.Drive(ctx, common.NewEndpointPairPull(sender, receiver)) j.mainTask.Finish() diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index f3e6c92..bebfbd3 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -13,7 +13,8 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/cmd/replication.v2" + "github.com/zrepl/zrepl/cmd/replication" + "github.com/zrepl/zrepl/cmd/replication/common" ) type PullJob struct { @@ -29,7 +30,7 @@ type PullJob struct { Debug JobDebugSettings task *Task - rep *replication.Replication + rep replication.Replication } func parsePullJob(c JobParsingContext, name string, i map[string]interface{}) (j *PullJob, err error) { @@ -188,13 +189,12 @@ func (j *PullJob) doRun(ctx context.Context) { } - ctx = replication.ContextWithLogger(ctx, replicationLogAdaptor{j.task.Log().WithField("subsystem", "replication")}) + ctx = common.ContextWithLogger(ctx, replicationLogAdaptor{j.task.Log().WithField("subsystem", "replication")}) ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{j.task.Log().WithField("subsystem", "rpc.protocol")}) ctx = context.WithValue(ctx, contextKeyLog, j.task.Log().WithField("subsystem", "rpc.endpoint")) j.rep = replication.NewReplication() - retryNow := make(chan struct{}) - j.rep.Drive(ctx, replication.NewEndpointPairPull(sender, puller), retryNow) + j.rep.Drive(ctx, common.NewEndpointPairPull(sender, puller)) client.Close() j.task.Finish() diff --git a/cmd/replication.go b/cmd/replication.go index b7d1c56..d0f3ed9 100644 --- a/cmd/replication.go +++ b/cmd/replication.go @@ -2,7 +2,8 @@ package cmd import ( "fmt" - "github.com/zrepl/zrepl/cmd/replication.v2" + "github.com/zrepl/zrepl/cmd/replication/common" + "github.com/zrepl/zrepl/cmd/replication/pdu" "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/zfs" "io" @@ -31,14 +32,14 @@ func NewSenderEndpoint(fsf zfs.DatasetFilter, fsvf zfs.FilesystemVersionFilter) return &SenderEndpoint{fsf, fsvf} } -func (p *SenderEndpoint) ListFilesystems(ctx context.Context) ([]*replication.Filesystem, error) { +func (p *SenderEndpoint) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) { fss, err := zfs.ZFSListMapping(p.FSFilter) if err != nil { return nil, err } - rfss := make([]*replication.Filesystem, len(fss)) + rfss := make([]*pdu.Filesystem, len(fss)) for i := range fss { - rfss[i] = &replication.Filesystem{ + rfss[i] = &pdu.Filesystem{ Path: fss[i].ToString(), // FIXME: not supporting ResumeToken yet } @@ -46,7 +47,7 @@ func (p *SenderEndpoint) ListFilesystems(ctx context.Context) ([]*replication.Fi return rfss, nil } -func (p *SenderEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ([]*replication.FilesystemVersion, error) { +func (p *SenderEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) { dp, err := zfs.NewDatasetPath(fs) if err != nil { return nil, err @@ -56,20 +57,20 @@ func (p *SenderEndpoint) ListFilesystemVersions(ctx context.Context, fs string) return nil, err } if !pass { - return nil, replication.NewFilteredError(fs) + return nil, common.NewFilteredError(fs) } fsvs, err := zfs.ZFSListFilesystemVersions(dp, p.FilesystemVersionFilter) if err != nil { return nil, err } - rfsvs := make([]*replication.FilesystemVersion, len(fsvs)) + rfsvs := make([]*pdu.FilesystemVersion, len(fsvs)) for i := range fsvs { - rfsvs[i] = replication.FilesystemVersionFromZFS(fsvs[i]) + rfsvs[i] = pdu.FilesystemVersionFromZFS(fsvs[i]) } return rfsvs, nil } -func (p *SenderEndpoint) Send(ctx context.Context, r *replication.SendReq) (*replication.SendRes, io.ReadCloser, error) { +func (p *SenderEndpoint) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) { dp, err := zfs.NewDatasetPath(r.Filesystem) if err != nil { return nil, nil, err @@ -79,16 +80,16 @@ func (p *SenderEndpoint) Send(ctx context.Context, r *replication.SendReq) (*rep return nil, nil, err } if !pass { - return nil, nil, replication.NewFilteredError(r.Filesystem) + return nil, nil, common.NewFilteredError(r.Filesystem) } stream, err := zfs.ZFSSend(r.Filesystem, r.From, r.To) if err != nil { return nil, nil, err } - return &replication.SendRes{}, stream, nil + return &pdu.SendRes{}, stream, nil } -func (p *SenderEndpoint) Receive(ctx context.Context, r *replication.ReceiveReq, sendStream io.ReadCloser) (error) { +func (p *SenderEndpoint) Receive(ctx context.Context, r *pdu.ReceiveReq, sendStream io.ReadCloser) (error) { return fmt.Errorf("sender endpoint does not receive") } @@ -108,23 +109,23 @@ func NewReceiverEndpoint(fsmap *DatasetMapFilter, fsvf zfs.FilesystemVersionFilt return &ReceiverEndpoint{fsmapInv, fsmap, fsvf}, nil } -func (e *ReceiverEndpoint) ListFilesystems(ctx context.Context) ([]*replication.Filesystem, error) { +func (e *ReceiverEndpoint) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) { filtered, err := zfs.ZFSListMapping(e.fsmapInv.AsFilter()) if err != nil { return nil, errors.Wrap(err, "error checking client permission") } - fss := make([]*replication.Filesystem, len(filtered)) + fss := make([]*pdu.Filesystem, len(filtered)) for i, a := range filtered { mapped, err := e.fsmapInv.Map(a) if err != nil { return nil, err } - fss[i] = &replication.Filesystem{Path: mapped.ToString()} + fss[i] = &pdu.Filesystem{Path: mapped.ToString()} } return fss, nil } -func (e *ReceiverEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ([]*replication.FilesystemVersion, error) { +func (e *ReceiverEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) { p, err := zfs.NewDatasetPath(fs) if err != nil { return nil, err @@ -142,19 +143,19 @@ func (e *ReceiverEndpoint) ListFilesystemVersions(ctx context.Context, fs string return nil, err } - rfsvs := make([]*replication.FilesystemVersion, len(fsvs)) + rfsvs := make([]*pdu.FilesystemVersion, len(fsvs)) for i := range fsvs { - rfsvs[i] = replication.FilesystemVersionFromZFS(fsvs[i]) + rfsvs[i] = pdu.FilesystemVersionFromZFS(fsvs[i]) } return rfsvs, nil } -func (e *ReceiverEndpoint) Send(ctx context.Context, req *replication.SendReq) (*replication.SendRes, io.ReadCloser, error) { +func (e *ReceiverEndpoint) Send(ctx context.Context, req *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) { return nil, nil, errors.New("receiver endpoint does not send") } -func (e *ReceiverEndpoint) Receive(ctx context.Context, req *replication.ReceiveReq, sendStream io.ReadCloser) error { +func (e *ReceiverEndpoint) Receive(ctx context.Context, req *pdu.ReceiveReq, sendStream io.ReadCloser) error { defer sendStream.Close() p, err := zfs.NewDatasetPath(req.Filesystem) @@ -236,8 +237,8 @@ type RemoteEndpoint struct { *streamrpc.Client } -func (s RemoteEndpoint) ListFilesystems(ctx context.Context) ([]*replication.Filesystem, error) { - req := replication.ListFilesystemReq{} +func (s RemoteEndpoint) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) { + req := pdu.ListFilesystemReq{} b, err := proto.Marshal(&req) if err != nil { return nil, err @@ -250,15 +251,15 @@ func (s RemoteEndpoint) ListFilesystems(ctx context.Context) ([]*replication.Fil rs.Close() return nil, errors.New("response contains unexpected stream") } - var res replication.ListFilesystemRes + var res pdu.ListFilesystemRes if err := proto.Unmarshal(rb.Bytes(), &res); err != nil { return nil, err } return res.Filesystems, nil } -func (s RemoteEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ([]*replication.FilesystemVersion, error) { - req := replication.ListFilesystemVersionsReq{ +func (s RemoteEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) { + req := pdu.ListFilesystemVersionsReq{ Filesystem: fs, } b, err := proto.Marshal(&req) @@ -273,14 +274,14 @@ func (s RemoteEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ( rs.Close() return nil, errors.New("response contains unexpected stream") } - var res replication.ListFilesystemVersionsRes + var res pdu.ListFilesystemVersionsRes if err := proto.Unmarshal(rb.Bytes(), &res); err != nil { return nil, err } return res.Versions, nil } -func (s RemoteEndpoint) Send(ctx context.Context, r *replication.SendReq) (*replication.SendRes, io.ReadCloser, error) { +func (s RemoteEndpoint) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) { b, err := proto.Marshal(r) if err != nil { return nil, nil, err @@ -292,7 +293,7 @@ func (s RemoteEndpoint) Send(ctx context.Context, r *replication.SendReq) (*repl if rs == nil { return nil, nil, errors.New("response does not contain a stream") } - var res replication.SendRes + var res pdu.SendRes if err := proto.Unmarshal(rb.Bytes(), &res); err != nil { rs.Close() return nil, nil, err @@ -301,7 +302,7 @@ func (s RemoteEndpoint) Send(ctx context.Context, r *replication.SendReq) (*repl return &res, rs, nil } -func (s RemoteEndpoint) Receive(ctx context.Context, r *replication.ReceiveReq, sendStream io.ReadCloser) (error) { +func (s RemoteEndpoint) Receive(ctx context.Context, r *pdu.ReceiveReq, sendStream io.ReadCloser) (error) { defer sendStream.Close() b, err := proto.Marshal(r) if err != nil { @@ -315,7 +316,7 @@ func (s RemoteEndpoint) Receive(ctx context.Context, r *replication.ReceiveReq, rs.Close() return errors.New("response contains unexpected stream") } - var res replication.ReceiveRes + var res pdu.ReceiveRes if err := proto.Unmarshal(rb.Bytes(), &res); err != nil { return err } @@ -323,14 +324,14 @@ func (s RemoteEndpoint) Receive(ctx context.Context, r *replication.ReceiveReq, } type HandlerAdaptor struct { - ep replication.ReplicationEndpoint + ep common.ReplicationEndpoint } func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructured *bytes.Buffer, reqStream io.ReadCloser) (resStructured *bytes.Buffer, resStream io.ReadCloser, err error) { switch endpoint { case RPCListFilesystems: - var req replication.ListFilesystemReq + var req pdu.ListFilesystemReq if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { return nil, nil, err } @@ -338,7 +339,7 @@ func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructu if err != nil { return nil, nil, err } - res := &replication.ListFilesystemRes{ + res := &pdu.ListFilesystemRes{ Filesystems: fsses, } b, err := proto.Marshal(res) @@ -349,7 +350,7 @@ func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructu case RPCListFilesystemVersions: - var req replication.ListFilesystemVersionsReq + var req pdu.ListFilesystemVersionsReq if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { return nil, nil, err } @@ -357,7 +358,7 @@ func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructu if err != nil { return nil, nil, err } - res := &replication.ListFilesystemVersionsRes{ + res := &pdu.ListFilesystemVersionsRes{ Versions: fsvs, } b, err := proto.Marshal(res) @@ -368,7 +369,7 @@ func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructu case RPCSend: - var req replication.SendReq + var req pdu.SendReq if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { return nil, nil, err } @@ -384,7 +385,7 @@ func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructu case RPCReceive: - var req replication.ReceiveReq + var req pdu.ReceiveReq if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { return nil, nil, err } @@ -392,7 +393,7 @@ func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructu if err != nil { return nil, nil, err } - b, err := proto.Marshal(&replication.ReceiveRes{}) + b, err := proto.Marshal(&pdu.ReceiveRes{}) if err != nil { return nil, nil, err } diff --git a/cmd/replication.v2/diff_test.go b/cmd/replication.v2/diff_test.go deleted file mode 100644 index cf9b771..0000000 --- a/cmd/replication.v2/diff_test.go +++ /dev/null @@ -1,268 +0,0 @@ -package replication_test - -import ( - "github.com/stretchr/testify/assert" - "github.com/zrepl/zrepl/cmd/replication" - "strconv" - "strings" - "testing" - "time" -) - -func fsvlist(fsv ...string) (r []*replication.FilesystemVersion) { - - r = make([]*replication.FilesystemVersion, len(fsv)) - for i, f := range fsv { - - // parse the id from fsvlist. it is used to derivce Guid,CreateTXG and Creation attrs - split := strings.Split(f, ",") - if len(split) != 2 { - panic("invalid fsv spec") - } - id, err := strconv.Atoi(split[1]) - if err != nil { - panic(err) - } - - if strings.HasPrefix(f, "#") { - r[i] = &replication.FilesystemVersion{ - Name: strings.TrimPrefix(f, "#"), - Type: replication.FilesystemVersion_Bookmark, - Guid: uint64(id), - CreateTXG: uint64(id), - Creation: time.Unix(0, 0).Add(time.Duration(id) * time.Second).Format(time.RFC3339), - } - } else if strings.HasPrefix(f, "@") { - r[i] = &replication.FilesystemVersion{ - Name: strings.TrimPrefix(f, "@"), - Type: replication.FilesystemVersion_Snapshot, - Guid: uint64(id), - CreateTXG: uint64(id), - Creation: time.Unix(0, 0).Add(time.Duration(id) * time.Second).Format(time.RFC3339), - } - } else { - panic("invalid character") - } - } - return -} - -type incPathResult struct { - incPath []*replication.FilesystemVersion - conflict error -} - -type IncrementalPathTest struct { - Msg string - Receiver, Sender []*replication.FilesystemVersion - ExpectIncPath []*replication.FilesystemVersion - ExpectNoCommonAncestor bool - ExpectDiverged *replication.ConflictDiverged - ExpectPanic bool -} - -func (tt *IncrementalPathTest) Test(t *testing.T) { - - t.Logf("test: %s", tt.Msg) - - if tt.ExpectPanic { - assert.Panics(t, func() { - replication.IncrementalPath(tt.Receiver, tt.Sender) - }) - return - } - - incPath, conflict := replication.IncrementalPath(tt.Receiver, tt.Sender) - - if tt.ExpectIncPath != nil { - assert.Nil(t, conflict) - assert.True(t, len(incPath) == 0 || len(incPath) >= 2) - assert.Equal(t, tt.ExpectIncPath, incPath) - return - } - if conflict == nil { - t.Logf("conflict is (unexpectly) \nincPath: %#v", incPath) - } - if tt.ExpectNoCommonAncestor { - assert.IsType(t, &replication.ConflictNoCommonAncestor{}, conflict) - // TODO check sorting - return - } - if tt.ExpectDiverged != nil { - if !assert.IsType(t, &replication.ConflictDiverged{}, conflict) { - return - } - c := conflict.(*replication.ConflictDiverged) - // TODO check sorting - assert.NotZero(t, c.CommonAncestor) - assert.NotEmpty(t, c.ReceiverOnly) - assert.Equal(t, tt.ExpectDiverged.ReceiverOnly, c.ReceiverOnly) - assert.Equal(t, tt.ExpectDiverged.SenderOnly, c.SenderOnly) - return - } - -} - -func TestIncrementalPlan_IncrementalSnapshots(t *testing.T) { - l := fsvlist - - tbl := []IncrementalPathTest{ - { - Msg: "basic functionality", - Receiver: l("@a,1", "@b,2"), - Sender: l("@a,1", "@b,2", "@c,3", "@d,4"), - ExpectIncPath: l("@b,2", "@c,3", "@d,4"), - }, - { - Msg: "no snaps on receiver yields no common ancestor", - Receiver: l(), - Sender: l("@a,1"), - ExpectNoCommonAncestor: true, - }, - { - Msg: "no snapshots on sender yields empty incremental path", - Receiver: l(), - Sender: l(), - ExpectIncPath: l(), - }, - { - Msg: "nothing to do yields empty incremental path", - Receiver: l("@a,1"), - Sender: l("@a,1"), - ExpectIncPath: l(), - }, - { - Msg: "drifting apart", - Receiver: l("@a,1", "@b,2"), - Sender: l("@c,3", "@d,4"), - ExpectNoCommonAncestor: true, - }, - { - Msg: "different snapshots on sender and receiver", - Receiver: l("@a,1", "@c,2"), - Sender: l("@a,1", "@b,3"), - ExpectDiverged: &replication.ConflictDiverged{ - CommonAncestor: l("@a,1")[0], - SenderOnly: l("@b,3"), - ReceiverOnly: l("@c,2"), - }, - }, - { - Msg: "snapshot on receiver not present on sender", - Receiver: l("@a,1", "@b,2"), - Sender: l("@a,1"), - ExpectDiverged: &replication.ConflictDiverged{ - CommonAncestor: l("@a,1")[0], - SenderOnly: l(), - ReceiverOnly: l("@b,2"), - }, - }, - { - Msg: "gaps before most recent common ancestor do not matter", - Receiver: l("@a,1", "@b,2", "@c,3"), - Sender: l("@a,1", "@c,3", "@d,4"), - ExpectIncPath: l("@c,3", "@d,4"), - }, - } - - for _, test := range tbl { - test.Test(t) - } - -} - -func TestIncrementalPlan_BookmarksSupport(t *testing.T) { - l := fsvlist - - tbl := []IncrementalPathTest{ - { - Msg: "bookmarks are used", - Receiver: l("@a,1"), - Sender: l("#a,1", "@b,2"), - ExpectIncPath: l("#a,1", "@b,2"), - }, - { - Msg: "boomarks are stripped from incPath (cannot send incrementally)", - Receiver: l("@a,1"), - Sender: l("#a,1", "#b,2", "@c,3"), - ExpectIncPath: l("#a,1", "@c,3"), - }, - { - Msg: "bookmarks are preferred over snapshots for start of incPath", - Receiver: l("@a,1"), - Sender: l("#a,1", "@a,1", "@b,2"), - ExpectIncPath: l("#a,1", "@b,2"), - }, - { - Msg: "bookmarks are preferred over snapshots for start of incPath (regardless of order)", - Receiver: l("@a,1"), - Sender: l("@a,1", "#a,1", "@b,2"), - ExpectIncPath: l("#a,1", "@b,2"), - }, - } - - for _, test := range tbl { - test.Test(t) - } - -} - -func TestSortVersionListByCreateTXGThenBookmarkLTSnapshot(t *testing.T) { - - type Test struct { - Msg string - Input, Output []*replication.FilesystemVersion - } - - l := fsvlist - - tbl := []Test{ - { - "snapshot sorting already sorted", - l("@a,1", "@b,2"), - l("@a,1", "@b,2"), - }, - { - "bookmark sorting already sorted", - l("#a,1", "#b,2"), - l("#a,1", "#b,2"), - }, - { - "snapshot sorting", - l("@b,2", "@a,1"), - l("@a,1", "@b,2"), - }, - { - "bookmark sorting", - l("#b,2", "#a,1"), - l("#a,1", "#b,2"), - }, - } - - for _, test := range tbl { - t.Logf("test: %s", test.Msg) - inputlen := len(test.Input) - sorted := replication.SortVersionListByCreateTXGThenBookmarkLTSnapshot(test.Input) - if len(sorted) != inputlen { - t.Errorf("lenghts of input and output do not match: %d vs %d", inputlen, len(sorted)) - continue - } - if !assert.Equal(t, test.Output, sorted) { - continue - } - last := sorted[0] - for _, s := range sorted[1:] { - if s.CreateTXG < last.CreateTXG { - t.Errorf("must be sorted ascending, got:\n\t%#v", sorted) - break - } - if s.CreateTXG == last.CreateTXG { - if last.Type == replication.FilesystemVersion_Bookmark && s.Type != replication.FilesystemVersion_Snapshot { - t.Errorf("snapshots must come after bookmarks") - } - } - last = s - } - } - -} diff --git a/cmd/replication.v2/plan.go b/cmd/replication.v2/plan.go deleted file mode 100644 index b0045ec..0000000 --- a/cmd/replication.v2/plan.go +++ /dev/null @@ -1,666 +0,0 @@ -package replication - -import ( - "context" - "errors" - "fmt" - "io" - "math/bits" - "net" - "sort" - "sync" - "time" -) - -//go:generate stringer -type=ReplicationState -type ReplicationState uint - -const ( - Planning ReplicationState = 1 << iota - PlanningError - Working - WorkingWait - Completed - ContextDone -) - -func (s ReplicationState) rsf() replicationStateFunc { - idx := bits.TrailingZeros(uint(s)) - if idx == bits.UintSize { - panic(s) // invalid value - } - m := []replicationStateFunc{ - rsfPlanning, - rsfPlanningError, - rsfWorking, - rsfWorkingWait, - nil, - nil, - } - return m[idx] -} - -type replicationQueueItem struct { - retriesSinceLastError int - // duplicates fsr.state to avoid accessing and locking fsr - state FSReplicationState - // duplicates fsr.current.nextStepDate to avoid accessing & locking fsr - nextStepDate time.Time - - fsr *FSReplication -} - -type Replication struct { - // lock protects all fields of this struct (but not the fields behind pointers!) - lock sync.Mutex - - state ReplicationState - - // Working, WorkingWait, Completed, ContextDone - queue *replicationQueue - completed []*FSReplication - active *FSReplication - - // PlanningError - planningError error - - // ContextDone - contextError error - - // PlanningError, WorkingWait - sleepUntil time.Time -} - -type replicationUpdater func(func(*Replication)) (newState ReplicationState) -type replicationStateFunc func(context.Context, EndpointPair, replicationUpdater) replicationStateFunc - -//go:generate stringer -type=FSReplicationState -type FSReplicationState uint - -const ( - FSReady FSReplicationState = 1 << iota - FSRetryWait - FSPermanentError - FSCompleted -) - -func (s FSReplicationState) fsrsf() fsrsf { - idx := bits.TrailingZeros(uint(s)) - if idx == bits.UintSize { - panic(s) - } - m := []fsrsf{ - fsrsfReady, - fsrsfRetryWait, - nil, - nil, - } - return m[idx] -} - -type FSReplication struct { - // lock protects all fields in this struct, but not the data behind pointers - lock sync.Mutex - state FSReplicationState - fs *Filesystem - err error - retryWaitUntil time.Time - completed, pending []*FSReplicationStep - current *FSReplicationStep -} - -func newReplicationQueueItemPermanentError(fs *Filesystem, err error) *replicationQueueItem { - return &replicationQueueItem{ - retriesSinceLastError: 0, - state: FSPermanentError, - fsr: &FSReplication{ - state: FSPermanentError, - fs: fs, - err: err, - }, - } -} - -func (b *replicationQueueItemBuilder) Complete() *replicationQueueItem { - if len(b.r.pending) > 0 { - b.r.state = FSReady - } else { - b.r.state = FSCompleted - } - r := b.r - return &replicationQueueItem{0, b.r.state, time.Time{}, r} -} - -//go:generate stringer -type=FSReplicationStepState -type FSReplicationStepState uint - -const ( - StepReady FSReplicationStepState = 1 << iota - StepRetry - StepPermanentError - StepCompleted -) - -type FSReplicationStep struct { - // only protects state, err - // from, to and fsrep are assumed to be immutable - lock sync.Mutex - - state FSReplicationStepState - from, to *FilesystemVersion - fsrep *FSReplication - - // both retry and permanent error - err error -} - -func (r *Replication) Drive(ctx context.Context, ep EndpointPair, retryNow chan struct{}) { - - var u replicationUpdater = func(f func(*Replication)) ReplicationState { - r.lock.Lock() - defer r.lock.Unlock() - if f != nil { - f(r) - } - return r.state - } - - var s replicationStateFunc = rsfPlanning - var pre, post ReplicationState - for s != nil { - preTime := time.Now() - pre = u(nil) - s = s(ctx, ep, u) - delta := time.Now().Sub(preTime) - post = u(nil) - getLogger(ctx). - WithField("transition", fmt.Sprintf("%s => %s", pre, post)). - WithField("duration", delta). - Debug("main state transition") - } - - getLogger(ctx). - WithField("final_state", post). - Debug("main final state") -} - -func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { - - log := getLogger(ctx) - - handlePlanningError := func(err error) replicationStateFunc { - return u(func(r *Replication) { - r.planningError = err - r.state = PlanningError - }).rsf() - } - - sfss, err := ep.Sender().ListFilesystems(ctx) - if err != nil { - log.WithError(err).Error("error listing sender filesystems") - return handlePlanningError(err) - } - - rfss, err := ep.Receiver().ListFilesystems(ctx) - if err != nil { - log.WithError(err).Error("error listing receiver filesystems") - return handlePlanningError(err) - } - - q := newReplicationQueue() - mainlog := log - for _, fs := range sfss { - - log := mainlog.WithField("filesystem", fs.Path) - - log.Info("assessing filesystem") - - sfsvs, err := ep.Sender().ListFilesystemVersions(ctx, fs.Path) - if err != nil { - log.WithError(err).Error("cannot get remote filesystem versions") - return handlePlanningError(err) - } - - if len(sfsvs) <= 1 { - err := errors.New("sender does not have any versions") - log.Error(err.Error()) - q.Add(newReplicationQueueItemPermanentError(fs, err)) - continue - } - - receiverFSExists := false - for _, rfs := range rfss { - if rfs.Path == fs.Path { - receiverFSExists = true - } - } - - var rfsvs []*FilesystemVersion - if receiverFSExists { - rfsvs, err = ep.Receiver().ListFilesystemVersions(ctx, fs.Path) - if err != nil { - if _, ok := err.(FilteredError); ok { - log.Info("receiver ignores filesystem") - continue - } - log.WithError(err).Error("receiver error") - return handlePlanningError(err) - } - } else { - rfsvs = []*FilesystemVersion{} - } - - path, conflict := IncrementalPath(rfsvs, sfsvs) - if conflict != nil { - var msg string - path, msg = resolveConflict(conflict) // no shadowing allowed! - if path != nil { - log.WithField("conflict", conflict).Info("conflict") - log.WithField("resolution", msg).Info("automatically resolved") - } else { - log.WithField("conflict", conflict).Error("conflict") - log.WithField("problem", msg).Error("cannot resolve conflict") - } - } - if path == nil { - q.Add(newReplicationQueueItemPermanentError(fs, conflict)) - continue - } - - builder := buildReplicationQueueItem(fs) - if len(path) == 1 { - builder.AddStep(nil, path[0]) - } else { - for i := 0; i < len(path)-1; i++ { - builder.AddStep(path[i], path[i+1]) - } - } - qitem := builder.Complete() - q.Add(qitem) - } - - return u(func(r *Replication) { - r.completed = nil - r.queue = q - r.planningError = nil - r.state = Working - }).rsf() -} - -func rsfPlanningError(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { - sleepTime := 10 * time.Second - u(func(r *Replication) { - r.sleepUntil = time.Now().Add(sleepTime) - }) - t := time.NewTimer(sleepTime) // FIXME make constant onfigurable - defer t.Stop() - select { - case <-ctx.Done(): - return u(func(r *Replication) { - r.state = ContextDone - r.contextError = ctx.Err() - }).rsf() - case <-t.C: - return u(func(r *Replication) { - r.state = Planning - }).rsf() - } -} - -func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { - - var active *replicationQueueItemHandle - - rsfNext := u(func(r *Replication) { - done, next := r.queue.GetNext() - r.completed = append(r.completed, done...) - if next == nil { - r.state = Completed - } - active = next - }).rsf() - - if active == nil { - return rsfNext - } - - state, nextStepDate := active.GetFSReplication().takeStep(ctx, ep) - - return u(func(r *Replication) { - active.Update(state, nextStepDate) - }).rsf() -} - -func rsfWorkingWait(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { - sleepTime := 10 * time.Second - u(func(r *Replication) { - r.sleepUntil = time.Now().Add(sleepTime) - }) - t := time.NewTimer(sleepTime) - defer t.Stop() - select { - case <-ctx.Done(): - return u(func(r *Replication) { - r.state = ContextDone - r.contextError = ctx.Err() - }).rsf() - case <-t.C: - return u(func(r *Replication) { - r.state = Working - }).rsf() - } -} - -type replicationQueueItemBuilder struct { - r *FSReplication - steps []*FSReplicationStep -} - -func buildReplicationQueueItem(fs *Filesystem) *replicationQueueItemBuilder { - return &replicationQueueItemBuilder{ - r: &FSReplication{ - fs: fs, - pending: make([]*FSReplicationStep, 0), - }, - } -} - -func (b *replicationQueueItemBuilder) AddStep(from, to *FilesystemVersion) *replicationQueueItemBuilder { - step := &FSReplicationStep{ - state: StepReady, - fsrep: b.r, - from: from, - to: to, - } - b.r.pending = append(b.r.pending, step) - return b -} - -type replicationQueue []*replicationQueueItem - -func newReplicationQueue() *replicationQueue { - q := make(replicationQueue, 0) - return &q -} - -func (q replicationQueue) Len() int { return len(q) } -func (q replicationQueue) Swap(i, j int) { q[i], q[j] = q[j], q[i] } -func (q replicationQueue) Less(i, j int) bool { - a, b := q[i], q[j] - statePrio := func(x *replicationQueueItem) int { - if x.state&(FSReady|FSRetryWait) == 0 { - panic(x) - } - if x.state == FSReady { - return 0 - } - return 1 - } - - aprio, bprio := statePrio(a), statePrio(b) - if aprio != bprio { - return aprio < bprio - } - // now we know they are the same state - if a.state == FSReady { - return a.nextStepDate.Before(b.nextStepDate) - } - if a.state == FSRetryWait { - return a.retriesSinceLastError < b.retriesSinceLastError - } - panic("should not be reached") -} - -func (q *replicationQueue) sort() (done []*FSReplication) { - // pre-scan for everything that is not ready - newq := make(replicationQueue, 0, len(*q)) - done = make([]*FSReplication, 0, len(*q)) - for _, qitem := range *q { - if qitem.state&(FSReady|FSRetryWait) == 0 { - done = append(done, qitem.fsr) - } else { - newq = append(newq, qitem) - } - } - sort.SortStable(newq) // stable to avoid flickering in reports - *q = newq - return done -} - -// next remains valid until the next call to GetNext() -func (q *replicationQueue) GetNext() (done []*FSReplication, next *replicationQueueItemHandle) { - done = q.sort() - if len(*q) == 0 { - return done, nil - } - next = &replicationQueueItemHandle{(*q)[0]} - return done, next -} - -func (q *replicationQueue) Add(qitem *replicationQueueItem) { - *q = append(*q, qitem) -} - -func (q *replicationQueue) Foreach(fu func(*replicationQueueItemHandle)) { - for _, qitem := range *q { - fu(&replicationQueueItemHandle{qitem}) - } -} - -type replicationQueueItemHandle struct { - i *replicationQueueItem -} - -func (h replicationQueueItemHandle) GetFSReplication() *FSReplication { - return h.i.fsr -} - -func (h replicationQueueItemHandle) Update(newState FSReplicationState, nextStepDate time.Time) { - h.i.state = newState - h.i.nextStepDate = nextStepDate - if h.i.state&FSReady != 0 { - h.i.retriesSinceLastError = 0 - } else if h.i.state&FSRetryWait != 0 { - h.i.retriesSinceLastError++ - } -} - -func (f *FSReplication) takeStep(ctx context.Context, ep EndpointPair) (post FSReplicationState, nextStepDate time.Time) { - - var u fsrUpdater = func(fu func(*FSReplication)) FSReplicationState { - f.lock.Lock() - defer f.lock.Unlock() - if fu != nil { - fu(f) - } - return f.state - } - var s fsrsf = u(nil).fsrsf() - - pre := u(nil) - preTime := time.Now() - s = s(ctx, ep, u) - delta := time.Now().Sub(preTime) - post = u(func(f *FSReplication) { - if f.state != FSReady { - return - } - ct, err := f.current.to.CreationAsTime() - if err != nil { - panic(err) // FIXME - } - nextStepDate = ct - }) - - getLogger(ctx). - WithField("fs", f.fs.Path). - WithField("transition", fmt.Sprintf("%s => %s", pre, post)). - WithField("duration", delta). - Debug("fsr step taken") - - return post, nextStepDate -} - -type fsrUpdater func(func(fsr *FSReplication)) FSReplicationState -type fsrsf func(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf - -func fsrsfReady(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf { - - var current *FSReplicationStep - s := u(func(f *FSReplication) { - if f.current == nil { - if len(f.pending) == 0 { - f.state = FSCompleted - return - } - f.current = f.pending[0] - f.pending = f.pending[1:] - } - current = f.current - }) - if s != FSReady { - return s.fsrsf() - } - - stepState := current.do(ctx, ep) - - return u(func(f *FSReplication) { - switch stepState { - case StepCompleted: - f.completed = append(f.completed, f.current) - f.current = nil - if len(f.pending) > 0 { - f.state = FSReady - } else { - f.state = FSCompleted - } - case StepRetry: - f.retryWaitUntil = time.Now().Add(10 * time.Second) // FIXME make configurable - f.state = FSRetryWait - case StepPermanentError: - f.state = FSPermanentError - f.err = errors.New("a replication step failed with a permanent error") - default: - panic(f) - } - }).fsrsf() -} - -func fsrsfRetryWait(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf { - var sleepUntil time.Time - u(func(f *FSReplication) { - sleepUntil = f.retryWaitUntil - }) - t := time.NewTimer(sleepUntil.Sub(time.Now())) - defer t.Stop() - select { - case <-ctx.Done(): - return u(func(f *FSReplication) { - f.state = FSPermanentError - f.err = ctx.Err() - }).fsrsf() - case <-t.C: - } - return u(func(f *FSReplication) { - f.state = FSReady - }).fsrsf() -} - -func (s *FSReplicationStep) do(ctx context.Context, ep EndpointPair) FSReplicationStepState { - - fs := s.fsrep.fs - - log := getLogger(ctx). - WithField("filesystem", fs.Path). - WithField("step", s.String()) - - updateStateError := func(err error) FSReplicationStepState { - s.lock.Lock() - defer s.lock.Unlock() - - s.err = err - switch err { - case io.EOF: - fallthrough - case io.ErrUnexpectedEOF: - fallthrough - case io.ErrClosedPipe: - s.state = StepRetry - return s.state - } - if _, ok := err.(net.Error); ok { - s.state = StepRetry - return s.state - } - s.state = StepPermanentError - return s.state - } - - updateStateCompleted := func() FSReplicationStepState { - s.lock.Lock() - defer s.lock.Unlock() - s.err = nil - s.state = StepCompleted - return s.state - } - - // FIXME refresh fs resume token - fs.ResumeToken = "" - - var sr *SendReq - if fs.ResumeToken != "" { - sr = &SendReq{ - Filesystem: fs.Path, - ResumeToken: fs.ResumeToken, - } - } else if s.from == nil { - sr = &SendReq{ - Filesystem: fs.Path, - From: s.to.RelName(), // FIXME fix protocol to use To, like zfs does internally - } - } else { - sr = &SendReq{ - Filesystem: fs.Path, - From: s.from.RelName(), - To: s.to.RelName(), - } - } - - log.WithField("request", sr).Debug("initiate send request") - sres, sstream, err := ep.Sender().Send(ctx, sr) - if err != nil { - log.WithError(err).Error("send request failed") - return updateStateError(err) - } - if sstream == nil { - err := errors.New("send request did not return a stream, broken endpoint implementation") - return updateStateError(err) - } - - rr := &ReceiveReq{ - Filesystem: fs.Path, - ClearResumeToken: !sres.UsedResumeToken, - } - log.WithField("request", rr).Debug("initiate receive request") - err = ep.Receiver().Receive(ctx, rr, sstream) - if err != nil { - log.WithError(err).Error("receive request failed (might also be error on sender)") - sstream.Close() - // This failure could be due to - // - an unexpected exit of ZFS on the sending side - // - an unexpected exit of ZFS on the receiving side - // - a connectivity issue - return updateStateError(err) - } - log.Info("receive finished") - return updateStateCompleted() - -} - -func (s *FSReplicationStep) String() string { - if s.from == nil { // FIXME: ZFS semantics are that to is nil on non-incremental send - return fmt.Sprintf("%s%s (full)", s.fsrep.fs.Path, s.to.RelName()) - } else { - return fmt.Sprintf("%s(%s => %s)", s.fsrep.fs.Path, s.from.RelName(), s.to.RelName()) - } -} diff --git a/cmd/replication.v2/replication.go b/cmd/replication.v2/replication.go deleted file mode 100644 index 83fd38b..0000000 --- a/cmd/replication.v2/replication.go +++ /dev/null @@ -1,181 +0,0 @@ -package replication - -import ( - "os" - "syscall" - "encoding/json" - "context" - "fmt" - "github.com/zrepl/zrepl/logger" - "io" - "os/signal" -) - -type ReplicationEndpoint interface { - // Does not include placeholder filesystems - ListFilesystems(ctx context.Context) ([]*Filesystem, error) - ListFilesystemVersions(ctx context.Context, fs string) ([]*FilesystemVersion, error) // fix depS - Send(ctx context.Context, r *SendReq) (*SendRes, io.ReadCloser, error) - Receive(ctx context.Context, r *ReceiveReq, sendStream io.ReadCloser) error -} - -type FilteredError struct{ fs string } - -func NewFilteredError(fs string) FilteredError { - return FilteredError{fs} -} - -func (f FilteredError) Error() string { return "endpoint does not allow access to filesystem " + f.fs } - -type ReplicationMode int - -const ( - ReplicationModePull ReplicationMode = iota - ReplicationModePush -) - -type EndpointPair struct { - a, b ReplicationEndpoint - m ReplicationMode -} - -func NewEndpointPairPull(sender, receiver ReplicationEndpoint) EndpointPair { - return EndpointPair{sender, receiver, ReplicationModePull} -} - -func NewEndpointPairPush(sender, receiver ReplicationEndpoint) EndpointPair { - return EndpointPair{receiver, sender, ReplicationModePush} -} - -func (p EndpointPair) Sender() ReplicationEndpoint { - switch p.m { - case ReplicationModePull: - return p.a - case ReplicationModePush: - return p.b - } - panic("should not be reached") - return nil -} - -func (p EndpointPair) Receiver() ReplicationEndpoint { - switch p.m { - case ReplicationModePull: - return p.b - case ReplicationModePush: - return p.a - } - panic("should not be reached") - return nil -} - -func (p EndpointPair) Mode() ReplicationMode { - return p.m -} - -type contextKey int - -const ( - contextKeyLog contextKey = iota -) - -//type Logger interface { -// Infof(fmt string, args ...interface{}) -// Errorf(fmt string, args ...interface{}) -//} - -//var _ Logger = nullLogger{} - -//type nullLogger struct{} -// -//func (nullLogger) Infof(fmt string, args ...interface{}) {} -//func (nullLogger) Errorf(fmt string, args ...interface{}) {} - -type Logger = logger.Logger - -func ContextWithLogger(ctx context.Context, l Logger) context.Context { - return context.WithValue(ctx, contextKeyLog, l) -} - -func getLogger(ctx context.Context) Logger { - l, ok := ctx.Value(contextKeyLog).(Logger) - if !ok { - l = logger.NewNullLogger() - } - return l -} - -func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) { - if noCommonAncestor, ok := conflict.(*ConflictNoCommonAncestor); ok { - if len(noCommonAncestor.SortedReceiverVersions) == 0 { - // FIXME hard-coded replication policy: most recent - // snapshot as source - var mostRecentSnap *FilesystemVersion - for n := len(noCommonAncestor.SortedSenderVersions) - 1; n >= 0; n-- { - if noCommonAncestor.SortedSenderVersions[n].Type == FilesystemVersion_Snapshot { - mostRecentSnap = noCommonAncestor.SortedSenderVersions[n] - break - } - } - if mostRecentSnap == nil { - return nil, "no snapshots available on sender side" - } - return []*FilesystemVersion{mostRecentSnap}, fmt.Sprintf("start replication at most recent snapshot %s", mostRecentSnap.RelName()) - } - } - return nil, "no automated way to handle conflict type" -} - -func NewReplication() *Replication { - r := Replication{ - state: Planning, - } - return &r -} -// Replicate replicates filesystems from ep.Sender() to ep.Receiver(). -// -// All filesystems presented by the sending side are replicated, -// unless the receiver rejects a Receive request with a *FilteredError. -// -// If an error occurs when replicating a filesystem, that error is logged to the logger in ctx. -// Replicate continues with the replication of the remaining file systems. -// Depending on the type of error, failed replications are retried in an unspecified order (currently FIFO). -func Replicate(ctx context.Context, ep EndpointPair, retryNow chan struct{}) { - r := Replication{ - state: Planning, - } - - c := make(chan os.Signal) - defer close(c) - signal.Notify(c, syscall.SIGHUP) - go func() { - f, err := os.OpenFile("/tmp/report", os.O_CREATE|os.O_APPEND|os.O_WRONLY, 0600) - if err != nil { - getLogger(ctx).WithError(err).Error("cannot open report file") - panic(err) - } - defer f.Close() - for { - select { - case <-ctx.Done(): - return - case sig := <-c: - if sig == nil { - return - } - report := r.Report() - enc := json.NewEncoder(f) - enc.SetIndent(" ", " ") - if err := enc.Encode(report); err != nil { - getLogger(ctx).WithError(err).Error("cannot encode report") - panic(err) - } - f.Write([]byte("\n")) - f.Sync() - } - } - }() - - r.Drive(ctx, ep, retryNow) -} - diff --git a/cmd/replication.v2/replication_test.go b/cmd/replication.v2/replication_test.go deleted file mode 100644 index 00b3868..0000000 --- a/cmd/replication.v2/replication_test.go +++ /dev/null @@ -1,181 +0,0 @@ -package replication_test - -import ( - "context" - "github.com/stretchr/testify/assert" - "github.com/zrepl/zrepl/cmd/replication" - "io" - "testing" -) - -type IncrementalPathSequenceStep struct { - SendRequest *replication.SendReq - SendResponse *replication.SendRes - SendReader io.ReadCloser - SendError error - ReceiveRequest *replication.ReceiveReq - ReceiveError error -} - -type MockIncrementalPathRecorder struct { - T *testing.T - Sequence []IncrementalPathSequenceStep - Pos int -} - -func (m *MockIncrementalPathRecorder) Receive(ctx context.Context, r *replication.ReceiveReq, rs io.ReadCloser) (error) { - if m.Pos >= len(m.Sequence) { - m.T.Fatal("unexpected Receive") - } - i := m.Sequence[m.Pos] - m.Pos++ - if !assert.Equal(m.T, i.ReceiveRequest, r) { - m.T.FailNow() - } - return i.ReceiveError -} - -func (m *MockIncrementalPathRecorder) Send(ctx context.Context, r *replication.SendReq) (*replication.SendRes, io.ReadCloser, error) { - if m.Pos >= len(m.Sequence) { - m.T.Fatal("unexpected Send") - } - i := m.Sequence[m.Pos] - m.Pos++ - if !assert.Equal(m.T, i.SendRequest, r) { - m.T.FailNow() - } - return i.SendResponse, i.SendReader, i.SendError -} - -func (m *MockIncrementalPathRecorder) Finished() bool { - return m.Pos == len(m.Sequence) -} - -//type IncrementalPathReplicatorTest struct { -// Msg string -// Filesystem *replication.Filesystem -// Path []*replication.FilesystemVersion -// Steps []IncrementalPathSequenceStep -//} -// -//func (test *IncrementalPathReplicatorTest) Test(t *testing.T) { -// -// t.Log(test.Msg) -// -// rec := &MockIncrementalPathRecorder{ -// T: t, -// Sequence: test.Steps, -// } -// -// ctx := replication.ContextWithLogger(context.Background(), testLog{t}) -// -// ipr := replication.NewIncrementalPathReplicator() -// ipr.Replicate( -// ctx, -// rec, -// rec, -// DiscardCopier{}, -// test.Filesystem, -// test.Path, -// ) -// -// assert.True(t, rec.Finished()) -// -//} - -//type testLog struct { -// t *testing.T -//} -// -//var _ replication.Logger = testLog{} -// -//func (t testLog) Infof(fmt string, args ...interface{}) { -// t.t.Logf(fmt, args) -//} -//func (t testLog) Errorf(fmt string, args ...interface{}) { -// t.t.Logf(fmt, args) -//} - - -//func TestIncrementalPathReplicator_Replicate(t *testing.T) { -// -// tbl := []IncrementalPathReplicatorTest{ -// { -// Msg: "generic happy place with resume token", -// Filesystem: &replication.Filesystem{ -// Path: "foo/bar", -// ResumeToken: "blafoo", -// }, -// Path: fsvlist("@a,1", "@b,2", "@c,3"), -// Steps: []IncrementalPathSequenceStep{ -// { -// SendRequest: &replication.SendReq{ -// Filesystem: "foo/bar", -// From: "@a,1", -// To: "@b,2", -// ResumeToken: "blafoo", -// }, -// SendResponse: &replication.SendRes{ -// UsedResumeToken: true, -// }, -// }, -// { -// ReceiveRequest: &replication.ReceiveReq{ -// Filesystem: "foo/bar", -// ClearResumeToken: false, -// }, -// }, -// { -// SendRequest: &replication.SendReq{ -// Filesystem: "foo/bar", -// From: "@b,2", -// To: "@c,3", -// }, -// }, -// { -// ReceiveRequest: &replication.ReceiveReq{ -// Filesystem: "foo/bar", -// }, -// }, -// }, -// }, -// { -// Msg: "no action on empty sequence", -// Filesystem: &replication.Filesystem{ -// Path: "foo/bar", -// }, -// Path: fsvlist(), -// Steps: []IncrementalPathSequenceStep{}, -// }, -// { -// Msg: "full send on single entry path", -// Filesystem: &replication.Filesystem{ -// Path: "foo/bar", -// }, -// Path: fsvlist("@justone,1"), -// Steps: []IncrementalPathSequenceStep{ -// { -// SendRequest: &replication.SendReq{ -// Filesystem: "foo/bar", -// From: "@justone,1", -// To: "", // empty means full send -// }, -// SendResponse: &replication.SendRes{ -// UsedResumeToken: false, -// }, -// }, -// { -// ReceiveRequest: &replication.ReceiveReq{ -// Filesystem: "foo/bar", -// ClearResumeToken: false, -// }, -// }, -// }, -// }, -// } -// -// for _, test := range tbl { -// test.Test(t) -// } -// -//} diff --git a/cmd/replication.v2/report.go b/cmd/replication.v2/report.go deleted file mode 100644 index 6df5b68..0000000 --- a/cmd/replication.v2/report.go +++ /dev/null @@ -1,94 +0,0 @@ -package replication - -type Report struct { - Status string - Problem string - Completed []*FilesystemReplicationReport - Pending []*FilesystemReplicationReport - Active *FilesystemReplicationReport -} - -type StepReport struct { - From, To string - Status string - Problem string -} - -type FilesystemReplicationReport struct { - Filesystem string - Status string - Problem string - Steps []*StepReport -} - -func stepReportFromStep(step *FSReplicationStep) *StepReport { - var from string // FIXME follow same convention as ZFS: to should be nil on full send - if step.from != nil { - from = step.from.RelName() - } - rep := StepReport{ - From: from, - To: step.to.RelName(), - Status: step.state.String(), - } - return &rep -} - -// access to fsr's members must be exclusive -func filesystemReplicationReport(fsr *FSReplication) *FilesystemReplicationReport { - fsr.lock.Lock() - defer fsr.lock.Unlock() - - rep := FilesystemReplicationReport{ - Filesystem: fsr.fs.Path, - Status: fsr.state.String(), - } - - if fsr.state&FSPermanentError != 0 { - rep.Problem = fsr.err.Error() - return &rep - } - - rep.Steps = make([]*StepReport, 0, len(fsr.completed)+len(fsr.pending) + 1) - for _, step := range fsr.completed { - rep.Steps = append(rep.Steps, stepReportFromStep(step)) - } - if fsr.current != nil { - rep.Steps = append(rep.Steps, stepReportFromStep(fsr.current)) - } - for _, step := range fsr.pending { - rep.Steps = append(rep.Steps, stepReportFromStep(step)) - } - return &rep -} - -func (r *Replication) Report() *Report { - r.lock.Lock() - defer r.lock.Unlock() - - rep := Report{ - Status: r.state.String(), - } - - if r.state&(Planning|PlanningError|ContextDone) != 0 { - switch r.state { - case PlanningError: - rep.Problem = r.planningError.Error() - case ContextDone: - rep.Problem = r.contextError.Error() - } - return &rep - } - - rep.Pending = make([]*FilesystemReplicationReport, 0, r.queue.Len()) - rep.Completed = make([]*FilesystemReplicationReport, 0, len(r.completed)) // room for active (potentially) - - r.queue.Foreach(func (h *replicationQueueItemHandle){ - rep.Pending = append(rep.Pending, filesystemReplicationReport(h.GetFSReplication())) - }) - for _, fsr := range r.completed { - rep.Completed = append(rep.Completed, filesystemReplicationReport(fsr)) - } - - return &rep -} diff --git a/cmd/replication/common/common.go b/cmd/replication/common/common.go new file mode 100644 index 0000000..3166bfc --- /dev/null +++ b/cmd/replication/common/common.go @@ -0,0 +1,91 @@ +package common + +import ( + "context" + "io" + + "github.com/zrepl/zrepl/cmd/replication/pdu" + "github.com/zrepl/zrepl/logger" +) + +type contextKey int + +const ( + contextKeyLog contextKey = iota +) + +type Logger = logger.Logger + +func ContextWithLogger(ctx context.Context, l Logger) context.Context { + return context.WithValue(ctx, contextKeyLog, l) +} + +func GetLogger(ctx context.Context) Logger { + l, ok := ctx.Value(contextKeyLog).(Logger) + if !ok { + l = logger.NewNullLogger() + } + return l +} + +type ReplicationEndpoint interface { + // Does not include placeholder filesystems + ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) + ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) // fix depS + Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) + Receive(ctx context.Context, r *pdu.ReceiveReq, sendStream io.ReadCloser) error +} + +type FilteredError struct{ fs string } + +func NewFilteredError(fs string) *FilteredError { + return &FilteredError{fs} +} + +func (f FilteredError) Error() string { return "endpoint does not allow access to filesystem " + f.fs } + +type ReplicationMode int + +const ( + ReplicationModePull ReplicationMode = iota + ReplicationModePush +) + +type EndpointPair struct { + a, b ReplicationEndpoint + m ReplicationMode +} + +func NewEndpointPairPull(sender, receiver ReplicationEndpoint) EndpointPair { + return EndpointPair{sender, receiver, ReplicationModePull} +} + +func NewEndpointPairPush(sender, receiver ReplicationEndpoint) EndpointPair { + return EndpointPair{receiver, sender, ReplicationModePush} +} + +func (p EndpointPair) Sender() ReplicationEndpoint { + switch p.m { + case ReplicationModePull: + return p.a + case ReplicationModePush: + return p.b + } + panic("should not be reached") + return nil +} + +func (p EndpointPair) Receiver() ReplicationEndpoint { + switch p.m { + case ReplicationModePull: + return p.b + case ReplicationModePush: + return p.a + } + panic("should not be reached") + return nil +} + +func (p EndpointPair) Mode() ReplicationMode { + return p.m +} diff --git a/cmd/replication/internal/fsfsm/fsfsm.go b/cmd/replication/internal/fsfsm/fsfsm.go new file mode 100644 index 0000000..d5c3629 --- /dev/null +++ b/cmd/replication/internal/fsfsm/fsfsm.go @@ -0,0 +1,361 @@ +package fsfsm + +import ( + "context" + "errors" + "fmt" + "io" + "math/bits" + "net" + "sync" + "time" + + "github.com/zrepl/zrepl/cmd/replication/pdu" + . "github.com/zrepl/zrepl/cmd/replication/common" +) + +type StepReport struct { + From, To string + Status string + Problem string +} + +type FilesystemReplicationReport struct { + Filesystem string + Status string + Problem string + Completed,Pending []*StepReport +} + + +//go:generate stringer -type=FSReplicationState +type FSReplicationState uint + +const ( + FSReady FSReplicationState = 1 << iota + FSRetryWait + FSPermanentError + FSCompleted +) + +func (s FSReplicationState) fsrsf() fsrsf { + idx := bits.TrailingZeros(uint(s)) + if idx == bits.UintSize { + panic(s) + } + m := []fsrsf{ + fsrsfReady, + fsrsfRetryWait, + nil, + nil, + } + return m[idx] +} + +type FSReplication struct { + // lock protects all fields in this struct, but not the data behind pointers + lock sync.Mutex + state FSReplicationState + fs string + err error + retryWaitUntil time.Time + completed, pending []*FSReplicationStep +} + +func (f *FSReplication) State() FSReplicationState { + f.lock.Lock() + defer f.lock.Unlock() + return f.state +} + +type FSReplicationBuilder struct { + r *FSReplication +} + +func BuildFSReplication(fs string) *FSReplicationBuilder { + return &FSReplicationBuilder{&FSReplication{fs: fs}} +} + +func (b *FSReplicationBuilder) AddStep(from, to FilesystemVersion) *FSReplicationBuilder { + step := &FSReplicationStep{ + state: StepReady, + fsrep: b.r, + from: from, + to: to, + } + b.r.pending = append(b.r.pending, step) + return b +} + +func (b *FSReplicationBuilder) Done() (r *FSReplication) { + if len(b.r.pending) > 0 { + b.r.state = FSReady + } else { + b.r.state = FSCompleted + } + r = b.r + b.r = nil + return r +} + +func NewFSReplicationWithPermanentError(fs string, err error) *FSReplication { + return &FSReplication{ + state: FSPermanentError, + fs: fs, + err: err, + } +} + + +//go:generate stringer -type=FSReplicationStepState +type FSReplicationStepState uint + +const ( + StepReady FSReplicationStepState = 1 << iota + StepRetry + StepPermanentError + StepCompleted +) + +type FilesystemVersion interface { + SnapshotTime() time.Time + RelName() string +} + +type FSReplicationStep struct { + // only protects state, err + // from, to and fsrep are assumed to be immutable + lock sync.Mutex + + state FSReplicationStepState + from, to FilesystemVersion + fsrep *FSReplication + + // both retry and permanent error + err error +} + +func (f *FSReplication) TakeStep(ctx context.Context, ep EndpointPair) (post FSReplicationState, nextStepDate time.Time) { + + var u fsrUpdater = func(fu func(*FSReplication)) FSReplicationState { + f.lock.Lock() + defer f.lock.Unlock() + if fu != nil { + fu(f) + } + return f.state + } + var s fsrsf = u(nil).fsrsf() + + pre := u(nil) + preTime := time.Now() + s = s(ctx, ep, u) + delta := time.Now().Sub(preTime) + post = u(func(f *FSReplication) { + if len(f.pending) == 0 { + return + } + nextStepDate = f.pending[0].to.SnapshotTime() + }) + + GetLogger(ctx). + WithField("fs", f.fs). + WithField("transition", fmt.Sprintf("%s => %s", pre, post)). + WithField("duration", delta). + Debug("fsr step taken") + + return post, nextStepDate +} + +type fsrUpdater func(func(fsr *FSReplication)) FSReplicationState + +type fsrsf func(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf + +func fsrsfReady(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf { + + var current *FSReplicationStep + s := u(func(f *FSReplication) { + if len(f.pending) == 0 { + f.state = FSCompleted + return + } + current = f.pending[0] + }) + if s != FSReady { + return s.fsrsf() + } + + stepState := current.do(ctx, ep) + + return u(func(f *FSReplication) { + switch stepState { + case StepCompleted: + f.completed = append(f.completed, current) + f.pending = f.pending[1:] + if len(f.pending) > 0 { + f.state = FSReady + } else { + f.state = FSCompleted + } + case StepRetry: + f.retryWaitUntil = time.Now().Add(10 * time.Second) // FIXME make configurable + f.state = FSRetryWait + case StepPermanentError: + f.state = FSPermanentError + f.err = errors.New("a replication step failed with a permanent error") + default: + panic(f) + } + }).fsrsf() +} + +func fsrsfRetryWait(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf { + var sleepUntil time.Time + u(func(f *FSReplication) { + sleepUntil = f.retryWaitUntil + }) + t := time.NewTimer(sleepUntil.Sub(time.Now())) + defer t.Stop() + select { + case <-ctx.Done(): + return u(func(f *FSReplication) { + f.state = FSPermanentError + f.err = ctx.Err() + }).fsrsf() + case <-t.C: + } + return u(func(f *FSReplication) { + f.state = FSReady + }).fsrsf() +} + +// access to fsr's members must be exclusive +func (fsr *FSReplication) Report() *FilesystemReplicationReport { + fsr.lock.Lock() + defer fsr.lock.Unlock() + + rep := FilesystemReplicationReport{ + Filesystem: fsr.fs, + Status: fsr.state.String(), + } + + if fsr.state&FSPermanentError != 0 { + rep.Problem = fsr.err.Error() + return &rep + } + + rep.Completed = make([]*StepReport, len(fsr.completed)) + for i := range fsr.completed { + rep.Completed[i] = fsr.completed[i].Report() + } + rep.Pending = make([]*StepReport, len(fsr.pending)) + for i := range fsr.pending { + rep.Pending[i] = fsr.pending[i].Report() + } + return &rep +} + +func (s *FSReplicationStep) do(ctx context.Context, ep EndpointPair) FSReplicationStepState { + + fs := s.fsrep.fs + + log := GetLogger(ctx). + WithField("filesystem", fs). + WithField("step", s.String()) + + updateStateError := func(err error) FSReplicationStepState { + s.lock.Lock() + defer s.lock.Unlock() + + s.err = err + switch err { + case io.EOF: + fallthrough + case io.ErrUnexpectedEOF: + fallthrough + case io.ErrClosedPipe: + s.state = StepRetry + return s.state + } + if _, ok := err.(net.Error); ok { + s.state = StepRetry + return s.state + } + s.state = StepPermanentError + return s.state + } + + updateStateCompleted := func() FSReplicationStepState { + s.lock.Lock() + defer s.lock.Unlock() + s.err = nil + s.state = StepCompleted + return s.state + } + + var sr *pdu.SendReq + if s.from == nil { + sr = &pdu.SendReq{ + Filesystem: fs, + From: s.to.RelName(), // FIXME fix protocol to use To, like zfs does internally + } + } else { + sr = &pdu.SendReq{ + Filesystem: fs, + From: s.from.RelName(), + To: s.to.RelName(), + } + } + + log.WithField("request", sr).Debug("initiate send request") + sres, sstream, err := ep.Sender().Send(ctx, sr) + if err != nil { + log.WithError(err).Error("send request failed") + return updateStateError(err) + } + if sstream == nil { + err := errors.New("send request did not return a stream, broken endpoint implementation") + return updateStateError(err) + } + + rr := &pdu.ReceiveReq{ + Filesystem: fs, + ClearResumeToken: !sres.UsedResumeToken, + } + log.WithField("request", rr).Debug("initiate receive request") + err = ep.Receiver().Receive(ctx, rr, sstream) + if err != nil { + log.WithError(err).Error("receive request failed (might also be error on sender)") + sstream.Close() + // This failure could be due to + // - an unexpected exit of ZFS on the sending side + // - an unexpected exit of ZFS on the receiving side + // - a connectivity issue + return updateStateError(err) + } + log.Info("receive finished") + return updateStateCompleted() + +} + +func (s *FSReplicationStep) String() string { + if s.from == nil { // FIXME: ZFS semantics are that to is nil on non-incremental send + return fmt.Sprintf("%s%s (full)", s.fsrep.fs, s.to.RelName()) + } else { + return fmt.Sprintf("%s(%s => %s)", s.fsrep.fs, s.from, s.to.RelName()) + } +} + +func (step *FSReplicationStep) Report() *StepReport { + var from string // FIXME follow same convention as ZFS: to should be nil on full send + if step.from != nil { + from = step.from.RelName() + } + rep := StepReport{ + From: from, + To: step.to.RelName(), + Status: step.state.String(), + } + return &rep +} + diff --git a/cmd/replication.v2/fsreplicationstate_string.go b/cmd/replication/internal/fsfsm/fsreplicationstate_string.go similarity index 97% rename from cmd/replication.v2/fsreplicationstate_string.go rename to cmd/replication/internal/fsfsm/fsreplicationstate_string.go index 6002737..225fee9 100644 --- a/cmd/replication.v2/fsreplicationstate_string.go +++ b/cmd/replication/internal/fsfsm/fsreplicationstate_string.go @@ -1,6 +1,6 @@ // Code generated by "stringer -type=FSReplicationState"; DO NOT EDIT. -package replication +package fsfsm import "strconv" diff --git a/cmd/replication.v2/fsreplicationstepstate_string.go b/cmd/replication/internal/fsfsm/fsreplicationstepstate_string.go similarity index 97% rename from cmd/replication.v2/fsreplicationstepstate_string.go rename to cmd/replication/internal/fsfsm/fsreplicationstepstate_string.go index 3d075c1..3d64be0 100644 --- a/cmd/replication.v2/fsreplicationstepstate_string.go +++ b/cmd/replication/internal/fsfsm/fsreplicationstepstate_string.go @@ -1,6 +1,6 @@ // Code generated by "stringer -type=FSReplicationStepState"; DO NOT EDIT. -package replication +package fsfsm import "strconv" diff --git a/cmd/replication.v2/diff.go b/cmd/replication/internal/mainfsm/diff.go similarity index 98% rename from cmd/replication.v2/diff.go rename to cmd/replication/internal/mainfsm/diff.go index bb6cb17..f1f4af6 100644 --- a/cmd/replication.v2/diff.go +++ b/cmd/replication/internal/mainfsm/diff.go @@ -1,7 +1,9 @@ -package replication +package mainfsm import ( "sort" + + . "github.com/zrepl/zrepl/cmd/replication/pdu" ) type ConflictNoCommonAncestor struct { diff --git a/cmd/replication/internal/mainfsm/mainfsm.go b/cmd/replication/internal/mainfsm/mainfsm.go new file mode 100644 index 0000000..883d6d5 --- /dev/null +++ b/cmd/replication/internal/mainfsm/mainfsm.go @@ -0,0 +1,342 @@ +package mainfsm + +import ( + "context" + "errors" + "fmt" + "math/bits" + "sync" + "time" + + . "github.com/zrepl/zrepl/cmd/replication/common" + "github.com/zrepl/zrepl/cmd/replication/pdu" + "github.com/zrepl/zrepl/cmd/replication/internal/fsfsm" + . "github.com/zrepl/zrepl/cmd/replication/internal/mainfsm/queue" +) + +//go:generate stringer -type=ReplicationState +type ReplicationState uint + +const ( + Planning ReplicationState = 1 << iota + PlanningError + Working + WorkingWait + Completed + ContextDone +) + +func (s ReplicationState) rsf() replicationStateFunc { + idx := bits.TrailingZeros(uint(s)) + if idx == bits.UintSize { + panic(s) // invalid value + } + m := []replicationStateFunc{ + rsfPlanning, + rsfPlanningError, + rsfWorking, + rsfWorkingWait, + nil, + nil, + } + return m[idx] +} + +type Replication struct { + // lock protects all fields of this struct (but not the fields behind pointers!) + lock sync.Mutex + + state ReplicationState + + // Working, WorkingWait, Completed, ContextDone + queue *ReplicationQueue + completed []*fsfsm.FSReplication + active *ReplicationQueueItemHandle + + // PlanningError + planningError error + + // ContextDone + contextError error + + // PlanningError, WorkingWait + sleepUntil time.Time +} + +type Report struct { + Status string + Problem string + Completed []*fsfsm.FilesystemReplicationReport + Pending []*fsfsm.FilesystemReplicationReport + Active *fsfsm.FilesystemReplicationReport +} + + +func NewReplication() *Replication { + r := Replication{ + state: Planning, + } + return &r +} + +type replicationUpdater func(func(*Replication)) (newState ReplicationState) +type replicationStateFunc func(context.Context, EndpointPair, replicationUpdater) replicationStateFunc + +func (r *Replication) Drive(ctx context.Context, ep EndpointPair) { + + var u replicationUpdater = func(f func(*Replication)) ReplicationState { + r.lock.Lock() + defer r.lock.Unlock() + if f != nil { + f(r) + } + return r.state + } + + var s replicationStateFunc = rsfPlanning + var pre, post ReplicationState + for s != nil { + preTime := time.Now() + pre = u(nil) + s = s(ctx, ep, u) + delta := time.Now().Sub(preTime) + post = u(nil) + GetLogger(ctx). + WithField("transition", fmt.Sprintf("%s => %s", pre, post)). + WithField("duration", delta). + Debug("main state transition") + } + + GetLogger(ctx). + WithField("final_state", post). + Debug("main final state") +} + +func resolveConflict(conflict error) (path []*pdu.FilesystemVersion, msg string) { + if noCommonAncestor, ok := conflict.(*ConflictNoCommonAncestor); ok { + if len(noCommonAncestor.SortedReceiverVersions) == 0 { + // FIXME hard-coded replication policy: most recent + // snapshot as source + var mostRecentSnap *pdu.FilesystemVersion + for n := len(noCommonAncestor.SortedSenderVersions) - 1; n >= 0; n-- { + if noCommonAncestor.SortedSenderVersions[n].Type == pdu.FilesystemVersion_Snapshot { + mostRecentSnap = noCommonAncestor.SortedSenderVersions[n] + break + } + } + if mostRecentSnap == nil { + return nil, "no snapshots available on sender side" + } + return []*pdu.FilesystemVersion{mostRecentSnap}, fmt.Sprintf("start replication at most recent snapshot %s", mostRecentSnap.RelName()) + } + } + return nil, "no automated way to handle conflict type" +} + +func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { + + log := GetLogger(ctx) + + handlePlanningError := func(err error) replicationStateFunc { + return u(func(r *Replication) { + r.planningError = err + r.state = PlanningError + }).rsf() + } + + sfss, err := ep.Sender().ListFilesystems(ctx) + if err != nil { + log.WithError(err).Error("error listing sender filesystems") + return handlePlanningError(err) + } + + rfss, err := ep.Receiver().ListFilesystems(ctx) + if err != nil { + log.WithError(err).Error("error listing receiver filesystems") + return handlePlanningError(err) + } + + q := NewReplicationQueue() + mainlog := log + for _, fs := range sfss { + + log := mainlog.WithField("filesystem", fs.Path) + + log.Info("assessing filesystem") + + sfsvs, err := ep.Sender().ListFilesystemVersions(ctx, fs.Path) + if err != nil { + log.WithError(err).Error("cannot get remote filesystem versions") + return handlePlanningError(err) + } + + if len(sfsvs) <= 1 { + err := errors.New("sender does not have any versions") + log.Error(err.Error()) + q.Add(fsfsm.NewFSReplicationWithPermanentError(fs.Path, err)) + continue + } + + receiverFSExists := false + for _, rfs := range rfss { + if rfs.Path == fs.Path { + receiverFSExists = true + } + } + + var rfsvs []*pdu.FilesystemVersion + if receiverFSExists { + rfsvs, err = ep.Receiver().ListFilesystemVersions(ctx, fs.Path) + if err != nil { + if _, ok := err.(*FilteredError); ok { + log.Info("receiver ignores filesystem") + continue + } + log.WithError(err).Error("receiver error") + return handlePlanningError(err) + } + } else { + rfsvs = []*pdu.FilesystemVersion{} + } + + path, conflict := IncrementalPath(rfsvs, sfsvs) + if conflict != nil { + var msg string + path, msg = resolveConflict(conflict) // no shadowing allowed! + if path != nil { + log.WithField("conflict", conflict).Info("conflict") + log.WithField("resolution", msg).Info("automatically resolved") + } else { + log.WithField("conflict", conflict).Error("conflict") + log.WithField("problem", msg).Error("cannot resolve conflict") + } + } + if path == nil { + q.Add(fsfsm.NewFSReplicationWithPermanentError(fs.Path, conflict)) + continue + } + + fsrfsm := fsfsm.BuildFSReplication(fs.Path) + if len(path) == 1 { + fsrfsm.AddStep(nil, path[0]) + } else { + for i := 0; i < len(path)-1; i++ { + fsrfsm.AddStep(path[i], path[i+1]) + } + } + qitem := fsrfsm.Done() + q.Add(qitem) + } + + return u(func(r *Replication) { + r.completed = nil + r.queue = q + r.planningError = nil + r.state = Working + }).rsf() +} + +func rsfPlanningError(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { + sleepTime := 10 * time.Second + u(func(r *Replication) { + r.sleepUntil = time.Now().Add(sleepTime) + }) + t := time.NewTimer(sleepTime) // FIXME make constant onfigurable + defer t.Stop() + select { + case <-ctx.Done(): + return u(func(r *Replication) { + r.state = ContextDone + r.contextError = ctx.Err() + }).rsf() + case <-t.C: + return u(func(r *Replication) { + r.state = Planning + }).rsf() + } +} + +func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { + + var active *ReplicationQueueItemHandle + rsfNext := u(func(r *Replication) { + done, next := r.queue.GetNext() + r.completed = append(r.completed, done...) + if next == nil { + r.state = Completed + } + r.active = next + active = next + }).rsf() + + if active == nil { + return rsfNext + } + + state, nextStepDate := active.GetFSReplication().TakeStep(ctx, ep) + + return u(func(r *Replication) { + active.Update(state, nextStepDate) + r.active = nil + }).rsf() +} + +func rsfWorkingWait(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { + sleepTime := 10 * time.Second + u(func(r *Replication) { + r.sleepUntil = time.Now().Add(sleepTime) + }) + t := time.NewTimer(sleepTime) + defer t.Stop() + select { + case <-ctx.Done(): + return u(func(r *Replication) { + r.state = ContextDone + r.contextError = ctx.Err() + }).rsf() + case <-t.C: + return u(func(r *Replication) { + r.state = Working + }).rsf() + } +} + +func (r *Replication) Report() *Report { + r.lock.Lock() + defer r.lock.Unlock() + + rep := Report{ + Status: r.state.String(), + } + + if r.state&(Planning|PlanningError|ContextDone) != 0 { + switch r.state { + case PlanningError: + rep.Problem = r.planningError.Error() + case ContextDone: + rep.Problem = r.contextError.Error() + } + return &rep + } + + rep.Pending = make([]*fsfsm.FilesystemReplicationReport, 0, r.queue.Len()) + rep.Completed = make([]*fsfsm.FilesystemReplicationReport, 0, len(r.completed)) // room for active (potentially) + + var active *fsfsm.FSReplication + if r.active != nil { + active = r.active.GetFSReplication() + rep.Active = active.Report() + } + r.queue.Foreach(func (h *ReplicationQueueItemHandle){ + fsr := h.GetFSReplication() + if active != fsr { + rep.Pending = append(rep.Pending, fsr.Report()) + } + }) + for _, fsr := range r.completed { + rep.Completed = append(rep.Completed, fsr.Report()) + } + + return &rep +} + diff --git a/cmd/replication/internal/mainfsm/queue/queue.go b/cmd/replication/internal/mainfsm/queue/queue.go new file mode 100644 index 0000000..e2b588d --- /dev/null +++ b/cmd/replication/internal/mainfsm/queue/queue.go @@ -0,0 +1,124 @@ +package queue + +import ( + "time" + "sort" + + . "github.com/zrepl/zrepl/cmd/replication/internal/fsfsm" +) + +type replicationQueueItem struct { + retriesSinceLastError int + // duplicates fsr.state to avoid accessing and locking fsr + state FSReplicationState + // duplicates fsr.current.nextStepDate to avoid accessing & locking fsr + nextStepDate time.Time + + fsr *FSReplication +} + +type ReplicationQueue []*replicationQueueItem + +func NewReplicationQueue() *ReplicationQueue { + q := make(ReplicationQueue, 0) + return &q +} + +func (q ReplicationQueue) Len() int { return len(q) } +func (q ReplicationQueue) Swap(i, j int) { q[i], q[j] = q[j], q[i] } + +type lessmapEntry struct{ + prio int + less func(a,b *replicationQueueItem) bool +} + +var lessmap = map[FSReplicationState]lessmapEntry { + FSReady: { + prio: 0, + less: func(a, b *replicationQueueItem) bool { + return a.nextStepDate.Before(b.nextStepDate) + }, + }, + FSRetryWait: { + prio: 1, + less: func(a, b *replicationQueueItem) bool { + return a.retriesSinceLastError < b.retriesSinceLastError + }, + }, +} + +func (q ReplicationQueue) Less(i, j int) bool { + + a, b := q[i], q[j] + al, aok := lessmap[a.state] + if !aok { + panic(a) + } + bl, bok := lessmap[b.state] + if !bok { + panic(b) + } + + if al.prio != bl.prio { + return al.prio < bl.prio + } + + return al.less(a, b) +} + +func (q *ReplicationQueue) sort() (done []*FSReplication) { + // pre-scan for everything that is not ready + newq := make(ReplicationQueue, 0, len(*q)) + done = make([]*FSReplication, 0, len(*q)) + for _, qitem := range *q { + if _, ok := lessmap[qitem.state]; !ok { + done = append(done, qitem.fsr) + } else { + newq = append(newq, qitem) + } + } + sort.Stable(newq) // stable to avoid flickering in reports + *q = newq + return done +} + +// next remains valid until the next call to GetNext() +func (q *ReplicationQueue) GetNext() (done []*FSReplication, next *ReplicationQueueItemHandle) { + done = q.sort() + if len(*q) == 0 { + return done, nil + } + next = &ReplicationQueueItemHandle{(*q)[0]} + return done, next +} + +func (q *ReplicationQueue) Add(fsr *FSReplication) { + *q = append(*q, &replicationQueueItem{ + fsr: fsr, + state: fsr.State(), + }) +} + +func (q *ReplicationQueue) Foreach(fu func(*ReplicationQueueItemHandle)) { + for _, qitem := range *q { + fu(&ReplicationQueueItemHandle{qitem}) + } +} + +type ReplicationQueueItemHandle struct { + i *replicationQueueItem +} + +func (h ReplicationQueueItemHandle) GetFSReplication() *FSReplication { + return h.i.fsr +} + +func (h ReplicationQueueItemHandle) Update(newState FSReplicationState, nextStepDate time.Time) { + h.i.state = newState + h.i.nextStepDate = nextStepDate + if h.i.state&FSReady != 0 { + h.i.retriesSinceLastError = 0 + } else if h.i.state&FSRetryWait != 0 { + h.i.retriesSinceLastError++ + } +} diff --git a/cmd/replication.v2/replicationstate_string.go b/cmd/replication/internal/mainfsm/replicationstate_string.go similarity index 97% rename from cmd/replication.v2/replicationstate_string.go rename to cmd/replication/internal/mainfsm/replicationstate_string.go index e4381ba..46511af 100644 --- a/cmd/replication.v2/replicationstate_string.go +++ b/cmd/replication/internal/mainfsm/replicationstate_string.go @@ -1,6 +1,6 @@ // Code generated by "stringer -type=ReplicationState"; DO NOT EDIT. -package replication +package mainfsm import "strconv" diff --git a/cmd/replication.v2/pdu.pb.go b/cmd/replication/pdu/pdu.pb.go similarity index 72% rename from cmd/replication.v2/pdu.pb.go rename to cmd/replication/pdu/pdu.pb.go index 1d382bd..d9bed35 100644 --- a/cmd/replication.v2/pdu.pb.go +++ b/cmd/replication/pdu/pdu.pb.go @@ -2,7 +2,7 @@ // source: pdu.proto /* -Package replication is a generated protocol buffer package. +Package pdu is a generated protocol buffer package. It is generated from these files: pdu.proto @@ -20,7 +20,7 @@ It has these top-level messages: ReceiveReq ReceiveRes */ -package replication +package pdu import proto "github.com/golang/protobuf/proto" import fmt "fmt" @@ -141,7 +141,7 @@ func (m *ListFilesystemVersionsRes) GetVersions() []*FilesystemVersion { } type FilesystemVersion struct { - Type FilesystemVersion_VersionType `protobuf:"varint,1,opt,name=Type,enum=replication.FilesystemVersion_VersionType" json:"Type,omitempty"` + Type FilesystemVersion_VersionType `protobuf:"varint,1,opt,name=Type,enum=pdu.FilesystemVersion_VersionType" json:"Type,omitempty"` Name string `protobuf:"bytes,2,opt,name=Name" json:"Name,omitempty"` Guid uint64 `protobuf:"varint,3,opt,name=Guid" json:"Guid,omitempty"` CreateTXG uint64 `protobuf:"varint,4,opt,name=CreateTXG" json:"CreateTXG,omitempty"` @@ -191,7 +191,8 @@ func (m *FilesystemVersion) GetCreation() string { type SendReq struct { Filesystem string `protobuf:"bytes,1,opt,name=Filesystem" json:"Filesystem,omitempty"` From string `protobuf:"bytes,2,opt,name=From" json:"From,omitempty"` - To string `protobuf:"bytes,3,opt,name=To" json:"To,omitempty"` + // May be empty / null to request a full transfer of From + To string `protobuf:"bytes,3,opt,name=To" json:"To,omitempty"` // If ResumeToken is not empty, the resume token that CAN be tried for 'zfs send' by the sender. // The sender MUST indicate in SendRes.UsedResumeToken // If it does not work, the sender SHOULD clear the resume token on their side @@ -334,51 +335,50 @@ func (*ReceiveRes) ProtoMessage() {} func (*ReceiveRes) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} } func init() { - proto.RegisterType((*ListFilesystemReq)(nil), "replication.ListFilesystemReq") - proto.RegisterType((*ListFilesystemRes)(nil), "replication.ListFilesystemRes") - proto.RegisterType((*Filesystem)(nil), "replication.Filesystem") - proto.RegisterType((*ListFilesystemVersionsReq)(nil), "replication.ListFilesystemVersionsReq") - proto.RegisterType((*ListFilesystemVersionsRes)(nil), "replication.ListFilesystemVersionsRes") - proto.RegisterType((*FilesystemVersion)(nil), "replication.FilesystemVersion") - proto.RegisterType((*SendReq)(nil), "replication.SendReq") - proto.RegisterType((*Property)(nil), "replication.Property") - proto.RegisterType((*SendRes)(nil), "replication.SendRes") - proto.RegisterType((*ReceiveReq)(nil), "replication.ReceiveReq") - proto.RegisterType((*ReceiveRes)(nil), "replication.ReceiveRes") - proto.RegisterEnum("replication.FilesystemVersion_VersionType", FilesystemVersion_VersionType_name, FilesystemVersion_VersionType_value) + proto.RegisterType((*ListFilesystemReq)(nil), "pdu.ListFilesystemReq") + proto.RegisterType((*ListFilesystemRes)(nil), "pdu.ListFilesystemRes") + proto.RegisterType((*Filesystem)(nil), "pdu.Filesystem") + proto.RegisterType((*ListFilesystemVersionsReq)(nil), "pdu.ListFilesystemVersionsReq") + proto.RegisterType((*ListFilesystemVersionsRes)(nil), "pdu.ListFilesystemVersionsRes") + proto.RegisterType((*FilesystemVersion)(nil), "pdu.FilesystemVersion") + proto.RegisterType((*SendReq)(nil), "pdu.SendReq") + proto.RegisterType((*Property)(nil), "pdu.Property") + proto.RegisterType((*SendRes)(nil), "pdu.SendRes") + proto.RegisterType((*ReceiveReq)(nil), "pdu.ReceiveReq") + proto.RegisterType((*ReceiveRes)(nil), "pdu.ReceiveRes") + proto.RegisterEnum("pdu.FilesystemVersion_VersionType", FilesystemVersion_VersionType_name, FilesystemVersion_VersionType_value) } func init() { proto.RegisterFile("pdu.proto", fileDescriptor0) } var fileDescriptor0 = []byte{ - // 454 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x53, 0x4d, 0x6f, 0xd3, 0x40, - 0x10, 0x65, 0x53, 0xa7, 0x38, 0xe3, 0xaa, 0xa4, 0x4b, 0x11, 0x06, 0xa1, 0x2a, 0xda, 0x53, 0xe8, - 0x21, 0x87, 0x02, 0x07, 0x40, 0xe2, 0xd0, 0xa2, 0xf6, 0x82, 0xaa, 0x6a, 0x6b, 0x4a, 0xaf, 0xa6, - 0x1e, 0xa9, 0x4b, 0x62, 0xaf, 0xbb, 0x63, 0x23, 0xe5, 0xe7, 0xf0, 0xcf, 0xf8, 0x29, 0xc8, 0x53, - 0x3b, 0xd9, 0x26, 0x2a, 0xca, 0xc9, 0xf3, 0xde, 0x7c, 0xbd, 0x7d, 0xeb, 0x85, 0x41, 0x99, 0xd5, - 0x93, 0xd2, 0xd9, 0xca, 0xca, 0xc8, 0x61, 0x39, 0x33, 0x37, 0x69, 0x65, 0x6c, 0xa1, 0x9e, 0xc3, - 0xde, 0x37, 0x43, 0xd5, 0xa9, 0x99, 0x21, 0xcd, 0xa9, 0xc2, 0x5c, 0xe3, 0x9d, 0x3a, 0x5f, 0x27, - 0x49, 0x7e, 0x84, 0x68, 0x49, 0x50, 0x2c, 0x46, 0x5b, 0xe3, 0xe8, 0xe8, 0xe5, 0xc4, 0x1b, 0x36, - 0xf1, 0x1a, 0xfc, 0x5a, 0x75, 0x0c, 0xb0, 0x84, 0x52, 0x42, 0x70, 0x91, 0x56, 0xb7, 0xb1, 0x18, - 0x89, 0xf1, 0x40, 0x73, 0x2c, 0x47, 0x10, 0x69, 0xa4, 0x3a, 0xc7, 0xc4, 0x4e, 0xb1, 0x88, 0x7b, - 0x9c, 0xf2, 0x29, 0xf5, 0x19, 0x5e, 0x3d, 0xd4, 0x74, 0x85, 0x8e, 0x8c, 0x2d, 0x48, 0xe3, 0x9d, - 0x3c, 0xf0, 0x17, 0xb4, 0x83, 0x3d, 0x46, 0xfd, 0x78, 0xbc, 0x99, 0xe4, 0x27, 0x08, 0x3b, 0xd8, - 0x9e, 0xea, 0xe0, 0x91, 0x53, 0xb5, 0x65, 0x7a, 0x51, 0xaf, 0xfe, 0x0a, 0xd8, 0x5b, 0xcb, 0xcb, - 0x2f, 0x10, 0x24, 0xf3, 0x12, 0x59, 0xc8, 0xee, 0xd1, 0xe1, 0xff, 0xa7, 0x4d, 0xda, 0x6f, 0xd3, - 0xa1, 0xb9, 0xaf, 0x71, 0xe8, 0x3c, 0xcd, 0xb1, 0xb5, 0x81, 0xe3, 0x86, 0x3b, 0xab, 0x4d, 0x16, - 0x6f, 0x8d, 0xc4, 0x38, 0xd0, 0x1c, 0xcb, 0x37, 0x30, 0x38, 0x71, 0x98, 0x56, 0x98, 0x5c, 0x9f, - 0xc5, 0x01, 0x27, 0x96, 0x84, 0x7c, 0x0d, 0x21, 0x03, 0x63, 0x8b, 0xb8, 0xcf, 0x93, 0x16, 0x58, - 0xbd, 0x85, 0xc8, 0x5b, 0x2b, 0x77, 0x20, 0xbc, 0x2c, 0xd2, 0x92, 0x6e, 0x6d, 0x35, 0x7c, 0xd2, - 0xa0, 0x63, 0x6b, 0xa7, 0x79, 0xea, 0xa6, 0x43, 0xa1, 0xfe, 0x08, 0x78, 0x7a, 0x89, 0x45, 0xb6, - 0x81, 0xcf, 0x8d, 0xc8, 0x53, 0x67, 0xf3, 0x4e, 0x78, 0x13, 0xcb, 0x5d, 0xe8, 0x25, 0x96, 0x65, - 0x0f, 0x74, 0x2f, 0xb1, 0xab, 0x57, 0x1d, 0xac, 0x5d, 0x35, 0x0b, 0xb7, 0x79, 0xe9, 0x90, 0x88, - 0x85, 0x87, 0x7a, 0x81, 0xe5, 0x3e, 0xf4, 0xbf, 0x62, 0x56, 0x97, 0xf1, 0x36, 0x27, 0xee, 0x81, - 0x7a, 0x0f, 0xe1, 0x85, 0xb3, 0x25, 0xba, 0x6a, 0xbe, 0x30, 0x4f, 0x78, 0xe6, 0xed, 0x43, 0xff, - 0x2a, 0x9d, 0xd5, 0x9d, 0xa3, 0xf7, 0x40, 0xfd, 0xea, 0x0e, 0x46, 0x72, 0x0c, 0xcf, 0xbe, 0x13, - 0x66, 0xbe, 0x30, 0xc1, 0x0b, 0x56, 0x69, 0xf9, 0x01, 0xa0, 0x5d, 0x65, 0x90, 0xe2, 0x1e, 0xff, - 0x2f, 0x2f, 0x1e, 0xdc, 0x70, 0xa7, 0x44, 0x7b, 0x85, 0xea, 0x1a, 0x40, 0xe3, 0x0d, 0x9a, 0xdf, - 0xb8, 0x89, 0x8f, 0x87, 0x30, 0x3c, 0x99, 0x61, 0xea, 0x56, 0xdf, 0x44, 0xa8, 0xd7, 0x78, 0xb5, - 0xe3, 0x4d, 0xa6, 0x9f, 0xdb, 0xfc, 0xc6, 0xdf, 0xfd, 0x0b, 0x00, 0x00, 0xff, 0xff, 0xa4, 0x5a, - 0xf6, 0xa7, 0xf0, 0x03, 0x00, 0x00, + // 445 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x53, 0xc1, 0x6e, 0xd3, 0x40, + 0x10, 0x65, 0x13, 0xa7, 0x38, 0x93, 0xd2, 0xa6, 0x4b, 0x85, 0x0c, 0x42, 0x28, 0xda, 0x53, 0x40, + 0x22, 0x12, 0x01, 0x71, 0xe1, 0xd6, 0xa2, 0xf4, 0x82, 0xa0, 0xda, 0x9a, 0xaa, 0x57, 0x17, 0x8f, + 0x54, 0x2b, 0xb1, 0x77, 0xbb, 0x63, 0x23, 0xe5, 0x73, 0xf8, 0x2b, 0x3e, 0x07, 0x79, 0x6a, 0x27, + 0x4b, 0x0c, 0x52, 0x4e, 0x99, 0xf7, 0x66, 0x32, 0xf3, 0xe6, 0xcd, 0x1a, 0x86, 0x36, 0xad, 0x66, + 0xd6, 0x99, 0xd2, 0xc8, 0xbe, 0x4d, 0x2b, 0xf5, 0x14, 0x4e, 0xbe, 0x64, 0x54, 0x2e, 0xb2, 0x15, + 0xd2, 0x9a, 0x4a, 0xcc, 0x35, 0xde, 0xab, 0x45, 0x97, 0x24, 0xf9, 0x0e, 0x46, 0x5b, 0x82, 0x22, + 0x31, 0xe9, 0x4f, 0x47, 0xf3, 0xe3, 0x59, 0xdd, 0xcf, 0x2b, 0xf4, 0x6b, 0xd4, 0x19, 0xc0, 0x16, + 0x4a, 0x09, 0xc1, 0x65, 0x52, 0xde, 0x45, 0x62, 0x22, 0xa6, 0x43, 0xcd, 0xb1, 0x9c, 0xc0, 0x48, + 0x23, 0x55, 0x39, 0xc6, 0x66, 0x89, 0x45, 0xd4, 0xe3, 0x94, 0x4f, 0xa9, 0x4f, 0xf0, 0xfc, 0x6f, + 0x2d, 0xd7, 0xe8, 0x28, 0x33, 0x05, 0x69, 0xbc, 0x97, 0xaf, 0xfc, 0x01, 0x4d, 0x63, 0x8f, 0x51, + 0xdf, 0xfe, 0xff, 0x67, 0x92, 0x73, 0x08, 0x5b, 0xd8, 0x6c, 0xf3, 0x6c, 0x67, 0x9b, 0x26, 0xad, + 0x37, 0x75, 0xea, 0xb7, 0x80, 0x93, 0x4e, 0x5e, 0x7e, 0x84, 0x20, 0x5e, 0x5b, 0x64, 0x01, 0x47, + 0x73, 0xf5, 0xef, 0x2e, 0xb3, 0xe6, 0xb7, 0xae, 0xd4, 0x5c, 0x5f, 0x3b, 0xf2, 0x35, 0xc9, 0xb1, + 0x59, 0x9b, 0xe3, 0x9a, 0xbb, 0xa8, 0xb2, 0x34, 0xea, 0x4f, 0xc4, 0x34, 0xd0, 0x1c, 0xcb, 0x97, + 0x30, 0x3c, 0x77, 0x98, 0x94, 0x18, 0xdf, 0x5c, 0x44, 0x01, 0x27, 0xb6, 0x84, 0x7c, 0x01, 0x21, + 0x83, 0xcc, 0x14, 0xd1, 0x80, 0x3b, 0x6d, 0xb0, 0x7a, 0x0d, 0x23, 0x6f, 0xac, 0x3c, 0x84, 0xf0, + 0xaa, 0x48, 0x2c, 0xdd, 0x99, 0x72, 0xfc, 0xa8, 0x46, 0x67, 0xc6, 0x2c, 0xf3, 0xc4, 0x2d, 0xc7, + 0x42, 0xfd, 0x12, 0xf0, 0xf8, 0x0a, 0x8b, 0x74, 0x0f, 0x5f, 0x6b, 0x91, 0x0b, 0x67, 0xf2, 0x56, + 0x78, 0x1d, 0xcb, 0x23, 0xe8, 0xc5, 0x86, 0x65, 0x0f, 0x75, 0x2f, 0x36, 0xbb, 0xa7, 0x0d, 0x3a, + 0xa7, 0x65, 0xe1, 0x26, 0xb7, 0x0e, 0x89, 0x58, 0x78, 0xa8, 0x37, 0x58, 0x9e, 0xc2, 0xe0, 0x33, + 0xa6, 0x95, 0x8d, 0x0e, 0x38, 0xf1, 0x00, 0xd4, 0x07, 0x08, 0x2f, 0x9d, 0xb1, 0xe8, 0xca, 0xf5, + 0xc6, 0x3c, 0xe1, 0x99, 0x77, 0x0a, 0x83, 0xeb, 0x64, 0x55, 0xb5, 0x8e, 0x3e, 0x00, 0x75, 0xdb, + 0x2e, 0x46, 0x72, 0x0a, 0xc7, 0xdf, 0x09, 0x53, 0x5f, 0x98, 0xe0, 0x01, 0xbb, 0xb4, 0x7c, 0x0b, + 0xd0, 0x8c, 0xca, 0x90, 0xa2, 0x1e, 0xbf, 0x8f, 0x27, 0x7c, 0xd9, 0x56, 0x81, 0xf6, 0x0a, 0xd4, + 0x0d, 0x80, 0xc6, 0x1f, 0x98, 0xfd, 0xc4, 0x7d, 0xfc, 0x7b, 0x03, 0xe3, 0xf3, 0x15, 0x26, 0x6e, + 0xf7, 0xed, 0x87, 0xba, 0xc3, 0xab, 0x43, 0xaf, 0x33, 0xdd, 0x1e, 0xf0, 0xb7, 0xfb, 0xfe, 0x4f, + 0x00, 0x00, 0x00, 0xff, 0xff, 0xfd, 0xc4, 0x8d, 0xb7, 0xc8, 0x03, 0x00, 0x00, } diff --git a/cmd/replication.v2/pdu.proto b/cmd/replication/pdu/pdu.proto similarity index 98% rename from cmd/replication.v2/pdu.proto rename to cmd/replication/pdu/pdu.proto index 72fc56c..cac47b1 100644 --- a/cmd/replication.v2/pdu.proto +++ b/cmd/replication/pdu/pdu.proto @@ -1,6 +1,6 @@ syntax = "proto3"; -package replication; +package pdu; message ListFilesystemReq {} diff --git a/cmd/replication.v2/pdu_extras.go b/cmd/replication/pdu/pdu_extras.go similarity index 87% rename from cmd/replication.v2/pdu_extras.go rename to cmd/replication/pdu/pdu_extras.go index 0848bd6..19e27e8 100644 --- a/cmd/replication.v2/pdu_extras.go +++ b/cmd/replication/pdu/pdu_extras.go @@ -1,4 +1,4 @@ -package replication +package pdu import ( "fmt" @@ -45,6 +45,15 @@ func (v *FilesystemVersion) CreationAsTime() (time.Time, error) { return time.Parse(time.RFC3339, v.Creation) } +// implement fsfsm.FilesystemVersion +func (v *FilesystemVersion) SnapshotTime() time.Time { + t, err := v.CreationAsTime() + if err != nil { + panic(err) // FIXME + } + return t +} + func (v *FilesystemVersion) ZFSFilesystemVersion() *zfs.FilesystemVersion { ct := time.Time{} if v.Creation != "" { @@ -61,4 +70,4 @@ func (v *FilesystemVersion) ZFSFilesystemVersion() *zfs.FilesystemVersion { CreateTXG: v.CreateTXG, Creation: ct, } -} \ No newline at end of file +} diff --git a/cmd/replication.v2/pdu_test.go b/cmd/replication/pdu/pdu_test.go similarity index 98% rename from cmd/replication.v2/pdu_test.go rename to cmd/replication/pdu/pdu_test.go index 3b26572..86e2b49 100644 --- a/cmd/replication.v2/pdu_test.go +++ b/cmd/replication/pdu/pdu_test.go @@ -1,4 +1,4 @@ -package replication +package pdu import ( "testing" diff --git a/cmd/replication/replication.go b/cmd/replication/replication.go new file mode 100644 index 0000000..75b1dd5 --- /dev/null +++ b/cmd/replication/replication.go @@ -0,0 +1,19 @@ +package replication + +import ( + "context" + + "github.com/zrepl/zrepl/cmd/replication/common" + "github.com/zrepl/zrepl/cmd/replication/internal/mainfsm" +) + +type Report = mainfsm.Report + +type Replication interface { + Drive(ctx context.Context, ep common.EndpointPair) + Report() *Report +} + +func NewReplication() Replication { + return mainfsm.NewReplication() +} From 2f205d205bcdb867d8ce5544a1b172ee85c5181f Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 21 Aug 2018 22:15:00 +0200 Subject: [PATCH 017/167] remove EndpointPair abstraction --- cmd/replication/common/common.go | 46 --------------------- cmd/replication/internal/fsfsm/fsfsm.go | 18 ++++---- cmd/replication/internal/mainfsm/mainfsm.go | 28 ++++++------- cmd/replication/replication.go | 2 +- 4 files changed, 24 insertions(+), 70 deletions(-) diff --git a/cmd/replication/common/common.go b/cmd/replication/common/common.go index 3166bfc..2aa37b0 100644 --- a/cmd/replication/common/common.go +++ b/cmd/replication/common/common.go @@ -43,49 +43,3 @@ func NewFilteredError(fs string) *FilteredError { } func (f FilteredError) Error() string { return "endpoint does not allow access to filesystem " + f.fs } - -type ReplicationMode int - -const ( - ReplicationModePull ReplicationMode = iota - ReplicationModePush -) - -type EndpointPair struct { - a, b ReplicationEndpoint - m ReplicationMode -} - -func NewEndpointPairPull(sender, receiver ReplicationEndpoint) EndpointPair { - return EndpointPair{sender, receiver, ReplicationModePull} -} - -func NewEndpointPairPush(sender, receiver ReplicationEndpoint) EndpointPair { - return EndpointPair{receiver, sender, ReplicationModePush} -} - -func (p EndpointPair) Sender() ReplicationEndpoint { - switch p.m { - case ReplicationModePull: - return p.a - case ReplicationModePush: - return p.b - } - panic("should not be reached") - return nil -} - -func (p EndpointPair) Receiver() ReplicationEndpoint { - switch p.m { - case ReplicationModePull: - return p.b - case ReplicationModePush: - return p.a - } - panic("should not be reached") - return nil -} - -func (p EndpointPair) Mode() ReplicationMode { - return p.m -} diff --git a/cmd/replication/internal/fsfsm/fsfsm.go b/cmd/replication/internal/fsfsm/fsfsm.go index d5c3629..467c716 100644 --- a/cmd/replication/internal/fsfsm/fsfsm.go +++ b/cmd/replication/internal/fsfsm/fsfsm.go @@ -135,7 +135,7 @@ type FSReplicationStep struct { err error } -func (f *FSReplication) TakeStep(ctx context.Context, ep EndpointPair) (post FSReplicationState, nextStepDate time.Time) { +func (f *FSReplication) TakeStep(ctx context.Context, sender, receiver ReplicationEndpoint) (post FSReplicationState, nextStepDate time.Time) { var u fsrUpdater = func(fu func(*FSReplication)) FSReplicationState { f.lock.Lock() @@ -149,7 +149,7 @@ func (f *FSReplication) TakeStep(ctx context.Context, ep EndpointPair) (post FSR pre := u(nil) preTime := time.Now() - s = s(ctx, ep, u) + s = s(ctx, sender, receiver, u) delta := time.Now().Sub(preTime) post = u(func(f *FSReplication) { if len(f.pending) == 0 { @@ -169,9 +169,9 @@ func (f *FSReplication) TakeStep(ctx context.Context, ep EndpointPair) (post FSR type fsrUpdater func(func(fsr *FSReplication)) FSReplicationState -type fsrsf func(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf +type fsrsf func(ctx context.Context, sender, receiver ReplicationEndpoint, u fsrUpdater) fsrsf -func fsrsfReady(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf { +func fsrsfReady(ctx context.Context, sender, receiver ReplicationEndpoint, u fsrUpdater) fsrsf { var current *FSReplicationStep s := u(func(f *FSReplication) { @@ -185,7 +185,7 @@ func fsrsfReady(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf { return s.fsrsf() } - stepState := current.do(ctx, ep) + stepState := current.do(ctx, sender, receiver) return u(func(f *FSReplication) { switch stepState { @@ -209,7 +209,7 @@ func fsrsfReady(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf { }).fsrsf() } -func fsrsfRetryWait(ctx context.Context, ep EndpointPair, u fsrUpdater) fsrsf { +func fsrsfRetryWait(ctx context.Context, sender, receiver ReplicationEndpoint, u fsrUpdater) fsrsf { var sleepUntil time.Time u(func(f *FSReplication) { sleepUntil = f.retryWaitUntil @@ -255,7 +255,7 @@ func (fsr *FSReplication) Report() *FilesystemReplicationReport { return &rep } -func (s *FSReplicationStep) do(ctx context.Context, ep EndpointPair) FSReplicationStepState { +func (s *FSReplicationStep) do(ctx context.Context, sender, receiver ReplicationEndpoint) FSReplicationStepState { fs := s.fsrep.fs @@ -308,7 +308,7 @@ func (s *FSReplicationStep) do(ctx context.Context, ep EndpointPair) FSReplicati } log.WithField("request", sr).Debug("initiate send request") - sres, sstream, err := ep.Sender().Send(ctx, sr) + sres, sstream, err := sender.Send(ctx, sr) if err != nil { log.WithError(err).Error("send request failed") return updateStateError(err) @@ -323,7 +323,7 @@ func (s *FSReplicationStep) do(ctx context.Context, ep EndpointPair) FSReplicati ClearResumeToken: !sres.UsedResumeToken, } log.WithField("request", rr).Debug("initiate receive request") - err = ep.Receiver().Receive(ctx, rr, sstream) + err = receiver.Receive(ctx, rr, sstream) if err != nil { log.WithError(err).Error("receive request failed (might also be error on sender)") sstream.Close() diff --git a/cmd/replication/internal/mainfsm/mainfsm.go b/cmd/replication/internal/mainfsm/mainfsm.go index 883d6d5..5d5f944 100644 --- a/cmd/replication/internal/mainfsm/mainfsm.go +++ b/cmd/replication/internal/mainfsm/mainfsm.go @@ -80,9 +80,9 @@ func NewReplication() *Replication { } type replicationUpdater func(func(*Replication)) (newState ReplicationState) -type replicationStateFunc func(context.Context, EndpointPair, replicationUpdater) replicationStateFunc +type replicationStateFunc func(ctx context.Context, sender, receiver ReplicationEndpoint, u replicationUpdater) replicationStateFunc -func (r *Replication) Drive(ctx context.Context, ep EndpointPair) { +func (r *Replication) Drive(ctx context.Context, sender, receiver ReplicationEndpoint) { var u replicationUpdater = func(f func(*Replication)) ReplicationState { r.lock.Lock() @@ -98,7 +98,7 @@ func (r *Replication) Drive(ctx context.Context, ep EndpointPair) { for s != nil { preTime := time.Now() pre = u(nil) - s = s(ctx, ep, u) + s = s(ctx, sender, receiver, u) delta := time.Now().Sub(preTime) post = u(nil) GetLogger(ctx). @@ -133,7 +133,7 @@ func resolveConflict(conflict error) (path []*pdu.FilesystemVersion, msg string) return nil, "no automated way to handle conflict type" } -func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { +func rsfPlanning(ctx context.Context, sender, receiver ReplicationEndpoint, u replicationUpdater) replicationStateFunc { log := GetLogger(ctx) @@ -144,13 +144,13 @@ func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) rep }).rsf() } - sfss, err := ep.Sender().ListFilesystems(ctx) + sfss, err := sender.ListFilesystems(ctx) if err != nil { log.WithError(err).Error("error listing sender filesystems") return handlePlanningError(err) } - rfss, err := ep.Receiver().ListFilesystems(ctx) + rfss, err := receiver.ListFilesystems(ctx) if err != nil { log.WithError(err).Error("error listing receiver filesystems") return handlePlanningError(err) @@ -164,7 +164,7 @@ func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) rep log.Info("assessing filesystem") - sfsvs, err := ep.Sender().ListFilesystemVersions(ctx, fs.Path) + sfsvs, err := sender.ListFilesystemVersions(ctx, fs.Path) if err != nil { log.WithError(err).Error("cannot get remote filesystem versions") return handlePlanningError(err) @@ -186,7 +186,7 @@ func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) rep var rfsvs []*pdu.FilesystemVersion if receiverFSExists { - rfsvs, err = ep.Receiver().ListFilesystemVersions(ctx, fs.Path) + rfsvs, err = receiver.ListFilesystemVersions(ctx, fs.Path) if err != nil { if _, ok := err.(*FilteredError); ok { log.Info("receiver ignores filesystem") @@ -236,7 +236,7 @@ func rsfPlanning(ctx context.Context, ep EndpointPair, u replicationUpdater) rep }).rsf() } -func rsfPlanningError(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { +func rsfPlanningError(ctx context.Context, sender, receiver ReplicationEndpoint, u replicationUpdater) replicationStateFunc { sleepTime := 10 * time.Second u(func(r *Replication) { r.sleepUntil = time.Now().Add(sleepTime) @@ -256,7 +256,7 @@ func rsfPlanningError(ctx context.Context, ep EndpointPair, u replicationUpdater } } -func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { +func rsfWorking(ctx context.Context, sender, receiver ReplicationEndpoint, u replicationUpdater) replicationStateFunc { var active *ReplicationQueueItemHandle rsfNext := u(func(r *Replication) { @@ -273,7 +273,7 @@ func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) repl return rsfNext } - state, nextStepDate := active.GetFSReplication().TakeStep(ctx, ep) + state, nextStepDate := active.GetFSReplication().TakeStep(ctx, sender, receiver) return u(func(r *Replication) { active.Update(state, nextStepDate) @@ -281,7 +281,7 @@ func rsfWorking(ctx context.Context, ep EndpointPair, u replicationUpdater) repl }).rsf() } -func rsfWorkingWait(ctx context.Context, ep EndpointPair, u replicationUpdater) replicationStateFunc { +func rsfWorkingWait(ctx context.Context, sender, receiver ReplicationEndpoint, u replicationUpdater) replicationStateFunc { sleepTime := 10 * time.Second u(func(r *Replication) { r.sleepUntil = time.Now().Add(sleepTime) @@ -327,14 +327,14 @@ func (r *Replication) Report() *Report { active = r.active.GetFSReplication() rep.Active = active.Report() } - r.queue.Foreach(func (h *ReplicationQueueItemHandle){ + r.queue.Foreach(func(h *ReplicationQueueItemHandle) { fsr := h.GetFSReplication() if active != fsr { rep.Pending = append(rep.Pending, fsr.Report()) } }) for _, fsr := range r.completed { - rep.Completed = append(rep.Completed, fsr.Report()) + rep.Completed = append(rep.Completed, fsr.Report()) } return &rep diff --git a/cmd/replication/replication.go b/cmd/replication/replication.go index 75b1dd5..5d103a2 100644 --- a/cmd/replication/replication.go +++ b/cmd/replication/replication.go @@ -10,7 +10,7 @@ import ( type Report = mainfsm.Report type Replication interface { - Drive(ctx context.Context, ep common.EndpointPair) + Drive(ctx context.Context, sender, receiver common.ReplicationEndpoint) Report() *Report } From 301c7b2dd531b2db0362cc394821da72785730ee Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 22 Aug 2018 00:10:09 +0200 Subject: [PATCH 018/167] restructure and rename, making mainfsm the replication package itself --- Makefile | 7 +- cmd/config_job_local.go | 3 +- cmd/config_job_pull.go | 7 +- cmd/replication.go | 27 ++- cmd/replication/common/common.go | 45 ----- cmd/replication/context.go | 30 +++ .../{internal/fsfsm => fsrep}/fsfsm.go | 181 ++++++++++-------- cmd/replication/fsrep/state_string.go | 29 +++ cmd/replication/fsrep/stepstate_string.go | 29 +++ .../internal/{mainfsm => diff}/diff.go | 0 .../fsfsm/fsreplicationstate_string.go | 29 --- .../fsfsm/fsreplicationstepstate_string.go | 29 --- .../mainfsm/replicationstate_string.go | 35 ---- .../internal/{mainfsm => }/queue/queue.go | 28 +-- .../{internal/mainfsm => }/mainfsm.go | 118 ++++++++---- cmd/replication/replication.go | 19 -- cmd/replication/state_string.go | 35 ++++ 17 files changed, 348 insertions(+), 303 deletions(-) delete mode 100644 cmd/replication/common/common.go create mode 100644 cmd/replication/context.go rename cmd/replication/{internal/fsfsm => fsrep}/fsfsm.go (59%) create mode 100644 cmd/replication/fsrep/state_string.go create mode 100644 cmd/replication/fsrep/stepstate_string.go rename cmd/replication/internal/{mainfsm => diff}/diff.go (100%) delete mode 100644 cmd/replication/internal/fsfsm/fsreplicationstate_string.go delete mode 100644 cmd/replication/internal/fsfsm/fsreplicationstepstate_string.go delete mode 100644 cmd/replication/internal/mainfsm/replicationstate_string.go rename cmd/replication/internal/{mainfsm => }/queue/queue.go (75%) rename cmd/replication/{internal/mainfsm => }/mainfsm.go (65%) delete mode 100644 cmd/replication/replication.go create mode 100644 cmd/replication/state_string.go diff --git a/Makefile b/Makefile index 5944a64..619df9c 100644 --- a/Makefile +++ b/Makefile @@ -4,9 +4,10 @@ ROOT := github.com/zrepl/zrepl SUBPKGS := cmd SUBPKGS += cmd/replication -SUBPKGS += cmd/replication/internal/common -SUBPKGS += cmd/replication/internal/mainfsm -SUBPKGS += cmd/replication/internal/fsfsm +SUBPKGS += cmd/replication/fsrep +SUBPKGS += cmd/replication/pdu +SUBPKGS += cmd/replication/internal/queue +SUBPKGS += cmd/replication/internal/diff SUBPKGS += logger util zfs _TESTPKGS := $(ROOT) $(foreach p,$(SUBPKGS),$(ROOT)/$(p)) diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index 347a9d8..21fd6a6 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -9,7 +9,6 @@ import ( "github.com/zrepl/zrepl/zfs" "sync" "github.com/zrepl/zrepl/cmd/replication" - "github.com/zrepl/zrepl/cmd/replication/common" ) type LocalJob struct { @@ -148,7 +147,7 @@ outer: j.mainTask.Enter("replicate") rep := replication.NewReplication() - rep.Drive(ctx, common.NewEndpointPairPull(sender, receiver)) + rep.Drive(ctx, sender, receiver) j.mainTask.Finish() diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index bebfbd3..fe65d4f 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -14,7 +14,6 @@ import ( "github.com/pkg/errors" "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/cmd/replication" - "github.com/zrepl/zrepl/cmd/replication/common" ) type PullJob struct { @@ -30,7 +29,7 @@ type PullJob struct { Debug JobDebugSettings task *Task - rep replication.Replication + rep *replication.Replication } func parsePullJob(c JobParsingContext, name string, i map[string]interface{}) (j *PullJob, err error) { @@ -189,12 +188,12 @@ func (j *PullJob) doRun(ctx context.Context) { } - ctx = common.ContextWithLogger(ctx, replicationLogAdaptor{j.task.Log().WithField("subsystem", "replication")}) + ctx = replication.WithLogger(ctx, replicationLogAdaptor{j.task.Log().WithField("subsystem", "replication")}) ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{j.task.Log().WithField("subsystem", "rpc.protocol")}) ctx = context.WithValue(ctx, contextKeyLog, j.task.Log().WithField("subsystem", "rpc.endpoint")) j.rep = replication.NewReplication() - j.rep.Drive(ctx, common.NewEndpointPairPull(sender, puller)) + j.rep.Drive(ctx, sender, puller) client.Close() j.task.Finish() diff --git a/cmd/replication.go b/cmd/replication.go index d0f3ed9..0e1cc8a 100644 --- a/cmd/replication.go +++ b/cmd/replication.go @@ -2,7 +2,6 @@ package cmd import ( "fmt" - "github.com/zrepl/zrepl/cmd/replication/common" "github.com/zrepl/zrepl/cmd/replication/pdu" "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/zfs" @@ -11,6 +10,7 @@ import ( "github.com/golang/protobuf/proto" "bytes" "context" + "github.com/zrepl/zrepl/cmd/replication" ) type InitialReplPolicy string @@ -57,7 +57,7 @@ func (p *SenderEndpoint) ListFilesystemVersions(ctx context.Context, fs string) return nil, err } if !pass { - return nil, common.NewFilteredError(fs) + return nil, replication.NewFilteredError(fs) } fsvs, err := zfs.ZFSListFilesystemVersions(dp, p.FilesystemVersionFilter) if err != nil { @@ -80,7 +80,7 @@ func (p *SenderEndpoint) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes return nil, nil, err } if !pass { - return nil, nil, common.NewFilteredError(r.Filesystem) + return nil, nil, replication.NewFilteredError(r.Filesystem) } stream, err := zfs.ZFSSend(r.Filesystem, r.From, r.To) if err != nil { @@ -324,7 +324,7 @@ func (s RemoteEndpoint) Receive(ctx context.Context, r *pdu.ReceiveReq, sendStre } type HandlerAdaptor struct { - ep common.ReplicationEndpoint + ep replication.Endpoint } func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructured *bytes.Buffer, reqStream io.ReadCloser) (resStructured *bytes.Buffer, resStream io.ReadCloser, err error) { @@ -369,11 +369,16 @@ func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructu case RPCSend: + sender, ok := a.ep.(replication.Sender) + if !ok { + goto Err + } + var req pdu.SendReq if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { return nil, nil, err } - res, sendStream, err := a.ep.Send(ctx, &req) + res, sendStream, err := sender.Send(ctx, &req) if err != nil { return nil, nil, err } @@ -385,11 +390,16 @@ func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructu case RPCReceive: + receiver, ok := a.ep.(replication.Receiver) + if !ok { + goto Err + } + var req pdu.ReceiveReq if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { return nil, nil, err } - err := a.ep.Receive(ctx, &req, reqStream) + err := receiver.Receive(ctx, &req, reqStream) if err != nil { return nil, nil, err } @@ -399,8 +409,7 @@ func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructu } return bytes.NewBuffer(b), nil, err - - default: - return nil, nil, errors.New("no handler for given endpoint") } + Err: + return nil, nil, errors.New("no handler for given endpoint") } diff --git a/cmd/replication/common/common.go b/cmd/replication/common/common.go deleted file mode 100644 index 2aa37b0..0000000 --- a/cmd/replication/common/common.go +++ /dev/null @@ -1,45 +0,0 @@ -package common - -import ( - "context" - "io" - - "github.com/zrepl/zrepl/cmd/replication/pdu" - "github.com/zrepl/zrepl/logger" -) - -type contextKey int - -const ( - contextKeyLog contextKey = iota -) - -type Logger = logger.Logger - -func ContextWithLogger(ctx context.Context, l Logger) context.Context { - return context.WithValue(ctx, contextKeyLog, l) -} - -func GetLogger(ctx context.Context) Logger { - l, ok := ctx.Value(contextKeyLog).(Logger) - if !ok { - l = logger.NewNullLogger() - } - return l -} - -type ReplicationEndpoint interface { - // Does not include placeholder filesystems - ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) - ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) // fix depS - Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) - Receive(ctx context.Context, r *pdu.ReceiveReq, sendStream io.ReadCloser) error -} - -type FilteredError struct{ fs string } - -func NewFilteredError(fs string) *FilteredError { - return &FilteredError{fs} -} - -func (f FilteredError) Error() string { return "endpoint does not allow access to filesystem " + f.fs } diff --git a/cmd/replication/context.go b/cmd/replication/context.go new file mode 100644 index 0000000..775ee24 --- /dev/null +++ b/cmd/replication/context.go @@ -0,0 +1,30 @@ +package replication + +import ( + "github.com/zrepl/zrepl/logger" + "context" + "github.com/zrepl/zrepl/cmd/replication/fsrep" +) + +type contextKey int + +const ( + contextKeyLog contextKey = iota +) + +type Logger = logger.Logger + +func WithLogger(ctx context.Context, l Logger) context.Context { + ctx = context.WithValue(ctx, contextKeyLog, l) + ctx = fsrep.WithLogger(ctx, l) + return ctx +} + +func getLogger(ctx context.Context) Logger { + l, ok := ctx.Value(contextKeyLog).(Logger) + if !ok { + l = logger.NewNullLogger() + } + return l +} + diff --git a/cmd/replication/internal/fsfsm/fsfsm.go b/cmd/replication/fsrep/fsfsm.go similarity index 59% rename from cmd/replication/internal/fsfsm/fsfsm.go rename to cmd/replication/fsrep/fsfsm.go index 467c716..d70b84b 100644 --- a/cmd/replication/internal/fsfsm/fsfsm.go +++ b/cmd/replication/fsrep/fsfsm.go @@ -1,4 +1,6 @@ -package fsfsm +// Package fsrep implements replication of a single file system with existing versions +// from a sender to a receiver. +package fsrep import ( "context" @@ -11,16 +13,44 @@ import ( "time" "github.com/zrepl/zrepl/cmd/replication/pdu" - . "github.com/zrepl/zrepl/cmd/replication/common" + "github.com/zrepl/zrepl/logger" ) +type contextKey int + +const ( + contextKeyLogger contextKey = iota +) + +type Logger = logger.Logger + +func WithLogger(ctx context.Context, log Logger) context.Context { + return context.WithValue(ctx, contextKeyLogger, log) +} + +func getLogger(ctx context.Context) Logger { + l, ok := ctx.Value(contextKeyLogger).(Logger) + if !ok { + l = logger.NewNullLogger() + } + return l +} + +type Sender interface { + Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) +} + +type Receiver interface { + Receive(ctx context.Context, r *pdu.ReceiveReq, sendStream io.ReadCloser) error +} + type StepReport struct { From, To string Status string Problem string } -type FilesystemReplicationReport struct { +type Report struct { Filesystem string Status string Problem string @@ -28,90 +58,90 @@ type FilesystemReplicationReport struct { } -//go:generate stringer -type=FSReplicationState -type FSReplicationState uint +//go:generate stringer -type=State +type State uint const ( - FSReady FSReplicationState = 1 << iota - FSRetryWait - FSPermanentError - FSCompleted + Ready State = 1 << iota + RetryWait + PermanentError + Completed ) -func (s FSReplicationState) fsrsf() fsrsf { +func (s State) fsrsf() state { idx := bits.TrailingZeros(uint(s)) if idx == bits.UintSize { panic(s) } - m := []fsrsf{ - fsrsfReady, - fsrsfRetryWait, + m := []state{ + stateReady, + stateRetryWait, nil, nil, } return m[idx] } -type FSReplication struct { +type Replication struct { // lock protects all fields in this struct, but not the data behind pointers lock sync.Mutex - state FSReplicationState + state State fs string err error retryWaitUntil time.Time - completed, pending []*FSReplicationStep + completed, pending []*ReplicationStep } -func (f *FSReplication) State() FSReplicationState { +func (f *Replication) State() State { f.lock.Lock() defer f.lock.Unlock() return f.state } -type FSReplicationBuilder struct { - r *FSReplication +type ReplicationBuilder struct { + r *Replication } -func BuildFSReplication(fs string) *FSReplicationBuilder { - return &FSReplicationBuilder{&FSReplication{fs: fs}} +func BuildReplication(fs string) *ReplicationBuilder { + return &ReplicationBuilder{&Replication{fs: fs}} } -func (b *FSReplicationBuilder) AddStep(from, to FilesystemVersion) *FSReplicationBuilder { - step := &FSReplicationStep{ - state: StepReady, - fsrep: b.r, - from: from, - to: to, +func (b *ReplicationBuilder) AddStep(from, to FilesystemVersion) *ReplicationBuilder { + step := &ReplicationStep{ + state: StepReady, + parent: b.r, + from: from, + to: to, } b.r.pending = append(b.r.pending, step) return b } -func (b *FSReplicationBuilder) Done() (r *FSReplication) { +func (b *ReplicationBuilder) Done() (r *Replication) { if len(b.r.pending) > 0 { - b.r.state = FSReady + b.r.state = Ready } else { - b.r.state = FSCompleted + b.r.state = Completed } r = b.r b.r = nil return r } -func NewFSReplicationWithPermanentError(fs string, err error) *FSReplication { - return &FSReplication{ - state: FSPermanentError, +func NewReplicationWithPermanentError(fs string, err error) *Replication { + return &Replication{ + state: PermanentError, fs: fs, err: err, } } -//go:generate stringer -type=FSReplicationStepState -type FSReplicationStepState uint +//go:generate stringer -type=StepState +type StepState uint const ( - StepReady FSReplicationStepState = 1 << iota + StepReady StepState = 1 << iota StepRetry StepPermanentError StepCompleted @@ -122,22 +152,22 @@ type FilesystemVersion interface { RelName() string } -type FSReplicationStep struct { +type ReplicationStep struct { // only protects state, err - // from, to and fsrep are assumed to be immutable + // from, to and parent are assumed to be immutable lock sync.Mutex - state FSReplicationStepState + state StepState from, to FilesystemVersion - fsrep *FSReplication + parent *Replication // both retry and permanent error err error } -func (f *FSReplication) TakeStep(ctx context.Context, sender, receiver ReplicationEndpoint) (post FSReplicationState, nextStepDate time.Time) { +func (f *Replication) TakeStep(ctx context.Context, sender Sender, receiver Receiver) (post State, nextStepDate time.Time) { - var u fsrUpdater = func(fu func(*FSReplication)) FSReplicationState { + var u updater = func(fu func(*Replication)) State { f.lock.Lock() defer f.lock.Unlock() if fu != nil { @@ -145,20 +175,20 @@ func (f *FSReplication) TakeStep(ctx context.Context, sender, receiver Replicati } return f.state } - var s fsrsf = u(nil).fsrsf() + var s state = u(nil).fsrsf() pre := u(nil) preTime := time.Now() s = s(ctx, sender, receiver, u) delta := time.Now().Sub(preTime) - post = u(func(f *FSReplication) { + post = u(func(f *Replication) { if len(f.pending) == 0 { return } nextStepDate = f.pending[0].to.SnapshotTime() }) - GetLogger(ctx). + getLogger(ctx). WithField("fs", f.fs). WithField("transition", fmt.Sprintf("%s => %s", pre, post)). WithField("duration", delta). @@ -167,41 +197,41 @@ func (f *FSReplication) TakeStep(ctx context.Context, sender, receiver Replicati return post, nextStepDate } -type fsrUpdater func(func(fsr *FSReplication)) FSReplicationState +type updater func(func(fsr *Replication)) State -type fsrsf func(ctx context.Context, sender, receiver ReplicationEndpoint, u fsrUpdater) fsrsf +type state func(ctx context.Context, sender Sender, receiver Receiver, u updater) state -func fsrsfReady(ctx context.Context, sender, receiver ReplicationEndpoint, u fsrUpdater) fsrsf { +func stateReady(ctx context.Context, sender Sender, receiver Receiver, u updater) state { - var current *FSReplicationStep - s := u(func(f *FSReplication) { + var current *ReplicationStep + s := u(func(f *Replication) { if len(f.pending) == 0 { - f.state = FSCompleted + f.state = Completed return } current = f.pending[0] }) - if s != FSReady { + if s != Ready { return s.fsrsf() } stepState := current.do(ctx, sender, receiver) - return u(func(f *FSReplication) { + return u(func(f *Replication) { switch stepState { case StepCompleted: f.completed = append(f.completed, current) f.pending = f.pending[1:] if len(f.pending) > 0 { - f.state = FSReady + f.state = Ready } else { - f.state = FSCompleted + f.state = Completed } case StepRetry: f.retryWaitUntil = time.Now().Add(10 * time.Second) // FIXME make configurable - f.state = FSRetryWait + f.state = RetryWait case StepPermanentError: - f.state = FSPermanentError + f.state = PermanentError f.err = errors.New("a replication step failed with a permanent error") default: panic(f) @@ -209,37 +239,36 @@ func fsrsfReady(ctx context.Context, sender, receiver ReplicationEndpoint, u fsr }).fsrsf() } -func fsrsfRetryWait(ctx context.Context, sender, receiver ReplicationEndpoint, u fsrUpdater) fsrsf { +func stateRetryWait(ctx context.Context, sender Sender, receiver Receiver, u updater) state { var sleepUntil time.Time - u(func(f *FSReplication) { + u(func(f *Replication) { sleepUntil = f.retryWaitUntil }) t := time.NewTimer(sleepUntil.Sub(time.Now())) defer t.Stop() select { case <-ctx.Done(): - return u(func(f *FSReplication) { - f.state = FSPermanentError + return u(func(f *Replication) { + f.state = PermanentError f.err = ctx.Err() }).fsrsf() case <-t.C: } - return u(func(f *FSReplication) { - f.state = FSReady + return u(func(f *Replication) { + f.state = Ready }).fsrsf() } -// access to fsr's members must be exclusive -func (fsr *FSReplication) Report() *FilesystemReplicationReport { +func (fsr *Replication) Report() *Report { fsr.lock.Lock() defer fsr.lock.Unlock() - rep := FilesystemReplicationReport{ + rep := Report{ Filesystem: fsr.fs, Status: fsr.state.String(), } - if fsr.state&FSPermanentError != 0 { + if fsr.state&PermanentError != 0 { rep.Problem = fsr.err.Error() return &rep } @@ -255,15 +284,15 @@ func (fsr *FSReplication) Report() *FilesystemReplicationReport { return &rep } -func (s *FSReplicationStep) do(ctx context.Context, sender, receiver ReplicationEndpoint) FSReplicationStepState { +func (s *ReplicationStep) do(ctx context.Context, sender Sender, receiver Receiver) StepState { - fs := s.fsrep.fs + fs := s.parent.fs - log := GetLogger(ctx). + log := getLogger(ctx). WithField("filesystem", fs). WithField("step", s.String()) - updateStateError := func(err error) FSReplicationStepState { + updateStateError := func(err error) StepState { s.lock.Lock() defer s.lock.Unlock() @@ -285,7 +314,7 @@ func (s *FSReplicationStep) do(ctx context.Context, sender, receiver Replication return s.state } - updateStateCompleted := func() FSReplicationStepState { + updateStateCompleted := func() StepState { s.lock.Lock() defer s.lock.Unlock() s.err = nil @@ -338,15 +367,15 @@ func (s *FSReplicationStep) do(ctx context.Context, sender, receiver Replication } -func (s *FSReplicationStep) String() string { +func (s *ReplicationStep) String() string { if s.from == nil { // FIXME: ZFS semantics are that to is nil on non-incremental send - return fmt.Sprintf("%s%s (full)", s.fsrep.fs, s.to.RelName()) + return fmt.Sprintf("%s%s (full)", s.parent.fs, s.to.RelName()) } else { - return fmt.Sprintf("%s(%s => %s)", s.fsrep.fs, s.from, s.to.RelName()) + return fmt.Sprintf("%s(%s => %s)", s.parent.fs, s.from, s.to.RelName()) } } -func (step *FSReplicationStep) Report() *StepReport { +func (step *ReplicationStep) Report() *StepReport { var from string // FIXME follow same convention as ZFS: to should be nil on full send if step.from != nil { from = step.from.RelName() diff --git a/cmd/replication/fsrep/state_string.go b/cmd/replication/fsrep/state_string.go new file mode 100644 index 0000000..8bf26f4 --- /dev/null +++ b/cmd/replication/fsrep/state_string.go @@ -0,0 +1,29 @@ +// Code generated by "stringer -type=State"; DO NOT EDIT. + +package fsrep + +import "strconv" + +const ( + _State_name_0 = "ReadyRetryWait" + _State_name_1 = "PermanentError" + _State_name_2 = "Completed" +) + +var ( + _State_index_0 = [...]uint8{0, 5, 14} +) + +func (i State) String() string { + switch { + case 1 <= i && i <= 2: + i -= 1 + return _State_name_0[_State_index_0[i]:_State_index_0[i+1]] + case i == 4: + return _State_name_1 + case i == 8: + return _State_name_2 + default: + return "State(" + strconv.FormatInt(int64(i), 10) + ")" + } +} diff --git a/cmd/replication/fsrep/stepstate_string.go b/cmd/replication/fsrep/stepstate_string.go new file mode 100644 index 0000000..8228bfc --- /dev/null +++ b/cmd/replication/fsrep/stepstate_string.go @@ -0,0 +1,29 @@ +// Code generated by "stringer -type=StepState"; DO NOT EDIT. + +package fsrep + +import "strconv" + +const ( + _StepState_name_0 = "StepReadyStepRetry" + _StepState_name_1 = "StepPermanentError" + _StepState_name_2 = "StepCompleted" +) + +var ( + _StepState_index_0 = [...]uint8{0, 9, 18} +) + +func (i StepState) String() string { + switch { + case 1 <= i && i <= 2: + i -= 1 + return _StepState_name_0[_StepState_index_0[i]:_StepState_index_0[i+1]] + case i == 4: + return _StepState_name_1 + case i == 8: + return _StepState_name_2 + default: + return "StepState(" + strconv.FormatInt(int64(i), 10) + ")" + } +} diff --git a/cmd/replication/internal/mainfsm/diff.go b/cmd/replication/internal/diff/diff.go similarity index 100% rename from cmd/replication/internal/mainfsm/diff.go rename to cmd/replication/internal/diff/diff.go diff --git a/cmd/replication/internal/fsfsm/fsreplicationstate_string.go b/cmd/replication/internal/fsfsm/fsreplicationstate_string.go deleted file mode 100644 index 225fee9..0000000 --- a/cmd/replication/internal/fsfsm/fsreplicationstate_string.go +++ /dev/null @@ -1,29 +0,0 @@ -// Code generated by "stringer -type=FSReplicationState"; DO NOT EDIT. - -package fsfsm - -import "strconv" - -const ( - _FSReplicationState_name_0 = "FSReadyFSRetryWait" - _FSReplicationState_name_1 = "FSPermanentError" - _FSReplicationState_name_2 = "FSCompleted" -) - -var ( - _FSReplicationState_index_0 = [...]uint8{0, 7, 18} -) - -func (i FSReplicationState) String() string { - switch { - case 1 <= i && i <= 2: - i -= 1 - return _FSReplicationState_name_0[_FSReplicationState_index_0[i]:_FSReplicationState_index_0[i+1]] - case i == 4: - return _FSReplicationState_name_1 - case i == 8: - return _FSReplicationState_name_2 - default: - return "FSReplicationState(" + strconv.FormatInt(int64(i), 10) + ")" - } -} diff --git a/cmd/replication/internal/fsfsm/fsreplicationstepstate_string.go b/cmd/replication/internal/fsfsm/fsreplicationstepstate_string.go deleted file mode 100644 index 3d64be0..0000000 --- a/cmd/replication/internal/fsfsm/fsreplicationstepstate_string.go +++ /dev/null @@ -1,29 +0,0 @@ -// Code generated by "stringer -type=FSReplicationStepState"; DO NOT EDIT. - -package fsfsm - -import "strconv" - -const ( - _FSReplicationStepState_name_0 = "StepReadyStepRetry" - _FSReplicationStepState_name_1 = "StepPermanentError" - _FSReplicationStepState_name_2 = "StepCompleted" -) - -var ( - _FSReplicationStepState_index_0 = [...]uint8{0, 9, 18} -) - -func (i FSReplicationStepState) String() string { - switch { - case 1 <= i && i <= 2: - i -= 1 - return _FSReplicationStepState_name_0[_FSReplicationStepState_index_0[i]:_FSReplicationStepState_index_0[i+1]] - case i == 4: - return _FSReplicationStepState_name_1 - case i == 8: - return _FSReplicationStepState_name_2 - default: - return "FSReplicationStepState(" + strconv.FormatInt(int64(i), 10) + ")" - } -} diff --git a/cmd/replication/internal/mainfsm/replicationstate_string.go b/cmd/replication/internal/mainfsm/replicationstate_string.go deleted file mode 100644 index 46511af..0000000 --- a/cmd/replication/internal/mainfsm/replicationstate_string.go +++ /dev/null @@ -1,35 +0,0 @@ -// Code generated by "stringer -type=ReplicationState"; DO NOT EDIT. - -package mainfsm - -import "strconv" - -const ( - _ReplicationState_name_0 = "PlanningPlanningError" - _ReplicationState_name_1 = "Working" - _ReplicationState_name_2 = "WorkingWait" - _ReplicationState_name_3 = "Completed" - _ReplicationState_name_4 = "ContextDone" -) - -var ( - _ReplicationState_index_0 = [...]uint8{0, 8, 21} -) - -func (i ReplicationState) String() string { - switch { - case 1 <= i && i <= 2: - i -= 1 - return _ReplicationState_name_0[_ReplicationState_index_0[i]:_ReplicationState_index_0[i+1]] - case i == 4: - return _ReplicationState_name_1 - case i == 8: - return _ReplicationState_name_2 - case i == 16: - return _ReplicationState_name_3 - case i == 32: - return _ReplicationState_name_4 - default: - return "ReplicationState(" + strconv.FormatInt(int64(i), 10) + ")" - } -} diff --git a/cmd/replication/internal/mainfsm/queue/queue.go b/cmd/replication/internal/queue/queue.go similarity index 75% rename from cmd/replication/internal/mainfsm/queue/queue.go rename to cmd/replication/internal/queue/queue.go index e2b588d..6f7496c 100644 --- a/cmd/replication/internal/mainfsm/queue/queue.go +++ b/cmd/replication/internal/queue/queue.go @@ -4,17 +4,17 @@ import ( "time" "sort" - . "github.com/zrepl/zrepl/cmd/replication/internal/fsfsm" + . "github.com/zrepl/zrepl/cmd/replication/fsrep" ) type replicationQueueItem struct { retriesSinceLastError int // duplicates fsr.state to avoid accessing and locking fsr - state FSReplicationState + state State // duplicates fsr.current.nextStepDate to avoid accessing & locking fsr nextStepDate time.Time - fsr *FSReplication + fsr *Replication } type ReplicationQueue []*replicationQueueItem @@ -32,14 +32,14 @@ type lessmapEntry struct{ less func(a,b *replicationQueueItem) bool } -var lessmap = map[FSReplicationState]lessmapEntry { - FSReady: { +var lessmap = map[State]lessmapEntry { + Ready: { prio: 0, less: func(a, b *replicationQueueItem) bool { return a.nextStepDate.Before(b.nextStepDate) }, }, - FSRetryWait: { + RetryWait: { prio: 1, less: func(a, b *replicationQueueItem) bool { return a.retriesSinceLastError < b.retriesSinceLastError @@ -66,10 +66,10 @@ func (q ReplicationQueue) Less(i, j int) bool { return al.less(a, b) } -func (q *ReplicationQueue) sort() (done []*FSReplication) { +func (q *ReplicationQueue) sort() (done []*Replication) { // pre-scan for everything that is not ready newq := make(ReplicationQueue, 0, len(*q)) - done = make([]*FSReplication, 0, len(*q)) + done = make([]*Replication, 0, len(*q)) for _, qitem := range *q { if _, ok := lessmap[qitem.state]; !ok { done = append(done, qitem.fsr) @@ -83,7 +83,7 @@ func (q *ReplicationQueue) sort() (done []*FSReplication) { } // next remains valid until the next call to GetNext() -func (q *ReplicationQueue) GetNext() (done []*FSReplication, next *ReplicationQueueItemHandle) { +func (q *ReplicationQueue) GetNext() (done []*Replication, next *ReplicationQueueItemHandle) { done = q.sort() if len(*q) == 0 { return done, nil @@ -92,7 +92,7 @@ func (q *ReplicationQueue) GetNext() (done []*FSReplication, next *ReplicationQu return done, next } -func (q *ReplicationQueue) Add(fsr *FSReplication) { +func (q *ReplicationQueue) Add(fsr *Replication) { *q = append(*q, &replicationQueueItem{ fsr: fsr, state: fsr.State(), @@ -109,16 +109,16 @@ type ReplicationQueueItemHandle struct { i *replicationQueueItem } -func (h ReplicationQueueItemHandle) GetFSReplication() *FSReplication { +func (h ReplicationQueueItemHandle) GetFSReplication() *Replication { return h.i.fsr } -func (h ReplicationQueueItemHandle) Update(newState FSReplicationState, nextStepDate time.Time) { +func (h ReplicationQueueItemHandle) Update(newState State, nextStepDate time.Time) { h.i.state = newState h.i.nextStepDate = nextStepDate - if h.i.state&FSReady != 0 { + if h.i.state&Ready != 0 { h.i.retriesSinceLastError = 0 - } else if h.i.state&FSRetryWait != 0 { + } else if h.i.state&RetryWait != 0 { h.i.retriesSinceLastError++ } } diff --git a/cmd/replication/internal/mainfsm/mainfsm.go b/cmd/replication/mainfsm.go similarity index 65% rename from cmd/replication/internal/mainfsm/mainfsm.go rename to cmd/replication/mainfsm.go index 5d5f944..d9d6c83 100644 --- a/cmd/replication/internal/mainfsm/mainfsm.go +++ b/cmd/replication/mainfsm.go @@ -1,4 +1,6 @@ -package mainfsm +// Package replication implements replication of filesystems with existing +// versions (snapshots) from a sender to a receiver. +package replication import ( "context" @@ -8,17 +10,17 @@ import ( "sync" "time" - . "github.com/zrepl/zrepl/cmd/replication/common" "github.com/zrepl/zrepl/cmd/replication/pdu" - "github.com/zrepl/zrepl/cmd/replication/internal/fsfsm" - . "github.com/zrepl/zrepl/cmd/replication/internal/mainfsm/queue" + "github.com/zrepl/zrepl/cmd/replication/fsrep" + . "github.com/zrepl/zrepl/cmd/replication/internal/queue" + . "github.com/zrepl/zrepl/cmd/replication/internal/diff" ) -//go:generate stringer -type=ReplicationState -type ReplicationState uint +//go:generate stringer -type=State +type State uint const ( - Planning ReplicationState = 1 << iota + Planning State = 1 << iota PlanningError Working WorkingWait @@ -26,31 +28,35 @@ const ( ContextDone ) -func (s ReplicationState) rsf() replicationStateFunc { +func (s State) rsf() state { idx := bits.TrailingZeros(uint(s)) if idx == bits.UintSize { panic(s) // invalid value } - m := []replicationStateFunc{ - rsfPlanning, - rsfPlanningError, - rsfWorking, - rsfWorkingWait, + m := []state{ + statePlanning, + statePlanningError, + stateWorking, + stateWorkingWait, nil, nil, } return m[idx] } +// Replication implements the replication of multiple file systems from a Sender to a Receiver. +// +// It is a state machine that is driven by the Drive method +// and provides asynchronous reporting via the Report method (i.e. from another goroutine). type Replication struct { // lock protects all fields of this struct (but not the fields behind pointers!) lock sync.Mutex - state ReplicationState + state State // Working, WorkingWait, Completed, ContextDone queue *ReplicationQueue - completed []*fsfsm.FSReplication + completed []*fsrep.Replication active *ReplicationQueueItemHandle // PlanningError @@ -66,9 +72,9 @@ type Replication struct { type Report struct { Status string Problem string - Completed []*fsfsm.FilesystemReplicationReport - Pending []*fsfsm.FilesystemReplicationReport - Active *fsfsm.FilesystemReplicationReport + Completed []*fsrep.Report + Pending []*fsrep.Report + Active *fsrep.Report } @@ -79,12 +85,45 @@ func NewReplication() *Replication { return &r } -type replicationUpdater func(func(*Replication)) (newState ReplicationState) -type replicationStateFunc func(ctx context.Context, sender, receiver ReplicationEndpoint, u replicationUpdater) replicationStateFunc +type Endpoint interface { + // Does not include placeholder filesystems + ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) + ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) // fix depS +} -func (r *Replication) Drive(ctx context.Context, sender, receiver ReplicationEndpoint) { +type Sender interface { + Endpoint + fsrep.Sender +} - var u replicationUpdater = func(f func(*Replication)) ReplicationState { +type Receiver interface { + Endpoint + fsrep.Receiver +} + + +type FilteredError struct{ fs string } + +func NewFilteredError(fs string) *FilteredError { + return &FilteredError{fs} +} + +func (f FilteredError) Error() string { return "endpoint does not allow access to filesystem " + f.fs } + + +type updater func(func(*Replication)) (newState State) +type state func(ctx context.Context, sender Sender, receiver Receiver, u updater) state + +// Drive starts the state machine and returns only after replication has finished (with or without errors). +// The Logger in ctx is used for both debug and error logging, but is not guaranteed to be stable +// or end-user friendly. +// User-facing replication progress reports and can be obtained using the Report method, +// whose output will not change after Drive returns. +// +// FIXME: Drive may be only called once per instance of Replication +func (r *Replication) Drive(ctx context.Context, sender Sender, receiver Receiver) { + + var u updater = func(f func(*Replication)) State { r.lock.Lock() defer r.lock.Unlock() if f != nil { @@ -93,21 +132,21 @@ func (r *Replication) Drive(ctx context.Context, sender, receiver ReplicationEnd return r.state } - var s replicationStateFunc = rsfPlanning - var pre, post ReplicationState + var s state = statePlanning + var pre, post State for s != nil { preTime := time.Now() pre = u(nil) s = s(ctx, sender, receiver, u) delta := time.Now().Sub(preTime) post = u(nil) - GetLogger(ctx). + getLogger(ctx). WithField("transition", fmt.Sprintf("%s => %s", pre, post)). WithField("duration", delta). Debug("main state transition") } - GetLogger(ctx). + getLogger(ctx). WithField("final_state", post). Debug("main final state") } @@ -133,11 +172,11 @@ func resolveConflict(conflict error) (path []*pdu.FilesystemVersion, msg string) return nil, "no automated way to handle conflict type" } -func rsfPlanning(ctx context.Context, sender, receiver ReplicationEndpoint, u replicationUpdater) replicationStateFunc { +func statePlanning(ctx context.Context, sender Sender, receiver Receiver, u updater) state { - log := GetLogger(ctx) + log := getLogger(ctx) - handlePlanningError := func(err error) replicationStateFunc { + handlePlanningError := func(err error) state { return u(func(r *Replication) { r.planningError = err r.state = PlanningError @@ -173,7 +212,7 @@ func rsfPlanning(ctx context.Context, sender, receiver ReplicationEndpoint, u re if len(sfsvs) <= 1 { err := errors.New("sender does not have any versions") log.Error(err.Error()) - q.Add(fsfsm.NewFSReplicationWithPermanentError(fs.Path, err)) + q.Add(fsrep.NewReplicationWithPermanentError(fs.Path, err)) continue } @@ -212,11 +251,11 @@ func rsfPlanning(ctx context.Context, sender, receiver ReplicationEndpoint, u re } } if path == nil { - q.Add(fsfsm.NewFSReplicationWithPermanentError(fs.Path, conflict)) + q.Add(fsrep.NewReplicationWithPermanentError(fs.Path, conflict)) continue } - fsrfsm := fsfsm.BuildFSReplication(fs.Path) + fsrfsm := fsrep.BuildReplication(fs.Path) if len(path) == 1 { fsrfsm.AddStep(nil, path[0]) } else { @@ -236,7 +275,7 @@ func rsfPlanning(ctx context.Context, sender, receiver ReplicationEndpoint, u re }).rsf() } -func rsfPlanningError(ctx context.Context, sender, receiver ReplicationEndpoint, u replicationUpdater) replicationStateFunc { +func statePlanningError(ctx context.Context, sender Sender, receiver Receiver, u updater) state { sleepTime := 10 * time.Second u(func(r *Replication) { r.sleepUntil = time.Now().Add(sleepTime) @@ -256,7 +295,7 @@ func rsfPlanningError(ctx context.Context, sender, receiver ReplicationEndpoint, } } -func rsfWorking(ctx context.Context, sender, receiver ReplicationEndpoint, u replicationUpdater) replicationStateFunc { +func stateWorking(ctx context.Context, sender Sender, receiver Receiver, u updater) state { var active *ReplicationQueueItemHandle rsfNext := u(func(r *Replication) { @@ -281,7 +320,7 @@ func rsfWorking(ctx context.Context, sender, receiver ReplicationEndpoint, u rep }).rsf() } -func rsfWorkingWait(ctx context.Context, sender, receiver ReplicationEndpoint, u replicationUpdater) replicationStateFunc { +func stateWorkingWait(ctx context.Context, sender Sender, receiver Receiver, u updater) state { sleepTime := 10 * time.Second u(func(r *Replication) { r.sleepUntil = time.Now().Add(sleepTime) @@ -301,6 +340,9 @@ func rsfWorkingWait(ctx context.Context, sender, receiver ReplicationEndpoint, u } } +// Report provides a summary of the progress of the Replication, +// i.e., a condensed dump of the internal state machine. +// Report is safe to be called asynchronously while Drive is running. func (r *Replication) Report() *Report { r.lock.Lock() defer r.lock.Unlock() @@ -319,10 +361,10 @@ func (r *Replication) Report() *Report { return &rep } - rep.Pending = make([]*fsfsm.FilesystemReplicationReport, 0, r.queue.Len()) - rep.Completed = make([]*fsfsm.FilesystemReplicationReport, 0, len(r.completed)) // room for active (potentially) + rep.Pending = make([]*fsrep.Report, 0, r.queue.Len()) + rep.Completed = make([]*fsrep.Report, 0, len(r.completed)) // room for active (potentially) - var active *fsfsm.FSReplication + var active *fsrep.Replication if r.active != nil { active = r.active.GetFSReplication() rep.Active = active.Report() diff --git a/cmd/replication/replication.go b/cmd/replication/replication.go deleted file mode 100644 index 5d103a2..0000000 --- a/cmd/replication/replication.go +++ /dev/null @@ -1,19 +0,0 @@ -package replication - -import ( - "context" - - "github.com/zrepl/zrepl/cmd/replication/common" - "github.com/zrepl/zrepl/cmd/replication/internal/mainfsm" -) - -type Report = mainfsm.Report - -type Replication interface { - Drive(ctx context.Context, sender, receiver common.ReplicationEndpoint) - Report() *Report -} - -func NewReplication() Replication { - return mainfsm.NewReplication() -} diff --git a/cmd/replication/state_string.go b/cmd/replication/state_string.go new file mode 100644 index 0000000..a4514d2 --- /dev/null +++ b/cmd/replication/state_string.go @@ -0,0 +1,35 @@ +// Code generated by "stringer -type=State"; DO NOT EDIT. + +package replication + +import "strconv" + +const ( + _State_name_0 = "PlanningPlanningError" + _State_name_1 = "Working" + _State_name_2 = "WorkingWait" + _State_name_3 = "Completed" + _State_name_4 = "ContextDone" +) + +var ( + _State_index_0 = [...]uint8{0, 8, 21} +) + +func (i State) String() string { + switch { + case 1 <= i && i <= 2: + i -= 1 + return _State_name_0[_State_index_0[i]:_State_index_0[i+1]] + case i == 4: + return _State_name_1 + case i == 8: + return _State_name_2 + case i == 16: + return _State_name_3 + case i == 32: + return _State_name_4 + default: + return "State(" + strconv.FormatInt(int64(i), 10) + ")" + } +} From 7b3a84e2a396ea10f1aef1570857bf162119e22d Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 22 Aug 2018 00:19:03 +0200 Subject: [PATCH 019/167] move replication package to project root (independent of cmd package) --- cmd/config_job_local.go | 2 +- cmd/config_job_pull.go | 2 +- cmd/replication.go | 4 ++-- {cmd/replication => replication}/context.go | 2 +- {cmd/replication => replication}/fsrep/fsfsm.go | 2 +- {cmd/replication => replication}/fsrep/state_string.go | 0 .../replication => replication}/fsrep/stepstate_string.go | 0 {cmd/replication => replication}/internal/diff/diff.go | 2 +- {cmd/replication => replication}/internal/queue/queue.go | 2 +- {cmd/replication => replication}/mainfsm.go | 8 ++++---- {cmd/replication => replication}/pdu/pdu.pb.go | 0 {cmd/replication => replication}/pdu/pdu.proto | 0 {cmd/replication => replication}/pdu/pdu_extras.go | 0 {cmd/replication => replication}/pdu/pdu_test.go | 0 {cmd/replication => replication}/state_string.go | 0 15 files changed, 12 insertions(+), 12 deletions(-) rename {cmd/replication => replication}/context.go (90%) rename {cmd/replication => replication}/fsrep/fsfsm.go (99%) rename {cmd/replication => replication}/fsrep/state_string.go (100%) rename {cmd/replication => replication}/fsrep/stepstate_string.go (100%) rename {cmd/replication => replication}/internal/diff/diff.go (98%) rename {cmd/replication => replication}/internal/queue/queue.go (98%) rename {cmd/replication => replication}/mainfsm.go (97%) rename {cmd/replication => replication}/pdu/pdu.pb.go (100%) rename {cmd/replication => replication}/pdu/pdu.proto (100%) rename {cmd/replication => replication}/pdu/pdu_extras.go (100%) rename {cmd/replication => replication}/pdu/pdu_test.go (100%) rename {cmd/replication => replication}/state_string.go (100%) diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index 21fd6a6..4755beb 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -8,7 +8,7 @@ import ( "github.com/pkg/errors" "github.com/zrepl/zrepl/zfs" "sync" - "github.com/zrepl/zrepl/cmd/replication" + "github.com/zrepl/zrepl/replication" ) type LocalJob struct { diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index fe65d4f..a54fd8d 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -13,7 +13,7 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/cmd/replication" + "github.com/zrepl/zrepl/replication" ) type PullJob struct { diff --git a/cmd/replication.go b/cmd/replication.go index 0e1cc8a..11c6abe 100644 --- a/cmd/replication.go +++ b/cmd/replication.go @@ -2,7 +2,7 @@ package cmd import ( "fmt" - "github.com/zrepl/zrepl/cmd/replication/pdu" + "github.com/zrepl/zrepl/replication/pdu" "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/zfs" "io" @@ -10,7 +10,7 @@ import ( "github.com/golang/protobuf/proto" "bytes" "context" - "github.com/zrepl/zrepl/cmd/replication" + "github.com/zrepl/zrepl/replication" ) type InitialReplPolicy string diff --git a/cmd/replication/context.go b/replication/context.go similarity index 90% rename from cmd/replication/context.go rename to replication/context.go index 775ee24..7b4b0e9 100644 --- a/cmd/replication/context.go +++ b/replication/context.go @@ -3,7 +3,7 @@ package replication import ( "github.com/zrepl/zrepl/logger" "context" - "github.com/zrepl/zrepl/cmd/replication/fsrep" + "github.com/zrepl/zrepl/replication/fsrep" ) type contextKey int diff --git a/cmd/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go similarity index 99% rename from cmd/replication/fsrep/fsfsm.go rename to replication/fsrep/fsfsm.go index d70b84b..d07fd3d 100644 --- a/cmd/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -12,7 +12,7 @@ import ( "sync" "time" - "github.com/zrepl/zrepl/cmd/replication/pdu" + "github.com/zrepl/zrepl/replication/pdu" "github.com/zrepl/zrepl/logger" ) diff --git a/cmd/replication/fsrep/state_string.go b/replication/fsrep/state_string.go similarity index 100% rename from cmd/replication/fsrep/state_string.go rename to replication/fsrep/state_string.go diff --git a/cmd/replication/fsrep/stepstate_string.go b/replication/fsrep/stepstate_string.go similarity index 100% rename from cmd/replication/fsrep/stepstate_string.go rename to replication/fsrep/stepstate_string.go diff --git a/cmd/replication/internal/diff/diff.go b/replication/internal/diff/diff.go similarity index 98% rename from cmd/replication/internal/diff/diff.go rename to replication/internal/diff/diff.go index f1f4af6..fed2392 100644 --- a/cmd/replication/internal/diff/diff.go +++ b/replication/internal/diff/diff.go @@ -3,7 +3,7 @@ package mainfsm import ( "sort" - . "github.com/zrepl/zrepl/cmd/replication/pdu" + . "github.com/zrepl/zrepl/replication/pdu" ) type ConflictNoCommonAncestor struct { diff --git a/cmd/replication/internal/queue/queue.go b/replication/internal/queue/queue.go similarity index 98% rename from cmd/replication/internal/queue/queue.go rename to replication/internal/queue/queue.go index 6f7496c..a187026 100644 --- a/cmd/replication/internal/queue/queue.go +++ b/replication/internal/queue/queue.go @@ -4,7 +4,7 @@ import ( "time" "sort" - . "github.com/zrepl/zrepl/cmd/replication/fsrep" + . "github.com/zrepl/zrepl/replication/fsrep" ) type replicationQueueItem struct { diff --git a/cmd/replication/mainfsm.go b/replication/mainfsm.go similarity index 97% rename from cmd/replication/mainfsm.go rename to replication/mainfsm.go index d9d6c83..ea1308f 100644 --- a/cmd/replication/mainfsm.go +++ b/replication/mainfsm.go @@ -10,10 +10,10 @@ import ( "sync" "time" - "github.com/zrepl/zrepl/cmd/replication/pdu" - "github.com/zrepl/zrepl/cmd/replication/fsrep" - . "github.com/zrepl/zrepl/cmd/replication/internal/queue" - . "github.com/zrepl/zrepl/cmd/replication/internal/diff" + "github.com/zrepl/zrepl/replication/pdu" + "github.com/zrepl/zrepl/replication/fsrep" + . "github.com/zrepl/zrepl/replication/internal/queue" + . "github.com/zrepl/zrepl/replication/internal/diff" ) //go:generate stringer -type=State diff --git a/cmd/replication/pdu/pdu.pb.go b/replication/pdu/pdu.pb.go similarity index 100% rename from cmd/replication/pdu/pdu.pb.go rename to replication/pdu/pdu.pb.go diff --git a/cmd/replication/pdu/pdu.proto b/replication/pdu/pdu.proto similarity index 100% rename from cmd/replication/pdu/pdu.proto rename to replication/pdu/pdu.proto diff --git a/cmd/replication/pdu/pdu_extras.go b/replication/pdu/pdu_extras.go similarity index 100% rename from cmd/replication/pdu/pdu_extras.go rename to replication/pdu/pdu_extras.go diff --git a/cmd/replication/pdu/pdu_test.go b/replication/pdu/pdu_test.go similarity index 100% rename from cmd/replication/pdu/pdu_test.go rename to replication/pdu/pdu_test.go diff --git a/cmd/replication/state_string.go b/replication/state_string.go similarity index 100% rename from cmd/replication/state_string.go rename to replication/state_string.go From 70aad0940f37e611c20165a0d4c300e6e209d304 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 22 Aug 2018 00:43:58 +0200 Subject: [PATCH 020/167] cmd: move replication endpoints into subpackage --- cmd/config_job_local.go | 9 ++++--- cmd/config_job_pull.go | 11 +++++---- cmd/config_job_source.go | 5 ++-- cmd/config_mapfilter.go | 12 ++++----- cmd/config_parse.go | 7 +++--- cmd/endpoint/context.go | 26 ++++++++++++++++++++ cmd/{replication.go => endpoint/endpoint.go} | 25 ++++++++++++++++--- 7 files changed, 71 insertions(+), 24 deletions(-) create mode 100644 cmd/endpoint/context.go rename cmd/{replication.go => endpoint/endpoint.go} (94%) diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index 4755beb..4cb7091 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -9,6 +9,7 @@ import ( "github.com/zrepl/zrepl/zfs" "sync" "github.com/zrepl/zrepl/replication" + "github.com/zrepl/zrepl/cmd/endpoint" ) type LocalJob struct { @@ -16,7 +17,7 @@ type LocalJob struct { Mapping *DatasetMapFilter SnapshotPrefix string Interval time.Duration - InitialReplPolicy InitialReplPolicy + InitialReplPolicy endpoint.InitialReplPolicy PruneLHS PrunePolicy PruneRHS PrunePolicy Debug JobDebugSettings @@ -59,7 +60,7 @@ func parseLocalJob(c JobParsingContext, name string, i map[string]interface{}) ( return } - if j.InitialReplPolicy, err = parseInitialReplPolicy(asMap.InitialReplPolicy, DEFAULT_INITIAL_REPL_POLICY); err != nil { + if j.InitialReplPolicy, err = parseInitialReplPolicy(asMap.InitialReplPolicy, endpoint.DEFAULT_INITIAL_REPL_POLICY); err != nil { return } @@ -103,9 +104,9 @@ func (j *LocalJob) JobStart(ctx context.Context) { // We can pay this small performance penalty for now. wildcardMapFilter := NewDatasetMapFilter(1, false) wildcardMapFilter.Add("<", "<") - sender := &SenderEndpoint{wildcardMapFilter, NewPrefixFilter(j.SnapshotPrefix)} + sender := &endpoint.SenderEndpoint{wildcardMapFilter, NewPrefixFilter(j.SnapshotPrefix)} - receiver, err := NewReceiverEndpoint(j.Mapping, NewPrefixFilter(j.SnapshotPrefix)) + receiver, err := endpoint.NewReceiverEndpoint(j.Mapping, NewPrefixFilter(j.SnapshotPrefix)) if err != nil { rootLog.WithError(err).Error("unexpected error setting up local handler") } diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index a54fd8d..6fda29e 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -14,6 +14,7 @@ import ( "github.com/pkg/errors" "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/replication" + "github.com/zrepl/zrepl/cmd/endpoint" ) type PullJob struct { @@ -24,7 +25,7 @@ type PullJob struct { // constructed from mapping during parsing pruneFilter *DatasetMapFilter SnapshotPrefix string - InitialReplPolicy InitialReplPolicy + InitialReplPolicy endpoint.InitialReplPolicy Prune PrunePolicy Debug JobDebugSettings @@ -73,7 +74,7 @@ func parsePullJob(c JobParsingContext, name string, i map[string]interface{}) (j return nil, err } - j.InitialReplPolicy, err = parseInitialReplPolicy(asMap.InitialReplPolicy, DEFAULT_INITIAL_REPL_POLICY) + j.InitialReplPolicy, err = parseInitialReplPolicy(asMap.InitialReplPolicy, endpoint.DEFAULT_INITIAL_REPL_POLICY) if err != nil { err = errors.Wrap(err, "cannot parse 'initial_repl_policy'") return @@ -175,9 +176,9 @@ func (j *PullJob) doRun(ctx context.Context) { j.task.Enter("pull") - sender := RemoteEndpoint{client} + sender := endpoint.RemoteEndpoint{client} - puller, err := NewReceiverEndpoint( + puller, err := endpoint.NewReceiverEndpoint( j.Mapping, NewPrefixFilter(j.SnapshotPrefix), ) @@ -229,7 +230,7 @@ func (j *PullJob) Pruner(task *Task, side PrunePolicySide, dryRun bool) (p Prune return } -func closeRPCWithTimeout(task *Task, remote RemoteEndpoint, timeout time.Duration, goodbye string) { +func closeRPCWithTimeout(task *Task, remote endpoint.RemoteEndpoint, timeout time.Duration, goodbye string) { task.Log().Info("closing rpc connection") diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index a3ff349..5a4405a 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -8,6 +8,7 @@ import ( "github.com/pkg/errors" "github.com/problame/go-streamrpc" "net" + "github.com/zrepl/zrepl/cmd/endpoint" ) type SourceJob struct { @@ -211,12 +212,12 @@ func (j *SourceJob) handleConnection(conn net.Conn, task *Task) { task.Log().Info("handling client connection") - senderEP := NewSenderEndpoint(j.Filesystems, NewPrefixFilter(j.SnapshotPrefix)) + senderEP := endpoint.NewSenderEndpoint(j.Filesystems, NewPrefixFilter(j.SnapshotPrefix)) ctx := context.Background() ctx = context.WithValue(ctx, contextKeyLog, task.Log().WithField("subsystem", "rpc.endpoint")) ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{task.Log().WithField("subsystem", "rpc.protocol")}) - handler := HandlerAdaptor{senderEP} + handler := endpoint.NewHandlerAdaptor(senderEP) if err := streamrpc.ServeConn(ctx, conn, STREAMRPC_CONFIG, handler.Handle); err != nil { task.Log().WithError(err).Error("error serving connection") } else { diff --git a/cmd/config_mapfilter.go b/cmd/config_mapfilter.go index 04d7076..e59f482 100644 --- a/cmd/config_mapfilter.go +++ b/cmd/config_mapfilter.go @@ -7,6 +7,7 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/zrepl/zrepl/zfs" + "github.com/zrepl/zrepl/cmd/endpoint" ) type DatasetMapFilter struct { @@ -187,11 +188,10 @@ func (m DatasetMapFilter) InvertedFilter() (inv *DatasetMapFilter, err error) { } // FIXME investigate whether we can support more... -func (m DatasetMapFilter) Invert() (inv *DatasetMapFilter, err error) { +func (m DatasetMapFilter) Invert() (endpoint.FSMap, error) { if m.filterMode { - err = errors.Errorf("can only invert mappings") - return + return nil, errors.Errorf("can only invert mappings") } if len(m.entries) != 1 { @@ -200,7 +200,7 @@ func (m DatasetMapFilter) Invert() (inv *DatasetMapFilter, err error) { e := m.entries[0] - inv = &DatasetMapFilter{ + inv := &DatasetMapFilter{ make([]datasetMapFilterEntry, len(m.entries)), false, } @@ -221,9 +221,9 @@ func (m DatasetMapFilter) Invert() (inv *DatasetMapFilter, err error) { // Creates a new DatasetMapFilter in filter mode from a mapping // All accepting mapping results are mapped to accepting filter results // All rejecting mapping results are mapped to rejecting filter results -func (m DatasetMapFilter) AsFilter() (f *DatasetMapFilter) { +func (m DatasetMapFilter) AsFilter() endpoint.FSFilter { - f = &DatasetMapFilter{ + f := &DatasetMapFilter{ make([]datasetMapFilterEntry, len(m.entries)), true, } diff --git a/cmd/config_parse.go b/cmd/config_parse.go index 6f6784c..eef40ec 100644 --- a/cmd/config_parse.go +++ b/cmd/config_parse.go @@ -12,6 +12,7 @@ import ( "strconv" "time" "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/cmd/endpoint" ) var ConfigFileDefaultLocations []string = []string{ @@ -225,7 +226,7 @@ func parseConnect(i map[string]interface{}) (c streamrpc.Connecter, err error) { } -func parseInitialReplPolicy(v interface{}, defaultPolicy InitialReplPolicy) (p InitialReplPolicy, err error) { +func parseInitialReplPolicy(v interface{}, defaultPolicy endpoint.InitialReplPolicy) (p endpoint.InitialReplPolicy, err error) { s, ok := v.(string) if !ok { goto err @@ -235,9 +236,9 @@ func parseInitialReplPolicy(v interface{}, defaultPolicy InitialReplPolicy) (p I case s == "": p = defaultPolicy case s == "most_recent": - p = InitialReplPolicyMostRecent + p = endpoint.InitialReplPolicyMostRecent case s == "all": - p = InitialReplPolicyAll + p = endpoint.InitialReplPolicyAll default: goto err } diff --git a/cmd/endpoint/context.go b/cmd/endpoint/context.go new file mode 100644 index 0000000..a528930 --- /dev/null +++ b/cmd/endpoint/context.go @@ -0,0 +1,26 @@ +package endpoint + +import ( + "context" + "github.com/zrepl/zrepl/logger" +) + +type contextKey int + +const ( + contextKeyLogger contextKey = iota +) + +type Logger = logger.Logger + +func WithLogger(ctx context.Context, log Logger) context.Context { + return context.WithValue(ctx, contextKeyLogger, log) +} + +func getLogger(ctx context.Context) Logger { + l, ok := ctx.Value(contextKeyLogger).(Logger) + if !ok { + l = logger.NewNullLogger() + } + return l +} diff --git a/cmd/replication.go b/cmd/endpoint/endpoint.go similarity index 94% rename from cmd/replication.go rename to cmd/endpoint/endpoint.go index 11c6abe..9ecbf36 100644 --- a/cmd/replication.go +++ b/cmd/endpoint/endpoint.go @@ -1,4 +1,4 @@ -package cmd +package endpoint import ( "fmt" @@ -13,6 +13,7 @@ import ( "github.com/zrepl/zrepl/replication" ) +// FIXME: remove this type InitialReplPolicy string const ( @@ -20,6 +21,7 @@ const ( InitialReplPolicyAll InitialReplPolicy = "all" ) +// FIXME: remove this const DEFAULT_INITIAL_REPL_POLICY = InitialReplPolicyMostRecent // SenderEndpoint implements replication.ReplicationEndpoint for a sending side @@ -93,15 +95,26 @@ func (p *SenderEndpoint) Receive(ctx context.Context, r *pdu.ReceiveReq, sendStr return fmt.Errorf("sender endpoint does not receive") } +type FSFilter interface { + Filter(path *zfs.DatasetPath) (pass bool, err error) +} + +// FIXME: can we get away without error types here? +type FSMap interface { + FSFilter + Map(path *zfs.DatasetPath) (*zfs.DatasetPath,error) + Invert() (FSMap,error) + AsFilter() (FSFilter) +} // ReceiverEndpoint implements replication.ReplicationEndpoint for a receiving side type ReceiverEndpoint struct { - fsmapInv *DatasetMapFilter - fsmap *DatasetMapFilter + fsmapInv FSMap + fsmap FSMap fsvf zfs.FilesystemVersionFilter } -func NewReceiverEndpoint(fsmap *DatasetMapFilter, fsvf zfs.FilesystemVersionFilter) (*ReceiverEndpoint, error) { +func NewReceiverEndpoint(fsmap FSMap, fsvf zfs.FilesystemVersionFilter) (*ReceiverEndpoint, error) { fsmapInv, err := fsmap.Invert() if err != nil { return nil, err @@ -327,6 +340,10 @@ type HandlerAdaptor struct { ep replication.Endpoint } +func NewHandlerAdaptor(ep replication.Endpoint) HandlerAdaptor { + return HandlerAdaptor{ep} +} + func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructured *bytes.Buffer, reqStream io.ReadCloser) (resStructured *bytes.Buffer, resStream io.ReadCloser, err error) { switch endpoint { From 9b537ec704229f72c186775dcd14e4855407188e Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 22 Aug 2018 00:52:46 +0200 Subject: [PATCH 021/167] simplify naming in endpoint package --- cmd/config_job_local.go | 4 +-- cmd/config_job_pull.go | 33 ++---------------- cmd/config_job_source.go | 4 +-- cmd/endpoint/endpoint.go | 72 +++++++++++++++++++--------------------- 4 files changed, 42 insertions(+), 71 deletions(-) diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index 4cb7091..c168c18 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -104,9 +104,9 @@ func (j *LocalJob) JobStart(ctx context.Context) { // We can pay this small performance penalty for now. wildcardMapFilter := NewDatasetMapFilter(1, false) wildcardMapFilter.Add("<", "<") - sender := &endpoint.SenderEndpoint{wildcardMapFilter, NewPrefixFilter(j.SnapshotPrefix)} + sender := &endpoint.Sender{wildcardMapFilter, NewPrefixFilter(j.SnapshotPrefix)} - receiver, err := endpoint.NewReceiverEndpoint(j.Mapping, NewPrefixFilter(j.SnapshotPrefix)) + receiver, err := endpoint.NewReceiver(j.Mapping, NewPrefixFilter(j.SnapshotPrefix)) if err != nil { rootLog.WithError(err).Error("unexpected error setting up local handler") } diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index 6fda29e..255e47c 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -8,7 +8,6 @@ import ( "time" "context" - "fmt" "github.com/mitchellh/mapstructure" "github.com/pkg/errors" @@ -176,9 +175,9 @@ func (j *PullJob) doRun(ctx context.Context) { j.task.Enter("pull") - sender := endpoint.RemoteEndpoint{client} + sender := endpoint.NewRemote(client) - puller, err := endpoint.NewReceiverEndpoint( + puller, err := endpoint.NewReceiver( j.Mapping, NewPrefixFilter(j.SnapshotPrefix), ) @@ -188,10 +187,9 @@ func (j *PullJob) doRun(ctx context.Context) { return } - ctx = replication.WithLogger(ctx, replicationLogAdaptor{j.task.Log().WithField("subsystem", "replication")}) ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{j.task.Log().WithField("subsystem", "rpc.protocol")}) - ctx = context.WithValue(ctx, contextKeyLog, j.task.Log().WithField("subsystem", "rpc.endpoint")) + ctx = endpoint.WithLogger(ctx, j.task.Log().WithField("subsystem", "rpc.endpoint")) j.rep = replication.NewReplication() j.rep.Drive(ctx, sender, puller) @@ -229,28 +227,3 @@ func (j *PullJob) Pruner(task *Task, side PrunePolicySide, dryRun bool) (p Prune } return } - -func closeRPCWithTimeout(task *Task, remote endpoint.RemoteEndpoint, timeout time.Duration, goodbye string) { - - task.Log().Info("closing rpc connection") - - ch := make(chan error) - go func() { - remote.Close() - ch <- nil - close(ch) - }() - - var err error - select { - case <-time.After(timeout): - err = fmt.Errorf("timeout exceeded (%s)", timeout) - case closeRequestErr := <-ch: - err = closeRequestErr - } - - if err != nil { - task.Log().WithError(err).Error("error closing connection") - } - return -} diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index 5a4405a..b91179d 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -212,12 +212,12 @@ func (j *SourceJob) handleConnection(conn net.Conn, task *Task) { task.Log().Info("handling client connection") - senderEP := endpoint.NewSenderEndpoint(j.Filesystems, NewPrefixFilter(j.SnapshotPrefix)) + senderEP := endpoint.NewSender(j.Filesystems, NewPrefixFilter(j.SnapshotPrefix)) ctx := context.Background() ctx = context.WithValue(ctx, contextKeyLog, task.Log().WithField("subsystem", "rpc.endpoint")) ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{task.Log().WithField("subsystem", "rpc.protocol")}) - handler := endpoint.NewHandlerAdaptor(senderEP) + handler := endpoint.NewHandler(senderEP) if err := streamrpc.ServeConn(ctx, conn, STREAMRPC_CONFIG, handler.Handle); err != nil { task.Log().WithError(err).Error("error serving connection") } else { diff --git a/cmd/endpoint/endpoint.go b/cmd/endpoint/endpoint.go index 9ecbf36..9f31412 100644 --- a/cmd/endpoint/endpoint.go +++ b/cmd/endpoint/endpoint.go @@ -1,7 +1,7 @@ +// Package endpoint implements replication endpoints for use with package replication. package endpoint import ( - "fmt" "github.com/zrepl/zrepl/replication/pdu" "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/zfs" @@ -24,17 +24,17 @@ const ( // FIXME: remove this const DEFAULT_INITIAL_REPL_POLICY = InitialReplPolicyMostRecent -// SenderEndpoint implements replication.ReplicationEndpoint for a sending side -type SenderEndpoint struct { +// Sender implements replication.ReplicationEndpoint for a sending side +type Sender struct { FSFilter zfs.DatasetFilter FilesystemVersionFilter zfs.FilesystemVersionFilter } -func NewSenderEndpoint(fsf zfs.DatasetFilter, fsvf zfs.FilesystemVersionFilter) *SenderEndpoint { - return &SenderEndpoint{fsf, fsvf} +func NewSender(fsf zfs.DatasetFilter, fsvf zfs.FilesystemVersionFilter) *Sender { + return &Sender{fsf, fsvf} } -func (p *SenderEndpoint) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) { +func (p *Sender) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) { fss, err := zfs.ZFSListMapping(p.FSFilter) if err != nil { return nil, err @@ -49,7 +49,7 @@ func (p *SenderEndpoint) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem return rfss, nil } -func (p *SenderEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) { +func (p *Sender) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) { dp, err := zfs.NewDatasetPath(fs) if err != nil { return nil, err @@ -72,7 +72,7 @@ func (p *SenderEndpoint) ListFilesystemVersions(ctx context.Context, fs string) return rfsvs, nil } -func (p *SenderEndpoint) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) { +func (p *Sender) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) { dp, err := zfs.NewDatasetPath(r.Filesystem) if err != nil { return nil, nil, err @@ -91,10 +91,6 @@ func (p *SenderEndpoint) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes return &pdu.SendRes{}, stream, nil } -func (p *SenderEndpoint) Receive(ctx context.Context, r *pdu.ReceiveReq, sendStream io.ReadCloser) (error) { - return fmt.Errorf("sender endpoint does not receive") -} - type FSFilter interface { Filter(path *zfs.DatasetPath) (pass bool, err error) } @@ -107,22 +103,22 @@ type FSMap interface { AsFilter() (FSFilter) } -// ReceiverEndpoint implements replication.ReplicationEndpoint for a receiving side -type ReceiverEndpoint struct { +// Receiver implements replication.ReplicationEndpoint for a receiving side +type Receiver struct { fsmapInv FSMap fsmap FSMap fsvf zfs.FilesystemVersionFilter } -func NewReceiverEndpoint(fsmap FSMap, fsvf zfs.FilesystemVersionFilter) (*ReceiverEndpoint, error) { +func NewReceiver(fsmap FSMap, fsvf zfs.FilesystemVersionFilter) (*Receiver, error) { fsmapInv, err := fsmap.Invert() if err != nil { return nil, err } - return &ReceiverEndpoint{fsmapInv, fsmap, fsvf}, nil + return &Receiver{fsmapInv, fsmap, fsvf}, nil } -func (e *ReceiverEndpoint) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) { +func (e *Receiver) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) { filtered, err := zfs.ZFSListMapping(e.fsmapInv.AsFilter()) if err != nil { return nil, errors.Wrap(err, "error checking client permission") @@ -138,7 +134,7 @@ func (e *ReceiverEndpoint) ListFilesystems(ctx context.Context) ([]*pdu.Filesyst return fss, nil } -func (e *ReceiverEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) { +func (e *Receiver) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) { p, err := zfs.NewDatasetPath(fs) if err != nil { return nil, err @@ -164,11 +160,7 @@ func (e *ReceiverEndpoint) ListFilesystemVersions(ctx context.Context, fs string return rfsvs, nil } -func (e *ReceiverEndpoint) Send(ctx context.Context, req *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) { - return nil, nil, errors.New("receiver endpoint does not send") -} - -func (e *ReceiverEndpoint) Receive(ctx context.Context, req *pdu.ReceiveReq, sendStream io.ReadCloser) error { +func (e *Receiver) Receive(ctx context.Context, req *pdu.ReceiveReq, sendStream io.ReadCloser) error { defer sendStream.Close() p, err := zfs.NewDatasetPath(req.Filesystem) @@ -246,17 +238,22 @@ const ( RPCSend = "Send" ) -type RemoteEndpoint struct { - *streamrpc.Client +// Remote implements an endpoint stub that uses streamrpc as a transport. +type Remote struct { + c *streamrpc.Client } -func (s RemoteEndpoint) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) { +func NewRemote(c *streamrpc.Client) Remote { + return Remote{c} +} + +func (s Remote) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) { req := pdu.ListFilesystemReq{} b, err := proto.Marshal(&req) if err != nil { return nil, err } - rb, rs, err := s.RequestReply(ctx, RPCListFilesystems, bytes.NewBuffer(b), nil) + rb, rs, err := s.c.RequestReply(ctx, RPCListFilesystems, bytes.NewBuffer(b), nil) if err != nil { return nil, err } @@ -271,7 +268,7 @@ func (s RemoteEndpoint) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, return res.Filesystems, nil } -func (s RemoteEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) { +func (s Remote) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) { req := pdu.ListFilesystemVersionsReq{ Filesystem: fs, } @@ -279,7 +276,7 @@ func (s RemoteEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ( if err != nil { return nil, err } - rb, rs, err := s.RequestReply(ctx, RPCListFilesystemVersions, bytes.NewBuffer(b), nil) + rb, rs, err := s.c.RequestReply(ctx, RPCListFilesystemVersions, bytes.NewBuffer(b), nil) if err != nil { return nil, err } @@ -294,12 +291,12 @@ func (s RemoteEndpoint) ListFilesystemVersions(ctx context.Context, fs string) ( return res.Versions, nil } -func (s RemoteEndpoint) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) { +func (s Remote) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) { b, err := proto.Marshal(r) if err != nil { return nil, nil, err } - rb, rs, err := s.RequestReply(ctx, RPCSend, bytes.NewBuffer(b), nil) + rb, rs, err := s.c.RequestReply(ctx, RPCSend, bytes.NewBuffer(b), nil) if err != nil { return nil, nil, err } @@ -315,13 +312,13 @@ func (s RemoteEndpoint) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, return &res, rs, nil } -func (s RemoteEndpoint) Receive(ctx context.Context, r *pdu.ReceiveReq, sendStream io.ReadCloser) (error) { +func (s Remote) Receive(ctx context.Context, r *pdu.ReceiveReq, sendStream io.ReadCloser) (error) { defer sendStream.Close() b, err := proto.Marshal(r) if err != nil { return err } - rb, rs, err := s.RequestReply(ctx, RPCReceive, bytes.NewBuffer(b), sendStream) + rb, rs, err := s.c.RequestReply(ctx, RPCReceive, bytes.NewBuffer(b), sendStream) if err != nil { return err } @@ -336,15 +333,16 @@ func (s RemoteEndpoint) Receive(ctx context.Context, r *pdu.ReceiveReq, sendStre return nil } -type HandlerAdaptor struct { +// Handler implements the server-side streamrpc.HandlerFunc for a Remote endpoint stub. +type Handler struct { ep replication.Endpoint } -func NewHandlerAdaptor(ep replication.Endpoint) HandlerAdaptor { - return HandlerAdaptor{ep} +func NewHandler(ep replication.Endpoint) Handler { + return Handler{ep} } -func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructured *bytes.Buffer, reqStream io.ReadCloser) (resStructured *bytes.Buffer, resStream io.ReadCloser, err error) { +func (a *Handler) Handle(ctx context.Context, endpoint string, reqStructured *bytes.Buffer, reqStream io.ReadCloser) (resStructured *bytes.Buffer, resStream io.ReadCloser, err error) { switch endpoint { case RPCListFilesystems: From 54c9dcb7c1fed440a58dbe36f79b26dcb2afd4c6 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 22 Aug 2018 10:11:14 +0200 Subject: [PATCH 022/167] move replication policy constants to package replication --- cmd/config_job_local.go | 4 ++-- cmd/config_job_pull.go | 4 ++-- cmd/config_parse.go | 8 ++++---- cmd/endpoint/endpoint.go | 11 ----------- replication/policy.go | 12 ++++++++++++ 5 files changed, 20 insertions(+), 19 deletions(-) create mode 100644 replication/policy.go diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index c168c18..fafac4d 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -17,7 +17,7 @@ type LocalJob struct { Mapping *DatasetMapFilter SnapshotPrefix string Interval time.Duration - InitialReplPolicy endpoint.InitialReplPolicy + InitialReplPolicy replication.InitialReplPolicy PruneLHS PrunePolicy PruneRHS PrunePolicy Debug JobDebugSettings @@ -60,7 +60,7 @@ func parseLocalJob(c JobParsingContext, name string, i map[string]interface{}) ( return } - if j.InitialReplPolicy, err = parseInitialReplPolicy(asMap.InitialReplPolicy, endpoint.DEFAULT_INITIAL_REPL_POLICY); err != nil { + if j.InitialReplPolicy, err = parseInitialReplPolicy(asMap.InitialReplPolicy, replication.DEFAULT_INITIAL_REPL_POLICY); err != nil { return } diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index 255e47c..48ce8de 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -24,7 +24,7 @@ type PullJob struct { // constructed from mapping during parsing pruneFilter *DatasetMapFilter SnapshotPrefix string - InitialReplPolicy endpoint.InitialReplPolicy + InitialReplPolicy replication.InitialReplPolicy Prune PrunePolicy Debug JobDebugSettings @@ -73,7 +73,7 @@ func parsePullJob(c JobParsingContext, name string, i map[string]interface{}) (j return nil, err } - j.InitialReplPolicy, err = parseInitialReplPolicy(asMap.InitialReplPolicy, endpoint.DEFAULT_INITIAL_REPL_POLICY) + j.InitialReplPolicy, err = parseInitialReplPolicy(asMap.InitialReplPolicy, replication.DEFAULT_INITIAL_REPL_POLICY) if err != nil { err = errors.Wrap(err, "cannot parse 'initial_repl_policy'") return diff --git a/cmd/config_parse.go b/cmd/config_parse.go index eef40ec..1a773c6 100644 --- a/cmd/config_parse.go +++ b/cmd/config_parse.go @@ -11,8 +11,8 @@ import ( "regexp" "strconv" "time" + "github.com/zrepl/zrepl/replication" "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/cmd/endpoint" ) var ConfigFileDefaultLocations []string = []string{ @@ -226,7 +226,7 @@ func parseConnect(i map[string]interface{}) (c streamrpc.Connecter, err error) { } -func parseInitialReplPolicy(v interface{}, defaultPolicy endpoint.InitialReplPolicy) (p endpoint.InitialReplPolicy, err error) { +func parseInitialReplPolicy(v interface{}, defaultPolicy replication.InitialReplPolicy) (p replication.InitialReplPolicy, err error) { s, ok := v.(string) if !ok { goto err @@ -236,9 +236,9 @@ func parseInitialReplPolicy(v interface{}, defaultPolicy endpoint.InitialReplPol case s == "": p = defaultPolicy case s == "most_recent": - p = endpoint.InitialReplPolicyMostRecent + p = replication.InitialReplPolicyMostRecent case s == "all": - p = endpoint.InitialReplPolicyAll + p = replication.InitialReplPolicyAll default: goto err } diff --git a/cmd/endpoint/endpoint.go b/cmd/endpoint/endpoint.go index 9f31412..927cdd6 100644 --- a/cmd/endpoint/endpoint.go +++ b/cmd/endpoint/endpoint.go @@ -13,17 +13,6 @@ import ( "github.com/zrepl/zrepl/replication" ) -// FIXME: remove this -type InitialReplPolicy string - -const ( - InitialReplPolicyMostRecent InitialReplPolicy = "most_recent" - InitialReplPolicyAll InitialReplPolicy = "all" -) - -// FIXME: remove this -const DEFAULT_INITIAL_REPL_POLICY = InitialReplPolicyMostRecent - // Sender implements replication.ReplicationEndpoint for a sending side type Sender struct { FSFilter zfs.DatasetFilter diff --git a/replication/policy.go b/replication/policy.go new file mode 100644 index 0000000..ea9b190 --- /dev/null +++ b/replication/policy.go @@ -0,0 +1,12 @@ +package replication + +// FIXME: Leftovers from previous versions, not used currently +type InitialReplPolicy string + +const ( + InitialReplPolicyMostRecent InitialReplPolicy = "most_recent" + InitialReplPolicyAll InitialReplPolicy = "all" +) + +const DEFAULT_INITIAL_REPL_POLICY = InitialReplPolicyMostRecent + From 873c64ecc3f6c5e1f03c64e296200c8b7b61fbd4 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 22 Aug 2018 10:15:27 +0200 Subject: [PATCH 023/167] update README to reflect restructuring --- README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/README.md b/README.md index bca53ad..0fc4d2c 100644 --- a/README.md +++ b/README.md @@ -43,6 +43,7 @@ Make sure to develop an understanding how zrepl is typically used by studying th ``` ├── cmd +│   ├── endpoint # implementations of endpoints for package replication │   ├── sampleconf # example configuration ├── docs # sphinx-based documentation │   ├── **/*.rst # documentation in reStructuredText @@ -52,6 +53,7 @@ Make sure to develop an understanding how zrepl is typically used by studying th │   ├── publish.sh # shell script for automated rendering & deploy to zrepl.github.io repo │   ├── public_git # checkout of zrepl.github.io managed by above shell script ├── logger # logger package used by zrepl +├── replication # replication functionality ├── rpc # rpc protocol implementation ├── util └── zfs # ZFS wrappers, filesystemm diffing From d677cde6d0d33d5d3af3ba5f716876392329c9c7 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 25 Aug 2018 12:58:17 +0200 Subject: [PATCH 024/167] implement tcp and tcp+tls transports --- cmd/config_connect.go | 78 +++++++++++++++++++++++++ cmd/config_job_pull.go | 7 +-- cmd/config_job_source.go | 4 +- cmd/config_logging.go | 66 ++++++++++----------- cmd/config_parse.go | 4 ++ cmd/config_serve_tcp.go | 86 ++++++++++++++++++++++++++++ cmd/tlsconf/tlsconf.go | 121 +++++++++++++++++++++++++++++++++++++++ 7 files changed, 325 insertions(+), 41 deletions(-) create mode 100644 cmd/config_serve_tcp.go create mode 100644 cmd/tlsconf/tlsconf.go diff --git a/cmd/config_connect.go b/cmd/config_connect.go index d94f065..83c7976 100644 --- a/cmd/config_connect.go +++ b/cmd/config_connect.go @@ -1,6 +1,7 @@ package cmd import ( + "crypto/tls" "fmt" "net" @@ -11,6 +12,7 @@ import ( "github.com/problame/go-netssh" "github.com/problame/go-streamrpc" "time" + "github.com/zrepl/zrepl/cmd/tlsconf" ) type SSHStdinserverConnecter struct { @@ -74,3 +76,79 @@ func (c *SSHStdinserverConnecter) Connect(dialCtx context.Context) (net.Conn, er } return netsshConnToConn{nconn}, nil } + +type TCPConnecter struct { + Host string + Port uint16 + dialer net.Dialer + tlsConfig *tls.Config +} + +func parseTCPConnecter(i map[string]interface{}) (*TCPConnecter, error) { + var in struct { + Host string + Port uint16 + DialTimeout string `mapstructure:"dial_timeout"` + TLS map[string]interface{} + } + if err := mapstructure.Decode(i, &in); err != nil { + return nil, errors.Wrap(err, "mapstructure error") + } + + if in.Host == "" || in.Port == 0 { + return nil, errors.New("fields 'host' and 'port' must not be empty") + } + dialTimeout, err := parsePostitiveDuration(in.DialTimeout) + if err != nil { + if in.DialTimeout != "" { + return nil, errors.Wrap(err, "cannot parse field 'dial_timeout'") + } + dialTimeout = 10 * time.Second + } + dialer := net.Dialer{ + Timeout: dialTimeout, + } + + var tlsConfig *tls.Config + if in.TLS != nil { + tlsConfig, err = func(i map[string]interface{}) (config *tls.Config, err error) { + var in struct { + CA string + Cert string + Key string + ServerCN string `mapstructure:"server_cn"` + } + if err := mapstructure.Decode(i, &in); err != nil { + return nil, errors.Wrap(err, "mapstructure error") + } + if in.CA == "" || in.Cert == "" || in.Key == "" || in.ServerCN == "" { + return nil, errors.New("fields 'ca', 'cert', 'key' and 'server_cn' must be specified") + } + + ca, err := tlsconf.ParseCAFile(in.CA) + if err != nil { + return nil, errors.Wrap(err, "cannot parse ca file") + } + + cert, err := tls.LoadX509KeyPair(in.Cert, in.Key) + if err != nil { + return nil, errors.Wrap(err, "cannot parse cert/key pair") + } + + return tlsconf.ClientAuthClient(in.ServerCN, ca, cert) + }(in.TLS) + if err != nil { + return nil, errors.Wrap(err, "cannot parse TLS config in field 'tls'") + } + } + + return &TCPConnecter{in.Host, in.Port, dialer, tlsConfig}, nil +} + +func (c *TCPConnecter) Connect(dialCtx context.Context) (conn net.Conn, err error) { + addr := fmt.Sprintf("%s:%d", c.Host, c.Port) + if c.tlsConfig != nil { + return tls.DialWithDialer(&c.dialer, "tcp", addr, c.tlsConfig) + } + return c.dialer.DialContext(dialCtx, "tcp", addr) +} diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index 48ce8de..4ee4d35 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -51,7 +51,7 @@ func parsePullJob(c JobParsingContext, name string, i map[string]interface{}) (j j = &PullJob{Name: name} - j.Connect, err = parseSSHStdinserverConnecter(asMap.Connect) + j.Connect, err = parseConnect(asMap.Connect) if err != nil { err = errors.Wrap(err, "cannot parse 'connect'") return nil, err @@ -167,10 +167,7 @@ func (j *PullJob) doRun(ctx context.Context) { ConnConfig: STREAMRPC_CONFIG, } - //client, err := streamrpc.NewClient(j.Connect, clientConf) - client, err := streamrpc.NewClient(&tcpConnecter{net.Dialer{ - Timeout: 10*time.Second, - }}, clientConf) + client, err := streamrpc.NewClient(j.Connect, clientConf) defer client.Close() j.task.Enter("pull") diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index b91179d..9afe61c 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -147,9 +147,7 @@ func (j *SourceJob) Pruner(task *Task, side PrunePolicySide, dryRun bool) (p Pru func (j *SourceJob) serve(ctx context.Context, task *Task) { - //listener, err := j.Serve.Listen() - - listener, err := net.Listen("tcp", ":8888") + listener, err := j.Serve.Listen() if err != nil { task.Log().WithError(err).Error("error listening") return diff --git a/cmd/config_logging.go b/cmd/config_logging.go index 927fbe6..00707ee 100644 --- a/cmd/config_logging.go +++ b/cmd/config_logging.go @@ -1,15 +1,15 @@ package cmd import ( - "crypto/tls" - "crypto/x509" "github.com/mattn/go-isatty" "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/zrepl/zrepl/logger" - "io/ioutil" "os" "time" + "crypto/tls" + "crypto/x509" + "github.com/zrepl/zrepl/cmd/tlsconf" ) type LoggingConfig struct { @@ -164,11 +164,7 @@ func parseTCPOutlet(i interface{}, formatter EntryFormatter) (out *TCPOutlet, er Net string Address string RetryInterval string `mapstructure:"retry_interval"` - TLS *struct { - CA string - Cert string - Key string - } + TLS map[string]interface{} } if err = mapstructure.Decode(i, &in); err != nil { return nil, errors.Wrap(err, "mapstructure error") @@ -188,37 +184,41 @@ func parseTCPOutlet(i interface{}, formatter EntryFormatter) (out *TCPOutlet, er var tlsConfig *tls.Config if in.TLS != nil { - - cert, err := tls.LoadX509KeyPair(in.TLS.Cert, in.TLS.Key) - if err != nil { - return nil, errors.Wrap(err, "cannot load client cert") - } - - var rootCAs *x509.CertPool - if in.TLS.CA == "" { - if rootCAs, err = x509.SystemCertPool(); err != nil { - return nil, errors.Wrap(err, "cannot open system cert pool") + tlsConfig, err = func(m map[string]interface{}, host string) (*tls.Config, error) { + var in struct { + CA string + Cert string + Key string } - } else { - rootCAs = x509.NewCertPool() - rootCAPEM, err := ioutil.ReadFile(in.TLS.CA) + if err := mapstructure.Decode(m, &in); err != nil { + return nil, errors.Wrap(err, "mapstructure error") + } + + clientCert, err := tls.LoadX509KeyPair(in.Cert, in.Key) if err != nil { - return nil, errors.Wrap(err, "cannot load CA cert") + return nil, errors.Wrap(err, "cannot load client cert") } - if !rootCAs.AppendCertsFromPEM(rootCAPEM) { - return nil, errors.New("cannot parse CA cert") + + var rootCAs *x509.CertPool + if in.CA == "" { + if rootCAs, err = x509.SystemCertPool(); err != nil { + return nil, errors.Wrap(err, "cannot open system cert pool") + } + } else { + rootCAs, err = tlsconf.ParseCAFile(in.CA) + if err != nil { + return nil, errors.Wrap(err, "cannot parse CA cert") + } + } + if rootCAs == nil { + panic("invariant violated") } - } - if err != nil && in.TLS.CA == "" { - return nil, errors.Wrap(err, "cannot load root ca pool") - } - tlsConfig = &tls.Config{ - Certificates: []tls.Certificate{cert}, - RootCAs: rootCAs, + return tlsconf.ClientAuthClient(host, rootCAs, clientCert) + }(in.TLS, in.Address) + if err != nil { + return nil, errors.New("cannot not parse TLS config in field 'tls'") } - - tlsConfig.BuildNameToCertificate() } formatter.SetMetadataFlags(MetadataAll) diff --git a/cmd/config_parse.go b/cmd/config_parse.go index 1a773c6..9c334ae 100644 --- a/cmd/config_parse.go +++ b/cmd/config_parse.go @@ -220,6 +220,8 @@ func parseConnect(i map[string]interface{}) (c streamrpc.Connecter, err error) { switch t { case "ssh+stdinserver": return parseSSHStdinserverConnecter(i) + case "tcp": + return parseTCPConnecter(i) default: return nil, errors.Errorf("unknown connection type '%s'", t) } @@ -278,6 +280,8 @@ func parseAuthenticatedChannelListenerFactory(c JobParsingContext, v map[string] switch t { case "stdinserver": return parseStdinserverListenerFactory(c, v) + case "tcp": + return parseTCPListenerFactory(c, v) default: err = errors.Errorf("unknown type '%s'", t) return diff --git a/cmd/config_serve_tcp.go b/cmd/config_serve_tcp.go new file mode 100644 index 0000000..b85d01a --- /dev/null +++ b/cmd/config_serve_tcp.go @@ -0,0 +1,86 @@ +package cmd + +import ( + "crypto/tls" + "crypto/x509" + "net" + "time" + + "github.com/mitchellh/mapstructure" + "github.com/pkg/errors" + "github.com/zrepl/zrepl/cmd/tlsconf" +) + +type TCPListenerFactory struct { + Address string + tls bool + clientCA *x509.CertPool + serverCert tls.Certificate + clientCommonName string +} + +func parseTCPListenerFactory(c JobParsingContext, i map[string]interface{}) (*TCPListenerFactory, error) { + + var in struct { + Address string + TLS map[string]interface{} + } + if err := mapstructure.Decode(i, &in); err != nil { + return nil, errors.Wrap(err, "mapstructure error") + } + + lf := &TCPListenerFactory{} + + if in.Address == "" { + return nil, errors.New("must specify field 'address'") + } + lf.Address = in.Address + + if in.TLS != nil { + err := func(i map[string]interface{}) (err error) { + var in struct { + CA string + Cert string + Key string + ClientCN string `mapstructure:"client_cn"` + } + if err := mapstructure.Decode(i, &in); err != nil { + return errors.Wrap(err, "mapstructure error") + } + + if in.CA == "" || in.Cert == "" || in.Key == "" || in.ClientCN == "" { + return errors.New("fields 'ca', 'cert', 'key' and 'client_cn' must be specified") + } + + lf.clientCommonName = in.ClientCN + + lf.clientCA, err = tlsconf.ParseCAFile(in.CA) + if err != nil { + return errors.Wrap(err,"cannot parse ca file") + } + + lf.serverCert, err = tls.LoadX509KeyPair(in.Cert, in.Key) + if err != nil { + return errors.Wrap(err, "cannot parse cer/key pair") + } + + lf.tls = true // mark success + return nil + }(in.TLS) + if err != nil { + return nil, errors.Wrap(err, "error parsing TLS config in field 'tls'") + } + } + + return lf, nil +} + +func (f *TCPListenerFactory) Listen() (net.Listener, error) { + l, err := net.Listen("tcp", f.Address) + if !f.tls || err != nil { + return l, err + } + + tl := tlsconf.NewClientAuthListener(l, f.clientCA, f.serverCert, f.clientCommonName, 10*time.Second) + return tl, nil +} diff --git a/cmd/tlsconf/tlsconf.go b/cmd/tlsconf/tlsconf.go new file mode 100644 index 0000000..77372f8 --- /dev/null +++ b/cmd/tlsconf/tlsconf.go @@ -0,0 +1,121 @@ +package tlsconf + +import ( + "crypto/tls" + "crypto/x509" + "errors" + "fmt" + "io/ioutil" + "net" + "time" +) + +func ParseCAFile(certfile string) (*x509.CertPool, error) { + pool := x509.NewCertPool() + pem, err := ioutil.ReadFile(certfile) + if err != nil { + return nil, err + } + if !pool.AppendCertsFromPEM(pem) { + return nil, errors.New("PEM parsing error") + } + return pool, nil +} + +type ClientAuthListener struct { + l net.Listener + clientCommonName string + handshakeTimeout time.Duration +} + +func NewClientAuthListener( + l net.Listener, ca *x509.CertPool, serverCert tls.Certificate, + clientCommonName string, handshakeTimeout time.Duration) *ClientAuthListener { + + if ca == nil { + panic(ca) + } + if serverCert.Certificate == nil || serverCert.PrivateKey == nil { + panic(serverCert) + } + if clientCommonName == "" { + panic(clientCommonName) + } + + tlsConf := tls.Config{ + Certificates: []tls.Certificate{serverCert}, + ClientCAs: ca, + ClientAuth: tls.RequireAndVerifyClientCert, + PreferServerCipherSuites: true, + } + l = tls.NewListener(l, &tlsConf) + return &ClientAuthListener{ + l, + clientCommonName, + handshakeTimeout, + } +} + +func (l *ClientAuthListener) Accept() (c net.Conn, err error) { + c, err = l.l.Accept() + if err != nil { + return nil, err + } + tlsConn, ok := c.(*tls.Conn) + if !ok { + return c, err + } + + var ( + cn string + peerCerts []*x509.Certificate + ) + if err = tlsConn.SetDeadline(time.Now().Add(l.handshakeTimeout)); err != nil { + goto CloseAndErr + } + if err = tlsConn.Handshake(); err != nil { + goto CloseAndErr + } + + peerCerts = tlsConn.ConnectionState().PeerCertificates + if len(peerCerts) != 1 { + err = errors.New("unexpected number of certificates presented by TLS client") + goto CloseAndErr + } + cn = peerCerts[0].Subject.CommonName + if cn != l.clientCommonName { + err = fmt.Errorf("client cert common name does not match client_identity: %q != %q", cn, l.clientCommonName) + goto CloseAndErr + } + return c, nil +CloseAndErr: + c.Close() + return nil, err +} + +func (l *ClientAuthListener) Addr() net.Addr { + return l.l.Addr() +} + +func (l *ClientAuthListener) Close() error { + return l.l.Close() +} + +func ClientAuthClient(serverName string, rootCA *x509.CertPool, clientCert tls.Certificate) (*tls.Config, error) { + if serverName == "" { + panic(serverName) + } + if rootCA == nil { + panic(rootCA) + } + if clientCert.Certificate == nil || clientCert.PrivateKey == nil { + panic(clientCert) + } + tlsConfig := &tls.Config{ + Certificates: []tls.Certificate{clientCert}, + RootCAs: rootCA, + ServerName: serverName, + } + tlsConfig.BuildNameToCertificate() + return tlsConfig, nil +} \ No newline at end of file From 51cfcfe79b0b5faf5a88af784cb36fdeec2fed04 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 25 Aug 2018 13:00:51 +0200 Subject: [PATCH 025/167] job source: do not stop listener on accept() errors refs #77 --- cmd/config_job_source.go | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index 9afe61c..eaa9457 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -160,16 +160,11 @@ func (j *SourceJob) serve(ctx context.Context, task *Task) { connChan := make(chan connChanMsg) // Serve connections until interrupted or error -outer: + outer: for { go func() { rwc, err := listener.Accept() - if err != nil { - connChan <- connChanMsg{rwc, err} - close(connChan) - return - } connChan <- connChanMsg{rwc, err} }() @@ -178,8 +173,8 @@ outer: case rwcMsg := <-connChan: if rwcMsg.err != nil { - task.Log().WithError(err).Error("error accepting connection") - break outer + task.Log().WithError(rwcMsg.err).Error("error accepting connection") + continue } j.handleConnection(rwcMsg.conn, task) @@ -187,11 +182,12 @@ outer: case <-ctx.Done(): task.Log().WithError(ctx.Err()).Info("context") break outer - } } + task.Log().Info("closing listener") + task.Enter("close_listener") defer task.Finish() err = listener.Close() From f46d1bc338ac7c38d86f9fd9b6c6c3d83c265774 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 25 Aug 2018 13:02:38 +0200 Subject: [PATCH 026/167] fixup 70aad0940f37e611c20165a0d4c300e6e209d304: fix broken config_test.go --- cmd/config_test.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cmd/config_test.go b/cmd/config_test.go index 5bfcb2b..253e9fc 100644 --- a/cmd/config_test.go +++ b/cmd/config_test.go @@ -249,8 +249,10 @@ func TestDatasetMapFilter_Invert(t *testing.T) { m, err := parseDatasetMapFilter(mapspec, false) assert.NoError(t, err) - inv, err := m.Invert() + invI, err := m.Invert() assert.NoError(t, err) + inv, ok := invI.(*DatasetMapFilter) + assert.True(t, ok) expectMapping := func(m *DatasetMapFilter, input, expect string, expErr bool, expEmpty bool) { p, err := zfs.NewDatasetPath(input) From b56e236874431352bf258c47b915f80578dfab5e Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 25 Aug 2018 15:14:27 +0200 Subject: [PATCH 027/167] add go-streamrpc to Gopkg.toml --- Gopkg.lock | 106 ++++++++++++++++++++++++++++++++++++++++++++++++++--- Gopkg.toml | 4 ++ 2 files changed, 105 insertions(+), 5 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index 5b51855..0713a08 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -3,163 +3,259 @@ [[projects]] branch = "master" + digest = "1:c0bec5f9b98d0bc872ff5e834fac186b807b656683bd29cb82fb207a1513fabb" name = "github.com/beorn7/perks" packages = ["quantile"] + pruneopts = "" revision = "3a771d992973f24aa725d07868b467d1ddfceafb" [[projects]] + digest = "1:56c130d885a4aacae1dd9c7b71cfe39912c7ebc1ff7d2b46083c8812996dc43b" name = "github.com/davecgh/go-spew" packages = ["spew"] + pruneopts = "" revision = "346938d642f2ec3594ed81d874461961cd0faa76" version = "v1.1.0" [[projects]] branch = "master" + digest = "1:ae162f9b5c46f6d5ff4bd53a3d78f72e2eb6676c11c5d33b8b106c36f87ddb31" name = "github.com/dustin/go-humanize" packages = ["."] + pruneopts = "" revision = "bb3d318650d48840a39aa21a027c6630e198e626" [[projects]] branch = "master" + digest = "1:5d0a2385edf4ba44f3b7b76bc0436ceb8f62bf55aa5d540a9eb9ec6c58d86809" name = "github.com/ftrvxmtrx/fd" packages = ["."] + pruneopts = "" revision = "c6d800382fff6dc1412f34269f71b7f83bd059ad" [[projects]] + digest = "1:6a4a01d58b227c4b6b11111b9f172ec5c17682b82724e58e6daf3f19f4faccd8" name = "github.com/go-logfmt/logfmt" packages = ["."] + pruneopts = "" revision = "390ab7935ee28ec6b286364bba9b4dd6410cb3d5" version = "v0.3.0" [[projects]] branch = "v2" + digest = "1:81314a486195626940617e43740b4fa073f265b0715c9f54ce2027fee1cb5f61" name = "github.com/go-yaml/yaml" packages = ["."] + pruneopts = "" revision = "eb3733d160e74a9c7e442f435eb3bea458e1d19f" [[projects]] + digest = "1:bcb38c8fc9b21bb8682ce2d605a7d4aeb618abc7f827e3ac0b27c0371fdb23fb" name = "github.com/golang/protobuf" packages = ["proto"] + pruneopts = "" revision = "925541529c1fa6821df4e44ce2723319eb2be768" version = "v1.0.0" [[projects]] branch = "master" + digest = "1:cb09475f771b9167fb9333629f5d6a7161572602ea040f1094602b0dc8709878" name = "github.com/jinzhu/copier" packages = ["."] + pruneopts = "" revision = "db4671f3a9b8df855e993f7c94ec5ef1ffb0a23b" [[projects]] branch = "master" + digest = "1:1ed9eeebdf24aadfbca57eb50e6455bd1d2474525e0f0d4454de8c8e9bc7ee9a" name = "github.com/kr/logfmt" packages = ["."] + pruneopts = "" revision = "b84e30acd515aadc4b783ad4ff83aff3299bdfe0" [[projects]] branch = "master" + digest = "1:591a2778aa6e896980757ea87e659b3aa13d8c0e790310614028463a31c0998b" name = "github.com/kr/pretty" packages = ["."] + pruneopts = "" revision = "cfb55aafdaf3ec08f0db22699ab822c50091b1c4" [[projects]] branch = "master" + digest = "1:109e0a9b9e74c5c8adf94a2dc4dd4a9ca2a183d4d87ffecd4e62db69a5ede55a" name = "github.com/kr/text" packages = ["."] + pruneopts = "" revision = "7cafcd837844e784b526369c9bce262804aebc60" [[projects]] + digest = "1:78229b46ddb7434f881390029bd1af7661294af31f6802e0e1bedaad4ab0af3c" name = "github.com/mattn/go-isatty" packages = ["."] + pruneopts = "" revision = "0360b2af4f38e8d38c7fce2a9f4e702702d73a39" version = "v0.0.3" [[projects]] + digest = "1:4c23ced97a470b17d9ffd788310502a077b9c1f60221a85563e49696276b4147" name = "github.com/matttproud/golang_protobuf_extensions" packages = ["pbutil"] + pruneopts = "" revision = "3247c84500bff8d9fb6d579d800f20b3e091582c" version = "v1.0.0" [[projects]] branch = "master" + digest = "1:c9ede10a9ded782d25d1f0be87c680e11409c23554828f19a19d691a95e76130" name = "github.com/mitchellh/mapstructure" packages = ["."] + pruneopts = "" revision = "d0303fe809921458f417bcf828397a65db30a7e4" [[projects]] + digest = "1:7365acd48986e205ccb8652cc746f09c8b7876030d53710ea6ef7d0bd0dcd7ca" name = "github.com/pkg/errors" packages = ["."] + pruneopts = "" revision = "645ef00459ed84a119197bfb8d8205042c6df63d" version = "v0.8.0" [[projects]] + digest = "1:256484dbbcd271f9ecebc6795b2df8cad4c458dd0f5fd82a8c2fa0c29f233411" name = "github.com/pmezard/go-difflib" packages = ["difflib"] + pruneopts = "" revision = "792786c7400a136282c1664665ae0a8db921c6c2" version = "v1.0.0" [[projects]] branch = "master" + digest = "1:1392748e290ca66ac8447ef24961f8ae9e1d846a53af0f58a5a0256982ce0577" name = "github.com/problame/go-netssh" packages = ["."] + pruneopts = "" revision = "53a2e445f8ace7ec678f2d8cdd9c1428dfef4562" [[projects]] branch = "master" + digest = "1:8c63c44f018bd52b03ebad65c9df26aabbc6793138e421df1c8c84c285a45bc6" name = "github.com/problame/go-rwccmd" packages = ["."] + pruneopts = "" revision = "391d2c78c8404a9683d79f75dd24ab53040f89f7" +[[projects]] + digest = "1:f80c558f761d9cbfb0cd1a317ece8b12a55ec670f8cef52acc515ab76463e6b6" + name = "github.com/problame/go-streamrpc" + packages = [ + ".", + "internal/pdu", + ] + pruneopts = "" + revision = "de264a1c39cd8fc42a6b5e902c6eac7fd9683521" + version = "0.1" + [[projects]] branch = "master" + digest = "1:ebf8ffdde9bdbf9c83e22121875c68c01d821776523546554b2a3ff6f72773ab" name = "github.com/prometheus/client_golang" - packages = ["prometheus","prometheus/promhttp"] + packages = [ + "prometheus", + "prometheus/promhttp", + ] + pruneopts = "" revision = "e11c6ff8170beca9d5fd8b938e71165eeec53ac6" [[projects]] branch = "master" + digest = "1:60aca47f4eeeb972f1b9da7e7db51dee15ff6c59f7b401c1588b8e6771ba15ef" name = "github.com/prometheus/client_model" packages = ["go"] + pruneopts = "" revision = "99fa1f4be8e564e8a6b613da7fa6f46c9edafc6c" [[projects]] branch = "master" + digest = "1:af21ee3e0a8212f17bb317cd7237f9920bcb2641a291ac111f30f63b3cab817f" name = "github.com/prometheus/common" - packages = ["expfmt","internal/bitbucket.org/ww/goautoneg","model"] + packages = [ + "expfmt", + "internal/bitbucket.org/ww/goautoneg", + "model", + ] + pruneopts = "" revision = "d0f7cd64bda49e08b22ae8a730aa57aa0db125d6" [[projects]] branch = "master" + digest = "1:61df0898746840afc7be5dc2c3eeec83022fab70df11ecee5b16c85e912cf5ed" name = "github.com/prometheus/procfs" - packages = [".","internal/util","nfs","xfs"] + packages = [ + ".", + "internal/util", + "nfs", + "xfs", + ] + pruneopts = "" revision = "8b1c2da0d56deffdbb9e48d4414b4e674bd8083e" [[projects]] branch = "master" + digest = "1:146327ce93be37e68bd3ff8541090d96da8cb3adc9e35d57570e9170a29f6bf6" name = "github.com/spf13/cobra" packages = ["."] + pruneopts = "" revision = "b78744579491c1ceeaaa3b40205e56b0591b93a3" [[projects]] + digest = "1:261bc565833ef4f02121450d74eb88d5ae4bd74bfe5d0e862cddb8550ec35000" name = "github.com/spf13/pflag" packages = ["."] + pruneopts = "" revision = "e57e3eeb33f795204c1ca35f56c44f83227c6e66" version = "v1.0.0" [[projects]] + digest = "1:3926a4ec9a4ff1a072458451aa2d9b98acd059a45b38f7335d31e06c3d6a0159" name = "github.com/stretchr/testify" - packages = ["assert"] + packages = [ + "assert", + "require", + ] + pruneopts = "" revision = "69483b4bd14f5845b5a1e55bca19e954e827f1d0" version = "v1.1.4" [[projects]] branch = "master" + digest = "1:9c286cf11d0ca56368185bada5dd6d97b6be4648fc26c354fcba8df7293718f7" name = "golang.org/x/sys" packages = ["unix"] + pruneopts = "" revision = "bf42f188b9bc6f2cf5b8ee5a912ef1aedd0eba4c" [solve-meta] analyzer-name = "dep" analyzer-version = 1 - inputs-digest = "36731e77252dcc851fdfa2b0b0778b980597e3c1a47b5e2af3bd0bcb802662ec" + input-imports = [ + "github.com/dustin/go-humanize", + "github.com/go-logfmt/logfmt", + "github.com/go-yaml/yaml", + "github.com/golang/protobuf/proto", + "github.com/jinzhu/copier", + "github.com/kr/pretty", + "github.com/mattn/go-isatty", + "github.com/mitchellh/mapstructure", + "github.com/pkg/errors", + "github.com/problame/go-netssh", + "github.com/problame/go-rwccmd", + "github.com/problame/go-streamrpc", + "github.com/prometheus/client_golang/prometheus", + "github.com/prometheus/client_golang/prometheus/promhttp", + "github.com/spf13/cobra", + "github.com/stretchr/testify/assert", + "github.com/stretchr/testify/require", + ] solver-name = "gps-cdcl" solver-version = 1 diff --git a/Gopkg.toml b/Gopkg.toml index f526185..ae6f5d8 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -55,3 +55,7 @@ ignored = [ "github.com/inconshreveable/mousetrap" ] [[constraint]] name = "github.com/prometheus/client_golang" branch = "master" + +[[constraint]] + name = "github.com/problame/go-streamrpc" + version = "0.1.0" From e082816de5d8cd5a19c42f818955b1bb6844e1a9 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 25 Aug 2018 15:16:38 +0200 Subject: [PATCH 028/167] fixup d677cde6d0d33d5d3af3ba5f716876392329c9c7: unused import --- cmd/config_job_pull.go | 1 - 1 file changed, 1 deletion(-) diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index 4ee4d35..bec1196 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -1,7 +1,6 @@ package cmd import ( - "net" "os" "os/signal" "syscall" From e30ae972f4c76940b4ecf9aa555f2c6a20b093de Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 25 Aug 2018 21:30:25 +0200 Subject: [PATCH 029/167] gofmt --- cmd/adaptors.go | 38 ++++++++++++++--------------- cmd/config_connect.go | 8 +++--- cmd/config_job_control.go | 2 +- cmd/config_job_local.go | 4 +-- cmd/config_job_pull.go | 7 +++--- cmd/config_job_source.go | 4 +-- cmd/config_logging.go | 6 ++--- cmd/config_mapfilter.go | 6 ++--- cmd/config_parse.go | 4 +-- cmd/config_serve_tcp.go | 6 ++--- cmd/config_test.go | 4 +-- cmd/endpoint/endpoint.go | 37 ++++++++++++++-------------- cmd/tlsconf/tlsconf.go | 6 ++--- logger/nulllogger.go | 22 ++++++++--------- replication/context.go | 3 +-- replication/fsrep/fsfsm.go | 19 ++++++--------- replication/internal/queue/queue.go | 10 ++++---- replication/mainfsm.go | 10 +++----- replication/pdu/pdu_extras.go | 16 ++++++------ replication/pdu/pdu_test.go | 8 +++--- replication/policy.go | 1 - util/contextflexibletimeout.go | 12 ++++----- util/contextflexibletimeout_test.go | 30 +++++++++++------------ util/io.go | 2 +- zfs/diff.go | 4 +-- zfs/mapping.go | 6 ++--- zfs/versions.go | 2 +- 27 files changed, 130 insertions(+), 147 deletions(-) diff --git a/cmd/adaptors.go b/cmd/adaptors.go index 86e7c37..acf1a0d 100644 --- a/cmd/adaptors.go +++ b/cmd/adaptors.go @@ -9,8 +9,8 @@ import ( "time" "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/util" "github.com/zrepl/zrepl/logger" + "github.com/zrepl/zrepl/util" ) type logNetConnConnecter struct { @@ -29,36 +29,35 @@ func (l logNetConnConnecter) Connect(ctx context.Context) (net.Conn, error) { } type logListenerFactory struct { - ListenerFactory - ReadDump, WriteDump string + ListenerFactory + ReadDump, WriteDump string } var _ ListenerFactory = logListenerFactory{} type logListener struct { - net.Listener - ReadDump, WriteDump string + net.Listener + ReadDump, WriteDump string } var _ net.Listener = logListener{} func (m logListenerFactory) Listen() (net.Listener, error) { - l, err := m.ListenerFactory.Listen() - if err != nil { - return nil, err - } - return logListener{l, m.ReadDump, m.WriteDump}, nil + l, err := m.ListenerFactory.Listen() + if err != nil { + return nil, err + } + return logListener{l, m.ReadDump, m.WriteDump}, nil } func (l logListener) Accept() (net.Conn, error) { - conn, err := l.Listener.Accept() - if err != nil { - return nil, err - } - return util.NewNetConnLogger(conn, l.ReadDump, l.WriteDump) + conn, err := l.Listener.Accept() + if err != nil { + return nil, err + } + return util.NewNetConnLogger(conn, l.ReadDump, l.WriteDump) } - type netsshAddr struct{} func (netsshAddr) Network() string { return "netssh" } @@ -78,7 +77,7 @@ func (netsshConnToNetConnAdatper) SetReadDeadline(t time.Time) error { return ni func (netsshConnToNetConnAdatper) SetWriteDeadline(t time.Time) error { return nil } -type streamrpcLogAdaptor = twoClassLogAdaptor +type streamrpcLogAdaptor = twoClassLogAdaptor type replicationLogAdaptor = twoClassLogAdaptor type twoClassLogAdaptor struct { @@ -87,7 +86,7 @@ type twoClassLogAdaptor struct { var _ streamrpc.Logger = twoClassLogAdaptor{} -func (a twoClassLogAdaptor) Errorf(fmtStr string, args... interface{}) { +func (a twoClassLogAdaptor) Errorf(fmtStr string, args ...interface{}) { const errorSuffix = ": %s" if len(args) == 1 { if err, ok := args[0].(error); ok && strings.HasSuffix(fmtStr, errorSuffix) { @@ -99,7 +98,6 @@ func (a twoClassLogAdaptor) Errorf(fmtStr string, args... interface{}) { a.Logger.Error(fmt.Sprintf(fmtStr, args...)) } -func (a twoClassLogAdaptor) Infof(fmtStr string, args... interface{}) { +func (a twoClassLogAdaptor) Infof(fmtStr string, args ...interface{}) { a.Logger.Info(fmt.Sprintf(fmtStr, args...)) } - diff --git a/cmd/config_connect.go b/cmd/config_connect.go index 83c7976..e13274a 100644 --- a/cmd/config_connect.go +++ b/cmd/config_connect.go @@ -11,8 +11,8 @@ import ( "github.com/pkg/errors" "github.com/problame/go-netssh" "github.com/problame/go-streamrpc" - "time" "github.com/zrepl/zrepl/cmd/tlsconf" + "time" ) type SSHStdinserverConnecter struct { @@ -51,12 +51,12 @@ func parseSSHStdinserverConnecter(i map[string]interface{}) (c *SSHStdinserverCo } -type netsshConnToConn struct { *netssh.SSHConn } +type netsshConnToConn struct{ *netssh.SSHConn } var _ net.Conn = netsshConnToConn{} -func (netsshConnToConn) SetDeadline(dl time.Time) error { return nil } -func (netsshConnToConn) SetReadDeadline(dl time.Time) error { return nil } +func (netsshConnToConn) SetDeadline(dl time.Time) error { return nil } +func (netsshConnToConn) SetReadDeadline(dl time.Time) error { return nil } func (netsshConnToConn) SetWriteDeadline(dl time.Time) error { return nil } func (c *SSHStdinserverConnecter) Connect(dialCtx context.Context) (net.Conn, error) { diff --git a/cmd/config_job_control.go b/cmd/config_job_control.go index 300e4b0..85d906e 100644 --- a/cmd/config_job_control.go +++ b/cmd/config_job_control.go @@ -5,10 +5,10 @@ import ( "context" "encoding/json" "github.com/pkg/errors" + "github.com/zrepl/zrepl/logger" "io" "net" "net/http" - "github.com/zrepl/zrepl/logger" ) type ControlJob struct { diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index fafac4d..fce3708 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -6,10 +6,10 @@ import ( "context" "github.com/mitchellh/mapstructure" "github.com/pkg/errors" + "github.com/zrepl/zrepl/cmd/endpoint" + "github.com/zrepl/zrepl/replication" "github.com/zrepl/zrepl/zfs" "sync" - "github.com/zrepl/zrepl/replication" - "github.com/zrepl/zrepl/cmd/endpoint" ) type LocalJob struct { diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index bec1196..df6dab1 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -11,8 +11,8 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/replication" "github.com/zrepl/zrepl/cmd/endpoint" + "github.com/zrepl/zrepl/replication" ) type PullJob struct { @@ -28,7 +28,7 @@ type PullJob struct { Debug JobDebugSettings task *Task - rep *replication.Replication + rep *replication.Replication } func parsePullJob(c JobParsingContext, name string, i map[string]interface{}) (j *PullJob, err error) { @@ -155,7 +155,6 @@ var STREAMRPC_CONFIG = &streamrpc.ConnConfig{ // FIXME oversight and configurabi }, } - func (j *PullJob) doRun(ctx context.Context) { j.task.Enter("run") @@ -184,7 +183,7 @@ func (j *PullJob) doRun(ctx context.Context) { } ctx = replication.WithLogger(ctx, replicationLogAdaptor{j.task.Log().WithField("subsystem", "replication")}) - ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{j.task.Log().WithField("subsystem", "rpc.protocol")}) + ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{j.task.Log().WithField("subsystem", "rpc.protocol")}) ctx = endpoint.WithLogger(ctx, j.task.Log().WithField("subsystem", "rpc.endpoint")) j.rep = replication.NewReplication() diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index eaa9457..c79d3c9 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -7,8 +7,8 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-streamrpc" - "net" "github.com/zrepl/zrepl/cmd/endpoint" + "net" ) type SourceJob struct { @@ -160,7 +160,7 @@ func (j *SourceJob) serve(ctx context.Context, task *Task) { connChan := make(chan connChanMsg) // Serve connections until interrupted or error - outer: +outer: for { go func() { diff --git a/cmd/config_logging.go b/cmd/config_logging.go index 00707ee..200f365 100644 --- a/cmd/config_logging.go +++ b/cmd/config_logging.go @@ -1,15 +1,15 @@ package cmd import ( + "crypto/tls" + "crypto/x509" "github.com/mattn/go-isatty" "github.com/mitchellh/mapstructure" "github.com/pkg/errors" + "github.com/zrepl/zrepl/cmd/tlsconf" "github.com/zrepl/zrepl/logger" "os" "time" - "crypto/tls" - "crypto/x509" - "github.com/zrepl/zrepl/cmd/tlsconf" ) type LoggingConfig struct { diff --git a/cmd/config_mapfilter.go b/cmd/config_mapfilter.go index e59f482..0a1de54 100644 --- a/cmd/config_mapfilter.go +++ b/cmd/config_mapfilter.go @@ -6,8 +6,8 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" - "github.com/zrepl/zrepl/zfs" "github.com/zrepl/zrepl/cmd/endpoint" + "github.com/zrepl/zrepl/zfs" ) type DatasetMapFilter struct { @@ -210,8 +210,8 @@ func (m DatasetMapFilter) Invert() (endpoint.FSMap, error) { } inv.entries[0] = datasetMapFilterEntry{ - path: mp, - mapping: e.path.ToString(), + path: mp, + mapping: e.path.ToString(), subtreeMatch: e.subtreeMatch, } diff --git a/cmd/config_parse.go b/cmd/config_parse.go index 9c334ae..573436a 100644 --- a/cmd/config_parse.go +++ b/cmd/config_parse.go @@ -7,12 +7,12 @@ import ( yaml "github.com/go-yaml/yaml" "github.com/mitchellh/mapstructure" "github.com/pkg/errors" + "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/replication" "os" "regexp" "strconv" "time" - "github.com/zrepl/zrepl/replication" - "github.com/problame/go-streamrpc" ) var ConfigFileDefaultLocations []string = []string{ diff --git a/cmd/config_serve_tcp.go b/cmd/config_serve_tcp.go index b85d01a..bfc4c49 100644 --- a/cmd/config_serve_tcp.go +++ b/cmd/config_serve_tcp.go @@ -22,8 +22,8 @@ type TCPListenerFactory struct { func parseTCPListenerFactory(c JobParsingContext, i map[string]interface{}) (*TCPListenerFactory, error) { var in struct { - Address string - TLS map[string]interface{} + Address string + TLS map[string]interface{} } if err := mapstructure.Decode(i, &in); err != nil { return nil, errors.Wrap(err, "mapstructure error") @@ -56,7 +56,7 @@ func parseTCPListenerFactory(c JobParsingContext, i map[string]interface{}) (*TC lf.clientCA, err = tlsconf.ParseCAFile(in.CA) if err != nil { - return errors.Wrap(err,"cannot parse ca file") + return errors.Wrap(err, "cannot parse ca file") } lf.serverCert, err = tls.LoadX509KeyPair(in.Cert, in.Key) diff --git a/cmd/config_test.go b/cmd/config_test.go index 253e9fc..f53453a 100644 --- a/cmd/config_test.go +++ b/cmd/config_test.go @@ -136,7 +136,7 @@ func TestDatasetMapFilter(t *testing.T) { expectMapping(map1, "q/r", "root4/1/2/r") map2 := map[string]string{ // identity mapping - "<":"", + "<": "", } expectMapping(map2, "foo/bar", "foo/bar") @@ -280,4 +280,4 @@ func TestDatasetMapFilter_Invert(t *testing.T) { expectMapping(inv, "foo/bar", "", false, true) expectMapping(inv, "foo/bar/bee", "bee", false, false) -} \ No newline at end of file +} diff --git a/cmd/endpoint/endpoint.go b/cmd/endpoint/endpoint.go index 927cdd6..bd03a6a 100644 --- a/cmd/endpoint/endpoint.go +++ b/cmd/endpoint/endpoint.go @@ -2,20 +2,20 @@ package endpoint import ( - "github.com/zrepl/zrepl/replication/pdu" - "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/zfs" - "io" - "github.com/pkg/errors" - "github.com/golang/protobuf/proto" "bytes" "context" + "github.com/golang/protobuf/proto" + "github.com/pkg/errors" + "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/replication" + "github.com/zrepl/zrepl/replication/pdu" + "github.com/zrepl/zrepl/zfs" + "io" ) // Sender implements replication.ReplicationEndpoint for a sending side type Sender struct { - FSFilter zfs.DatasetFilter + FSFilter zfs.DatasetFilter FilesystemVersionFilter zfs.FilesystemVersionFilter } @@ -87,16 +87,16 @@ type FSFilter interface { // FIXME: can we get away without error types here? type FSMap interface { FSFilter - Map(path *zfs.DatasetPath) (*zfs.DatasetPath,error) - Invert() (FSMap,error) - AsFilter() (FSFilter) + Map(path *zfs.DatasetPath) (*zfs.DatasetPath, error) + Invert() (FSMap, error) + AsFilter() FSFilter } // Receiver implements replication.ReplicationEndpoint for a receiving side type Receiver struct { fsmapInv FSMap - fsmap FSMap - fsvf zfs.FilesystemVersionFilter + fsmap FSMap + fsvf zfs.FilesystemVersionFilter } func NewReceiver(fsmap FSMap, fsvf zfs.FilesystemVersionFilter) (*Receiver, error) { @@ -219,12 +219,11 @@ func (e *Receiver) Receive(ctx context.Context, req *pdu.ReceiveReq, sendStream // RPC STUBS // =-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= - const ( - RPCListFilesystems = "ListFilesystems" + RPCListFilesystems = "ListFilesystems" RPCListFilesystemVersions = "ListFilesystemVersions" - RPCReceive = "Receive" - RPCSend = "Send" + RPCReceive = "Receive" + RPCSend = "Send" ) // Remote implements an endpoint stub that uses streamrpc as a transport. @@ -295,13 +294,13 @@ func (s Remote) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.Read var res pdu.SendRes if err := proto.Unmarshal(rb.Bytes(), &res); err != nil { rs.Close() - return nil, nil, err + return nil, nil, err } // FIXME make sure the consumer will read the reader until the end... return &res, rs, nil } -func (s Remote) Receive(ctx context.Context, r *pdu.ReceiveReq, sendStream io.ReadCloser) (error) { +func (s Remote) Receive(ctx context.Context, r *pdu.ReceiveReq, sendStream io.ReadCloser) error { defer sendStream.Close() b, err := proto.Marshal(r) if err != nil { @@ -414,6 +413,6 @@ func (a *Handler) Handle(ctx context.Context, endpoint string, reqStructured *by return bytes.NewBuffer(b), nil, err } - Err: +Err: return nil, nil, errors.New("no handler for given endpoint") } diff --git a/cmd/tlsconf/tlsconf.go b/cmd/tlsconf/tlsconf.go index 77372f8..cf47968 100644 --- a/cmd/tlsconf/tlsconf.go +++ b/cmd/tlsconf/tlsconf.go @@ -43,7 +43,7 @@ func NewClientAuthListener( } tlsConf := tls.Config{ - Certificates: []tls.Certificate{serverCert}, + Certificates: []tls.Certificate{serverCert}, ClientCAs: ca, ClientAuth: tls.RequireAndVerifyClientCert, PreferServerCipherSuites: true, @@ -114,8 +114,8 @@ func ClientAuthClient(serverName string, rootCA *x509.CertPool, clientCert tls.C tlsConfig := &tls.Config{ Certificates: []tls.Certificate{clientCert}, RootCAs: rootCA, - ServerName: serverName, + ServerName: serverName, } tlsConfig.BuildNameToCertificate() return tlsConfig, nil -} \ No newline at end of file +} diff --git a/logger/nulllogger.go b/logger/nulllogger.go index 3233dfe..ec954f7 100644 --- a/logger/nulllogger.go +++ b/logger/nulllogger.go @@ -1,7 +1,6 @@ package logger - -type nullLogger struct {} +type nullLogger struct{} var _ Logger = nullLogger{} @@ -9,14 +8,13 @@ func NewNullLogger() Logger { return nullLogger{} } -func (n nullLogger) WithOutlet(outlet Outlet, level Level) Logger { return n } +func (n nullLogger) WithOutlet(outlet Outlet, level Level) Logger { return n } func (n nullLogger) ReplaceField(field string, val interface{}) Logger { return n } -func (n nullLogger) WithField(field string, val interface{}) Logger { return n } -func (n nullLogger) WithFields(fields Fields) Logger { return n } -func (n nullLogger) WithError(err error) Logger { return n } -func (nullLogger) Debug(msg string) {} -func (nullLogger) Info(msg string) {} -func (nullLogger) Warn(msg string) {} -func (nullLogger) Error(msg string) {} -func (nullLogger) Printf(format string, args ...interface{}) {} - +func (n nullLogger) WithField(field string, val interface{}) Logger { return n } +func (n nullLogger) WithFields(fields Fields) Logger { return n } +func (n nullLogger) WithError(err error) Logger { return n } +func (nullLogger) Debug(msg string) {} +func (nullLogger) Info(msg string) {} +func (nullLogger) Warn(msg string) {} +func (nullLogger) Error(msg string) {} +func (nullLogger) Printf(format string, args ...interface{}) {} diff --git a/replication/context.go b/replication/context.go index 7b4b0e9..7e43981 100644 --- a/replication/context.go +++ b/replication/context.go @@ -1,8 +1,8 @@ package replication import ( - "github.com/zrepl/zrepl/logger" "context" + "github.com/zrepl/zrepl/logger" "github.com/zrepl/zrepl/replication/fsrep" ) @@ -27,4 +27,3 @@ func getLogger(ctx context.Context) Logger { } return l } - diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index d07fd3d..d542456 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -12,8 +12,8 @@ import ( "sync" "time" - "github.com/zrepl/zrepl/replication/pdu" "github.com/zrepl/zrepl/logger" + "github.com/zrepl/zrepl/replication/pdu" ) type contextKey int @@ -51,18 +51,17 @@ type StepReport struct { } type Report struct { - Filesystem string - Status string - Problem string - Completed,Pending []*StepReport + Filesystem string + Status string + Problem string + Completed, Pending []*StepReport } - //go:generate stringer -type=State type State uint const ( - Ready State = 1 << iota + Ready State = 1 << iota RetryWait PermanentError Completed @@ -123,7 +122,7 @@ func (b *ReplicationBuilder) Done() (r *Replication) { } else { b.r.state = Completed } - r = b.r + r = b.r b.r = nil return r } @@ -136,12 +135,11 @@ func NewReplicationWithPermanentError(fs string, err error) *Replication { } } - //go:generate stringer -type=StepState type StepState uint const ( - StepReady StepState = 1 << iota + StepReady StepState = 1 << iota StepRetry StepPermanentError StepCompleted @@ -387,4 +385,3 @@ func (step *ReplicationStep) Report() *StepReport { } return &rep } - diff --git a/replication/internal/queue/queue.go b/replication/internal/queue/queue.go index a187026..a3d7c0c 100644 --- a/replication/internal/queue/queue.go +++ b/replication/internal/queue/queue.go @@ -1,8 +1,8 @@ package queue import ( - "time" "sort" + "time" . "github.com/zrepl/zrepl/replication/fsrep" ) @@ -27,12 +27,12 @@ func NewReplicationQueue() *ReplicationQueue { func (q ReplicationQueue) Len() int { return len(q) } func (q ReplicationQueue) Swap(i, j int) { q[i], q[j] = q[j], q[i] } -type lessmapEntry struct{ +type lessmapEntry struct { prio int - less func(a,b *replicationQueueItem) bool + less func(a, b *replicationQueueItem) bool } -var lessmap = map[State]lessmapEntry { +var lessmap = map[State]lessmapEntry{ Ready: { prio: 0, less: func(a, b *replicationQueueItem) bool { @@ -94,7 +94,7 @@ func (q *ReplicationQueue) GetNext() (done []*Replication, next *ReplicationQueu func (q *ReplicationQueue) Add(fsr *Replication) { *q = append(*q, &replicationQueueItem{ - fsr: fsr, + fsr: fsr, state: fsr.State(), }) } diff --git a/replication/mainfsm.go b/replication/mainfsm.go index ea1308f..f9a0dcb 100644 --- a/replication/mainfsm.go +++ b/replication/mainfsm.go @@ -10,17 +10,17 @@ import ( "sync" "time" - "github.com/zrepl/zrepl/replication/pdu" "github.com/zrepl/zrepl/replication/fsrep" - . "github.com/zrepl/zrepl/replication/internal/queue" . "github.com/zrepl/zrepl/replication/internal/diff" + . "github.com/zrepl/zrepl/replication/internal/queue" + "github.com/zrepl/zrepl/replication/pdu" ) //go:generate stringer -type=State type State uint const ( - Planning State = 1 << iota + Planning State = 1 << iota PlanningError Working WorkingWait @@ -77,7 +77,6 @@ type Report struct { Active *fsrep.Report } - func NewReplication() *Replication { r := Replication{ state: Planning, @@ -101,7 +100,6 @@ type Receiver interface { fsrep.Receiver } - type FilteredError struct{ fs string } func NewFilteredError(fs string) *FilteredError { @@ -110,7 +108,6 @@ func NewFilteredError(fs string) *FilteredError { func (f FilteredError) Error() string { return "endpoint does not allow access to filesystem " + f.fs } - type updater func(func(*Replication)) (newState State) type state func(ctx context.Context, sender Sender, receiver Receiver, u updater) state @@ -381,4 +378,3 @@ func (r *Replication) Report() *Report { return &rep } - diff --git a/replication/pdu/pdu_extras.go b/replication/pdu/pdu_extras.go index 19e27e8..b009d3b 100644 --- a/replication/pdu/pdu_extras.go +++ b/replication/pdu/pdu_extras.go @@ -33,11 +33,11 @@ func FilesystemVersionFromZFS(fsv zfs.FilesystemVersion) *FilesystemVersion { panic("unknown fsv.Type: " + fsv.Type) } return &FilesystemVersion{ - Type: t, - Name: fsv.Name, - Guid: fsv.Guid, + Type: t, + Name: fsv.Name, + Guid: fsv.Guid, CreateTXG: fsv.CreateTXG, - Creation: fsv.Creation.Format(time.RFC3339), + Creation: fsv.Creation.Format(time.RFC3339), } } @@ -64,10 +64,10 @@ func (v *FilesystemVersion) ZFSFilesystemVersion() *zfs.FilesystemVersion { } } return &zfs.FilesystemVersion{ - Type: v.Type.ZFSVersionType(), - Name: v.Name, - Guid: v.Guid, + Type: v.Type.ZFSVersionType(), + Name: v.Name, + Guid: v.Guid, CreateTXG: v.CreateTXG, - Creation: ct, + Creation: ct, } } diff --git a/replication/pdu/pdu_test.go b/replication/pdu/pdu_test.go index 86e2b49..6a1d596 100644 --- a/replication/pdu/pdu_test.go +++ b/replication/pdu/pdu_test.go @@ -1,15 +1,15 @@ package pdu import ( - "testing" "github.com/stretchr/testify/assert" + "testing" ) func TestFilesystemVersion_RelName(t *testing.T) { type TestCase struct { - In FilesystemVersion - Out string + In FilesystemVersion + Out string Panic bool } @@ -56,7 +56,7 @@ func TestFilesystemVersion_ZFSFilesystemVersion(t *testing.T) { emptyZFS := empty.ZFSFilesystemVersion() assert.Zero(t, emptyZFS.Creation) - dateInvalid := &FilesystemVersion{Creation:"foobar"} + dateInvalid := &FilesystemVersion{Creation: "foobar"} assert.Panics(t, func() { dateInvalid.ZFSFilesystemVersion() }) diff --git a/replication/policy.go b/replication/policy.go index ea9b190..83398af 100644 --- a/replication/policy.go +++ b/replication/policy.go @@ -9,4 +9,3 @@ const ( ) const DEFAULT_INITIAL_REPL_POLICY = InitialReplPolicyMostRecent - diff --git a/util/contextflexibletimeout.go b/util/contextflexibletimeout.go index dc84549..422318d 100644 --- a/util/contextflexibletimeout.go +++ b/util/contextflexibletimeout.go @@ -2,18 +2,18 @@ package util import ( "context" - "time" "sync" + "time" ) type contextWithOptionalDeadline struct { context.Context - m sync.Mutex + m sync.Mutex deadline time.Time done chan struct{} - err error + err error } func (c *contextWithOptionalDeadline) Deadline() (deadline time.Time, ok bool) { @@ -28,7 +28,7 @@ func (c *contextWithOptionalDeadline) Err() error { return c.err } -func (c *contextWithOptionalDeadline) Done() (<-chan struct{}) { +func (c *contextWithOptionalDeadline) Done() <-chan struct{} { return c.done } @@ -37,8 +37,8 @@ func ContextWithOptionalDeadline(pctx context.Context) (ctx context.Context, enf // mctx can only be cancelled by cancelMctx, not by a potential cancel of pctx rctx := &contextWithOptionalDeadline{ Context: pctx, - done: make(chan struct{}), - err: nil, + done: make(chan struct{}), + err: nil, } enforceDeadline = func(deadline time.Time) { diff --git a/util/contextflexibletimeout_test.go b/util/contextflexibletimeout_test.go index 77031d5..e6a1128 100644 --- a/util/contextflexibletimeout_test.go +++ b/util/contextflexibletimeout_test.go @@ -1,11 +1,11 @@ package util import ( - "testing" "context" - "time" - "github.com/stretchr/testify/require" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "testing" + "time" ) func TestContextWithOptionalDeadline(t *testing.T) { @@ -18,14 +18,14 @@ func TestContextWithOptionalDeadline(t *testing.T) { var cancellationError error go func() { select { - case <- cctx.Done(): + case <-cctx.Done(): receivedCancellation = time.Now() cancellationError = cctx.Err() - case <- time.After(600*time.Millisecond): + case <-time.After(600 * time.Millisecond): t.Fatalf("should have been cancelled by deadline") } }() - time.Sleep(100*time.Millisecond) + time.Sleep(100 * time.Millisecond) if !receivedCancellation.IsZero() { t.Fatalf("no enforcement means no cancellation") } @@ -33,11 +33,11 @@ func TestContextWithOptionalDeadline(t *testing.T) { dl, ok := cctx.Deadline() require.False(t, ok) require.Zero(t, dl) - enforceDeadline(begin.Add(200*time.Millisecond)) + enforceDeadline(begin.Add(200 * time.Millisecond)) // second call must be ignored, i.e. we expect the deadline to be at begin+200ms, not begin+400ms - enforceDeadline(begin.Add(400*time.Millisecond)) + enforceDeadline(begin.Add(400 * time.Millisecond)) - time.Sleep(300*time.Millisecond) // 100ms margin for scheduler + time.Sleep(300 * time.Millisecond) // 100ms margin for scheduler if receivedCancellation.Sub(begin) > 250*time.Millisecond { t.Fatalf("cancellation is beyond acceptable scheduler latency") } @@ -47,7 +47,7 @@ func TestContextWithOptionalDeadline(t *testing.T) { func TestContextWithOptionalDeadlineNegativeDeadline(t *testing.T) { ctx := context.Background() cctx, enforceDeadline := ContextWithOptionalDeadline(ctx) - enforceDeadline(time.Now().Add(-10*time.Second)) + enforceDeadline(time.Now().Add(-10 * time.Second)) select { case <-cctx.Done(): default: @@ -62,10 +62,10 @@ func TestContextWithOptionalDeadlineParentCancellation(t *testing.T) { // 0 ms start := time.Now() - enforceDeadline(start.Add(400*time.Millisecond)) - time.Sleep(100*time.Millisecond) - cancel() // cancel @ ~100ms - time.Sleep(100*time.Millisecond) // give 100ms time to propagate cancel + enforceDeadline(start.Add(400 * time.Millisecond)) + time.Sleep(100 * time.Millisecond) + cancel() // cancel @ ~100ms + time.Sleep(100 * time.Millisecond) // give 100ms time to propagate cancel // @ ~200ms select { case <-cctx.Done(): @@ -81,4 +81,4 @@ func TestContextWithOptionalDeadlineValue(t *testing.T) { pctx := context.WithValue(context.Background(), "key", "value") cctx, _ := ContextWithOptionalDeadline(pctx) assert.Equal(t, "value", cctx.Value("key")) -} \ No newline at end of file +} diff --git a/util/io.go b/util/io.go index 857df26..969e9c7 100644 --- a/util/io.go +++ b/util/io.go @@ -14,7 +14,7 @@ type NetConnLogger struct { func NewNetConnLogger(conn net.Conn, readlog, writelog string) (l *NetConnLogger, err error) { l = &NetConnLogger{ - Conn: conn, + Conn: conn, } flags := os.O_CREATE | os.O_WRONLY if readlog != "" { diff --git a/zfs/diff.go b/zfs/diff.go index c3564a1..e7a7e0c 100644 --- a/zfs/diff.go +++ b/zfs/diff.go @@ -5,9 +5,9 @@ import ( "crypto/sha512" "encoding/hex" "fmt" + "io" "os/exec" "sort" - "io" ) type fsbyCreateTXG []FilesystemVersion @@ -255,7 +255,7 @@ func ZFSIsPlaceholderFilesystem(p *DatasetPath) (isPlaceholder bool, err error) } else if err != nil { return false, err } - isPlaceholder, _ = IsPlaceholder(p, props.Get(ZREPL_PLACEHOLDER_PROPERTY_NAME)) + isPlaceholder, _ = IsPlaceholder(p, props.Get(ZREPL_PLACEHOLDER_PROPERTY_NAME)) return } diff --git a/zfs/mapping.go b/zfs/mapping.go index 8fcbd9a..56a85b3 100644 --- a/zfs/mapping.go +++ b/zfs/mapping.go @@ -39,7 +39,7 @@ func ZFSListMappingProperties(filter DatasetFilter, properties []string) (datase panic("properties must not contain 'name'") } } - newProps := make([]string, len(properties) + 1) + newProps := make([]string, len(properties)+1) newProps[0] = "name" copy(newProps[1:], properties) properties = newProps @@ -69,7 +69,7 @@ func ZFSListMappingProperties(filter DatasetFilter, properties []string) (datase } if pass { datasets = append(datasets, ZFSListMappingPropertiesResult{ - Path: path, + Path: path, Fields: r.Fields[1:], }) } @@ -78,5 +78,3 @@ func ZFSListMappingProperties(filter DatasetFilter, properties []string) (datase return } - - diff --git a/zfs/versions.go b/zfs/versions.go index 552af35..4bf2d40 100644 --- a/zfs/versions.go +++ b/zfs/versions.go @@ -6,10 +6,10 @@ import ( "errors" "fmt" "github.com/prometheus/client_golang/prometheus" + "io" "strconv" "strings" "time" - "io" ) type VersionType string From 861e5f831328bf896253c745ffc90164334ecdfe Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 25 Aug 2018 22:15:37 +0200 Subject: [PATCH 030/167] special logging fields: from now on only 'job', 'task', 'subsystem' --- cmd/autosnap.go | 5 +++-- cmd/config_job_pull.go | 6 +++--- cmd/config_job_source.go | 4 ++-- cmd/logging_formatters.go | 24 +++--------------------- cmd/prune.go | 4 ++-- 5 files changed, 13 insertions(+), 30 deletions(-) diff --git a/cmd/autosnap.go b/cmd/autosnap.go index 776c387..fe063a7 100644 --- a/cmd/autosnap.go +++ b/cmd/autosnap.go @@ -48,7 +48,7 @@ func (a *IntervalAutosnap) findSyncPoint(fss []*zfs.DatasetPath) (syncPoint time a.task.Log().Debug("examine filesystem state") for _, d := range fss { - l := a.task.Log().WithField(logFSField, d.ToString()) + l := a.task.Log().WithField("fs", d.ToString()) fsvs, err := zfs.ZFSListFilesystemVersions(d, NewPrefixFilter(a.Prefix)) if err != nil { @@ -175,7 +175,8 @@ func (a *IntervalAutosnap) doSnapshots(didSnaps chan struct{}) { suffix := time.Now().In(time.UTC).Format("20060102_150405_000") snapname := fmt.Sprintf("%s%s", a.Prefix, suffix) - l := a.task.Log().WithField(logFSField, d.ToString()). + l := a.task.Log(). + WithField("fs", d.ToString()). WithField("snapname", snapname) l.Info("create snapshot") diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index df6dab1..871166c 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -182,9 +182,9 @@ func (j *PullJob) doRun(ctx context.Context) { return } - ctx = replication.WithLogger(ctx, replicationLogAdaptor{j.task.Log().WithField("subsystem", "replication")}) - ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{j.task.Log().WithField("subsystem", "rpc.protocol")}) - ctx = endpoint.WithLogger(ctx, j.task.Log().WithField("subsystem", "rpc.endpoint")) + ctx = replication.WithLogger(ctx, replicationLogAdaptor{j.task.Log().WithField(logSubsysField, "replication")}) + ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{j.task.Log().WithField(logSubsysField, "rpc.protocol")}) + ctx = endpoint.WithLogger(ctx, j.task.Log().WithField(logSubsysField, "rpc.endpoint")) j.rep = replication.NewReplication() j.rep.Drive(ctx, sender, puller) diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index c79d3c9..200b01d 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -209,8 +209,8 @@ func (j *SourceJob) handleConnection(conn net.Conn, task *Task) { senderEP := endpoint.NewSender(j.Filesystems, NewPrefixFilter(j.SnapshotPrefix)) ctx := context.Background() - ctx = context.WithValue(ctx, contextKeyLog, task.Log().WithField("subsystem", "rpc.endpoint")) - ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{task.Log().WithField("subsystem", "rpc.protocol")}) + ctx = context.WithValue(ctx, contextKeyLog, task.Log().WithField(logSubsysField, "rpc.endpoint")) + ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{task.Log().WithField(logSubsysField, "rpc.protocol")}) handler := endpoint.NewHandler(senderEP) if err := streamrpc.ServeConn(ctx, conn, STREAMRPC_CONFIG, handler.Handle); err != nil { task.Log().WithError(err).Error("error serving connection") diff --git a/cmd/logging_formatters.go b/cmd/logging_formatters.go index 02795e8..444f3d3 100644 --- a/cmd/logging_formatters.go +++ b/cmd/logging_formatters.go @@ -22,14 +22,9 @@ const ( ) const ( - logJobField string = "job" - logTaskField string = "task" - logFSField string = "filesystem" - logSubsysField string = "subsystem" - logMapFromField string = "map_from" - logMapToField string = "map_to" - logIncFromField string = "inc_from" - logIncToField string = "inc_to" + logJobField string = "job" + logTaskField string = "task" + logSubsysField string = "subsystem" ) type NoFormatter struct{} @@ -91,19 +86,6 @@ func (f *HumanFormatter) Format(e *logger.Entry) (out []byte, err error) { break } } - // even more prefix fields - //mapFrom, mapFromOk := e.Fields[logMapFromField].(string) - //mapTo, mapToOk := e.Fields[logMapToField].(string) - //if mapFromOk && mapToOk && !f.ignored(logMapFromField) && !f.ignored(logMapToField) { - // fmt.Fprintf(&line, "[%s => %s]", mapFrom, mapTo) - // prefixed[logMapFromField], prefixed[logMapToField] = true, true - //} - //incFrom, incFromOk := e.Fields[logIncFromField].(string) - //incTo, incToOk := e.Fields[logIncToField].(string) - //if incFromOk && incToOk && !f.ignored(logIncFromField) && !f.ignored(logMapToField) { - // fmt.Fprintf(&line, "[%s => %s]", incFrom, incTo) - // prefixed[logIncFromField], prefixed[logIncToField] = true, true - //} if line.Len() > 0 { fmt.Fprint(&line, ": ") diff --git a/cmd/prune.go b/cmd/prune.go index 73ea7e1..82d4425 100644 --- a/cmd/prune.go +++ b/cmd/prune.go @@ -41,7 +41,7 @@ func (p *Pruner) filterFilesystems() (filesystems []*zfs.DatasetPath, stop bool) func (p *Pruner) filterVersions(fs *zfs.DatasetPath) (fsversions []zfs.FilesystemVersion, stop bool) { p.task.Enter("filter_versions") defer p.task.Finish() - log := p.task.Log().WithField(logFSField, fs.ToString()) + log := p.task.Log().WithField("fs", fs.ToString()) filter := NewPrefixFilter(p.SnapshotPrefix) fsversions, err := zfs.ZFSListFilesystemVersions(fs, filter) @@ -59,7 +59,7 @@ func (p *Pruner) filterVersions(fs *zfs.DatasetPath) (fsversions []zfs.Filesyste func (p *Pruner) pruneFilesystem(fs *zfs.DatasetPath) (r PruneResult, valid bool) { p.task.Enter("prune_fs") defer p.task.Finish() - log := p.task.Log().WithField(logFSField, fs.ToString()) + log := p.task.Log().WithField("fs", fs.ToString()) fsversions, stop := p.filterVersions(fs) if stop { From 88de8ba8bb00ec805c715027e581b50f3f43b265 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 25 Aug 2018 22:23:47 +0200 Subject: [PATCH 031/167] initial repl policy: get rid of unimplemented options --- cmd/config_job_local.go | 5 ----- cmd/config_job_pull.go | 7 ------ cmd/config_parse.go | 27 +----------------------- cmd/sampleconf/localbackup/host1.yml | 1 - cmd/sampleconf/pullbackup/backuphost.yml | 1 - docs/configuration/jobs.rst | 8 ++----- docs/tutorial.rst | 1 - replication/mainfsm.go | 3 +-- replication/policy.go | 11 ---------- 9 files changed, 4 insertions(+), 60 deletions(-) delete mode 100644 replication/policy.go diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index fce3708..7fa88ae 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -17,7 +17,6 @@ type LocalJob struct { Mapping *DatasetMapFilter SnapshotPrefix string Interval time.Duration - InitialReplPolicy replication.InitialReplPolicy PruneLHS PrunePolicy PruneRHS PrunePolicy Debug JobDebugSettings @@ -60,10 +59,6 @@ func parseLocalJob(c JobParsingContext, name string, i map[string]interface{}) ( return } - if j.InitialReplPolicy, err = parseInitialReplPolicy(asMap.InitialReplPolicy, replication.DEFAULT_INITIAL_REPL_POLICY); err != nil { - return - } - if j.PruneLHS, err = parsePrunePolicy(asMap.PruneLHS, true); err != nil { err = errors.Wrap(err, "cannot parse 'prune_lhs'") return diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index 871166c..e4aba5c 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -23,7 +23,6 @@ type PullJob struct { // constructed from mapping during parsing pruneFilter *DatasetMapFilter SnapshotPrefix string - InitialReplPolicy replication.InitialReplPolicy Prune PrunePolicy Debug JobDebugSettings @@ -72,12 +71,6 @@ func parsePullJob(c JobParsingContext, name string, i map[string]interface{}) (j return nil, err } - j.InitialReplPolicy, err = parseInitialReplPolicy(asMap.InitialReplPolicy, replication.DEFAULT_INITIAL_REPL_POLICY) - if err != nil { - err = errors.Wrap(err, "cannot parse 'initial_repl_policy'") - return - } - if j.SnapshotPrefix, err = parseSnapshotPrefix(asMap.SnapshotPrefix); err != nil { return } diff --git a/cmd/config_parse.go b/cmd/config_parse.go index 573436a..3a5b1eb 100644 --- a/cmd/config_parse.go +++ b/cmd/config_parse.go @@ -4,11 +4,10 @@ import ( "io/ioutil" "fmt" - yaml "github.com/go-yaml/yaml" + "github.com/go-yaml/yaml" "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/replication" "os" "regexp" "strconv" @@ -228,30 +227,6 @@ func parseConnect(i map[string]interface{}) (c streamrpc.Connecter, err error) { } -func parseInitialReplPolicy(v interface{}, defaultPolicy replication.InitialReplPolicy) (p replication.InitialReplPolicy, err error) { - s, ok := v.(string) - if !ok { - goto err - } - - switch { - case s == "": - p = defaultPolicy - case s == "most_recent": - p = replication.InitialReplPolicyMostRecent - case s == "all": - p = replication.InitialReplPolicyAll - default: - goto err - } - - return - -err: - err = errors.New(fmt.Sprintf("expected InitialReplPolicy, got %#v", v)) - return -} - func parsePrunePolicy(v map[string]interface{}, willSeeBookmarks bool) (p PrunePolicy, err error) { policyName, err := extractStringField(v, "policy", true) diff --git a/cmd/sampleconf/localbackup/host1.yml b/cmd/sampleconf/localbackup/host1.yml index e48cf02..5ef4f65 100644 --- a/cmd/sampleconf/localbackup/host1.yml +++ b/cmd/sampleconf/localbackup/host1.yml @@ -12,7 +12,6 @@ jobs: } snapshot_prefix: zrepl_ interval: 10m - initial_repl_policy: most_recent # keep one hour of 10m interval snapshots of filesystems matched by # the left-hand-side of the mapping diff --git a/cmd/sampleconf/pullbackup/backuphost.yml b/cmd/sampleconf/pullbackup/backuphost.yml index f6871d0..206e947 100644 --- a/cmd/sampleconf/pullbackup/backuphost.yml +++ b/cmd/sampleconf/pullbackup/backuphost.yml @@ -17,7 +17,6 @@ jobs: mapping: { "<":"storage/backups/zrepl/pull/prod1.example.com" } - initial_repl_policy: most_recent # follow a grandfathering scheme for filesystems on the right-hand-side of the mapping snapshot_prefix: zrepl_ diff --git a/docs/configuration/jobs.rst b/docs/configuration/jobs.rst index b921e79..1854bd5 100644 --- a/docs/configuration/jobs.rst +++ b/docs/configuration/jobs.rst @@ -88,8 +88,6 @@ Example: :sampleconf:`pullbackup/backuphost.yml` - Interval between pull attempts * - ``mapping`` - |mapping| for remote to local filesystems - * - ``initial_repl_policy`` - - default = ``most_recent``, initial replication policy * - ``snapshot_prefix`` - prefix snapshots must match to be considered for replication & pruning * - ``prune`` @@ -103,7 +101,7 @@ Example: :sampleconf:`pullbackup/backuphost.yml` #. Only snapshots with prefix ``snapshot_prefix`` are replicated. #. If possible, incremental replication takes place. - #. If the local target filesystem does not exist, ``initial_repl_policy`` is used. + #. If the local target filesystem does not exist, the most recent snapshot is sent fully (non-incremental). #. On conflicts, an error is logged but replication of other filesystems with mapping continues. #. The ``prune`` policy is evaluated for all *target filesystems* @@ -135,8 +133,6 @@ Example: :sampleconf:`localbackup/host1.yml` - prefix for ZFS snapshots taken by this job * - ``interval`` - snapshotting & replication interval - * - ``initial_repl_policy`` - - default = ``most_recent``, initial replication policy * - ``prune_lhs`` - pruning policy on left-hand-side (source) * - ``prune_rhs`` @@ -151,7 +147,7 @@ Example: :sampleconf:`localbackup/host1.yml` #. Only snapshots with prefix ``snapshot_prefix`` are replicated. #. If possible, incremental replication takes place. - #. If the *target filesystem* does not exist, ``initial_repl_policy`` is used. + #. If the *target filesystem* does not exist, the most recent snapshot is sent fully (non-incremental). #. On conflicts, an error is logged but replication of other *mapped filesystems* continues. #. The ``prune_lhs`` policy is triggered for all *mapped filesystems* diff --git a/docs/tutorial.rst b/docs/tutorial.rst index 40192d8..fa769ed 100644 --- a/docs/tutorial.rst +++ b/docs/tutorial.rst @@ -75,7 +75,6 @@ We define a **pull job** named ``pull_app-srv`` in the |mainconfig| on host ``ba mapping: { "<":"storage/zrepl/pull/app-srv" } - initial_repl_policy: most_recent snapshot_prefix: zrepl_pull_backup_ prune: policy: grid diff --git a/replication/mainfsm.go b/replication/mainfsm.go index f9a0dcb..68fefc4 100644 --- a/replication/mainfsm.go +++ b/replication/mainfsm.go @@ -151,8 +151,7 @@ func (r *Replication) Drive(ctx context.Context, sender Sender, receiver Receive func resolveConflict(conflict error) (path []*pdu.FilesystemVersion, msg string) { if noCommonAncestor, ok := conflict.(*ConflictNoCommonAncestor); ok { if len(noCommonAncestor.SortedReceiverVersions) == 0 { - // FIXME hard-coded replication policy: most recent - // snapshot as source + // TODO this is hard-coded replication policy: most recent snapshot as source var mostRecentSnap *pdu.FilesystemVersion for n := len(noCommonAncestor.SortedSenderVersions) - 1; n >= 0; n-- { if noCommonAncestor.SortedSenderVersions[n].Type == pdu.FilesystemVersion_Snapshot { diff --git a/replication/policy.go b/replication/policy.go deleted file mode 100644 index 83398af..0000000 --- a/replication/policy.go +++ /dev/null @@ -1,11 +0,0 @@ -package replication - -// FIXME: Leftovers from previous versions, not used currently -type InitialReplPolicy string - -const ( - InitialReplPolicyMostRecent InitialReplPolicy = "most_recent" - InitialReplPolicyAll InitialReplPolicy = "all" -) - -const DEFAULT_INITIAL_REPL_POLICY = InitialReplPolicyMostRecent From 71203ab325ea4ce55e2f6889fce995d606f5a9af Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 25 Aug 2018 22:30:16 +0200 Subject: [PATCH 032/167] move various timeouts to package-level variables --- cmd/config_serve_tcp.go | 4 +++- replication/fsrep/fsfsm.go | 4 +++- replication/mainfsm.go | 12 ++++++------ 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/cmd/config_serve_tcp.go b/cmd/config_serve_tcp.go index bfc4c49..c3c1785 100644 --- a/cmd/config_serve_tcp.go +++ b/cmd/config_serve_tcp.go @@ -75,12 +75,14 @@ func parseTCPListenerFactory(c JobParsingContext, i map[string]interface{}) (*TC return lf, nil } +var TCPListenerHandshakeTimeout = 10 * time.Second // FIXME make configurable + func (f *TCPListenerFactory) Listen() (net.Listener, error) { l, err := net.Listen("tcp", f.Address) if !f.tls || err != nil { return l, err } - tl := tlsconf.NewClientAuthListener(l, f.clientCA, f.serverCert, f.clientCommonName, 10*time.Second) + tl := tlsconf.NewClientAuthListener(l, f.clientCA, f.serverCert, f.clientCommonName, TCPListenerHandshakeTimeout) return tl, nil } diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index d542456..025cbb1 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -199,6 +199,8 @@ type updater func(func(fsr *Replication)) State type state func(ctx context.Context, sender Sender, receiver Receiver, u updater) state +var RetrySleepDuration = 10 * time.Second // FIXME make configurable + func stateReady(ctx context.Context, sender Sender, receiver Receiver, u updater) state { var current *ReplicationStep @@ -226,7 +228,7 @@ func stateReady(ctx context.Context, sender Sender, receiver Receiver, u updater f.state = Completed } case StepRetry: - f.retryWaitUntil = time.Now().Add(10 * time.Second) // FIXME make configurable + f.retryWaitUntil = time.Now().Add(RetrySleepDuration) f.state = RetryWait case StepPermanentError: f.state = PermanentError diff --git a/replication/mainfsm.go b/replication/mainfsm.go index 68fefc4..4a5abcf 100644 --- a/replication/mainfsm.go +++ b/replication/mainfsm.go @@ -271,12 +271,13 @@ func statePlanning(ctx context.Context, sender Sender, receiver Receiver, u upda }).rsf() } +var RetrySleepDuration = 10 * time.Second // FIXME make constant onfigurable + func statePlanningError(ctx context.Context, sender Sender, receiver Receiver, u updater) state { - sleepTime := 10 * time.Second u(func(r *Replication) { - r.sleepUntil = time.Now().Add(sleepTime) + r.sleepUntil = time.Now().Add(RetrySleepDuration) }) - t := time.NewTimer(sleepTime) // FIXME make constant onfigurable + t := time.NewTimer(RetrySleepDuration) defer t.Stop() select { case <-ctx.Done(): @@ -317,11 +318,10 @@ func stateWorking(ctx context.Context, sender Sender, receiver Receiver, u updat } func stateWorkingWait(ctx context.Context, sender Sender, receiver Receiver, u updater) state { - sleepTime := 10 * time.Second u(func(r *Replication) { - r.sleepUntil = time.Now().Add(sleepTime) + r.sleepUntil = time.Now().Add(RetrySleepDuration) }) - t := time.NewTimer(sleepTime) + t := time.NewTimer(RetrySleepDuration) defer t.Stop() select { case <-ctx.Done(): From ea0e3a29e463bb26d83770e6c6c2224b48408f4f Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 25 Aug 2018 22:30:44 +0200 Subject: [PATCH 033/167] fixup 88de8ba8bb00ec805c715027e581b50f3f43b265: gofmt --- cmd/config_job_local.go | 24 ++++++++++++------------ cmd/config_job_pull.go | 8 ++++---- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index 7fa88ae..41f8219 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -13,18 +13,18 @@ import ( ) type LocalJob struct { - Name string - Mapping *DatasetMapFilter - SnapshotPrefix string - Interval time.Duration - PruneLHS PrunePolicy - PruneRHS PrunePolicy - Debug JobDebugSettings - snapperTask *Task - mainTask *Task - handlerTask *Task - pruneRHSTask *Task - pruneLHSTask *Task + Name string + Mapping *DatasetMapFilter + SnapshotPrefix string + Interval time.Duration + PruneLHS PrunePolicy + PruneRHS PrunePolicy + Debug JobDebugSettings + snapperTask *Task + mainTask *Task + handlerTask *Task + pruneRHSTask *Task + pruneLHSTask *Task } func parseLocalJob(c JobParsingContext, name string, i map[string]interface{}) (j *LocalJob, err error) { diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index e4aba5c..233f150 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -21,10 +21,10 @@ type PullJob struct { Interval time.Duration Mapping *DatasetMapFilter // constructed from mapping during parsing - pruneFilter *DatasetMapFilter - SnapshotPrefix string - Prune PrunePolicy - Debug JobDebugSettings + pruneFilter *DatasetMapFilter + SnapshotPrefix string + Prune PrunePolicy + Debug JobDebugSettings task *Task rep *replication.Replication From cf01086df599305cff40c88eb7a300a7a9fadbc3 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 26 Aug 2018 14:35:18 +0200 Subject: [PATCH 034/167] build: pin protoc version and update protobuf + regenerate --- Gopkg.lock | 6 +- Gopkg.toml | 4 + Makefile | 14 +- build.Dockerfile | 7 +- lazy.sh | 3 +- replication/pdu/pdu.pb.go | 399 +++++++++++++++++++++++++++++--------- 6 files changed, 332 insertions(+), 101 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index 0713a08..4868440 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -50,12 +50,12 @@ revision = "eb3733d160e74a9c7e442f435eb3bea458e1d19f" [[projects]] - digest = "1:bcb38c8fc9b21bb8682ce2d605a7d4aeb618abc7f827e3ac0b27c0371fdb23fb" + digest = "1:3dd078fda7500c341bc26cfbc6c6a34614f295a2457149fc1045cab767cbcf18" name = "github.com/golang/protobuf" packages = ["proto"] pruneopts = "" - revision = "925541529c1fa6821df4e44ce2723319eb2be768" - version = "v1.0.0" + revision = "aa810b61a9c79d51363740d207bb46cf8e620ed5" + version = "v1.2.0" [[projects]] branch = "master" diff --git a/Gopkg.toml b/Gopkg.toml index ae6f5d8..81becc1 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -59,3 +59,7 @@ ignored = [ "github.com/inconshreveable/mousetrap" ] [[constraint]] name = "github.com/problame/go-streamrpc" version = "0.1.0" + +[[constraint]] + name = "github.com/golang/protobuf" + version = "1.2.0" diff --git a/Makefile b/Makefile index 619df9c..fd22ce1 100644 --- a/Makefile +++ b/Makefile @@ -3,11 +3,11 @@ ROOT := github.com/zrepl/zrepl SUBPKGS := cmd -SUBPKGS += cmd/replication -SUBPKGS += cmd/replication/fsrep -SUBPKGS += cmd/replication/pdu -SUBPKGS += cmd/replication/internal/queue -SUBPKGS += cmd/replication/internal/diff +SUBPKGS += replication +SUBPKGS += replication/fsrep +SUBPKGS += replication/pdu +SUBPKGS += replication/internal/queue +SUBPKGS += replication/internal/diff SUBPKGS += logger util zfs _TESTPKGS := $(ROOT) $(foreach p,$(SUBPKGS),$(ROOT)/$(p)) @@ -32,12 +32,10 @@ vendordeps: dep ensure -v -vendor-only generate: #not part of the build, must do that manually - protoc -I=cmd/replication/pdu --go_out=cmd/replication/pdu cmd/replication/pdu/pdu.proto + protoc -I=replication/pdu --go_out=replication/pdu replication/pdu/pdu.proto @for pkg in $(_TESTPKGS); do\ go generate "$$pkg" || exit 1; \ done; - # FIXME fix docker build! - build: @echo "INFO: In case of missing dependencies, run 'make vendordeps'" diff --git a/build.Dockerfile b/build.Dockerfile index 47cec34..d7729e5 100644 --- a/build.Dockerfile +++ b/build.Dockerfile @@ -1,7 +1,12 @@ FROM golang:latest RUN apt-get update && apt-get install -y \ - python3-pip + python3-pip \ + unzip + +RUN wget https://github.com/protocolbuffers/protobuf/releases/download/v3.6.1/protoc-3.6.1-linux-x86_64.zip +RUN echo "6003de742ea3fcf703cfec1cd4a3380fd143081a2eb0e559065563496af27807 protoc-3.6.1-linux-x86_64.zip" | sha256sum -c +RUN unzip -d /usr protoc-3.6.1-linux-x86_64.zip ADD lazy.sh /tmp/lazy.sh ADD docs/requirements.txt /tmp/requirements.txt diff --git a/lazy.sh b/lazy.sh index 8d785cb..a8c7316 100755 --- a/lazy.sh +++ b/lazy.sh @@ -29,7 +29,8 @@ builddep() { step "Install build depdencies using 'go get' to \$GOPATH/bin" go get -u golang.org/x/tools/cmd/stringer go get -u github.com/golang/dep/cmd/dep - if ! type stringer || ! type dep; then + go get -u github.com/golang/protobuf/protoc-gen-go + if ! type stringer || ! type dep || ! type protoc-gen-go; then echo "Installed dependencies but can't find them in \$PATH, adjust it to contain \$GOPATH/bin" 1>&2 exit 1 fi diff --git a/replication/pdu/pdu.pb.go b/replication/pdu/pdu.pb.go index d9bed35..a770a17 100644 --- a/replication/pdu/pdu.pb.go +++ b/replication/pdu/pdu.pb.go @@ -1,25 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // source: pdu.proto -/* -Package pdu is a generated protocol buffer package. - -It is generated from these files: - pdu.proto - -It has these top-level messages: - ListFilesystemReq - ListFilesystemRes - Filesystem - ListFilesystemVersionsReq - ListFilesystemVersionsRes - FilesystemVersion - SendReq - Property - SendRes - ReceiveReq - ReceiveRes -*/ package pdu import proto "github.com/golang/protobuf/proto" @@ -57,25 +38,69 @@ func (x FilesystemVersion_VersionType) String() string { return proto.EnumName(FilesystemVersion_VersionType_name, int32(x)) } func (FilesystemVersion_VersionType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor0, []int{5, 0} + return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{5, 0} } type ListFilesystemReq struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ListFilesystemReq) Reset() { *m = ListFilesystemReq{} } -func (m *ListFilesystemReq) String() string { return proto.CompactTextString(m) } -func (*ListFilesystemReq) ProtoMessage() {} -func (*ListFilesystemReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} } +func (m *ListFilesystemReq) Reset() { *m = ListFilesystemReq{} } +func (m *ListFilesystemReq) String() string { return proto.CompactTextString(m) } +func (*ListFilesystemReq) ProtoMessage() {} +func (*ListFilesystemReq) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{0} +} +func (m *ListFilesystemReq) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ListFilesystemReq.Unmarshal(m, b) +} +func (m *ListFilesystemReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ListFilesystemReq.Marshal(b, m, deterministic) +} +func (dst *ListFilesystemReq) XXX_Merge(src proto.Message) { + xxx_messageInfo_ListFilesystemReq.Merge(dst, src) +} +func (m *ListFilesystemReq) XXX_Size() int { + return xxx_messageInfo_ListFilesystemReq.Size(m) +} +func (m *ListFilesystemReq) XXX_DiscardUnknown() { + xxx_messageInfo_ListFilesystemReq.DiscardUnknown(m) +} + +var xxx_messageInfo_ListFilesystemReq proto.InternalMessageInfo type ListFilesystemRes struct { - Filesystems []*Filesystem `protobuf:"bytes,1,rep,name=Filesystems" json:"Filesystems,omitempty"` + Filesystems []*Filesystem `protobuf:"bytes,1,rep,name=Filesystems,proto3" json:"Filesystems,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ListFilesystemRes) Reset() { *m = ListFilesystemRes{} } -func (m *ListFilesystemRes) String() string { return proto.CompactTextString(m) } -func (*ListFilesystemRes) ProtoMessage() {} -func (*ListFilesystemRes) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } +func (m *ListFilesystemRes) Reset() { *m = ListFilesystemRes{} } +func (m *ListFilesystemRes) String() string { return proto.CompactTextString(m) } +func (*ListFilesystemRes) ProtoMessage() {} +func (*ListFilesystemRes) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{1} +} +func (m *ListFilesystemRes) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ListFilesystemRes.Unmarshal(m, b) +} +func (m *ListFilesystemRes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ListFilesystemRes.Marshal(b, m, deterministic) +} +func (dst *ListFilesystemRes) XXX_Merge(src proto.Message) { + xxx_messageInfo_ListFilesystemRes.Merge(dst, src) +} +func (m *ListFilesystemRes) XXX_Size() int { + return xxx_messageInfo_ListFilesystemRes.Size(m) +} +func (m *ListFilesystemRes) XXX_DiscardUnknown() { + xxx_messageInfo_ListFilesystemRes.DiscardUnknown(m) +} + +var xxx_messageInfo_ListFilesystemRes proto.InternalMessageInfo func (m *ListFilesystemRes) GetFilesystems() []*Filesystem { if m != nil { @@ -85,14 +110,36 @@ func (m *ListFilesystemRes) GetFilesystems() []*Filesystem { } type Filesystem struct { - Path string `protobuf:"bytes,1,opt,name=Path" json:"Path,omitempty"` - ResumeToken string `protobuf:"bytes,2,opt,name=ResumeToken" json:"ResumeToken,omitempty"` + Path string `protobuf:"bytes,1,opt,name=Path,proto3" json:"Path,omitempty"` + ResumeToken string `protobuf:"bytes,2,opt,name=ResumeToken,proto3" json:"ResumeToken,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Filesystem) Reset() { *m = Filesystem{} } -func (m *Filesystem) String() string { return proto.CompactTextString(m) } -func (*Filesystem) ProtoMessage() {} -func (*Filesystem) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} } +func (m *Filesystem) Reset() { *m = Filesystem{} } +func (m *Filesystem) String() string { return proto.CompactTextString(m) } +func (*Filesystem) ProtoMessage() {} +func (*Filesystem) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{2} +} +func (m *Filesystem) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_Filesystem.Unmarshal(m, b) +} +func (m *Filesystem) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_Filesystem.Marshal(b, m, deterministic) +} +func (dst *Filesystem) XXX_Merge(src proto.Message) { + xxx_messageInfo_Filesystem.Merge(dst, src) +} +func (m *Filesystem) XXX_Size() int { + return xxx_messageInfo_Filesystem.Size(m) +} +func (m *Filesystem) XXX_DiscardUnknown() { + xxx_messageInfo_Filesystem.DiscardUnknown(m) +} + +var xxx_messageInfo_Filesystem proto.InternalMessageInfo func (m *Filesystem) GetPath() string { if m != nil { @@ -109,13 +156,35 @@ func (m *Filesystem) GetResumeToken() string { } type ListFilesystemVersionsReq struct { - Filesystem string `protobuf:"bytes,1,opt,name=Filesystem" json:"Filesystem,omitempty"` + Filesystem string `protobuf:"bytes,1,opt,name=Filesystem,proto3" json:"Filesystem,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ListFilesystemVersionsReq) Reset() { *m = ListFilesystemVersionsReq{} } -func (m *ListFilesystemVersionsReq) String() string { return proto.CompactTextString(m) } -func (*ListFilesystemVersionsReq) ProtoMessage() {} -func (*ListFilesystemVersionsReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} } +func (m *ListFilesystemVersionsReq) Reset() { *m = ListFilesystemVersionsReq{} } +func (m *ListFilesystemVersionsReq) String() string { return proto.CompactTextString(m) } +func (*ListFilesystemVersionsReq) ProtoMessage() {} +func (*ListFilesystemVersionsReq) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{3} +} +func (m *ListFilesystemVersionsReq) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ListFilesystemVersionsReq.Unmarshal(m, b) +} +func (m *ListFilesystemVersionsReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ListFilesystemVersionsReq.Marshal(b, m, deterministic) +} +func (dst *ListFilesystemVersionsReq) XXX_Merge(src proto.Message) { + xxx_messageInfo_ListFilesystemVersionsReq.Merge(dst, src) +} +func (m *ListFilesystemVersionsReq) XXX_Size() int { + return xxx_messageInfo_ListFilesystemVersionsReq.Size(m) +} +func (m *ListFilesystemVersionsReq) XXX_DiscardUnknown() { + xxx_messageInfo_ListFilesystemVersionsReq.DiscardUnknown(m) +} + +var xxx_messageInfo_ListFilesystemVersionsReq proto.InternalMessageInfo func (m *ListFilesystemVersionsReq) GetFilesystem() string { if m != nil { @@ -125,13 +194,35 @@ func (m *ListFilesystemVersionsReq) GetFilesystem() string { } type ListFilesystemVersionsRes struct { - Versions []*FilesystemVersion `protobuf:"bytes,1,rep,name=Versions" json:"Versions,omitempty"` + Versions []*FilesystemVersion `protobuf:"bytes,1,rep,name=Versions,proto3" json:"Versions,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ListFilesystemVersionsRes) Reset() { *m = ListFilesystemVersionsRes{} } -func (m *ListFilesystemVersionsRes) String() string { return proto.CompactTextString(m) } -func (*ListFilesystemVersionsRes) ProtoMessage() {} -func (*ListFilesystemVersionsRes) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} } +func (m *ListFilesystemVersionsRes) Reset() { *m = ListFilesystemVersionsRes{} } +func (m *ListFilesystemVersionsRes) String() string { return proto.CompactTextString(m) } +func (*ListFilesystemVersionsRes) ProtoMessage() {} +func (*ListFilesystemVersionsRes) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{4} +} +func (m *ListFilesystemVersionsRes) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ListFilesystemVersionsRes.Unmarshal(m, b) +} +func (m *ListFilesystemVersionsRes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ListFilesystemVersionsRes.Marshal(b, m, deterministic) +} +func (dst *ListFilesystemVersionsRes) XXX_Merge(src proto.Message) { + xxx_messageInfo_ListFilesystemVersionsRes.Merge(dst, src) +} +func (m *ListFilesystemVersionsRes) XXX_Size() int { + return xxx_messageInfo_ListFilesystemVersionsRes.Size(m) +} +func (m *ListFilesystemVersionsRes) XXX_DiscardUnknown() { + xxx_messageInfo_ListFilesystemVersionsRes.DiscardUnknown(m) +} + +var xxx_messageInfo_ListFilesystemVersionsRes proto.InternalMessageInfo func (m *ListFilesystemVersionsRes) GetVersions() []*FilesystemVersion { if m != nil { @@ -141,17 +232,39 @@ func (m *ListFilesystemVersionsRes) GetVersions() []*FilesystemVersion { } type FilesystemVersion struct { - Type FilesystemVersion_VersionType `protobuf:"varint,1,opt,name=Type,enum=pdu.FilesystemVersion_VersionType" json:"Type,omitempty"` - Name string `protobuf:"bytes,2,opt,name=Name" json:"Name,omitempty"` - Guid uint64 `protobuf:"varint,3,opt,name=Guid" json:"Guid,omitempty"` - CreateTXG uint64 `protobuf:"varint,4,opt,name=CreateTXG" json:"CreateTXG,omitempty"` - Creation string `protobuf:"bytes,5,opt,name=Creation" json:"Creation,omitempty"` + Type FilesystemVersion_VersionType `protobuf:"varint,1,opt,name=Type,proto3,enum=pdu.FilesystemVersion_VersionType" json:"Type,omitempty"` + Name string `protobuf:"bytes,2,opt,name=Name,proto3" json:"Name,omitempty"` + Guid uint64 `protobuf:"varint,3,opt,name=Guid,proto3" json:"Guid,omitempty"` + CreateTXG uint64 `protobuf:"varint,4,opt,name=CreateTXG,proto3" json:"CreateTXG,omitempty"` + Creation string `protobuf:"bytes,5,opt,name=Creation,proto3" json:"Creation,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *FilesystemVersion) Reset() { *m = FilesystemVersion{} } -func (m *FilesystemVersion) String() string { return proto.CompactTextString(m) } -func (*FilesystemVersion) ProtoMessage() {} -func (*FilesystemVersion) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} } +func (m *FilesystemVersion) Reset() { *m = FilesystemVersion{} } +func (m *FilesystemVersion) String() string { return proto.CompactTextString(m) } +func (*FilesystemVersion) ProtoMessage() {} +func (*FilesystemVersion) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{5} +} +func (m *FilesystemVersion) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_FilesystemVersion.Unmarshal(m, b) +} +func (m *FilesystemVersion) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_FilesystemVersion.Marshal(b, m, deterministic) +} +func (dst *FilesystemVersion) XXX_Merge(src proto.Message) { + xxx_messageInfo_FilesystemVersion.Merge(dst, src) +} +func (m *FilesystemVersion) XXX_Size() int { + return xxx_messageInfo_FilesystemVersion.Size(m) +} +func (m *FilesystemVersion) XXX_DiscardUnknown() { + xxx_messageInfo_FilesystemVersion.DiscardUnknown(m) +} + +var xxx_messageInfo_FilesystemVersion proto.InternalMessageInfo func (m *FilesystemVersion) GetType() FilesystemVersion_VersionType { if m != nil { @@ -189,10 +302,10 @@ func (m *FilesystemVersion) GetCreation() string { } type SendReq struct { - Filesystem string `protobuf:"bytes,1,opt,name=Filesystem" json:"Filesystem,omitempty"` - From string `protobuf:"bytes,2,opt,name=From" json:"From,omitempty"` + Filesystem string `protobuf:"bytes,1,opt,name=Filesystem,proto3" json:"Filesystem,omitempty"` + From string `protobuf:"bytes,2,opt,name=From,proto3" json:"From,omitempty"` // May be empty / null to request a full transfer of From - To string `protobuf:"bytes,3,opt,name=To" json:"To,omitempty"` + To string `protobuf:"bytes,3,opt,name=To,proto3" json:"To,omitempty"` // If ResumeToken is not empty, the resume token that CAN be tried for 'zfs send' by the sender. // The sender MUST indicate in SendRes.UsedResumeToken // If it does not work, the sender SHOULD clear the resume token on their side @@ -200,15 +313,37 @@ type SendReq struct { // If ResumeToken is not empty, the GUIDs of From and To // MUST correspond to those encoded in the ResumeToken. // Otherwise, the Sender MUST return an error. - ResumeToken string `protobuf:"bytes,4,opt,name=ResumeToken" json:"ResumeToken,omitempty"` - Compress bool `protobuf:"varint,5,opt,name=Compress" json:"Compress,omitempty"` - Dedup bool `protobuf:"varint,6,opt,name=Dedup" json:"Dedup,omitempty"` + ResumeToken string `protobuf:"bytes,4,opt,name=ResumeToken,proto3" json:"ResumeToken,omitempty"` + Compress bool `protobuf:"varint,5,opt,name=Compress,proto3" json:"Compress,omitempty"` + Dedup bool `protobuf:"varint,6,opt,name=Dedup,proto3" json:"Dedup,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SendReq) Reset() { *m = SendReq{} } -func (m *SendReq) String() string { return proto.CompactTextString(m) } -func (*SendReq) ProtoMessage() {} -func (*SendReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} } +func (m *SendReq) Reset() { *m = SendReq{} } +func (m *SendReq) String() string { return proto.CompactTextString(m) } +func (*SendReq) ProtoMessage() {} +func (*SendReq) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{6} +} +func (m *SendReq) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SendReq.Unmarshal(m, b) +} +func (m *SendReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SendReq.Marshal(b, m, deterministic) +} +func (dst *SendReq) XXX_Merge(src proto.Message) { + xxx_messageInfo_SendReq.Merge(dst, src) +} +func (m *SendReq) XXX_Size() int { + return xxx_messageInfo_SendReq.Size(m) +} +func (m *SendReq) XXX_DiscardUnknown() { + xxx_messageInfo_SendReq.DiscardUnknown(m) +} + +var xxx_messageInfo_SendReq proto.InternalMessageInfo func (m *SendReq) GetFilesystem() string { if m != nil { @@ -253,14 +388,36 @@ func (m *SendReq) GetDedup() bool { } type Property struct { - Name string `protobuf:"bytes,1,opt,name=Name" json:"Name,omitempty"` - Value string `protobuf:"bytes,2,opt,name=Value" json:"Value,omitempty"` + Name string `protobuf:"bytes,1,opt,name=Name,proto3" json:"Name,omitempty"` + Value string `protobuf:"bytes,2,opt,name=Value,proto3" json:"Value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Property) Reset() { *m = Property{} } -func (m *Property) String() string { return proto.CompactTextString(m) } -func (*Property) ProtoMessage() {} -func (*Property) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} } +func (m *Property) Reset() { *m = Property{} } +func (m *Property) String() string { return proto.CompactTextString(m) } +func (*Property) ProtoMessage() {} +func (*Property) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{7} +} +func (m *Property) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_Property.Unmarshal(m, b) +} +func (m *Property) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_Property.Marshal(b, m, deterministic) +} +func (dst *Property) XXX_Merge(src proto.Message) { + xxx_messageInfo_Property.Merge(dst, src) +} +func (m *Property) XXX_Size() int { + return xxx_messageInfo_Property.Size(m) +} +func (m *Property) XXX_DiscardUnknown() { + xxx_messageInfo_Property.DiscardUnknown(m) +} + +var xxx_messageInfo_Property proto.InternalMessageInfo func (m *Property) GetName() string { if m != nil { @@ -278,14 +435,36 @@ func (m *Property) GetValue() string { type SendRes struct { // Whether the resume token provided in the request has been used or not. - UsedResumeToken bool `protobuf:"varint,1,opt,name=UsedResumeToken" json:"UsedResumeToken,omitempty"` - Properties []*Property `protobuf:"bytes,2,rep,name=Properties" json:"Properties,omitempty"` + UsedResumeToken bool `protobuf:"varint,1,opt,name=UsedResumeToken,proto3" json:"UsedResumeToken,omitempty"` + Properties []*Property `protobuf:"bytes,2,rep,name=Properties,proto3" json:"Properties,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SendRes) Reset() { *m = SendRes{} } -func (m *SendRes) String() string { return proto.CompactTextString(m) } -func (*SendRes) ProtoMessage() {} -func (*SendRes) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} } +func (m *SendRes) Reset() { *m = SendRes{} } +func (m *SendRes) String() string { return proto.CompactTextString(m) } +func (*SendRes) ProtoMessage() {} +func (*SendRes) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{8} +} +func (m *SendRes) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SendRes.Unmarshal(m, b) +} +func (m *SendRes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SendRes.Marshal(b, m, deterministic) +} +func (dst *SendRes) XXX_Merge(src proto.Message) { + xxx_messageInfo_SendRes.Merge(dst, src) +} +func (m *SendRes) XXX_Size() int { + return xxx_messageInfo_SendRes.Size(m) +} +func (m *SendRes) XXX_DiscardUnknown() { + xxx_messageInfo_SendRes.DiscardUnknown(m) +} + +var xxx_messageInfo_SendRes proto.InternalMessageInfo func (m *SendRes) GetUsedResumeToken() bool { if m != nil { @@ -302,15 +481,37 @@ func (m *SendRes) GetProperties() []*Property { } type ReceiveReq struct { - Filesystem string `protobuf:"bytes,1,opt,name=Filesystem" json:"Filesystem,omitempty"` + Filesystem string `protobuf:"bytes,1,opt,name=Filesystem,proto3" json:"Filesystem,omitempty"` // If true, the receiver should clear the resume token before perfoming the zfs recv of the stream in the request - ClearResumeToken bool `protobuf:"varint,2,opt,name=ClearResumeToken" json:"ClearResumeToken,omitempty"` + ClearResumeToken bool `protobuf:"varint,2,opt,name=ClearResumeToken,proto3" json:"ClearResumeToken,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ReceiveReq) Reset() { *m = ReceiveReq{} } -func (m *ReceiveReq) String() string { return proto.CompactTextString(m) } -func (*ReceiveReq) ProtoMessage() {} -func (*ReceiveReq) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} } +func (m *ReceiveReq) Reset() { *m = ReceiveReq{} } +func (m *ReceiveReq) String() string { return proto.CompactTextString(m) } +func (*ReceiveReq) ProtoMessage() {} +func (*ReceiveReq) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{9} +} +func (m *ReceiveReq) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ReceiveReq.Unmarshal(m, b) +} +func (m *ReceiveReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ReceiveReq.Marshal(b, m, deterministic) +} +func (dst *ReceiveReq) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReceiveReq.Merge(dst, src) +} +func (m *ReceiveReq) XXX_Size() int { + return xxx_messageInfo_ReceiveReq.Size(m) +} +func (m *ReceiveReq) XXX_DiscardUnknown() { + xxx_messageInfo_ReceiveReq.DiscardUnknown(m) +} + +var xxx_messageInfo_ReceiveReq proto.InternalMessageInfo func (m *ReceiveReq) GetFilesystem() string { if m != nil { @@ -327,12 +528,34 @@ func (m *ReceiveReq) GetClearResumeToken() bool { } type ReceiveRes struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ReceiveRes) Reset() { *m = ReceiveRes{} } -func (m *ReceiveRes) String() string { return proto.CompactTextString(m) } -func (*ReceiveRes) ProtoMessage() {} -func (*ReceiveRes) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} } +func (m *ReceiveRes) Reset() { *m = ReceiveRes{} } +func (m *ReceiveRes) String() string { return proto.CompactTextString(m) } +func (*ReceiveRes) ProtoMessage() {} +func (*ReceiveRes) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{10} +} +func (m *ReceiveRes) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ReceiveRes.Unmarshal(m, b) +} +func (m *ReceiveRes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ReceiveRes.Marshal(b, m, deterministic) +} +func (dst *ReceiveRes) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReceiveRes.Merge(dst, src) +} +func (m *ReceiveRes) XXX_Size() int { + return xxx_messageInfo_ReceiveRes.Size(m) +} +func (m *ReceiveRes) XXX_DiscardUnknown() { + xxx_messageInfo_ReceiveRes.DiscardUnknown(m) +} + +var xxx_messageInfo_ReceiveRes proto.InternalMessageInfo func init() { proto.RegisterType((*ListFilesystemReq)(nil), "pdu.ListFilesystemReq") @@ -349,9 +572,9 @@ func init() { proto.RegisterEnum("pdu.FilesystemVersion_VersionType", FilesystemVersion_VersionType_name, FilesystemVersion_VersionType_value) } -func init() { proto.RegisterFile("pdu.proto", fileDescriptor0) } +func init() { proto.RegisterFile("pdu.proto", fileDescriptor_pdu_25ca9e5977a7ce3f) } -var fileDescriptor0 = []byte{ +var fileDescriptor_pdu_25ca9e5977a7ce3f = []byte{ // 445 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x53, 0xc1, 0x6e, 0xd3, 0x40, 0x10, 0x65, 0x13, 0xa7, 0x38, 0x93, 0xd2, 0xa6, 0x4b, 0x85, 0x0c, 0x42, 0x28, 0xda, 0x53, 0x40, From 666ead2646792a01e238c709114dc23ebd0dab1a Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 26 Aug 2018 14:51:20 +0200 Subject: [PATCH 035/167] make go vet happy --- cmd/config_job_local.go | 2 +- zfs/resume_token.go | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index 41f8219..e9e1d2c 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -99,7 +99,7 @@ func (j *LocalJob) JobStart(ctx context.Context) { // We can pay this small performance penalty for now. wildcardMapFilter := NewDatasetMapFilter(1, false) wildcardMapFilter.Add("<", "<") - sender := &endpoint.Sender{wildcardMapFilter, NewPrefixFilter(j.SnapshotPrefix)} + sender := endpoint.NewSender(wildcardMapFilter, NewPrefixFilter(j.SnapshotPrefix)) receiver, err := endpoint.NewReceiver(j.Mapping, NewPrefixFilter(j.SnapshotPrefix)) if err != nil { diff --git a/zfs/resume_token.go b/zfs/resume_token.go index 5bf1632..e83c8f2 100644 --- a/zfs/resume_token.go +++ b/zfs/resume_token.go @@ -48,7 +48,8 @@ func ParseResumeToken(ctx context.Context, token string) (*ResumeToken, error) { // toname = pool1/test@b //cannot resume send: 'pool1/test@b' used in the initial send no longer exists - ctx, _ = context.WithTimeout(ctx, 500*time.Millisecond) + ctx, cancel := context.WithTimeout(ctx, 500*time.Millisecond) + defer cancel() cmd := exec.CommandContext(ctx, ZFS_BINARY, "send", "-nvt", string(token)) output, err := cmd.CombinedOutput() if err != nil { From f6be5b776b854f230827190d6c47e99f2d00a964 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 26 Aug 2018 14:58:57 +0200 Subject: [PATCH 036/167] cmd: clean up usage of contextKeyLog through getter and setter functions --- cmd/config_job_control.go | 2 +- cmd/config_job_local.go | 2 +- cmd/config_job_prometheus.go | 2 +- cmd/config_job_pull.go | 2 +- cmd/config_job_source.go | 5 ++--- cmd/daemon.go | 11 +++++++---- cmd/test.go | 2 +- 7 files changed, 14 insertions(+), 12 deletions(-) diff --git a/cmd/config_job_control.go b/cmd/config_job_control.go index 85d906e..83de88a 100644 --- a/cmd/config_job_control.go +++ b/cmd/config_job_control.go @@ -46,7 +46,7 @@ const ( func (j *ControlJob) JobStart(ctx context.Context) { - log := ctx.Value(contextKeyLog).(Logger) + log := getLogger(ctx) defer log.Info("control job finished") daemon := ctx.Value(contextKeyDaemon).(*Daemon) diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index e9e1d2c..1c3f314 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -84,7 +84,7 @@ func (j *LocalJob) JobType() JobType { return JobTypeLocal } func (j *LocalJob) JobStart(ctx context.Context) { - rootLog := ctx.Value(contextKeyLog).(Logger) + rootLog := getLogger(ctx) j.snapperTask = NewTask("snapshot", j, rootLog) j.mainTask = NewTask("main", j, rootLog) diff --git a/cmd/config_job_prometheus.go b/cmd/config_job_prometheus.go index 512d01b..29af3c1 100644 --- a/cmd/config_job_prometheus.go +++ b/cmd/config_job_prometheus.go @@ -69,7 +69,7 @@ func (j *PrometheusJob) JobStart(ctx context.Context) { panic(err) } - log := ctx.Value(contextKeyLog).(Logger) + log := getLogger(ctx) task := NewTask("main", j, log) log = task.Log() diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index 233f150..5d30b2b 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -105,7 +105,7 @@ func (j *PullJob) JobType() JobType { return JobTypePull } func (j *PullJob) JobStart(ctx context.Context) { - log := ctx.Value(contextKeyLog).(Logger) + log := getLogger(ctx) defer log.Info("exiting") j.task = NewTask("main", j, log) diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index 200b01d..3d409f2 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -89,7 +89,7 @@ func (j *SourceJob) JobType() JobType { return JobTypeSource } func (j *SourceJob) JobStart(ctx context.Context) { - log := ctx.Value(contextKeyLog).(Logger) + log := getLogger(ctx) defer log.Info("exiting") j.autosnapTask = NewTask("autosnap", j, log) @@ -196,7 +196,6 @@ outer: } return - } func (j *SourceJob) handleConnection(conn net.Conn, task *Task) { @@ -209,7 +208,7 @@ func (j *SourceJob) handleConnection(conn net.Conn, task *Task) { senderEP := endpoint.NewSender(j.Filesystems, NewPrefixFilter(j.SnapshotPrefix)) ctx := context.Background() - ctx = context.WithValue(ctx, contextKeyLog, task.Log().WithField(logSubsysField, "rpc.endpoint")) + ctx = endpoint.WithLogger(ctx, task.Log().WithField(logSubsysField, "rpc.endpoint")) ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{task.Log().WithField(logSubsysField, "rpc.protocol")}) handler := endpoint.NewHandler(senderEP) if err := streamrpc.ServeConn(ctx, conn, STREAMRPC_CONFIG, handler.Handle); err != nil { diff --git a/cmd/daemon.go b/cmd/daemon.go index 81a718a..4ccd6f8 100644 --- a/cmd/daemon.go +++ b/cmd/daemon.go @@ -73,8 +73,7 @@ func doDaemon(cmd *cobra.Command, args []string) { log.Info(NewZreplVersionInformation().String()) log.Debug("starting daemon") - ctx := context.WithValue(context.Background(), contextKeyLog, log) - ctx = context.WithValue(ctx, contextKeyLog, log) + ctx := WithLogger(context.Background(), log) d := NewDaemon(conf) d.Loop(ctx) @@ -92,6 +91,10 @@ func getLogger(ctx context.Context) Logger { return ctx.Value(contextKeyLog).(Logger) } +func WithLogger(ctx context.Context, l Logger) context.Context { + return context.WithValue(ctx, contextKeyLog, l) +} + type Daemon struct { conf *Config startedAt time.Time @@ -105,7 +108,7 @@ func (d *Daemon) Loop(ctx context.Context) { d.startedAt = time.Now() - log := ctx.Value(contextKeyLog).(Logger) + log := getLogger(ctx) ctx, cancel := context.WithCancel(ctx) ctx = context.WithValue(ctx, contextKeyDaemon, d) @@ -121,7 +124,7 @@ func (d *Daemon) Loop(ctx context.Context) { logger := log.WithField(logJobField, job.JobName()) logger.Info("starting") i++ - jobCtx := context.WithValue(ctx, contextKeyLog, logger) + jobCtx := WithLogger(ctx, logger) go func(j Job) { j.JobStart(jobCtx) finishs <- j diff --git a/cmd/test.go b/cmd/test.go index d6b1f80..08d34b5 100644 --- a/cmd/test.go +++ b/cmd/test.go @@ -179,7 +179,7 @@ func doTestPrunePolicy(cmd *cobra.Command, args []string) { log.Printf("start pruning") - ctx := context.WithValue(context.Background(), contextKeyLog, log) + ctx := WithLogger(context.Background(), log) result, err := pruner.Run(ctx) if err != nil { log.Printf("error running pruner: %s", err) From 7ff72fb6d98342daa18f760aca83a266dba1d4d0 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 26 Aug 2018 15:12:43 +0200 Subject: [PATCH 037/167] replication: document most important aspects of Endpoint interface --- cmd/endpoint/endpoint.go | 1 - replication/fsrep/fsfsm.go | 8 ++++++++ replication/mainfsm.go | 6 ++++++ 3 files changed, 14 insertions(+), 1 deletion(-) diff --git a/cmd/endpoint/endpoint.go b/cmd/endpoint/endpoint.go index bd03a6a..fca060b 100644 --- a/cmd/endpoint/endpoint.go +++ b/cmd/endpoint/endpoint.go @@ -296,7 +296,6 @@ func (s Remote) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.Read rs.Close() return nil, nil, err } - // FIXME make sure the consumer will read the reader until the end... return &res, rs, nil } diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index 025cbb1..52f6c31 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -36,11 +36,19 @@ func getLogger(ctx context.Context) Logger { return l } +// A Sender is usually part of a github.com/zrepl/zrepl/replication.Endpoint. type Sender interface { + // If a non-nil io.ReadCloser is returned, it is guaranteed to be closed before + // any next call to the parent github.com/zrepl/zrepl/replication.Endpoint. Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) } +// A Sender is usually part of a github.com/zrepl/zrepl/replication.Endpoint. type Receiver interface { + // Receive sends r and sendStream (the latter containing a ZFS send stream) + // to the parent github.com/zrepl/zrepl/replication.Endpoint. + // Implementors must guarantee that Close was called on sendStream before + // the call to Receive returns. Receive(ctx context.Context, r *pdu.ReceiveReq, sendStream io.ReadCloser) error } diff --git a/replication/mainfsm.go b/replication/mainfsm.go index 4a5abcf..d7b2462 100644 --- a/replication/mainfsm.go +++ b/replication/mainfsm.go @@ -84,9 +84,14 @@ func NewReplication() *Replication { return &r } +// Endpoint represents one side of the replication. +// +// An endpoint is either in Sender or Receiver mode, represented by the correspondingly +// named interfaces defined in this package. type Endpoint interface { // Does not include placeholder filesystems ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) + // FIXME document FilteredError handling ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) // fix depS } @@ -100,6 +105,7 @@ type Receiver interface { fsrep.Receiver } + type FilteredError struct{ fs string } func NewFilteredError(fs string) *FilteredError { From a0f72b585b0d60864b4d39d888814a48cdc1e3d6 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 26 Aug 2018 16:49:40 +0200 Subject: [PATCH 038/167] remove JobStatus, Task abstraction and 'control status' subcommand Control status will be replaced by job-specific output at some point. Task was not useful anymore with state machine, may reintroduce something similar at a later point, but consider alternatives: - opentracing.io - embedding everything in ctx - activity stack would work easily - log entries via proxy logger.Logger object - progress reporting should be in status reports of individial jobs --- cmd/autosnap.go | 62 +++---- cmd/config.go | 2 +- cmd/config_job_control.go | 6 +- cmd/config_job_local.go | 68 ++------ cmd/config_job_prometheus.go | 5 - cmd/config_job_pull.go | 52 +++--- cmd/config_job_source.go | 69 +++----- cmd/control.go | 151 ----------------- cmd/daemon.go | 318 +---------------------------------- cmd/prune.go | 36 ++-- cmd/test.go | 3 +- 11 files changed, 100 insertions(+), 672 deletions(-) diff --git a/cmd/autosnap.go b/cmd/autosnap.go index fe063a7..983513f 100644 --- a/cmd/autosnap.go +++ b/cmd/autosnap.go @@ -9,29 +9,24 @@ import ( ) type IntervalAutosnap struct { - task *Task DatasetFilter zfs.DatasetFilter Prefix string SnapshotInterval time.Duration } -func (a *IntervalAutosnap) filterFilesystems() (fss []*zfs.DatasetPath, stop bool) { - a.task.Enter("filter_filesystems") - defer a.task.Finish() +func (a *IntervalAutosnap) filterFilesystems(ctx context.Context) (fss []*zfs.DatasetPath, stop bool) { fss, err := zfs.ZFSListMapping(a.DatasetFilter) stop = err != nil if err != nil { - a.task.Log().WithError(err).Error("cannot list datasets") + getLogger(ctx).WithError(err).Error("cannot list datasets") } if len(fss) == 0 { - a.task.Log().Warn("no filesystem matching filesystem filter") + getLogger(ctx).Warn("no filesystem matching filesystem filter") } return fss, stop } -func (a *IntervalAutosnap) findSyncPoint(fss []*zfs.DatasetPath) (syncPoint time.Time, err error) { - a.task.Enter("find_sync_point") - defer a.task.Finish() +func (a *IntervalAutosnap) findSyncPoint(log Logger, fss []*zfs.DatasetPath) (syncPoint time.Time, err error) { type snapTime struct { ds *zfs.DatasetPath time time.Time @@ -45,10 +40,10 @@ func (a *IntervalAutosnap) findSyncPoint(fss []*zfs.DatasetPath) (syncPoint time now := time.Now() - a.task.Log().Debug("examine filesystem state") + log.Debug("examine filesystem state") for _, d := range fss { - l := a.task.Log().WithField("fs", d.ToString()) + l := log.WithField("fs", d.ToString()) fsvs, err := zfs.ZFSListFilesystemVersions(d, NewPrefixFilter(a.Prefix)) if err != nil { @@ -96,76 +91,71 @@ func (a *IntervalAutosnap) findSyncPoint(fss []*zfs.DatasetPath) (syncPoint time } func (a *IntervalAutosnap) waitForSyncPoint(ctx context.Context, syncPoint time.Time) { - a.task.Enter("wait_sync_point") - defer a.task.Finish() const LOG_TIME_FMT string = time.ANSIC - a.task.Log().WithField("sync_point", syncPoint.Format(LOG_TIME_FMT)). + getLogger(ctx). + WithField("sync_point", syncPoint.Format(LOG_TIME_FMT)). Info("wait for sync point") select { case <-ctx.Done(): - a.task.Log().WithError(ctx.Err()).Info("context done") + getLogger(ctx).WithError(ctx.Err()).Info("context done") return case <-time.After(syncPoint.Sub(time.Now())): } } func (a *IntervalAutosnap) syncUpRun(ctx context.Context, didSnaps chan struct{}) (stop bool) { - a.task.Enter("sync_up") - defer a.task.Finish() - - fss, stop := a.filterFilesystems() + fss, stop := a.filterFilesystems(ctx) if stop { return true } - syncPoint, err := a.findSyncPoint(fss) + syncPoint, err := a.findSyncPoint(getLogger(ctx), fss) if err != nil { return true } a.waitForSyncPoint(ctx, syncPoint) - a.task.Log().Debug("snapshot all filesystems to enable further snaps in lockstep") - a.doSnapshots(didSnaps) + getLogger(ctx).Debug("snapshot all filesystems to enable further snaps in lockstep") + a.doSnapshots(ctx, didSnaps) return false } func (a *IntervalAutosnap) Run(ctx context.Context, didSnaps chan struct{}) { + log := getLogger(ctx) + if a.syncUpRun(ctx, didSnaps) { - a.task.Log().Error("stoppping autosnap after error in sync up") + log.Error("stoppping autosnap after error in sync up") return } // task drops back to idle here - a.task.Log().Debug("setting up ticker in SnapshotInterval") + log.Debug("setting up ticker in SnapshotInterval") ticker := time.NewTicker(a.SnapshotInterval) for { select { case <-ctx.Done(): ticker.Stop() - a.task.Log().WithError(ctx.Err()).Info("context done") + log.WithError(ctx.Err()).Info("context done") return - case <-ticker.C: - a.doSnapshots(didSnaps) + a.doSnapshots(ctx, didSnaps) } } } -func (a *IntervalAutosnap) doSnapshots(didSnaps chan struct{}) { - - a.task.Enter("do_snapshots") - defer a.task.Finish() +func (a *IntervalAutosnap) doSnapshots(ctx context.Context, didSnaps chan struct{}) { + log := getLogger(ctx) // don't cache the result from previous run in case the user added // a new dataset in the meantime - ds, stop := a.filterFilesystems() + ds, stop := a.filterFilesystems(ctx) if stop { return } @@ -175,20 +165,20 @@ func (a *IntervalAutosnap) doSnapshots(didSnaps chan struct{}) { suffix := time.Now().In(time.UTC).Format("20060102_150405_000") snapname := fmt.Sprintf("%s%s", a.Prefix, suffix) - l := a.task.Log(). + l := log. WithField("fs", d.ToString()). WithField("snapname", snapname) l.Info("create snapshot") err := zfs.ZFSSnapshot(d, snapname, false) if err != nil { - a.task.Log().WithError(err).Error("cannot create snapshot") + l.WithError(err).Error("cannot create snapshot") } l.Info("create corresponding bookmark") err = zfs.ZFSBookmark(d, snapname, snapname) if err != nil { - a.task.Log().WithError(err).Error("cannot create bookmark") + l.WithError(err).Error("cannot create bookmark") } } @@ -196,7 +186,7 @@ func (a *IntervalAutosnap) doSnapshots(didSnaps chan struct{}) { select { case didSnaps <- struct{}{}: default: - a.task.Log().Error("warning: callback channel is full, discarding") + log.Error("warning: callback channel is full, discarding") } } diff --git a/cmd/config.go b/cmd/config.go index 1361d75..69ce086 100644 --- a/cmd/config.go +++ b/cmd/config.go @@ -57,7 +57,7 @@ type PrunePolicy interface { } type PruningJob interface { - Pruner(task *Task, side PrunePolicySide, dryRun bool) (Pruner, error) + Pruner(side PrunePolicySide, dryRun bool) (Pruner, error) } // A type for constants describing different prune policies of a PruningJob diff --git a/cmd/config_job_control.go b/cmd/config_job_control.go index 83de88a..3c0d802 100644 --- a/cmd/config_job_control.go +++ b/cmd/config_job_control.go @@ -34,10 +34,6 @@ func (j *ControlJob) JobName() string { func (j *ControlJob) JobType() JobType { return JobTypeControl } -func (j *ControlJob) JobStatus(ctx context.Context) (*JobStatus, error) { - return &JobStatus{Tasks: nil}, nil -} - const ( ControlJobEndpointPProf string = "/debug/pprof" ControlJobEndpointVersion string = "/version" @@ -76,7 +72,7 @@ func (j *ControlJob) JobStart(ctx context.Context) { }}}) mux.Handle(ControlJobEndpointStatus, requestLogger{log: log, handler: jsonResponder{func() (interface{}, error) { - return daemon.Status(), nil + panic("FIXME") // FIXME }}}) mux.Handle("/pulljobreport", requestLogger{log: log, handler: jsonResponder{func() (interface{}, error) { diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index 1c3f314..03b063a 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -20,11 +20,6 @@ type LocalJob struct { PruneLHS PrunePolicy PruneRHS PrunePolicy Debug JobDebugSettings - snapperTask *Task - mainTask *Task - handlerTask *Task - pruneRHSTask *Task - pruneLHSTask *Task } func parseLocalJob(c JobParsingContext, name string, i map[string]interface{}) (j *LocalJob, err error) { @@ -84,13 +79,7 @@ func (j *LocalJob) JobType() JobType { return JobTypeLocal } func (j *LocalJob) JobStart(ctx context.Context) { - rootLog := getLogger(ctx) - - j.snapperTask = NewTask("snapshot", j, rootLog) - j.mainTask = NewTask("main", j, rootLog) - j.handlerTask = NewTask("handler", j, rootLog) - j.pruneRHSTask = NewTask("prune_rhs", j, rootLog) - j.pruneLHSTask = NewTask("prune_lhs", j, rootLog) + log := getLogger(ctx) // Allow access to any dataset since we control what mapping // is passed to the pull routine. @@ -103,89 +92,67 @@ func (j *LocalJob) JobStart(ctx context.Context) { receiver, err := endpoint.NewReceiver(j.Mapping, NewPrefixFilter(j.SnapshotPrefix)) if err != nil { - rootLog.WithError(err).Error("unexpected error setting up local handler") + log.WithError(err).Error("unexpected error setting up local handler") } snapper := IntervalAutosnap{ - task: j.snapperTask, DatasetFilter: j.Mapping.AsFilter(), Prefix: j.SnapshotPrefix, SnapshotInterval: j.Interval, } - plhs, err := j.Pruner(j.pruneLHSTask, PrunePolicySideLeft, false) + plhs, err := j.Pruner(PrunePolicySideLeft, false) if err != nil { - rootLog.WithError(err).Error("error creating lhs pruner") + log.WithError(err).Error("error creating lhs pruner") return } - prhs, err := j.Pruner(j.pruneRHSTask, PrunePolicySideRight, false) + prhs, err := j.Pruner(PrunePolicySideRight, false) if err != nil { - rootLog.WithError(err).Error("error creating rhs pruner") + log.WithError(err).Error("error creating rhs pruner") return } didSnaps := make(chan struct{}) - go snapper.Run(ctx, didSnaps) + go snapper.Run(WithLogger(ctx, log.WithField(logSubsysField, "snap")), didSnaps) outer: for { select { case <-ctx.Done(): - j.mainTask.Log().WithError(ctx.Err()).Info("context") + log.WithError(ctx.Err()).Info("context") break outer case <-didSnaps: - j.mainTask.Log().Debug("finished taking snapshots") - j.mainTask.Log().Info("starting replication procedure") + log.Debug("finished taking snapshots") + log.Info("starting replication procedure") } - j.mainTask.Log().Debug("replicating from lhs to rhs") - j.mainTask.Enter("replicate") - - rep := replication.NewReplication() - rep.Drive(ctx, sender, receiver) - - j.mainTask.Finish() - - // use a ctx as soon as Pull gains ctx support - select { - case <-ctx.Done(): - break outer - default: + { + ctx := WithLogger(ctx, log.WithField(logSubsysField, "replication")) + rep := replication.NewReplication() + rep.Drive(ctx, sender, receiver) } var wg sync.WaitGroup - j.mainTask.Log().Info("pruning lhs") wg.Add(1) go func() { - plhs.Run(ctx) + plhs.Run(WithLogger(ctx, log.WithField(logSubsysField, "prune_lhs"))) wg.Done() }() - j.mainTask.Log().Info("pruning rhs") wg.Add(1) go func() { - prhs.Run(ctx) + prhs.Run(WithLogger(ctx, log.WithField(logSubsysField, "prune_rhs"))) wg.Done() }() wg.Wait() - } } -func (j *LocalJob) JobStatus(ctxt context.Context) (*JobStatus, error) { - return &JobStatus{Tasks: []*TaskStatus{ - j.snapperTask.Status(), - j.pruneLHSTask.Status(), - j.pruneRHSTask.Status(), - j.mainTask.Status(), - }}, nil -} - -func (j *LocalJob) Pruner(task *Task, side PrunePolicySide, dryRun bool) (p Pruner, err error) { +func (j *LocalJob) Pruner(side PrunePolicySide, dryRun bool) (p Pruner, err error) { var dsfilter zfs.DatasetFilter var pp PrunePolicy @@ -206,7 +173,6 @@ func (j *LocalJob) Pruner(task *Task, side PrunePolicySide, dryRun bool) (p Prun } p = Pruner{ - task, time.Now(), dryRun, dsfilter, diff --git a/cmd/config_job_prometheus.go b/cmd/config_job_prometheus.go index 29af3c1..8e0427b 100644 --- a/cmd/config_job_prometheus.go +++ b/cmd/config_job_prometheus.go @@ -70,8 +70,6 @@ func (j *PrometheusJob) JobStart(ctx context.Context) { } log := getLogger(ctx) - task := NewTask("main", j, log) - log = task.Log() l, err := net.Listen("tcp", j.Listen) if err != nil { @@ -94,6 +92,3 @@ func (j *PrometheusJob) JobStart(ctx context.Context) { } -func (*PrometheusJob) JobStatus(ctxt context.Context) (*JobStatus, error) { - return &JobStatus{}, nil -} diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index 5d30b2b..711e3d6 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -26,7 +26,6 @@ type PullJob struct { Prune PrunePolicy Debug JobDebugSettings - task *Task rep *replication.Replication } @@ -107,7 +106,6 @@ func (j *PullJob) JobStart(ctx context.Context) { log := getLogger(ctx) defer log.Info("exiting") - j.task = NewTask("main", j, log) // j.task is idle here idle here usr1 := make(chan os.Signal) @@ -120,14 +118,14 @@ func (j *PullJob) JobStart(ctx context.Context) { j.doRun(ctx) duration := time.Now().Sub(begin) if duration > j.Interval { - j.task.Log(). + log. WithField("actual_duration", duration). WithField("configured_interval", j.Interval). Warn("pull run took longer than configured interval") } select { case <-ctx.Done(): - j.task.Log().WithError(ctx.Err()).Info("context") + log.WithError(ctx.Err()).Info("context") return case <-ticker.C: case <-usr1: @@ -150,9 +148,7 @@ var STREAMRPC_CONFIG = &streamrpc.ConnConfig{ // FIXME oversight and configurabi func (j *PullJob) doRun(ctx context.Context) { - j.task.Enter("run") - defer j.task.Finish() - + log := getLogger(ctx) // FIXME clientConf := &streamrpc.ClientConfig{ ConnConfig: STREAMRPC_CONFIG, @@ -161,8 +157,6 @@ func (j *PullJob) doRun(ctx context.Context) { client, err := streamrpc.NewClient(j.Connect, clientConf) defer client.Close() - j.task.Enter("pull") - sender := endpoint.NewRemote(client) puller, err := endpoint.NewReceiver( @@ -170,43 +164,37 @@ func (j *PullJob) doRun(ctx context.Context) { NewPrefixFilter(j.SnapshotPrefix), ) if err != nil { - j.task.Log().WithError(err).Error("error creating receiver endpoint") - j.task.Finish() + log.WithError(err).Error("error creating receiver endpoint") return } - ctx = replication.WithLogger(ctx, replicationLogAdaptor{j.task.Log().WithField(logSubsysField, "replication")}) - ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{j.task.Log().WithField(logSubsysField, "rpc.protocol")}) - ctx = endpoint.WithLogger(ctx, j.task.Log().WithField(logSubsysField, "rpc.endpoint")) - - j.rep = replication.NewReplication() - j.rep.Drive(ctx, sender, puller) + { + ctx := replication.WithLogger(ctx, replicationLogAdaptor{log.WithField(logSubsysField, "replication")}) + ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{log.WithField(logSubsysField, "rpc")}) + ctx = endpoint.WithLogger(ctx, log.WithField(logSubsysField, "endpoint")) + j.rep = replication.NewReplication() + j.rep.Drive(ctx, sender, puller) + } client.Close() - j.task.Finish() - j.task.Enter("prune") - pruner, err := j.Pruner(j.task, PrunePolicySideDefault, false) - if err != nil { - j.task.Log().WithError(err).Error("error creating pruner") - } else { - pruner.Run(ctx) + { + ctx := WithLogger(ctx, log.WithField(logSubsysField, "prune")) + pruner, err := j.Pruner(PrunePolicySideDefault, false) + if err != nil { + log.WithError(err).Error("error creating pruner") + } else { + pruner.Run(ctx) + } } - j.task.Finish() - } func (j *PullJob) Report() *replication.Report { return j.rep.Report() } -func (j *PullJob) JobStatus(ctxt context.Context) (*JobStatus, error) { - return &JobStatus{Tasks: []*TaskStatus{j.task.Status()}}, nil -} - -func (j *PullJob) Pruner(task *Task, side PrunePolicySide, dryRun bool) (p Pruner, err error) { +func (j *PullJob) Pruner(side PrunePolicySide, dryRun bool) (p Pruner, err error) { p = Pruner{ - task, time.Now(), dryRun, j.pruneFilter, diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index 3d409f2..9e2d53c 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -19,9 +19,6 @@ type SourceJob struct { Interval time.Duration Prune PrunePolicy Debug JobDebugSettings - serveTask *Task - autosnapTask *Task - pruneTask *Task } func parseSourceJob(c JobParsingContext, name string, i map[string]interface{}) (j *SourceJob, err error) { @@ -92,12 +89,8 @@ func (j *SourceJob) JobStart(ctx context.Context) { log := getLogger(ctx) defer log.Info("exiting") - j.autosnapTask = NewTask("autosnap", j, log) - j.pruneTask = NewTask("prune", j, log) - j.serveTask = NewTask("serve", j, log) - - a := IntervalAutosnap{j.autosnapTask, j.Filesystems, j.SnapshotPrefix, j.Interval} - p, err := j.Pruner(j.pruneTask, PrunePolicySideDefault, false) + a := IntervalAutosnap{j.Filesystems, j.SnapshotPrefix, j.Interval} + p, err := j.Pruner(PrunePolicySideDefault, false) if err != nil { log.WithError(err).Error("error creating pruner") @@ -106,8 +99,8 @@ func (j *SourceJob) JobStart(ctx context.Context) { didSnaps := make(chan struct{}) - go j.serve(ctx, j.serveTask) - go a.Run(ctx, didSnaps) + go j.serve(ctx) // logSubsysField set by handleConnection + go a.Run(WithLogger(ctx, log.WithField(logSubsysField, "snap")), didSnaps) outer: for { @@ -115,27 +108,15 @@ outer: case <-ctx.Done(): break outer case <-didSnaps: - log.Info("starting pruner") - p.Run(ctx) - log.Info("pruner done") + p.Run(WithLogger(ctx, log.WithField(logSubsysField, "prune"))) } } log.WithError(ctx.Err()).Info("context") } -func (j *SourceJob) JobStatus(ctxt context.Context) (*JobStatus, error) { - return &JobStatus{ - Tasks: []*TaskStatus{ - j.autosnapTask.Status(), - j.pruneTask.Status(), - j.serveTask.Status(), - }}, nil -} - -func (j *SourceJob) Pruner(task *Task, side PrunePolicySide, dryRun bool) (p Pruner, err error) { +func (j *SourceJob) Pruner(side PrunePolicySide, dryRun bool) (p Pruner, err error) { p = Pruner{ - task, time.Now(), dryRun, j.Filesystems, @@ -145,11 +126,13 @@ func (j *SourceJob) Pruner(task *Task, side PrunePolicySide, dryRun bool) (p Pru return } -func (j *SourceJob) serve(ctx context.Context, task *Task) { +func (j *SourceJob) serve(ctx context.Context) { + + log := getLogger(ctx) listener, err := j.Serve.Listen() if err != nil { - task.Log().WithError(err).Error("error listening") + getLogger(ctx).WithError(err).Error("error listening") return } @@ -173,48 +156,40 @@ outer: case rwcMsg := <-connChan: if rwcMsg.err != nil { - task.Log().WithError(rwcMsg.err).Error("error accepting connection") + log.WithError(rwcMsg.err).Error("error accepting connection") continue } - j.handleConnection(rwcMsg.conn, task) + j.handleConnection(ctx, rwcMsg.conn) case <-ctx.Done(): - task.Log().WithError(ctx.Err()).Info("context") + log.WithError(ctx.Err()).Info("context") break outer } } - task.Log().Info("closing listener") - - task.Enter("close_listener") - defer task.Finish() + log.Info("closing listener") err = listener.Close() if err != nil { - task.Log().WithError(err).Error("error closing listener") + log.WithError(err).Error("error closing listener") } return } -func (j *SourceJob) handleConnection(conn net.Conn, task *Task) { - - task.Enter("handle_connection") - defer task.Finish() - - task.Log().Info("handling client connection") +func (j *SourceJob) handleConnection(ctx context.Context, conn net.Conn) { + log := getLogger(ctx) + log.Info("handling client connection") senderEP := endpoint.NewSender(j.Filesystems, NewPrefixFilter(j.SnapshotPrefix)) - ctx := context.Background() - ctx = endpoint.WithLogger(ctx, task.Log().WithField(logSubsysField, "rpc.endpoint")) - ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{task.Log().WithField(logSubsysField, "rpc.protocol")}) + ctx = endpoint.WithLogger(ctx, log.WithField(logSubsysField, "serve")) + ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{log.WithField(logSubsysField, "rpc")}) handler := endpoint.NewHandler(senderEP) if err := streamrpc.ServeConn(ctx, conn, STREAMRPC_CONFIG, handler.Handle); err != nil { - task.Log().WithError(err).Error("error serving connection") + log.WithError(err).Error("error serving connection") } else { - task.Log().Info("client closed connection") + log.Info("client closed connection") } - } diff --git a/cmd/control.go b/cmd/control.go index 3c9e7df..b65579c 100644 --- a/cmd/control.go +++ b/cmd/control.go @@ -5,7 +5,6 @@ import ( "context" "encoding/json" "fmt" - "github.com/dustin/go-humanize" "github.com/pkg/errors" "github.com/spf13/cobra" "github.com/zrepl/zrepl/logger" @@ -14,9 +13,6 @@ import ( "net" "net/http" "os" - "sort" - "strings" - "time" ) var controlCmd = &cobra.Command{ @@ -67,20 +63,11 @@ var controlStatusCmdArgs struct { onlyShowJob string } -var controlStatusCmd = &cobra.Command{ - Use: "status [JOB_NAME]", - Short: "get current status", - Run: doControlStatusCmd, -} - func init() { RootCmd.AddCommand(controlCmd) controlCmd.AddCommand(pprofCmd) controlCmd.AddCommand(controlVersionCmd) - controlCmd.AddCommand(controlStatusCmd) - controlStatusCmd.Flags().StringVar(&controlStatusCmdArgs.format, "format", "human", "output format (human|raw)") controlStatusCmdArgs.level = logger.Warn - controlStatusCmd.Flags().Var(&controlStatusCmdArgs.level, "level", "minimum log level to show") } func controlHttpClient() (client http.Client, err error) { @@ -165,141 +152,3 @@ func doControLVersionCmd(cmd *cobra.Command, args []string) { fmt.Println(info.String()) } - -func doControlStatusCmd(cmd *cobra.Command, args []string) { - - log := golog.New(os.Stderr, "", 0) - - die := func() { - log.Print("exiting after error") - os.Exit(1) - } - - if len(args) == 1 { - controlStatusCmdArgs.onlyShowJob = args[0] - } else if len(args) > 1 { - log.Print("can only specify one job as positional argument") - cmd.Usage() - die() - } - - httpc, err := controlHttpClient() - if err != nil { - log.Printf("could not connect to daemon: %s", err) - die() - } - - resp, err := httpc.Get("http://unix" + ControlJobEndpointStatus) - if err != nil { - log.Printf("error: %s", err) - die() - } else if resp.StatusCode != http.StatusOK { - var msg bytes.Buffer - io.CopyN(&msg, resp.Body, 4096) - log.Printf("error: %s", msg.String()) - die() - } - - var status DaemonStatus - err = json.NewDecoder(resp.Body).Decode(&status) - if err != nil { - log.Printf("error unmarshaling response: %s", err) - die() - } - - switch controlStatusCmdArgs.format { - case "raw": - enc := json.NewEncoder(os.Stdout) - enc.SetIndent("", " ") - if err := enc.Encode(status); err != nil { - log.Panic(err) - } - case "human": - - formatter := HumanFormatter{} - formatter.SetMetadataFlags(MetadataAll) - formatter.SetIgnoreFields([]string{ - logJobField, - }) - jobNames := make([]string, 0, len(status.Jobs)) - for name, _ := range status.Jobs { - jobNames = append(jobNames, name) - } - sort.Slice(jobNames, func(i, j int) bool { - return strings.Compare(jobNames[i], jobNames[j]) == -1 - }) - now := time.Now() - for _, name := range jobNames { - - if controlStatusCmdArgs.onlyShowJob != "" && name != controlStatusCmdArgs.onlyShowJob { - continue - } - - job := status.Jobs[name] - jobLogEntries := make([]logger.Entry, 0) - informAboutError := false - - fmt.Printf("Job '%s':\n", name) - for _, task := range job.Tasks { - - var header bytes.Buffer - fmt.Fprintf(&header, " Task '%s': ", task.Name) - if !task.Idle { - fmt.Fprint(&header, strings.Join(task.ActivityStack, ".")) - } else { - fmt.Fprint(&header, "") - } - fmt.Fprint(&header, " ") - const TASK_STALLED_HOLDOFF_DURATION = 10 * time.Second - sinceLastUpdate := now.Sub(task.LastUpdate) - if !task.Idle || task.ProgressRx != 0 || task.ProgressTx != 0 { - fmt.Fprintf(&header, "(%s / %s , Rx/Tx", - humanize.Bytes(uint64(task.ProgressRx)), - humanize.Bytes(uint64(task.ProgressTx))) - if task.Idle { - fmt.Fprint(&header, ", values from last run") - } - fmt.Fprint(&header, ")") - } - fmt.Fprint(&header, "\n") - if !task.Idle && !task.LastUpdate.IsZero() && sinceLastUpdate >= TASK_STALLED_HOLDOFF_DURATION { - informAboutError = true - fmt.Fprintf(&header, " WARNING: last update %s ago at %s)", - sinceLastUpdate.String(), - task.LastUpdate.Format(HumanFormatterDateFormat)) - fmt.Fprint(&header, "\n") - } - io.Copy(os.Stdout, &header) - - jobLogEntries = append(jobLogEntries, task.LogEntries...) - informAboutError = informAboutError || task.MaxLogLevel >= logger.Warn - } - - sort.Slice(jobLogEntries, func(i, j int) bool { - return jobLogEntries[i].Time.Before(jobLogEntries[j].Time) - }) - if informAboutError { - fmt.Println(" WARNING: Some tasks encountered problems since the last time they left idle state:") - fmt.Println(" check the logs below or your log file for more information.") - fmt.Println(" Use the --level flag if you need debug information.") - fmt.Println() - } - for _, e := range jobLogEntries { - if e.Level < controlStatusCmdArgs.level { - continue - } - formatted, err := formatter.Format(&e) - if err != nil { - panic(err) - } - fmt.Printf(" %s\n", string(formatted)) - } - fmt.Println() - - } - default: - log.Printf("invalid output format '%s'", controlStatusCmdArgs.format) - die() - } - -} diff --git a/cmd/daemon.go b/cmd/daemon.go index 4ccd6f8..6593114 100644 --- a/cmd/daemon.go +++ b/cmd/daemon.go @@ -1,16 +1,13 @@ package cmd import ( - "container/list" "context" "fmt" "github.com/spf13/cobra" "github.com/zrepl/zrepl/logger" - "io" "os" "os/signal" - //"strings" - "sync" + "syscall" "time" ) @@ -30,7 +27,6 @@ type Job interface { JobName() string JobType() JobType JobStart(ctxt context.Context) - JobStatus(ctxt context.Context) (*JobStatus, error) } type JobType string @@ -155,315 +151,3 @@ outer: log.Info("exiting") } - -// Representation of a Job's status that is composed of Tasks -type JobStatus struct { - // Statuses of all tasks of this job - Tasks []*TaskStatus - // Error != "" if JobStatus() returned an error - JobStatusError string -} - -// Representation of a Daemon's status that is composed of Jobs -type DaemonStatus struct { - StartedAt time.Time - Jobs map[string]*JobStatus -} - -func (d *Daemon) Status() (s *DaemonStatus) { - - s = &DaemonStatus{} - s.StartedAt = d.startedAt - - s.Jobs = make(map[string]*JobStatus, len(d.conf.Jobs)) - - for name, j := range d.conf.Jobs { - status, err := j.JobStatus(context.TODO()) - if err != nil { - s.Jobs[name] = &JobStatus{nil, err.Error()} - continue - } - s.Jobs[name] = status - } - - return -} - -// Representation of a Task's status -type TaskStatus struct { - Name string - // Whether the task is idle. - Idle bool - // The stack of activities the task is currently executing. - // The first element is the root activity and equal to Name. - ActivityStack []string - // Number of bytes received by the task since it last left idle state. - ProgressRx int64 - // Number of bytes sent by the task since it last left idle state. - ProgressTx int64 - // Log entries emitted by the task since it last left idle state. - // Only contains the log entries emitted through the task's logger - // (provided by Task.Log()). - LogEntries []logger.Entry - // The maximum log level of LogEntries. - // Only valid if len(LogEntries) > 0. - MaxLogLevel logger.Level - // Last time something about the Task changed - LastUpdate time.Time -} - -// An instance of Task tracks a single thread of activity that is part of a Job. -type Task struct { - name string // immutable - parent Job // immutable - - // Stack of activities the task is currently in - // Members are instances of taskActivity - activities *list.List - // Last time activities was changed (not the activities inside, the list) - activitiesLastUpdate time.Time - // Protects Task members from modification - rwl sync.RWMutex -} - -// Structure that describes the progress a Task has made -type taskProgress struct { - rx int64 - tx int64 - creation time.Time - lastUpdate time.Time - logEntries []logger.Entry - mtx sync.RWMutex -} - -func newTaskProgress() (p *taskProgress) { - return &taskProgress{ - creation: time.Now(), - logEntries: make([]logger.Entry, 0), - } -} - -func (p *taskProgress) UpdateIO(drx, dtx int64) { - p.mtx.Lock() - defer p.mtx.Unlock() - p.rx += drx - p.tx += dtx - p.lastUpdate = time.Now() -} - -func (p *taskProgress) UpdateLogEntry(entry logger.Entry) { - p.mtx.Lock() - defer p.mtx.Unlock() - // FIXME: ensure maximum size (issue #48) - p.logEntries = append(p.logEntries, entry) - p.lastUpdate = time.Now() -} - -func (p *taskProgress) DeepCopy() (out taskProgress) { - p.mtx.RLock() - defer p.mtx.RUnlock() - out.rx, out.tx = p.rx, p.tx - out.creation = p.creation - out.lastUpdate = p.lastUpdate - out.logEntries = make([]logger.Entry, len(p.logEntries)) - for i := range p.logEntries { - out.logEntries[i] = p.logEntries[i] - } - return -} - -// returns a copy of this taskProgress, the mutex carries no semantic value -func (p *taskProgress) Read() (out taskProgress) { - p.mtx.RLock() - defer p.mtx.RUnlock() - return p.DeepCopy() -} - -// Element of a Task's activity stack -type taskActivity struct { - name string - idle bool - logger logger.Logger - // The progress of the task that is updated by UpdateIO() and UpdateLogEntry() - // - // Progress happens on a task-level and is thus global to the task. - // That's why progress is just a pointer to the current taskProgress: - // we reset progress when leaving the idle root activity - progress *taskProgress -} - -func NewTask(name string, parent Job, lg logger.Logger) *Task { - t := &Task{ - name: name, - parent: parent, - activities: list.New(), - } - rootLogger := lg.ReplaceField(logTaskField, name). - WithOutlet(t, logger.Debug) - rootAct := &taskActivity{name, true, rootLogger, newTaskProgress()} - t.activities.PushFront(rootAct) - return t -} - -// callers must hold t.rwl -func (t *Task) cur() *taskActivity { - return t.activities.Front().Value.(*taskActivity) -} - -// buildActivityStack returns the stack of activity names -// t.rwl must be held, but the slice can be returned since strings are immutable -func (t *Task) buildActivityStack() []string { - comps := make([]string, 0, t.activities.Len()) - for e := t.activities.Back(); e != nil; e = e.Prev() { - act := e.Value.(*taskActivity) - comps = append(comps, act.name) - } - return comps -} - -// Start a sub-activity. -// Must always be matched with a call to t.Finish() -// --- consider using defer for this purpose. -func (t *Task) Enter(activity string) { - t.rwl.Lock() - defer t.rwl.Unlock() - - prev := t.cur() - if prev.idle { - // reset progress when leaving idle task - // we leave the old progress dangling to have the user not worry about - prev.progress = newTaskProgress() - - prom.taskLastActiveStart.WithLabelValues( - t.parent.JobName(), - t.parent.JobType().String(), - t.name). - Set(float64(prev.progress.creation.UnixNano()) / 1e9) - - } - act := &taskActivity{activity, false, nil, prev.progress} - t.activities.PushFront(act) - //stack := t.buildActivityStack() - //activityField := strings.Join(stack, ".") - act.logger = prev.logger - // act.logger = prev.logger.ReplaceField(logTaskField, activityField) - - t.activitiesLastUpdate = time.Now() -} - -func (t *Task) UpdateProgress(dtx, drx int64) { - t.rwl.RLock() - p := t.cur().progress // protected by own rwlock - t.rwl.RUnlock() - p.UpdateIO(dtx, drx) -} - -// Returns a wrapper io.Reader that updates this task's _current_ progress value. -// Progress updates after this task resets its progress value are discarded. -func (t *Task) ProgressUpdater(r io.Reader) *IOProgressUpdater { - t.rwl.RLock() - defer t.rwl.RUnlock() - return &IOProgressUpdater{r, t.cur().progress} -} - -func (t *Task) Status() *TaskStatus { - t.rwl.RLock() - defer t.rwl.RUnlock() - // NOTE - // do not return any state in TaskStatus that is protected by t.rwl - - cur := t.cur() - stack := t.buildActivityStack() - prog := cur.progress.Read() - - var maxLevel logger.Level - for _, entry := range prog.logEntries { - if maxLevel < entry.Level { - maxLevel = entry.Level - } - } - - lastUpdate := prog.lastUpdate - if lastUpdate.Before(t.activitiesLastUpdate) { - lastUpdate = t.activitiesLastUpdate - } - - s := &TaskStatus{ - Name: stack[0], - ActivityStack: stack, - Idle: cur.idle, - ProgressRx: prog.rx, - ProgressTx: prog.tx, - LogEntries: prog.logEntries, - MaxLogLevel: maxLevel, - LastUpdate: lastUpdate, - } - - return s -} - -// Finish a sub-activity. -// Corresponds to a preceding call to t.Enter() -func (t *Task) Finish() { - t.rwl.Lock() - defer t.rwl.Unlock() - top := t.activities.Front() - if top.Next() == nil { - return // cannot remove root activity - } - t.activities.Remove(top) - t.activitiesLastUpdate = time.Now() - - // prometheus - front := t.activities.Front() - if front != nil && front == t.activities.Back() { - idleAct := front.Value.(*taskActivity) - if !idleAct.idle { - panic("inconsistent implementation") - } - progress := idleAct.progress.Read() - non_idle_time := t.activitiesLastUpdate.Sub(progress.creation) // use same time - prom.taskLastActiveDuration.WithLabelValues( - t.parent.JobName(), - t.parent.JobType().String(), - t.name).Set(non_idle_time.Seconds()) - } - -} - -// Returns a logger derived from the logger passed to the constructor function. -// The logger's task field contains the current activity stack joined by '.'. -func (t *Task) Log() logger.Logger { - t.rwl.RLock() - defer t.rwl.RUnlock() - // FIXME should influence TaskStatus's LastUpdate field - return t.cur().logger -} - -// implement logger.Outlet interface -func (t *Task) WriteEntry(entry logger.Entry) error { - t.rwl.RLock() - defer t.rwl.RUnlock() - t.cur().progress.UpdateLogEntry(entry) - - prom.taskLogEntries.WithLabelValues( - t.parent.JobName(), - t.parent.JobType().String(), - t.name, - entry.Level.String()). - Inc() - - return nil -} - -type IOProgressUpdater struct { - r io.Reader - p *taskProgress -} - -func (u *IOProgressUpdater) Read(p []byte) (n int, err error) { - n, err = u.r.Read(p) - u.p.UpdateIO(int64(n), 0) - return - -} diff --git a/cmd/prune.go b/cmd/prune.go index 82d4425..33ba755 100644 --- a/cmd/prune.go +++ b/cmd/prune.go @@ -8,7 +8,6 @@ import ( ) type Pruner struct { - task *Task Now time.Time DryRun bool DatasetFilter zfs.DatasetFilter @@ -23,25 +22,21 @@ type PruneResult struct { Remove []zfs.FilesystemVersion } -func (p *Pruner) filterFilesystems() (filesystems []*zfs.DatasetPath, stop bool) { - p.task.Enter("filter_fs") - defer p.task.Finish() +func (p *Pruner) filterFilesystems(ctx context.Context) (filesystems []*zfs.DatasetPath, stop bool) { filesystems, err := zfs.ZFSListMapping(p.DatasetFilter) if err != nil { - p.task.Log().WithError(err).Error("error applying filesystem filter") + getLogger(ctx).WithError(err).Error("error applying filesystem filter") return nil, true } if len(filesystems) <= 0 { - p.task.Log().Info("no filesystems matching filter") + getLogger(ctx).Info("no filesystems matching filter") return nil, true } return filesystems, false } -func (p *Pruner) filterVersions(fs *zfs.DatasetPath) (fsversions []zfs.FilesystemVersion, stop bool) { - p.task.Enter("filter_versions") - defer p.task.Finish() - log := p.task.Log().WithField("fs", fs.ToString()) +func (p *Pruner) filterVersions(ctx context.Context, fs *zfs.DatasetPath) (fsversions []zfs.FilesystemVersion, stop bool) { + log := getLogger(ctx).WithField("fs", fs.ToString()) filter := NewPrefixFilter(p.SnapshotPrefix) fsversions, err := zfs.ZFSListFilesystemVersions(fs, filter) @@ -56,19 +51,15 @@ func (p *Pruner) filterVersions(fs *zfs.DatasetPath) (fsversions []zfs.Filesyste return fsversions, false } -func (p *Pruner) pruneFilesystem(fs *zfs.DatasetPath) (r PruneResult, valid bool) { - p.task.Enter("prune_fs") - defer p.task.Finish() - log := p.task.Log().WithField("fs", fs.ToString()) +func (p *Pruner) pruneFilesystem(ctx context.Context, fs *zfs.DatasetPath) (r PruneResult, valid bool) { + log := getLogger(ctx).WithField("fs", fs.ToString()) - fsversions, stop := p.filterVersions(fs) + fsversions, stop := p.filterVersions(ctx, fs) if stop { return } - p.task.Enter("prune_policy") keep, remove, err := p.PrunePolicy.Prune(fs, fsversions) - p.task.Finish() if err != nil { log.WithError(err).Error("error evaluating prune policy") return @@ -100,9 +91,7 @@ func (p *Pruner) pruneFilesystem(fs *zfs.DatasetPath) (r PruneResult, valid bool // TODO special handling for EBUSY (zfs hold) // TODO error handling for clones? just echo to cli, skip over, and exit with non-zero status code (we're idempotent) if !p.DryRun { - p.task.Enter("destroy") err := zfs.ZFSDestroyFilesystemVersion(fs, v) - p.task.Finish() if err != nil { log.WithFields(fields).WithError(err).Error("error destroying version") } @@ -112,14 +101,11 @@ func (p *Pruner) pruneFilesystem(fs *zfs.DatasetPath) (r PruneResult, valid bool } func (p *Pruner) Run(ctx context.Context) (r []PruneResult, err error) { - p.task.Enter("run") - defer p.task.Finish() - if p.DryRun { - p.task.Log().Info("doing dry run") + getLogger(ctx).Info("doing dry run") } - filesystems, stop := p.filterFilesystems() + filesystems, stop := p.filterFilesystems(ctx) if stop { return } @@ -127,7 +113,7 @@ func (p *Pruner) Run(ctx context.Context) (r []PruneResult, err error) { r = make([]PruneResult, 0, len(filesystems)) for _, fs := range filesystems { - res, ok := p.pruneFilesystem(fs) + res, ok := p.pruneFilesystem(ctx, fs) if ok { r = append(r, res) } diff --git a/cmd/test.go b/cmd/test.go index 08d34b5..78f4e71 100644 --- a/cmd/test.go +++ b/cmd/test.go @@ -170,8 +170,7 @@ func doTestPrunePolicy(cmd *cobra.Command, args []string) { log.Printf("job dump:\n%s", pretty.Sprint(jobp)) - task := NewTask("", jobi, log) - pruner, err := jobp.Pruner(task, testPrunePolicyArgs.side, true) + pruner, err := jobp.Pruner(testPrunePolicyArgs.side, true) if err != nil { log.Printf("cannot create test pruner: %s", err) os.Exit(1) From ee5445777db3c9b91b3e2ceb1a7f3b424793f9f6 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 26 Aug 2018 19:03:50 +0200 Subject: [PATCH 039/167] logging format 'human': continue printing prefixed fields if some are missing --- cmd/config_job_prometheus.go | 1 - cmd/config_job_pull.go | 2 +- cmd/logging_formatters.go | 13 ++++++------- replication/mainfsm.go | 1 - 4 files changed, 7 insertions(+), 10 deletions(-) diff --git a/cmd/config_job_prometheus.go b/cmd/config_job_prometheus.go index 8e0427b..62400f2 100644 --- a/cmd/config_job_prometheus.go +++ b/cmd/config_job_prometheus.go @@ -91,4 +91,3 @@ func (j *PrometheusJob) JobStart(ctx context.Context) { } } - diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index 711e3d6..a77e161 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -26,7 +26,7 @@ type PullJob struct { Prune PrunePolicy Debug JobDebugSettings - rep *replication.Replication + rep *replication.Replication } func parsePullJob(c JobParsingContext, name string, i map[string]interface{}) (j *PullJob, err error) { diff --git a/cmd/logging_formatters.go b/cmd/logging_formatters.go index 444f3d3..49ef88b 100644 --- a/cmd/logging_formatters.go +++ b/cmd/logging_formatters.go @@ -77,13 +77,12 @@ func (f *HumanFormatter) Format(e *logger.Entry) (out []byte, err error) { prefixed := make(map[string]bool, len(prefixFields)+2) for _, field := range prefixFields { val, ok := e.Fields[field].(string) - if ok { - if !f.ignored(field) { - fmt.Fprintf(&line, "[%s]", val) - prefixed[field] = true - } - } else { - break + if !ok { + continue + } + if !f.ignored(field) { + fmt.Fprintf(&line, "[%s]", val) + prefixed[field] = true } } diff --git a/replication/mainfsm.go b/replication/mainfsm.go index d7b2462..7d9bfc5 100644 --- a/replication/mainfsm.go +++ b/replication/mainfsm.go @@ -105,7 +105,6 @@ type Receiver interface { fsrep.Receiver } - type FilteredError struct{ fs string } func NewFilteredError(fs string) *FilteredError { From 428339e1addf5c011b1f171ca8cd1845efd42a9c Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 26 Aug 2018 21:57:19 +0200 Subject: [PATCH 040/167] move version info to separate package --- Makefile | 2 +- cmd/control.go | 2 +- cmd/main.go | 4 ---- cmd/version.go | 24 ++---------------------- version/version.go | 31 +++++++++++++++++++++++++++++++ 5 files changed, 35 insertions(+), 28 deletions(-) create mode 100644 version/version.go diff --git a/Makefile b/Makefile index fd22ce1..d9f4670 100644 --- a/Makefile +++ b/Makefile @@ -20,7 +20,7 @@ ifndef ZREPL_VERSION $(error cannot infer variable ZREPL_VERSION using git and variable is not overriden by make invocation) endif endif -GO_LDFLAGS := "-X github.com/zrepl/zrepl/cmd.zreplVersion=$(ZREPL_VERSION)" +GO_LDFLAGS := "-X github.com/zrepl/zrepl/version.zreplVersion=$(ZREPL_VERSION)" GO_BUILD := go build -ldflags $(GO_LDFLAGS) diff --git a/cmd/control.go b/cmd/control.go index b65579c..2cec607 100644 --- a/cmd/control.go +++ b/cmd/control.go @@ -142,7 +142,7 @@ func doControLVersionCmd(cmd *cobra.Command, args []string) { die() } - var info ZreplVersionInformation + var info version.ZreplVersionInformation err = json.NewDecoder(resp.Body).Decode(&info) if err != nil { log.Printf("error unmarshaling response: %s", err) diff --git a/cmd/main.go b/cmd/main.go index 242367b..d459cd5 100644 --- a/cmd/main.go +++ b/cmd/main.go @@ -20,10 +20,6 @@ import ( // Printf(format string, v ...interface{}) //} -var ( - zreplVersion string // set by build infrastructure -) - type Logger = logger.Logger var RootCmd = &cobra.Command{ diff --git a/cmd/version.go b/cmd/version.go index 4597d2d..d4abe03 100644 --- a/cmd/version.go +++ b/cmd/version.go @@ -3,7 +3,7 @@ package cmd import ( "fmt" "github.com/spf13/cobra" - "runtime" + "github.com/zrepl/zrepl/version" ) var versionCmd = &cobra.Command{ @@ -17,26 +17,6 @@ func init() { } func doVersion(cmd *cobra.Command, args []string) { - fmt.Println(NewZreplVersionInformation().String()) + fmt.Println(version.NewZreplVersionInformation().String()) } -type ZreplVersionInformation struct { - Version string - RuntimeGOOS string - RuntimeGOARCH string - RUNTIMECompiler string -} - -func NewZreplVersionInformation() *ZreplVersionInformation { - return &ZreplVersionInformation{ - Version: zreplVersion, - RuntimeGOOS: runtime.GOOS, - RuntimeGOARCH: runtime.GOARCH, - RUNTIMECompiler: runtime.Compiler, - } -} - -func (i *ZreplVersionInformation) String() string { - return fmt.Sprintf("zrepl version=%s GOOS=%s GOARCH=%s Compiler=%s", - i.Version, i.RuntimeGOOS, i.RuntimeGOARCH, i.RUNTIMECompiler) -} diff --git a/version/version.go b/version/version.go new file mode 100644 index 0000000..8866f6f --- /dev/null +++ b/version/version.go @@ -0,0 +1,31 @@ +package version + +import ( + "runtime" + "fmt" +) + +var ( + zreplVersion string // set by build infrastructure +) + +type ZreplVersionInformation struct { + Version string + RuntimeGOOS string + RuntimeGOARCH string + RUNTIMECompiler string +} + +func NewZreplVersionInformation() *ZreplVersionInformation { + return &ZreplVersionInformation{ + Version: zreplVersion, + RuntimeGOOS: runtime.GOOS, + RuntimeGOARCH: runtime.GOARCH, + RUNTIMECompiler: runtime.Compiler, + } +} + +func (i *ZreplVersionInformation) String() string { + return fmt.Sprintf("zrepl version=%s GOOS=%s GOARCH=%s Compiler=%s", + i.Version, i.RuntimeGOOS, i.RuntimeGOARCH, i.RUNTIMECompiler) +} From 6425c26b1b5f9d8d5751bc547304b5dc601b9c74 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 26 Aug 2018 21:58:58 +0200 Subject: [PATCH 041/167] start refactoring: move daemon into subpackage --- cmd/config_parse.go | 9 - cmd/config_serve_stdinserver.go | 3 +- cmd/control.go | 8 +- cmd/daemon.go | 28 ++- .../control.go} | 37 ++-- cmd/daemon/daemon.go | 162 ++++++++++++++++++ cmd/daemon/job/job.go | 47 +++++ cmd/{control_pprof.go => daemon/pprof.go} | 2 +- .../prometheus.go} | 37 ++-- cmd/{ => helpers}/helpers.go | 6 +- 10 files changed, 271 insertions(+), 68 deletions(-) rename cmd/{config_job_control.go => daemon/control.go} (78%) create mode 100644 cmd/daemon/daemon.go create mode 100644 cmd/daemon/job/job.go rename cmd/{control_pprof.go => daemon/pprof.go} (99%) rename cmd/{config_job_prometheus.go => daemon/prometheus.go} (69%) rename cmd/{ => helpers}/helpers.go (98%) diff --git a/cmd/config_parse.go b/cmd/config_parse.go index 3a5b1eb..36a2406 100644 --- a/cmd/config_parse.go +++ b/cmd/config_parse.go @@ -137,13 +137,6 @@ func parseConfig(i interface{}) (c *Config, err error) { c.Jobs[job.JobName()] = job } - cj, err := NewControlJob(JobNameControl, jpc.Global.Control.Sockpath) - if err != nil { - err = errors.Wrap(err, "cannot create control job") - return - } - c.Jobs[JobNameControl] = cj - return c, nil } @@ -201,8 +194,6 @@ func parseJob(c JobParsingContext, i map[string]interface{}) (j Job, err error) return parseSourceJob(c, name, i) case JobTypeLocal: return parseLocalJob(c, name, i) - case JobTypePrometheus: - return parsePrometheusJob(c, name, i) default: panic(fmt.Sprintf("implementation error: unknown job type %s", jobtype)) } diff --git a/cmd/config_serve_stdinserver.go b/cmd/config_serve_stdinserver.go index 2380cc8..4dd8e86 100644 --- a/cmd/config_serve_stdinserver.go +++ b/cmd/config_serve_stdinserver.go @@ -6,6 +6,7 @@ import ( "github.com/problame/go-netssh" "net" "path" + "github.com/zrepl/zrepl/cmd/helpers" ) type StdinserverListenerFactory struct { @@ -32,7 +33,7 @@ func parseStdinserverListenerFactory(c JobParsingContext, i map[string]interface func (f *StdinserverListenerFactory) Listen() (net.Listener, error) { - if err := PreparePrivateSockpath(f.sockpath); err != nil { + if err := helpers.PreparePrivateSockpath(f.sockpath); err != nil { return nil, err } diff --git a/cmd/control.go b/cmd/control.go index 2cec607..fee5349 100644 --- a/cmd/control.go +++ b/cmd/control.go @@ -13,6 +13,8 @@ import ( "net" "net/http" "os" + "github.com/zrepl/zrepl/version" + "github.com/zrepl/zrepl/cmd/daemon" ) var controlCmd = &cobra.Command{ @@ -48,7 +50,7 @@ var pprofCmd = &cobra.Command{ }, } var pprofCmdArgs struct { - msg PprofServerControlMsg + msg daemon.PprofServerControlMsg } var controlVersionCmd = &cobra.Command{ @@ -107,7 +109,7 @@ func doControlPProf(cmd *cobra.Command, args []string) { log.Printf("error marshaling request: %s", err) die() } - _, err = httpc.Post("http://unix"+ControlJobEndpointPProf, "application/json", &buf) + _, err = httpc.Post("http://unix"+daemon.ControlJobEndpointPProf, "application/json", &buf) if err != nil { log.Printf("error: %s", err) die() @@ -131,7 +133,7 @@ func doControLVersionCmd(cmd *cobra.Command, args []string) { die() } - resp, err := httpc.Get("http://unix" + ControlJobEndpointVersion) + resp, err := httpc.Get("http://unix" + daemon.ControlJobEndpointVersion) if err != nil { log.Printf("error: %s", err) die() diff --git a/cmd/daemon.go b/cmd/daemon.go index 6593114..94a32d4 100644 --- a/cmd/daemon.go +++ b/cmd/daemon.go @@ -4,12 +4,14 @@ import ( "context" "fmt" "github.com/spf13/cobra" - "github.com/zrepl/zrepl/logger" "os" "os/signal" "syscall" "time" + "github.com/zrepl/zrepl/cmd/daemon" + "github.com/zrepl/zrepl/cmd/daemon/job" + "github.com/zrepl/zrepl/logger" ) // daemonCmd represents the daemon command @@ -57,6 +59,20 @@ func (j JobType) String() string { return string(j) } +type daemonJobAdaptor struct { + j Job +} + +func (a daemonJobAdaptor) Name() string { + return a.j.JobName() +} + +func (a daemonJobAdaptor) Run(ctx context.Context) { + a.j.JobStart(ctx) +} + +func (a daemonJobAdaptor) Status() interface{} { return nil } + func doDaemon(cmd *cobra.Command, args []string) { conf, err := ParseConfig(rootArgs.configFile) @@ -66,13 +82,13 @@ func doDaemon(cmd *cobra.Command, args []string) { } log := logger.NewLogger(conf.Global.logging.Outlets, 1*time.Second) - - log.Info(NewZreplVersionInformation().String()) - log.Debug("starting daemon") ctx := WithLogger(context.Background(), log) - d := NewDaemon(conf) - d.Loop(ctx) + daemonJobs := make([]job.Job, 0, len(conf.Jobs)) + for i := range conf.Jobs { + daemonJobs = append(daemonJobs, daemonJobAdaptor{conf.Jobs[i]}) + } + daemon.Run(ctx, conf.Global.Control.Sockpath, conf.Global.logging.Outlets, daemonJobs) } diff --git a/cmd/config_job_control.go b/cmd/daemon/control.go similarity index 78% rename from cmd/config_job_control.go rename to cmd/daemon/control.go index 3c0d802..c7d17a1 100644 --- a/cmd/config_job_control.go +++ b/cmd/daemon/control.go @@ -1,4 +1,4 @@ -package cmd +package daemon import ( "bytes" @@ -9,15 +9,18 @@ import ( "io" "net" "net/http" + "github.com/zrepl/zrepl/cmd/daemon/job" + "github.com/zrepl/zrepl/version" + "github.com/zrepl/zrepl/cmd/helpers" ) -type ControlJob struct { - Name string +type controlJob struct { sockaddr *net.UnixAddr + jobs *jobs } -func NewControlJob(name, sockpath string) (j *ControlJob, err error) { - j = &ControlJob{Name: name} +func newControlJob(sockpath string, jobs *jobs) (j *controlJob, err error) { + j = &controlJob{jobs: jobs} j.sockaddr, err = net.ResolveUnixAddr("unix", sockpath) if err != nil { @@ -28,11 +31,9 @@ func NewControlJob(name, sockpath string) (j *ControlJob, err error) { return } -func (j *ControlJob) JobName() string { - return j.Name -} +func (j *controlJob) Name() string { return jobNameControl } -func (j *ControlJob) JobType() JobType { return JobTypeControl } +func (j *controlJob) Status() interface{} { return nil } const ( ControlJobEndpointPProf string = "/debug/pprof" @@ -40,14 +41,12 @@ const ( ControlJobEndpointStatus string = "/status" ) -func (j *ControlJob) JobStart(ctx context.Context) { +func (j *controlJob) Run(ctx context.Context) { - log := getLogger(ctx) + log := job.GetLogger(ctx) defer log.Info("control job finished") - daemon := ctx.Value(contextKeyDaemon).(*Daemon) - - l, err := ListenUnixPrivate(j.sockaddr) + l, err := helpers.ListenUnixPrivate(j.sockaddr) if err != nil { log.WithError(err).Error("error listening") return @@ -68,16 +67,12 @@ func (j *ControlJob) JobStart(ctx context.Context) { }}) mux.Handle(ControlJobEndpointVersion, requestLogger{log: log, handler: jsonResponder{func() (interface{}, error) { - return NewZreplVersionInformation(), nil + return version.NewZreplVersionInformation(), nil }}}) mux.Handle(ControlJobEndpointStatus, requestLogger{log: log, handler: jsonResponder{func() (interface{}, error) { - panic("FIXME") // FIXME - }}}) - mux.Handle("/pulljobreport", - requestLogger{log: log, handler: jsonResponder{func() (interface{}, error) { - j := daemon.conf.Jobs["debian"] - return j.(*PullJob).Report(), nil + s := j.jobs.status() + return s, nil }}}) server := http.Server{Handler: mux} diff --git a/cmd/daemon/daemon.go b/cmd/daemon/daemon.go new file mode 100644 index 0000000..6240d01 --- /dev/null +++ b/cmd/daemon/daemon.go @@ -0,0 +1,162 @@ +package daemon + +import ( + "context" + "os" + "os/signal" + "syscall" + "sync" + "fmt" + "github.com/zrepl/zrepl/cmd/daemon/job" + "strings" + "github.com/zrepl/zrepl/logger" + "github.com/zrepl/zrepl/version" + "time" +) + + +func Run(ctx context.Context, controlSockpath string, outlets *logger.Outlets, confJobs []job.Job) { + + ctx, cancel := context.WithCancel(ctx) + defer cancel() + sigChan := make(chan os.Signal, 1) + signal.Notify(sigChan, syscall.SIGINT, syscall.SIGTERM) + go func() { + <-sigChan + cancel() + }() + + log := logger.NewLogger(outlets, 1*time.Second) + log.Info(version.NewZreplVersionInformation().String()) + + // parse config + for _, job := range confJobs { + if IsInternalJobName(job.Name()) { + panic(fmt.Sprintf("internal job name used for config job '%s'", job.Name())) //FIXME + } + } + + ctx = job.WithLogger(ctx, log) + + jobs := newJobs() + + // start control socket + controlJob, err := newControlJob(controlSockpath, jobs) + if err != nil { + panic(err) // FIXME + } + jobs.start(ctx, controlJob, true) + + // start prometheus + //var promJob *prometheusJob // FIXME + //jobs.start(ctx, promJob, true) + + log.Info("starting daemon") + + // start regular jobs + for _, j := range confJobs { + jobs.start(ctx, j, false) + } + + select { + case <-jobs.wait(): + log.Info("all jobs finished") + case <-ctx.Done(): + log.WithError(ctx.Err()).Info("context finished") + } + log.Info("daemon exiting") +} + +type jobs struct { + wg sync.WaitGroup + + // m protects all fields below it + m sync.RWMutex + wakeups map[string]job.WakeupChan // by JobName + jobs map[string]job.Job +} + +func newJobs() *jobs { + return &jobs{ + wakeups: make(map[string]job.WakeupChan), + jobs: make(map[string]job.Job), + } +} + +const ( + logJobField string = "job" + logTaskField string = "task" + logSubsysField string = "subsystem" +) + +func (s *jobs) wait() <-chan struct{} { + ch := make(chan struct{}) + go func() { + s.wg.Wait() + }() + return ch +} + +func (s *jobs) status() map[string]interface{} { + s.m.RLock() + defer s.m.RUnlock() + + type res struct { + name string + status interface{} + } + var wg sync.WaitGroup + c := make(chan res, len(s.jobs)) + for name, j := range s.jobs { + wg.Add(1) + go func(name string, j job.Job) { + defer wg.Done() + c <- res{name: name, status: j.Status()} + }(name, j) + } + wg.Wait() + close(c) + ret := make(map[string]interface{}, len(s.jobs)) + for res := range c { + ret[res.name] = res.status + } + return ret +} + +const ( + jobNamePrometheus = "_prometheus" + jobNameControl = "_control" +) + +func IsInternalJobName(s string) bool { + return strings.HasPrefix(s, "_") +} + +func (s *jobs) start(ctx context.Context, j job.Job, internal bool) { + s.m.Lock() + defer s.m.Unlock() + + jobLog := job.GetLogger(ctx).WithField(logJobField, j.Name()) + jobName := j.Name() + if !internal && IsInternalJobName(jobName) { + panic(fmt.Sprintf("internal job name used for non-internal job %s", jobName)) + } + if internal && !IsInternalJobName(jobName) { + panic(fmt.Sprintf("internal job does not use internal job name %s", jobName)) + } + if _, ok := s.jobs[jobName]; ok { + panic(fmt.Sprintf("duplicate job name %s", jobName)) + } + s.jobs[jobName] = j + ctx = job.WithLogger(ctx, jobLog) + ctx, wakeupChan := job.WithWakeup(ctx) + s.wakeups[jobName] = wakeupChan + + s.wg.Add(1) + go func() { + defer s.wg.Done() + jobLog.Info("starting job") + defer jobLog.Info("job exited") + j.Run(ctx) + }() +} diff --git a/cmd/daemon/job/job.go b/cmd/daemon/job/job.go new file mode 100644 index 0000000..59cc147 --- /dev/null +++ b/cmd/daemon/job/job.go @@ -0,0 +1,47 @@ +package job + +import ( + "github.com/zrepl/zrepl/logger" + "context" +) + +type Logger = logger.Logger + +type contextKey int + +const ( + contextKeyLog contextKey = iota + contextKeyWakeup +) + +func GetLogger(ctx context.Context) Logger { + if l, ok := ctx.Value(contextKeyLog).(Logger); ok { + return l + } + return logger.NewNullLogger() +} + +func WithLogger(ctx context.Context, l Logger) context.Context { + return context.WithValue(ctx, contextKeyLog, l) +} + +func WithWakeup(ctx context.Context) (context.Context, WakeupChan) { + wc := make(chan struct{}, 1) + return context.WithValue(ctx, contextKeyWakeup, wc), wc +} + +type Job interface { + Name() string + Run(ctx context.Context) + Status() interface{} +} + +type WakeupChan <-chan struct{} + +func WaitWakeup(ctx context.Context) WakeupChan { + wc, ok := ctx.Value(contextKeyWakeup).(WakeupChan) + if !ok { + wc = make(chan struct{}) + } + return wc +} diff --git a/cmd/control_pprof.go b/cmd/daemon/pprof.go similarity index 99% rename from cmd/control_pprof.go rename to cmd/daemon/pprof.go index a6b6939..2296ebd 100644 --- a/cmd/control_pprof.go +++ b/cmd/daemon/pprof.go @@ -1,4 +1,4 @@ -package cmd +package daemon import ( "net/http" diff --git a/cmd/config_job_prometheus.go b/cmd/daemon/prometheus.go similarity index 69% rename from cmd/config_job_prometheus.go rename to cmd/daemon/prometheus.go index 62400f2..afe4fd5 100644 --- a/cmd/config_job_prometheus.go +++ b/cmd/daemon/prometheus.go @@ -1,19 +1,21 @@ -package cmd +package daemon import ( "context" - "github.com/mitchellh/mapstructure" - "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" "github.com/zrepl/zrepl/zfs" "net" "net/http" + "github.com/zrepl/zrepl/cmd/daemon/job" ) -type PrometheusJob struct { - Name string - Listen string +type prometheusJob struct { + listen string +} + +func newPrometheusJob(listen string) *prometheusJob { + return &prometheusJob{listen} } var prom struct { @@ -46,32 +48,19 @@ func init() { prometheus.MustRegister(prom.taskLogEntries) } -func parsePrometheusJob(c JobParsingContext, name string, i map[string]interface{}) (j *PrometheusJob, err error) { - var s struct { - Listen string - } - if err := mapstructure.Decode(i, &s); err != nil { - return nil, errors.Wrap(err, "mapstructure error") - } - if s.Listen == "" { - return nil, errors.New("must specify 'listen' attribute") - } - return &PrometheusJob{name, s.Listen}, nil -} +func (j *prometheusJob) Name() string { return jobNamePrometheus } -func (j *PrometheusJob) JobName() string { return j.Name } +func (j *prometheusJob) Status() interface{} { return nil } -func (j *PrometheusJob) JobType() JobType { return JobTypePrometheus } - -func (j *PrometheusJob) JobStart(ctx context.Context) { +func (j *prometheusJob) Run(ctx context.Context) { if err := zfs.PrometheusRegister(prometheus.DefaultRegisterer); err != nil { panic(err) } - log := getLogger(ctx) + log := job.GetLogger(ctx) - l, err := net.Listen("tcp", j.Listen) + l, err := net.Listen("tcp", j.listen) if err != nil { log.WithError(err).Error("cannot listen") } diff --git a/cmd/helpers.go b/cmd/helpers/helpers.go similarity index 98% rename from cmd/helpers.go rename to cmd/helpers/helpers.go index be3864e..01d6a9c 100644 --- a/cmd/helpers.go +++ b/cmd/helpers/helpers.go @@ -1,10 +1,10 @@ -package cmd +package helpers import ( + "path/filepath" + "os" "github.com/pkg/errors" "net" - "os" - "path/filepath" ) func PreparePrivateSockpath(sockpath string) error { From 38bb78b6424a507e090431bb87aaea2150a6b626 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Sun, 26 Aug 2018 15:17:15 +0200 Subject: [PATCH 042/167] WIP new config format --- Gopkg.lock | 9 ++ Gopkg.toml | 2 +- cmd/config/config.go | 223 ++++++++++++++++++++++++++++++++++++ cmd/config/config_test.go | 27 +++++ cmd/config/samples/push.yml | 30 +++++ cmd/config/samples/sink.yml | 15 +++ 6 files changed, 305 insertions(+), 1 deletion(-) create mode 100644 cmd/config/config.go create mode 100644 cmd/config/config_test.go create mode 100644 cmd/config/samples/push.yml create mode 100644 cmd/config/samples/sink.yml diff --git a/Gopkg.lock b/Gopkg.lock index 4868440..f6a26f3 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -227,6 +227,14 @@ revision = "69483b4bd14f5845b5a1e55bca19e954e827f1d0" version = "v1.1.4" +[[projects]] + branch = "v2" + digest = "1:f0620375dd1f6251d9973b5f2596228cc8042e887cd7f827e4220bc1ce8c30e2" + name = "github.com/zrepl/yaml-config" + packages = ["."] + pruneopts = "" + revision = "5420a8b6744d3b0345ab293f6fcba19c978f1183" + [[projects]] branch = "master" digest = "1:9c286cf11d0ca56368185bada5dd6d97b6be4648fc26c354fcba8df7293718f7" @@ -256,6 +264,7 @@ "github.com/spf13/cobra", "github.com/stretchr/testify/assert", "github.com/stretchr/testify/require", + "github.com/zrepl/yaml-config", ] solver-name = "gps-cdcl" solver-version = 1 diff --git a/Gopkg.toml b/Gopkg.toml index 81becc1..041e666 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -38,7 +38,7 @@ ignored = [ "github.com/inconshreveable/mousetrap" ] [[constraint]] branch = "v2" - name = "github.com/go-yaml/yaml" + name = "github.com/zrepl/yaml-config" [[constraint]] name = "github.com/go-logfmt/logfmt" diff --git a/cmd/config/config.go b/cmd/config/config.go new file mode 100644 index 0000000..27af8cd --- /dev/null +++ b/cmd/config/config.go @@ -0,0 +1,223 @@ +package config + +import ( + "github.com/zrepl/yaml-config" + "fmt" + "time" + "os" + "github.com/pkg/errors" + "io/ioutil" +) + +type NodeEnum struct { + Ret interface{} +} + +type PushNode struct { + Type string `yaml:"type"` + Replication PushReplication `yaml:"replication"` + Snapshotting Snapshotting `yaml:"snapshotting"` + Pruning Pruning `yaml:"pruning"` + Global Global `yaml:"global"` +} + +type SinkNode struct { + Type string `yaml:"type"` + Replication SinkReplication `yaml:"replication"` + Global Global `yaml:"global"` +} + +type PushReplication struct { + Connect ConnectEnum `yaml:"connect"` + Filesystems map[string]bool `yaml:"filesystems"` +} + +type SinkReplication struct { + RootDataset string `yaml:"root_dataset"` + Serve ServeEnum `yaml:"serve"` +} + +type Snapshotting struct { + SnapshotPrefix string `yaml:"snapshot_prefix"` + Interval time.Duration `yaml:"interval"` +} + +type Pruning struct { + KeepLocal []PruningEnum `yaml:"keep_local"` + KeepRemote []PruningEnum `yaml:"keep_remote"` +} + +type Global struct { + Logging []LoggingOutlet `yaml:"logging"` +} + +type ConnectEnum struct { + Ret interface{} +} + +type TCPConnect struct { + Type string `yaml:"type"` + Address string `yaml:"address"` +} + +type TLSConnect struct { + Type string `yaml:"type"` + Address string `yaml:"address"` + Ca string `yaml:"ca"` + Cert string `yaml:"cert"` + Key string `yaml:"key"` +} + +type ServeEnum struct { + Ret interface{} +} + +type TCPServe struct { + Type string `yaml:"type"` + Listen string `yaml:"listen"` + Clients map[string]string `yaml:"clients"` +} + +type TLSServe struct { + Type string `yaml:"type"` + Listen string `yaml:"listen"` + Ca string `yaml:"ca"` + Cert string `yaml:"cert"` + Key string `yaml:"key"` +} + +type PruningEnum struct { + Ret interface{} +} + +type PruneKeepNotReplicated struct { + Type string `yaml:"type"` +} + +type PruneKeepLastN struct { + Type string `yaml:"type"` + Count int `yaml:"count"` +} + +type PruneGrid struct { + Type string `yaml:"type"` + Grid string `yaml:"grid"` +} + +type LoggingOutlet struct { + Outlet LoggingOutletEnum `yaml:"outlet"` + Level string `yaml:"level"` + Format string `yaml:"format"` +} + +type LoggingOutletEnum struct { + Ret interface{} +} + +type StdoutLoggingOutlet struct { + Type string `yaml:"type"` + Time bool `yaml:"time"` +} + +type SyslogLoggingOutlet struct { + Type string `yaml:"type"` + RetryInterval time.Duration `yaml:"retry_interval"` +} + +func enumUnmarshal(u func(interface{}, bool) error, types map[string]interface{}) (interface{}, error) { + var in struct { + Type string + } + if err := u(&in, true); err != nil { + return nil, err + } + if in.Type == "" { + return nil, &yaml.TypeError{[]string{"must specify type"}} + } + + v, ok := types[in.Type] + if !ok { + return nil, &yaml.TypeError{[]string{fmt.Sprintf("invalid type name %q", in.Type)}} + } + if err := u(v, false); err != nil { + return nil, err + } + return v, nil +} + +func (t *NodeEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { + t.Ret, err = enumUnmarshal(u, map[string]interface{}{ + "push": &PushNode{}, + "sink": &SinkNode{}, + }) + return +} + +func (t *ConnectEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { + t.Ret, err = enumUnmarshal(u, map[string]interface{}{ + "tcp": &TCPConnect{}, + "tls": &TLSConnect{}, + }) + return +} + +func (t *ServeEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { + t.Ret, err = enumUnmarshal(u, map[string]interface{}{ + "tcp": &TCPServe{}, + "tls": &TLSServe{}, + }) + return +} + +func (t *PruningEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { + t.Ret, err = enumUnmarshal(u, map[string]interface{}{ + "not_replicated": &PruneKeepNotReplicated{}, + "last_n": &PruneKeepLastN{}, + "grid": &PruneGrid{}, + }) + return +} + +func (t *LoggingOutletEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { + t.Ret, err = enumUnmarshal(u, map[string]interface{}{ + "stdout": &StdoutLoggingOutlet{}, + "syslog": &SyslogLoggingOutlet{}, + }) + return +} + +var ConfigFileDefaultLocations = []string{ + "/etc/zrepl/zrepl.yml", + "/usr/local/etc/zrepl/zrepl.yml", +} + +func ParseConfig(path string) (i NodeEnum, err error) { + + if path == "" { + // Try default locations + for _, l := range ConfigFileDefaultLocations { + stat, statErr := os.Stat(l) + if statErr != nil { + continue + } + if !stat.Mode().IsRegular() { + err = errors.Errorf("file at default location is not a regular file: %s", l) + return + } + path = l + break + } + } + + var bytes []byte + + if bytes, err = ioutil.ReadFile(path); err != nil { + return + } + + if err = yaml.UnmarshalStrict(bytes, &i); err != nil { + return + } + + return +} diff --git a/cmd/config/config_test.go b/cmd/config/config_test.go new file mode 100644 index 0000000..caab41a --- /dev/null +++ b/cmd/config/config_test.go @@ -0,0 +1,27 @@ +package config + +import ( + "testing" + "github.com/kr/pretty" + "path/filepath" +) + +func TestSampleConfigsAreParsedWithoutErrors(t *testing.T) { + paths, err := filepath.Glob("./samples/*") + if err != nil { + t.Errorf("glob failed: %+v", err) + } + + for _, p := range paths { + + c, err := ParseConfig(p) + if err != nil { + t.Errorf("error parsing %s:\n%+v", p, err) + } + + t.Logf("file: %s", p) + t.Log(pretty.Sprint(c)) + + } + +} diff --git a/cmd/config/samples/push.yml b/cmd/config/samples/push.yml new file mode 100644 index 0000000..9bec4d6 --- /dev/null +++ b/cmd/config/samples/push.yml @@ -0,0 +1,30 @@ +type: push +replication: + connect: + type: tcp + address: "backup-server.foo.bar:8888" + filesystems: { + "<": true, + "tmp": false + } +snapshotting: + snapshot_prefix: zrepl_ + interval: 10m +pruning: + keep_local: + - type: not_replicated + - type: last_n + count: 10 + - type: grid + grid: 1x1h(keep=all) | 24x1h | 14x1d + + keep_remote: + - type: grid + grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d +global: + logging: + - outlet: + type: "stdout" + time: true + level: "warn" + format: "human" diff --git a/cmd/config/samples/sink.yml b/cmd/config/samples/sink.yml new file mode 100644 index 0000000..e55236e --- /dev/null +++ b/cmd/config/samples/sink.yml @@ -0,0 +1,15 @@ +type: sink +replication: + root_dataset: "pool2/backup_laptops" + serve: + type: tls + listen: "192.168.122.189:8888" + ca: "ca.pem" + cert: "cert.pem" + key: "key.pem" +global: + logging: + - outlet: + type: "syslog" + level: "warn" + format: "human" From 13dc63bd23f8802b2d110767397584b84f0ce846 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Sun, 26 Aug 2018 16:44:34 +0200 Subject: [PATCH 043/167] build logger from new config --- cmd/config/config.go | 116 +++++++++++++++----------- cmd/config/config_test.go | 2 +- cmd/config/samples/push.yml | 50 ++++++------ cmd/config/samples/sink.yml | 27 +++--- cmd/config_logging.go | 159 ++++++++++++------------------------ cmd/daemon.go | 14 +++- 6 files changed, 170 insertions(+), 198 deletions(-) diff --git a/cmd/config/config.go b/cmd/config/config.go index 27af8cd..d7b52d9 100644 --- a/cmd/config/config.go +++ b/cmd/config/config.go @@ -1,54 +1,57 @@ package config import ( - "github.com/zrepl/yaml-config" "fmt" - "time" - "os" "github.com/pkg/errors" + "github.com/zrepl/yaml-config" "io/ioutil" + "os" + "time" ) -type NodeEnum struct { +type Config struct { + Jobs []JobEnum `yaml:"jobs"` + Global Global `yaml:"global"` +} + +type JobEnum struct { Ret interface{} } -type PushNode struct { - Type string `yaml:"type"` - Replication PushReplication `yaml:"replication"` - Snapshotting Snapshotting `yaml:"snapshotting"` - Pruning Pruning `yaml:"pruning"` - Global Global `yaml:"global"` +type PushJob struct { + Type string `yaml:"type"` + Replication PushReplication `yaml:"replication"` + Snapshotting Snapshotting `yaml:"snapshotting"` + Pruning Pruning `yaml:"pruning"` } -type SinkNode struct { - Type string `yaml:"type"` +type SinkJob struct { + Type string `yaml:"type"` Replication SinkReplication `yaml:"replication"` - Global Global `yaml:"global"` } type PushReplication struct { - Connect ConnectEnum `yaml:"connect"` + Connect ConnectEnum `yaml:"connect"` Filesystems map[string]bool `yaml:"filesystems"` } type SinkReplication struct { - RootDataset string `yaml:"root_dataset"` - Serve ServeEnum `yaml:"serve"` + RootDataset string `yaml:"root_dataset"` + Serve ServeEnum `yaml:"serve"` } type Snapshotting struct { - SnapshotPrefix string `yaml:"snapshot_prefix"` - Interval time.Duration `yaml:"interval"` + SnapshotPrefix string `yaml:"snapshot_prefix"` + Interval time.Duration `yaml:"interval"` } type Pruning struct { - KeepLocal []PruningEnum `yaml:"keep_local"` + KeepLocal []PruningEnum `yaml:"keep_local"` KeepRemote []PruningEnum `yaml:"keep_remote"` } type Global struct { - Logging []LoggingOutlet `yaml:"logging"` + Logging []LoggingOutletEnum `yaml:"logging"` } type ConnectEnum struct { @@ -56,16 +59,16 @@ type ConnectEnum struct { } type TCPConnect struct { - Type string `yaml:"type"` + Type string `yaml:"type"` Address string `yaml:"address"` } type TLSConnect struct { - Type string `yaml:"type"` + Type string `yaml:"type"` Address string `yaml:"address"` - Ca string `yaml:"ca"` - Cert string `yaml:"cert"` - Key string `yaml:"key"` + Ca string `yaml:"ca"` + Cert string `yaml:"cert"` + Key string `yaml:"key"` } type ServeEnum struct { @@ -73,17 +76,17 @@ type ServeEnum struct { } type TCPServe struct { - Type string `yaml:"type"` - Listen string `yaml:"listen"` + Type string `yaml:"type"` + Listen string `yaml:"listen"` Clients map[string]string `yaml:"clients"` } type TLSServe struct { - Type string `yaml:"type"` + Type string `yaml:"type"` Listen string `yaml:"listen"` - Ca string `yaml:"ca"` - Cert string `yaml:"cert"` - Key string `yaml:"key"` + Ca string `yaml:"ca"` + Cert string `yaml:"cert"` + Key string `yaml:"key"` } type PruningEnum struct { @@ -95,8 +98,8 @@ type PruneKeepNotReplicated struct { } type PruneKeepLastN struct { - Type string `yaml:"type"` - Count int `yaml:"count"` + Type string `yaml:"type"` + Count int `yaml:"count"` } type PruneGrid struct { @@ -104,24 +107,38 @@ type PruneGrid struct { Grid string `yaml:"grid"` } -type LoggingOutlet struct { - Outlet LoggingOutletEnum `yaml:"outlet"` - Level string `yaml:"level"` - Format string `yaml:"format"` -} - type LoggingOutletEnum struct { Ret interface{} } +type LoggingOutletCommon struct { + Type string `yaml:"type"` + Level string `yaml:"level"` + Format string `yaml:"format"` +} + type StdoutLoggingOutlet struct { - Type string `yaml:"type"` - Time bool `yaml:"time"` + LoggingOutletCommon `yaml:",inline"` + Time bool `yaml:"time"` } type SyslogLoggingOutlet struct { - Type string `yaml:"type"` - RetryInterval time.Duration `yaml:"retry_interval"` + LoggingOutletCommon `yaml:",inline"` + RetryInterval time.Duration `yaml:"retry_interval"` +} + +type TCPLoggingOutlet struct { + LoggingOutletCommon `yaml:",inline"` + Address string `yaml:"address"` //TODO required + Net string `yaml:"net"` //TODO default tcp + RetryInterval time.Duration `yaml:"retry_interval"` + TLS *TCPLoggingOutletTLS +} + +type TCPLoggingOutletTLS struct { + CA string `yaml:"ca"` + Cert string `yaml:"cert"` + Key string `yaml:"key"` } func enumUnmarshal(u func(interface{}, bool) error, types map[string]interface{}) (interface{}, error) { @@ -145,10 +162,10 @@ func enumUnmarshal(u func(interface{}, bool) error, types map[string]interface{} return v, nil } -func (t *NodeEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { +func (t *JobEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { t.Ret, err = enumUnmarshal(u, map[string]interface{}{ - "push": &PushNode{}, - "sink": &SinkNode{}, + "push": &PushJob{}, + "sink": &SinkJob{}, }) return } @@ -172,8 +189,8 @@ func (t *ServeEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { func (t *PruningEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { t.Ret, err = enumUnmarshal(u, map[string]interface{}{ "not_replicated": &PruneKeepNotReplicated{}, - "last_n": &PruneKeepLastN{}, - "grid": &PruneGrid{}, + "last_n": &PruneKeepLastN{}, + "grid": &PruneGrid{}, }) return } @@ -182,6 +199,7 @@ func (t *LoggingOutletEnum) UnmarshalYAML(u func(interface{}, bool) error) (err t.Ret, err = enumUnmarshal(u, map[string]interface{}{ "stdout": &StdoutLoggingOutlet{}, "syslog": &SyslogLoggingOutlet{}, + "tcp": &TCPLoggingOutlet{}, }) return } @@ -191,7 +209,7 @@ var ConfigFileDefaultLocations = []string{ "/usr/local/etc/zrepl/zrepl.yml", } -func ParseConfig(path string) (i NodeEnum, err error) { +func ParseConfig(path string) (i Config, err error) { if path == "" { // Try default locations diff --git a/cmd/config/config_test.go b/cmd/config/config_test.go index caab41a..ef4ca0c 100644 --- a/cmd/config/config_test.go +++ b/cmd/config/config_test.go @@ -1,9 +1,9 @@ package config import ( - "testing" "github.com/kr/pretty" "path/filepath" + "testing" ) func TestSampleConfigsAreParsedWithoutErrors(t *testing.T) { diff --git a/cmd/config/samples/push.yml b/cmd/config/samples/push.yml index 9bec4d6..5966465 100644 --- a/cmd/config/samples/push.yml +++ b/cmd/config/samples/push.yml @@ -1,30 +1,30 @@ -type: push -replication: - connect: - type: tcp - address: "backup-server.foo.bar:8888" - filesystems: { - "<": true, - "tmp": false - } -snapshotting: - snapshot_prefix: zrepl_ - interval: 10m -pruning: - keep_local: - - type: not_replicated - - type: last_n - count: 10 - - type: grid - grid: 1x1h(keep=all) | 24x1h | 14x1d +jobs: + - type: push + replication: + connect: + type: tcp + address: "backup-server.foo.bar:8888" + filesystems: { + "<": true, + "tmp": false + } + snapshotting: + snapshot_prefix: zrepl_ + interval: 10m + pruning: + keep_local: + - type: not_replicated + - type: last_n + count: 10 + - type: grid + grid: 1x1h(keep=all) | 24x1h | 14x1d - keep_remote: - - type: grid - grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d + keep_remote: + - type: grid + grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d global: logging: - - outlet: - type: "stdout" - time: true + - type: "stdout" + time: true level: "warn" format: "human" diff --git a/cmd/config/samples/sink.yml b/cmd/config/samples/sink.yml index e55236e..95f4500 100644 --- a/cmd/config/samples/sink.yml +++ b/cmd/config/samples/sink.yml @@ -1,15 +1,20 @@ -type: sink -replication: - root_dataset: "pool2/backup_laptops" - serve: - type: tls - listen: "192.168.122.189:8888" - ca: "ca.pem" - cert: "cert.pem" - key: "key.pem" +jobs: + - type: sink + replication: + root_dataset: "pool2/backup_laptops" + serve: + type: tls + listen: "192.168.122.189:8888" + ca: "ca.pem" + cert: "cert.pem" + key: "key.pem" global: logging: - - outlet: - type: "syslog" + - type: "tcp" + address: "123.123.123.123:1234" + tls: + ca: "ca.pem" + cert: "cert.pem" + key: "key.pem" level: "warn" format: "human" diff --git a/cmd/config_logging.go b/cmd/config_logging.go index 200f365..4f18f0a 100644 --- a/cmd/config_logging.go +++ b/cmd/config_logging.go @@ -4,12 +4,11 @@ import ( "crypto/tls" "crypto/x509" "github.com/mattn/go-isatty" - "github.com/mitchellh/mapstructure" "github.com/pkg/errors" + "github.com/zrepl/zrepl/cmd/config" "github.com/zrepl/zrepl/cmd/tlsconf" "github.com/zrepl/zrepl/logger" "os" - "time" ) type LoggingConfig struct { @@ -26,16 +25,12 @@ const ( MetadataAll MetadataFlags = ^0 ) -func parseLogging(i interface{}) (c *LoggingConfig, err error) { +func parseLogging(in []config.LoggingOutletEnum) (c *LoggingConfig, err error) { c = &LoggingConfig{} c.Outlets = logger.NewOutlets() - var asList []interface{} - if err = mapstructure.Decode(i, &asList); err != nil { - return nil, errors.Wrap(err, "mapstructure error") - } - if len(asList) == 0 { + if len(in) == 0 { // Default config out := WriterOutlet{&HumanFormatter{}, os.Stdout} c.Outlets.Add(out, logger.Warn) @@ -43,7 +38,7 @@ func parseLogging(i interface{}) (c *LoggingConfig, err error) { } var syslogOutlets, stdoutOutlets int - for lei, le := range asList { + for lei, le := range in { outlet, minLevel, err := parseOutlet(le) if err != nil { @@ -95,56 +90,52 @@ func parseLogFormat(i interface{}) (f EntryFormatter, err error) { } -func parseOutlet(i interface{}) (o logger.Outlet, level logger.Level, err error) { +func parseOutlet(in config.LoggingOutletEnum) (o logger.Outlet, level logger.Level, err error) { - var in struct { - Outlet string - Level string - Format string - } - if err = mapstructure.Decode(i, &in); err != nil { - err = errors.Wrap(err, "mapstructure error") - return - } - if in.Outlet == "" || in.Level == "" || in.Format == "" { - err = errors.Errorf("must specify 'outlet', 'level' and 'format' field") - return + parseCommon := func(common config.LoggingOutletCommon) (logger.Level, EntryFormatter, error) { + if common.Level == "" || common.Format == "" { + return 0, nil, errors.Errorf("must specify 'level' and 'format' field") + } + + minLevel, err := logger.ParseLevel(common.Level) + if err != nil { + return 0, nil, errors.Wrap(err, "cannot parse 'level' field") + } + formatter, err := parseLogFormat(common.Format) + if err != nil { + return 0, nil, errors.Wrap(err, "cannot parse 'formatter' field") + } + return minLevel, formatter, nil } - minLevel, err := logger.ParseLevel(in.Level) - if err != nil { - err = errors.Wrap(err, "cannot parse 'level' field") - return - } - formatter, err := parseLogFormat(in.Format) - if err != nil { - err = errors.Wrap(err, "cannot parse") - return - } + var f EntryFormatter - switch in.Outlet { - case "stdout": - o, err = parseStdoutOutlet(i, formatter) - case "tcp": - o, err = parseTCPOutlet(i, formatter) - case "syslog": - o, err = parseSyslogOutlet(i, formatter) + switch v := in.Ret.(type) { + case config.StdoutLoggingOutlet: + level, f, err = parseCommon(v.LoggingOutletCommon) + if err != nil { + break + } + o, err = parseStdoutOutlet(v, f) + case config.TCPLoggingOutlet: + level, f, err = parseCommon(v.LoggingOutletCommon) + if err != nil { + break + } + o, err = parseTCPOutlet(v, f) + case config.SyslogLoggingOutlet: + level, f, err = parseCommon(v.LoggingOutletCommon) + if err != nil { + break + } + o, err = parseSyslogOutlet(v, f) default: - err = errors.Errorf("unknown outlet type '%s'", in.Outlet) + panic(v) } - return o, minLevel, err - + return o, level, err } -func parseStdoutOutlet(i interface{}, formatter EntryFormatter) (WriterOutlet, error) { - - var in struct { - Time bool - } - if err := mapstructure.Decode(i, &in); err != nil { - return WriterOutlet{}, errors.Wrap(err, "invalid structure for stdout outlet") - } - +func parseStdoutOutlet(in config.StdoutLoggingOutlet, formatter EntryFormatter) (WriterOutlet, error) { flags := MetadataAll writer := os.Stdout if !isatty.IsTerminal(writer.Fd()) && !in.Time { @@ -158,54 +149,22 @@ func parseStdoutOutlet(i interface{}, formatter EntryFormatter) (WriterOutlet, e }, nil } -func parseTCPOutlet(i interface{}, formatter EntryFormatter) (out *TCPOutlet, err error) { - - var in struct { - Net string - Address string - RetryInterval string `mapstructure:"retry_interval"` - TLS map[string]interface{} - } - if err = mapstructure.Decode(i, &in); err != nil { - return nil, errors.Wrap(err, "mapstructure error") - } - - retryInterval, err := time.ParseDuration(in.RetryInterval) - if err != nil { - return nil, errors.Wrap(err, "cannot parse 'retry_interval'") - } - - if len(in.Net) == 0 { - return nil, errors.New("field 'net' must not be empty") - } - if len(in.Address) == 0 { - return nil, errors.New("field 'address' must not be empty") - } - +func parseTCPOutlet(in config.TCPLoggingOutlet, formatter EntryFormatter) (out *TCPOutlet, err error) { var tlsConfig *tls.Config if in.TLS != nil { - tlsConfig, err = func(m map[string]interface{}, host string) (*tls.Config, error) { - var in struct { - CA string - Cert string - Key string - } - if err := mapstructure.Decode(m, &in); err != nil { - return nil, errors.Wrap(err, "mapstructure error") - } - - clientCert, err := tls.LoadX509KeyPair(in.Cert, in.Key) + tlsConfig, err = func(m *config.TCPLoggingOutletTLS, host string) (*tls.Config, error) { + clientCert, err := tls.LoadX509KeyPair(m.Cert, m.Key) if err != nil { return nil, errors.Wrap(err, "cannot load client cert") } var rootCAs *x509.CertPool - if in.CA == "" { + if m.CA == "" { if rootCAs, err = x509.SystemCertPool(); err != nil { return nil, errors.Wrap(err, "cannot open system cert pool") } } else { - rootCAs, err = tlsconf.ParseCAFile(in.CA) + rootCAs, err = tlsconf.ParseCAFile(m.CA) if err != nil { return nil, errors.Wrap(err, "cannot parse CA cert") } @@ -222,30 +181,14 @@ func parseTCPOutlet(i interface{}, formatter EntryFormatter) (out *TCPOutlet, er } formatter.SetMetadataFlags(MetadataAll) - return NewTCPOutlet(formatter, in.Net, in.Address, tlsConfig, retryInterval), nil + return NewTCPOutlet(formatter, in.Net, in.Address, tlsConfig, in.RetryInterval), nil } -func parseSyslogOutlet(i interface{}, formatter EntryFormatter) (out *SyslogOutlet, err error) { - - var in struct { - RetryInterval string `mapstructure:"retry_interval"` - } - if err = mapstructure.Decode(i, &in); err != nil { - return nil, errors.Wrap(err, "mapstructure error") - } - +func parseSyslogOutlet(in config.SyslogLoggingOutlet, formatter EntryFormatter) (out *SyslogOutlet, err error) { out = &SyslogOutlet{} out.Formatter = formatter out.Formatter.SetMetadataFlags(MetadataNone) - - out.RetryInterval = 0 // default to 0 as we assume local syslog will just work - if in.RetryInterval != "" { - out.RetryInterval, err = time.ParseDuration(in.RetryInterval) - if err != nil { - return nil, errors.Wrap(err, "cannot parse 'retry_interval'") - } - } - - return + out.RetryInterval = in.RetryInterval + return out, nil } diff --git a/cmd/daemon.go b/cmd/daemon.go index 94a32d4..3fd8d2f 100644 --- a/cmd/daemon.go +++ b/cmd/daemon.go @@ -4,14 +4,14 @@ import ( "context" "fmt" "github.com/spf13/cobra" + "github.com/zrepl/zrepl/cmd/config" + "github.com/zrepl/zrepl/logger" "os" "os/signal" - "syscall" "time" "github.com/zrepl/zrepl/cmd/daemon" "github.com/zrepl/zrepl/cmd/daemon/job" - "github.com/zrepl/zrepl/logger" ) // daemonCmd represents the daemon command @@ -75,13 +75,19 @@ func (a daemonJobAdaptor) Status() interface{} { return nil } func doDaemon(cmd *cobra.Command, args []string) { - conf, err := ParseConfig(rootArgs.configFile) + conf, err := config.ParseConfig(rootArgs.configFile) if err != nil { fmt.Fprintf(os.Stderr, "error parsing config: %s\n", err) os.Exit(1) } - log := logger.NewLogger(conf.Global.logging.Outlets, 1*time.Second) + outlets, err := parseLogging(conf.Global.Logging) + if err != nil { + fmt.Fprintf(os.Stderr, "failed to generate logger: %s\n", err) + return + } + log := logger.NewLogger(outlets.Outlets, 1*time.Second) + ctx := WithLogger(context.Background(), log) daemonJobs := make([]job.Job, 0, len(conf.Jobs)) From 4ec5e2345790b9d576e91a97cc867e06c8812164 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Sun, 26 Aug 2018 16:45:49 +0200 Subject: [PATCH 044/167] set channel buffer to prevent leaking goroutine --- cmd/config_job_source.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index 9e2d53c..dba2556 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -140,7 +140,7 @@ func (j *SourceJob) serve(ctx context.Context) { conn net.Conn err error } - connChan := make(chan connChanMsg) + connChan := make(chan connChanMsg, 1) // Serve connections until interrupted or error outer: From cd9a42884194f6295a3043424181b830511e722b Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Sun, 26 Aug 2018 16:46:09 +0200 Subject: [PATCH 045/167] rename variable --- cmd/config_job_pull.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index a77e161..ac2eee0 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -159,7 +159,7 @@ func (j *PullJob) doRun(ctx context.Context) { sender := endpoint.NewRemote(client) - puller, err := endpoint.NewReceiver( + receiver, err := endpoint.NewReceiver( j.Mapping, NewPrefixFilter(j.SnapshotPrefix), ) @@ -173,7 +173,7 @@ func (j *PullJob) doRun(ctx context.Context) { ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{log.WithField(logSubsysField, "rpc")}) ctx = endpoint.WithLogger(ctx, log.WithField(logSubsysField, "endpoint")) j.rep = replication.NewReplication() - j.rep.Drive(ctx, sender, puller) + j.rep.Drive(ctx, sender, receiver) } client.Close() From add1b69809aefd46a5f0d40200a8a71a0705fdb7 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Sun, 26 Aug 2018 19:20:08 +0200 Subject: [PATCH 046/167] move retentiongrid to own package --- cmd/config/config.go | 45 +++- cmd/config/retentiongrid.go | 123 +++++++++ cmd/config_parse.go | 38 --- cmd/config_prune_grid.go | 245 ------------------ .../retentiongrid/config_prune_grid.go | 156 +++++++++++ .../pruning/retentiongrid}/retentiongrid.go | 31 +-- .../retentiongrid}/retentiongrid_test.go | 27 +- 7 files changed, 355 insertions(+), 310 deletions(-) create mode 100644 cmd/config/retentiongrid.go delete mode 100644 cmd/config_prune_grid.go create mode 100644 cmd/pruning/retentiongrid/config_prune_grid.go rename {util => cmd/pruning/retentiongrid}/retentiongrid.go (71%) rename {util => cmd/pruning/retentiongrid}/retentiongrid_test.go (87%) diff --git a/cmd/config/config.go b/cmd/config/config.go index d7b52d9..3c3a6ce 100644 --- a/cmd/config/config.go +++ b/cmd/config/config.go @@ -6,6 +6,8 @@ import ( "github.com/zrepl/yaml-config" "io/ioutil" "os" + "regexp" + "strconv" "time" ) @@ -102,11 +104,6 @@ type PruneKeepLastN struct { Count int `yaml:"count"` } -type PruneGrid struct { - Type string `yaml:"type"` - Grid string `yaml:"grid"` -} - type LoggingOutletEnum struct { Ret interface{} } @@ -239,3 +236,41 @@ func ParseConfig(path string) (i Config, err error) { return } + +var durationStringRegex *regexp.Regexp = regexp.MustCompile(`^\s*(\d+)\s*(s|m|h|d|w)\s*$`) + +func parsePostitiveDuration(e string) (d time.Duration, err error) { + comps := durationStringRegex.FindStringSubmatch(e) + if len(comps) != 3 { + err = fmt.Errorf("does not match regex: %s %#v", e, comps) + return + } + + durationFactor, err := strconv.ParseInt(comps[1], 10, 64) + if err != nil { + return 0, err + } + if durationFactor <= 0 { + return 0, errors.New("duration must be positive integer") + } + + var durationUnit time.Duration + switch comps[2] { + case "s": + durationUnit = time.Second + case "m": + durationUnit = time.Minute + case "h": + durationUnit = time.Hour + case "d": + durationUnit = 24 * time.Hour + case "w": + durationUnit = 24 * 7 * time.Hour + default: + err = fmt.Errorf("contains unknown time unit '%s'", comps[2]) + return + } + + d = time.Duration(durationFactor) * durationUnit + return +} diff --git a/cmd/config/retentiongrid.go b/cmd/config/retentiongrid.go new file mode 100644 index 0000000..e208941 --- /dev/null +++ b/cmd/config/retentiongrid.go @@ -0,0 +1,123 @@ +package config + +import ( + "fmt" + "regexp" + "strconv" + "strings" + "time" +) + +type RetentionIntervalList []RetentionInterval + +type PruneGrid struct { + Type string `yaml:"type"` + Grid RetentionIntervalList `yaml:"grid"` + KeepBookmarks string `yaml:"keep_bookmarks"` +} + +type RetentionInterval struct { + length time.Duration + keepCount int +} + +func (i *RetentionInterval) Length() time.Duration { + return i.length +} + +func (i *RetentionInterval) KeepCount() int { + return i.keepCount +} + +const RetentionGridKeepCountAll int = -1 + +type RetentionGrid struct { + intervals []RetentionInterval +} + +func (t *RetentionIntervalList) UnmarshalYAML(u func(interface{}, bool) error) (err error) { + var in string + if err := u(&in, true); err != nil { + return err + } + + intervals, err := parseRetentionGridIntervalsString(in) + if err != nil { + return err + } + + *t = intervals + + return nil +} + +var retentionStringIntervalRegex *regexp.Regexp = regexp.MustCompile(`^\s*(\d+)\s*x\s*([^\(]+)\s*(\((.*)\))?\s*$`) + +func parseRetentionGridIntervalString(e string) (intervals []RetentionInterval, err error) { + + comps := retentionStringIntervalRegex.FindStringSubmatch(e) + if comps == nil { + err = fmt.Errorf("retention string does not match expected format") + return + } + + times, err := strconv.Atoi(comps[1]) + if err != nil { + return nil, err + } else if times <= 0 { + return nil, fmt.Errorf("contains factor <= 0") + } + + duration, err := parsePostitiveDuration(comps[2]) + if err != nil { + return nil, err + } + + keepCount := 1 + if comps[3] != "" { + // Decompose key=value, comma separated + // For now, only keep_count is supported + re := regexp.MustCompile(`^\s*keep=(.+)\s*$`) + res := re.FindStringSubmatch(comps[4]) + if res == nil || len(res) != 2 { + err = fmt.Errorf("interval parameter contains unknown parameters") + return + } + if res[1] == "all" { + keepCount = RetentionGridKeepCountAll + } else { + keepCount, err = strconv.Atoi(res[1]) + if err != nil { + err = fmt.Errorf("cannot parse keep_count value") + return + } + } + } + + intervals = make([]RetentionInterval, times) + for i := range intervals { + intervals[i] = RetentionInterval{ + length: duration, + keepCount: keepCount, + } + } + + return + +} + +func parseRetentionGridIntervalsString(s string) (intervals []RetentionInterval, err error) { + + ges := strings.Split(s, "|") + intervals = make([]RetentionInterval, 0, 7*len(ges)) + + for intervalIdx, e := range ges { + parsed, err := parseRetentionGridIntervalString(e) + if err != nil { + return nil, fmt.Errorf("cannot parse interval %d of %d: %s: %s", intervalIdx+1, len(ges), err, strings.TrimSpace(e)) + } + intervals = append(intervals, parsed...) + } + + return +} diff --git a/cmd/config_parse.go b/cmd/config_parse.go index 36a2406..0afbb13 100644 --- a/cmd/config_parse.go +++ b/cmd/config_parse.go @@ -254,41 +254,3 @@ func parseAuthenticatedChannelListenerFactory(c JobParsingContext, v map[string] } } - -var durationStringRegex *regexp.Regexp = regexp.MustCompile(`^\s*(\d+)\s*(s|m|h|d|w)\s*$`) - -func parsePostitiveDuration(e string) (d time.Duration, err error) { - comps := durationStringRegex.FindStringSubmatch(e) - if len(comps) != 3 { - err = fmt.Errorf("does not match regex: %s %#v", e, comps) - return - } - - durationFactor, err := strconv.ParseInt(comps[1], 10, 64) - if err != nil { - return 0, err - } - if durationFactor <= 0 { - return 0, errors.New("duration must be positive integer") - } - - var durationUnit time.Duration - switch comps[2] { - case "s": - durationUnit = time.Second - case "m": - durationUnit = time.Minute - case "h": - durationUnit = time.Hour - case "d": - durationUnit = 24 * time.Hour - case "w": - durationUnit = 24 * 7 * time.Hour - default: - err = fmt.Errorf("contains unknown time unit '%s'", comps[2]) - return - } - - d = time.Duration(durationFactor) * durationUnit - return -} diff --git a/cmd/config_prune_grid.go b/cmd/config_prune_grid.go deleted file mode 100644 index 5b567f3..0000000 --- a/cmd/config_prune_grid.go +++ /dev/null @@ -1,245 +0,0 @@ -package cmd - -import ( - "fmt" - "github.com/mitchellh/mapstructure" - "github.com/pkg/errors" - "github.com/zrepl/zrepl/util" - "github.com/zrepl/zrepl/zfs" - "math" - "regexp" - "sort" - "strconv" - "strings" - "time" -) - -type GridPrunePolicy struct { - RetentionGrid *util.RetentionGrid - MaxBookmarks int -} - -const GridPrunePolicyMaxBookmarksKeepAll = -1 - -type retentionGridAdaptor struct { - zfs.FilesystemVersion -} - -func (a retentionGridAdaptor) Date() time.Time { - return a.Creation -} - -func (a retentionGridAdaptor) LessThan(b util.RetentionGridEntry) bool { - return a.CreateTXG < b.(retentionGridAdaptor).CreateTXG -} - -// Prune filters snapshots with the retention grid. -// Bookmarks are deleted such that KeepBookmarks are kept in the end. -// The oldest bookmarks are removed first. -func (p *GridPrunePolicy) Prune(_ *zfs.DatasetPath, versions []zfs.FilesystemVersion) (keep, remove []zfs.FilesystemVersion, err error) { - skeep, sremove := p.pruneSnapshots(versions) - keep, remove = p.pruneBookmarks(skeep) - remove = append(remove, sremove...) - return keep, remove, nil -} - -func (p *GridPrunePolicy) pruneSnapshots(versions []zfs.FilesystemVersion) (keep, remove []zfs.FilesystemVersion) { - - // Build adaptors for retention grid - keep = []zfs.FilesystemVersion{} - adaptors := make([]util.RetentionGridEntry, 0) - for fsv := range versions { - if versions[fsv].Type != zfs.Snapshot { - keep = append(keep, versions[fsv]) - continue - } - adaptors = append(adaptors, retentionGridAdaptor{versions[fsv]}) - } - - sort.SliceStable(adaptors, func(i, j int) bool { - return adaptors[i].LessThan(adaptors[j]) - }) - now := adaptors[len(adaptors)-1].Date() - - // Evaluate retention grid - keepa, removea := p.RetentionGrid.FitEntries(now, adaptors) - - // Revert adaptors - for i := range keepa { - keep = append(keep, keepa[i].(retentionGridAdaptor).FilesystemVersion) - } - remove = make([]zfs.FilesystemVersion, len(removea)) - for i := range removea { - remove[i] = removea[i].(retentionGridAdaptor).FilesystemVersion - } - return - -} - -func (p *GridPrunePolicy) pruneBookmarks(versions []zfs.FilesystemVersion) (keep, remove []zfs.FilesystemVersion) { - - if p.MaxBookmarks == GridPrunePolicyMaxBookmarksKeepAll { - return versions, []zfs.FilesystemVersion{} - } - - keep = []zfs.FilesystemVersion{} - bookmarks := make([]zfs.FilesystemVersion, 0) - for fsv := range versions { - if versions[fsv].Type != zfs.Bookmark { - keep = append(keep, versions[fsv]) - continue - } - bookmarks = append(bookmarks, versions[fsv]) - } - - if len(bookmarks) == 0 { - return keep, []zfs.FilesystemVersion{} - } - if len(bookmarks) < p.MaxBookmarks { - keep = append(keep, bookmarks...) - return keep, []zfs.FilesystemVersion{} - } - - // NOTE: sorting descending by descending by createtxg <=> sorting ascending wrt creation time - sort.SliceStable(bookmarks, func(i, j int) bool { - return (bookmarks[i].CreateTXG > bookmarks[j].CreateTXG) - }) - - keep = append(keep, bookmarks[:p.MaxBookmarks]...) - remove = bookmarks[p.MaxBookmarks:] - - return keep, remove -} - -func parseGridPrunePolicy(e map[string]interface{}, willSeeBookmarks bool) (p *GridPrunePolicy, err error) { - - const KeepBookmarksAllString = "all" - var i struct { - Grid string - KeepBookmarks string `mapstructure:"keep_bookmarks"` - } - - dec, err := mapstructure.NewDecoder(&mapstructure.DecoderConfig{Result: &i, WeaklyTypedInput: true}) - if err != nil { - err = errors.Wrap(err, "mapstructure error") - return - } - if err = dec.Decode(e); err != nil { - err = errors.Wrapf(err, "mapstructure error") - return - } - - // Parse grid - intervals, err := parseRetentionGridIntervalsString(i.Grid) - if err != nil { - err = fmt.Errorf("cannot parse retention grid: %s", err) - return - } - // Assert intervals are of increasing length (not necessarily required, but indicates config mistake) - lastDuration := time.Duration(0) - for i := range intervals { - - if intervals[i].Length < lastDuration { - // If all intervals before were keep=all, this is ok - allPrevKeepCountAll := true - for j := i - 1; allPrevKeepCountAll && j >= 0; j-- { - allPrevKeepCountAll = intervals[j].KeepCount == util.RetentionGridKeepCountAll - } - if allPrevKeepCountAll { - goto isMonotonicIncrease - } - err = errors.New("retention grid interval length must be monotonically increasing") - return - } - isMonotonicIncrease: - lastDuration = intervals[i].Length - - } - - // Parse KeepBookmarks - keepBookmarks := 0 - if i.KeepBookmarks == KeepBookmarksAllString || (i.KeepBookmarks == "" && !willSeeBookmarks) { - keepBookmarks = GridPrunePolicyMaxBookmarksKeepAll - } else { - i, err := strconv.ParseInt(i.KeepBookmarks, 10, 32) - if err != nil || i <= 0 || i > math.MaxInt32 { - return nil, errors.Errorf("keep_bookmarks must be positive integer or 'all'") - } - keepBookmarks = int(i) - } - return &GridPrunePolicy{ - util.NewRetentionGrid(intervals), - keepBookmarks, - }, nil -} - -var retentionStringIntervalRegex *regexp.Regexp = regexp.MustCompile(`^\s*(\d+)\s*x\s*([^\(]+)\s*(\((.*)\))?\s*$`) - -func parseRetentionGridIntervalString(e string) (intervals []util.RetentionInterval, err error) { - - comps := retentionStringIntervalRegex.FindStringSubmatch(e) - if comps == nil { - err = fmt.Errorf("retention string does not match expected format") - return - } - - times, err := strconv.Atoi(comps[1]) - if err != nil { - return nil, err - } else if times <= 0 { - return nil, fmt.Errorf("contains factor <= 0") - } - - duration, err := parsePostitiveDuration(comps[2]) - if err != nil { - return nil, err - } - - keepCount := 1 - if comps[3] != "" { - // Decompose key=value, comma separated - // For now, only keep_count is supported - re := regexp.MustCompile(`^\s*keep=(.+)\s*$`) - res := re.FindStringSubmatch(comps[4]) - if res == nil || len(res) != 2 { - err = fmt.Errorf("interval parameter contains unknown parameters") - return - } - if res[1] == "all" { - keepCount = util.RetentionGridKeepCountAll - } else { - keepCount, err = strconv.Atoi(res[1]) - if err != nil { - err = fmt.Errorf("cannot parse keep_count value") - return - } - } - } - - intervals = make([]util.RetentionInterval, times) - for i := range intervals { - intervals[i] = util.RetentionInterval{ - Length: duration, - KeepCount: keepCount, - } - } - - return - -} - -func parseRetentionGridIntervalsString(s string) (intervals []util.RetentionInterval, err error) { - - ges := strings.Split(s, "|") - intervals = make([]util.RetentionInterval, 0, 7*len(ges)) - - for intervalIdx, e := range ges { - parsed, err := parseRetentionGridIntervalString(e) - if err != nil { - return nil, fmt.Errorf("cannot parse interval %d of %d: %s: %s", intervalIdx+1, len(ges), err, strings.TrimSpace(e)) - } - intervals = append(intervals, parsed...) - } - - return -} diff --git a/cmd/pruning/retentiongrid/config_prune_grid.go b/cmd/pruning/retentiongrid/config_prune_grid.go new file mode 100644 index 0000000..5223b9d --- /dev/null +++ b/cmd/pruning/retentiongrid/config_prune_grid.go @@ -0,0 +1,156 @@ +package retentiongrid + +import ( + "github.com/pkg/errors" + "github.com/zrepl/zrepl/cmd/config" + "github.com/zrepl/zrepl/zfs" + "math" + "sort" + "strconv" + "time" +) + +type GridPrunePolicy struct { + retentionGrid *retentionGrid + keepBookmarks int +} + +const GridPrunePolicyMaxBookmarksKeepAll = -1 + +type retentionGridAdaptor struct { + zfs.FilesystemVersion +} + +func (a retentionGridAdaptor) Date() time.Time { + return a.Creation +} + +func (a retentionGridAdaptor) LessThan(b RetentionGridEntry) bool { + return a.CreateTXG < b.(retentionGridAdaptor).CreateTXG +} + +// Prune filters snapshots with the retention grid. +// Bookmarks are deleted such that keepBookmarks are kept in the end. +// The oldest bookmarks are removed first. +func (p *GridPrunePolicy) Prune(_ *zfs.DatasetPath, versions []zfs.FilesystemVersion) (keep, remove []zfs.FilesystemVersion, err error) { + skeep, sremove := p.pruneSnapshots(versions) + keep, remove = p.pruneBookmarks(skeep) + remove = append(remove, sremove...) + return keep, remove, nil +} + +func (p *GridPrunePolicy) pruneSnapshots(versions []zfs.FilesystemVersion) (keep, remove []zfs.FilesystemVersion) { + + // Build adaptors for retention grid + keep = []zfs.FilesystemVersion{} + adaptors := make([]RetentionGridEntry, 0) + for fsv := range versions { + if versions[fsv].Type != zfs.Snapshot { + keep = append(keep, versions[fsv]) + continue + } + adaptors = append(adaptors, retentionGridAdaptor{versions[fsv]}) + } + + sort.SliceStable(adaptors, func(i, j int) bool { + return adaptors[i].LessThan(adaptors[j]) + }) + now := adaptors[len(adaptors)-1].Date() + + // Evaluate retention grid + keepa, removea := p.retentionGrid.FitEntries(now, adaptors) + + // Revert adaptors + for i := range keepa { + keep = append(keep, keepa[i].(retentionGridAdaptor).FilesystemVersion) + } + remove = make([]zfs.FilesystemVersion, len(removea)) + for i := range removea { + remove[i] = removea[i].(retentionGridAdaptor).FilesystemVersion + } + return + +} + +func (p *GridPrunePolicy) pruneBookmarks(versions []zfs.FilesystemVersion) (keep, remove []zfs.FilesystemVersion) { + + if p.keepBookmarks == GridPrunePolicyMaxBookmarksKeepAll { + return versions, []zfs.FilesystemVersion{} + } + + keep = []zfs.FilesystemVersion{} + bookmarks := make([]zfs.FilesystemVersion, 0) + for fsv := range versions { + if versions[fsv].Type != zfs.Bookmark { + keep = append(keep, versions[fsv]) + continue + } + bookmarks = append(bookmarks, versions[fsv]) + } + + if len(bookmarks) == 0 { + return keep, []zfs.FilesystemVersion{} + } + if len(bookmarks) < p.keepBookmarks { + keep = append(keep, bookmarks...) + return keep, []zfs.FilesystemVersion{} + } + + // NOTE: sorting descending by descending by createtxg <=> sorting ascending wrt creation time + sort.SliceStable(bookmarks, func(i, j int) bool { + return (bookmarks[i].CreateTXG > bookmarks[j].CreateTXG) + }) + + keep = append(keep, bookmarks[:p.keepBookmarks]...) + remove = bookmarks[p.keepBookmarks:] + + return keep, remove +} + +func ParseGridPrunePolicy(in config.PruneGrid, willSeeBookmarks bool) (p *GridPrunePolicy, err error) { + + const KeepBookmarksAllString = "all" + + // Assert intervals are of increasing length (not necessarily required, but indicates config mistake) + lastDuration := time.Duration(0) + for i := range in.Grid { + + if in.Grid[i].Length() < lastDuration { + // If all intervals before were keep=all, this is ok + allPrevKeepCountAll := true + for j := i - 1; allPrevKeepCountAll && j >= 0; j-- { + allPrevKeepCountAll = in.Grid[j].KeepCount() == config.RetentionGridKeepCountAll + } + if allPrevKeepCountAll { + goto isMonotonicIncrease + } + err = errors.New("retention grid interval length must be monotonically increasing") + return + } + isMonotonicIncrease: + lastDuration = in.Grid[i].Length() + + } + + // Parse keepBookmarks + keepBookmarks := 0 + if in.KeepBookmarks == KeepBookmarksAllString || (in.KeepBookmarks == "" && !willSeeBookmarks) { + keepBookmarks = GridPrunePolicyMaxBookmarksKeepAll + } else { + i, err := strconv.ParseInt(in.KeepBookmarks, 10, 32) + if err != nil || i <= 0 || i > math.MaxInt32 { + return nil, errors.Errorf("keep_bookmarks must be positive integer or 'all'") + } + keepBookmarks = int(i) + } + + retentionIntervals := make([]RetentionInterval, len(in.Grid)) + for i := range in.Grid { + retentionIntervals[i] = &in.Grid[i] + } + + return &GridPrunePolicy{ + newRetentionGrid(retentionIntervals), + keepBookmarks, + }, nil +} diff --git a/util/retentiongrid.go b/cmd/pruning/retentiongrid/retentiongrid.go similarity index 71% rename from util/retentiongrid.go rename to cmd/pruning/retentiongrid/retentiongrid.go index 9e114b9..4813933 100644 --- a/util/retentiongrid.go +++ b/cmd/pruning/retentiongrid/retentiongrid.go @@ -1,45 +1,46 @@ -package util +package retentiongrid import ( "sort" "time" ) -type RetentionInterval struct { - Length time.Duration - KeepCount int +type RetentionInterval interface { + Length() time.Duration + KeepCount() int } const RetentionGridKeepCountAll int = -1 -type RetentionGrid struct { +type retentionGrid struct { intervals []RetentionInterval } +//A point inside the grid, i.e. a thing the grid can decide to remove type RetentionGridEntry interface { Date() time.Time LessThan(b RetentionGridEntry) bool } func dateInInterval(date, startDateInterval time.Time, i RetentionInterval) bool { - return date.After(startDateInterval) && date.Before(startDateInterval.Add(i.Length)) + return date.After(startDateInterval) && date.Before(startDateInterval.Add(i.Length())) } -func NewRetentionGrid(l []RetentionInterval) *RetentionGrid { +func newRetentionGrid(l []RetentionInterval) *retentionGrid { // TODO Maybe check for ascending interval lengths here, although the algorithm // itself doesn't care about that. - return &RetentionGrid{l} + return &retentionGrid{l} } -// Partition a list of RetentionGridEntries into the RetentionGrid, +// Partition a list of RetentionGridEntries into the retentionGrid, // relative to a given start date `now`. // -// The `KeepCount` oldest entries per `RetentionInterval` are kept (`keep`), +// The `keepCount` oldest entries per `RetentionInterval` are kept (`keep`), // the others are removed (`remove`). // // Entries that are younger than `now` are always kept. // Those that are older than the earliest beginning of an interval are removed. -func (g RetentionGrid) FitEntries(now time.Time, entries []RetentionGridEntry) (keep, remove []RetentionGridEntry) { +func (g retentionGrid) FitEntries(now time.Time, entries []RetentionGridEntry) (keep, remove []RetentionGridEntry) { type bucket struct { entries []RetentionGridEntry @@ -51,7 +52,7 @@ func (g RetentionGrid) FitEntries(now time.Time, entries []RetentionGridEntry) ( oldestIntervalStart := now for i := range g.intervals { - oldestIntervalStart = oldestIntervalStart.Add(-g.intervals[i].Length) + oldestIntervalStart = oldestIntervalStart.Add(-g.intervals[i].Length()) } for ei := 0; ei < len(entries); ei++ { @@ -69,7 +70,7 @@ func (g RetentionGrid) FitEntries(now time.Time, entries []RetentionGridEntry) ( iStartTime := now for i := 0; i < len(g.intervals); i++ { - iStartTime = iStartTime.Add(-g.intervals[i].Length) + iStartTime = iStartTime.Add(-g.intervals[i].Length()) if date == iStartTime || dateInInterval(date, iStartTime, g.intervals[i]) { buckets[i].entries = append(buckets[i].entries, e) } @@ -78,14 +79,14 @@ func (g RetentionGrid) FitEntries(now time.Time, entries []RetentionGridEntry) ( for bi, b := range buckets { - interval := &g.intervals[bi] + interval := g.intervals[bi] sort.SliceStable(b.entries, func(i, j int) bool { return b.entries[i].LessThan((b.entries[j])) }) i := 0 - for ; (interval.KeepCount == RetentionGridKeepCountAll || i < interval.KeepCount) && i < len(b.entries); i++ { + for ; (interval.KeepCount() == RetentionGridKeepCountAll || i < interval.KeepCount()) && i < len(b.entries); i++ { keep = append(keep, b.entries[i]) } for ; i < len(b.entries); i++ { diff --git a/util/retentiongrid_test.go b/cmd/pruning/retentiongrid/retentiongrid_test.go similarity index 87% rename from util/retentiongrid_test.go rename to cmd/pruning/retentiongrid/retentiongrid_test.go index b2fb0fb..70c8ccd 100644 --- a/util/retentiongrid_test.go +++ b/cmd/pruning/retentiongrid/retentiongrid_test.go @@ -1,4 +1,4 @@ -package util +package retentiongrid import ( "fmt" @@ -9,9 +9,22 @@ import ( "time" ) -func retentionGridFromString(gs string) (g *RetentionGrid) { +type retentionIntervalStub struct { + length time.Duration + keepCount int +} + +func (i *retentionIntervalStub) Length() time.Duration { + return i.length +} + +func (i *retentionIntervalStub) KeepCount() int { + return i.keepCount +} + +func retentionGridFromString(gs string) (g *retentionGrid) { intervals := strings.Split(gs, "|") - g = &RetentionGrid{ + g = &retentionGrid{ intervals: make([]RetentionInterval, len(intervals)), } for idx, i := range intervals { @@ -25,16 +38,16 @@ func retentionGridFromString(gs string) (g *RetentionGrid) { } var err error - var interval RetentionInterval + var interval retentionIntervalStub - if interval.KeepCount, err = strconv.Atoi(numSnapsStr); err != nil { + if interval.keepCount, err = strconv.Atoi(numSnapsStr); err != nil { panic(err) } - if interval.Length, err = time.ParseDuration(durationStr); err != nil { + if interval.length, err = time.ParseDuration(durationStr); err != nil { panic(err) } - g.intervals[idx] = interval + g.intervals[idx] = &interval } return } From fbb8a253206cdc5f40ad2aa6754b29733795ad35 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Sun, 26 Aug 2018 20:25:06 +0200 Subject: [PATCH 047/167] add prometheus monitoring to config --- cmd/config/config.go | 19 ++++++++++++++++++- cmd/config/samples/push.yml | 3 +++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/cmd/config/config.go b/cmd/config/config.go index 3c3a6ce..e2ab447 100644 --- a/cmd/config/config.go +++ b/cmd/config/config.go @@ -53,7 +53,8 @@ type Pruning struct { } type Global struct { - Logging []LoggingOutletEnum `yaml:"logging"` + Logging []LoggingOutletEnum `yaml:"logging"` + Monitoring []MonitoringEnum `yaml:"monitoring"` } type ConnectEnum struct { @@ -138,6 +139,15 @@ type TCPLoggingOutletTLS struct { Key string `yaml:"key"` } +type MonitoringEnum struct { + Ret interface{} +} + +type PrometheusMonitoring struct { + Type string `yaml:"type"` + Listen string `yaml:"listen"` +} + func enumUnmarshal(u func(interface{}, bool) error, types map[string]interface{}) (interface{}, error) { var in struct { Type string @@ -201,6 +211,13 @@ func (t *LoggingOutletEnum) UnmarshalYAML(u func(interface{}, bool) error) (err return } +func (t *MonitoringEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { + t.Ret, err = enumUnmarshal(u, map[string]interface{}{ + "prometheus": &PrometheusMonitoring{}, + }) + return +} + var ConfigFileDefaultLocations = []string{ "/etc/zrepl/zrepl.yml", "/usr/local/etc/zrepl/zrepl.yml", diff --git a/cmd/config/samples/push.yml b/cmd/config/samples/push.yml index 5966465..4d762af 100644 --- a/cmd/config/samples/push.yml +++ b/cmd/config/samples/push.yml @@ -28,3 +28,6 @@ global: time: true level: "warn" format: "human" + monitoring: + - type: "prometheus" + listen: ":9091" \ No newline at end of file From e2bf557d17b7a57ee0ba40dc2c69e6435970e8c0 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Sun, 26 Aug 2018 21:37:29 +0200 Subject: [PATCH 048/167] use optional and default feature of yaml-config --- cmd/config/config.go | 8 ++++---- cmd/config/samples/push.yml | 2 ++ cmd/config/samples/sink.yml | 1 + 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/cmd/config/config.go b/cmd/config/config.go index e2ab447..6f79352 100644 --- a/cmd/config/config.go +++ b/cmd/config/config.go @@ -54,7 +54,7 @@ type Pruning struct { type Global struct { Logging []LoggingOutletEnum `yaml:"logging"` - Monitoring []MonitoringEnum `yaml:"monitoring"` + Monitoring []MonitoringEnum `yaml:"monitoring,optional"` } type ConnectEnum struct { @@ -127,10 +127,10 @@ type SyslogLoggingOutlet struct { type TCPLoggingOutlet struct { LoggingOutletCommon `yaml:",inline"` - Address string `yaml:"address"` //TODO required - Net string `yaml:"net"` //TODO default tcp + Address string `yaml:"address"` + Net string `yaml:"net,default=tcp"` RetryInterval time.Duration `yaml:"retry_interval"` - TLS *TCPLoggingOutletTLS + TLS *TCPLoggingOutletTLS `yaml:"tls,optional"` } type TCPLoggingOutletTLS struct { diff --git a/cmd/config/samples/push.yml b/cmd/config/samples/push.yml index 4d762af..d12c57a 100644 --- a/cmd/config/samples/push.yml +++ b/cmd/config/samples/push.yml @@ -18,10 +18,12 @@ jobs: count: 10 - type: grid grid: 1x1h(keep=all) | 24x1h | 14x1d + keep_bookmarks: all keep_remote: - type: grid grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d + keep_bookmarks: all global: logging: - type: "stdout" diff --git a/cmd/config/samples/sink.yml b/cmd/config/samples/sink.yml index 95f4500..f1e550f 100644 --- a/cmd/config/samples/sink.yml +++ b/cmd/config/samples/sink.yml @@ -12,6 +12,7 @@ global: logging: - type: "tcp" address: "123.123.123.123:1234" + retry_interval: 10s tls: ca: "ca.pem" cert: "cert.pem" From 48a08e4f4df0edaa233bcc090b143b53cdb5fa36 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Sun, 26 Aug 2018 23:11:50 +0200 Subject: [PATCH 049/167] config for pull and source --- cmd/config/config.go | 71 +++++++++++++++++++++++++++++------ cmd/config/config_test.go | 14 ++++--- cmd/config/samples/pull.yml | 39 +++++++++++++++++++ cmd/config/samples/push.yml | 12 ++++-- cmd/config/samples/sink.yml | 6 +++ cmd/config/samples/source.yml | 40 ++++++++++++++++++++ 6 files changed, 161 insertions(+), 21 deletions(-) create mode 100644 cmd/config/samples/pull.yml create mode 100644 cmd/config/samples/source.yml diff --git a/cmd/config/config.go b/cmd/config/config.go index 6f79352..e45637a 100644 --- a/cmd/config/config.go +++ b/cmd/config/config.go @@ -21,17 +21,39 @@ type JobEnum struct { } type PushJob struct { - Type string `yaml:"type"` - Replication PushReplication `yaml:"replication"` - Snapshotting Snapshotting `yaml:"snapshotting"` - Pruning Pruning `yaml:"pruning"` + Type string `yaml:"type"` + Name string `yaml:"name"` + Replication PushReplication `yaml:"replication"` + Snapshotting Snapshotting `yaml:"snapshotting"` + Pruning PruningSenderReceiver `yaml:"pruning"` } type SinkJob struct { Type string `yaml:"type"` + Name string `yaml:"name"` Replication SinkReplication `yaml:"replication"` } +type PullJob struct { + Type string `yaml:"type"` + Name string `yaml:"name"` + Replication PullReplication `yaml:"replication"` + Pruning PruningSenderReceiver `yaml:"pruning"` +} + +type PullReplication struct { + Connect ConnectEnum `yaml:"connect"` + RootDataset string `yaml:"root_dataset"` +} + +type SourceJob struct { + Type string `yaml:"type"` + Name string `yaml:"name"` + Replication SourceReplication `yaml:"replication"` + Snapshotting Snapshotting `yaml:"snapshotting"` + Pruning PruningLocal `yaml:"pruning"` +} + type PushReplication struct { Connect ConnectEnum `yaml:"connect"` Filesystems map[string]bool `yaml:"filesystems"` @@ -42,19 +64,30 @@ type SinkReplication struct { Serve ServeEnum `yaml:"serve"` } +type SourceReplication struct { + Serve ServeEnum `yaml:"serve"` + Filesystems map[string]bool `yaml:"filesystems"` +} + type Snapshotting struct { SnapshotPrefix string `yaml:"snapshot_prefix"` Interval time.Duration `yaml:"interval"` } -type Pruning struct { - KeepLocal []PruningEnum `yaml:"keep_local"` - KeepRemote []PruningEnum `yaml:"keep_remote"` +type PruningSenderReceiver struct { + KeepSender []PruningEnum `yaml:"keep_sender"` + KeepReceiver []PruningEnum `yaml:"keep_receiver"` +} + +type PruningLocal struct { + Keep []PruningEnum `yaml:"keep"` } type Global struct { Logging []LoggingOutletEnum `yaml:"logging"` Monitoring []MonitoringEnum `yaml:"monitoring,optional"` + Control GlobalControl `yaml:"control"` + Serve GlobalServe `yaml:"serve"` } type ConnectEnum struct { @@ -127,9 +160,9 @@ type SyslogLoggingOutlet struct { type TCPLoggingOutlet struct { LoggingOutletCommon `yaml:",inline"` - Address string `yaml:"address"` - Net string `yaml:"net,default=tcp"` - RetryInterval time.Duration `yaml:"retry_interval"` + Address string `yaml:"address"` + Net string `yaml:"net,default=tcp"` + RetryInterval time.Duration `yaml:"retry_interval"` TLS *TCPLoggingOutletTLS `yaml:"tls,optional"` } @@ -148,6 +181,18 @@ type PrometheusMonitoring struct { Listen string `yaml:"listen"` } +type GlobalControl struct { + SockPath string `yaml:"sockpath,default=/var/run/zrepl/control"` +} + +type GlobalServe struct { + StdinServer GlobalStdinServer `yaml:"stdinserver"` +} + +type GlobalStdinServer struct { + SockDir string `yaml:"sockdir,default=/var/run/zrepl/stdinserver"` +} + func enumUnmarshal(u func(interface{}, bool) error, types map[string]interface{}) (interface{}, error) { var in struct { Type string @@ -171,8 +216,10 @@ func enumUnmarshal(u func(interface{}, bool) error, types map[string]interface{} func (t *JobEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { t.Ret, err = enumUnmarshal(u, map[string]interface{}{ - "push": &PushJob{}, - "sink": &SinkJob{}, + "push": &PushJob{}, + "sink": &SinkJob{}, + "pull": &PullJob{}, + "source": &SourceJob{}, }) return } diff --git a/cmd/config/config_test.go b/cmd/config/config_test.go index ef4ca0c..ad975d0 100644 --- a/cmd/config/config_test.go +++ b/cmd/config/config_test.go @@ -14,13 +14,15 @@ func TestSampleConfigsAreParsedWithoutErrors(t *testing.T) { for _, p := range paths { - c, err := ParseConfig(p) - if err != nil { - t.Errorf("error parsing %s:\n%+v", p, err) - } + t.Run(p, func(t *testing.T) { + c, err := ParseConfig(p) + if err != nil { + t.Errorf("error parsing %s:\n%+v", p, err) + } - t.Logf("file: %s", p) - t.Log(pretty.Sprint(c)) + t.Logf("file: %s", p) + t.Log(pretty.Sprint(c)) + }) } diff --git a/cmd/config/samples/pull.yml b/cmd/config/samples/pull.yml new file mode 100644 index 0000000..dce4706 --- /dev/null +++ b/cmd/config/samples/pull.yml @@ -0,0 +1,39 @@ +jobs: + +- name: pull_servers + type: pull + replication: + connect: + type: tls + address: "server1.foo.bar:8888" + ca: /certs/ca.crt + cert: /certs/cert.crt + key: /certs/key.pem + root_dataset: "pool2/backup_servers" + pruning: + keep_sender: + - type: not_replicated + - type: last_n + count: 10 + - type: grid + grid: 1x1h(keep=all) | 24x1h | 14x1d + keep_bookmarks: all + keep_receiver: + - type: grid + grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d + keep_bookmarks: all + +global: + logging: + - type: "stdout" + time: true + level: "warn" + format: "human" + monitoring: + - type: "prometheus" + listen: ":9091" + control: + sockpath: /var/run/zrepl/control + serve: + stdinserver: + sockdir: /var/run/zrepl/stdinserver \ No newline at end of file diff --git a/cmd/config/samples/push.yml b/cmd/config/samples/push.yml index d12c57a..5765a87 100644 --- a/cmd/config/samples/push.yml +++ b/cmd/config/samples/push.yml @@ -1,5 +1,6 @@ jobs: - type: push + name: "push" replication: connect: type: tcp @@ -12,7 +13,7 @@ jobs: snapshot_prefix: zrepl_ interval: 10m pruning: - keep_local: + keep_sender: - type: not_replicated - type: last_n count: 10 @@ -20,7 +21,7 @@ jobs: grid: 1x1h(keep=all) | 24x1h | 14x1d keep_bookmarks: all - keep_remote: + keep_receiver: - type: grid grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d keep_bookmarks: all @@ -32,4 +33,9 @@ global: format: "human" monitoring: - type: "prometheus" - listen: ":9091" \ No newline at end of file + listen: ":9091" + control: + sockpath: /var/run/zrepl/control + serve: + stdinserver: + sockdir: /var/run/zrepl/stdinserver \ No newline at end of file diff --git a/cmd/config/samples/sink.yml b/cmd/config/samples/sink.yml index f1e550f..1803512 100644 --- a/cmd/config/samples/sink.yml +++ b/cmd/config/samples/sink.yml @@ -1,5 +1,6 @@ jobs: - type: sink + name: "laptop_sink" replication: root_dataset: "pool2/backup_laptops" serve: @@ -19,3 +20,8 @@ global: key: "key.pem" level: "warn" format: "human" + control: + sockpath: /var/run/zrepl/control + serve: + stdinserver: + sockdir: /var/run/zrepl/stdinserver \ No newline at end of file diff --git a/cmd/config/samples/source.yml b/cmd/config/samples/source.yml new file mode 100644 index 0000000..79a4847 --- /dev/null +++ b/cmd/config/samples/source.yml @@ -0,0 +1,40 @@ +jobs: +- name: pull_source + type: source + replication: + serve: + type: tcp + listen: "0.0.0.0:8888" + clients: { + "192.168.122.123" : "client1" + } + filesystems: { + "<": true, + "secret": false + } + snapshotting: + snapshot_prefix: zrepl_ + interval: 10m + pruning: + keep: + - type: not_replicated + - type: last_n + count: 10 + - type: grid + grid: 1x1h(keep=all) | 24x1h | 14x1d + keep_bookmarks: all + +global: + logging: + - type: "stdout" + time: true + level: "warn" + format: "human" + monitoring: + - type: "prometheus" + listen: ":9091" + control: + sockpath: /var/run/zrepl/control + serve: + stdinserver: + sockdir: /var/run/zrepl/stdinserver \ No newline at end of file From 5e51595d7fc67bd8d25cb52296e7d73cde8a08aa Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Sun, 26 Aug 2018 23:29:57 +0200 Subject: [PATCH 050/167] local job config --- cmd/config/config.go | 14 ++++++++++++ cmd/config/samples/local.yml | 42 ++++++++++++++++++++++++++++++++++++ 2 files changed, 56 insertions(+) create mode 100644 cmd/config/samples/local.yml diff --git a/cmd/config/config.go b/cmd/config/config.go index e45637a..38c9a38 100644 --- a/cmd/config/config.go +++ b/cmd/config/config.go @@ -54,6 +54,14 @@ type SourceJob struct { Pruning PruningLocal `yaml:"pruning"` } +type LocalJob struct { + Type string `yaml:"type"` + Name string `yaml:"name"` + Replication LocalReplication `yaml:"replication"` + Snapshotting Snapshotting `yaml:"snapshotting"` + Pruning PruningSenderReceiver `yaml:"pruning"` +} + type PushReplication struct { Connect ConnectEnum `yaml:"connect"` Filesystems map[string]bool `yaml:"filesystems"` @@ -69,6 +77,11 @@ type SourceReplication struct { Filesystems map[string]bool `yaml:"filesystems"` } +type LocalReplication struct { + Filesystems map[string]bool `yaml:"filesystems"` + RootDataset string `yaml:"root_dataset"` +} + type Snapshotting struct { SnapshotPrefix string `yaml:"snapshot_prefix"` Interval time.Duration `yaml:"interval"` @@ -220,6 +233,7 @@ func (t *JobEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { "sink": &SinkJob{}, "pull": &PullJob{}, "source": &SourceJob{}, + "local": &LocalJob{}, }) return } diff --git a/cmd/config/samples/local.yml b/cmd/config/samples/local.yml new file mode 100644 index 0000000..5473b42 --- /dev/null +++ b/cmd/config/samples/local.yml @@ -0,0 +1,42 @@ + +jobs: +- name: mirror_local + type: local + # snapshot the filesystems matched by the left-hand-side of the mapping + # every 10m with zrepl_ as prefix + replication: + filesystems: { + "pool1/var/db<": true, + "pool1/usr/home<": true, + "pool1/usr/home/paranoid": false, #don't backup paranoid user + "pool1/poudriere/ports<": false #don't backup the ports trees + } + # TODO FIXME enforce that the tree under root_dataset and the trees allowed (true) by filesystems are non-overlapping + root_dataset: "pool2/backups/pool1" + + snapshotting: + snapshot_prefix: zrepl_ + interval: 10m + + pruning: + keep_sender: + - type: not_replicated + keep_receiver: + - type: grid + grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d + keep_bookmarks: all + +global: + logging: + - type: "stdout" + time: true + level: "warn" + format: "human" + monitoring: + - type: "prometheus" + listen: ":9091" + control: + sockpath: /var/run/zrepl/control + serve: + stdinserver: + sockdir: /var/run/zrepl/stdinserver From b4ea5f56b20663666aae8af48b40da4c18fc339f Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Sun, 26 Aug 2018 23:46:59 +0200 Subject: [PATCH 051/167] ssh config --- cmd/config/config.go | 16 ++++++++++++ cmd/config/samples/pull_ssh.yml | 41 +++++++++++++++++++++++++++++++ cmd/config/samples/source_ssh.yml | 37 ++++++++++++++++++++++++++++ 3 files changed, 94 insertions(+) create mode 100644 cmd/config/samples/pull_ssh.yml create mode 100644 cmd/config/samples/source_ssh.yml diff --git a/cmd/config/config.go b/cmd/config/config.go index 38c9a38..3bf2304 100644 --- a/cmd/config/config.go +++ b/cmd/config/config.go @@ -120,6 +120,15 @@ type TLSConnect struct { Key string `yaml:"key"` } +type SSHStdinserverConnect struct { + Type string `yaml:"type"` + Host string `yaml:"host"` + User string `yaml:"user"` + Port uint16 `yaml:"port"` + IdentityFile string `yaml:"identity_file"` + Options []string `yaml:"options"` +} + type ServeEnum struct { Ret interface{} } @@ -138,6 +147,11 @@ type TLSServe struct { Key string `yaml:"key"` } +type StdinserverServer struct { + Type string `yaml:"type"` + ClientIdentity string `yaml:"client_identity"` +} + type PruningEnum struct { Ret interface{} } @@ -242,6 +256,7 @@ func (t *ConnectEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) t.Ret, err = enumUnmarshal(u, map[string]interface{}{ "tcp": &TCPConnect{}, "tls": &TLSConnect{}, + "ssh+stdinserver": &SSHStdinserverConnect{}, }) return } @@ -250,6 +265,7 @@ func (t *ServeEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { t.Ret, err = enumUnmarshal(u, map[string]interface{}{ "tcp": &TCPServe{}, "tls": &TLSServe{}, + "stdinserver": &StdinserverServer{}, }) return } diff --git a/cmd/config/samples/pull_ssh.yml b/cmd/config/samples/pull_ssh.yml new file mode 100644 index 0000000..6a7b6a0 --- /dev/null +++ b/cmd/config/samples/pull_ssh.yml @@ -0,0 +1,41 @@ +jobs: + +- name: pull_servers + type: pull + replication: + connect: + type: ssh+stdinserver + host: app-srv.example.com + user: root + port: 22 + identity_file: /etc/zrepl/ssh/identity + options: # optional, default [], `-o` arguments passed to ssh + - "Compression=on" + root_dataset: "pool2/backup_servers" + pruning: + keep_sender: + - type: not_replicated + - type: last_n + count: 10 + - type: grid + grid: 1x1h(keep=all) | 24x1h | 14x1d + keep_bookmarks: all + keep_receiver: + - type: grid + grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d + keep_bookmarks: all + +global: + logging: + - type: "stdout" + time: true + level: "warn" + format: "human" + monitoring: + - type: "prometheus" + listen: ":9091" + control: + sockpath: /var/run/zrepl/control + serve: + stdinserver: + sockdir: /var/run/zrepl/stdinserver \ No newline at end of file diff --git a/cmd/config/samples/source_ssh.yml b/cmd/config/samples/source_ssh.yml new file mode 100644 index 0000000..2b2ac9c --- /dev/null +++ b/cmd/config/samples/source_ssh.yml @@ -0,0 +1,37 @@ +jobs: +- name: pull_source + type: source + replication: + serve: + type: stdinserver + client_identity: "client1" + filesystems: { + "<": true, + "secret": false + } + snapshotting: + snapshot_prefix: zrepl_ + interval: 10m + pruning: + keep: + - type: not_replicated + - type: last_n + count: 10 + - type: grid + grid: 1x1h(keep=all) | 24x1h | 14x1d + keep_bookmarks: all + +global: + logging: + - type: "stdout" + time: true + level: "warn" + format: "human" + monitoring: + - type: "prometheus" + listen: ":9091" + control: + sockpath: /var/run/zrepl/control + serve: + stdinserver: + sockdir: /var/run/zrepl/stdinserver \ No newline at end of file From ecd9db4ac69f3bb5a4832a5203bdcfd0e29b16f8 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 27 Aug 2018 15:09:24 +0200 Subject: [PATCH 052/167] start pruning reimplementation in cmd/pruning subpackage --- cmd/pruning/keep_helpers.go | 17 ++++++ cmd/pruning/keep_helpers_test.go | 22 ++++++++ cmd/pruning/keep_last_n.go | 32 ++++++++++++ cmd/pruning/keep_prefix.go | 33 ++++++++++++ cmd/pruning/pruning.go | 39 ++++++++++++++ cmd/pruning/pruning_test.go | 88 ++++++++++++++++++++++++++++++++ 6 files changed, 231 insertions(+) create mode 100644 cmd/pruning/keep_helpers.go create mode 100644 cmd/pruning/keep_helpers_test.go create mode 100644 cmd/pruning/keep_last_n.go create mode 100644 cmd/pruning/keep_prefix.go create mode 100644 cmd/pruning/pruning.go create mode 100644 cmd/pruning/pruning_test.go diff --git a/cmd/pruning/keep_helpers.go b/cmd/pruning/keep_helpers.go new file mode 100644 index 0000000..a756ce7 --- /dev/null +++ b/cmd/pruning/keep_helpers.go @@ -0,0 +1,17 @@ +package pruning + +func filterSnapList(snaps []Snapshot, predicate func(Snapshot) bool) []Snapshot { + r := make([]Snapshot, 0, len(snaps)) + for i := range snaps { + if predicate(snaps[i]) { + r = append(r, snaps[i]) + } + } + return r +} + +func shallowCopySnapList(snaps []Snapshot) []Snapshot { + c := make([]Snapshot, len(snaps)) + copy(c, snaps) + return c +} diff --git a/cmd/pruning/keep_helpers_test.go b/cmd/pruning/keep_helpers_test.go new file mode 100644 index 0000000..bf8b399 --- /dev/null +++ b/cmd/pruning/keep_helpers_test.go @@ -0,0 +1,22 @@ +package pruning + +import ( + "github.com/stretchr/testify/assert" + "testing" +) + +func TestShallowCopySnapList(t *testing.T) { + + l1 := []Snapshot{ + stubSnap{name: "foo"}, + stubSnap{name: "bar"}, + } + l2 := shallowCopySnapList(l1) + + assert.Equal(t, l1, l2) + + l1[0] = stubSnap{name: "baz"} + assert.Equal(t, "baz", l1[0].Name()) + assert.Equal(t, "foo", l2[0].Name()) + +} diff --git a/cmd/pruning/keep_last_n.go b/cmd/pruning/keep_last_n.go new file mode 100644 index 0000000..5301141 --- /dev/null +++ b/cmd/pruning/keep_last_n.go @@ -0,0 +1,32 @@ +package pruning + +import ( + "github.com/pkg/errors" + "sort" +) + +type KeepLastN struct { + n int +} + +func NewKeepLastN(n int) (*KeepLastN, error) { + if n <= 0 { + return nil, errors.Errorf("must specify positive number as 'keep last count', got %d", n) + } + return &KeepLastN{n}, nil +} + +func (k KeepLastN) KeepRule(snaps []Snapshot) []Snapshot { + + if k.n > len(snaps) { + return snaps + } + + res := shallowCopySnapList(snaps) + + sort.Slice(res, func(i, j int) bool { + return res[i].Date().After(res[j].Date()) + }) + + return res[:k.n] +} diff --git a/cmd/pruning/keep_prefix.go b/cmd/pruning/keep_prefix.go new file mode 100644 index 0000000..f8a5956 --- /dev/null +++ b/cmd/pruning/keep_prefix.go @@ -0,0 +1,33 @@ +package pruning + +import ( + "regexp" +) + +type KeepRegex struct { + expr *regexp.Regexp +} + +var _ KeepRule = &KeepRegex{} + +func NewKeepRegex(expr string) (*KeepRegex, error) { + re, err := regexp.Compile(expr) + if err != nil { + return nil, err + } + return &KeepRegex{re}, nil +} + +func MustKeepRegex(expr string) *KeepRegex { + k, err := NewKeepRegex(expr) + if err != nil { + panic(err) + } + return k +} + +func (k *KeepRegex) KeepRule(snaps []Snapshot) []Snapshot { + return filterSnapList(snaps, func(s Snapshot) bool { + return k.expr.FindStringIndex(s.Name()) == nil + }) +} diff --git a/cmd/pruning/pruning.go b/cmd/pruning/pruning.go new file mode 100644 index 0000000..97b866b --- /dev/null +++ b/cmd/pruning/pruning.go @@ -0,0 +1,39 @@ +package pruning + +import ( + "time" +) + +type KeepRule interface { + KeepRule(snaps []Snapshot) []Snapshot +} + +type Snapshot interface { + Name() string + Replicated() bool + Date() time.Time +} + +func PruneSnapshots(snaps []Snapshot, keepRules []KeepRule) []Snapshot { + + if len(keepRules) == 0 { + return snaps + } + + remCount := make(map[Snapshot]int, len(snaps)) + for _, r := range keepRules { + ruleRems := r.KeepRule(snaps) + for _, ruleRem := range ruleRems { + remCount[ruleRem]++ + } + } + + remove := make([]Snapshot, 0, len(snaps)) + for snap, rc := range remCount { + if rc == len(keepRules) { + remove = append(remove, snap) + } + } + + return remove +} diff --git a/cmd/pruning/pruning_test.go b/cmd/pruning/pruning_test.go new file mode 100644 index 0000000..cd97311 --- /dev/null +++ b/cmd/pruning/pruning_test.go @@ -0,0 +1,88 @@ +package pruning + +import ( + "github.com/stretchr/testify/assert" + "testing" + "time" +) + +type stubSnap struct { + name string + replicated bool + date time.Time +} + +func (s stubSnap) Name() string { return s.name } + +func (s stubSnap) Replicated() bool { return s.replicated } + +func (s stubSnap) Date() time.Time { return s.date } + +func TestPruneSnapshots(t *testing.T) { + + type testCase struct { + inputs []Snapshot + rules []KeepRule + exp, eff []Snapshot + } + + inputs := map[string][]Snapshot{ + "s1": []Snapshot{ + stubSnap{name: "foo_123"}, + stubSnap{name: "foo_456"}, + stubSnap{name: "bar_123"}, + }, + } + + tcs := map[string]testCase{ + "simple": { + inputs: inputs["s1"], + rules: []KeepRule{ + MustKeepRegex("foo_"), + }, + exp: []Snapshot{ + stubSnap{name: "bar_123"}, + }, + }, + "multipleRules": { + inputs: inputs["s1"], + rules: []KeepRule{ + MustKeepRegex("foo_"), + MustKeepRegex("bar_"), + }, + exp: []Snapshot{}, + }, + "onlyThoseRemovedByAllAreRemoved": { + inputs: inputs["s1"], + rules: []KeepRule{ + MustKeepRegex("notInS1"), // would remove all + MustKeepRegex("bar_"), // would remove all but bar_, i.e. foo_.* + }, + exp: []Snapshot{ + stubSnap{name: "foo_123"}, + stubSnap{name: "foo_456"}, + }, + }, + "noRulesKeepsAll": { + inputs: inputs["s1"], + rules: []KeepRule{}, + exp: inputs["s1"], + }, + "noSnaps": { + inputs: []Snapshot{}, + rules: []KeepRule{ + MustKeepRegex("foo_"), + }, + exp: []Snapshot{}, + }, + } + + for name := range tcs { + t.Run(name, func(t *testing.T) { + tc := tcs[name] + tc.eff = PruneSnapshots(tc.inputs, tc.rules) + assert.Equal(t, tc.exp, tc.eff) + }) + } + +} From b955d308d9acfdddb364a067509007356f16b1bb Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Mon, 27 Aug 2018 15:18:08 +0200 Subject: [PATCH 053/167] add to config --- cmd/config/config.go | 100 ++++++++++++++++++++------------ cmd/config/samples/pull.yml | 9 +-- cmd/config/samples/pull_ssh.yml | 3 + cmd/config/samples/sink.yml | 1 + 4 files changed, 72 insertions(+), 41 deletions(-) diff --git a/cmd/config/config.go b/cmd/config/config.go index 3bf2304..a999732 100644 --- a/cmd/config/config.go +++ b/cmd/config/config.go @@ -26,12 +26,14 @@ type PushJob struct { Replication PushReplication `yaml:"replication"` Snapshotting Snapshotting `yaml:"snapshotting"` Pruning PruningSenderReceiver `yaml:"pruning"` + Debug JobDebugSettings `yaml:"debug,optional"` } type SinkJob struct { - Type string `yaml:"type"` - Name string `yaml:"name"` - Replication SinkReplication `yaml:"replication"` + Type string `yaml:"type"` + Name string `yaml:"name"` + Replication SinkReplication `yaml:"replication"` + Debug JobDebugSettings `yaml:"debug,optional"` } type PullJob struct { @@ -39,11 +41,13 @@ type PullJob struct { Name string `yaml:"name"` Replication PullReplication `yaml:"replication"` Pruning PruningSenderReceiver `yaml:"pruning"` + Debug JobDebugSettings `yaml:"debug,optional"` } type PullReplication struct { - Connect ConnectEnum `yaml:"connect"` - RootDataset string `yaml:"root_dataset"` + Connect ConnectEnum `yaml:"connect"` + RootDataset string `yaml:"root_dataset"` + Interval time.Duration `yaml:"interval,positive"` } type SourceJob struct { @@ -52,14 +56,16 @@ type SourceJob struct { Replication SourceReplication `yaml:"replication"` Snapshotting Snapshotting `yaml:"snapshotting"` Pruning PruningLocal `yaml:"pruning"` + Debug JobDebugSettings `yaml:"debug,optional"` } type LocalJob struct { - Type string `yaml:"type"` - Name string `yaml:"name"` - Replication LocalReplication `yaml:"replication"` + Type string `yaml:"type"` + Name string `yaml:"name"` + Replication LocalReplication `yaml:"replication"` Snapshotting Snapshotting `yaml:"snapshotting"` - Pruning PruningSenderReceiver `yaml:"pruning"` + Pruning PruningSenderReceiver `yaml:"pruning"` + Debug JobDebugSettings `yaml:"debug,optional"` } type PushReplication struct { @@ -79,12 +85,12 @@ type SourceReplication struct { type LocalReplication struct { Filesystems map[string]bool `yaml:"filesystems"` - RootDataset string `yaml:"root_dataset"` + RootDataset string `yaml:"root_dataset"` } type Snapshotting struct { SnapshotPrefix string `yaml:"snapshot_prefix"` - Interval time.Duration `yaml:"interval"` + Interval time.Duration `yaml:"interval,positive"` } type PruningSenderReceiver struct { @@ -108,25 +114,31 @@ type ConnectEnum struct { } type TCPConnect struct { - Type string `yaml:"type"` - Address string `yaml:"address"` + Type string `yaml:"type"` + Address string `yaml:"address"` + DialTimeout time.Duration `yaml:"dial_timeout,positive,default=10s"` } type TLSConnect struct { - Type string `yaml:"type"` - Address string `yaml:"address"` - Ca string `yaml:"ca"` - Cert string `yaml:"cert"` - Key string `yaml:"key"` + Type string `yaml:"type"` + Address string `yaml:"address"` + Ca string `yaml:"ca"` + Cert string `yaml:"cert"` + Key string `yaml:"key"` + ServerCN string `yaml:"server_cn"` + DialTimeout time.Duration `yaml:"dial_timeout,positive,default=10s"` } type SSHStdinserverConnect struct { - Type string `yaml:"type"` - Host string `yaml:"host"` - User string `yaml:"user"` - Port uint16 `yaml:"port"` - IdentityFile string `yaml:"identity_file"` - Options []string `yaml:"options"` + Type string `yaml:"type"` + Host string `yaml:"host"` + User string `yaml:"user"` + Port uint16 `yaml:"port"` + IdentityFile string `yaml:"identity_file"` + TransportOpenCommand []string `yaml:"transport_open_command,optional"` //TODO unused + SSHCommand string `yaml:"ssh_command,optional"` //TODO unused + Options []string `yaml:"options"` + DialTimeout time.Duration `yaml:"dial_timeout,positive,default=10s"` } type ServeEnum struct { @@ -140,15 +152,16 @@ type TCPServe struct { } type TLSServe struct { - Type string `yaml:"type"` - Listen string `yaml:"listen"` - Ca string `yaml:"ca"` - Cert string `yaml:"cert"` - Key string `yaml:"key"` + Type string `yaml:"type"` + Listen string `yaml:"listen"` + Ca string `yaml:"ca"` + Cert string `yaml:"cert"` + Key string `yaml:"key"` + ClientCN string `yaml:"client_cn"` } type StdinserverServer struct { - Type string `yaml:"type"` + Type string `yaml:"type"` ClientIdentity string `yaml:"client_identity"` } @@ -165,6 +178,11 @@ type PruneKeepLastN struct { Count int `yaml:"count"` } +type PruneKeepPrefix struct { // FIXME rename to KeepPrefix + Type string `yaml:"type"` + Prefix string `yaml:"prefix"` +} + type LoggingOutletEnum struct { Ret interface{} } @@ -182,14 +200,14 @@ type StdoutLoggingOutlet struct { type SyslogLoggingOutlet struct { LoggingOutletCommon `yaml:",inline"` - RetryInterval time.Duration `yaml:"retry_interval"` + RetryInterval time.Duration `yaml:"retry_interval,positive"` } type TCPLoggingOutlet struct { LoggingOutletCommon `yaml:",inline"` Address string `yaml:"address"` Net string `yaml:"net,default=tcp"` - RetryInterval time.Duration `yaml:"retry_interval"` + RetryInterval time.Duration `yaml:"retry_interval,positive"` TLS *TCPLoggingOutletTLS `yaml:"tls,optional"` } @@ -220,6 +238,13 @@ type GlobalStdinServer struct { SockDir string `yaml:"sockdir,default=/var/run/zrepl/stdinserver"` } +type JobDebugSettings struct { + Conn struct { + ReadDump string `yaml:"read_dump"` + WriteDump string `yaml:"write_dump"` + } `yaml:"conn"` +} + func enumUnmarshal(u func(interface{}, bool) error, types map[string]interface{}) (interface{}, error) { var in struct { Type string @@ -247,15 +272,15 @@ func (t *JobEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { "sink": &SinkJob{}, "pull": &PullJob{}, "source": &SourceJob{}, - "local": &LocalJob{}, + "local": &LocalJob{}, }) return } func (t *ConnectEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { t.Ret, err = enumUnmarshal(u, map[string]interface{}{ - "tcp": &TCPConnect{}, - "tls": &TLSConnect{}, + "tcp": &TCPConnect{}, + "tls": &TLSConnect{}, "ssh+stdinserver": &SSHStdinserverConnect{}, }) return @@ -263,8 +288,8 @@ func (t *ConnectEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) func (t *ServeEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { t.Ret, err = enumUnmarshal(u, map[string]interface{}{ - "tcp": &TCPServe{}, - "tls": &TLSServe{}, + "tcp": &TCPServe{}, + "tls": &TLSServe{}, "stdinserver": &StdinserverServer{}, }) return @@ -275,6 +300,7 @@ func (t *PruningEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) "not_replicated": &PruneKeepNotReplicated{}, "last_n": &PruneKeepLastN{}, "grid": &PruneGrid{}, + "prefix": &PruneKeepPrefix{}, }) return } diff --git a/cmd/config/samples/pull.yml b/cmd/config/samples/pull.yml index dce4706..a6a37c7 100644 --- a/cmd/config/samples/pull.yml +++ b/cmd/config/samples/pull.yml @@ -1,15 +1,16 @@ jobs: - - name: pull_servers type: pull replication: connect: type: tls address: "server1.foo.bar:8888" - ca: /certs/ca.crt - cert: /certs/cert.crt - key: /certs/key.pem + ca: "/certs/ca.crt" + cert: "/certs/cert.crt" + key: "/certs/key.pem" + server_cn: "server1" root_dataset: "pool2/backup_servers" + interval: 10m pruning: keep_sender: - type: not_replicated diff --git a/cmd/config/samples/pull_ssh.yml b/cmd/config/samples/pull_ssh.yml index 6a7b6a0..317aae1 100644 --- a/cmd/config/samples/pull_ssh.yml +++ b/cmd/config/samples/pull_ssh.yml @@ -12,6 +12,7 @@ jobs: options: # optional, default [], `-o` arguments passed to ssh - "Compression=on" root_dataset: "pool2/backup_servers" + interval: 10m pruning: keep_sender: - type: not_replicated @@ -21,6 +22,8 @@ jobs: grid: 1x1h(keep=all) | 24x1h | 14x1d keep_bookmarks: all keep_receiver: + - type: prefix + prefix: keep_ - type: grid grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d keep_bookmarks: all diff --git a/cmd/config/samples/sink.yml b/cmd/config/samples/sink.yml index 1803512..bd3769d 100644 --- a/cmd/config/samples/sink.yml +++ b/cmd/config/samples/sink.yml @@ -9,6 +9,7 @@ jobs: ca: "ca.pem" cert: "cert.pem" key: "key.pem" + client_cn: "laptop1" global: logging: - type: "tcp" From 16e139626193a984afa52eca85799cce2e5b2a39 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Mon, 27 Aug 2018 15:19:17 +0200 Subject: [PATCH 054/167] connecters with new config --- cmd/config_connect.go | 133 ++++++++++++++++-------------------------- 1 file changed, 49 insertions(+), 84 deletions(-) diff --git a/cmd/config_connect.go b/cmd/config_connect.go index e13274a..efb84d4 100644 --- a/cmd/config_connect.go +++ b/cmd/config_connect.go @@ -2,15 +2,14 @@ package cmd import ( "crypto/tls" - "fmt" "net" "context" "github.com/jinzhu/copier" - "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-netssh" "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/cmd/config" "github.com/zrepl/zrepl/cmd/tlsconf" "time" ) @@ -19,34 +18,26 @@ type SSHStdinserverConnecter struct { Host string User string Port uint16 - IdentityFile string `mapstructure:"identity_file"` - TransportOpenCommand []string `mapstructure:"transport_open_command"` - SSHCommand string `mapstructure:"ssh_command"` + IdentityFile string + TransportOpenCommand []string + SSHCommand string Options []string - DialTimeout string `mapstructure:"dial_timeout"` dialTimeout time.Duration } var _ streamrpc.Connecter = &SSHStdinserverConnecter{} -func parseSSHStdinserverConnecter(i map[string]interface{}) (c *SSHStdinserverConnecter, err error) { +func parseSSHStdinserverConnecter(in config.SSHStdinserverConnect) (c *SSHStdinserverConnecter, err error) { - c = &SSHStdinserverConnecter{} - if err = mapstructure.Decode(i, c); err != nil { - err = errors.New(fmt.Sprintf("could not parse ssh transport: %s", err)) - return nil, err + c = &SSHStdinserverConnecter{ + Host: in.Host, + User: in.User, + Port: in.Port, + IdentityFile: in.IdentityFile, + SSHCommand: in.SSHCommand, + Options: in.Options, + dialTimeout: in.DialTimeout, } - - if c.DialTimeout != "" { - c.dialTimeout, err = time.ParseDuration(c.DialTimeout) - if err != nil { - return nil, errors.Wrap(err, "cannot parse dial_timeout") - } - } else { - c.dialTimeout = 10 * time.Second - } - - // TODO assert fields are filled return } @@ -78,77 +69,51 @@ func (c *SSHStdinserverConnecter) Connect(dialCtx context.Context) (net.Conn, er } type TCPConnecter struct { - Host string - Port uint16 + Address string + dialer net.Dialer +} + +func parseTCPConnecter(in config.TCPConnect) (*TCPConnecter, error) { + dialer := net.Dialer{ + Timeout: in.DialTimeout, + } + + return &TCPConnecter{in.Address, dialer}, nil +} + +func (c *TCPConnecter) Connect(dialCtx context.Context) (conn net.Conn, err error) { + return c.dialer.DialContext(dialCtx, "tcp", c.Address) +} + +type TLSConnecter struct { + Address string dialer net.Dialer tlsConfig *tls.Config } -func parseTCPConnecter(i map[string]interface{}) (*TCPConnecter, error) { - var in struct { - Host string - Port uint16 - DialTimeout string `mapstructure:"dial_timeout"` - TLS map[string]interface{} - } - if err := mapstructure.Decode(i, &in); err != nil { - return nil, errors.Wrap(err, "mapstructure error") - } - - if in.Host == "" || in.Port == 0 { - return nil, errors.New("fields 'host' and 'port' must not be empty") - } - dialTimeout, err := parsePostitiveDuration(in.DialTimeout) - if err != nil { - if in.DialTimeout != "" { - return nil, errors.Wrap(err, "cannot parse field 'dial_timeout'") - } - dialTimeout = 10 * time.Second - } +func parseTLSConnecter(in config.TLSConnect) (*TLSConnecter, error) { dialer := net.Dialer{ - Timeout: dialTimeout, + Timeout: in.DialTimeout, } - var tlsConfig *tls.Config - if in.TLS != nil { - tlsConfig, err = func(i map[string]interface{}) (config *tls.Config, err error) { - var in struct { - CA string - Cert string - Key string - ServerCN string `mapstructure:"server_cn"` - } - if err := mapstructure.Decode(i, &in); err != nil { - return nil, errors.Wrap(err, "mapstructure error") - } - if in.CA == "" || in.Cert == "" || in.Key == "" || in.ServerCN == "" { - return nil, errors.New("fields 'ca', 'cert', 'key' and 'server_cn' must be specified") - } - - ca, err := tlsconf.ParseCAFile(in.CA) - if err != nil { - return nil, errors.Wrap(err, "cannot parse ca file") - } - - cert, err := tls.LoadX509KeyPair(in.Cert, in.Key) - if err != nil { - return nil, errors.Wrap(err, "cannot parse cert/key pair") - } - - return tlsconf.ClientAuthClient(in.ServerCN, ca, cert) - }(in.TLS) - if err != nil { - return nil, errors.Wrap(err, "cannot parse TLS config in field 'tls'") - } + ca, err := tlsconf.ParseCAFile(in.Ca) + if err != nil { + return nil, errors.Wrap(err, "cannot parse ca file") } - return &TCPConnecter{in.Host, in.Port, dialer, tlsConfig}, nil + cert, err := tls.LoadX509KeyPair(in.Cert, in.Key) + if err != nil { + return nil, errors.Wrap(err, "cannot parse cert/key pair") + } + + tlsConfig, err := tlsconf.ClientAuthClient(in.ServerCN, ca, cert) + if err != nil { + return nil, errors.Wrap(err, "cannot build tls config") + } + + return &TLSConnecter{in.Address, dialer, tlsConfig}, nil } -func (c *TCPConnecter) Connect(dialCtx context.Context) (conn net.Conn, err error) { - addr := fmt.Sprintf("%s:%d", c.Host, c.Port) - if c.tlsConfig != nil { - return tls.DialWithDialer(&c.dialer, "tcp", addr, c.tlsConfig) - } - return c.dialer.DialContext(dialCtx, "tcp", addr) +func (c *TLSConnecter) Connect(dialCtx context.Context) (conn net.Conn, err error) { + return tls.DialWithDialer(&c.dialer, "tcp", c.Address, c.tlsConfig) } From b0d17803f08abee90b62c85b2c2124bfeb854d90 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Mon, 27 Aug 2018 15:19:56 +0200 Subject: [PATCH 055/167] job source with new config --- cmd/config_job_source.go | 47 ++++++++++------------------------------ 1 file changed, 12 insertions(+), 35 deletions(-) diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index dba2556..76d2be5 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -4,9 +4,9 @@ import ( "context" "time" - "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/cmd/config" "github.com/zrepl/zrepl/cmd/endpoint" "net" ) @@ -18,59 +18,36 @@ type SourceJob struct { SnapshotPrefix string Interval time.Duration Prune PrunePolicy - Debug JobDebugSettings } -func parseSourceJob(c JobParsingContext, name string, i map[string]interface{}) (j *SourceJob, err error) { - - var asMap struct { - Serve map[string]interface{} - Filesystems map[string]string - SnapshotPrefix string `mapstructure:"snapshot_prefix"` - Interval string - Prune map[string]interface{} - Debug map[string]interface{} +func parseSourceJob(c config.Global, in config.SourceJob) (j *SourceJob, err error) { + j = &SourceJob{ + Name: in.Name, + Interval: in.Snapshotting.Interval, } - if err = mapstructure.Decode(i, &asMap); err != nil { - err = errors.Wrap(err, "mapstructure error") - return nil, err - } - - j = &SourceJob{Name: name} - - if j.Serve, err = parseAuthenticatedChannelListenerFactory(c, asMap.Serve); err != nil { + if j.Serve, err = parseAuthenticatedChannelListenerFactory(c, in.Replication.Serve); err != nil { return } - if j.Filesystems, err = parseDatasetMapFilter(asMap.Filesystems, true); err != nil { + if j.Filesystems, err = parseDatasetMapFilter(in.Replication.Filesystems, true); err != nil { return } - if j.SnapshotPrefix, err = parseSnapshotPrefix(asMap.SnapshotPrefix); err != nil { + if j.SnapshotPrefix, err = parseSnapshotPrefix(in.Snapshotting.SnapshotPrefix); err != nil { return } - if j.Interval, err = parsePostitiveDuration(asMap.Interval); err != nil { - err = errors.Wrap(err, "cannot parse 'interval'") - return - } - - if j.Prune, err = parsePrunePolicy(asMap.Prune, true); err != nil { + if j.Prune, err = parsePrunePolicy(in.Pruning, true); err != nil { err = errors.Wrap(err, "cannot parse 'prune'") return } - if err = mapstructure.Decode(asMap.Debug, &j.Debug); err != nil { - err = errors.Wrap(err, "cannot parse 'debug'") - return - } - - if j.Debug.Conn.ReadDump != "" || j.Debug.Conn.WriteDump != "" { + if in.Debug.Conn.ReadDump != "" || in.Debug.Conn.WriteDump != "" { logServe := logListenerFactory{ ListenerFactory: j.Serve, - ReadDump: j.Debug.Conn.ReadDump, - WriteDump: j.Debug.Conn.WriteDump, + ReadDump: in.Debug.Conn.ReadDump, + WriteDump: in.Debug.Conn.WriteDump, } j.Serve = logServe } From c2b04d10c5e3dfab558878dfa36d180918c145f4 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Mon, 27 Aug 2018 15:22:32 +0200 Subject: [PATCH 056/167] wip floocode backup --- Gopkg.lock | 9 -- cmd/config_fsvfilter.go | 6 ++ cmd/config_job_local.go | 3 +- cmd/config_job_pull.go | 71 ++++------------ cmd/config_mapfilter.go | 17 ++-- cmd/config_parse.go | 141 +++++++++++--------------------- cmd/config_serve_stdinserver.go | 21 ++--- cmd/config_serve_tcp.go | 74 ++--------------- cmd/config_serve_tls.go | 78 ++++++++++++++++++ cmd/control.go | 4 +- cmd/daemon.go | 5 +- cmd/daemon/control.go | 8 +- cmd/daemon/daemon.go | 29 ++++--- cmd/daemon/job/job.go | 2 +- cmd/daemon/prometheus.go | 4 +- cmd/helpers/helpers.go | 4 +- cmd/prune.go | 13 ++- cmd/version.go | 1 - version/version.go | 2 +- 19 files changed, 206 insertions(+), 286 deletions(-) create mode 100644 cmd/config_serve_tls.go diff --git a/Gopkg.lock b/Gopkg.lock index f6a26f3..6d9e63c 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -17,14 +17,6 @@ revision = "346938d642f2ec3594ed81d874461961cd0faa76" version = "v1.1.0" -[[projects]] - branch = "master" - digest = "1:ae162f9b5c46f6d5ff4bd53a3d78f72e2eb6676c11c5d33b8b106c36f87ddb31" - name = "github.com/dustin/go-humanize" - packages = ["."] - pruneopts = "" - revision = "bb3d318650d48840a39aa21a027c6630e198e626" - [[projects]] branch = "master" digest = "1:5d0a2385edf4ba44f3b7b76bc0436ceb8f62bf55aa5d540a9eb9ec6c58d86809" @@ -247,7 +239,6 @@ analyzer-name = "dep" analyzer-version = 1 input-imports = [ - "github.com/dustin/go-humanize", "github.com/go-logfmt/logfmt", "github.com/go-yaml/yaml", "github.com/golang/protobuf/proto", diff --git a/cmd/config_fsvfilter.go b/cmd/config_fsvfilter.go index 3391780..fc8839d 100644 --- a/cmd/config_fsvfilter.go +++ b/cmd/config_fsvfilter.go @@ -6,6 +6,12 @@ import ( "strings" ) +type AnyFSVFilter struct{} + +func (AnyFSVFilter) Filter(t zfs.VersionType, name string) (accept bool, err error) { + return true, nil +} + type PrefixFilter struct { prefix string fstype zfs.VersionType diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index 03b063a..c8b8c00 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -6,6 +6,7 @@ import ( "context" "github.com/mitchellh/mapstructure" "github.com/pkg/errors" + "github.com/zrepl/zrepl/cmd/config" "github.com/zrepl/zrepl/cmd/endpoint" "github.com/zrepl/zrepl/replication" "github.com/zrepl/zrepl/zfs" @@ -22,7 +23,7 @@ type LocalJob struct { Debug JobDebugSettings } -func parseLocalJob(c JobParsingContext, name string, i map[string]interface{}) (j *LocalJob, err error) { +func parseLocalJob(c config.Global, in source.LocalJob) (j *LocalJob, err error) { var asMap struct { Mapping map[string]string diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index ac2eee0..b20db21 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -11,6 +11,7 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/cmd/config" "github.com/zrepl/zrepl/cmd/endpoint" "github.com/zrepl/zrepl/replication" ) @@ -21,74 +22,44 @@ type PullJob struct { Interval time.Duration Mapping *DatasetMapFilter // constructed from mapping during parsing - pruneFilter *DatasetMapFilter - SnapshotPrefix string - Prune PrunePolicy - Debug JobDebugSettings + pruneFilter *DatasetMapFilter + Prune PrunePolicy rep *replication.Replication } -func parsePullJob(c JobParsingContext, name string, i map[string]interface{}) (j *PullJob, err error) { +func parsePullJob(c config.Global, in config.PullJob) (j *PullJob, err error) { - var asMap struct { - Connect map[string]interface{} - Interval string - Mapping map[string]string - InitialReplPolicy string `mapstructure:"initial_repl_policy"` - Prune map[string]interface{} - SnapshotPrefix string `mapstructure:"snapshot_prefix"` - Debug map[string]interface{} - } + j = &PullJob{Name: in.Name} - if err = mapstructure.Decode(i, &asMap); err != nil { - err = errors.Wrap(err, "mapstructure error") - return nil, err - } - - j = &PullJob{Name: name} - - j.Connect, err = parseConnect(asMap.Connect) + j.Connect, err = parseConnect(in.Replication.Connect) if err != nil { err = errors.Wrap(err, "cannot parse 'connect'") return nil, err } - if j.Interval, err = parsePostitiveDuration(asMap.Interval); err != nil { - err = errors.Wrap(err, "cannot parse 'interval'") + j.Interval = in.Replication.Interval + + j.Mapping = NewDatasetMapFilter(1, false) + if err := j.Mapping.Add("<", in.Replication.RootDataset); err != nil { return nil, err } - j.Mapping, err = parseDatasetMapFilter(asMap.Mapping, false) - if err != nil { - err = errors.Wrap(err, "cannot parse 'mapping'") + j.pruneFilter = NewDatasetMapFilter(1, true) + if err := j.pruneFilter.Add(in.Replication.RootDataset, MapFilterResultOk); err != nil { return nil, err } - if j.pruneFilter, err = j.Mapping.InvertedFilter(); err != nil { - err = errors.Wrap(err, "cannot automatically invert 'mapping' for prune job") - return nil, err - } - - if j.SnapshotPrefix, err = parseSnapshotPrefix(asMap.SnapshotPrefix); err != nil { - return - } - if j.Prune, err = parsePrunePolicy(asMap.Prune, false); err != nil { err = errors.Wrap(err, "cannot parse prune policy") return } - if err = mapstructure.Decode(asMap.Debug, &j.Debug); err != nil { - err = errors.Wrap(err, "cannot parse 'debug'") - return - } - - if j.Debug.Conn.ReadDump != "" || j.Debug.Conn.WriteDump != "" { + if in.Debug.Conn.ReadDump != "" || j.Debug.Conn.WriteDump != "" { logConnecter := logNetConnConnecter{ Connecter: j.Connect, - ReadDump: j.Debug.Conn.ReadDump, - WriteDump: j.Debug.Conn.WriteDump, + ReadDump: in.Debug.Conn.ReadDump, + WriteDump: in.Debug.Conn.WriteDump, } j.Connect = logConnecter } @@ -96,11 +67,7 @@ func parsePullJob(c JobParsingContext, name string, i map[string]interface{}) (j return } -func (j *PullJob) JobName() string { - return j.Name -} - -func (j *PullJob) JobType() JobType { return JobTypePull } +func (j *PullJob) JobName() string { return j.Name } func (j *PullJob) JobStart(ctx context.Context) { @@ -159,10 +126,7 @@ func (j *PullJob) doRun(ctx context.Context) { sender := endpoint.NewRemote(client) - receiver, err := endpoint.NewReceiver( - j.Mapping, - NewPrefixFilter(j.SnapshotPrefix), - ) + receiver, err := endpoint.NewReceiver(j.Mapping, AnyFSVFilter{}) if err != nil { log.WithError(err).Error("error creating receiver endpoint") return @@ -198,7 +162,6 @@ func (j *PullJob) Pruner(side PrunePolicySide, dryRun bool) (p Pruner, err error time.Now(), dryRun, j.pruneFilter, - j.SnapshotPrefix, j.Prune, } return diff --git a/cmd/config_mapfilter.go b/cmd/config_mapfilter.go index 0a1de54..f1182b7 100644 --- a/cmd/config_mapfilter.go +++ b/cmd/config_mapfilter.go @@ -4,7 +4,6 @@ import ( "fmt" "strings" - "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/zrepl/zrepl/cmd/endpoint" "github.com/zrepl/zrepl/zfs" @@ -258,16 +257,14 @@ func (m DatasetMapFilter) parseDatasetFilterResult(result string) (pass bool, er return false, fmt.Errorf("'%s' is not a valid filter result", result) } -func parseDatasetMapFilter(mi interface{}, filterMode bool) (f *DatasetMapFilter, err error) { +func parseDatasetMapFilterFilesystems(in map[string]bool) (f *DatasetMapFilter, err error) { - var m map[string]string - if err = mapstructure.Decode(mi, &m); err != nil { - err = fmt.Errorf("maps / filters must be specified as map[string]string: %s", err) - return - } - - f = NewDatasetMapFilter(len(m), filterMode) - for pathPattern, mapping := range m { + f = NewDatasetMapFilter(len(in), true) + for pathPattern, accept := range in { + mapping := MapFilterResultOmit + if accept { + mapping = MapFilterResultOk + } if err = f.Add(pathPattern, mapping); err != nil { err = fmt.Errorf("invalid mapping entry ['%s':'%s']: %s", pathPattern, mapping, err) return diff --git a/cmd/config_parse.go b/cmd/config_parse.go index 0afbb13..0419b9e 100644 --- a/cmd/config_parse.go +++ b/cmd/config_parse.go @@ -8,10 +8,9 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/cmd/config" + "github.com/zrepl/zrepl/cmd/pruning/retentiongrid" "os" - "regexp" - "strconv" - "time" ) var ConfigFileDefaultLocations []string = []string{ @@ -141,116 +140,72 @@ func parseConfig(i interface{}) (c *Config, err error) { } -func extractStringField(i map[string]interface{}, key string, notempty bool) (field string, err error) { - vi, ok := i[key] - if !ok { - err = errors.Errorf("must have field '%s'", key) - return "", err - } - field, ok = vi.(string) - if !ok { - err = errors.Errorf("'%s' field must have type string", key) - return "", err - } - if notempty && len(field) <= 0 { - err = errors.Errorf("'%s' field must not be empty", key) - return "", err - } - return -} - type JobParsingContext struct { ConfigParsingContext } -func parseJob(c JobParsingContext, i map[string]interface{}) (j Job, err error) { +func parseJob(c config.Global, in config.JobEnum) (j Job, err error) { - name, err := extractStringField(i, "name", true) - if err != nil { - return nil, err + switch v := in.Ret.(type) { + case config.PullJob: + return parsePullJob(c, v) + case config.SourceJob: + return parseSourceJob(c, v) + case config.LocalJob: + return parseLocalJob(c, v) + default: + panic(fmt.Sprintf("implementation error: unknown job type %s", v)) } - for _, r := range ReservedJobNames { - if name == r { - err = errors.Errorf("job name '%s' is reserved", name) - return nil, err +} + +func parseConnect(in config.ConnectEnum) (c streamrpc.Connecter, err error) { + switch v := in.Ret.(type) { + case config.SSHStdinserverConnect: + return parseSSHStdinserverConnecter(v) + case config.TCPConnect: + return parseTCPConnecter(v) + case config.TLSConnect: + return parseTLSConnecter(v) + default: + panic(fmt.Sprintf("unknown connect type %v", v)) + } +} + +func parsePruning(in []config.PruningEnum, willSeeBookmarks bool) (p Pruner, err error) { + + policies := make([]PrunePolicy, len(in)) + for i := range in { + if policies[i], err = parseKeepRule(in[i]); err != nil { + return nil, errors.Wrapf(err, "invalid keep rule #%d:", i) } } - jobtypeStr, err := extractStringField(i, "type", true) - if err != nil { - return nil, err - } - - jobtype, err := ParseUserJobType(jobtypeStr) - if err != nil { - return nil, err - } - - switch jobtype { - case JobTypePull: - return parsePullJob(c, name, i) - case JobTypeSource: - return parseSourceJob(c, name, i) - case JobTypeLocal: - return parseLocalJob(c, name, i) - default: - panic(fmt.Sprintf("implementation error: unknown job type %s", jobtype)) - } - } -func parseConnect(i map[string]interface{}) (c streamrpc.Connecter, err error) { - - t, err := extractStringField(i, "type", true) - if err != nil { - return nil, err - } - - switch t { - case "ssh+stdinserver": - return parseSSHStdinserverConnecter(i) - case "tcp": - return parseTCPConnecter(i) +func parseKeepRule(in config.PruningEnum) (p PrunePolicy, err error) { + switch v := in.Ret.(type) { + case config.PruneGrid: + return retentiongrid.ParseGridPrunePolicy(v, willSeeBookmarks) + //case config.PruneKeepLastN: + //case config.PruneKeepPrefix: + //case config.PruneKeepNotReplicated: default: - return nil, errors.Errorf("unknown connection type '%s'", t) - } - -} - -func parsePrunePolicy(v map[string]interface{}, willSeeBookmarks bool) (p PrunePolicy, err error) { - - policyName, err := extractStringField(v, "policy", true) - if err != nil { - return - } - - switch policyName { - case "grid": - return parseGridPrunePolicy(v, willSeeBookmarks) - case "noprune": - return NoPrunePolicy{}, nil - default: - err = errors.Errorf("unknown policy '%s'", policyName) - return + panic(fmt.Sprintf("unknown keep rule type %v", v)) } } -func parseAuthenticatedChannelListenerFactory(c JobParsingContext, v map[string]interface{}) (p ListenerFactory, err error) { +func parseAuthenticatedChannelListenerFactory(c config.Global, in config.ServeEnum) (p ListenerFactory, err error) { - t, err := extractStringField(v, "type", true) - if err != nil { - return nil, err - } - - switch t { - case "stdinserver": + switch v := in.Ret.(type) { + case config.StdinserverServer: return parseStdinserverListenerFactory(c, v) - case "tcp": + case config.TCPServe: return parseTCPListenerFactory(c, v) + case config.TLSServe: + return parseTLSListenerFactory(c, v) default: - err = errors.Errorf("unknown type '%s'", t) - return + panic(fmt.Sprintf("unknown listener type %v", v)) } } diff --git a/cmd/config_serve_stdinserver.go b/cmd/config_serve_stdinserver.go index 4dd8e86..5fb19b9 100644 --- a/cmd/config_serve_stdinserver.go +++ b/cmd/config_serve_stdinserver.go @@ -1,32 +1,25 @@ package cmd import ( - "github.com/mitchellh/mapstructure" - "github.com/pkg/errors" "github.com/problame/go-netssh" + "github.com/zrepl/zrepl/cmd/config" + "github.com/zrepl/zrepl/cmd/helpers" "net" "path" - "github.com/zrepl/zrepl/cmd/helpers" ) type StdinserverListenerFactory struct { - ClientIdentity string `mapstructure:"client_identity"` + ClientIdentity string sockpath string } -func parseStdinserverListenerFactory(c JobParsingContext, i map[string]interface{}) (f *StdinserverListenerFactory, err error) { +func parseStdinserverListenerFactory(c config.Global, in config.StdinserverServer) (f *StdinserverListenerFactory, err error) { - f = &StdinserverListenerFactory{} - - if err = mapstructure.Decode(i, f); err != nil { - return nil, errors.Wrap(err, "mapstructure error") - } - if !(len(f.ClientIdentity) > 0) { - err = errors.Errorf("must specify 'client_identity'") - return + f = &StdinserverListenerFactory{ + ClientIdentity: in.ClientIdentity, } - f.sockpath = path.Join(c.Global.Serve.Stdinserver.SockDir, f.ClientIdentity) + f.sockpath = path.Join(c.Serve.StdinServer.SockDir, f.ClientIdentity) return } diff --git a/cmd/config_serve_tcp.go b/cmd/config_serve_tcp.go index c3c1785..9757b00 100644 --- a/cmd/config_serve_tcp.go +++ b/cmd/config_serve_tcp.go @@ -1,88 +1,26 @@ package cmd import ( - "crypto/tls" - "crypto/x509" "net" "time" - "github.com/mitchellh/mapstructure" - "github.com/pkg/errors" - "github.com/zrepl/zrepl/cmd/tlsconf" + "github.com/zrepl/zrepl/cmd/config" ) type TCPListenerFactory struct { - Address string - tls bool - clientCA *x509.CertPool - serverCert tls.Certificate - clientCommonName string + Address string } -func parseTCPListenerFactory(c JobParsingContext, i map[string]interface{}) (*TCPListenerFactory, error) { +func parseTCPListenerFactory(c config.Global, in config.TCPServe) (*TCPListenerFactory, error) { - var in struct { - Address string - TLS map[string]interface{} + lf := &TCPListenerFactory{ + Address: in.Listen, } - if err := mapstructure.Decode(i, &in); err != nil { - return nil, errors.Wrap(err, "mapstructure error") - } - - lf := &TCPListenerFactory{} - - if in.Address == "" { - return nil, errors.New("must specify field 'address'") - } - lf.Address = in.Address - - if in.TLS != nil { - err := func(i map[string]interface{}) (err error) { - var in struct { - CA string - Cert string - Key string - ClientCN string `mapstructure:"client_cn"` - } - if err := mapstructure.Decode(i, &in); err != nil { - return errors.Wrap(err, "mapstructure error") - } - - if in.CA == "" || in.Cert == "" || in.Key == "" || in.ClientCN == "" { - return errors.New("fields 'ca', 'cert', 'key' and 'client_cn' must be specified") - } - - lf.clientCommonName = in.ClientCN - - lf.clientCA, err = tlsconf.ParseCAFile(in.CA) - if err != nil { - return errors.Wrap(err, "cannot parse ca file") - } - - lf.serverCert, err = tls.LoadX509KeyPair(in.Cert, in.Key) - if err != nil { - return errors.Wrap(err, "cannot parse cer/key pair") - } - - lf.tls = true // mark success - return nil - }(in.TLS) - if err != nil { - return nil, errors.Wrap(err, "error parsing TLS config in field 'tls'") - } - } - return lf, nil } var TCPListenerHandshakeTimeout = 10 * time.Second // FIXME make configurable func (f *TCPListenerFactory) Listen() (net.Listener, error) { - l, err := net.Listen("tcp", f.Address) - if !f.tls || err != nil { - return l, err - } - - tl := tlsconf.NewClientAuthListener(l, f.clientCA, f.serverCert, f.clientCommonName, TCPListenerHandshakeTimeout) - return tl, nil + return net.Listen("tcp", f.Address) } diff --git a/cmd/config_serve_tls.go b/cmd/config_serve_tls.go new file mode 100644 index 0000000..54ab53d --- /dev/null +++ b/cmd/config_serve_tls.go @@ -0,0 +1,78 @@ +package cmd + +import ( + "crypto/tls" + "crypto/x509" + "net" + "time" + + "github.com/mitchellh/mapstructure" + "github.com/pkg/errors" + "github.com/zrepl/zrepl/cmd/config" + "github.com/zrepl/zrepl/cmd/tlsconf" +) + +type TCPListenerFactory struct { + Address string + tls bool + clientCA *x509.CertPool + serverCert tls.Certificate + clientCommonName string +} + +func parseTCPListenerFactory(c config.Global, in config.TCPServe) (*TCPListenerFactory, error) { + + lf := &TCPListenerFactory{ + Address: in.Listen, + } + + if in.TLS != nil { + err := func(i map[string]interface{}) (err error) { + var in struct { + CA string + Cert string + Key string + ClientCN string `mapstructure:"client_cn"` + } + if err := mapstructure.Decode(i, &in); err != nil { + return errors.Wrap(err, "mapstructure error") + } + + if in.CA == "" || in.Cert == "" || in.Key == "" || in.ClientCN == "" { + return errors.New("fields 'ca', 'cert', 'key' and 'client_cn' must be specified") + } + + lf.clientCommonName = in.ClientCN + + lf.clientCA, err = tlsconf.ParseCAFile(in.CA) + if err != nil { + return errors.Wrap(err, "cannot parse ca file") + } + + lf.serverCert, err = tls.LoadX509KeyPair(in.Cert, in.Key) + if err != nil { + return errors.Wrap(err, "cannot parse cer/key pair") + } + + lf.tls = true // mark success + return nil + }(in.TLS) + if err != nil { + return nil, errors.Wrap(err, "error parsing TLS config in field 'tls'") + } + } + + return lf, nil +} + +var TCPListenerHandshakeTimeout = 10 * time.Second // FIXME make configurable + +func (f *TCPListenerFactory) Listen() (net.Listener, error) { + l, err := net.Listen("tcp", f.Address) + if !f.tls || err != nil { + return l, err + } + + tl := tlsconf.NewClientAuthListener(l, f.clientCA, f.serverCert, f.clientCommonName, TCPListenerHandshakeTimeout) + return tl, nil +} diff --git a/cmd/control.go b/cmd/control.go index fee5349..77daa44 100644 --- a/cmd/control.go +++ b/cmd/control.go @@ -7,14 +7,14 @@ import ( "fmt" "github.com/pkg/errors" "github.com/spf13/cobra" + "github.com/zrepl/zrepl/cmd/daemon" "github.com/zrepl/zrepl/logger" + "github.com/zrepl/zrepl/version" "io" golog "log" "net" "net/http" "os" - "github.com/zrepl/zrepl/version" - "github.com/zrepl/zrepl/cmd/daemon" ) var controlCmd = &cobra.Command{ diff --git a/cmd/daemon.go b/cmd/daemon.go index 3fd8d2f..dd45234 100644 --- a/cmd/daemon.go +++ b/cmd/daemon.go @@ -5,13 +5,13 @@ import ( "fmt" "github.com/spf13/cobra" "github.com/zrepl/zrepl/cmd/config" + "github.com/zrepl/zrepl/cmd/daemon" + "github.com/zrepl/zrepl/cmd/daemon/job" "github.com/zrepl/zrepl/logger" "os" "os/signal" "syscall" "time" - "github.com/zrepl/zrepl/cmd/daemon" - "github.com/zrepl/zrepl/cmd/daemon/job" ) // daemonCmd represents the daemon command @@ -92,6 +92,7 @@ func doDaemon(cmd *cobra.Command, args []string) { daemonJobs := make([]job.Job, 0, len(conf.Jobs)) for i := range conf.Jobs { + parseJob() daemonJobs = append(daemonJobs, daemonJobAdaptor{conf.Jobs[i]}) } daemon.Run(ctx, conf.Global.Control.Sockpath, conf.Global.logging.Outlets, daemonJobs) diff --git a/cmd/daemon/control.go b/cmd/daemon/control.go index c7d17a1..0aff1ce 100644 --- a/cmd/daemon/control.go +++ b/cmd/daemon/control.go @@ -5,18 +5,18 @@ import ( "context" "encoding/json" "github.com/pkg/errors" + "github.com/zrepl/zrepl/cmd/daemon/job" + "github.com/zrepl/zrepl/cmd/helpers" "github.com/zrepl/zrepl/logger" + "github.com/zrepl/zrepl/version" "io" "net" "net/http" - "github.com/zrepl/zrepl/cmd/daemon/job" - "github.com/zrepl/zrepl/version" - "github.com/zrepl/zrepl/cmd/helpers" ) type controlJob struct { sockaddr *net.UnixAddr - jobs *jobs + jobs *jobs } func newControlJob(sockpath string, jobs *jobs) (j *controlJob, err error) { diff --git a/cmd/daemon/daemon.go b/cmd/daemon/daemon.go index 6240d01..b3d54b5 100644 --- a/cmd/daemon/daemon.go +++ b/cmd/daemon/daemon.go @@ -2,19 +2,18 @@ package daemon import ( "context" - "os" - "os/signal" - "syscall" - "sync" "fmt" "github.com/zrepl/zrepl/cmd/daemon/job" - "strings" "github.com/zrepl/zrepl/logger" "github.com/zrepl/zrepl/version" + "os" + "os/signal" + "strings" + "sync" + "syscall" "time" ) - func Run(ctx context.Context, controlSockpath string, outlets *logger.Outlets, confJobs []job.Job) { ctx, cancel := context.WithCancel(ctx) @@ -59,10 +58,10 @@ func Run(ctx context.Context, controlSockpath string, outlets *logger.Outlets, c } select { - case <-jobs.wait(): - log.Info("all jobs finished") - case <-ctx.Done(): - log.WithError(ctx.Err()).Info("context finished") + case <-jobs.wait(): + log.Info("all jobs finished") + case <-ctx.Done(): + log.WithError(ctx.Err()).Info("context finished") } log.Info("daemon exiting") } @@ -71,15 +70,15 @@ type jobs struct { wg sync.WaitGroup // m protects all fields below it - m sync.RWMutex + m sync.RWMutex wakeups map[string]job.WakeupChan // by JobName - jobs map[string]job.Job + jobs map[string]job.Job } func newJobs() *jobs { return &jobs{ wakeups: make(map[string]job.WakeupChan), - jobs: make(map[string]job.Job), + jobs: make(map[string]job.Job), } } @@ -102,7 +101,7 @@ func (s *jobs) status() map[string]interface{} { defer s.m.RUnlock() type res struct { - name string + name string status interface{} } var wg sync.WaitGroup @@ -125,7 +124,7 @@ func (s *jobs) status() map[string]interface{} { const ( jobNamePrometheus = "_prometheus" - jobNameControl = "_control" + jobNameControl = "_control" ) func IsInternalJobName(s string) bool { diff --git a/cmd/daemon/job/job.go b/cmd/daemon/job/job.go index 59cc147..56e25af 100644 --- a/cmd/daemon/job/job.go +++ b/cmd/daemon/job/job.go @@ -1,8 +1,8 @@ package job import ( - "github.com/zrepl/zrepl/logger" "context" + "github.com/zrepl/zrepl/logger" ) type Logger = logger.Logger diff --git a/cmd/daemon/prometheus.go b/cmd/daemon/prometheus.go index afe4fd5..1cef3d0 100644 --- a/cmd/daemon/prometheus.go +++ b/cmd/daemon/prometheus.go @@ -4,10 +4,10 @@ import ( "context" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" + "github.com/zrepl/zrepl/cmd/daemon/job" "github.com/zrepl/zrepl/zfs" "net" "net/http" - "github.com/zrepl/zrepl/cmd/daemon/job" ) type prometheusJob struct { @@ -48,7 +48,7 @@ func init() { prometheus.MustRegister(prom.taskLogEntries) } -func (j *prometheusJob) Name() string { return jobNamePrometheus } +func (j *prometheusJob) Name() string { return jobNamePrometheus } func (j *prometheusJob) Status() interface{} { return nil } diff --git a/cmd/helpers/helpers.go b/cmd/helpers/helpers.go index 01d6a9c..bcf2cf5 100644 --- a/cmd/helpers/helpers.go +++ b/cmd/helpers/helpers.go @@ -1,10 +1,10 @@ package helpers import ( - "path/filepath" - "os" "github.com/pkg/errors" "net" + "os" + "path/filepath" ) func PreparePrivateSockpath(sockpath string) error { diff --git a/cmd/prune.go b/cmd/prune.go index 33ba755..6963451 100644 --- a/cmd/prune.go +++ b/cmd/prune.go @@ -8,11 +8,10 @@ import ( ) type Pruner struct { - Now time.Time - DryRun bool - DatasetFilter zfs.DatasetFilter - SnapshotPrefix string - PrunePolicy PrunePolicy + Now time.Time + DryRun bool + DatasetFilter zfs.DatasetFilter + policies []PrunePolicy } type PruneResult struct { @@ -38,14 +37,14 @@ func (p *Pruner) filterFilesystems(ctx context.Context) (filesystems []*zfs.Data func (p *Pruner) filterVersions(ctx context.Context, fs *zfs.DatasetPath) (fsversions []zfs.FilesystemVersion, stop bool) { log := getLogger(ctx).WithField("fs", fs.ToString()) - filter := NewPrefixFilter(p.SnapshotPrefix) + filter := AnyFSVFilter{} fsversions, err := zfs.ZFSListFilesystemVersions(fs, filter) if err != nil { log.WithError(err).Error("error listing filesytem versions") return nil, true } if len(fsversions) == 0 { - log.WithField("prefix", p.SnapshotPrefix).Info("no filesystem versions matching prefix") + log.Info("no filesystem versions matching prefix") return nil, true } return fsversions, false diff --git a/cmd/version.go b/cmd/version.go index d4abe03..ab838b2 100644 --- a/cmd/version.go +++ b/cmd/version.go @@ -19,4 +19,3 @@ func init() { func doVersion(cmd *cobra.Command, args []string) { fmt.Println(version.NewZreplVersionInformation().String()) } - diff --git a/version/version.go b/version/version.go index 8866f6f..19e7aa9 100644 --- a/version/version.go +++ b/version/version.go @@ -1,8 +1,8 @@ package version import ( - "runtime" "fmt" + "runtime" ) var ( From 4073c5dfb0439d5ebd6e50d66a09732f66337ab0 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Mon, 27 Aug 2018 16:17:39 +0200 Subject: [PATCH 057/167] change pruning testing function to use set compare on names --- cmd/pruning/pruning_test.go | 66 +++++++++++++++++++++++++------------ 1 file changed, 45 insertions(+), 21 deletions(-) diff --git a/cmd/pruning/pruning_test.go b/cmd/pruning/pruning_test.go index cd97311..4cdeb45 100644 --- a/cmd/pruning/pruning_test.go +++ b/cmd/pruning/pruning_test.go @@ -1,6 +1,7 @@ package pruning import ( + "fmt" "github.com/stretchr/testify/assert" "testing" "time" @@ -18,14 +19,40 @@ func (s stubSnap) Replicated() bool { return s.replicated } func (s stubSnap) Date() time.Time { return s.date } -func TestPruneSnapshots(t *testing.T) { +type testCase struct { + inputs []Snapshot + rules []KeepRule + exp, eff map[string]bool +} - type testCase struct { - inputs []Snapshot - rules []KeepRule - exp, eff []Snapshot +func testTable(tcs map[string]testCase, t *testing.T) { + mapEqual := func(a, b map[string]bool) bool { + if len(a) != len(b) { + return false + } + for k, v := range a { + if w, ok := b[k]; !ok || v != w { + return false + } + } + return true } + for name := range tcs { + t.Run(name, func(t *testing.T) { + tc := tcs[name] + remove := PruneSnapshots(tc.inputs, tc.rules) + tc.eff = make(map[string]bool) + for _, s := range remove { + tc.eff[s.Name()] = true + } + assert.True(t, mapEqual(tc.exp, tc.eff), fmt.Sprintf("is %v but should be %v", tc.eff, tc.exp)) + }) + } +} + +func TestPruneSnapshots(t *testing.T) { + inputs := map[string][]Snapshot{ "s1": []Snapshot{ stubSnap{name: "foo_123"}, @@ -40,8 +67,8 @@ func TestPruneSnapshots(t *testing.T) { rules: []KeepRule{ MustKeepRegex("foo_"), }, - exp: []Snapshot{ - stubSnap{name: "bar_123"}, + exp: map[string]bool{ + "bar_123": true, }, }, "multipleRules": { @@ -50,7 +77,7 @@ func TestPruneSnapshots(t *testing.T) { MustKeepRegex("foo_"), MustKeepRegex("bar_"), }, - exp: []Snapshot{}, + exp: map[string]bool{}, }, "onlyThoseRemovedByAllAreRemoved": { inputs: inputs["s1"], @@ -58,31 +85,28 @@ func TestPruneSnapshots(t *testing.T) { MustKeepRegex("notInS1"), // would remove all MustKeepRegex("bar_"), // would remove all but bar_, i.e. foo_.* }, - exp: []Snapshot{ - stubSnap{name: "foo_123"}, - stubSnap{name: "foo_456"}, + exp: map[string]bool{ + "foo_123": true, + "foo_456": true, }, }, "noRulesKeepsAll": { inputs: inputs["s1"], rules: []KeepRule{}, - exp: inputs["s1"], + exp: map[string]bool{ + "foo_123": true, + "foo_456": true, + "bar_123": true, + }, }, "noSnaps": { inputs: []Snapshot{}, rules: []KeepRule{ MustKeepRegex("foo_"), }, - exp: []Snapshot{}, + exp: map[string]bool{}, }, } - for name := range tcs { - t.Run(name, func(t *testing.T) { - tc := tcs[name] - tc.eff = PruneSnapshots(tc.inputs, tc.rules) - assert.Equal(t, tc.exp, tc.eff) - }) - } - + testTable(tcs, t) } From c7237cb09d6d0b10f36c86a2550af3f46cf8c0da Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Mon, 27 Aug 2018 16:24:19 +0200 Subject: [PATCH 058/167] test keep last n pruning --- cmd/pruning/keep_last_n_test.go | 77 +++++++++++++++++++++++++++++++++ 1 file changed, 77 insertions(+) create mode 100644 cmd/pruning/keep_last_n_test.go diff --git a/cmd/pruning/keep_last_n_test.go b/cmd/pruning/keep_last_n_test.go new file mode 100644 index 0000000..21e19f8 --- /dev/null +++ b/cmd/pruning/keep_last_n_test.go @@ -0,0 +1,77 @@ +package pruning + +import ( + "github.com/stretchr/testify/assert" + "testing" + "time" +) + +func TestKeepLastN(t *testing.T) { + + o := func(minutes int) time.Time { + return time.Unix(123, 0).Add(time.Duration(minutes) * time.Minute) + } + + inputs := map[string][]Snapshot{ + "s1": []Snapshot{ + stubSnap{name: "1", date: o(10)}, + stubSnap{name: "2", date: o(20)}, + stubSnap{name: "3", date: o(15)}, + stubSnap{name: "4", date: o(30)}, + stubSnap{name: "5", date: o(30)}, + }, + "s2": []Snapshot{}, + } + + tcs := map[string]testCase{ + "keep2": { + inputs: inputs["s1"], + rules: []KeepRule{ + KeepLastN{2}, + }, + exp: map[string]bool{ + "4": true, "5": true, + }, + }, + "keep1": { // Keep one of two with same time + inputs: inputs["s1"], + rules: []KeepRule{ + KeepLastN{1}, + }, + exp: map[string]bool{ + "4": true, //5 would be ok too + }, + }, + "keepMany": { + inputs: inputs["s1"], + rules: []KeepRule{ + KeepLastN{100}, + }, + exp: map[string]bool{ + "1": true, + "2": true, + "3": true, + "4": true, + "5": true, + }, + }, + "empty": { + inputs: inputs["s2"], + rules: []KeepRule{ + KeepLastN{100}, + }, + exp: map[string]bool{}, + }, + } + + testTable(tcs, t) + + t.Run("mustBePositive", func(t *testing.T) { + var err error + _, err = NewKeepLastN(0) + assert.Error(t, err) + _, err = NewKeepLastN(-5) + assert.Error(t, err) + }) + +} From 89dc267780b42b7216cc0ec2db9b640f8e1f1608 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 27 Aug 2018 19:10:55 +0200 Subject: [PATCH 059/167] start implementing new daemon in package daemon --- daemon/control.go | 142 ++++++++++++++++++ daemon/daemon.go | 176 +++++++++++++++++++++++ daemon/job/build_jobs.go | 27 ++++ daemon/job/job.go | 47 ++++++ daemon/logging/build_logging.go | 181 +++++++++++++++++++++++ daemon/logging/logging_formatters.go | 208 +++++++++++++++++++++++++++ daemon/logging/logging_outlets.go | 167 +++++++++++++++++++++ daemon/main.go | 8 ++ daemon/pprof.go | 80 +++++++++++ daemon/prometheus.go | 82 +++++++++++ main.go | 42 +++++- 11 files changed, 1158 insertions(+), 2 deletions(-) create mode 100644 daemon/control.go create mode 100644 daemon/daemon.go create mode 100644 daemon/job/build_jobs.go create mode 100644 daemon/job/job.go create mode 100644 daemon/logging/build_logging.go create mode 100644 daemon/logging/logging_formatters.go create mode 100644 daemon/logging/logging_outlets.go create mode 100644 daemon/main.go create mode 100644 daemon/pprof.go create mode 100644 daemon/prometheus.go diff --git a/daemon/control.go b/daemon/control.go new file mode 100644 index 0000000..0aff1ce --- /dev/null +++ b/daemon/control.go @@ -0,0 +1,142 @@ +package daemon + +import ( + "bytes" + "context" + "encoding/json" + "github.com/pkg/errors" + "github.com/zrepl/zrepl/cmd/daemon/job" + "github.com/zrepl/zrepl/cmd/helpers" + "github.com/zrepl/zrepl/logger" + "github.com/zrepl/zrepl/version" + "io" + "net" + "net/http" +) + +type controlJob struct { + sockaddr *net.UnixAddr + jobs *jobs +} + +func newControlJob(sockpath string, jobs *jobs) (j *controlJob, err error) { + j = &controlJob{jobs: jobs} + + j.sockaddr, err = net.ResolveUnixAddr("unix", sockpath) + if err != nil { + err = errors.Wrap(err, "cannot resolve unix address") + return + } + + return +} + +func (j *controlJob) Name() string { return jobNameControl } + +func (j *controlJob) Status() interface{} { return nil } + +const ( + ControlJobEndpointPProf string = "/debug/pprof" + ControlJobEndpointVersion string = "/version" + ControlJobEndpointStatus string = "/status" +) + +func (j *controlJob) Run(ctx context.Context) { + + log := job.GetLogger(ctx) + defer log.Info("control job finished") + + l, err := helpers.ListenUnixPrivate(j.sockaddr) + if err != nil { + log.WithError(err).Error("error listening") + return + } + + pprofServer := NewPProfServer(ctx) + + mux := http.NewServeMux() + mux.Handle(ControlJobEndpointPProf, requestLogger{log: log, handlerFunc: func(w http.ResponseWriter, r *http.Request) { + var msg PprofServerControlMsg + err := json.NewDecoder(r.Body).Decode(&msg) + if err != nil { + log.WithError(err).Error("bad pprof request from client") + w.WriteHeader(http.StatusBadRequest) + } + pprofServer.Control(msg) + w.WriteHeader(200) + }}) + mux.Handle(ControlJobEndpointVersion, + requestLogger{log: log, handler: jsonResponder{func() (interface{}, error) { + return version.NewZreplVersionInformation(), nil + }}}) + mux.Handle(ControlJobEndpointStatus, + requestLogger{log: log, handler: jsonResponder{func() (interface{}, error) { + s := j.jobs.status() + return s, nil + }}}) + server := http.Server{Handler: mux} + +outer: + for { + + served := make(chan error) + go func() { + served <- server.Serve(l) + close(served) + }() + + select { + case <-ctx.Done(): + log.WithError(ctx.Err()).Info("context done") + server.Shutdown(context.Background()) + break outer + case err = <-served: + if err != nil { + log.WithError(err).Error("error serving") + break outer + } + } + + } + +} + +type jsonResponder struct { + producer func() (interface{}, error) +} + +func (j jsonResponder) ServeHTTP(w http.ResponseWriter, r *http.Request) { + res, err := j.producer() + if err != nil { + w.WriteHeader(http.StatusInternalServerError) + io.WriteString(w, err.Error()) + return + } + var buf bytes.Buffer + err = json.NewEncoder(&buf).Encode(res) + if err != nil { + w.WriteHeader(http.StatusInternalServerError) + io.WriteString(w, err.Error()) + } else { + io.Copy(w, &buf) + } +} + +type requestLogger struct { + log logger.Logger + handler http.Handler + handlerFunc http.HandlerFunc +} + +func (l requestLogger) ServeHTTP(w http.ResponseWriter, r *http.Request) { + log := l.log.WithField("method", r.Method).WithField("url", r.URL) + log.Info("start") + if l.handlerFunc != nil { + l.handlerFunc(w, r) + } else if l.handler != nil { + l.handler.ServeHTTP(w, r) + } else { + log.Error("no handler or handlerFunc configured") + } + log.Info("finish") +} diff --git a/daemon/daemon.go b/daemon/daemon.go new file mode 100644 index 0000000..a408aa3 --- /dev/null +++ b/daemon/daemon.go @@ -0,0 +1,176 @@ +package daemon + +import ( + "context" + "fmt" + "github.com/zrepl/zrepl/daemon/job" + "github.com/zrepl/zrepl/logger" + "github.com/zrepl/zrepl/version" + "os" + "os/signal" + "strings" + "sync" + "syscall" + "time" + "github.com/zrepl/zrepl/cmd/config" + "github.com/zrepl/zrepl/daemon/logging" + "github.com/pkg/errors" +) + + +func Run(conf config.Config) error { + + ctx, cancel := context.WithCancel(context.Background()) + + defer cancel() + sigChan := make(chan os.Signal, 1) + signal.Notify(sigChan, syscall.SIGINT, syscall.SIGTERM) + go func() { + <-sigChan + cancel() + }() + + outlets, err := logging.OutletsFromConfig(conf.Global.Logging) + if err != nil { + return errors.Wrap(err, "cannot build logging from config") + } + + confJobs, err := job.JobsFromConfig(conf) + if err != nil { + return errors.Wrap(err, "cannot build jobs from config") + } + + log := logger.NewLogger(outlets, 1*time.Second) + log.Info(version.NewZreplVersionInformation().String()) + + for _, job := range confJobs { + if IsInternalJobName(job.Name()) { + panic(fmt.Sprintf("internal job name used for config job '%s'", job.Name())) //FIXME + } + } + + ctx = job.WithLogger(ctx, log) + + jobs := newJobs() + + // start control socket + controlJob, err := newControlJob(conf.Global.Control.SockPath, jobs) + if err != nil { + panic(err) // FIXME + } + jobs.start(ctx, controlJob, true) + + // start prometheus + //var promJob *prometheusJob // FIXME + //jobs.start(ctx, promJob, true) + + log.Info("starting daemon") + + // start regular jobs + for _, j := range confJobs { + jobs.start(ctx, j, false) + } + + select { + case <-jobs.wait(): + log.Info("all jobs finished") + case <-ctx.Done(): + log.WithError(ctx.Err()).Info("context finished") + } + log.Info("daemon exiting") + return nil +} + +type jobs struct { + wg sync.WaitGroup + + // m protects all fields below it + m sync.RWMutex + wakeups map[string]job.WakeupChan // by JobName + jobs map[string]job.Job +} + +func newJobs() *jobs { + return &jobs{ + wakeups: make(map[string]job.WakeupChan), + jobs: make(map[string]job.Job), + } +} + +const ( + logJobField string = "job" + logTaskField string = "task" + logSubsysField string = "subsystem" +) + +func (s *jobs) wait() <-chan struct{} { + ch := make(chan struct{}) + go func() { + s.wg.Wait() + }() + return ch +} + +func (s *jobs) status() map[string]interface{} { + s.m.RLock() + defer s.m.RUnlock() + + type res struct { + name string + status interface{} + } + var wg sync.WaitGroup + c := make(chan res, len(s.jobs)) + for name, j := range s.jobs { + wg.Add(1) + go func(name string, j job.Job) { + defer wg.Done() + c <- res{name: name, status: j.Status()} + }(name, j) + } + wg.Wait() + close(c) + ret := make(map[string]interface{}, len(s.jobs)) + for res := range c { + ret[res.name] = res.status + } + return ret +} + +const ( + jobNamePrometheus = "_prometheus" + jobNameControl = "_control" +) + +func IsInternalJobName(s string) bool { + return strings.HasPrefix(s, "_") +} + +func (s *jobs) start(ctx context.Context, j job.Job, internal bool) { + s.m.Lock() + defer s.m.Unlock() + + jobLog := job.GetLogger(ctx).WithField(logJobField, j.Name()) + jobName := j.Name() + if !internal && IsInternalJobName(jobName) { + panic(fmt.Sprintf("internal job name used for non-internal job %s", jobName)) + } + if internal && !IsInternalJobName(jobName) { + panic(fmt.Sprintf("internal job does not use internal job name %s", jobName)) + } + if _, ok := s.jobs[jobName]; ok { + panic(fmt.Sprintf("duplicate job name %s", jobName)) + } + s.jobs[jobName] = j + ctx = job.WithLogger(ctx, jobLog) + ctx, wakeupChan := job.WithWakeup(ctx) + s.wakeups[jobName] = wakeupChan + + s.wg.Add(1) + go func() { + defer s.wg.Done() + jobLog.Info("starting job") + defer jobLog.Info("job exited") + j.Run(ctx) + }() +} diff --git a/daemon/job/build_jobs.go b/daemon/job/build_jobs.go new file mode 100644 index 0000000..b36fc90 --- /dev/null +++ b/daemon/job/build_jobs.go @@ -0,0 +1,27 @@ +package job + +import ( + "github.com/zrepl/zrepl/cmd/config" + "fmt" +) + +func JobsFromConfig(c config.Config) ([]Job, error) { + js := make([]Job, len(c.Jobs)) + for i := range c.Jobs { + j, err := buildJob(c.Global, c.Jobs[i]) + if err != nil { + return nil, err + } + js[i] = j + } + return js, nil +} + +func buildJob(c config.Global, in config.JobEnum) (j Job, err error) { + + switch v := in.Ret.(type) { + default: + panic(fmt.Sprintf("implementation error: unknown job type %s", v)) + } + +} \ No newline at end of file diff --git a/daemon/job/job.go b/daemon/job/job.go new file mode 100644 index 0000000..56e25af --- /dev/null +++ b/daemon/job/job.go @@ -0,0 +1,47 @@ +package job + +import ( + "context" + "github.com/zrepl/zrepl/logger" +) + +type Logger = logger.Logger + +type contextKey int + +const ( + contextKeyLog contextKey = iota + contextKeyWakeup +) + +func GetLogger(ctx context.Context) Logger { + if l, ok := ctx.Value(contextKeyLog).(Logger); ok { + return l + } + return logger.NewNullLogger() +} + +func WithLogger(ctx context.Context, l Logger) context.Context { + return context.WithValue(ctx, contextKeyLog, l) +} + +func WithWakeup(ctx context.Context) (context.Context, WakeupChan) { + wc := make(chan struct{}, 1) + return context.WithValue(ctx, contextKeyWakeup, wc), wc +} + +type Job interface { + Name() string + Run(ctx context.Context) + Status() interface{} +} + +type WakeupChan <-chan struct{} + +func WaitWakeup(ctx context.Context) WakeupChan { + wc, ok := ctx.Value(contextKeyWakeup).(WakeupChan) + if !ok { + wc = make(chan struct{}) + } + return wc +} diff --git a/daemon/logging/build_logging.go b/daemon/logging/build_logging.go new file mode 100644 index 0000000..6e2b4d6 --- /dev/null +++ b/daemon/logging/build_logging.go @@ -0,0 +1,181 @@ +package logging + +import ( + "github.com/zrepl/zrepl/cmd/config" + "os" + "github.com/mattn/go-isatty" + "crypto/tls" + "github.com/pkg/errors" + "crypto/x509" + "github.com/zrepl/zrepl/cmd/tlsconf" + "github.com/zrepl/zrepl/logger" +) + +func OutletsFromConfig(in []config.LoggingOutletEnum) (*logger.Outlets, error) { + + outlets := logger.NewOutlets() + + if len(in) == 0 { + // Default config + out := WriterOutlet{&HumanFormatter{}, os.Stdout} + outlets.Add(out, logger.Warn) + return outlets, nil + } + + var syslogOutlets, stdoutOutlets int + for lei, le := range in { + + outlet, minLevel, err := parseOutlet(le) + if err != nil { + return nil, errors.Wrapf(err, "cannot parse outlet #%d", lei) + } + var _ logger.Outlet = WriterOutlet{} + var _ logger.Outlet = &SyslogOutlet{} + switch outlet.(type) { + case *SyslogOutlet: + syslogOutlets++ + case WriterOutlet: + stdoutOutlets++ + } + + outlets.Add(outlet, minLevel) + + } + + if syslogOutlets > 1 { + return nil, errors.Errorf("can only define one 'syslog' outlet") + } + if stdoutOutlets > 1 { + return nil, errors.Errorf("can only define one 'stdout' outlet") + } + + return outlets, nil + +} + + +func parseLogFormat(i interface{}) (f EntryFormatter, err error) { + var is string + switch j := i.(type) { + case string: + is = j + default: + return nil, errors.Errorf("invalid log format: wrong type: %T", i) + } + + switch is { + case "human": + return &HumanFormatter{}, nil + case "logfmt": + return &LogfmtFormatter{}, nil + case "json": + return &JSONFormatter{}, nil + default: + return nil, errors.Errorf("invalid log format: '%s'", is) + } + +} + +func parseOutlet(in config.LoggingOutletEnum) (o logger.Outlet, level logger.Level, err error) { + + parseCommon := func(common config.LoggingOutletCommon) (logger.Level, EntryFormatter, error) { + if common.Level == "" || common.Format == "" { + return 0, nil, errors.Errorf("must specify 'level' and 'format' field") + } + + minLevel, err := logger.ParseLevel(common.Level) + if err != nil { + return 0, nil, errors.Wrap(err, "cannot parse 'level' field") + } + formatter, err := parseLogFormat(common.Format) + if err != nil { + return 0, nil, errors.Wrap(err, "cannot parse 'formatter' field") + } + return minLevel, formatter, nil + } + + var f EntryFormatter + + switch v := in.Ret.(type) { + case config.StdoutLoggingOutlet: + level, f, err = parseCommon(v.LoggingOutletCommon) + if err != nil { + break + } + o, err = parseStdoutOutlet(v, f) + case config.TCPLoggingOutlet: + level, f, err = parseCommon(v.LoggingOutletCommon) + if err != nil { + break + } + o, err = parseTCPOutlet(v, f) + case config.SyslogLoggingOutlet: + level, f, err = parseCommon(v.LoggingOutletCommon) + if err != nil { + break + } + o, err = parseSyslogOutlet(v, f) + default: + panic(v) + } + return o, level, err +} + +func parseStdoutOutlet(in config.StdoutLoggingOutlet, formatter EntryFormatter) (WriterOutlet, error) { + flags := MetadataAll + writer := os.Stdout + if !isatty.IsTerminal(writer.Fd()) && !in.Time { + flags &= ^MetadataTime + } + + formatter.SetMetadataFlags(flags) + return WriterOutlet{ + formatter, + os.Stdout, + }, nil +} + +func parseTCPOutlet(in config.TCPLoggingOutlet, formatter EntryFormatter) (out *TCPOutlet, err error) { + var tlsConfig *tls.Config + if in.TLS != nil { + tlsConfig, err = func(m *config.TCPLoggingOutletTLS, host string) (*tls.Config, error) { + clientCert, err := tls.LoadX509KeyPair(m.Cert, m.Key) + if err != nil { + return nil, errors.Wrap(err, "cannot load client cert") + } + + var rootCAs *x509.CertPool + if m.CA == "" { + if rootCAs, err = x509.SystemCertPool(); err != nil { + return nil, errors.Wrap(err, "cannot open system cert pool") + } + } else { + rootCAs, err = tlsconf.ParseCAFile(m.CA) + if err != nil { + return nil, errors.Wrap(err, "cannot parse CA cert") + } + } + if rootCAs == nil { + panic("invariant violated") + } + + return tlsconf.ClientAuthClient(host, rootCAs, clientCert) + }(in.TLS, in.Address) + if err != nil { + return nil, errors.New("cannot not parse TLS config in field 'tls'") + } + } + + formatter.SetMetadataFlags(MetadataAll) + return NewTCPOutlet(formatter, in.Net, in.Address, tlsConfig, in.RetryInterval), nil + +} + +func parseSyslogOutlet(in config.SyslogLoggingOutlet, formatter EntryFormatter) (out *SyslogOutlet, err error) { + out = &SyslogOutlet{} + out.Formatter = formatter + out.Formatter.SetMetadataFlags(MetadataNone) + out.RetryInterval = in.RetryInterval + return out, nil +} + diff --git a/daemon/logging/logging_formatters.go b/daemon/logging/logging_formatters.go new file mode 100644 index 0000000..b968531 --- /dev/null +++ b/daemon/logging/logging_formatters.go @@ -0,0 +1,208 @@ +package logging + +import ( + "bytes" + "encoding/json" + "fmt" + "github.com/go-logfmt/logfmt" + "github.com/pkg/errors" + "github.com/zrepl/zrepl/logger" + "time" +) + +const ( + FieldLevel = "level" + FieldMessage = "msg" + FieldTime = "time" +) + +const ( + logJobField string = "job" + logTaskField string = "task" + logSubsysField string = "subsystem" +) + + +type MetadataFlags int64 + +const ( + MetadataTime MetadataFlags = 1 << iota + MetadataLevel + + MetadataNone MetadataFlags = 0 + MetadataAll MetadataFlags = ^0 +) + + +type NoFormatter struct{} + +func (f NoFormatter) SetMetadataFlags(flags MetadataFlags) {} + +func (f NoFormatter) Format(e *logger.Entry) ([]byte, error) { + return []byte(e.Message), nil +} + +type HumanFormatter struct { + metadataFlags MetadataFlags + ignoreFields map[string]bool +} + +const HumanFormatterDateFormat = time.RFC3339 + +func (f *HumanFormatter) SetMetadataFlags(flags MetadataFlags) { + f.metadataFlags = flags +} + +func (f *HumanFormatter) SetIgnoreFields(ignore []string) { + if ignore == nil { + f.ignoreFields = nil + return + } + f.ignoreFields = make(map[string]bool, len(ignore)) + + for _, field := range ignore { + f.ignoreFields[field] = true + } +} + +func (f *HumanFormatter) ignored(field string) bool { + return f.ignoreFields != nil && f.ignoreFields[field] +} + +func (f *HumanFormatter) Format(e *logger.Entry) (out []byte, err error) { + + var line bytes.Buffer + + if f.metadataFlags&MetadataTime != 0 { + fmt.Fprintf(&line, "%s ", e.Time.Format(HumanFormatterDateFormat)) + } + if f.metadataFlags&MetadataLevel != 0 { + fmt.Fprintf(&line, "[%s]", e.Level.Short()) + } + + prefixFields := []string{logJobField, logTaskField, logSubsysField} + prefixed := make(map[string]bool, len(prefixFields)+2) + for _, field := range prefixFields { + val, ok := e.Fields[field].(string) + if !ok { + continue + } + if !f.ignored(field) { + fmt.Fprintf(&line, "[%s]", val) + prefixed[field] = true + } + } + + if line.Len() > 0 { + fmt.Fprint(&line, ": ") + } + fmt.Fprint(&line, e.Message) + + if len(e.Fields)-len(prefixed) > 0 { + fmt.Fprint(&line, " ") + enc := logfmt.NewEncoder(&line) + for field, value := range e.Fields { + if prefixed[field] || f.ignored(field) { + continue + } + if err := logfmtTryEncodeKeyval(enc, field, value); err != nil { + return nil, err + } + } + } + + return line.Bytes(), nil +} + +type JSONFormatter struct { + metadataFlags MetadataFlags +} + +func (f *JSONFormatter) SetMetadataFlags(flags MetadataFlags) { + f.metadataFlags = flags +} + +func (f *JSONFormatter) Format(e *logger.Entry) ([]byte, error) { + data := make(logger.Fields, len(e.Fields)+3) + for k, v := range e.Fields { + switch v := v.(type) { + case error: + // Otherwise errors are ignored by `encoding/json` + // https://github.com/sirupsen/logrus/issues/137 + data[k] = v.Error() + default: + _, err := json.Marshal(v) + if err != nil { + return nil, errors.Errorf("field is not JSON encodable: %s", k) + } + data[k] = v + } + } + + data[FieldMessage] = e.Message + data[FieldTime] = e.Time.Format(time.RFC3339) + data[FieldLevel] = e.Level + + return json.Marshal(data) + +} + +type LogfmtFormatter struct { + metadataFlags MetadataFlags +} + +func (f *LogfmtFormatter) SetMetadataFlags(flags MetadataFlags) { + f.metadataFlags = flags +} + +func (f *LogfmtFormatter) Format(e *logger.Entry) ([]byte, error) { + var buf bytes.Buffer + enc := logfmt.NewEncoder(&buf) + + if f.metadataFlags&MetadataTime != 0 { + enc.EncodeKeyval(FieldTime, e.Time) + } + if f.metadataFlags&MetadataLevel != 0 { + enc.EncodeKeyval(FieldLevel, e.Level) + } + + // at least try and put job and task in front + prefixed := make(map[string]bool, 2) + prefix := []string{logJobField, logTaskField, logSubsysField} + for _, pf := range prefix { + v, ok := e.Fields[pf] + if !ok { + break + } + if err := logfmtTryEncodeKeyval(enc, pf, v); err != nil { + return nil, err // unlikely + } + prefixed[pf] = true + } + + enc.EncodeKeyval(FieldMessage, e.Message) + + for k, v := range e.Fields { + if !prefixed[k] { + if err := logfmtTryEncodeKeyval(enc, k, v); err != nil { + return nil, err + } + } + } + + return buf.Bytes(), nil +} + +func logfmtTryEncodeKeyval(enc *logfmt.Encoder, field, value interface{}) error { + + err := enc.EncodeKeyval(field, value) + switch err { + case nil: // ok + return nil + case logfmt.ErrUnsupportedValueType: + enc.EncodeKeyval(field, fmt.Sprintf("<%T>", value)) + return nil + } + return errors.Wrapf(err, "cannot encode field '%s'", field) + +} diff --git a/daemon/logging/logging_outlets.go b/daemon/logging/logging_outlets.go new file mode 100644 index 0000000..4a0fd7a --- /dev/null +++ b/daemon/logging/logging_outlets.go @@ -0,0 +1,167 @@ +package logging + +import ( + "bytes" + "context" + "crypto/tls" + "github.com/pkg/errors" + "github.com/zrepl/zrepl/logger" + "io" + "log/syslog" + "net" + "time" +) + + +type EntryFormatter interface { + SetMetadataFlags(flags MetadataFlags) + Format(e *logger.Entry) ([]byte, error) +} + +type WriterOutlet struct { + formatter EntryFormatter + writer io.Writer +} + +func (h WriterOutlet) WriteEntry(entry logger.Entry) error { + bytes, err := h.formatter.Format(&entry) + if err != nil { + return err + } + _, err = h.writer.Write(bytes) + h.writer.Write([]byte("\n")) + return err +} + +type TCPOutlet struct { + formatter EntryFormatter + // Specifies how much time must pass between a connection error and a reconnection attempt + // Log entries written to the outlet during this time interval are silently dropped. + connect func(ctx context.Context) (net.Conn, error) + entryChan chan *bytes.Buffer +} + +func NewTCPOutlet(formatter EntryFormatter, network, address string, tlsConfig *tls.Config, retryInterval time.Duration) *TCPOutlet { + + connect := func(ctx context.Context) (conn net.Conn, err error) { + deadl, ok := ctx.Deadline() + if !ok { + deadl = time.Time{} + } + dialer := net.Dialer{ + Deadline: deadl, + } + if tlsConfig != nil { + conn, err = tls.DialWithDialer(&dialer, network, address, tlsConfig) + } else { + conn, err = dialer.DialContext(ctx, network, address) + } + return + } + + entryChan := make(chan *bytes.Buffer, 1) // allow one message in flight while previos is in io.Copy() + + o := &TCPOutlet{ + formatter: formatter, + connect: connect, + entryChan: entryChan, + } + + go o.outLoop(retryInterval) + + return o +} + +// FIXME: use this method +func (h *TCPOutlet) Close() { + close(h.entryChan) +} + +func (h *TCPOutlet) outLoop(retryInterval time.Duration) { + + var retry time.Time + var conn net.Conn + for msg := range h.entryChan { + var err error + for conn == nil { + time.Sleep(time.Until(retry)) + ctx, cancel := context.WithDeadline(context.TODO(), time.Now().Add(retryInterval)) + conn, err = h.connect(ctx) + cancel() + if err != nil { + retry = time.Now().Add(retryInterval) + conn = nil + } + } + conn.SetWriteDeadline(time.Now().Add(retryInterval)) + _, err = io.Copy(conn, msg) + if err != nil { + retry = time.Now().Add(retryInterval) + conn.Close() + conn = nil + } + } +} + +func (h *TCPOutlet) WriteEntry(e logger.Entry) error { + + ebytes, err := h.formatter.Format(&e) + if err != nil { + return err + } + + buf := new(bytes.Buffer) + buf.Write(ebytes) + buf.WriteString("\n") + + select { + case h.entryChan <- buf: + return nil + default: + return errors.New("connection broken or not fast enough") + } +} + +type SyslogOutlet struct { + Formatter EntryFormatter + RetryInterval time.Duration + writer *syslog.Writer + lastConnectAttempt time.Time +} + +func (o *SyslogOutlet) WriteEntry(entry logger.Entry) error { + + bytes, err := o.Formatter.Format(&entry) + if err != nil { + return err + } + + s := string(bytes) + + if o.writer == nil { + now := time.Now() + if now.Sub(o.lastConnectAttempt) < o.RetryInterval { + return nil // not an error toward logger + } + o.writer, err = syslog.New(syslog.LOG_LOCAL0, "zrepl") + o.lastConnectAttempt = time.Now() + if err != nil { + o.writer = nil + return err + } + } + + switch entry.Level { + case logger.Debug: + return o.writer.Debug(s) + case logger.Info: + return o.writer.Info(s) + case logger.Warn: + return o.writer.Warning(s) + case logger.Error: + return o.writer.Err(s) + default: + return o.writer.Err(s) // write as error as reaching this case is in fact an error + } + +} diff --git a/daemon/main.go b/daemon/main.go new file mode 100644 index 0000000..488b020 --- /dev/null +++ b/daemon/main.go @@ -0,0 +1,8 @@ +package daemon + +import ( + "github.com/zrepl/zrepl/logger" +) + +type Logger = logger.Logger + diff --git a/daemon/pprof.go b/daemon/pprof.go new file mode 100644 index 0000000..2296ebd --- /dev/null +++ b/daemon/pprof.go @@ -0,0 +1,80 @@ +package daemon + +import ( + "net/http" + // FIXME: importing this package has the side-effect of poisoning the http.DefaultServeMux + // FIXME: with the /debug/pprof endpoints + "context" + "net" + "net/http/pprof" +) + +type PProfServer struct { + cc chan PprofServerControlMsg + state PprofServerControlMsg + listener net.Listener +} + +type PprofServerControlMsg struct { + // Whether the server should listen for requests on the given address + Run bool + // Must be set if Run is true, undefined otherwise + HttpListenAddress string +} + +func NewPProfServer(ctx context.Context) *PProfServer { + + s := &PProfServer{ + cc: make(chan PprofServerControlMsg), + } + + go s.controlLoop(ctx) + return s +} + +func (s *PProfServer) controlLoop(ctx context.Context) { +outer: + for { + + var msg PprofServerControlMsg + select { + case <-ctx.Done(): + if s.listener != nil { + s.listener.Close() + } + break outer + case msg = <-s.cc: + // proceed + } + + var err error + if msg.Run && s.listener == nil { + + s.listener, err = net.Listen("tcp", msg.HttpListenAddress) + if err != nil { + s.listener = nil + continue + } + + // FIXME: because net/http/pprof does not provide a mux, + mux := http.NewServeMux() + mux.Handle("/debug/pprof/", http.HandlerFunc(pprof.Index)) + mux.Handle("/debug/pprof/cmdline", http.HandlerFunc(pprof.Cmdline)) + mux.Handle("/debug/pprof/profile", http.HandlerFunc(pprof.Profile)) + mux.Handle("/debug/pprof/symbol", http.HandlerFunc(pprof.Symbol)) + mux.Handle("/debug/pprof/trace", http.HandlerFunc(pprof.Trace)) + go http.Serve(s.listener, mux) + continue + } + + if !msg.Run && s.listener != nil { + s.listener.Close() + s.listener = nil + continue + } + } +} + +func (s *PProfServer) Control(msg PprofServerControlMsg) { + s.cc <- msg +} diff --git a/daemon/prometheus.go b/daemon/prometheus.go new file mode 100644 index 0000000..1cef3d0 --- /dev/null +++ b/daemon/prometheus.go @@ -0,0 +1,82 @@ +package daemon + +import ( + "context" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promhttp" + "github.com/zrepl/zrepl/cmd/daemon/job" + "github.com/zrepl/zrepl/zfs" + "net" + "net/http" +) + +type prometheusJob struct { + listen string +} + +func newPrometheusJob(listen string) *prometheusJob { + return &prometheusJob{listen} +} + +var prom struct { + taskLastActiveStart *prometheus.GaugeVec + taskLastActiveDuration *prometheus.GaugeVec + taskLogEntries *prometheus.CounterVec +} + +func init() { + prom.taskLastActiveStart = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "zrepl", + Subsystem: "daemon", + Name: "task_last_active_start", + Help: "point in time at which the job task last left idle state", + }, []string{"zrepl_job", "job_type", "task"}) + prom.taskLastActiveDuration = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "zrepl", + Subsystem: "daemon", + Name: "task_last_active_duration", + Help: "seconds that the last run ob a job task spent between leaving and re-entering idle state", + }, []string{"zrepl_job", "job_type", "task"}) + prom.taskLogEntries = prometheus.NewCounterVec(prometheus.CounterOpts{ + Namespace: "zrepl", + Subsystem: "daemon", + Name: "task_log_entries", + Help: "number of log entries per job task and level", + }, []string{"zrepl_job", "job_type", "task", "level"}) + prometheus.MustRegister(prom.taskLastActiveStart) + prometheus.MustRegister(prom.taskLastActiveDuration) + prometheus.MustRegister(prom.taskLogEntries) +} + +func (j *prometheusJob) Name() string { return jobNamePrometheus } + +func (j *prometheusJob) Status() interface{} { return nil } + +func (j *prometheusJob) Run(ctx context.Context) { + + if err := zfs.PrometheusRegister(prometheus.DefaultRegisterer); err != nil { + panic(err) + } + + log := job.GetLogger(ctx) + + l, err := net.Listen("tcp", j.listen) + if err != nil { + log.WithError(err).Error("cannot listen") + } + go func() { + select { + case <-ctx.Done(): + l.Close() + } + }() + + mux := http.NewServeMux() + mux.Handle("/metrics", promhttp.Handler()) + + err = http.Serve(l, mux) + if err != nil { + log.WithError(err).Error("error while serving") + } + +} diff --git a/main.go b/main.go index a7919b6..452b02d 100644 --- a/main.go +++ b/main.go @@ -2,13 +2,51 @@ package main import ( - "github.com/zrepl/zrepl/cmd" "log" "os" + "github.com/spf13/cobra" + "github.com/zrepl/zrepl/daemon" + "github.com/zrepl/zrepl/cmd/config" ) +var rootCmd = &cobra.Command{ + Use: "zrepl", + Short: "ZFS dataset replication", + Long: `Replicate ZFS filesystems & volumes between pools: + + - push & pull mode + - automatic snapshot creation & pruning + - local / over the network + - ACLs instead of blank SSH access`, +} + + +var daemonCmd = &cobra.Command{ + Use: "daemon", + Short: "daemon", + RunE: func(cmd *cobra.Command, args []string) error { + conf, err := config.ParseConfig(rootArgs.configFile) + if err != nil { + return err + } + return daemon.Run(conf) + }, +} + +var rootArgs struct { + configFile string +} + +func init() { + //cobra.OnInitialize(initConfig) + rootCmd.PersistentFlags().StringVar(&rootArgs.configFile, "config", "", "config file path") + rootCmd.AddCommand(daemonCmd) +} + func main() { - if err := cmd.RootCmd.Execute(); err != nil { + + + if err := rootCmd.Execute(); err != nil { log.Printf("error executing root command: %s", err) os.Exit(1) } From df6e1bc64d14e0af76e5351eafe228b3d2ec451a Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 27 Aug 2018 19:13:35 +0200 Subject: [PATCH 060/167] privatize pprofServer --- daemon/pprof.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/daemon/pprof.go b/daemon/pprof.go index 2296ebd..6c96251 100644 --- a/daemon/pprof.go +++ b/daemon/pprof.go @@ -9,7 +9,7 @@ import ( "net/http/pprof" ) -type PProfServer struct { +type pprofServer struct { cc chan PprofServerControlMsg state PprofServerControlMsg listener net.Listener @@ -22,9 +22,9 @@ type PprofServerControlMsg struct { HttpListenAddress string } -func NewPProfServer(ctx context.Context) *PProfServer { +func NewPProfServer(ctx context.Context) *pprofServer { - s := &PProfServer{ + s := &pprofServer{ cc: make(chan PprofServerControlMsg), } @@ -32,7 +32,7 @@ func NewPProfServer(ctx context.Context) *PProfServer { return s } -func (s *PProfServer) controlLoop(ctx context.Context) { +func (s *pprofServer) controlLoop(ctx context.Context) { outer: for { @@ -75,6 +75,6 @@ outer: } } -func (s *PProfServer) Control(msg PprofServerControlMsg) { +func (s *pprofServer) Control(msg PprofServerControlMsg) { s.cc <- msg } From c69ebd3806715477b7980e0d15938b503db7b0df Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 27 Aug 2018 22:21:45 +0200 Subject: [PATCH 061/167] WIP rewrite the daemon cmd subdir does not build on purpose, it's only left in tree to grab old code and move it to github.com/zrepl/zrepl/daemon --- cmd/config_connect.go | 2 +- cmd/config_job_local.go | 4 +- cmd/config_job_pull.go | 4 +- cmd/config_job_source.go | 4 +- cmd/config_logging.go | 2 +- cmd/config_mapfilter.go | 2 +- cmd/config_parse.go | 2 +- cmd/config_serve_stdinserver.go | 2 +- cmd/config_serve_tcp.go | 5 +- cmd/config_serve_tls.go | 2 +- cmd/{daemon => daemon.deact}/control.go | 0 cmd/{daemon => daemon.deact}/daemon.go | 13 + cmd/{daemon => daemon.deact}/job/job.go | 0 cmd/{daemon => daemon.deact}/pprof.go | 0 cmd/{daemon => daemon.deact}/prometheus.go | 0 cmd/daemon.go | 2 +- .../retentiongrid/config_prune_grid.go | 2 +- {cmd/config => config}/config.go | 19 +- {cmd/config => config}/config_test.go | 0 {cmd/config => config}/retentiongrid.go | 0 {cmd/config => config}/samples/local.yml | 0 {cmd/config => config}/samples/pull.yml | 0 {cmd/config => config}/samples/pull_ssh.yml | 0 {cmd/config => config}/samples/push.yml | 0 {cmd/config => config}/samples/sink.yml | 0 {cmd/config => config}/samples/source.yml | 0 {cmd/config => config}/samples/source_ssh.yml | 0 daemon/connecter/connect_ssh.go | 66 +++++ daemon/connecter/connect_tcp.go | 24 ++ daemon/connecter/connect_tls.go | 43 +++ daemon/connecter/connecter.go | 20 ++ daemon/control.go | 81 +++++- daemon/daemon.go | 22 +- daemon/filters/fsmapfilter.go | 273 ++++++++++++++++++ daemon/filters/fsvfilter.go | 15 + daemon/job/build_jobs.go | 10 +- daemon/job/job.go | 40 ++- daemon/job/push.go | 86 ++++++ daemon/job/sink.go | 115 ++++++++ daemon/logging/adaptors.go | 32 ++ daemon/logging/build_logging.go | 39 ++- daemon/logging/logging_formatters.go | 11 +- daemon/logging/logging_outlets.go | 1 - daemon/main.go | 1 - {cmd/helpers => daemon/nethelpers}/helpers.go | 2 +- daemon/prometheus.go | 2 +- daemon/serve/serve.go | 26 ++ daemon/serve/serve_stdinserver.go | 79 +++++ daemon/serve/serve_tcp.go | 21 ++ daemon/serve/serve_tls.go | 52 ++++ {cmd/endpoint => endpoint}/context.go | 0 {cmd/endpoint => endpoint}/endpoint.go | 0 main.go | 23 +- {cmd/tlsconf => tlsconf}/tlsconf.go | 0 wakeup.go | 68 +++++ 55 files changed, 1133 insertions(+), 84 deletions(-) rename cmd/{daemon => daemon.deact}/control.go (100%) rename cmd/{daemon => daemon.deact}/daemon.go (95%) rename cmd/{daemon => daemon.deact}/job/job.go (100%) rename cmd/{daemon => daemon.deact}/pprof.go (100%) rename cmd/{daemon => daemon.deact}/prometheus.go (100%) rename {cmd/config => config}/config.go (95%) rename {cmd/config => config}/config_test.go (100%) rename {cmd/config => config}/retentiongrid.go (100%) rename {cmd/config => config}/samples/local.yml (100%) rename {cmd/config => config}/samples/pull.yml (100%) rename {cmd/config => config}/samples/pull_ssh.yml (100%) rename {cmd/config => config}/samples/push.yml (100%) rename {cmd/config => config}/samples/sink.yml (100%) rename {cmd/config => config}/samples/source.yml (100%) rename {cmd/config => config}/samples/source_ssh.yml (100%) create mode 100644 daemon/connecter/connect_ssh.go create mode 100644 daemon/connecter/connect_tcp.go create mode 100644 daemon/connecter/connect_tls.go create mode 100644 daemon/connecter/connecter.go create mode 100644 daemon/filters/fsmapfilter.go create mode 100644 daemon/filters/fsvfilter.go create mode 100644 daemon/job/push.go create mode 100644 daemon/job/sink.go create mode 100644 daemon/logging/adaptors.go rename {cmd/helpers => daemon/nethelpers}/helpers.go (98%) create mode 100644 daemon/serve/serve.go create mode 100644 daemon/serve/serve_stdinserver.go create mode 100644 daemon/serve/serve_tcp.go create mode 100644 daemon/serve/serve_tls.go rename {cmd/endpoint => endpoint}/context.go (100%) rename {cmd/endpoint => endpoint}/endpoint.go (100%) rename {cmd/tlsconf => tlsconf}/tlsconf.go (100%) create mode 100644 wakeup.go diff --git a/cmd/config_connect.go b/cmd/config_connect.go index efb84d4..81e9803 100644 --- a/cmd/config_connect.go +++ b/cmd/config_connect.go @@ -9,7 +9,7 @@ import ( "github.com/pkg/errors" "github.com/problame/go-netssh" "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/cmd/config" + "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/cmd/tlsconf" "time" ) diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go index c8b8c00..8bf4fa3 100644 --- a/cmd/config_job_local.go +++ b/cmd/config_job_local.go @@ -6,8 +6,8 @@ import ( "context" "github.com/mitchellh/mapstructure" "github.com/pkg/errors" - "github.com/zrepl/zrepl/cmd/config" - "github.com/zrepl/zrepl/cmd/endpoint" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/endpoint" "github.com/zrepl/zrepl/replication" "github.com/zrepl/zrepl/zfs" "sync" diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go index b20db21..f3167c5 100644 --- a/cmd/config_job_pull.go +++ b/cmd/config_job_pull.go @@ -11,8 +11,8 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/cmd/config" - "github.com/zrepl/zrepl/cmd/endpoint" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/endpoint" "github.com/zrepl/zrepl/replication" ) diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go index 76d2be5..8e1c283 100644 --- a/cmd/config_job_source.go +++ b/cmd/config_job_source.go @@ -6,8 +6,8 @@ import ( "github.com/pkg/errors" "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/cmd/config" - "github.com/zrepl/zrepl/cmd/endpoint" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/endpoint" "net" ) diff --git a/cmd/config_logging.go b/cmd/config_logging.go index 4f18f0a..2d3fe26 100644 --- a/cmd/config_logging.go +++ b/cmd/config_logging.go @@ -5,7 +5,7 @@ import ( "crypto/x509" "github.com/mattn/go-isatty" "github.com/pkg/errors" - "github.com/zrepl/zrepl/cmd/config" + "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/cmd/tlsconf" "github.com/zrepl/zrepl/logger" "os" diff --git a/cmd/config_mapfilter.go b/cmd/config_mapfilter.go index f1182b7..4c8f0a7 100644 --- a/cmd/config_mapfilter.go +++ b/cmd/config_mapfilter.go @@ -5,7 +5,7 @@ import ( "strings" "github.com/pkg/errors" - "github.com/zrepl/zrepl/cmd/endpoint" + "github.com/zrepl/zrepl/endpoint" "github.com/zrepl/zrepl/zfs" ) diff --git a/cmd/config_parse.go b/cmd/config_parse.go index 0419b9e..f30a9a6 100644 --- a/cmd/config_parse.go +++ b/cmd/config_parse.go @@ -8,7 +8,7 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/cmd/config" + "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/cmd/pruning/retentiongrid" "os" ) diff --git a/cmd/config_serve_stdinserver.go b/cmd/config_serve_stdinserver.go index 5fb19b9..63ef839 100644 --- a/cmd/config_serve_stdinserver.go +++ b/cmd/config_serve_stdinserver.go @@ -2,7 +2,7 @@ package cmd import ( "github.com/problame/go-netssh" - "github.com/zrepl/zrepl/cmd/config" + "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/cmd/helpers" "net" "path" diff --git a/cmd/config_serve_tcp.go b/cmd/config_serve_tcp.go index 9757b00..351578b 100644 --- a/cmd/config_serve_tcp.go +++ b/cmd/config_serve_tcp.go @@ -1,10 +1,9 @@ package cmd import ( - "net" + "github.com/zrepl/zrepl/config" "time" - - "github.com/zrepl/zrepl/cmd/config" + "net" ) type TCPListenerFactory struct { diff --git a/cmd/config_serve_tls.go b/cmd/config_serve_tls.go index 54ab53d..565bc07 100644 --- a/cmd/config_serve_tls.go +++ b/cmd/config_serve_tls.go @@ -8,7 +8,7 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" - "github.com/zrepl/zrepl/cmd/config" + "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/cmd/tlsconf" ) diff --git a/cmd/daemon/control.go b/cmd/daemon.deact/control.go similarity index 100% rename from cmd/daemon/control.go rename to cmd/daemon.deact/control.go diff --git a/cmd/daemon/daemon.go b/cmd/daemon.deact/daemon.go similarity index 95% rename from cmd/daemon/daemon.go rename to cmd/daemon.deact/daemon.go index b3d54b5..070d4fe 100644 --- a/cmd/daemon/daemon.go +++ b/cmd/daemon.deact/daemon.go @@ -1,5 +1,18 @@ package daemon +import ( + "context" + "github.com/zrepl/zrepl/logger" + "os" + "os/signal" + "syscall" + "time" + "github.com/zrepl/zrepl/version" + "fmt" +) + +.daesdfadsfsafjlsjfda + import ( "context" "fmt" diff --git a/cmd/daemon/job/job.go b/cmd/daemon.deact/job/job.go similarity index 100% rename from cmd/daemon/job/job.go rename to cmd/daemon.deact/job/job.go diff --git a/cmd/daemon/pprof.go b/cmd/daemon.deact/pprof.go similarity index 100% rename from cmd/daemon/pprof.go rename to cmd/daemon.deact/pprof.go diff --git a/cmd/daemon/prometheus.go b/cmd/daemon.deact/prometheus.go similarity index 100% rename from cmd/daemon/prometheus.go rename to cmd/daemon.deact/prometheus.go diff --git a/cmd/daemon.go b/cmd/daemon.go index dd45234..621c591 100644 --- a/cmd/daemon.go +++ b/cmd/daemon.go @@ -4,7 +4,7 @@ import ( "context" "fmt" "github.com/spf13/cobra" - "github.com/zrepl/zrepl/cmd/config" + "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/cmd/daemon" "github.com/zrepl/zrepl/cmd/daemon/job" "github.com/zrepl/zrepl/logger" diff --git a/cmd/pruning/retentiongrid/config_prune_grid.go b/cmd/pruning/retentiongrid/config_prune_grid.go index 5223b9d..de1c6ba 100644 --- a/cmd/pruning/retentiongrid/config_prune_grid.go +++ b/cmd/pruning/retentiongrid/config_prune_grid.go @@ -2,7 +2,7 @@ package retentiongrid import ( "github.com/pkg/errors" - "github.com/zrepl/zrepl/cmd/config" + "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/zfs" "math" "sort" diff --git a/cmd/config/config.go b/config/config.go similarity index 95% rename from cmd/config/config.go rename to config/config.go index a999732..57bb701 100644 --- a/cmd/config/config.go +++ b/config/config.go @@ -152,12 +152,13 @@ type TCPServe struct { } type TLSServe struct { - Type string `yaml:"type"` - Listen string `yaml:"listen"` - Ca string `yaml:"ca"` - Cert string `yaml:"cert"` - Key string `yaml:"key"` - ClientCN string `yaml:"client_cn"` + Type string `yaml:"type"` + Listen string `yaml:"listen"` + Ca string `yaml:"ca"` + Cert string `yaml:"cert"` + Key string `yaml:"key"` + ClientCN string `yaml:"client_cn"` + HandshakeTimeout time.Duration `yaml:"handshake_timeout,positive,default=10s"` } type StdinserverServer struct { @@ -195,19 +196,19 @@ type LoggingOutletCommon struct { type StdoutLoggingOutlet struct { LoggingOutletCommon `yaml:",inline"` - Time bool `yaml:"time"` + Time bool `yaml:"time,default=true"` } type SyslogLoggingOutlet struct { LoggingOutletCommon `yaml:",inline"` - RetryInterval time.Duration `yaml:"retry_interval,positive"` + RetryInterval time.Duration `yaml:"retry_interval,positive,default=10s"` } type TCPLoggingOutlet struct { LoggingOutletCommon `yaml:",inline"` Address string `yaml:"address"` Net string `yaml:"net,default=tcp"` - RetryInterval time.Duration `yaml:"retry_interval,positive"` + RetryInterval time.Duration `yaml:"retry_interval,positive,default=10s"` TLS *TCPLoggingOutletTLS `yaml:"tls,optional"` } diff --git a/cmd/config/config_test.go b/config/config_test.go similarity index 100% rename from cmd/config/config_test.go rename to config/config_test.go diff --git a/cmd/config/retentiongrid.go b/config/retentiongrid.go similarity index 100% rename from cmd/config/retentiongrid.go rename to config/retentiongrid.go diff --git a/cmd/config/samples/local.yml b/config/samples/local.yml similarity index 100% rename from cmd/config/samples/local.yml rename to config/samples/local.yml diff --git a/cmd/config/samples/pull.yml b/config/samples/pull.yml similarity index 100% rename from cmd/config/samples/pull.yml rename to config/samples/pull.yml diff --git a/cmd/config/samples/pull_ssh.yml b/config/samples/pull_ssh.yml similarity index 100% rename from cmd/config/samples/pull_ssh.yml rename to config/samples/pull_ssh.yml diff --git a/cmd/config/samples/push.yml b/config/samples/push.yml similarity index 100% rename from cmd/config/samples/push.yml rename to config/samples/push.yml diff --git a/cmd/config/samples/sink.yml b/config/samples/sink.yml similarity index 100% rename from cmd/config/samples/sink.yml rename to config/samples/sink.yml diff --git a/cmd/config/samples/source.yml b/config/samples/source.yml similarity index 100% rename from cmd/config/samples/source.yml rename to config/samples/source.yml diff --git a/cmd/config/samples/source_ssh.yml b/config/samples/source_ssh.yml similarity index 100% rename from cmd/config/samples/source_ssh.yml rename to config/samples/source_ssh.yml diff --git a/daemon/connecter/connect_ssh.go b/daemon/connecter/connect_ssh.go new file mode 100644 index 0000000..7efeec5 --- /dev/null +++ b/daemon/connecter/connect_ssh.go @@ -0,0 +1,66 @@ +package connecter + +import ( + "context" + "github.com/jinzhu/copier" + "github.com/pkg/errors" + "github.com/problame/go-netssh" + "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/config" + "net" + "time" +) + +type SSHStdinserverConnecter struct { + Host string + User string + Port uint16 + IdentityFile string + TransportOpenCommand []string + SSHCommand string + Options []string + dialTimeout time.Duration +} + +var _ streamrpc.Connecter = &SSHStdinserverConnecter{} + +func SSHStdinserverConnecterFromConfig(in *config.SSHStdinserverConnect) (c *SSHStdinserverConnecter, err error) { + + c = &SSHStdinserverConnecter{ + Host: in.Host, + User: in.User, + Port: in.Port, + IdentityFile: in.IdentityFile, + SSHCommand: in.SSHCommand, + Options: in.Options, + dialTimeout: in.DialTimeout, + } + return + +} + +type netsshConnToConn struct{ *netssh.SSHConn } + +var _ net.Conn = netsshConnToConn{} + +func (netsshConnToConn) SetDeadline(dl time.Time) error { return nil } +func (netsshConnToConn) SetReadDeadline(dl time.Time) error { return nil } +func (netsshConnToConn) SetWriteDeadline(dl time.Time) error { return nil } + +func (c *SSHStdinserverConnecter) Connect(dialCtx context.Context) (net.Conn, error) { + + var endpoint netssh.Endpoint + if err := copier.Copy(&endpoint, c); err != nil { + return nil, errors.WithStack(err) + } + dialCtx, dialCancel := context.WithTimeout(dialCtx, c.dialTimeout) // context.TODO tied to error handling below + defer dialCancel() + nconn, err := netssh.Dial(dialCtx, endpoint) + if err != nil { + if err == context.DeadlineExceeded { + err = errors.Errorf("dial_timeout of %s exceeded", c.dialTimeout) + } + return nil, err + } + return netsshConnToConn{nconn}, nil +} diff --git a/daemon/connecter/connect_tcp.go b/daemon/connecter/connect_tcp.go new file mode 100644 index 0000000..3d8b77e --- /dev/null +++ b/daemon/connecter/connect_tcp.go @@ -0,0 +1,24 @@ +package connecter + +import ( + "context" + "github.com/zrepl/zrepl/config" + "net" +) + +type TCPConnecter struct { + Address string + dialer net.Dialer +} + +func TCPConnecterFromConfig(in *config.TCPConnect) (*TCPConnecter, error) { + dialer := net.Dialer{ + Timeout: in.DialTimeout, + } + + return &TCPConnecter{in.Address, dialer}, nil +} + +func (c *TCPConnecter) Connect(dialCtx context.Context) (conn net.Conn, err error) { + return c.dialer.DialContext(dialCtx, "tcp", c.Address) +} diff --git a/daemon/connecter/connect_tls.go b/daemon/connecter/connect_tls.go new file mode 100644 index 0000000..28f3440 --- /dev/null +++ b/daemon/connecter/connect_tls.go @@ -0,0 +1,43 @@ +package connecter + +import ( + "context" + "crypto/tls" + "github.com/pkg/errors" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/tlsconf" + "net" +) + +type TLSConnecter struct { + Address string + dialer net.Dialer + tlsConfig *tls.Config +} + +func TLSConnecterFromConfig(in *config.TLSConnect) (*TLSConnecter, error) { + dialer := net.Dialer{ + Timeout: in.DialTimeout, + } + + ca, err := tlsconf.ParseCAFile(in.Ca) + if err != nil { + return nil, errors.Wrap(err, "cannot parse ca file") + } + + cert, err := tls.LoadX509KeyPair(in.Cert, in.Key) + if err != nil { + return nil, errors.Wrap(err, "cannot parse cert/key pair") + } + + tlsConfig, err := tlsconf.ClientAuthClient(in.ServerCN, ca, cert) + if err != nil { + return nil, errors.Wrap(err, "cannot build tls config") + } + + return &TLSConnecter{in.Address, dialer, tlsConfig}, nil +} + +func (c *TLSConnecter) Connect(dialCtx context.Context) (conn net.Conn, err error) { + return tls.DialWithDialer(&c.dialer, "tcp", c.Address, c.tlsConfig) +} diff --git a/daemon/connecter/connecter.go b/daemon/connecter/connecter.go new file mode 100644 index 0000000..a27ba7d --- /dev/null +++ b/daemon/connecter/connecter.go @@ -0,0 +1,20 @@ +package connecter + +import ( + "fmt" + "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/config" +) + +func FromConfig(g config.Global, in config.ConnectEnum) (streamrpc.Connecter, error) { + switch v := in.Ret.(type) { + case *config.SSHStdinserverConnect: + return SSHStdinserverConnecterFromConfig(v) + case *config.TCPConnect: + return TCPConnecterFromConfig(v) + case *config.TLSConnect: + return TLSConnecterFromConfig(v) + default: + panic(fmt.Sprintf("implementation error: unknown connecter type %T", v)) + } +} diff --git a/daemon/control.go b/daemon/control.go index 0aff1ce..c7e1307 100644 --- a/daemon/control.go +++ b/daemon/control.go @@ -5,8 +5,8 @@ import ( "context" "encoding/json" "github.com/pkg/errors" - "github.com/zrepl/zrepl/cmd/daemon/job" - "github.com/zrepl/zrepl/cmd/helpers" + "github.com/zrepl/zrepl/daemon/job" + "github.com/zrepl/zrepl/daemon/nethelpers" "github.com/zrepl/zrepl/logger" "github.com/zrepl/zrepl/version" "io" @@ -39,6 +39,7 @@ const ( ControlJobEndpointPProf string = "/debug/pprof" ControlJobEndpointVersion string = "/version" ControlJobEndpointStatus string = "/status" + ControlJobEndpointWakeup string = "/wakeup" ) func (j *controlJob) Run(ctx context.Context) { @@ -46,7 +47,7 @@ func (j *controlJob) Run(ctx context.Context) { log := job.GetLogger(ctx) defer log.Info("control job finished") - l, err := helpers.ListenUnixPrivate(j.sockaddr) + l, err := nethelpers.ListenUnixPrivate(j.sockaddr) if err != nil { log.WithError(err).Error("error listening") return @@ -55,25 +56,42 @@ func (j *controlJob) Run(ctx context.Context) { pprofServer := NewPProfServer(ctx) mux := http.NewServeMux() - mux.Handle(ControlJobEndpointPProf, requestLogger{log: log, handlerFunc: func(w http.ResponseWriter, r *http.Request) { - var msg PprofServerControlMsg - err := json.NewDecoder(r.Body).Decode(&msg) - if err != nil { - log.WithError(err).Error("bad pprof request from client") - w.WriteHeader(http.StatusBadRequest) - } - pprofServer.Control(msg) - w.WriteHeader(200) - }}) + mux.Handle(ControlJobEndpointPProf, + requestLogger{log: log, handler: jsonRequestResponder{func(decoder jsonDecoder) (interface{}, error) { + var msg PprofServerControlMsg + err := decoder(&msg) + if err != nil { + return nil, errors.Errorf("decode failed") + } + pprofServer.Control(msg) + return struct{}{}, nil + }}}) + mux.Handle(ControlJobEndpointVersion, requestLogger{log: log, handler: jsonResponder{func() (interface{}, error) { return version.NewZreplVersionInformation(), nil }}}) + mux.Handle(ControlJobEndpointStatus, requestLogger{log: log, handler: jsonResponder{func() (interface{}, error) { s := j.jobs.status() return s, nil }}}) + + mux.Handle(ControlJobEndpointWakeup, + requestLogger{log: log, handler: jsonRequestResponder{func(decoder jsonDecoder) (interface{}, error) { + type reqT struct { + Name string + } + var req reqT + if decoder(&req) != nil { + return nil, errors.Errorf("decode failed") + } + + err := j.jobs.wakeup(req.Name) + + return struct{}{}, err + }}}) server := http.Server{Handler: mux} outer: @@ -122,6 +140,43 @@ func (j jsonResponder) ServeHTTP(w http.ResponseWriter, r *http.Request) { } } +type jsonDecoder = func(interface{}) error + +type jsonRequestResponder struct { + producer func(decoder jsonDecoder) (interface{}, error) +} + +func (j jsonRequestResponder) ServeHTTP(w http.ResponseWriter, r *http.Request) { + var decodeError error + decoder := func(i interface{}) error { + err := json.NewDecoder(r.Body).Decode(&i) + decodeError = err + return err + } + res, producerErr := j.producer(decoder) + + //If we had a decode error ignore output of producer and return error + if decodeError != nil { + w.WriteHeader(http.StatusBadRequest) + io.WriteString(w, decodeError.Error()) + return + } + if producerErr != nil { + w.WriteHeader(http.StatusInternalServerError) + io.WriteString(w, producerErr.Error()) + return + } + + var buf bytes.Buffer + encodeErr := json.NewEncoder(&buf).Encode(res) + if encodeErr != nil { + w.WriteHeader(http.StatusInternalServerError) + io.WriteString(w, encodeErr.Error()) + } else { + io.Copy(w, &buf) + } +} + type requestLogger struct { log logger.Logger handler http.Handler diff --git a/daemon/daemon.go b/daemon/daemon.go index a408aa3..e231d72 100644 --- a/daemon/daemon.go +++ b/daemon/daemon.go @@ -3,7 +3,10 @@ package daemon import ( "context" "fmt" + "github.com/pkg/errors" + "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon/job" + "github.com/zrepl/zrepl/daemon/logging" "github.com/zrepl/zrepl/logger" "github.com/zrepl/zrepl/version" "os" @@ -12,12 +15,8 @@ import ( "sync" "syscall" "time" - "github.com/zrepl/zrepl/cmd/config" - "github.com/zrepl/zrepl/daemon/logging" - "github.com/pkg/errors" ) - func Run(conf config.Config) error { ctx, cancel := context.WithCancel(context.Background()) @@ -86,13 +85,13 @@ type jobs struct { // m protects all fields below it m sync.RWMutex - wakeups map[string]job.WakeupChan // by JobName + wakeups map[string]job.WakeupFunc // by Job.Name jobs map[string]job.Job } func newJobs() *jobs { return &jobs{ - wakeups: make(map[string]job.WakeupChan), + wakeups: make(map[string]job.WakeupFunc), jobs: make(map[string]job.Job), } } @@ -137,6 +136,17 @@ func (s *jobs) status() map[string]interface{} { return ret } +func (s *jobs) wakeup(job string) error { + s.m.RLock() + defer s.m.RUnlock() + + wu, ok := s.wakeups[job] + if !ok { + return errors.Errorf("Job %s does not exist", job) + } + return wu() +} + const ( jobNamePrometheus = "_prometheus" jobNameControl = "_control" diff --git a/daemon/filters/fsmapfilter.go b/daemon/filters/fsmapfilter.go new file mode 100644 index 0000000..7fc6bbd --- /dev/null +++ b/daemon/filters/fsmapfilter.go @@ -0,0 +1,273 @@ +package filters + +import ( + "fmt" + "github.com/pkg/errors" + "github.com/zrepl/zrepl/endpoint" + "github.com/zrepl/zrepl/zfs" + "strings" +) + +type DatasetMapFilter struct { + entries []datasetMapFilterEntry + + // if set, only valid filter entries can be added using Add() + // and Map() will always return an error + filterMode bool +} + +type datasetMapFilterEntry struct { + path *zfs.DatasetPath + // the mapping. since this datastructure acts as both mapping and filter + // we have to convert it to the desired rep dynamically + mapping string + subtreeMatch bool +} + +func NewDatasetMapFilter(capacity int, filterMode bool) *DatasetMapFilter { + return &DatasetMapFilter{ + entries: make([]datasetMapFilterEntry, 0, capacity), + filterMode: filterMode, + } +} + +func (m *DatasetMapFilter) Add(pathPattern, mapping string) (err error) { + + if m.filterMode { + if _, err = m.parseDatasetFilterResult(mapping); err != nil { + return + } + } + + // assert path glob adheres to spec + const SUBTREE_PATTERN string = "<" + patternCount := strings.Count(pathPattern, SUBTREE_PATTERN) + switch { + case patternCount > 1: + case patternCount == 1 && !strings.HasSuffix(pathPattern, SUBTREE_PATTERN): + err = fmt.Errorf("pattern invalid: only one '<' at end of string allowed") + return + } + + pathStr := strings.TrimSuffix(pathPattern, SUBTREE_PATTERN) + path, err := zfs.NewDatasetPath(pathStr) + if err != nil { + return fmt.Errorf("pattern is not a dataset path: %s", err) + } + + entry := datasetMapFilterEntry{ + path: path, + mapping: mapping, + subtreeMatch: patternCount > 0, + } + m.entries = append(m.entries, entry) + return + +} + +// find the most specific prefix mapping we have +// +// longer prefix wins over shorter prefix, direct wins over glob +func (m DatasetMapFilter) mostSpecificPrefixMapping(path *zfs.DatasetPath) (idx int, found bool) { + lcp, lcp_entry_idx := -1, -1 + direct_idx := -1 + for e := range m.entries { + entry := m.entries[e] + ep := m.entries[e].path + lep := ep.Length() + + switch { + case !entry.subtreeMatch && ep.Equal(path): + direct_idx = e + continue + case entry.subtreeMatch && path.HasPrefix(ep) && lep > lcp: + lcp = lep + lcp_entry_idx = e + default: + continue + } + } + + if lcp_entry_idx >= 0 || direct_idx >= 0 { + found = true + switch { + case direct_idx >= 0: + idx = direct_idx + case lcp_entry_idx >= 0: + idx = lcp_entry_idx + } + } + return +} + +// Returns target == nil if there is no mapping +func (m DatasetMapFilter) Map(source *zfs.DatasetPath) (target *zfs.DatasetPath, err error) { + + if m.filterMode { + err = fmt.Errorf("using a filter for mapping simply does not work") + return + } + + mi, hasMapping := m.mostSpecificPrefixMapping(source) + if !hasMapping { + return nil, nil + } + me := m.entries[mi] + + if me.mapping == "" { + // Special case treatment: 'foo/bar<' => '' + if !me.subtreeMatch { + return nil, fmt.Errorf("mapping to '' must be a subtree match") + } + // ok... + } else { + if strings.HasPrefix("!", me.mapping) { + // reject mapping + return nil, nil + } + } + + target, err = zfs.NewDatasetPath(me.mapping) + if err != nil { + err = fmt.Errorf("mapping target is not a dataset path: %s", err) + return + } + if me.subtreeMatch { + // strip common prefix ('<' wildcards are no special case here) + extendComps := source.Copy() + extendComps.TrimPrefix(me.path) + target.Extend(extendComps) + } + return +} + +func (m DatasetMapFilter) Filter(p *zfs.DatasetPath) (pass bool, err error) { + + if !m.filterMode { + err = fmt.Errorf("using a mapping as a filter does not work") + return + } + + mi, hasMapping := m.mostSpecificPrefixMapping(p) + if !hasMapping { + pass = false + return + } + me := m.entries[mi] + pass, err = m.parseDatasetFilterResult(me.mapping) + return +} + +// Construct a new filter-only DatasetMapFilter from a mapping +// The new filter allows excactly those paths that were not forbidden by the mapping. +func (m DatasetMapFilter) InvertedFilter() (inv *DatasetMapFilter, err error) { + + if m.filterMode { + err = errors.Errorf("can only invert mappings") + return + } + + inv = &DatasetMapFilter{ + make([]datasetMapFilterEntry, len(m.entries)), + true, + } + + for i, e := range m.entries { + inv.entries[i].path, err = zfs.NewDatasetPath(e.mapping) + if err != nil { + err = errors.Wrapf(err, "mapping cannot be inverted: '%s' is not a dataset path: %s", e.mapping) + return + } + inv.entries[i].mapping = MapFilterResultOk + inv.entries[i].subtreeMatch = e.subtreeMatch + } + + return inv, nil +} + +// FIXME investigate whether we can support more... +func (m DatasetMapFilter) Invert() (endpoint.FSMap, error) { + + if m.filterMode { + return nil, errors.Errorf("can only invert mappings") + } + + if len(m.entries) != 1 { + return nil, errors.Errorf("inversion of complicated mappings is not implemented") // FIXME + } + + e := m.entries[0] + + inv := &DatasetMapFilter{ + make([]datasetMapFilterEntry, len(m.entries)), + false, + } + mp, err := zfs.NewDatasetPath(e.mapping) + if err != nil { + return nil, err + } + + inv.entries[0] = datasetMapFilterEntry{ + path: mp, + mapping: e.path.ToString(), + subtreeMatch: e.subtreeMatch, + } + + return inv, nil +} + +// Creates a new DatasetMapFilter in filter mode from a mapping +// All accepting mapping results are mapped to accepting filter results +// All rejecting mapping results are mapped to rejecting filter results +func (m DatasetMapFilter) AsFilter() endpoint.FSFilter { + + f := &DatasetMapFilter{ + make([]datasetMapFilterEntry, len(m.entries)), + true, + } + + for i, e := range m.entries { + var newe datasetMapFilterEntry = e + if strings.HasPrefix(newe.mapping, "!") { + newe.mapping = MapFilterResultOmit + } else { + newe.mapping = MapFilterResultOk + } + f.entries[i] = newe + } + + return f +} + +const ( + MapFilterResultOk string = "ok" + MapFilterResultOmit string = "!" +) + +// Parse a dataset filter result +func (m DatasetMapFilter) parseDatasetFilterResult(result string) (pass bool, err error) { + l := strings.ToLower(result) + if l == MapFilterResultOk { + return true, nil + } + if l == MapFilterResultOmit { + return false, nil + } + return false, fmt.Errorf("'%s' is not a valid filter result", result) +} + +func DatasetMapFilterFromConfig(in map[string]bool) (f *DatasetMapFilter, err error) { + + f = NewDatasetMapFilter(len(in), true) + for pathPattern, accept := range in { + mapping := MapFilterResultOmit + if accept { + mapping = MapFilterResultOk + } + if err = f.Add(pathPattern, mapping); err != nil { + err = fmt.Errorf("invalid mapping entry ['%s':'%s']: %s", pathPattern, mapping, err) + return + } + } + return +} diff --git a/daemon/filters/fsvfilter.go b/daemon/filters/fsvfilter.go new file mode 100644 index 0000000..0ec6225 --- /dev/null +++ b/daemon/filters/fsvfilter.go @@ -0,0 +1,15 @@ +package filters + +import "github.com/zrepl/zrepl/zfs" + +type AnyFSVFilter struct{} + +func NewAnyFSVFilter() AnyFSVFilter { + return AnyFSVFilter{} +} + +var _ zfs.FilesystemVersionFilter = AnyFSVFilter{} + +func (AnyFSVFilter) Filter(t zfs.VersionType, name string) (accept bool, err error) { + return true, nil +} diff --git a/daemon/job/build_jobs.go b/daemon/job/build_jobs.go index b36fc90..f062918 100644 --- a/daemon/job/build_jobs.go +++ b/daemon/job/build_jobs.go @@ -1,8 +1,8 @@ package job import ( - "github.com/zrepl/zrepl/cmd/config" "fmt" + "github.com/zrepl/zrepl/config" ) func JobsFromConfig(c config.Config) ([]Job, error) { @@ -20,8 +20,12 @@ func JobsFromConfig(c config.Config) ([]Job, error) { func buildJob(c config.Global, in config.JobEnum) (j Job, err error) { switch v := in.Ret.(type) { + case *config.SinkJob: + return SinkFromConfig(c, v) + case *config.PushJob: + return PushFromConfig(c, v) default: - panic(fmt.Sprintf("implementation error: unknown job type %s", v)) + panic(fmt.Sprintf("implementation error: unknown job type %T", v)) } -} \ No newline at end of file +} diff --git a/daemon/job/job.go b/daemon/job/job.go index 56e25af..4ea199a 100644 --- a/daemon/job/job.go +++ b/daemon/job/job.go @@ -2,7 +2,10 @@ package job import ( "context" + "errors" + "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/logger" + "time" ) type Logger = logger.Logger @@ -25,9 +28,21 @@ func WithLogger(ctx context.Context, l Logger) context.Context { return context.WithValue(ctx, contextKeyLog, l) } -func WithWakeup(ctx context.Context) (context.Context, WakeupChan) { - wc := make(chan struct{}, 1) - return context.WithValue(ctx, contextKeyWakeup, wc), wc +type WakeupFunc func() error + +var AlreadyWokenUp = errors.New("already woken up") + +func WithWakeup(ctx context.Context) (context.Context, WakeupFunc) { + wc := make(chan struct{}) + wuf := func() error { + select { + case wc <- struct{}{}: + return nil + default: + return AlreadyWokenUp + } + } + return context.WithValue(ctx, contextKeyWakeup, wc), wuf } type Job interface { @@ -36,12 +51,23 @@ type Job interface { Status() interface{} } -type WakeupChan <-chan struct{} - -func WaitWakeup(ctx context.Context) WakeupChan { - wc, ok := ctx.Value(contextKeyWakeup).(WakeupChan) +func WaitWakeup(ctx context.Context) <-chan struct{} { + wc, ok := ctx.Value(contextKeyWakeup).(chan struct{}) if !ok { wc = make(chan struct{}) } return wc } + +var STREAMRPC_CONFIG = &streamrpc.ConnConfig{ // FIXME oversight and configurability + RxHeaderMaxLen: 4096, + RxStructuredMaxLen: 4096 * 4096, + RxStreamMaxChunkSize: 4096 * 4096, + TxChunkSize: 4096 * 4096, + RxTimeout: streamrpc.Timeout{ + Progress: 10 * time.Second, + }, + TxTimeout: streamrpc.Timeout{ + Progress: 10 * time.Second, + }, +} diff --git a/daemon/job/push.go b/daemon/job/push.go new file mode 100644 index 0000000..f234981 --- /dev/null +++ b/daemon/job/push.go @@ -0,0 +1,86 @@ +package job + +import ( + "context" + "github.com/pkg/errors" + "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon/connecter" + "github.com/zrepl/zrepl/daemon/filters" + "github.com/zrepl/zrepl/daemon/logging" + "github.com/zrepl/zrepl/endpoint" + "github.com/zrepl/zrepl/replication" + "sync" +) + +type Push struct { + name string + connecter streamrpc.Connecter + fsfilter endpoint.FSFilter + + mtx sync.Mutex + replication *replication.Replication +} + +func PushFromConfig(g config.Global, in *config.PushJob) (j *Push, err error) { + + j = &Push{} + j.name = in.Name + + j.connecter, err = connecter.FromConfig(g, in.Replication.Connect) + + if j.fsfilter, err = filters.DatasetMapFilterFromConfig(in.Replication.Filesystems); err != nil { + return nil, errors.Wrap(err, "cannnot build filesystem filter") + } + + return j, nil +} + +func (j *Push) Name() string { return j.name } + +func (j *Push) Status() interface{} { + return nil // FIXME +} + +func (j *Push) Run(ctx context.Context) { + log := GetLogger(ctx) + + defer log.Info("job exiting") + + log.Debug("wait for wakeups") + + invocationCount := 0 +outer: + for { + select { + case <-ctx.Done(): + log.WithError(ctx.Err()).Info("context") + break outer + case <-WaitWakeup(ctx): + invocationCount++ + invLog := log.WithField("invocation", invocationCount) + j.do(WithLogger(ctx, invLog)) + } + } +} + +func (j *Push) do(ctx context.Context) { + + log := GetLogger(ctx) + + client, err := streamrpc.NewClient(j.connecter, &streamrpc.ClientConfig{STREAMRPC_CONFIG}) + if err != nil { + log.WithError(err).Error("cannot create streamrpc client") + } + defer client.Close() + + sender := endpoint.NewSender(j.fsfilter, filters.NewAnyFSVFilter()) + receiver := endpoint.NewRemote(client) + + j.mtx.Lock() + rep := replication.NewReplication() + j.mtx.Unlock() + + ctx = logging.WithSubsystemLoggers(ctx, log) + rep.Drive(ctx, sender, receiver) +} diff --git a/daemon/job/sink.go b/daemon/job/sink.go new file mode 100644 index 0000000..618ced7 --- /dev/null +++ b/daemon/job/sink.go @@ -0,0 +1,115 @@ +package job + +import ( + "context" + "github.com/pkg/errors" + "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon/filters" + "github.com/zrepl/zrepl/daemon/logging" + "github.com/zrepl/zrepl/daemon/serve" + "github.com/zrepl/zrepl/endpoint" + "net" +) + +type Sink struct { + name string + l serve.ListenerFactory + fsmap endpoint.FSMap + fsmapInv endpoint.FSFilter +} + +func SinkFromConfig(g config.Global, in *config.SinkJob) (s *Sink, err error) { + + // FIXME multi client support + + s = &Sink{name: in.Name} + if s.l, err = serve.FromConfig(g, in.Replication.Serve); err != nil { + return nil, errors.Wrap(err, "cannot build server") + } + + fsmap := filters.NewDatasetMapFilter(1, false) // FIXME multi-client support + if err := fsmap.Add("<", in.Replication.RootDataset); err != nil { + return nil, errors.Wrap(err, "unexpected error: cannot build filesystem mapping") + } + s.fsmap = fsmap + + return s, nil +} + +func (j *Sink) Name() string { return j.name } + +func (*Sink) Status() interface{} { + // FIXME + return nil +} + +func (j *Sink) Run(ctx context.Context) { + + log := GetLogger(ctx) + defer log.Info("job exiting") + + l, err := j.l.Listen() + if err != nil { + log.WithError(err).Error("cannot listen") + return + } + + log.WithField("addr", l.Addr()).Debug("accepting connections") + + var connId int + +outer: + for { + + select { + case res := <-accept(l): + if res.err != nil { + log.WithError(err).Info("accept error") + break outer + } + connId++ + connLog := log. + WithField("connID", connId) + j.handleConnection(WithLogger(ctx, connLog), res.conn) + + case <-ctx.Done(): + break outer + } + + } + +} + +func (j *Sink) handleConnection(ctx context.Context, conn net.Conn) { + log := GetLogger(ctx) + log.WithField("addr", conn.RemoteAddr()).Info("handling connection") + defer log.Info("finished handling connection") + + logging.WithSubsystemLoggers(ctx, log) + + local, err := endpoint.NewReceiver(j.fsmap, filters.NewAnyFSVFilter()) + if err != nil { + log.WithError(err).Error("unexpected error: cannot convert mapping to filter") + return + } + + handler := endpoint.NewHandler(local) + if err := streamrpc.ServeConn(ctx, conn, STREAMRPC_CONFIG, handler.Handle); err != nil { + log.WithError(err).Error("error serving client") + } +} + +type acceptResult struct { + conn net.Conn + err error +} + +func accept(listener net.Listener) <-chan acceptResult { + c := make(chan acceptResult, 1) + go func() { + conn, err := listener.Accept() + c <- acceptResult{conn, err} + }() + return c +} diff --git a/daemon/logging/adaptors.go b/daemon/logging/adaptors.go new file mode 100644 index 0000000..7f0b21b --- /dev/null +++ b/daemon/logging/adaptors.go @@ -0,0 +1,32 @@ +package logging + +import ( + "fmt" + "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/logger" + "strings" +) + +type streamrpcLogAdaptor = twoClassLogAdaptor + +type twoClassLogAdaptor struct { + logger.Logger +} + +var _ streamrpc.Logger = twoClassLogAdaptor{} + +func (a twoClassLogAdaptor) Errorf(fmtStr string, args ...interface{}) { + const errorSuffix = ": %s" + if len(args) == 1 { + if err, ok := args[0].(error); ok && strings.HasSuffix(fmtStr, errorSuffix) { + msg := strings.TrimSuffix(fmtStr, errorSuffix) + a.WithError(err).Error(msg) + return + } + } + a.Logger.Error(fmt.Sprintf(fmtStr, args...)) +} + +func (a twoClassLogAdaptor) Infof(fmtStr string, args ...interface{}) { + a.Logger.Info(fmt.Sprintf(fmtStr, args...)) +} diff --git a/daemon/logging/build_logging.go b/daemon/logging/build_logging.go index 6e2b4d6..f673033 100644 --- a/daemon/logging/build_logging.go +++ b/daemon/logging/build_logging.go @@ -1,14 +1,18 @@ package logging import ( - "github.com/zrepl/zrepl/cmd/config" - "os" - "github.com/mattn/go-isatty" + "context" "crypto/tls" - "github.com/pkg/errors" "crypto/x509" - "github.com/zrepl/zrepl/cmd/tlsconf" + "github.com/mattn/go-isatty" + "github.com/pkg/errors" + "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/endpoint" "github.com/zrepl/zrepl/logger" + "github.com/zrepl/zrepl/replication" + "github.com/zrepl/zrepl/tlsconf" + "os" ) func OutletsFromConfig(in []config.LoggingOutletEnum) (*logger.Outlets, error) { @@ -53,6 +57,18 @@ func OutletsFromConfig(in []config.LoggingOutletEnum) (*logger.Outlets, error) { } +const ( + SubsysReplication = "repl" + SubsysStreamrpc = "rpc" + SubsyEndpoint = "endpoint" +) + +func WithSubsystemLoggers(ctx context.Context, log logger.Logger) context.Context { + ctx = replication.WithLogger(ctx, log.WithField(SubsysField, "repl")) + ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{log.WithField(SubsysField, "rpc")}) + ctx = endpoint.WithLogger(ctx, log.WithField(SubsysField, "endpoint")) + return ctx +} func parseLogFormat(i interface{}) (f EntryFormatter, err error) { var is string @@ -97,19 +113,19 @@ func parseOutlet(in config.LoggingOutletEnum) (o logger.Outlet, level logger.Lev var f EntryFormatter switch v := in.Ret.(type) { - case config.StdoutLoggingOutlet: + case *config.StdoutLoggingOutlet: level, f, err = parseCommon(v.LoggingOutletCommon) if err != nil { break } o, err = parseStdoutOutlet(v, f) - case config.TCPLoggingOutlet: + case *config.TCPLoggingOutlet: level, f, err = parseCommon(v.LoggingOutletCommon) if err != nil { break } o, err = parseTCPOutlet(v, f) - case config.SyslogLoggingOutlet: + case *config.SyslogLoggingOutlet: level, f, err = parseCommon(v.LoggingOutletCommon) if err != nil { break @@ -121,7 +137,7 @@ func parseOutlet(in config.LoggingOutletEnum) (o logger.Outlet, level logger.Lev return o, level, err } -func parseStdoutOutlet(in config.StdoutLoggingOutlet, formatter EntryFormatter) (WriterOutlet, error) { +func parseStdoutOutlet(in *config.StdoutLoggingOutlet, formatter EntryFormatter) (WriterOutlet, error) { flags := MetadataAll writer := os.Stdout if !isatty.IsTerminal(writer.Fd()) && !in.Time { @@ -135,7 +151,7 @@ func parseStdoutOutlet(in config.StdoutLoggingOutlet, formatter EntryFormatter) }, nil } -func parseTCPOutlet(in config.TCPLoggingOutlet, formatter EntryFormatter) (out *TCPOutlet, err error) { +func parseTCPOutlet(in *config.TCPLoggingOutlet, formatter EntryFormatter) (out *TCPOutlet, err error) { var tlsConfig *tls.Config if in.TLS != nil { tlsConfig, err = func(m *config.TCPLoggingOutletTLS, host string) (*tls.Config, error) { @@ -171,11 +187,10 @@ func parseTCPOutlet(in config.TCPLoggingOutlet, formatter EntryFormatter) (out * } -func parseSyslogOutlet(in config.SyslogLoggingOutlet, formatter EntryFormatter) (out *SyslogOutlet, err error) { +func parseSyslogOutlet(in *config.SyslogLoggingOutlet, formatter EntryFormatter) (out *SyslogOutlet, err error) { out = &SyslogOutlet{} out.Formatter = formatter out.Formatter.SetMetadataFlags(MetadataNone) out.RetryInterval = in.RetryInterval return out, nil } - diff --git a/daemon/logging/logging_formatters.go b/daemon/logging/logging_formatters.go index b968531..de79ee0 100644 --- a/daemon/logging/logging_formatters.go +++ b/daemon/logging/logging_formatters.go @@ -17,12 +17,10 @@ const ( ) const ( - logJobField string = "job" - logTaskField string = "task" - logSubsysField string = "subsystem" + JobField string = "job" + SubsysField string = "subsystem" ) - type MetadataFlags int64 const ( @@ -33,7 +31,6 @@ const ( MetadataAll MetadataFlags = ^0 ) - type NoFormatter struct{} func (f NoFormatter) SetMetadataFlags(flags MetadataFlags) {} @@ -80,7 +77,7 @@ func (f *HumanFormatter) Format(e *logger.Entry) (out []byte, err error) { fmt.Fprintf(&line, "[%s]", e.Level.Short()) } - prefixFields := []string{logJobField, logTaskField, logSubsysField} + prefixFields := []string{JobField, SubsysField} prefixed := make(map[string]bool, len(prefixFields)+2) for _, field := range prefixFields { val, ok := e.Fields[field].(string) @@ -168,7 +165,7 @@ func (f *LogfmtFormatter) Format(e *logger.Entry) ([]byte, error) { // at least try and put job and task in front prefixed := make(map[string]bool, 2) - prefix := []string{logJobField, logTaskField, logSubsysField} + prefix := []string{JobField, SubsysField} for _, pf := range prefix { v, ok := e.Fields[pf] if !ok { diff --git a/daemon/logging/logging_outlets.go b/daemon/logging/logging_outlets.go index 4a0fd7a..5a00d42 100644 --- a/daemon/logging/logging_outlets.go +++ b/daemon/logging/logging_outlets.go @@ -12,7 +12,6 @@ import ( "time" ) - type EntryFormatter interface { SetMetadataFlags(flags MetadataFlags) Format(e *logger.Entry) ([]byte, error) diff --git a/daemon/main.go b/daemon/main.go index 488b020..2dd2590 100644 --- a/daemon/main.go +++ b/daemon/main.go @@ -5,4 +5,3 @@ import ( ) type Logger = logger.Logger - diff --git a/cmd/helpers/helpers.go b/daemon/nethelpers/helpers.go similarity index 98% rename from cmd/helpers/helpers.go rename to daemon/nethelpers/helpers.go index bcf2cf5..994b9d2 100644 --- a/cmd/helpers/helpers.go +++ b/daemon/nethelpers/helpers.go @@ -1,4 +1,4 @@ -package helpers +package nethelpers import ( "github.com/pkg/errors" diff --git a/daemon/prometheus.go b/daemon/prometheus.go index 1cef3d0..3836624 100644 --- a/daemon/prometheus.go +++ b/daemon/prometheus.go @@ -4,7 +4,7 @@ import ( "context" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" - "github.com/zrepl/zrepl/cmd/daemon/job" + "github.com/zrepl/zrepl/daemon/job" "github.com/zrepl/zrepl/zfs" "net" "net/http" diff --git a/daemon/serve/serve.go b/daemon/serve/serve.go new file mode 100644 index 0000000..8ab7945 --- /dev/null +++ b/daemon/serve/serve.go @@ -0,0 +1,26 @@ +package serve + +import ( + "github.com/pkg/errors" + "github.com/zrepl/zrepl/config" + "net" +) + +type ListenerFactory interface { + Listen() (net.Listener, error) +} + +func FromConfig(g config.Global, in config.ServeEnum) (ListenerFactory, error) { + + switch v := in.Ret.(type) { + case *config.TCPServe: + return TCPListenerFactoryFromConfig(g, v) + case *config.TLSServe: + return TLSListenerFactoryFromConfig(g, v) + case *config.StdinserverServer: + return StdinserverListenerFactoryFromConfig(g, v) + default: + return nil, errors.Errorf("internal error: unknown serve type %T", v) + } + +} diff --git a/daemon/serve/serve_stdinserver.go b/daemon/serve/serve_stdinserver.go new file mode 100644 index 0000000..4048d65 --- /dev/null +++ b/daemon/serve/serve_stdinserver.go @@ -0,0 +1,79 @@ +package serve + +import ( + "github.com/problame/go-netssh" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon/nethelpers" + "io" + "net" + "path" + "time" +) + +type StdinserverListenerFactory struct { + ClientIdentity string + sockpath string +} + +func StdinserverListenerFactoryFromConfig(g config.Global, in *config.StdinserverServer) (f *StdinserverListenerFactory, err error) { + + f = &StdinserverListenerFactory{ + ClientIdentity: in.ClientIdentity, + } + + f.sockpath = path.Join(g.Serve.StdinServer.SockDir, f.ClientIdentity) + + return +} + +func (f *StdinserverListenerFactory) Listen() (net.Listener, error) { + + if err := nethelpers.PreparePrivateSockpath(f.sockpath); err != nil { + return nil, err + } + + l, err := netssh.Listen(f.sockpath) + if err != nil { + return nil, err + } + return StdinserverListener{l}, nil +} + +type StdinserverListener struct { + l *netssh.Listener +} + +func (l StdinserverListener) Addr() net.Addr { + return netsshAddr{} +} + +func (l StdinserverListener) Accept() (net.Conn, error) { + c, err := l.l.Accept() + if err != nil { + return nil, err + } + return netsshConnToNetConnAdatper{c}, nil +} + +func (l StdinserverListener) Close() (err error) { + return l.l.Close() +} + +type netsshAddr struct{} + +func (netsshAddr) Network() string { return "netssh" } +func (netsshAddr) String() string { return "???" } + +type netsshConnToNetConnAdatper struct { + io.ReadWriteCloser // works for both netssh.SSHConn and netssh.ServeConn +} + +func (netsshConnToNetConnAdatper) LocalAddr() net.Addr { return netsshAddr{} } + +func (netsshConnToNetConnAdatper) RemoteAddr() net.Addr { return netsshAddr{} } + +func (netsshConnToNetConnAdatper) SetDeadline(t time.Time) error { return nil } + +func (netsshConnToNetConnAdatper) SetReadDeadline(t time.Time) error { return nil } + +func (netsshConnToNetConnAdatper) SetWriteDeadline(t time.Time) error { return nil } diff --git a/daemon/serve/serve_tcp.go b/daemon/serve/serve_tcp.go new file mode 100644 index 0000000..a5bad28 --- /dev/null +++ b/daemon/serve/serve_tcp.go @@ -0,0 +1,21 @@ +package serve + +import ( + "github.com/zrepl/zrepl/config" + "net" +) + +type TCPListenerFactory struct { + Address string +} + +func TCPListenerFactoryFromConfig(c config.Global, in *config.TCPServe) (*TCPListenerFactory, error) { + lf := &TCPListenerFactory{ + Address: in.Listen, + } + return lf, nil +} + +func (f *TCPListenerFactory) Listen() (net.Listener, error) { + return net.Listen("tcp", f.Address) +} diff --git a/daemon/serve/serve_tls.go b/daemon/serve/serve_tls.go new file mode 100644 index 0000000..f24f5ad --- /dev/null +++ b/daemon/serve/serve_tls.go @@ -0,0 +1,52 @@ +package serve + +import ( + "crypto/tls" + "crypto/x509" + "github.com/pkg/errors" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/tlsconf" + "net" + "time" +) + +type TLSListenerFactory struct { + address string + clientCA *x509.CertPool + serverCert tls.Certificate + clientCommonName string + handshakeTimeout time.Duration +} + +func TLSListenerFactoryFromConfig(c config.Global, in *config.TLSServe) (lf *TLSListenerFactory, err error) { + lf = &TLSListenerFactory{ + address: in.Listen, + } + + if in.Ca == "" || in.Cert == "" || in.Key == "" || in.ClientCN == "" { + return nil, errors.New("fields 'ca', 'cert', 'key' and 'client_cn' must be specified") + } + + lf.clientCommonName = in.ClientCN + + lf.clientCA, err = tlsconf.ParseCAFile(in.Ca) + if err != nil { + return nil, errors.Wrap(err, "cannot parse ca file") + } + + lf.serverCert, err = tls.LoadX509KeyPair(in.Cert, in.Key) + if err != nil { + return nil, errors.Wrap(err, "cannot parse cer/key pair") + } + + return lf, nil +} + +func (f *TLSListenerFactory) Listen() (net.Listener, error) { + l, err := net.Listen("tcp", f.address) + if err != nil { + return nil, err + } + tl := tlsconf.NewClientAuthListener(l, f.clientCA, f.serverCert, f.clientCommonName, f.handshakeTimeout) + return tl, nil +} diff --git a/cmd/endpoint/context.go b/endpoint/context.go similarity index 100% rename from cmd/endpoint/context.go rename to endpoint/context.go diff --git a/cmd/endpoint/endpoint.go b/endpoint/endpoint.go similarity index 100% rename from cmd/endpoint/endpoint.go rename to endpoint/endpoint.go diff --git a/main.go b/main.go index 452b02d..251ffe2 100644 --- a/main.go +++ b/main.go @@ -2,11 +2,11 @@ package main import ( + "github.com/spf13/cobra" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon" "log" "os" - "github.com/spf13/cobra" - "github.com/zrepl/zrepl/daemon" - "github.com/zrepl/zrepl/cmd/config" ) var rootCmd = &cobra.Command{ @@ -20,9 +20,8 @@ var rootCmd = &cobra.Command{ - ACLs instead of blank SSH access`, } - var daemonCmd = &cobra.Command{ - Use: "daemon", + Use: "daemon", Short: "daemon", RunE: func(cmd *cobra.Command, args []string) error { conf, err := config.ParseConfig(rootArgs.configFile) @@ -33,6 +32,18 @@ var daemonCmd = &cobra.Command{ }, } +var wakeupCmd = &cobra.Command{ + Use: "wakeup", + Short: "wake up jobs", + RunE: func(cmd *cobra.Command, args []string) error { + conf, err := config.ParseConfig(rootArgs.configFile) + if err != nil { + return err + } + return RunWakeup(conf, args) + }, +} + var rootArgs struct { configFile string } @@ -41,11 +52,11 @@ func init() { //cobra.OnInitialize(initConfig) rootCmd.PersistentFlags().StringVar(&rootArgs.configFile, "config", "", "config file path") rootCmd.AddCommand(daemonCmd) + rootCmd.AddCommand(wakeupCmd) } func main() { - if err := rootCmd.Execute(); err != nil { log.Printf("error executing root command: %s", err) os.Exit(1) diff --git a/cmd/tlsconf/tlsconf.go b/tlsconf/tlsconf.go similarity index 100% rename from cmd/tlsconf/tlsconf.go rename to tlsconf/tlsconf.go diff --git a/wakeup.go b/wakeup.go new file mode 100644 index 0000000..df389de --- /dev/null +++ b/wakeup.go @@ -0,0 +1,68 @@ +package main + +import ( + "bytes" + "context" + "encoding/json" + "github.com/pkg/errors" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon" + "io" + "net" + "net/http" +) + +func RunWakeup(config config.Config, args []string) error { + if len(args) != 1 { + return errors.Errorf("Expected 1 argument: job") + } + + httpc, err := controlHttpClient(config.Global.Control.SockPath) + if err != nil { + return err + } + + err = jsonRequestResponse(httpc, daemon.ControlJobEndpointWakeup, + struct { + Name string + }{ + Name: args[0], + }, + struct{}{}, + ) + return err +} + +func controlHttpClient(sockpath string) (client http.Client, err error) { + return http.Client{ + Transport: &http.Transport{ + DialContext: func(_ context.Context, _, _ string) (net.Conn, error) { + return net.Dial("unix", sockpath) + }, + }, + }, nil +} + +func jsonRequestResponse(c http.Client, endpoint string, req interface{}, res interface{}) error { + var buf bytes.Buffer + encodeErr := json.NewEncoder(&buf).Encode(req) + if encodeErr != nil { + return encodeErr + } + + resp, err := c.Post("http://unix"+endpoint, "application/json", &buf) + if err != nil { + return err + } else if resp.StatusCode != http.StatusOK { + var msg bytes.Buffer + io.CopyN(&msg, resp.Body, 4096) + return errors.Errorf("%s", msg.String()) + } + + decodeError := json.NewDecoder(resp.Body).Decode(&res) + if decodeError != nil { + return decodeError + } + + return nil +} From fb0a8d8b40b165754a1c8b15f2b96a6e902a018d Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 29 Aug 2018 14:54:29 +0200 Subject: [PATCH 062/167] gofmt cmd/ --- cmd/config_connect.go | 2 +- cmd/config_logging.go | 2 +- cmd/config_parse.go | 2 +- cmd/config_serve_stdinserver.go | 2 +- cmd/config_serve_tcp.go | 2 +- cmd/config_serve_tls.go | 2 +- cmd/daemon.go | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/cmd/config_connect.go b/cmd/config_connect.go index 81e9803..ea71ac3 100644 --- a/cmd/config_connect.go +++ b/cmd/config_connect.go @@ -9,8 +9,8 @@ import ( "github.com/pkg/errors" "github.com/problame/go-netssh" "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/cmd/tlsconf" + "github.com/zrepl/zrepl/config" "time" ) diff --git a/cmd/config_logging.go b/cmd/config_logging.go index 2d3fe26..88e8cb8 100644 --- a/cmd/config_logging.go +++ b/cmd/config_logging.go @@ -5,8 +5,8 @@ import ( "crypto/x509" "github.com/mattn/go-isatty" "github.com/pkg/errors" - "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/cmd/tlsconf" + "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/logger" "os" ) diff --git a/cmd/config_parse.go b/cmd/config_parse.go index f30a9a6..a6cd27d 100644 --- a/cmd/config_parse.go +++ b/cmd/config_parse.go @@ -8,8 +8,8 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/cmd/pruning/retentiongrid" + "github.com/zrepl/zrepl/config" "os" ) diff --git a/cmd/config_serve_stdinserver.go b/cmd/config_serve_stdinserver.go index 63ef839..1fef93c 100644 --- a/cmd/config_serve_stdinserver.go +++ b/cmd/config_serve_stdinserver.go @@ -2,8 +2,8 @@ package cmd import ( "github.com/problame/go-netssh" - "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/cmd/helpers" + "github.com/zrepl/zrepl/config" "net" "path" ) diff --git a/cmd/config_serve_tcp.go b/cmd/config_serve_tcp.go index 351578b..7afafd6 100644 --- a/cmd/config_serve_tcp.go +++ b/cmd/config_serve_tcp.go @@ -2,8 +2,8 @@ package cmd import ( "github.com/zrepl/zrepl/config" - "time" "net" + "time" ) type TCPListenerFactory struct { diff --git a/cmd/config_serve_tls.go b/cmd/config_serve_tls.go index 565bc07..83dcf16 100644 --- a/cmd/config_serve_tls.go +++ b/cmd/config_serve_tls.go @@ -8,8 +8,8 @@ import ( "github.com/mitchellh/mapstructure" "github.com/pkg/errors" - "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/cmd/tlsconf" + "github.com/zrepl/zrepl/config" ) type TCPListenerFactory struct { diff --git a/cmd/daemon.go b/cmd/daemon.go index 621c591..4d4cb1b 100644 --- a/cmd/daemon.go +++ b/cmd/daemon.go @@ -4,9 +4,9 @@ import ( "context" "fmt" "github.com/spf13/cobra" - "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/cmd/daemon" "github.com/zrepl/zrepl/cmd/daemon/job" + "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/logger" "os" "os/signal" From 0de17fd05136e19f9704740015b40e98b93f0913 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 27 Aug 2018 16:11:35 +0200 Subject: [PATCH 063/167] move cmd/pruning to pruning, as it's independent of the command implementation --- {cmd/pruning => pruning}/keep_helpers.go | 0 {cmd/pruning => pruning}/keep_helpers_test.go | 0 {cmd/pruning => pruning}/keep_last_n.go | 0 {cmd/pruning => pruning}/keep_prefix.go | 0 {cmd/pruning => pruning}/pruning.go | 0 {cmd/pruning => pruning}/pruning_test.go | 0 {cmd/pruning => pruning}/retentiongrid/config_prune_grid.go | 0 {cmd/pruning => pruning}/retentiongrid/retentiongrid.go | 0 {cmd/pruning => pruning}/retentiongrid/retentiongrid_test.go | 0 9 files changed, 0 insertions(+), 0 deletions(-) rename {cmd/pruning => pruning}/keep_helpers.go (100%) rename {cmd/pruning => pruning}/keep_helpers_test.go (100%) rename {cmd/pruning => pruning}/keep_last_n.go (100%) rename {cmd/pruning => pruning}/keep_prefix.go (100%) rename {cmd/pruning => pruning}/pruning.go (100%) rename {cmd/pruning => pruning}/pruning_test.go (100%) rename {cmd/pruning => pruning}/retentiongrid/config_prune_grid.go (100%) rename {cmd/pruning => pruning}/retentiongrid/retentiongrid.go (100%) rename {cmd/pruning => pruning}/retentiongrid/retentiongrid_test.go (100%) diff --git a/cmd/pruning/keep_helpers.go b/pruning/keep_helpers.go similarity index 100% rename from cmd/pruning/keep_helpers.go rename to pruning/keep_helpers.go diff --git a/cmd/pruning/keep_helpers_test.go b/pruning/keep_helpers_test.go similarity index 100% rename from cmd/pruning/keep_helpers_test.go rename to pruning/keep_helpers_test.go diff --git a/cmd/pruning/keep_last_n.go b/pruning/keep_last_n.go similarity index 100% rename from cmd/pruning/keep_last_n.go rename to pruning/keep_last_n.go diff --git a/cmd/pruning/keep_prefix.go b/pruning/keep_prefix.go similarity index 100% rename from cmd/pruning/keep_prefix.go rename to pruning/keep_prefix.go diff --git a/cmd/pruning/pruning.go b/pruning/pruning.go similarity index 100% rename from cmd/pruning/pruning.go rename to pruning/pruning.go diff --git a/cmd/pruning/pruning_test.go b/pruning/pruning_test.go similarity index 100% rename from cmd/pruning/pruning_test.go rename to pruning/pruning_test.go diff --git a/cmd/pruning/retentiongrid/config_prune_grid.go b/pruning/retentiongrid/config_prune_grid.go similarity index 100% rename from cmd/pruning/retentiongrid/config_prune_grid.go rename to pruning/retentiongrid/config_prune_grid.go diff --git a/cmd/pruning/retentiongrid/retentiongrid.go b/pruning/retentiongrid/retentiongrid.go similarity index 100% rename from cmd/pruning/retentiongrid/retentiongrid.go rename to pruning/retentiongrid/retentiongrid.go diff --git a/cmd/pruning/retentiongrid/retentiongrid_test.go b/pruning/retentiongrid/retentiongrid_test.go similarity index 100% rename from cmd/pruning/retentiongrid/retentiongrid_test.go rename to pruning/retentiongrid/retentiongrid_test.go From a2aa8e7bd7e00974179460435477b3c54bff7f76 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 29 Aug 2018 19:00:45 +0200 Subject: [PATCH 064/167] finish pruner implementation --- daemon/job/push.go | 15 ++ daemon/logging/build_logging.go | 2 + daemon/pruner/pruner.go | 297 ++++++++++++++++++++++++++++++++ daemon/pruner/pruner_test.go | 192 +++++++++++++++++++++ daemon/pruner/state_string.go | 35 ++++ logger/testlogger.go | 28 +++ pruning/pruning.go | 1 + replication/pdu/pdu_extras.go | 4 + 8 files changed, 574 insertions(+) create mode 100644 daemon/pruner/pruner.go create mode 100644 daemon/pruner/pruner_test.go create mode 100644 daemon/pruner/state_string.go create mode 100644 logger/testlogger.go diff --git a/daemon/job/push.go b/daemon/job/push.go index f234981..a292847 100644 --- a/daemon/job/push.go +++ b/daemon/job/push.go @@ -11,6 +11,8 @@ import ( "github.com/zrepl/zrepl/endpoint" "github.com/zrepl/zrepl/replication" "sync" + "github.com/zrepl/zrepl/daemon/pruner" + "github.com/zrepl/zrepl/pruning" ) type Push struct { @@ -18,6 +20,9 @@ type Push struct { connecter streamrpc.Connecter fsfilter endpoint.FSFilter + keepRulesSender []pruning.KeepRule + keepRulesReceiver []pruning.KeepRule + mtx sync.Mutex replication *replication.Replication } @@ -83,4 +88,14 @@ func (j *Push) do(ctx context.Context) { ctx = logging.WithSubsystemLoggers(ctx, log) rep.Drive(ctx, sender, receiver) + + // Prune sender + senderPruner := pruner.NewPruner(sender, receiver, j.keepRulesSender) + senderPruner.Prune(ctx) + + // Prune receiver + receiverPruner := pruner.NewPruner(receiver, receiver, j.keepRulesReceiver) + receiverPruner.Prune(ctx) + } + diff --git a/daemon/logging/build_logging.go b/daemon/logging/build_logging.go index f673033..c6bea78 100644 --- a/daemon/logging/build_logging.go +++ b/daemon/logging/build_logging.go @@ -13,6 +13,7 @@ import ( "github.com/zrepl/zrepl/replication" "github.com/zrepl/zrepl/tlsconf" "os" + "github.com/zrepl/zrepl/daemon/pruner" ) func OutletsFromConfig(in []config.LoggingOutletEnum) (*logger.Outlets, error) { @@ -67,6 +68,7 @@ func WithSubsystemLoggers(ctx context.Context, log logger.Logger) context.Contex ctx = replication.WithLogger(ctx, log.WithField(SubsysField, "repl")) ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{log.WithField(SubsysField, "rpc")}) ctx = endpoint.WithLogger(ctx, log.WithField(SubsysField, "endpoint")) + ctx = pruner.WithLogger(ctx, log.WithField(SubsysField, "pruning")) return ctx } diff --git a/daemon/pruner/pruner.go b/daemon/pruner/pruner.go new file mode 100644 index 0000000..303ba14 --- /dev/null +++ b/daemon/pruner/pruner.go @@ -0,0 +1,297 @@ +package pruner + +import ( + "context" + "github.com/zrepl/zrepl/pruning" + "github.com/zrepl/zrepl/replication/pdu" + "sync" + "time" + "fmt" + "net" + "github.com/zrepl/zrepl/logger" +) + +// Try to keep it compatible with gitub.com/zrepl/zrepl/replication.Endpoint +type Receiver interface { + HasFilesystemVersion(ctx context.Context, fs string, version *pdu.FilesystemVersion) (bool, error) +} + +type Target interface { + ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) + ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) // fix depS + DestroySnapshots(ctx context.Context, fs string, snaps []*pdu.FilesystemVersion) ([]*pdu.FilesystemVersion, error) +} + +type Logger = logger.Logger + +type contextKey int + +const contextKeyLogger contextKey = 0 + +func WithLogger(ctx context.Context, log Logger) context.Context { + return context.WithValue(ctx, contextKeyLogger, log) +} + +func getLogger(ctx context.Context) Logger { + if l, ok := ctx.Value(contextKeyLogger).(Logger); ok { + return l + } + return logger.NewNullLogger() +} + +type args struct { + ctx context.Context + target Target + receiver Receiver + rules []pruning.KeepRule + retryWait time.Duration +} + +type Pruner struct { + + args args + + mtx sync.RWMutex + + state State + + // State ErrWait|ErrPerm + sleepUntil time.Time + err error + + // State Exec + prunePending []fs + pruneCompleted []fs + +} + +func NewPruner(retryWait time.Duration, target Target, receiver Receiver, rules []pruning.KeepRule) *Pruner { + p := &Pruner{ + args: args{nil, target, receiver, rules, retryWait}, // ctx is filled in Prune() + state: Plan, + } + return p +} + +//go:generate stringer -type=State +type State int + +const ( + Plan State = 1 << iota + PlanWait + Exec + ExecWait + ErrPerm + Done +) + + +func (s State) statefunc() state { + var statemap = map[State]state{ + Plan: statePlan, + PlanWait: statePlanWait, + Exec: stateExec, + ExecWait: stateExecWait, + ErrPerm: nil, + Done: nil, + } + return statemap[s] +} + +type updater func(func(*Pruner)) State +type state func(args *args, u updater) state + +func (p *Pruner) Prune(ctx context.Context) { + p.args.ctx = ctx + p.prune(p.args) +} + +func (p *Pruner) prune(args args) { + s := p.state.statefunc() + for s != nil { + pre := p.state + s = s(&args, func(f func(*Pruner)) State { + p.mtx.Lock() + defer p.mtx.Unlock() + f(p) + return p.state + }) + post := p.state + getLogger(args.ctx). + WithField("transition", fmt.Sprintf("%s=>%s", pre, post)). + Debug("state transition") + } +} + +func (p *Pruner) Report() interface{} { + return nil // FIXME TODO +} + +type fs struct { + path string + snaps []pruning.Snapshot + + mtx sync.RWMutex + // for Plan + err error +} + +func (f* fs) Update(err error) { + f.mtx.Lock() + defer f.mtx.Unlock() + f.err = err +} + +type snapshot struct { + replicated bool + date time.Time + fsv *pdu.FilesystemVersion +} + +var _ pruning.Snapshot = snapshot{} + +func (s snapshot) Name() string { return s.fsv.Name } + +func (s snapshot) Replicated() bool { return s.replicated } + +func (s snapshot) Date() time.Time { return s.date } + +func shouldRetry(e error) bool { + switch e.(type) { + case nil: + return true + case net.Error: + return true + } + return false +} + +func onErr(u updater, e error) state { + return u(func(p *Pruner) { + p.err = e + if !shouldRetry(e) { + p.state = ErrPerm + return + } + switch p.state { + case Plan: p.state = PlanWait + case Exec: p.state = ExecWait + default: panic(p.state) + } + }).statefunc() +} + +func statePlan(a *args, u updater) state { + + ctx, target, receiver := a.ctx, a.target, a.receiver + + tfss, err := target.ListFilesystems(ctx) + if err != nil { + return onErr(u, err) + } + + pfss := make([]fs, len(tfss)) + for i, tfs := range tfss { + tfsvs, err := target.ListFilesystemVersions(ctx, tfs.Path) + if err != nil { + return onErr(u, err) + } + + pfs := fs{ + path: tfs.Path, + snaps: make([]pruning.Snapshot, 0, len(tfsvs)), + } + + for _, tfsv := range tfsvs { + if tfsv.Type != pdu.FilesystemVersion_Snapshot { + continue + } + creation, err := tfsv.CreationAsTime() + if err != nil { + return onErr(u, fmt.Errorf("%s%s has invalid creation date: %s", tfs, tfsv.RelName(), err)) + } + replicated, err := receiver.HasFilesystemVersion(ctx, tfs.Path, tfsv) + if err != nil && shouldRetry(err) { + return onErr(u, err) + } else if err != nil { + pfs.err = err + pfs.snaps = nil + break + } + pfs.snaps = append(pfs.snaps, snapshot{ + replicated: replicated, + date: creation, + fsv: tfsv, + }) + + } + + pfss[i] = pfs + + } + + return u(func(pruner *Pruner) { + for _, pfs := range pfss { + if pfs.err != nil { + pruner.pruneCompleted = append(pruner.pruneCompleted, pfs) + } else { + pruner.prunePending = append(pruner.prunePending, pfs) + } + } + pruner.state = Exec + }).statefunc() +} + +func stateExec(a *args, u updater) state { + + var pfs fs + state := u(func(pruner *Pruner) { + if len(pruner.prunePending) == 0 { + pruner.state = Done + return + } + pfs = pruner.prunePending[0] + }) + if state != Exec { + return state.statefunc() + } + + destroyListI := pruning.PruneSnapshots(pfs.snaps, a.rules) + destroyList := make([]*pdu.FilesystemVersion, len(destroyListI)) + for i := range destroyList { + destroyList[i] = destroyListI[i].(snapshot).fsv + } + pfs.Update(nil) + _, err := a.target.DestroySnapshots(a.ctx, pfs.path, destroyList) + pfs.Update(err) + if err != nil && shouldRetry(err) { + return onErr(u, err) + } + // if it's not retryable, treat is like as being done + + return u(func(pruner *Pruner) { + pruner.pruneCompleted = append(pruner.pruneCompleted, pfs) + pruner.prunePending = pruner.prunePending[1:] + }).statefunc() +} + +func stateExecWait(a *args, u updater) state { + return doWait(Exec, a, u) +} + +func statePlanWait(a *args, u updater) state { + return doWait(Plan, a, u) +} + +func doWait(goback State, a *args, u updater) state { + timer := time.NewTimer(a.retryWait) + defer timer.Stop() + select { + case <-timer.C: + return u(func(pruner *Pruner) { + pruner.state = goback + }).statefunc() + case <-a.ctx.Done(): + return onErr(u, a.ctx.Err()) + } +} diff --git a/daemon/pruner/pruner_test.go b/daemon/pruner/pruner_test.go new file mode 100644 index 0000000..6b560ef --- /dev/null +++ b/daemon/pruner/pruner_test.go @@ -0,0 +1,192 @@ +package pruner + +import ( + "testing" + "github.com/zrepl/zrepl/replication/pdu" + "context" + "github.com/zrepl/zrepl/pruning" + "fmt" + "time" + "github.com/stretchr/testify/assert" + "net" + "github.com/zrepl/zrepl/logger" +) + +type mockFS struct { + path string + snaps []string +} + +func (m *mockFS) Filesystem() *pdu.Filesystem { + return &pdu.Filesystem{ + Path: m.path, + } +} + +func (m *mockFS) FilesystemVersions() []*pdu.FilesystemVersion { + versions := make([]*pdu.FilesystemVersion, len(m.snaps)) + for i, v := range m.snaps { + versions[i] = &pdu.FilesystemVersion{ + Type: pdu.FilesystemVersion_Snapshot, + Name: v, + Creation: pdu.FilesystemVersionCreation(time.Unix(0, 0)), + } + } + return versions +} + +type mockTarget struct { + fss []mockFS + destroyed map[string][]string + listVersionsErrs map[string][]error + listFilesystemsErr []error + destroyErrs map[string][]error +} + +func (t *mockTarget) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) { + if len(t.listFilesystemsErr) > 0 { + e := t.listFilesystemsErr[0] + t.listFilesystemsErr = t.listFilesystemsErr[1:] + return nil, e + } + fss := make([]*pdu.Filesystem, len(t.fss)) + for i := range fss { + fss[i] = t.fss[i].Filesystem() + } + return fss, nil +} + +func (t *mockTarget) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) { + if len(t.listVersionsErrs[fs]) != 0 { + e := t.listVersionsErrs[fs][0] + t.listVersionsErrs[fs] = t.listVersionsErrs[fs][1:] + return nil, e + } + + for _, mfs := range t.fss { + if mfs.path != fs { + continue + } + return mfs.FilesystemVersions(), nil + } + return nil, fmt.Errorf("filesystem %s does not exist", fs) +} + +func (t *mockTarget) DestroySnapshots(ctx context.Context, fs string, snaps []*pdu.FilesystemVersion) ([]*pdu.FilesystemVersion, error) { + if len(t.destroyErrs[fs]) != 0 { + e := t.destroyErrs[fs][0] + t.destroyErrs[fs] = t.destroyErrs[fs][1:] + return nil, e + } + destroyed := t.destroyed[fs] + for _, s := range snaps { + destroyed = append(destroyed, s.Name) + } + t.destroyed[fs] = destroyed + return snaps, nil +} + +type mockReceiver struct { + fss []mockFS + errs map[string][]error +} + +func (r *mockReceiver) HasFilesystemVersion(ctx context.Context, fs string, version *pdu.FilesystemVersion) (bool, error) { + + if len(r.errs[fs]) > 0 { + e := r.errs[fs][0] + r.errs[fs] = r.errs[fs][1:] + return false, e + } + + for _, mfs := range r.fss { + if mfs.path != fs { + continue + } + for _, v := range mfs.FilesystemVersions() { + if v.Type == version.Type && v.Name == v.Name && v.CreateTXG == version.CreateTXG { + return true, nil + } + } + } + return false, nil +} + +func TestPruner_Prune(t *testing.T) { + + var _ net.Error = &net.OpError{} // we use it below + target := &mockTarget{ + listFilesystemsErr: []error{ + &net.OpError{Op: "fakerror0"}, + }, + listVersionsErrs: map[string][]error{ + "zroot/foo": { + &net.OpError{Op: "fakeerror1"}, // should be classified as temporaty + &net.OpError{Op: "fakeerror2"}, + }, + }, + destroyErrs: map[string][]error{ + "zroot/foo": { + fmt.Errorf("permanent error"), + }, + "zroot/bar": { + &net.OpError{Op: "fakeerror3"}, + }, + }, + destroyed: make(map[string][]string), + fss: []mockFS{ + { + path: "zroot/foo", + snaps: []string{ + "keep_a", + "keep_b", + "drop_c", + "keep_d", + }, + }, + { + path: "zroot/bar", + snaps: []string{ + "keep_e", + "keep_f", + "drop_g", + }, + }, + { + path: "zroot/baz", + snaps: []string{ + "keep_h", + "drop_i", + }, + }, + }, + } + receiver := &mockReceiver{ + errs: map[string][]error{ + "zroot/foo": { + &net.OpError{Op: "fakeerror4"}, + }, + "zroot/baz": { + fmt.Errorf("permanent error2"), + }, + }, + } + + keepRules := []pruning.KeepRule{pruning.MustKeepRegex("^keep")} + + p := NewPruner(10*time.Millisecond, target, receiver, keepRules) + ctx := context.Background() + ctx = WithLogger(ctx, logger.NewTestLogger(t)) + p.Prune(ctx) + + exp := map[string][]string{ + "zroot/bar":{"drop_g"}, + // drop_c is prohibited by failing destroy + // drop_i is prohibiteed by failing HasFilesystemVersion call + } + + assert.Equal(t, exp, target.destroyed) + + //assert.Equal(t, map[string][]error{}, target.listVersionsErrs, "retried") + +} diff --git a/daemon/pruner/state_string.go b/daemon/pruner/state_string.go new file mode 100644 index 0000000..68e04e1 --- /dev/null +++ b/daemon/pruner/state_string.go @@ -0,0 +1,35 @@ +// Code generated by "stringer -type=State"; DO NOT EDIT. + +package pruner + +import "strconv" + +const ( + _State_name_0 = "PlanPlanWait" + _State_name_1 = "Exec" + _State_name_2 = "ExecWait" + _State_name_3 = "ErrPerm" + _State_name_4 = "Done" +) + +var ( + _State_index_0 = [...]uint8{0, 4, 12} +) + +func (i State) String() string { + switch { + case 1 <= i && i <= 2: + i -= 1 + return _State_name_0[_State_index_0[i]:_State_index_0[i+1]] + case i == 4: + return _State_name_1 + case i == 8: + return _State_name_2 + case i == 16: + return _State_name_3 + case i == 32: + return _State_name_4 + default: + return "State(" + strconv.FormatInt(int64(i), 10) + ")" + } +} diff --git a/logger/testlogger.go b/logger/testlogger.go new file mode 100644 index 0000000..c416109 --- /dev/null +++ b/logger/testlogger.go @@ -0,0 +1,28 @@ +package logger + +import ( + "testing" +) + +type testLogger struct { + Logger +} + +type testingLoggerOutlet struct { + t *testing.T +} + +func (o testingLoggerOutlet) WriteEntry(entry Entry) error { + o.t.Logf("%#v", entry) + return nil +} + +var _ Logger = testLogger{} + +func NewTestLogger(t *testing.T) Logger { + outlets := NewOutlets() + outlets.Add(&testingLoggerOutlet{t}, Debug) + return &testLogger{ + Logger: NewLogger(outlets, 0), + } +} diff --git a/pruning/pruning.go b/pruning/pruning.go index 97b866b..2a754ec 100644 --- a/pruning/pruning.go +++ b/pruning/pruning.go @@ -14,6 +14,7 @@ type Snapshot interface { Date() time.Time } +// The returned snapshot list is guaranteed to only contains elements of input parameter snaps func PruneSnapshots(snaps []Snapshot, keepRules []KeepRule) []Snapshot { if len(keepRules) == 0 { diff --git a/replication/pdu/pdu_extras.go b/replication/pdu/pdu_extras.go index b009d3b..4d718a8 100644 --- a/replication/pdu/pdu_extras.go +++ b/replication/pdu/pdu_extras.go @@ -41,6 +41,10 @@ func FilesystemVersionFromZFS(fsv zfs.FilesystemVersion) *FilesystemVersion { } } +func FilesystemVersionCreation(t time.Time) string { + return t.Format(time.RFC3339) +} + func (v *FilesystemVersion) CreationAsTime() (time.Time, error) { return time.Parse(time.RFC3339, v.Creation) } From d684302864236c2ddac4298aa14d6a3708d5b289 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 30 Aug 2018 11:44:43 +0200 Subject: [PATCH 065/167] pruning: fix tests + implement 'not_replicated' and 'keep_regex' keep rule tests expected that a KeepRule returns a *keep* list whereas it actually returns a *destroy* list. --- cmd/config_parse.go | 2 +- config/config.go | 8 +-- pruning/keep_last_n.go | 6 +-- {cmd/pruning => pruning}/keep_last_n_test.go | 21 +++----- pruning/keep_not_replicated.go | 15 ++++++ pruning/keep_not_replicated_test.go | 39 ++++++++++++++ pruning/{keep_prefix.go => keep_regex.go} | 0 pruning/pruning.go | 33 ++++++++++-- pruning/pruning_test.go | 53 +++++++++++++------- 9 files changed, 134 insertions(+), 43 deletions(-) rename {cmd/pruning => pruning}/keep_last_n_test.go (75%) create mode 100644 pruning/keep_not_replicated.go create mode 100644 pruning/keep_not_replicated_test.go rename pruning/{keep_prefix.go => keep_regex.go} (100%) diff --git a/cmd/config_parse.go b/cmd/config_parse.go index a6cd27d..3e72072 100644 --- a/cmd/config_parse.go +++ b/cmd/config_parse.go @@ -188,7 +188,7 @@ func parseKeepRule(in config.PruningEnum) (p PrunePolicy, err error) { case config.PruneGrid: return retentiongrid.ParseGridPrunePolicy(v, willSeeBookmarks) //case config.PruneKeepLastN: - //case config.PruneKeepPrefix: + //case config.PruneKeepRegex: //case config.PruneKeepNotReplicated: default: panic(fmt.Sprintf("unknown keep rule type %v", v)) diff --git a/config/config.go b/config/config.go index 57bb701..2d618cf 100644 --- a/config/config.go +++ b/config/config.go @@ -179,9 +179,9 @@ type PruneKeepLastN struct { Count int `yaml:"count"` } -type PruneKeepPrefix struct { // FIXME rename to KeepPrefix - Type string `yaml:"type"` - Prefix string `yaml:"prefix"` +type PruneKeepRegex struct { // FIXME rename to KeepRegex + Type string `yaml:"type"` + Regex string `yaml:"prefix"` } type LoggingOutletEnum struct { @@ -301,7 +301,7 @@ func (t *PruningEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) "not_replicated": &PruneKeepNotReplicated{}, "last_n": &PruneKeepLastN{}, "grid": &PruneGrid{}, - "prefix": &PruneKeepPrefix{}, + "prefix": &PruneKeepRegex{}, }) return } diff --git a/pruning/keep_last_n.go b/pruning/keep_last_n.go index 5301141..a275154 100644 --- a/pruning/keep_last_n.go +++ b/pruning/keep_last_n.go @@ -16,10 +16,10 @@ func NewKeepLastN(n int) (*KeepLastN, error) { return &KeepLastN{n}, nil } -func (k KeepLastN) KeepRule(snaps []Snapshot) []Snapshot { +func (k KeepLastN) KeepRule(snaps []Snapshot) (destroyList []Snapshot) { if k.n > len(snaps) { - return snaps + return []Snapshot{} } res := shallowCopySnapList(snaps) @@ -28,5 +28,5 @@ func (k KeepLastN) KeepRule(snaps []Snapshot) []Snapshot { return res[i].Date().After(res[j].Date()) }) - return res[:k.n] + return res[k.n:] } diff --git a/cmd/pruning/keep_last_n_test.go b/pruning/keep_last_n_test.go similarity index 75% rename from cmd/pruning/keep_last_n_test.go rename to pruning/keep_last_n_test.go index 21e19f8..8ef0000 100644 --- a/cmd/pruning/keep_last_n_test.go +++ b/pruning/keep_last_n_test.go @@ -29,17 +29,18 @@ func TestKeepLastN(t *testing.T) { rules: []KeepRule{ KeepLastN{2}, }, - exp: map[string]bool{ - "4": true, "5": true, + expDestroy: map[string]bool{ + "1": true, "2": true, "3": true, }, }, - "keep1": { // Keep one of two with same time + "keep1OfTwoWithSameTime": { // Keep one of two with same time inputs: inputs["s1"], rules: []KeepRule{ KeepLastN{1}, }, - exp: map[string]bool{ - "4": true, //5 would be ok too + expDestroyAlternatives: []map[string]bool{ + {"1": true, "2": true, "3": true, "4": true}, + {"1": true, "2": true, "3": true, "5": true}, }, }, "keepMany": { @@ -47,20 +48,14 @@ func TestKeepLastN(t *testing.T) { rules: []KeepRule{ KeepLastN{100}, }, - exp: map[string]bool{ - "1": true, - "2": true, - "3": true, - "4": true, - "5": true, - }, + expDestroy: map[string]bool{}, }, "empty": { inputs: inputs["s2"], rules: []KeepRule{ KeepLastN{100}, }, - exp: map[string]bool{}, + expDestroy: map[string]bool{}, }, } diff --git a/pruning/keep_not_replicated.go b/pruning/keep_not_replicated.go new file mode 100644 index 0000000..955d9e6 --- /dev/null +++ b/pruning/keep_not_replicated.go @@ -0,0 +1,15 @@ +package pruning + +type KeepNotReplicated struct { + forceConstructor struct{} +} + +func (*KeepNotReplicated) KeepRule(snaps []Snapshot) (destroyList []Snapshot) { + return filterSnapList(snaps, func(snapshot Snapshot) bool { + return snapshot.Replicated() + }) +} + +func NewKeepNotReplicated() *KeepNotReplicated { + return &KeepNotReplicated{} +} diff --git a/pruning/keep_not_replicated_test.go b/pruning/keep_not_replicated_test.go new file mode 100644 index 0000000..47aa37d --- /dev/null +++ b/pruning/keep_not_replicated_test.go @@ -0,0 +1,39 @@ +package pruning + +import ( + "testing" +) + +func TestNewKeepNotReplicated(t *testing.T) { + + inputs := map[string][]Snapshot{ + "s1": []Snapshot{ + stubSnap{name: "1", replicated: true}, + stubSnap{name: "2", replicated: false}, + stubSnap{name: "3", replicated: true}, + }, + "s2": []Snapshot{}, + } + + tcs := map[string]testCase{ + "destroysOnlyReplicated": { + inputs: inputs["s1"], + rules: []KeepRule{ + NewKeepNotReplicated(), + }, + expDestroy: map[string]bool{ + "1": true, "3": true, + }, + }, + "empty": { + inputs: inputs["s2"], + rules: []KeepRule{ + NewKeepNotReplicated(), + }, + expDestroy: map[string]bool{}, + }, + } + + testTable(tcs, t) + +} diff --git a/pruning/keep_prefix.go b/pruning/keep_regex.go similarity index 100% rename from pruning/keep_prefix.go rename to pruning/keep_regex.go diff --git a/pruning/pruning.go b/pruning/pruning.go index 2a754ec..eb575c2 100644 --- a/pruning/pruning.go +++ b/pruning/pruning.go @@ -1,11 +1,14 @@ package pruning import ( + "fmt" + "github.com/pkg/errors" + "github.com/zrepl/zrepl/config" "time" ) type KeepRule interface { - KeepRule(snaps []Snapshot) []Snapshot + KeepRule(snaps []Snapshot) (destroyList []Snapshot) } type Snapshot interface { @@ -17,8 +20,8 @@ type Snapshot interface { // The returned snapshot list is guaranteed to only contains elements of input parameter snaps func PruneSnapshots(snaps []Snapshot, keepRules []KeepRule) []Snapshot { - if len(keepRules) == 0 { - return snaps + if keepRules == nil || len(keepRules) == 0 { + return []Snapshot{} } remCount := make(map[Snapshot]int, len(snaps)) @@ -38,3 +41,27 @@ func PruneSnapshots(snaps []Snapshot, keepRules []KeepRule) []Snapshot { return remove } + +func RulesFromConfig(in []config.PruningEnum) (rules []KeepRule, err error) { + rules = make([]KeepRule, len(in)) + for i := range in { + rules[i], err = RuleFromConfig(in[i]) + if err != nil { + return nil, errors.Wrapf(err, "cannot build rule #%d", i) + } + } + return rules, nil +} + +func RuleFromConfig(in config.PruningEnum) (KeepRule, error) { + switch v := in.Ret.(type) { + case *config.PruneKeepNotReplicated: + return NewKeepNotReplicated(), nil + case *config.PruneKeepLastN: + return NewKeepLastN(v.Count) + case *config.PruneKeepRegex: + return NewKeepRegex(v.Regex) + default: + return nil, fmt.Errorf("unknown keep rule type %T", v) + } +} diff --git a/pruning/pruning_test.go b/pruning/pruning_test.go index 4cdeb45..678dc50 100644 --- a/pruning/pruning_test.go +++ b/pruning/pruning_test.go @@ -1,8 +1,6 @@ package pruning import ( - "fmt" - "github.com/stretchr/testify/assert" "testing" "time" ) @@ -20,9 +18,10 @@ func (s stubSnap) Replicated() bool { return s.replicated } func (s stubSnap) Date() time.Time { return s.date } type testCase struct { - inputs []Snapshot - rules []KeepRule - exp, eff map[string]bool + inputs []Snapshot + rules []KeepRule + expDestroy, effDestroy map[string]bool + expDestroyAlternatives []map[string]bool } func testTable(tcs map[string]testCase, t *testing.T) { @@ -42,11 +41,26 @@ func testTable(tcs map[string]testCase, t *testing.T) { t.Run(name, func(t *testing.T) { tc := tcs[name] remove := PruneSnapshots(tc.inputs, tc.rules) - tc.eff = make(map[string]bool) + tc.effDestroy = make(map[string]bool) for _, s := range remove { - tc.eff[s.Name()] = true + tc.effDestroy[s.Name()] = true + } + if tc.expDestroyAlternatives == nil { + if tc.expDestroy == nil { + panic("must specify either expDestroyAlternatives or expDestroy") + } + tc.expDestroyAlternatives = []map[string]bool{tc.expDestroy} + } + var okAlt map[string]bool = nil + for _, alt := range tc.expDestroyAlternatives { + t.Logf("testing possible result: %v", alt) + if mapEqual(alt, tc.effDestroy) { + okAlt = alt + } + } + if okAlt == nil { + t.Errorf("no alternatives matched result: %v", tc.effDestroy) } - assert.True(t, mapEqual(tc.exp, tc.eff), fmt.Sprintf("is %v but should be %v", tc.eff, tc.exp)) }) } } @@ -67,7 +81,7 @@ func TestPruneSnapshots(t *testing.T) { rules: []KeepRule{ MustKeepRegex("foo_"), }, - exp: map[string]bool{ + expDestroy: map[string]bool{ "bar_123": true, }, }, @@ -77,7 +91,7 @@ func TestPruneSnapshots(t *testing.T) { MustKeepRegex("foo_"), MustKeepRegex("bar_"), }, - exp: map[string]bool{}, + expDestroy: map[string]bool{}, }, "onlyThoseRemovedByAllAreRemoved": { inputs: inputs["s1"], @@ -85,26 +99,27 @@ func TestPruneSnapshots(t *testing.T) { MustKeepRegex("notInS1"), // would remove all MustKeepRegex("bar_"), // would remove all but bar_, i.e. foo_.* }, - exp: map[string]bool{ + expDestroy: map[string]bool{ "foo_123": true, "foo_456": true, }, }, "noRulesKeepsAll": { - inputs: inputs["s1"], - rules: []KeepRule{}, - exp: map[string]bool{ - "foo_123": true, - "foo_456": true, - "bar_123": true, - }, + inputs: inputs["s1"], + rules: []KeepRule{}, + expDestroy: map[string]bool{}, + }, + "nilRulesKeepsAll": { + inputs: inputs["s1"], + rules: nil, + expDestroy: map[string]bool{}, }, "noSnaps": { inputs: []Snapshot{}, rules: []KeepRule{ MustKeepRegex("foo_"), }, - exp: map[string]bool{}, + expDestroy: map[string]bool{}, }, } From 12dd240b5fc3eee78c562c8562f8d251b760dca4 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 30 Aug 2018 11:49:06 +0200 Subject: [PATCH 066/167] fixup pruner --- daemon/pruner/pruner.go | 77 +++++++++++++++++++++--------------- daemon/pruner/pruner_test.go | 45 +++++++++++---------- 2 files changed, 70 insertions(+), 52 deletions(-) diff --git a/daemon/pruner/pruner.go b/daemon/pruner/pruner.go index 303ba14..8041cc0 100644 --- a/daemon/pruner/pruner.go +++ b/daemon/pruner/pruner.go @@ -2,24 +2,24 @@ package pruner import ( "context" + "fmt" + "github.com/zrepl/zrepl/logger" "github.com/zrepl/zrepl/pruning" "github.com/zrepl/zrepl/replication/pdu" + "net" "sync" "time" - "fmt" - "net" - "github.com/zrepl/zrepl/logger" ) // Try to keep it compatible with gitub.com/zrepl/zrepl/replication.Endpoint -type Receiver interface { - HasFilesystemVersion(ctx context.Context, fs string, version *pdu.FilesystemVersion) (bool, error) +type History interface { + SnapshotReplicationStatus(ctx context.Context, req *pdu.SnapshotReplicationStatusReq) (*pdu.SnapshotReplicationStatusRes, error) } type Target interface { ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) // fix depS - DestroySnapshots(ctx context.Context, fs string, snaps []*pdu.FilesystemVersion) ([]*pdu.FilesystemVersion, error) + DestroySnapshots(ctx context.Context, req *pdu.DestroySnapshotsReq) (*pdu.DestroySnapshotsRes, error) } type Logger = logger.Logger @@ -32,7 +32,7 @@ func WithLogger(ctx context.Context, log Logger) context.Context { return context.WithValue(ctx, contextKeyLogger, log) } -func getLogger(ctx context.Context) Logger { +func GetLogger(ctx context.Context) Logger { if l, ok := ctx.Value(contextKeyLogger).(Logger); ok { return l } @@ -40,15 +40,14 @@ func getLogger(ctx context.Context) Logger { } type args struct { - ctx context.Context - target Target - receiver Receiver - rules []pruning.KeepRule - retryWait time.Duration + ctx context.Context + target Target + receiver History + rules []pruning.KeepRule + retryWait time.Duration } type Pruner struct { - args args mtx sync.RWMutex @@ -60,14 +59,13 @@ type Pruner struct { err error // State Exec - prunePending []fs + prunePending []fs pruneCompleted []fs - } -func NewPruner(retryWait time.Duration, target Target, receiver Receiver, rules []pruning.KeepRule) *Pruner { +func NewPruner(retryWait time.Duration, target Target, receiver History, rules []pruning.KeepRule) *Pruner { p := &Pruner{ - args: args{nil, target, receiver, rules, retryWait}, // ctx is filled in Prune() + args: args{nil, target, receiver, rules, retryWait}, // ctx is filled in Prune() state: Plan, } return p @@ -85,7 +83,6 @@ const ( Done ) - func (s State) statefunc() state { var statemap = map[State]state{ Plan: statePlan, @@ -117,7 +114,7 @@ func (p *Pruner) prune(args args) { return p.state }) post := p.state - getLogger(args.ctx). + GetLogger(args.ctx). WithField("transition", fmt.Sprintf("%s=>%s", pre, post)). Debug("state transition") } @@ -128,7 +125,7 @@ func (p *Pruner) Report() interface{} { } type fs struct { - path string + path string snaps []pruning.Snapshot mtx sync.RWMutex @@ -136,7 +133,7 @@ type fs struct { err error } -func (f* fs) Update(err error) { +func (f *fs) Update(err error) { f.mtx.Lock() defer f.mtx.Unlock() f.err = err @@ -144,8 +141,8 @@ func (f* fs) Update(err error) { type snapshot struct { replicated bool - date time.Time - fsv *pdu.FilesystemVersion + date time.Time + fsv *pdu.FilesystemVersion } var _ pruning.Snapshot = snapshot{} @@ -174,9 +171,12 @@ func onErr(u updater, e error) state { return } switch p.state { - case Plan: p.state = PlanWait - case Exec: p.state = ExecWait - default: panic(p.state) + case Plan: + p.state = PlanWait + case Exec: + p.state = ExecWait + default: + panic(p.state) } }).statefunc() } @@ -210,7 +210,12 @@ func statePlan(a *args, u updater) state { if err != nil { return onErr(u, fmt.Errorf("%s%s has invalid creation date: %s", tfs, tfsv.RelName(), err)) } - replicated, err := receiver.HasFilesystemVersion(ctx, tfs.Path, tfsv) + req := pdu.SnapshotReplicationStatusReq{ + Filesystem: tfs.Path, + Snapshot: tfsv.Name, + Op: pdu.SnapshotReplicationStatusReq_Get, + } + res, err := receiver.SnapshotReplicationStatus(ctx, &req) if err != nil && shouldRetry(err) { return onErr(u, err) } else if err != nil { @@ -218,10 +223,11 @@ func statePlan(a *args, u updater) state { pfs.snaps = nil break } + pfs.snaps = append(pfs.snaps, snapshot{ - replicated: replicated, + replicated: res.Replicated, date: creation, - fsv: tfsv, + fsv: tfsv, }) } @@ -256,15 +262,24 @@ func stateExec(a *args, u updater) state { return state.statefunc() } + GetLogger(a.ctx).Debug(fmt.Sprintf("%#v", a.rules)) destroyListI := pruning.PruneSnapshots(pfs.snaps, a.rules) destroyList := make([]*pdu.FilesystemVersion, len(destroyListI)) for i := range destroyList { destroyList[i] = destroyListI[i].(snapshot).fsv + GetLogger(a.ctx). + WithField("fs", pfs.path). + WithField("destroy_snap", destroyList[i].Name). + Debug("policy destroys snapshot") } pfs.Update(nil) - _, err := a.target.DestroySnapshots(a.ctx, pfs.path, destroyList) + req := pdu.DestroySnapshotsReq{ + Filesystem: pfs.path, + Snapshots: destroyList, + } + _, err := a.target.DestroySnapshots(a.ctx, &req) pfs.Update(err) - if err != nil && shouldRetry(err) { + if err != nil && shouldRetry(err) { return onErr(u, err) } // if it's not retryable, treat is like as being done diff --git a/daemon/pruner/pruner_test.go b/daemon/pruner/pruner_test.go index 6b560ef..ce8fb7c 100644 --- a/daemon/pruner/pruner_test.go +++ b/daemon/pruner/pruner_test.go @@ -1,15 +1,15 @@ package pruner import ( - "testing" - "github.com/zrepl/zrepl/replication/pdu" "context" - "github.com/zrepl/zrepl/pruning" "fmt" - "time" "github.com/stretchr/testify/assert" - "net" "github.com/zrepl/zrepl/logger" + "github.com/zrepl/zrepl/pruning" + "github.com/zrepl/zrepl/replication/pdu" + "net" + "testing" + "time" ) type mockFS struct { @@ -27,8 +27,8 @@ func (m *mockFS) FilesystemVersions() []*pdu.FilesystemVersion { versions := make([]*pdu.FilesystemVersion, len(m.snaps)) for i, v := range m.snaps { versions[i] = &pdu.FilesystemVersion{ - Type: pdu.FilesystemVersion_Snapshot, - Name: v, + Type: pdu.FilesystemVersion_Snapshot, + Name: v, Creation: pdu.FilesystemVersionCreation(time.Unix(0, 0)), } } @@ -36,11 +36,11 @@ func (m *mockFS) FilesystemVersions() []*pdu.FilesystemVersion { } type mockTarget struct { - fss []mockFS - destroyed map[string][]string - listVersionsErrs map[string][]error + fss []mockFS + destroyed map[string][]string + listVersionsErrs map[string][]error listFilesystemsErr []error - destroyErrs map[string][]error + destroyErrs map[string][]error } func (t *mockTarget) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) { @@ -72,26 +72,29 @@ func (t *mockTarget) ListFilesystemVersions(ctx context.Context, fs string) ([]* return nil, fmt.Errorf("filesystem %s does not exist", fs) } -func (t *mockTarget) DestroySnapshots(ctx context.Context, fs string, snaps []*pdu.FilesystemVersion) ([]*pdu.FilesystemVersion, error) { +func (t *mockTarget) DestroySnapshots(ctx context.Context, req *pdu.DestroySnapshotsReq) (*pdu.DestroySnapshotsRes, error) { + fs, snaps := req.Filesystem, req.Snapshots if len(t.destroyErrs[fs]) != 0 { e := t.destroyErrs[fs][0] t.destroyErrs[fs] = t.destroyErrs[fs][1:] return nil, e } destroyed := t.destroyed[fs] - for _, s := range snaps { + res := make([]*pdu.DestroySnapshotRes, len(snaps)) + for i, s := range snaps { destroyed = append(destroyed, s.Name) + res[i] = &pdu.DestroySnapshotRes{Error: "", Snapshot: s} } t.destroyed[fs] = destroyed - return snaps, nil + return &pdu.DestroySnapshotsRes{Results: res}, nil } -type mockReceiver struct { - fss []mockFS +type mockHistory struct { + fss []mockFS errs map[string][]error } -func (r *mockReceiver) HasFilesystemVersion(ctx context.Context, fs string, version *pdu.FilesystemVersion) (bool, error) { +func (r *mockHistory) WasSnapshotReplicated(ctx context.Context, fs string, version *pdu.FilesystemVersion) (bool, error) { if len(r.errs[fs]) > 0 { e := r.errs[fs][0] @@ -161,7 +164,7 @@ func TestPruner_Prune(t *testing.T) { }, }, } - receiver := &mockReceiver{ + history := &mockHistory{ errs: map[string][]error{ "zroot/foo": { &net.OpError{Op: "fakeerror4"}, @@ -174,15 +177,15 @@ func TestPruner_Prune(t *testing.T) { keepRules := []pruning.KeepRule{pruning.MustKeepRegex("^keep")} - p := NewPruner(10*time.Millisecond, target, receiver, keepRules) + p := NewPruner(10*time.Millisecond, target, history, keepRules) ctx := context.Background() ctx = WithLogger(ctx, logger.NewTestLogger(t)) p.Prune(ctx) exp := map[string][]string{ - "zroot/bar":{"drop_g"}, + "zroot/bar": {"drop_g"}, // drop_c is prohibited by failing destroy - // drop_i is prohibiteed by failing HasFilesystemVersion call + // drop_i is prohibiteed by failing WasSnapshotReplicated call } assert.Equal(t, exp, target.destroyed) From 22ca80eb7e71c10eba577e96e8356922d42ed8f3 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 30 Aug 2018 11:51:47 +0200 Subject: [PATCH 067/167] remote snapshot destruction & replication status zfs property --- endpoint/endpoint.go | 173 ++++++++++++- replication/fsrep/fsfsm.go | 118 ++++++--- replication/fsrep/stepstate_string.go | 14 +- replication/mainfsm.go | 1 + replication/pdu/pdu.pb.go | 348 ++++++++++++++++++++++---- replication/pdu/pdu.proto | 29 +++ replication/pdu/pdu_extras.go | 21 +- zfs/replication_history.go | 25 ++ zfs/versions.go | 2 +- zfs/zfs.go | 29 ++- 10 files changed, 662 insertions(+), 98 deletions(-) create mode 100644 zfs/replication_history.go diff --git a/endpoint/endpoint.go b/endpoint/endpoint.go index fca060b..408a155 100644 --- a/endpoint/endpoint.go +++ b/endpoint/endpoint.go @@ -4,6 +4,7 @@ package endpoint import ( "bytes" "context" + "fmt" "github.com/golang/protobuf/proto" "github.com/pkg/errors" "github.com/problame/go-streamrpc" @@ -56,7 +57,7 @@ func (p *Sender) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu. } rfsvs := make([]*pdu.FilesystemVersion, len(fsvs)) for i := range fsvs { - rfsvs[i] = pdu.FilesystemVersionFromZFS(fsvs[i]) + rfsvs[i] = pdu.FilesystemVersionFromZFS(&fsvs[i]) } return rfsvs, nil } @@ -80,6 +81,62 @@ func (p *Sender) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.Rea return &pdu.SendRes{}, stream, nil } +func (p *Sender) DestroySnapshots(ctx context.Context, req *pdu.DestroySnapshotsReq) (*pdu.DestroySnapshotsRes, error) { + dp, err := zfs.NewDatasetPath(req.Filesystem) + if err != nil { + return nil, err + } + pass, err := p.FSFilter.Filter(dp) + if err != nil { + return nil, err + } + if !pass { + return nil, replication.NewFilteredError(req.Filesystem) + } + + return doDestroySnapshots(ctx, dp, req.Snapshots) +} + +// Since replication always happens from sender to receiver, this method is only ipmlemented for the sender. +// If this method returns a *zfs.DatasetDoesNotExist as an error, it might be a good indicator +// that something is wrong with the pruning logic, which is the only consumer of this method. +func (p *Sender) SnapshotReplicationStatus(ctx context.Context, req *pdu.SnapshotReplicationStatusReq) (*pdu.SnapshotReplicationStatusRes, error) { + dp, err := zfs.NewDatasetPath(req.Filesystem) + if err != nil { + return nil, err + } + pass, err := p.FSFilter.Filter(dp) + if err != nil { + return nil, err + } + if !pass { + return nil, replication.NewFilteredError(req.Filesystem) + } + + version := zfs.FilesystemVersion{ + Type: zfs.Snapshot, + Name: req.Snapshot, //FIXME validation + } + + replicated := false + switch req.Op { + case pdu.SnapshotReplicationStatusReq_Get: + replicated, err = zfs.ZFSGetReplicatedProperty(dp, &version) + if err != nil { + return nil, err + } + case pdu.SnapshotReplicationStatusReq_SetReplicated: + err = zfs.ZFSSetReplicatedProperty(dp, &version, true) + if err != nil { + return nil, err + } + replicated = true + default: + return nil, errors.Errorf("unknown opcode %v", req.Op) + } + return &pdu.SnapshotReplicationStatusRes{Replicated: replicated}, nil +} + type FSFilter interface { Filter(path *zfs.DatasetPath) (pass bool, err error) } @@ -143,7 +200,7 @@ func (e *Receiver) ListFilesystemVersions(ctx context.Context, fs string) ([]*pd rfsvs := make([]*pdu.FilesystemVersion, len(fsvs)) for i := range fsvs { - rfsvs[i] = pdu.FilesystemVersionFromZFS(fsvs[i]) + rfsvs[i] = pdu.FilesystemVersionFromZFS(&fsvs[i]) } return rfsvs, nil @@ -215,15 +272,61 @@ func (e *Receiver) Receive(ctx context.Context, req *pdu.ReceiveReq, sendStream return nil } +func (e *Receiver) DestroySnapshots(ctx context.Context, req *pdu.DestroySnapshotsReq) (*pdu.DestroySnapshotsRes, error) { + dp, err := zfs.NewDatasetPath(req.Filesystem) + if err != nil { + return nil, err + } + lp, err := e.fsmap.Map(dp) + if err != nil { + return nil, err + } + if lp == nil { + return nil, errors.New("access to filesystem denied") + } + return doDestroySnapshots(ctx, lp, req.Snapshots) +} + +func doDestroySnapshots(ctx context.Context, lp *zfs.DatasetPath, snaps []*pdu.FilesystemVersion) (*pdu.DestroySnapshotsRes, error) { + fsvs := make([]*zfs.FilesystemVersion, len(snaps)) + for i, fsv := range snaps { + if fsv.Type != pdu.FilesystemVersion_Snapshot { + return nil, fmt.Errorf("version %q is not a snapshot", fsv.Name) + } + var err error + fsvs[i], err = fsv.ZFSFilesystemVersion() + if err != nil { + return nil, err + } + } + res := &pdu.DestroySnapshotsRes{ + Results: make([]*pdu.DestroySnapshotRes, len(fsvs)), + } + for i, fsv := range fsvs { + err := zfs.ZFSDestroyFilesystemVersion(lp, fsv) + errMsg := "" + if err != nil { + errMsg = err.Error() + } + res.Results[i] = &pdu.DestroySnapshotRes{ + Snapshot: pdu.FilesystemVersionFromZFS(fsv), + Error: errMsg, + } + } + return res, nil +} + // =-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= // RPC STUBS // =-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= const ( - RPCListFilesystems = "ListFilesystems" - RPCListFilesystemVersions = "ListFilesystemVersions" - RPCReceive = "Receive" - RPCSend = "Send" + RPCListFilesystems = "ListFilesystems" + RPCListFilesystemVersions = "ListFilesystemVersions" + RPCReceive = "Receive" + RPCSend = "Send" + RPCSDestroySnapshots = "DestroySnapshots" + RPCSnapshotReplicationStatus = "SnapshotReplicationStatus" ) // Remote implements an endpoint stub that uses streamrpc as a transport. @@ -320,6 +423,26 @@ func (s Remote) Receive(ctx context.Context, r *pdu.ReceiveReq, sendStream io.Re return nil } +func (s Remote) DestroySnapshots(ctx context.Context, r *pdu.DestroySnapshotsReq) (*pdu.DestroySnapshotsRes, error) { + b, err := proto.Marshal(r) + if err != nil { + return nil, err + } + rb, rs, err := s.c.RequestReply(ctx, RPCSDestroySnapshots, bytes.NewBuffer(b), nil) + if err != nil { + return nil, err + } + if rs != nil { + rs.Close() + return nil, errors.New("response contains unexpected stream") + } + var res pdu.DestroySnapshotsRes + if err := proto.Unmarshal(rb.Bytes(), &res); err != nil { + return nil, err + } + return &res, nil +} + // Handler implements the server-side streamrpc.HandlerFunc for a Remote endpoint stub. type Handler struct { ep replication.Endpoint @@ -411,6 +534,44 @@ func (a *Handler) Handle(ctx context.Context, endpoint string, reqStructured *by } return bytes.NewBuffer(b), nil, err + case RPCSDestroySnapshots: + + var req pdu.DestroySnapshotsReq + if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { + return nil, nil, err + } + + res, err := a.ep.DestroySnapshots(ctx, &req) + if err != nil { + return nil, nil, err + } + b, err := proto.Marshal(res) + if err != nil { + return nil, nil, err + } + return bytes.NewBuffer(b), nil, nil + + case RPCSnapshotReplicationStatus: + + sender, ok := a.ep.(replication.Sender) + if !ok { + goto Err + } + + var req pdu.SnapshotReplicationStatusReq + if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { + return nil, nil, err + } + res, err := sender.SnapshotReplicationStatus(ctx, &req) + if err != nil { + return nil, nil, err + } + b, err := proto.Marshal(res) + if err != nil { + return nil, nil, err + } + return bytes.NewBuffer(b), nil, nil + } Err: return nil, nil, errors.New("no handler for given endpoint") diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index 52f6c31..5957dda 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -7,7 +7,6 @@ import ( "errors" "fmt" "io" - "math/bits" "net" "sync" "time" @@ -41,6 +40,7 @@ type Sender interface { // If a non-nil io.ReadCloser is returned, it is guaranteed to be closed before // any next call to the parent github.com/zrepl/zrepl/replication.Endpoint. Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) + SnapshotReplicationStatus(ctx context.Context, r *pdu.SnapshotReplicationStatusReq) (*pdu.SnapshotReplicationStatusRes, error) } // A Sender is usually part of a github.com/zrepl/zrepl/replication.Endpoint. @@ -76,17 +76,13 @@ const ( ) func (s State) fsrsf() state { - idx := bits.TrailingZeros(uint(s)) - if idx == bits.UintSize { - panic(s) + m := map[State]state{ + Ready: stateReady, + RetryWait: stateRetryWait, + PermanentError: nil, + Completed: nil, } - m := []state{ - stateReady, - stateRetryWait, - nil, - nil, - } - return m[idx] + return m[s] } type Replication struct { @@ -115,7 +111,7 @@ func BuildReplication(fs string) *ReplicationBuilder { func (b *ReplicationBuilder) AddStep(from, to FilesystemVersion) *ReplicationBuilder { step := &ReplicationStep{ - state: StepReady, + state: StepReplicationReady, parent: b.r, from: from, to: to, @@ -147,15 +143,18 @@ func NewReplicationWithPermanentError(fs string, err error) *Replication { type StepState uint const ( - StepReady StepState = 1 << iota - StepRetry + StepReplicationReady StepState = 1 << iota + StepReplicationRetry + StepMarkReplicatedReady + StepMarkReplicatedRetry StepPermanentError StepCompleted ) type FilesystemVersion interface { SnapshotTime() time.Time - RelName() string + GetName() string // name without @ or # + RelName() string // name with @ or # } type ReplicationStep struct { @@ -223,7 +222,7 @@ func stateReady(ctx context.Context, sender Sender, receiver Receiver, u updater return s.fsrsf() } - stepState := current.do(ctx, sender, receiver) + stepState := current.doReplication(ctx, sender, receiver) return u(func(f *Replication) { switch stepState { @@ -235,7 +234,9 @@ func stateReady(ctx context.Context, sender Sender, receiver Receiver, u updater } else { f.state = Completed } - case StepRetry: + case StepReplicationRetry: + fallthrough + case StepMarkReplicatedRetry: f.retryWaitUntil = time.Now().Add(RetrySleepDuration) f.state = RetryWait case StepPermanentError: @@ -292,7 +293,22 @@ func (fsr *Replication) Report() *Report { return &rep } -func (s *ReplicationStep) do(ctx context.Context, sender Sender, receiver Receiver) StepState { +func shouldRetry(err error) bool { + switch err { + case io.EOF: + fallthrough + case io.ErrUnexpectedEOF: + fallthrough + case io.ErrClosedPipe: + return true + } + if _, ok := err.(net.Error); ok { + return true + } + return false +} + +func (s *ReplicationStep) doReplication(ctx context.Context, sender Sender, receiver Receiver) StepState { fs := s.parent.fs @@ -305,17 +321,8 @@ func (s *ReplicationStep) do(ctx context.Context, sender Sender, receiver Receiv defer s.lock.Unlock() s.err = err - switch err { - case io.EOF: - fallthrough - case io.ErrUnexpectedEOF: - fallthrough - case io.ErrClosedPipe: - s.state = StepRetry - return s.state - } - if _, ok := err.(net.Error); ok { - s.state = StepRetry + if shouldRetry(s.err) { + s.state = StepReplicationRetry return s.state } s.state = StepPermanentError @@ -326,7 +333,7 @@ func (s *ReplicationStep) do(ctx context.Context, sender Sender, receiver Receiv s.lock.Lock() defer s.lock.Unlock() s.err = nil - s.state = StepCompleted + s.state = StepMarkReplicatedReady return s.state } @@ -371,8 +378,57 @@ func (s *ReplicationStep) do(ctx context.Context, sender Sender, receiver Receiv return updateStateError(err) } log.Info("receive finished") - return updateStateCompleted() + updateStateCompleted() + + return s.doMarkReplicated(ctx, sender) + +} + +func (s *ReplicationStep) doMarkReplicated(ctx context.Context, sender Sender) StepState { + + log := getLogger(ctx). + WithField("filesystem", s.parent.fs). + WithField("step", s.String()) + + updateStateError := func(err error) StepState { + s.lock.Lock() + defer s.lock.Unlock() + + s.err = err + if shouldRetry(s.err) { + s.state = StepMarkReplicatedRetry + return s.state + } + s.state = StepPermanentError + return s.state + } + + updateStateCompleted := func() StepState { + s.lock.Lock() + defer s.lock.Unlock() + s.state = StepCompleted + return s.state + } + + log.Info("mark snapshot as replicated") + req := pdu.SnapshotReplicationStatusReq{ + Filesystem: s.parent.fs, + Snapshot: s.to.GetName(), + Op: pdu.SnapshotReplicationStatusReq_SetReplicated, + } + res, err := sender.SnapshotReplicationStatus(ctx, &req) + if err != nil { + log.WithError(err).Error("error marking snapshot as replicated") + return updateStateError(err) + } + if res.Replicated != true { + err := fmt.Errorf("sender did not report snapshot as replicated") + log.Error(err.Error()) + return updateStateError(err) + } + + return updateStateCompleted() } func (s *ReplicationStep) String() string { diff --git a/replication/fsrep/stepstate_string.go b/replication/fsrep/stepstate_string.go index 8228bfc..aeac7e1 100644 --- a/replication/fsrep/stepstate_string.go +++ b/replication/fsrep/stepstate_string.go @@ -5,13 +5,15 @@ package fsrep import "strconv" const ( - _StepState_name_0 = "StepReadyStepRetry" - _StepState_name_1 = "StepPermanentError" - _StepState_name_2 = "StepCompleted" + _StepState_name_0 = "StepReplicationReadyStepReplicationRetry" + _StepState_name_1 = "StepMarkReplicatedReady" + _StepState_name_2 = "StepMarkReplicatedRetry" + _StepState_name_3 = "StepPermanentError" + _StepState_name_4 = "StepCompleted" ) var ( - _StepState_index_0 = [...]uint8{0, 9, 18} + _StepState_index_0 = [...]uint8{0, 20, 40} ) func (i StepState) String() string { @@ -23,6 +25,10 @@ func (i StepState) String() string { return _StepState_name_1 case i == 8: return _StepState_name_2 + case i == 16: + return _StepState_name_3 + case i == 32: + return _StepState_name_4 default: return "StepState(" + strconv.FormatInt(int64(i), 10) + ")" } diff --git a/replication/mainfsm.go b/replication/mainfsm.go index 7d9bfc5..2fc5cc3 100644 --- a/replication/mainfsm.go +++ b/replication/mainfsm.go @@ -93,6 +93,7 @@ type Endpoint interface { ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) // FIXME document FilteredError handling ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) // fix depS + DestroySnapshots(ctx context.Context, req *pdu.DestroySnapshotsReq) (*pdu.DestroySnapshotsRes, error) } type Sender interface { diff --git a/replication/pdu/pdu.pb.go b/replication/pdu/pdu.pb.go index a770a17..db361c3 100644 --- a/replication/pdu/pdu.pb.go +++ b/replication/pdu/pdu.pb.go @@ -38,7 +38,30 @@ func (x FilesystemVersion_VersionType) String() string { return proto.EnumName(FilesystemVersion_VersionType_name, int32(x)) } func (FilesystemVersion_VersionType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{5, 0} + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{5, 0} +} + +type SnapshotReplicationStatusReq_Op int32 + +const ( + SnapshotReplicationStatusReq_Get SnapshotReplicationStatusReq_Op = 0 + SnapshotReplicationStatusReq_SetReplicated SnapshotReplicationStatusReq_Op = 1 +) + +var SnapshotReplicationStatusReq_Op_name = map[int32]string{ + 0: "Get", + 1: "SetReplicated", +} +var SnapshotReplicationStatusReq_Op_value = map[string]int32{ + "Get": 0, + "SetReplicated": 1, +} + +func (x SnapshotReplicationStatusReq_Op) String() string { + return proto.EnumName(SnapshotReplicationStatusReq_Op_name, int32(x)) +} +func (SnapshotReplicationStatusReq_Op) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{14, 0} } type ListFilesystemReq struct { @@ -51,7 +74,7 @@ func (m *ListFilesystemReq) Reset() { *m = ListFilesystemReq{} } func (m *ListFilesystemReq) String() string { return proto.CompactTextString(m) } func (*ListFilesystemReq) ProtoMessage() {} func (*ListFilesystemReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{0} + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{0} } func (m *ListFilesystemReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemReq.Unmarshal(m, b) @@ -82,7 +105,7 @@ func (m *ListFilesystemRes) Reset() { *m = ListFilesystemRes{} } func (m *ListFilesystemRes) String() string { return proto.CompactTextString(m) } func (*ListFilesystemRes) ProtoMessage() {} func (*ListFilesystemRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{1} + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{1} } func (m *ListFilesystemRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemRes.Unmarshal(m, b) @@ -121,7 +144,7 @@ func (m *Filesystem) Reset() { *m = Filesystem{} } func (m *Filesystem) String() string { return proto.CompactTextString(m) } func (*Filesystem) ProtoMessage() {} func (*Filesystem) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{2} + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{2} } func (m *Filesystem) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Filesystem.Unmarshal(m, b) @@ -166,7 +189,7 @@ func (m *ListFilesystemVersionsReq) Reset() { *m = ListFilesystemVersion func (m *ListFilesystemVersionsReq) String() string { return proto.CompactTextString(m) } func (*ListFilesystemVersionsReq) ProtoMessage() {} func (*ListFilesystemVersionsReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{3} + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{3} } func (m *ListFilesystemVersionsReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemVersionsReq.Unmarshal(m, b) @@ -204,7 +227,7 @@ func (m *ListFilesystemVersionsRes) Reset() { *m = ListFilesystemVersion func (m *ListFilesystemVersionsRes) String() string { return proto.CompactTextString(m) } func (*ListFilesystemVersionsRes) ProtoMessage() {} func (*ListFilesystemVersionsRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{4} + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{4} } func (m *ListFilesystemVersionsRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemVersionsRes.Unmarshal(m, b) @@ -246,7 +269,7 @@ func (m *FilesystemVersion) Reset() { *m = FilesystemVersion{} } func (m *FilesystemVersion) String() string { return proto.CompactTextString(m) } func (*FilesystemVersion) ProtoMessage() {} func (*FilesystemVersion) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{5} + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{5} } func (m *FilesystemVersion) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_FilesystemVersion.Unmarshal(m, b) @@ -325,7 +348,7 @@ func (m *SendReq) Reset() { *m = SendReq{} } func (m *SendReq) String() string { return proto.CompactTextString(m) } func (*SendReq) ProtoMessage() {} func (*SendReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{6} + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{6} } func (m *SendReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SendReq.Unmarshal(m, b) @@ -399,7 +422,7 @@ func (m *Property) Reset() { *m = Property{} } func (m *Property) String() string { return proto.CompactTextString(m) } func (*Property) ProtoMessage() {} func (*Property) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{7} + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{7} } func (m *Property) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Property.Unmarshal(m, b) @@ -446,7 +469,7 @@ func (m *SendRes) Reset() { *m = SendRes{} } func (m *SendRes) String() string { return proto.CompactTextString(m) } func (*SendRes) ProtoMessage() {} func (*SendRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{8} + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{8} } func (m *SendRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SendRes.Unmarshal(m, b) @@ -493,7 +516,7 @@ func (m *ReceiveReq) Reset() { *m = ReceiveReq{} } func (m *ReceiveReq) String() string { return proto.CompactTextString(m) } func (*ReceiveReq) ProtoMessage() {} func (*ReceiveReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{9} + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{9} } func (m *ReceiveReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ReceiveReq.Unmarshal(m, b) @@ -537,7 +560,7 @@ func (m *ReceiveRes) Reset() { *m = ReceiveRes{} } func (m *ReceiveRes) String() string { return proto.CompactTextString(m) } func (*ReceiveRes) ProtoMessage() {} func (*ReceiveRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_25ca9e5977a7ce3f, []int{10} + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{10} } func (m *ReceiveRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ReceiveRes.Unmarshal(m, b) @@ -557,6 +580,229 @@ func (m *ReceiveRes) XXX_DiscardUnknown() { var xxx_messageInfo_ReceiveRes proto.InternalMessageInfo +type DestroySnapshotsReq struct { + Filesystem string `protobuf:"bytes,1,opt,name=Filesystem,proto3" json:"Filesystem,omitempty"` + // Path to filesystem, snapshot or bookmark to be destroyed + Snapshots []*FilesystemVersion `protobuf:"bytes,2,rep,name=Snapshots,proto3" json:"Snapshots,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DestroySnapshotsReq) Reset() { *m = DestroySnapshotsReq{} } +func (m *DestroySnapshotsReq) String() string { return proto.CompactTextString(m) } +func (*DestroySnapshotsReq) ProtoMessage() {} +func (*DestroySnapshotsReq) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{11} +} +func (m *DestroySnapshotsReq) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_DestroySnapshotsReq.Unmarshal(m, b) +} +func (m *DestroySnapshotsReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_DestroySnapshotsReq.Marshal(b, m, deterministic) +} +func (dst *DestroySnapshotsReq) XXX_Merge(src proto.Message) { + xxx_messageInfo_DestroySnapshotsReq.Merge(dst, src) +} +func (m *DestroySnapshotsReq) XXX_Size() int { + return xxx_messageInfo_DestroySnapshotsReq.Size(m) +} +func (m *DestroySnapshotsReq) XXX_DiscardUnknown() { + xxx_messageInfo_DestroySnapshotsReq.DiscardUnknown(m) +} + +var xxx_messageInfo_DestroySnapshotsReq proto.InternalMessageInfo + +func (m *DestroySnapshotsReq) GetFilesystem() string { + if m != nil { + return m.Filesystem + } + return "" +} + +func (m *DestroySnapshotsReq) GetSnapshots() []*FilesystemVersion { + if m != nil { + return m.Snapshots + } + return nil +} + +type DestroySnapshotRes struct { + Snapshot *FilesystemVersion `protobuf:"bytes,1,opt,name=Snapshot,proto3" json:"Snapshot,omitempty"` + Error string `protobuf:"bytes,2,opt,name=Error,proto3" json:"Error,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DestroySnapshotRes) Reset() { *m = DestroySnapshotRes{} } +func (m *DestroySnapshotRes) String() string { return proto.CompactTextString(m) } +func (*DestroySnapshotRes) ProtoMessage() {} +func (*DestroySnapshotRes) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{12} +} +func (m *DestroySnapshotRes) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_DestroySnapshotRes.Unmarshal(m, b) +} +func (m *DestroySnapshotRes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_DestroySnapshotRes.Marshal(b, m, deterministic) +} +func (dst *DestroySnapshotRes) XXX_Merge(src proto.Message) { + xxx_messageInfo_DestroySnapshotRes.Merge(dst, src) +} +func (m *DestroySnapshotRes) XXX_Size() int { + return xxx_messageInfo_DestroySnapshotRes.Size(m) +} +func (m *DestroySnapshotRes) XXX_DiscardUnknown() { + xxx_messageInfo_DestroySnapshotRes.DiscardUnknown(m) +} + +var xxx_messageInfo_DestroySnapshotRes proto.InternalMessageInfo + +func (m *DestroySnapshotRes) GetSnapshot() *FilesystemVersion { + if m != nil { + return m.Snapshot + } + return nil +} + +func (m *DestroySnapshotRes) GetError() string { + if m != nil { + return m.Error + } + return "" +} + +type DestroySnapshotsRes struct { + Results []*DestroySnapshotRes `protobuf:"bytes,1,rep,name=Results,proto3" json:"Results,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DestroySnapshotsRes) Reset() { *m = DestroySnapshotsRes{} } +func (m *DestroySnapshotsRes) String() string { return proto.CompactTextString(m) } +func (*DestroySnapshotsRes) ProtoMessage() {} +func (*DestroySnapshotsRes) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{13} +} +func (m *DestroySnapshotsRes) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_DestroySnapshotsRes.Unmarshal(m, b) +} +func (m *DestroySnapshotsRes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_DestroySnapshotsRes.Marshal(b, m, deterministic) +} +func (dst *DestroySnapshotsRes) XXX_Merge(src proto.Message) { + xxx_messageInfo_DestroySnapshotsRes.Merge(dst, src) +} +func (m *DestroySnapshotsRes) XXX_Size() int { + return xxx_messageInfo_DestroySnapshotsRes.Size(m) +} +func (m *DestroySnapshotsRes) XXX_DiscardUnknown() { + xxx_messageInfo_DestroySnapshotsRes.DiscardUnknown(m) +} + +var xxx_messageInfo_DestroySnapshotsRes proto.InternalMessageInfo + +func (m *DestroySnapshotsRes) GetResults() []*DestroySnapshotRes { + if m != nil { + return m.Results + } + return nil +} + +type SnapshotReplicationStatusReq struct { + Filesystem string `protobuf:"bytes,1,opt,name=Filesystem,proto3" json:"Filesystem,omitempty"` + Snapshot string `protobuf:"bytes,2,opt,name=Snapshot,proto3" json:"Snapshot,omitempty"` + Op SnapshotReplicationStatusReq_Op `protobuf:"varint,3,opt,name=op,proto3,enum=pdu.SnapshotReplicationStatusReq_Op" json:"op,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SnapshotReplicationStatusReq) Reset() { *m = SnapshotReplicationStatusReq{} } +func (m *SnapshotReplicationStatusReq) String() string { return proto.CompactTextString(m) } +func (*SnapshotReplicationStatusReq) ProtoMessage() {} +func (*SnapshotReplicationStatusReq) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{14} +} +func (m *SnapshotReplicationStatusReq) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SnapshotReplicationStatusReq.Unmarshal(m, b) +} +func (m *SnapshotReplicationStatusReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SnapshotReplicationStatusReq.Marshal(b, m, deterministic) +} +func (dst *SnapshotReplicationStatusReq) XXX_Merge(src proto.Message) { + xxx_messageInfo_SnapshotReplicationStatusReq.Merge(dst, src) +} +func (m *SnapshotReplicationStatusReq) XXX_Size() int { + return xxx_messageInfo_SnapshotReplicationStatusReq.Size(m) +} +func (m *SnapshotReplicationStatusReq) XXX_DiscardUnknown() { + xxx_messageInfo_SnapshotReplicationStatusReq.DiscardUnknown(m) +} + +var xxx_messageInfo_SnapshotReplicationStatusReq proto.InternalMessageInfo + +func (m *SnapshotReplicationStatusReq) GetFilesystem() string { + if m != nil { + return m.Filesystem + } + return "" +} + +func (m *SnapshotReplicationStatusReq) GetSnapshot() string { + if m != nil { + return m.Snapshot + } + return "" +} + +func (m *SnapshotReplicationStatusReq) GetOp() SnapshotReplicationStatusReq_Op { + if m != nil { + return m.Op + } + return SnapshotReplicationStatusReq_Get +} + +type SnapshotReplicationStatusRes struct { + Replicated bool `protobuf:"varint,1,opt,name=Replicated,proto3" json:"Replicated,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SnapshotReplicationStatusRes) Reset() { *m = SnapshotReplicationStatusRes{} } +func (m *SnapshotReplicationStatusRes) String() string { return proto.CompactTextString(m) } +func (*SnapshotReplicationStatusRes) ProtoMessage() {} +func (*SnapshotReplicationStatusRes) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{15} +} +func (m *SnapshotReplicationStatusRes) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SnapshotReplicationStatusRes.Unmarshal(m, b) +} +func (m *SnapshotReplicationStatusRes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SnapshotReplicationStatusRes.Marshal(b, m, deterministic) +} +func (dst *SnapshotReplicationStatusRes) XXX_Merge(src proto.Message) { + xxx_messageInfo_SnapshotReplicationStatusRes.Merge(dst, src) +} +func (m *SnapshotReplicationStatusRes) XXX_Size() int { + return xxx_messageInfo_SnapshotReplicationStatusRes.Size(m) +} +func (m *SnapshotReplicationStatusRes) XXX_DiscardUnknown() { + xxx_messageInfo_SnapshotReplicationStatusRes.DiscardUnknown(m) +} + +var xxx_messageInfo_SnapshotReplicationStatusRes proto.InternalMessageInfo + +func (m *SnapshotReplicationStatusRes) GetReplicated() bool { + if m != nil { + return m.Replicated + } + return false +} + func init() { proto.RegisterType((*ListFilesystemReq)(nil), "pdu.ListFilesystemReq") proto.RegisterType((*ListFilesystemRes)(nil), "pdu.ListFilesystemRes") @@ -569,39 +815,55 @@ func init() { proto.RegisterType((*SendRes)(nil), "pdu.SendRes") proto.RegisterType((*ReceiveReq)(nil), "pdu.ReceiveReq") proto.RegisterType((*ReceiveRes)(nil), "pdu.ReceiveRes") + proto.RegisterType((*DestroySnapshotsReq)(nil), "pdu.DestroySnapshotsReq") + proto.RegisterType((*DestroySnapshotRes)(nil), "pdu.DestroySnapshotRes") + proto.RegisterType((*DestroySnapshotsRes)(nil), "pdu.DestroySnapshotsRes") + proto.RegisterType((*SnapshotReplicationStatusReq)(nil), "pdu.SnapshotReplicationStatusReq") + proto.RegisterType((*SnapshotReplicationStatusRes)(nil), "pdu.SnapshotReplicationStatusRes") proto.RegisterEnum("pdu.FilesystemVersion_VersionType", FilesystemVersion_VersionType_name, FilesystemVersion_VersionType_value) + proto.RegisterEnum("pdu.SnapshotReplicationStatusReq_Op", SnapshotReplicationStatusReq_Op_name, SnapshotReplicationStatusReq_Op_value) } -func init() { proto.RegisterFile("pdu.proto", fileDescriptor_pdu_25ca9e5977a7ce3f) } +func init() { proto.RegisterFile("pdu.proto", fileDescriptor_pdu_e1dccbd3b8cde5a3) } -var fileDescriptor_pdu_25ca9e5977a7ce3f = []byte{ - // 445 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x53, 0xc1, 0x6e, 0xd3, 0x40, - 0x10, 0x65, 0x13, 0xa7, 0x38, 0x93, 0xd2, 0xa6, 0x4b, 0x85, 0x0c, 0x42, 0x28, 0xda, 0x53, 0x40, - 0x22, 0x12, 0x01, 0x71, 0xe1, 0xd6, 0xa2, 0xf4, 0x82, 0xa0, 0xda, 0x9a, 0xaa, 0x57, 0x17, 0x8f, - 0x54, 0x2b, 0xb1, 0x77, 0xbb, 0x63, 0x23, 0xe5, 0x73, 0xf8, 0x2b, 0x3e, 0x07, 0x79, 0x6a, 0x27, - 0x4b, 0x0c, 0x52, 0x4e, 0x99, 0xf7, 0x66, 0x32, 0xf3, 0xe6, 0xcd, 0x1a, 0x86, 0x36, 0xad, 0x66, - 0xd6, 0x99, 0xd2, 0xc8, 0xbe, 0x4d, 0x2b, 0xf5, 0x14, 0x4e, 0xbe, 0x64, 0x54, 0x2e, 0xb2, 0x15, - 0xd2, 0x9a, 0x4a, 0xcc, 0x35, 0xde, 0xab, 0x45, 0x97, 0x24, 0xf9, 0x0e, 0x46, 0x5b, 0x82, 0x22, - 0x31, 0xe9, 0x4f, 0x47, 0xf3, 0xe3, 0x59, 0xdd, 0xcf, 0x2b, 0xf4, 0x6b, 0xd4, 0x19, 0xc0, 0x16, - 0x4a, 0x09, 0xc1, 0x65, 0x52, 0xde, 0x45, 0x62, 0x22, 0xa6, 0x43, 0xcd, 0xb1, 0x9c, 0xc0, 0x48, - 0x23, 0x55, 0x39, 0xc6, 0x66, 0x89, 0x45, 0xd4, 0xe3, 0x94, 0x4f, 0xa9, 0x4f, 0xf0, 0xfc, 0x6f, - 0x2d, 0xd7, 0xe8, 0x28, 0x33, 0x05, 0x69, 0xbc, 0x97, 0xaf, 0xfc, 0x01, 0x4d, 0x63, 0x8f, 0x51, - 0xdf, 0xfe, 0xff, 0x67, 0x92, 0x73, 0x08, 0x5b, 0xd8, 0x6c, 0xf3, 0x6c, 0x67, 0x9b, 0x26, 0xad, - 0x37, 0x75, 0xea, 0xb7, 0x80, 0x93, 0x4e, 0x5e, 0x7e, 0x84, 0x20, 0x5e, 0x5b, 0x64, 0x01, 0x47, - 0x73, 0xf5, 0xef, 0x2e, 0xb3, 0xe6, 0xb7, 0xae, 0xd4, 0x5c, 0x5f, 0x3b, 0xf2, 0x35, 0xc9, 0xb1, - 0x59, 0x9b, 0xe3, 0x9a, 0xbb, 0xa8, 0xb2, 0x34, 0xea, 0x4f, 0xc4, 0x34, 0xd0, 0x1c, 0xcb, 0x97, - 0x30, 0x3c, 0x77, 0x98, 0x94, 0x18, 0xdf, 0x5c, 0x44, 0x01, 0x27, 0xb6, 0x84, 0x7c, 0x01, 0x21, - 0x83, 0xcc, 0x14, 0xd1, 0x80, 0x3b, 0x6d, 0xb0, 0x7a, 0x0d, 0x23, 0x6f, 0xac, 0x3c, 0x84, 0xf0, - 0xaa, 0x48, 0x2c, 0xdd, 0x99, 0x72, 0xfc, 0xa8, 0x46, 0x67, 0xc6, 0x2c, 0xf3, 0xc4, 0x2d, 0xc7, - 0x42, 0xfd, 0x12, 0xf0, 0xf8, 0x0a, 0x8b, 0x74, 0x0f, 0x5f, 0x6b, 0x91, 0x0b, 0x67, 0xf2, 0x56, - 0x78, 0x1d, 0xcb, 0x23, 0xe8, 0xc5, 0x86, 0x65, 0x0f, 0x75, 0x2f, 0x36, 0xbb, 0xa7, 0x0d, 0x3a, - 0xa7, 0x65, 0xe1, 0x26, 0xb7, 0x0e, 0x89, 0x58, 0x78, 0xa8, 0x37, 0x58, 0x9e, 0xc2, 0xe0, 0x33, - 0xa6, 0x95, 0x8d, 0x0e, 0x38, 0xf1, 0x00, 0xd4, 0x07, 0x08, 0x2f, 0x9d, 0xb1, 0xe8, 0xca, 0xf5, - 0xc6, 0x3c, 0xe1, 0x99, 0x77, 0x0a, 0x83, 0xeb, 0x64, 0x55, 0xb5, 0x8e, 0x3e, 0x00, 0x75, 0xdb, - 0x2e, 0x46, 0x72, 0x0a, 0xc7, 0xdf, 0x09, 0x53, 0x5f, 0x98, 0xe0, 0x01, 0xbb, 0xb4, 0x7c, 0x0b, - 0xd0, 0x8c, 0xca, 0x90, 0xa2, 0x1e, 0xbf, 0x8f, 0x27, 0x7c, 0xd9, 0x56, 0x81, 0xf6, 0x0a, 0xd4, - 0x0d, 0x80, 0xc6, 0x1f, 0x98, 0xfd, 0xc4, 0x7d, 0xfc, 0x7b, 0x03, 0xe3, 0xf3, 0x15, 0x26, 0x6e, - 0xf7, 0xed, 0x87, 0xba, 0xc3, 0xab, 0x43, 0xaf, 0x33, 0xdd, 0x1e, 0xf0, 0xb7, 0xfb, 0xfe, 0x4f, - 0x00, 0x00, 0x00, 0xff, 0xff, 0xfd, 0xc4, 0x8d, 0xb7, 0xc8, 0x03, 0x00, 0x00, +var fileDescriptor_pdu_e1dccbd3b8cde5a3 = []byte{ + // 595 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x54, 0xdd, 0x6e, 0x12, 0x41, + 0x14, 0x76, 0x17, 0x5a, 0x96, 0x43, 0x4b, 0x61, 0xda, 0xe8, 0xda, 0x34, 0x86, 0x4c, 0xbc, 0x40, + 0x13, 0x49, 0x44, 0xe2, 0x8d, 0x89, 0x17, 0x6d, 0x05, 0x2f, 0x8c, 0x34, 0x03, 0x36, 0xbd, 0x32, + 0xd9, 0x76, 0x4f, 0xd2, 0x0d, 0x2c, 0x33, 0x9d, 0xd9, 0x35, 0xe1, 0x71, 0x7c, 0x05, 0x9f, 0xc6, + 0xc7, 0x31, 0x33, 0xec, 0xcf, 0x08, 0x2d, 0xe1, 0x8a, 0xf9, 0xce, 0xf9, 0x38, 0xe7, 0x3b, 0x7f, + 0x0b, 0x75, 0x11, 0xa6, 0x3d, 0x21, 0x79, 0xc2, 0x49, 0x45, 0x84, 0x29, 0x3d, 0x86, 0xf6, 0xb7, + 0x48, 0x25, 0xc3, 0x68, 0x8e, 0x6a, 0xa9, 0x12, 0x8c, 0x19, 0x3e, 0xd0, 0xe1, 0xa6, 0x51, 0x91, + 0xf7, 0xd0, 0x28, 0x0d, 0xca, 0x77, 0x3a, 0x95, 0x6e, 0xa3, 0x7f, 0xd4, 0xd3, 0xf1, 0x2c, 0xa2, + 0xcd, 0xa1, 0xe7, 0x00, 0x25, 0x24, 0x04, 0xaa, 0x57, 0x41, 0x72, 0xef, 0x3b, 0x1d, 0xa7, 0x5b, + 0x67, 0xe6, 0x4d, 0x3a, 0xd0, 0x60, 0xa8, 0xd2, 0x18, 0xa7, 0x7c, 0x86, 0x0b, 0xdf, 0x35, 0x2e, + 0xdb, 0x44, 0x3f, 0xc1, 0xcb, 0xff, 0xb5, 0x5c, 0xa3, 0x54, 0x11, 0x5f, 0x28, 0x86, 0x0f, 0xe4, + 0x95, 0x9d, 0x20, 0x0b, 0x6c, 0x59, 0xe8, 0xf8, 0xe9, 0x3f, 0x2b, 0xd2, 0x07, 0x2f, 0x87, 0x59, + 0x35, 0xcf, 0xd7, 0xaa, 0xc9, 0xdc, 0xac, 0xe0, 0xd1, 0xbf, 0x0e, 0xb4, 0x37, 0xfc, 0xe4, 0x23, + 0x54, 0xa7, 0x4b, 0x81, 0x46, 0x40, 0xb3, 0x4f, 0x1f, 0x8f, 0xd2, 0xcb, 0x7e, 0x35, 0x93, 0x19, + 0xbe, 0xee, 0xc8, 0xf7, 0x20, 0xc6, 0xac, 0x6c, 0xf3, 0xd6, 0xb6, 0x51, 0x1a, 0x85, 0x7e, 0xa5, + 0xe3, 0x74, 0xab, 0xcc, 0xbc, 0xc9, 0x19, 0xd4, 0x2f, 0x24, 0x06, 0x09, 0x4e, 0x6f, 0x46, 0x7e, + 0xd5, 0x38, 0x4a, 0x03, 0x39, 0x05, 0xcf, 0x80, 0x88, 0x2f, 0xfc, 0x3d, 0x13, 0xa9, 0xc0, 0xf4, + 0x0d, 0x34, 0xac, 0xb4, 0xe4, 0x00, 0xbc, 0xc9, 0x22, 0x10, 0xea, 0x9e, 0x27, 0xad, 0x67, 0x1a, + 0x9d, 0x73, 0x3e, 0x8b, 0x03, 0x39, 0x6b, 0x39, 0xf4, 0xb7, 0x03, 0xb5, 0x09, 0x2e, 0xc2, 0x1d, + 0xfa, 0xaa, 0x45, 0x0e, 0x25, 0x8f, 0x73, 0xe1, 0xfa, 0x4d, 0x9a, 0xe0, 0x4e, 0xb9, 0x91, 0x5d, + 0x67, 0xee, 0x94, 0xaf, 0x8f, 0xb6, 0xba, 0x31, 0x5a, 0x23, 0x9c, 0xc7, 0x42, 0xa2, 0x52, 0x46, + 0xb8, 0xc7, 0x0a, 0x4c, 0x4e, 0x60, 0xef, 0x12, 0xc3, 0x54, 0xf8, 0xfb, 0xc6, 0xb1, 0x02, 0x74, + 0x00, 0xde, 0x95, 0xe4, 0x02, 0x65, 0xb2, 0x2c, 0x9a, 0xe7, 0x58, 0xcd, 0x3b, 0x81, 0xbd, 0xeb, + 0x60, 0x9e, 0xe6, 0x1d, 0x5d, 0x01, 0x7a, 0x9b, 0x17, 0xa6, 0x48, 0x17, 0x8e, 0x7e, 0x28, 0x0c, + 0x6d, 0x61, 0x8e, 0x49, 0xb0, 0x6e, 0x26, 0xef, 0x00, 0xb2, 0x54, 0x11, 0x2a, 0xdf, 0x35, 0xfb, + 0x71, 0x68, 0x26, 0x9b, 0x2b, 0x60, 0x16, 0x81, 0xde, 0x00, 0x30, 0xbc, 0xc3, 0xe8, 0x17, 0xee, + 0xd2, 0xbf, 0xb7, 0xd0, 0xba, 0x98, 0x63, 0x20, 0xd7, 0x77, 0xdf, 0x63, 0x1b, 0x76, 0x7a, 0x60, + 0x45, 0x56, 0x74, 0x06, 0xc7, 0x97, 0xa8, 0x12, 0xc9, 0x97, 0xf9, 0x20, 0x77, 0x39, 0x04, 0x32, + 0x80, 0x7a, 0xc1, 0xcf, 0x8a, 0x79, 0x6a, 0xd9, 0x4b, 0x22, 0xfd, 0x09, 0x64, 0x2d, 0x59, 0x76, + 0x37, 0x39, 0x34, 0x99, 0xb6, 0xdc, 0x4d, 0xce, 0xd3, 0x83, 0xf9, 0x22, 0x25, 0x97, 0xf9, 0x60, + 0x0c, 0xa0, 0x5f, 0x1f, 0x2b, 0x46, 0x7f, 0x69, 0x6a, 0xba, 0x01, 0xf3, 0x24, 0xbf, 0xcb, 0x17, + 0x26, 0xfe, 0xa6, 0x14, 0x96, 0xf3, 0xe8, 0x1f, 0x07, 0xce, 0x4a, 0x87, 0x98, 0x47, 0x77, 0x66, + 0xff, 0x27, 0x49, 0x90, 0xa4, 0x3b, 0x35, 0xe8, 0xd4, 0x2a, 0x6a, 0xa5, 0xb1, 0x14, 0x3f, 0x00, + 0x97, 0x0b, 0xb3, 0xd9, 0xcd, 0xfe, 0x6b, 0x23, 0x65, 0x5b, 0xaa, 0xde, 0x58, 0x30, 0x97, 0x0b, + 0xda, 0x01, 0x77, 0x2c, 0x48, 0x0d, 0x2a, 0x23, 0xd4, 0xc7, 0xd6, 0x86, 0xc3, 0x09, 0x16, 0x7f, + 0xc0, 0xb0, 0xe5, 0xd0, 0xcf, 0x5b, 0x35, 0x2b, 0xad, 0xb9, 0xe4, 0x67, 0x7b, 0x6a, 0x59, 0x6e, + 0xf7, 0xcd, 0x77, 0xfc, 0xc3, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xf6, 0x1a, 0xe7, 0xf6, 0xd4, + 0x05, 0x00, 0x00, } diff --git a/replication/pdu/pdu.proto b/replication/pdu/pdu.proto index cac47b1..0d1103c 100644 --- a/replication/pdu/pdu.proto +++ b/replication/pdu/pdu.proto @@ -76,3 +76,32 @@ message ReceiveReq { } message ReceiveRes {} + +message DestroySnapshotsReq { + string Filesystem = 1; + // Path to filesystem, snapshot or bookmark to be destroyed + repeated FilesystemVersion Snapshots = 2; +} + +message DestroySnapshotRes { + FilesystemVersion Snapshot = 1; + string Error = 2; +} + +message DestroySnapshotsRes { + repeated DestroySnapshotRes Results = 1; +} + +message SnapshotReplicationStatusReq { + string Filesystem = 1; + string Snapshot = 2; + enum Op { + Get = 0; + SetReplicated = 1; + } + Op op = 3; +} + +message SnapshotReplicationStatusRes { + bool Replicated = 1; +} diff --git a/replication/pdu/pdu_extras.go b/replication/pdu/pdu_extras.go index 4d718a8..9cf52c9 100644 --- a/replication/pdu/pdu_extras.go +++ b/replication/pdu/pdu_extras.go @@ -7,7 +7,10 @@ import ( ) func (v *FilesystemVersion) RelName() string { - zv := v.ZFSFilesystemVersion() + zv, err := v.ZFSFilesystemVersion() + if err != nil { + panic(err) + } return zv.String() } @@ -22,7 +25,7 @@ func (v FilesystemVersion_VersionType) ZFSVersionType() zfs.VersionType { } } -func FilesystemVersionFromZFS(fsv zfs.FilesystemVersion) *FilesystemVersion { +func FilesystemVersionFromZFS(fsv *zfs.FilesystemVersion) *FilesystemVersion { var t FilesystemVersion_VersionType switch fsv.Type { case zfs.Bookmark: @@ -58,14 +61,10 @@ func (v *FilesystemVersion) SnapshotTime() time.Time { return t } -func (v *FilesystemVersion) ZFSFilesystemVersion() *zfs.FilesystemVersion { - ct := time.Time{} - if v.Creation != "" { - var err error - ct, err = time.Parse(time.RFC3339, v.Creation) - if err != nil { - panic(err) - } +func (v *FilesystemVersion) ZFSFilesystemVersion() (*zfs.FilesystemVersion, error) { + ct, err := v.CreationAsTime() + if err != nil { + return nil, err } return &zfs.FilesystemVersion{ Type: v.Type.ZFSVersionType(), @@ -73,5 +72,5 @@ func (v *FilesystemVersion) ZFSFilesystemVersion() *zfs.FilesystemVersion { Guid: v.Guid, CreateTXG: v.CreateTXG, Creation: ct, - } + }, nil } diff --git a/zfs/replication_history.go b/zfs/replication_history.go new file mode 100644 index 0000000..c650451 --- /dev/null +++ b/zfs/replication_history.go @@ -0,0 +1,25 @@ +package zfs + +const ReplicatedProperty = "zrepl:replicated" + +// May return *DatasetDoesNotExist as an error +func ZFSGetReplicatedProperty(fs *DatasetPath, v *FilesystemVersion) (replicated bool, err error) { + props, err := zfsGet(v.ToAbsPath(fs), []string{ReplicatedProperty}) + if err != nil { + return false, err + } + if props.Get(ReplicatedProperty) == "yes" { + return true, nil + } + return false, nil +} + +func ZFSSetReplicatedProperty(fs *DatasetPath, v *FilesystemVersion, replicated bool) error { + val := "no" + if replicated { + val = "yes" + } + props := NewZFSProperties() + props.Set(ReplicatedProperty, val) + return zfsSet(v.ToAbsPath(fs), props) +} diff --git a/zfs/versions.go b/zfs/versions.go index 4bf2d40..9deb10d 100644 --- a/zfs/versions.go +++ b/zfs/versions.go @@ -154,7 +154,7 @@ func ZFSListFilesystemVersions(fs *DatasetPath, filter FilesystemVersionFilter) return } -func ZFSDestroyFilesystemVersion(filesystem *DatasetPath, version FilesystemVersion) (err error) { +func ZFSDestroyFilesystemVersion(filesystem *DatasetPath, version *FilesystemVersion) (err error) { promTimer := prometheus.NewTimer(prom.ZFSDestroyFilesystemVersionDuration.WithLabelValues(filesystem.ToString(), version.Type.String())) defer promTimer.ObserveDuration() diff --git a/zfs/zfs.go b/zfs/zfs.go index 2b11da3..58ed6d3 100644 --- a/zfs/zfs.go +++ b/zfs/zfs.go @@ -14,6 +14,7 @@ import ( "github.com/problame/go-rwccmd" "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/util" + "regexp" ) type DatasetPath struct { @@ -397,14 +398,17 @@ func (p *ZFSProperties) appendArgs(args *[]string) (err error) { } func ZFSSet(fs *DatasetPath, props *ZFSProperties) (err error) { + return zfsSet(fs.ToString(), props) +} +func zfsSet(path string, props *ZFSProperties) (err error) { args := make([]string, 0) args = append(args, "set") err = props.appendArgs(&args) if err != nil { return err } - args = append(args, fs.ToString()) + args = append(args, path) cmd := exec.Command(ZFS_BINARY, args...) @@ -426,11 +430,32 @@ func ZFSSet(fs *DatasetPath, props *ZFSProperties) (err error) { } func ZFSGet(fs *DatasetPath, props []string) (*ZFSProperties, error) { - args := []string{"get", "-Hp", "-o", "property,value", strings.Join(props, ","), fs.ToString()} + return zfsGet(fs.ToString(), props) +} +var zfsGetDatasetDoesNotExistRegexp = regexp.MustCompile(`^cannot open '(\s+)': dataset does not exist`) + +type DatasetDoesNotExist struct { + Path string +} + +func (d *DatasetDoesNotExist) Error() string { return fmt.Sprintf("dataset %q does not exist", d.Path) } + +func zfsGet(path string, props []string) (*ZFSProperties, error) { + args := []string{"get", "-Hp", "-o", "property,value", strings.Join(props, ","), path} cmd := exec.Command(ZFS_BINARY, args...) output, err := cmd.CombinedOutput() if err != nil { + if exitErr, ok := err.(*exec.ExitError); ok { + if exitErr.Exited() { + // screen-scrape output + if sm := zfsGetDatasetDoesNotExistRegexp.FindSubmatch(output); sm != nil { + if string(sm[1]) == path { + return nil, &DatasetDoesNotExist{path} + } + } + } + } return nil, err } o := string(output) From 7dd49b835a797236de9a1ccef6b64f82f1fd2369 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 30 Aug 2018 11:52:05 +0200 Subject: [PATCH 068/167] finish pruning implementation in push job --- daemon/job/push.go | 26 ++++++++++++++++++-------- 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/daemon/job/push.go b/daemon/job/push.go index a292847..8802c27 100644 --- a/daemon/job/push.go +++ b/daemon/job/push.go @@ -8,11 +8,12 @@ import ( "github.com/zrepl/zrepl/daemon/connecter" "github.com/zrepl/zrepl/daemon/filters" "github.com/zrepl/zrepl/daemon/logging" + "github.com/zrepl/zrepl/daemon/pruner" "github.com/zrepl/zrepl/endpoint" + "github.com/zrepl/zrepl/pruning" "github.com/zrepl/zrepl/replication" "sync" - "github.com/zrepl/zrepl/daemon/pruner" - "github.com/zrepl/zrepl/pruning" + "time" ) type Push struct { @@ -20,7 +21,7 @@ type Push struct { connecter streamrpc.Connecter fsfilter endpoint.FSFilter - keepRulesSender []pruning.KeepRule + keepRulesSender []pruning.KeepRule keepRulesReceiver []pruning.KeepRule mtx sync.Mutex @@ -38,6 +39,16 @@ func PushFromConfig(g config.Global, in *config.PushJob) (j *Push, err error) { return nil, errors.Wrap(err, "cannnot build filesystem filter") } + j.keepRulesReceiver, err = pruning.RulesFromConfig(in.Pruning.KeepReceiver) + if err != nil { + return nil, errors.Wrap(err, "cannot build receiver pruning rules") + } + + j.keepRulesSender, err = pruning.RulesFromConfig(in.Pruning.KeepSender) + if err != nil { + return nil, errors.Wrap(err, "cannot build sender pruning rules") + } + return j, nil } @@ -90,12 +101,11 @@ func (j *Push) do(ctx context.Context) { rep.Drive(ctx, sender, receiver) // Prune sender - senderPruner := pruner.NewPruner(sender, receiver, j.keepRulesSender) - senderPruner.Prune(ctx) + senderPruner := pruner.NewPruner(10*time.Second, sender, sender, j.keepRulesSender) // FIXME constant + senderPruner.Prune(pruner.WithLogger(ctx, pruner.GetLogger(ctx).WithField("prune_side", "sender"))) // Prune receiver - receiverPruner := pruner.NewPruner(receiver, receiver, j.keepRulesReceiver) - receiverPruner.Prune(ctx) + receiverPruner := pruner.NewPruner(10*time.Second, receiver, sender, j.keepRulesReceiver) // FIXME constant + receiverPruner.Prune(pruner.WithLogger(ctx, pruner.GetLogger(ctx).WithField("prune_side", "receiver"))) } - From e4958248345a283d29418708c1111a24a94dedd3 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Wed, 29 Aug 2018 19:17:45 +0200 Subject: [PATCH 069/167] move wakeup to client package and extract http client --- wakeup.go => client/jsonclient.go | 38 +++++++------------------------ client/wakeup.go | 28 +++++++++++++++++++++++ 2 files changed, 36 insertions(+), 30 deletions(-) rename wakeup.go => client/jsonclient.go (66%) create mode 100644 client/wakeup.go diff --git a/wakeup.go b/client/jsonclient.go similarity index 66% rename from wakeup.go rename to client/jsonclient.go index df389de..144f9b6 100644 --- a/wakeup.go +++ b/client/jsonclient.go @@ -1,38 +1,15 @@ -package main +package client import ( - "bytes" - "context" - "encoding/json" - "github.com/pkg/errors" - "github.com/zrepl/zrepl/config" - "github.com/zrepl/zrepl/daemon" - "io" - "net" "net/http" + "net" + "context" + "bytes" + "encoding/json" + "io" + "github.com/pkg/errors" ) -func RunWakeup(config config.Config, args []string) error { - if len(args) != 1 { - return errors.Errorf("Expected 1 argument: job") - } - - httpc, err := controlHttpClient(config.Global.Control.SockPath) - if err != nil { - return err - } - - err = jsonRequestResponse(httpc, daemon.ControlJobEndpointWakeup, - struct { - Name string - }{ - Name: args[0], - }, - struct{}{}, - ) - return err -} - func controlHttpClient(sockpath string) (client http.Client, err error) { return http.Client{ Transport: &http.Transport{ @@ -66,3 +43,4 @@ func jsonRequestResponse(c http.Client, endpoint string, req interface{}, res in return nil } + diff --git a/client/wakeup.go b/client/wakeup.go new file mode 100644 index 0000000..97fb180 --- /dev/null +++ b/client/wakeup.go @@ -0,0 +1,28 @@ +package client + +import ( + "github.com/pkg/errors" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon" +) + +func RunWakeup(config config.Config, args []string) error { + if len(args) != 1 { + return errors.Errorf("Expected 1 argument: job") + } + + httpc, err := controlHttpClient(config.Global.Control.SockPath) + if err != nil { + return err + } + + err = jsonRequestResponse(httpc, daemon.ControlJobEndpointWakeup, + struct { + Name string + }{ + Name: args[0], + }, + struct{}{}, + ) + return err +} From 6cedd0a2e8f0202a1fe76002590c70295a1e6f3b Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Wed, 29 Aug 2018 19:18:54 +0200 Subject: [PATCH 070/167] add status command --- Gopkg.lock | 17 ++++ Gopkg.toml | 4 + client/status.go | 227 +++++++++++++++++++++++++++++++++++++++++++++ daemon/job/push.go | 17 +++- main.go | 16 +++- 5 files changed, 277 insertions(+), 4 deletions(-) create mode 100644 client/status.go diff --git a/Gopkg.lock b/Gopkg.lock index 6d9e63c..5e7387e 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -89,6 +89,14 @@ revision = "0360b2af4f38e8d38c7fce2a9f4e702702d73a39" version = "v0.0.3" +[[projects]] + digest = "1:82b912465c1da0668582a7d1117339c278e786c2536b3c3623029a0c7141c2d0" + name = "github.com/mattn/go-runewidth" + packages = ["."] + pruneopts = "" + revision = "ce7b0b5c7b45a81508558cd1dba6bb1e4ddb51bb" + version = "v0.0.3" + [[projects]] digest = "1:4c23ced97a470b17d9ffd788310502a077b9c1f60221a85563e49696276b4147" name = "github.com/matttproud/golang_protobuf_extensions" @@ -105,6 +113,14 @@ pruneopts = "" revision = "d0303fe809921458f417bcf828397a65db30a7e4" +[[projects]] + branch = "master" + digest = "1:20a553eff588d7abe1f05addf5f57cdbaef1d0f992427a0099b7eb51274b79cf" + name = "github.com/nsf/termbox-go" + packages = ["."] + pruneopts = "" + revision = "b66b20ab708e289ff1eb3e218478302e6aec28ce" + [[projects]] digest = "1:7365acd48986e205ccb8652cc746f09c8b7876030d53710ea6ef7d0bd0dcd7ca" name = "github.com/pkg/errors" @@ -246,6 +262,7 @@ "github.com/kr/pretty", "github.com/mattn/go-isatty", "github.com/mitchellh/mapstructure", + "github.com/nsf/termbox-go", "github.com/pkg/errors", "github.com/problame/go-netssh", "github.com/problame/go-rwccmd", diff --git a/Gopkg.toml b/Gopkg.toml index 041e666..0e74efb 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -63,3 +63,7 @@ ignored = [ "github.com/inconshreveable/mousetrap" ] [[constraint]] name = "github.com/golang/protobuf" version = "1.2.0" + +[[constraint]] + name = "github.com/nsf/termbox-go" + branch = "master" \ No newline at end of file diff --git a/client/status.go b/client/status.go new file mode 100644 index 0000000..c9235d5 --- /dev/null +++ b/client/status.go @@ -0,0 +1,227 @@ +package client + +import ( + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon" + "fmt" + "github.com/zrepl/zrepl/replication" + "github.com/mitchellh/mapstructure" + "github.com/zrepl/zrepl/replication/fsrep" + "github.com/nsf/termbox-go" + "time" + "github.com/pkg/errors" + "sort" + "sync" +) + +type tui struct { + x, y int + indent int + + lock sync.Mutex //For report and error + report map[string]interface{} + err error +} + +func newTui() tui { + return tui{} +} + +func (t *tui) moveCursor(x, y int) { + t.x += x + t.y += y +} + +func (t *tui) moveLine(dl int, col int) { + t.y += dl + t.x = t.indent * 4 + col +} + +func (t *tui) write(text string) { + for _, c := range text { + termbox.SetCell(t.x, t.y, c, termbox.ColorDefault, termbox.ColorDefault) + t.x += 1 + } +} + +func (t *tui) printf(text string, a ...interface{}) { + t.write(fmt.Sprintf(text, a...)) +} + +func (t *tui) newline() { + t.moveLine(1, 0) +} + +func (t *tui) setIndent(indent int) { + t.indent = indent + t.moveLine(0, 0) +} + +func (t *tui) addIndent(indent int) { + t.indent += indent + t.moveLine(0, 0) +} + + +func RunStatus(config config.Config, args []string) error { + httpc, err := controlHttpClient(config.Global.Control.SockPath) + if err != nil { + return err + } + + t := newTui() + t.lock.Lock() + t.err = errors.New("Got no report yet") + t.lock.Unlock() + + err = termbox.Init() + if err != nil { + return err + } + defer termbox.Close() + + ticker := time.NewTicker(500 * time.Millisecond) + defer ticker.Stop() + go func() { + for _ = range ticker.C { + m := make(map[string]interface{}) + + err2 := jsonRequestResponse(httpc, daemon.ControlJobEndpointStatus, + struct {}{}, + &m, + ) + + t.lock.Lock() + t.err = err2 + t.report = m + t.lock.Unlock() + t.draw() + } + }() + + termbox.HideCursor() + termbox.Clear(termbox.ColorDefault, termbox.ColorDefault) + + loop: + for { + switch ev := termbox.PollEvent(); ev.Type { + case termbox.EventKey: + switch ev.Key { + case termbox.KeyEsc: + break loop + case termbox.KeyCtrlC: + break loop + } + case termbox.EventResize: + t.draw() + } + } + + return nil + +} + +func (t *tui) draw() { + t.lock.Lock() + defer t.lock.Unlock() + + termbox.Clear(termbox.ColorDefault, termbox.ColorDefault) + t.x = 0 + t.y = 0 + t.indent = 0 + + if t.err != nil { + t.write(t.err.Error()) + } else { + //Iterate over map in alphabetical order + keys := make([]string, len(t.report)) + i := 0 + for k, _ := range t.report { + keys[i] = k + i++ + } + sort.Strings(keys) + + for _, k := range keys { + v := t.report[k] + if len(k) == 0 || k[0] == '_' { //Internal job + continue + } + t.setIndent(0) + + t.printf("Job: %s", k) + t.setIndent(1) + t.newline() + + if v == nil { + t.printf("No report generated yet") + t.newline() + continue + } + rep := replication.Report{} + err := mapstructure.Decode(v, &rep) + if err != nil { + t.printf("Failed to decode report: %s", err.Error()) + t.newline() + continue + } + t.printf("Status: %s", rep.Status) + t.newline() + t.printf("Problem: %s", rep.Problem) + t.newline() + + for _, fs := range rep.Completed { + printFilesystem(fs, t) + } + if rep.Active != nil { + printFilesystem(rep.Active, t) + } + for _, fs := range rep.Pending { + printFilesystem(fs, t) + } + + } + } + termbox.Flush() +} + +func times(str string, n int) (out string) { + for i := 0; i < n; i++ { + out += str + } + return +} + +func rightPad(str string, length int, pad string) string { + return str + times(pad, length-len(str)) +} + +func (t *tui) drawBar(name string, status string, total int, done int) { + t.write(rightPad(name, 20, " ")) + t.write(" ") + t.write(rightPad(status, 20, " ")) + + if total > 0 { + length := 50 + completedLength := length * done / total + + //FIXME finished bar has 1 off size compared to not finished bar + t.write(times("=", completedLength-1)) + t.write(">") + t.write(times("-", length-completedLength)) + + t.printf(" %d/%d", done, total) + } + + t.newline() +} + +func printFilesystem(rep *fsrep.Report, t *tui) { + t.drawBar(rep.Filesystem, rep.Status, len(rep.Completed) + len(rep.Pending), len(rep.Completed)) + if (rep.Problem != "") { + t.addIndent(1) + t.printf("Problem: %s", rep.Problem) + t.newline() + t.addIndent(-1) + } +} \ No newline at end of file diff --git a/daemon/job/push.go b/daemon/job/push.go index 8802c27..97746f7 100644 --- a/daemon/job/push.go +++ b/daemon/job/push.go @@ -55,7 +55,18 @@ func PushFromConfig(g config.Global, in *config.PushJob) (j *Push, err error) { func (j *Push) Name() string { return j.name } func (j *Push) Status() interface{} { - return nil // FIXME + rep := func() *replication.Replication { + j.mtx.Lock() + defer j.mtx.Unlock() + if j.replication == nil { + return nil + } + return j.replication + }() + if rep == nil { + return nil + } + return rep.Report() } func (j *Push) Run(ctx context.Context) { @@ -94,11 +105,11 @@ func (j *Push) do(ctx context.Context) { receiver := endpoint.NewRemote(client) j.mtx.Lock() - rep := replication.NewReplication() + j.replication = replication.NewReplication() j.mtx.Unlock() ctx = logging.WithSubsystemLoggers(ctx, log) - rep.Drive(ctx, sender, receiver) + j.replication.Drive(ctx, sender, receiver) // Prune sender senderPruner := pruner.NewPruner(10*time.Second, sender, sender, j.keepRulesSender) // FIXME constant diff --git a/main.go b/main.go index 251ffe2..38f7138 100644 --- a/main.go +++ b/main.go @@ -5,6 +5,7 @@ import ( "github.com/spf13/cobra" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon" + "github.com/zrepl/zrepl/client" "log" "os" ) @@ -40,7 +41,19 @@ var wakeupCmd = &cobra.Command{ if err != nil { return err } - return RunWakeup(conf, args) + return client.RunWakeup(conf, args) + }, +} + +var statusCmd = &cobra.Command{ + Use: "status", + Short: "status", + RunE: func(cmd *cobra.Command, args []string) error { + conf, err := config.ParseConfig(rootArgs.configFile) + if err != nil { + return err + } + return client.RunStatus(conf, args) }, } @@ -53,6 +66,7 @@ func init() { rootCmd.PersistentFlags().StringVar(&rootArgs.configFile, "config", "", "config file path") rootCmd.AddCommand(daemonCmd) rootCmd.AddCommand(wakeupCmd) + rootCmd.AddCommand(statusCmd) } func main() { From b2f01e454fd1aa8b4beae0caba1fbf1077cd94e6 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Wed, 29 Aug 2018 19:19:32 +0200 Subject: [PATCH 071/167] bug in ZFSListFilesystemVersions? --- zfs/versions.go | 1 + 1 file changed, 1 insertion(+) diff --git a/zfs/versions.go b/zfs/versions.go index 9deb10d..daf5bdc 100644 --- a/zfs/versions.go +++ b/zfs/versions.go @@ -87,6 +87,7 @@ type FilesystemVersionFilter interface { Filter(t VersionType, name string) (accept bool, err error) } +//FIXME Seems to always forget first snapshot func ZFSListFilesystemVersions(fs *DatasetPath, filter FilesystemVersionFilter) (res []FilesystemVersion, err error) { listResults := make(chan ZFSListResult) From 42056f7a321a73c8e9346864c3a9c55f0932797a Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Wed, 29 Aug 2018 19:25:34 +0200 Subject: [PATCH 072/167] status: do not show problem field when none exists --- client/status.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/client/status.go b/client/status.go index c9235d5..62cd7bc 100644 --- a/client/status.go +++ b/client/status.go @@ -167,8 +167,10 @@ func (t *tui) draw() { } t.printf("Status: %s", rep.Status) t.newline() - t.printf("Problem: %s", rep.Problem) - t.newline() + if (rep.Problem != "") { + t.printf("Problem: %s", rep.Problem) + t.newline() + } for _, fs := range rep.Completed { printFilesystem(fs, t) From 6ca11a73916db15483c73d2416fe2e04d4c975cb Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Wed, 29 Aug 2018 22:06:24 +0200 Subject: [PATCH 073/167] byte counter for status --- client/jsonclient.go | 9 ++-- client/status.go | 86 +++++++++++++++++++++++++------------- main.go | 2 +- replication/fsrep/fsfsm.go | 23 +++++++--- util/io.go | 26 ++++++++++++ 5 files changed, 104 insertions(+), 42 deletions(-) diff --git a/client/jsonclient.go b/client/jsonclient.go index 144f9b6..43bc903 100644 --- a/client/jsonclient.go +++ b/client/jsonclient.go @@ -1,13 +1,13 @@ package client import ( - "net/http" - "net" - "context" "bytes" + "context" "encoding/json" - "io" "github.com/pkg/errors" + "io" + "net" + "net/http" ) func controlHttpClient(sockpath string) (client http.Client, err error) { @@ -43,4 +43,3 @@ func jsonRequestResponse(c http.Client, endpoint string, req interface{}, res in return nil } - diff --git a/client/status.go b/client/status.go index 62cd7bc..4824f2d 100644 --- a/client/status.go +++ b/client/status.go @@ -1,26 +1,26 @@ package client import ( + "fmt" + "github.com/mitchellh/mapstructure" + "github.com/nsf/termbox-go" + "github.com/pkg/errors" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon" - "fmt" "github.com/zrepl/zrepl/replication" - "github.com/mitchellh/mapstructure" "github.com/zrepl/zrepl/replication/fsrep" - "github.com/nsf/termbox-go" - "time" - "github.com/pkg/errors" "sort" "sync" + "time" ) type tui struct { - x, y int + x, y int indent int - lock sync.Mutex //For report and error + lock sync.Mutex //For report and error report map[string]interface{} - err error + err error } func newTui() tui { @@ -34,7 +34,7 @@ func (t *tui) moveCursor(x, y int) { func (t *tui) moveLine(dl int, col int) { t.y += dl - t.x = t.indent * 4 + col + t.x = t.indent*4 + col } func (t *tui) write(text string) { @@ -62,7 +62,6 @@ func (t *tui) addIndent(indent int) { t.moveLine(0, 0) } - func RunStatus(config config.Config, args []string) error { httpc, err := controlHttpClient(config.Global.Control.SockPath) if err != nil { @@ -80,29 +79,34 @@ func RunStatus(config config.Config, args []string) error { } defer termbox.Close() + update := func() { + m := make(map[string]interface{}) + + err2 := jsonRequestResponse(httpc, daemon.ControlJobEndpointStatus, + struct{}{}, + &m, + ) + + t.lock.Lock() + t.err = err2 + t.report = m + t.lock.Unlock() + t.draw() + } + update() + ticker := time.NewTicker(500 * time.Millisecond) defer ticker.Stop() go func() { for _ = range ticker.C { - m := make(map[string]interface{}) - - err2 := jsonRequestResponse(httpc, daemon.ControlJobEndpointStatus, - struct {}{}, - &m, - ) - - t.lock.Lock() - t.err = err2 - t.report = m - t.lock.Unlock() - t.draw() + update() } }() termbox.HideCursor() termbox.Clear(termbox.ColorDefault, termbox.ColorDefault) - loop: +loop: for { switch ev := termbox.PollEvent(); ev.Type { case termbox.EventKey: @@ -167,7 +171,7 @@ func (t *tui) draw() { } t.printf("Status: %s", rep.Status) t.newline() - if (rep.Problem != "") { + if rep.Problem != "" { t.printf("Problem: %s", rep.Problem) t.newline() } @@ -198,7 +202,7 @@ func rightPad(str string, length int, pad string) string { return str + times(pad, length-len(str)) } -func (t *tui) drawBar(name string, status string, total int, done int) { +func (t *tui) drawBar(name string, status string, total int, done int, bytes int64) { t.write(rightPad(name, 20, " ")) t.write(" ") t.write(rightPad(status, 20, " ")) @@ -207,11 +211,12 @@ func (t *tui) drawBar(name string, status string, total int, done int) { length := 50 completedLength := length * done / total - //FIXME finished bar has 1 off size compared to not finished bar - t.write(times("=", completedLength-1)) + t.write(times("=", completedLength)) t.write(">") t.write(times("-", length-completedLength)) + t.write(" ") + t.write(rightPad(ByteCountBinary(bytes), 8, " ")) t.printf(" %d/%d", done, total) } @@ -219,11 +224,32 @@ func (t *tui) drawBar(name string, status string, total int, done int) { } func printFilesystem(rep *fsrep.Report, t *tui) { - t.drawBar(rep.Filesystem, rep.Status, len(rep.Completed) + len(rep.Pending), len(rep.Completed)) - if (rep.Problem != "") { + bytes := int64(0) + for _, s := range rep.Pending { + bytes += s.Bytes + } + for _, s := range rep.Completed { + bytes += s.Bytes + } + + t.drawBar(rep.Filesystem, rep.Status, len(rep.Completed)+len(rep.Pending), len(rep.Completed), bytes) + if rep.Problem != "" { t.addIndent(1) t.printf("Problem: %s", rep.Problem) t.newline() t.addIndent(-1) } -} \ No newline at end of file +} + +func ByteCountBinary(b int64) string { + const unit = 1024 + if b < unit { + return fmt.Sprintf("%d B", b) + } + div, exp := int64(unit), 0 + for n := b / unit; n >= unit; n /= unit { + div *= unit + exp++ + } + return fmt.Sprintf("%.1f %ciB", float64(b)/float64(div), "KMGTPE"[exp]) +} diff --git a/main.go b/main.go index 38f7138..40f1f33 100644 --- a/main.go +++ b/main.go @@ -3,9 +3,9 @@ package main import ( "github.com/spf13/cobra" + "github.com/zrepl/zrepl/client" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon" - "github.com/zrepl/zrepl/client" "log" "os" ) diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index 5957dda..0bf7fb6 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -13,6 +13,7 @@ import ( "github.com/zrepl/zrepl/logger" "github.com/zrepl/zrepl/replication/pdu" + "github.com/zrepl/zrepl/util" ) type contextKey int @@ -56,6 +57,7 @@ type StepReport struct { From, To string Status string Problem string + Bytes int64 } type Report struct { @@ -167,7 +169,8 @@ type ReplicationStep struct { parent *Replication // both retry and permanent error - err error + err error + byteCounter *util.ByteCounterReader } func (f *Replication) TakeStep(ctx context.Context, sender Sender, receiver Receiver) (post State, nextStepDate time.Time) { @@ -362,6 +365,9 @@ func (s *ReplicationStep) doReplication(ctx context.Context, sender Sender, rece return updateStateError(err) } + s.byteCounter = util.NewByteCounterReader(sstream) + sstream = s.byteCounter + rr := &pdu.ReceiveReq{ Filesystem: fs, ClearResumeToken: !sres.UsedResumeToken, @@ -439,15 +445,20 @@ func (s *ReplicationStep) String() string { } } -func (step *ReplicationStep) Report() *StepReport { +func (s *ReplicationStep) Report() *StepReport { var from string // FIXME follow same convention as ZFS: to should be nil on full send - if step.from != nil { - from = step.from.RelName() + if s.from != nil { + from = s.from.RelName() + } + bytes := int64(0) + if s.byteCounter != nil { + bytes = s.byteCounter.Bytes() } rep := StepReport{ From: from, - To: step.to.RelName(), - Status: step.state.String(), + To: s.to.RelName(), + Status: s.state.String(), + Bytes: bytes, } return &rep } diff --git a/util/io.go b/util/io.go index 969e9c7..f9210b3 100644 --- a/util/io.go +++ b/util/io.go @@ -4,6 +4,7 @@ import ( "io" "net" "os" + "sync/atomic" ) type NetConnLogger struct { @@ -97,3 +98,28 @@ func (c *ChainedReader) Read(buf []byte) (n int, err error) { return } + +type ByteCounterReader struct { + reader io.ReadCloser + bytes int64 +} + +func NewByteCounterReader(reader io.ReadCloser) *ByteCounterReader { + return &ByteCounterReader{ + reader: reader, + } +} + +func (b *ByteCounterReader) Close() error { + return b.reader.Close() +} + +func (b *ByteCounterReader) Read(p []byte) (n int, err error) { + n, err = b.reader.Read(p) + atomic.AddInt64(&b.bytes, int64(n)) + return n, err +} + +func (b *ByteCounterReader) Bytes() int64 { + return atomic.LoadInt64(&b.bytes) +} From 98f3f3dfd81915c93d4639fe2773adba47985e48 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Wed, 29 Aug 2018 23:29:45 +0200 Subject: [PATCH 074/167] show expected size of current send Needs to be changed to send sizes for all planned steps --- client/status.go | 28 +++++--- endpoint/endpoint.go | 8 ++- replication/fsrep/fsfsm.go | 10 ++- replication/pdu/pdu.pb.go | 132 ++++++++++++++++++++----------------- replication/pdu/pdu.proto | 5 +- zfs/zfs.go | 44 +++++++++++++ 6 files changed, 152 insertions(+), 75 deletions(-) diff --git a/client/status.go b/client/status.go index 4824f2d..b42b069 100644 --- a/client/status.go +++ b/client/status.go @@ -177,13 +177,13 @@ func (t *tui) draw() { } for _, fs := range rep.Completed { - printFilesystem(fs, t) + printFilesystem(fs, t, false) } if rep.Active != nil { - printFilesystem(rep.Active, t) + printFilesystem(rep.Active, t, true) } for _, fs := range rep.Pending { - printFilesystem(fs, t) + printFilesystem(fs, t, false) } } @@ -202,43 +202,51 @@ func rightPad(str string, length int, pad string) string { return str + times(pad, length-len(str)) } -func (t *tui) drawBar(name string, status string, total int, done int, bytes int64) { +func (t *tui) drawBar(name string, status string, bytes int64, totalBytes int64) { t.write(rightPad(name, 20, " ")) t.write(" ") t.write(rightPad(status, 20, " ")) - if total > 0 { + if totalBytes > 0 { length := 50 - completedLength := length * done / total + completedLength := int(int64(length) * bytes / totalBytes) t.write(times("=", completedLength)) t.write(">") t.write(times("-", length-completedLength)) t.write(" ") - t.write(rightPad(ByteCountBinary(bytes), 8, " ")) - t.printf(" %d/%d", done, total) + t.write(rightPad(ByteCountBinary(bytes) + "/" + ByteCountBinary(bytes), 16, " ")) } t.newline() } -func printFilesystem(rep *fsrep.Report, t *tui) { +func printFilesystem(rep *fsrep.Report, t *tui, versions bool) { bytes := int64(0) + totalBytes := int64(0) for _, s := range rep.Pending { bytes += s.Bytes + totalBytes += s.ExpectedBytes } for _, s := range rep.Completed { bytes += s.Bytes + totalBytes += s.ExpectedBytes } - t.drawBar(rep.Filesystem, rep.Status, len(rep.Completed)+len(rep.Pending), len(rep.Completed), bytes) + t.drawBar(rep.Filesystem, rep.Status, bytes, totalBytes) if rep.Problem != "" { t.addIndent(1) t.printf("Problem: %s", rep.Problem) t.newline() t.addIndent(-1) } + if versions { + vs := append(rep.Completed, rep.Pending...) + for _, v := range vs { + t.drawBar(" " + v.To, v.Status, v.Bytes, v.ExpectedBytes) + } + } } func ByteCountBinary(b int64) string { diff --git a/endpoint/endpoint.go b/endpoint/endpoint.go index 408a155..55c41c5 100644 --- a/endpoint/endpoint.go +++ b/endpoint/endpoint.go @@ -74,11 +74,17 @@ func (p *Sender) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.Rea if !pass { return nil, nil, replication.NewFilteredError(r.Filesystem) } + + size, err := zfs.ZFSSendDry(r.Filesystem, r.From, r.To) + if err != nil { + return nil, nil, err + } + stream, err := zfs.ZFSSend(r.Filesystem, r.From, r.To) if err != nil { return nil, nil, err } - return &pdu.SendRes{}, stream, nil + return &pdu.SendRes{ExpectedSize: size}, stream, nil } func (p *Sender) DestroySnapshots(ctx context.Context, req *pdu.DestroySnapshotsReq) (*pdu.DestroySnapshotsRes, error) { diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index 0bf7fb6..43bbf59 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -40,6 +40,7 @@ func getLogger(ctx context.Context) Logger { type Sender interface { // If a non-nil io.ReadCloser is returned, it is guaranteed to be closed before // any next call to the parent github.com/zrepl/zrepl/replication.Endpoint. + // returned int64 is the expected size of the stream Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) SnapshotReplicationStatus(ctx context.Context, r *pdu.SnapshotReplicationStatusReq) (*pdu.SnapshotReplicationStatusRes, error) } @@ -58,6 +59,7 @@ type StepReport struct { Status string Problem string Bytes int64 + ExpectedBytes int64 } type Report struct { @@ -169,8 +171,10 @@ type ReplicationStep struct { parent *Replication // both retry and permanent error - err error - byteCounter *util.ByteCounterReader + err error + + byteCounter *util.ByteCounterReader + expectedSize int64 } func (f *Replication) TakeStep(ctx context.Context, sender Sender, receiver Receiver) (post State, nextStepDate time.Time) { @@ -365,6 +369,7 @@ func (s *ReplicationStep) doReplication(ctx context.Context, sender Sender, rece return updateStateError(err) } + s.expectedSize = sres.ExpectedSize s.byteCounter = util.NewByteCounterReader(sstream) sstream = s.byteCounter @@ -459,6 +464,7 @@ func (s *ReplicationStep) Report() *StepReport { To: s.to.RelName(), Status: s.state.String(), Bytes: bytes, + ExpectedBytes: s.expectedSize, } return &rep } diff --git a/replication/pdu/pdu.pb.go b/replication/pdu/pdu.pb.go index db361c3..3ca7077 100644 --- a/replication/pdu/pdu.pb.go +++ b/replication/pdu/pdu.pb.go @@ -38,7 +38,7 @@ func (x FilesystemVersion_VersionType) String() string { return proto.EnumName(FilesystemVersion_VersionType_name, int32(x)) } func (FilesystemVersion_VersionType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{5, 0} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{5, 0} } type SnapshotReplicationStatusReq_Op int32 @@ -61,7 +61,7 @@ func (x SnapshotReplicationStatusReq_Op) String() string { return proto.EnumName(SnapshotReplicationStatusReq_Op_name, int32(x)) } func (SnapshotReplicationStatusReq_Op) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{14, 0} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{14, 0} } type ListFilesystemReq struct { @@ -74,7 +74,7 @@ func (m *ListFilesystemReq) Reset() { *m = ListFilesystemReq{} } func (m *ListFilesystemReq) String() string { return proto.CompactTextString(m) } func (*ListFilesystemReq) ProtoMessage() {} func (*ListFilesystemReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{0} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{0} } func (m *ListFilesystemReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemReq.Unmarshal(m, b) @@ -105,7 +105,7 @@ func (m *ListFilesystemRes) Reset() { *m = ListFilesystemRes{} } func (m *ListFilesystemRes) String() string { return proto.CompactTextString(m) } func (*ListFilesystemRes) ProtoMessage() {} func (*ListFilesystemRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{1} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{1} } func (m *ListFilesystemRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemRes.Unmarshal(m, b) @@ -144,7 +144,7 @@ func (m *Filesystem) Reset() { *m = Filesystem{} } func (m *Filesystem) String() string { return proto.CompactTextString(m) } func (*Filesystem) ProtoMessage() {} func (*Filesystem) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{2} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{2} } func (m *Filesystem) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Filesystem.Unmarshal(m, b) @@ -189,7 +189,7 @@ func (m *ListFilesystemVersionsReq) Reset() { *m = ListFilesystemVersion func (m *ListFilesystemVersionsReq) String() string { return proto.CompactTextString(m) } func (*ListFilesystemVersionsReq) ProtoMessage() {} func (*ListFilesystemVersionsReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{3} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{3} } func (m *ListFilesystemVersionsReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemVersionsReq.Unmarshal(m, b) @@ -227,7 +227,7 @@ func (m *ListFilesystemVersionsRes) Reset() { *m = ListFilesystemVersion func (m *ListFilesystemVersionsRes) String() string { return proto.CompactTextString(m) } func (*ListFilesystemVersionsRes) ProtoMessage() {} func (*ListFilesystemVersionsRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{4} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{4} } func (m *ListFilesystemVersionsRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemVersionsRes.Unmarshal(m, b) @@ -269,7 +269,7 @@ func (m *FilesystemVersion) Reset() { *m = FilesystemVersion{} } func (m *FilesystemVersion) String() string { return proto.CompactTextString(m) } func (*FilesystemVersion) ProtoMessage() {} func (*FilesystemVersion) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{5} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{5} } func (m *FilesystemVersion) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_FilesystemVersion.Unmarshal(m, b) @@ -348,7 +348,7 @@ func (m *SendReq) Reset() { *m = SendReq{} } func (m *SendReq) String() string { return proto.CompactTextString(m) } func (*SendReq) ProtoMessage() {} func (*SendReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{6} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{6} } func (m *SendReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SendReq.Unmarshal(m, b) @@ -422,7 +422,7 @@ func (m *Property) Reset() { *m = Property{} } func (m *Property) String() string { return proto.CompactTextString(m) } func (*Property) ProtoMessage() {} func (*Property) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{7} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{7} } func (m *Property) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Property.Unmarshal(m, b) @@ -458,8 +458,10 @@ func (m *Property) GetValue() string { type SendRes struct { // Whether the resume token provided in the request has been used or not. - UsedResumeToken bool `protobuf:"varint,1,opt,name=UsedResumeToken,proto3" json:"UsedResumeToken,omitempty"` - Properties []*Property `protobuf:"bytes,2,rep,name=Properties,proto3" json:"Properties,omitempty"` + UsedResumeToken bool `protobuf:"varint,1,opt,name=UsedResumeToken,proto3" json:"UsedResumeToken,omitempty"` + // Expected stream size determined by dry run, not exact + ExpectedSize int64 `protobuf:"varint,2,opt,name=ExpectedSize,proto3" json:"ExpectedSize,omitempty"` + Properties []*Property `protobuf:"bytes,3,rep,name=Properties,proto3" json:"Properties,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -469,7 +471,7 @@ func (m *SendRes) Reset() { *m = SendRes{} } func (m *SendRes) String() string { return proto.CompactTextString(m) } func (*SendRes) ProtoMessage() {} func (*SendRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{8} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{8} } func (m *SendRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SendRes.Unmarshal(m, b) @@ -496,6 +498,13 @@ func (m *SendRes) GetUsedResumeToken() bool { return false } +func (m *SendRes) GetExpectedSize() int64 { + if m != nil { + return m.ExpectedSize + } + return 0 +} + func (m *SendRes) GetProperties() []*Property { if m != nil { return m.Properties @@ -516,7 +525,7 @@ func (m *ReceiveReq) Reset() { *m = ReceiveReq{} } func (m *ReceiveReq) String() string { return proto.CompactTextString(m) } func (*ReceiveReq) ProtoMessage() {} func (*ReceiveReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{9} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{9} } func (m *ReceiveReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ReceiveReq.Unmarshal(m, b) @@ -560,7 +569,7 @@ func (m *ReceiveRes) Reset() { *m = ReceiveRes{} } func (m *ReceiveRes) String() string { return proto.CompactTextString(m) } func (*ReceiveRes) ProtoMessage() {} func (*ReceiveRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{10} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{10} } func (m *ReceiveRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ReceiveRes.Unmarshal(m, b) @@ -593,7 +602,7 @@ func (m *DestroySnapshotsReq) Reset() { *m = DestroySnapshotsReq{} } func (m *DestroySnapshotsReq) String() string { return proto.CompactTextString(m) } func (*DestroySnapshotsReq) ProtoMessage() {} func (*DestroySnapshotsReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{11} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{11} } func (m *DestroySnapshotsReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DestroySnapshotsReq.Unmarshal(m, b) @@ -639,7 +648,7 @@ func (m *DestroySnapshotRes) Reset() { *m = DestroySnapshotRes{} } func (m *DestroySnapshotRes) String() string { return proto.CompactTextString(m) } func (*DestroySnapshotRes) ProtoMessage() {} func (*DestroySnapshotRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{12} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{12} } func (m *DestroySnapshotRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DestroySnapshotRes.Unmarshal(m, b) @@ -684,7 +693,7 @@ func (m *DestroySnapshotsRes) Reset() { *m = DestroySnapshotsRes{} } func (m *DestroySnapshotsRes) String() string { return proto.CompactTextString(m) } func (*DestroySnapshotsRes) ProtoMessage() {} func (*DestroySnapshotsRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{13} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{13} } func (m *DestroySnapshotsRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DestroySnapshotsRes.Unmarshal(m, b) @@ -724,7 +733,7 @@ func (m *SnapshotReplicationStatusReq) Reset() { *m = SnapshotReplicatio func (m *SnapshotReplicationStatusReq) String() string { return proto.CompactTextString(m) } func (*SnapshotReplicationStatusReq) ProtoMessage() {} func (*SnapshotReplicationStatusReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{14} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{14} } func (m *SnapshotReplicationStatusReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SnapshotReplicationStatusReq.Unmarshal(m, b) @@ -776,7 +785,7 @@ func (m *SnapshotReplicationStatusRes) Reset() { *m = SnapshotReplicatio func (m *SnapshotReplicationStatusRes) String() string { return proto.CompactTextString(m) } func (*SnapshotReplicationStatusRes) ProtoMessage() {} func (*SnapshotReplicationStatusRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_e1dccbd3b8cde5a3, []int{15} + return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{15} } func (m *SnapshotReplicationStatusRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SnapshotReplicationStatusRes.Unmarshal(m, b) @@ -824,46 +833,47 @@ func init() { proto.RegisterEnum("pdu.SnapshotReplicationStatusReq_Op", SnapshotReplicationStatusReq_Op_name, SnapshotReplicationStatusReq_Op_value) } -func init() { proto.RegisterFile("pdu.proto", fileDescriptor_pdu_e1dccbd3b8cde5a3) } +func init() { proto.RegisterFile("pdu.proto", fileDescriptor_pdu_cc43fc3f9f8b50e6) } -var fileDescriptor_pdu_e1dccbd3b8cde5a3 = []byte{ - // 595 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x54, 0xdd, 0x6e, 0x12, 0x41, - 0x14, 0x76, 0x17, 0x5a, 0x96, 0x43, 0x4b, 0x61, 0xda, 0xe8, 0xda, 0x34, 0x86, 0x4c, 0xbc, 0x40, - 0x13, 0x49, 0x44, 0xe2, 0x8d, 0x89, 0x17, 0x6d, 0x05, 0x2f, 0x8c, 0x34, 0x03, 0x36, 0xbd, 0x32, - 0xd9, 0x76, 0x4f, 0xd2, 0x0d, 0x2c, 0x33, 0x9d, 0xd9, 0x35, 0xe1, 0x71, 0x7c, 0x05, 0x9f, 0xc6, - 0xc7, 0x31, 0x33, 0xec, 0xcf, 0x08, 0x2d, 0xe1, 0x8a, 0xf9, 0xce, 0xf9, 0x38, 0xe7, 0x3b, 0x7f, - 0x0b, 0x75, 0x11, 0xa6, 0x3d, 0x21, 0x79, 0xc2, 0x49, 0x45, 0x84, 0x29, 0x3d, 0x86, 0xf6, 0xb7, - 0x48, 0x25, 0xc3, 0x68, 0x8e, 0x6a, 0xa9, 0x12, 0x8c, 0x19, 0x3e, 0xd0, 0xe1, 0xa6, 0x51, 0x91, - 0xf7, 0xd0, 0x28, 0x0d, 0xca, 0x77, 0x3a, 0x95, 0x6e, 0xa3, 0x7f, 0xd4, 0xd3, 0xf1, 0x2c, 0xa2, - 0xcd, 0xa1, 0xe7, 0x00, 0x25, 0x24, 0x04, 0xaa, 0x57, 0x41, 0x72, 0xef, 0x3b, 0x1d, 0xa7, 0x5b, - 0x67, 0xe6, 0x4d, 0x3a, 0xd0, 0x60, 0xa8, 0xd2, 0x18, 0xa7, 0x7c, 0x86, 0x0b, 0xdf, 0x35, 0x2e, - 0xdb, 0x44, 0x3f, 0xc1, 0xcb, 0xff, 0xb5, 0x5c, 0xa3, 0x54, 0x11, 0x5f, 0x28, 0x86, 0x0f, 0xe4, - 0x95, 0x9d, 0x20, 0x0b, 0x6c, 0x59, 0xe8, 0xf8, 0xe9, 0x3f, 0x2b, 0xd2, 0x07, 0x2f, 0x87, 0x59, - 0x35, 0xcf, 0xd7, 0xaa, 0xc9, 0xdc, 0xac, 0xe0, 0xd1, 0xbf, 0x0e, 0xb4, 0x37, 0xfc, 0xe4, 0x23, - 0x54, 0xa7, 0x4b, 0x81, 0x46, 0x40, 0xb3, 0x4f, 0x1f, 0x8f, 0xd2, 0xcb, 0x7e, 0x35, 0x93, 0x19, - 0xbe, 0xee, 0xc8, 0xf7, 0x20, 0xc6, 0xac, 0x6c, 0xf3, 0xd6, 0xb6, 0x51, 0x1a, 0x85, 0x7e, 0xa5, - 0xe3, 0x74, 0xab, 0xcc, 0xbc, 0xc9, 0x19, 0xd4, 0x2f, 0x24, 0x06, 0x09, 0x4e, 0x6f, 0x46, 0x7e, - 0xd5, 0x38, 0x4a, 0x03, 0x39, 0x05, 0xcf, 0x80, 0x88, 0x2f, 0xfc, 0x3d, 0x13, 0xa9, 0xc0, 0xf4, - 0x0d, 0x34, 0xac, 0xb4, 0xe4, 0x00, 0xbc, 0xc9, 0x22, 0x10, 0xea, 0x9e, 0x27, 0xad, 0x67, 0x1a, - 0x9d, 0x73, 0x3e, 0x8b, 0x03, 0x39, 0x6b, 0x39, 0xf4, 0xb7, 0x03, 0xb5, 0x09, 0x2e, 0xc2, 0x1d, - 0xfa, 0xaa, 0x45, 0x0e, 0x25, 0x8f, 0x73, 0xe1, 0xfa, 0x4d, 0x9a, 0xe0, 0x4e, 0xb9, 0x91, 0x5d, - 0x67, 0xee, 0x94, 0xaf, 0x8f, 0xb6, 0xba, 0x31, 0x5a, 0x23, 0x9c, 0xc7, 0x42, 0xa2, 0x52, 0x46, - 0xb8, 0xc7, 0x0a, 0x4c, 0x4e, 0x60, 0xef, 0x12, 0xc3, 0x54, 0xf8, 0xfb, 0xc6, 0xb1, 0x02, 0x74, - 0x00, 0xde, 0x95, 0xe4, 0x02, 0x65, 0xb2, 0x2c, 0x9a, 0xe7, 0x58, 0xcd, 0x3b, 0x81, 0xbd, 0xeb, - 0x60, 0x9e, 0xe6, 0x1d, 0x5d, 0x01, 0x7a, 0x9b, 0x17, 0xa6, 0x48, 0x17, 0x8e, 0x7e, 0x28, 0x0c, - 0x6d, 0x61, 0x8e, 0x49, 0xb0, 0x6e, 0x26, 0xef, 0x00, 0xb2, 0x54, 0x11, 0x2a, 0xdf, 0x35, 0xfb, - 0x71, 0x68, 0x26, 0x9b, 0x2b, 0x60, 0x16, 0x81, 0xde, 0x00, 0x30, 0xbc, 0xc3, 0xe8, 0x17, 0xee, - 0xd2, 0xbf, 0xb7, 0xd0, 0xba, 0x98, 0x63, 0x20, 0xd7, 0x77, 0xdf, 0x63, 0x1b, 0x76, 0x7a, 0x60, - 0x45, 0x56, 0x74, 0x06, 0xc7, 0x97, 0xa8, 0x12, 0xc9, 0x97, 0xf9, 0x20, 0x77, 0x39, 0x04, 0x32, - 0x80, 0x7a, 0xc1, 0xcf, 0x8a, 0x79, 0x6a, 0xd9, 0x4b, 0x22, 0xfd, 0x09, 0x64, 0x2d, 0x59, 0x76, - 0x37, 0x39, 0x34, 0x99, 0xb6, 0xdc, 0x4d, 0xce, 0xd3, 0x83, 0xf9, 0x22, 0x25, 0x97, 0xf9, 0x60, - 0x0c, 0xa0, 0x5f, 0x1f, 0x2b, 0x46, 0x7f, 0x69, 0x6a, 0xba, 0x01, 0xf3, 0x24, 0xbf, 0xcb, 0x17, - 0x26, 0xfe, 0xa6, 0x14, 0x96, 0xf3, 0xe8, 0x1f, 0x07, 0xce, 0x4a, 0x87, 0x98, 0x47, 0x77, 0x66, - 0xff, 0x27, 0x49, 0x90, 0xa4, 0x3b, 0x35, 0xe8, 0xd4, 0x2a, 0x6a, 0xa5, 0xb1, 0x14, 0x3f, 0x00, - 0x97, 0x0b, 0xb3, 0xd9, 0xcd, 0xfe, 0x6b, 0x23, 0x65, 0x5b, 0xaa, 0xde, 0x58, 0x30, 0x97, 0x0b, - 0xda, 0x01, 0x77, 0x2c, 0x48, 0x0d, 0x2a, 0x23, 0xd4, 0xc7, 0xd6, 0x86, 0xc3, 0x09, 0x16, 0x7f, - 0xc0, 0xb0, 0xe5, 0xd0, 0xcf, 0x5b, 0x35, 0x2b, 0xad, 0xb9, 0xe4, 0x67, 0x7b, 0x6a, 0x59, 0x6e, - 0xf7, 0xcd, 0x77, 0xfc, 0xc3, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xf6, 0x1a, 0xe7, 0xf6, 0xd4, - 0x05, 0x00, 0x00, +var fileDescriptor_pdu_cc43fc3f9f8b50e6 = []byte{ + // 616 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x54, 0xdb, 0x6e, 0x13, 0x3d, + 0x10, 0xfe, 0x77, 0x37, 0x6d, 0x37, 0x93, 0x1e, 0x52, 0xb7, 0xfa, 0x59, 0xaa, 0x0a, 0x45, 0x16, + 0x17, 0x01, 0x89, 0x48, 0x84, 0x8a, 0x1b, 0x24, 0x2e, 0x7a, 0xe4, 0x02, 0xd1, 0xca, 0x09, 0x55, + 0xaf, 0x90, 0x96, 0xee, 0x48, 0x5d, 0x25, 0x1b, 0xbb, 0xb6, 0x17, 0x11, 0x1e, 0x80, 0xf7, 0xe0, + 0x15, 0x78, 0x1a, 0x1e, 0x07, 0xd9, 0xd9, 0x83, 0x9b, 0x94, 0x28, 0x57, 0xeb, 0xf9, 0xe6, 0xdb, + 0x99, 0x6f, 0x66, 0x3c, 0x86, 0xa6, 0x48, 0xf2, 0x9e, 0x90, 0x5c, 0x73, 0x12, 0x88, 0x24, 0xa7, + 0x7b, 0xb0, 0xfb, 0x31, 0x55, 0xfa, 0x3c, 0x1d, 0xa3, 0x9a, 0x2a, 0x8d, 0x19, 0xc3, 0x7b, 0x7a, + 0xbe, 0x08, 0x2a, 0xf2, 0x1a, 0x5a, 0x35, 0xa0, 0x22, 0xaf, 0x13, 0x74, 0x5b, 0xfd, 0x9d, 0x9e, + 0x89, 0xe7, 0x10, 0x5d, 0x0e, 0x3d, 0x06, 0xa8, 0x4d, 0x42, 0xa0, 0x71, 0x15, 0xeb, 0xbb, 0xc8, + 0xeb, 0x78, 0xdd, 0x26, 0xb3, 0x67, 0xd2, 0x81, 0x16, 0x43, 0x95, 0x67, 0x38, 0xe4, 0x23, 0x9c, + 0x44, 0xbe, 0x75, 0xb9, 0x10, 0x7d, 0x07, 0x4f, 0x1f, 0x6a, 0xb9, 0x46, 0xa9, 0x52, 0x3e, 0x51, + 0x0c, 0xef, 0xc9, 0x33, 0x37, 0x41, 0x11, 0xd8, 0x41, 0xe8, 0xe5, 0xbf, 0x7f, 0x56, 0xa4, 0x0f, + 0x61, 0x69, 0x16, 0xd5, 0xfc, 0x3f, 0x57, 0x4d, 0xe1, 0x66, 0x15, 0x8f, 0xfe, 0xf1, 0x60, 0x77, + 0xc1, 0x4f, 0xde, 0x42, 0x63, 0x38, 0x15, 0x68, 0x05, 0x6c, 0xf7, 0xe9, 0xe3, 0x51, 0x7a, 0xc5, + 0xd7, 0x30, 0x99, 0xe5, 0x9b, 0x8e, 0x7c, 0x8a, 0x33, 0x2c, 0xca, 0xb6, 0x67, 0x83, 0x5d, 0xe4, + 0x69, 0x12, 0x05, 0x1d, 0xaf, 0xdb, 0x60, 0xf6, 0x4c, 0x0e, 0xa1, 0x79, 0x22, 0x31, 0xd6, 0x38, + 0xbc, 0xb9, 0x88, 0x1a, 0xd6, 0x51, 0x03, 0xe4, 0x00, 0x42, 0x6b, 0xa4, 0x7c, 0x12, 0xad, 0xd9, + 0x48, 0x95, 0x4d, 0x5f, 0x40, 0xcb, 0x49, 0x4b, 0x36, 0x21, 0x1c, 0x4c, 0x62, 0xa1, 0xee, 0xb8, + 0x6e, 0xff, 0x67, 0xac, 0x63, 0xce, 0x47, 0x59, 0x2c, 0x47, 0x6d, 0x8f, 0xfe, 0xf2, 0x60, 0x63, + 0x80, 0x93, 0x64, 0x85, 0xbe, 0x1a, 0x91, 0xe7, 0x92, 0x67, 0xa5, 0x70, 0x73, 0x26, 0xdb, 0xe0, + 0x0f, 0xb9, 0x95, 0xdd, 0x64, 0xfe, 0x90, 0xcf, 0x8f, 0xb6, 0xb1, 0x30, 0x5a, 0x2b, 0x9c, 0x67, + 0x42, 0xa2, 0x52, 0x56, 0x78, 0xc8, 0x2a, 0x9b, 0xec, 0xc3, 0xda, 0x29, 0x26, 0xb9, 0x88, 0xd6, + 0xad, 0x63, 0x66, 0xd0, 0x23, 0x08, 0xaf, 0x24, 0x17, 0x28, 0xf5, 0xb4, 0x6a, 0x9e, 0xe7, 0x34, + 0x6f, 0x1f, 0xd6, 0xae, 0xe3, 0x71, 0x5e, 0x76, 0x74, 0x66, 0xd0, 0x9f, 0x55, 0x65, 0x8a, 0x74, + 0x61, 0xe7, 0xb3, 0xc2, 0xc4, 0x55, 0xe6, 0xd9, 0x0c, 0xf3, 0x30, 0xa1, 0xb0, 0x79, 0xf6, 0x5d, + 0xe0, 0xad, 0xc6, 0x64, 0x90, 0xfe, 0x98, 0x85, 0x0c, 0xd8, 0x03, 0x8c, 0xbc, 0x02, 0x28, 0xf4, + 0xa4, 0xa8, 0xa2, 0xc0, 0x5e, 0xa2, 0x2d, 0x3b, 0xfe, 0x52, 0x26, 0x73, 0x08, 0xf4, 0x06, 0x80, + 0xe1, 0x2d, 0xa6, 0xdf, 0x70, 0x95, 0x26, 0xbf, 0x84, 0xf6, 0xc9, 0x18, 0x63, 0x39, 0xbf, 0x20, + 0x21, 0x5b, 0xc0, 0xe9, 0xa6, 0x13, 0x59, 0xd1, 0x11, 0xec, 0x9d, 0xa2, 0xd2, 0x92, 0x4f, 0xcb, + 0x69, 0xaf, 0xb2, 0x2d, 0xe4, 0x08, 0x9a, 0x15, 0x3f, 0xf2, 0x97, 0x6e, 0x44, 0x4d, 0xa4, 0x5f, + 0x80, 0xcc, 0x25, 0x2b, 0x96, 0xab, 0x34, 0x6d, 0xa6, 0x25, 0xcb, 0x55, 0xf2, 0xcc, 0xf4, 0xce, + 0xa4, 0xe4, 0xb2, 0x9c, 0x9e, 0x35, 0xe8, 0x87, 0xc7, 0x8a, 0x31, 0xcf, 0xd1, 0x86, 0x69, 0xc0, + 0x58, 0x97, 0xcb, 0xfb, 0xc4, 0xc6, 0x5f, 0x94, 0xc2, 0x4a, 0x1e, 0xfd, 0xed, 0xc1, 0x61, 0xed, + 0x10, 0xe3, 0xf4, 0xd6, 0x2e, 0xc9, 0x40, 0xc7, 0x3a, 0x5f, 0xa9, 0x41, 0x07, 0x4e, 0x51, 0x33, + 0x8d, 0xb5, 0xf8, 0x23, 0xf0, 0xb9, 0xb0, 0xd7, 0x7f, 0xbb, 0xff, 0xdc, 0x4a, 0x59, 0x96, 0xaa, + 0x77, 0x29, 0x98, 0xcf, 0x05, 0xed, 0x80, 0x7f, 0x29, 0xc8, 0x06, 0x04, 0x17, 0x68, 0x36, 0x72, + 0x17, 0xb6, 0x06, 0x58, 0xfd, 0x80, 0x49, 0xdb, 0xa3, 0xef, 0x97, 0x6a, 0x56, 0x46, 0x73, 0xcd, + 0x2f, 0xee, 0xb2, 0x83, 0x7c, 0x5d, 0xb7, 0x8f, 0xfd, 0x9b, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, + 0x47, 0xbd, 0x08, 0x65, 0xf9, 0x05, 0x00, 0x00, } diff --git a/replication/pdu/pdu.proto b/replication/pdu/pdu.proto index 0d1103c..2e6f045 100644 --- a/replication/pdu/pdu.proto +++ b/replication/pdu/pdu.proto @@ -63,7 +63,10 @@ message SendRes { // Whether the resume token provided in the request has been used or not. bool UsedResumeToken = 1; - repeated Property Properties = 2; + // Expected stream size determined by dry run, not exact + int64 ExpectedSize = 2; + + repeated Property Properties = 3; } message ReceiveReq { diff --git a/zfs/zfs.go b/zfs/zfs.go index 58ed6d3..5040567 100644 --- a/zfs/zfs.go +++ b/zfs/zfs.go @@ -15,6 +15,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/util" "regexp" + "strconv" ) type DatasetPath struct { @@ -308,6 +309,49 @@ func ZFSSend(fs string, from, to string) (stream io.ReadCloser, err error) { return } +func ZFSSendDry(fs string, from, to string) (size int64, err error) { + + fromV, err := absVersion(fs, from) + if err != nil { + return 0, err + } + + toV := "" + if to != "" { + toV, err = absVersion(fs, to) + if err != nil { + return 0, err + } + } + + args := make([]string, 0) + args = append(args, "send", "-n", "-v", "-P") + + if toV == "" { // Initial + args = append(args, fromV) + } else { + args = append(args, "-i", fromV, toV) + } + + cmd := exec.Command(ZFS_BINARY, args...) + output, err := cmd.CombinedOutput() + if err != nil { + return 0, err + } + o := string(output) + lines := strings.Split(o, "\n") + if len(lines) < 2 { + return 0, errors.New("zfs send -n did not return the expected number of lines") + } + fields := strings.Fields(lines[1]) + if len(fields) != 2 { + return 0, errors.New("zfs send -n returned unexpexted output") + } + + size, err = strconv.ParseInt(fields[1], 10, 64) + return size, err +} + func ZFSRecv(fs string, stream io.Reader, additionalArgs ...string) (err error) { if err := validateZFSFilesystem(fs); err != nil { From 583773025fa4f0878414287d13fdde299d226daf Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Thu, 30 Aug 2018 12:47:57 +0200 Subject: [PATCH 075/167] nicer progress bar --- client/status.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/client/status.go b/client/status.go index b42b069..c2deccf 100644 --- a/client/status.go +++ b/client/status.go @@ -205,15 +205,18 @@ func rightPad(str string, length int, pad string) string { func (t *tui) drawBar(name string, status string, bytes int64, totalBytes int64) { t.write(rightPad(name, 20, " ")) t.write(" ") - t.write(rightPad(status, 20, " ")) + t.write(rightPad(status, 14, " ")) + t.write(" ") if totalBytes > 0 { length := 50 completedLength := int(int64(length) * bytes / totalBytes) + t.write("[") t.write(times("=", completedLength)) t.write(">") t.write(times("-", length-completedLength)) + t.write("]") t.write(" ") t.write(rightPad(ByteCountBinary(bytes) + "/" + ByteCountBinary(bytes), 16, " ")) From 47d8a5a7cdea630ab604e494c761991e957547ca Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Thu, 30 Aug 2018 12:48:27 +0200 Subject: [PATCH 076/167] status: only show active not all versions of active filesystem --- client/status.go | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/client/status.go b/client/status.go index c2deccf..cc171a4 100644 --- a/client/status.go +++ b/client/status.go @@ -244,11 +244,9 @@ func printFilesystem(rep *fsrep.Report, t *tui, versions bool) { t.newline() t.addIndent(-1) } - if versions { - vs := append(rep.Completed, rep.Pending...) - for _, v := range vs { - t.drawBar(" " + v.To, v.Status, v.Bytes, v.ExpectedBytes) - } + if versions && len(rep.Pending) > 0 { + v := rep.Pending[0] + t.drawBar(" " + v.To, v.Status, v.Bytes, v.ExpectedBytes) } } From b5957aca37575938ea197dc1b380d9fe179947f1 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Thu, 30 Aug 2018 12:52:08 +0200 Subject: [PATCH 077/167] do dry runs in planning stage to estimate size of all sends --- endpoint/endpoint.go | 21 +++--- replication/fsrep/fsfsm.go | 66 ++++++++++++++----- replication/mainfsm.go | 4 ++ replication/pdu/pdu.pb.go | 129 ++++++++++++++++++++----------------- replication/pdu/pdu.proto | 2 + 5 files changed, 138 insertions(+), 84 deletions(-) diff --git a/endpoint/endpoint.go b/endpoint/endpoint.go index 55c41c5..8a26730 100644 --- a/endpoint/endpoint.go +++ b/endpoint/endpoint.go @@ -75,16 +75,19 @@ func (p *Sender) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.Rea return nil, nil, replication.NewFilteredError(r.Filesystem) } - size, err := zfs.ZFSSendDry(r.Filesystem, r.From, r.To) - if err != nil { - return nil, nil, err + if r.DryRun { + size, err := zfs.ZFSSendDry(r.Filesystem, r.From, r.To) + if err != nil { + return nil, nil, err + } + return &pdu.SendRes{ExpectedSize: size}, nil, nil + } else { + stream, err := zfs.ZFSSend(r.Filesystem, r.From, r.To) + if err != nil { + return nil, nil, err + } + return &pdu.SendRes{}, stream, nil } - - stream, err := zfs.ZFSSend(r.Filesystem, r.From, r.To) - if err != nil { - return nil, nil, err - } - return &pdu.SendRes{ExpectedSize: size}, stream, nil } func (p *Sender) DestroySnapshots(ctx context.Context, req *pdu.DestroySnapshotsReq) (*pdu.DestroySnapshotsRes, error) { diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index 43bbf59..2462cba 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -40,7 +40,7 @@ func getLogger(ctx context.Context) Logger { type Sender interface { // If a non-nil io.ReadCloser is returned, it is guaranteed to be closed before // any next call to the parent github.com/zrepl/zrepl/replication.Endpoint. - // returned int64 is the expected size of the stream + // If the send request is for dry run the io.ReadCloser will be nil Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) SnapshotReplicationStatus(ctx context.Context, r *pdu.SnapshotReplicationStatusReq) (*pdu.SnapshotReplicationStatusRes, error) } @@ -105,6 +105,17 @@ func (f *Replication) State() State { return f.state } +func (f *Replication) UpdateSizeEsitmate(ctx context.Context, sender Sender) error { + f.lock.Lock() + defer f.lock.Unlock() + for _, e := range f.pending { + if err := e.updateSizeEstimate(ctx, sender); err != nil { + return err + } + } + return nil +} + type ReplicationBuilder struct { r *Replication } @@ -344,19 +355,7 @@ func (s *ReplicationStep) doReplication(ctx context.Context, sender Sender, rece return s.state } - var sr *pdu.SendReq - if s.from == nil { - sr = &pdu.SendReq{ - Filesystem: fs, - From: s.to.RelName(), // FIXME fix protocol to use To, like zfs does internally - } - } else { - sr = &pdu.SendReq{ - Filesystem: fs, - From: s.from.RelName(), - To: s.to.RelName(), - } - } + sr := s.buildSendRequest(false) log.WithField("request", sr).Debug("initiate send request") sres, sstream, err := sender.Send(ctx, sr) @@ -369,7 +368,6 @@ func (s *ReplicationStep) doReplication(ctx context.Context, sender Sender, rece return updateStateError(err) } - s.expectedSize = sres.ExpectedSize s.byteCounter = util.NewByteCounterReader(sstream) sstream = s.byteCounter @@ -442,6 +440,44 @@ func (s *ReplicationStep) doMarkReplicated(ctx context.Context, sender Sender) S return updateStateCompleted() } +func (s *ReplicationStep) updateSizeEstimate(ctx context.Context, sender Sender) error { + + fs := s.parent.fs + + log := getLogger(ctx). + WithField("filesystem", fs). + WithField("step", s.String()) + + sr := s.buildSendRequest(true) + + log.WithField("request", sr).Debug("initiate dry run send request") + sres, _, err := sender.Send(ctx, sr) + if err != nil { + log.WithError(err).Error("dry run send request failed") + } + s.expectedSize = sres.ExpectedSize + return nil +} + +func (s *ReplicationStep) buildSendRequest(dryRun bool) (sr *pdu.SendReq) { + fs := s.parent.fs + if s.from == nil { + sr = &pdu.SendReq{ + Filesystem: fs, + From: s.to.RelName(), // FIXME fix protocol to use To, like zfs does internally + DryRun: dryRun, + } + } else { + sr = &pdu.SendReq{ + Filesystem: fs, + From: s.from.RelName(), + To: s.to.RelName(), + DryRun: dryRun, + } + } + return sr +} + func (s *ReplicationStep) String() string { if s.from == nil { // FIXME: ZFS semantics are that to is nil on non-incremental send return fmt.Sprintf("%s%s (full)", s.parent.fs, s.to.RelName()) diff --git a/replication/mainfsm.go b/replication/mainfsm.go index 2fc5cc3..d30f417 100644 --- a/replication/mainfsm.go +++ b/replication/mainfsm.go @@ -266,6 +266,10 @@ func statePlanning(ctx context.Context, sender Sender, receiver Receiver, u upda } } qitem := fsrfsm.Done() + if err = qitem.UpdateSizeEsitmate(ctx, sender); err != nil { + log.WithError(err).Error("cannot get size estimate") + return handlePlanningError(err) + } q.Add(qitem) } diff --git a/replication/pdu/pdu.pb.go b/replication/pdu/pdu.pb.go index 3ca7077..a1876f8 100644 --- a/replication/pdu/pdu.pb.go +++ b/replication/pdu/pdu.pb.go @@ -38,7 +38,7 @@ func (x FilesystemVersion_VersionType) String() string { return proto.EnumName(FilesystemVersion_VersionType_name, int32(x)) } func (FilesystemVersion_VersionType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{5, 0} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{5, 0} } type SnapshotReplicationStatusReq_Op int32 @@ -61,7 +61,7 @@ func (x SnapshotReplicationStatusReq_Op) String() string { return proto.EnumName(SnapshotReplicationStatusReq_Op_name, int32(x)) } func (SnapshotReplicationStatusReq_Op) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{14, 0} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{14, 0} } type ListFilesystemReq struct { @@ -74,7 +74,7 @@ func (m *ListFilesystemReq) Reset() { *m = ListFilesystemReq{} } func (m *ListFilesystemReq) String() string { return proto.CompactTextString(m) } func (*ListFilesystemReq) ProtoMessage() {} func (*ListFilesystemReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{0} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{0} } func (m *ListFilesystemReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemReq.Unmarshal(m, b) @@ -105,7 +105,7 @@ func (m *ListFilesystemRes) Reset() { *m = ListFilesystemRes{} } func (m *ListFilesystemRes) String() string { return proto.CompactTextString(m) } func (*ListFilesystemRes) ProtoMessage() {} func (*ListFilesystemRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{1} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{1} } func (m *ListFilesystemRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemRes.Unmarshal(m, b) @@ -144,7 +144,7 @@ func (m *Filesystem) Reset() { *m = Filesystem{} } func (m *Filesystem) String() string { return proto.CompactTextString(m) } func (*Filesystem) ProtoMessage() {} func (*Filesystem) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{2} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{2} } func (m *Filesystem) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Filesystem.Unmarshal(m, b) @@ -189,7 +189,7 @@ func (m *ListFilesystemVersionsReq) Reset() { *m = ListFilesystemVersion func (m *ListFilesystemVersionsReq) String() string { return proto.CompactTextString(m) } func (*ListFilesystemVersionsReq) ProtoMessage() {} func (*ListFilesystemVersionsReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{3} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{3} } func (m *ListFilesystemVersionsReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemVersionsReq.Unmarshal(m, b) @@ -227,7 +227,7 @@ func (m *ListFilesystemVersionsRes) Reset() { *m = ListFilesystemVersion func (m *ListFilesystemVersionsRes) String() string { return proto.CompactTextString(m) } func (*ListFilesystemVersionsRes) ProtoMessage() {} func (*ListFilesystemVersionsRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{4} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{4} } func (m *ListFilesystemVersionsRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemVersionsRes.Unmarshal(m, b) @@ -269,7 +269,7 @@ func (m *FilesystemVersion) Reset() { *m = FilesystemVersion{} } func (m *FilesystemVersion) String() string { return proto.CompactTextString(m) } func (*FilesystemVersion) ProtoMessage() {} func (*FilesystemVersion) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{5} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{5} } func (m *FilesystemVersion) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_FilesystemVersion.Unmarshal(m, b) @@ -339,6 +339,7 @@ type SendReq struct { ResumeToken string `protobuf:"bytes,4,opt,name=ResumeToken,proto3" json:"ResumeToken,omitempty"` Compress bool `protobuf:"varint,5,opt,name=Compress,proto3" json:"Compress,omitempty"` Dedup bool `protobuf:"varint,6,opt,name=Dedup,proto3" json:"Dedup,omitempty"` + DryRun bool `protobuf:"varint,7,opt,name=DryRun,proto3" json:"DryRun,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -348,7 +349,7 @@ func (m *SendReq) Reset() { *m = SendReq{} } func (m *SendReq) String() string { return proto.CompactTextString(m) } func (*SendReq) ProtoMessage() {} func (*SendReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{6} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{6} } func (m *SendReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SendReq.Unmarshal(m, b) @@ -410,6 +411,13 @@ func (m *SendReq) GetDedup() bool { return false } +func (m *SendReq) GetDryRun() bool { + if m != nil { + return m.DryRun + } + return false +} + type Property struct { Name string `protobuf:"bytes,1,opt,name=Name,proto3" json:"Name,omitempty"` Value string `protobuf:"bytes,2,opt,name=Value,proto3" json:"Value,omitempty"` @@ -422,7 +430,7 @@ func (m *Property) Reset() { *m = Property{} } func (m *Property) String() string { return proto.CompactTextString(m) } func (*Property) ProtoMessage() {} func (*Property) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{7} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{7} } func (m *Property) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Property.Unmarshal(m, b) @@ -471,7 +479,7 @@ func (m *SendRes) Reset() { *m = SendRes{} } func (m *SendRes) String() string { return proto.CompactTextString(m) } func (*SendRes) ProtoMessage() {} func (*SendRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{8} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{8} } func (m *SendRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SendRes.Unmarshal(m, b) @@ -525,7 +533,7 @@ func (m *ReceiveReq) Reset() { *m = ReceiveReq{} } func (m *ReceiveReq) String() string { return proto.CompactTextString(m) } func (*ReceiveReq) ProtoMessage() {} func (*ReceiveReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{9} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{9} } func (m *ReceiveReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ReceiveReq.Unmarshal(m, b) @@ -569,7 +577,7 @@ func (m *ReceiveRes) Reset() { *m = ReceiveRes{} } func (m *ReceiveRes) String() string { return proto.CompactTextString(m) } func (*ReceiveRes) ProtoMessage() {} func (*ReceiveRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{10} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{10} } func (m *ReceiveRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ReceiveRes.Unmarshal(m, b) @@ -602,7 +610,7 @@ func (m *DestroySnapshotsReq) Reset() { *m = DestroySnapshotsReq{} } func (m *DestroySnapshotsReq) String() string { return proto.CompactTextString(m) } func (*DestroySnapshotsReq) ProtoMessage() {} func (*DestroySnapshotsReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{11} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{11} } func (m *DestroySnapshotsReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DestroySnapshotsReq.Unmarshal(m, b) @@ -648,7 +656,7 @@ func (m *DestroySnapshotRes) Reset() { *m = DestroySnapshotRes{} } func (m *DestroySnapshotRes) String() string { return proto.CompactTextString(m) } func (*DestroySnapshotRes) ProtoMessage() {} func (*DestroySnapshotRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{12} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{12} } func (m *DestroySnapshotRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DestroySnapshotRes.Unmarshal(m, b) @@ -693,7 +701,7 @@ func (m *DestroySnapshotsRes) Reset() { *m = DestroySnapshotsRes{} } func (m *DestroySnapshotsRes) String() string { return proto.CompactTextString(m) } func (*DestroySnapshotsRes) ProtoMessage() {} func (*DestroySnapshotsRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{13} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{13} } func (m *DestroySnapshotsRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DestroySnapshotsRes.Unmarshal(m, b) @@ -733,7 +741,7 @@ func (m *SnapshotReplicationStatusReq) Reset() { *m = SnapshotReplicatio func (m *SnapshotReplicationStatusReq) String() string { return proto.CompactTextString(m) } func (*SnapshotReplicationStatusReq) ProtoMessage() {} func (*SnapshotReplicationStatusReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{14} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{14} } func (m *SnapshotReplicationStatusReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SnapshotReplicationStatusReq.Unmarshal(m, b) @@ -785,7 +793,7 @@ func (m *SnapshotReplicationStatusRes) Reset() { *m = SnapshotReplicatio func (m *SnapshotReplicationStatusRes) String() string { return proto.CompactTextString(m) } func (*SnapshotReplicationStatusRes) ProtoMessage() {} func (*SnapshotReplicationStatusRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_cc43fc3f9f8b50e6, []int{15} + return fileDescriptor_pdu_ae88208ed9690d0f, []int{15} } func (m *SnapshotReplicationStatusRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SnapshotReplicationStatusRes.Unmarshal(m, b) @@ -833,47 +841,48 @@ func init() { proto.RegisterEnum("pdu.SnapshotReplicationStatusReq_Op", SnapshotReplicationStatusReq_Op_name, SnapshotReplicationStatusReq_Op_value) } -func init() { proto.RegisterFile("pdu.proto", fileDescriptor_pdu_cc43fc3f9f8b50e6) } +func init() { proto.RegisterFile("pdu.proto", fileDescriptor_pdu_ae88208ed9690d0f) } -var fileDescriptor_pdu_cc43fc3f9f8b50e6 = []byte{ - // 616 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x54, 0xdb, 0x6e, 0x13, 0x3d, - 0x10, 0xfe, 0x77, 0x37, 0x6d, 0x37, 0x93, 0x1e, 0x52, 0xb7, 0xfa, 0x59, 0xaa, 0x0a, 0x45, 0x16, - 0x17, 0x01, 0x89, 0x48, 0x84, 0x8a, 0x1b, 0x24, 0x2e, 0x7a, 0xe4, 0x02, 0xd1, 0xca, 0x09, 0x55, - 0xaf, 0x90, 0x96, 0xee, 0x48, 0x5d, 0x25, 0x1b, 0xbb, 0xb6, 0x17, 0x11, 0x1e, 0x80, 0xf7, 0xe0, - 0x15, 0x78, 0x1a, 0x1e, 0x07, 0xd9, 0xd9, 0x83, 0x9b, 0x94, 0x28, 0x57, 0xeb, 0xf9, 0xe6, 0xdb, - 0x99, 0x6f, 0x66, 0x3c, 0x86, 0xa6, 0x48, 0xf2, 0x9e, 0x90, 0x5c, 0x73, 0x12, 0x88, 0x24, 0xa7, - 0x7b, 0xb0, 0xfb, 0x31, 0x55, 0xfa, 0x3c, 0x1d, 0xa3, 0x9a, 0x2a, 0x8d, 0x19, 0xc3, 0x7b, 0x7a, - 0xbe, 0x08, 0x2a, 0xf2, 0x1a, 0x5a, 0x35, 0xa0, 0x22, 0xaf, 0x13, 0x74, 0x5b, 0xfd, 0x9d, 0x9e, - 0x89, 0xe7, 0x10, 0x5d, 0x0e, 0x3d, 0x06, 0xa8, 0x4d, 0x42, 0xa0, 0x71, 0x15, 0xeb, 0xbb, 0xc8, - 0xeb, 0x78, 0xdd, 0x26, 0xb3, 0x67, 0xd2, 0x81, 0x16, 0x43, 0x95, 0x67, 0x38, 0xe4, 0x23, 0x9c, - 0x44, 0xbe, 0x75, 0xb9, 0x10, 0x7d, 0x07, 0x4f, 0x1f, 0x6a, 0xb9, 0x46, 0xa9, 0x52, 0x3e, 0x51, - 0x0c, 0xef, 0xc9, 0x33, 0x37, 0x41, 0x11, 0xd8, 0x41, 0xe8, 0xe5, 0xbf, 0x7f, 0x56, 0xa4, 0x0f, - 0x61, 0x69, 0x16, 0xd5, 0xfc, 0x3f, 0x57, 0x4d, 0xe1, 0x66, 0x15, 0x8f, 0xfe, 0xf1, 0x60, 0x77, - 0xc1, 0x4f, 0xde, 0x42, 0x63, 0x38, 0x15, 0x68, 0x05, 0x6c, 0xf7, 0xe9, 0xe3, 0x51, 0x7a, 0xc5, - 0xd7, 0x30, 0x99, 0xe5, 0x9b, 0x8e, 0x7c, 0x8a, 0x33, 0x2c, 0xca, 0xb6, 0x67, 0x83, 0x5d, 0xe4, - 0x69, 0x12, 0x05, 0x1d, 0xaf, 0xdb, 0x60, 0xf6, 0x4c, 0x0e, 0xa1, 0x79, 0x22, 0x31, 0xd6, 0x38, - 0xbc, 0xb9, 0x88, 0x1a, 0xd6, 0x51, 0x03, 0xe4, 0x00, 0x42, 0x6b, 0xa4, 0x7c, 0x12, 0xad, 0xd9, - 0x48, 0x95, 0x4d, 0x5f, 0x40, 0xcb, 0x49, 0x4b, 0x36, 0x21, 0x1c, 0x4c, 0x62, 0xa1, 0xee, 0xb8, - 0x6e, 0xff, 0x67, 0xac, 0x63, 0xce, 0x47, 0x59, 0x2c, 0x47, 0x6d, 0x8f, 0xfe, 0xf2, 0x60, 0x63, - 0x80, 0x93, 0x64, 0x85, 0xbe, 0x1a, 0x91, 0xe7, 0x92, 0x67, 0xa5, 0x70, 0x73, 0x26, 0xdb, 0xe0, - 0x0f, 0xb9, 0x95, 0xdd, 0x64, 0xfe, 0x90, 0xcf, 0x8f, 0xb6, 0xb1, 0x30, 0x5a, 0x2b, 0x9c, 0x67, - 0x42, 0xa2, 0x52, 0x56, 0x78, 0xc8, 0x2a, 0x9b, 0xec, 0xc3, 0xda, 0x29, 0x26, 0xb9, 0x88, 0xd6, - 0xad, 0x63, 0x66, 0xd0, 0x23, 0x08, 0xaf, 0x24, 0x17, 0x28, 0xf5, 0xb4, 0x6a, 0x9e, 0xe7, 0x34, - 0x6f, 0x1f, 0xd6, 0xae, 0xe3, 0x71, 0x5e, 0x76, 0x74, 0x66, 0xd0, 0x9f, 0x55, 0x65, 0x8a, 0x74, - 0x61, 0xe7, 0xb3, 0xc2, 0xc4, 0x55, 0xe6, 0xd9, 0x0c, 0xf3, 0x30, 0xa1, 0xb0, 0x79, 0xf6, 0x5d, - 0xe0, 0xad, 0xc6, 0x64, 0x90, 0xfe, 0x98, 0x85, 0x0c, 0xd8, 0x03, 0x8c, 0xbc, 0x02, 0x28, 0xf4, - 0xa4, 0xa8, 0xa2, 0xc0, 0x5e, 0xa2, 0x2d, 0x3b, 0xfe, 0x52, 0x26, 0x73, 0x08, 0xf4, 0x06, 0x80, - 0xe1, 0x2d, 0xa6, 0xdf, 0x70, 0x95, 0x26, 0xbf, 0x84, 0xf6, 0xc9, 0x18, 0x63, 0x39, 0xbf, 0x20, - 0x21, 0x5b, 0xc0, 0xe9, 0xa6, 0x13, 0x59, 0xd1, 0x11, 0xec, 0x9d, 0xa2, 0xd2, 0x92, 0x4f, 0xcb, - 0x69, 0xaf, 0xb2, 0x2d, 0xe4, 0x08, 0x9a, 0x15, 0x3f, 0xf2, 0x97, 0x6e, 0x44, 0x4d, 0xa4, 0x5f, - 0x80, 0xcc, 0x25, 0x2b, 0x96, 0xab, 0x34, 0x6d, 0xa6, 0x25, 0xcb, 0x55, 0xf2, 0xcc, 0xf4, 0xce, - 0xa4, 0xe4, 0xb2, 0x9c, 0x9e, 0x35, 0xe8, 0x87, 0xc7, 0x8a, 0x31, 0xcf, 0xd1, 0x86, 0x69, 0xc0, - 0x58, 0x97, 0xcb, 0xfb, 0xc4, 0xc6, 0x5f, 0x94, 0xc2, 0x4a, 0x1e, 0xfd, 0xed, 0xc1, 0x61, 0xed, - 0x10, 0xe3, 0xf4, 0xd6, 0x2e, 0xc9, 0x40, 0xc7, 0x3a, 0x5f, 0xa9, 0x41, 0x07, 0x4e, 0x51, 0x33, - 0x8d, 0xb5, 0xf8, 0x23, 0xf0, 0xb9, 0xb0, 0xd7, 0x7f, 0xbb, 0xff, 0xdc, 0x4a, 0x59, 0x96, 0xaa, - 0x77, 0x29, 0x98, 0xcf, 0x05, 0xed, 0x80, 0x7f, 0x29, 0xc8, 0x06, 0x04, 0x17, 0x68, 0x36, 0x72, - 0x17, 0xb6, 0x06, 0x58, 0xfd, 0x80, 0x49, 0xdb, 0xa3, 0xef, 0x97, 0x6a, 0x56, 0x46, 0x73, 0xcd, - 0x2f, 0xee, 0xb2, 0x83, 0x7c, 0x5d, 0xb7, 0x8f, 0xfd, 0x9b, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, - 0x47, 0xbd, 0x08, 0x65, 0xf9, 0x05, 0x00, 0x00, +var fileDescriptor_pdu_ae88208ed9690d0f = []byte{ + // 627 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0xdb, 0x6e, 0x13, 0x31, + 0x10, 0x65, 0x37, 0x69, 0x93, 0x4c, 0x7a, 0x49, 0xdd, 0xaa, 0x2c, 0x55, 0x85, 0x22, 0x8b, 0x87, + 0x80, 0x44, 0x24, 0x42, 0xc5, 0x0b, 0x12, 0x0f, 0xbd, 0xf2, 0x80, 0x68, 0xe5, 0x84, 0xaa, 0x4f, + 0x48, 0x4b, 0x77, 0xa4, 0xae, 0x72, 0xb1, 0x6b, 0xef, 0x22, 0xc2, 0x07, 0xf0, 0x41, 0xbc, 0xf1, + 0x27, 0x7c, 0x0e, 0xf2, 0x64, 0x2f, 0x6e, 0x52, 0xa2, 0x3c, 0xd5, 0xe7, 0xcc, 0xd9, 0x99, 0x33, + 0x63, 0x4f, 0x03, 0x0d, 0x15, 0xa5, 0x5d, 0xa5, 0x65, 0x22, 0x59, 0x45, 0x45, 0x29, 0xdf, 0x85, + 0x9d, 0x4f, 0xb1, 0x49, 0xce, 0xe3, 0x11, 0x9a, 0xa9, 0x49, 0x70, 0x2c, 0xf0, 0x9e, 0x9f, 0x2f, + 0x92, 0x86, 0xbd, 0x81, 0x66, 0x49, 0x98, 0xc0, 0x6b, 0x57, 0x3a, 0xcd, 0xde, 0x76, 0xd7, 0xe6, + 0x73, 0x84, 0xae, 0x86, 0x1f, 0x03, 0x94, 0x90, 0x31, 0xa8, 0x5e, 0x85, 0xc9, 0x5d, 0xe0, 0xb5, + 0xbd, 0x4e, 0x43, 0xd0, 0x99, 0xb5, 0xa1, 0x29, 0xd0, 0xa4, 0x63, 0x1c, 0xc8, 0x21, 0x4e, 0x02, + 0x9f, 0x42, 0x2e, 0xc5, 0xdf, 0xc3, 0xb3, 0x87, 0x5e, 0xae, 0x51, 0x9b, 0x58, 0x4e, 0x8c, 0xc0, + 0x7b, 0xf6, 0xdc, 0x2d, 0x90, 0x25, 0x76, 0x18, 0x7e, 0xf9, 0xff, 0x8f, 0x0d, 0xeb, 0x41, 0x3d, + 0x87, 0x59, 0x37, 0xfb, 0x73, 0xdd, 0x64, 0x61, 0x51, 0xe8, 0xf8, 0x5f, 0x0f, 0x76, 0x16, 0xe2, + 0xec, 0x1d, 0x54, 0x07, 0x53, 0x85, 0x64, 0x60, 0xab, 0xc7, 0x1f, 0xcf, 0xd2, 0xcd, 0xfe, 0x5a, + 0xa5, 0x20, 0xbd, 0x9d, 0xc8, 0xe7, 0x70, 0x8c, 0x59, 0xdb, 0x74, 0xb6, 0xdc, 0x45, 0x1a, 0x47, + 0x41, 0xa5, 0xed, 0x75, 0xaa, 0x82, 0xce, 0xec, 0x10, 0x1a, 0x27, 0x1a, 0xc3, 0x04, 0x07, 0x37, + 0x17, 0x41, 0x95, 0x02, 0x25, 0xc1, 0x0e, 0xa0, 0x4e, 0x20, 0x96, 0x93, 0x60, 0x8d, 0x32, 0x15, + 0x98, 0xbf, 0x84, 0xa6, 0x53, 0x96, 0x6d, 0x40, 0xbd, 0x3f, 0x09, 0x95, 0xb9, 0x93, 0x49, 0xeb, + 0x89, 0x45, 0xc7, 0x52, 0x0e, 0xc7, 0xa1, 0x1e, 0xb6, 0x3c, 0xfe, 0xc7, 0x83, 0x5a, 0x1f, 0x27, + 0xd1, 0x0a, 0x73, 0xb5, 0x26, 0xcf, 0xb5, 0x1c, 0xe7, 0xc6, 0xed, 0x99, 0x6d, 0x81, 0x3f, 0x90, + 0x64, 0xbb, 0x21, 0xfc, 0x81, 0x9c, 0xbf, 0xda, 0xea, 0xc2, 0xd5, 0x92, 0x71, 0x39, 0x56, 0x1a, + 0x8d, 0x21, 0xe3, 0x75, 0x51, 0x60, 0xb6, 0x07, 0x6b, 0xa7, 0x18, 0xa5, 0x2a, 0x58, 0xa7, 0xc0, + 0x0c, 0xb0, 0x7d, 0x58, 0x3f, 0xd5, 0x53, 0x91, 0x4e, 0x82, 0x1a, 0xd1, 0x19, 0xe2, 0x47, 0x50, + 0xbf, 0xd2, 0x52, 0xa1, 0x4e, 0xa6, 0xc5, 0x50, 0x3d, 0x67, 0xa8, 0x7b, 0xb0, 0x76, 0x1d, 0x8e, + 0xd2, 0x7c, 0xd2, 0x33, 0xc0, 0x7f, 0x15, 0x1d, 0x1b, 0xd6, 0x81, 0xed, 0x2f, 0x06, 0x23, 0xd7, + 0xb1, 0x47, 0x25, 0xe6, 0x69, 0xc6, 0x61, 0xe3, 0xec, 0x87, 0xc2, 0xdb, 0x04, 0xa3, 0x7e, 0xfc, + 0x73, 0x96, 0xb2, 0x22, 0x1e, 0x70, 0xec, 0x35, 0x40, 0xe6, 0x27, 0x46, 0x13, 0x54, 0xe8, 0x71, + 0x6d, 0xd2, 0xb3, 0xc8, 0x6d, 0x0a, 0x47, 0xc0, 0x6f, 0x00, 0x04, 0xde, 0x62, 0xfc, 0x1d, 0x57, + 0x19, 0xfe, 0x2b, 0x68, 0x9d, 0x8c, 0x30, 0xd4, 0xf3, 0x8b, 0x53, 0x17, 0x0b, 0x3c, 0xdf, 0x70, + 0x32, 0x1b, 0x3e, 0x84, 0xdd, 0x53, 0x34, 0x89, 0x96, 0xd3, 0xfc, 0x15, 0xac, 0xb2, 0x45, 0xec, + 0x08, 0x1a, 0x85, 0x3e, 0xf0, 0x97, 0x6e, 0x4a, 0x29, 0xe4, 0x5f, 0x81, 0xcd, 0x15, 0xcb, 0x96, + 0x2e, 0x87, 0x54, 0x69, 0xc9, 0xd2, 0xe5, 0x3a, 0x7b, 0x7b, 0x67, 0x5a, 0x4b, 0x9d, 0xdf, 0x1e, + 0x01, 0xfe, 0xf1, 0xb1, 0x66, 0xec, 0xbf, 0xa9, 0x9a, 0x1d, 0xc0, 0x28, 0xc9, 0x97, 0xfa, 0x29, + 0xe5, 0x5f, 0xb4, 0x22, 0x72, 0x1d, 0xff, 0xed, 0xc1, 0x61, 0x19, 0x50, 0xa3, 0xf8, 0x96, 0x96, + 0xa7, 0x9f, 0x84, 0x49, 0xba, 0xd2, 0x80, 0x0e, 0x9c, 0xa6, 0x66, 0x1e, 0x4b, 0xf3, 0x47, 0xe0, + 0x4b, 0x45, 0x6b, 0xb1, 0xd5, 0x7b, 0x41, 0x56, 0x96, 0x95, 0xea, 0x5e, 0x2a, 0xe1, 0x4b, 0xc5, + 0xdb, 0xe0, 0x5f, 0x2a, 0x56, 0x83, 0xca, 0x05, 0xda, 0x4d, 0xdd, 0x81, 0xcd, 0x3e, 0x16, 0x1f, + 0x60, 0xd4, 0xf2, 0xf8, 0x87, 0xa5, 0x9e, 0x8d, 0xf5, 0x5c, 0xea, 0xb3, 0xb7, 0xec, 0x30, 0xdf, + 0xd6, 0xe9, 0x47, 0xe0, 0xed, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xf0, 0xc1, 0x08, 0xf6, 0x11, + 0x06, 0x00, 0x00, } diff --git a/replication/pdu/pdu.proto b/replication/pdu/pdu.proto index 2e6f045..211fc0e 100644 --- a/replication/pdu/pdu.proto +++ b/replication/pdu/pdu.proto @@ -50,6 +50,8 @@ message SendReq { string ResumeToken = 4; bool Compress = 5; bool Dedup = 6; + + bool DryRun = 7; } message Property { From 1690339440eda19e929483a50bc2077d25dfb992 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 30 Aug 2018 13:30:18 +0200 Subject: [PATCH 078/167] colorized stdout logger if stdout is tty --- Gopkg.lock | 17 +++++++++++++++++ Gopkg.toml | 5 ++++- config/config.go | 1 + daemon/logging/build_logging.go | 5 ++++- daemon/logging/logging_formatters.go | 16 +++++++++------- logger/datastructures.go | 16 ++++++++++++++++ 6 files changed, 51 insertions(+), 9 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index 5e7387e..cda2e51 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -17,6 +17,14 @@ revision = "346938d642f2ec3594ed81d874461961cd0faa76" version = "v1.1.0" +[[projects]] + digest = "1:e988ed0ca0d81f4d28772760c02ee95084961311291bdfefc1b04617c178b722" + name = "github.com/fatih/color" + packages = ["."] + pruneopts = "" + revision = "5b77d2a35fb0ede96d138fc9a99f5c9b6aef11b4" + version = "v1.7.0" + [[projects]] branch = "master" digest = "1:5d0a2385edf4ba44f3b7b76bc0436ceb8f62bf55aa5d540a9eb9ec6c58d86809" @@ -81,6 +89,14 @@ pruneopts = "" revision = "7cafcd837844e784b526369c9bce262804aebc60" +[[projects]] + digest = "1:9ea83adf8e96d6304f394d40436f2eb44c1dc3250d223b74088cc253a6cd0a1c" + name = "github.com/mattn/go-colorable" + packages = ["."] + pruneopts = "" + revision = "167de6bfdfba052fa6b2d3664c8f5272e23c9072" + version = "v0.0.9" + [[projects]] digest = "1:78229b46ddb7434f881390029bd1af7661294af31f6802e0e1bedaad4ab0af3c" name = "github.com/mattn/go-isatty" @@ -255,6 +271,7 @@ analyzer-name = "dep" analyzer-version = 1 input-imports = [ + "github.com/fatih/color", "github.com/go-logfmt/logfmt", "github.com/go-yaml/yaml", "github.com/golang/protobuf/proto", diff --git a/Gopkg.toml b/Gopkg.toml index 0e74efb..ecb2919 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -66,4 +66,7 @@ ignored = [ "github.com/inconshreveable/mousetrap" ] [[constraint]] name = "github.com/nsf/termbox-go" - branch = "master" \ No newline at end of file + branch = "master" +[[constraint]] + name = "github.com/fatih/color" + version = "1.7.0" diff --git a/config/config.go b/config/config.go index 2d618cf..93a7a29 100644 --- a/config/config.go +++ b/config/config.go @@ -197,6 +197,7 @@ type LoggingOutletCommon struct { type StdoutLoggingOutlet struct { LoggingOutletCommon `yaml:",inline"` Time bool `yaml:"time,default=true"` + Color bool `yaml:"color,default=true""` } type SyslogLoggingOutlet struct { diff --git a/daemon/logging/build_logging.go b/daemon/logging/build_logging.go index c6bea78..9d88d41 100644 --- a/daemon/logging/build_logging.go +++ b/daemon/logging/build_logging.go @@ -8,12 +8,12 @@ import ( "github.com/pkg/errors" "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon/pruner" "github.com/zrepl/zrepl/endpoint" "github.com/zrepl/zrepl/logger" "github.com/zrepl/zrepl/replication" "github.com/zrepl/zrepl/tlsconf" "os" - "github.com/zrepl/zrepl/daemon/pruner" ) func OutletsFromConfig(in []config.LoggingOutletEnum) (*logger.Outlets, error) { @@ -145,6 +145,9 @@ func parseStdoutOutlet(in *config.StdoutLoggingOutlet, formatter EntryFormatter) if !isatty.IsTerminal(writer.Fd()) && !in.Time { flags &= ^MetadataTime } + if isatty.IsTerminal(writer.Fd()) && !in.Color { + flags &= ^MetadataColor + } formatter.SetMetadataFlags(flags) return WriterOutlet{ diff --git a/daemon/logging/logging_formatters.go b/daemon/logging/logging_formatters.go index de79ee0..3f3a7a8 100644 --- a/daemon/logging/logging_formatters.go +++ b/daemon/logging/logging_formatters.go @@ -4,6 +4,7 @@ import ( "bytes" "encoding/json" "fmt" + "github.com/fatih/color" "github.com/go-logfmt/logfmt" "github.com/pkg/errors" "github.com/zrepl/zrepl/logger" @@ -26,6 +27,7 @@ type MetadataFlags int64 const ( MetadataTime MetadataFlags = 1 << iota MetadataLevel + MetadataColor MetadataNone MetadataFlags = 0 MetadataAll MetadataFlags = ^0 @@ -69,12 +71,16 @@ func (f *HumanFormatter) ignored(field string) bool { func (f *HumanFormatter) Format(e *logger.Entry) (out []byte, err error) { var line bytes.Buffer + col := color.New() + if f.metadataFlags&MetadataColor != 0 { + col = e.Color() + } if f.metadataFlags&MetadataTime != 0 { fmt.Fprintf(&line, "%s ", e.Time.Format(HumanFormatterDateFormat)) } if f.metadataFlags&MetadataLevel != 0 { - fmt.Fprintf(&line, "[%s]", e.Level.Short()) + fmt.Fprintf(&line, "[%s]", col.Sprint(e.Level.Short())) } prefixFields := []string{JobField, SubsysField} @@ -85,7 +91,7 @@ func (f *HumanFormatter) Format(e *logger.Entry) (out []byte, err error) { continue } if !f.ignored(field) { - fmt.Fprintf(&line, "[%s]", val) + fmt.Fprintf(&line, "[%s]", col.Sprint(val)) prefixed[field] = true } } @@ -96,15 +102,11 @@ func (f *HumanFormatter) Format(e *logger.Entry) (out []byte, err error) { fmt.Fprint(&line, e.Message) if len(e.Fields)-len(prefixed) > 0 { - fmt.Fprint(&line, " ") - enc := logfmt.NewEncoder(&line) for field, value := range e.Fields { if prefixed[field] || f.ignored(field) { continue } - if err := logfmtTryEncodeKeyval(enc, field, value); err != nil { - return nil, err - } + fmt.Fprintf(&line, " %s=%q", col.Sprint(field), fmt.Sprint(value)) } } diff --git a/logger/datastructures.go b/logger/datastructures.go index d6460ee..782c8e9 100644 --- a/logger/datastructures.go +++ b/logger/datastructures.go @@ -4,6 +4,7 @@ import ( "bytes" "encoding/json" "fmt" + "github.com/fatih/color" "github.com/pkg/errors" "sync" "time" @@ -105,6 +106,21 @@ type Entry struct { Fields Fields } +func (e Entry) Color() *color.Color { + c := color.New() + switch e.Level { + case Debug: + c.Add(color.FgHiBlue) + case Info: + c.Add(color.FgHiGreen) + case Warn: + c.Add(color.FgHiYellow) + case Error: + c.Add(color.FgHiRed) + } + return c +} + // An outlet receives log entries produced by the Logger and writes them to some destination. type Outlet interface { // Write the entry to the destination. From acd2a68cfb7100c6b75137a496dd60b268637c01 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 30 Aug 2018 13:40:28 +0200 Subject: [PATCH 079/167] fix build: bump yaml-config --- Gopkg.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index cda2e51..4c185ab 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -253,11 +253,11 @@ [[projects]] branch = "v2" - digest = "1:f0620375dd1f6251d9973b5f2596228cc8042e887cd7f827e4220bc1ce8c30e2" + digest = "1:1d4340fa24f9e4a81097fb8e56f509e528bfc7456e2e9f672831c7d9e9cb8109" name = "github.com/zrepl/yaml-config" packages = ["."] pruneopts = "" - revision = "5420a8b6744d3b0345ab293f6fcba19c978f1183" + revision = "3f4c23177f5097f12b95dd3d053481015815a6cd" [[projects]] branch = "master" From 48feaff0543bada8b74463eda225721a1866a755 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Thu, 30 Aug 2018 14:07:28 +0200 Subject: [PATCH 080/167] fix some status display alignment --- client/status.go | 23 +++++++++++++++++++++-- replication/fsrep/fsfsm.go | 4 ++-- 2 files changed, 23 insertions(+), 4 deletions(-) diff --git a/client/status.go b/client/status.go index cc171a4..25e9212 100644 --- a/client/status.go +++ b/client/status.go @@ -199,6 +199,9 @@ func times(str string, n int) (out string) { } func rightPad(str string, length int, pad string) string { + if len(str) > length { + return str[:length] + } return str + times(pad, length-len(str)) } @@ -211,6 +214,9 @@ func (t *tui) drawBar(name string, status string, bytes int64, totalBytes int64) if totalBytes > 0 { length := 50 completedLength := int(int64(length) * bytes / totalBytes) + if completedLength > length { + completedLength = length + } t.write("[") t.write(times("=", completedLength)) @@ -219,12 +225,25 @@ func (t *tui) drawBar(name string, status string, bytes int64, totalBytes int64) t.write("]") t.write(" ") - t.write(rightPad(ByteCountBinary(bytes) + "/" + ByteCountBinary(bytes), 16, " ")) + t.write(rightPad(ByteCountBinary(bytes) + "/" + ByteCountBinary(totalBytes), 20, " ")) } t.newline() } +func StringStepState(s fsrep.StepState) string { + switch s { + case fsrep.StepReplicationReady: return "Ready" + case fsrep.StepReplicationRetry: return "Retry" + case fsrep.StepMarkReplicatedReady: return "MarkReady" + case fsrep.StepMarkReplicatedRetry: return "MarkRetry" + case fsrep.StepPermanentError: return "PermanentError" + case fsrep.StepCompleted: return "Completed" + default: + return fmt.Sprintf("UNKNOWN %d", s) + } +} + func printFilesystem(rep *fsrep.Report, t *tui, versions bool) { bytes := int64(0) totalBytes := int64(0) @@ -246,7 +265,7 @@ func printFilesystem(rep *fsrep.Report, t *tui, versions bool) { } if versions && len(rep.Pending) > 0 { v := rep.Pending[0] - t.drawBar(" " + v.To, v.Status, v.Bytes, v.ExpectedBytes) + t.drawBar(" " + v.To, StringStepState(v.Status), v.Bytes, v.ExpectedBytes) } } diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index 2462cba..df550cc 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -56,7 +56,7 @@ type Receiver interface { type StepReport struct { From, To string - Status string + Status StepState Problem string Bytes int64 ExpectedBytes int64 @@ -498,7 +498,7 @@ func (s *ReplicationStep) Report() *StepReport { rep := StepReport{ From: from, To: s.to.RelName(), - Status: s.state.String(), + Status: s.state, Bytes: bytes, ExpectedBytes: s.expectedSize, } From 5442d8e7d5f328702aa43c178e8acb1db3b6e19a Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Thu, 30 Aug 2018 15:20:44 +0200 Subject: [PATCH 081/167] status: calculate max fs name length --- client/status.go | 40 ++++++++++++++++++++++++++++++++-------- 1 file changed, 32 insertions(+), 8 deletions(-) diff --git a/client/status.go b/client/status.go index 25e9212..5182772 100644 --- a/client/status.go +++ b/client/status.go @@ -176,14 +176,16 @@ func (t *tui) draw() { t.newline() } + maxNameLength := calculateMaxFilesystemAndVersionNameLength(&rep) + for _, fs := range rep.Completed { - printFilesystem(fs, t, false) + printFilesystem(fs, t, false, maxNameLength) } if rep.Active != nil { - printFilesystem(rep.Active, t, true) + printFilesystem(rep.Active, t, true, maxNameLength) } for _, fs := range rep.Pending { - printFilesystem(fs, t, false) + printFilesystem(fs, t, false, maxNameLength) } } @@ -191,6 +193,28 @@ func (t *tui) draw() { termbox.Flush() } +const snapshotIndent = 1 +func calculateMaxFilesystemAndVersionNameLength(report *replication.Report) int { + all := append(report.Completed, report.Active) + all = append(all, report.Pending...) + maxLen := 0 + for _, e := range all { + if e == nil { //active can be nil + continue + } + if len(e.Filesystem) > maxLen { + maxLen = len(e.Filesystem) + } + all2 := append(e.Pending, e.Completed...) + for _, e2 := range all2 { + if len(e2.To) + snapshotIndent > maxLen { + maxLen = len(e2.To) + snapshotIndent + } + } + } + return maxLen +} + func times(str string, n int) (out string) { for i := 0; i < n; i++ { out += str @@ -205,8 +229,8 @@ func rightPad(str string, length int, pad string) string { return str + times(pad, length-len(str)) } -func (t *tui) drawBar(name string, status string, bytes int64, totalBytes int64) { - t.write(rightPad(name, 20, " ")) +func (t *tui) drawBar(name string, maxNameLength int, status string, bytes int64, totalBytes int64) { + t.write(rightPad(name, maxNameLength + 1, " ")) t.write(" ") t.write(rightPad(status, 14, " ")) t.write(" ") @@ -244,7 +268,7 @@ func StringStepState(s fsrep.StepState) string { } } -func printFilesystem(rep *fsrep.Report, t *tui, versions bool) { +func printFilesystem(rep *fsrep.Report, t *tui, versions bool, maxNameLength int) { bytes := int64(0) totalBytes := int64(0) for _, s := range rep.Pending { @@ -256,7 +280,7 @@ func printFilesystem(rep *fsrep.Report, t *tui, versions bool) { totalBytes += s.ExpectedBytes } - t.drawBar(rep.Filesystem, rep.Status, bytes, totalBytes) + t.drawBar(rep.Filesystem, maxNameLength, rep.Status, bytes, totalBytes) if rep.Problem != "" { t.addIndent(1) t.printf("Problem: %s", rep.Problem) @@ -265,7 +289,7 @@ func printFilesystem(rep *fsrep.Report, t *tui, versions bool) { } if versions && len(rep.Pending) > 0 { v := rep.Pending[0] - t.drawBar(" " + v.To, StringStepState(v.Status), v.Bytes, v.ExpectedBytes) + t.drawBar(times(" ", snapshotIndent) + v.To, maxNameLength, StringStepState(v.Status), v.Bytes, v.ExpectedBytes) } } From c0a3e1f1210c54350cb8adc5a6cfa7f6f962e0d4 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Thu, 30 Aug 2018 17:40:02 +0200 Subject: [PATCH 082/167] wrap error in buildJob with job name --- daemon/job/build_jobs.go | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/daemon/job/build_jobs.go b/daemon/job/build_jobs.go index f062918..8843b55 100644 --- a/daemon/job/build_jobs.go +++ b/daemon/job/build_jobs.go @@ -3,6 +3,7 @@ package job import ( "fmt" "github.com/zrepl/zrepl/config" + "github.com/pkg/errors" ) func JobsFromConfig(c config.Config) ([]Job, error) { @@ -18,14 +19,19 @@ func JobsFromConfig(c config.Config) ([]Job, error) { } func buildJob(c config.Global, in config.JobEnum) (j Job, err error) { - switch v := in.Ret.(type) { case *config.SinkJob: - return SinkFromConfig(c, v) + j, err = SinkFromConfig(c, v) + if err != nil { + return nil, errors.Wrapf(err, "cannot build job %q", v.Name) + } case *config.PushJob: - return PushFromConfig(c, v) + j, err = PushFromConfig(c, v) + if err != nil { + return nil, errors.Wrapf(err, "cannot build job %q", v.Name) + } default: panic(fmt.Sprintf("implementation error: unknown job type %T", v)) } - + return j, err } From 32391adf4f3be6b902b9e70a5c0cea24990b097e Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Thu, 30 Aug 2018 17:40:45 +0200 Subject: [PATCH 083/167] build pruner in factory and check prune rules --- daemon/job/push.go | 22 ++++--------- daemon/pruner/pruner.go | 71 ++++++++++++++++++++++++++++++++++++++--- 2 files changed, 74 insertions(+), 19 deletions(-) diff --git a/daemon/job/push.go b/daemon/job/push.go index 97746f7..62fbb24 100644 --- a/daemon/job/push.go +++ b/daemon/job/push.go @@ -10,10 +10,8 @@ import ( "github.com/zrepl/zrepl/daemon/logging" "github.com/zrepl/zrepl/daemon/pruner" "github.com/zrepl/zrepl/endpoint" - "github.com/zrepl/zrepl/pruning" "github.com/zrepl/zrepl/replication" "sync" - "time" ) type Push struct { @@ -21,8 +19,7 @@ type Push struct { connecter streamrpc.Connecter fsfilter endpoint.FSFilter - keepRulesSender []pruning.KeepRule - keepRulesReceiver []pruning.KeepRule + prunerFactory *pruner.PrunerFactory mtx sync.Mutex replication *replication.Replication @@ -39,14 +36,9 @@ func PushFromConfig(g config.Global, in *config.PushJob) (j *Push, err error) { return nil, errors.Wrap(err, "cannnot build filesystem filter") } - j.keepRulesReceiver, err = pruning.RulesFromConfig(in.Pruning.KeepReceiver) + j.prunerFactory, err = pruner.NewPrunerFactory(in.Pruning) if err != nil { - return nil, errors.Wrap(err, "cannot build receiver pruning rules") - } - - j.keepRulesSender, err = pruning.RulesFromConfig(in.Pruning.KeepSender) - if err != nil { - return nil, errors.Wrap(err, "cannot build sender pruning rules") + return nil, err } return j, nil @@ -112,11 +104,11 @@ func (j *Push) do(ctx context.Context) { j.replication.Drive(ctx, sender, receiver) // Prune sender - senderPruner := pruner.NewPruner(10*time.Second, sender, sender, j.keepRulesSender) // FIXME constant - senderPruner.Prune(pruner.WithLogger(ctx, pruner.GetLogger(ctx).WithField("prune_side", "sender"))) + senderPruner := j.prunerFactory.BuildSenderPruner(ctx, sender, sender) + senderPruner.Prune() // Prune receiver - receiverPruner := pruner.NewPruner(10*time.Second, receiver, sender, j.keepRulesReceiver) // FIXME constant - receiverPruner.Prune(pruner.WithLogger(ctx, pruner.GetLogger(ctx).WithField("prune_side", "receiver"))) + receiverPruner := j.prunerFactory.BuildReceiverPruner(ctx, receiver, sender) + receiverPruner.Prune() } diff --git a/daemon/pruner/pruner.go b/daemon/pruner/pruner.go index 8041cc0..f912a90 100644 --- a/daemon/pruner/pruner.go +++ b/daemon/pruner/pruner.go @@ -3,6 +3,8 @@ package pruner import ( "context" "fmt" + "github.com/pkg/errors" + "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/logger" "github.com/zrepl/zrepl/pruning" "github.com/zrepl/zrepl/replication/pdu" @@ -63,9 +65,71 @@ type Pruner struct { pruneCompleted []fs } -func NewPruner(retryWait time.Duration, target Target, receiver History, rules []pruning.KeepRule) *Pruner { +type PrunerFactory struct { + senderRules []pruning.KeepRule + receiverRules []pruning.KeepRule + retryWait time.Duration +} + +func checkContainsKeep1(rules []pruning.KeepRule) error { + if len(rules) == 0 { + return nil //No keep rules means keep all - ok + } + for _, e := range rules { + switch e.(type) { + case *pruning.KeepLastN: + return nil + } + } + return errors.New("sender keep rules must contain last_n or be empty so that the last snapshot is definitely kept") +} + +func NewPrunerFactory(in config.PruningSenderReceiver) (*PrunerFactory, error) { + keepRulesReceiver, err := pruning.RulesFromConfig(in.KeepReceiver) + if err != nil { + return nil, errors.Wrap(err, "cannot build receiver pruning rules") + } + + keepRulesSender, err := pruning.RulesFromConfig(in.KeepSender) + if err != nil { + return nil, errors.Wrap(err, "cannot build sender pruning rules") + } + + if err := checkContainsKeep1(keepRulesSender); err != nil { + return nil, err + } + + f := &PrunerFactory{ + keepRulesSender, + keepRulesReceiver, + 10 * time.Second, //FIXME constant + } + return f, nil +} + +func (f *PrunerFactory) BuildSenderPruner(ctx context.Context, target Target, receiver History) *Pruner { p := &Pruner{ - args: args{nil, target, receiver, rules, retryWait}, // ctx is filled in Prune() + args: args{ + WithLogger(ctx, GetLogger(ctx).WithField("prune_side", "sender")), + target, + receiver, + f.senderRules, + f.retryWait, + }, + state: Plan, + } + return p +} + +func (f *PrunerFactory) BuildReceiverPruner(ctx context.Context, target Target, receiver History) *Pruner { + p := &Pruner{ + args: args{ + WithLogger(ctx, GetLogger(ctx).WithField("prune_side", "receiver")), + target, + receiver, + f.receiverRules, + f.retryWait, + }, state: Plan, } return p @@ -98,8 +162,7 @@ func (s State) statefunc() state { type updater func(func(*Pruner)) State type state func(args *args, u updater) state -func (p *Pruner) Prune(ctx context.Context) { - p.args.ctx = ctx +func (p *Pruner) Prune() { p.prune(p.args) } From f387e2321485021a167a59b212255ff8ff3f4444 Mon Sep 17 00:00:00 2001 From: Anton Schirg Date: Thu, 30 Aug 2018 17:50:43 +0200 Subject: [PATCH 084/167] fix: at least two snapshots were needed to start replication --- replication/mainfsm.go | 2 +- zfs/versions.go | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/replication/mainfsm.go b/replication/mainfsm.go index d30f417..1cae748 100644 --- a/replication/mainfsm.go +++ b/replication/mainfsm.go @@ -211,7 +211,7 @@ func statePlanning(ctx context.Context, sender Sender, receiver Receiver, u upda return handlePlanningError(err) } - if len(sfsvs) <= 1 { + if len(sfsvs) < 1 { err := errors.New("sender does not have any versions") log.Error(err.Error()) q.Add(fsrep.NewReplicationWithPermanentError(fs.Path, err)) diff --git a/zfs/versions.go b/zfs/versions.go index daf5bdc..9deb10d 100644 --- a/zfs/versions.go +++ b/zfs/versions.go @@ -87,7 +87,6 @@ type FilesystemVersionFilter interface { Filter(t VersionType, name string) (accept bool, err error) } -//FIXME Seems to always forget first snapshot func ZFSListFilesystemVersions(fs *DatasetPath, filter FilesystemVersionFilter) (res []FilesystemVersion, err error) { listResults := make(chan ZFSListResult) From b95e983d0d7b3102a7917f7aca8aef215b1ab2be Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 31 Aug 2018 16:26:11 +0200 Subject: [PATCH 085/167] bump go-streamrpc to 0.2, cleanup logging logging should be user-friendly in INFO mode --- Gopkg.lock | 6 +++--- Gopkg.toml | 8 ++++---- daemon/control.go | 4 ++-- daemon/job/job.go | 5 +---- daemon/job/push.go | 20 +++++++++++--------- daemon/job/sink.go | 2 +- daemon/logging/adaptors.go | 2 +- endpoint/context.go | 7 +++---- endpoint/endpoint.go | 8 ++++++++ replication/fsrep/fsfsm.go | 17 ++++++++++------- replication/mainfsm.go | 20 ++++++++++++++++---- 11 files changed, 60 insertions(+), 39 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index 4c185ab..e6bdc39 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -170,15 +170,15 @@ revision = "391d2c78c8404a9683d79f75dd24ab53040f89f7" [[projects]] - digest = "1:f80c558f761d9cbfb0cd1a317ece8b12a55ec670f8cef52acc515ab76463e6b6" + digest = "1:23576326ee8d287fd295807d5de5db9fcc9572ec2b4123c8ec2394a683edf1e1" name = "github.com/problame/go-streamrpc" packages = [ ".", "internal/pdu", ] pruneopts = "" - revision = "de264a1c39cd8fc42a6b5e902c6eac7fd9683521" - version = "0.1" + revision = "504ffed1faf6af51c057d7b11d79e9367678c666" + version = "v0.2" [[projects]] branch = "master" diff --git a/Gopkg.toml b/Gopkg.toml index ecb2919..49065ab 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -56,10 +56,6 @@ ignored = [ "github.com/inconshreveable/mousetrap" ] name = "github.com/prometheus/client_golang" branch = "master" -[[constraint]] - name = "github.com/problame/go-streamrpc" - version = "0.1.0" - [[constraint]] name = "github.com/golang/protobuf" version = "1.2.0" @@ -70,3 +66,7 @@ ignored = [ "github.com/inconshreveable/mousetrap" ] [[constraint]] name = "github.com/fatih/color" version = "1.7.0" + +[[constraint]] + name = "github.com/problame/go-streamrpc" + version = "0.2.0" diff --git a/daemon/control.go b/daemon/control.go index c7e1307..35a278c 100644 --- a/daemon/control.go +++ b/daemon/control.go @@ -185,7 +185,7 @@ type requestLogger struct { func (l requestLogger) ServeHTTP(w http.ResponseWriter, r *http.Request) { log := l.log.WithField("method", r.Method).WithField("url", r.URL) - log.Info("start") + log.Debug("start") if l.handlerFunc != nil { l.handlerFunc(w, r) } else if l.handler != nil { @@ -193,5 +193,5 @@ func (l requestLogger) ServeHTTP(w http.ResponseWriter, r *http.Request) { } else { log.Error("no handler or handlerFunc configured") } - log.Info("finish") + log.Debug("finish") } diff --git a/daemon/job/job.go b/daemon/job/job.go index 4ea199a..b95f852 100644 --- a/daemon/job/job.go +++ b/daemon/job/job.go @@ -64,10 +64,7 @@ var STREAMRPC_CONFIG = &streamrpc.ConnConfig{ // FIXME oversight and configurabi RxStructuredMaxLen: 4096 * 4096, RxStreamMaxChunkSize: 4096 * 4096, TxChunkSize: 4096 * 4096, - RxTimeout: streamrpc.Timeout{ - Progress: 10 * time.Second, - }, - TxTimeout: streamrpc.Timeout{ + Timeout: streamrpc.Timeout{ Progress: 10 * time.Second, }, } diff --git a/daemon/job/push.go b/daemon/job/push.go index 62fbb24..e4750c2 100644 --- a/daemon/job/push.go +++ b/daemon/job/push.go @@ -7,11 +7,11 @@ import ( "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon/connecter" "github.com/zrepl/zrepl/daemon/filters" - "github.com/zrepl/zrepl/daemon/logging" "github.com/zrepl/zrepl/daemon/pruner" "github.com/zrepl/zrepl/endpoint" "github.com/zrepl/zrepl/replication" "sync" + "github.com/zrepl/zrepl/daemon/logging" ) type Push struct { @@ -66,11 +66,10 @@ func (j *Push) Run(ctx context.Context) { defer log.Info("job exiting") - log.Debug("wait for wakeups") - invocationCount := 0 outer: for { + log.Info("wait for wakeups") select { case <-ctx.Done(): log.WithError(ctx.Err()).Info("context") @@ -86,12 +85,13 @@ outer: func (j *Push) do(ctx context.Context) { log := GetLogger(ctx) + ctx = logging.WithSubsystemLoggers(ctx, log) client, err := streamrpc.NewClient(j.connecter, &streamrpc.ClientConfig{STREAMRPC_CONFIG}) if err != nil { log.WithError(err).Error("cannot create streamrpc client") } - defer client.Close() + defer client.Close(ctx) sender := endpoint.NewSender(j.fsfilter, filters.NewAnyFSVFilter()) receiver := endpoint.NewRemote(client) @@ -100,15 +100,17 @@ func (j *Push) do(ctx context.Context) { j.replication = replication.NewReplication() j.mtx.Unlock() - ctx = logging.WithSubsystemLoggers(ctx, log) + log.Info("start replication") j.replication.Drive(ctx, sender, receiver) - // Prune sender - senderPruner := j.prunerFactory.BuildSenderPruner(ctx, sender, sender) + log.Info("start pruning sender") + psCtx := pruner.WithLogger(ctx, pruner.GetLogger(ctx).WithField("prune_side", "sender")) + senderPruner := j.prunerFactory.BuildSenderPruner(psCtx, sender, sender) // FIXME ctx as member senderPruner.Prune() - // Prune receiver - receiverPruner := j.prunerFactory.BuildReceiverPruner(ctx, receiver, sender) + log.Info("start pruning receiver") + prCtx := pruner.WithLogger(ctx, pruner.GetLogger(ctx).WithField("prune_side", "receiver")) + receiverPruner := j.prunerFactory.BuildReceiverPruner(prCtx, receiver, sender) // FIXME ctx as member receiverPruner.Prune() } diff --git a/daemon/job/sink.go b/daemon/job/sink.go index 618ced7..8c9f496 100644 --- a/daemon/job/sink.go +++ b/daemon/job/sink.go @@ -86,7 +86,7 @@ func (j *Sink) handleConnection(ctx context.Context, conn net.Conn) { log.WithField("addr", conn.RemoteAddr()).Info("handling connection") defer log.Info("finished handling connection") - logging.WithSubsystemLoggers(ctx, log) + ctx = logging.WithSubsystemLoggers(ctx, log) local, err := endpoint.NewReceiver(j.fsmap, filters.NewAnyFSVFilter()) if err != nil { diff --git a/daemon/logging/adaptors.go b/daemon/logging/adaptors.go index 7f0b21b..c5a7196 100644 --- a/daemon/logging/adaptors.go +++ b/daemon/logging/adaptors.go @@ -28,5 +28,5 @@ func (a twoClassLogAdaptor) Errorf(fmtStr string, args ...interface{}) { } func (a twoClassLogAdaptor) Infof(fmtStr string, args ...interface{}) { - a.Logger.Info(fmt.Sprintf(fmtStr, args...)) + a.Logger.Debug(fmt.Sprintf(fmtStr, args...)) } diff --git a/endpoint/context.go b/endpoint/context.go index a528930..09f9032 100644 --- a/endpoint/context.go +++ b/endpoint/context.go @@ -18,9 +18,8 @@ func WithLogger(ctx context.Context, log Logger) context.Context { } func getLogger(ctx context.Context) Logger { - l, ok := ctx.Value(contextKeyLogger).(Logger) - if !ok { - l = logger.NewNullLogger() + if l, ok := ctx.Value(contextKeyLogger).(Logger); ok { + return l } - return l + return logger.NewNullLogger() } diff --git a/endpoint/endpoint.go b/endpoint/endpoint.go index 8a26730..561e509 100644 --- a/endpoint/endpoint.go +++ b/endpoint/endpoint.go @@ -230,6 +230,8 @@ func (e *Receiver) Receive(ctx context.Context, req *pdu.ReceiveReq, sendStream return errors.New("receive to filesystem denied") } + getLogger(ctx).Debug("incoming Receive") + // create placeholder parent filesystems as appropriate var visitErr error f := zfs.NewDatasetPathForest() @@ -276,6 +278,11 @@ func (e *Receiver) Receive(ctx context.Context, req *pdu.ReceiveReq, sendStream getLogger(ctx).Debug("start receive command") if err := zfs.ZFSRecv(lp.ToString(), sendStream, args...); err != nil { + getLogger(ctx). + WithError(err). + WithField("args", args). + Error("zfs receive failed") + sendStream.Close() return err } return nil @@ -418,6 +425,7 @@ func (s Remote) Receive(ctx context.Context, r *pdu.ReceiveReq, sendStream io.Re return err } rb, rs, err := s.c.RequestReply(ctx, RPCReceive, bytes.NewBuffer(b), sendStream) + getLogger(ctx).WithField("err", err).Debug("Remote.Receive RequestReplyReturned") if err != nil { return err } diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index df550cc..2203639 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -357,7 +357,7 @@ func (s *ReplicationStep) doReplication(ctx context.Context, sender Sender, rece sr := s.buildSendRequest(false) - log.WithField("request", sr).Debug("initiate send request") + log.Debug("initiate send request") sres, sstream, err := sender.Send(ctx, sr) if err != nil { log.WithError(err).Error("send request failed") @@ -375,10 +375,13 @@ func (s *ReplicationStep) doReplication(ctx context.Context, sender Sender, rece Filesystem: fs, ClearResumeToken: !sres.UsedResumeToken, } - log.WithField("request", rr).Debug("initiate receive request") + log.Debug("initiate receive request") err = receiver.Receive(ctx, rr, sstream) if err != nil { - log.WithError(err).Error("receive request failed (might also be error on sender)") + log. + WithError(err). + WithField("errType", fmt.Sprintf("%T", err)). + Error("receive request failed (might also be error on sender)") sstream.Close() // This failure could be due to // - an unexpected exit of ZFS on the sending side @@ -386,7 +389,7 @@ func (s *ReplicationStep) doReplication(ctx context.Context, sender Sender, rece // - a connectivity issue return updateStateError(err) } - log.Info("receive finished") + log.Debug("receive finished") updateStateCompleted() @@ -420,7 +423,7 @@ func (s *ReplicationStep) doMarkReplicated(ctx context.Context, sender Sender) S return s.state } - log.Info("mark snapshot as replicated") + log.Debug("mark snapshot as replicated") req := pdu.SnapshotReplicationStatusReq{ Filesystem: s.parent.fs, Snapshot: s.to.GetName(), @@ -450,7 +453,7 @@ func (s *ReplicationStep) updateSizeEstimate(ctx context.Context, sender Sender) sr := s.buildSendRequest(true) - log.WithField("request", sr).Debug("initiate dry run send request") + log.Debug("initiate dry run send request") sres, _, err := sender.Send(ctx, sr) if err != nil { log.WithError(err).Error("dry run send request failed") @@ -482,7 +485,7 @@ func (s *ReplicationStep) String() string { if s.from == nil { // FIXME: ZFS semantics are that to is nil on non-incremental send return fmt.Sprintf("%s%s (full)", s.parent.fs, s.to.RelName()) } else { - return fmt.Sprintf("%s(%s => %s)", s.parent.fs, s.from, s.to.RelName()) + return fmt.Sprintf("%s(%s => %s)", s.parent.fs, s.from.RelName(), s.to.RelName()) } } diff --git a/replication/mainfsm.go b/replication/mainfsm.go index 1cae748..c829442 100644 --- a/replication/mainfsm.go +++ b/replication/mainfsm.go @@ -147,6 +147,9 @@ func (r *Replication) Drive(ctx context.Context, sender Sender, receiver Receive WithField("transition", fmt.Sprintf("%s => %s", pre, post)). WithField("duration", delta). Debug("main state transition") + if post == Working && pre != post { + getLogger(ctx).Info("start working") + } } getLogger(ctx). @@ -178,6 +181,8 @@ func statePlanning(ctx context.Context, sender Sender, receiver Receiver, u upda log := getLogger(ctx) + log.Info("start planning") + handlePlanningError := func(err error) state { return u(func(r *Replication) { r.planningError = err @@ -203,7 +208,7 @@ func statePlanning(ctx context.Context, sender Sender, receiver Receiver, u upda log := mainlog.WithField("filesystem", fs.Path) - log.Info("assessing filesystem") + log.Debug("assessing filesystem") sfsvs, err := sender.ListFilesystemVersions(ctx, fs.Path) if err != nil { @@ -266,8 +271,10 @@ func statePlanning(ctx context.Context, sender Sender, receiver Receiver, u upda } } qitem := fsrfsm.Done() + + log.Debug("compute send size estimate") if err = qitem.UpdateSizeEsitmate(ctx, sender); err != nil { - log.WithError(err).Error("cannot get size estimate") + log.WithError(err).Error("error computing size estimate") return handlePlanningError(err) } q.Add(qitem) @@ -284,10 +291,13 @@ func statePlanning(ctx context.Context, sender Sender, receiver Receiver, u upda var RetrySleepDuration = 10 * time.Second // FIXME make constant onfigurable func statePlanningError(ctx context.Context, sender Sender, receiver Receiver, u updater) state { + + sleepUntil := time.Now().Add(RetrySleepDuration) u(func(r *Replication) { - r.sleepUntil = time.Now().Add(RetrySleepDuration) + r.sleepUntil = sleepUntil }) t := time.NewTimer(RetrySleepDuration) + getLogger(ctx).WithField("until", sleepUntil).Info("retry wait after planning error") defer t.Stop() select { case <-ctx.Done(): @@ -328,10 +338,12 @@ func stateWorking(ctx context.Context, sender Sender, receiver Receiver, u updat } func stateWorkingWait(ctx context.Context, sender Sender, receiver Receiver, u updater) state { + sleepUntil := time.Now().Add(RetrySleepDuration) u(func(r *Replication) { - r.sleepUntil = time.Now().Add(RetrySleepDuration) + r.sleepUntil = sleepUntil }) t := time.NewTimer(RetrySleepDuration) + getLogger(ctx).WithField("until", sleepUntil).Info("retry wait after send/recv error") defer t.Stop() select { case <-ctx.Done(): From d55a271ac7987ab01fa17b737e36b99571922b66 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 31 Aug 2018 21:50:59 +0200 Subject: [PATCH 086/167] WIP adopt updated yaml-config with 'fromdefaults' struct tag --- client/status.go | 2 +- client/wakeup.go | 2 +- config/config.go | 76 ++++++++++++++++++++++++++----- config/config_minimal_test.go | 41 +++++++++++++++++ config/config_rpc_test.go | 61 +++++++++++++++++++++++++ config/config_test.go | 26 +++++++++++ daemon/connecter/connecter.go | 2 +- daemon/daemon.go | 4 +- daemon/job/build_jobs.go | 4 +- daemon/job/push.go | 2 +- daemon/job/sink.go | 2 +- daemon/logging/build_logging.go | 2 +- daemon/serve/serve.go | 2 +- daemon/serve/serve_stdinserver.go | 2 +- daemon/serve/serve_tcp.go | 2 +- daemon/serve/serve_tls.go | 2 +- 16 files changed, 206 insertions(+), 26 deletions(-) create mode 100644 config/config_minimal_test.go create mode 100644 config/config_rpc_test.go diff --git a/client/status.go b/client/status.go index 5182772..7847c84 100644 --- a/client/status.go +++ b/client/status.go @@ -62,7 +62,7 @@ func (t *tui) addIndent(indent int) { t.moveLine(0, 0) } -func RunStatus(config config.Config, args []string) error { +func RunStatus(config *config.Config, args []string) error { httpc, err := controlHttpClient(config.Global.Control.SockPath) if err != nil { return err diff --git a/client/wakeup.go b/client/wakeup.go index 97fb180..d6a224f 100644 --- a/client/wakeup.go +++ b/client/wakeup.go @@ -6,7 +6,7 @@ import ( "github.com/zrepl/zrepl/daemon" ) -func RunWakeup(config config.Config, args []string) error { +func RunWakeup(config *config.Config, args []string) error { if len(args) != 1 { return errors.Errorf("Expected 1 argument: job") } diff --git a/config/config.go b/config/config.go index 93a7a29..d6a334f 100644 --- a/config/config.go +++ b/config/config.go @@ -9,11 +9,12 @@ import ( "regexp" "strconv" "time" + "reflect" ) type Config struct { Jobs []JobEnum `yaml:"jobs"` - Global Global `yaml:"global"` + Global *Global `yaml:"global,optional,fromdefaults"` } type JobEnum struct { @@ -102,11 +103,51 @@ type PruningLocal struct { Keep []PruningEnum `yaml:"keep"` } +type LoggingOutletEnumList []LoggingOutletEnum + +func (l *LoggingOutletEnumList) SetDefault() { + def := ` +type: "stdout" +time: true +level: "warn" +format: "human" +` + s := StdoutLoggingOutlet{} + err := yaml.UnmarshalStrict([]byte(def), &s) + if err != nil { + panic(err) + } + *l = []LoggingOutletEnum{LoggingOutletEnum{Ret: s}} +} + +var _ yaml.Defaulter = &LoggingOutletEnumList{} + type Global struct { - Logging []LoggingOutletEnum `yaml:"logging"` + Logging *LoggingOutletEnumList `yaml:"logging,optional,fromdefaults"` Monitoring []MonitoringEnum `yaml:"monitoring,optional"` - Control GlobalControl `yaml:"control"` - Serve GlobalServe `yaml:"serve"` + Control *GlobalControl `yaml:"control,optional,fromdefaults"` + Serve *GlobalServe `yaml:"serve,optional,fromdefaults"` + RPC *RPCConfig `yaml:"rpc,optional,fromdefaults"` +} + +func Default(i interface{}) { + v := reflect.ValueOf(i) + if v.Kind() != reflect.Ptr { + panic(v) + } + y := `{}` + err := yaml.Unmarshal([]byte(y), v.Interface()) + if err != nil { + panic(err) + } +} + +type RPCConfig struct { + Timeout time.Duration `yaml:"timeout,optional,positive,default=10s"` + TxChunkSize uint `yaml:"tx_chunk_size,optional,default=32768"` + RxStructuredMaxLen uint `yaml:"rx_structured_max,optional,default=16777216"` + RxStreamChunkMaxLen uint `yaml:"rx_stream_chunk_max,optional,default=16777216"` + RxHeaderMaxLen uint `yaml:"rx_header_max,optional,default=40960"` } type ConnectEnum struct { @@ -115,12 +156,14 @@ type ConnectEnum struct { type TCPConnect struct { Type string `yaml:"type"` + RPC *RPCConfig `yaml:"rpc,optional"` Address string `yaml:"address"` DialTimeout time.Duration `yaml:"dial_timeout,positive,default=10s"` } type TLSConnect struct { Type string `yaml:"type"` + RPC *RPCConfig `yaml:"rpc,optional"` Address string `yaml:"address"` Ca string `yaml:"ca"` Cert string `yaml:"cert"` @@ -131,6 +174,7 @@ type TLSConnect struct { type SSHStdinserverConnect struct { Type string `yaml:"type"` + RPC *RPCConfig `yaml:"rpc,optional"` Host string `yaml:"host"` User string `yaml:"user"` Port uint16 `yaml:"port"` @@ -233,7 +277,7 @@ type GlobalControl struct { } type GlobalServe struct { - StdinServer GlobalStdinServer `yaml:"stdinserver"` + StdinServer *GlobalStdinServer `yaml:"stdinserver,optional,fromdefaults"` } type GlobalStdinServer struct { @@ -241,10 +285,11 @@ type GlobalStdinServer struct { } type JobDebugSettings struct { - Conn struct { + Conn *struct { ReadDump string `yaml:"read_dump"` WriteDump string `yaml:"write_dump"` - } `yaml:"conn"` + } `yaml:"conn,optional"` + RPCLog bool `yaml:"rpc_log,optional,default=false"` } func enumUnmarshal(u func(interface{}, bool) error, types map[string]interface{}) (interface{}, error) { @@ -328,7 +373,7 @@ var ConfigFileDefaultLocations = []string{ "/usr/local/etc/zrepl/zrepl.yml", } -func ParseConfig(path string) (i Config, err error) { +func ParseConfig(path string) (i *Config, err error) { if path == "" { // Try default locations @@ -352,11 +397,18 @@ func ParseConfig(path string) (i Config, err error) { return } - if err = yaml.UnmarshalStrict(bytes, &i); err != nil { - return - } + return ParseConfigBytes(bytes) +} - return +func ParseConfigBytes(bytes []byte) (*Config, error) { + var c *Config + if err := yaml.UnmarshalStrict(bytes, &c); err != nil { + return nil, err + } + if c == nil { + return nil, fmt.Errorf("config is empty or only consists of comments") + } + return c, nil } var durationStringRegex *regexp.Regexp = regexp.MustCompile(`^\s*(\d+)\s*(s|m|h|d|w)\s*$`) diff --git a/config/config_minimal_test.go b/config/config_minimal_test.go new file mode 100644 index 0000000..82edde8 --- /dev/null +++ b/config/config_minimal_test.go @@ -0,0 +1,41 @@ +package config + +import ( + "testing" + "github.com/stretchr/testify/assert" +) + +func TestConfigEmptyFails(t *testing.T) { + conf, err := testConfig(t, "\n") + assert.Nil(t, conf) + assert.Error(t, err) +} + +func TestJobsOnlyWorks(t *testing.T) { + testValidConfig(t, ` +jobs: +- name: push + type: push + # snapshot the filesystems matched by the left-hand-side of the mapping + # every 10m with zrepl_ as prefix + replication: + connect: + type: tcp + address: localhost:2342 + filesystems: { + "pool1/var/db<": true, + "pool1/usr/home<": true, + "pool1/usr/home/paranoid": false, #don't backup paranoid user + "pool1/poudriere/ports<": false #don't backup the ports trees + } + snapshotting: + snapshot_prefix: zrepl_ + interval: 10m + pruning: + keep_sender: + - type: not_replicated + keep_receiver: + - type: last_n + count: 1 +`) +} \ No newline at end of file diff --git a/config/config_rpc_test.go b/config/config_rpc_test.go new file mode 100644 index 0000000..0fd2a4c --- /dev/null +++ b/config/config_rpc_test.go @@ -0,0 +1,61 @@ +package config + +import ( + "testing" + "github.com/stretchr/testify/assert" + "time" +) + +func TestRPC (t *testing.T) { + conf := testValidConfig(t, ` +jobs: +- name: pull_servers + type: pull + replication: + connect: + type: tcp + address: "server1.foo.bar:8888" + rpc: + timeout: 20s # different form default, should merge + root_dataset: "pool2/backup_servers" + interval: 10m + pruning: + keep_sender: + - type: not_replicated + keep_receiver: + - type: last_n + count: 100 + +- name: pull_servers2 + type: pull + replication: + connect: + type: tcp + address: "server1.foo.bar:8888" + rpc: + tx_chunk_size: 0xabcd # different from default, should merge + root_dataset: "pool2/backup_servers" + interval: 10m + pruning: + keep_sender: + - type: not_replicated + keep_receiver: + - type: last_n + count: 100 +`) + + assert.Equal(t, 20*time.Second, conf.Jobs[0].Ret.(*PullJob).Replication.Connect.Ret.(*TCPConnect).RPC.Timeout) + assert.Equal(t, uint(0xabcd), conf.Jobs[1].Ret.(*PullJob).Replication.Connect.Ret.(*TCPConnect).RPC.TxChunkSize) + defConf := RPCConfig{} + Default(&defConf) + assert.Equal(t, defConf.Timeout, conf.Global.RPC.Timeout) +} + +func TestGlobal_DefaultRPCConfig(t *testing.T) { + assert.NotPanics(t, func() { + var c RPCConfig + Default(&c) + assert.NotNil(t, c) + assert.Equal(t, c.TxChunkSize, uint(1)<<15) + }) +} diff --git a/config/config_test.go b/config/config_test.go index ad975d0..64a12f9 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -4,6 +4,9 @@ import ( "github.com/kr/pretty" "path/filepath" "testing" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/zrepl/yaml-config" ) func TestSampleConfigsAreParsedWithoutErrors(t *testing.T) { @@ -27,3 +30,26 @@ func TestSampleConfigsAreParsedWithoutErrors(t *testing.T) { } } + +func TestLoggingOutletEnumList_SetDefaults(t *testing.T) { + e := &LoggingOutletEnumList{} + var i yaml.Defaulter = e + require.NotPanics(t, func() { + i.SetDefault() + assert.Equal(t, "warn", (*e)[0].Ret.(StdoutLoggingOutlet).Level) + }) +} + + +func testValidConfig(t *testing.T, input string) (*Config) { + t.Helper() + conf, err := testConfig(t, input) + require.NoError(t, err) + require.NotNil(t, conf) + return conf +} + +func testConfig(t *testing.T, input string) (*Config, error) { + t.Helper() + return ParseConfigBytes([]byte(input)) +} \ No newline at end of file diff --git a/daemon/connecter/connecter.go b/daemon/connecter/connecter.go index a27ba7d..a5d95ad 100644 --- a/daemon/connecter/connecter.go +++ b/daemon/connecter/connecter.go @@ -6,7 +6,7 @@ import ( "github.com/zrepl/zrepl/config" ) -func FromConfig(g config.Global, in config.ConnectEnum) (streamrpc.Connecter, error) { +func FromConfig(g *config.Global, in config.ConnectEnum) (streamrpc.Connecter, error) { switch v := in.Ret.(type) { case *config.SSHStdinserverConnect: return SSHStdinserverConnecterFromConfig(v) diff --git a/daemon/daemon.go b/daemon/daemon.go index e231d72..078f1f4 100644 --- a/daemon/daemon.go +++ b/daemon/daemon.go @@ -17,7 +17,7 @@ import ( "time" ) -func Run(conf config.Config) error { +func Run(conf *config.Config) error { ctx, cancel := context.WithCancel(context.Background()) @@ -29,7 +29,7 @@ func Run(conf config.Config) error { cancel() }() - outlets, err := logging.OutletsFromConfig(conf.Global.Logging) + outlets, err := logging.OutletsFromConfig(*conf.Global.Logging) if err != nil { return errors.Wrap(err, "cannot build logging from config") } diff --git a/daemon/job/build_jobs.go b/daemon/job/build_jobs.go index 8843b55..b438cac 100644 --- a/daemon/job/build_jobs.go +++ b/daemon/job/build_jobs.go @@ -6,7 +6,7 @@ import ( "github.com/pkg/errors" ) -func JobsFromConfig(c config.Config) ([]Job, error) { +func JobsFromConfig(c *config.Config) ([]Job, error) { js := make([]Job, len(c.Jobs)) for i := range c.Jobs { j, err := buildJob(c.Global, c.Jobs[i]) @@ -18,7 +18,7 @@ func JobsFromConfig(c config.Config) ([]Job, error) { return js, nil } -func buildJob(c config.Global, in config.JobEnum) (j Job, err error) { +func buildJob(c *config.Global, in config.JobEnum) (j Job, err error) { switch v := in.Ret.(type) { case *config.SinkJob: j, err = SinkFromConfig(c, v) diff --git a/daemon/job/push.go b/daemon/job/push.go index e4750c2..088e863 100644 --- a/daemon/job/push.go +++ b/daemon/job/push.go @@ -25,7 +25,7 @@ type Push struct { replication *replication.Replication } -func PushFromConfig(g config.Global, in *config.PushJob) (j *Push, err error) { +func PushFromConfig(g *config.Global, in *config.PushJob) (j *Push, err error) { j = &Push{} j.name = in.Name diff --git a/daemon/job/sink.go b/daemon/job/sink.go index 8c9f496..3c20ae3 100644 --- a/daemon/job/sink.go +++ b/daemon/job/sink.go @@ -19,7 +19,7 @@ type Sink struct { fsmapInv endpoint.FSFilter } -func SinkFromConfig(g config.Global, in *config.SinkJob) (s *Sink, err error) { +func SinkFromConfig(g *config.Global, in *config.SinkJob) (s *Sink, err error) { // FIXME multi client support diff --git a/daemon/logging/build_logging.go b/daemon/logging/build_logging.go index 9d88d41..fbaf9c2 100644 --- a/daemon/logging/build_logging.go +++ b/daemon/logging/build_logging.go @@ -16,7 +16,7 @@ import ( "os" ) -func OutletsFromConfig(in []config.LoggingOutletEnum) (*logger.Outlets, error) { +func OutletsFromConfig(in config.LoggingOutletEnumList) (*logger.Outlets, error) { outlets := logger.NewOutlets() diff --git a/daemon/serve/serve.go b/daemon/serve/serve.go index 8ab7945..4df8488 100644 --- a/daemon/serve/serve.go +++ b/daemon/serve/serve.go @@ -10,7 +10,7 @@ type ListenerFactory interface { Listen() (net.Listener, error) } -func FromConfig(g config.Global, in config.ServeEnum) (ListenerFactory, error) { +func FromConfig(g *config.Global, in config.ServeEnum) (ListenerFactory, error) { switch v := in.Ret.(type) { case *config.TCPServe: diff --git a/daemon/serve/serve_stdinserver.go b/daemon/serve/serve_stdinserver.go index 4048d65..f6403d3 100644 --- a/daemon/serve/serve_stdinserver.go +++ b/daemon/serve/serve_stdinserver.go @@ -15,7 +15,7 @@ type StdinserverListenerFactory struct { sockpath string } -func StdinserverListenerFactoryFromConfig(g config.Global, in *config.StdinserverServer) (f *StdinserverListenerFactory, err error) { +func StdinserverListenerFactoryFromConfig(g *config.Global, in *config.StdinserverServer) (f *StdinserverListenerFactory, err error) { f = &StdinserverListenerFactory{ ClientIdentity: in.ClientIdentity, diff --git a/daemon/serve/serve_tcp.go b/daemon/serve/serve_tcp.go index a5bad28..21cab59 100644 --- a/daemon/serve/serve_tcp.go +++ b/daemon/serve/serve_tcp.go @@ -9,7 +9,7 @@ type TCPListenerFactory struct { Address string } -func TCPListenerFactoryFromConfig(c config.Global, in *config.TCPServe) (*TCPListenerFactory, error) { +func TCPListenerFactoryFromConfig(c *config.Global, in *config.TCPServe) (*TCPListenerFactory, error) { lf := &TCPListenerFactory{ Address: in.Listen, } diff --git a/daemon/serve/serve_tls.go b/daemon/serve/serve_tls.go index f24f5ad..0b80345 100644 --- a/daemon/serve/serve_tls.go +++ b/daemon/serve/serve_tls.go @@ -18,7 +18,7 @@ type TLSListenerFactory struct { handshakeTimeout time.Duration } -func TLSListenerFactoryFromConfig(c config.Global, in *config.TLSServe) (lf *TLSListenerFactory, err error) { +func TLSListenerFactoryFromConfig(c *config.Global, in *config.TLSServe) (lf *TLSListenerFactory, err error) { lf = &TLSListenerFactory{ address: in.Listen, } From 3d8e552c6aa5b134932c1839f812571a96c6870f Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 31 Aug 2018 21:51:44 +0200 Subject: [PATCH 087/167] streamrpc 0.3 + config from daemon/config --- Gopkg.toml | 2 +- config/config.go | 67 +++++++++++++---------- config/config_rpc_test.go | 23 ++++++-- daemon/connecter/connecter.go | 35 ++++++++++-- daemon/job/job.go | 11 ---- daemon/job/push.go | 12 ++-- daemon/job/sink.go | 5 +- daemon/serve/serve.go | 29 ++++++++-- daemon/streamrpcconfig/streamrpcconfig.go | 22 ++++++++ 9 files changed, 144 insertions(+), 62 deletions(-) create mode 100644 daemon/streamrpcconfig/streamrpcconfig.go diff --git a/Gopkg.toml b/Gopkg.toml index 49065ab..b08613e 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -69,4 +69,4 @@ ignored = [ "github.com/inconshreveable/mousetrap" ] [[constraint]] name = "github.com/problame/go-streamrpc" - version = "0.2.0" + version = "0.3.0" diff --git a/config/config.go b/config/config.go index d6a334f..d312f57 100644 --- a/config/config.go +++ b/config/config.go @@ -6,15 +6,15 @@ import ( "github.com/zrepl/yaml-config" "io/ioutil" "os" + "reflect" "regexp" "strconv" "time" - "reflect" ) type Config struct { Jobs []JobEnum `yaml:"jobs"` - Global *Global `yaml:"global,optional,fromdefaults"` + Global *Global `yaml:"global,optional,fromdefaults"` } type JobEnum struct { @@ -124,10 +124,10 @@ var _ yaml.Defaulter = &LoggingOutletEnumList{} type Global struct { Logging *LoggingOutletEnumList `yaml:"logging,optional,fromdefaults"` - Monitoring []MonitoringEnum `yaml:"monitoring,optional"` - Control *GlobalControl `yaml:"control,optional,fromdefaults"` - Serve *GlobalServe `yaml:"serve,optional,fromdefaults"` - RPC *RPCConfig `yaml:"rpc,optional,fromdefaults"` + Monitoring []MonitoringEnum `yaml:"monitoring,optional"` + Control *GlobalControl `yaml:"control,optional,fromdefaults"` + Serve *GlobalServe `yaml:"serve,optional,fromdefaults"` + RPC *RPCConfig `yaml:"rpc,optional,fromdefaults"` } func Default(i interface{}) { @@ -143,38 +143,40 @@ func Default(i interface{}) { } type RPCConfig struct { - Timeout time.Duration `yaml:"timeout,optional,positive,default=10s"` - TxChunkSize uint `yaml:"tx_chunk_size,optional,default=32768"` - RxStructuredMaxLen uint `yaml:"rx_structured_max,optional,default=16777216"` - RxStreamChunkMaxLen uint `yaml:"rx_stream_chunk_max,optional,default=16777216"` - RxHeaderMaxLen uint `yaml:"rx_header_max,optional,default=40960"` + Timeout time.Duration `yaml:"timeout,optional,positive,default=10s"` + TxChunkSize uint32 `yaml:"tx_chunk_size,optional,default=32768"` + RxStructuredMaxLen uint32 `yaml:"rx_structured_max,optional,default=16777216"` + RxStreamChunkMaxLen uint32 `yaml:"rx_stream_chunk_max,optional,default=16777216"` + RxHeaderMaxLen uint32 `yaml:"rx_header_max,optional,default=40960"` } type ConnectEnum struct { Ret interface{} } +type ConnectCommon struct { + Type string `yaml:"type"` + RPC *RPCConfig `yaml:"rpc,optional"` +} + type TCPConnect struct { - Type string `yaml:"type"` - RPC *RPCConfig `yaml:"rpc,optional"` - Address string `yaml:"address"` - DialTimeout time.Duration `yaml:"dial_timeout,positive,default=10s"` + ConnectCommon `yaml:",inline"` + Address string `yaml:"address"` + DialTimeout time.Duration `yaml:"dial_timeout,positive,default=10s"` } type TLSConnect struct { - Type string `yaml:"type"` - RPC *RPCConfig `yaml:"rpc,optional"` - Address string `yaml:"address"` - Ca string `yaml:"ca"` - Cert string `yaml:"cert"` - Key string `yaml:"key"` - ServerCN string `yaml:"server_cn"` - DialTimeout time.Duration `yaml:"dial_timeout,positive,default=10s"` + ConnectCommon `yaml:",inline"` + Address string `yaml:"address"` + Ca string `yaml:"ca"` + Cert string `yaml:"cert"` + Key string `yaml:"key"` + ServerCN string `yaml:"server_cn"` + DialTimeout time.Duration `yaml:"dial_timeout,positive,default=10s"` } type SSHStdinserverConnect struct { - Type string `yaml:"type"` - RPC *RPCConfig `yaml:"rpc,optional"` + ConnectCommon `yaml:",inline"` Host string `yaml:"host"` User string `yaml:"user"` Port uint16 `yaml:"port"` @@ -189,14 +191,19 @@ type ServeEnum struct { Ret interface{} } +type ServeCommon struct { + Type string `yaml:"type"` + RPC *RPCConfig `yaml:"rpc,optional"` +} + type TCPServe struct { - Type string `yaml:"type"` - Listen string `yaml:"listen"` - Clients map[string]string `yaml:"clients"` + ServeCommon `yaml:",inline"` + Listen string `yaml:"listen"` + Clients map[string]string `yaml:"clients"` } type TLSServe struct { - Type string `yaml:"type"` + ServeCommon `yaml:",inline"` Listen string `yaml:"listen"` Ca string `yaml:"ca"` Cert string `yaml:"cert"` @@ -206,7 +213,7 @@ type TLSServe struct { } type StdinserverServer struct { - Type string `yaml:"type"` + ServeCommon `yaml:",inline"` ClientIdentity string `yaml:"client_identity"` } diff --git a/config/config_rpc_test.go b/config/config_rpc_test.go index 0fd2a4c..fa041b4 100644 --- a/config/config_rpc_test.go +++ b/config/config_rpc_test.go @@ -1,12 +1,12 @@ package config import ( - "testing" "github.com/stretchr/testify/assert" + "testing" "time" ) -func TestRPC (t *testing.T) { +func TestRPC(t *testing.T) { conf := testValidConfig(t, ` jobs: - name: pull_servers @@ -42,10 +42,25 @@ jobs: keep_receiver: - type: last_n count: 100 + +- type: sink + name: "laptop_sink" + replication: + root_dataset: "pool2/backup_laptops" + serve: + type: tcp + listen: "192.168.122.189:8888" + clients: { + "10.23.42.23":"client1" + } + rpc: + rx_structured_max: 0x2342 + `) assert.Equal(t, 20*time.Second, conf.Jobs[0].Ret.(*PullJob).Replication.Connect.Ret.(*TCPConnect).RPC.Timeout) - assert.Equal(t, uint(0xabcd), conf.Jobs[1].Ret.(*PullJob).Replication.Connect.Ret.(*TCPConnect).RPC.TxChunkSize) + assert.Equal(t, uint32(0xabcd), conf.Jobs[1].Ret.(*PullJob).Replication.Connect.Ret.(*TCPConnect).RPC.TxChunkSize) + assert.Equal(t, uint32(0x2342), conf.Jobs[2].Ret.(*SinkJob).Replication.Serve.Ret.(*TCPServe).RPC.RxStructuredMaxLen) defConf := RPCConfig{} Default(&defConf) assert.Equal(t, defConf.Timeout, conf.Global.RPC.Timeout) @@ -56,6 +71,6 @@ func TestGlobal_DefaultRPCConfig(t *testing.T) { var c RPCConfig Default(&c) assert.NotNil(t, c) - assert.Equal(t, c.TxChunkSize, uint(1)<<15) + assert.Equal(t, c.TxChunkSize, uint32(1)<<15) }) } diff --git a/daemon/connecter/connecter.go b/daemon/connecter/connecter.go index a5d95ad..e6803ce 100644 --- a/daemon/connecter/connecter.go +++ b/daemon/connecter/connecter.go @@ -4,17 +4,44 @@ import ( "fmt" "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon/streamrpcconfig" ) -func FromConfig(g *config.Global, in config.ConnectEnum) (streamrpc.Connecter, error) { +func FromConfig(g *config.Global, in config.ConnectEnum) (*ClientFactory, error) { + var ( + connecter streamrpc.Connecter + errConnecter, errRPC error + connConf *streamrpc.ConnConfig + ) switch v := in.Ret.(type) { case *config.SSHStdinserverConnect: - return SSHStdinserverConnecterFromConfig(v) + connecter, errConnecter = SSHStdinserverConnecterFromConfig(v) + connConf, errRPC = streamrpcconfig.FromDaemonConfig(g, v.RPC) case *config.TCPConnect: - return TCPConnecterFromConfig(v) + connecter, errConnecter = TCPConnecterFromConfig(v) + connConf, errRPC = streamrpcconfig.FromDaemonConfig(g, v.RPC) case *config.TLSConnect: - return TLSConnecterFromConfig(v) + connecter, errConnecter = TLSConnecterFromConfig(v) + connConf, errRPC = streamrpcconfig.FromDaemonConfig(g, v.RPC) default: panic(fmt.Sprintf("implementation error: unknown connecter type %T", v)) } + + if errConnecter != nil { + return nil, errConnecter + } + if errRPC != nil { + return nil, errRPC + } + + return &ClientFactory{connecter: connecter, config: &streamrpc.ClientConfig{connConf}}, nil +} + +type ClientFactory struct { + connecter streamrpc.Connecter + config *streamrpc.ClientConfig +} + +func (f ClientFactory) NewClient() (*streamrpc.Client, error) { + return streamrpc.NewClient(f.connecter, f.config) } diff --git a/daemon/job/job.go b/daemon/job/job.go index b95f852..12f738e 100644 --- a/daemon/job/job.go +++ b/daemon/job/job.go @@ -3,9 +3,7 @@ package job import ( "context" "errors" - "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/logger" - "time" ) type Logger = logger.Logger @@ -59,12 +57,3 @@ func WaitWakeup(ctx context.Context) <-chan struct{} { return wc } -var STREAMRPC_CONFIG = &streamrpc.ConnConfig{ // FIXME oversight and configurability - RxHeaderMaxLen: 4096, - RxStructuredMaxLen: 4096 * 4096, - RxStreamMaxChunkSize: 4096 * 4096, - TxChunkSize: 4096 * 4096, - Timeout: streamrpc.Timeout{ - Progress: 10 * time.Second, - }, -} diff --git a/daemon/job/push.go b/daemon/job/push.go index 088e863..5c7004f 100644 --- a/daemon/job/push.go +++ b/daemon/job/push.go @@ -3,7 +3,6 @@ package job import ( "context" "github.com/pkg/errors" - "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon/connecter" "github.com/zrepl/zrepl/daemon/filters" @@ -16,7 +15,7 @@ import ( type Push struct { name string - connecter streamrpc.Connecter + clientFactory *connecter.ClientFactory fsfilter endpoint.FSFilter prunerFactory *pruner.PrunerFactory @@ -30,7 +29,10 @@ func PushFromConfig(g *config.Global, in *config.PushJob) (j *Push, err error) { j = &Push{} j.name = in.Name - j.connecter, err = connecter.FromConfig(g, in.Replication.Connect) + j.clientFactory, err = connecter.FromConfig(g, in.Replication.Connect) + if err != nil { + return nil, errors.Wrap(err, "cannot build client") + } if j.fsfilter, err = filters.DatasetMapFilterFromConfig(in.Replication.Filesystems); err != nil { return nil, errors.Wrap(err, "cannnot build filesystem filter") @@ -87,9 +89,9 @@ func (j *Push) do(ctx context.Context) { log := GetLogger(ctx) ctx = logging.WithSubsystemLoggers(ctx, log) - client, err := streamrpc.NewClient(j.connecter, &streamrpc.ClientConfig{STREAMRPC_CONFIG}) + client, err := j.clientFactory.NewClient() if err != nil { - log.WithError(err).Error("cannot create streamrpc client") + log.WithError(err).Error("factory cannot instantiate streamrpc client") } defer client.Close(ctx) diff --git a/daemon/job/sink.go b/daemon/job/sink.go index 3c20ae3..6afbb5d 100644 --- a/daemon/job/sink.go +++ b/daemon/job/sink.go @@ -15,6 +15,7 @@ import ( type Sink struct { name string l serve.ListenerFactory + rpcConf *streamrpc.ConnConfig fsmap endpoint.FSMap fsmapInv endpoint.FSFilter } @@ -24,7 +25,7 @@ func SinkFromConfig(g *config.Global, in *config.SinkJob) (s *Sink, err error) { // FIXME multi client support s = &Sink{name: in.Name} - if s.l, err = serve.FromConfig(g, in.Replication.Serve); err != nil { + if s.l, s.rpcConf, err = serve.FromConfig(g, in.Replication.Serve); err != nil { return nil, errors.Wrap(err, "cannot build server") } @@ -95,7 +96,7 @@ func (j *Sink) handleConnection(ctx context.Context, conn net.Conn) { } handler := endpoint.NewHandler(local) - if err := streamrpc.ServeConn(ctx, conn, STREAMRPC_CONFIG, handler.Handle); err != nil { + if err := streamrpc.ServeConn(ctx, conn, j.rpcConf, handler.Handle); err != nil { log.WithError(err).Error("error serving client") } } diff --git a/daemon/serve/serve.go b/daemon/serve/serve.go index 4df8488..fa7bb4a 100644 --- a/daemon/serve/serve.go +++ b/daemon/serve/serve.go @@ -4,23 +4,42 @@ import ( "github.com/pkg/errors" "github.com/zrepl/zrepl/config" "net" + "github.com/zrepl/zrepl/daemon/streamrpcconfig" + "github.com/problame/go-streamrpc" ) type ListenerFactory interface { Listen() (net.Listener, error) } -func FromConfig(g *config.Global, in config.ServeEnum) (ListenerFactory, error) { +func FromConfig(g *config.Global, in config.ServeEnum) (lf ListenerFactory, conf *streamrpc.ConnConfig, _ error) { + var ( + lfError, rpcErr error + ) switch v := in.Ret.(type) { case *config.TCPServe: - return TCPListenerFactoryFromConfig(g, v) + lf, lfError = TCPListenerFactoryFromConfig(g, v) + conf, rpcErr = streamrpcconfig.FromDaemonConfig(g, v.RPC) case *config.TLSServe: - return TLSListenerFactoryFromConfig(g, v) + lf, lfError = TLSListenerFactoryFromConfig(g, v) + conf, rpcErr = streamrpcconfig.FromDaemonConfig(g, v.RPC) case *config.StdinserverServer: - return StdinserverListenerFactoryFromConfig(g, v) + lf, lfError = StdinserverListenerFactoryFromConfig(g, v) + conf, rpcErr = streamrpcconfig.FromDaemonConfig(g, v.RPC) default: - return nil, errors.Errorf("internal error: unknown serve type %T", v) + return nil, nil, errors.Errorf("internal error: unknown serve type %T", v) } + if lfError != nil { + return nil, nil, lfError + } + if rpcErr != nil { + return nil, nil, rpcErr + } + + return lf, conf, nil + } + + diff --git a/daemon/streamrpcconfig/streamrpcconfig.go b/daemon/streamrpcconfig/streamrpcconfig.go new file mode 100644 index 0000000..3292577 --- /dev/null +++ b/daemon/streamrpcconfig/streamrpcconfig.go @@ -0,0 +1,22 @@ +package streamrpcconfig + +import ( + "github.com/problame/go-streamrpc" + "github.com/zrepl/zrepl/config" +) + +func FromDaemonConfig(g *config.Global, in *config.RPCConfig) (*streamrpc.ConnConfig, error) { + conf := in + if conf == nil { + conf = g.RPC + } + return &streamrpc.ConnConfig{ + RxHeaderMaxLen: conf.RxHeaderMaxLen, + RxStructuredMaxLen: conf.RxStructuredMaxLen, + RxStreamMaxChunkSize: conf.RxStreamChunkMaxLen, + TxChunkSize: conf.TxChunkSize, + Timeout: streamrpc.Timeout{ + Progress: conf.Timeout, + }, + }, nil +} From ad28fd1ecb3fc88957f2861c91066f176a884f8d Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 2 Sep 2018 15:31:10 -0700 Subject: [PATCH 088/167] replication: diff does not need special case for receiver/sender == nil --- replication/internal/diff/diff.go | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/replication/internal/diff/diff.go b/replication/internal/diff/diff.go index fed2392..213bc77 100644 --- a/replication/internal/diff/diff.go +++ b/replication/internal/diff/diff.go @@ -49,20 +49,9 @@ func SortVersionListByCreateTXGThenBookmarkLTSnapshot(fsvslice []*FilesystemVers // conflict may be a *ConflictDiverged or a *ConflictNoCommonAncestor func IncrementalPath(receiver, sender []*FilesystemVersion) (incPath []*FilesystemVersion, conflict error) { - if receiver == nil { - panic("receiver must not be nil") - } - if sender == nil { - panic("sender must not be nil") - } - receiver = SortVersionListByCreateTXGThenBookmarkLTSnapshot(receiver) sender = SortVersionListByCreateTXGThenBookmarkLTSnapshot(sender) - if len(sender) == 0 { - return []*FilesystemVersion{}, nil - } - // Find most recent common ancestor by name, preferring snapshots over bookmarks mrcaRcv := len(receiver) - 1 From f0860767f53480a0f8105b087962ac72979715c4 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 2 Sep 2018 15:32:08 -0700 Subject: [PATCH 089/167] zfs: include stderr of command in ZFSError.Error() Since we don't implement screen-scraping of ZFS output ATM, this is better than nothing, as user's may be able to figure out what' sthe problem from the logs / status reports. --- zfs/zfs.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zfs/zfs.go b/zfs/zfs.go index 5040567..577449e 100644 --- a/zfs/zfs.go +++ b/zfs/zfs.go @@ -136,7 +136,7 @@ type ZFSError struct { } func (e ZFSError) Error() string { - return fmt.Sprintf("zfs exited with error: %s", e.WaitErr.Error()) + return fmt.Sprintf("zfs exited with error: %s\nstderr:\n%s", e.WaitErr.Error(), e.Stderr) } var ZFS_BINARY string = "zfs" From 2da0e51fda88a3c7aaf2e05965592eb2f1c23309 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 2 Sep 2018 15:49:17 -0700 Subject: [PATCH 090/167] Update Gopkg.lock to latest versions of streamrpc and yaml-config --- Gopkg.lock | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index e6bdc39..07b31e6 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -170,15 +170,15 @@ revision = "391d2c78c8404a9683d79f75dd24ab53040f89f7" [[projects]] - digest = "1:23576326ee8d287fd295807d5de5db9fcc9572ec2b4123c8ec2394a683edf1e1" + digest = "1:f6bc2d2f9c00737754ef87dba06c851458523fa382a61e1e2e5270c1f39a6622" name = "github.com/problame/go-streamrpc" packages = [ ".", "internal/pdu", ] pruneopts = "" - revision = "504ffed1faf6af51c057d7b11d79e9367678c666" - version = "v0.2" + revision = "a67d93db93b3dc8c2d6901a1b2183ff42dea8f7d" + version = "v0.3" [[projects]] branch = "master" @@ -253,11 +253,11 @@ [[projects]] branch = "v2" - digest = "1:1d4340fa24f9e4a81097fb8e56f509e528bfc7456e2e9f672831c7d9e9cb8109" + digest = "1:9d92186f609a73744232323416ddafd56fae67cb552162cc190ab903e36900dd" name = "github.com/zrepl/yaml-config" packages = ["."] pruneopts = "" - revision = "3f4c23177f5097f12b95dd3d053481015815a6cd" + revision = "af27d27978ad95808723a62d87557d63c3ff0605" [[projects]] branch = "master" From 03f9f81cb5f71b07f314d8c860bb485c7193adc4 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 3 Sep 2018 13:17:53 -0700 Subject: [PATCH 091/167] fixup 3d8e552c6aa5b134932c1839f812571a96c6870f: validate streamrpc config in factory constructors --- daemon/connecter/connecter.go | 6 +++++- daemon/streamrpcconfig/streamrpcconfig.go | 8 ++++++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/daemon/connecter/connecter.go b/daemon/connecter/connecter.go index e6803ce..c26d1de 100644 --- a/daemon/connecter/connecter.go +++ b/daemon/connecter/connecter.go @@ -34,7 +34,11 @@ func FromConfig(g *config.Global, in config.ConnectEnum) (*ClientFactory, error) return nil, errRPC } - return &ClientFactory{connecter: connecter, config: &streamrpc.ClientConfig{connConf}}, nil + config := streamrpc.ClientConfig{connConf} + if err := config.Validate(); err != nil { + return nil, err + } + return &ClientFactory{connecter: connecter, config: &config}, nil } type ClientFactory struct { diff --git a/daemon/streamrpcconfig/streamrpcconfig.go b/daemon/streamrpcconfig/streamrpcconfig.go index 3292577..8dc64f6 100644 --- a/daemon/streamrpcconfig/streamrpcconfig.go +++ b/daemon/streamrpcconfig/streamrpcconfig.go @@ -10,7 +10,7 @@ func FromDaemonConfig(g *config.Global, in *config.RPCConfig) (*streamrpc.ConnCo if conf == nil { conf = g.RPC } - return &streamrpc.ConnConfig{ + srpcConf := &streamrpc.ConnConfig{ RxHeaderMaxLen: conf.RxHeaderMaxLen, RxStructuredMaxLen: conf.RxStructuredMaxLen, RxStreamMaxChunkSize: conf.RxStreamChunkMaxLen, @@ -18,5 +18,9 @@ func FromDaemonConfig(g *config.Global, in *config.RPCConfig) (*streamrpc.ConnCo Timeout: streamrpc.Timeout{ Progress: conf.Timeout, }, - }, nil + } + if err := srpcConf.Validate(); err != nil { + return nil, err + } + return srpcConf, nil } From 8799108b5509d41dd9ef18700d983ecbd9553b09 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 3 Sep 2018 13:19:56 -0700 Subject: [PATCH 092/167] fixup b95e983d0d7b3102a7917f7aca8aef215b1ab2be: prunerFactory: fix duplicate logger fields --- daemon/job/push.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/daemon/job/push.go b/daemon/job/push.go index 5c7004f..d5406b1 100644 --- a/daemon/job/push.go +++ b/daemon/job/push.go @@ -106,13 +106,11 @@ func (j *Push) do(ctx context.Context) { j.replication.Drive(ctx, sender, receiver) log.Info("start pruning sender") - psCtx := pruner.WithLogger(ctx, pruner.GetLogger(ctx).WithField("prune_side", "sender")) - senderPruner := j.prunerFactory.BuildSenderPruner(psCtx, sender, sender) // FIXME ctx as member + senderPruner := j.prunerFactory.BuildSenderPruner(ctx, sender, sender) senderPruner.Prune() log.Info("start pruning receiver") - prCtx := pruner.WithLogger(ctx, pruner.GetLogger(ctx).WithField("prune_side", "receiver")) - receiverPruner := j.prunerFactory.BuildReceiverPruner(prCtx, receiver, sender) // FIXME ctx as member + receiverPruner := j.prunerFactory.BuildReceiverPruner(ctx, receiver, sender) receiverPruner.Prune() } From 0c4a3f8dc442568c6e3d6208c86135b056b8b89a Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 4 Sep 2018 13:29:51 -0700 Subject: [PATCH 093/167] pruning/history: properly communicate via rpc if snapshot does not exist --- daemon/pruner/pruner.go | 12 ++- endpoint/endpoint.go | 19 +++-- replication/fsrep/fsfsm.go | 4 +- replication/pdu/pdu.pb.go | 165 ++++++++++++++++++++++--------------- replication/pdu/pdu.proto | 7 +- 5 files changed, 128 insertions(+), 79 deletions(-) diff --git a/daemon/pruner/pruner.go b/daemon/pruner/pruner.go index f912a90..2b49c76 100644 --- a/daemon/pruner/pruner.go +++ b/daemon/pruner/pruner.go @@ -279,6 +279,11 @@ func statePlan(a *args, u updater) state { Op: pdu.SnapshotReplicationStatusReq_Get, } res, err := receiver.SnapshotReplicationStatus(ctx, &req) + if err != nil { + GetLogger(ctx). + WithField("req", req.String()). + WithError(err).Error("cannot get snapshot replication status") + } if err != nil && shouldRetry(err) { return onErr(u, err) } else if err != nil { @@ -286,9 +291,12 @@ func statePlan(a *args, u updater) state { pfs.snaps = nil break } - + if res.Status == pdu.SnapshotReplicationStatusRes_Nonexistent { + GetLogger(ctx). + Debug("snapshot does not exist in history, assuming was replicated") + } pfs.snaps = append(pfs.snaps, snapshot{ - replicated: res.Replicated, + replicated: !(res.Status != pdu.SnapshotReplicationStatusRes_Replicated), date: creation, fsv: tfsv, }) diff --git a/endpoint/endpoint.go b/endpoint/endpoint.go index 561e509..d1ee34f 100644 --- a/endpoint/endpoint.go +++ b/endpoint/endpoint.go @@ -127,23 +127,30 @@ func (p *Sender) SnapshotReplicationStatus(ctx context.Context, req *pdu.Snapsho Name: req.Snapshot, //FIXME validation } - replicated := false + var status pdu.SnapshotReplicationStatusRes_Status switch req.Op { case pdu.SnapshotReplicationStatusReq_Get: - replicated, err = zfs.ZFSGetReplicatedProperty(dp, &version) - if err != nil { - return nil, err + replicated, err := zfs.ZFSGetReplicatedProperty(dp, &version) + if _, ok := err.(*zfs.DatasetDoesNotExist); ok { + status = pdu.SnapshotReplicationStatusRes_Nonexistent + } else if err != nil { + + } + if replicated { + status = pdu.SnapshotReplicationStatusRes_Replicated + } else { + status = pdu.SnapshotReplicationStatusRes_NotReplicated } case pdu.SnapshotReplicationStatusReq_SetReplicated: err = zfs.ZFSSetReplicatedProperty(dp, &version, true) if err != nil { return nil, err } - replicated = true + status = pdu.SnapshotReplicationStatusRes_Replicated default: return nil, errors.Errorf("unknown opcode %v", req.Op) } - return &pdu.SnapshotReplicationStatusRes{Replicated: replicated}, nil + return &pdu.SnapshotReplicationStatusRes{Status: status}, nil } type FSFilter interface { diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index 2203639..e1a3ffe 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -434,8 +434,8 @@ func (s *ReplicationStep) doMarkReplicated(ctx context.Context, sender Sender) S log.WithError(err).Error("error marking snapshot as replicated") return updateStateError(err) } - if res.Replicated != true { - err := fmt.Errorf("sender did not report snapshot as replicated") + if res.Status != pdu.SnapshotReplicationStatusRes_Replicated { + err := fmt.Errorf("sender did not report snapshot as replicated: %s", res.Status) log.Error(err.Error()) return updateStateError(err) } diff --git a/replication/pdu/pdu.pb.go b/replication/pdu/pdu.pb.go index a1876f8..be22dea 100644 --- a/replication/pdu/pdu.pb.go +++ b/replication/pdu/pdu.pb.go @@ -38,7 +38,7 @@ func (x FilesystemVersion_VersionType) String() string { return proto.EnumName(FilesystemVersion_VersionType_name, int32(x)) } func (FilesystemVersion_VersionType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{5, 0} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{5, 0} } type SnapshotReplicationStatusReq_Op int32 @@ -61,7 +61,33 @@ func (x SnapshotReplicationStatusReq_Op) String() string { return proto.EnumName(SnapshotReplicationStatusReq_Op_name, int32(x)) } func (SnapshotReplicationStatusReq_Op) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{14, 0} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{14, 0} +} + +type SnapshotReplicationStatusRes_Status int32 + +const ( + SnapshotReplicationStatusRes_Nonexistent SnapshotReplicationStatusRes_Status = 0 + SnapshotReplicationStatusRes_NotReplicated SnapshotReplicationStatusRes_Status = 1 + SnapshotReplicationStatusRes_Replicated SnapshotReplicationStatusRes_Status = 2 +) + +var SnapshotReplicationStatusRes_Status_name = map[int32]string{ + 0: "Nonexistent", + 1: "NotReplicated", + 2: "Replicated", +} +var SnapshotReplicationStatusRes_Status_value = map[string]int32{ + "Nonexistent": 0, + "NotReplicated": 1, + "Replicated": 2, +} + +func (x SnapshotReplicationStatusRes_Status) String() string { + return proto.EnumName(SnapshotReplicationStatusRes_Status_name, int32(x)) +} +func (SnapshotReplicationStatusRes_Status) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{15, 0} } type ListFilesystemReq struct { @@ -74,7 +100,7 @@ func (m *ListFilesystemReq) Reset() { *m = ListFilesystemReq{} } func (m *ListFilesystemReq) String() string { return proto.CompactTextString(m) } func (*ListFilesystemReq) ProtoMessage() {} func (*ListFilesystemReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{0} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{0} } func (m *ListFilesystemReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemReq.Unmarshal(m, b) @@ -105,7 +131,7 @@ func (m *ListFilesystemRes) Reset() { *m = ListFilesystemRes{} } func (m *ListFilesystemRes) String() string { return proto.CompactTextString(m) } func (*ListFilesystemRes) ProtoMessage() {} func (*ListFilesystemRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{1} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{1} } func (m *ListFilesystemRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemRes.Unmarshal(m, b) @@ -144,7 +170,7 @@ func (m *Filesystem) Reset() { *m = Filesystem{} } func (m *Filesystem) String() string { return proto.CompactTextString(m) } func (*Filesystem) ProtoMessage() {} func (*Filesystem) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{2} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{2} } func (m *Filesystem) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Filesystem.Unmarshal(m, b) @@ -189,7 +215,7 @@ func (m *ListFilesystemVersionsReq) Reset() { *m = ListFilesystemVersion func (m *ListFilesystemVersionsReq) String() string { return proto.CompactTextString(m) } func (*ListFilesystemVersionsReq) ProtoMessage() {} func (*ListFilesystemVersionsReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{3} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{3} } func (m *ListFilesystemVersionsReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemVersionsReq.Unmarshal(m, b) @@ -227,7 +253,7 @@ func (m *ListFilesystemVersionsRes) Reset() { *m = ListFilesystemVersion func (m *ListFilesystemVersionsRes) String() string { return proto.CompactTextString(m) } func (*ListFilesystemVersionsRes) ProtoMessage() {} func (*ListFilesystemVersionsRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{4} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{4} } func (m *ListFilesystemVersionsRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemVersionsRes.Unmarshal(m, b) @@ -269,7 +295,7 @@ func (m *FilesystemVersion) Reset() { *m = FilesystemVersion{} } func (m *FilesystemVersion) String() string { return proto.CompactTextString(m) } func (*FilesystemVersion) ProtoMessage() {} func (*FilesystemVersion) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{5} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{5} } func (m *FilesystemVersion) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_FilesystemVersion.Unmarshal(m, b) @@ -349,7 +375,7 @@ func (m *SendReq) Reset() { *m = SendReq{} } func (m *SendReq) String() string { return proto.CompactTextString(m) } func (*SendReq) ProtoMessage() {} func (*SendReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{6} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{6} } func (m *SendReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SendReq.Unmarshal(m, b) @@ -430,7 +456,7 @@ func (m *Property) Reset() { *m = Property{} } func (m *Property) String() string { return proto.CompactTextString(m) } func (*Property) ProtoMessage() {} func (*Property) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{7} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{7} } func (m *Property) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Property.Unmarshal(m, b) @@ -479,7 +505,7 @@ func (m *SendRes) Reset() { *m = SendRes{} } func (m *SendRes) String() string { return proto.CompactTextString(m) } func (*SendRes) ProtoMessage() {} func (*SendRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{8} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{8} } func (m *SendRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SendRes.Unmarshal(m, b) @@ -533,7 +559,7 @@ func (m *ReceiveReq) Reset() { *m = ReceiveReq{} } func (m *ReceiveReq) String() string { return proto.CompactTextString(m) } func (*ReceiveReq) ProtoMessage() {} func (*ReceiveReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{9} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{9} } func (m *ReceiveReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ReceiveReq.Unmarshal(m, b) @@ -577,7 +603,7 @@ func (m *ReceiveRes) Reset() { *m = ReceiveRes{} } func (m *ReceiveRes) String() string { return proto.CompactTextString(m) } func (*ReceiveRes) ProtoMessage() {} func (*ReceiveRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{10} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{10} } func (m *ReceiveRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ReceiveRes.Unmarshal(m, b) @@ -610,7 +636,7 @@ func (m *DestroySnapshotsReq) Reset() { *m = DestroySnapshotsReq{} } func (m *DestroySnapshotsReq) String() string { return proto.CompactTextString(m) } func (*DestroySnapshotsReq) ProtoMessage() {} func (*DestroySnapshotsReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{11} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{11} } func (m *DestroySnapshotsReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DestroySnapshotsReq.Unmarshal(m, b) @@ -656,7 +682,7 @@ func (m *DestroySnapshotRes) Reset() { *m = DestroySnapshotRes{} } func (m *DestroySnapshotRes) String() string { return proto.CompactTextString(m) } func (*DestroySnapshotRes) ProtoMessage() {} func (*DestroySnapshotRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{12} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{12} } func (m *DestroySnapshotRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DestroySnapshotRes.Unmarshal(m, b) @@ -701,7 +727,7 @@ func (m *DestroySnapshotsRes) Reset() { *m = DestroySnapshotsRes{} } func (m *DestroySnapshotsRes) String() string { return proto.CompactTextString(m) } func (*DestroySnapshotsRes) ProtoMessage() {} func (*DestroySnapshotsRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{13} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{13} } func (m *DestroySnapshotsRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DestroySnapshotsRes.Unmarshal(m, b) @@ -741,7 +767,7 @@ func (m *SnapshotReplicationStatusReq) Reset() { *m = SnapshotReplicatio func (m *SnapshotReplicationStatusReq) String() string { return proto.CompactTextString(m) } func (*SnapshotReplicationStatusReq) ProtoMessage() {} func (*SnapshotReplicationStatusReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{14} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{14} } func (m *SnapshotReplicationStatusReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SnapshotReplicationStatusReq.Unmarshal(m, b) @@ -783,17 +809,17 @@ func (m *SnapshotReplicationStatusReq) GetOp() SnapshotReplicationStatusReq_Op { } type SnapshotReplicationStatusRes struct { - Replicated bool `protobuf:"varint,1,opt,name=Replicated,proto3" json:"Replicated,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + Status SnapshotReplicationStatusRes_Status `protobuf:"varint,1,opt,name=status,proto3,enum=pdu.SnapshotReplicationStatusRes_Status" json:"status,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *SnapshotReplicationStatusRes) Reset() { *m = SnapshotReplicationStatusRes{} } func (m *SnapshotReplicationStatusRes) String() string { return proto.CompactTextString(m) } func (*SnapshotReplicationStatusRes) ProtoMessage() {} func (*SnapshotReplicationStatusRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_ae88208ed9690d0f, []int{15} + return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{15} } func (m *SnapshotReplicationStatusRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SnapshotReplicationStatusRes.Unmarshal(m, b) @@ -813,11 +839,11 @@ func (m *SnapshotReplicationStatusRes) XXX_DiscardUnknown() { var xxx_messageInfo_SnapshotReplicationStatusRes proto.InternalMessageInfo -func (m *SnapshotReplicationStatusRes) GetReplicated() bool { +func (m *SnapshotReplicationStatusRes) GetStatus() SnapshotReplicationStatusRes_Status { if m != nil { - return m.Replicated + return m.Status } - return false + return SnapshotReplicationStatusRes_Nonexistent } func init() { @@ -839,50 +865,53 @@ func init() { proto.RegisterType((*SnapshotReplicationStatusRes)(nil), "pdu.SnapshotReplicationStatusRes") proto.RegisterEnum("pdu.FilesystemVersion_VersionType", FilesystemVersion_VersionType_name, FilesystemVersion_VersionType_value) proto.RegisterEnum("pdu.SnapshotReplicationStatusReq_Op", SnapshotReplicationStatusReq_Op_name, SnapshotReplicationStatusReq_Op_value) + proto.RegisterEnum("pdu.SnapshotReplicationStatusRes_Status", SnapshotReplicationStatusRes_Status_name, SnapshotReplicationStatusRes_Status_value) } -func init() { proto.RegisterFile("pdu.proto", fileDescriptor_pdu_ae88208ed9690d0f) } +func init() { proto.RegisterFile("pdu.proto", fileDescriptor_pdu_b3a98b3542e9fb4e) } -var fileDescriptor_pdu_ae88208ed9690d0f = []byte{ - // 627 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0xdb, 0x6e, 0x13, 0x31, - 0x10, 0x65, 0x37, 0x69, 0x93, 0x4c, 0x7a, 0x49, 0xdd, 0xaa, 0x2c, 0x55, 0x85, 0x22, 0x8b, 0x87, - 0x80, 0x44, 0x24, 0x42, 0xc5, 0x0b, 0x12, 0x0f, 0xbd, 0xf2, 0x80, 0x68, 0xe5, 0x84, 0xaa, 0x4f, - 0x48, 0x4b, 0x77, 0xa4, 0xae, 0x72, 0xb1, 0x6b, 0xef, 0x22, 0xc2, 0x07, 0xf0, 0x41, 0xbc, 0xf1, - 0x27, 0x7c, 0x0e, 0xf2, 0x64, 0x2f, 0x6e, 0x52, 0xa2, 0x3c, 0xd5, 0xe7, 0xcc, 0xd9, 0x99, 0x33, - 0x63, 0x4f, 0x03, 0x0d, 0x15, 0xa5, 0x5d, 0xa5, 0x65, 0x22, 0x59, 0x45, 0x45, 0x29, 0xdf, 0x85, - 0x9d, 0x4f, 0xb1, 0x49, 0xce, 0xe3, 0x11, 0x9a, 0xa9, 0x49, 0x70, 0x2c, 0xf0, 0x9e, 0x9f, 0x2f, - 0x92, 0x86, 0xbd, 0x81, 0x66, 0x49, 0x98, 0xc0, 0x6b, 0x57, 0x3a, 0xcd, 0xde, 0x76, 0xd7, 0xe6, - 0x73, 0x84, 0xae, 0x86, 0x1f, 0x03, 0x94, 0x90, 0x31, 0xa8, 0x5e, 0x85, 0xc9, 0x5d, 0xe0, 0xb5, - 0xbd, 0x4e, 0x43, 0xd0, 0x99, 0xb5, 0xa1, 0x29, 0xd0, 0xa4, 0x63, 0x1c, 0xc8, 0x21, 0x4e, 0x02, - 0x9f, 0x42, 0x2e, 0xc5, 0xdf, 0xc3, 0xb3, 0x87, 0x5e, 0xae, 0x51, 0x9b, 0x58, 0x4e, 0x8c, 0xc0, - 0x7b, 0xf6, 0xdc, 0x2d, 0x90, 0x25, 0x76, 0x18, 0x7e, 0xf9, 0xff, 0x8f, 0x0d, 0xeb, 0x41, 0x3d, - 0x87, 0x59, 0x37, 0xfb, 0x73, 0xdd, 0x64, 0x61, 0x51, 0xe8, 0xf8, 0x5f, 0x0f, 0x76, 0x16, 0xe2, - 0xec, 0x1d, 0x54, 0x07, 0x53, 0x85, 0x64, 0x60, 0xab, 0xc7, 0x1f, 0xcf, 0xd2, 0xcd, 0xfe, 0x5a, - 0xa5, 0x20, 0xbd, 0x9d, 0xc8, 0xe7, 0x70, 0x8c, 0x59, 0xdb, 0x74, 0xb6, 0xdc, 0x45, 0x1a, 0x47, - 0x41, 0xa5, 0xed, 0x75, 0xaa, 0x82, 0xce, 0xec, 0x10, 0x1a, 0x27, 0x1a, 0xc3, 0x04, 0x07, 0x37, - 0x17, 0x41, 0x95, 0x02, 0x25, 0xc1, 0x0e, 0xa0, 0x4e, 0x20, 0x96, 0x93, 0x60, 0x8d, 0x32, 0x15, - 0x98, 0xbf, 0x84, 0xa6, 0x53, 0x96, 0x6d, 0x40, 0xbd, 0x3f, 0x09, 0x95, 0xb9, 0x93, 0x49, 0xeb, - 0x89, 0x45, 0xc7, 0x52, 0x0e, 0xc7, 0xa1, 0x1e, 0xb6, 0x3c, 0xfe, 0xc7, 0x83, 0x5a, 0x1f, 0x27, - 0xd1, 0x0a, 0x73, 0xb5, 0x26, 0xcf, 0xb5, 0x1c, 0xe7, 0xc6, 0xed, 0x99, 0x6d, 0x81, 0x3f, 0x90, - 0x64, 0xbb, 0x21, 0xfc, 0x81, 0x9c, 0xbf, 0xda, 0xea, 0xc2, 0xd5, 0x92, 0x71, 0x39, 0x56, 0x1a, - 0x8d, 0x21, 0xe3, 0x75, 0x51, 0x60, 0xb6, 0x07, 0x6b, 0xa7, 0x18, 0xa5, 0x2a, 0x58, 0xa7, 0xc0, - 0x0c, 0xb0, 0x7d, 0x58, 0x3f, 0xd5, 0x53, 0x91, 0x4e, 0x82, 0x1a, 0xd1, 0x19, 0xe2, 0x47, 0x50, - 0xbf, 0xd2, 0x52, 0xa1, 0x4e, 0xa6, 0xc5, 0x50, 0x3d, 0x67, 0xa8, 0x7b, 0xb0, 0x76, 0x1d, 0x8e, - 0xd2, 0x7c, 0xd2, 0x33, 0xc0, 0x7f, 0x15, 0x1d, 0x1b, 0xd6, 0x81, 0xed, 0x2f, 0x06, 0x23, 0xd7, - 0xb1, 0x47, 0x25, 0xe6, 0x69, 0xc6, 0x61, 0xe3, 0xec, 0x87, 0xc2, 0xdb, 0x04, 0xa3, 0x7e, 0xfc, - 0x73, 0x96, 0xb2, 0x22, 0x1e, 0x70, 0xec, 0x35, 0x40, 0xe6, 0x27, 0x46, 0x13, 0x54, 0xe8, 0x71, - 0x6d, 0xd2, 0xb3, 0xc8, 0x6d, 0x0a, 0x47, 0xc0, 0x6f, 0x00, 0x04, 0xde, 0x62, 0xfc, 0x1d, 0x57, - 0x19, 0xfe, 0x2b, 0x68, 0x9d, 0x8c, 0x30, 0xd4, 0xf3, 0x8b, 0x53, 0x17, 0x0b, 0x3c, 0xdf, 0x70, - 0x32, 0x1b, 0x3e, 0x84, 0xdd, 0x53, 0x34, 0x89, 0x96, 0xd3, 0xfc, 0x15, 0xac, 0xb2, 0x45, 0xec, - 0x08, 0x1a, 0x85, 0x3e, 0xf0, 0x97, 0x6e, 0x4a, 0x29, 0xe4, 0x5f, 0x81, 0xcd, 0x15, 0xcb, 0x96, - 0x2e, 0x87, 0x54, 0x69, 0xc9, 0xd2, 0xe5, 0x3a, 0x7b, 0x7b, 0x67, 0x5a, 0x4b, 0x9d, 0xdf, 0x1e, - 0x01, 0xfe, 0xf1, 0xb1, 0x66, 0xec, 0xbf, 0xa9, 0x9a, 0x1d, 0xc0, 0x28, 0xc9, 0x97, 0xfa, 0x29, - 0xe5, 0x5f, 0xb4, 0x22, 0x72, 0x1d, 0xff, 0xed, 0xc1, 0x61, 0x19, 0x50, 0xa3, 0xf8, 0x96, 0x96, - 0xa7, 0x9f, 0x84, 0x49, 0xba, 0xd2, 0x80, 0x0e, 0x9c, 0xa6, 0x66, 0x1e, 0x4b, 0xf3, 0x47, 0xe0, - 0x4b, 0x45, 0x6b, 0xb1, 0xd5, 0x7b, 0x41, 0x56, 0x96, 0x95, 0xea, 0x5e, 0x2a, 0xe1, 0x4b, 0xc5, - 0xdb, 0xe0, 0x5f, 0x2a, 0x56, 0x83, 0xca, 0x05, 0xda, 0x4d, 0xdd, 0x81, 0xcd, 0x3e, 0x16, 0x1f, - 0x60, 0xd4, 0xf2, 0xf8, 0x87, 0xa5, 0x9e, 0x8d, 0xf5, 0x5c, 0xea, 0xb3, 0xb7, 0xec, 0x30, 0xdf, - 0xd6, 0xe9, 0x47, 0xe0, 0xed, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xf0, 0xc1, 0x08, 0xf6, 0x11, - 0x06, 0x00, 0x00, +var fileDescriptor_pdu_b3a98b3542e9fb4e = []byte{ + // 666 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0xcf, 0x6e, 0xd3, 0x4e, + 0x10, 0xfe, 0xd9, 0x49, 0xf3, 0x67, 0xd2, 0xa6, 0xe9, 0xb6, 0xea, 0xcf, 0x54, 0x15, 0x8a, 0x56, + 0x1c, 0x02, 0x12, 0x91, 0x08, 0x15, 0x17, 0x38, 0xa0, 0xfe, 0xe5, 0x80, 0xda, 0x6a, 0x13, 0xaa, + 0x9e, 0x90, 0x4c, 0x3d, 0x52, 0xad, 0x24, 0xde, 0xed, 0xee, 0x1a, 0x35, 0x3c, 0x00, 0x8f, 0xc1, + 0x43, 0x70, 0xe3, 0x4d, 0x78, 0x1c, 0xe4, 0x89, 0xed, 0xb8, 0x49, 0x09, 0x39, 0x65, 0xbe, 0x6f, + 0x66, 0x67, 0xbe, 0x99, 0xdd, 0x71, 0xa0, 0xae, 0x82, 0xb8, 0xab, 0xb4, 0xb4, 0x92, 0x95, 0x54, + 0x10, 0xf3, 0x6d, 0xd8, 0xfa, 0x18, 0x1a, 0x7b, 0x1a, 0x8e, 0xd0, 0x4c, 0x8c, 0xc5, 0xb1, 0xc0, + 0x3b, 0x7e, 0xba, 0x48, 0x1a, 0xf6, 0x0a, 0x1a, 0x33, 0xc2, 0x78, 0x4e, 0xbb, 0xd4, 0x69, 0xf4, + 0x36, 0xbb, 0x49, 0xbe, 0x42, 0x60, 0x31, 0x86, 0x1f, 0x02, 0xcc, 0x20, 0x63, 0x50, 0xbe, 0xf4, + 0xed, 0xad, 0xe7, 0xb4, 0x9d, 0x4e, 0x5d, 0x90, 0xcd, 0xda, 0xd0, 0x10, 0x68, 0xe2, 0x31, 0x0e, + 0xe4, 0x10, 0x23, 0xcf, 0x25, 0x57, 0x91, 0xe2, 0x6f, 0xe1, 0xc9, 0x43, 0x2d, 0x57, 0xa8, 0x4d, + 0x28, 0x23, 0x23, 0xf0, 0x8e, 0x3d, 0x2d, 0x16, 0x48, 0x13, 0x17, 0x18, 0x7e, 0xf1, 0xf7, 0xc3, + 0x86, 0xf5, 0xa0, 0x96, 0xc1, 0xb4, 0x9b, 0xdd, 0xb9, 0x6e, 0x52, 0xb7, 0xc8, 0xe3, 0xf8, 0x6f, + 0x07, 0xb6, 0x16, 0xfc, 0xec, 0x0d, 0x94, 0x07, 0x13, 0x85, 0x24, 0xa0, 0xd9, 0xe3, 0x8f, 0x67, + 0xe9, 0xa6, 0xbf, 0x49, 0xa4, 0xa0, 0xf8, 0x64, 0x22, 0xe7, 0xfe, 0x18, 0xd3, 0xb6, 0xc9, 0x4e, + 0xb8, 0xb3, 0x38, 0x0c, 0xbc, 0x52, 0xdb, 0xe9, 0x94, 0x05, 0xd9, 0x6c, 0x1f, 0xea, 0x47, 0x1a, + 0x7d, 0x8b, 0x83, 0xeb, 0x33, 0xaf, 0x4c, 0x8e, 0x19, 0xc1, 0xf6, 0xa0, 0x46, 0x20, 0x94, 0x91, + 0xb7, 0x46, 0x99, 0x72, 0xcc, 0x9f, 0x43, 0xa3, 0x50, 0x96, 0xad, 0x43, 0xad, 0x1f, 0xf9, 0xca, + 0xdc, 0x4a, 0xdb, 0xfa, 0x2f, 0x41, 0x87, 0x52, 0x0e, 0xc7, 0xbe, 0x1e, 0xb6, 0x1c, 0xfe, 0xcb, + 0x81, 0x6a, 0x1f, 0xa3, 0x60, 0x85, 0xb9, 0x26, 0x22, 0x4f, 0xb5, 0x1c, 0x67, 0xc2, 0x13, 0x9b, + 0x35, 0xc1, 0x1d, 0x48, 0x92, 0x5d, 0x17, 0xee, 0x40, 0xce, 0x5f, 0x6d, 0x79, 0xe1, 0x6a, 0x49, + 0xb8, 0x1c, 0x2b, 0x8d, 0xc6, 0x90, 0xf0, 0x9a, 0xc8, 0x31, 0xdb, 0x81, 0xb5, 0x63, 0x0c, 0x62, + 0xe5, 0x55, 0xc8, 0x31, 0x05, 0x6c, 0x17, 0x2a, 0xc7, 0x7a, 0x22, 0xe2, 0xc8, 0xab, 0x12, 0x9d, + 0x22, 0x7e, 0x00, 0xb5, 0x4b, 0x2d, 0x15, 0x6a, 0x3b, 0xc9, 0x87, 0xea, 0x14, 0x86, 0xba, 0x03, + 0x6b, 0x57, 0xfe, 0x28, 0xce, 0x26, 0x3d, 0x05, 0xfc, 0x7b, 0xde, 0xb1, 0x61, 0x1d, 0xd8, 0xfc, + 0x64, 0x30, 0x28, 0x2a, 0x76, 0xa8, 0xc4, 0x3c, 0xcd, 0x38, 0xac, 0x9f, 0xdc, 0x2b, 0xbc, 0xb1, + 0x18, 0xf4, 0xc3, 0x6f, 0xd3, 0x94, 0x25, 0xf1, 0x80, 0x63, 0x2f, 0x01, 0x52, 0x3d, 0x21, 0x1a, + 0xaf, 0x44, 0x8f, 0x6b, 0x83, 0x9e, 0x45, 0x26, 0x53, 0x14, 0x02, 0xf8, 0x35, 0x80, 0xc0, 0x1b, + 0x0c, 0xbf, 0xe2, 0x2a, 0xc3, 0x7f, 0x01, 0xad, 0xa3, 0x11, 0xfa, 0x7a, 0x7e, 0x71, 0x6a, 0x62, + 0x81, 0xe7, 0xeb, 0x85, 0xcc, 0x86, 0x0f, 0x61, 0xfb, 0x18, 0x8d, 0xd5, 0x72, 0x92, 0xbd, 0x82, + 0x55, 0xb6, 0x88, 0x1d, 0x40, 0x3d, 0x8f, 0xf7, 0xdc, 0xa5, 0x9b, 0x32, 0x0b, 0xe4, 0x9f, 0x81, + 0xcd, 0x15, 0x4b, 0x97, 0x2e, 0x83, 0x54, 0x69, 0xc9, 0xd2, 0x65, 0x71, 0xc9, 0xed, 0x9d, 0x68, + 0x2d, 0x75, 0x76, 0x7b, 0x04, 0xf8, 0x87, 0xc7, 0x9a, 0x49, 0x3e, 0x53, 0xd5, 0x64, 0x00, 0x23, + 0x9b, 0x2d, 0xf5, 0xff, 0x94, 0x7f, 0x51, 0x8a, 0xc8, 0xe2, 0xf8, 0x4f, 0x07, 0xf6, 0x67, 0x0e, + 0x35, 0x0a, 0x6f, 0x68, 0x79, 0xfa, 0xd6, 0xb7, 0xf1, 0x4a, 0x03, 0xda, 0x2b, 0x34, 0x35, 0xd5, + 0x38, 0x13, 0x7f, 0x00, 0xae, 0x54, 0xb4, 0x16, 0xcd, 0xde, 0x33, 0x92, 0xb2, 0xac, 0x54, 0xf7, + 0x42, 0x09, 0x57, 0x2a, 0xde, 0x06, 0xf7, 0x42, 0xb1, 0x2a, 0x94, 0xce, 0x30, 0xd9, 0xd4, 0x2d, + 0xd8, 0xe8, 0x63, 0x7e, 0x00, 0x83, 0x96, 0xc3, 0x7f, 0x2c, 0x17, 0x6d, 0xd8, 0x7b, 0xa8, 0x18, + 0x02, 0xe9, 0x67, 0xa9, 0xf3, 0xaf, 0xe2, 0xa6, 0x9b, 0x5a, 0xe9, 0x39, 0xfe, 0x0e, 0x2a, 0x53, + 0x86, 0x6d, 0x42, 0xe3, 0x5c, 0x46, 0x78, 0x1f, 0x1a, 0x8b, 0x51, 0x2a, 0xe8, 0x5c, 0x3e, 0x10, + 0xc4, 0x9a, 0xc9, 0x53, 0xcb, 0xb1, 0xfb, 0xa5, 0x42, 0xff, 0x32, 0xaf, 0xff, 0x04, 0x00, 0x00, + 0xff, 0xff, 0x3a, 0x00, 0x8b, 0x6e, 0x72, 0x06, 0x00, 0x00, } diff --git a/replication/pdu/pdu.proto b/replication/pdu/pdu.proto index 211fc0e..f390fd8 100644 --- a/replication/pdu/pdu.proto +++ b/replication/pdu/pdu.proto @@ -108,5 +108,10 @@ message SnapshotReplicationStatusReq { } message SnapshotReplicationStatusRes { - bool Replicated = 1; + enum Status { + Nonexistent = 0; + NotReplicated = 1; + Replicated = 2; + } + Status status = 1; } From 4336af295f9c76f4fc79fd91e1f4c2e98eb9e3e2 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 4 Sep 2018 13:30:52 -0700 Subject: [PATCH 094/167] fixup 22ca80eb7e71c10eba577e96e8356922d42ed8f3: scraping regex was broken and potentially mixed with stdout --- zfs/zfs.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/zfs/zfs.go b/zfs/zfs.go index 577449e..a218c72 100644 --- a/zfs/zfs.go +++ b/zfs/zfs.go @@ -477,7 +477,7 @@ func ZFSGet(fs *DatasetPath, props []string) (*ZFSProperties, error) { return zfsGet(fs.ToString(), props) } -var zfsGetDatasetDoesNotExistRegexp = regexp.MustCompile(`^cannot open '(\s+)': dataset does not exist`) +var zfsGetDatasetDoesNotExistRegexp = regexp.MustCompile(`^cannot open '(\S+)': dataset does not exist`) type DatasetDoesNotExist struct { Path string @@ -488,12 +488,12 @@ func (d *DatasetDoesNotExist) Error() string { return fmt.Sprintf("dataset %q do func zfsGet(path string, props []string) (*ZFSProperties, error) { args := []string{"get", "-Hp", "-o", "property,value", strings.Join(props, ","), path} cmd := exec.Command(ZFS_BINARY, args...) - output, err := cmd.CombinedOutput() + stdout, err := cmd.Output() if err != nil { if exitErr, ok := err.(*exec.ExitError); ok { if exitErr.Exited() { // screen-scrape output - if sm := zfsGetDatasetDoesNotExistRegexp.FindSubmatch(output); sm != nil { + if sm := zfsGetDatasetDoesNotExistRegexp.FindSubmatch(exitErr.Stderr); sm != nil { if string(sm[1]) == path { return nil, &DatasetDoesNotExist{path} } @@ -502,7 +502,7 @@ func zfsGet(path string, props []string) (*ZFSProperties, error) { } return nil, err } - o := string(output) + o := string(stdout) lines := strings.Split(o, "\n") if len(lines) < 1 || // account for newlines len(lines)-1 != len(props) { From be57d6ce8efc458fcfec94430cb6020d7eb5ae8a Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 4 Sep 2018 13:32:19 -0700 Subject: [PATCH 095/167] replication/diff: replace invalid comparison of CreateTXG with Creation --- replication/internal/diff/diff.go | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/replication/internal/diff/diff.go b/replication/internal/diff/diff.go index 213bc77..c76f5de 100644 --- a/replication/internal/diff/diff.go +++ b/replication/internal/diff/diff.go @@ -65,7 +65,16 @@ func IncrementalPath(receiver, sender []*FilesystemVersion) (incPath []*Filesyst } break } - if receiver[mrcaRcv].CreateTXG < sender[mrcaSnd].CreateTXG { + receiverCreation, err := receiver[mrcaRcv].CreationAsTime() + if err != nil { + panic(err) // FIXME move this to a sorting phase before + } + senderCreation, err := sender[mrcaSnd].CreationAsTime() + if err != nil { + panic(err) // FIXME move this to the sorting phase before + } + + if receiverCreation.Before(senderCreation) { mrcaSnd-- } else { mrcaRcv-- From 754b253043c5cfba8bb092d4f54f6976b2bf2092 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 4 Sep 2018 14:44:45 -0700 Subject: [PATCH 096/167] config: no-field for replication anymore It's closer to the original config and we don't want users to specify 'filesystems' and similar multiple times in a single job definition. --- config/config.go | 40 +++++++------------------ config/config_minimal_test.go | 19 ++++++------ config/config_rpc_test.go | 55 +++++++++++++++++------------------ config/samples/local.yml | 17 +++++------ config/samples/pull.yml | 19 ++++++------ config/samples/pull_ssh.yml | 21 +++++++------ config/samples/push.yml | 15 +++++----- config/samples/sink.yml | 17 +++++------ config/samples/source.yml | 19 ++++++------ config/samples/source_ssh.yml | 15 +++++----- daemon/job/push.go | 4 +-- daemon/job/sink.go | 4 +-- 12 files changed, 108 insertions(+), 137 deletions(-) diff --git a/config/config.go b/config/config.go index d312f57..ab6f339 100644 --- a/config/config.go +++ b/config/config.go @@ -24,7 +24,8 @@ type JobEnum struct { type PushJob struct { Type string `yaml:"type"` Name string `yaml:"name"` - Replication PushReplication `yaml:"replication"` + Connect ConnectEnum `yaml:"connect"` + Filesystems FilesystemsFilter `yaml:"filesystems"` Snapshotting Snapshotting `yaml:"snapshotting"` Pruning PruningSenderReceiver `yaml:"pruning"` Debug JobDebugSettings `yaml:"debug,optional"` @@ -33,28 +34,26 @@ type PushJob struct { type SinkJob struct { Type string `yaml:"type"` Name string `yaml:"name"` - Replication SinkReplication `yaml:"replication"` + RootDataset string `yaml:"root_dataset"` + Serve ServeEnum `yaml:"serve"` Debug JobDebugSettings `yaml:"debug,optional"` } type PullJob struct { Type string `yaml:"type"` Name string `yaml:"name"` - Replication PullReplication `yaml:"replication"` - Pruning PruningSenderReceiver `yaml:"pruning"` - Debug JobDebugSettings `yaml:"debug,optional"` -} - -type PullReplication struct { Connect ConnectEnum `yaml:"connect"` RootDataset string `yaml:"root_dataset"` Interval time.Duration `yaml:"interval,positive"` + Pruning PruningSenderReceiver `yaml:"pruning"` + Debug JobDebugSettings `yaml:"debug,optional"` } type SourceJob struct { Type string `yaml:"type"` Name string `yaml:"name"` - Replication SourceReplication `yaml:"replication"` + Serve ServeEnum `yaml:"serve"` + Filesystems FilesystemsFilter `yaml:"filesystems"` Snapshotting Snapshotting `yaml:"snapshotting"` Pruning PruningLocal `yaml:"pruning"` Debug JobDebugSettings `yaml:"debug,optional"` @@ -63,31 +62,14 @@ type SourceJob struct { type LocalJob struct { Type string `yaml:"type"` Name string `yaml:"name"` - Replication LocalReplication `yaml:"replication"` + Filesystems FilesystemsFilter `yaml:"filesystems"` + RootDataset string `yaml:"root_dataset"` Snapshotting Snapshotting `yaml:"snapshotting"` Pruning PruningSenderReceiver `yaml:"pruning"` Debug JobDebugSettings `yaml:"debug,optional"` } -type PushReplication struct { - Connect ConnectEnum `yaml:"connect"` - Filesystems map[string]bool `yaml:"filesystems"` -} - -type SinkReplication struct { - RootDataset string `yaml:"root_dataset"` - Serve ServeEnum `yaml:"serve"` -} - -type SourceReplication struct { - Serve ServeEnum `yaml:"serve"` - Filesystems map[string]bool `yaml:"filesystems"` -} - -type LocalReplication struct { - Filesystems map[string]bool `yaml:"filesystems"` - RootDataset string `yaml:"root_dataset"` -} +type FilesystemsFilter map[string]bool type Snapshotting struct { SnapshotPrefix string `yaml:"snapshot_prefix"` diff --git a/config/config_minimal_test.go b/config/config_minimal_test.go index 82edde8..fa9c780 100644 --- a/config/config_minimal_test.go +++ b/config/config_minimal_test.go @@ -18,16 +18,15 @@ jobs: type: push # snapshot the filesystems matched by the left-hand-side of the mapping # every 10m with zrepl_ as prefix - replication: - connect: - type: tcp - address: localhost:2342 - filesystems: { - "pool1/var/db<": true, - "pool1/usr/home<": true, - "pool1/usr/home/paranoid": false, #don't backup paranoid user - "pool1/poudriere/ports<": false #don't backup the ports trees - } + connect: + type: tcp + address: localhost:2342 + filesystems: { + "pool1/var/db<": true, + "pool1/usr/home<": true, + "pool1/usr/home/paranoid": false, #don't backup paranoid user + "pool1/poudriere/ports<": false #don't backup the ports trees + } snapshotting: snapshot_prefix: zrepl_ interval: 10m diff --git a/config/config_rpc_test.go b/config/config_rpc_test.go index fa041b4..e295752 100644 --- a/config/config_rpc_test.go +++ b/config/config_rpc_test.go @@ -11,14 +11,13 @@ func TestRPC(t *testing.T) { jobs: - name: pull_servers type: pull - replication: - connect: - type: tcp - address: "server1.foo.bar:8888" - rpc: - timeout: 20s # different form default, should merge - root_dataset: "pool2/backup_servers" - interval: 10m + connect: + type: tcp + address: "server1.foo.bar:8888" + rpc: + timeout: 20s # different form default, should merge + root_dataset: "pool2/backup_servers" + interval: 10m pruning: keep_sender: - type: not_replicated @@ -28,14 +27,13 @@ jobs: - name: pull_servers2 type: pull - replication: - connect: - type: tcp - address: "server1.foo.bar:8888" - rpc: - tx_chunk_size: 0xabcd # different from default, should merge - root_dataset: "pool2/backup_servers" - interval: 10m + connect: + type: tcp + address: "server1.foo.bar:8888" + rpc: + tx_chunk_size: 0xabcd # different from default, should merge + root_dataset: "pool2/backup_servers" + interval: 10m pruning: keep_sender: - type: not_replicated @@ -45,22 +43,21 @@ jobs: - type: sink name: "laptop_sink" - replication: - root_dataset: "pool2/backup_laptops" - serve: - type: tcp - listen: "192.168.122.189:8888" - clients: { - "10.23.42.23":"client1" - } - rpc: - rx_structured_max: 0x2342 + root_dataset: "pool2/backup_laptops" + serve: + type: tcp + listen: "192.168.122.189:8888" + clients: { + "10.23.42.23":"client1" + } + rpc: + rx_structured_max: 0x2342 `) - assert.Equal(t, 20*time.Second, conf.Jobs[0].Ret.(*PullJob).Replication.Connect.Ret.(*TCPConnect).RPC.Timeout) - assert.Equal(t, uint32(0xabcd), conf.Jobs[1].Ret.(*PullJob).Replication.Connect.Ret.(*TCPConnect).RPC.TxChunkSize) - assert.Equal(t, uint32(0x2342), conf.Jobs[2].Ret.(*SinkJob).Replication.Serve.Ret.(*TCPServe).RPC.RxStructuredMaxLen) + assert.Equal(t, 20*time.Second, conf.Jobs[0].Ret.(*PullJob).Connect.Ret.(*TCPConnect).RPC.Timeout) + assert.Equal(t, uint32(0xabcd), conf.Jobs[1].Ret.(*PullJob).Connect.Ret.(*TCPConnect).RPC.TxChunkSize) + assert.Equal(t, uint32(0x2342), conf.Jobs[2].Ret.(*SinkJob).Serve.Ret.(*TCPServe).RPC.RxStructuredMaxLen) defConf := RPCConfig{} Default(&defConf) assert.Equal(t, defConf.Timeout, conf.Global.RPC.Timeout) diff --git a/config/samples/local.yml b/config/samples/local.yml index 5473b42..3c56373 100644 --- a/config/samples/local.yml +++ b/config/samples/local.yml @@ -4,15 +4,14 @@ jobs: type: local # snapshot the filesystems matched by the left-hand-side of the mapping # every 10m with zrepl_ as prefix - replication: - filesystems: { - "pool1/var/db<": true, - "pool1/usr/home<": true, - "pool1/usr/home/paranoid": false, #don't backup paranoid user - "pool1/poudriere/ports<": false #don't backup the ports trees - } - # TODO FIXME enforce that the tree under root_dataset and the trees allowed (true) by filesystems are non-overlapping - root_dataset: "pool2/backups/pool1" + filesystems: { + "pool1/var/db<": true, + "pool1/usr/home<": true, + "pool1/usr/home/paranoid": false, #don't backup paranoid user + "pool1/poudriere/ports<": false #don't backup the ports trees + } + # TODO FIXME enforce that the tree under root_dataset and the trees allowed (true) by filesystems are non-overlapping + root_dataset: "pool2/backups/pool1" snapshotting: snapshot_prefix: zrepl_ diff --git a/config/samples/pull.yml b/config/samples/pull.yml index a6a37c7..45406c6 100644 --- a/config/samples/pull.yml +++ b/config/samples/pull.yml @@ -1,16 +1,15 @@ jobs: - name: pull_servers type: pull - replication: - connect: - type: tls - address: "server1.foo.bar:8888" - ca: "/certs/ca.crt" - cert: "/certs/cert.crt" - key: "/certs/key.pem" - server_cn: "server1" - root_dataset: "pool2/backup_servers" - interval: 10m + connect: + type: tls + address: "server1.foo.bar:8888" + ca: "/certs/ca.crt" + cert: "/certs/cert.crt" + key: "/certs/key.pem" + server_cn: "server1" + root_dataset: "pool2/backup_servers" + interval: 10m pruning: keep_sender: - type: not_replicated diff --git a/config/samples/pull_ssh.yml b/config/samples/pull_ssh.yml index 317aae1..9830b01 100644 --- a/config/samples/pull_ssh.yml +++ b/config/samples/pull_ssh.yml @@ -2,17 +2,16 @@ jobs: - name: pull_servers type: pull - replication: - connect: - type: ssh+stdinserver - host: app-srv.example.com - user: root - port: 22 - identity_file: /etc/zrepl/ssh/identity - options: # optional, default [], `-o` arguments passed to ssh - - "Compression=on" - root_dataset: "pool2/backup_servers" - interval: 10m + connect: + type: ssh+stdinserver + host: app-srv.example.com + user: root + port: 22 + identity_file: /etc/zrepl/ssh/identity + options: # optional, default [], `-o` arguments passed to ssh + - "Compression=on" + root_dataset: "pool2/backup_servers" + interval: 10m pruning: keep_sender: - type: not_replicated diff --git a/config/samples/push.yml b/config/samples/push.yml index 5765a87..862a8f5 100644 --- a/config/samples/push.yml +++ b/config/samples/push.yml @@ -1,14 +1,13 @@ jobs: - type: push name: "push" - replication: - connect: - type: tcp - address: "backup-server.foo.bar:8888" - filesystems: { - "<": true, - "tmp": false - } + filesystems: { + "<": true, + "tmp": false + } + connect: + type: tcp + address: "backup-server.foo.bar:8888" snapshotting: snapshot_prefix: zrepl_ interval: 10m diff --git a/config/samples/sink.yml b/config/samples/sink.yml index bd3769d..7fbf716 100644 --- a/config/samples/sink.yml +++ b/config/samples/sink.yml @@ -1,15 +1,14 @@ jobs: - type: sink name: "laptop_sink" - replication: - root_dataset: "pool2/backup_laptops" - serve: - type: tls - listen: "192.168.122.189:8888" - ca: "ca.pem" - cert: "cert.pem" - key: "key.pem" - client_cn: "laptop1" + root_dataset: "pool2/backup_laptops" + serve: + type: tls + listen: "192.168.122.189:8888" + ca: "ca.pem" + cert: "cert.pem" + key: "key.pem" + client_cn: "laptop1" global: logging: - type: "tcp" diff --git a/config/samples/source.yml b/config/samples/source.yml index 79a4847..128cfa1 100644 --- a/config/samples/source.yml +++ b/config/samples/source.yml @@ -1,17 +1,16 @@ jobs: - name: pull_source type: source - replication: - serve: - type: tcp - listen: "0.0.0.0:8888" - clients: { - "192.168.122.123" : "client1" - } - filesystems: { - "<": true, - "secret": false + serve: + type: tcp + listen: "0.0.0.0:8888" + clients: { + "192.168.122.123" : "client1" } + filesystems: { + "<": true, + "secret": false + } snapshotting: snapshot_prefix: zrepl_ interval: 10m diff --git a/config/samples/source_ssh.yml b/config/samples/source_ssh.yml index 2b2ac9c..c707f83 100644 --- a/config/samples/source_ssh.yml +++ b/config/samples/source_ssh.yml @@ -1,14 +1,13 @@ jobs: - name: pull_source type: source - replication: - serve: - type: stdinserver - client_identity: "client1" - filesystems: { - "<": true, - "secret": false - } + serve: + type: stdinserver + client_identity: "client1" + filesystems: { + "<": true, + "secret": false + } snapshotting: snapshot_prefix: zrepl_ interval: 10m diff --git a/daemon/job/push.go b/daemon/job/push.go index d5406b1..fa7fd45 100644 --- a/daemon/job/push.go +++ b/daemon/job/push.go @@ -29,12 +29,12 @@ func PushFromConfig(g *config.Global, in *config.PushJob) (j *Push, err error) { j = &Push{} j.name = in.Name - j.clientFactory, err = connecter.FromConfig(g, in.Replication.Connect) + j.clientFactory, err = connecter.FromConfig(g, in.Connect) if err != nil { return nil, errors.Wrap(err, "cannot build client") } - if j.fsfilter, err = filters.DatasetMapFilterFromConfig(in.Replication.Filesystems); err != nil { + if j.fsfilter, err = filters.DatasetMapFilterFromConfig(in.Filesystems); err != nil { return nil, errors.Wrap(err, "cannnot build filesystem filter") } diff --git a/daemon/job/sink.go b/daemon/job/sink.go index 6afbb5d..7622167 100644 --- a/daemon/job/sink.go +++ b/daemon/job/sink.go @@ -25,12 +25,12 @@ func SinkFromConfig(g *config.Global, in *config.SinkJob) (s *Sink, err error) { // FIXME multi client support s = &Sink{name: in.Name} - if s.l, s.rpcConf, err = serve.FromConfig(g, in.Replication.Serve); err != nil { + if s.l, s.rpcConf, err = serve.FromConfig(g, in.Serve); err != nil { return nil, errors.Wrap(err, "cannot build server") } fsmap := filters.NewDatasetMapFilter(1, false) // FIXME multi-client support - if err := fsmap.Add("<", in.Replication.RootDataset); err != nil { + if err := fsmap.Add("<", in.RootDataset); err != nil { return nil, errors.Wrap(err, "unexpected error: cannot build filesystem mapping") } s.fsmap = fsmap From e161347e47279ad1e40564d9d8bbc979c92175b9 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 4 Sep 2018 14:46:02 -0700 Subject: [PATCH 097/167] Implement periodic snapshotting. --- daemon/filters/fsvfilter.go | 28 ++- daemon/job/push.go | 27 ++- daemon/logging/build_logging.go | 2 + daemon/snapper/snapper.go | 358 +++++++++++++++++++++++++++++ daemon/snapper/snapstate_string.go | 29 +++ daemon/snapper/state_string.go | 32 +++ 6 files changed, 471 insertions(+), 5 deletions(-) create mode 100644 daemon/snapper/snapper.go create mode 100644 daemon/snapper/snapstate_string.go create mode 100644 daemon/snapper/state_string.go diff --git a/daemon/filters/fsvfilter.go b/daemon/filters/fsvfilter.go index 0ec6225..3abcca8 100644 --- a/daemon/filters/fsvfilter.go +++ b/daemon/filters/fsvfilter.go @@ -1,6 +1,9 @@ package filters -import "github.com/zrepl/zrepl/zfs" +import ( + "github.com/zrepl/zrepl/zfs" + "strings" +) type AnyFSVFilter struct{} @@ -13,3 +16,26 @@ var _ zfs.FilesystemVersionFilter = AnyFSVFilter{} func (AnyFSVFilter) Filter(t zfs.VersionType, name string) (accept bool, err error) { return true, nil } + + +type PrefixFilter struct { + prefix string + fstype zfs.VersionType + fstypeSet bool // optionals anyone? +} + +var _ zfs.FilesystemVersionFilter = &PrefixFilter{} + +func NewPrefixFilter(prefix string) *PrefixFilter { + return &PrefixFilter{prefix: prefix} +} + +func NewTypedPrefixFilter(prefix string, versionType zfs.VersionType) *PrefixFilter { + return &PrefixFilter{prefix, versionType, true} +} + +func (f *PrefixFilter) Filter(t zfs.VersionType, name string) (accept bool, err error) { + fstypeMatches := (!f.fstypeSet || t == f.fstype) + prefixMatches := strings.HasPrefix(name, f.prefix) + return fstypeMatches && prefixMatches, nil +} diff --git a/daemon/job/push.go b/daemon/job/push.go index fa7fd45..ee9d00d 100644 --- a/daemon/job/push.go +++ b/daemon/job/push.go @@ -11,6 +11,7 @@ import ( "github.com/zrepl/zrepl/replication" "sync" "github.com/zrepl/zrepl/daemon/logging" + "github.com/zrepl/zrepl/daemon/snapper" ) type Push struct { @@ -20,6 +21,8 @@ type Push struct { prunerFactory *pruner.PrunerFactory + snapper *snapper.Snapper + mtx sync.Mutex replication *replication.Replication } @@ -34,15 +37,21 @@ func PushFromConfig(g *config.Global, in *config.PushJob) (j *Push, err error) { return nil, errors.Wrap(err, "cannot build client") } - if j.fsfilter, err = filters.DatasetMapFilterFromConfig(in.Filesystems); err != nil { + fsf, err := filters.DatasetMapFilterFromConfig(in.Filesystems) + if err != nil { return nil, errors.Wrap(err, "cannnot build filesystem filter") } + j.fsfilter = fsf j.prunerFactory, err = pruner.NewPrunerFactory(in.Pruning) if err != nil { return nil, err } + if j.snapper, err = snapper.FromConfig(g, fsf, &in.Snapshotting); err != nil { + return nil, errors.Wrap(err, "cannot build snapper") + } + return j, nil } @@ -68,6 +77,14 @@ func (j *Push) Run(ctx context.Context) { defer log.Info("job exiting") + snapshotsTaken := make(chan struct{}) + { + ctx, cancel := context.WithCancel(ctx) + defer cancel() + ctx = logging.WithSubsystemLoggers(ctx, log) + go j.snapper.Run(ctx, snapshotsTaken) + } + invocationCount := 0 outer: for { @@ -76,11 +93,13 @@ outer: case <-ctx.Done(): log.WithError(ctx.Err()).Info("context") break outer + case <-WaitWakeup(ctx): - invocationCount++ - invLog := log.WithField("invocation", invocationCount) - j.do(WithLogger(ctx, invLog)) + case <-snapshotsTaken: } + invocationCount++ + invLog := log.WithField("invocation", invocationCount) + j.do(WithLogger(ctx, invLog)) } } diff --git a/daemon/logging/build_logging.go b/daemon/logging/build_logging.go index fbaf9c2..56099bb 100644 --- a/daemon/logging/build_logging.go +++ b/daemon/logging/build_logging.go @@ -14,6 +14,7 @@ import ( "github.com/zrepl/zrepl/replication" "github.com/zrepl/zrepl/tlsconf" "os" + "github.com/zrepl/zrepl/daemon/snapper" ) func OutletsFromConfig(in config.LoggingOutletEnumList) (*logger.Outlets, error) { @@ -69,6 +70,7 @@ func WithSubsystemLoggers(ctx context.Context, log logger.Logger) context.Contex ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{log.WithField(SubsysField, "rpc")}) ctx = endpoint.WithLogger(ctx, log.WithField(SubsysField, "endpoint")) ctx = pruner.WithLogger(ctx, log.WithField(SubsysField, "pruning")) + ctx = snapper.WithLogger(ctx, log.WithField(SubsysField, "snapshot")) return ctx } diff --git a/daemon/snapper/snapper.go b/daemon/snapper/snapper.go new file mode 100644 index 0000000..2071389 --- /dev/null +++ b/daemon/snapper/snapper.go @@ -0,0 +1,358 @@ +package snapper + +import ( + "github.com/zrepl/zrepl/config" + "github.com/pkg/errors" + "time" + "context" + "github.com/zrepl/zrepl/daemon/filters" + "fmt" + "github.com/zrepl/zrepl/zfs" + "sort" + "github.com/zrepl/zrepl/logger" + "sync" +) + + +//go:generate stringer -type=SnapState +type SnapState uint + +const ( + SnapPending SnapState = 1 << iota + SnapStarted + SnapDone + SnapError +) + +type snapProgress struct { + state SnapState + + // SnapStarted, SnapDone, SnapError + name string + startAt time.Time + + // SnapDone + doneAt time.Time + + // SnapErr + err error +} + +type args struct { + ctx context.Context + log Logger + prefix string + interval time.Duration + fsf *filters.DatasetMapFilter + snapshotsTaken chan<-struct{} +} + +type Snapper struct { + args args + + mtx sync.Mutex + state State + + // set in state Plan, used in Waiting + lastInvocation time.Time + + // valid for state Snapshotting + plan map[*zfs.DatasetPath]snapProgress + + // valid for state SyncUp and Waiting + sleepUntil time.Time + + // valid for state Err + err error +} + +//go:generate stringer -type=State +type State uint + +const ( + SyncUp State = 1<%s", pre, post)). + Debug("state transition") + + } + +} + +func onErr(err error, u updater) state { + return u(func(s *Snapper) { + s.err = err + s.state = Error + }).sf() +} + +func syncUp(a args, u updater) state { + fss, err := listFSes(a.fsf) + if err != nil { + return onErr(err, u) + } + syncPoint, err := findSyncPoint(a.log, fss, a.prefix, a.interval) + if err != nil { + return onErr(err, u) + } + u(func(s *Snapper){ + s.sleepUntil = syncPoint + }) + t := time.NewTimer(syncPoint.Sub(time.Now())) + defer t.Stop() + select { + case <-t.C: + return u(func(s *Snapper) { + s.state = Planning + }).sf() + case <-a.ctx.Done(): + return onErr(err, u) + } +} + +func plan(a args, u updater) state { + u(func(snapper *Snapper) { + snapper.lastInvocation = time.Now() + }) + fss, err := listFSes(a.fsf) + if err != nil { + return onErr(err, u) + } + + plan := make(map[*zfs.DatasetPath]snapProgress, len(fss)) + for _, fs := range fss { + plan[fs] = snapProgress{state: SnapPending} + } + return u(func(s *Snapper) { + s.state = Snapshotting + s.plan = plan + }).sf() +} + +func snapshot(a args, u updater) state { + + var plan map[*zfs.DatasetPath]snapProgress + u(func(snapper *Snapper) { + plan = snapper.plan + }) + + hadErr := false + // TODO channel programs -> allow a little jitter? + for fs, progress := range plan { + suffix := time.Now().In(time.UTC).Format("20060102_150405_000") + snapname := fmt.Sprintf("%s%s", a.prefix, suffix) + + l := a.log. + WithField("fs", fs.ToString()). + WithField("snap", snapname) + + u(func(snapper *Snapper) { + progress.name = snapname + progress.startAt = time.Now() + progress.state = SnapStarted + }) + + l.Debug("create snapshot") + err := zfs.ZFSSnapshot(fs, snapname, false) + if err != nil { + hadErr = true + l.WithError(err).Error("cannot create snapshot") + } + doneAt := time.Now() + + u(func(snapper *Snapper) { + progress.doneAt = doneAt + progress.state = SnapDone + if err != nil { + progress.state = SnapError + progress.err = err + } + }) + } + + select { + case a.snapshotsTaken <- struct{}{}: + default: + a.log.Warn("callback channel is full, discarding snapshot update event") + } + + return u(func(snapper *Snapper) { + if hadErr { + snapper.state = Error + snapper.err = errors.New("one or more snapshots could not be created") + } else { + snapper.state = Waiting + } + }).sf() +} + +func wait(a args, u updater) state { + var sleepUntil time.Time + u(func(snapper *Snapper) { + lastTick := snapper.lastInvocation + snapper.sleepUntil = lastTick.Add(a.interval) + sleepUntil = snapper.sleepUntil + }) + + t := time.NewTimer(sleepUntil.Sub(time.Now())) + defer t.Stop() + + select { + case <-t.C: + return u(func(snapper *Snapper) { + snapper.state = Planning + }).sf() + case <-a.ctx.Done(): + return onErr(a.ctx.Err(), u) + } +} + +func listFSes(mf *filters.DatasetMapFilter) (fss []*zfs.DatasetPath, err error) { + return zfs.ZFSListMapping(mf) +} + +func findSyncPoint(log Logger, fss []*zfs.DatasetPath, prefix string, interval time.Duration) (syncPoint time.Time, err error) { + type snapTime struct { + ds *zfs.DatasetPath + time time.Time + } + + if len(fss) == 0 { + return time.Now(), nil + } + + snaptimes := make([]snapTime, 0, len(fss)) + + now := time.Now() + + log.Debug("examine filesystem state") + for _, d := range fss { + + l := log.WithField("fs", d.ToString()) + + fsvs, err := zfs.ZFSListFilesystemVersions(d, filters.NewTypedPrefixFilter(prefix, zfs.Snapshot)) + if err != nil { + l.WithError(err).Error("cannot list filesystem versions") + continue + } + if len(fsvs) <= 0 { + l.WithField("prefix", prefix).Debug("no filesystem versions with prefix") + continue + } + + // Sort versions by creation + sort.SliceStable(fsvs, func(i, j int) bool { + return fsvs[i].CreateTXG < fsvs[j].CreateTXG + }) + + latest := fsvs[len(fsvs)-1] + l.WithField("creation", latest.Creation). + Debug("found latest snapshot") + + since := now.Sub(latest.Creation) + if since < 0 { + l.WithField("snapshot", latest.Name). + WithField("creation", latest.Creation). + Error("snapshot is from the future") + continue + } + next := now + if since < interval { + next = latest.Creation.Add(interval) + } + snaptimes = append(snaptimes, snapTime{d, next}) + } + + if len(snaptimes) == 0 { + snaptimes = append(snaptimes, snapTime{nil, now}) + } + + sort.Slice(snaptimes, func(i, j int) bool { + return snaptimes[i].time.Before(snaptimes[j].time) + }) + + return snaptimes[0].time, nil + +} + diff --git a/daemon/snapper/snapstate_string.go b/daemon/snapper/snapstate_string.go new file mode 100644 index 0000000..faa7347 --- /dev/null +++ b/daemon/snapper/snapstate_string.go @@ -0,0 +1,29 @@ +// Code generated by "stringer -type=SnapState"; DO NOT EDIT. + +package snapper + +import "strconv" + +const ( + _SnapState_name_0 = "SnapPendingSnapStarted" + _SnapState_name_1 = "SnapDone" + _SnapState_name_2 = "SnapError" +) + +var ( + _SnapState_index_0 = [...]uint8{0, 11, 22} +) + +func (i SnapState) String() string { + switch { + case 1 <= i && i <= 2: + i -= 1 + return _SnapState_name_0[_SnapState_index_0[i]:_SnapState_index_0[i+1]] + case i == 4: + return _SnapState_name_1 + case i == 8: + return _SnapState_name_2 + default: + return "SnapState(" + strconv.FormatInt(int64(i), 10) + ")" + } +} diff --git a/daemon/snapper/state_string.go b/daemon/snapper/state_string.go new file mode 100644 index 0000000..485e7d9 --- /dev/null +++ b/daemon/snapper/state_string.go @@ -0,0 +1,32 @@ +// Code generated by "stringer -type=State"; DO NOT EDIT. + +package snapper + +import "strconv" + +const ( + _State_name_0 = "SyncUpPlanning" + _State_name_1 = "Snapshotting" + _State_name_2 = "Waiting" + _State_name_3 = "Error" +) + +var ( + _State_index_0 = [...]uint8{0, 6, 14} +) + +func (i State) String() string { + switch { + case 1 <= i && i <= 2: + i -= 1 + return _State_name_0[_State_index_0[i]:_State_index_0[i+1]] + case i == 4: + return _State_name_1 + case i == 8: + return _State_name_2 + case i == 16: + return _State_name_3 + default: + return "State(" + strconv.FormatInt(int64(i), 10) + ")" + } +} From 308e5e35fb05ef4652f92009e200f8bac7b55eb3 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 4 Sep 2018 16:41:54 -0700 Subject: [PATCH 098/167] Multi-client servers + bring back stdinserver support --- client/stdinserver.go | 41 ++++++++++ config/config.go | 6 +- config/samples/sink.yml | 4 +- config/samples/source_ssh.yml | 4 +- daemon/job/sink.go | 47 +++++++----- daemon/logging/build_logging.go | 2 + daemon/serve/serve.go | 63 +++++++++++++++- daemon/serve/serve_stdinserver.go | 119 +++++++++++++++++++++++++----- daemon/serve/serve_tcp.go | 78 +++++++++++++++++++- daemon/serve/serve_tls.go | 28 +++++-- main.go | 13 ++++ tlsconf/tlsconf.go | 22 ++---- 12 files changed, 356 insertions(+), 71 deletions(-) create mode 100644 client/stdinserver.go diff --git a/client/stdinserver.go b/client/stdinserver.go new file mode 100644 index 0000000..9d47a25 --- /dev/null +++ b/client/stdinserver.go @@ -0,0 +1,41 @@ +package client + +import ( + "os" + + "context" + "github.com/problame/go-netssh" + "log" + "path" + "github.com/zrepl/zrepl/config" + "errors" +) + + +func RunStdinserver(config *config.Config, args []string) error { + + // NOTE: the netssh proxying protocol requires exiting with non-zero status if anything goes wrong + defer os.Exit(1) + + log := log.New(os.Stderr, "", log.LUTC|log.Ldate|log.Ltime) + + if len(args) != 1 || args[0] == "" { + err := errors.New("must specify client_identity as positional argument") + return err + } + + identity := args[0] + unixaddr := path.Join(config.Global.Serve.StdinServer.SockDir, identity) + + log.Printf("proxying client identity '%s' to zrepl daemon '%s'", identity, unixaddr) + + ctx := netssh.ContextWithLog(context.TODO(), log) + + err := netssh.Proxy(ctx, unixaddr) + if err == nil { + log.Print("proxying finished successfully, exiting with status 0") + os.Exit(0) + } + log.Printf("error proxying: %s", err) + return nil +} diff --git a/config/config.go b/config/config.go index ab6f339..03fb662 100644 --- a/config/config.go +++ b/config/config.go @@ -165,7 +165,7 @@ type SSHStdinserverConnect struct { IdentityFile string `yaml:"identity_file"` TransportOpenCommand []string `yaml:"transport_open_command,optional"` //TODO unused SSHCommand string `yaml:"ssh_command,optional"` //TODO unused - Options []string `yaml:"options"` + Options []string `yaml:"options,optional"` DialTimeout time.Duration `yaml:"dial_timeout,positive,default=10s"` } @@ -190,13 +190,13 @@ type TLSServe struct { Ca string `yaml:"ca"` Cert string `yaml:"cert"` Key string `yaml:"key"` - ClientCN string `yaml:"client_cn"` + ClientCNs []string `yaml:"client_cns"` HandshakeTimeout time.Duration `yaml:"handshake_timeout,positive,default=10s"` } type StdinserverServer struct { ServeCommon `yaml:",inline"` - ClientIdentity string `yaml:"client_identity"` + ClientIdentities []string `yaml:"client_identities"` } type PruningEnum struct { diff --git a/config/samples/sink.yml b/config/samples/sink.yml index 7fbf716..b927041 100644 --- a/config/samples/sink.yml +++ b/config/samples/sink.yml @@ -8,7 +8,9 @@ jobs: ca: "ca.pem" cert: "cert.pem" key: "key.pem" - client_cn: "laptop1" + client_cns: + - "laptop1" + - "homeserver" global: logging: - type: "tcp" diff --git a/config/samples/source_ssh.yml b/config/samples/source_ssh.yml index c707f83..b1c034d 100644 --- a/config/samples/source_ssh.yml +++ b/config/samples/source_ssh.yml @@ -3,7 +3,9 @@ jobs: type: source serve: type: stdinserver - client_identity: "client1" + client_identities: + - "client1" + - "client2" filesystems: { "<": true, "secret": false diff --git a/daemon/job/sink.go b/daemon/job/sink.go index 7622167..e69a7fb 100644 --- a/daemon/job/sink.go +++ b/daemon/job/sink.go @@ -9,31 +9,27 @@ import ( "github.com/zrepl/zrepl/daemon/logging" "github.com/zrepl/zrepl/daemon/serve" "github.com/zrepl/zrepl/endpoint" - "net" + "path" ) type Sink struct { name string l serve.ListenerFactory rpcConf *streamrpc.ConnConfig - fsmap endpoint.FSMap - fsmapInv endpoint.FSFilter + rootDataset string } func SinkFromConfig(g *config.Global, in *config.SinkJob) (s *Sink, err error) { - // FIXME multi client support - s = &Sink{name: in.Name} if s.l, s.rpcConf, err = serve.FromConfig(g, in.Serve); err != nil { return nil, errors.Wrap(err, "cannot build server") } - fsmap := filters.NewDatasetMapFilter(1, false) // FIXME multi-client support - if err := fsmap.Add("<", in.RootDataset); err != nil { - return nil, errors.Wrap(err, "unexpected error: cannot build filesystem mapping") + if in.RootDataset == "" { + return nil, errors.Wrap(err, "must specify root dataset") } - s.fsmap = fsmap + s.rootDataset = in.RootDataset return s, nil } @@ -55,6 +51,7 @@ func (j *Sink) Run(ctx context.Context) { log.WithError(err).Error("cannot listen") return } + defer l.Close() log.WithField("addr", l.Addr()).Debug("accepting connections") @@ -64,10 +61,10 @@ outer: for { select { - case res := <-accept(l): + case res := <-accept(ctx, l): if res.err != nil { - log.WithError(err).Info("accept error") - break outer + log.WithError(res.err).Info("accept error") + continue } connId++ connLog := log. @@ -82,14 +79,28 @@ outer: } -func (j *Sink) handleConnection(ctx context.Context, conn net.Conn) { +func (j *Sink) handleConnection(ctx context.Context, conn serve.AuthenticatedConn) { + defer conn.Close() + log := GetLogger(ctx) - log.WithField("addr", conn.RemoteAddr()).Info("handling connection") + log. + WithField("addr", conn.RemoteAddr()). + WithField("client_identity", conn.ClientIdentity()). + Info("handling connection") defer log.Info("finished handling connection") + clientRoot := path.Join(j.rootDataset, conn.ClientIdentity()) + log.WithField("client_root", clientRoot).Debug("client root") + fsmap := filters.NewDatasetMapFilter(1, false) + if err := fsmap.Add("<", clientRoot); err != nil { + log.WithError(err). + WithField("client_identity", conn.ClientIdentity()). + Error("cannot build client filesystem map (client identity must be a valid ZFS FS name") + } + ctx = logging.WithSubsystemLoggers(ctx, log) - local, err := endpoint.NewReceiver(j.fsmap, filters.NewAnyFSVFilter()) + local, err := endpoint.NewReceiver(fsmap, filters.NewAnyFSVFilter()) if err != nil { log.WithError(err).Error("unexpected error: cannot convert mapping to filter") return @@ -102,14 +113,14 @@ func (j *Sink) handleConnection(ctx context.Context, conn net.Conn) { } type acceptResult struct { - conn net.Conn + conn serve.AuthenticatedConn err error } -func accept(listener net.Listener) <-chan acceptResult { +func accept(ctx context.Context, listener serve.AuthenticatedListener) <-chan acceptResult { c := make(chan acceptResult, 1) go func() { - conn, err := listener.Accept() + conn, err := listener.Accept(ctx) c <- acceptResult{conn, err} }() return c diff --git a/daemon/logging/build_logging.go b/daemon/logging/build_logging.go index 56099bb..754fa2f 100644 --- a/daemon/logging/build_logging.go +++ b/daemon/logging/build_logging.go @@ -15,6 +15,7 @@ import ( "github.com/zrepl/zrepl/tlsconf" "os" "github.com/zrepl/zrepl/daemon/snapper" + "github.com/zrepl/zrepl/daemon/serve" ) func OutletsFromConfig(in config.LoggingOutletEnumList) (*logger.Outlets, error) { @@ -71,6 +72,7 @@ func WithSubsystemLoggers(ctx context.Context, log logger.Logger) context.Contex ctx = endpoint.WithLogger(ctx, log.WithField(SubsysField, "endpoint")) ctx = pruner.WithLogger(ctx, log.WithField(SubsysField, "pruning")) ctx = snapper.WithLogger(ctx, log.WithField(SubsysField, "snapshot")) + ctx = serve.WithLogger(ctx, log.WithField(SubsysField, "serve")) return ctx } diff --git a/daemon/serve/serve.go b/daemon/serve/serve.go index fa7bb4a..8000a94 100644 --- a/daemon/serve/serve.go +++ b/daemon/serve/serve.go @@ -6,10 +6,69 @@ import ( "net" "github.com/zrepl/zrepl/daemon/streamrpcconfig" "github.com/problame/go-streamrpc" + "context" + "github.com/zrepl/zrepl/logger" + "github.com/zrepl/zrepl/zfs" ) +type contextKey int + +const contextKeyLog contextKey = 0 + +type Logger = logger.Logger + +func WithLogger(ctx context.Context, log Logger) context.Context { + return context.WithValue(ctx, contextKeyLog, log) +} + +func getLogger(ctx context.Context) Logger { + if log, ok := ctx.Value(contextKeyLog).(Logger); ok { + return log + } + return logger.NewNullLogger() +} + +type AuthenticatedConn interface { + net.Conn + // ClientIdentity must be a string that satisfies ValidateClientIdentity + ClientIdentity() string +} + +// A client identity must be a single component in a ZFS filesystem path +func ValidateClientIdentity(in string) (err error) { + path, err := zfs.NewDatasetPath(in) + if err != nil { + return err + } + if path.Length() != 1 { + return errors.New("client identity must be a single path comonent (not empty, no '/')") + } + return nil +} + +type authConn struct { + net.Conn + clientIdentity string +} + +var _ AuthenticatedConn = authConn{} + +func (c authConn) ClientIdentity() string { + if err := ValidateClientIdentity(c.clientIdentity); err != nil { + panic(err) + } + return c.clientIdentity +} + +// like net.Listener, but with an AuthenticatedConn instead of net.Conn +type AuthenticatedListener interface { + Addr() (net.Addr) + Accept(ctx context.Context) (AuthenticatedConn, error) + Close() error +} + type ListenerFactory interface { - Listen() (net.Listener, error) + Listen() (AuthenticatedListener, error) } func FromConfig(g *config.Global, in config.ServeEnum) (lf ListenerFactory, conf *streamrpc.ConnConfig, _ error) { @@ -25,7 +84,7 @@ func FromConfig(g *config.Global, in config.ServeEnum) (lf ListenerFactory, conf lf, lfError = TLSListenerFactoryFromConfig(g, v) conf, rpcErr = streamrpcconfig.FromDaemonConfig(g, v.RPC) case *config.StdinserverServer: - lf, lfError = StdinserverListenerFactoryFromConfig(g, v) + lf, lfError = MultiStdinserverListenerFactoryFromConfig(g, v) conf, rpcErr = streamrpcconfig.FromDaemonConfig(g, v.RPC) default: return nil, nil, errors.Errorf("internal error: unknown serve type %T", v) diff --git a/daemon/serve/serve_stdinserver.go b/daemon/serve/serve_stdinserver.go index f6403d3..baa8c88 100644 --- a/daemon/serve/serve_stdinserver.go +++ b/daemon/serve/serve_stdinserver.go @@ -8,54 +8,133 @@ import ( "net" "path" "time" + "context" + "github.com/pkg/errors" + "sync/atomic" + "fmt" + "os" ) type StdinserverListenerFactory struct { - ClientIdentity string - sockpath string + ClientIdentities []string + Sockdir string } -func StdinserverListenerFactoryFromConfig(g *config.Global, in *config.StdinserverServer) (f *StdinserverListenerFactory, err error) { +func MultiStdinserverListenerFactoryFromConfig(g *config.Global, in *config.StdinserverServer) (f *multiStdinserverListenerFactory, err error) { - f = &StdinserverListenerFactory{ - ClientIdentity: in.ClientIdentity, + for _, ci := range in.ClientIdentities { + if err := ValidateClientIdentity(ci); err != nil { + return nil, errors.Wrapf(err, "invalid client identity %q", ci) + } } - f.sockpath = path.Join(g.Serve.StdinServer.SockDir, f.ClientIdentity) + f = &multiStdinserverListenerFactory{ + ClientIdentities: in.ClientIdentities, + Sockdir: g.Serve.StdinServer.SockDir, + } return } -func (f *StdinserverListenerFactory) Listen() (net.Listener, error) { +type multiStdinserverListenerFactory struct { + ClientIdentities []string + Sockdir string +} - if err := nethelpers.PreparePrivateSockpath(f.sockpath); err != nil { - return nil, err +func (f *multiStdinserverListenerFactory) Listen() (AuthenticatedListener, error) { + return multiStdinserverListenerFromClientIdentities(f.Sockdir, f.ClientIdentities) +} + +type multiStdinserverAcceptRes struct { + conn AuthenticatedConn + err error +} + +type MultiStdinserverListener struct { + listeners []*stdinserverListener + accepts chan multiStdinserverAcceptRes + closed int32 +} + +// client identities must be validated +func multiStdinserverListenerFromClientIdentities(sockdir string, cis []string) (*MultiStdinserverListener, error) { + listeners := make([]*stdinserverListener, 0, len(cis)) + var err error + for _, ci := range cis { + sockpath := path.Join(sockdir, ci) + l := &stdinserverListener{clientIdentity: ci} + if err = nethelpers.PreparePrivateSockpath(sockpath); err != nil { + break + } + if l.l, err = netssh.Listen(sockpath); err != nil { + break + } + listeners = append(listeners, l) } - - l, err := netssh.Listen(f.sockpath) if err != nil { + for _, l := range listeners { + l.Close() // FIXME error reporting? + } return nil, err } - return StdinserverListener{l}, nil + return &MultiStdinserverListener{listeners: listeners}, nil } -type StdinserverListener struct { - l *netssh.Listener +func (m *MultiStdinserverListener) Accept(ctx context.Context) (AuthenticatedConn, error){ + + if m.accepts == nil { + m.accepts = make(chan multiStdinserverAcceptRes, len(m.listeners)) + for i := range m.listeners { + go func(i int) { + for atomic.LoadInt32(&m.closed) == 0 { + fmt.Fprintf(os.Stderr, "accepting\n") + conn, err := m.listeners[i].Accept(context.TODO()) + fmt.Fprintf(os.Stderr, "incoming\n") + m.accepts <- multiStdinserverAcceptRes{conn, err} + } + }(i) + } + } + + res := <- m.accepts + return res.conn, res.err + } -func (l StdinserverListener) Addr() net.Addr { +func (m *MultiStdinserverListener) Addr() (net.Addr) { return netsshAddr{} } -func (l StdinserverListener) Accept() (net.Conn, error) { +func (m *MultiStdinserverListener) Close() error { + atomic.StoreInt32(&m.closed, 1) + var oneErr error + for _, l := range m.listeners { + if err := l.Close(); err != nil && oneErr == nil { + oneErr = err + } + } + return oneErr +} + +// a single stdinserverListener (part of multiStinserverListener) +type stdinserverListener struct { + l *netssh.Listener + clientIdentity string +} + +func (l stdinserverListener) Addr() net.Addr { + return netsshAddr{} +} + +func (l stdinserverListener) Accept(ctx context.Context) (AuthenticatedConn, error) { c, err := l.l.Accept() if err != nil { return nil, err } - return netsshConnToNetConnAdatper{c}, nil + return netsshConnToNetConnAdatper{c, l.clientIdentity}, nil } -func (l StdinserverListener) Close() (err error) { +func (l stdinserverListener) Close() (err error) { return l.l.Close() } @@ -66,12 +145,16 @@ func (netsshAddr) String() string { return "???" } type netsshConnToNetConnAdatper struct { io.ReadWriteCloser // works for both netssh.SSHConn and netssh.ServeConn + clientIdentity string } +func (a netsshConnToNetConnAdatper) ClientIdentity() string { return a.clientIdentity } + func (netsshConnToNetConnAdatper) LocalAddr() net.Addr { return netsshAddr{} } func (netsshConnToNetConnAdatper) RemoteAddr() net.Addr { return netsshAddr{} } +// FIXME log warning once! func (netsshConnToNetConnAdatper) SetDeadline(t time.Time) error { return nil } func (netsshConnToNetConnAdatper) SetReadDeadline(t time.Time) error { return nil } diff --git a/daemon/serve/serve_tcp.go b/daemon/serve/serve_tcp.go index 21cab59..957d3b9 100644 --- a/daemon/serve/serve_tcp.go +++ b/daemon/serve/serve_tcp.go @@ -3,19 +3,89 @@ package serve import ( "github.com/zrepl/zrepl/config" "net" + "github.com/pkg/errors" + "context" ) type TCPListenerFactory struct { - Address string + address *net.TCPAddr + clientMap *ipMap +} + +type ipMapEntry struct { + ip net.IP + ident string +} + +type ipMap struct { + entries []ipMapEntry +} + +func ipMapFromConfig(clients map[string]string) (*ipMap, error) { + entries := make([]ipMapEntry, 0, len(clients)) + for clientIPString, clientIdent := range clients { + clientIP := net.ParseIP(clientIPString) + if clientIP == nil { + return nil, errors.Errorf("cannot parse client IP %q", clientIPString) + } + if err := ValidateClientIdentity(clientIdent); err != nil { + return nil, errors.Wrapf(err,"invalid client identity for IP %q", clientIPString) + } + entries = append(entries, ipMapEntry{clientIP, clientIdent}) + } + return &ipMap{entries: entries}, nil +} + +func (m *ipMap) Get(ip net.IP) (string, error) { + for _, e := range m.entries { + if e.ip.Equal(ip) { + return e.ident, nil + } + } + return "", errors.Errorf("no identity mapping for client IP %s", ip) } func TCPListenerFactoryFromConfig(c *config.Global, in *config.TCPServe) (*TCPListenerFactory, error) { + addr, err := net.ResolveTCPAddr("tcp", in.Listen) + if err != nil { + return nil, errors.Wrap(err, "cannot parse listen address") + } + clientMap, err := ipMapFromConfig(in.Clients) + if err != nil { + return nil, errors.Wrap(err, "cannot parse client IP map") + } lf := &TCPListenerFactory{ - Address: in.Listen, + address: addr, + clientMap: clientMap, } return lf, nil } -func (f *TCPListenerFactory) Listen() (net.Listener, error) { - return net.Listen("tcp", f.Address) +func (f *TCPListenerFactory) Listen() (AuthenticatedListener, error) { + l, err := net.ListenTCP("tcp", f.address) + if err != nil { + return nil, err + } + return &TCPAuthListener{l, f.clientMap}, nil } + +type TCPAuthListener struct { + *net.TCPListener + clientMap *ipMap +} + +func (f *TCPAuthListener) Accept(ctx context.Context) (AuthenticatedConn, error) { + nc, err := f.TCPListener.Accept() + if err != nil { + return nil, err + } + clientIP := nc.RemoteAddr().(*net.TCPAddr).IP + clientIdent, err := f.clientMap.Get(clientIP) + if err != nil { + getLogger(ctx).WithField("ip", clientIP).Error("client IP not in client map") + nc.Close() + return nil, err + } + return authConn{nc, clientIdent}, nil +} + diff --git a/daemon/serve/serve_tls.go b/daemon/serve/serve_tls.go index 0b80345..8f5e527 100644 --- a/daemon/serve/serve_tls.go +++ b/daemon/serve/serve_tls.go @@ -8,13 +8,13 @@ import ( "github.com/zrepl/zrepl/tlsconf" "net" "time" + "context" ) type TLSListenerFactory struct { address string clientCA *x509.CertPool serverCert tls.Certificate - clientCommonName string handshakeTimeout time.Duration } @@ -23,12 +23,10 @@ func TLSListenerFactoryFromConfig(c *config.Global, in *config.TLSServe) (lf *TL address: in.Listen, } - if in.Ca == "" || in.Cert == "" || in.Key == "" || in.ClientCN == "" { - return nil, errors.New("fields 'ca', 'cert', 'key' and 'client_cn' must be specified") + if in.Ca == "" || in.Cert == "" || in.Key == "" { + return nil, errors.New("fields 'ca', 'cert' and 'key'must be specified") } - lf.clientCommonName = in.ClientCN - lf.clientCA, err = tlsconf.ParseCAFile(in.Ca) if err != nil { return nil, errors.Wrap(err, "cannot parse ca file") @@ -42,11 +40,25 @@ func TLSListenerFactoryFromConfig(c *config.Global, in *config.TLSServe) (lf *TL return lf, nil } -func (f *TLSListenerFactory) Listen() (net.Listener, error) { +func (f *TLSListenerFactory) Listen() (AuthenticatedListener, error) { l, err := net.Listen("tcp", f.address) if err != nil { return nil, err } - tl := tlsconf.NewClientAuthListener(l, f.clientCA, f.serverCert, f.clientCommonName, f.handshakeTimeout) - return tl, nil + tl := tlsconf.NewClientAuthListener(l, f.clientCA, f.serverCert, f.handshakeTimeout) + return tlsAuthListener{tl}, nil } + +type tlsAuthListener struct { + *tlsconf.ClientAuthListener +} + +func (l tlsAuthListener) Accept(ctx context.Context) (AuthenticatedConn, error) { + c, cn, err := l.ClientAuthListener.Accept() + if err != nil { + return nil, err + } + return authConn{c, cn}, nil +} + + diff --git a/main.go b/main.go index 40f1f33..c520ce3 100644 --- a/main.go +++ b/main.go @@ -57,6 +57,18 @@ var statusCmd = &cobra.Command{ }, } +var stdinserverCmd = &cobra.Command{ + Use: "stdinserver CLIENT_IDENTITY", + Short: "start in stdinserver mode (from authorized_keys file)", + RunE: func(cmd *cobra.Command, args []string) error { + conf, err := config.ParseConfig(rootArgs.configFile) + if err != nil { + return err + } + return client.RunStdinserver(conf, args) + }, +} + var rootArgs struct { configFile string } @@ -67,6 +79,7 @@ func init() { rootCmd.AddCommand(daemonCmd) rootCmd.AddCommand(wakeupCmd) rootCmd.AddCommand(statusCmd) + rootCmd.AddCommand(stdinserverCmd) } func main() { diff --git a/tlsconf/tlsconf.go b/tlsconf/tlsconf.go index cf47968..48fc382 100644 --- a/tlsconf/tlsconf.go +++ b/tlsconf/tlsconf.go @@ -4,7 +4,6 @@ import ( "crypto/tls" "crypto/x509" "errors" - "fmt" "io/ioutil" "net" "time" @@ -24,13 +23,12 @@ func ParseCAFile(certfile string) (*x509.CertPool, error) { type ClientAuthListener struct { l net.Listener - clientCommonName string handshakeTimeout time.Duration } func NewClientAuthListener( l net.Listener, ca *x509.CertPool, serverCert tls.Certificate, - clientCommonName string, handshakeTimeout time.Duration) *ClientAuthListener { + handshakeTimeout time.Duration) *ClientAuthListener { if ca == nil { panic(ca) @@ -38,9 +36,6 @@ func NewClientAuthListener( if serverCert.Certificate == nil || serverCert.PrivateKey == nil { panic(serverCert) } - if clientCommonName == "" { - panic(clientCommonName) - } tlsConf := tls.Config{ Certificates: []tls.Certificate{serverCert}, @@ -51,19 +46,18 @@ func NewClientAuthListener( l = tls.NewListener(l, &tlsConf) return &ClientAuthListener{ l, - clientCommonName, handshakeTimeout, } } -func (l *ClientAuthListener) Accept() (c net.Conn, err error) { +func (l *ClientAuthListener) Accept() (c net.Conn, clientCN string, err error) { c, err = l.l.Accept() if err != nil { - return nil, err + return nil, "", err } tlsConn, ok := c.(*tls.Conn) if !ok { - return c, err + return c, "", err } var ( @@ -83,14 +77,10 @@ func (l *ClientAuthListener) Accept() (c net.Conn, err error) { goto CloseAndErr } cn = peerCerts[0].Subject.CommonName - if cn != l.clientCommonName { - err = fmt.Errorf("client cert common name does not match client_identity: %q != %q", cn, l.clientCommonName) - goto CloseAndErr - } - return c, nil + return c, cn, nil CloseAndErr: c.Close() - return nil, err + return nil, "", err } func (l *ClientAuthListener) Addr() net.Addr { From 8eade3d20a2b64e8083d585763ff56161970fb16 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 4 Sep 2018 17:01:46 -0700 Subject: [PATCH 099/167] replication/pdu: fix broken test --- replication/pdu/pdu_test.go | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/replication/pdu/pdu_test.go b/replication/pdu/pdu_test.go index 6a1d596..79315a6 100644 --- a/replication/pdu/pdu_test.go +++ b/replication/pdu/pdu_test.go @@ -3,6 +3,7 @@ package pdu import ( "github.com/stretchr/testify/assert" "testing" + "time" ) func TestFilesystemVersion_RelName(t *testing.T) { @@ -13,11 +14,13 @@ func TestFilesystemVersion_RelName(t *testing.T) { Panic bool } + creat := FilesystemVersionCreation(time.Now()) tcs := []TestCase{ { In: FilesystemVersion{ Type: FilesystemVersion_Snapshot, Name: "foobar", + Creation: creat, }, Out: "@foobar", }, @@ -25,6 +28,7 @@ func TestFilesystemVersion_RelName(t *testing.T) { In: FilesystemVersion{ Type: FilesystemVersion_Bookmark, Name: "foobar", + Creation: creat, }, Out: "#foobar", }, @@ -32,6 +36,7 @@ func TestFilesystemVersion_RelName(t *testing.T) { In: FilesystemVersion{ Type: 2342, Name: "foobar", + Creation: creat, }, Panic: true, }, @@ -53,12 +58,11 @@ func TestFilesystemVersion_RelName(t *testing.T) { func TestFilesystemVersion_ZFSFilesystemVersion(t *testing.T) { empty := &FilesystemVersion{} - emptyZFS := empty.ZFSFilesystemVersion() - assert.Zero(t, emptyZFS.Creation) + _, err:= empty.ZFSFilesystemVersion() + assert.Error(t, err) dateInvalid := &FilesystemVersion{Creation: "foobar"} - assert.Panics(t, func() { - dateInvalid.ZFSFilesystemVersion() - }) + _, err = dateInvalid.ZFSFilesystemVersion() + assert.Error(t, err) } From 1e27720b9981d8b0a13c957e310ab2c6392918ab Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 4 Sep 2018 17:02:02 -0700 Subject: [PATCH 100/167] zfs: skip test with ZFS_BINARY mock (doesn't work in parallel) --- zfs/zfs_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/zfs/zfs_test.go b/zfs/zfs_test.go index cf89b40..dcfe0f9 100644 --- a/zfs/zfs_test.go +++ b/zfs/zfs_test.go @@ -6,6 +6,8 @@ import ( ) func TestZFSListHandlesProducesZFSErrorOnNonZeroExit(t *testing.T) { + t.SkipNow() // FIXME ZFS_BINARY does not work if tests run in parallel + var err error ZFS_BINARY = "./test_helpers/zfs_failer.sh" From bfc631f6a6131bd7bac1f4607255768681d7c6d3 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 4 Sep 2018 17:19:39 -0700 Subject: [PATCH 101/167] fix broken pruner exercise (don't call it test...) --- daemon/pruner/pruner_test.go | 60 +++++++++++++++++++++--------------- 1 file changed, 35 insertions(+), 25 deletions(-) diff --git a/daemon/pruner/pruner_test.go b/daemon/pruner/pruner_test.go index ce8fb7c..dea3d5f 100644 --- a/daemon/pruner/pruner_test.go +++ b/daemon/pruner/pruner_test.go @@ -90,42 +90,45 @@ func (t *mockTarget) DestroySnapshots(ctx context.Context, req *pdu.DestroySnaps } type mockHistory struct { - fss []mockFS errs map[string][]error } -func (r *mockHistory) WasSnapshotReplicated(ctx context.Context, fs string, version *pdu.FilesystemVersion) (bool, error) { - +func (r *mockHistory) SnapshotReplicationStatus(ctx context.Context, req *pdu.SnapshotReplicationStatusReq) (*pdu.SnapshotReplicationStatusRes, error) { + fs := req.Filesystem if len(r.errs[fs]) > 0 { e := r.errs[fs][0] r.errs[fs] = r.errs[fs][1:] - return false, e + return nil, e } - - for _, mfs := range r.fss { - if mfs.path != fs { - continue - } - for _, v := range mfs.FilesystemVersions() { - if v.Type == version.Type && v.Name == v.Name && v.CreateTXG == version.CreateTXG { - return true, nil - } - } - } - return false, nil + return &pdu.SnapshotReplicationStatusRes{Status: pdu.SnapshotReplicationStatusRes_Nonexistent}, nil } +type stubNetErr struct { + msg string + temporary, timeout bool +} + +var _ net.Error = stubNetErr{} + +func (e stubNetErr) Error() string { + return e.msg +} + +func (e stubNetErr) Temporary() bool { return e.temporary } + +func (e stubNetErr) Timeout() bool { return e.timeout } + func TestPruner_Prune(t *testing.T) { var _ net.Error = &net.OpError{} // we use it below target := &mockTarget{ listFilesystemsErr: []error{ - &net.OpError{Op: "fakerror0"}, + stubNetErr{msg: "fakerror0"}, }, listVersionsErrs: map[string][]error{ "zroot/foo": { - &net.OpError{Op: "fakeerror1"}, // should be classified as temporaty - &net.OpError{Op: "fakeerror2"}, + stubNetErr{msg: "fakeerror1"}, // should be classified as temporaty + stubNetErr{msg: "fakeerror2"}, }, }, destroyErrs: map[string][]error{ @@ -133,7 +136,7 @@ func TestPruner_Prune(t *testing.T) { fmt.Errorf("permanent error"), }, "zroot/bar": { - &net.OpError{Op: "fakeerror3"}, + stubNetErr{msg: "fakeerror3"}, }, }, destroyed: make(map[string][]string), @@ -167,7 +170,7 @@ func TestPruner_Prune(t *testing.T) { history := &mockHistory{ errs: map[string][]error{ "zroot/foo": { - &net.OpError{Op: "fakeerror4"}, + stubNetErr{msg: "fakeerror4"}, }, "zroot/baz": { fmt.Errorf("permanent error2"), @@ -177,10 +180,17 @@ func TestPruner_Prune(t *testing.T) { keepRules := []pruning.KeepRule{pruning.MustKeepRegex("^keep")} - p := NewPruner(10*time.Millisecond, target, history, keepRules) - ctx := context.Background() - ctx = WithLogger(ctx, logger.NewTestLogger(t)) - p.Prune(ctx) + p := Pruner{ + args: args{ + ctx: WithLogger(context.Background(), logger.NewTestLogger(t)), + target: target, + receiver: history, + rules: keepRules, + retryWait: 10*time.Millisecond, + }, + state: Plan, + } + p.Prune() exp := map[string][]string{ "zroot/bar": {"drop_g"}, From 52f0c0c33bcd9fa62e338985bd9ad64a5adb4159 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 4 Sep 2018 17:02:16 -0700 Subject: [PATCH 102/167] update Makefile --- Makefile | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/Makefile b/Makefile index d9f4670..5b3091e 100644 --- a/Makefile +++ b/Makefile @@ -2,13 +2,30 @@ .DEFAULT_GOAL := build ROOT := github.com/zrepl/zrepl -SUBPKGS := cmd +SUBPKGS += client +SUBPKGS += config +SUBPKGS += daemon +SUBPKGS += daemon/connecter +SUBPKGS += daemon/filters +SUBPKGS += daemon/job +SUBPKGS += daemon/logging +SUBPKGS += daemon/nethelpers +SUBPKGS += daemon/pruner +SUBPKGS += daemon/serve +SUBPKGS += daemon/snapper +SUBPKGS += daemon/streamrpcconfig +SUBPKGS += endpoint +SUBPKGS += logger +SUBPKGS += pruning SUBPKGS += replication SUBPKGS += replication/fsrep SUBPKGS += replication/pdu SUBPKGS += replication/internal/queue SUBPKGS += replication/internal/diff -SUBPKGS += logger util zfs +SUBPKGS += tlsconf +SUBPKGS += util +SUBPKGS += version +SUBPKGS += zfs _TESTPKGS := $(ROOT) $(foreach p,$(SUBPKGS),$(ROOT)/$(p)) From adab06405b645a01849d1d3343392580185248ef Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 4 Sep 2018 17:25:10 -0700 Subject: [PATCH 103/167] make go vet happy --- config/config.go | 6 +++--- daemon/connecter/connecter.go | 2 +- daemon/pruner/pruner.go | 10 +++++----- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/config/config.go b/config/config.go index 03fb662..32b3b4d 100644 --- a/config/config.go +++ b/config/config.go @@ -230,7 +230,7 @@ type LoggingOutletCommon struct { type StdoutLoggingOutlet struct { LoggingOutletCommon `yaml:",inline"` Time bool `yaml:"time,default=true"` - Color bool `yaml:"color,default=true""` + Color bool `yaml:"color,default=true"` } type SyslogLoggingOutlet struct { @@ -289,12 +289,12 @@ func enumUnmarshal(u func(interface{}, bool) error, types map[string]interface{} return nil, err } if in.Type == "" { - return nil, &yaml.TypeError{[]string{"must specify type"}} + return nil, &yaml.TypeError{Errors: []string{"must specify type"}} } v, ok := types[in.Type] if !ok { - return nil, &yaml.TypeError{[]string{fmt.Sprintf("invalid type name %q", in.Type)}} + return nil, &yaml.TypeError{Errors: []string{fmt.Sprintf("invalid type name %q", in.Type)}} } if err := u(v, false); err != nil { return nil, err diff --git a/daemon/connecter/connecter.go b/daemon/connecter/connecter.go index c26d1de..4708cb3 100644 --- a/daemon/connecter/connecter.go +++ b/daemon/connecter/connecter.go @@ -34,7 +34,7 @@ func FromConfig(g *config.Global, in config.ConnectEnum) (*ClientFactory, error) return nil, errRPC } - config := streamrpc.ClientConfig{connConf} + config := streamrpc.ClientConfig{ConnConfig: connConf} if err := config.Validate(); err != nil { return nil, err } diff --git a/daemon/pruner/pruner.go b/daemon/pruner/pruner.go index 2b49c76..f8e3ab2 100644 --- a/daemon/pruner/pruner.go +++ b/daemon/pruner/pruner.go @@ -61,8 +61,8 @@ type Pruner struct { err error // State Exec - prunePending []fs - pruneCompleted []fs + prunePending []*fs + pruneCompleted []*fs } type PrunerFactory struct { @@ -253,14 +253,14 @@ func statePlan(a *args, u updater) state { return onErr(u, err) } - pfss := make([]fs, len(tfss)) + pfss := make([]*fs, len(tfss)) for i, tfs := range tfss { tfsvs, err := target.ListFilesystemVersions(ctx, tfs.Path) if err != nil { return onErr(u, err) } - pfs := fs{ + pfs := &fs{ path: tfs.Path, snaps: make([]pruning.Snapshot, 0, len(tfsvs)), } @@ -321,7 +321,7 @@ func statePlan(a *args, u updater) state { func stateExec(a *args, u updater) state { - var pfs fs + var pfs *fs state := u(func(pruner *Pruner) { if len(pruner.prunePending) == 0 { pruner.state = Done From 6c31c665625bb58ab0bc4977c7e310b021067ea2 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 4 Sep 2018 17:27:20 -0700 Subject: [PATCH 104/167] hidden bashcomp command --- main.go | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/main.go b/main.go index c520ce3..575ac86 100644 --- a/main.go +++ b/main.go @@ -8,6 +8,7 @@ import ( "github.com/zrepl/zrepl/daemon" "log" "os" + "fmt" ) var rootCmd = &cobra.Command{ @@ -69,6 +70,24 @@ var stdinserverCmd = &cobra.Command{ }, } + +var bashcompCmd = &cobra.Command{ + Use: "bashcomp path/to/out/file", + Short: "generate bash completions", + Run: func(cmd *cobra.Command, args []string) { + if len(args) != 1 { + fmt.Fprintf(os.Stderr, "specify exactly one positional agument\n") + cmd.Usage() + os.Exit(1) + } + if err := rootCmd.GenBashCompletionFile(args[0]); err != nil { + fmt.Fprintf(os.Stderr, "error generating bash completion: %s", err) + os.Exit(1) + } + }, + Hidden: true, +} + var rootArgs struct { configFile string } @@ -80,6 +99,7 @@ func init() { rootCmd.AddCommand(wakeupCmd) rootCmd.AddCommand(statusCmd) rootCmd.AddCommand(stdinserverCmd) + rootCmd.AddCommand(bashcompCmd) } func main() { From 4b39a18178c732ba9d6e2e0db289e0d815d16672 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 4 Sep 2018 17:31:46 -0700 Subject: [PATCH 105/167] zfs: disable resume token test because it doesn't work in docker --- zfs/resume_token_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/zfs/resume_token_test.go b/zfs/resume_token_test.go index 33f560a..4a35595 100644 --- a/zfs/resume_token_test.go +++ b/zfs/resume_token_test.go @@ -31,6 +31,8 @@ func (rtt *ResumeTokenTest) Test(t *testing.T) { func TestParseResumeToken(t *testing.T) { + t.SkipNow() // FIXME not compatible with docker + tbl := []ResumeTokenTest{ { Msg: "normal send (non-incremental)", From 6c988d0ebba8b92fe6017c40594307aeade9f3f5 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 5 Sep 2018 08:32:38 -0700 Subject: [PATCH 106/167] add small subcommand to validate config --- client/configcheck.go | 8 ++++++++ main.go | 13 +++++++++++++ 2 files changed, 21 insertions(+) create mode 100644 client/configcheck.go diff --git a/client/configcheck.go b/client/configcheck.go new file mode 100644 index 0000000..b50ac80 --- /dev/null +++ b/client/configcheck.go @@ -0,0 +1,8 @@ +package client + +import "github.com/zrepl/zrepl/config" + +func RunConfigcheck(conf *config.Config, args []string) error { + // TODO: do the 'build' steps, e.g. build the jobs and see if that fails + return nil +} diff --git a/main.go b/main.go index 575ac86..905e2f1 100644 --- a/main.go +++ b/main.go @@ -88,6 +88,18 @@ var bashcompCmd = &cobra.Command{ Hidden: true, } +var configcheckCmd = &cobra.Command{ + Use: "configcheck", + Short: "validate config file", + RunE: func(cmd *cobra.Command, args []string) error { + conf, err := config.ParseConfig(rootArgs.configFile) + if err != nil { + return err + } + return client.RunConfigcheck(conf, args) + }, +} + var rootArgs struct { configFile string } @@ -100,6 +112,7 @@ func init() { rootCmd.AddCommand(statusCmd) rootCmd.AddCommand(stdinserverCmd) rootCmd.AddCommand(bashcompCmd) + rootCmd.AddCommand(configcheckCmd) } func main() { From c21222ef1337b5c87fd124c8cf58a3d02e265f3f Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 5 Sep 2018 08:32:59 -0700 Subject: [PATCH 107/167] serve/tls: use handshake timeout from config --- daemon/serve/serve_tls.go | 1 + 1 file changed, 1 insertion(+) diff --git a/daemon/serve/serve_tls.go b/daemon/serve/serve_tls.go index 8f5e527..9172b32 100644 --- a/daemon/serve/serve_tls.go +++ b/daemon/serve/serve_tls.go @@ -21,6 +21,7 @@ type TLSListenerFactory struct { func TLSListenerFactoryFromConfig(c *config.Global, in *config.TLSServe) (lf *TLSListenerFactory, err error) { lf = &TLSListenerFactory{ address: in.Listen, + handshakeTimeout: in.HandshakeTimeout, } if in.Ca == "" || in.Cert == "" || in.Key == "" { From 9eca269ad8d4ea31064a9678521b4171e4664f59 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 5 Sep 2018 08:35:31 -0700 Subject: [PATCH 108/167] fixup 308e5e35fb05ef4652f92009e200f8bac7b55eb3: remove fprintf debug output --- daemon/serve/serve_stdinserver.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/daemon/serve/serve_stdinserver.go b/daemon/serve/serve_stdinserver.go index baa8c88..f02bf20 100644 --- a/daemon/serve/serve_stdinserver.go +++ b/daemon/serve/serve_stdinserver.go @@ -11,8 +11,6 @@ import ( "context" "github.com/pkg/errors" "sync/atomic" - "fmt" - "os" ) type StdinserverListenerFactory struct { @@ -87,9 +85,7 @@ func (m *MultiStdinserverListener) Accept(ctx context.Context) (AuthenticatedCon for i := range m.listeners { go func(i int) { for atomic.LoadInt32(&m.closed) == 0 { - fmt.Fprintf(os.Stderr, "accepting\n") conn, err := m.listeners[i].Accept(context.TODO()) - fmt.Fprintf(os.Stderr, "incoming\n") m.accepts <- multiStdinserverAcceptRes{conn, err} } }(i) From acd2418803ec42576786122c80428d80e4a5d520 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 5 Sep 2018 17:41:25 -0700 Subject: [PATCH 109/167] handle DryRun send size estimate errors with bookmarks --- client/status.go | 27 ++++++++++++++++----------- endpoint/endpoint.go | 3 +++ replication/fsrep/fsfsm.go | 5 +++-- replication/pdu/pdu.pb.go | 3 ++- replication/pdu/pdu.proto | 3 ++- zfs/zfs.go | 13 +++++++++++++ 6 files changed, 39 insertions(+), 15 deletions(-) diff --git a/client/status.go b/client/status.go index 7847c84..e19d474 100644 --- a/client/status.go +++ b/client/status.go @@ -235,23 +235,28 @@ func (t *tui) drawBar(name string, maxNameLength int, status string, bytes int64 t.write(rightPad(status, 14, " ")) t.write(" ") + length := 50 + var completedLength int if totalBytes > 0 { - length := 50 - completedLength := int(int64(length) * bytes / totalBytes) + completedLength = int(int64(length) * bytes / totalBytes) if completedLength > length { completedLength = length } - - t.write("[") - t.write(times("=", completedLength)) - t.write(">") - t.write(times("-", length-completedLength)) - t.write("]") - - t.write(" ") - t.write(rightPad(ByteCountBinary(bytes) + "/" + ByteCountBinary(totalBytes), 20, " ")) } + t.write("[") + t.write(times("=", completedLength)) + t.write(">") + t.write(times("-", length-completedLength)) + t.write("]") + + t.write(" ") + totalBytesStr := ByteCountBinary(totalBytes) + if totalBytes == 0 { + totalBytesStr = "??? B" + } + t.write(rightPad(ByteCountBinary(bytes)+"/"+totalBytesStr, 20, " ")) + t.newline() } diff --git a/endpoint/endpoint.go b/endpoint/endpoint.go index d1ee34f..c6c5b37 100644 --- a/endpoint/endpoint.go +++ b/endpoint/endpoint.go @@ -77,6 +77,9 @@ func (p *Sender) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.Rea if r.DryRun { size, err := zfs.ZFSSendDry(r.Filesystem, r.From, r.To) + if err == zfs.BookmarkSizeEstimationNotSupported { + return &pdu.SendRes{ExpectedSize: 0}, nil, nil + } if err != nil { return nil, nil, err } diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index e1a3ffe..2b992c7 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -59,7 +59,7 @@ type StepReport struct { Status StepState Problem string Bytes int64 - ExpectedBytes int64 + ExpectedBytes int64 // 0 means no size estimate possible } type Report struct { @@ -185,7 +185,7 @@ type ReplicationStep struct { err error byteCounter *util.ByteCounterReader - expectedSize int64 + expectedSize int64 // 0 means no size estimate present / possible } func (f *Replication) TakeStep(ctx context.Context, sender Sender, receiver Receiver) (post State, nextStepDate time.Time) { @@ -457,6 +457,7 @@ func (s *ReplicationStep) updateSizeEstimate(ctx context.Context, sender Sender) sres, _, err := sender.Send(ctx, sr) if err != nil { log.WithError(err).Error("dry run send request failed") + return err } s.expectedSize = sres.ExpectedSize return nil diff --git a/replication/pdu/pdu.pb.go b/replication/pdu/pdu.pb.go index be22dea..52f596e 100644 --- a/replication/pdu/pdu.pb.go +++ b/replication/pdu/pdu.pb.go @@ -493,7 +493,8 @@ func (m *Property) GetValue() string { type SendRes struct { // Whether the resume token provided in the request has been used or not. UsedResumeToken bool `protobuf:"varint,1,opt,name=UsedResumeToken,proto3" json:"UsedResumeToken,omitempty"` - // Expected stream size determined by dry run, not exact + // Expected stream size determined by dry run, not exact. + // 0 indicates that for the given SendReq, no size estimate could be made. ExpectedSize int64 `protobuf:"varint,2,opt,name=ExpectedSize,proto3" json:"ExpectedSize,omitempty"` Properties []*Property `protobuf:"bytes,3,rep,name=Properties,proto3" json:"Properties,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` diff --git a/replication/pdu/pdu.proto b/replication/pdu/pdu.proto index f390fd8..1238243 100644 --- a/replication/pdu/pdu.proto +++ b/replication/pdu/pdu.proto @@ -65,7 +65,8 @@ message SendRes { // Whether the resume token provided in the request has been used or not. bool UsedResumeToken = 1; - // Expected stream size determined by dry run, not exact + // Expected stream size determined by dry run, not exact. + // 0 indicates that for the given SendReq, no size estimate could be made. int64 ExpectedSize = 2; repeated Property Properties = 3; diff --git a/zfs/zfs.go b/zfs/zfs.go index a218c72..9c4d6b7 100644 --- a/zfs/zfs.go +++ b/zfs/zfs.go @@ -309,6 +309,9 @@ func ZFSSend(fs string, from, to string) (stream io.ReadCloser, err error) { return } +var BookmarkSizeEstimationNotSupported error = fmt.Errorf("size estimation is not supported for bookmarks") + +// May return BookmarkSizeEstimationNotSupported as err if from is a bookmark. func ZFSSendDry(fs string, from, to string) (size int64, err error) { fromV, err := absVersion(fs, from) @@ -324,6 +327,16 @@ func ZFSSendDry(fs string, from, to string) (size int64, err error) { } } + if strings.Contains(fromV, "#") { + /* TODO: + * ZFS at the time of writing does not support dry-run send because size-estimation + * uses fromSnap's deadlist. However, for a bookmark, that deadlist no longer exists. + * Redacted send & recv will bring this functionality, see + * https://github.com/openzfs/openzfs/pull/484 + */ + return 0, BookmarkSizeEstimationNotSupported + } + args := make([]string, 0) args = append(args, "send", "-n", "-v", "-P") From 975fdee217edccff693e997f3731a59f416826ea Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 5 Sep 2018 18:24:15 -0700 Subject: [PATCH 110/167] replication & pruning: ditch replicated-property, use bookmark as cursor instead A bookmark with a well-known name is used to track which version was last successfully received by the receiver. The createtxg that can be retrieved from the bookmark using `zfs get` is used to set the Replicated attribute of each snap on the sender: If the snap's CreateTXG > the cursor's, it is not yet replicated, otherwise it has been. There is an optional config option to change the behvior to `CreateTXG >= the cursor's`, and the implementation defaults to that. The reason: While things work just fine with `CreateTXG > the cursor's`, ZFS does not provide size estimates in a `zfs send` dry run (see acd2418). However, to enable the use case of keeping the snapshot only around for the replication, the config flag exists. --- config/config.go | 1 + daemon/pruner/pruner.go | 106 +++--- endpoint/endpoint.go | 61 ++-- replication/fsrep/fsfsm.go | 21 +- replication/internal/diff/diff.go | 7 +- replication/pdu/pdu.pb.go | 568 +++++++++++++++++++++--------- replication/pdu/pdu.proto | 24 +- zfs/replication_history.go | 63 +++- zfs/zfs.go | 2 +- 9 files changed, 559 insertions(+), 294 deletions(-) diff --git a/config/config.go b/config/config.go index 32b3b4d..08262a9 100644 --- a/config/config.go +++ b/config/config.go @@ -205,6 +205,7 @@ type PruningEnum struct { type PruneKeepNotReplicated struct { Type string `yaml:"type"` + KeepSnapshotAtCursor bool `yaml:"keep_snapshot_at_cursor,optional,default=true"` } type PruneKeepLastN struct { diff --git a/daemon/pruner/pruner.go b/daemon/pruner/pruner.go index f8e3ab2..e3d5c8c 100644 --- a/daemon/pruner/pruner.go +++ b/daemon/pruner/pruner.go @@ -9,13 +9,14 @@ import ( "github.com/zrepl/zrepl/pruning" "github.com/zrepl/zrepl/replication/pdu" "net" + "sort" "sync" "time" ) // Try to keep it compatible with gitub.com/zrepl/zrepl/replication.Endpoint type History interface { - SnapshotReplicationStatus(ctx context.Context, req *pdu.SnapshotReplicationStatusReq) (*pdu.SnapshotReplicationStatusRes, error) + ReplicationCursor(ctx context.Context, req *pdu.ReplicationCursorReq) (*pdu.ReplicationCursorRes, error) } type Target interface { @@ -42,11 +43,12 @@ func GetLogger(ctx context.Context) Logger { } type args struct { - ctx context.Context - target Target - receiver History - rules []pruning.KeepRule - retryWait time.Duration + ctx context.Context + target Target + receiver History + rules []pruning.KeepRule + retryWait time.Duration + considerSnapAtCursorReplicated bool } type Pruner struct { @@ -66,9 +68,10 @@ type Pruner struct { } type PrunerFactory struct { - senderRules []pruning.KeepRule - receiverRules []pruning.KeepRule - retryWait time.Duration + senderRules []pruning.KeepRule + receiverRules []pruning.KeepRule + retryWait time.Duration + considerSnapAtCursorReplicated bool } func checkContainsKeep1(rules []pruning.KeepRule) error { @@ -95,14 +98,19 @@ func NewPrunerFactory(in config.PruningSenderReceiver) (*PrunerFactory, error) { return nil, errors.Wrap(err, "cannot build sender pruning rules") } - if err := checkContainsKeep1(keepRulesSender); err != nil { - return nil, err + considerSnapAtCursorReplicated := false + for _, r := range in.KeepSender { + knr, ok := r.Ret.(*config.PruneKeepNotReplicated) + if !ok { + continue + } + considerSnapAtCursorReplicated = considerSnapAtCursorReplicated || !knr.KeepSnapshotAtCursor } - f := &PrunerFactory{ keepRulesSender, keepRulesReceiver, 10 * time.Second, //FIXME constant + considerSnapAtCursorReplicated, } return f, nil } @@ -115,6 +123,7 @@ func (f *PrunerFactory) BuildSenderPruner(ctx context.Context, target Target, re receiver, f.senderRules, f.retryWait, + f.considerSnapAtCursorReplicated, }, state: Plan, } @@ -129,6 +138,7 @@ func (f *PrunerFactory) BuildReceiverPruner(ctx context.Context, target Target, receiver, f.receiverRules, f.retryWait, + false, // senseless here anyways }, state: Plan, } @@ -254,56 +264,66 @@ func statePlan(a *args, u updater) state { } pfss := make([]*fs, len(tfss)) +fsloop: for i, tfs := range tfss { + + l := GetLogger(ctx).WithField("fs", tfs.Path) + l.Debug("plan filesystem") + tfsvs, err := target.ListFilesystemVersions(ctx, tfs.Path) if err != nil { + l.WithError(err).Error("cannot list filesystem versions") return onErr(u, err) } + rcReq := &pdu.ReplicationCursorReq{ + Filesystem: tfs.Path, + Op: &pdu.ReplicationCursorReq_Get{}, + } + rc, err := receiver.ReplicationCursor(ctx, rcReq) + if err != nil { + l.WithError(err).Error("cannot get replication cursor") + return onErr(u, err) + } + if rc.GetError() != "" { + l.WithField("reqErr", rc.GetError()).Error("cannot get replication cursor") + return onErr(u, fmt.Errorf("%s", rc.GetError())) + } + pfs := &fs{ path: tfs.Path, snaps: make([]pruning.Snapshot, 0, len(tfsvs)), } + pfss[i] = pfs + // scan from older to newer, all snapshots older than cursor are interpreted as replicated + sort.Slice(tfsvs, func(i, j int) bool { + return tfsvs[i].CreateTXG < tfsvs[j].CreateTXG + }) + preCursor := true for _, tfsv := range tfsvs { if tfsv.Type != pdu.FilesystemVersion_Snapshot { continue } creation, err := tfsv.CreationAsTime() if err != nil { - return onErr(u, fmt.Errorf("%s%s has invalid creation date: %s", tfs, tfsv.RelName(), err)) - } - req := pdu.SnapshotReplicationStatusReq{ - Filesystem: tfs.Path, - Snapshot: tfsv.Name, - Op: pdu.SnapshotReplicationStatusReq_Get, - } - res, err := receiver.SnapshotReplicationStatus(ctx, &req) - if err != nil { - GetLogger(ctx). - WithField("req", req.String()). - WithError(err).Error("cannot get snapshot replication status") - } - if err != nil && shouldRetry(err) { - return onErr(u, err) - } else if err != nil { - pfs.err = err - pfs.snaps = nil - break - } - if res.Status == pdu.SnapshotReplicationStatusRes_Nonexistent { - GetLogger(ctx). - Debug("snapshot does not exist in history, assuming was replicated") + pfs.err = fmt.Errorf("%s%s has invalid creation date: %s", tfs, tfsv.RelName(), err) + l.WithError(pfs.err).Error("") + continue fsloop } + atCursor := tfsv.Guid == rc.GetGuid() + preCursor = preCursor && !atCursor pfs.snaps = append(pfs.snaps, snapshot{ - replicated: !(res.Status != pdu.SnapshotReplicationStatusRes_Replicated), + replicated: preCursor || (a.considerSnapAtCursorReplicated && atCursor), date: creation, fsv: tfsv, }) - } - - pfss[i] = pfs + if preCursor { + pfs.err = fmt.Errorf("replication cursor not found in prune target filesystem versions") + l.WithError(pfs.err).Error("") + continue fsloop + } } @@ -324,7 +344,13 @@ func stateExec(a *args, u updater) state { var pfs *fs state := u(func(pruner *Pruner) { if len(pruner.prunePending) == 0 { - pruner.state = Done + nextState := Done + for _, pfs := range pruner.pruneCompleted { + if pfs.err != nil { + nextState = ErrPerm + } + } + pruner.state = nextState return } pfs = pruner.prunePending[0] diff --git a/endpoint/endpoint.go b/endpoint/endpoint.go index c6c5b37..bd3e2f7 100644 --- a/endpoint/endpoint.go +++ b/endpoint/endpoint.go @@ -109,10 +109,7 @@ func (p *Sender) DestroySnapshots(ctx context.Context, req *pdu.DestroySnapshots return doDestroySnapshots(ctx, dp, req.Snapshots) } -// Since replication always happens from sender to receiver, this method is only ipmlemented for the sender. -// If this method returns a *zfs.DatasetDoesNotExist as an error, it might be a good indicator -// that something is wrong with the pruning logic, which is the only consumer of this method. -func (p *Sender) SnapshotReplicationStatus(ctx context.Context, req *pdu.SnapshotReplicationStatusReq) (*pdu.SnapshotReplicationStatusRes, error) { +func (p *Sender) ReplicationCursor(ctx context.Context, req *pdu.ReplicationCursorReq) (*pdu.ReplicationCursorRes, error) { dp, err := zfs.NewDatasetPath(req.Filesystem) if err != nil { return nil, err @@ -125,35 +122,25 @@ func (p *Sender) SnapshotReplicationStatus(ctx context.Context, req *pdu.Snapsho return nil, replication.NewFilteredError(req.Filesystem) } - version := zfs.FilesystemVersion{ - Type: zfs.Snapshot, - Name: req.Snapshot, //FIXME validation - } - - var status pdu.SnapshotReplicationStatusRes_Status - switch req.Op { - case pdu.SnapshotReplicationStatusReq_Get: - replicated, err := zfs.ZFSGetReplicatedProperty(dp, &version) - if _, ok := err.(*zfs.DatasetDoesNotExist); ok { - status = pdu.SnapshotReplicationStatusRes_Nonexistent - } else if err != nil { - - } - if replicated { - status = pdu.SnapshotReplicationStatusRes_Replicated - } else { - status = pdu.SnapshotReplicationStatusRes_NotReplicated - } - case pdu.SnapshotReplicationStatusReq_SetReplicated: - err = zfs.ZFSSetReplicatedProperty(dp, &version, true) + switch op := req.Op.(type) { + case *pdu.ReplicationCursorReq_Get: + cursor, err := zfs.ZFSGetReplicationCursor(dp) if err != nil { return nil, err } - status = pdu.SnapshotReplicationStatusRes_Replicated + if cursor == nil { + return &pdu.ReplicationCursorRes{Result: &pdu.ReplicationCursorRes_Error{Error: "cursor does not exist"}}, nil + } + return &pdu.ReplicationCursorRes{Result: &pdu.ReplicationCursorRes_Guid{cursor.Guid}}, nil + case *pdu.ReplicationCursorReq_Set: + guid, err := zfs.ZFSSetReplicationCursor(dp, op.Set.Snapshot) + if err != nil { + return nil, err + } + return &pdu.ReplicationCursorRes{Result: &pdu.ReplicationCursorRes_Guid{Guid: guid}}, nil default: - return nil, errors.Errorf("unknown opcode %v", req.Op) + return nil, errors.Errorf("unknown op %T", op) } - return &pdu.SnapshotReplicationStatusRes{Status: status}, nil } type FSFilter interface { @@ -347,12 +334,12 @@ func doDestroySnapshots(ctx context.Context, lp *zfs.DatasetPath, snaps []*pdu.F // =-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= const ( - RPCListFilesystems = "ListFilesystems" - RPCListFilesystemVersions = "ListFilesystemVersions" - RPCReceive = "Receive" - RPCSend = "Send" - RPCSDestroySnapshots = "DestroySnapshots" - RPCSnapshotReplicationStatus = "SnapshotReplicationStatus" + RPCListFilesystems = "ListFilesystems" + RPCListFilesystemVersions = "ListFilesystemVersions" + RPCReceive = "Receive" + RPCSend = "Send" + RPCSDestroySnapshots = "DestroySnapshots" + RPCReplicationCursor = "ReplicationCursor" ) // Remote implements an endpoint stub that uses streamrpc as a transport. @@ -578,18 +565,18 @@ func (a *Handler) Handle(ctx context.Context, endpoint string, reqStructured *by } return bytes.NewBuffer(b), nil, nil - case RPCSnapshotReplicationStatus: + case RPCReplicationCursor: sender, ok := a.ep.(replication.Sender) if !ok { goto Err } - var req pdu.SnapshotReplicationStatusReq + var req pdu.ReplicationCursorReq if err := proto.Unmarshal(reqStructured.Bytes(), &req); err != nil { return nil, nil, err } - res, err := sender.SnapshotReplicationStatus(ctx, &req) + res, err := sender.ReplicationCursor(ctx, &req) if err != nil { return nil, nil, err } diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index 2b992c7..31dcdc1 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -42,7 +42,7 @@ type Sender interface { // any next call to the parent github.com/zrepl/zrepl/replication.Endpoint. // If the send request is for dry run the io.ReadCloser will be nil Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) - SnapshotReplicationStatus(ctx context.Context, r *pdu.SnapshotReplicationStatusReq) (*pdu.SnapshotReplicationStatusRes, error) + ReplicationCursor(ctx context.Context, req *pdu.ReplicationCursorReq) (*pdu.ReplicationCursorRes, error) } // A Sender is usually part of a github.com/zrepl/zrepl/replication.Endpoint. @@ -423,19 +423,22 @@ func (s *ReplicationStep) doMarkReplicated(ctx context.Context, sender Sender) S return s.state } - log.Debug("mark snapshot as replicated") - req := pdu.SnapshotReplicationStatusReq{ + log.Debug("advance replication cursor") + req := &pdu.ReplicationCursorReq{ Filesystem: s.parent.fs, - Snapshot: s.to.GetName(), - Op: pdu.SnapshotReplicationStatusReq_SetReplicated, + Op: &pdu.ReplicationCursorReq_Set{ + Set: &pdu.ReplicationCursorReq_SetOp{ + Snapshot: s.to.GetName(), + }, + }, } - res, err := sender.SnapshotReplicationStatus(ctx, &req) + res, err := sender.ReplicationCursor(ctx, req) if err != nil { - log.WithError(err).Error("error marking snapshot as replicated") + log.WithError(err).Error("error advancing replication cursor") return updateStateError(err) } - if res.Status != pdu.SnapshotReplicationStatusRes_Replicated { - err := fmt.Errorf("sender did not report snapshot as replicated: %s", res.Status) + if res.GetError() != "" { + err := fmt.Errorf("cannot advance replication cursor: %s", res.GetError()) log.Error(err.Error()) return updateStateError(err) } diff --git a/replication/internal/diff/diff.go b/replication/internal/diff/diff.go index c76f5de..6af5246 100644 --- a/replication/internal/diff/diff.go +++ b/replication/internal/diff/diff.go @@ -59,10 +59,9 @@ func IncrementalPath(receiver, sender []*FilesystemVersion) (incPath []*Filesyst for mrcaRcv >= 0 && mrcaSnd >= 0 { if receiver[mrcaRcv].Guid == sender[mrcaSnd].Guid { - if mrcaSnd-1 >= 0 && sender[mrcaSnd-1].Guid == sender[mrcaSnd].Guid && sender[mrcaSnd-1].Type == FilesystemVersion_Bookmark { - // prefer bookmarks over snapshots as the snapshot might go away sooner - mrcaSnd -= 1 - } + // Since we arrive from the end of the array, and because we defined bookmark < snapshot, + // this condition will match snapshot first, which is what we want because it gives us + // size estimation break } receiverCreation, err := receiver[mrcaRcv].CreationAsTime() diff --git a/replication/pdu/pdu.pb.go b/replication/pdu/pdu.pb.go index 52f596e..1171e07 100644 --- a/replication/pdu/pdu.pb.go +++ b/replication/pdu/pdu.pb.go @@ -38,56 +38,7 @@ func (x FilesystemVersion_VersionType) String() string { return proto.EnumName(FilesystemVersion_VersionType_name, int32(x)) } func (FilesystemVersion_VersionType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{5, 0} -} - -type SnapshotReplicationStatusReq_Op int32 - -const ( - SnapshotReplicationStatusReq_Get SnapshotReplicationStatusReq_Op = 0 - SnapshotReplicationStatusReq_SetReplicated SnapshotReplicationStatusReq_Op = 1 -) - -var SnapshotReplicationStatusReq_Op_name = map[int32]string{ - 0: "Get", - 1: "SetReplicated", -} -var SnapshotReplicationStatusReq_Op_value = map[string]int32{ - "Get": 0, - "SetReplicated": 1, -} - -func (x SnapshotReplicationStatusReq_Op) String() string { - return proto.EnumName(SnapshotReplicationStatusReq_Op_name, int32(x)) -} -func (SnapshotReplicationStatusReq_Op) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{14, 0} -} - -type SnapshotReplicationStatusRes_Status int32 - -const ( - SnapshotReplicationStatusRes_Nonexistent SnapshotReplicationStatusRes_Status = 0 - SnapshotReplicationStatusRes_NotReplicated SnapshotReplicationStatusRes_Status = 1 - SnapshotReplicationStatusRes_Replicated SnapshotReplicationStatusRes_Status = 2 -) - -var SnapshotReplicationStatusRes_Status_name = map[int32]string{ - 0: "Nonexistent", - 1: "NotReplicated", - 2: "Replicated", -} -var SnapshotReplicationStatusRes_Status_value = map[string]int32{ - "Nonexistent": 0, - "NotReplicated": 1, - "Replicated": 2, -} - -func (x SnapshotReplicationStatusRes_Status) String() string { - return proto.EnumName(SnapshotReplicationStatusRes_Status_name, int32(x)) -} -func (SnapshotReplicationStatusRes_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{15, 0} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{5, 0} } type ListFilesystemReq struct { @@ -100,7 +51,7 @@ func (m *ListFilesystemReq) Reset() { *m = ListFilesystemReq{} } func (m *ListFilesystemReq) String() string { return proto.CompactTextString(m) } func (*ListFilesystemReq) ProtoMessage() {} func (*ListFilesystemReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{0} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{0} } func (m *ListFilesystemReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemReq.Unmarshal(m, b) @@ -131,7 +82,7 @@ func (m *ListFilesystemRes) Reset() { *m = ListFilesystemRes{} } func (m *ListFilesystemRes) String() string { return proto.CompactTextString(m) } func (*ListFilesystemRes) ProtoMessage() {} func (*ListFilesystemRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{1} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{1} } func (m *ListFilesystemRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemRes.Unmarshal(m, b) @@ -170,7 +121,7 @@ func (m *Filesystem) Reset() { *m = Filesystem{} } func (m *Filesystem) String() string { return proto.CompactTextString(m) } func (*Filesystem) ProtoMessage() {} func (*Filesystem) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{2} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{2} } func (m *Filesystem) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Filesystem.Unmarshal(m, b) @@ -215,7 +166,7 @@ func (m *ListFilesystemVersionsReq) Reset() { *m = ListFilesystemVersion func (m *ListFilesystemVersionsReq) String() string { return proto.CompactTextString(m) } func (*ListFilesystemVersionsReq) ProtoMessage() {} func (*ListFilesystemVersionsReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{3} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{3} } func (m *ListFilesystemVersionsReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemVersionsReq.Unmarshal(m, b) @@ -253,7 +204,7 @@ func (m *ListFilesystemVersionsRes) Reset() { *m = ListFilesystemVersion func (m *ListFilesystemVersionsRes) String() string { return proto.CompactTextString(m) } func (*ListFilesystemVersionsRes) ProtoMessage() {} func (*ListFilesystemVersionsRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{4} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{4} } func (m *ListFilesystemVersionsRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ListFilesystemVersionsRes.Unmarshal(m, b) @@ -295,7 +246,7 @@ func (m *FilesystemVersion) Reset() { *m = FilesystemVersion{} } func (m *FilesystemVersion) String() string { return proto.CompactTextString(m) } func (*FilesystemVersion) ProtoMessage() {} func (*FilesystemVersion) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{5} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{5} } func (m *FilesystemVersion) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_FilesystemVersion.Unmarshal(m, b) @@ -375,7 +326,7 @@ func (m *SendReq) Reset() { *m = SendReq{} } func (m *SendReq) String() string { return proto.CompactTextString(m) } func (*SendReq) ProtoMessage() {} func (*SendReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{6} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{6} } func (m *SendReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SendReq.Unmarshal(m, b) @@ -456,7 +407,7 @@ func (m *Property) Reset() { *m = Property{} } func (m *Property) String() string { return proto.CompactTextString(m) } func (*Property) ProtoMessage() {} func (*Property) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{7} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{7} } func (m *Property) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Property.Unmarshal(m, b) @@ -506,7 +457,7 @@ func (m *SendRes) Reset() { *m = SendRes{} } func (m *SendRes) String() string { return proto.CompactTextString(m) } func (*SendRes) ProtoMessage() {} func (*SendRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{8} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{8} } func (m *SendRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SendRes.Unmarshal(m, b) @@ -560,7 +511,7 @@ func (m *ReceiveReq) Reset() { *m = ReceiveReq{} } func (m *ReceiveReq) String() string { return proto.CompactTextString(m) } func (*ReceiveReq) ProtoMessage() {} func (*ReceiveReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{9} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{9} } func (m *ReceiveReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ReceiveReq.Unmarshal(m, b) @@ -604,7 +555,7 @@ func (m *ReceiveRes) Reset() { *m = ReceiveRes{} } func (m *ReceiveRes) String() string { return proto.CompactTextString(m) } func (*ReceiveRes) ProtoMessage() {} func (*ReceiveRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{10} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{10} } func (m *ReceiveRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ReceiveRes.Unmarshal(m, b) @@ -637,7 +588,7 @@ func (m *DestroySnapshotsReq) Reset() { *m = DestroySnapshotsReq{} } func (m *DestroySnapshotsReq) String() string { return proto.CompactTextString(m) } func (*DestroySnapshotsReq) ProtoMessage() {} func (*DestroySnapshotsReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{11} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{11} } func (m *DestroySnapshotsReq) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DestroySnapshotsReq.Unmarshal(m, b) @@ -683,7 +634,7 @@ func (m *DestroySnapshotRes) Reset() { *m = DestroySnapshotRes{} } func (m *DestroySnapshotRes) String() string { return proto.CompactTextString(m) } func (*DestroySnapshotRes) ProtoMessage() {} func (*DestroySnapshotRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{12} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{12} } func (m *DestroySnapshotRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DestroySnapshotRes.Unmarshal(m, b) @@ -728,7 +679,7 @@ func (m *DestroySnapshotsRes) Reset() { *m = DestroySnapshotsRes{} } func (m *DestroySnapshotsRes) String() string { return proto.CompactTextString(m) } func (*DestroySnapshotsRes) ProtoMessage() {} func (*DestroySnapshotsRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{13} + return fileDescriptor_pdu_cbdc4740ab26577c, []int{13} } func (m *DestroySnapshotsRes) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DestroySnapshotsRes.Unmarshal(m, b) @@ -755,96 +706,361 @@ func (m *DestroySnapshotsRes) GetResults() []*DestroySnapshotRes { return nil } -type SnapshotReplicationStatusReq struct { - Filesystem string `protobuf:"bytes,1,opt,name=Filesystem,proto3" json:"Filesystem,omitempty"` - Snapshot string `protobuf:"bytes,2,opt,name=Snapshot,proto3" json:"Snapshot,omitempty"` - Op SnapshotReplicationStatusReq_Op `protobuf:"varint,3,opt,name=op,proto3,enum=pdu.SnapshotReplicationStatusReq_Op" json:"op,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` +type ReplicationCursorReq struct { + Filesystem string `protobuf:"bytes,1,opt,name=Filesystem,proto3" json:"Filesystem,omitempty"` + // Types that are valid to be assigned to Op: + // *ReplicationCursorReq_Get + // *ReplicationCursorReq_Set + Op isReplicationCursorReq_Op `protobuf_oneof:"op"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SnapshotReplicationStatusReq) Reset() { *m = SnapshotReplicationStatusReq{} } -func (m *SnapshotReplicationStatusReq) String() string { return proto.CompactTextString(m) } -func (*SnapshotReplicationStatusReq) ProtoMessage() {} -func (*SnapshotReplicationStatusReq) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{14} +func (m *ReplicationCursorReq) Reset() { *m = ReplicationCursorReq{} } +func (m *ReplicationCursorReq) String() string { return proto.CompactTextString(m) } +func (*ReplicationCursorReq) ProtoMessage() {} +func (*ReplicationCursorReq) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_cbdc4740ab26577c, []int{14} } -func (m *SnapshotReplicationStatusReq) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_SnapshotReplicationStatusReq.Unmarshal(m, b) +func (m *ReplicationCursorReq) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ReplicationCursorReq.Unmarshal(m, b) } -func (m *SnapshotReplicationStatusReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_SnapshotReplicationStatusReq.Marshal(b, m, deterministic) +func (m *ReplicationCursorReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ReplicationCursorReq.Marshal(b, m, deterministic) } -func (dst *SnapshotReplicationStatusReq) XXX_Merge(src proto.Message) { - xxx_messageInfo_SnapshotReplicationStatusReq.Merge(dst, src) +func (dst *ReplicationCursorReq) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReplicationCursorReq.Merge(dst, src) } -func (m *SnapshotReplicationStatusReq) XXX_Size() int { - return xxx_messageInfo_SnapshotReplicationStatusReq.Size(m) +func (m *ReplicationCursorReq) XXX_Size() int { + return xxx_messageInfo_ReplicationCursorReq.Size(m) } -func (m *SnapshotReplicationStatusReq) XXX_DiscardUnknown() { - xxx_messageInfo_SnapshotReplicationStatusReq.DiscardUnknown(m) +func (m *ReplicationCursorReq) XXX_DiscardUnknown() { + xxx_messageInfo_ReplicationCursorReq.DiscardUnknown(m) } -var xxx_messageInfo_SnapshotReplicationStatusReq proto.InternalMessageInfo +var xxx_messageInfo_ReplicationCursorReq proto.InternalMessageInfo -func (m *SnapshotReplicationStatusReq) GetFilesystem() string { +func (m *ReplicationCursorReq) GetFilesystem() string { if m != nil { return m.Filesystem } return "" } -func (m *SnapshotReplicationStatusReq) GetSnapshot() string { +type isReplicationCursorReq_Op interface { + isReplicationCursorReq_Op() +} + +type ReplicationCursorReq_Get struct { + Get *ReplicationCursorReq_GetOp `protobuf:"bytes,2,opt,name=get,proto3,oneof"` +} + +type ReplicationCursorReq_Set struct { + Set *ReplicationCursorReq_SetOp `protobuf:"bytes,3,opt,name=set,proto3,oneof"` +} + +func (*ReplicationCursorReq_Get) isReplicationCursorReq_Op() {} + +func (*ReplicationCursorReq_Set) isReplicationCursorReq_Op() {} + +func (m *ReplicationCursorReq) GetOp() isReplicationCursorReq_Op { + if m != nil { + return m.Op + } + return nil +} + +func (m *ReplicationCursorReq) GetGet() *ReplicationCursorReq_GetOp { + if x, ok := m.GetOp().(*ReplicationCursorReq_Get); ok { + return x.Get + } + return nil +} + +func (m *ReplicationCursorReq) GetSet() *ReplicationCursorReq_SetOp { + if x, ok := m.GetOp().(*ReplicationCursorReq_Set); ok { + return x.Set + } + return nil +} + +// XXX_OneofFuncs is for the internal use of the proto package. +func (*ReplicationCursorReq) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { + return _ReplicationCursorReq_OneofMarshaler, _ReplicationCursorReq_OneofUnmarshaler, _ReplicationCursorReq_OneofSizer, []interface{}{ + (*ReplicationCursorReq_Get)(nil), + (*ReplicationCursorReq_Set)(nil), + } +} + +func _ReplicationCursorReq_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { + m := msg.(*ReplicationCursorReq) + // op + switch x := m.Op.(type) { + case *ReplicationCursorReq_Get: + b.EncodeVarint(2<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Get); err != nil { + return err + } + case *ReplicationCursorReq_Set: + b.EncodeVarint(3<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Set); err != nil { + return err + } + case nil: + default: + return fmt.Errorf("ReplicationCursorReq.Op has unexpected type %T", x) + } + return nil +} + +func _ReplicationCursorReq_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) { + m := msg.(*ReplicationCursorReq) + switch tag { + case 2: // op.get + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ReplicationCursorReq_GetOp) + err := b.DecodeMessage(msg) + m.Op = &ReplicationCursorReq_Get{msg} + return true, err + case 3: // op.set + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ReplicationCursorReq_SetOp) + err := b.DecodeMessage(msg) + m.Op = &ReplicationCursorReq_Set{msg} + return true, err + default: + return false, nil + } +} + +func _ReplicationCursorReq_OneofSizer(msg proto.Message) (n int) { + m := msg.(*ReplicationCursorReq) + // op + switch x := m.Op.(type) { + case *ReplicationCursorReq_Get: + s := proto.Size(x.Get) + n += 1 // tag and wire + n += proto.SizeVarint(uint64(s)) + n += s + case *ReplicationCursorReq_Set: + s := proto.Size(x.Set) + n += 1 // tag and wire + n += proto.SizeVarint(uint64(s)) + n += s + case nil: + default: + panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) + } + return n +} + +type ReplicationCursorReq_GetOp struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ReplicationCursorReq_GetOp) Reset() { *m = ReplicationCursorReq_GetOp{} } +func (m *ReplicationCursorReq_GetOp) String() string { return proto.CompactTextString(m) } +func (*ReplicationCursorReq_GetOp) ProtoMessage() {} +func (*ReplicationCursorReq_GetOp) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_cbdc4740ab26577c, []int{14, 0} +} +func (m *ReplicationCursorReq_GetOp) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ReplicationCursorReq_GetOp.Unmarshal(m, b) +} +func (m *ReplicationCursorReq_GetOp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ReplicationCursorReq_GetOp.Marshal(b, m, deterministic) +} +func (dst *ReplicationCursorReq_GetOp) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReplicationCursorReq_GetOp.Merge(dst, src) +} +func (m *ReplicationCursorReq_GetOp) XXX_Size() int { + return xxx_messageInfo_ReplicationCursorReq_GetOp.Size(m) +} +func (m *ReplicationCursorReq_GetOp) XXX_DiscardUnknown() { + xxx_messageInfo_ReplicationCursorReq_GetOp.DiscardUnknown(m) +} + +var xxx_messageInfo_ReplicationCursorReq_GetOp proto.InternalMessageInfo + +type ReplicationCursorReq_SetOp struct { + Snapshot string `protobuf:"bytes,2,opt,name=Snapshot,proto3" json:"Snapshot,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ReplicationCursorReq_SetOp) Reset() { *m = ReplicationCursorReq_SetOp{} } +func (m *ReplicationCursorReq_SetOp) String() string { return proto.CompactTextString(m) } +func (*ReplicationCursorReq_SetOp) ProtoMessage() {} +func (*ReplicationCursorReq_SetOp) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_cbdc4740ab26577c, []int{14, 1} +} +func (m *ReplicationCursorReq_SetOp) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ReplicationCursorReq_SetOp.Unmarshal(m, b) +} +func (m *ReplicationCursorReq_SetOp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ReplicationCursorReq_SetOp.Marshal(b, m, deterministic) +} +func (dst *ReplicationCursorReq_SetOp) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReplicationCursorReq_SetOp.Merge(dst, src) +} +func (m *ReplicationCursorReq_SetOp) XXX_Size() int { + return xxx_messageInfo_ReplicationCursorReq_SetOp.Size(m) +} +func (m *ReplicationCursorReq_SetOp) XXX_DiscardUnknown() { + xxx_messageInfo_ReplicationCursorReq_SetOp.DiscardUnknown(m) +} + +var xxx_messageInfo_ReplicationCursorReq_SetOp proto.InternalMessageInfo + +func (m *ReplicationCursorReq_SetOp) GetSnapshot() string { if m != nil { return m.Snapshot } return "" } -func (m *SnapshotReplicationStatusReq) GetOp() SnapshotReplicationStatusReq_Op { +type ReplicationCursorRes struct { + // Types that are valid to be assigned to Result: + // *ReplicationCursorRes_Guid + // *ReplicationCursorRes_Error + Result isReplicationCursorRes_Result `protobuf_oneof:"Result"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ReplicationCursorRes) Reset() { *m = ReplicationCursorRes{} } +func (m *ReplicationCursorRes) String() string { return proto.CompactTextString(m) } +func (*ReplicationCursorRes) ProtoMessage() {} +func (*ReplicationCursorRes) Descriptor() ([]byte, []int) { + return fileDescriptor_pdu_cbdc4740ab26577c, []int{15} +} +func (m *ReplicationCursorRes) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ReplicationCursorRes.Unmarshal(m, b) +} +func (m *ReplicationCursorRes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ReplicationCursorRes.Marshal(b, m, deterministic) +} +func (dst *ReplicationCursorRes) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReplicationCursorRes.Merge(dst, src) +} +func (m *ReplicationCursorRes) XXX_Size() int { + return xxx_messageInfo_ReplicationCursorRes.Size(m) +} +func (m *ReplicationCursorRes) XXX_DiscardUnknown() { + xxx_messageInfo_ReplicationCursorRes.DiscardUnknown(m) +} + +var xxx_messageInfo_ReplicationCursorRes proto.InternalMessageInfo + +type isReplicationCursorRes_Result interface { + isReplicationCursorRes_Result() +} + +type ReplicationCursorRes_Guid struct { + Guid uint64 `protobuf:"varint,1,opt,name=Guid,proto3,oneof"` +} + +type ReplicationCursorRes_Error struct { + Error string `protobuf:"bytes,2,opt,name=Error,proto3,oneof"` +} + +func (*ReplicationCursorRes_Guid) isReplicationCursorRes_Result() {} + +func (*ReplicationCursorRes_Error) isReplicationCursorRes_Result() {} + +func (m *ReplicationCursorRes) GetResult() isReplicationCursorRes_Result { if m != nil { - return m.Op + return m.Result } - return SnapshotReplicationStatusReq_Get + return nil } -type SnapshotReplicationStatusRes struct { - Status SnapshotReplicationStatusRes_Status `protobuf:"varint,1,opt,name=status,proto3,enum=pdu.SnapshotReplicationStatusRes_Status" json:"status,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *SnapshotReplicationStatusRes) Reset() { *m = SnapshotReplicationStatusRes{} } -func (m *SnapshotReplicationStatusRes) String() string { return proto.CompactTextString(m) } -func (*SnapshotReplicationStatusRes) ProtoMessage() {} -func (*SnapshotReplicationStatusRes) Descriptor() ([]byte, []int) { - return fileDescriptor_pdu_b3a98b3542e9fb4e, []int{15} -} -func (m *SnapshotReplicationStatusRes) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_SnapshotReplicationStatusRes.Unmarshal(m, b) -} -func (m *SnapshotReplicationStatusRes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_SnapshotReplicationStatusRes.Marshal(b, m, deterministic) -} -func (dst *SnapshotReplicationStatusRes) XXX_Merge(src proto.Message) { - xxx_messageInfo_SnapshotReplicationStatusRes.Merge(dst, src) -} -func (m *SnapshotReplicationStatusRes) XXX_Size() int { - return xxx_messageInfo_SnapshotReplicationStatusRes.Size(m) -} -func (m *SnapshotReplicationStatusRes) XXX_DiscardUnknown() { - xxx_messageInfo_SnapshotReplicationStatusRes.DiscardUnknown(m) -} - -var xxx_messageInfo_SnapshotReplicationStatusRes proto.InternalMessageInfo - -func (m *SnapshotReplicationStatusRes) GetStatus() SnapshotReplicationStatusRes_Status { - if m != nil { - return m.Status +func (m *ReplicationCursorRes) GetGuid() uint64 { + if x, ok := m.GetResult().(*ReplicationCursorRes_Guid); ok { + return x.Guid } - return SnapshotReplicationStatusRes_Nonexistent + return 0 +} + +func (m *ReplicationCursorRes) GetError() string { + if x, ok := m.GetResult().(*ReplicationCursorRes_Error); ok { + return x.Error + } + return "" +} + +// XXX_OneofFuncs is for the internal use of the proto package. +func (*ReplicationCursorRes) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { + return _ReplicationCursorRes_OneofMarshaler, _ReplicationCursorRes_OneofUnmarshaler, _ReplicationCursorRes_OneofSizer, []interface{}{ + (*ReplicationCursorRes_Guid)(nil), + (*ReplicationCursorRes_Error)(nil), + } +} + +func _ReplicationCursorRes_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { + m := msg.(*ReplicationCursorRes) + // Result + switch x := m.Result.(type) { + case *ReplicationCursorRes_Guid: + b.EncodeVarint(1<<3 | proto.WireVarint) + b.EncodeVarint(uint64(x.Guid)) + case *ReplicationCursorRes_Error: + b.EncodeVarint(2<<3 | proto.WireBytes) + b.EncodeStringBytes(x.Error) + case nil: + default: + return fmt.Errorf("ReplicationCursorRes.Result has unexpected type %T", x) + } + return nil +} + +func _ReplicationCursorRes_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) { + m := msg.(*ReplicationCursorRes) + switch tag { + case 1: // Result.Guid + if wire != proto.WireVarint { + return true, proto.ErrInternalBadWireType + } + x, err := b.DecodeVarint() + m.Result = &ReplicationCursorRes_Guid{x} + return true, err + case 2: // Result.Error + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + x, err := b.DecodeStringBytes() + m.Result = &ReplicationCursorRes_Error{x} + return true, err + default: + return false, nil + } +} + +func _ReplicationCursorRes_OneofSizer(msg proto.Message) (n int) { + m := msg.(*ReplicationCursorRes) + // Result + switch x := m.Result.(type) { + case *ReplicationCursorRes_Guid: + n += 1 // tag and wire + n += proto.SizeVarint(uint64(x.Guid)) + case *ReplicationCursorRes_Error: + n += 1 // tag and wire + n += proto.SizeVarint(uint64(len(x.Error))) + n += len(x.Error) + case nil: + default: + panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) + } + return n } func init() { @@ -862,57 +1078,57 @@ func init() { proto.RegisterType((*DestroySnapshotsReq)(nil), "pdu.DestroySnapshotsReq") proto.RegisterType((*DestroySnapshotRes)(nil), "pdu.DestroySnapshotRes") proto.RegisterType((*DestroySnapshotsRes)(nil), "pdu.DestroySnapshotsRes") - proto.RegisterType((*SnapshotReplicationStatusReq)(nil), "pdu.SnapshotReplicationStatusReq") - proto.RegisterType((*SnapshotReplicationStatusRes)(nil), "pdu.SnapshotReplicationStatusRes") + proto.RegisterType((*ReplicationCursorReq)(nil), "pdu.ReplicationCursorReq") + proto.RegisterType((*ReplicationCursorReq_GetOp)(nil), "pdu.ReplicationCursorReq.GetOp") + proto.RegisterType((*ReplicationCursorReq_SetOp)(nil), "pdu.ReplicationCursorReq.SetOp") + proto.RegisterType((*ReplicationCursorRes)(nil), "pdu.ReplicationCursorRes") proto.RegisterEnum("pdu.FilesystemVersion_VersionType", FilesystemVersion_VersionType_name, FilesystemVersion_VersionType_value) - proto.RegisterEnum("pdu.SnapshotReplicationStatusReq_Op", SnapshotReplicationStatusReq_Op_name, SnapshotReplicationStatusReq_Op_value) - proto.RegisterEnum("pdu.SnapshotReplicationStatusRes_Status", SnapshotReplicationStatusRes_Status_name, SnapshotReplicationStatusRes_Status_value) } -func init() { proto.RegisterFile("pdu.proto", fileDescriptor_pdu_b3a98b3542e9fb4e) } +func init() { proto.RegisterFile("pdu.proto", fileDescriptor_pdu_cbdc4740ab26577c) } -var fileDescriptor_pdu_b3a98b3542e9fb4e = []byte{ - // 666 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0xcf, 0x6e, 0xd3, 0x4e, - 0x10, 0xfe, 0xd9, 0x49, 0xf3, 0x67, 0xd2, 0xa6, 0xe9, 0xb6, 0xea, 0xcf, 0x54, 0x15, 0x8a, 0x56, - 0x1c, 0x02, 0x12, 0x91, 0x08, 0x15, 0x17, 0x38, 0xa0, 0xfe, 0xe5, 0x80, 0xda, 0x6a, 0x13, 0xaa, - 0x9e, 0x90, 0x4c, 0x3d, 0x52, 0xad, 0x24, 0xde, 0xed, 0xee, 0x1a, 0x35, 0x3c, 0x00, 0x8f, 0xc1, - 0x43, 0x70, 0xe3, 0x4d, 0x78, 0x1c, 0xe4, 0x89, 0xed, 0xb8, 0x49, 0x09, 0x39, 0x65, 0xbe, 0x6f, - 0x66, 0x67, 0xbe, 0x99, 0xdd, 0x71, 0xa0, 0xae, 0x82, 0xb8, 0xab, 0xb4, 0xb4, 0x92, 0x95, 0x54, - 0x10, 0xf3, 0x6d, 0xd8, 0xfa, 0x18, 0x1a, 0x7b, 0x1a, 0x8e, 0xd0, 0x4c, 0x8c, 0xc5, 0xb1, 0xc0, - 0x3b, 0x7e, 0xba, 0x48, 0x1a, 0xf6, 0x0a, 0x1a, 0x33, 0xc2, 0x78, 0x4e, 0xbb, 0xd4, 0x69, 0xf4, - 0x36, 0xbb, 0x49, 0xbe, 0x42, 0x60, 0x31, 0x86, 0x1f, 0x02, 0xcc, 0x20, 0x63, 0x50, 0xbe, 0xf4, - 0xed, 0xad, 0xe7, 0xb4, 0x9d, 0x4e, 0x5d, 0x90, 0xcd, 0xda, 0xd0, 0x10, 0x68, 0xe2, 0x31, 0x0e, - 0xe4, 0x10, 0x23, 0xcf, 0x25, 0x57, 0x91, 0xe2, 0x6f, 0xe1, 0xc9, 0x43, 0x2d, 0x57, 0xa8, 0x4d, - 0x28, 0x23, 0x23, 0xf0, 0x8e, 0x3d, 0x2d, 0x16, 0x48, 0x13, 0x17, 0x18, 0x7e, 0xf1, 0xf7, 0xc3, - 0x86, 0xf5, 0xa0, 0x96, 0xc1, 0xb4, 0x9b, 0xdd, 0xb9, 0x6e, 0x52, 0xb7, 0xc8, 0xe3, 0xf8, 0x6f, - 0x07, 0xb6, 0x16, 0xfc, 0xec, 0x0d, 0x94, 0x07, 0x13, 0x85, 0x24, 0xa0, 0xd9, 0xe3, 0x8f, 0x67, - 0xe9, 0xa6, 0xbf, 0x49, 0xa4, 0xa0, 0xf8, 0x64, 0x22, 0xe7, 0xfe, 0x18, 0xd3, 0xb6, 0xc9, 0x4e, - 0xb8, 0xb3, 0x38, 0x0c, 0xbc, 0x52, 0xdb, 0xe9, 0x94, 0x05, 0xd9, 0x6c, 0x1f, 0xea, 0x47, 0x1a, - 0x7d, 0x8b, 0x83, 0xeb, 0x33, 0xaf, 0x4c, 0x8e, 0x19, 0xc1, 0xf6, 0xa0, 0x46, 0x20, 0x94, 0x91, - 0xb7, 0x46, 0x99, 0x72, 0xcc, 0x9f, 0x43, 0xa3, 0x50, 0x96, 0xad, 0x43, 0xad, 0x1f, 0xf9, 0xca, - 0xdc, 0x4a, 0xdb, 0xfa, 0x2f, 0x41, 0x87, 0x52, 0x0e, 0xc7, 0xbe, 0x1e, 0xb6, 0x1c, 0xfe, 0xcb, - 0x81, 0x6a, 0x1f, 0xa3, 0x60, 0x85, 0xb9, 0x26, 0x22, 0x4f, 0xb5, 0x1c, 0x67, 0xc2, 0x13, 0x9b, - 0x35, 0xc1, 0x1d, 0x48, 0x92, 0x5d, 0x17, 0xee, 0x40, 0xce, 0x5f, 0x6d, 0x79, 0xe1, 0x6a, 0x49, - 0xb8, 0x1c, 0x2b, 0x8d, 0xc6, 0x90, 0xf0, 0x9a, 0xc8, 0x31, 0xdb, 0x81, 0xb5, 0x63, 0x0c, 0x62, - 0xe5, 0x55, 0xc8, 0x31, 0x05, 0x6c, 0x17, 0x2a, 0xc7, 0x7a, 0x22, 0xe2, 0xc8, 0xab, 0x12, 0x9d, - 0x22, 0x7e, 0x00, 0xb5, 0x4b, 0x2d, 0x15, 0x6a, 0x3b, 0xc9, 0x87, 0xea, 0x14, 0x86, 0xba, 0x03, - 0x6b, 0x57, 0xfe, 0x28, 0xce, 0x26, 0x3d, 0x05, 0xfc, 0x7b, 0xde, 0xb1, 0x61, 0x1d, 0xd8, 0xfc, - 0x64, 0x30, 0x28, 0x2a, 0x76, 0xa8, 0xc4, 0x3c, 0xcd, 0x38, 0xac, 0x9f, 0xdc, 0x2b, 0xbc, 0xb1, - 0x18, 0xf4, 0xc3, 0x6f, 0xd3, 0x94, 0x25, 0xf1, 0x80, 0x63, 0x2f, 0x01, 0x52, 0x3d, 0x21, 0x1a, - 0xaf, 0x44, 0x8f, 0x6b, 0x83, 0x9e, 0x45, 0x26, 0x53, 0x14, 0x02, 0xf8, 0x35, 0x80, 0xc0, 0x1b, - 0x0c, 0xbf, 0xe2, 0x2a, 0xc3, 0x7f, 0x01, 0xad, 0xa3, 0x11, 0xfa, 0x7a, 0x7e, 0x71, 0x6a, 0x62, - 0x81, 0xe7, 0xeb, 0x85, 0xcc, 0x86, 0x0f, 0x61, 0xfb, 0x18, 0x8d, 0xd5, 0x72, 0x92, 0xbd, 0x82, - 0x55, 0xb6, 0x88, 0x1d, 0x40, 0x3d, 0x8f, 0xf7, 0xdc, 0xa5, 0x9b, 0x32, 0x0b, 0xe4, 0x9f, 0x81, - 0xcd, 0x15, 0x4b, 0x97, 0x2e, 0x83, 0x54, 0x69, 0xc9, 0xd2, 0x65, 0x71, 0xc9, 0xed, 0x9d, 0x68, - 0x2d, 0x75, 0x76, 0x7b, 0x04, 0xf8, 0x87, 0xc7, 0x9a, 0x49, 0x3e, 0x53, 0xd5, 0x64, 0x00, 0x23, - 0x9b, 0x2d, 0xf5, 0xff, 0x94, 0x7f, 0x51, 0x8a, 0xc8, 0xe2, 0xf8, 0x4f, 0x07, 0xf6, 0x67, 0x0e, - 0x35, 0x0a, 0x6f, 0x68, 0x79, 0xfa, 0xd6, 0xb7, 0xf1, 0x4a, 0x03, 0xda, 0x2b, 0x34, 0x35, 0xd5, - 0x38, 0x13, 0x7f, 0x00, 0xae, 0x54, 0xb4, 0x16, 0xcd, 0xde, 0x33, 0x92, 0xb2, 0xac, 0x54, 0xf7, - 0x42, 0x09, 0x57, 0x2a, 0xde, 0x06, 0xf7, 0x42, 0xb1, 0x2a, 0x94, 0xce, 0x30, 0xd9, 0xd4, 0x2d, - 0xd8, 0xe8, 0x63, 0x7e, 0x00, 0x83, 0x96, 0xc3, 0x7f, 0x2c, 0x17, 0x6d, 0xd8, 0x7b, 0xa8, 0x18, - 0x02, 0xe9, 0x67, 0xa9, 0xf3, 0xaf, 0xe2, 0xa6, 0x9b, 0x5a, 0xe9, 0x39, 0xfe, 0x0e, 0x2a, 0x53, - 0x86, 0x6d, 0x42, 0xe3, 0x5c, 0x46, 0x78, 0x1f, 0x1a, 0x8b, 0x51, 0x2a, 0xe8, 0x5c, 0x3e, 0x10, - 0xc4, 0x9a, 0xc9, 0x53, 0xcb, 0xb1, 0xfb, 0xa5, 0x42, 0xff, 0x32, 0xaf, 0xff, 0x04, 0x00, 0x00, - 0xff, 0xff, 0x3a, 0x00, 0x8b, 0x6e, 0x72, 0x06, 0x00, 0x00, +var fileDescriptor_pdu_cbdc4740ab26577c = []byte{ + // 657 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0xcb, 0x6e, 0xdb, 0x3a, + 0x10, 0xb5, 0x6c, 0xd9, 0x96, 0xc7, 0xb9, 0x79, 0x30, 0x41, 0xae, 0x6e, 0x70, 0x71, 0xaf, 0xc1, + 0x6e, 0xdc, 0x02, 0x35, 0x50, 0x27, 0xe8, 0xa6, 0x3b, 0xe7, 0xe5, 0x45, 0x91, 0x04, 0xb4, 0x1b, + 0x64, 0x55, 0x40, 0x8d, 0x06, 0x8d, 0xe0, 0x07, 0x15, 0x92, 0x2a, 0xea, 0x7e, 0x40, 0xff, 0xa9, + 0xff, 0xd1, 0x45, 0x3f, 0xa7, 0xe0, 0x58, 0x92, 0x15, 0xdb, 0x0d, 0xbc, 0x32, 0xcf, 0xf0, 0x70, + 0xe6, 0xcc, 0xa1, 0x86, 0x86, 0x46, 0x1c, 0x26, 0x9d, 0x58, 0x49, 0x23, 0x59, 0x25, 0x0e, 0x13, + 0xbe, 0x0f, 0x7b, 0xef, 0x23, 0x6d, 0x2e, 0xa2, 0x31, 0xea, 0x99, 0x36, 0x38, 0x11, 0xf8, 0xc8, + 0x2f, 0x56, 0x83, 0x9a, 0xbd, 0x81, 0xe6, 0x22, 0xa0, 0x7d, 0xa7, 0x55, 0x69, 0x37, 0xbb, 0x3b, + 0x1d, 0x9b, 0xaf, 0x40, 0x2c, 0x72, 0x78, 0x0f, 0x60, 0x01, 0x19, 0x03, 0xf7, 0x26, 0x30, 0x0f, + 0xbe, 0xd3, 0x72, 0xda, 0x0d, 0x41, 0x6b, 0xd6, 0x82, 0xa6, 0x40, 0x9d, 0x4c, 0x70, 0x28, 0x47, + 0x38, 0xf5, 0xcb, 0xb4, 0x55, 0x0c, 0xf1, 0x77, 0xf0, 0xcf, 0x53, 0x2d, 0xb7, 0xa8, 0x74, 0x24, + 0xa7, 0x5a, 0xe0, 0x23, 0xfb, 0xaf, 0x58, 0x20, 0x4d, 0x5c, 0x88, 0xf0, 0xeb, 0x3f, 0x1f, 0xd6, + 0xac, 0x0b, 0x5e, 0x06, 0xd3, 0x6e, 0x0e, 0x97, 0xba, 0x49, 0xb7, 0x45, 0xce, 0xe3, 0xbf, 0x1c, + 0xd8, 0x5b, 0xd9, 0x67, 0x6f, 0xc1, 0x1d, 0xce, 0x62, 0x24, 0x01, 0xdb, 0x5d, 0xbe, 0x3e, 0x4b, + 0x27, 0xfd, 0xb5, 0x4c, 0x41, 0x7c, 0xeb, 0xc8, 0x55, 0x30, 0xc1, 0xb4, 0x6d, 0x5a, 0xdb, 0xd8, + 0x65, 0x12, 0x85, 0x7e, 0xa5, 0xe5, 0xb4, 0x5d, 0x41, 0x6b, 0xf6, 0x2f, 0x34, 0x4e, 0x15, 0x06, + 0x06, 0x87, 0x77, 0x97, 0xbe, 0x4b, 0x1b, 0x8b, 0x00, 0x3b, 0x02, 0x8f, 0x40, 0x24, 0xa7, 0x7e, + 0x95, 0x32, 0xe5, 0x98, 0xbf, 0x84, 0x66, 0xa1, 0x2c, 0xdb, 0x02, 0x6f, 0x30, 0x0d, 0x62, 0xfd, + 0x20, 0xcd, 0x6e, 0xc9, 0xa2, 0x9e, 0x94, 0xa3, 0x49, 0xa0, 0x46, 0xbb, 0x0e, 0xff, 0xe1, 0x40, + 0x7d, 0x80, 0xd3, 0x70, 0x03, 0x5f, 0xad, 0xc8, 0x0b, 0x25, 0x27, 0x99, 0x70, 0xbb, 0x66, 0xdb, + 0x50, 0x1e, 0x4a, 0x92, 0xdd, 0x10, 0xe5, 0xa1, 0x5c, 0xbe, 0x5a, 0x77, 0xe5, 0x6a, 0x49, 0xb8, + 0x9c, 0xc4, 0x0a, 0xb5, 0x26, 0xe1, 0x9e, 0xc8, 0x31, 0x3b, 0x80, 0xea, 0x19, 0x86, 0x49, 0xec, + 0xd7, 0x68, 0x63, 0x0e, 0xd8, 0x21, 0xd4, 0xce, 0xd4, 0x4c, 0x24, 0x53, 0xbf, 0x4e, 0xe1, 0x14, + 0xf1, 0x13, 0xf0, 0x6e, 0x94, 0x8c, 0x51, 0x99, 0x59, 0x6e, 0xaa, 0x53, 0x30, 0xf5, 0x00, 0xaa, + 0xb7, 0xc1, 0x38, 0xc9, 0x9c, 0x9e, 0x03, 0xfe, 0x3d, 0xef, 0x58, 0xb3, 0x36, 0xec, 0x7c, 0xd0, + 0x18, 0x16, 0x15, 0x3b, 0x54, 0x62, 0x39, 0xcc, 0x38, 0x6c, 0x9d, 0x7f, 0x8d, 0xf1, 0xde, 0x60, + 0x38, 0x88, 0xbe, 0xcd, 0x53, 0x56, 0xc4, 0x93, 0x18, 0x7b, 0x0d, 0x90, 0xea, 0x89, 0x50, 0xfb, + 0x15, 0xfa, 0xb8, 0xfe, 0xa2, 0xcf, 0x22, 0x93, 0x29, 0x0a, 0x04, 0x7e, 0x07, 0x20, 0xf0, 0x1e, + 0xa3, 0x2f, 0xb8, 0x89, 0xf9, 0xaf, 0x60, 0xf7, 0x74, 0x8c, 0x81, 0x5a, 0x1e, 0x1c, 0x4f, 0xac, + 0xc4, 0xf9, 0x56, 0x21, 0xb3, 0xe6, 0x23, 0xd8, 0x3f, 0x43, 0x6d, 0x94, 0x9c, 0x65, 0x5f, 0xc1, + 0x26, 0x53, 0xc4, 0x4e, 0xa0, 0x91, 0xf3, 0xfd, 0xf2, 0xb3, 0x93, 0xb2, 0x20, 0xf2, 0x8f, 0xc0, + 0x96, 0x8a, 0xa5, 0x43, 0x97, 0x41, 0xaa, 0xf4, 0xcc, 0xd0, 0x65, 0x3c, 0x7b, 0x7b, 0xe7, 0x4a, + 0x49, 0x95, 0xdd, 0x1e, 0x01, 0xde, 0x5f, 0xd7, 0x8c, 0x7d, 0xa6, 0xea, 0xd6, 0x80, 0xb1, 0xc9, + 0x86, 0xfa, 0x6f, 0xca, 0xbf, 0x2a, 0x45, 0x64, 0x3c, 0xfe, 0xd3, 0x81, 0x03, 0x81, 0xf1, 0x38, + 0xba, 0xa7, 0xa1, 0x39, 0x4d, 0x94, 0x96, 0x6a, 0x13, 0x63, 0x8e, 0xa1, 0xf2, 0x19, 0x0d, 0xc9, + 0x6a, 0x76, 0xff, 0xa7, 0x3a, 0xeb, 0xf2, 0x74, 0x2e, 0xd1, 0x5c, 0xc7, 0xfd, 0x92, 0xb0, 0x6c, + 0x7b, 0x48, 0xa3, 0xa1, 0x41, 0x79, 0xf6, 0xd0, 0x20, 0x3b, 0xa4, 0xd1, 0x1c, 0xd5, 0xa1, 0x4a, + 0x49, 0x8e, 0x5e, 0x40, 0x95, 0x36, 0xec, 0xf0, 0xe4, 0x46, 0xce, 0x7d, 0xc9, 0x71, 0xcf, 0x85, + 0xb2, 0x8c, 0xf9, 0xd5, 0xda, 0xae, 0xec, 0x68, 0xcd, 0x5f, 0x18, 0xdb, 0x8f, 0xdb, 0x2f, 0xa5, + 0x6f, 0xcc, 0xe1, 0x13, 0x93, 0xfb, 0xa5, 0xd4, 0xe6, 0x9e, 0x07, 0xb5, 0xb9, 0x4f, 0x9f, 0x6a, + 0xf4, 0xb7, 0x71, 0xfc, 0x3b, 0x00, 0x00, 0xff, 0xff, 0x02, 0x35, 0xe7, 0x43, 0x43, 0x06, 0x00, + 0x00, } diff --git a/replication/pdu/pdu.proto b/replication/pdu/pdu.proto index 1238243..d07902c 100644 --- a/replication/pdu/pdu.proto +++ b/replication/pdu/pdu.proto @@ -98,21 +98,21 @@ message DestroySnapshotsRes { repeated DestroySnapshotRes Results = 1; } -message SnapshotReplicationStatusReq { +message ReplicationCursorReq { string Filesystem = 1; - string Snapshot = 2; - enum Op { - Get = 0; - SetReplicated = 1; + message GetOp {} + message SetOp { + string Snapshot = 2; + } + oneof op { + GetOp get = 2; + SetOp set = 3; } - Op op = 3; } -message SnapshotReplicationStatusRes { - enum Status { - Nonexistent = 0; - NotReplicated = 1; - Replicated = 2; +message ReplicationCursorRes { + oneof Result { + uint64 Guid = 1; + string Error = 2; } - Status status = 1; } diff --git a/zfs/replication_history.go b/zfs/replication_history.go index c650451..c5b46c3 100644 --- a/zfs/replication_history.go +++ b/zfs/replication_history.go @@ -1,25 +1,58 @@ package zfs -const ReplicatedProperty = "zrepl:replicated" +import ( + "fmt" + "github.com/pkg/errors" + "strconv" +) -// May return *DatasetDoesNotExist as an error -func ZFSGetReplicatedProperty(fs *DatasetPath, v *FilesystemVersion) (replicated bool, err error) { - props, err := zfsGet(v.ToAbsPath(fs), []string{ReplicatedProperty}) +const ReplicationCursorBookmarkName = "zrepl_replication_cursor" + +// may return nil for both values, indicating there is no cursor +func ZFSGetReplicationCursor(fs *DatasetPath) (*FilesystemVersion, error) { + versions, err := ZFSListFilesystemVersions(fs, nil) if err != nil { - return false, err + return nil, err } - if props.Get(ReplicatedProperty) == "yes" { - return true, nil + for _, v := range versions { + if v.Type == Bookmark && v.Name == ReplicationCursorBookmarkName { + return &v, nil + } } - return false, nil + return nil, nil } -func ZFSSetReplicatedProperty(fs *DatasetPath, v *FilesystemVersion, replicated bool) error { - val := "no" - if replicated { - val = "yes" +func ZFSSetReplicationCursor(fs *DatasetPath, snapname string) (guid uint64, err error) { + snapPath := fmt.Sprintf("%s@%s", fs.ToString(), snapname) + propsSnap, err := zfsGet(snapPath, []string{"createtxg", "guid"}) + if err != nil { + return 0, err } - props := NewZFSProperties() - props.Set(ReplicatedProperty, val) - return zfsSet(v.ToAbsPath(fs), props) + snapGuid, err := strconv.ParseUint(propsSnap.Get("guid"), 10, 64) + bookmarkPath := fmt.Sprintf("%s#%s", fs.ToString(), ReplicationCursorBookmarkName) + propsBookmark, err := zfsGet(bookmarkPath, []string{"createtxg"}) + _, bookmarkNotExistErr := err.(*DatasetDoesNotExist) + if err != nil && !bookmarkNotExistErr { + return 0, err + } + if err == nil { + bookmarkTxg, err := strconv.ParseUint(propsBookmark.Get("createtxg"), 10, 64) + if err != nil { + return 0, errors.Wrap(err, "cannot parse bookmark createtxg") + } + snapTxg, err := strconv.ParseUint(propsSnap.Get("createtxg"), 10, 64) + if err != nil { + return 0, errors.Wrap(err, "cannot parse snapshot createtxg") + } + if snapTxg < bookmarkTxg { + return 0, errors.New("replication cursor can only be advanced, not set back") + } + if err := ZFSDestroy(bookmarkPath); err != nil { // FIXME make safer by using new temporary bookmark, then rename, possible with channel programs + return 0, err + } + } + if err := ZFSBookmark(fs, snapname, ReplicationCursorBookmarkName); err != nil { + return 0, err + } + return snapGuid, nil } diff --git a/zfs/zfs.go b/zfs/zfs.go index 9c4d6b7..22a4d49 100644 --- a/zfs/zfs.go +++ b/zfs/zfs.go @@ -490,7 +490,7 @@ func ZFSGet(fs *DatasetPath, props []string) (*ZFSProperties, error) { return zfsGet(fs.ToString(), props) } -var zfsGetDatasetDoesNotExistRegexp = regexp.MustCompile(`^cannot open '(\S+)': dataset does not exist`) +var zfsGetDatasetDoesNotExistRegexp = regexp.MustCompile(`^cannot open '(\S+)': (dataset does not exist|no such pool or dataset)`) type DatasetDoesNotExist struct { Path string From 1323a30a0c4b96b4e7235b871abb127ba32d6e67 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 5 Sep 2018 19:44:35 -0700 Subject: [PATCH 111/167] zfs: ability to specify sources for zfsGet fix use for Placeholder, leave rest as previous behavior --- zfs/diff.go | 2 +- zfs/replication_history.go | 4 ++-- zfs/zfs.go | 44 ++++++++++++++++++++++++++++++++------ zfs/zfs_test.go | 38 ++++++++++++++++++++++++++++++++ 4 files changed, 78 insertions(+), 10 deletions(-) diff --git a/zfs/diff.go b/zfs/diff.go index e7a7e0c..31886e3 100644 --- a/zfs/diff.go +++ b/zfs/diff.go @@ -248,7 +248,7 @@ func IsPlaceholder(p *DatasetPath, placeholderPropertyValue string) (isPlacehold // for nonexistent FS, isPlaceholder == false && err == nil func ZFSIsPlaceholderFilesystem(p *DatasetPath) (isPlaceholder bool, err error) { - props, err := ZFSGet(p, []string{ZREPL_PLACEHOLDER_PROPERTY_NAME}) + props, err := zfsGet(p.ToString(), []string{ZREPL_PLACEHOLDER_PROPERTY_NAME}, sourceLocal) if err == io.ErrUnexpectedEOF { // interpret this as an early exit of the zfs binary due to the fs not existing return false, nil diff --git a/zfs/replication_history.go b/zfs/replication_history.go index c5b46c3..2ad5abf 100644 --- a/zfs/replication_history.go +++ b/zfs/replication_history.go @@ -24,13 +24,13 @@ func ZFSGetReplicationCursor(fs *DatasetPath) (*FilesystemVersion, error) { func ZFSSetReplicationCursor(fs *DatasetPath, snapname string) (guid uint64, err error) { snapPath := fmt.Sprintf("%s@%s", fs.ToString(), snapname) - propsSnap, err := zfsGet(snapPath, []string{"createtxg", "guid"}) + propsSnap, err := zfsGet(snapPath, []string{"createtxg", "guid"}, sourceAny) if err != nil { return 0, err } snapGuid, err := strconv.ParseUint(propsSnap.Get("guid"), 10, 64) bookmarkPath := fmt.Sprintf("%s#%s", fs.ToString(), ReplicationCursorBookmarkName) - propsBookmark, err := zfsGet(bookmarkPath, []string{"createtxg"}) + propsBookmark, err := zfsGet(bookmarkPath, []string{"createtxg"}, sourceAny) _, bookmarkNotExistErr := err.(*DatasetDoesNotExist) if err != nil && !bookmarkNotExistErr { return 0, err diff --git a/zfs/zfs.go b/zfs/zfs.go index 22a4d49..8cd4fd2 100644 --- a/zfs/zfs.go +++ b/zfs/zfs.go @@ -487,7 +487,7 @@ func zfsSet(path string, props *ZFSProperties) (err error) { } func ZFSGet(fs *DatasetPath, props []string) (*ZFSProperties, error) { - return zfsGet(fs.ToString(), props) + return zfsGet(fs.ToString(), props, sourceAny) } var zfsGetDatasetDoesNotExistRegexp = regexp.MustCompile(`^cannot open '(\S+)': (dataset does not exist|no such pool or dataset)`) @@ -498,8 +498,30 @@ type DatasetDoesNotExist struct { func (d *DatasetDoesNotExist) Error() string { return fmt.Sprintf("dataset %q does not exist", d.Path) } -func zfsGet(path string, props []string) (*ZFSProperties, error) { - args := []string{"get", "-Hp", "-o", "property,value", strings.Join(props, ","), path} +type zfsPropertySource uint + +const ( + sourceLocal zfsPropertySource = 1 << iota + sourceDefault + sourceInherited + sourceNone + sourceTemporary + + sourceAny zfsPropertySource = ^zfsPropertySource(0) +) + +func (s zfsPropertySource) zfsGetSourceFieldPrefixes() []string { + prefixes := make([]string, 0, 5) + if s&sourceLocal != 0 {prefixes = append(prefixes, "local")} + if s&sourceDefault != 0 {prefixes = append(prefixes, "default")} + if s&sourceInherited != 0 {prefixes = append(prefixes, "inherited")} + if s&sourceNone != 0 {prefixes = append(prefixes, "-")} + if s&sourceTemporary != 0 { prefixes = append(prefixes, "temporary")} + return prefixes +} + +func zfsGet(path string, props []string, allowedSources zfsPropertySource) (*ZFSProperties, error) { + args := []string{"get", "-Hp", "-o", "property,value,source", strings.Join(props, ","), path} cmd := exec.Command(ZFS_BINARY, args...) stdout, err := cmd.Output() if err != nil { @@ -524,12 +546,20 @@ func zfsGet(path string, props []string) (*ZFSProperties, error) { res := &ZFSProperties{ make(map[string]string, len(lines)), } + allowedPrefixes := allowedSources.zfsGetSourceFieldPrefixes() for _, line := range lines[:len(lines)-1] { - fields := strings.Fields(line) - if len(fields) != 2 { - return nil, fmt.Errorf("zfs get did not return property value pairs") + fields := strings.FieldsFunc(line, func(r rune) bool { + return r == '\t' + }) + if len(fields) != 3 { + return nil, fmt.Errorf("zfs get did not return property,value,source tuples") + } + for _, p := range allowedPrefixes { + if strings.HasPrefix(fields[2],p) { + res.m[fields[0]] = fields[1] + break + } } - res.m[fields[0]] = fields[1] } return res, nil } diff --git a/zfs/zfs_test.go b/zfs/zfs_test.go index dcfe0f9..b46a36a 100644 --- a/zfs/zfs_test.go +++ b/zfs/zfs_test.go @@ -30,3 +30,41 @@ func TestDatasetPathTrimNPrefixComps(t *testing.T) { p.TrimNPrefixComps((1)) assert.True(t, p.Empty(), "empty trimming shouldn't do harm") } + +func TestZFSPropertySource(t *testing.T) { + + tcs := []struct{ + in zfsPropertySource + exp []string + }{ + { + in: sourceAny, + exp: []string{"local", "default", "inherited", "-", "temporary"}, + }, + { + in: sourceTemporary, + exp: []string{"temporary"}, + }, + { + in: sourceLocal|sourceInherited, + exp: []string{"local", "inherited"}, + }, + } + + toSet := func(in []string) map[string]struct{} { + m := make(map[string]struct{}, len(in)) + for _, s := range in { + m[s] = struct{}{} + } + return m + } + + for _, tc := range tcs { + + res := tc.in.zfsGetSourceFieldPrefixes() + resSet := toSet(res) + expSet := toSet(tc.exp) + assert.Equal(t, expSet, resSet) + } + +} From 2c25f28972bf81adfcb2cadbd33b84e905df5d25 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 5 Sep 2018 19:49:26 -0700 Subject: [PATCH 112/167] simplify mapping & filtering in endpoints (re-rooting only) --- daemon/job/push.go | 2 +- daemon/job/sink.go | 24 ++++--- endpoint/endpoint.go | 151 +++++++++++++++++++++---------------------- 3 files changed, 89 insertions(+), 88 deletions(-) diff --git a/daemon/job/push.go b/daemon/job/push.go index ee9d00d..36159b4 100644 --- a/daemon/job/push.go +++ b/daemon/job/push.go @@ -114,7 +114,7 @@ func (j *Push) do(ctx context.Context) { } defer client.Close(ctx) - sender := endpoint.NewSender(j.fsfilter, filters.NewAnyFSVFilter()) + sender := endpoint.NewSender(j.fsfilter) receiver := endpoint.NewRemote(client) j.mtx.Lock() diff --git a/daemon/job/sink.go b/daemon/job/sink.go index e69a7fb..6e308ff 100644 --- a/daemon/job/sink.go +++ b/daemon/job/sink.go @@ -5,18 +5,18 @@ import ( "github.com/pkg/errors" "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/config" - "github.com/zrepl/zrepl/daemon/filters" "github.com/zrepl/zrepl/daemon/logging" "github.com/zrepl/zrepl/daemon/serve" "github.com/zrepl/zrepl/endpoint" "path" + "github.com/zrepl/zrepl/zfs" ) type Sink struct { name string l serve.ListenerFactory rpcConf *streamrpc.ConnConfig - rootDataset string + rootDataset *zfs.DatasetPath } func SinkFromConfig(g *config.Global, in *config.SinkJob) (s *Sink, err error) { @@ -26,10 +26,14 @@ func SinkFromConfig(g *config.Global, in *config.SinkJob) (s *Sink, err error) { return nil, errors.Wrap(err, "cannot build server") } - if in.RootDataset == "" { - return nil, errors.Wrap(err, "must specify root dataset") + s.rootDataset, err = zfs.NewDatasetPath(in.RootDataset) + if err != nil { + return nil, errors.New("root dataset is not a valid zfs filesystem path") } - s.rootDataset = in.RootDataset + if s.rootDataset.Length() <= 0 { + return nil, errors.New("root dataset must not be empty") // duplicates error check of receiver + } + return s, nil } @@ -89,18 +93,18 @@ func (j *Sink) handleConnection(ctx context.Context, conn serve.AuthenticatedCon Info("handling connection") defer log.Info("finished handling connection") - clientRoot := path.Join(j.rootDataset, conn.ClientIdentity()) - log.WithField("client_root", clientRoot).Debug("client root") - fsmap := filters.NewDatasetMapFilter(1, false) - if err := fsmap.Add("<", clientRoot); err != nil { + clientRootStr := path.Join(j.rootDataset.ToString(), conn.ClientIdentity()) + clientRoot, err := zfs.NewDatasetPath(clientRootStr) + if err != nil { log.WithError(err). WithField("client_identity", conn.ClientIdentity()). Error("cannot build client filesystem map (client identity must be a valid ZFS FS name") } + log.WithField("client_root", clientRoot).Debug("client root") ctx = logging.WithSubsystemLoggers(ctx, log) - local, err := endpoint.NewReceiver(fsmap, filters.NewAnyFSVFilter()) + local, err := endpoint.NewReceiver(clientRoot) if err != nil { log.WithError(err).Error("unexpected error: cannot convert mapping to filter") return diff --git a/endpoint/endpoint.go b/endpoint/endpoint.go index bd3e2f7..bb1a051 100644 --- a/endpoint/endpoint.go +++ b/endpoint/endpoint.go @@ -17,11 +17,28 @@ import ( // Sender implements replication.ReplicationEndpoint for a sending side type Sender struct { FSFilter zfs.DatasetFilter - FilesystemVersionFilter zfs.FilesystemVersionFilter } -func NewSender(fsf zfs.DatasetFilter, fsvf zfs.FilesystemVersionFilter) *Sender { - return &Sender{fsf, fsvf} +func NewSender(fsf zfs.DatasetFilter) *Sender { + return &Sender{FSFilter: fsf} +} + +func (s *Sender) filterCheckFS(fs string) (*zfs.DatasetPath, error) { + dp, err := zfs.NewDatasetPath(fs) + if err != nil { + return nil, err + } + if dp.Length() == 0 { + return nil, errors.New("empty filesystem not allowed") + } + pass, err := s.FSFilter.Filter(dp) + if err != nil { + return nil, err + } + if !pass { + return nil, replication.NewFilteredError(fs) + } + return dp, nil } func (p *Sender) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) { @@ -40,18 +57,11 @@ func (p *Sender) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) } func (p *Sender) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) { - dp, err := zfs.NewDatasetPath(fs) + lp, err := p.filterCheckFS(fs) if err != nil { return nil, err } - pass, err := p.FSFilter.Filter(dp) - if err != nil { - return nil, err - } - if !pass { - return nil, replication.NewFilteredError(fs) - } - fsvs, err := zfs.ZFSListFilesystemVersions(dp, p.FilesystemVersionFilter) + fsvs, err := zfs.ZFSListFilesystemVersions(lp, nil) if err != nil { return nil, err } @@ -63,17 +73,10 @@ func (p *Sender) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu. } func (p *Sender) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.ReadCloser, error) { - dp, err := zfs.NewDatasetPath(r.Filesystem) + _, err := p.filterCheckFS(r.Filesystem) if err != nil { return nil, nil, err } - pass, err := p.FSFilter.Filter(dp) - if err != nil { - return nil, nil, err - } - if !pass { - return nil, nil, replication.NewFilteredError(r.Filesystem) - } if r.DryRun { size, err := zfs.ZFSSendDry(r.Filesystem, r.From, r.To) @@ -94,33 +97,18 @@ func (p *Sender) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.Rea } func (p *Sender) DestroySnapshots(ctx context.Context, req *pdu.DestroySnapshotsReq) (*pdu.DestroySnapshotsRes, error) { - dp, err := zfs.NewDatasetPath(req.Filesystem) + dp, err := p.filterCheckFS(req.Filesystem) if err != nil { return nil, err } - pass, err := p.FSFilter.Filter(dp) - if err != nil { - return nil, err - } - if !pass { - return nil, replication.NewFilteredError(req.Filesystem) - } - return doDestroySnapshots(ctx, dp, req.Snapshots) } func (p *Sender) ReplicationCursor(ctx context.Context, req *pdu.ReplicationCursorReq) (*pdu.ReplicationCursorRes, error) { - dp, err := zfs.NewDatasetPath(req.Filesystem) + dp, err := p.filterCheckFS(req.Filesystem) if err != nil { return nil, err } - pass, err := p.FSFilter.Filter(dp) - if err != nil { - return nil, err - } - if !pass { - return nil, replication.NewFilteredError(req.Filesystem) - } switch op := req.Op.(type) { case *pdu.ReplicationCursorReq_Get: @@ -143,12 +131,12 @@ func (p *Sender) ReplicationCursor(ctx context.Context, req *pdu.ReplicationCurs } } -type FSFilter interface { +type FSFilter interface { // FIXME unused Filter(path *zfs.DatasetPath) (pass bool, err error) } // FIXME: can we get away without error types here? -type FSMap interface { +type FSMap interface { // FIXME unused FSFilter Map(path *zfs.DatasetPath) (*zfs.DatasetPath, error) Invert() (FSMap, error) @@ -157,49 +145,72 @@ type FSMap interface { // Receiver implements replication.ReplicationEndpoint for a receiving side type Receiver struct { - fsmapInv FSMap - fsmap FSMap - fsvf zfs.FilesystemVersionFilter + root *zfs.DatasetPath } -func NewReceiver(fsmap FSMap, fsvf zfs.FilesystemVersionFilter) (*Receiver, error) { - fsmapInv, err := fsmap.Invert() +func NewReceiver(rootDataset *zfs.DatasetPath) (*Receiver, error) { + if rootDataset.Length() <= 0 { + return nil, errors.New("root dataset must not be an empty path") + } + return &Receiver{root: rootDataset.Copy()}, nil +} + +type subroot struct { + localRoot *zfs.DatasetPath +} + +var _ zfs.DatasetFilter = subroot{} + +// Filters local p +func (f subroot) Filter(p *zfs.DatasetPath) (pass bool, err error) { + return p.HasPrefix(f.localRoot) && !p.Equal(f.localRoot), nil +} + +func (f subroot) MapToLocal(fs string) (*zfs.DatasetPath, error) { + p, err := zfs.NewDatasetPath(fs) if err != nil { return nil, err } - return &Receiver{fsmapInv, fsmap, fsvf}, nil + if p.Length() == 0 { + return nil, errors.Errorf("cannot map empty filesystem") + } + c := f.localRoot.Copy() + c.Extend(p) + return c, nil } func (e *Receiver) ListFilesystems(ctx context.Context) ([]*pdu.Filesystem, error) { - filtered, err := zfs.ZFSListMapping(e.fsmapInv.AsFilter()) + filtered, err := zfs.ZFSListMapping(subroot{e.root}) if err != nil { - return nil, errors.Wrap(err, "error checking client permission") + return nil, err } - fss := make([]*pdu.Filesystem, len(filtered)) - for i, a := range filtered { - mapped, err := e.fsmapInv.Map(a) + // present without prefix, and only those that are not placeholders + fss := make([]*pdu.Filesystem, 0, len(filtered)) + for _, a := range filtered { + ph, err := zfs.ZFSIsPlaceholderFilesystem(a) if err != nil { - return nil, err + getLogger(ctx). + WithError(err). + WithField("fs", a). + Error("inconsistent placeholder property") + return nil, errors.New("server error, see logs") // don't leak path } - fss[i] = &pdu.Filesystem{Path: mapped.ToString()} + if ph { + continue + } + a.TrimPrefix(e.root) + fss = append(fss, &pdu.Filesystem{Path: a.ToString()}) } return fss, nil } func (e *Receiver) ListFilesystemVersions(ctx context.Context, fs string) ([]*pdu.FilesystemVersion, error) { - p, err := zfs.NewDatasetPath(fs) + lp, err := subroot{e.root}.MapToLocal(fs) if err != nil { return nil, err } - lp, err := e.fsmap.Map(p) - if err != nil { - return nil, err - } - if lp == nil { - return nil, errors.New("access to filesystem denied") - } - fsvs, err := zfs.ZFSListFilesystemVersions(lp, e.fsvf) + fsvs, err := zfs.ZFSListFilesystemVersions(lp, nil) if err != nil { return nil, err } @@ -215,17 +226,10 @@ func (e *Receiver) ListFilesystemVersions(ctx context.Context, fs string) ([]*pd func (e *Receiver) Receive(ctx context.Context, req *pdu.ReceiveReq, sendStream io.ReadCloser) error { defer sendStream.Close() - p, err := zfs.NewDatasetPath(req.Filesystem) + lp, err := subroot{e.root}.MapToLocal(req.Filesystem) if err != nil { return err } - lp, err := e.fsmap.Map(p) - if err != nil { - return err - } - if lp == nil { - return errors.New("receive to filesystem denied") - } getLogger(ctx).Debug("incoming Receive") @@ -286,17 +290,10 @@ func (e *Receiver) Receive(ctx context.Context, req *pdu.ReceiveReq, sendStream } func (e *Receiver) DestroySnapshots(ctx context.Context, req *pdu.DestroySnapshotsReq) (*pdu.DestroySnapshotsRes, error) { - dp, err := zfs.NewDatasetPath(req.Filesystem) + lp, err := subroot{e.root}.MapToLocal(req.Filesystem) if err != nil { return nil, err } - lp, err := e.fsmap.Map(dp) - if err != nil { - return nil, err - } - if lp == nil { - return nil, errors.New("access to filesystem denied") - } return doDestroySnapshots(ctx, lp, req.Snapshots) } From 0f75677e591d99c0695dc6dd9a8b520558ec47ed Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 5 Sep 2018 21:47:44 -0700 Subject: [PATCH 113/167] daemon/pruner: fix exercise (don't call it test) --- daemon/pruner/pruner.go | 27 +++++++++++++++++++-------- daemon/pruner/pruner_test.go | 12 +++++++++--- 2 files changed, 28 insertions(+), 11 deletions(-) diff --git a/daemon/pruner/pruner.go b/daemon/pruner/pruner.go index e3d5c8c..64e6487 100644 --- a/daemon/pruner/pruner.go +++ b/daemon/pruner/pruner.go @@ -270,11 +270,22 @@ fsloop: l := GetLogger(ctx).WithField("fs", tfs.Path) l.Debug("plan filesystem") + + pfs := &fs{ + path: tfs.Path, + } + pfss[i] = pfs + tfsvs, err := target.ListFilesystemVersions(ctx, tfs.Path) if err != nil { l.WithError(err).Error("cannot list filesystem versions") - return onErr(u, err) + if shouldRetry(err) { + return onErr(u, err) + } + pfs.err = err + continue fsloop } + pfs.snaps = make([]pruning.Snapshot, 0, len(tfsvs)) rcReq := &pdu.ReplicationCursorReq{ Filesystem: tfs.Path, @@ -283,18 +294,18 @@ fsloop: rc, err := receiver.ReplicationCursor(ctx, rcReq) if err != nil { l.WithError(err).Error("cannot get replication cursor") - return onErr(u, err) + if shouldRetry(err) { + return onErr(u, err) + } + pfs.err = err + continue fsloop } if rc.GetError() != "" { l.WithField("reqErr", rc.GetError()).Error("cannot get replication cursor") - return onErr(u, fmt.Errorf("%s", rc.GetError())) + pfs.err = fmt.Errorf("%s", rc.GetError()) + continue fsloop } - pfs := &fs{ - path: tfs.Path, - snaps: make([]pruning.Snapshot, 0, len(tfsvs)), - } - pfss[i] = pfs // scan from older to newer, all snapshots older than cursor are interpreted as replicated sort.Slice(tfsvs, func(i, j int) bool { diff --git a/daemon/pruner/pruner_test.go b/daemon/pruner/pruner_test.go index dea3d5f..a3e1656 100644 --- a/daemon/pruner/pruner_test.go +++ b/daemon/pruner/pruner_test.go @@ -30,6 +30,7 @@ func (m *mockFS) FilesystemVersions() []*pdu.FilesystemVersion { Type: pdu.FilesystemVersion_Snapshot, Name: v, Creation: pdu.FilesystemVersionCreation(time.Unix(0, 0)), + Guid: uint64(i), } } return versions @@ -89,18 +90,23 @@ func (t *mockTarget) DestroySnapshots(ctx context.Context, req *pdu.DestroySnaps return &pdu.DestroySnapshotsRes{Results: res}, nil } +type mockCursor struct { + snapname string + guid uint64 +} type mockHistory struct { errs map[string][]error + cursors map[string]*mockCursor } -func (r *mockHistory) SnapshotReplicationStatus(ctx context.Context, req *pdu.SnapshotReplicationStatusReq) (*pdu.SnapshotReplicationStatusRes, error) { +func (r *mockHistory) ReplicationCursor(ctx context.Context, req *pdu.ReplicationCursorReq) (*pdu.ReplicationCursorRes, error) { fs := req.Filesystem if len(r.errs[fs]) > 0 { e := r.errs[fs][0] r.errs[fs] = r.errs[fs][1:] return nil, e } - return &pdu.SnapshotReplicationStatusRes{Status: pdu.SnapshotReplicationStatusRes_Nonexistent}, nil + return &pdu.ReplicationCursorRes{Result: &pdu.ReplicationCursorRes_Guid{Guid: 0}}, nil } type stubNetErr struct { @@ -195,7 +201,7 @@ func TestPruner_Prune(t *testing.T) { exp := map[string][]string{ "zroot/bar": {"drop_g"}, // drop_c is prohibited by failing destroy - // drop_i is prohibiteed by failing WasSnapshotReplicated call + // drop_i is prohibiteed by failing ReplicationCursor call } assert.Equal(t, exp, target.destroyed) From 82d51cd0dc804481c345646f0d4874ec0bcc3d7d Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 5 Sep 2018 21:48:52 -0700 Subject: [PATCH 114/167] go vet fix --- endpoint/endpoint.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/endpoint/endpoint.go b/endpoint/endpoint.go index bb1a051..863e960 100644 --- a/endpoint/endpoint.go +++ b/endpoint/endpoint.go @@ -119,7 +119,7 @@ func (p *Sender) ReplicationCursor(ctx context.Context, req *pdu.ReplicationCurs if cursor == nil { return &pdu.ReplicationCursorRes{Result: &pdu.ReplicationCursorRes_Error{Error: "cursor does not exist"}}, nil } - return &pdu.ReplicationCursorRes{Result: &pdu.ReplicationCursorRes_Guid{cursor.Guid}}, nil + return &pdu.ReplicationCursorRes{Result: &pdu.ReplicationCursorRes_Guid{Guid: cursor.Guid}}, nil case *pdu.ReplicationCursorReq_Set: guid, err := zfs.ZFSSetReplicationCursor(dp, op.Set.Snapshot) if err != nil { From c60ed78bc5b484c9401989ac46fd6d18d369c338 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 6 Sep 2018 11:05:32 -0700 Subject: [PATCH 115/167] status subcommand: only draw one big progress bar of the entire replication more details on progress per step in text form --- client/status.go | 141 ++++++++++++++++++++++++++++++----------------- 1 file changed, 90 insertions(+), 51 deletions(-) diff --git a/client/status.go b/client/status.go index e19d474..a039f04 100644 --- a/client/status.go +++ b/client/status.go @@ -169,50 +169,74 @@ func (t *tui) draw() { t.newline() continue } + + + all := make([]*fsrep.Report, 0, len(rep.Completed)+len(rep.Pending) + 1) + all = append(all, rep.Completed...) + all = append(all, rep.Pending...) + if rep.Active != nil { + all = append(all, rep.Active) + } + sort.Slice(all, func(i, j int) bool { + return all[i].Filesystem < all[j].Filesystem + }) + t.printf("Status: %s", rep.Status) t.newline() if rep.Problem != "" { t.printf("Problem: %s", rep.Problem) t.newline() } - - maxNameLength := calculateMaxFilesystemAndVersionNameLength(&rep) - - for _, fs := range rep.Completed { - printFilesystem(fs, t, false, maxNameLength) - } - if rep.Active != nil { - printFilesystem(rep.Active, t, true, maxNameLength) - } - for _, fs := range rep.Pending { - printFilesystem(fs, t, false, maxNameLength) + { // Progress: [---------------] + sumUpFSRep := func(rep *fsrep.Report) (transferred, total int64) { + for _, s := range rep.Pending { + transferred += s.Bytes + total += s.ExpectedBytes + } + for _, s := range rep.Completed { + transferred += s.Bytes + total += s.ExpectedBytes + } + return + } + var transferred, total int64 + for _, fs := range all { + fstx, fstotal := sumUpFSRep(fs) + transferred += fstx + total += fstotal + } + t.write("Progress: ") + t.drawBar(80, transferred, total) + t.write(fmt.Sprintf(" %s / %s", ByteCountBinary(transferred), ByteCountBinary(total))) + t.newline() } + maxFS, maxStatus := calculateMaxFilesystemAndVersionNameLength(all) + for _, fs := range all { + printFilesystemStatus(fs, t, fs == rep.Active, maxFS, maxStatus) + } } } termbox.Flush() } const snapshotIndent = 1 -func calculateMaxFilesystemAndVersionNameLength(report *replication.Report) int { - all := append(report.Completed, report.Active) - all = append(all, report.Pending...) - maxLen := 0 +func calculateMaxFilesystemAndVersionNameLength(all []*fsrep.Report) (maxFS, maxStatus int) { for _, e := range all { - if e == nil { //active can be nil - continue + if len(e.Filesystem) > maxFS { + maxFS = len(e.Filesystem) } - if len(e.Filesystem) > maxLen { - maxLen = len(e.Filesystem) - } - all2 := append(e.Pending, e.Completed...) + all2 := make([]*fsrep.StepReport, 0, len(e.Pending) + len(e.Completed)) + all2 = append(all2, e.Pending...) + all2 = append(all2, e.Completed...) for _, e2 := range all2 { - if len(e2.To) + snapshotIndent > maxLen { - maxLen = len(e2.To) + snapshotIndent + elen := len(e2.Problem) + len(e2.From) + len(e2.To) + 60 // random spacing, units, labels, etc + if elen > maxStatus { + maxStatus = elen } } } - return maxLen + return } func times(str string, n int) (out string) { @@ -229,13 +253,15 @@ func rightPad(str string, length int, pad string) string { return str + times(pad, length-len(str)) } -func (t *tui) drawBar(name string, maxNameLength int, status string, bytes int64, totalBytes int64) { - t.write(rightPad(name, maxNameLength + 1, " ")) - t.write(" ") - t.write(rightPad(status, 14, " ")) - t.write(" ") - length := 50 +func leftPad(str string, length int, pad string) string { + if len(str) > length { + return str[len(str)-length:] + } + return times(pad, length-len(str)) + str +} + +func (t *tui) drawBar(length int, bytes, totalBytes int64) { var completedLength int if totalBytes > 0 { completedLength = int(int64(length) * bytes / totalBytes) @@ -249,15 +275,6 @@ func (t *tui) drawBar(name string, maxNameLength int, status string, bytes int64 t.write(">") t.write(times("-", length-completedLength)) t.write("]") - - t.write(" ") - totalBytesStr := ByteCountBinary(totalBytes) - if totalBytes == 0 { - totalBytesStr = "??? B" - } - t.write(rightPad(ByteCountBinary(bytes)+"/"+totalBytesStr, 20, " ")) - - t.newline() } func StringStepState(s fsrep.StepState) string { @@ -273,9 +290,9 @@ func StringStepState(s fsrep.StepState) string { } } -func printFilesystem(rep *fsrep.Report, t *tui, versions bool, maxNameLength int) { - bytes := int64(0) - totalBytes := int64(0) +func filesystemStatusString(rep *fsrep.Report, active bool, maxFS, maxStatus int) (totalStatus string, bytes, totalBytes int64) { + bytes = int64(0) + totalBytes = int64(0) for _, s := range rep.Pending { bytes += s.Bytes totalBytes += s.ExpectedBytes @@ -285,17 +302,39 @@ func printFilesystem(rep *fsrep.Report, t *tui, versions bool, maxNameLength int totalBytes += s.ExpectedBytes } - t.drawBar(rep.Filesystem, maxNameLength, rep.Status, bytes, totalBytes) - if rep.Problem != "" { - t.addIndent(1) - t.printf("Problem: %s", rep.Problem) - t.newline() - t.addIndent(-1) + nextStep := "" + if len(rep.Pending) > 0 { + if rep.Pending[0].From != "" { + nextStep = fmt.Sprintf("%s => %s", rep.Pending[0].From, rep.Pending[0].To) + } else { + nextStep = fmt.Sprintf("%s (full)", rep.Pending[0].To) + } } - if versions && len(rep.Pending) > 0 { - v := rep.Pending[0] - t.drawBar(times(" ", snapshotIndent) + v.To, maxNameLength, StringStepState(v.Status), v.Bytes, v.ExpectedBytes) + status := fmt.Sprintf("%s (step %d/%d, %s/%s)", + rep.Status, + len(rep.Completed), len(rep.Pending) + len(rep.Completed), + ByteCountBinary(bytes), ByteCountBinary(totalBytes), + ) + if rep.Problem == "" && nextStep != "" { + status += fmt.Sprintf(" next: %s", nextStep) + } else if rep.Problem != "" { + status += fmt.Sprintf(" problem: %s", rep.Problem) } + activeIndicator := " " + if active { + activeIndicator = "*" + } + totalStatus = fmt.Sprintf("%s %s %s", + activeIndicator, + rightPad(rep.Filesystem, maxFS, " "), + rightPad(status, maxStatus, " ")) + return totalStatus, bytes, totalBytes +} + +func printFilesystemStatus(rep *fsrep.Report, t *tui, active bool, maxFS, maxTotal int) { + totalStatus, _, _ := filesystemStatusString(rep, active, maxFS, maxTotal) + t.write(totalStatus) + t.newline() } func ByteCountBinary(b int64) string { From 1edf020ce7844e8ee112f8140d15f994124f51ba Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 6 Sep 2018 11:46:02 -0700 Subject: [PATCH 116/167] status command: better handling of 'nothing to do' Complete state --- client/status.go | 45 +++++++++++++++++++++++++-------------------- 1 file changed, 25 insertions(+), 20 deletions(-) diff --git a/client/status.go b/client/status.go index a039f04..2b94f35 100644 --- a/client/status.go +++ b/client/status.go @@ -211,9 +211,14 @@ func (t *tui) draw() { t.newline() } - maxFS, maxStatus := calculateMaxFilesystemAndVersionNameLength(all) + var maxFSLen int for _, fs := range all { - printFilesystemStatus(fs, t, fs == rep.Active, maxFS, maxStatus) + if len(fs.Filesystem) > maxFSLen { + maxFSLen = len(fs.Filesystem) + } + } + for _, fs := range all { + printFilesystemStatus(fs, t, fs == rep.Active, maxFSLen) } } } @@ -221,7 +226,7 @@ func (t *tui) draw() { } const snapshotIndent = 1 -func calculateMaxFilesystemAndVersionNameLength(all []*fsrep.Report) (maxFS, maxStatus int) { +func calculateMaxFSLength(all []*fsrep.Report) (maxFS, maxStatus int) { for _, e := range all { if len(e.Filesystem) > maxFS { maxFS = len(e.Filesystem) @@ -268,6 +273,8 @@ func (t *tui) drawBar(length int, bytes, totalBytes int64) { if completedLength > length { completedLength = length } + } else if totalBytes == bytes { + completedLength = length } t.write("[") @@ -290,7 +297,7 @@ func StringStepState(s fsrep.StepState) string { } } -func filesystemStatusString(rep *fsrep.Report, active bool, maxFS, maxStatus int) (totalStatus string, bytes, totalBytes int64) { +func filesystemStatusString(rep *fsrep.Report, active bool, fsWidth int) (line string, bytes, totalBytes int64) { bytes = int64(0) totalBytes = int64(0) for _, s := range rep.Pending { @@ -302,37 +309,35 @@ func filesystemStatusString(rep *fsrep.Report, active bool, maxFS, maxStatus int totalBytes += s.ExpectedBytes } - nextStep := "" - if len(rep.Pending) > 0 { + next := "" + if rep.Problem != "" { + next = " problem: " + rep.Problem + } else if len(rep.Pending) > 0 { if rep.Pending[0].From != "" { - nextStep = fmt.Sprintf("%s => %s", rep.Pending[0].From, rep.Pending[0].To) + next = fmt.Sprintf("next: %s => %s", rep.Pending[0].From, rep.Pending[0].To) } else { - nextStep = fmt.Sprintf("%s (full)", rep.Pending[0].To) + next = fmt.Sprintf("next: %s (full)", rep.Pending[0].To) } } - status := fmt.Sprintf("%s (step %d/%d, %s/%s)", + status := fmt.Sprintf("%s (step %d/%d, %s/%s)%s", rep.Status, len(rep.Completed), len(rep.Pending) + len(rep.Completed), ByteCountBinary(bytes), ByteCountBinary(totalBytes), + next, ) - if rep.Problem == "" && nextStep != "" { - status += fmt.Sprintf(" next: %s", nextStep) - } else if rep.Problem != "" { - status += fmt.Sprintf(" problem: %s", rep.Problem) - } activeIndicator := " " if active { activeIndicator = "*" } - totalStatus = fmt.Sprintf("%s %s %s", + line = fmt.Sprintf("%s %s %s", activeIndicator, - rightPad(rep.Filesystem, maxFS, " "), - rightPad(status, maxStatus, " ")) - return totalStatus, bytes, totalBytes + rightPad(rep.Filesystem, fsWidth, " "), + status) + return line, bytes, totalBytes } -func printFilesystemStatus(rep *fsrep.Report, t *tui, active bool, maxFS, maxTotal int) { - totalStatus, _, _ := filesystemStatusString(rep, active, maxFS, maxTotal) +func printFilesystemStatus(rep *fsrep.Report, t *tui, active bool, maxFS int) { + totalStatus, _, _ := filesystemStatusString(rep, active, maxFS) t.write(totalStatus) t.newline() } From 7836ea36fc47717b1af39033a005122dc6d1e43f Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 6 Sep 2018 13:34:39 -0700 Subject: [PATCH 117/167] serve TLS: validate client CNs against whitelist in config file --- daemon/serve/serve_tls.go | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/daemon/serve/serve_tls.go b/daemon/serve/serve_tls.go index 9172b32..bc95e41 100644 --- a/daemon/serve/serve_tls.go +++ b/daemon/serve/serve_tls.go @@ -3,6 +3,7 @@ package serve import ( "crypto/tls" "crypto/x509" + "fmt" "github.com/pkg/errors" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/tlsconf" @@ -16,6 +17,7 @@ type TLSListenerFactory struct { clientCA *x509.CertPool serverCert tls.Certificate handshakeTimeout time.Duration + clientCNs map[string]struct{} } func TLSListenerFactoryFromConfig(c *config.Global, in *config.TLSServe) (lf *TLSListenerFactory, err error) { @@ -38,6 +40,15 @@ func TLSListenerFactoryFromConfig(c *config.Global, in *config.TLSServe) (lf *TL return nil, errors.Wrap(err, "cannot parse cer/key pair") } + lf.clientCNs = make(map[string]struct{}, len(in.ClientCNs)) + for i, cn := range in.ClientCNs { + if err := ValidateClientIdentity(cn); err != nil { + return nil, errors.Wrapf(err, "unsuitable client_cn #%d %q", i, cn) + } + // dupes are ok fr now + lf.clientCNs[cn] = struct{}{} + } + return lf, nil } @@ -47,11 +58,12 @@ func (f *TLSListenerFactory) Listen() (AuthenticatedListener, error) { return nil, err } tl := tlsconf.NewClientAuthListener(l, f.clientCA, f.serverCert, f.handshakeTimeout) - return tlsAuthListener{tl}, nil + return tlsAuthListener{tl, f.clientCNs}, nil } type tlsAuthListener struct { *tlsconf.ClientAuthListener + clientCNs map[string]struct{} } func (l tlsAuthListener) Accept(ctx context.Context) (AuthenticatedConn, error) { @@ -59,6 +71,12 @@ func (l tlsAuthListener) Accept(ctx context.Context) (AuthenticatedConn, error) if err != nil { return nil, err } + if _, ok := l.clientCNs[cn]; !ok { + if err := c.Close(); err != nil { + getLogger(ctx).WithError(err).Error("error closing connection with unauthorized common name") + } + return nil, fmt.Errorf("unauthorized client common name %q from %s", cn, c.RemoteAddr()) + } return authConn{c, cn}, nil } From bf5099baacb807b7661935ac300fa8550a505bda Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 6 Sep 2018 23:52:11 -0700 Subject: [PATCH 118/167] version subcommand: unified client & server --- client/version.go | 57 +++++++++++++++++++++++++++++++++++++++++++++++ main.go | 15 +++++++++++++ 2 files changed, 72 insertions(+) create mode 100644 client/version.go diff --git a/client/version.go b/client/version.go new file mode 100644 index 0000000..e584a80 --- /dev/null +++ b/client/version.go @@ -0,0 +1,57 @@ +package client + +import ( + "fmt" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon" + "github.com/zrepl/zrepl/version" + "os" +) + +type VersionArgs struct { + Show string + Config *config.Config +} + +func RunVersion(args VersionArgs) { + + die := func() { + fmt.Fprintf(os.Stderr, "exiting after error\n") + os.Exit(1) + } + + if args.Show != "daemon" && args.Show != "client" && args.Show != "" { + fmt.Fprintf(os.Stderr, "show flag must be 'client' or 'server' or be left empty") + die() + } + + var clientVersion, daemonVersion *version.ZreplVersionInformation + if args.Show == "client" || args.Show == "" { + clientVersion = version.NewZreplVersionInformation() + fmt.Printf("client: %s\n", clientVersion.String()) + } + if args.Show == "daemon" || args.Show == "" { + + httpc, err := controlHttpClient(args.Config.Global.Control.SockPath) + if err != nil { + fmt.Fprintf(os.Stderr, "server: error: %s\n", err) + die() + } + + var info version.ZreplVersionInformation + err = jsonRequestResponse(httpc, daemon.ControlJobEndpointVersion, "", &info) + if err != nil { + fmt.Fprintf(os.Stderr, "server: error: %s\n", err) + die() + } + daemonVersion = &info + fmt.Printf("server: %s\n", daemonVersion.String()) + } + + if args.Show == "" { + if clientVersion.Version != daemonVersion.Version { + fmt.Fprintf(os.Stderr, "WARNING: client version != daemon version, restart zrepl daemon\n") + } + } + +} diff --git a/main.go b/main.go index 905e2f1..a41b080 100644 --- a/main.go +++ b/main.go @@ -100,6 +100,19 @@ var configcheckCmd = &cobra.Command{ }, } +var versionCmdArgs client.VersionArgs +var versionCmd = &cobra.Command{ + Use: "version", + Short: "print version of zrepl binary (for running daemon 'zrepl control version' command)", + Run: func(cmd *cobra.Command, args []string) { + conf, err := config.ParseConfig(rootArgs.configFile) + if err == nil { + versionCmdArgs.Config = conf + } + client.RunVersion(versionCmdArgs) + }, +} + var rootArgs struct { configFile string } @@ -113,6 +126,8 @@ func init() { rootCmd.AddCommand(stdinserverCmd) rootCmd.AddCommand(bashcompCmd) rootCmd.AddCommand(configcheckCmd) + versionCmd.Flags().StringVar(&versionCmdArgs.Show, "show", "", "version info to show (client|daemon)") + rootCmd.AddCommand(versionCmd) } func main() { From 0c2ac3a1684a9768e7d009e4f8711e1e80512160 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 7 Sep 2018 00:04:03 -0700 Subject: [PATCH 119/167] pprof subcommand --- client/pprof.go | 35 +++++++++++++++++++++++++++++++++++ main.go | 36 ++++++++++++++++++++++++++++++++++++ 2 files changed, 71 insertions(+) create mode 100644 client/pprof.go diff --git a/client/pprof.go b/client/pprof.go new file mode 100644 index 0000000..7afb4cb --- /dev/null +++ b/client/pprof.go @@ -0,0 +1,35 @@ +package client + +import ( + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon" + "log" + "os" +) + +type PProfArgs struct { + daemon.PprofServerControlMsg +} + +func RunPProf(conf *config.Config, args PProfArgs) { + log := log.New(os.Stderr, "", 0) + + die := func() { + log.Printf("exiting after error") + os.Exit(1) + } + + log.Printf("connecting to zrepl daemon") + + httpc, err := controlHttpClient(conf.Global.Control.SockPath) + if err != nil { + log.Printf("error creating http client: %s", err) + die() + } + err = jsonRequestResponse(httpc, daemon.ControlJobEndpointPProf, args.PprofServerControlMsg, struct{}{}) + if err != nil { + log.Printf("error sending control message: %s", err) + die() + } + log.Printf("finished") +} diff --git a/main.go b/main.go index a41b080..9ab031d 100644 --- a/main.go +++ b/main.go @@ -113,6 +113,41 @@ var versionCmd = &cobra.Command{ }, } +var pprofCmd = &cobra.Command{ + Use: "pprof off | [on TCP_LISTEN_ADDRESS]", + Short: "start a http server exposing go-tool-compatible profiling endpoints at TCP_LISTEN_ADDRESS", + RunE: func(cmd *cobra.Command, args []string) error { + conf, err := config.ParseConfig(rootArgs.configFile) + if err != nil { + return err + } + + var pprofCmdArgs client.PProfArgs + if cmd.Flags().NArg() < 1 { + goto enargs + } + switch cmd.Flags().Arg(0) { + case "on": + pprofCmdArgs.Run = true + if cmd.Flags().NArg() != 2 { + return errors.New("must specify TCP_LISTEN_ADDRESS as second positional argument") + } + pprofCmdArgs.HttpListenAddress = cmd.Flags().Arg(1) + case "off": + if cmd.Flags().NArg() != 1 { + goto enargs + } + pprofCmdArgs.Run = false + } + + client.RunPProf(conf, pprofCmdArgs) + return nil + enargs: + return errors.New("invalid number of positional arguments") + + }, +} + var rootArgs struct { configFile string } @@ -128,6 +163,7 @@ func init() { rootCmd.AddCommand(configcheckCmd) versionCmd.Flags().StringVar(&versionCmdArgs.Show, "show", "", "version info to show (client|daemon)") rootCmd.AddCommand(versionCmd) + rootCmd.AddCommand(pprofCmd) } func main() { From ab9446137f41bade1bd3a5e5c6469ceca26e349d Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 7 Sep 2018 22:03:51 -0700 Subject: [PATCH 120/167] fix missing import of errors pacakge --- main.go | 1 + 1 file changed, 1 insertion(+) diff --git a/main.go b/main.go index 9ab031d..f6e35aa 100644 --- a/main.go +++ b/main.go @@ -2,6 +2,7 @@ package main import ( + "errors" "github.com/spf13/cobra" "github.com/zrepl/zrepl/client" "github.com/zrepl/zrepl/config" From fa47667f315607bbb056c0df01b73dc5e45fd064 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 7 Sep 2018 22:03:41 -0700 Subject: [PATCH 121/167] bring back prometheus metrics, with new metrics for replication state machine --- daemon/control.go | 3 +++ daemon/daemon.go | 28 +++++++++++++++++++---- daemon/job/job.go | 2 ++ daemon/job/push.go | 42 ++++++++++++++++++++++++++++++---- daemon/job/sink.go | 3 +++ daemon/prometheus.go | 47 ++++++++++++++++++++++---------------- daemon/pruner/pruner.go | 8 ++++++- replication/fsrep/fsfsm.go | 13 ++++++++--- replication/mainfsm.go | 18 ++++++++++++--- zfs/prometheus.go | 26 ++++++++++----------- zfs/versions.go | 3 --- zfs/zfs.go | 15 ++++++++++++ 12 files changed, 156 insertions(+), 52 deletions(-) diff --git a/daemon/control.go b/daemon/control.go index 35a278c..34b879c 100644 --- a/daemon/control.go +++ b/daemon/control.go @@ -5,6 +5,7 @@ import ( "context" "encoding/json" "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/daemon/job" "github.com/zrepl/zrepl/daemon/nethelpers" "github.com/zrepl/zrepl/logger" @@ -35,6 +36,8 @@ func (j *controlJob) Name() string { return jobNameControl } func (j *controlJob) Status() interface{} { return nil } +func (j *controlJob) RegisterMetrics(registerer prometheus.Registerer) {} + const ( ControlJobEndpointPProf string = "/debug/pprof" ControlJobEndpointVersion string = "/version" diff --git a/daemon/daemon.go b/daemon/daemon.go index 078f1f4..91cc71b 100644 --- a/daemon/daemon.go +++ b/daemon/daemon.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon/job" "github.com/zrepl/zrepl/daemon/logging" @@ -59,9 +60,23 @@ func Run(conf *config.Config) error { } jobs.start(ctx, controlJob, true) - // start prometheus - //var promJob *prometheusJob // FIXME - //jobs.start(ctx, promJob, true) + for i, jc := range conf.Global.Monitoring { + var ( + job job.Job + err error + ) + switch v := jc.Ret.(type) { + case *config.PrometheusMonitoring: + job, err = newPrometheusJobFromConfig(v) + default: + return errors.Errorf("unknown monitoring job #%d (type %T)", i, v) + } + if err != nil { + return errors.Wrapf(err,"cannot build monitorin gjob #%d", i) + } + jobs.start(ctx, job, true) + } + log.Info("starting daemon") @@ -160,7 +175,9 @@ func (s *jobs) start(ctx context.Context, j job.Job, internal bool) { s.m.Lock() defer s.m.Unlock() - jobLog := job.GetLogger(ctx).WithField(logJobField, j.Name()) + jobLog := job.GetLogger(ctx). + WithField(logJobField, j.Name()). + WithOutlet(newPrometheusLogOutlet(j.Name()), logger.Debug) jobName := j.Name() if !internal && IsInternalJobName(jobName) { panic(fmt.Sprintf("internal job name used for non-internal job %s", jobName)) @@ -171,6 +188,9 @@ func (s *jobs) start(ctx context.Context, j job.Job, internal bool) { if _, ok := s.jobs[jobName]; ok { panic(fmt.Sprintf("duplicate job name %s", jobName)) } + + j.RegisterMetrics(prometheus.DefaultRegisterer) + s.jobs[jobName] = j ctx = job.WithLogger(ctx, jobLog) ctx, wakeupChan := job.WithWakeup(ctx) diff --git a/daemon/job/job.go b/daemon/job/job.go index 12f738e..3e338c0 100644 --- a/daemon/job/job.go +++ b/daemon/job/job.go @@ -3,6 +3,7 @@ package job import ( "context" "errors" + "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/logger" ) @@ -47,6 +48,7 @@ type Job interface { Name() string Run(ctx context.Context) Status() interface{} + RegisterMetrics(registerer prometheus.Registerer) } func WaitWakeup(ctx context.Context) <-chan struct{} { diff --git a/daemon/job/push.go b/daemon/job/push.go index 36159b4..496a44c 100644 --- a/daemon/job/push.go +++ b/daemon/job/push.go @@ -3,6 +3,7 @@ package job import ( "context" "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon/connecter" "github.com/zrepl/zrepl/daemon/filters" @@ -15,14 +16,18 @@ import ( ) type Push struct { - name string - clientFactory *connecter.ClientFactory - fsfilter endpoint.FSFilter + name string + clientFactory *connecter.ClientFactory + fsfilter endpoint.FSFilter prunerFactory *pruner.PrunerFactory snapper *snapper.Snapper + promRepStateSecs *prometheus.HistogramVec // labels: state + promPruneSecs *prometheus.HistogramVec // labels: prune_side + promBytesReplicated *prometheus.CounterVec // labels: filesystem + mtx sync.Mutex replication *replication.Replication } @@ -31,6 +36,20 @@ func PushFromConfig(g *config.Global, in *config.PushJob) (j *Push, err error) { j = &Push{} j.name = in.Name + j.promRepStateSecs = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "zrepl", + Subsystem: "replication", + Name: "state_time", + Help: "seconds spent during replication", + ConstLabels: prometheus.Labels{"zrepl_job":j.name}, + }, []string{"state"}) + j.promBytesReplicated = prometheus.NewCounterVec(prometheus.CounterOpts{ + Namespace: "zrepl", + Subsystem: "replication", + Name: "bytes_replicated", + Help: "number of bytes replicated from sender to receiver per filesystem", + ConstLabels: prometheus.Labels{"zrepl_job":j.name}, + }, []string{"filesystem"}) j.clientFactory, err = connecter.FromConfig(g, in.Connect) if err != nil { @@ -43,7 +62,14 @@ func PushFromConfig(g *config.Global, in *config.PushJob) (j *Push, err error) { } j.fsfilter = fsf - j.prunerFactory, err = pruner.NewPrunerFactory(in.Pruning) + j.promPruneSecs = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "zrepl", + Subsystem: "pruning", + Name: "time", + Help: "seconds spent in pruner", + ConstLabels: prometheus.Labels{"zrepl_job":j.name}, + }, []string{"prune_side"}) + j.prunerFactory, err = pruner.NewPrunerFactory(in.Pruning, j.promPruneSecs) if err != nil { return nil, err } @@ -55,6 +81,12 @@ func PushFromConfig(g *config.Global, in *config.PushJob) (j *Push, err error) { return j, nil } +func (j *Push) RegisterMetrics(registerer prometheus.Registerer) { + registerer.MustRegister(j.promRepStateSecs) + registerer.MustRegister(j.promPruneSecs) + registerer.MustRegister(j.promBytesReplicated) +} + func (j *Push) Name() string { return j.name } func (j *Push) Status() interface{} { @@ -118,7 +150,7 @@ func (j *Push) do(ctx context.Context) { receiver := endpoint.NewRemote(client) j.mtx.Lock() - j.replication = replication.NewReplication() + j.replication = replication.NewReplication(j.promRepStateSecs, j.promBytesReplicated) j.mtx.Unlock() log.Info("start replication") diff --git a/daemon/job/sink.go b/daemon/job/sink.go index 6e308ff..18c009c 100644 --- a/daemon/job/sink.go +++ b/daemon/job/sink.go @@ -4,6 +4,7 @@ import ( "context" "github.com/pkg/errors" "github.com/problame/go-streamrpc" + "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon/logging" "github.com/zrepl/zrepl/daemon/serve" @@ -45,6 +46,8 @@ func (*Sink) Status() interface{} { return nil } +func (*Sink) RegisterMetrics(registerer prometheus.Registerer) {} + func (j *Sink) Run(ctx context.Context) { log := GetLogger(ctx) diff --git a/daemon/prometheus.go b/daemon/prometheus.go index 3836624..b39910c 100644 --- a/daemon/prometheus.go +++ b/daemon/prometheus.go @@ -4,7 +4,9 @@ import ( "context" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" + "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon/job" + "github.com/zrepl/zrepl/logger" "github.com/zrepl/zrepl/zfs" "net" "net/http" @@ -14,37 +16,24 @@ type prometheusJob struct { listen string } -func newPrometheusJob(listen string) *prometheusJob { - return &prometheusJob{listen} +func newPrometheusJobFromConfig(in *config.PrometheusMonitoring) (*prometheusJob, error) { + if _, _, err := net.SplitHostPort(in.Listen); err != nil { + return nil, err + } + return &prometheusJob{in.Listen}, nil } var prom struct { - taskLastActiveStart *prometheus.GaugeVec - taskLastActiveDuration *prometheus.GaugeVec taskLogEntries *prometheus.CounterVec } func init() { - prom.taskLastActiveStart = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "zrepl", - Subsystem: "daemon", - Name: "task_last_active_start", - Help: "point in time at which the job task last left idle state", - }, []string{"zrepl_job", "job_type", "task"}) - prom.taskLastActiveDuration = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "zrepl", - Subsystem: "daemon", - Name: "task_last_active_duration", - Help: "seconds that the last run ob a job task spent between leaving and re-entering idle state", - }, []string{"zrepl_job", "job_type", "task"}) prom.taskLogEntries = prometheus.NewCounterVec(prometheus.CounterOpts{ Namespace: "zrepl", Subsystem: "daemon", - Name: "task_log_entries", + Name: "log_entries", Help: "number of log entries per job task and level", - }, []string{"zrepl_job", "job_type", "task", "level"}) - prometheus.MustRegister(prom.taskLastActiveStart) - prometheus.MustRegister(prom.taskLastActiveDuration) + }, []string{"zrepl_job", "level"}) prometheus.MustRegister(prom.taskLogEntries) } @@ -52,6 +41,8 @@ func (j *prometheusJob) Name() string { return jobNamePrometheus } func (j *prometheusJob) Status() interface{} { return nil } +func (j *prometheusJob) RegisterMetrics(registerer prometheus.Registerer) {} + func (j *prometheusJob) Run(ctx context.Context) { if err := zfs.PrometheusRegister(prometheus.DefaultRegisterer); err != nil { @@ -80,3 +71,19 @@ func (j *prometheusJob) Run(ctx context.Context) { } } + +type prometheusJobOutlet struct { + jobName string +} + +var _ logger.Outlet = prometheusJobOutlet{} + +func newPrometheusLogOutlet(jobName string) prometheusJobOutlet { + return prometheusJobOutlet{jobName} +} + +func (o prometheusJobOutlet) WriteEntry(entry logger.Entry) error { + prom.taskLogEntries.WithLabelValues(o.jobName, entry.Level.String()).Inc() + return nil +} + diff --git a/daemon/pruner/pruner.go b/daemon/pruner/pruner.go index 64e6487..013c161 100644 --- a/daemon/pruner/pruner.go +++ b/daemon/pruner/pruner.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/logger" "github.com/zrepl/zrepl/pruning" @@ -49,6 +50,7 @@ type args struct { rules []pruning.KeepRule retryWait time.Duration considerSnapAtCursorReplicated bool + promPruneSecs prometheus.Observer } type Pruner struct { @@ -72,6 +74,7 @@ type PrunerFactory struct { receiverRules []pruning.KeepRule retryWait time.Duration considerSnapAtCursorReplicated bool + promPruneSecs *prometheus.HistogramVec } func checkContainsKeep1(rules []pruning.KeepRule) error { @@ -87,7 +90,7 @@ func checkContainsKeep1(rules []pruning.KeepRule) error { return errors.New("sender keep rules must contain last_n or be empty so that the last snapshot is definitely kept") } -func NewPrunerFactory(in config.PruningSenderReceiver) (*PrunerFactory, error) { +func NewPrunerFactory(in config.PruningSenderReceiver, promPruneSecs *prometheus.HistogramVec) (*PrunerFactory, error) { keepRulesReceiver, err := pruning.RulesFromConfig(in.KeepReceiver) if err != nil { return nil, errors.Wrap(err, "cannot build receiver pruning rules") @@ -111,6 +114,7 @@ func NewPrunerFactory(in config.PruningSenderReceiver) (*PrunerFactory, error) { keepRulesReceiver, 10 * time.Second, //FIXME constant considerSnapAtCursorReplicated, + promPruneSecs, } return f, nil } @@ -124,6 +128,7 @@ func (f *PrunerFactory) BuildSenderPruner(ctx context.Context, target Target, re f.senderRules, f.retryWait, f.considerSnapAtCursorReplicated, + f.promPruneSecs.WithLabelValues("sender"), }, state: Plan, } @@ -139,6 +144,7 @@ func (f *PrunerFactory) BuildReceiverPruner(ctx context.Context, target Target, f.receiverRules, f.retryWait, false, // senseless here anyways + f.promPruneSecs.WithLabelValues("receiver"), }, state: Plan, } diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index 31dcdc1..a8c836e 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -6,6 +6,7 @@ import ( "context" "errors" "fmt" + "github.com/prometheus/client_golang/prometheus" "io" "net" "sync" @@ -90,7 +91,9 @@ func (s State) fsrsf() state { } type Replication struct { - // lock protects all fields in this struct, but not the data behind pointers + promBytesReplicated prometheus.Counter + + // lock protects all fields below it in this struct, but not the data behind pointers lock sync.Mutex state State fs string @@ -120,8 +123,8 @@ type ReplicationBuilder struct { r *Replication } -func BuildReplication(fs string) *ReplicationBuilder { - return &ReplicationBuilder{&Replication{fs: fs}} +func BuildReplication(fs string, promBytesReplicated prometheus.Counter) *ReplicationBuilder { + return &ReplicationBuilder{&Replication{fs: fs, promBytesReplicated: promBytesReplicated}} } func (b *ReplicationBuilder) AddStep(from, to FilesystemVersion) *ReplicationBuilder { @@ -204,6 +207,7 @@ func (f *Replication) TakeStep(ctx context.Context, sender Sender, receiver Rece preTime := time.Now() s = s(ctx, sender, receiver, u) delta := time.Now().Sub(preTime) + post = u(func(f *Replication) { if len(f.pending) == 0 { return @@ -369,6 +373,9 @@ func (s *ReplicationStep) doReplication(ctx context.Context, sender Sender, rece } s.byteCounter = util.NewByteCounterReader(sstream) + defer func() { + s.parent.promBytesReplicated.Add(float64(s.byteCounter.Bytes())) + }() sstream = s.byteCounter rr := &pdu.ReceiveReq{ diff --git a/replication/mainfsm.go b/replication/mainfsm.go index c829442..b4cec9b 100644 --- a/replication/mainfsm.go +++ b/replication/mainfsm.go @@ -6,6 +6,7 @@ import ( "context" "errors" "fmt" + "github.com/prometheus/client_golang/prometheus" "math/bits" "sync" "time" @@ -49,6 +50,10 @@ func (s State) rsf() state { // It is a state machine that is driven by the Drive method // and provides asynchronous reporting via the Report method (i.e. from another goroutine). type Replication struct { + // not protected by lock + promSecsPerState *prometheus.HistogramVec // labels: state + promBytesReplicated *prometheus.CounterVec // labels: filesystem + // lock protects all fields of this struct (but not the fields behind pointers!) lock sync.Mutex @@ -77,9 +82,11 @@ type Report struct { Active *fsrep.Report } -func NewReplication() *Replication { +func NewReplication(secsPerState *prometheus.HistogramVec, bytesReplicated *prometheus.CounterVec) *Replication { r := Replication{ - state: Planning, + promSecsPerState: secsPerState, + promBytesReplicated: bytesReplicated, + state: Planning, } return &r } @@ -142,6 +149,7 @@ func (r *Replication) Drive(ctx context.Context, sender Sender, receiver Receive pre = u(nil) s = s(ctx, sender, receiver, u) delta := time.Now().Sub(preTime) + r.promSecsPerState.WithLabelValues(pre.String()).Observe(delta.Seconds()) post = u(nil) getLogger(ctx). WithField("transition", fmt.Sprintf("%s => %s", pre, post)). @@ -262,7 +270,11 @@ func statePlanning(ctx context.Context, sender Sender, receiver Receiver, u upda continue } - fsrfsm := fsrep.BuildReplication(fs.Path) + var promBytesReplicated *prometheus.CounterVec + u(func(replication *Replication) { // FIXME args struct like in pruner (also use for sender and receiver) + promBytesReplicated = replication.promBytesReplicated + }) + fsrfsm := fsrep.BuildReplication(fs.Path, promBytesReplicated.WithLabelValues(fs.Path)) if len(path) == 1 { fsrfsm.AddStep(nil, path[0]) } else { diff --git a/zfs/prometheus.go b/zfs/prometheus.go index f27d797..3934418 100644 --- a/zfs/prometheus.go +++ b/zfs/prometheus.go @@ -3,10 +3,10 @@ package zfs import "github.com/prometheus/client_golang/prometheus" var prom struct { - ZFSListFilesystemVersionDuration *prometheus.HistogramVec - ZFSDestroyFilesystemVersionDuration *prometheus.HistogramVec - ZFSSnapshotDuration *prometheus.HistogramVec - ZFSBookmarkDuration *prometheus.HistogramVec + ZFSListFilesystemVersionDuration *prometheus.HistogramVec + ZFSSnapshotDuration *prometheus.HistogramVec + ZFSBookmarkDuration *prometheus.HistogramVec + ZFSDestroyDuration *prometheus.HistogramVec } func init() { @@ -16,12 +16,6 @@ func init() { Name: "list_filesystem_versions_duration", Help: "Seconds it took for listing the versions of a given filesystem", }, []string{"filesystem"}) - prom.ZFSDestroyFilesystemVersionDuration = prometheus.NewHistogramVec(prometheus.HistogramOpts{ - Namespace: "zrepl", - Subsystem: "zfs", - Name: "destroy_filesystem_version_duration", - Help: "Seconds it took to destroy a version of a given filesystem", - }, []string{"filesystem", "version_type"}) prom.ZFSSnapshotDuration = prometheus.NewHistogramVec(prometheus.HistogramOpts{ Namespace: "zrepl", Subsystem: "zfs", @@ -34,20 +28,26 @@ func init() { Name: "bookmark_duration", Help: "Duration it took to bookmark a given snapshot", }, []string{"filesystem"}) + prom.ZFSDestroyDuration = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "zrepl", + Subsystem: "zfs", + Name: "destroy_duration", + Help: "Duration it took to destroy a dataset", + }, []string{"dataset_type", "filesystem"}) } func PrometheusRegister(registry prometheus.Registerer) error { if err := registry.Register(prom.ZFSListFilesystemVersionDuration); err != nil { return err } - if err := registry.Register(prom.ZFSDestroyFilesystemVersionDuration); err != nil { - return err - } if err := registry.Register(prom.ZFSBookmarkDuration); err != nil { return err } if err := registry.Register(prom.ZFSSnapshotDuration); err != nil { return err } + if err := registry.Register(prom.ZFSDestroyDuration); err != nil { + return err + } return nil } diff --git a/zfs/versions.go b/zfs/versions.go index 9deb10d..29303c0 100644 --- a/zfs/versions.go +++ b/zfs/versions.go @@ -156,9 +156,6 @@ func ZFSListFilesystemVersions(fs *DatasetPath, filter FilesystemVersionFilter) func ZFSDestroyFilesystemVersion(filesystem *DatasetPath, version *FilesystemVersion) (err error) { - promTimer := prometheus.NewTimer(prom.ZFSDestroyFilesystemVersionDuration.WithLabelValues(filesystem.ToString(), version.Type.String())) - defer promTimer.ObserveDuration() - datasetPath := version.ToAbsPath(filesystem) // Sanity check... diff --git a/zfs/zfs.go b/zfs/zfs.go index 8cd4fd2..8200c61 100644 --- a/zfs/zfs.go +++ b/zfs/zfs.go @@ -566,6 +566,21 @@ func zfsGet(path string, props []string, allowedSources zfsPropertySource) (*ZFS func ZFSDestroy(dataset string) (err error) { + var dstype, filesystem string + idx := strings.IndexAny(dataset, "@#") + if idx == -1 { + dstype = "filesystem" + filesystem = dataset + } else { + switch dataset[idx] { + case '@': dstype = "snapshot" + case '#': dstype = "bookmark" + } + filesystem = dataset[:idx] + } + + defer prometheus.NewTimer(prom.ZFSDestroyDuration.WithLabelValues(dstype, filesystem)) + cmd := exec.Command(ZFS_BINARY, "destroy", dataset) stderr := bytes.NewBuffer(make([]byte, 0, 1024)) From 6c3f442f13ede969097e72e12c0db2e2152873e8 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 13 Sep 2018 12:44:46 +0200 Subject: [PATCH 122/167] daemon control / jsonclient: fix connection leak due to open request body Also: - Defensive measures in control http server (1s timeouts) (prevent the leak, even if request body is not closed) - Add prometheus metrics to track control socket latencies (were used for debugging) --- client/jsonclient.go | 5 ++++- daemon/control.go | 34 ++++++++++++++++++++++++++++++++-- 2 files changed, 36 insertions(+), 3 deletions(-) diff --git a/client/jsonclient.go b/client/jsonclient.go index 43bc903..41c272d 100644 --- a/client/jsonclient.go +++ b/client/jsonclient.go @@ -30,7 +30,10 @@ func jsonRequestResponse(c http.Client, endpoint string, req interface{}, res in resp, err := c.Post("http://unix"+endpoint, "application/json", &buf) if err != nil { return err - } else if resp.StatusCode != http.StatusOK { + } + defer resp.Body.Close() + + if resp.StatusCode != http.StatusOK { var msg bytes.Buffer io.CopyN(&msg, resp.Body, 4096) return errors.Errorf("%s", msg.String()) diff --git a/daemon/control.go b/daemon/control.go index 34b879c..ffacfc8 100644 --- a/daemon/control.go +++ b/daemon/control.go @@ -13,6 +13,7 @@ import ( "io" "net" "net/http" + "time" ) type controlJob struct { @@ -36,7 +37,29 @@ func (j *controlJob) Name() string { return jobNameControl } func (j *controlJob) Status() interface{} { return nil } -func (j *controlJob) RegisterMetrics(registerer prometheus.Registerer) {} +var promControl struct { + requestBegin *prometheus.CounterVec + requestFinished *prometheus.HistogramVec +} + +func (j *controlJob) RegisterMetrics(registerer prometheus.Registerer) { + promControl.requestBegin = prometheus.NewCounterVec(prometheus.CounterOpts{ + Namespace: "zrepl", + Subsystem: "control", + Name: "request_begin", + Help: "number of request we started to handle", + }, []string{"endpoint"}) + + promControl.requestFinished = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "zrepl", + Subsystem: "control", + Name: "request_finished", + Help: "time it took a request to finih", + Buckets: []float64{1e-6, 10e-6, 100e-6, 500e-6, 1e-3,10e-3, 100e-3, 200e-3,400e-3,800e-3, 1, 10, 20}, + }, []string{"endpoint"}) + registerer.MustRegister(promControl.requestBegin) + registerer.MustRegister(promControl.requestFinished) +} const ( ControlJobEndpointPProf string = "/debug/pprof" @@ -95,7 +118,12 @@ func (j *controlJob) Run(ctx context.Context) { return struct{}{}, err }}}) - server := http.Server{Handler: mux} + server := http.Server{ + Handler: mux, + // control socket is local, 1s timeout should be more than sufficient, even on a loaded system + WriteTimeout: 1*time.Second, + ReadTimeout: 1*time.Second, + } outer: for { @@ -189,6 +217,8 @@ type requestLogger struct { func (l requestLogger) ServeHTTP(w http.ResponseWriter, r *http.Request) { log := l.log.WithField("method", r.Method).WithField("url", r.URL) log.Debug("start") + promControl.requestBegin.WithLabelValues(r.URL.Path).Inc() + defer prometheus.NewTimer(promControl.requestFinished.WithLabelValues(r.URL.Path)).ObserveDuration() if l.handlerFunc != nil { l.handlerFunc(w, r) } else if l.handler != nil { From 7f9eb626408434599c33663b160994029d887bb3 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 18 Sep 2018 22:44:00 +0200 Subject: [PATCH 123/167] sink: concurrent connection handling --- daemon/job/sink.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/daemon/job/sink.go b/daemon/job/sink.go index 18c009c..627b999 100644 --- a/daemon/job/sink.go +++ b/daemon/job/sink.go @@ -76,7 +76,7 @@ outer: connId++ connLog := log. WithField("connID", connId) - j.handleConnection(WithLogger(ctx, connLog), res.conn) + go j.handleConnection(WithLogger(ctx, connLog), res.conn) case <-ctx.Done(): break outer From 9dd662df08e8e5008093698e249be192b3dd792a Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 23 Sep 2018 14:44:53 +0200 Subject: [PATCH 124/167] status: raw output subcommand --- client/status.go | 26 +++++++++++++++++++++++++- main.go | 5 ++++- 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/client/status.go b/client/status.go index 2b94f35..bd743d7 100644 --- a/client/status.go +++ b/client/status.go @@ -12,6 +12,9 @@ import ( "sort" "sync" "time" + "io" + "os" + "net/http" ) type tui struct { @@ -62,12 +65,33 @@ func (t *tui) addIndent(indent int) { t.moveLine(0, 0) } -func RunStatus(config *config.Config, args []string) error { +type StatusFlags struct { + Raw bool +} + +func RunStatus(flags StatusFlags, config *config.Config, args []string) error { httpc, err := controlHttpClient(config.Global.Control.SockPath) if err != nil { return err } + if flags.Raw { + resp, err := httpc.Get("http://unix"+daemon.ControlJobEndpointStatus) + if err != nil { + return err + } + defer resp.Body.Close() + if resp.StatusCode != http.StatusOK { + fmt.Fprintf(os.Stderr, "Received error response:\n") + io.CopyN(os.Stderr, resp.Body, 4096) + return errors.Errorf("exit") + } + if _, err := io.Copy(os.Stdout, resp.Body); err != nil { + return err + } + return nil + } + t := newTui() t.lock.Lock() t.err = errors.New("Got no report yet") diff --git a/main.go b/main.go index f6e35aa..b2985c9 100644 --- a/main.go +++ b/main.go @@ -47,6 +47,8 @@ var wakeupCmd = &cobra.Command{ }, } +var statusCmdFlags client.StatusFlags + var statusCmd = &cobra.Command{ Use: "status", Short: "status", @@ -55,7 +57,7 @@ var statusCmd = &cobra.Command{ if err != nil { return err } - return client.RunStatus(conf, args) + return client.RunStatus(statusCmdFlags, conf, args) }, } @@ -158,6 +160,7 @@ func init() { rootCmd.PersistentFlags().StringVar(&rootArgs.configFile, "config", "", "config file path") rootCmd.AddCommand(daemonCmd) rootCmd.AddCommand(wakeupCmd) + statusCmd.Flags().BoolVar(&statusCmdFlags.Raw, "raw", false, "dump raw response from zrepl daemon") rootCmd.AddCommand(statusCmd) rootCmd.AddCommand(stdinserverCmd) rootCmd.AddCommand(bashcompCmd) From 4a6160baf3cad1254384d93952152d9ddb47ebee Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 23 Sep 2018 20:28:30 +0200 Subject: [PATCH 125/167] update to streamrpc 0.4 & adjust config (not breaking) --- Gopkg.lock | 6 +++--- Gopkg.toml | 2 +- config/config.go | 2 ++ config/config_rpc_test.go | 13 +++++++++++++ daemon/streamrpcconfig/streamrpcconfig.go | 5 ++--- 5 files changed, 21 insertions(+), 7 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index 07b31e6..6684973 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -170,15 +170,15 @@ revision = "391d2c78c8404a9683d79f75dd24ab53040f89f7" [[projects]] - digest = "1:f6bc2d2f9c00737754ef87dba06c851458523fa382a61e1e2e5270c1f39a6622" + digest = "1:c2ba1c9dc003c15856e4529dac028cacba08ee8924300f058b3467cde9acf7a9" name = "github.com/problame/go-streamrpc" packages = [ ".", "internal/pdu", ] pruneopts = "" - revision = "a67d93db93b3dc8c2d6901a1b2183ff42dea8f7d" - version = "v0.3" + revision = "de6f6a4041c77f700f02d8fe749e54efa50811f7" + version = "v0.4" [[projects]] branch = "master" diff --git a/Gopkg.toml b/Gopkg.toml index b08613e..fe541e1 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -69,4 +69,4 @@ ignored = [ "github.com/inconshreveable/mousetrap" ] [[constraint]] name = "github.com/problame/go-streamrpc" - version = "0.3.0" + version = "0.4.0" diff --git a/config/config.go b/config/config.go index 08262a9..d661d4a 100644 --- a/config/config.go +++ b/config/config.go @@ -130,6 +130,8 @@ type RPCConfig struct { RxStructuredMaxLen uint32 `yaml:"rx_structured_max,optional,default=16777216"` RxStreamChunkMaxLen uint32 `yaml:"rx_stream_chunk_max,optional,default=16777216"` RxHeaderMaxLen uint32 `yaml:"rx_header_max,optional,default=40960"` + SendHeartbeatInterval time.Duration `yaml:"send_heartbeat_interval,optional,positive,default=5s"` + } type ConnectEnum struct { diff --git a/config/config_rpc_test.go b/config/config_rpc_test.go index e295752..f74581d 100644 --- a/config/config_rpc_test.go +++ b/config/config_rpc_test.go @@ -53,11 +53,24 @@ jobs: rpc: rx_structured_max: 0x2342 +- type: sink + name: "other_sink" + root_dataset: "pool2/backup_laptops" + serve: + type: tcp + listen: "192.168.122.189:8888" + clients: { + "10.23.42.23":"client1" + } + rpc: + send_heartbeat_interval: 10s + `) assert.Equal(t, 20*time.Second, conf.Jobs[0].Ret.(*PullJob).Connect.Ret.(*TCPConnect).RPC.Timeout) assert.Equal(t, uint32(0xabcd), conf.Jobs[1].Ret.(*PullJob).Connect.Ret.(*TCPConnect).RPC.TxChunkSize) assert.Equal(t, uint32(0x2342), conf.Jobs[2].Ret.(*SinkJob).Serve.Ret.(*TCPServe).RPC.RxStructuredMaxLen) + assert.Equal(t, 10*time.Second, conf.Jobs[3].Ret.(*SinkJob).Serve.Ret.(*TCPServe).RPC.SendHeartbeatInterval) defConf := RPCConfig{} Default(&defConf) assert.Equal(t, defConf.Timeout, conf.Global.RPC.Timeout) diff --git a/daemon/streamrpcconfig/streamrpcconfig.go b/daemon/streamrpcconfig/streamrpcconfig.go index 8dc64f6..da28d5d 100644 --- a/daemon/streamrpcconfig/streamrpcconfig.go +++ b/daemon/streamrpcconfig/streamrpcconfig.go @@ -15,9 +15,8 @@ func FromDaemonConfig(g *config.Global, in *config.RPCConfig) (*streamrpc.ConnCo RxStructuredMaxLen: conf.RxStructuredMaxLen, RxStreamMaxChunkSize: conf.RxStreamChunkMaxLen, TxChunkSize: conf.TxChunkSize, - Timeout: streamrpc.Timeout{ - Progress: conf.Timeout, - }, + Timeout: conf.Timeout, + SendHeartbeatInterval: conf.SendHeartbeatInterval, } if err := srpcConf.Validate(); err != nil { return nil, err From 9446b51a1fdf9bad515e19bfa904a7374bb94942 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 23 Sep 2018 21:08:03 +0200 Subject: [PATCH 126/167] status: infra for reporting jobs instead of just replication.Report --- client/status.go | 45 ++++++++++++++++++++--------- daemon/control.go | 2 +- daemon/daemon.go | 6 ++-- daemon/job/job.go | 67 +++++++++++++++++++++++++++++++++++++++++++- daemon/job/push.go | 12 ++++++-- daemon/job/sink.go | 7 +++-- daemon/prometheus.go | 2 +- 7 files changed, 116 insertions(+), 25 deletions(-) diff --git a/client/status.go b/client/status.go index bd743d7..5d7a95a 100644 --- a/client/status.go +++ b/client/status.go @@ -2,12 +2,12 @@ package client import ( "fmt" - "github.com/mitchellh/mapstructure" "github.com/nsf/termbox-go" "github.com/pkg/errors" + "github.com/zrepl/yaml-config" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon" - "github.com/zrepl/zrepl/replication" + "github.com/zrepl/zrepl/daemon/job" "github.com/zrepl/zrepl/replication/fsrep" "sort" "sync" @@ -22,7 +22,7 @@ type tui struct { indent int lock sync.Mutex //For report and error - report map[string]interface{} + report map[string]job.Status err error } @@ -42,6 +42,10 @@ func (t *tui) moveLine(dl int, col int) { func (t *tui) write(text string) { for _, c := range text { + if c == '\n' { + t.newline() + continue + } termbox.SetCell(t.x, t.y, c, termbox.ColorDefault, termbox.ColorDefault) t.x += 1 } @@ -104,7 +108,7 @@ func RunStatus(flags StatusFlags, config *config.Config, args []string) error { defer termbox.Close() update := func() { - m := make(map[string]interface{}) + m := make(map[string]job.Status) err2 := jsonRequestResponse(httpc, daemon.ControlJobEndpointStatus, struct{}{}, @@ -172,7 +176,7 @@ func (t *tui) draw() { for _, k := range keys { v := t.report[k] - if len(k) == 0 || k[0] == '_' { //Internal job + if len(k) == 0 || daemon.IsInternalJobName(k) { //Internal job continue } t.setIndent(0) @@ -180,20 +184,35 @@ func (t *tui) draw() { t.printf("Job: %s", k) t.setIndent(1) t.newline() + t.printf("Type: %s", v.Type) + t.setIndent(1) + t.newline() - if v == nil { - t.printf("No report generated yet") + if v.Type != job.TypePush { + t.printf("No status representation for job type '%s', dumping as YAML", v.Type) t.newline() - continue - } - rep := replication.Report{} - err := mapstructure.Decode(v, &rep) - if err != nil { - t.printf("Failed to decode report: %s", err.Error()) + asYaml, err := yaml.Marshal(v.JobSpecific) + if err != nil { + t.printf("Error marshaling status to YAML: %s", err) + t.newline() + continue + } + t.write(string(asYaml)) t.newline() continue } + pushStatus, ok := v.JobSpecific.(*job.PushStatus) + if !ok || pushStatus == nil { + t.printf("PushStatus is null") + t.newline() + continue + } + rep := pushStatus.Replication + if rep == nil { + t.newline() + continue + } all := make([]*fsrep.Report, 0, len(rep.Completed)+len(rep.Pending) + 1) all = append(all, rep.Completed...) diff --git a/daemon/control.go b/daemon/control.go index ffacfc8..0be8a0d 100644 --- a/daemon/control.go +++ b/daemon/control.go @@ -35,7 +35,7 @@ func newControlJob(sockpath string, jobs *jobs) (j *controlJob, err error) { func (j *controlJob) Name() string { return jobNameControl } -func (j *controlJob) Status() interface{} { return nil } +func (j *controlJob) Status() *job.Status { return &job.Status{Type: job.TypeInternal} } var promControl struct { requestBegin *prometheus.CounterVec diff --git a/daemon/daemon.go b/daemon/daemon.go index 91cc71b..8ada3c9 100644 --- a/daemon/daemon.go +++ b/daemon/daemon.go @@ -125,13 +125,13 @@ func (s *jobs) wait() <-chan struct{} { return ch } -func (s *jobs) status() map[string]interface{} { +func (s *jobs) status() map[string]*job.Status { s.m.RLock() defer s.m.RUnlock() type res struct { name string - status interface{} + status *job.Status } var wg sync.WaitGroup c := make(chan res, len(s.jobs)) @@ -144,7 +144,7 @@ func (s *jobs) status() map[string]interface{} { } wg.Wait() close(c) - ret := make(map[string]interface{}, len(s.jobs)) + ret := make(map[string]*job.Status, len(s.jobs)) for res := range c { ret[res.name] = res.status } diff --git a/daemon/job/job.go b/daemon/job/job.go index 3e338c0..23f580d 100644 --- a/daemon/job/job.go +++ b/daemon/job/job.go @@ -2,7 +2,9 @@ package job import ( "context" + "encoding/json" "errors" + "fmt" "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/logger" ) @@ -47,10 +49,73 @@ func WithWakeup(ctx context.Context) (context.Context, WakeupFunc) { type Job interface { Name() string Run(ctx context.Context) - Status() interface{} + Status() *Status RegisterMetrics(registerer prometheus.Registerer) } +type Type string + +const ( + TypeInternal Type = "internal" + TypePush Type = "push" + TypeSink Type = "sink" +) + +type Status struct { + Type Type + JobSpecific interface{} +} + +func (s *Status) MarshalJSON() ([]byte, error) { + typeJson, err := json.Marshal(s.Type) + if err != nil { + return nil, err + } + jobJSON, err := json.Marshal(s.JobSpecific) + if err != nil { + return nil, err + } + m := map[string]json.RawMessage { + "type": typeJson, + string(s.Type): jobJSON, + } + return json.Marshal(m) +} + +func (s *Status) UnmarshalJSON(in []byte) (err error) { + var m map[string]json.RawMessage + if err := json.Unmarshal(in, &m); err != nil { + return err + } + tJSON, ok := m["type"] + if !ok { + return fmt.Errorf("field 'type' not found") + } + if err := json.Unmarshal(tJSON, &s.Type); err != nil { + return err + } + key := string(s.Type) + jobJSON, ok := m[key] + if !ok { + return fmt.Errorf("field '%s', not found", key) + } + switch s.Type { + case TypePush: + var st PushStatus + err = json.Unmarshal(jobJSON, &st) + s.JobSpecific = &st + case TypeSink: + var st SinkStatus + err = json.Unmarshal(jobJSON, &st) + s.JobSpecific = &st + case TypeInternal: + // internal jobs do not report specifics + default: + err = fmt.Errorf("unknown job type '%s'", key) + } + return err +} + func WaitWakeup(ctx context.Context) <-chan struct{} { wc, ok := ctx.Value(contextKeyWakeup).(chan struct{}) if !ok { diff --git a/daemon/job/push.go b/daemon/job/push.go index 496a44c..c77e4d3 100644 --- a/daemon/job/push.go +++ b/daemon/job/push.go @@ -89,7 +89,11 @@ func (j *Push) RegisterMetrics(registerer prometheus.Registerer) { func (j *Push) Name() string { return j.name } -func (j *Push) Status() interface{} { +type PushStatus struct { + Replication *replication.Report +} + +func (j *Push) Status() *Status { rep := func() *replication.Replication { j.mtx.Lock() defer j.mtx.Unlock() @@ -98,10 +102,12 @@ func (j *Push) Status() interface{} { } return j.replication }() + s := &PushStatus{} if rep == nil { - return nil + return &Status{Type: TypePush, JobSpecific: s} } - return rep.Report() + s.Replication = rep.Report() + return &Status{Type: TypePush, JobSpecific: s} } func (j *Push) Run(ctx context.Context) { diff --git a/daemon/job/sink.go b/daemon/job/sink.go index 627b999..0c1c114 100644 --- a/daemon/job/sink.go +++ b/daemon/job/sink.go @@ -41,9 +41,10 @@ func SinkFromConfig(g *config.Global, in *config.SinkJob) (s *Sink, err error) { func (j *Sink) Name() string { return j.name } -func (*Sink) Status() interface{} { - // FIXME - return nil +type SinkStatus struct {} + +func (*Sink) Status() *Status { + return &Status{Type: TypeSink} // FIXME SinkStatus } func (*Sink) RegisterMetrics(registerer prometheus.Registerer) {} diff --git a/daemon/prometheus.go b/daemon/prometheus.go index b39910c..7607b94 100644 --- a/daemon/prometheus.go +++ b/daemon/prometheus.go @@ -39,7 +39,7 @@ func init() { func (j *prometheusJob) Name() string { return jobNamePrometheus } -func (j *prometheusJob) Status() interface{} { return nil } +func (j *prometheusJob) Status() *job.Status { return &job.Status{Type: job.TypeInternal} } func (j *prometheusJob) RegisterMetrics(registerer prometheus.Registerer) {} From e3be120d8804d39406c17073f9bea17c154a9253 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sun, 23 Sep 2018 23:04:31 +0200 Subject: [PATCH 127/167] refactor push + source into active + passive 'sides' with push and source 'modes' --- client/status.go | 4 +- config/config.go | 32 ++++---- daemon/job/{push.go => active.go} | 99 ++++++++++++++--------- daemon/job/build_jobs.go | 24 ++++-- daemon/job/job.go | 6 +- daemon/job/{sink.go => passive.go} | 123 +++++++++++++++++------------ 6 files changed, 174 insertions(+), 114 deletions(-) rename daemon/job/{push.go => active.go} (67%) rename daemon/job/{sink.go => passive.go} (55%) diff --git a/client/status.go b/client/status.go index 5d7a95a..c72ca10 100644 --- a/client/status.go +++ b/client/status.go @@ -202,9 +202,9 @@ func (t *tui) draw() { continue } - pushStatus, ok := v.JobSpecific.(*job.PushStatus) + pushStatus, ok := v.JobSpecific.(*job.ActiveSideStatus) if !ok || pushStatus == nil { - t.printf("PushStatus is null") + t.printf("ActiveSideStatus is null") t.newline() continue } diff --git a/config/config.go b/config/config.go index d661d4a..081e335 100644 --- a/config/config.go +++ b/config/config.go @@ -21,42 +21,42 @@ type JobEnum struct { Ret interface{} } -type PushJob struct { +type ActiveJob struct { Type string `yaml:"type"` Name string `yaml:"name"` Connect ConnectEnum `yaml:"connect"` - Filesystems FilesystemsFilter `yaml:"filesystems"` - Snapshotting Snapshotting `yaml:"snapshotting"` Pruning PruningSenderReceiver `yaml:"pruning"` Debug JobDebugSettings `yaml:"debug,optional"` } -type SinkJob struct { +type PassiveJob struct { Type string `yaml:"type"` Name string `yaml:"name"` - RootDataset string `yaml:"root_dataset"` Serve ServeEnum `yaml:"serve"` Debug JobDebugSettings `yaml:"debug,optional"` } +type PushJob struct { + ActiveJob `yaml:",inline"` + Snapshotting Snapshotting `yaml:"snapshotting"` + Filesystems FilesystemsFilter `yaml:"filesystems"` +} + type PullJob struct { - Type string `yaml:"type"` - Name string `yaml:"name"` - Connect ConnectEnum `yaml:"connect"` + ActiveJob `yaml:",inline"` RootDataset string `yaml:"root_dataset"` Interval time.Duration `yaml:"interval,positive"` - Pruning PruningSenderReceiver `yaml:"pruning"` - Debug JobDebugSettings `yaml:"debug,optional"` +} + +type SinkJob struct { + PassiveJob `yaml:",inline"` + RootDataset string `yaml:"root_dataset"` } type SourceJob struct { - Type string `yaml:"type"` - Name string `yaml:"name"` - Serve ServeEnum `yaml:"serve"` - Filesystems FilesystemsFilter `yaml:"filesystems"` + PassiveJob `yaml:",inline"` Snapshotting Snapshotting `yaml:"snapshotting"` - Pruning PruningLocal `yaml:"pruning"` - Debug JobDebugSettings `yaml:"debug,optional"` + Filesystems FilesystemsFilter `yaml:"filesystems"` } type LocalJob struct { diff --git a/daemon/job/push.go b/daemon/job/active.go similarity index 67% rename from daemon/job/push.go rename to daemon/job/active.go index c77e4d3..5d20f65 100644 --- a/daemon/job/push.go +++ b/daemon/job/active.go @@ -3,6 +3,7 @@ package job import ( "context" "github.com/pkg/errors" + "github.com/problame/go-streamrpc" "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon/connecter" @@ -15,14 +16,13 @@ import ( "github.com/zrepl/zrepl/daemon/snapper" ) -type Push struct { - name string - clientFactory *connecter.ClientFactory - fsfilter endpoint.FSFilter +type ActiveSide struct { + mode activeMode + name string + clientFactory *connecter.ClientFactory prunerFactory *pruner.PrunerFactory - snapper *snapper.Snapper promRepStateSecs *prometheus.HistogramVec // labels: state promPruneSecs *prometheus.HistogramVec // labels: prune_side @@ -32,9 +32,48 @@ type Push struct { replication *replication.Replication } -func PushFromConfig(g *config.Global, in *config.PushJob) (j *Push, err error) { +type activeMode interface { + SenderReceiver(client *streamrpc.Client) (replication.Sender, replication.Receiver, error) + Type() Type + RunPeriodic(ctx context.Context, wakeUpCommon chan<- struct{}) +} - j = &Push{} +type modePush struct { + fsfilter endpoint.FSFilter + snapper *snapper.Snapper +} + +func (m *modePush) SenderReceiver(client *streamrpc.Client) (replication.Sender, replication.Receiver, error) { + sender := endpoint.NewSender(m.fsfilter) + receiver := endpoint.NewRemote(client) + return sender, receiver, nil +} + +func (m *modePush) Type() Type { return TypePush } + +func (m *modePush) RunPeriodic(ctx context.Context, wakeUpCommon chan <- struct{}) { + m.snapper.Run(ctx, wakeUpCommon) +} + + +func modePushFromConfig(g *config.Global, in *config.PushJob) (*modePush, error) { + m := &modePush{} + fsf, err := filters.DatasetMapFilterFromConfig(in.Filesystems) + if err != nil { + return nil, errors.Wrap(err, "cannnot build filesystem filter") + } + m.fsfilter = fsf + + if m.snapper, err = snapper.FromConfig(g, fsf, &in.Snapshotting); err != nil { + return nil, errors.Wrap(err, "cannot build snapper") + } + + return m, nil +} + +func activeSide(g *config.Global, in *config.ActiveJob, mode activeMode) (j *ActiveSide, err error) { + + j = &ActiveSide{mode: mode} j.name = in.Name j.promRepStateSecs = prometheus.NewHistogramVec(prometheus.HistogramOpts{ Namespace: "zrepl", @@ -56,12 +95,6 @@ func PushFromConfig(g *config.Global, in *config.PushJob) (j *Push, err error) { return nil, errors.Wrap(err, "cannot build client") } - fsf, err := filters.DatasetMapFilterFromConfig(in.Filesystems) - if err != nil { - return nil, errors.Wrap(err, "cannnot build filesystem filter") - } - j.fsfilter = fsf - j.promPruneSecs = prometheus.NewHistogramVec(prometheus.HistogramOpts{ Namespace: "zrepl", Subsystem: "pruning", @@ -74,26 +107,22 @@ func PushFromConfig(g *config.Global, in *config.PushJob) (j *Push, err error) { return nil, err } - if j.snapper, err = snapper.FromConfig(g, fsf, &in.Snapshotting); err != nil { - return nil, errors.Wrap(err, "cannot build snapper") - } - return j, nil } -func (j *Push) RegisterMetrics(registerer prometheus.Registerer) { +func (j *ActiveSide) RegisterMetrics(registerer prometheus.Registerer) { registerer.MustRegister(j.promRepStateSecs) registerer.MustRegister(j.promPruneSecs) registerer.MustRegister(j.promBytesReplicated) } -func (j *Push) Name() string { return j.name } +func (j *ActiveSide) Name() string { return j.name } -type PushStatus struct { +type ActiveSideStatus struct { Replication *replication.Report } -func (j *Push) Status() *Status { +func (j *ActiveSide) Status() *Status { rep := func() *replication.Replication { j.mtx.Lock() defer j.mtx.Unlock() @@ -102,26 +131,25 @@ func (j *Push) Status() *Status { } return j.replication }() - s := &PushStatus{} + s := &ActiveSideStatus{} + t := j.mode.Type() if rep == nil { - return &Status{Type: TypePush, JobSpecific: s} + return &Status{Type: t, JobSpecific: s} } s.Replication = rep.Report() - return &Status{Type: TypePush, JobSpecific: s} + return &Status{Type: t, JobSpecific: s} } -func (j *Push) Run(ctx context.Context) { +func (j *ActiveSide) Run(ctx context.Context) { log := GetLogger(ctx) + ctx = logging.WithSubsystemLoggers(ctx, log) defer log.Info("job exiting") - snapshotsTaken := make(chan struct{}) - { - ctx, cancel := context.WithCancel(ctx) - defer cancel() - ctx = logging.WithSubsystemLoggers(ctx, log) - go j.snapper.Run(ctx, snapshotsTaken) - } + periodicDone := make(chan struct{}) + ctx, cancel := context.WithCancel(ctx) + defer cancel() + go j.mode.RunPeriodic(ctx, periodicDone) invocationCount := 0 outer: @@ -133,7 +161,7 @@ outer: break outer case <-WaitWakeup(ctx): - case <-snapshotsTaken: + case <-periodicDone: } invocationCount++ invLog := log.WithField("invocation", invocationCount) @@ -141,7 +169,7 @@ outer: } } -func (j *Push) do(ctx context.Context) { +func (j *ActiveSide) do(ctx context.Context) { log := GetLogger(ctx) ctx = logging.WithSubsystemLoggers(ctx, log) @@ -152,8 +180,7 @@ func (j *Push) do(ctx context.Context) { } defer client.Close(ctx) - sender := endpoint.NewSender(j.fsfilter) - receiver := endpoint.NewRemote(client) + sender, receiver, err := j.mode.SenderReceiver(client) j.mtx.Lock() j.replication = replication.NewReplication(j.promRepStateSecs, j.promBytesReplicated) diff --git a/daemon/job/build_jobs.go b/daemon/job/build_jobs.go index b438cac..368d9bf 100644 --- a/daemon/job/build_jobs.go +++ b/daemon/job/build_jobs.go @@ -2,8 +2,8 @@ package job import ( "fmt" - "github.com/zrepl/zrepl/config" "github.com/pkg/errors" + "github.com/zrepl/zrepl/config" ) func JobsFromConfig(c *config.Config) ([]Job, error) { @@ -19,19 +19,31 @@ func JobsFromConfig(c *config.Config) ([]Job, error) { } func buildJob(c *config.Global, in config.JobEnum) (j Job, err error) { + cannotBuildJob := func(e error, name string) (Job, error) { + return nil, errors.Wrapf(err, "cannot build job %q", name) + } switch v := in.Ret.(type) { case *config.SinkJob: - j, err = SinkFromConfig(c, v) + m, err := modeSinkFromConfig(c, v) if err != nil { - return nil, errors.Wrapf(err, "cannot build job %q", v.Name) + return cannotBuildJob(err, v.Name) + } + j, err = passiveSideFromConfig(c, &v.PassiveJob, m) + if err != nil { + return cannotBuildJob(err, v.Name) } case *config.PushJob: - j, err = PushFromConfig(c, v) + m, err := modePushFromConfig(c, v) if err != nil { - return nil, errors.Wrapf(err, "cannot build job %q", v.Name) + return cannotBuildJob(err, v.Name) + } + j, err = activeSide(c, &v.ActiveJob, m) + if err != nil { + return cannotBuildJob(err, v.Name) } default: panic(fmt.Sprintf("implementation error: unknown job type %T", v)) } - return j, err + return j, nil + } diff --git a/daemon/job/job.go b/daemon/job/job.go index 23f580d..1cac343 100644 --- a/daemon/job/job.go +++ b/daemon/job/job.go @@ -59,6 +59,8 @@ const ( TypeInternal Type = "internal" TypePush Type = "push" TypeSink Type = "sink" + TypePull Type = "pull" + TypeSource Type = "source" ) type Status struct { @@ -101,11 +103,11 @@ func (s *Status) UnmarshalJSON(in []byte) (err error) { } switch s.Type { case TypePush: - var st PushStatus + var st ActiveSideStatus err = json.Unmarshal(jobJSON, &st) s.JobSpecific = &st case TypeSink: - var st SinkStatus + var st PassiveStatus err = json.Unmarshal(jobJSON, &st) s.JobSpecific = &st case TypeInternal: diff --git a/daemon/job/sink.go b/daemon/job/passive.go similarity index 55% rename from daemon/job/sink.go rename to daemon/job/passive.go index 0c1c114..4326562 100644 --- a/daemon/job/sink.go +++ b/daemon/job/passive.go @@ -13,43 +13,79 @@ import ( "github.com/zrepl/zrepl/zfs" ) -type Sink struct { +type PassiveSide struct { + mode passiveMode name string l serve.ListenerFactory rpcConf *streamrpc.ConnConfig +} + +type passiveMode interface { + ConnHandleFunc(ctx context.Context, conn serve.AuthenticatedConn) streamrpc.HandlerFunc + Type() Type +} + +type modeSink struct { rootDataset *zfs.DatasetPath } -func SinkFromConfig(g *config.Global, in *config.SinkJob) (s *Sink, err error) { +func (m *modeSink) Type() Type { return TypeSink } - s = &Sink{name: in.Name} +func (m *modeSink) ConnHandleFunc(ctx context.Context, conn serve.AuthenticatedConn) streamrpc.HandlerFunc { + log := GetLogger(ctx) + + clientRootStr := path.Join(m.rootDataset.ToString(), conn.ClientIdentity()) + clientRoot, err := zfs.NewDatasetPath(clientRootStr) + if err != nil { + log.WithError(err). + WithField("client_identity", conn.ClientIdentity()). + Error("cannot build client filesystem map (client identity must be a valid ZFS FS name") + } + log.WithField("client_root", clientRoot).Debug("client root") + + local, err := endpoint.NewReceiver(clientRoot) + if err != nil { + log.WithError(err).Error("unexpected error: cannot convert mapping to filter") + return nil + } + + h := endpoint.NewHandler(local) + return h.Handle +} + +func modeSinkFromConfig(g *config.Global, in *config.SinkJob) (m *modeSink, err error) { + m = &modeSink{} + m.rootDataset, err = zfs.NewDatasetPath(in.RootDataset) + if err != nil { + return nil, errors.New("root dataset is not a valid zfs filesystem path") + } + if m.rootDataset.Length() <= 0 { + return nil, errors.New("root dataset must not be empty") // duplicates error check of receiver + } + return m, nil +} + +func passiveSideFromConfig(g *config.Global, in *config.PassiveJob, mode passiveMode) (s *PassiveSide, err error) { + + s = &PassiveSide{mode: mode, name: in.Name} if s.l, s.rpcConf, err = serve.FromConfig(g, in.Serve); err != nil { return nil, errors.Wrap(err, "cannot build server") } - s.rootDataset, err = zfs.NewDatasetPath(in.RootDataset) - if err != nil { - return nil, errors.New("root dataset is not a valid zfs filesystem path") - } - if s.rootDataset.Length() <= 0 { - return nil, errors.New("root dataset must not be empty") // duplicates error check of receiver - } - - return s, nil } -func (j *Sink) Name() string { return j.name } +func (j *PassiveSide) Name() string { return j.name } -type SinkStatus struct {} +type PassiveStatus struct {} -func (*Sink) Status() *Status { - return &Status{Type: TypeSink} // FIXME SinkStatus +func (s *PassiveSide) Status() *Status { + return &Status{Type: s.mode.Type()} // FIXME PassiveStatus } -func (*Sink) RegisterMetrics(registerer prometheus.Registerer) {} +func (*PassiveSide) RegisterMetrics(registerer prometheus.Registerer) {} -func (j *Sink) Run(ctx context.Context) { +func (j *PassiveSide) Run(ctx context.Context) { log := GetLogger(ctx) defer log.Info("job exiting") @@ -74,10 +110,26 @@ outer: log.WithError(res.err).Info("accept error") continue } + conn := res.conn connId++ connLog := log. WithField("connID", connId) - go j.handleConnection(WithLogger(ctx, connLog), res.conn) + connLog. + WithField("addr", conn.RemoteAddr()). + WithField("client_identity", conn.ClientIdentity()). + Info("handling connection") + go func() { + defer connLog.Info("finished handling connection") + defer conn.Close() + ctx := logging.WithSubsystemLoggers(ctx, connLog) + handleFunc := j.mode.ConnHandleFunc(ctx, conn) + if handleFunc == nil { + return + } + if err := streamrpc.ServeConn(ctx, conn, j.rpcConf, handleFunc); err != nil { + log.WithError(err).Error("error serving client") + } + }() case <-ctx.Done(): break outer @@ -87,39 +139,6 @@ outer: } -func (j *Sink) handleConnection(ctx context.Context, conn serve.AuthenticatedConn) { - defer conn.Close() - - log := GetLogger(ctx) - log. - WithField("addr", conn.RemoteAddr()). - WithField("client_identity", conn.ClientIdentity()). - Info("handling connection") - defer log.Info("finished handling connection") - - clientRootStr := path.Join(j.rootDataset.ToString(), conn.ClientIdentity()) - clientRoot, err := zfs.NewDatasetPath(clientRootStr) - if err != nil { - log.WithError(err). - WithField("client_identity", conn.ClientIdentity()). - Error("cannot build client filesystem map (client identity must be a valid ZFS FS name") - } - log.WithField("client_root", clientRoot).Debug("client root") - - ctx = logging.WithSubsystemLoggers(ctx, log) - - local, err := endpoint.NewReceiver(clientRoot) - if err != nil { - log.WithError(err).Error("unexpected error: cannot convert mapping to filter") - return - } - - handler := endpoint.NewHandler(local) - if err := streamrpc.ServeConn(ctx, conn, j.rpcConf, handler.Handle); err != nil { - log.WithError(err).Error("error serving client") - } -} - type acceptResult struct { conn serve.AuthenticatedConn err error From ffe33aff3d14bd18e240f2389d5641c7d30ee6df Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 24 Sep 2018 12:26:55 +0200 Subject: [PATCH 128/167] fix pruner: protobuf one-ofs require non-zero value, even if no public fields --- daemon/pruner/pruner.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/daemon/pruner/pruner.go b/daemon/pruner/pruner.go index 013c161..83dd1bb 100644 --- a/daemon/pruner/pruner.go +++ b/daemon/pruner/pruner.go @@ -295,7 +295,9 @@ fsloop: rcReq := &pdu.ReplicationCursorReq{ Filesystem: tfs.Path, - Op: &pdu.ReplicationCursorReq_Get{}, + Op: &pdu.ReplicationCursorReq_Get{ + Get: &pdu.ReplicationCursorReq_GetOp{}, + }, } rc, err := receiver.ReplicationCursor(ctx, rcReq) if err != nil { From 9c86e03384551a84bce346f29852b578e504caef Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 24 Sep 2018 12:27:57 +0200 Subject: [PATCH 129/167] endpoint Remote: fix broken Send endpoint for DryRun=true --- endpoint/endpoint.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/endpoint/endpoint.go b/endpoint/endpoint.go index 863e960..a92d41a 100644 --- a/endpoint/endpoint.go +++ b/endpoint/endpoint.go @@ -401,9 +401,13 @@ func (s Remote) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, io.Read if err != nil { return nil, nil, err } - if rs == nil { + if !r.DryRun && rs == nil { return nil, nil, errors.New("response does not contain a stream") } + if r.DryRun && rs != nil { + rs.Close() + return nil, nil, errors.New("response contains unexpected stream (was dry run)") + } var res pdu.SendRes if err := proto.Unmarshal(rb.Bytes(), &res); err != nil { rs.Close() From 6889f441b29275bd270f053fb1962ec1835612cf Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 24 Sep 2018 12:34:53 +0200 Subject: [PATCH 130/167] endpoint: support remote ReplicationCursor endpoint --- endpoint/endpoint.go | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/endpoint/endpoint.go b/endpoint/endpoint.go index a92d41a..fb1347a 100644 --- a/endpoint/endpoint.go +++ b/endpoint/endpoint.go @@ -458,6 +458,26 @@ func (s Remote) DestroySnapshots(ctx context.Context, r *pdu.DestroySnapshotsReq return &res, nil } +func (s Remote) ReplicationCursor(ctx context.Context, req *pdu.ReplicationCursorReq) (*pdu.ReplicationCursorRes, error) { + b, err := proto.Marshal(req) + if err != nil { + return nil, err + } + rb, rs, err := s.c.RequestReply(ctx, RPCReplicationCursor, bytes.NewBuffer(b), nil) + if err != nil { + return nil, err + } + if rs != nil { + rs.Close() + return nil, errors.New("response contains unexpected stream") + } + var res pdu.ReplicationCursorRes + if err := proto.Unmarshal(rb.Bytes(), &res); err != nil { + return nil, err + } + return &res, nil +} + // Handler implements the server-side streamrpc.HandlerFunc for a Remote endpoint stub. type Handler struct { ep replication.Endpoint From d04b9713c4cbf9ea8c2f064ea993d144ae43f3b4 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 24 Sep 2018 12:31:29 +0200 Subject: [PATCH 131/167] implement pull + sink modes for active and passive side --- client/status.go | 2 +- daemon/job/active.go | 53 ++++++++++++++++++++++++++++++++++++++++ daemon/job/build_jobs.go | 19 ++++++++++++++ daemon/job/job.go | 2 ++ daemon/job/passive.go | 48 +++++++++++++++++++++++++++++++++--- 5 files changed, 120 insertions(+), 4 deletions(-) diff --git a/client/status.go b/client/status.go index c72ca10..2ac80ca 100644 --- a/client/status.go +++ b/client/status.go @@ -188,7 +188,7 @@ func (t *tui) draw() { t.setIndent(1) t.newline() - if v.Type != job.TypePush { + if v.Type != job.TypePush && v.Type != job.TypePull { t.printf("No status representation for job type '%s', dumping as YAML", v.Type) t.newline() asYaml, err := yaml.Marshal(v.JobSpecific) diff --git a/daemon/job/active.go b/daemon/job/active.go index 5d20f65..a03871e 100644 --- a/daemon/job/active.go +++ b/daemon/job/active.go @@ -11,9 +11,11 @@ import ( "github.com/zrepl/zrepl/daemon/pruner" "github.com/zrepl/zrepl/endpoint" "github.com/zrepl/zrepl/replication" + "github.com/zrepl/zrepl/zfs" "sync" "github.com/zrepl/zrepl/daemon/logging" "github.com/zrepl/zrepl/daemon/snapper" + "time" ) type ActiveSide struct { @@ -71,6 +73,57 @@ func modePushFromConfig(g *config.Global, in *config.PushJob) (*modePush, error) return m, nil } +type modePull struct { + rootDataset *zfs.DatasetPath + interval time.Duration +} + +func (m *modePull) SenderReceiver(client *streamrpc.Client) (replication.Sender, replication.Receiver, error) { + sender := endpoint.NewRemote(client) + receiver, err := endpoint.NewReceiver(m.rootDataset) + return sender, receiver, err +} + +func (*modePull) Type() Type { return TypePull } + +func (m *modePull) RunPeriodic(ctx context.Context, wakeUpCommon chan<- struct{}) { + t := time.NewTicker(m.interval) + defer t.Stop() + for { + select { + case <-t.C: + select { + case wakeUpCommon <- struct{}{}: + default: + GetLogger(ctx). + WithField("pull_interval", m.interval). + Warn("pull job took longer than pull interval") + wakeUpCommon <- struct{}{} // block anyways, to queue up the wakeup + } + case <-ctx.Done(): + return + } + } +} + +func modePullFromConfig(g *config.Global, in *config.PullJob) (m *modePull, err error) { + m = &modePull{} + if in.Interval <= 0 { + return nil, errors.New("interval must be positive") + } + m.interval = in.Interval + + m.rootDataset, err = zfs.NewDatasetPath(in.RootDataset) + if err != nil { + return nil, errors.New("root dataset is not a valid zfs filesystem path") + } + if m.rootDataset.Length() <= 0 { + return nil, errors.New("root dataset must not be empty") // duplicates error check of receiver + } + + return m, nil +} + func activeSide(g *config.Global, in *config.ActiveJob, mode activeMode) (j *ActiveSide, err error) { j = &ActiveSide{mode: mode} diff --git a/daemon/job/build_jobs.go b/daemon/job/build_jobs.go index 368d9bf..372f51a 100644 --- a/daemon/job/build_jobs.go +++ b/daemon/job/build_jobs.go @@ -22,6 +22,7 @@ func buildJob(c *config.Global, in config.JobEnum) (j Job, err error) { cannotBuildJob := func(e error, name string) (Job, error) { return nil, errors.Wrapf(err, "cannot build job %q", name) } + // FIXME prettify this switch v := in.Ret.(type) { case *config.SinkJob: m, err := modeSinkFromConfig(c, v) @@ -32,6 +33,15 @@ func buildJob(c *config.Global, in config.JobEnum) (j Job, err error) { if err != nil { return cannotBuildJob(err, v.Name) } + case *config.SourceJob: + m, err := modeSourceFromConfig(c, v) + if err != nil { + return cannotBuildJob(err, v.Name) + } + j, err = passiveSideFromConfig(c, &v.PassiveJob, m) + if err != nil { + return cannotBuildJob(err, v.Name) + } case *config.PushJob: m, err := modePushFromConfig(c, v) if err != nil { @@ -41,6 +51,15 @@ func buildJob(c *config.Global, in config.JobEnum) (j Job, err error) { if err != nil { return cannotBuildJob(err, v.Name) } + case *config.PullJob: + m, err := modePullFromConfig(c, v) + if err != nil { + return cannotBuildJob(err, v.Name) + } + j, err = activeSide(c, &v.ActiveJob, m) + if err != nil { + return cannotBuildJob(err, v.Name) + } default: panic(fmt.Sprintf("implementation error: unknown job type %T", v)) } diff --git a/daemon/job/job.go b/daemon/job/job.go index 1cac343..85334a5 100644 --- a/daemon/job/job.go +++ b/daemon/job/job.go @@ -102,10 +102,12 @@ func (s *Status) UnmarshalJSON(in []byte) (err error) { return fmt.Errorf("field '%s', not found", key) } switch s.Type { + case TypePull: fallthrough case TypePush: var st ActiveSideStatus err = json.Unmarshal(jobJSON, &st) s.JobSpecific = &st + case TypeSource: fallthrough case TypeSink: var st PassiveStatus err = json.Unmarshal(jobJSON, &st) diff --git a/daemon/job/passive.go b/daemon/job/passive.go index 4326562..6e5cbb4 100644 --- a/daemon/job/passive.go +++ b/daemon/job/passive.go @@ -6,11 +6,13 @@ import ( "github.com/problame/go-streamrpc" "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon/filters" "github.com/zrepl/zrepl/daemon/logging" "github.com/zrepl/zrepl/daemon/serve" + "github.com/zrepl/zrepl/daemon/snapper" "github.com/zrepl/zrepl/endpoint" - "path" "github.com/zrepl/zrepl/zfs" + "path" ) type PassiveSide struct { @@ -22,6 +24,7 @@ type PassiveSide struct { type passiveMode interface { ConnHandleFunc(ctx context.Context, conn serve.AuthenticatedConn) streamrpc.HandlerFunc + RunPeriodic(ctx context.Context) Type() Type } @@ -53,6 +56,8 @@ func (m *modeSink) ConnHandleFunc(ctx context.Context, conn serve.AuthenticatedC return h.Handle } +func (m *modeSink) RunPeriodic(_ context.Context) {} + func modeSinkFromConfig(g *config.Global, in *config.SinkJob) (m *modeSink, err error) { m = &modeSink{} m.rootDataset, err = zfs.NewDatasetPath(in.RootDataset) @@ -65,6 +70,39 @@ func modeSinkFromConfig(g *config.Global, in *config.SinkJob) (m *modeSink, err return m, nil } +type modeSource struct { + fsfilter zfs.DatasetFilter + snapper *snapper.Snapper +} + +func modeSourceFromConfig(g *config.Global, in *config.SourceJob) (m *modeSource, err error) { + // FIXME exact dedup of modePush + m = &modeSource{} + fsf, err := filters.DatasetMapFilterFromConfig(in.Filesystems) + if err != nil { + return nil, errors.Wrap(err, "cannnot build filesystem filter") + } + m.fsfilter = fsf + + if m.snapper, err = snapper.FromConfig(g, fsf, &in.Snapshotting); err != nil { + return nil, errors.Wrap(err, "cannot build snapper") + } + + return m, nil +} + +func (m *modeSource) Type() Type { return TypeSource } + +func (m *modeSource) ConnHandleFunc(ctx context.Context, conn serve.AuthenticatedConn) streamrpc.HandlerFunc { + sender := endpoint.NewSender(m.fsfilter) + h := endpoint.NewHandler(sender) + return h.Handle +} + +func (m *modeSource) RunPeriodic(ctx context.Context) { + m.snapper.Run(ctx, nil) +} + func passiveSideFromConfig(g *config.Global, in *config.PassiveJob, mode passiveMode) (s *PassiveSide, err error) { s = &PassiveSide{mode: mode, name: in.Name} @@ -97,10 +135,14 @@ func (j *PassiveSide) Run(ctx context.Context) { } defer l.Close() + { + ctx, cancel := context.WithCancel(logging.WithSubsystemLoggers(ctx, log)) // shadowing + defer cancel() + go j.mode.RunPeriodic(ctx) + } + log.WithField("addr", l.Addr()).Debug("accepting connections") - var connId int - outer: for { From 4e04f8d3d22f497f098977b199cfffa26ec3a7a6 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 24 Sep 2018 12:29:34 +0200 Subject: [PATCH 132/167] snapper: make error mode an error wait mode Just because taking one snapshot fails does not mean snapper needs to stop for all others. Since users are advised to monitor error logs, snapshot-taking errors can still be addressed. The ErrorWait mode allows a potential future Report / Status command to distinguish normal waits from error waits. --- daemon/snapper/snapper.go | 14 ++++++++------ daemon/snapper/state_string.go | 2 +- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/daemon/snapper/snapper.go b/daemon/snapper/snapper.go index 2071389..8a6d811 100644 --- a/daemon/snapper/snapper.go +++ b/daemon/snapper/snapper.go @@ -74,7 +74,7 @@ const ( Planning Snapshotting Waiting - Error + ErrorWait ) func (s State) sf() state { @@ -83,7 +83,7 @@ func (s State) sf() state { Planning: plan, Snapshotting: snapshot, Waiting: wait, - Error: nil, + ErrorWait: wait, } return m[s] } @@ -163,7 +163,7 @@ func (s *Snapper) Run(ctx context.Context, snapshotsTaken chan<- struct{}) { func onErr(err error, u updater) state { return u(func(s *Snapper) { s.err = err - s.state = Error + s.state = ErrorWait }).sf() } @@ -254,13 +254,15 @@ func snapshot(a args, u updater) state { select { case a.snapshotsTaken <- struct{}{}: default: - a.log.Warn("callback channel is full, discarding snapshot update event") + if a.snapshotsTaken != nil { + a.log.Warn("callback channel is full, discarding snapshot update event") + } } return u(func(snapper *Snapper) { if hadErr { - snapper.state = Error - snapper.err = errors.New("one or more snapshots could not be created") + snapper.state = ErrorWait + snapper.err = errors.New("one or more snapshots could not be created, check logs for details") } else { snapper.state = Waiting } diff --git a/daemon/snapper/state_string.go b/daemon/snapper/state_string.go index 485e7d9..136baf5 100644 --- a/daemon/snapper/state_string.go +++ b/daemon/snapper/state_string.go @@ -8,7 +8,7 @@ const ( _State_name_0 = "SyncUpPlanning" _State_name_1 = "Snapshotting" _State_name_2 = "Waiting" - _State_name_3 = "Error" + _State_name_3 = "ErrorWait" ) var ( From cf5d63ee88f97dfe4788c7ac3d234ce97f485132 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 24 Sep 2018 12:40:02 +0200 Subject: [PATCH 133/167] config: fix broken tests + reduce example configs --- config/samples/local.yml | 15 --------------- config/samples/pull.yml | 15 --------------- config/samples/pull_ssh.yml | 15 --------------- config/samples/push.yml | 14 -------------- config/samples/sink.yml | 16 ---------------- config/samples/source.yml | 23 ----------------------- config/samples/source_ssh.yml | 23 ----------------------- 7 files changed, 121 deletions(-) diff --git a/config/samples/local.yml b/config/samples/local.yml index 3c56373..56441a2 100644 --- a/config/samples/local.yml +++ b/config/samples/local.yml @@ -24,18 +24,3 @@ jobs: - type: grid grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d keep_bookmarks: all - -global: - logging: - - type: "stdout" - time: true - level: "warn" - format: "human" - monitoring: - - type: "prometheus" - listen: ":9091" - control: - sockpath: /var/run/zrepl/control - serve: - stdinserver: - sockdir: /var/run/zrepl/stdinserver diff --git a/config/samples/pull.yml b/config/samples/pull.yml index 45406c6..4fd162b 100644 --- a/config/samples/pull.yml +++ b/config/samples/pull.yml @@ -22,18 +22,3 @@ jobs: - type: grid grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d keep_bookmarks: all - -global: - logging: - - type: "stdout" - time: true - level: "warn" - format: "human" - monitoring: - - type: "prometheus" - listen: ":9091" - control: - sockpath: /var/run/zrepl/control - serve: - stdinserver: - sockdir: /var/run/zrepl/stdinserver \ No newline at end of file diff --git a/config/samples/pull_ssh.yml b/config/samples/pull_ssh.yml index 9830b01..3e809a6 100644 --- a/config/samples/pull_ssh.yml +++ b/config/samples/pull_ssh.yml @@ -26,18 +26,3 @@ jobs: - type: grid grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d keep_bookmarks: all - -global: - logging: - - type: "stdout" - time: true - level: "warn" - format: "human" - monitoring: - - type: "prometheus" - listen: ":9091" - control: - sockpath: /var/run/zrepl/control - serve: - stdinserver: - sockdir: /var/run/zrepl/stdinserver \ No newline at end of file diff --git a/config/samples/push.yml b/config/samples/push.yml index 862a8f5..b017dad 100644 --- a/config/samples/push.yml +++ b/config/samples/push.yml @@ -24,17 +24,3 @@ jobs: - type: grid grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d keep_bookmarks: all -global: - logging: - - type: "stdout" - time: true - level: "warn" - format: "human" - monitoring: - - type: "prometheus" - listen: ":9091" - control: - sockpath: /var/run/zrepl/control - serve: - stdinserver: - sockdir: /var/run/zrepl/stdinserver \ No newline at end of file diff --git a/config/samples/sink.yml b/config/samples/sink.yml index b927041..01990da 100644 --- a/config/samples/sink.yml +++ b/config/samples/sink.yml @@ -11,19 +11,3 @@ jobs: client_cns: - "laptop1" - "homeserver" -global: - logging: - - type: "tcp" - address: "123.123.123.123:1234" - retry_interval: 10s - tls: - ca: "ca.pem" - cert: "cert.pem" - key: "key.pem" - level: "warn" - format: "human" - control: - sockpath: /var/run/zrepl/control - serve: - stdinserver: - sockdir: /var/run/zrepl/stdinserver \ No newline at end of file diff --git a/config/samples/source.yml b/config/samples/source.yml index 128cfa1..9162421 100644 --- a/config/samples/source.yml +++ b/config/samples/source.yml @@ -14,26 +14,3 @@ jobs: snapshotting: snapshot_prefix: zrepl_ interval: 10m - pruning: - keep: - - type: not_replicated - - type: last_n - count: 10 - - type: grid - grid: 1x1h(keep=all) | 24x1h | 14x1d - keep_bookmarks: all - -global: - logging: - - type: "stdout" - time: true - level: "warn" - format: "human" - monitoring: - - type: "prometheus" - listen: ":9091" - control: - sockpath: /var/run/zrepl/control - serve: - stdinserver: - sockdir: /var/run/zrepl/stdinserver \ No newline at end of file diff --git a/config/samples/source_ssh.yml b/config/samples/source_ssh.yml index b1c034d..b1e6e11 100644 --- a/config/samples/source_ssh.yml +++ b/config/samples/source_ssh.yml @@ -13,26 +13,3 @@ jobs: snapshotting: snapshot_prefix: zrepl_ interval: 10m - pruning: - keep: - - type: not_replicated - - type: last_n - count: 10 - - type: grid - grid: 1x1h(keep=all) | 24x1h | 14x1d - keep_bookmarks: all - -global: - logging: - - type: "stdout" - time: true - level: "warn" - format: "human" - monitoring: - - type: "prometheus" - listen: ":9091" - control: - sockpath: /var/run/zrepl/control - serve: - stdinserver: - sockdir: /var/run/zrepl/stdinserver \ No newline at end of file From f3e8eda04dfb9b88ed9624e57dbbe9435e7e923e Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 24 Sep 2018 14:40:47 +0200 Subject: [PATCH 134/167] fixup 4e04f8d3d22f497f098977b199cfffa26ec3a7a6: snapper with separate stopped state for clean shutdown would tight loop in ErrorWait --- daemon/snapper/snapper.go | 13 +++++++++++-- daemon/snapper/state_string.go | 3 +++ 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/daemon/snapper/snapper.go b/daemon/snapper/snapper.go index 8a6d811..bfaeb47 100644 --- a/daemon/snapper/snapper.go +++ b/daemon/snapper/snapper.go @@ -75,6 +75,7 @@ const ( Snapshotting Waiting ErrorWait + Stopped ) func (s State) sf() state { @@ -84,6 +85,7 @@ func (s State) sf() state { Snapshotting: snapshot, Waiting: wait, ErrorWait: wait, + Stopped: nil, } return m[s] } @@ -167,6 +169,13 @@ func onErr(err error, u updater) state { }).sf() } +func onMainCtxDone(ctx context.Context, u updater) state { + return u(func(s *Snapper) { + s.err = ctx.Err() + s.state = Stopped + }).sf() +} + func syncUp(a args, u updater) state { fss, err := listFSes(a.fsf) if err != nil { @@ -187,7 +196,7 @@ func syncUp(a args, u updater) state { s.state = Planning }).sf() case <-a.ctx.Done(): - return onErr(err, u) + return onMainCtxDone(a.ctx, u) } } @@ -286,7 +295,7 @@ func wait(a args, u updater) state { snapper.state = Planning }).sf() case <-a.ctx.Done(): - return onErr(a.ctx.Err(), u) + return onMainCtxDone(a.ctx, u) } } diff --git a/daemon/snapper/state_string.go b/daemon/snapper/state_string.go index 136baf5..53796b0 100644 --- a/daemon/snapper/state_string.go +++ b/daemon/snapper/state_string.go @@ -9,6 +9,7 @@ const ( _State_name_1 = "Snapshotting" _State_name_2 = "Waiting" _State_name_3 = "ErrorWait" + _State_name_4 = "Stopped" ) var ( @@ -26,6 +27,8 @@ func (i State) String() string { return _State_name_2 case i == 16: return _State_name_3 + case i == 32: + return _State_name_4 default: return "State(" + strconv.FormatInt(int64(i), 10) + ")" } From 1ce0c69e4f4e787bc0c00d30684bb60ba278482d Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 24 Sep 2018 14:43:53 +0200 Subject: [PATCH 135/167] implement local replication using new local transport The new local transport uses socketpair() and a switchboard based on client identities. The special local job type is gone, which is good since it does not fit into the 'Active/Passive side ' + 'mode' concept used to implement the duality of push/sink | pull/source. --- config/config.go | 22 ++-- config/samples/local.yml | 48 +++---- daemon/connecter/connect_local.go | 26 ++++ daemon/connecter/connecter.go | 3 + daemon/serve/serve.go | 3 + daemon/serve/serve_local.go | 211 ++++++++++++++++++++++++++++++ 6 files changed, 279 insertions(+), 34 deletions(-) create mode 100644 daemon/connecter/connect_local.go create mode 100644 daemon/serve/serve_local.go diff --git a/config/config.go b/config/config.go index 081e335..e53a1bf 100644 --- a/config/config.go +++ b/config/config.go @@ -59,16 +59,6 @@ type SourceJob struct { Filesystems FilesystemsFilter `yaml:"filesystems"` } -type LocalJob struct { - Type string `yaml:"type"` - Name string `yaml:"name"` - Filesystems FilesystemsFilter `yaml:"filesystems"` - RootDataset string `yaml:"root_dataset"` - Snapshotting Snapshotting `yaml:"snapshotting"` - Pruning PruningSenderReceiver `yaml:"pruning"` - Debug JobDebugSettings `yaml:"debug,optional"` -} - type FilesystemsFilter map[string]bool type Snapshotting struct { @@ -171,6 +161,11 @@ type SSHStdinserverConnect struct { DialTimeout time.Duration `yaml:"dial_timeout,positive,default=10s"` } +type LocalConnect struct { + ConnectCommon `yaml:",inline"` + ClientIdentity string `yaml:"client_identity"` +} + type ServeEnum struct { Ret interface{} } @@ -201,6 +196,10 @@ type StdinserverServer struct { ClientIdentities []string `yaml:"client_identities"` } +type LocalServe struct { + ServeCommon `yaml:",inline"` +} + type PruningEnum struct { Ret interface{} } @@ -311,7 +310,6 @@ func (t *JobEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { "sink": &SinkJob{}, "pull": &PullJob{}, "source": &SourceJob{}, - "local": &LocalJob{}, }) return } @@ -321,6 +319,7 @@ func (t *ConnectEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) "tcp": &TCPConnect{}, "tls": &TLSConnect{}, "ssh+stdinserver": &SSHStdinserverConnect{}, + "local": &LocalConnect{}, }) return } @@ -330,6 +329,7 @@ func (t *ServeEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { "tcp": &TCPServe{}, "tls": &TLSServe{}, "stdinserver": &StdinserverServer{}, + "local" : &LocalServe{}, }) return } diff --git a/config/samples/local.yml b/config/samples/local.yml index 56441a2..01d73f0 100644 --- a/config/samples/local.yml +++ b/config/samples/local.yml @@ -1,26 +1,28 @@ jobs: -- name: mirror_local - type: local - # snapshot the filesystems matched by the left-hand-side of the mapping - # every 10m with zrepl_ as prefix - filesystems: { - "pool1/var/db<": true, - "pool1/usr/home<": true, - "pool1/usr/home/paranoid": false, #don't backup paranoid user - "pool1/poudriere/ports<": false #don't backup the ports trees - } - # TODO FIXME enforce that the tree under root_dataset and the trees allowed (true) by filesystems are non-overlapping - root_dataset: "pool2/backups/pool1" + - type: sink + name: "local_sink" + root_dataset: "storage/zrepl/sink" + serve: + type: local - snapshotting: - snapshot_prefix: zrepl_ - interval: 10m - - pruning: - keep_sender: - - type: not_replicated - keep_receiver: - - type: grid - grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d - keep_bookmarks: all + - type: push + name: "backup_system" + connect: + type: local + client_identity: local_backup + filesystems: { + "system<": true, + } + snapshotting: + snapshot_prefix: zrepl_ + interval: 10m + pruning: + keep_sender: + - type: not_replicated + - type: last_n + count: 10 + keep_receiver: + - type: grid + grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d + keep_bookmarks: all diff --git a/daemon/connecter/connect_local.go b/daemon/connecter/connect_local.go new file mode 100644 index 0000000..252a9b6 --- /dev/null +++ b/daemon/connecter/connect_local.go @@ -0,0 +1,26 @@ +package connecter + +import ( + "context" + "fmt" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon/serve" + "net" +) + +type LocalConnecter struct { + clientIdentity string +} + +func LocalConnecterFromConfig(in *config.LocalConnect) (*LocalConnecter, error) { + if in.ClientIdentity == "" { + return nil, fmt.Errorf("ClientIdentity must not be empty") + } + return &LocalConnecter{in.ClientIdentity}, nil +} + +func (c *LocalConnecter) Connect(dialCtx context.Context) (conn net.Conn, err error) { + switchboard := serve.GetLocalListenerSwitchboard() + return switchboard.DialContext(dialCtx, c.clientIdentity) +} + diff --git a/daemon/connecter/connecter.go b/daemon/connecter/connecter.go index 4708cb3..3c0f0de 100644 --- a/daemon/connecter/connecter.go +++ b/daemon/connecter/connecter.go @@ -23,6 +23,9 @@ func FromConfig(g *config.Global, in config.ConnectEnum) (*ClientFactory, error) case *config.TLSConnect: connecter, errConnecter = TLSConnecterFromConfig(v) connConf, errRPC = streamrpcconfig.FromDaemonConfig(g, v.RPC) + case *config.LocalConnect: + connecter, errConnecter = LocalConnecterFromConfig(v) + connConf, errRPC = streamrpcconfig.FromDaemonConfig(g, v.RPC) default: panic(fmt.Sprintf("implementation error: unknown connecter type %T", v)) } diff --git a/daemon/serve/serve.go b/daemon/serve/serve.go index 8000a94..15e6ba3 100644 --- a/daemon/serve/serve.go +++ b/daemon/serve/serve.go @@ -86,6 +86,9 @@ func FromConfig(g *config.Global, in config.ServeEnum) (lf ListenerFactory, conf case *config.StdinserverServer: lf, lfError = MultiStdinserverListenerFactoryFromConfig(g, v) conf, rpcErr = streamrpcconfig.FromDaemonConfig(g, v.RPC) + case *config.LocalServe: + lf, lfError = LocalListenerFactoryFromConfig(g, v) + conf, rpcErr = streamrpcconfig.FromDaemonConfig(g, v.RPC) default: return nil, nil, errors.Errorf("internal error: unknown serve type %T", v) } diff --git a/daemon/serve/serve_local.go b/daemon/serve/serve_local.go new file mode 100644 index 0000000..ba122ae --- /dev/null +++ b/daemon/serve/serve_local.go @@ -0,0 +1,211 @@ +package serve + +import ( + "context" + "fmt" + "github.com/zrepl/zrepl/config" + "golang.org/x/sys/unix" + "net" + "os" + "sync" +) + +var localListenerSwitchboardSingleton struct { + s *LocalListenerSwitchboard + once sync.Once +} + +func GetLocalListenerSwitchboard() (*LocalListenerSwitchboard) { + localListenerSwitchboardSingleton.once.Do(func() { + localListenerSwitchboardSingleton.s = &LocalListenerSwitchboard{ + connects: make(chan connectRequest), + } + }) + return localListenerSwitchboardSingleton.s +} + +type connectRequest struct { + clientIdentity string + callback chan connectResult +} + +type connectResult struct { + conn net.Conn + err error +} + +type LocalListenerSwitchboard struct { + connects chan connectRequest +} + +func (l *LocalListenerSwitchboard) DialContext(dialCtx context.Context, clientIdentity string) (conn net.Conn, err error) { + + // place request + req := connectRequest{ + clientIdentity: clientIdentity, + callback: make(chan connectResult), + } + select { + case l.connects <- req: + case <-dialCtx.Done(): + return nil, dialCtx.Err() + } + + // wait for listener response + select { + case connRes := <- req.callback: + conn, err = connRes.conn, connRes.err + case <-dialCtx.Done(): + close(req.callback) // sending to the channel afterwards will panic, the listener has to catch this + conn, err = nil, dialCtx.Err() + } + + return conn, err +} + +type localAddr struct { + S string +} + +func (localAddr) Network() string { return "local" } + +func (a localAddr) String() string { return a.S } + +func (l *LocalListenerSwitchboard) Addr() (net.Addr) { return localAddr{""} } + +type localConn struct { + net.Conn + clientIdentity string +} + +func (l localConn) ClientIdentity() string { return l.clientIdentity } + +func (l *LocalListenerSwitchboard) Accept(ctx context.Context) (AuthenticatedConn, error) { + respondToRequest := func(req connectRequest, res connectResult) (err error) { + getLogger(ctx). + WithField("res.conn", res.conn).WithField("res.err", res.err). + Debug("responding to client request") + defer func() { + errv := recover() + getLogger(ctx).WithField("recover_err", errv). + Debug("panic on send to client callback, likely a legitimate client-side timeout") + }() + select { + case req.callback <- res: + err = nil + default: + err = fmt.Errorf("client-provided callback did block on send") + } + close(req.callback) + return err + } + + getLogger(ctx).Debug("waiting for local client connect requests") + var req connectRequest + select { + case req = <-l.connects: + case <-ctx.Done(): + return nil, ctx.Err() + } + + getLogger(ctx).WithField("client_identity", req.clientIdentity).Debug("got connect request") + if req.clientIdentity == "" { + res := connectResult{nil, fmt.Errorf("client identity must not be empty")} + if err := respondToRequest(req, res); err != nil { + return nil, err + } + return nil, fmt.Errorf("client connected with empty client identity") + } + + getLogger(ctx).Debug("creating socketpair") + left, right, err := makeSocketpairConn() + if err != nil { + res := connectResult{nil, fmt.Errorf("server error: %s", err)} + if respErr := respondToRequest(req, res); respErr != nil { + // returning the socketpair error properly is more important than the error sent to the client + getLogger(ctx).WithError(respErr).Error("error responding to client") + } + return nil, err + } + + getLogger(ctx).Debug("responding with left side of socketpair") + res := connectResult{left, nil} + if err := respondToRequest(req, res); err != nil { + getLogger(ctx).WithError(err).Error("error responding to client") + if err := left.Close(); err != nil { + getLogger(ctx).WithError(err).Error("cannot close left side of socketpair") + } + if err := right.Close(); err != nil { + getLogger(ctx).WithError(err).Error("cannot close right side of socketpair") + } + return nil, err + } + + return localConn{right, req.clientIdentity}, nil +} + +type fileConn struct { + net.Conn // net.FileConn + f *os.File +} + +func (c fileConn) Close() error { + if err := c.Conn.Close(); err != nil { + return err + } + if err := c.f.Close(); err != nil { + return err + } + return nil +} + +func makeSocketpairConn() (a, b net.Conn, err error) { + // don't use net.Pipe, as it doesn't implement things like lingering, which our code relies on + sockpair, err := unix.Socketpair(unix.AF_UNIX, unix.SOCK_STREAM, 0) + if err != nil { + return nil, nil, err + } + toConn := func(fd int) (net.Conn, error) { + f := os.NewFile(uintptr(fd), "fileconn") + if f == nil { + panic(fd) + } + c, err := net.FileConn(f) + if err != nil { + f.Close() + return nil, err + } + return fileConn{Conn: c, f: f}, nil + } + if a, err = toConn(sockpair[0]); err != nil { // shadowing + return nil, nil, err + } + if b, err = toConn(sockpair[1]); err != nil { // shadowing + a.Close() + return nil, nil, err + } + return a, b, nil +} + +func (l *LocalListenerSwitchboard) Close() error { + // FIXME: make sure concurrent Accepts return with error, and further Accepts return that error, too + // Example impl: for each accept, do context.WithCancel, and store the cancel in a list + // When closing, set a member variable to state=closed, make sure accept will exit early + // and then call all cancels in the list + // The code path from Accept entry over check if state=closed to list entry must be protected by a mutex. + return nil +} + +type LocalListenerFactory struct { + clients []string +} + +func LocalListenerFactoryFromConfig(g *config.Global, in *config.LocalServe) (f *LocalListenerFactory, err error) { + return &LocalListenerFactory{}, nil +} + + +func (*LocalListenerFactory) Listen() (AuthenticatedListener, error) { + return GetLocalListenerSwitchboard(), nil +} + From 328ac687f670888cd12fce33681fa893dd507cbe Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 24 Sep 2018 14:48:12 +0200 Subject: [PATCH 136/167] Remove obsolete cmd/** package + subpackages --- cmd/adaptors.go | 103 ------- cmd/autosnap.go | 192 ------------ cmd/bashcomp.go | 29 -- cmd/config.go | 94 ------ cmd/config_connect.go | 119 -------- cmd/config_fsvfilter.go | 42 --- cmd/config_job_local.go | 185 ------------ cmd/config_job_pull.go | 168 ----------- cmd/config_job_source.go | 172 ----------- cmd/config_logging.go | 194 ------------ cmd/config_mapfilter.go | 274 ----------------- cmd/config_parse.go | 211 ------------- cmd/config_prune_none.go | 11 - cmd/config_serve_stdinserver.go | 58 ---- cmd/config_serve_tcp.go | 25 -- cmd/config_serve_tls.go | 78 ----- cmd/config_test.go | 283 ------------------ cmd/control.go | 156 ---------- cmd/daemon.deact/control.go | 142 --------- cmd/daemon.deact/daemon.go | 174 ----------- cmd/daemon.deact/job/job.go | 47 --- cmd/daemon.deact/pprof.go | 80 ----- cmd/daemon.deact/prometheus.go | 82 ----- cmd/daemon.go | 176 ----------- cmd/logging_formatters.go | 201 ------------- cmd/logging_outlets.go | 161 ---------- cmd/main.go | 43 --- cmd/prune.go | 123 -------- cmd/sampleconf/localbackup/host1.yml | 28 -- cmd/sampleconf/pullbackup/backuphost.yml | 26 -- cmd/sampleconf/pullbackup/productionhost.yml | 47 --- cmd/sampleconf/pushbackup/backuphost.yml | 20 -- cmd/sampleconf/pushbackup/productionhost.yml | 26 -- cmd/sampleconf/random/debugging.yml | 33 -- cmd/sampleconf/random/logging/client.crt | 19 -- cmd/sampleconf/random/logging/client.csr | 16 - cmd/sampleconf/random/logging/client.key | 28 -- cmd/sampleconf/random/logging/logserver.crt | 21 -- cmd/sampleconf/random/logging/logserver.key | 28 -- .../random/logging_and_monitoring.yml | 28 -- cmd/stdinserver.go | 55 ---- cmd/test.go | 214 ------------- cmd/version.go | 21 -- 43 files changed, 4233 deletions(-) delete mode 100644 cmd/adaptors.go delete mode 100644 cmd/autosnap.go delete mode 100644 cmd/bashcomp.go delete mode 100644 cmd/config.go delete mode 100644 cmd/config_connect.go delete mode 100644 cmd/config_fsvfilter.go delete mode 100644 cmd/config_job_local.go delete mode 100644 cmd/config_job_pull.go delete mode 100644 cmd/config_job_source.go delete mode 100644 cmd/config_logging.go delete mode 100644 cmd/config_mapfilter.go delete mode 100644 cmd/config_parse.go delete mode 100644 cmd/config_prune_none.go delete mode 100644 cmd/config_serve_stdinserver.go delete mode 100644 cmd/config_serve_tcp.go delete mode 100644 cmd/config_serve_tls.go delete mode 100644 cmd/config_test.go delete mode 100644 cmd/control.go delete mode 100644 cmd/daemon.deact/control.go delete mode 100644 cmd/daemon.deact/daemon.go delete mode 100644 cmd/daemon.deact/job/job.go delete mode 100644 cmd/daemon.deact/pprof.go delete mode 100644 cmd/daemon.deact/prometheus.go delete mode 100644 cmd/daemon.go delete mode 100644 cmd/logging_formatters.go delete mode 100644 cmd/logging_outlets.go delete mode 100644 cmd/main.go delete mode 100644 cmd/prune.go delete mode 100644 cmd/sampleconf/localbackup/host1.yml delete mode 100644 cmd/sampleconf/pullbackup/backuphost.yml delete mode 100644 cmd/sampleconf/pullbackup/productionhost.yml delete mode 100644 cmd/sampleconf/pushbackup/backuphost.yml delete mode 100644 cmd/sampleconf/pushbackup/productionhost.yml delete mode 100644 cmd/sampleconf/random/debugging.yml delete mode 100644 cmd/sampleconf/random/logging/client.crt delete mode 100644 cmd/sampleconf/random/logging/client.csr delete mode 100644 cmd/sampleconf/random/logging/client.key delete mode 100644 cmd/sampleconf/random/logging/logserver.crt delete mode 100644 cmd/sampleconf/random/logging/logserver.key delete mode 100644 cmd/sampleconf/random/logging_and_monitoring.yml delete mode 100644 cmd/stdinserver.go delete mode 100644 cmd/test.go delete mode 100644 cmd/version.go diff --git a/cmd/adaptors.go b/cmd/adaptors.go deleted file mode 100644 index acf1a0d..0000000 --- a/cmd/adaptors.go +++ /dev/null @@ -1,103 +0,0 @@ -package cmd - -import ( - "context" - "fmt" - "io" - "net" - "strings" - "time" - - "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/logger" - "github.com/zrepl/zrepl/util" -) - -type logNetConnConnecter struct { - streamrpc.Connecter - ReadDump, WriteDump string -} - -var _ streamrpc.Connecter = logNetConnConnecter{} - -func (l logNetConnConnecter) Connect(ctx context.Context) (net.Conn, error) { - conn, err := l.Connecter.Connect(ctx) - if err != nil { - return nil, err - } - return util.NewNetConnLogger(conn, l.ReadDump, l.WriteDump) -} - -type logListenerFactory struct { - ListenerFactory - ReadDump, WriteDump string -} - -var _ ListenerFactory = logListenerFactory{} - -type logListener struct { - net.Listener - ReadDump, WriteDump string -} - -var _ net.Listener = logListener{} - -func (m logListenerFactory) Listen() (net.Listener, error) { - l, err := m.ListenerFactory.Listen() - if err != nil { - return nil, err - } - return logListener{l, m.ReadDump, m.WriteDump}, nil -} - -func (l logListener) Accept() (net.Conn, error) { - conn, err := l.Listener.Accept() - if err != nil { - return nil, err - } - return util.NewNetConnLogger(conn, l.ReadDump, l.WriteDump) -} - -type netsshAddr struct{} - -func (netsshAddr) Network() string { return "netssh" } -func (netsshAddr) String() string { return "???" } - -type netsshConnToNetConnAdatper struct { - io.ReadWriteCloser // works for both netssh.SSHConn and netssh.ServeConn -} - -func (netsshConnToNetConnAdatper) LocalAddr() net.Addr { return netsshAddr{} } - -func (netsshConnToNetConnAdatper) RemoteAddr() net.Addr { return netsshAddr{} } - -func (netsshConnToNetConnAdatper) SetDeadline(t time.Time) error { return nil } - -func (netsshConnToNetConnAdatper) SetReadDeadline(t time.Time) error { return nil } - -func (netsshConnToNetConnAdatper) SetWriteDeadline(t time.Time) error { return nil } - -type streamrpcLogAdaptor = twoClassLogAdaptor -type replicationLogAdaptor = twoClassLogAdaptor - -type twoClassLogAdaptor struct { - logger.Logger -} - -var _ streamrpc.Logger = twoClassLogAdaptor{} - -func (a twoClassLogAdaptor) Errorf(fmtStr string, args ...interface{}) { - const errorSuffix = ": %s" - if len(args) == 1 { - if err, ok := args[0].(error); ok && strings.HasSuffix(fmtStr, errorSuffix) { - msg := strings.TrimSuffix(fmtStr, errorSuffix) - a.WithError(err).Error(msg) - return - } - } - a.Logger.Error(fmt.Sprintf(fmtStr, args...)) -} - -func (a twoClassLogAdaptor) Infof(fmtStr string, args ...interface{}) { - a.Logger.Info(fmt.Sprintf(fmtStr, args...)) -} diff --git a/cmd/autosnap.go b/cmd/autosnap.go deleted file mode 100644 index 983513f..0000000 --- a/cmd/autosnap.go +++ /dev/null @@ -1,192 +0,0 @@ -package cmd - -import ( - "context" - "fmt" - "github.com/zrepl/zrepl/zfs" - "sort" - "time" -) - -type IntervalAutosnap struct { - DatasetFilter zfs.DatasetFilter - Prefix string - SnapshotInterval time.Duration -} - -func (a *IntervalAutosnap) filterFilesystems(ctx context.Context) (fss []*zfs.DatasetPath, stop bool) { - fss, err := zfs.ZFSListMapping(a.DatasetFilter) - stop = err != nil - if err != nil { - getLogger(ctx).WithError(err).Error("cannot list datasets") - } - if len(fss) == 0 { - getLogger(ctx).Warn("no filesystem matching filesystem filter") - } - return fss, stop -} - -func (a *IntervalAutosnap) findSyncPoint(log Logger, fss []*zfs.DatasetPath) (syncPoint time.Time, err error) { - type snapTime struct { - ds *zfs.DatasetPath - time time.Time - } - - if len(fss) == 0 { - return time.Now(), nil - } - - snaptimes := make([]snapTime, 0, len(fss)) - - now := time.Now() - - log.Debug("examine filesystem state") - for _, d := range fss { - - l := log.WithField("fs", d.ToString()) - - fsvs, err := zfs.ZFSListFilesystemVersions(d, NewPrefixFilter(a.Prefix)) - if err != nil { - l.WithError(err).Error("cannot list filesystem versions") - continue - } - if len(fsvs) <= 0 { - l.WithField("prefix", a.Prefix).Info("no filesystem versions with prefix") - continue - } - - // Sort versions by creation - sort.SliceStable(fsvs, func(i, j int) bool { - return fsvs[i].CreateTXG < fsvs[j].CreateTXG - }) - - latest := fsvs[len(fsvs)-1] - l.WithField("creation", latest.Creation). - Debug("found latest snapshot") - - since := now.Sub(latest.Creation) - if since < 0 { - l.WithField("snapshot", latest.Name). - WithField("creation", latest.Creation). - Error("snapshot is from the future") - continue - } - next := now - if since < a.SnapshotInterval { - next = latest.Creation.Add(a.SnapshotInterval) - } - snaptimes = append(snaptimes, snapTime{d, next}) - } - - if len(snaptimes) == 0 { - snaptimes = append(snaptimes, snapTime{nil, now}) - } - - sort.Slice(snaptimes, func(i, j int) bool { - return snaptimes[i].time.Before(snaptimes[j].time) - }) - - return snaptimes[0].time, nil - -} - -func (a *IntervalAutosnap) waitForSyncPoint(ctx context.Context, syncPoint time.Time) { - - const LOG_TIME_FMT string = time.ANSIC - - getLogger(ctx). - WithField("sync_point", syncPoint.Format(LOG_TIME_FMT)). - Info("wait for sync point") - - select { - case <-ctx.Done(): - getLogger(ctx).WithError(ctx.Err()).Info("context done") - return - case <-time.After(syncPoint.Sub(time.Now())): - } -} - -func (a *IntervalAutosnap) syncUpRun(ctx context.Context, didSnaps chan struct{}) (stop bool) { - fss, stop := a.filterFilesystems(ctx) - if stop { - return true - } - - syncPoint, err := a.findSyncPoint(getLogger(ctx), fss) - if err != nil { - return true - } - - a.waitForSyncPoint(ctx, syncPoint) - - getLogger(ctx).Debug("snapshot all filesystems to enable further snaps in lockstep") - a.doSnapshots(ctx, didSnaps) - return false -} - -func (a *IntervalAutosnap) Run(ctx context.Context, didSnaps chan struct{}) { - - log := getLogger(ctx) - - if a.syncUpRun(ctx, didSnaps) { - log.Error("stoppping autosnap after error in sync up") - return - } - - // task drops back to idle here - - log.Debug("setting up ticker in SnapshotInterval") - ticker := time.NewTicker(a.SnapshotInterval) - for { - select { - case <-ctx.Done(): - ticker.Stop() - log.WithError(ctx.Err()).Info("context done") - return - case <-ticker.C: - a.doSnapshots(ctx, didSnaps) - } - } - -} - -func (a *IntervalAutosnap) doSnapshots(ctx context.Context, didSnaps chan struct{}) { - log := getLogger(ctx) - - // don't cache the result from previous run in case the user added - // a new dataset in the meantime - ds, stop := a.filterFilesystems(ctx) - if stop { - return - } - - // TODO channel programs -> allow a little jitter? - for _, d := range ds { - suffix := time.Now().In(time.UTC).Format("20060102_150405_000") - snapname := fmt.Sprintf("%s%s", a.Prefix, suffix) - - l := log. - WithField("fs", d.ToString()). - WithField("snapname", snapname) - - l.Info("create snapshot") - err := zfs.ZFSSnapshot(d, snapname, false) - if err != nil { - l.WithError(err).Error("cannot create snapshot") - } - - l.Info("create corresponding bookmark") - err = zfs.ZFSBookmark(d, snapname, snapname) - if err != nil { - l.WithError(err).Error("cannot create bookmark") - } - - } - - select { - case didSnaps <- struct{}{}: - default: - log.Error("warning: callback channel is full, discarding") - } - -} diff --git a/cmd/bashcomp.go b/cmd/bashcomp.go deleted file mode 100644 index 3b39252..0000000 --- a/cmd/bashcomp.go +++ /dev/null @@ -1,29 +0,0 @@ -package cmd - -import ( - "fmt" - - "github.com/spf13/cobra" - "os" -) - -var bashcompCmd = &cobra.Command{ - Use: "bashcomp path/to/out/file", - Short: "generate bash completions", - Run: func(cmd *cobra.Command, args []string) { - if len(args) != 1 { - fmt.Fprintf(os.Stderr, "specify exactly one positional agument\n") - cmd.Usage() - os.Exit(1) - } - if err := RootCmd.GenBashCompletionFile(args[0]); err != nil { - fmt.Fprintf(os.Stderr, "error generating bash completion: %s", err) - os.Exit(1) - } - }, - Hidden: true, -} - -func init() { - RootCmd.AddCommand(bashcompCmd) -} diff --git a/cmd/config.go b/cmd/config.go deleted file mode 100644 index 69ce086..0000000 --- a/cmd/config.go +++ /dev/null @@ -1,94 +0,0 @@ -package cmd - -import ( - "net" - - "fmt" - "github.com/pkg/errors" - "github.com/zrepl/zrepl/zfs" -) - -type Config struct { - Global Global - Jobs map[string]Job -} - -func (c *Config) LookupJob(name string) (j Job, err error) { - j, ok := c.Jobs[name] - if !ok { - return nil, errors.Errorf("job '%s' is not defined", name) - } - return j, nil -} - -type Global struct { - Serve struct { - Stdinserver struct { - SockDir string - } - } - Control struct { - Sockpath string - } - logging *LoggingConfig -} - -type JobDebugSettings struct { - Conn struct { - ReadDump string `mapstructure:"read_dump"` - WriteDump string `mapstructure:"write_dump"` - } - RPC struct { - Log bool - } -} - -type ListenerFactory interface { - Listen() (net.Listener, error) -} - -type SSHStdinServerConnectDescr struct { -} - -type PrunePolicy interface { - // Prune filters versions and decide which to keep and which to remove. - // Prune **does not** implement the actual removal of the versions. - Prune(fs *zfs.DatasetPath, versions []zfs.FilesystemVersion) (keep, remove []zfs.FilesystemVersion, err error) -} - -type PruningJob interface { - Pruner(side PrunePolicySide, dryRun bool) (Pruner, error) -} - -// A type for constants describing different prune policies of a PruningJob -// This is mostly a special-case for LocalJob, which is the only job that has two prune policies -// instead of one. -// It implements github.com/spf13/pflag.Value to be used as CLI flag for the test subcommand -type PrunePolicySide string - -const ( - PrunePolicySideDefault PrunePolicySide = "" - PrunePolicySideLeft PrunePolicySide = "left" - PrunePolicySideRight PrunePolicySide = "right" -) - -func (s *PrunePolicySide) String() string { - return string(*s) -} - -func (s *PrunePolicySide) Set(news string) error { - p := PrunePolicySide(news) - switch p { - case PrunePolicySideRight: - fallthrough - case PrunePolicySideLeft: - *s = p - default: - return errors.Errorf("must be either %s or %s", PrunePolicySideLeft, PrunePolicySideRight) - } - return nil -} - -func (s *PrunePolicySide) Type() string { - return fmt.Sprintf("%s | %s", PrunePolicySideLeft, PrunePolicySideRight) -} diff --git a/cmd/config_connect.go b/cmd/config_connect.go deleted file mode 100644 index ea71ac3..0000000 --- a/cmd/config_connect.go +++ /dev/null @@ -1,119 +0,0 @@ -package cmd - -import ( - "crypto/tls" - "net" - - "context" - "github.com/jinzhu/copier" - "github.com/pkg/errors" - "github.com/problame/go-netssh" - "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/cmd/tlsconf" - "github.com/zrepl/zrepl/config" - "time" -) - -type SSHStdinserverConnecter struct { - Host string - User string - Port uint16 - IdentityFile string - TransportOpenCommand []string - SSHCommand string - Options []string - dialTimeout time.Duration -} - -var _ streamrpc.Connecter = &SSHStdinserverConnecter{} - -func parseSSHStdinserverConnecter(in config.SSHStdinserverConnect) (c *SSHStdinserverConnecter, err error) { - - c = &SSHStdinserverConnecter{ - Host: in.Host, - User: in.User, - Port: in.Port, - IdentityFile: in.IdentityFile, - SSHCommand: in.SSHCommand, - Options: in.Options, - dialTimeout: in.DialTimeout, - } - return - -} - -type netsshConnToConn struct{ *netssh.SSHConn } - -var _ net.Conn = netsshConnToConn{} - -func (netsshConnToConn) SetDeadline(dl time.Time) error { return nil } -func (netsshConnToConn) SetReadDeadline(dl time.Time) error { return nil } -func (netsshConnToConn) SetWriteDeadline(dl time.Time) error { return nil } - -func (c *SSHStdinserverConnecter) Connect(dialCtx context.Context) (net.Conn, error) { - - var endpoint netssh.Endpoint - if err := copier.Copy(&endpoint, c); err != nil { - return nil, errors.WithStack(err) - } - dialCtx, dialCancel := context.WithTimeout(dialCtx, c.dialTimeout) // context.TODO tied to error handling below - defer dialCancel() - nconn, err := netssh.Dial(dialCtx, endpoint) - if err != nil { - if err == context.DeadlineExceeded { - err = errors.Errorf("dial_timeout of %s exceeded", c.dialTimeout) - } - return nil, err - } - return netsshConnToConn{nconn}, nil -} - -type TCPConnecter struct { - Address string - dialer net.Dialer -} - -func parseTCPConnecter(in config.TCPConnect) (*TCPConnecter, error) { - dialer := net.Dialer{ - Timeout: in.DialTimeout, - } - - return &TCPConnecter{in.Address, dialer}, nil -} - -func (c *TCPConnecter) Connect(dialCtx context.Context) (conn net.Conn, err error) { - return c.dialer.DialContext(dialCtx, "tcp", c.Address) -} - -type TLSConnecter struct { - Address string - dialer net.Dialer - tlsConfig *tls.Config -} - -func parseTLSConnecter(in config.TLSConnect) (*TLSConnecter, error) { - dialer := net.Dialer{ - Timeout: in.DialTimeout, - } - - ca, err := tlsconf.ParseCAFile(in.Ca) - if err != nil { - return nil, errors.Wrap(err, "cannot parse ca file") - } - - cert, err := tls.LoadX509KeyPair(in.Cert, in.Key) - if err != nil { - return nil, errors.Wrap(err, "cannot parse cert/key pair") - } - - tlsConfig, err := tlsconf.ClientAuthClient(in.ServerCN, ca, cert) - if err != nil { - return nil, errors.Wrap(err, "cannot build tls config") - } - - return &TLSConnecter{in.Address, dialer, tlsConfig}, nil -} - -func (c *TLSConnecter) Connect(dialCtx context.Context) (conn net.Conn, err error) { - return tls.DialWithDialer(&c.dialer, "tcp", c.Address, c.tlsConfig) -} diff --git a/cmd/config_fsvfilter.go b/cmd/config_fsvfilter.go deleted file mode 100644 index fc8839d..0000000 --- a/cmd/config_fsvfilter.go +++ /dev/null @@ -1,42 +0,0 @@ -package cmd - -import ( - "github.com/pkg/errors" - "github.com/zrepl/zrepl/zfs" - "strings" -) - -type AnyFSVFilter struct{} - -func (AnyFSVFilter) Filter(t zfs.VersionType, name string) (accept bool, err error) { - return true, nil -} - -type PrefixFilter struct { - prefix string - fstype zfs.VersionType - fstypeSet bool // optionals anyone? -} - -func NewPrefixFilter(prefix string) *PrefixFilter { - return &PrefixFilter{prefix: prefix} -} - -func NewTypedPrefixFilter(prefix string, versionType zfs.VersionType) *PrefixFilter { - return &PrefixFilter{prefix, versionType, true} -} - -func parseSnapshotPrefix(i string) (p string, err error) { - if len(i) <= 0 { - err = errors.Errorf("snapshot prefix must not be empty string") - return - } - p = i - return -} - -func (f *PrefixFilter) Filter(t zfs.VersionType, name string) (accept bool, err error) { - fstypeMatches := (!f.fstypeSet || t == f.fstype) - prefixMatches := strings.HasPrefix(name, f.prefix) - return fstypeMatches && prefixMatches, nil -} diff --git a/cmd/config_job_local.go b/cmd/config_job_local.go deleted file mode 100644 index 8bf4fa3..0000000 --- a/cmd/config_job_local.go +++ /dev/null @@ -1,185 +0,0 @@ -package cmd - -import ( - "time" - - "context" - "github.com/mitchellh/mapstructure" - "github.com/pkg/errors" - "github.com/zrepl/zrepl/config" - "github.com/zrepl/zrepl/endpoint" - "github.com/zrepl/zrepl/replication" - "github.com/zrepl/zrepl/zfs" - "sync" -) - -type LocalJob struct { - Name string - Mapping *DatasetMapFilter - SnapshotPrefix string - Interval time.Duration - PruneLHS PrunePolicy - PruneRHS PrunePolicy - Debug JobDebugSettings -} - -func parseLocalJob(c config.Global, in source.LocalJob) (j *LocalJob, err error) { - - var asMap struct { - Mapping map[string]string - SnapshotPrefix string `mapstructure:"snapshot_prefix"` - Interval string - InitialReplPolicy string `mapstructure:"initial_repl_policy"` - PruneLHS map[string]interface{} `mapstructure:"prune_lhs"` - PruneRHS map[string]interface{} `mapstructure:"prune_rhs"` - Debug map[string]interface{} - } - - if err = mapstructure.Decode(i, &asMap); err != nil { - err = errors.Wrap(err, "mapstructure error") - return nil, err - } - - j = &LocalJob{Name: name} - - if j.Mapping, err = parseDatasetMapFilter(asMap.Mapping, false); err != nil { - return - } - - if j.SnapshotPrefix, err = parseSnapshotPrefix(asMap.SnapshotPrefix); err != nil { - return - } - - if j.Interval, err = parsePostitiveDuration(asMap.Interval); err != nil { - err = errors.Wrap(err, "cannot parse interval") - return - } - - if j.PruneLHS, err = parsePrunePolicy(asMap.PruneLHS, true); err != nil { - err = errors.Wrap(err, "cannot parse 'prune_lhs'") - return - } - if j.PruneRHS, err = parsePrunePolicy(asMap.PruneRHS, false); err != nil { - err = errors.Wrap(err, "cannot parse 'prune_rhs'") - return - } - - if err = mapstructure.Decode(asMap.Debug, &j.Debug); err != nil { - err = errors.Wrap(err, "cannot parse 'debug'") - return - } - - return -} - -func (j *LocalJob) JobName() string { - return j.Name -} - -func (j *LocalJob) JobType() JobType { return JobTypeLocal } - -func (j *LocalJob) JobStart(ctx context.Context) { - - log := getLogger(ctx) - - // Allow access to any dataset since we control what mapping - // is passed to the pull routine. - // All local datasets will be passed to its Map() function, - // but only those for which a mapping exists will actually be pulled. - // We can pay this small performance penalty for now. - wildcardMapFilter := NewDatasetMapFilter(1, false) - wildcardMapFilter.Add("<", "<") - sender := endpoint.NewSender(wildcardMapFilter, NewPrefixFilter(j.SnapshotPrefix)) - - receiver, err := endpoint.NewReceiver(j.Mapping, NewPrefixFilter(j.SnapshotPrefix)) - if err != nil { - log.WithError(err).Error("unexpected error setting up local handler") - } - - snapper := IntervalAutosnap{ - DatasetFilter: j.Mapping.AsFilter(), - Prefix: j.SnapshotPrefix, - SnapshotInterval: j.Interval, - } - - plhs, err := j.Pruner(PrunePolicySideLeft, false) - if err != nil { - log.WithError(err).Error("error creating lhs pruner") - return - } - prhs, err := j.Pruner(PrunePolicySideRight, false) - if err != nil { - log.WithError(err).Error("error creating rhs pruner") - return - } - - didSnaps := make(chan struct{}) - go snapper.Run(WithLogger(ctx, log.WithField(logSubsysField, "snap")), didSnaps) - -outer: - for { - - select { - case <-ctx.Done(): - log.WithError(ctx.Err()).Info("context") - break outer - case <-didSnaps: - log.Debug("finished taking snapshots") - log.Info("starting replication procedure") - } - - { - ctx := WithLogger(ctx, log.WithField(logSubsysField, "replication")) - rep := replication.NewReplication() - rep.Drive(ctx, sender, receiver) - } - - var wg sync.WaitGroup - - wg.Add(1) - go func() { - plhs.Run(WithLogger(ctx, log.WithField(logSubsysField, "prune_lhs"))) - wg.Done() - }() - - wg.Add(1) - go func() { - prhs.Run(WithLogger(ctx, log.WithField(logSubsysField, "prune_rhs"))) - wg.Done() - }() - - wg.Wait() - } - -} - -func (j *LocalJob) Pruner(side PrunePolicySide, dryRun bool) (p Pruner, err error) { - - var dsfilter zfs.DatasetFilter - var pp PrunePolicy - switch side { - case PrunePolicySideLeft: - pp = j.PruneLHS - dsfilter = j.Mapping.AsFilter() - case PrunePolicySideRight: - pp = j.PruneRHS - dsfilter, err = j.Mapping.InvertedFilter() - if err != nil { - err = errors.Wrap(err, "cannot invert mapping for prune_rhs") - return - } - default: - err = errors.Errorf("must be either left or right side") - return - } - - p = Pruner{ - time.Now(), - dryRun, - dsfilter, - j.SnapshotPrefix, - pp, - } - - return -} diff --git a/cmd/config_job_pull.go b/cmd/config_job_pull.go deleted file mode 100644 index f3167c5..0000000 --- a/cmd/config_job_pull.go +++ /dev/null @@ -1,168 +0,0 @@ -package cmd - -import ( - "os" - "os/signal" - "syscall" - "time" - - "context" - - "github.com/mitchellh/mapstructure" - "github.com/pkg/errors" - "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/config" - "github.com/zrepl/zrepl/endpoint" - "github.com/zrepl/zrepl/replication" -) - -type PullJob struct { - Name string - Connect streamrpc.Connecter - Interval time.Duration - Mapping *DatasetMapFilter - // constructed from mapping during parsing - pruneFilter *DatasetMapFilter - Prune PrunePolicy - - rep *replication.Replication -} - -func parsePullJob(c config.Global, in config.PullJob) (j *PullJob, err error) { - - j = &PullJob{Name: in.Name} - - j.Connect, err = parseConnect(in.Replication.Connect) - if err != nil { - err = errors.Wrap(err, "cannot parse 'connect'") - return nil, err - } - - j.Interval = in.Replication.Interval - - j.Mapping = NewDatasetMapFilter(1, false) - if err := j.Mapping.Add("<", in.Replication.RootDataset); err != nil { - return nil, err - } - - j.pruneFilter = NewDatasetMapFilter(1, true) - if err := j.pruneFilter.Add(in.Replication.RootDataset, MapFilterResultOk); err != nil { - return nil, err - } - - if j.Prune, err = parsePrunePolicy(asMap.Prune, false); err != nil { - err = errors.Wrap(err, "cannot parse prune policy") - return - } - - if in.Debug.Conn.ReadDump != "" || j.Debug.Conn.WriteDump != "" { - logConnecter := logNetConnConnecter{ - Connecter: j.Connect, - ReadDump: in.Debug.Conn.ReadDump, - WriteDump: in.Debug.Conn.WriteDump, - } - j.Connect = logConnecter - } - - return -} - -func (j *PullJob) JobName() string { return j.Name } - -func (j *PullJob) JobStart(ctx context.Context) { - - log := getLogger(ctx) - defer log.Info("exiting") - - // j.task is idle here idle here - usr1 := make(chan os.Signal) - signal.Notify(usr1, syscall.SIGUSR1) - defer signal.Stop(usr1) - - ticker := time.NewTicker(j.Interval) - for { - begin := time.Now() - j.doRun(ctx) - duration := time.Now().Sub(begin) - if duration > j.Interval { - log. - WithField("actual_duration", duration). - WithField("configured_interval", j.Interval). - Warn("pull run took longer than configured interval") - } - select { - case <-ctx.Done(): - log.WithError(ctx.Err()).Info("context") - return - case <-ticker.C: - case <-usr1: - } - } -} - -var STREAMRPC_CONFIG = &streamrpc.ConnConfig{ // FIXME oversight and configurability - RxHeaderMaxLen: 4096, - RxStructuredMaxLen: 4096 * 4096, - RxStreamMaxChunkSize: 4096 * 4096, - TxChunkSize: 4096 * 4096, - RxTimeout: streamrpc.Timeout{ - Progress: 10 * time.Second, - }, - TxTimeout: streamrpc.Timeout{ - Progress: 10 * time.Second, - }, -} - -func (j *PullJob) doRun(ctx context.Context) { - - log := getLogger(ctx) - // FIXME - clientConf := &streamrpc.ClientConfig{ - ConnConfig: STREAMRPC_CONFIG, - } - - client, err := streamrpc.NewClient(j.Connect, clientConf) - defer client.Close() - - sender := endpoint.NewRemote(client) - - receiver, err := endpoint.NewReceiver(j.Mapping, AnyFSVFilter{}) - if err != nil { - log.WithError(err).Error("error creating receiver endpoint") - return - } - - { - ctx := replication.WithLogger(ctx, replicationLogAdaptor{log.WithField(logSubsysField, "replication")}) - ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{log.WithField(logSubsysField, "rpc")}) - ctx = endpoint.WithLogger(ctx, log.WithField(logSubsysField, "endpoint")) - j.rep = replication.NewReplication() - j.rep.Drive(ctx, sender, receiver) - } - - client.Close() - - { - ctx := WithLogger(ctx, log.WithField(logSubsysField, "prune")) - pruner, err := j.Pruner(PrunePolicySideDefault, false) - if err != nil { - log.WithError(err).Error("error creating pruner") - } else { - pruner.Run(ctx) - } - } -} - -func (j *PullJob) Report() *replication.Report { - return j.rep.Report() -} - -func (j *PullJob) Pruner(side PrunePolicySide, dryRun bool) (p Pruner, err error) { - p = Pruner{ - time.Now(), - dryRun, - j.pruneFilter, - j.Prune, - } - return -} diff --git a/cmd/config_job_source.go b/cmd/config_job_source.go deleted file mode 100644 index 8e1c283..0000000 --- a/cmd/config_job_source.go +++ /dev/null @@ -1,172 +0,0 @@ -package cmd - -import ( - "context" - "time" - - "github.com/pkg/errors" - "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/config" - "github.com/zrepl/zrepl/endpoint" - "net" -) - -type SourceJob struct { - Name string - Serve ListenerFactory - Filesystems *DatasetMapFilter - SnapshotPrefix string - Interval time.Duration - Prune PrunePolicy -} - -func parseSourceJob(c config.Global, in config.SourceJob) (j *SourceJob, err error) { - j = &SourceJob{ - Name: in.Name, - Interval: in.Snapshotting.Interval, - } - - if j.Serve, err = parseAuthenticatedChannelListenerFactory(c, in.Replication.Serve); err != nil { - return - } - - if j.Filesystems, err = parseDatasetMapFilter(in.Replication.Filesystems, true); err != nil { - return - } - - if j.SnapshotPrefix, err = parseSnapshotPrefix(in.Snapshotting.SnapshotPrefix); err != nil { - return - } - - if j.Prune, err = parsePrunePolicy(in.Pruning, true); err != nil { - err = errors.Wrap(err, "cannot parse 'prune'") - return - } - - if in.Debug.Conn.ReadDump != "" || in.Debug.Conn.WriteDump != "" { - logServe := logListenerFactory{ - ListenerFactory: j.Serve, - ReadDump: in.Debug.Conn.ReadDump, - WriteDump: in.Debug.Conn.WriteDump, - } - j.Serve = logServe - } - - return -} - -func (j *SourceJob) JobName() string { - return j.Name -} - -func (j *SourceJob) JobType() JobType { return JobTypeSource } - -func (j *SourceJob) JobStart(ctx context.Context) { - - log := getLogger(ctx) - defer log.Info("exiting") - - a := IntervalAutosnap{j.Filesystems, j.SnapshotPrefix, j.Interval} - p, err := j.Pruner(PrunePolicySideDefault, false) - - if err != nil { - log.WithError(err).Error("error creating pruner") - return - } - - didSnaps := make(chan struct{}) - - go j.serve(ctx) // logSubsysField set by handleConnection - go a.Run(WithLogger(ctx, log.WithField(logSubsysField, "snap")), didSnaps) - -outer: - for { - select { - case <-ctx.Done(): - break outer - case <-didSnaps: - p.Run(WithLogger(ctx, log.WithField(logSubsysField, "prune"))) - } - } - log.WithError(ctx.Err()).Info("context") - -} - -func (j *SourceJob) Pruner(side PrunePolicySide, dryRun bool) (p Pruner, err error) { - p = Pruner{ - time.Now(), - dryRun, - j.Filesystems, - j.SnapshotPrefix, - j.Prune, - } - return -} - -func (j *SourceJob) serve(ctx context.Context) { - - log := getLogger(ctx) - - listener, err := j.Serve.Listen() - if err != nil { - getLogger(ctx).WithError(err).Error("error listening") - return - } - - type connChanMsg struct { - conn net.Conn - err error - } - connChan := make(chan connChanMsg, 1) - - // Serve connections until interrupted or error -outer: - for { - - go func() { - rwc, err := listener.Accept() - connChan <- connChanMsg{rwc, err} - }() - - select { - - case rwcMsg := <-connChan: - - if rwcMsg.err != nil { - log.WithError(rwcMsg.err).Error("error accepting connection") - continue - } - - j.handleConnection(ctx, rwcMsg.conn) - - case <-ctx.Done(): - log.WithError(ctx.Err()).Info("context") - break outer - } - - } - - log.Info("closing listener") - err = listener.Close() - if err != nil { - log.WithError(err).Error("error closing listener") - } - - return -} - -func (j *SourceJob) handleConnection(ctx context.Context, conn net.Conn) { - log := getLogger(ctx) - log.Info("handling client connection") - - senderEP := endpoint.NewSender(j.Filesystems, NewPrefixFilter(j.SnapshotPrefix)) - - ctx = endpoint.WithLogger(ctx, log.WithField(logSubsysField, "serve")) - ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{log.WithField(logSubsysField, "rpc")}) - handler := endpoint.NewHandler(senderEP) - if err := streamrpc.ServeConn(ctx, conn, STREAMRPC_CONFIG, handler.Handle); err != nil { - log.WithError(err).Error("error serving connection") - } else { - log.Info("client closed connection") - } -} diff --git a/cmd/config_logging.go b/cmd/config_logging.go deleted file mode 100644 index 88e8cb8..0000000 --- a/cmd/config_logging.go +++ /dev/null @@ -1,194 +0,0 @@ -package cmd - -import ( - "crypto/tls" - "crypto/x509" - "github.com/mattn/go-isatty" - "github.com/pkg/errors" - "github.com/zrepl/zrepl/cmd/tlsconf" - "github.com/zrepl/zrepl/config" - "github.com/zrepl/zrepl/logger" - "os" -) - -type LoggingConfig struct { - Outlets *logger.Outlets -} - -type MetadataFlags int64 - -const ( - MetadataTime MetadataFlags = 1 << iota - MetadataLevel - - MetadataNone MetadataFlags = 0 - MetadataAll MetadataFlags = ^0 -) - -func parseLogging(in []config.LoggingOutletEnum) (c *LoggingConfig, err error) { - - c = &LoggingConfig{} - c.Outlets = logger.NewOutlets() - - if len(in) == 0 { - // Default config - out := WriterOutlet{&HumanFormatter{}, os.Stdout} - c.Outlets.Add(out, logger.Warn) - return - } - - var syslogOutlets, stdoutOutlets int - for lei, le := range in { - - outlet, minLevel, err := parseOutlet(le) - if err != nil { - return nil, errors.Wrapf(err, "cannot parse outlet #%d", lei) - } - var _ logger.Outlet = WriterOutlet{} - var _ logger.Outlet = &SyslogOutlet{} - switch outlet.(type) { - case *SyslogOutlet: - syslogOutlets++ - case WriterOutlet: - stdoutOutlets++ - } - - c.Outlets.Add(outlet, minLevel) - - } - - if syslogOutlets > 1 { - return nil, errors.Errorf("can only define one 'syslog' outlet") - } - if stdoutOutlets > 1 { - return nil, errors.Errorf("can only define one 'stdout' outlet") - } - - return c, nil - -} - -func parseLogFormat(i interface{}) (f EntryFormatter, err error) { - var is string - switch j := i.(type) { - case string: - is = j - default: - return nil, errors.Errorf("invalid log format: wrong type: %T", i) - } - - switch is { - case "human": - return &HumanFormatter{}, nil - case "logfmt": - return &LogfmtFormatter{}, nil - case "json": - return &JSONFormatter{}, nil - default: - return nil, errors.Errorf("invalid log format: '%s'", is) - } - -} - -func parseOutlet(in config.LoggingOutletEnum) (o logger.Outlet, level logger.Level, err error) { - - parseCommon := func(common config.LoggingOutletCommon) (logger.Level, EntryFormatter, error) { - if common.Level == "" || common.Format == "" { - return 0, nil, errors.Errorf("must specify 'level' and 'format' field") - } - - minLevel, err := logger.ParseLevel(common.Level) - if err != nil { - return 0, nil, errors.Wrap(err, "cannot parse 'level' field") - } - formatter, err := parseLogFormat(common.Format) - if err != nil { - return 0, nil, errors.Wrap(err, "cannot parse 'formatter' field") - } - return minLevel, formatter, nil - } - - var f EntryFormatter - - switch v := in.Ret.(type) { - case config.StdoutLoggingOutlet: - level, f, err = parseCommon(v.LoggingOutletCommon) - if err != nil { - break - } - o, err = parseStdoutOutlet(v, f) - case config.TCPLoggingOutlet: - level, f, err = parseCommon(v.LoggingOutletCommon) - if err != nil { - break - } - o, err = parseTCPOutlet(v, f) - case config.SyslogLoggingOutlet: - level, f, err = parseCommon(v.LoggingOutletCommon) - if err != nil { - break - } - o, err = parseSyslogOutlet(v, f) - default: - panic(v) - } - return o, level, err -} - -func parseStdoutOutlet(in config.StdoutLoggingOutlet, formatter EntryFormatter) (WriterOutlet, error) { - flags := MetadataAll - writer := os.Stdout - if !isatty.IsTerminal(writer.Fd()) && !in.Time { - flags &= ^MetadataTime - } - - formatter.SetMetadataFlags(flags) - return WriterOutlet{ - formatter, - os.Stdout, - }, nil -} - -func parseTCPOutlet(in config.TCPLoggingOutlet, formatter EntryFormatter) (out *TCPOutlet, err error) { - var tlsConfig *tls.Config - if in.TLS != nil { - tlsConfig, err = func(m *config.TCPLoggingOutletTLS, host string) (*tls.Config, error) { - clientCert, err := tls.LoadX509KeyPair(m.Cert, m.Key) - if err != nil { - return nil, errors.Wrap(err, "cannot load client cert") - } - - var rootCAs *x509.CertPool - if m.CA == "" { - if rootCAs, err = x509.SystemCertPool(); err != nil { - return nil, errors.Wrap(err, "cannot open system cert pool") - } - } else { - rootCAs, err = tlsconf.ParseCAFile(m.CA) - if err != nil { - return nil, errors.Wrap(err, "cannot parse CA cert") - } - } - if rootCAs == nil { - panic("invariant violated") - } - - return tlsconf.ClientAuthClient(host, rootCAs, clientCert) - }(in.TLS, in.Address) - if err != nil { - return nil, errors.New("cannot not parse TLS config in field 'tls'") - } - } - - formatter.SetMetadataFlags(MetadataAll) - return NewTCPOutlet(formatter, in.Net, in.Address, tlsConfig, in.RetryInterval), nil - -} - -func parseSyslogOutlet(in config.SyslogLoggingOutlet, formatter EntryFormatter) (out *SyslogOutlet, err error) { - out = &SyslogOutlet{} - out.Formatter = formatter - out.Formatter.SetMetadataFlags(MetadataNone) - out.RetryInterval = in.RetryInterval - return out, nil -} diff --git a/cmd/config_mapfilter.go b/cmd/config_mapfilter.go deleted file mode 100644 index 4c8f0a7..0000000 --- a/cmd/config_mapfilter.go +++ /dev/null @@ -1,274 +0,0 @@ -package cmd - -import ( - "fmt" - "strings" - - "github.com/pkg/errors" - "github.com/zrepl/zrepl/endpoint" - "github.com/zrepl/zrepl/zfs" -) - -type DatasetMapFilter struct { - entries []datasetMapFilterEntry - - // if set, only valid filter entries can be added using Add() - // and Map() will always return an error - filterMode bool -} - -type datasetMapFilterEntry struct { - path *zfs.DatasetPath - // the mapping. since this datastructure acts as both mapping and filter - // we have to convert it to the desired rep dynamically - mapping string - subtreeMatch bool -} - -func NewDatasetMapFilter(capacity int, filterMode bool) *DatasetMapFilter { - return &DatasetMapFilter{ - entries: make([]datasetMapFilterEntry, 0, capacity), - filterMode: filterMode, - } -} - -func (m *DatasetMapFilter) Add(pathPattern, mapping string) (err error) { - - if m.filterMode { - if _, err = m.parseDatasetFilterResult(mapping); err != nil { - return - } - } - - // assert path glob adheres to spec - const SUBTREE_PATTERN string = "<" - patternCount := strings.Count(pathPattern, SUBTREE_PATTERN) - switch { - case patternCount > 1: - case patternCount == 1 && !strings.HasSuffix(pathPattern, SUBTREE_PATTERN): - err = fmt.Errorf("pattern invalid: only one '<' at end of string allowed") - return - } - - pathStr := strings.TrimSuffix(pathPattern, SUBTREE_PATTERN) - path, err := zfs.NewDatasetPath(pathStr) - if err != nil { - return fmt.Errorf("pattern is not a dataset path: %s", err) - } - - entry := datasetMapFilterEntry{ - path: path, - mapping: mapping, - subtreeMatch: patternCount > 0, - } - m.entries = append(m.entries, entry) - return - -} - -// find the most specific prefix mapping we have -// -// longer prefix wins over shorter prefix, direct wins over glob -func (m DatasetMapFilter) mostSpecificPrefixMapping(path *zfs.DatasetPath) (idx int, found bool) { - lcp, lcp_entry_idx := -1, -1 - direct_idx := -1 - for e := range m.entries { - entry := m.entries[e] - ep := m.entries[e].path - lep := ep.Length() - - switch { - case !entry.subtreeMatch && ep.Equal(path): - direct_idx = e - continue - case entry.subtreeMatch && path.HasPrefix(ep) && lep > lcp: - lcp = lep - lcp_entry_idx = e - default: - continue - } - } - - if lcp_entry_idx >= 0 || direct_idx >= 0 { - found = true - switch { - case direct_idx >= 0: - idx = direct_idx - case lcp_entry_idx >= 0: - idx = lcp_entry_idx - } - } - return -} - -// Returns target == nil if there is no mapping -func (m DatasetMapFilter) Map(source *zfs.DatasetPath) (target *zfs.DatasetPath, err error) { - - if m.filterMode { - err = fmt.Errorf("using a filter for mapping simply does not work") - return - } - - mi, hasMapping := m.mostSpecificPrefixMapping(source) - if !hasMapping { - return nil, nil - } - me := m.entries[mi] - - if me.mapping == "" { - // Special case treatment: 'foo/bar<' => '' - if !me.subtreeMatch { - return nil, fmt.Errorf("mapping to '' must be a subtree match") - } - // ok... - } else { - if strings.HasPrefix("!", me.mapping) { - // reject mapping - return nil, nil - } - } - - target, err = zfs.NewDatasetPath(me.mapping) - if err != nil { - err = fmt.Errorf("mapping target is not a dataset path: %s", err) - return - } - if me.subtreeMatch { - // strip common prefix ('<' wildcards are no special case here) - extendComps := source.Copy() - extendComps.TrimPrefix(me.path) - target.Extend(extendComps) - } - return -} - -func (m DatasetMapFilter) Filter(p *zfs.DatasetPath) (pass bool, err error) { - - if !m.filterMode { - err = fmt.Errorf("using a mapping as a filter does not work") - return - } - - mi, hasMapping := m.mostSpecificPrefixMapping(p) - if !hasMapping { - pass = false - return - } - me := m.entries[mi] - pass, err = m.parseDatasetFilterResult(me.mapping) - return -} - -// Construct a new filter-only DatasetMapFilter from a mapping -// The new filter allows excactly those paths that were not forbidden by the mapping. -func (m DatasetMapFilter) InvertedFilter() (inv *DatasetMapFilter, err error) { - - if m.filterMode { - err = errors.Errorf("can only invert mappings") - return - } - - inv = &DatasetMapFilter{ - make([]datasetMapFilterEntry, len(m.entries)), - true, - } - - for i, e := range m.entries { - inv.entries[i].path, err = zfs.NewDatasetPath(e.mapping) - if err != nil { - err = errors.Wrapf(err, "mapping cannot be inverted: '%s' is not a dataset path: %s", e.mapping) - return - } - inv.entries[i].mapping = MapFilterResultOk - inv.entries[i].subtreeMatch = e.subtreeMatch - } - - return inv, nil -} - -// FIXME investigate whether we can support more... -func (m DatasetMapFilter) Invert() (endpoint.FSMap, error) { - - if m.filterMode { - return nil, errors.Errorf("can only invert mappings") - } - - if len(m.entries) != 1 { - return nil, errors.Errorf("inversion of complicated mappings is not implemented") // FIXME - } - - e := m.entries[0] - - inv := &DatasetMapFilter{ - make([]datasetMapFilterEntry, len(m.entries)), - false, - } - mp, err := zfs.NewDatasetPath(e.mapping) - if err != nil { - return nil, err - } - - inv.entries[0] = datasetMapFilterEntry{ - path: mp, - mapping: e.path.ToString(), - subtreeMatch: e.subtreeMatch, - } - - return inv, nil -} - -// Creates a new DatasetMapFilter in filter mode from a mapping -// All accepting mapping results are mapped to accepting filter results -// All rejecting mapping results are mapped to rejecting filter results -func (m DatasetMapFilter) AsFilter() endpoint.FSFilter { - - f := &DatasetMapFilter{ - make([]datasetMapFilterEntry, len(m.entries)), - true, - } - - for i, e := range m.entries { - var newe datasetMapFilterEntry = e - if strings.HasPrefix(newe.mapping, "!") { - newe.mapping = MapFilterResultOmit - } else { - newe.mapping = MapFilterResultOk - } - f.entries[i] = newe - } - - return f -} - -const ( - MapFilterResultOk string = "ok" - MapFilterResultOmit string = "!" -) - -// Parse a dataset filter result -func (m DatasetMapFilter) parseDatasetFilterResult(result string) (pass bool, err error) { - l := strings.ToLower(result) - if l == MapFilterResultOk { - return true, nil - } - if l == MapFilterResultOmit { - return false, nil - } - return false, fmt.Errorf("'%s' is not a valid filter result", result) -} - -func parseDatasetMapFilterFilesystems(in map[string]bool) (f *DatasetMapFilter, err error) { - - f = NewDatasetMapFilter(len(in), true) - for pathPattern, accept := range in { - mapping := MapFilterResultOmit - if accept { - mapping = MapFilterResultOk - } - if err = f.Add(pathPattern, mapping); err != nil { - err = fmt.Errorf("invalid mapping entry ['%s':'%s']: %s", pathPattern, mapping, err) - return - } - } - return -} diff --git a/cmd/config_parse.go b/cmd/config_parse.go deleted file mode 100644 index 3e72072..0000000 --- a/cmd/config_parse.go +++ /dev/null @@ -1,211 +0,0 @@ -package cmd - -import ( - "io/ioutil" - - "fmt" - "github.com/go-yaml/yaml" - "github.com/mitchellh/mapstructure" - "github.com/pkg/errors" - "github.com/problame/go-streamrpc" - "github.com/zrepl/zrepl/cmd/pruning/retentiongrid" - "github.com/zrepl/zrepl/config" - "os" -) - -var ConfigFileDefaultLocations []string = []string{ - "/etc/zrepl/zrepl.yml", - "/usr/local/etc/zrepl/zrepl.yml", -} - -const ( - JobNameControl string = "control" -) - -var ReservedJobNames []string = []string{ - JobNameControl, -} - -type ConfigParsingContext struct { - Global *Global -} - -func ParseConfig(path string) (config *Config, err error) { - - if path == "" { - // Try default locations - for _, l := range ConfigFileDefaultLocations { - stat, err := os.Stat(l) - if err != nil { - continue - } - if !stat.Mode().IsRegular() { - err = errors.Errorf("file at default location is not a regular file: %s", l) - continue - } - path = l - break - } - } - - var i interface{} - - var bytes []byte - - if bytes, err = ioutil.ReadFile(path); err != nil { - err = errors.WithStack(err) - return - } - - if err = yaml.Unmarshal(bytes, &i); err != nil { - err = errors.WithStack(err) - return - } - - return parseConfig(i) -} - -func parseConfig(i interface{}) (c *Config, err error) { - - var asMap struct { - Global map[string]interface{} - Jobs []map[string]interface{} - } - if err := mapstructure.Decode(i, &asMap); err != nil { - return nil, errors.Wrap(err, "config root must be a dict") - } - - c = &Config{} - - // Parse global with defaults - c.Global.Serve.Stdinserver.SockDir = "/var/run/zrepl/stdinserver" - c.Global.Control.Sockpath = "/var/run/zrepl/control" - - err = mapstructure.Decode(asMap.Global, &c.Global) - if err != nil { - err = errors.Wrap(err, "mapstructure error on 'global' section: %s") - return - } - - if c.Global.logging, err = parseLogging(asMap.Global["logging"]); err != nil { - return nil, errors.Wrap(err, "cannot parse logging section") - } - - cpc := ConfigParsingContext{&c.Global} - jpc := JobParsingContext{cpc} - c.Jobs = make(map[string]Job, len(asMap.Jobs)) - - // FIXME internal jobs should not be mixed with user jobs - // Monitoring Jobs - var monJobs []map[string]interface{} - if err := mapstructure.Decode(asMap.Global["monitoring"], &monJobs); err != nil { - return nil, errors.Wrap(err, "cannot parse monitoring section") - } - for i, jc := range monJobs { - if jc["name"] == "" || jc["name"] == nil { - // FIXME internal jobs should not require a name... - jc["name"] = fmt.Sprintf("prometheus-%d", i) - } - job, err := parseJob(jpc, jc) - if err != nil { - return nil, errors.Wrapf(err, "cannot parse monitoring job #%d", i) - } - if job.JobType() != JobTypePrometheus { - return nil, errors.Errorf("monitoring job #%d has invalid job type", i) - } - c.Jobs[job.JobName()] = job - } - - // Regular Jobs - for i := range asMap.Jobs { - job, err := parseJob(jpc, asMap.Jobs[i]) - if err != nil { - // Try to find its name - namei, ok := asMap.Jobs[i]["name"] - if !ok { - namei = fmt.Sprintf("", i) - } - err = errors.Wrapf(err, "cannot parse job '%v'", namei) - return nil, err - } - jn := job.JobName() - if _, ok := c.Jobs[jn]; ok { - err = errors.Errorf("duplicate or invalid job name: %s", jn) - return nil, err - } - c.Jobs[job.JobName()] = job - } - - return c, nil - -} - -type JobParsingContext struct { - ConfigParsingContext -} - -func parseJob(c config.Global, in config.JobEnum) (j Job, err error) { - - switch v := in.Ret.(type) { - case config.PullJob: - return parsePullJob(c, v) - case config.SourceJob: - return parseSourceJob(c, v) - case config.LocalJob: - return parseLocalJob(c, v) - default: - panic(fmt.Sprintf("implementation error: unknown job type %s", v)) - } - -} - -func parseConnect(in config.ConnectEnum) (c streamrpc.Connecter, err error) { - switch v := in.Ret.(type) { - case config.SSHStdinserverConnect: - return parseSSHStdinserverConnecter(v) - case config.TCPConnect: - return parseTCPConnecter(v) - case config.TLSConnect: - return parseTLSConnecter(v) - default: - panic(fmt.Sprintf("unknown connect type %v", v)) - } -} - -func parsePruning(in []config.PruningEnum, willSeeBookmarks bool) (p Pruner, err error) { - - policies := make([]PrunePolicy, len(in)) - for i := range in { - if policies[i], err = parseKeepRule(in[i]); err != nil { - return nil, errors.Wrapf(err, "invalid keep rule #%d:", i) - } - } - -} - -func parseKeepRule(in config.PruningEnum) (p PrunePolicy, err error) { - switch v := in.Ret.(type) { - case config.PruneGrid: - return retentiongrid.ParseGridPrunePolicy(v, willSeeBookmarks) - //case config.PruneKeepLastN: - //case config.PruneKeepRegex: - //case config.PruneKeepNotReplicated: - default: - panic(fmt.Sprintf("unknown keep rule type %v", v)) - } -} - -func parseAuthenticatedChannelListenerFactory(c config.Global, in config.ServeEnum) (p ListenerFactory, err error) { - - switch v := in.Ret.(type) { - case config.StdinserverServer: - return parseStdinserverListenerFactory(c, v) - case config.TCPServe: - return parseTCPListenerFactory(c, v) - case config.TLSServe: - return parseTLSListenerFactory(c, v) - default: - panic(fmt.Sprintf("unknown listener type %v", v)) - } - -} diff --git a/cmd/config_prune_none.go b/cmd/config_prune_none.go deleted file mode 100644 index 785941b..0000000 --- a/cmd/config_prune_none.go +++ /dev/null @@ -1,11 +0,0 @@ -package cmd - -import "github.com/zrepl/zrepl/zfs" - -type NoPrunePolicy struct{} - -func (p NoPrunePolicy) Prune(fs *zfs.DatasetPath, versions []zfs.FilesystemVersion) (keep, remove []zfs.FilesystemVersion, err error) { - keep = versions - remove = []zfs.FilesystemVersion{} - return -} diff --git a/cmd/config_serve_stdinserver.go b/cmd/config_serve_stdinserver.go deleted file mode 100644 index 1fef93c..0000000 --- a/cmd/config_serve_stdinserver.go +++ /dev/null @@ -1,58 +0,0 @@ -package cmd - -import ( - "github.com/problame/go-netssh" - "github.com/zrepl/zrepl/cmd/helpers" - "github.com/zrepl/zrepl/config" - "net" - "path" -) - -type StdinserverListenerFactory struct { - ClientIdentity string - sockpath string -} - -func parseStdinserverListenerFactory(c config.Global, in config.StdinserverServer) (f *StdinserverListenerFactory, err error) { - - f = &StdinserverListenerFactory{ - ClientIdentity: in.ClientIdentity, - } - - f.sockpath = path.Join(c.Serve.StdinServer.SockDir, f.ClientIdentity) - - return -} - -func (f *StdinserverListenerFactory) Listen() (net.Listener, error) { - - if err := helpers.PreparePrivateSockpath(f.sockpath); err != nil { - return nil, err - } - - l, err := netssh.Listen(f.sockpath) - if err != nil { - return nil, err - } - return StdinserverListener{l}, nil -} - -type StdinserverListener struct { - l *netssh.Listener -} - -func (l StdinserverListener) Addr() net.Addr { - return netsshAddr{} -} - -func (l StdinserverListener) Accept() (net.Conn, error) { - c, err := l.l.Accept() - if err != nil { - return nil, err - } - return netsshConnToNetConnAdatper{c}, nil -} - -func (l StdinserverListener) Close() (err error) { - return l.l.Close() -} diff --git a/cmd/config_serve_tcp.go b/cmd/config_serve_tcp.go deleted file mode 100644 index 7afafd6..0000000 --- a/cmd/config_serve_tcp.go +++ /dev/null @@ -1,25 +0,0 @@ -package cmd - -import ( - "github.com/zrepl/zrepl/config" - "net" - "time" -) - -type TCPListenerFactory struct { - Address string -} - -func parseTCPListenerFactory(c config.Global, in config.TCPServe) (*TCPListenerFactory, error) { - - lf := &TCPListenerFactory{ - Address: in.Listen, - } - return lf, nil -} - -var TCPListenerHandshakeTimeout = 10 * time.Second // FIXME make configurable - -func (f *TCPListenerFactory) Listen() (net.Listener, error) { - return net.Listen("tcp", f.Address) -} diff --git a/cmd/config_serve_tls.go b/cmd/config_serve_tls.go deleted file mode 100644 index 83dcf16..0000000 --- a/cmd/config_serve_tls.go +++ /dev/null @@ -1,78 +0,0 @@ -package cmd - -import ( - "crypto/tls" - "crypto/x509" - "net" - "time" - - "github.com/mitchellh/mapstructure" - "github.com/pkg/errors" - "github.com/zrepl/zrepl/cmd/tlsconf" - "github.com/zrepl/zrepl/config" -) - -type TCPListenerFactory struct { - Address string - tls bool - clientCA *x509.CertPool - serverCert tls.Certificate - clientCommonName string -} - -func parseTCPListenerFactory(c config.Global, in config.TCPServe) (*TCPListenerFactory, error) { - - lf := &TCPListenerFactory{ - Address: in.Listen, - } - - if in.TLS != nil { - err := func(i map[string]interface{}) (err error) { - var in struct { - CA string - Cert string - Key string - ClientCN string `mapstructure:"client_cn"` - } - if err := mapstructure.Decode(i, &in); err != nil { - return errors.Wrap(err, "mapstructure error") - } - - if in.CA == "" || in.Cert == "" || in.Key == "" || in.ClientCN == "" { - return errors.New("fields 'ca', 'cert', 'key' and 'client_cn' must be specified") - } - - lf.clientCommonName = in.ClientCN - - lf.clientCA, err = tlsconf.ParseCAFile(in.CA) - if err != nil { - return errors.Wrap(err, "cannot parse ca file") - } - - lf.serverCert, err = tls.LoadX509KeyPair(in.Cert, in.Key) - if err != nil { - return errors.Wrap(err, "cannot parse cer/key pair") - } - - lf.tls = true // mark success - return nil - }(in.TLS) - if err != nil { - return nil, errors.Wrap(err, "error parsing TLS config in field 'tls'") - } - } - - return lf, nil -} - -var TCPListenerHandshakeTimeout = 10 * time.Second // FIXME make configurable - -func (f *TCPListenerFactory) Listen() (net.Listener, error) { - l, err := net.Listen("tcp", f.Address) - if !f.tls || err != nil { - return l, err - } - - tl := tlsconf.NewClientAuthListener(l, f.clientCA, f.serverCert, f.clientCommonName, TCPListenerHandshakeTimeout) - return tl, nil -} diff --git a/cmd/config_test.go b/cmd/config_test.go deleted file mode 100644 index f53453a..0000000 --- a/cmd/config_test.go +++ /dev/null @@ -1,283 +0,0 @@ -package cmd - -import ( - "testing" - "time" - - "github.com/kr/pretty" - "github.com/stretchr/testify/assert" - "github.com/zrepl/zrepl/util" - "github.com/zrepl/zrepl/zfs" -) - -func TestSampleConfigsAreParsedWithoutErrors(t *testing.T) { - - paths := []string{ - "./sampleconf/localbackup/host1.yml", - "./sampleconf/pullbackup/backuphost.yml", - "./sampleconf/pullbackup/productionhost.yml", - "./sampleconf/random/debugging.yml", - "./sampleconf/random/logging_and_monitoring.yml", - } - - for _, p := range paths { - - c, err := ParseConfig(p) - if err != nil { - t.Errorf("error parsing %s:\n%+v", p, err) - } - - t.Logf("file: %s", p) - t.Log(pretty.Sprint(c)) - - } - -} - -func TestParseRetentionGridStringParsing(t *testing.T) { - - intervals, err := parseRetentionGridIntervalsString("2x10m(keep=2) | 1x1h | 3x1w") - - assert.Nil(t, err) - assert.Len(t, intervals, 6) - proto := util.RetentionInterval{ - KeepCount: 2, - Length: 10 * time.Minute, - } - assert.EqualValues(t, proto, intervals[0]) - assert.EqualValues(t, proto, intervals[1]) - - proto.KeepCount = 1 - proto.Length = 1 * time.Hour - assert.EqualValues(t, proto, intervals[2]) - - proto.Length = 7 * 24 * time.Hour - assert.EqualValues(t, proto, intervals[3]) - assert.EqualValues(t, proto, intervals[4]) - assert.EqualValues(t, proto, intervals[5]) - - intervals, err = parseRetentionGridIntervalsString("|") - assert.Error(t, err) - intervals, err = parseRetentionGridIntervalsString("2x10m") - assert.NoError(t, err) - - intervals, err = parseRetentionGridIntervalsString("1x10m(keep=all)") - assert.NoError(t, err) - assert.Len(t, intervals, 1) - assert.EqualValues(t, util.RetentionGridKeepCountAll, intervals[0].KeepCount) - -} - -func TestDatasetMapFilter(t *testing.T) { - - expectMapping := func(m map[string]string, from, to string) { - dmf, err := parseDatasetMapFilter(m, false) - if err != nil { - t.Logf("expect test map to be valid: %s", err) - t.FailNow() - } - fromPath, err := zfs.NewDatasetPath(from) - if err != nil { - t.Logf("expect test from path to be valid: %s", err) - t.FailNow() - } - - res, err := dmf.Map(fromPath) - if to == "" { - assert.Nil(t, res) - assert.Nil(t, err) - t.Logf("%s => NOT MAPPED", fromPath.ToString()) - return - } - - assert.Nil(t, err) - toPath, err := zfs.NewDatasetPath(to) - if err != nil { - t.Logf("expect test to path to be valid: %s", err) - t.FailNow() - } - assert.True(t, res.Equal(toPath)) - } - - expectFilter := func(m map[string]string, path string, pass bool) { - dmf, err := parseDatasetMapFilter(m, true) - if err != nil { - t.Logf("expect test filter to be valid: %s", err) - t.FailNow() - } - p, err := zfs.NewDatasetPath(path) - if err != nil { - t.Logf("expect test path to be valid: %s", err) - t.FailNow() - } - res, err := dmf.Filter(p) - assert.Nil(t, err) - assert.Equal(t, pass, res) - } - - map1 := map[string]string{ - "a/b/c<": "root1", - "a/b<": "root2", - "<": "root3/b/c", - "b": "!", - "a/b/c/d/e<": "!", - "q<": "root4/1/2", - } - - expectMapping(map1, "a/b/c", "root1") - expectMapping(map1, "a/b/c/d", "root1/d") - expectMapping(map1, "a/b/c/d/e", "") - expectMapping(map1, "a/b/e", "root2/e") - expectMapping(map1, "a/b", "root2") - expectMapping(map1, "x", "root3/b/c/x") - expectMapping(map1, "x/y", "root3/b/c/x/y") - expectMapping(map1, "q", "root4/1/2") - expectMapping(map1, "b", "") - expectMapping(map1, "q/r", "root4/1/2/r") - - map2 := map[string]string{ // identity mapping - "<": "", - } - expectMapping(map2, "foo/bar", "foo/bar") - - map3 := map[string]string{ // subtree to local mapping, need that for Invert() - "foo/bar<": "", - } - { - m, _ := parseDatasetMapFilter(map3, false) - p, _ := zfs.NewDatasetPath("foo/bar") - tp, err := m.Map(p) - assert.Nil(t, err) - assert.True(t, tp.Empty()) - - expectMapping(map3, "foo/bar/x", "x") - expectMapping(map3, "x", "") - } - - filter1 := map[string]string{ - "<": "!", - "a<": "ok", - "a/b<": "!", - } - - expectFilter(filter1, "b", false) - expectFilter(filter1, "a", true) - expectFilter(filter1, "a/d", true) - expectFilter(filter1, "a/b", false) - expectFilter(filter1, "a/b/c", false) - - filter2 := map[string]string{} - expectFilter(filter2, "foo", false) // default to omit - -} - -func TestDatasetMapFilter_AsFilter(t *testing.T) { - - mapspec := map[string]string{ - "a/b/c<": "root1", - "a/b<": "root2", - "<": "root3/b/c", - "b": "!", - "a/b/c/d/e<": "!", - "q<": "root4/1/2", - } - - m, err := parseDatasetMapFilter(mapspec, false) - assert.Nil(t, err) - - f := m.AsFilter() - - t.Logf("Mapping:\n%s\nFilter:\n%s", pretty.Sprint(m), pretty.Sprint(f)) - - tf := func(f zfs.DatasetFilter, path string, pass bool) { - p, err := zfs.NewDatasetPath(path) - assert.Nil(t, err) - r, err := f.Filter(p) - assert.Nil(t, err) - assert.Equal(t, pass, r) - } - - tf(f, "a/b/c", true) - tf(f, "a/b", true) - tf(f, "b", false) - tf(f, "a/b/c/d/e", false) - tf(f, "a/b/c/d/e/f", false) - tf(f, "a", true) - -} - -func TestDatasetMapFilter_InvertedFilter(t *testing.T) { - mapspec := map[string]string{ - "a/b": "1/2", - "a/b/c<": "3", - "a/b/c/d<": "1/2/a", - "a/b/d": "!", - } - - m, err := parseDatasetMapFilter(mapspec, false) - assert.Nil(t, err) - - inv, err := m.InvertedFilter() - assert.Nil(t, err) - - t.Log(pretty.Sprint(inv)) - - expectMapping := func(m *DatasetMapFilter, ps string, expRes bool) { - p, err := zfs.NewDatasetPath(ps) - assert.Nil(t, err) - r, err := m.Filter(p) - assert.Nil(t, err) - assert.Equal(t, expRes, r) - } - - expectMapping(inv, "4", false) - expectMapping(inv, "3", true) - expectMapping(inv, "3/x", true) - expectMapping(inv, "1", false) - expectMapping(inv, "1/2", true) - expectMapping(inv, "1/2/3", false) - expectMapping(inv, "1/2/a/b", true) - -} - -func TestDatasetMapFilter_Invert(t *testing.T) { - - mapspec := map[string]string{ - "<": "foo/bar", - } - - m, err := parseDatasetMapFilter(mapspec, false) - assert.NoError(t, err) - - invI, err := m.Invert() - assert.NoError(t, err) - inv, ok := invI.(*DatasetMapFilter) - assert.True(t, ok) - - expectMapping := func(m *DatasetMapFilter, input, expect string, expErr bool, expEmpty bool) { - p, err := zfs.NewDatasetPath(input) - assert.Nil(t, err) - r, err := m.Map(p) - if expErr { - assert.Nil(t, r) - assert.Error(t, err) - return - } - if expEmpty { - assert.Nil(t, err) - assert.True(t, r.Empty()) - } else if expect == "" { - assert.Nil(t, r) - assert.Nil(t, err) - } else { - assert.Nil(t, err) - assert.NotNil(t, r) - assert.Equal(t, expect, r.ToString()) - } - } - - expectMapping(inv, "x", "", false, false) - expectMapping(inv, "foo/bar", "", false, true) - expectMapping(inv, "foo/bar/bee", "bee", false, false) - -} diff --git a/cmd/control.go b/cmd/control.go deleted file mode 100644 index 77daa44..0000000 --- a/cmd/control.go +++ /dev/null @@ -1,156 +0,0 @@ -package cmd - -import ( - "bytes" - "context" - "encoding/json" - "fmt" - "github.com/pkg/errors" - "github.com/spf13/cobra" - "github.com/zrepl/zrepl/cmd/daemon" - "github.com/zrepl/zrepl/logger" - "github.com/zrepl/zrepl/version" - "io" - golog "log" - "net" - "net/http" - "os" -) - -var controlCmd = &cobra.Command{ - Use: "control", - Short: "control zrepl daemon", -} - -var pprofCmd = &cobra.Command{ - Use: "pprof off | [on TCP_LISTEN_ADDRESS]", - Short: "start a http server exposing go-tool-compatible profiling endpoints at TCP_LISTEN_ADDRESS", - Run: doControlPProf, - PreRunE: func(cmd *cobra.Command, args []string) error { - if cmd.Flags().NArg() < 1 { - goto enargs - } - switch cmd.Flags().Arg(0) { - case "on": - pprofCmdArgs.msg.Run = true - if cmd.Flags().NArg() != 2 { - return errors.New("must specify TCP_LISTEN_ADDRESS as second positional argument") - } - pprofCmdArgs.msg.HttpListenAddress = cmd.Flags().Arg(1) - case "off": - if cmd.Flags().NArg() != 1 { - goto enargs - } - pprofCmdArgs.msg.Run = false - } - return nil - enargs: - return errors.New("invalid number of positional arguments") - - }, -} -var pprofCmdArgs struct { - msg daemon.PprofServerControlMsg -} - -var controlVersionCmd = &cobra.Command{ - Use: "version", - Short: "print version of running zrepl daemon", - Run: doControLVersionCmd, -} - -var controlStatusCmdArgs struct { - format string - level logger.Level - onlyShowJob string -} - -func init() { - RootCmd.AddCommand(controlCmd) - controlCmd.AddCommand(pprofCmd) - controlCmd.AddCommand(controlVersionCmd) - controlStatusCmdArgs.level = logger.Warn -} - -func controlHttpClient() (client http.Client, err error) { - - conf, err := ParseConfig(rootArgs.configFile) - if err != nil { - return http.Client{}, err - } - - return http.Client{ - Transport: &http.Transport{ - DialContext: func(_ context.Context, _, _ string) (net.Conn, error) { - return net.Dial("unix", conf.Global.Control.Sockpath) - }, - }, - }, nil -} - -func doControlPProf(cmd *cobra.Command, args []string) { - - log := golog.New(os.Stderr, "", 0) - - die := func() { - log.Printf("exiting after error") - os.Exit(1) - } - - log.Printf("connecting to zrepl daemon") - httpc, err := controlHttpClient() - if err != nil { - log.Printf("error parsing config: %s", err) - die() - } - - var buf bytes.Buffer - if err := json.NewEncoder(&buf).Encode(&pprofCmdArgs.msg); err != nil { - log.Printf("error marshaling request: %s", err) - die() - } - _, err = httpc.Post("http://unix"+daemon.ControlJobEndpointPProf, "application/json", &buf) - if err != nil { - log.Printf("error: %s", err) - die() - } - - log.Printf("finished") -} - -func doControLVersionCmd(cmd *cobra.Command, args []string) { - - log := golog.New(os.Stderr, "", 0) - - die := func() { - log.Printf("exiting after error") - os.Exit(1) - } - - httpc, err := controlHttpClient() - if err != nil { - log.Printf("could not connect to daemon: %s", err) - die() - } - - resp, err := httpc.Get("http://unix" + daemon.ControlJobEndpointVersion) - if err != nil { - log.Printf("error: %s", err) - die() - } else if resp.StatusCode != http.StatusOK { - var msg bytes.Buffer - io.CopyN(&msg, resp.Body, 4096) - log.Printf("error: %s", msg.String()) - die() - } - - var info version.ZreplVersionInformation - err = json.NewDecoder(resp.Body).Decode(&info) - if err != nil { - log.Printf("error unmarshaling response: %s", err) - die() - } - - fmt.Println(info.String()) - -} diff --git a/cmd/daemon.deact/control.go b/cmd/daemon.deact/control.go deleted file mode 100644 index 0aff1ce..0000000 --- a/cmd/daemon.deact/control.go +++ /dev/null @@ -1,142 +0,0 @@ -package daemon - -import ( - "bytes" - "context" - "encoding/json" - "github.com/pkg/errors" - "github.com/zrepl/zrepl/cmd/daemon/job" - "github.com/zrepl/zrepl/cmd/helpers" - "github.com/zrepl/zrepl/logger" - "github.com/zrepl/zrepl/version" - "io" - "net" - "net/http" -) - -type controlJob struct { - sockaddr *net.UnixAddr - jobs *jobs -} - -func newControlJob(sockpath string, jobs *jobs) (j *controlJob, err error) { - j = &controlJob{jobs: jobs} - - j.sockaddr, err = net.ResolveUnixAddr("unix", sockpath) - if err != nil { - err = errors.Wrap(err, "cannot resolve unix address") - return - } - - return -} - -func (j *controlJob) Name() string { return jobNameControl } - -func (j *controlJob) Status() interface{} { return nil } - -const ( - ControlJobEndpointPProf string = "/debug/pprof" - ControlJobEndpointVersion string = "/version" - ControlJobEndpointStatus string = "/status" -) - -func (j *controlJob) Run(ctx context.Context) { - - log := job.GetLogger(ctx) - defer log.Info("control job finished") - - l, err := helpers.ListenUnixPrivate(j.sockaddr) - if err != nil { - log.WithError(err).Error("error listening") - return - } - - pprofServer := NewPProfServer(ctx) - - mux := http.NewServeMux() - mux.Handle(ControlJobEndpointPProf, requestLogger{log: log, handlerFunc: func(w http.ResponseWriter, r *http.Request) { - var msg PprofServerControlMsg - err := json.NewDecoder(r.Body).Decode(&msg) - if err != nil { - log.WithError(err).Error("bad pprof request from client") - w.WriteHeader(http.StatusBadRequest) - } - pprofServer.Control(msg) - w.WriteHeader(200) - }}) - mux.Handle(ControlJobEndpointVersion, - requestLogger{log: log, handler: jsonResponder{func() (interface{}, error) { - return version.NewZreplVersionInformation(), nil - }}}) - mux.Handle(ControlJobEndpointStatus, - requestLogger{log: log, handler: jsonResponder{func() (interface{}, error) { - s := j.jobs.status() - return s, nil - }}}) - server := http.Server{Handler: mux} - -outer: - for { - - served := make(chan error) - go func() { - served <- server.Serve(l) - close(served) - }() - - select { - case <-ctx.Done(): - log.WithError(ctx.Err()).Info("context done") - server.Shutdown(context.Background()) - break outer - case err = <-served: - if err != nil { - log.WithError(err).Error("error serving") - break outer - } - } - - } - -} - -type jsonResponder struct { - producer func() (interface{}, error) -} - -func (j jsonResponder) ServeHTTP(w http.ResponseWriter, r *http.Request) { - res, err := j.producer() - if err != nil { - w.WriteHeader(http.StatusInternalServerError) - io.WriteString(w, err.Error()) - return - } - var buf bytes.Buffer - err = json.NewEncoder(&buf).Encode(res) - if err != nil { - w.WriteHeader(http.StatusInternalServerError) - io.WriteString(w, err.Error()) - } else { - io.Copy(w, &buf) - } -} - -type requestLogger struct { - log logger.Logger - handler http.Handler - handlerFunc http.HandlerFunc -} - -func (l requestLogger) ServeHTTP(w http.ResponseWriter, r *http.Request) { - log := l.log.WithField("method", r.Method).WithField("url", r.URL) - log.Info("start") - if l.handlerFunc != nil { - l.handlerFunc(w, r) - } else if l.handler != nil { - l.handler.ServeHTTP(w, r) - } else { - log.Error("no handler or handlerFunc configured") - } - log.Info("finish") -} diff --git a/cmd/daemon.deact/daemon.go b/cmd/daemon.deact/daemon.go deleted file mode 100644 index 070d4fe..0000000 --- a/cmd/daemon.deact/daemon.go +++ /dev/null @@ -1,174 +0,0 @@ -package daemon - -import ( - "context" - "github.com/zrepl/zrepl/logger" - "os" - "os/signal" - "syscall" - "time" - "github.com/zrepl/zrepl/version" - "fmt" -) - -.daesdfadsfsafjlsjfda - -import ( - "context" - "fmt" - "github.com/zrepl/zrepl/cmd/daemon/job" - "github.com/zrepl/zrepl/logger" - "github.com/zrepl/zrepl/version" - "os" - "os/signal" - "strings" - "sync" - "syscall" - "time" -) - -func Run(ctx context.Context, controlSockpath string, outlets *logger.Outlets, confJobs []job.Job) { - - ctx, cancel := context.WithCancel(ctx) - defer cancel() - sigChan := make(chan os.Signal, 1) - signal.Notify(sigChan, syscall.SIGINT, syscall.SIGTERM) - go func() { - <-sigChan - cancel() - }() - - log := logger.NewLogger(outlets, 1*time.Second) - log.Info(version.NewZreplVersionInformation().String()) - - // parse config - for _, job := range confJobs { - if IsInternalJobName(job.Name()) { - panic(fmt.Sprintf("internal job name used for config job '%s'", job.Name())) //FIXME - } - } - - ctx = job.WithLogger(ctx, log) - - jobs := newJobs() - - // start control socket - controlJob, err := newControlJob(controlSockpath, jobs) - if err != nil { - panic(err) // FIXME - } - jobs.start(ctx, controlJob, true) - - // start prometheus - //var promJob *prometheusJob // FIXME - //jobs.start(ctx, promJob, true) - - log.Info("starting daemon") - - // start regular jobs - for _, j := range confJobs { - jobs.start(ctx, j, false) - } - - select { - case <-jobs.wait(): - log.Info("all jobs finished") - case <-ctx.Done(): - log.WithError(ctx.Err()).Info("context finished") - } - log.Info("daemon exiting") -} - -type jobs struct { - wg sync.WaitGroup - - // m protects all fields below it - m sync.RWMutex - wakeups map[string]job.WakeupChan // by JobName - jobs map[string]job.Job -} - -func newJobs() *jobs { - return &jobs{ - wakeups: make(map[string]job.WakeupChan), - jobs: make(map[string]job.Job), - } -} - -const ( - logJobField string = "job" - logTaskField string = "task" - logSubsysField string = "subsystem" -) - -func (s *jobs) wait() <-chan struct{} { - ch := make(chan struct{}) - go func() { - s.wg.Wait() - }() - return ch -} - -func (s *jobs) status() map[string]interface{} { - s.m.RLock() - defer s.m.RUnlock() - - type res struct { - name string - status interface{} - } - var wg sync.WaitGroup - c := make(chan res, len(s.jobs)) - for name, j := range s.jobs { - wg.Add(1) - go func(name string, j job.Job) { - defer wg.Done() - c <- res{name: name, status: j.Status()} - }(name, j) - } - wg.Wait() - close(c) - ret := make(map[string]interface{}, len(s.jobs)) - for res := range c { - ret[res.name] = res.status - } - return ret -} - -const ( - jobNamePrometheus = "_prometheus" - jobNameControl = "_control" -) - -func IsInternalJobName(s string) bool { - return strings.HasPrefix(s, "_") -} - -func (s *jobs) start(ctx context.Context, j job.Job, internal bool) { - s.m.Lock() - defer s.m.Unlock() - - jobLog := job.GetLogger(ctx).WithField(logJobField, j.Name()) - jobName := j.Name() - if !internal && IsInternalJobName(jobName) { - panic(fmt.Sprintf("internal job name used for non-internal job %s", jobName)) - } - if internal && !IsInternalJobName(jobName) { - panic(fmt.Sprintf("internal job does not use internal job name %s", jobName)) - } - if _, ok := s.jobs[jobName]; ok { - panic(fmt.Sprintf("duplicate job name %s", jobName)) - } - s.jobs[jobName] = j - ctx = job.WithLogger(ctx, jobLog) - ctx, wakeupChan := job.WithWakeup(ctx) - s.wakeups[jobName] = wakeupChan - - s.wg.Add(1) - go func() { - defer s.wg.Done() - jobLog.Info("starting job") - defer jobLog.Info("job exited") - j.Run(ctx) - }() -} diff --git a/cmd/daemon.deact/job/job.go b/cmd/daemon.deact/job/job.go deleted file mode 100644 index 56e25af..0000000 --- a/cmd/daemon.deact/job/job.go +++ /dev/null @@ -1,47 +0,0 @@ -package job - -import ( - "context" - "github.com/zrepl/zrepl/logger" -) - -type Logger = logger.Logger - -type contextKey int - -const ( - contextKeyLog contextKey = iota - contextKeyWakeup -) - -func GetLogger(ctx context.Context) Logger { - if l, ok := ctx.Value(contextKeyLog).(Logger); ok { - return l - } - return logger.NewNullLogger() -} - -func WithLogger(ctx context.Context, l Logger) context.Context { - return context.WithValue(ctx, contextKeyLog, l) -} - -func WithWakeup(ctx context.Context) (context.Context, WakeupChan) { - wc := make(chan struct{}, 1) - return context.WithValue(ctx, contextKeyWakeup, wc), wc -} - -type Job interface { - Name() string - Run(ctx context.Context) - Status() interface{} -} - -type WakeupChan <-chan struct{} - -func WaitWakeup(ctx context.Context) WakeupChan { - wc, ok := ctx.Value(contextKeyWakeup).(WakeupChan) - if !ok { - wc = make(chan struct{}) - } - return wc -} diff --git a/cmd/daemon.deact/pprof.go b/cmd/daemon.deact/pprof.go deleted file mode 100644 index 2296ebd..0000000 --- a/cmd/daemon.deact/pprof.go +++ /dev/null @@ -1,80 +0,0 @@ -package daemon - -import ( - "net/http" - // FIXME: importing this package has the side-effect of poisoning the http.DefaultServeMux - // FIXME: with the /debug/pprof endpoints - "context" - "net" - "net/http/pprof" -) - -type PProfServer struct { - cc chan PprofServerControlMsg - state PprofServerControlMsg - listener net.Listener -} - -type PprofServerControlMsg struct { - // Whether the server should listen for requests on the given address - Run bool - // Must be set if Run is true, undefined otherwise - HttpListenAddress string -} - -func NewPProfServer(ctx context.Context) *PProfServer { - - s := &PProfServer{ - cc: make(chan PprofServerControlMsg), - } - - go s.controlLoop(ctx) - return s -} - -func (s *PProfServer) controlLoop(ctx context.Context) { -outer: - for { - - var msg PprofServerControlMsg - select { - case <-ctx.Done(): - if s.listener != nil { - s.listener.Close() - } - break outer - case msg = <-s.cc: - // proceed - } - - var err error - if msg.Run && s.listener == nil { - - s.listener, err = net.Listen("tcp", msg.HttpListenAddress) - if err != nil { - s.listener = nil - continue - } - - // FIXME: because net/http/pprof does not provide a mux, - mux := http.NewServeMux() - mux.Handle("/debug/pprof/", http.HandlerFunc(pprof.Index)) - mux.Handle("/debug/pprof/cmdline", http.HandlerFunc(pprof.Cmdline)) - mux.Handle("/debug/pprof/profile", http.HandlerFunc(pprof.Profile)) - mux.Handle("/debug/pprof/symbol", http.HandlerFunc(pprof.Symbol)) - mux.Handle("/debug/pprof/trace", http.HandlerFunc(pprof.Trace)) - go http.Serve(s.listener, mux) - continue - } - - if !msg.Run && s.listener != nil { - s.listener.Close() - s.listener = nil - continue - } - } -} - -func (s *PProfServer) Control(msg PprofServerControlMsg) { - s.cc <- msg -} diff --git a/cmd/daemon.deact/prometheus.go b/cmd/daemon.deact/prometheus.go deleted file mode 100644 index 1cef3d0..0000000 --- a/cmd/daemon.deact/prometheus.go +++ /dev/null @@ -1,82 +0,0 @@ -package daemon - -import ( - "context" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promhttp" - "github.com/zrepl/zrepl/cmd/daemon/job" - "github.com/zrepl/zrepl/zfs" - "net" - "net/http" -) - -type prometheusJob struct { - listen string -} - -func newPrometheusJob(listen string) *prometheusJob { - return &prometheusJob{listen} -} - -var prom struct { - taskLastActiveStart *prometheus.GaugeVec - taskLastActiveDuration *prometheus.GaugeVec - taskLogEntries *prometheus.CounterVec -} - -func init() { - prom.taskLastActiveStart = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "zrepl", - Subsystem: "daemon", - Name: "task_last_active_start", - Help: "point in time at which the job task last left idle state", - }, []string{"zrepl_job", "job_type", "task"}) - prom.taskLastActiveDuration = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "zrepl", - Subsystem: "daemon", - Name: "task_last_active_duration", - Help: "seconds that the last run ob a job task spent between leaving and re-entering idle state", - }, []string{"zrepl_job", "job_type", "task"}) - prom.taskLogEntries = prometheus.NewCounterVec(prometheus.CounterOpts{ - Namespace: "zrepl", - Subsystem: "daemon", - Name: "task_log_entries", - Help: "number of log entries per job task and level", - }, []string{"zrepl_job", "job_type", "task", "level"}) - prometheus.MustRegister(prom.taskLastActiveStart) - prometheus.MustRegister(prom.taskLastActiveDuration) - prometheus.MustRegister(prom.taskLogEntries) -} - -func (j *prometheusJob) Name() string { return jobNamePrometheus } - -func (j *prometheusJob) Status() interface{} { return nil } - -func (j *prometheusJob) Run(ctx context.Context) { - - if err := zfs.PrometheusRegister(prometheus.DefaultRegisterer); err != nil { - panic(err) - } - - log := job.GetLogger(ctx) - - l, err := net.Listen("tcp", j.listen) - if err != nil { - log.WithError(err).Error("cannot listen") - } - go func() { - select { - case <-ctx.Done(): - l.Close() - } - }() - - mux := http.NewServeMux() - mux.Handle("/metrics", promhttp.Handler()) - - err = http.Serve(l, mux) - if err != nil { - log.WithError(err).Error("error while serving") - } - -} diff --git a/cmd/daemon.go b/cmd/daemon.go deleted file mode 100644 index 4d4cb1b..0000000 --- a/cmd/daemon.go +++ /dev/null @@ -1,176 +0,0 @@ -package cmd - -import ( - "context" - "fmt" - "github.com/spf13/cobra" - "github.com/zrepl/zrepl/cmd/daemon" - "github.com/zrepl/zrepl/cmd/daemon/job" - "github.com/zrepl/zrepl/config" - "github.com/zrepl/zrepl/logger" - "os" - "os/signal" - "syscall" - "time" -) - -// daemonCmd represents the daemon command -var daemonCmd = &cobra.Command{ - Use: "daemon", - Short: "start daemon", - Run: doDaemon, -} - -func init() { - RootCmd.AddCommand(daemonCmd) -} - -type Job interface { - JobName() string - JobType() JobType - JobStart(ctxt context.Context) -} - -type JobType string - -const ( - JobTypePull JobType = "pull" - JobTypeSource JobType = "source" - JobTypeLocal JobType = "local" - JobTypePrometheus JobType = "prometheus" - JobTypeControl JobType = "control" -) - -func ParseUserJobType(s string) (JobType, error) { - switch s { - case "pull": - return JobTypePull, nil - case "source": - return JobTypeSource, nil - case "local": - return JobTypeLocal, nil - case "prometheus": - return JobTypePrometheus, nil - } - return "", fmt.Errorf("unknown job type '%s'", s) -} - -func (j JobType) String() string { - return string(j) -} - -type daemonJobAdaptor struct { - j Job -} - -func (a daemonJobAdaptor) Name() string { - return a.j.JobName() -} - -func (a daemonJobAdaptor) Run(ctx context.Context) { - a.j.JobStart(ctx) -} - -func (a daemonJobAdaptor) Status() interface{} { return nil } - -func doDaemon(cmd *cobra.Command, args []string) { - - conf, err := config.ParseConfig(rootArgs.configFile) - if err != nil { - fmt.Fprintf(os.Stderr, "error parsing config: %s\n", err) - os.Exit(1) - } - - outlets, err := parseLogging(conf.Global.Logging) - if err != nil { - fmt.Fprintf(os.Stderr, "failed to generate logger: %s\n", err) - return - } - log := logger.NewLogger(outlets.Outlets, 1*time.Second) - - ctx := WithLogger(context.Background(), log) - - daemonJobs := make([]job.Job, 0, len(conf.Jobs)) - for i := range conf.Jobs { - parseJob() - daemonJobs = append(daemonJobs, daemonJobAdaptor{conf.Jobs[i]}) - } - daemon.Run(ctx, conf.Global.Control.Sockpath, conf.Global.logging.Outlets, daemonJobs) - -} - -type contextKey string - -const ( - contextKeyLog contextKey = contextKey("log") - contextKeyDaemon contextKey = contextKey("daemon") -) - -func getLogger(ctx context.Context) Logger { - return ctx.Value(contextKeyLog).(Logger) -} - -func WithLogger(ctx context.Context, l Logger) context.Context { - return context.WithValue(ctx, contextKeyLog, l) -} - -type Daemon struct { - conf *Config - startedAt time.Time -} - -func NewDaemon(initialConf *Config) *Daemon { - return &Daemon{conf: initialConf} -} - -func (d *Daemon) Loop(ctx context.Context) { - - d.startedAt = time.Now() - - log := getLogger(ctx) - - ctx, cancel := context.WithCancel(ctx) - ctx = context.WithValue(ctx, contextKeyDaemon, d) - - sigChan := make(chan os.Signal, 1) - finishs := make(chan Job) - - signal.Notify(sigChan, syscall.SIGINT, syscall.SIGTERM) - - log.Info("starting jobs from config") - i := 0 - for _, job := range d.conf.Jobs { - logger := log.WithField(logJobField, job.JobName()) - logger.Info("starting") - i++ - jobCtx := WithLogger(ctx, logger) - go func(j Job) { - j.JobStart(jobCtx) - finishs <- j - }(job) - } - - finishCount := 0 -outer: - for { - select { - case <-finishs: - finishCount++ - if finishCount == len(d.conf.Jobs) { - log.Info("all jobs finished") - break outer - } - - case sig := <-sigChan: - log.WithField("signal", sig).Info("received signal") - log.Info("cancelling all jobs") - cancel() - } - } - - signal.Stop(sigChan) - cancel() // make go vet happy - - log.Info("exiting") - -} diff --git a/cmd/logging_formatters.go b/cmd/logging_formatters.go deleted file mode 100644 index 49ef88b..0000000 --- a/cmd/logging_formatters.go +++ /dev/null @@ -1,201 +0,0 @@ -package cmd - -import ( - "bytes" - "encoding/json" - "fmt" - "github.com/go-logfmt/logfmt" - "github.com/pkg/errors" - "github.com/zrepl/zrepl/logger" - "time" -) - -type EntryFormatter interface { - SetMetadataFlags(flags MetadataFlags) - Format(e *logger.Entry) ([]byte, error) -} - -const ( - FieldLevel = "level" - FieldMessage = "msg" - FieldTime = "time" -) - -const ( - logJobField string = "job" - logTaskField string = "task" - logSubsysField string = "subsystem" -) - -type NoFormatter struct{} - -func (f NoFormatter) SetMetadataFlags(flags MetadataFlags) {} - -func (f NoFormatter) Format(e *logger.Entry) ([]byte, error) { - return []byte(e.Message), nil -} - -type HumanFormatter struct { - metadataFlags MetadataFlags - ignoreFields map[string]bool -} - -const HumanFormatterDateFormat = time.RFC3339 - -func (f *HumanFormatter) SetMetadataFlags(flags MetadataFlags) { - f.metadataFlags = flags -} - -func (f *HumanFormatter) SetIgnoreFields(ignore []string) { - if ignore == nil { - f.ignoreFields = nil - return - } - f.ignoreFields = make(map[string]bool, len(ignore)) - - for _, field := range ignore { - f.ignoreFields[field] = true - } -} - -func (f *HumanFormatter) ignored(field string) bool { - return f.ignoreFields != nil && f.ignoreFields[field] -} - -func (f *HumanFormatter) Format(e *logger.Entry) (out []byte, err error) { - - var line bytes.Buffer - - if f.metadataFlags&MetadataTime != 0 { - fmt.Fprintf(&line, "%s ", e.Time.Format(HumanFormatterDateFormat)) - } - if f.metadataFlags&MetadataLevel != 0 { - fmt.Fprintf(&line, "[%s]", e.Level.Short()) - } - - prefixFields := []string{logJobField, logTaskField, logSubsysField} - prefixed := make(map[string]bool, len(prefixFields)+2) - for _, field := range prefixFields { - val, ok := e.Fields[field].(string) - if !ok { - continue - } - if !f.ignored(field) { - fmt.Fprintf(&line, "[%s]", val) - prefixed[field] = true - } - } - - if line.Len() > 0 { - fmt.Fprint(&line, ": ") - } - fmt.Fprint(&line, e.Message) - - if len(e.Fields)-len(prefixed) > 0 { - fmt.Fprint(&line, " ") - enc := logfmt.NewEncoder(&line) - for field, value := range e.Fields { - if prefixed[field] || f.ignored(field) { - continue - } - if err := logfmtTryEncodeKeyval(enc, field, value); err != nil { - return nil, err - } - } - } - - return line.Bytes(), nil -} - -type JSONFormatter struct { - metadataFlags MetadataFlags -} - -func (f *JSONFormatter) SetMetadataFlags(flags MetadataFlags) { - f.metadataFlags = flags -} - -func (f *JSONFormatter) Format(e *logger.Entry) ([]byte, error) { - data := make(logger.Fields, len(e.Fields)+3) - for k, v := range e.Fields { - switch v := v.(type) { - case error: - // Otherwise errors are ignored by `encoding/json` - // https://github.com/sirupsen/logrus/issues/137 - data[k] = v.Error() - default: - _, err := json.Marshal(v) - if err != nil { - return nil, errors.Errorf("field is not JSON encodable: %s", k) - } - data[k] = v - } - } - - data[FieldMessage] = e.Message - data[FieldTime] = e.Time.Format(time.RFC3339) - data[FieldLevel] = e.Level - - return json.Marshal(data) - -} - -type LogfmtFormatter struct { - metadataFlags MetadataFlags -} - -func (f *LogfmtFormatter) SetMetadataFlags(flags MetadataFlags) { - f.metadataFlags = flags -} - -func (f *LogfmtFormatter) Format(e *logger.Entry) ([]byte, error) { - var buf bytes.Buffer - enc := logfmt.NewEncoder(&buf) - - if f.metadataFlags&MetadataTime != 0 { - enc.EncodeKeyval(FieldTime, e.Time) - } - if f.metadataFlags&MetadataLevel != 0 { - enc.EncodeKeyval(FieldLevel, e.Level) - } - - // at least try and put job and task in front - prefixed := make(map[string]bool, 2) - prefix := []string{logJobField, logTaskField, logSubsysField} - for _, pf := range prefix { - v, ok := e.Fields[pf] - if !ok { - break - } - if err := logfmtTryEncodeKeyval(enc, pf, v); err != nil { - return nil, err // unlikely - } - prefixed[pf] = true - } - - enc.EncodeKeyval(FieldMessage, e.Message) - - for k, v := range e.Fields { - if !prefixed[k] { - if err := logfmtTryEncodeKeyval(enc, k, v); err != nil { - return nil, err - } - } - } - - return buf.Bytes(), nil -} - -func logfmtTryEncodeKeyval(enc *logfmt.Encoder, field, value interface{}) error { - - err := enc.EncodeKeyval(field, value) - switch err { - case nil: // ok - return nil - case logfmt.ErrUnsupportedValueType: - enc.EncodeKeyval(field, fmt.Sprintf("<%T>", value)) - return nil - } - return errors.Wrapf(err, "cannot encode field '%s'", field) - -} diff --git a/cmd/logging_outlets.go b/cmd/logging_outlets.go deleted file mode 100644 index d201ddc..0000000 --- a/cmd/logging_outlets.go +++ /dev/null @@ -1,161 +0,0 @@ -package cmd - -import ( - "bytes" - "context" - "crypto/tls" - "github.com/pkg/errors" - "github.com/zrepl/zrepl/logger" - "io" - "log/syslog" - "net" - "time" -) - -type WriterOutlet struct { - Formatter EntryFormatter - Writer io.Writer -} - -func (h WriterOutlet) WriteEntry(entry logger.Entry) error { - bytes, err := h.Formatter.Format(&entry) - if err != nil { - return err - } - _, err = h.Writer.Write(bytes) - h.Writer.Write([]byte("\n")) - return err -} - -type TCPOutlet struct { - formatter EntryFormatter - // Specifies how much time must pass between a connection error and a reconnection attempt - // Log entries written to the outlet during this time interval are silently dropped. - connect func(ctx context.Context) (net.Conn, error) - entryChan chan *bytes.Buffer -} - -func NewTCPOutlet(formatter EntryFormatter, network, address string, tlsConfig *tls.Config, retryInterval time.Duration) *TCPOutlet { - - connect := func(ctx context.Context) (conn net.Conn, err error) { - deadl, ok := ctx.Deadline() - if !ok { - deadl = time.Time{} - } - dialer := net.Dialer{ - Deadline: deadl, - } - if tlsConfig != nil { - conn, err = tls.DialWithDialer(&dialer, network, address, tlsConfig) - } else { - conn, err = dialer.DialContext(ctx, network, address) - } - return - } - - entryChan := make(chan *bytes.Buffer, 1) // allow one message in flight while previos is in io.Copy() - - o := &TCPOutlet{ - formatter: formatter, - connect: connect, - entryChan: entryChan, - } - - go o.outLoop(retryInterval) - - return o -} - -// FIXME: use this method -func (h *TCPOutlet) Close() { - close(h.entryChan) -} - -func (h *TCPOutlet) outLoop(retryInterval time.Duration) { - - var retry time.Time - var conn net.Conn - for msg := range h.entryChan { - var err error - for conn == nil { - time.Sleep(time.Until(retry)) - ctx, cancel := context.WithDeadline(context.TODO(), time.Now().Add(retryInterval)) - conn, err = h.connect(ctx) - cancel() - if err != nil { - retry = time.Now().Add(retryInterval) - conn = nil - } - } - conn.SetWriteDeadline(time.Now().Add(retryInterval)) - _, err = io.Copy(conn, msg) - if err != nil { - retry = time.Now().Add(retryInterval) - conn.Close() - conn = nil - } - } -} - -func (h *TCPOutlet) WriteEntry(e logger.Entry) error { - - ebytes, err := h.formatter.Format(&e) - if err != nil { - return err - } - - buf := new(bytes.Buffer) - buf.Write(ebytes) - buf.WriteString("\n") - - select { - case h.entryChan <- buf: - return nil - default: - return errors.New("connection broken or not fast enough") - } -} - -type SyslogOutlet struct { - Formatter EntryFormatter - RetryInterval time.Duration - writer *syslog.Writer - lastConnectAttempt time.Time -} - -func (o *SyslogOutlet) WriteEntry(entry logger.Entry) error { - - bytes, err := o.Formatter.Format(&entry) - if err != nil { - return err - } - - s := string(bytes) - - if o.writer == nil { - now := time.Now() - if now.Sub(o.lastConnectAttempt) < o.RetryInterval { - return nil // not an error toward logger - } - o.writer, err = syslog.New(syslog.LOG_LOCAL0, "zrepl") - o.lastConnectAttempt = time.Now() - if err != nil { - o.writer = nil - return err - } - } - - switch entry.Level { - case logger.Debug: - return o.writer.Debug(s) - case logger.Info: - return o.writer.Info(s) - case logger.Warn: - return o.writer.Warning(s) - case logger.Error: - return o.writer.Err(s) - default: - return o.writer.Err(s) // write as error as reaching this case is in fact an error - } - -} diff --git a/cmd/main.go b/cmd/main.go deleted file mode 100644 index d459cd5..0000000 --- a/cmd/main.go +++ /dev/null @@ -1,43 +0,0 @@ -// zrepl replicates ZFS filesystems & volumes between pools -// -// Code Organization -// -// The cmd package uses github.com/spf13/cobra for its CLI. -// -// It combines the other packages in the zrepl project to implement zrepl functionality. -// -// Each subcommand's code is in the corresponding *.go file. -// All other *.go files contain code shared by the subcommands. -package cmd - -import ( - "github.com/spf13/cobra" - "github.com/zrepl/zrepl/logger" -) - -// -//type Logger interface { -// Printf(format string, v ...interface{}) -//} - -type Logger = logger.Logger - -var RootCmd = &cobra.Command{ - Use: "zrepl", - Short: "ZFS dataset replication", - Long: `Replicate ZFS filesystems & volumes between pools: - - - push & pull mode - - automatic snapshot creation & pruning - - local / over the network - - ACLs instead of blank SSH access`, -} - -var rootArgs struct { - configFile string -} - -func init() { - //cobra.OnInitialize(initConfig) - RootCmd.PersistentFlags().StringVar(&rootArgs.configFile, "config", "", "config file path") -} diff --git a/cmd/prune.go b/cmd/prune.go deleted file mode 100644 index 6963451..0000000 --- a/cmd/prune.go +++ /dev/null @@ -1,123 +0,0 @@ -package cmd - -import ( - "context" - "fmt" - "github.com/zrepl/zrepl/zfs" - "time" -) - -type Pruner struct { - Now time.Time - DryRun bool - DatasetFilter zfs.DatasetFilter - policies []PrunePolicy -} - -type PruneResult struct { - Filesystem *zfs.DatasetPath - All []zfs.FilesystemVersion - Keep []zfs.FilesystemVersion - Remove []zfs.FilesystemVersion -} - -func (p *Pruner) filterFilesystems(ctx context.Context) (filesystems []*zfs.DatasetPath, stop bool) { - filesystems, err := zfs.ZFSListMapping(p.DatasetFilter) - if err != nil { - getLogger(ctx).WithError(err).Error("error applying filesystem filter") - return nil, true - } - if len(filesystems) <= 0 { - getLogger(ctx).Info("no filesystems matching filter") - return nil, true - } - return filesystems, false -} - -func (p *Pruner) filterVersions(ctx context.Context, fs *zfs.DatasetPath) (fsversions []zfs.FilesystemVersion, stop bool) { - log := getLogger(ctx).WithField("fs", fs.ToString()) - - filter := AnyFSVFilter{} - fsversions, err := zfs.ZFSListFilesystemVersions(fs, filter) - if err != nil { - log.WithError(err).Error("error listing filesytem versions") - return nil, true - } - if len(fsversions) == 0 { - log.Info("no filesystem versions matching prefix") - return nil, true - } - return fsversions, false -} - -func (p *Pruner) pruneFilesystem(ctx context.Context, fs *zfs.DatasetPath) (r PruneResult, valid bool) { - log := getLogger(ctx).WithField("fs", fs.ToString()) - - fsversions, stop := p.filterVersions(ctx, fs) - if stop { - return - } - - keep, remove, err := p.PrunePolicy.Prune(fs, fsversions) - if err != nil { - log.WithError(err).Error("error evaluating prune policy") - return - } - - log.WithField("fsversions", fsversions). - WithField("keep", keep). - WithField("remove", remove). - Debug("prune policy debug dump") - - r = PruneResult{fs, fsversions, keep, remove} - - makeFields := func(v zfs.FilesystemVersion) (fields map[string]interface{}) { - fields = make(map[string]interface{}) - fields["version"] = v.ToAbsPath(fs) - timeSince := v.Creation.Sub(p.Now) - fields["age_ns"] = timeSince - const day time.Duration = 24 * time.Hour - days := timeSince / day - remainder := timeSince % day - fields["age_str"] = fmt.Sprintf("%dd%s", days, remainder) - return - } - - for _, v := range remove { - fields := makeFields(v) - log.WithFields(fields).Info("destroying version") - // echo what we'll do and exec zfs destroy if not dry run - // TODO special handling for EBUSY (zfs hold) - // TODO error handling for clones? just echo to cli, skip over, and exit with non-zero status code (we're idempotent) - if !p.DryRun { - err := zfs.ZFSDestroyFilesystemVersion(fs, v) - if err != nil { - log.WithFields(fields).WithError(err).Error("error destroying version") - } - } - } - return r, true -} - -func (p *Pruner) Run(ctx context.Context) (r []PruneResult, err error) { - if p.DryRun { - getLogger(ctx).Info("doing dry run") - } - - filesystems, stop := p.filterFilesystems(ctx) - if stop { - return - } - - r = make([]PruneResult, 0, len(filesystems)) - - for _, fs := range filesystems { - res, ok := p.pruneFilesystem(ctx, fs) - if ok { - r = append(r, res) - } - } - - return - -} diff --git a/cmd/sampleconf/localbackup/host1.yml b/cmd/sampleconf/localbackup/host1.yml deleted file mode 100644 index 5ef4f65..0000000 --- a/cmd/sampleconf/localbackup/host1.yml +++ /dev/null @@ -1,28 +0,0 @@ -jobs: -- name: mirror_local - type: local - - # snapshot the filesystems matched by the left-hand-side of the mapping - # every 10m with zrepl_ as prefix - mapping: { - "zroot/var/db<": "storage/backups/local/zroot/var/db", - "zroot/usr/home<": "storage/backups/local/zroot/usr/home", - "zroot/usr/home/paranoid": "!", #don't backup paranoid user - "zroot/poudriere/ports<": "!", #don't backup the ports trees - } - snapshot_prefix: zrepl_ - interval: 10m - - # keep one hour of 10m interval snapshots of filesystems matched by - # the left-hand-side of the mapping - prune_lhs: - policy: grid - grid: 1x1h(keep=all) - keep_bookmarks: all - - # follow a grandfathering scheme for filesystems on the right-hand-side of the mapping - prune_rhs: - policy: grid - grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d - - diff --git a/cmd/sampleconf/pullbackup/backuphost.yml b/cmd/sampleconf/pullbackup/backuphost.yml deleted file mode 100644 index 206e947..0000000 --- a/cmd/sampleconf/pullbackup/backuphost.yml +++ /dev/null @@ -1,26 +0,0 @@ -jobs: -- name: fullbackup_prod1 - type: pull - # connect to remote using ssh / stdinserver command - connect: - type: ssh+stdinserver - host: prod1.example.com - user: root - port: 22 - identity_file: /root/.ssh/id_ed25519 - - # pull (=ask for new snapshots) every 10m, prune afterwards - # this will leave us at most 10m behind production - interval: 10m - - # pull all offered filesystems to storage/backups/zrepl/pull/prod1.example.com - mapping: { - "<":"storage/backups/zrepl/pull/prod1.example.com" - } - - # follow a grandfathering scheme for filesystems on the right-hand-side of the mapping - snapshot_prefix: zrepl_ - prune: - policy: grid - grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d - diff --git a/cmd/sampleconf/pullbackup/productionhost.yml b/cmd/sampleconf/pullbackup/productionhost.yml deleted file mode 100644 index d2914e5..0000000 --- a/cmd/sampleconf/pullbackup/productionhost.yml +++ /dev/null @@ -1,47 +0,0 @@ -global: - serve: - stdinserver: - # Directory where AF_UNIX sockets for stdinserver command are placed. - # - # `zrepl stdinserver CLIENT_IDENTITY` - # * connects to the socket in $sockdir/CLIENT_IDENTITY - # * sends its stdin / stdout file descriptors to the `zrepl daemon` process (see cmsg(3)) - # * does nothing more - # - # This enables a setup where `zrepl daemon` is not directly exposed to the internet - # but instead all traffic is tunnelled through SSH. - # The server with the source job has an authorized_keys file entry for the public key - # used by the corresponding pull job - # - # command="/mnt/zrepl stdinserver CLIENT_IDENTITY" ssh-ed25519 AAAAC3NzaC1E... zrepl@pullingserver - # - # Below is the default value. - sockdir: /var/run/zrepl/stdinserver - -jobs: - -- name: fullbackup_prod1 - # expect remote to connect via ssh+stdinserver with fullbackup_prod1 as client_identity - type: source - serve: - type: stdinserver # see global.serve.stdinserver for explanation - client_identity: fullbackup_prod1 - - # snapshot these filesystems every 10m with zrepl_ as prefix - filesystems: { - "zroot/var/db<": "ok", - "zroot/usr/home<": "ok", - "zroot/var/tmp": "!", #don't backup /tmp - } - snapshot_prefix: zrepl_ - interval: 10m - - - # keep 1 hour of snapshots (6 at 10m interval) - # and one day of bookmarks in case pull doesn't work (link down, etc) - # => keep_bookmarks = 24h / interval = 24h / 10m = 144 - prune: - policy: grid - grid: 1x1h(keep=all) - keep_bookmarks: 144 - diff --git a/cmd/sampleconf/pushbackup/backuphost.yml b/cmd/sampleconf/pushbackup/backuphost.yml deleted file mode 100644 index 25431a6..0000000 --- a/cmd/sampleconf/pushbackup/backuphost.yml +++ /dev/null @@ -1,20 +0,0 @@ -jobs: -- name: fullbackup_prod1 - - # expect remote to connect via ssh+stdinserver with fullbackup_prod1 as client_identity - type: push-sink - serve: - type: stdinserver - client_identity: fullbackup_prod1 - - # map all pushed datasets to storage/backups/zrepl/sink/prod1.example.com - mapping: { - "<":"storage/backups/zrepl/sink/prod1.example.com" - } - - # follow a grandfathering scheme for filesystems on the right-hand-side of the mapping - prune: - policy: grid - grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d - - diff --git a/cmd/sampleconf/pushbackup/productionhost.yml b/cmd/sampleconf/pushbackup/productionhost.yml deleted file mode 100644 index 5b81ecd..0000000 --- a/cmd/sampleconf/pushbackup/productionhost.yml +++ /dev/null @@ -1,26 +0,0 @@ -jobs: -- name: fullbackup_prod1 - - # connect to remote using ssh / stdinserver command - type: push - connect: - type: ssh+stdinserver - host: prod1.example.com - user: root - port: 22 - identity_file: /root/.ssh/id_ed25519 - - # snapshot these datsets every 10m with zrepl_ as prefix - filesystems: { - "zroot/var/db<": "ok", - "zroot/usr/home<": "!", - } - snapshot_prefix: zrepl_ - interval: 10m - - # keep a one day window 10m interval snapshots in case push doesn't work (link down, etc) - # (we cannot keep more than one day because this host will run out of disk space) - prune: - policy: grid - grid: 1x1d(keep=all) - diff --git a/cmd/sampleconf/random/debugging.yml b/cmd/sampleconf/random/debugging.yml deleted file mode 100644 index 237f3fe..0000000 --- a/cmd/sampleconf/random/debugging.yml +++ /dev/null @@ -1,33 +0,0 @@ -global: - serve: - stdinserver: - sockdir: /var/run/zrepl/stdinserver - -jobs: - -- name: debian2_pull - # JOB DEBUGGING OPTIONS - # should be equal for all job types, but each job implements the debugging itself - # => consult job documentation for supported options - debug: - conn: # debug the io.ReadWriteCloser connection - read_dump: /tmp/connlog_read # dump results of Read() invocations to this file - write_dump: /tmp/connlog_write # dump results of Write() invocations to this file - rpc: # debug the RPC protocol implementation - log: true # log output from rpc layer to the job log - - # ... just to make the unit tests pass. - # check other examples, e.g. localbackup or pullbackup for what the sutff below means - type: source - serve: - type: stdinserver - client_identity: debian2 - filesystems: { - "pool1/db<": ok - } - snapshot_prefix: zrepl_ - interval: 1s - prune: - policy: grid - grid: 1x10s(keep=all) - keep_bookmarks: all diff --git a/cmd/sampleconf/random/logging/client.crt b/cmd/sampleconf/random/logging/client.crt deleted file mode 100644 index 58dbfa5..0000000 --- a/cmd/sampleconf/random/logging/client.crt +++ /dev/null @@ -1,19 +0,0 @@ ------BEGIN CERTIFICATE----- -MIIDIzCCAgsCAQEwDQYJKoZIhvcNAQELBQAwWTELMAkGA1UEBhMCQVUxEzARBgNV -BAgMClNvbWUtU3RhdGUxITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0 -ZDESMBAGA1UEAwwJbG9nc2VydmVyMB4XDTE3MDkyNDEyMzAzNloXDTE3MTAyNDEy -MzAzNlowVjELMAkGA1UEBhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNV -BAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDEPMA0GA1UEAwwGY2xpZW50MIIB -IjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAt/xJTUlqApeJGzRD+w2J8sZS -Bo+s+04T987L/M6gaCo8aDSTEb/ZH3XSoU5JEmO6kPpwNNapOsaEhTCjndZQdm5F -uqiUtAg1uW0HCkBEIDkGr9bFHDKzpewGmmMgfQ2+hfiBR/4ZCrc/vd9P0W9BiWQS -Dtc7p22XraWPVL8HlSz5K/Ih+V6i8O+kBltZkusiJh2bWPoRp/netiTZuc6du+Wp -kpWp1OBaTU4GXIAlLj5afF14BBphRQK983Yhaz53BkA7OQ76XxowynMjmuLQVGmK -f1R9zEJuohTX9XIr1tp/ueRHcS4Awk6LcNZUMCV6270FNSIw2f4hbOZvep+t2wID -AQABMA0GCSqGSIb3DQEBCwUAA4IBAQACK3OeNzScpiNwz/jpg/usQzvXbZ/wDvml -YLjtzn/A65ox8a8BhxvH1ydyoCM2YAGYX7+y7qXJnMgRO/v8565CQIVcznHhg9ST -3828/WqZ3bXf2DV5GxKKQf7hPmBnyVUUhn/Ny91MECED27lZucWiX/bczN8ffDeh -M3+ngezcJxsOBd4x0gLrqIJCoaFRSeepOaFEW6GHQ8loxE9GmA7FQd2phIpJHFSd -Z7nQl7X5C1iN2OboEApJHwtmNVC45UlOpg53vo2sDTLhSfdogstiWi8x1HmvhIGM -j3XHs0Illvo9OwVrmgUph8zQ7pvr/AFrTOIbhgzl/9uVUk5ApwFM ------END CERTIFICATE----- diff --git a/cmd/sampleconf/random/logging/client.csr b/cmd/sampleconf/random/logging/client.csr deleted file mode 100644 index 380bb7f..0000000 --- a/cmd/sampleconf/random/logging/client.csr +++ /dev/null @@ -1,16 +0,0 @@ ------BEGIN CERTIFICATE REQUEST----- -MIICmzCCAYMCAQAwVjELMAkGA1UEBhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUx -ITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDEPMA0GA1UEAwwGY2xp -ZW50MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAt/xJTUlqApeJGzRD -+w2J8sZSBo+s+04T987L/M6gaCo8aDSTEb/ZH3XSoU5JEmO6kPpwNNapOsaEhTCj -ndZQdm5FuqiUtAg1uW0HCkBEIDkGr9bFHDKzpewGmmMgfQ2+hfiBR/4ZCrc/vd9P -0W9BiWQSDtc7p22XraWPVL8HlSz5K/Ih+V6i8O+kBltZkusiJh2bWPoRp/netiTZ -uc6du+WpkpWp1OBaTU4GXIAlLj5afF14BBphRQK983Yhaz53BkA7OQ76XxowynMj -muLQVGmKf1R9zEJuohTX9XIr1tp/ueRHcS4Awk6LcNZUMCV6270FNSIw2f4hbOZv -ep+t2wIDAQABoAAwDQYJKoZIhvcNAQELBQADggEBAKnlr0Qs5KYF85u2YA7DJ5pL -HwAx+qNoNbox5CS1aynrDBpDTWLaErviUJ+4WxRlRyTMEscMOIOKajbYhqqFmtGZ -mu3SshZnFihErw8TOQMyU1LGGG+l6r+6ve5TciwJRLla2Y75z7izr6cyvQNRWdLr -PvxL1/Yqr8LKha12+7o28R4SLf6/GY0GcedqoebRmtuwA/jES0PuGauEUD5lH4cj -Me8sqRrB+IMHQ5j8hlJX4DbA8UQRUBL64sHkQzeQfWu+qkWmS5I19CFfLNrcH+OV -yhyjGfN0q0jHyHdpckBhgzS7IIdo6P66AIlm4qpHM7Scra3JaGM7oaZPamJ6f8U= ------END CERTIFICATE REQUEST----- diff --git a/cmd/sampleconf/random/logging/client.key b/cmd/sampleconf/random/logging/client.key deleted file mode 100644 index a7f1565..0000000 --- a/cmd/sampleconf/random/logging/client.key +++ /dev/null @@ -1,28 +0,0 @@ ------BEGIN PRIVATE KEY----- -MIIEvgIBADANBgkqhkiG9w0BAQEFAASCBKgwggSkAgEAAoIBAQC3/ElNSWoCl4kb -NEP7DYnyxlIGj6z7ThP3zsv8zqBoKjxoNJMRv9kfddKhTkkSY7qQ+nA01qk6xoSF -MKOd1lB2bkW6qJS0CDW5bQcKQEQgOQav1sUcMrOl7AaaYyB9Db6F+IFH/hkKtz+9 -30/Rb0GJZBIO1zunbZetpY9UvweVLPkr8iH5XqLw76QGW1mS6yImHZtY+hGn+d62 -JNm5zp275amSlanU4FpNTgZcgCUuPlp8XXgEGmFFAr3zdiFrPncGQDs5DvpfGjDK -cyOa4tBUaYp/VH3MQm6iFNf1civW2n+55EdxLgDCTotw1lQwJXrbvQU1IjDZ/iFs -5m96n63bAgMBAAECggEAF4om0sWe06ARwbJJNFjCGpa3LfG5/xk5Qs5pmPnS2iD1 -Q5veaTnzjKvlfA/pF3o9B4mTS59fXY7Cq8vSU0J1XwGy2DPzeqlGPmgtq2kXjkvd -iCfhZj8ybvsoyR3/rSBSDRADcnOXPqC9fgyRSMmESBDOoql1D3HdIzF4ii46ySIU -/XQvExS6NWifbP+Ue6DETV8NhreO5PqjeXLITQhhndtc8MDL/8eCNOyN8XjYIWKX -smlBYtRQYOOY9BHOQgUn6yvPHrtKJNKci+qcQNvWir66mBhY1o40MH5wTIV+8yP2 -Vbm/VzoNKIYgeROsilBW7QTwGvkDn3R11zeTqfUNSQKBgQD0eFzhJAEZi4uBw6Tg -NKmBC5Y1IHPOsb5gKPNz9Z9j4qYRDySgYl6ISk+2EdhgUCo1NmTk8EIPQjIerUVf -S+EogFnpsj8U9LR3OM79DaGkNULxrHqhd209/g8DtVgk7yjkxL4vmVOv8qpHMp/7 -eWsylN7AOxj2RB/eXYQBPrw+jQKBgQDAqae9HasLmvpJ9ktTv30yZSKXC+LP4A0D -RBBmx410VpPd4CvcpCJxXmjer6B7+9L1xHYP2pvsnMBid5i0knuvyK28dYy7fldl -CzWvb+lqNA5YYPFXQED4oEdihlQczoI1Bm06SFizeAKD1Q9e2c+lgbR/51j8xuXi -twvhMj/YBwKBgQCZw97/iQrcC2Zq7yiUEOuQjD4lGk1c83U/vGIsTJC9XcCAOFsc -OeMlrD/oz96d7a4unBDn4qpaOJOXsfpRT0PGmrxy/jcpMiUUW/ntNpa11v5NTeQw -DRL8DAFbnsNbL8Yz5f+Nps35fBNYBuKTZLJlNTfKByHTO9QjpAQ0WEZEvQKBgQCi -Ovm83EuYVSKmvxcE6Tyx/8lVqTOO2Vn7wweQlD4/lVujvE0S2L8L+XSS9w5K+GzW -eFz10p3zarbw80YJ30L5bSEmjVE43BUZR4woMzM4M6dUsiTm1HshIE2b4ALZ0uZ/ -Ye794ceXL9nmSrVLqFsaQZLNFPCwwYb4FiyRry9lZwKBgAO9VbWcN8SEeBDKo3z8 -yRbRTc6sI+AdKY44Dfx0tqOPmTjO3mE4X1GU4sbfD2Bvg3DdjwTuxxC/jHaKu0GG -dTM0CbrZGbDAj7E87SOcN/PWEeBckSvuQq5H3DQfwIpTmlS1l5oZn9CxRGbLqC2G -ifnel8XWUG0ROybsr1tk4mzW ------END PRIVATE KEY----- diff --git a/cmd/sampleconf/random/logging/logserver.crt b/cmd/sampleconf/random/logging/logserver.crt deleted file mode 100644 index 87ec4bb..0000000 --- a/cmd/sampleconf/random/logging/logserver.crt +++ /dev/null @@ -1,21 +0,0 @@ ------BEGIN CERTIFICATE----- -MIIDiDCCAnCgAwIBAgIJALhp/WvTQeg/MA0GCSqGSIb3DQEBCwUAMFkxCzAJBgNV -BAYTAkFVMRMwEQYDVQQIDApTb21lLVN0YXRlMSEwHwYDVQQKDBhJbnRlcm5ldCBX -aWRnaXRzIFB0eSBMdGQxEjAQBgNVBAMMCWxvZ3NlcnZlcjAeFw0xNzA5MjQxMjI3 -MDRaFw0yNzA5MjIxMjI3MDRaMFkxCzAJBgNVBAYTAkFVMRMwEQYDVQQIDApTb21l -LVN0YXRlMSEwHwYDVQQKDBhJbnRlcm5ldCBXaWRnaXRzIFB0eSBMdGQxEjAQBgNV -BAMMCWxvZ3NlcnZlcjCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBAKs3 -TLYfXhV3hap71tOkhPQlM+m0EKRAo8Nua50Cci5UhDo4JkVpyYok1h+NFkqmjU2b -IiIuGvsZZPOWYjbWWnSJE4+n5pBFBzcfNQ4d8xVxjANImFn6Tcehhj0WkbDIv/Ge -364XUgywS7u3EGQj/FO7vZ8KHlUxBHNuPIOPHftwIVRyleh5K32UyBaSpSmnqGos -rvI1byMuznavcZpOs4vlebZ+Jy6a20iKf9fj/0f0t0O+F5x3JIk07D3zSywhJ4RM -M0mGIUmYXbh2SMh+f61KDZLDANpz/pMAPbUJe0mxEtBf0tnwK1gEqc3SLwA0EwiM -8Hnn2iaH5Ln20UE3LOkCAwEAAaNTMFEwHQYDVR0OBBYEFDXoDcwx9SngzZcRYCeP -BplBecfiMB8GA1UdIwQYMBaAFDXoDcwx9SngzZcRYCePBplBecfiMA8GA1UdEwEB -/wQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBADyNvs4AA91x3gurQb1pcPVhK6nR -mkYSTN1AsDKSRi/X2iCUmR7G7FlF7XW8mntTpHvVzcs+gr94WckH5wqEOA5iZnaw -PXUWexmdXUge4hmC2q6kBQ5e2ykhSJMRVZXvOLZOZV9qitceamHESV1cKZSNMvZM -aCSVA1RK61/nUzs04pVp5PFPv9gFxJp9ki39FYFdsgZmM5RZ5I/FqxxvTJzu4RnH -VPjsMopzARYwJw6dV2bKdFSYOE8B/Vs3Yv0GxjrABw2ko4PkBPTjLIz22x6+Hd9r -K9BQi4pVmQfvppF5+SORSftlHSS+N47b0DD1rW1f5R6QGi71dFuJGikOwvY= ------END CERTIFICATE----- diff --git a/cmd/sampleconf/random/logging/logserver.key b/cmd/sampleconf/random/logging/logserver.key deleted file mode 100644 index 8eb8faa..0000000 --- a/cmd/sampleconf/random/logging/logserver.key +++ /dev/null @@ -1,28 +0,0 @@ ------BEGIN PRIVATE KEY----- -MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCrN0y2H14Vd4Wq -e9bTpIT0JTPptBCkQKPDbmudAnIuVIQ6OCZFacmKJNYfjRZKpo1NmyIiLhr7GWTz -lmI21lp0iROPp+aQRQc3HzUOHfMVcYwDSJhZ+k3HoYY9FpGwyL/xnt+uF1IMsEu7 -txBkI/xTu72fCh5VMQRzbjyDjx37cCFUcpXoeSt9lMgWkqUpp6hqLK7yNW8jLs52 -r3GaTrOL5Xm2ficumttIin/X4/9H9LdDvhecdySJNOw980ssISeETDNJhiFJmF24 -dkjIfn+tSg2SwwDac/6TAD21CXtJsRLQX9LZ8CtYBKnN0i8ANBMIjPB559omh+S5 -9tFBNyzpAgMBAAECggEBAIY8ZwJq+WKvQLb3POjWFf8so9TY/ispGrwAeJKy9j5o -uPrERw0o8YBDfTVjclS43BQ6Srqtly3DLSjlgL8ps+WmCxYYN2ZpGE0ZRIl65bis -O2/fnML+wbiAZTTD2xnVatfPDeP6GLQmDFpyHoHEzPIBQZvNXRbBxZGSnhMvQ/x7 -FhqSBQG4kf3b1XDCENIbFEVOBOCg7WtMiIgjEGS7QnW3I65/Zt+Ts1LXRZbz+6na -Gmi0PGHA/oLUh1NRzsF4zuZn6fFzja5zw4mkt+JvCWEoxg1QhRAxRp6QQwmZ6MIc -1rw1D4Z+c5UEKyqHeIwZj4M6UNPhCfTXVm47c9eSiGECgYEA4U8pB+7eRo2fqX0C -nWsWMcmsULJvwplQnUSFenUayPn3E8ammS/ZBHksoKhj82vwIdDbtS1hQZn8Bzsi -atc8au0wz0YRDcVDzHX4HknXVQayHtP/FTPeSr5hwpoY8vhEbySuxBTBkXCrp4dx -u5ErfOiYEP3Q1ZvPRywelrATu20CgYEAwonV5dgOcen/4oAirlnvufc2NfqhAQwJ -FJ/JSVMAcXxPYu3sZMv0dGWrX8mLc+P1+XMCuV/7eBM/vU2LbDzmpeUV8sJfB2jw -wyKqKXZwBgeq60btriA4f+0ElwRGgU2KSiniUuuTX2JmyftFQx4cVAQRCFk27NY0 -09psSsYyre0CgYBo6unabdtH029EB5iOIW3GZXk+Yrk0TxyA/4WAjsOYTv5FUT4H -G4bdVGf5sDBLDDpYJOAKsEUXvVLlMx5FzlCuIiGWg7QxS2jU7yJJSG1jhKixPlsM -Toj3GUyAyC1SB1Ymw1g2qsuwpFzquGG3zFQJ6G3Xi7oRnmqZY+wik3+8yQKBgB11 -SdKYOPe++2SNCrNkIw0CBk9+OEs0S1u4Jn7X9sU4kbzlUlqhF89YZe8HUfqmlmTD -qbHwet/f6lL8HxSw1Cxi2EP+cu1oUqz53tKQgL4pAxTFlNA9SND2Ty+fEh4aY8p/ -NSphSduzxuTnC8HyGVAPnZSqDcsnVLCP7r4T7TCxAoGAbJygkkk/gZ9pT4fZoIaq -8CMR8FTfxtkwCuZsWccSMUOWtx9nqet3gbCpKHfyoYZiKB4ke+lnUz4uFS16Y3hG -kN0hFfvfoNa8eB2Ox7vs60cMMfWJac0H7KSaDDy+EvbhE2KtQADT0eWxMyhzGR8p -5CbIivB0QCjeQIA8dOQpE8E= ------END PRIVATE KEY----- diff --git a/cmd/sampleconf/random/logging_and_monitoring.yml b/cmd/sampleconf/random/logging_and_monitoring.yml deleted file mode 100644 index d739f99..0000000 --- a/cmd/sampleconf/random/logging_and_monitoring.yml +++ /dev/null @@ -1,28 +0,0 @@ -global: - logging: - - - outlet: stdout - level: warn - format: human - - - outlet: tcp - level: debug - format: json - net: tcp - address: 127.0.0.1:8080 - retry_interval: 1s - tls: # if not specified, use plain TCP - ca: sampleconf/random/logging/logserver.crt - cert: sampleconf/random/logging/client.crt - key: sampleconf/random/logging/client.key - - - outlet: syslog - level: debug - format: logfmt - - monitoring: - - - type: prometheus - listen: ':9090' - -jobs: [] diff --git a/cmd/stdinserver.go b/cmd/stdinserver.go deleted file mode 100644 index 6d5d41e..0000000 --- a/cmd/stdinserver.go +++ /dev/null @@ -1,55 +0,0 @@ -package cmd - -import ( - "os" - - "context" - "github.com/problame/go-netssh" - "github.com/spf13/cobra" - "log" - "path" -) - -var StdinserverCmd = &cobra.Command{ - Use: "stdinserver CLIENT_IDENTITY", - Short: "start in stdinserver mode (from authorized_keys file)", - Run: cmdStdinServer, -} - -func init() { - RootCmd.AddCommand(StdinserverCmd) -} - -func cmdStdinServer(cmd *cobra.Command, args []string) { - - // NOTE: the netssh proxying protocol requires exiting with non-zero status if anything goes wrong - defer os.Exit(1) - - log := log.New(os.Stderr, "", log.LUTC|log.Ldate|log.Ltime) - - conf, err := ParseConfig(rootArgs.configFile) - if err != nil { - log.Printf("error parsing config: %s", err) - return - } - - if len(args) != 1 || args[0] == "" { - log.Print("must specify client_identity as positional argument") - return - } - - identity := args[0] - unixaddr := path.Join(conf.Global.Serve.Stdinserver.SockDir, identity) - - log.Printf("proxying client identity '%s' to zrepl daemon '%s'", identity, unixaddr) - - ctx := netssh.ContextWithLog(context.TODO(), log) - - err = netssh.Proxy(ctx, unixaddr) - if err == nil { - log.Print("proxying finished successfully, exiting with status 0") - os.Exit(0) - } - log.Printf("error proxying: %s", err) - -} diff --git a/cmd/test.go b/cmd/test.go deleted file mode 100644 index 78f4e71..0000000 --- a/cmd/test.go +++ /dev/null @@ -1,214 +0,0 @@ -package cmd - -import ( - "os" - - "bytes" - "context" - "fmt" - "sort" - "strings" - - "github.com/kr/pretty" - "github.com/spf13/cobra" - "github.com/zrepl/zrepl/logger" - "github.com/zrepl/zrepl/zfs" - "time" -) - -var testCmd = &cobra.Command{ - Use: "test", - Short: "test configuration", - PersistentPreRun: testCmdGlobalInit, -} - -var testCmdGlobal struct { - log Logger - conf *Config -} - -var testConfigSyntaxCmd = &cobra.Command{ - Use: "config", - Short: "parse config file and dump parsed datastructure", - Run: doTestConfig, -} - -var testDatasetMapFilter = &cobra.Command{ - Use: "pattern jobname test/zfs/dataset/path", - Short: "test dataset mapping / filter specified in config", - Example: ` zrepl test pattern my_pull_job tank/tmp`, - Run: doTestDatasetMapFilter, -} - -var testPrunePolicyArgs struct { - side PrunePolicySide - showKept bool - showRemoved bool -} - -var testPrunePolicyCmd = &cobra.Command{ - Use: "prune jobname", - Short: "do a dry-run of the pruning part of a job", - Run: doTestPrunePolicy, -} - -func init() { - RootCmd.AddCommand(testCmd) - testCmd.AddCommand(testConfigSyntaxCmd) - testCmd.AddCommand(testDatasetMapFilter) - - testPrunePolicyCmd.Flags().VarP(&testPrunePolicyArgs.side, "side", "s", "prune_lhs (left) or prune_rhs (right)") - testPrunePolicyCmd.Flags().BoolVar(&testPrunePolicyArgs.showKept, "kept", false, "show kept snapshots") - testPrunePolicyCmd.Flags().BoolVar(&testPrunePolicyArgs.showRemoved, "removed", true, "show removed snapshots") - testCmd.AddCommand(testPrunePolicyCmd) -} - -func testCmdGlobalInit(cmd *cobra.Command, args []string) { - - out := logger.NewOutlets() - out.Add(WriterOutlet{&NoFormatter{}, os.Stdout}, logger.Info) - log := logger.NewLogger(out, 1*time.Second) - testCmdGlobal.log = log - - var err error - if testCmdGlobal.conf, err = ParseConfig(rootArgs.configFile); err != nil { - testCmdGlobal.log.Printf("error parsing config file: %s", err) - os.Exit(1) - } - -} - -func doTestConfig(cmd *cobra.Command, args []string) { - - log, conf := testCmdGlobal.log, testCmdGlobal.conf - - log.Printf("config ok") - log.Printf("%# v", pretty.Formatter(conf)) - return -} - -func doTestDatasetMapFilter(cmd *cobra.Command, args []string) { - - log, conf := testCmdGlobal.log, testCmdGlobal.conf - - if len(args) != 2 { - log.Printf("specify job name as first postitional argument, test input as second") - log.Printf(cmd.UsageString()) - os.Exit(1) - } - n, i := args[0], args[1] - - jobi, err := conf.LookupJob(n) - if err != nil { - log.Printf("%s", err) - os.Exit(1) - } - - var mf *DatasetMapFilter - switch j := jobi.(type) { - case *PullJob: - mf = j.Mapping - case *SourceJob: - mf = j.Filesystems - case *LocalJob: - mf = j.Mapping - default: - panic("incomplete implementation") - } - - ip, err := zfs.NewDatasetPath(i) - if err != nil { - log.Printf("cannot parse test input as ZFS dataset path: %s", err) - os.Exit(1) - } - - if mf.filterMode { - pass, err := mf.Filter(ip) - if err != nil { - log.Printf("error evaluating filter: %s", err) - os.Exit(1) - } - log.Printf("filter result: %v", pass) - } else { - res, err := mf.Map(ip) - if err != nil { - log.Printf("error evaluating mapping: %s", err) - os.Exit(1) - } - toStr := "NO MAPPING" - if res != nil { - toStr = res.ToString() - } - log.Printf("%s => %s", ip.ToString(), toStr) - - } - -} - -func doTestPrunePolicy(cmd *cobra.Command, args []string) { - - log, conf := testCmdGlobal.log, testCmdGlobal.conf - - if cmd.Flags().NArg() != 1 { - log.Printf("specify job name as first positional argument") - log.Printf(cmd.UsageString()) - os.Exit(1) - } - - jobname := cmd.Flags().Arg(0) - jobi, err := conf.LookupJob(jobname) - if err != nil { - log.Printf("%s", err) - os.Exit(1) - } - - jobp, ok := jobi.(PruningJob) - if !ok { - log.Printf("job doesn't do any prunes") - os.Exit(0) - } - - log.Printf("job dump:\n%s", pretty.Sprint(jobp)) - - pruner, err := jobp.Pruner(testPrunePolicyArgs.side, true) - if err != nil { - log.Printf("cannot create test pruner: %s", err) - os.Exit(1) - } - - log.Printf("start pruning") - - ctx := WithLogger(context.Background(), log) - result, err := pruner.Run(ctx) - if err != nil { - log.Printf("error running pruner: %s", err) - os.Exit(1) - } - - sort.Slice(result, func(i, j int) bool { - return strings.Compare(result[i].Filesystem.ToString(), result[j].Filesystem.ToString()) == -1 - }) - - var b bytes.Buffer - for _, r := range result { - fmt.Fprintf(&b, "%s\n", r.Filesystem.ToString()) - - if testPrunePolicyArgs.showKept { - fmt.Fprintf(&b, "\tkept:\n") - for _, v := range r.Keep { - fmt.Fprintf(&b, "\t- %s\n", v.Name) - } - } - - if testPrunePolicyArgs.showRemoved { - fmt.Fprintf(&b, "\tremoved:\n") - for _, v := range r.Remove { - fmt.Fprintf(&b, "\t- %s\n", v.Name) - } - } - - } - - log.Printf("pruning result:\n%s", b.String()) - -} diff --git a/cmd/version.go b/cmd/version.go deleted file mode 100644 index ab838b2..0000000 --- a/cmd/version.go +++ /dev/null @@ -1,21 +0,0 @@ -package cmd - -import ( - "fmt" - "github.com/spf13/cobra" - "github.com/zrepl/zrepl/version" -) - -var versionCmd = &cobra.Command{ - Use: "version", - Short: "print version of zrepl binary (for running daemon 'zrepl control version' command)", - Run: doVersion, -} - -func init() { - RootCmd.AddCommand(versionCmd) -} - -func doVersion(cmd *cobra.Command, args []string) { - fmt.Println(version.NewZreplVersionInformation().String()) -} From 9e941d5be5dba02f114b4a8e783b471cde0cb6e0 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 24 Sep 2018 17:30:03 +0200 Subject: [PATCH 137/167] pruning: implement 'grid' keep rule --- Makefile | 1 + config/retentiongrid.go | 2 +- config/samples/local.yml | 2 +- config/samples/pull.yml | 4 +- config/samples/pull_ssh.yml | 4 +- config/samples/push.yml | 5 +- pruning/keep_grid.go | 102 +++++++++++++ pruning/pruning.go | 2 + pruning/retentiongrid/config_prune_grid.go | 156 -------------------- pruning/retentiongrid/retentiongrid.go | 28 ++-- pruning/retentiongrid/retentiongrid_test.go | 22 +-- 11 files changed, 138 insertions(+), 190 deletions(-) create mode 100644 pruning/keep_grid.go delete mode 100644 pruning/retentiongrid/config_prune_grid.go diff --git a/Makefile b/Makefile index 5b3091e..f66192d 100644 --- a/Makefile +++ b/Makefile @@ -17,6 +17,7 @@ SUBPKGS += daemon/streamrpcconfig SUBPKGS += endpoint SUBPKGS += logger SUBPKGS += pruning +SUBPKGS += pruning/retentiongrid SUBPKGS += replication SUBPKGS += replication/fsrep SUBPKGS += replication/pdu diff --git a/config/retentiongrid.go b/config/retentiongrid.go index e208941..58b2ff3 100644 --- a/config/retentiongrid.go +++ b/config/retentiongrid.go @@ -13,7 +13,7 @@ type RetentionIntervalList []RetentionInterval type PruneGrid struct { Type string `yaml:"type"` Grid RetentionIntervalList `yaml:"grid"` - KeepBookmarks string `yaml:"keep_bookmarks"` + Regex string `yaml:"regex"` } type RetentionInterval struct { diff --git a/config/samples/local.yml b/config/samples/local.yml index 01d73f0..5cc6c2c 100644 --- a/config/samples/local.yml +++ b/config/samples/local.yml @@ -25,4 +25,4 @@ jobs: keep_receiver: - type: grid grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d - keep_bookmarks: all + regex: "zrepl_.*" \ No newline at end of file diff --git a/config/samples/pull.yml b/config/samples/pull.yml index 4fd162b..4eb5113 100644 --- a/config/samples/pull.yml +++ b/config/samples/pull.yml @@ -17,8 +17,8 @@ jobs: count: 10 - type: grid grid: 1x1h(keep=all) | 24x1h | 14x1d - keep_bookmarks: all + regex: "zrepl_.*" keep_receiver: - type: grid grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d - keep_bookmarks: all + regex: "zrepl_.*" \ No newline at end of file diff --git a/config/samples/pull_ssh.yml b/config/samples/pull_ssh.yml index 3e809a6..920f237 100644 --- a/config/samples/pull_ssh.yml +++ b/config/samples/pull_ssh.yml @@ -19,10 +19,10 @@ jobs: count: 10 - type: grid grid: 1x1h(keep=all) | 24x1h | 14x1d - keep_bookmarks: all + regex: "^zrepl_.*" keep_receiver: - type: prefix prefix: keep_ - type: grid grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d - keep_bookmarks: all + regex: "^zrepl_.*" \ No newline at end of file diff --git a/config/samples/push.yml b/config/samples/push.yml index b017dad..cf941f0 100644 --- a/config/samples/push.yml +++ b/config/samples/push.yml @@ -18,9 +18,8 @@ jobs: count: 10 - type: grid grid: 1x1h(keep=all) | 24x1h | 14x1d - keep_bookmarks: all - + regex: "^zrepl_.*" keep_receiver: - type: grid grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d - keep_bookmarks: all + regex: "^zrepl_.*" \ No newline at end of file diff --git a/pruning/keep_grid.go b/pruning/keep_grid.go new file mode 100644 index 0000000..24da21a --- /dev/null +++ b/pruning/keep_grid.go @@ -0,0 +1,102 @@ +package pruning + +import ( + "fmt" + "github.com/pkg/errors" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/pruning/retentiongrid" + "regexp" + "sort" + "time" +) + +// KeepGrid fits snapshots that match a given regex into a retentiongrid.Grid, +// uses the most recent snapshot among those that match the regex as 'now', +// and deletes all snapshots that do not fit the grid specification. +type KeepGrid struct { + retentionGrid *retentiongrid.Grid + re *regexp.Regexp +} + +func NewKeepGrid(in *config.PruneGrid) (p *KeepGrid, err error) { + + if in.Regex == "" { + return nil, fmt.Errorf("Regex must not be empty") + } + re, err := regexp.Compile(in.Regex) + if err != nil { + return nil, errors.Wrap(err, "Regex is invalid") + } + + // Assert intervals are of increasing length (not necessarily required, but indicates config mistake) + lastDuration := time.Duration(0) + for i := range in.Grid { + + if in.Grid[i].Length() < lastDuration { + // If all intervals before were keep=all, this is ok + allPrevKeepCountAll := true + for j := i - 1; allPrevKeepCountAll && j >= 0; j-- { + allPrevKeepCountAll = in.Grid[j].KeepCount() == config.RetentionGridKeepCountAll + } + if allPrevKeepCountAll { + goto isMonotonicIncrease + } + err = errors.New("retention grid interval length must be monotonically increasing") + return + } + isMonotonicIncrease: + lastDuration = in.Grid[i].Length() + + } + + retentionIntervals := make([]retentiongrid.Interval, len(in.Grid)) + for i := range in.Grid { + retentionIntervals[i] = &in.Grid[i] + } + + return &KeepGrid{ + retentiongrid.NewGrid(retentionIntervals), + re, + }, nil +} + +type retentionGridAdaptor struct { + Snapshot +} + +func (a retentionGridAdaptor) LessThan(b retentiongrid.Entry) bool { + return a.Date().Before(b.Date()) +} + +// Prune filters snapshots with the retention grid. +func (p *KeepGrid) KeepRule(snaps []Snapshot) (destroyList []Snapshot) { + + snaps = filterSnapList(snaps, func(snapshot Snapshot) bool { + return p.re.MatchString(snapshot.Name()) + }) + if len(snaps) == 0 { + return nil + } + + // Build adaptors for retention grid + adaptors := make([]retentiongrid.Entry, 0) + for i := range snaps { + adaptors = append(adaptors, retentionGridAdaptor{snaps[i]}) + } + + // determine 'now' edge + sort.SliceStable(adaptors, func(i, j int) bool { + return adaptors[i].LessThan(adaptors[j]) + }) + now := adaptors[len(adaptors)-1].Date() + + // Evaluate retention grid + _, removea := p.retentionGrid.FitEntries(now, adaptors) + + // Revert adaptors + destroyList = make([]Snapshot, len(removea)) + for i := range removea { + destroyList[i] = removea[i].(retentionGridAdaptor).Snapshot + } + return destroyList +} diff --git a/pruning/pruning.go b/pruning/pruning.go index eb575c2..2c4dec9 100644 --- a/pruning/pruning.go +++ b/pruning/pruning.go @@ -61,6 +61,8 @@ func RuleFromConfig(in config.PruningEnum) (KeepRule, error) { return NewKeepLastN(v.Count) case *config.PruneKeepRegex: return NewKeepRegex(v.Regex) + case *config.PruneGrid: + return NewKeepGrid(v) default: return nil, fmt.Errorf("unknown keep rule type %T", v) } diff --git a/pruning/retentiongrid/config_prune_grid.go b/pruning/retentiongrid/config_prune_grid.go deleted file mode 100644 index de1c6ba..0000000 --- a/pruning/retentiongrid/config_prune_grid.go +++ /dev/null @@ -1,156 +0,0 @@ -package retentiongrid - -import ( - "github.com/pkg/errors" - "github.com/zrepl/zrepl/config" - "github.com/zrepl/zrepl/zfs" - "math" - "sort" - "strconv" - "time" -) - -type GridPrunePolicy struct { - retentionGrid *retentionGrid - keepBookmarks int -} - -const GridPrunePolicyMaxBookmarksKeepAll = -1 - -type retentionGridAdaptor struct { - zfs.FilesystemVersion -} - -func (a retentionGridAdaptor) Date() time.Time { - return a.Creation -} - -func (a retentionGridAdaptor) LessThan(b RetentionGridEntry) bool { - return a.CreateTXG < b.(retentionGridAdaptor).CreateTXG -} - -// Prune filters snapshots with the retention grid. -// Bookmarks are deleted such that keepBookmarks are kept in the end. -// The oldest bookmarks are removed first. -func (p *GridPrunePolicy) Prune(_ *zfs.DatasetPath, versions []zfs.FilesystemVersion) (keep, remove []zfs.FilesystemVersion, err error) { - skeep, sremove := p.pruneSnapshots(versions) - keep, remove = p.pruneBookmarks(skeep) - remove = append(remove, sremove...) - return keep, remove, nil -} - -func (p *GridPrunePolicy) pruneSnapshots(versions []zfs.FilesystemVersion) (keep, remove []zfs.FilesystemVersion) { - - // Build adaptors for retention grid - keep = []zfs.FilesystemVersion{} - adaptors := make([]RetentionGridEntry, 0) - for fsv := range versions { - if versions[fsv].Type != zfs.Snapshot { - keep = append(keep, versions[fsv]) - continue - } - adaptors = append(adaptors, retentionGridAdaptor{versions[fsv]}) - } - - sort.SliceStable(adaptors, func(i, j int) bool { - return adaptors[i].LessThan(adaptors[j]) - }) - now := adaptors[len(adaptors)-1].Date() - - // Evaluate retention grid - keepa, removea := p.retentionGrid.FitEntries(now, adaptors) - - // Revert adaptors - for i := range keepa { - keep = append(keep, keepa[i].(retentionGridAdaptor).FilesystemVersion) - } - remove = make([]zfs.FilesystemVersion, len(removea)) - for i := range removea { - remove[i] = removea[i].(retentionGridAdaptor).FilesystemVersion - } - return - -} - -func (p *GridPrunePolicy) pruneBookmarks(versions []zfs.FilesystemVersion) (keep, remove []zfs.FilesystemVersion) { - - if p.keepBookmarks == GridPrunePolicyMaxBookmarksKeepAll { - return versions, []zfs.FilesystemVersion{} - } - - keep = []zfs.FilesystemVersion{} - bookmarks := make([]zfs.FilesystemVersion, 0) - for fsv := range versions { - if versions[fsv].Type != zfs.Bookmark { - keep = append(keep, versions[fsv]) - continue - } - bookmarks = append(bookmarks, versions[fsv]) - } - - if len(bookmarks) == 0 { - return keep, []zfs.FilesystemVersion{} - } - if len(bookmarks) < p.keepBookmarks { - keep = append(keep, bookmarks...) - return keep, []zfs.FilesystemVersion{} - } - - // NOTE: sorting descending by descending by createtxg <=> sorting ascending wrt creation time - sort.SliceStable(bookmarks, func(i, j int) bool { - return (bookmarks[i].CreateTXG > bookmarks[j].CreateTXG) - }) - - keep = append(keep, bookmarks[:p.keepBookmarks]...) - remove = bookmarks[p.keepBookmarks:] - - return keep, remove -} - -func ParseGridPrunePolicy(in config.PruneGrid, willSeeBookmarks bool) (p *GridPrunePolicy, err error) { - - const KeepBookmarksAllString = "all" - - // Assert intervals are of increasing length (not necessarily required, but indicates config mistake) - lastDuration := time.Duration(0) - for i := range in.Grid { - - if in.Grid[i].Length() < lastDuration { - // If all intervals before were keep=all, this is ok - allPrevKeepCountAll := true - for j := i - 1; allPrevKeepCountAll && j >= 0; j-- { - allPrevKeepCountAll = in.Grid[j].KeepCount() == config.RetentionGridKeepCountAll - } - if allPrevKeepCountAll { - goto isMonotonicIncrease - } - err = errors.New("retention grid interval length must be monotonically increasing") - return - } - isMonotonicIncrease: - lastDuration = in.Grid[i].Length() - - } - - // Parse keepBookmarks - keepBookmarks := 0 - if in.KeepBookmarks == KeepBookmarksAllString || (in.KeepBookmarks == "" && !willSeeBookmarks) { - keepBookmarks = GridPrunePolicyMaxBookmarksKeepAll - } else { - i, err := strconv.ParseInt(in.KeepBookmarks, 10, 32) - if err != nil || i <= 0 || i > math.MaxInt32 { - return nil, errors.Errorf("keep_bookmarks must be positive integer or 'all'") - } - keepBookmarks = int(i) - } - - retentionIntervals := make([]RetentionInterval, len(in.Grid)) - for i := range in.Grid { - retentionIntervals[i] = &in.Grid[i] - } - - return &GridPrunePolicy{ - newRetentionGrid(retentionIntervals), - keepBookmarks, - }, nil -} diff --git a/pruning/retentiongrid/retentiongrid.go b/pruning/retentiongrid/retentiongrid.go index 4813933..78f8767 100644 --- a/pruning/retentiongrid/retentiongrid.go +++ b/pruning/retentiongrid/retentiongrid.go @@ -5,50 +5,50 @@ import ( "time" ) -type RetentionInterval interface { +type Interval interface { Length() time.Duration KeepCount() int } const RetentionGridKeepCountAll int = -1 -type retentionGrid struct { - intervals []RetentionInterval +type Grid struct { + intervals []Interval } //A point inside the grid, i.e. a thing the grid can decide to remove -type RetentionGridEntry interface { +type Entry interface { Date() time.Time - LessThan(b RetentionGridEntry) bool + LessThan(b Entry) bool } -func dateInInterval(date, startDateInterval time.Time, i RetentionInterval) bool { +func dateInInterval(date, startDateInterval time.Time, i Interval) bool { return date.After(startDateInterval) && date.Before(startDateInterval.Add(i.Length())) } -func newRetentionGrid(l []RetentionInterval) *retentionGrid { +func NewGrid(l []Interval) *Grid { // TODO Maybe check for ascending interval lengths here, although the algorithm // itself doesn't care about that. - return &retentionGrid{l} + return &Grid{l} } -// Partition a list of RetentionGridEntries into the retentionGrid, +// Partition a list of RetentionGridEntries into the Grid, // relative to a given start date `now`. // -// The `keepCount` oldest entries per `RetentionInterval` are kept (`keep`), +// The `keepCount` oldest entries per `retentiongrid.Interval` are kept (`keep`), // the others are removed (`remove`). // // Entries that are younger than `now` are always kept. // Those that are older than the earliest beginning of an interval are removed. -func (g retentionGrid) FitEntries(now time.Time, entries []RetentionGridEntry) (keep, remove []RetentionGridEntry) { +func (g Grid) FitEntries(now time.Time, entries []Entry) (keep, remove []Entry) { type bucket struct { - entries []RetentionGridEntry + entries []Entry } buckets := make([]bucket, len(g.intervals)) - keep = make([]RetentionGridEntry, 0) - remove = make([]RetentionGridEntry, 0) + keep = make([]Entry, 0) + remove = make([]Entry, 0) oldestIntervalStart := now for i := range g.intervals { diff --git a/pruning/retentiongrid/retentiongrid_test.go b/pruning/retentiongrid/retentiongrid_test.go index 70c8ccd..3e5e52f 100644 --- a/pruning/retentiongrid/retentiongrid_test.go +++ b/pruning/retentiongrid/retentiongrid_test.go @@ -22,10 +22,10 @@ func (i *retentionIntervalStub) KeepCount() int { return i.keepCount } -func retentionGridFromString(gs string) (g *retentionGrid) { +func gridFromString(gs string) (g *Grid) { intervals := strings.Split(gs, "|") - g = &retentionGrid{ - intervals: make([]RetentionInterval, len(intervals)), + g = &Grid{ + intervals: make([]Interval, len(intervals)), } for idx, i := range intervals { comps := strings.SplitN(i, ",", 2) @@ -62,11 +62,11 @@ func (ds dummySnap) Date() time.Time { return ds.date } -func (ds dummySnap) LessThan(b RetentionGridEntry) bool { +func (ds dummySnap) LessThan(b Entry) bool { return ds.date.Before(b.(dummySnap).date) // don't have a txg here } -func validateRetentionGridFitEntries(t *testing.T, now time.Time, input, keep, remove []RetentionGridEntry) { +func validateRetentionGridFitEntries(t *testing.T, now time.Time, input, keep, remove []Entry) { snapDescr := func(d dummySnap) string { return fmt.Sprintf("%s@%s", d.Name, d.date.Sub(now)) @@ -101,8 +101,8 @@ func validateRetentionGridFitEntries(t *testing.T, now time.Time, input, keep, r } func TestRetentionGridFitEntriesEmptyInput(t *testing.T) { - g := retentionGridFromString("10m|10m|10m|1h") - keep, remove := g.FitEntries(time.Now(), []RetentionGridEntry{}) + g := gridFromString("10m|10m|10m|1h") + keep, remove := g.FitEntries(time.Now(), []Entry{}) assert.Empty(t, keep) assert.Empty(t, remove) } @@ -111,13 +111,13 @@ func TestRetentionGridFitEntriesIntervalBoundariesAndAlignment(t *testing.T) { // Intervals are (duration], i.e. 10min is in the first interval, not in the second - g := retentionGridFromString("10m|10m|10m") + g := gridFromString("10m|10m|10m") t.Logf("%#v\n", g) now := time.Unix(0, 0) - snaps := []RetentionGridEntry{ + snaps := []Entry{ dummySnap{"0", true, now.Add(1 * time.Minute)}, // before now dummySnap{"1", true, now}, // before now dummySnap{"2", true, now.Add(-10 * time.Minute)}, // 1st interval @@ -133,13 +133,13 @@ func TestRetentionGridFitEntriesIntervalBoundariesAndAlignment(t *testing.T) { func TestRetentionGridFitEntries(t *testing.T) { - g := retentionGridFromString("10m,-1|10m|10m,2|1h") + g := gridFromString("10m,-1|10m|10m,2|1h") t.Logf("%#v\n", g) now := time.Unix(0, 0) - snaps := []RetentionGridEntry{ + snaps := []Entry{ dummySnap{"1", true, now.Add(3 * time.Minute)}, // pre-now must always be kept dummySnap{"b1", true, now.Add(-6 * time.Minute)}, // 1st interval allows unlimited entries dummySnap{"b3", true, now.Add(-8 * time.Minute)}, // 1st interval allows unlimited entries From 75ba5874a5fade55fde1a552658580233546bc58 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 24 Sep 2018 17:48:45 +0200 Subject: [PATCH 138/167] active side: track activities in Run() as atomically updated member --- daemon/job/active.go | 58 ++++++++++++++++++++++++++++---------------- 1 file changed, 37 insertions(+), 21 deletions(-) diff --git a/daemon/job/active.go b/daemon/job/active.go index a03871e..3595bd0 100644 --- a/daemon/job/active.go +++ b/daemon/job/active.go @@ -30,8 +30,28 @@ type ActiveSide struct { promPruneSecs *prometheus.HistogramVec // labels: prune_side promBytesReplicated *prometheus.CounterVec // labels: filesystem - mtx sync.Mutex + tasksMtx sync.Mutex + tasks activeSideTasks +} + +type activeSideTasks struct { replication *replication.Replication + prunerSender, prunerReceiver *pruner.Pruner +} + +func (a *ActiveSide) updateTasks(u func(*activeSideTasks)) activeSideTasks { + a.tasksMtx.Lock() + var copy activeSideTasks + copy = a.tasks + a.tasksMtx.Unlock() + if u == nil { + return copy + } + u(©) + a.tasksMtx.Lock() + a.tasks = copy + a.tasksMtx.Unlock() + return copy } type activeMode interface { @@ -176,20 +196,14 @@ type ActiveSideStatus struct { } func (j *ActiveSide) Status() *Status { - rep := func() *replication.Replication { - j.mtx.Lock() - defer j.mtx.Unlock() - if j.replication == nil { - return nil - } - return j.replication - }() + tasks := j.updateTasks(nil) + s := &ActiveSideStatus{} t := j.mode.Type() - if rep == nil { + if tasks.replication == nil { return &Status{Type: t, JobSpecific: s} } - s.Replication = rep.Report() + s.Replication = tasks.replication.Report() return &Status{Type: t, JobSpecific: s} } @@ -235,19 +249,21 @@ func (j *ActiveSide) do(ctx context.Context) { sender, receiver, err := j.mode.SenderReceiver(client) - j.mtx.Lock() - j.replication = replication.NewReplication(j.promRepStateSecs, j.promBytesReplicated) - j.mtx.Unlock() + tasks := j.updateTasks(func(tasks *activeSideTasks) { + // reset it + *tasks = activeSideTasks{} + tasks.replication = replication.NewReplication(j.promRepStateSecs, j.promBytesReplicated) + }) log.Info("start replication") - j.replication.Drive(ctx, sender, receiver) + tasks.replication.Drive(ctx, sender, receiver) + tasks = j.updateTasks(func(tasks *activeSideTasks) { + tasks.prunerSender = j.prunerFactory.BuildSenderPruner(ctx, sender, sender) + tasks.prunerReceiver = j.prunerFactory.BuildReceiverPruner(ctx, receiver, sender) + }) log.Info("start pruning sender") - senderPruner := j.prunerFactory.BuildSenderPruner(ctx, sender, sender) - senderPruner.Prune() - + tasks.prunerSender.Prune() log.Info("start pruning receiver") - receiverPruner := j.prunerFactory.BuildReceiverPruner(ctx, receiver, sender) - receiverPruner.Prune() - + tasks.prunerReceiver.Prune() } From 29901935129045c619f00b084a18cd870241d23d Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 24 Sep 2018 19:23:10 +0200 Subject: [PATCH 139/167] replication: export SleepUntil in report --- replication/mainfsm.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/replication/mainfsm.go b/replication/mainfsm.go index b4cec9b..d876491 100644 --- a/replication/mainfsm.go +++ b/replication/mainfsm.go @@ -77,6 +77,7 @@ type Replication struct { type Report struct { Status string Problem string + SleepUntil time.Time Completed []*fsrep.Report Pending []*fsrep.Report Active *fsrep.Report @@ -379,6 +380,7 @@ func (r *Replication) Report() *Report { rep := Report{ Status: r.state.String(), + SleepUntil: r.sleepUntil, } if r.state&(Planning|PlanningError|ContextDone) != 0 { From 75e42fd860bda1170035279f0ba0ffb8bdd2c7de Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 24 Sep 2018 19:22:44 +0200 Subject: [PATCH 140/167] pruner: implement Report method + display in status command --- client/status.go | 228 ++++++++++++++++++++++++++++++---------- daemon/job/active.go | 12 ++- daemon/pruner/pruner.go | 96 +++++++++++++++-- 3 files changed, 273 insertions(+), 63 deletions(-) diff --git a/client/status.go b/client/status.go index 2ac80ca..df1b793 100644 --- a/client/status.go +++ b/client/status.go @@ -8,8 +8,12 @@ import ( "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon" "github.com/zrepl/zrepl/daemon/job" + "github.com/zrepl/zrepl/daemon/pruner" + "github.com/zrepl/zrepl/replication" "github.com/zrepl/zrepl/replication/fsrep" + "math" "sort" + "strings" "sync" "time" "io" @@ -208,66 +212,182 @@ func (t *tui) draw() { t.newline() continue } - rep := pushStatus.Replication - if rep == nil { - t.newline() - continue - } - all := make([]*fsrep.Report, 0, len(rep.Completed)+len(rep.Pending) + 1) - all = append(all, rep.Completed...) - all = append(all, rep.Pending...) - if rep.Active != nil { - all = append(all, rep.Active) - } - sort.Slice(all, func(i, j int) bool { - return all[i].Filesystem < all[j].Filesystem - }) - - t.printf("Status: %s", rep.Status) + t.printf("Replication:") t.newline() - if rep.Problem != "" { - t.printf("Problem: %s", rep.Problem) - t.newline() - } - { // Progress: [---------------] - sumUpFSRep := func(rep *fsrep.Report) (transferred, total int64) { - for _, s := range rep.Pending { - transferred += s.Bytes - total += s.ExpectedBytes - } - for _, s := range rep.Completed { - transferred += s.Bytes - total += s.ExpectedBytes - } - return - } - var transferred, total int64 - for _, fs := range all { - fstx, fstotal := sumUpFSRep(fs) - transferred += fstx - total += fstotal - } - t.write("Progress: ") - t.drawBar(80, transferred, total) - t.write(fmt.Sprintf(" %s / %s", ByteCountBinary(transferred), ByteCountBinary(total))) - t.newline() - } + t.addIndent(1) + t.renderReplicationReport(pushStatus.Replication) + t.addIndent(-1) + + t.printf("Pruning Sender:") + t.newline() + t.addIndent(1) + t.renderPrunerReport(pushStatus.PruningSender) + t.addIndent(-1) + + t.printf("Pruning Receiver:") + t.newline() + t.addIndent(1) + t.renderPrunerReport(pushStatus.PruningReceiver) + t.addIndent(-1) - var maxFSLen int - for _, fs := range all { - if len(fs.Filesystem) > maxFSLen { - maxFSLen = len(fs.Filesystem) - } - } - for _, fs := range all { - printFilesystemStatus(fs, t, fs == rep.Active, maxFSLen) - } } } termbox.Flush() } +func (t *tui) renderReplicationReport(rep *replication.Report) { + if rep == nil { + t.printf("...\n") + return + } + + all := make([]*fsrep.Report, 0, len(rep.Completed)+len(rep.Pending) + 1) + all = append(all, rep.Completed...) + all = append(all, rep.Pending...) + if rep.Active != nil { + all = append(all, rep.Active) + } + sort.Slice(all, func(i, j int) bool { + return all[i].Filesystem < all[j].Filesystem + }) + + t.printf("Status: %s", rep.Status) + t.newline() + if rep.Problem != "" { + t.printf("Problem: %s", rep.Problem) + t.newline() + } + if rep.SleepUntil.After(time.Now()) { + t.printf("Sleeping until %s (%s left)\n", rep.SleepUntil, rep.SleepUntil.Sub(time.Now())) + } + { // Progress: [---------------] + sumUpFSRep := func(rep *fsrep.Report) (transferred, total int64) { + for _, s := range rep.Pending { + transferred += s.Bytes + total += s.ExpectedBytes + } + for _, s := range rep.Completed { + transferred += s.Bytes + total += s.ExpectedBytes + } + return + } + var transferred, total int64 + for _, fs := range all { + fstx, fstotal := sumUpFSRep(fs) + transferred += fstx + total += fstotal + } + t.write("Progress: ") + t.drawBar(80, transferred, total) + t.write(fmt.Sprintf(" %s / %s", ByteCountBinary(transferred), ByteCountBinary(total))) + t.newline() + } + + var maxFSLen int + for _, fs := range all { + if len(fs.Filesystem) > maxFSLen { + maxFSLen = len(fs.Filesystem) + } + } + for _, fs := range all { + printFilesystemStatus(fs, t, fs == rep.Active, maxFSLen) + } +} + +func (t *tui) renderPrunerReport(r *pruner.Report) { + if r == nil { + t.printf("...\n") + return + } + + t.printf("Status: %s", r.State) + t.newline() + + if r.Error != "" { + t.printf("Error: %s\n", r.Error) + } + if r.SleepUntil.After(time.Now()) { + t.printf("Sleeping until %s (%s left)\n", r.SleepUntil, r.SleepUntil.Sub(time.Now())) + } + + type commonFS struct { + *pruner.FSReport + completed bool + } + all := make([]commonFS, 0, len(r.Pending) + len(r.Completed)) + for i := range r.Pending { + all = append(all, commonFS{&r.Pending[i], false}) + } + for i := range r.Completed { + all = append(all, commonFS{&r.Completed[i], true}) + } + + if r.State == pruner.Plan.String() { + return + } + + if len(all) == 0 { + t.printf("nothing to do\n") + return + } + + var totalDestroyCount, completedDestroyCount int + var maxFSname int + for _, fs := range all { + totalDestroyCount += len(fs.DestroyList) + if fs.completed { + completedDestroyCount += len(fs.DestroyList) + } + if maxFSname < len(fs.Filesystem) { + maxFSname = len(fs.Filesystem) + } + } + + // global progress bar + progress := int(math.Round(80 * float64(completedDestroyCount) / float64(totalDestroyCount))) + t.write("Progress: ") + t.write("[") + t.write(times("=", progress)) + t.write(">") + t.write(times("-", 80 - progress)) + t.write("]") + t.printf(" %d/%d snapshots", completedDestroyCount, totalDestroyCount) + t.newline() + + sort.SliceStable(all, func(i, j int) bool { + return strings.Compare(all[i].Filesystem, all[j].Filesystem) == -1 + }) + + // Draw a table-like representation of 'all' + for _, fs := range all { + t.write(rightPad(fs.Filesystem, maxFSname, " ")) + t.write(" ") + if fs.Error != "" { + t.printf("ERROR: %s\n", fs.Error) // whitespace is padding + continue + } + + pruneRuleActionStr := fmt.Sprintf("(destroy %d of %d snapshots)", + len(fs.DestroyList), len(fs.SnapshotList)) + + if fs.completed { + t.printf( "Completed %s\n", pruneRuleActionStr) + continue + } + + t.write("Pending ") // whitespace is padding 10 + if len(fs.DestroyList) == 1 { + t.write(fs.DestroyList[0].Name) + } else { + t.write(pruneRuleActionStr) + } + t.newline() + } + +} + const snapshotIndent = 1 func calculateMaxFSLength(all []*fsrep.Report) (maxFS, maxStatus int) { for _, e := range all { @@ -357,9 +477,9 @@ func filesystemStatusString(rep *fsrep.Report, active bool, fsWidth int) (line s next = " problem: " + rep.Problem } else if len(rep.Pending) > 0 { if rep.Pending[0].From != "" { - next = fmt.Sprintf("next: %s => %s", rep.Pending[0].From, rep.Pending[0].To) + next = fmt.Sprintf(" next: %s => %s", rep.Pending[0].From, rep.Pending[0].To) } else { - next = fmt.Sprintf("next: %s (full)", rep.Pending[0].To) + next = fmt.Sprintf(" next: %s (full)", rep.Pending[0].To) } } status := fmt.Sprintf("%s (step %d/%d, %s/%s)%s", diff --git a/daemon/job/active.go b/daemon/job/active.go index 3595bd0..69cd7e8 100644 --- a/daemon/job/active.go +++ b/daemon/job/active.go @@ -193,6 +193,7 @@ func (j *ActiveSide) Name() string { return j.name } type ActiveSideStatus struct { Replication *replication.Report + PruningSender, PruningReceiver *pruner.Report } func (j *ActiveSide) Status() *Status { @@ -200,10 +201,15 @@ func (j *ActiveSide) Status() *Status { s := &ActiveSideStatus{} t := j.mode.Type() - if tasks.replication == nil { - return &Status{Type: t, JobSpecific: s} + if tasks.replication != nil { + s.Replication = tasks.replication.Report() + } + if tasks.prunerSender != nil { + s.PruningSender = tasks.prunerSender.Report() + } + if tasks.prunerReceiver != nil { + s.PruningReceiver = tasks.prunerReceiver.Report() } - s.Replication = tasks.replication.Report() return &Status{Type: t, JobSpecific: s} } diff --git a/daemon/pruner/pruner.go b/daemon/pruner/pruner.go index 83dd1bb..ebdfe1c 100644 --- a/daemon/pruner/pruner.go +++ b/daemon/pruner/pruner.go @@ -199,13 +199,65 @@ func (p *Pruner) prune(args args) { } } -func (p *Pruner) Report() interface{} { - return nil // FIXME TODO +type Report struct { + State string + SleepUntil time.Time + Error string + Pending, Completed []FSReport +} + +type FSReport struct { + Filesystem string + SnapshotList, DestroyList []SnapshotReport + Error string +} + +type SnapshotReport struct { + Name string + Replicated bool + Date time.Time +} + +func (p *Pruner) Report() *Report { + p.mtx.Lock() + defer p.mtx.Unlock() + + r := Report{State: p.state.String()} + + if p.state & PlanWait|ExecWait != 0 { + r.SleepUntil = p.sleepUntil + } + if p.state & PlanWait|ExecWait|ErrPerm != 0 { + if p.err != nil { + r.Error = p.err.Error() + } + } + + if p.state & Plan|PlanWait == 0 { + return &r + } + + r.Pending = make([]FSReport, len(p.prunePending)) + for i, fs := range p.prunePending{ + r.Pending[i] = fs.Report() + } + r.Completed = make([]FSReport, len(p.pruneCompleted)) + for i, fs := range p.pruneCompleted{ + r.Completed[i] = fs.Report() + } + + return &r } type fs struct { path string + + // snapshots presented by target + // (type snapshot) snaps []pruning.Snapshot + // destroy list returned by pruning.PruneSnapshots(snaps) + // (type snapshot) + destroyList []pruning.Snapshot mtx sync.RWMutex // for Plan @@ -218,12 +270,43 @@ func (f *fs) Update(err error) { f.err = err } +func (f *fs) Report() FSReport { + f.mtx.Lock() + defer f.mtx.Unlock() + + r := FSReport{} + r.Filesystem = f.path + if f.err != nil { + r.Error = f.err.Error() + } + + r.SnapshotList = make([]SnapshotReport, len(f.snaps)) + for i, snap := range f.snaps { + r.SnapshotList[i] = snap.(snapshot).Report() + } + + r.DestroyList = make([]SnapshotReport, len(f.destroyList)) + for i, snap := range f.destroyList{ + r.DestroyList[i] = snap.(snapshot).Report() + } + + return r +} + type snapshot struct { replicated bool date time.Time fsv *pdu.FilesystemVersion } +func (s snapshot) Report() SnapshotReport { + return SnapshotReport{ + Name: s.Name(), + Replicated: s.Replicated(), + Date: s.Date(), + } +} + var _ pruning.Snapshot = snapshot{} func (s snapshot) Name() string { return s.fsv.Name } @@ -344,6 +427,9 @@ fsloop: continue fsloop } + // Apply prune rules + pfs.destroyList = pruning.PruneSnapshots(pfs.snaps, a.rules) + } return u(func(pruner *Pruner) { @@ -378,11 +464,9 @@ func stateExec(a *args, u updater) state { return state.statefunc() } - GetLogger(a.ctx).Debug(fmt.Sprintf("%#v", a.rules)) - destroyListI := pruning.PruneSnapshots(pfs.snaps, a.rules) - destroyList := make([]*pdu.FilesystemVersion, len(destroyListI)) + destroyList := make([]*pdu.FilesystemVersion, len(pfs.destroyList)) for i := range destroyList { - destroyList[i] = destroyListI[i].(snapshot).fsv + destroyList[i] = pfs.destroyList[i].(snapshot).fsv GetLogger(a.ctx). WithField("fs", pfs.path). WithField("destroy_snap", destroyList[i].Name). From 976c1f39292985d3058fe9dd8d63f3330b5c4e0a Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 27 Sep 2018 12:06:59 +0200 Subject: [PATCH 141/167] util.IOCommand: add stderr logging for unexpected crashes in calls to ProcessState.Sys() Crashes observed on a FreeBSD 11.2 system 2018-09-27T05:08:39+02:00 [INFO][csnas]: start replication invocation="62" 2018-09-27T05:08:39+02:00 [INFO][csnas][repl]: start planning invocation="62" 2018-09-27T05:08:58+02:00 [INFO][csnas][repl]: start working invocation="62" 2018-09-27T05:09:57+02:00 [INFO][csnas]: start pruning sender invocation="62" 2018-09-27T05:10:11+02:00 [INFO][csnas]: start pruning receiver invocation="62" 2018-09-27T05:10:32+02:00 [INFO][csnas]: wait for wakeups 2018-09-27T06:08:39+02:00 [INFO][csnas]: start replication invocation="63" 2018-09-27T06:08:39+02:00 [INFO][csnas][repl]: start planning invocation="63" 2018-09-27T06:08:44+02:00 [INFO][csnas][repl]: start working invocation="63" 2018-09-27T06:08:49+02:00 [ERRO][csnas][repl]: receive request failed (might also be error on sender) invocation="63" filesystem="" err="concurrent use of RPC connection" step="(@zrepl_20180927_030838_000 => @zrepl_20180927_040835_000)" errType="*errors.errorString" panic: runtime error: invalid memory address or nil pointer dereference [signal SIGSEGV: segmentation violation code=0x1 addr=0x8 pc=0x7d484b] goroutine 3938545 [running]: os.(*ProcessState).os.sys(...) /usr/lib/golang/src/os/exec_posix.go:78 os.(*ProcessState).Sys(...) /usr/lib/golang/src/os/exec.go:157 github.com/zrepl/zrepl/util.(*IOCommand).doWait(0xc4201b2d80, 0xc420070060, 0xc420070060) /go/github.com/zrepl/zrepl/util/iocommand.go:91 +0x4b github.com/zrepl/zrepl/util.(*IOCommand).Read(0xc4201b2d80, 0xc420790000, 0x8000, 0x8000, 0x800c76d90, 0x0, 0xc420067c10) /go/github.com/zrepl/zrepl/util/iocommand.go:82 +0xe4 github.com/zrepl/zrepl/util.(*ByteCounterReader).Read(0xc4202dc580, 0xc420790000, 0x8000, 0x8000, 0x8c6900, 0x7cb201, 0xc420790000) /go/github.com/zrepl/zrepl/util/io.go:118 +0x51 github.com/zrepl/zrepl/vendor/github.com/problame/go-streamrpc.(*chunkBuffer).readChunk(0xc42057e3c0, 0x800d1bbf0, 0xc4202dc580, 0xc420790000, 0x8000, 0x8000) /go/github.com/zrepl/zrepl/vendor/github.com/problame/go-streamrpc/stream.go:58 +0x5e github.com/zrepl/zrepl/vendor/github.com/problame/go-streamrpc.writeStream(0xa04620, 0xc4204a9c20, 0x9fe340, 0xc4200d6380, 0x800d1bbf0, 0xc4202dc580, 0x8000, 0xc42000e000, 0x900420) /go/github.com/zrepl/zrepl/vendor/github.com/problame/go-streamrpc/stream.go:101 +0x1ce github.com/zrepl/zrepl/vendor/github.com/problame/go-streamrpc.(*Conn).send(0xc4200d6380, 0xa04620, 0xc4204a9c20, 0xc42057e2c0, 0xc42013d570, 0x800d1bbf0, 0xc4202dc580, 0x0, 0x0) /go/github.com/zrepl/zrepl/vendor/github.com/problame/go-streamrpc/main.go:374 +0x557 github.com/zrepl/zrepl/vendor/github.com/problame/go-streamrpc.(*Client).RequestReply.func1(0x999741, 0x7, 0xc4200d6380, 0xa04620, 0xc4204a9c20, 0xc42013d570, 0xa00aa0, 0xc4202dc580, 0xc420516480) /go/github.com/zrepl/zrepl/vendor/github.com/problame/go-streamrpc/client.go:169 +0x148 created by github.com/zrepl/zrepl/vendor/github.com/problame/go-streamrpc.(*Client).RequestReply /go/github.com/zrepl/zrepl/vendor/github.com/problame/go-streamrpc/client.go:167 +0x227 --- util/iocommand.go | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/util/iocommand.go b/util/iocommand.go index 406e98f..0433446 100644 --- a/util/iocommand.go +++ b/util/iocommand.go @@ -4,6 +4,7 @@ import ( "bytes" "fmt" "io" + "os" "os/exec" "syscall" ) @@ -88,9 +89,23 @@ func (c *IOCommand) Read(buf []byte) (n int, err error) { func (c *IOCommand) doWait() (err error) { waitErr := c.Cmd.Wait() - waitStatus := c.Cmd.ProcessState.Sys().(syscall.WaitStatus) // Fail hard if we're not on UNIX + var wasUs bool = false + var waitStatus syscall.WaitStatus + if c.Cmd.ProcessState == nil { + fmt.Fprintf(os.Stderr, "util.IOCommand: c.Cmd.ProcessState is nil after c.Cmd.Wait()\n") + } + if c.Cmd.ProcessState != nil { + sysSpecific := c.Cmd.ProcessState.Sys() + var ok bool + waitStatus, ok = sysSpecific.(syscall.WaitStatus) + if !ok { + fmt.Fprintf(os.Stderr, "util.IOCommand: c.Cmd.ProcessState.Sys() could not be converted to syscall.WaitStatus: %T\n", sysSpecific) + os.Stderr.Sync() + panic(sysSpecific) // this can only be true if we are not on UNIX, and we don't support that + } + wasUs = waitStatus.Signaled() && waitStatus.Signal() == syscall.SIGTERM // in Close() + } - wasUs := waitStatus.Signaled() && waitStatus.Signal() == syscall.SIGTERM // in Close() if waitErr != nil && !wasUs { err = IOCommandError{ WaitErr: waitErr, From 01668a989e9e8d793c5b0a971b3a8f7f75a71c07 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 11 Oct 2018 13:06:47 +0200 Subject: [PATCH 142/167] transport local: named listeners + struct renaming --- config/config.go | 2 ++ config/samples/local.yml | 2 ++ daemon/connecter/connect_local.go | 10 ++++-- daemon/serve/serve_local.go | 56 +++++++++++++++++++++---------- 4 files changed, 49 insertions(+), 21 deletions(-) diff --git a/config/config.go b/config/config.go index e53a1bf..55fdfa6 100644 --- a/config/config.go +++ b/config/config.go @@ -163,6 +163,7 @@ type SSHStdinserverConnect struct { type LocalConnect struct { ConnectCommon `yaml:",inline"` + ListenerName string `yaml:"listener_name"` ClientIdentity string `yaml:"client_identity"` } @@ -198,6 +199,7 @@ type StdinserverServer struct { type LocalServe struct { ServeCommon `yaml:",inline"` + ListenerName string `yaml:"listener_name"` } type PruningEnum struct { diff --git a/config/samples/local.yml b/config/samples/local.yml index 5cc6c2c..e7080fd 100644 --- a/config/samples/local.yml +++ b/config/samples/local.yml @@ -5,11 +5,13 @@ jobs: root_dataset: "storage/zrepl/sink" serve: type: local + listener_name: localsink - type: push name: "backup_system" connect: type: local + listener_name: localsink client_identity: local_backup filesystems: { "system<": true, diff --git a/daemon/connecter/connect_local.go b/daemon/connecter/connect_local.go index 252a9b6..c70b285 100644 --- a/daemon/connecter/connect_local.go +++ b/daemon/connecter/connect_local.go @@ -9,6 +9,7 @@ import ( ) type LocalConnecter struct { + listenerName string clientIdentity string } @@ -16,11 +17,14 @@ func LocalConnecterFromConfig(in *config.LocalConnect) (*LocalConnecter, error) if in.ClientIdentity == "" { return nil, fmt.Errorf("ClientIdentity must not be empty") } - return &LocalConnecter{in.ClientIdentity}, nil + if in.ListenerName == "" { + return nil, fmt.Errorf("ListenerName must not be empty") + } + return &LocalConnecter{listenerName: in.ListenerName, clientIdentity: in.ClientIdentity}, nil } func (c *LocalConnecter) Connect(dialCtx context.Context) (conn net.Conn, err error) { - switchboard := serve.GetLocalListenerSwitchboard() - return switchboard.DialContext(dialCtx, c.clientIdentity) + l := serve.GetLocalListener(c.listenerName) + return l.Connect(dialCtx, c.clientIdentity) } diff --git a/daemon/serve/serve_local.go b/daemon/serve/serve_local.go index ba122ae..19498fc 100644 --- a/daemon/serve/serve_local.go +++ b/daemon/serve/serve_local.go @@ -10,18 +10,28 @@ import ( "sync" ) -var localListenerSwitchboardSingleton struct { - s *LocalListenerSwitchboard - once sync.Once +var localListeners struct { + m map[string]*LocalListener // listenerName -> listener + init sync.Once + mtx sync.Mutex } -func GetLocalListenerSwitchboard() (*LocalListenerSwitchboard) { - localListenerSwitchboardSingleton.once.Do(func() { - localListenerSwitchboardSingleton.s = &LocalListenerSwitchboard{ - connects: make(chan connectRequest), - } +func GetLocalListener(listenerName string) (*LocalListener) { + + localListeners.init.Do(func() { + localListeners.m = make(map[string]*LocalListener) }) - return localListenerSwitchboardSingleton.s + + localListeners.mtx.Lock() + defer localListeners.mtx.Unlock() + + l, ok := localListeners.m[listenerName] + if !ok { + l = newLocalListener() + localListeners.m[listenerName] = l + } + return l + } type connectRequest struct { @@ -34,11 +44,18 @@ type connectResult struct { err error } -type LocalListenerSwitchboard struct { +type LocalListener struct { connects chan connectRequest } -func (l *LocalListenerSwitchboard) DialContext(dialCtx context.Context, clientIdentity string) (conn net.Conn, err error) { +func newLocalListener() *LocalListener { + return &LocalListener{ + connects: make(chan connectRequest), + } +} + +// Connect to the LocalListener from a client with identity clientIdentity +func (l *LocalListener) Connect(dialCtx context.Context, clientIdentity string) (conn net.Conn, err error) { // place request req := connectRequest{ @@ -71,7 +88,7 @@ func (localAddr) Network() string { return "local" } func (a localAddr) String() string { return a.S } -func (l *LocalListenerSwitchboard) Addr() (net.Addr) { return localAddr{""} } +func (l *LocalListener) Addr() (net.Addr) { return localAddr{""} } type localConn struct { net.Conn @@ -80,7 +97,7 @@ type localConn struct { func (l localConn) ClientIdentity() string { return l.clientIdentity } -func (l *LocalListenerSwitchboard) Accept(ctx context.Context) (AuthenticatedConn, error) { +func (l *LocalListener) Accept(ctx context.Context) (AuthenticatedConn, error) { respondToRequest := func(req connectRequest, res connectResult) (err error) { getLogger(ctx). WithField("res.conn", res.conn).WithField("res.err", res.err). @@ -187,7 +204,7 @@ func makeSocketpairConn() (a, b net.Conn, err error) { return a, b, nil } -func (l *LocalListenerSwitchboard) Close() error { +func (l *LocalListener) Close() error { // FIXME: make sure concurrent Accepts return with error, and further Accepts return that error, too // Example impl: for each accept, do context.WithCancel, and store the cancel in a list // When closing, set a member variable to state=closed, make sure accept will exit early @@ -197,15 +214,18 @@ func (l *LocalListenerSwitchboard) Close() error { } type LocalListenerFactory struct { - clients []string + listenerName string } func LocalListenerFactoryFromConfig(g *config.Global, in *config.LocalServe) (f *LocalListenerFactory, err error) { - return &LocalListenerFactory{}, nil + if in.ListenerName == "" { + return nil, fmt.Errorf("ListenerName must not be empty") + } + return &LocalListenerFactory{listenerName: in.ListenerName}, nil } -func (*LocalListenerFactory) Listen() (AuthenticatedListener, error) { - return GetLocalListenerSwitchboard(), nil +func (lf *LocalListenerFactory) Listen() (AuthenticatedListener, error) { + return GetLocalListener(lf.listenerName), nil } From 93c90cd70588bc9ef6743fdb8f12a20bf84267d4 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 11 Oct 2018 13:07:52 +0200 Subject: [PATCH 143/167] pruning: fix YAML representation of PruneKeepRegex --- config/config.go | 4 ++-- config/samples/pull_ssh.yml | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/config/config.go b/config/config.go index 55fdfa6..add4a65 100644 --- a/config/config.go +++ b/config/config.go @@ -218,7 +218,7 @@ type PruneKeepLastN struct { type PruneKeepRegex struct { // FIXME rename to KeepRegex Type string `yaml:"type"` - Regex string `yaml:"prefix"` + Regex string `yaml:"regex"` } type LoggingOutletEnum struct { @@ -341,7 +341,7 @@ func (t *PruningEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) "not_replicated": &PruneKeepNotReplicated{}, "last_n": &PruneKeepLastN{}, "grid": &PruneGrid{}, - "prefix": &PruneKeepRegex{}, + "regex": &PruneKeepRegex{}, }) return } diff --git a/config/samples/pull_ssh.yml b/config/samples/pull_ssh.yml index 920f237..be5229b 100644 --- a/config/samples/pull_ssh.yml +++ b/config/samples/pull_ssh.yml @@ -21,8 +21,8 @@ jobs: grid: 1x1h(keep=all) | 24x1h | 14x1d regex: "^zrepl_.*" keep_receiver: - - type: prefix - prefix: keep_ + - type: regex + regex: keep_ - type: grid grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d - regex: "^zrepl_.*" \ No newline at end of file + regex: "^zrepl_.*" From 14febbeb4c4cc062effae8f9a74299dfd2b5b6bf Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 11 Oct 2018 13:09:04 +0200 Subject: [PATCH 144/167] config: skip files that do not end in .yml --- config/config_test.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/config/config_test.go b/config/config_test.go index 64a12f9..07bbbb8 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -2,6 +2,7 @@ package config import ( "github.com/kr/pretty" + "path" "path/filepath" "testing" "github.com/stretchr/testify/assert" @@ -17,6 +18,11 @@ func TestSampleConfigsAreParsedWithoutErrors(t *testing.T) { for _, p := range paths { + if path.Ext(p) != ".yml" { + t.Logf("skipping file %s", p) + continue + } + t.Run(p, func(t *testing.T) { c, err := ParseConfig(p) if err != nil { From 4e16952ad9dc506a6c7858771fae7ec9aa01ddee Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 11 Oct 2018 15:22:52 +0200 Subject: [PATCH 145/167] snapshotting: support 'periodic' and 'manual' mode 1. Change config format to support multiple types of snapshotting modes. 2. Implement a hacky way to support periodic or completely manual snaphots. In manual mode, the user has to trigger replication using the wakeup mechanism after they took snapshots using their own tooling. As indicated by the comment, a more general solution would be desirable, but we want to get the release out and 'manual' mode is a feature that some people requested... --- config/config.go | 27 +++++++++++--- config/config_minimal_test.go | 3 +- config/config_snapshotting_test.go | 57 ++++++++++++++++++++++++++++++ config/samples/local.yml | 3 +- config/samples/push.yml | 3 +- config/samples/source.yml | 3 +- config/samples/source_ssh.yml | 4 ++- daemon/job/active.go | 4 +-- daemon/job/passive.go | 4 +-- daemon/snapper/snapper.go | 6 ++-- daemon/snapper/snapper_all.go | 39 ++++++++++++++++++++ 11 files changed, 134 insertions(+), 19 deletions(-) create mode 100644 config/config_snapshotting_test.go create mode 100644 daemon/snapper/snapper_all.go diff --git a/config/config.go b/config/config.go index add4a65..7a49d65 100644 --- a/config/config.go +++ b/config/config.go @@ -38,7 +38,7 @@ type PassiveJob struct { type PushJob struct { ActiveJob `yaml:",inline"` - Snapshotting Snapshotting `yaml:"snapshotting"` + Snapshotting SnapshottingEnum `yaml:"snapshotting"` Filesystems FilesystemsFilter `yaml:"filesystems"` } @@ -55,15 +55,24 @@ type SinkJob struct { type SourceJob struct { PassiveJob `yaml:",inline"` - Snapshotting Snapshotting `yaml:"snapshotting"` + Snapshotting SnapshottingEnum `yaml:"snapshotting"` Filesystems FilesystemsFilter `yaml:"filesystems"` } type FilesystemsFilter map[string]bool -type Snapshotting struct { - SnapshotPrefix string `yaml:"snapshot_prefix"` - Interval time.Duration `yaml:"interval,positive"` +type SnapshottingEnum struct { + Ret interface{} +} + +type SnapshottingPeriodic struct { + Type string `yaml:"type"` + Prefix string `yaml:"prefix"` + Interval time.Duration `yaml:"interval,positive"` +} + +type SnapshottingManual struct { + Type string `yaml:"type"` } type PruningSenderReceiver struct { @@ -346,6 +355,14 @@ func (t *PruningEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) return } +func (t *SnapshottingEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { + t.Ret, err = enumUnmarshal(u, map[string]interface{}{ + "periodic": &SnapshottingPeriodic{}, + "manual": &SnapshottingManual{}, + }) + return +} + func (t *LoggingOutletEnum) UnmarshalYAML(u func(interface{}, bool) error) (err error) { t.Ret, err = enumUnmarshal(u, map[string]interface{}{ "stdout": &StdoutLoggingOutlet{}, diff --git a/config/config_minimal_test.go b/config/config_minimal_test.go index fa9c780..72f8752 100644 --- a/config/config_minimal_test.go +++ b/config/config_minimal_test.go @@ -28,8 +28,7 @@ jobs: "pool1/poudriere/ports<": false #don't backup the ports trees } snapshotting: - snapshot_prefix: zrepl_ - interval: 10m + type: manual pruning: keep_sender: - type: not_replicated diff --git a/config/config_snapshotting_test.go b/config/config_snapshotting_test.go new file mode 100644 index 0000000..e0f826b --- /dev/null +++ b/config/config_snapshotting_test.go @@ -0,0 +1,57 @@ +package config + +import ( + "fmt" + "github.com/stretchr/testify/assert" + "testing" + "time" +) + +func TestSnapshotting(t *testing.T) { + tmpl := ` +jobs: +- name: foo + type: push + connect: + type: local + listener_name: foo + client_identity: bar + filesystems: {"<": true} + %s + pruning: + keep_sender: + - type: last_n + count: 10 + keep_receiver: + - type: last_n + count: 10 +` + manual := ` + snapshotting: + type: manual +` + periodic := ` + snapshotting: + type: periodic + prefix: zrepl_ + interval: 10m +` + + fillSnapshotting := func(s string) string {return fmt.Sprintf(tmpl, s)} + var c *Config + + t.Run("manual", func(t *testing.T) { + c = testValidConfig(t, fillSnapshotting(manual)) + snm := c.Jobs[0].Ret.(*PushJob).Snapshotting.Ret.(*SnapshottingManual) + assert.Equal(t, "manual", snm.Type) + }) + + t.Run("periodic", func(t *testing.T) { + c = testValidConfig(t, fillSnapshotting(periodic)) + snp := c.Jobs[0].Ret.(*PushJob).Snapshotting.Ret.(*SnapshottingPeriodic) + assert.Equal(t, "periodic", snp.Type) + assert.Equal(t, 10*time.Minute, snp.Interval) + assert.Equal(t, "zrepl_" , snp.Prefix) + }) + +} diff --git a/config/samples/local.yml b/config/samples/local.yml index e7080fd..7c886ba 100644 --- a/config/samples/local.yml +++ b/config/samples/local.yml @@ -17,8 +17,9 @@ jobs: "system<": true, } snapshotting: - snapshot_prefix: zrepl_ + type: periodic interval: 10m + prefix: zrepl_ pruning: keep_sender: - type: not_replicated diff --git a/config/samples/push.yml b/config/samples/push.yml index cf941f0..6045d67 100644 --- a/config/samples/push.yml +++ b/config/samples/push.yml @@ -9,8 +9,7 @@ jobs: type: tcp address: "backup-server.foo.bar:8888" snapshotting: - snapshot_prefix: zrepl_ - interval: 10m + type: manual pruning: keep_sender: - type: not_replicated diff --git a/config/samples/source.yml b/config/samples/source.yml index 9162421..f10f14b 100644 --- a/config/samples/source.yml +++ b/config/samples/source.yml @@ -12,5 +12,6 @@ jobs: "secret": false } snapshotting: - snapshot_prefix: zrepl_ + type: periodic interval: 10m + prefix: zrepl_ diff --git a/config/samples/source_ssh.yml b/config/samples/source_ssh.yml index b1e6e11..f81f44d 100644 --- a/config/samples/source_ssh.yml +++ b/config/samples/source_ssh.yml @@ -11,5 +11,7 @@ jobs: "secret": false } snapshotting: - snapshot_prefix: zrepl_ + type: periodic interval: 10m + prefix: zrepl_ + diff --git a/daemon/job/active.go b/daemon/job/active.go index 69cd7e8..15af588 100644 --- a/daemon/job/active.go +++ b/daemon/job/active.go @@ -62,7 +62,7 @@ type activeMode interface { type modePush struct { fsfilter endpoint.FSFilter - snapper *snapper.Snapper + snapper *snapper.PeriodicOrManual } func (m *modePush) SenderReceiver(client *streamrpc.Client) (replication.Sender, replication.Receiver, error) { @@ -86,7 +86,7 @@ func modePushFromConfig(g *config.Global, in *config.PushJob) (*modePush, error) } m.fsfilter = fsf - if m.snapper, err = snapper.FromConfig(g, fsf, &in.Snapshotting); err != nil { + if m.snapper, err = snapper.FromConfig(g, fsf, in.Snapshotting); err != nil { return nil, errors.Wrap(err, "cannot build snapper") } diff --git a/daemon/job/passive.go b/daemon/job/passive.go index 6e5cbb4..02ac9b6 100644 --- a/daemon/job/passive.go +++ b/daemon/job/passive.go @@ -72,7 +72,7 @@ func modeSinkFromConfig(g *config.Global, in *config.SinkJob) (m *modeSink, err type modeSource struct { fsfilter zfs.DatasetFilter - snapper *snapper.Snapper + snapper *snapper.PeriodicOrManual } func modeSourceFromConfig(g *config.Global, in *config.SourceJob) (m *modeSource, err error) { @@ -84,7 +84,7 @@ func modeSourceFromConfig(g *config.Global, in *config.SourceJob) (m *modeSource } m.fsfilter = fsf - if m.snapper, err = snapper.FromConfig(g, fsf, &in.Snapshotting); err != nil { + if m.snapper, err = snapper.FromConfig(g, fsf, in.Snapshotting); err != nil { return nil, errors.Wrap(err, "cannot build snapper") } diff --git a/daemon/snapper/snapper.go b/daemon/snapper/snapper.go index bfaeb47..6cd5b98 100644 --- a/daemon/snapper/snapper.go +++ b/daemon/snapper/snapper.go @@ -112,8 +112,8 @@ func getLogger(ctx context.Context) Logger { return logger.NewNullLogger() } -func FromConfig(g *config.Global, fsf *filters.DatasetMapFilter, in *config.Snapshotting) (*Snapper, error) { - if in.SnapshotPrefix == "" { +func PeriodicFromConfig(g *config.Global, fsf *filters.DatasetMapFilter, in *config.SnapshottingPeriodic) (*Snapper, error) { + if in.Prefix == "" { return nil, errors.New("prefix must not be empty") } if in.Interval <= 0 { @@ -121,7 +121,7 @@ func FromConfig(g *config.Global, fsf *filters.DatasetMapFilter, in *config.Snap } args := args{ - prefix: in.SnapshotPrefix, + prefix: in.Prefix, interval: in.Interval, fsf: fsf, // ctx and log is set in Run() diff --git a/daemon/snapper/snapper_all.go b/daemon/snapper/snapper_all.go new file mode 100644 index 0000000..22281dc --- /dev/null +++ b/daemon/snapper/snapper_all.go @@ -0,0 +1,39 @@ +package snapper + +import ( + "context" + "fmt" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon/filters" +) + +// FIXME: properly abstract snapshotting: +// - split up things that trigger snapshotting from the mechanism +// - timer-based trigger (periodic) +// - call from control socket (manual) +// - mixed modes? +// - support a `zrepl snapshot JOBNAME` subcommand for config.SnapshottingManual +type PeriodicOrManual struct { + s *Snapper +} + +func (s *PeriodicOrManual) Run(ctx context.Context, wakeUpCommon chan <- struct{}) { + if s.s != nil { + s.s.Run(ctx, wakeUpCommon) + } +} + +func FromConfig(g *config.Global, fsf *filters.DatasetMapFilter, in config.SnapshottingEnum) (*PeriodicOrManual, error) { + switch v := in.Ret.(type) { + case *config.SnapshottingPeriodic: + snapper, err := PeriodicFromConfig(g, fsf, v) + if err != nil { + return nil, err + } + return &PeriodicOrManual{snapper}, nil + case *config.SnapshottingManual: + return &PeriodicOrManual{}, nil + default: + return nil, fmt.Errorf("unknown snapshotting type %T", v) + } +} From 525a87582536ab9c6ebb2c76baa7c4d99242e1e2 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 11 Oct 2018 17:45:41 +0200 Subject: [PATCH 146/167] main: better descriptions for root subcommands --- main.go | 24 +++++++++--------------- 1 file changed, 9 insertions(+), 15 deletions(-) diff --git a/main.go b/main.go index b2985c9..d23aae8 100644 --- a/main.go +++ b/main.go @@ -14,18 +14,12 @@ import ( var rootCmd = &cobra.Command{ Use: "zrepl", - Short: "ZFS dataset replication", - Long: `Replicate ZFS filesystems & volumes between pools: - - - push & pull mode - - automatic snapshot creation & pruning - - local / over the network - - ACLs instead of blank SSH access`, + Short: "One-stop ZFS replication solution", } var daemonCmd = &cobra.Command{ Use: "daemon", - Short: "daemon", + Short: "run the zrepl daemon", RunE: func(cmd *cobra.Command, args []string) error { conf, err := config.ParseConfig(rootArgs.configFile) if err != nil { @@ -36,8 +30,8 @@ var daemonCmd = &cobra.Command{ } var wakeupCmd = &cobra.Command{ - Use: "wakeup", - Short: "wake up jobs", + Use: "wakeup JOB", + Short: "trigger replication and subsequent pruning for a job", RunE: func(cmd *cobra.Command, args []string) error { conf, err := config.ParseConfig(rootArgs.configFile) if err != nil { @@ -51,7 +45,7 @@ var statusCmdFlags client.StatusFlags var statusCmd = &cobra.Command{ Use: "status", - Short: "status", + Short: "show job activity or dump as JSON for monitoring", RunE: func(cmd *cobra.Command, args []string) error { conf, err := config.ParseConfig(rootArgs.configFile) if err != nil { @@ -63,7 +57,7 @@ var statusCmd = &cobra.Command{ var stdinserverCmd = &cobra.Command{ Use: "stdinserver CLIENT_IDENTITY", - Short: "start in stdinserver mode (from authorized_keys file)", + Short: "stdinserver transport mode (started from authorized_keys file as forced command)", RunE: func(cmd *cobra.Command, args []string) error { conf, err := config.ParseConfig(rootArgs.configFile) if err != nil { @@ -93,7 +87,7 @@ var bashcompCmd = &cobra.Command{ var configcheckCmd = &cobra.Command{ Use: "configcheck", - Short: "validate config file", + Short: "check if config can be parsed without errors", RunE: func(cmd *cobra.Command, args []string) error { conf, err := config.ParseConfig(rootArgs.configFile) if err != nil { @@ -106,7 +100,7 @@ var configcheckCmd = &cobra.Command{ var versionCmdArgs client.VersionArgs var versionCmd = &cobra.Command{ Use: "version", - Short: "print version of zrepl binary (for running daemon 'zrepl control version' command)", + Short: "print version of zrepl binary and running daemon", Run: func(cmd *cobra.Command, args []string) { conf, err := config.ParseConfig(rootArgs.configFile) if err == nil { @@ -160,7 +154,7 @@ func init() { rootCmd.PersistentFlags().StringVar(&rootArgs.configFile, "config", "", "config file path") rootCmd.AddCommand(daemonCmd) rootCmd.AddCommand(wakeupCmd) - statusCmd.Flags().BoolVar(&statusCmdFlags.Raw, "raw", false, "dump raw response from zrepl daemon") + statusCmd.Flags().BoolVar(&statusCmdFlags.Raw, "raw", false, "dump raw status description from zrepl daemon") rootCmd.AddCommand(statusCmd) rootCmd.AddCommand(stdinserverCmd) rootCmd.AddCommand(bashcompCmd) From 0c3a694470c0238cd69769a516e9481cfc7114de Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 11 Oct 2018 17:46:41 +0200 Subject: [PATCH 147/167] fixup: add test for global section --- config/config_global_test.go | 82 ++++++++++++++++++++++++++++++++++++ config/config_test.go | 13 +----- 2 files changed, 83 insertions(+), 12 deletions(-) create mode 100644 config/config_global_test.go diff --git a/config/config_global_test.go b/config/config_global_test.go new file mode 100644 index 0000000..50948b7 --- /dev/null +++ b/config/config_global_test.go @@ -0,0 +1,82 @@ +package config + +import ( + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/zrepl/yaml-config" + "testing" +) + +func testValidGlobalSection(t *testing.T, s string) *Config { + jobdef := ` +jobs: +- name: dummyjob + type: sink + serve: + type: tcp + listen: ":2342" + clients: { + "10.0.0.1":"foo" + } + root_dataset: zoot/foo +` + _, err := ParseConfigBytes([]byte(jobdef)) + require.NoError(t, err) + return testValidConfig(t, s + jobdef) +} + +func TestOutletTypes(t *testing.T) { + conf := testValidGlobalSection(t, ` +global: + logging: + - type: stdout + level: debug + format: human + - type: syslog + level: info + retry_interval: 20s + format: human + - type: tcp + level: debug + format: json + address: logserver.example.com:1234 + - type: tcp + level: debug + format: json + address: encryptedlogserver.example.com:1234 + retry_interval: 20s + tls: + ca: /etc/zrepl/log/ca.crt + cert: /etc/zrepl/log/key.pem + key: /etc/zrepl/log/cert.pem +`) + assert.Equal(t, 4, len(*conf.Global.Logging)) + assert.NotNil(t, (*conf.Global.Logging)[3].Ret.(*TCPLoggingOutlet).TLS) +} + +func TestDefaultLoggingOutlet(t *testing.T) { + conf := testValidGlobalSection(t, "") + assert.Equal(t, 1, len(*conf.Global.Logging)) + o := (*conf.Global.Logging)[0].Ret.(StdoutLoggingOutlet) + assert.Equal(t, "warn", o.Level) + assert.Equal(t, "human", o.Format) +} + +func TestPrometheusMonitoring(t *testing.T) { + conf := testValidGlobalSection(t, ` +global: + monitoring: + - type: prometheus + listen: ':9091' +`) + assert.Equal(t, ":9091", conf.Global.Monitoring[0].Ret.(*PrometheusMonitoring).Listen) +} + +func TestLoggingOutletEnumList_SetDefaults(t *testing.T) { + e := &LoggingOutletEnumList{} + var i yaml.Defaulter = e + require.NotPanics(t, func() { + i.SetDefault() + assert.Equal(t, "warn", (*e)[0].Ret.(StdoutLoggingOutlet).Level) + }) +} diff --git a/config/config_test.go b/config/config_test.go index 07bbbb8..d8e1e2c 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -2,12 +2,10 @@ package config import ( "github.com/kr/pretty" + "github.com/stretchr/testify/require" "path" "path/filepath" "testing" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - "github.com/zrepl/yaml-config" ) func TestSampleConfigsAreParsedWithoutErrors(t *testing.T) { @@ -37,15 +35,6 @@ func TestSampleConfigsAreParsedWithoutErrors(t *testing.T) { } -func TestLoggingOutletEnumList_SetDefaults(t *testing.T) { - e := &LoggingOutletEnumList{} - var i yaml.Defaulter = e - require.NotPanics(t, func() { - i.SetDefault() - assert.Equal(t, "warn", (*e)[0].Ret.(StdoutLoggingOutlet).Level) - }) -} - func testValidConfig(t *testing.T, input string) (*Config) { t.Helper() From 125b561df3cc7a239e5972eaa9ef977673c0639a Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 11 Oct 2018 18:00:23 +0200 Subject: [PATCH 148/167] rename root_dataset to root_fs for receiving-side jobs --- config/config.go | 6 +++--- config/config_global_test.go | 2 +- config/config_rpc_test.go | 8 ++++---- config/samples/local.yml | 2 +- config/samples/pull.yml | 2 +- config/samples/pull_ssh.yml | 2 +- config/samples/sink.yml | 2 +- daemon/job/active.go | 12 ++++++------ daemon/job/passive.go | 2 +- 9 files changed, 19 insertions(+), 19 deletions(-) diff --git a/config/config.go b/config/config.go index 7a49d65..62d2cb3 100644 --- a/config/config.go +++ b/config/config.go @@ -44,13 +44,13 @@ type PushJob struct { type PullJob struct { ActiveJob `yaml:",inline"` - RootDataset string `yaml:"root_dataset"` - Interval time.Duration `yaml:"interval,positive"` + RootFS string `yaml:"root_fs"` + Interval time.Duration `yaml:"interval,positive"` } type SinkJob struct { PassiveJob `yaml:",inline"` - RootDataset string `yaml:"root_dataset"` + RootFS string `yaml:"root_fs"` } type SourceJob struct { diff --git a/config/config_global_test.go b/config/config_global_test.go index 50948b7..f73219e 100644 --- a/config/config_global_test.go +++ b/config/config_global_test.go @@ -18,7 +18,7 @@ jobs: clients: { "10.0.0.1":"foo" } - root_dataset: zoot/foo + root_fs: zoot/foo ` _, err := ParseConfigBytes([]byte(jobdef)) require.NoError(t, err) diff --git a/config/config_rpc_test.go b/config/config_rpc_test.go index f74581d..f02311e 100644 --- a/config/config_rpc_test.go +++ b/config/config_rpc_test.go @@ -16,7 +16,7 @@ jobs: address: "server1.foo.bar:8888" rpc: timeout: 20s # different form default, should merge - root_dataset: "pool2/backup_servers" + root_fs: "pool2/backup_servers" interval: 10m pruning: keep_sender: @@ -32,7 +32,7 @@ jobs: address: "server1.foo.bar:8888" rpc: tx_chunk_size: 0xabcd # different from default, should merge - root_dataset: "pool2/backup_servers" + root_fs: "pool2/backup_servers" interval: 10m pruning: keep_sender: @@ -43,7 +43,7 @@ jobs: - type: sink name: "laptop_sink" - root_dataset: "pool2/backup_laptops" + root_fs: "pool2/backup_laptops" serve: type: tcp listen: "192.168.122.189:8888" @@ -55,7 +55,7 @@ jobs: - type: sink name: "other_sink" - root_dataset: "pool2/backup_laptops" + root_fs: "pool2/backup_laptops" serve: type: tcp listen: "192.168.122.189:8888" diff --git a/config/samples/local.yml b/config/samples/local.yml index 7c886ba..fbf7861 100644 --- a/config/samples/local.yml +++ b/config/samples/local.yml @@ -2,7 +2,7 @@ jobs: - type: sink name: "local_sink" - root_dataset: "storage/zrepl/sink" + root_fs: "storage/zrepl/sink" serve: type: local listener_name: localsink diff --git a/config/samples/pull.yml b/config/samples/pull.yml index 4eb5113..9a32a43 100644 --- a/config/samples/pull.yml +++ b/config/samples/pull.yml @@ -8,7 +8,7 @@ jobs: cert: "/certs/cert.crt" key: "/certs/key.pem" server_cn: "server1" - root_dataset: "pool2/backup_servers" + root_fs: "pool2/backup_servers" interval: 10m pruning: keep_sender: diff --git a/config/samples/pull_ssh.yml b/config/samples/pull_ssh.yml index be5229b..cbe8559 100644 --- a/config/samples/pull_ssh.yml +++ b/config/samples/pull_ssh.yml @@ -10,7 +10,7 @@ jobs: identity_file: /etc/zrepl/ssh/identity options: # optional, default [], `-o` arguments passed to ssh - "Compression=on" - root_dataset: "pool2/backup_servers" + root_fs: "pool2/backup_servers" interval: 10m pruning: keep_sender: diff --git a/config/samples/sink.yml b/config/samples/sink.yml index 01990da..917a3cf 100644 --- a/config/samples/sink.yml +++ b/config/samples/sink.yml @@ -1,7 +1,7 @@ jobs: - type: sink name: "laptop_sink" - root_dataset: "pool2/backup_laptops" + root_fs: "pool2/backup_laptops" serve: type: tls listen: "192.168.122.189:8888" diff --git a/daemon/job/active.go b/daemon/job/active.go index 15af588..edf24e9 100644 --- a/daemon/job/active.go +++ b/daemon/job/active.go @@ -94,13 +94,13 @@ func modePushFromConfig(g *config.Global, in *config.PushJob) (*modePush, error) } type modePull struct { - rootDataset *zfs.DatasetPath + rootFS *zfs.DatasetPath interval time.Duration } func (m *modePull) SenderReceiver(client *streamrpc.Client) (replication.Sender, replication.Receiver, error) { sender := endpoint.NewRemote(client) - receiver, err := endpoint.NewReceiver(m.rootDataset) + receiver, err := endpoint.NewReceiver(m.rootFS) return sender, receiver, err } @@ -133,12 +133,12 @@ func modePullFromConfig(g *config.Global, in *config.PullJob) (m *modePull, err } m.interval = in.Interval - m.rootDataset, err = zfs.NewDatasetPath(in.RootDataset) + m.rootFS, err = zfs.NewDatasetPath(in.RootFS) if err != nil { - return nil, errors.New("root dataset is not a valid zfs filesystem path") + return nil, errors.New("RootFS is not a valid zfs filesystem path") } - if m.rootDataset.Length() <= 0 { - return nil, errors.New("root dataset must not be empty") // duplicates error check of receiver + if m.rootFS.Length() <= 0 { + return nil, errors.New("RootFS must not be empty") // duplicates error check of receiver } return m, nil diff --git a/daemon/job/passive.go b/daemon/job/passive.go index 02ac9b6..ac843b3 100644 --- a/daemon/job/passive.go +++ b/daemon/job/passive.go @@ -60,7 +60,7 @@ func (m *modeSink) RunPeriodic(_ context.Context) {} func modeSinkFromConfig(g *config.Global, in *config.SinkJob) (m *modeSink, err error) { m = &modeSink{} - m.rootDataset, err = zfs.NewDatasetPath(in.RootDataset) + m.rootDataset, err = zfs.NewDatasetPath(in.RootFS) if err != nil { return nil, errors.New("root dataset is not a valid zfs filesystem path") } From 1643198713aa9e7763b057111da417808e1352ea Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 11 Oct 2018 17:46:26 +0200 Subject: [PATCH 149/167] docs: reflect changes in replication_rewrite branch --- docs/changelog.rst | 144 +++++---- docs/configuration.rst | 2 +- docs/configuration/filter_syntax.rst | 67 ++++ docs/configuration/jobs.rst | 384 ++++++++++++++--------- docs/configuration/logging.rst | 25 +- docs/configuration/map_filter_syntax.rst | 108 ------- docs/configuration/monitoring.rst | 3 +- docs/configuration/preface.rst | 2 +- docs/configuration/prune.rst | 155 ++++++--- docs/configuration/transports.rst | 197 ++++++++++-- docs/implementation.rst | 25 +- docs/index.rst | 59 ++-- docs/pr.rst | 15 +- docs/sphinxconf/conf.py | 13 +- docs/tutorial.rst | 160 ++++------ docs/usage.rst | 18 +- 16 files changed, 823 insertions(+), 554 deletions(-) create mode 100644 docs/configuration/filter_syntax.rst delete mode 100644 docs/configuration/map_filter_syntax.rst diff --git a/docs/changelog.rst b/docs/changelog.rst index 744ae2e..1e339cb 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,81 +1,103 @@ -.. |break_config| replace:: **[BREAK]** +.. |break_config| replace:: **[CONFIG]** .. |break| replace:: **[BREAK]** .. |bugfix| replace:: [BUG] .. |docs| replace:: [DOCS] .. |feature| replace:: [FEATURE] +.. _changelog: + Changelog ========= -The changelog summarized bugfixes that are deemed relevant for users. +The changelog summarizes bugfixes that are deemed relevant for users and package maintainers. Developers should consult the git commit log or GitHub issue tracker. -0.0.4 (unreleased) +e use the following annotations for classifying changes: + +* |break_config| Change that breaks the config. + As a package maintainer, make sure to warn your users about config breakage somehow. +* |break| Change that breaks interoperability or persistent state representation with previous releases. + As a package maintainer, make sure to warn your users about config breakage somehow. + Note that even updating the package on both sides might not be sufficient, e.g. if persistent state needs to be migrated to a new format. +* |feature| Change that introduces new functionality. +* |bugfix| Change that fixes a bug, no regressions or incompatibilities expected. +* |docs| Change to the documentation. + +0.1 (unreleased) ------------------ +This release is a milestone for zrepl and required significant refactoring if not rewrites of substantial parts of the application. +It breaks both configuration and transport format, and thus requires manual intervention and updates on both sides of a replication setup. + +Notes to Package Maintainer +~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +* If the daemon crashes, the stack trace produced by the Go runtime and possibly diagnostic output of zrepl will be written to stderr. + This behavior is independent from the ``stdout`` outlet type. + Please make sure the stderr output of the daemon is captured to a file. + Rotation should not be necessary because stderr is not written to under normal circumstances. + To conserve precious stack traces, make sure that multiple service restarts do not directly discard previous stderr output. +* Make it obvious for users how to set the ``GOTRACEBACK`` environment variable to ``GOTRACEBACK=crash``. + This functionality will cause SIGABRT on panics and can be used to capture a coredump of the panicking process. + To that extend, make sure that your package build system, your OS's coredump collection and the Go delve debugger work together. + Use your build system to package the Go program in `this tutorial on Go coredumps and the delve debugger `_ , and make sure the symbol resolution etc. work on coredumps captured from the binary produced by your build system. (Special focus on symbol stripping, etc.) + +Changes +~~~~~~~ + +* |feature| :issue:`55` : Push replication (see :ref:`push job ` and :ref:`sink job `) +* |feature| :ref:`TCP Transport ` +* |feature| :ref:`TCP + TLS client authentication transport ` +* |feature| :issue:`78` TODO MERGE COMMIT Replication protocol rewrite + + * Uses ``github.com/problame/go-streamrpc`` for RPC layer + * |break| zrepl 0.1 and restart on both sides of a replication setup is required + * |feature| :issue:`83`: Improved error handling of network-level errors (zrepl retries instead of failing the entire job) + * |bugfix| :issue:`75` :issue:`81`: use connection timeouts and protocol-level heartbeats + * |break| |break_config|: mappings are no longer supported + + * Receiving sides (``pull`` and ``sink`` job) specify a single ``root_fs``. + Received filesystems are then stored *per client* in ``${root_fs}/${client_identity}``. + +* |feature| |break| |break_config| Manual snapshotting + triggering of replication + + * |feature| :issue:`69`: include manually created snapshots in replication + * |break_config| ``manual`` and ``periodic`` :ref:`snapshotting types ` + * |feature| ``zrepl wakeup JOB`` subcommand to trigger *just* replication + +* |feature| |break| |break_config| New pruning system + + * The active side of a replication (pull or push) decides what to prune for both sender and receiver. + The RPC protocol is used to execute the destroy operations on the remote side. + * New pruning policies (see :ref:`configuration documentation ` ) + + * The decision what snapshots shall be pruned is now made based on *keep rules* + * |feature| :issue:`68`: keep rule ``not_replicated`` prevents divergence of sender and receiver + + * |feature| |break| Bookmark pruning is no longer necessary + + * Per filesystem, zrepl creates a single bookmark (``#zrepl_replication_cursor``) and moves it forward with the most recent successfully replicated snapshot on the receiving side. + * Old bookmarks created prior to zrepl 0.1 (named like their corresponding snapshot) must be deleted manually. + * |break_config| ``keep_bookmarks`` parameter of the ``grid`` keep rule has been removed + +* |feature| ``zrepl status`` for live-updating replication progress (it's really cool!) * |feature| :issue:`67`: Expose `Prometheus `_ metrics via HTTP (:ref:`config docs `) -0.0.3 ------ +* |break_config| Logging outlet types must be specified using the ``type`` instead of ``outlet`` key +* |break| :issue:`53`: CLI: ``zrepl control *`` subcommands have been made direct subcommands of ``zrepl *`` -* |break_config| |feature| :issue:`34`: automatic bookmarking of snapshots +* |bugfix| :issue:`81` :issue:`77` : handle failed accepts correctly (``source`` job) - * Snapshots are automatically bookmarked and pruning of bookmarks **must** be configured. - * This breaks existing configuration: ``grid`` :ref:`prune policy ` specifications require the new ``keep_bookmarks`` parameter. - * Make sure to understand the meaning bookmarks have for :ref:`maximum replication downtime `. - * Example: :sampleconf:`pullbackup/productionhost.yml` +.. |lastrelease| replace:: 0.0.3 -* |break| :commit:`ccd062e`: ``ssh+stdinserver`` transport: changed protocol requires daemon restart on both sides +Previous Releases +----------------- - * The delicate procedure of talking to the serving-side zrepl daemon via the stdinserver proxy command now has better error handling. - * This includes handshakes between client+proxy and client + remote daemo, which is not implemented in previous versions of zrepl. - * The connecting side will therefore time out, with the message ``dial_timeout of 10s exceeded``. - * Both sides of a replication setup must be updated and restarted. Otherwise the connecting side will hang and not time out. - -* |break_config| :commit:`2bfcfa5`: first outlet in ``global.logging`` is now used for logging meta-errors, for example problems encountered when writing to other outlets. -* |feature| :issue:`10`: ``zrepl control status`` subcommand - - * Allows inspection of job activity per task and their log output at runtime. - * Supports ``--format raw`` option for JSON output, usable for monitoring from scripts. - -* |feature| :commit:`d7f3fb9`: subcommand bash completions - - * Package maintainers should install this as appropriate. - -* |bugfix| :issue:`61`: fix excessive memory usage -* |bugfix| :issue:`8` and :issue:`56`: ``ssh+stdinserver`` transport properly reaps SSH child processes -* |bugfix| :commit:`cef63ac`: ``human`` format now prints non-string values correctly -* |bugfix| :issue:`26`: slow TCP outlets no longer block the daemon -* |docs| :issue:`64`: tutorial: document ``known_host`` file entry - -0.0.2 ------ - -* |break_config| :commit:`b95260f`: ``global.logging`` is no longer a dictionary but a list - -* |break_config| :commit:`3e647c1`: ``source`` job field ``datasets`` renamed to ``filesystems`` - - * **NOTE**: zrepl will parse missing ``filesystems`` field as an empty filter, - i.e. no filesystems are presented to the other side. - -* |bugfix| :commit:`72d2885` fix aliasing bug with root `<` subtree wildcard - - * Filesystems paths with final match at blank `s` subtree wildcard are now appended to the target path - * Non-root subtree wildcards, e.g. `zroot/foo/bar<` still map directrly onto the target path - -* Support days (``d``) and weeks (``w``) in durations - -* Docs - - * Ditch Hugo, move to Python Sphinx - * Improve & simplify tutorial (single SSH key per installation) - * Document pruning policies - * Document job types - * Document logging - * Start updating implementation overview +.. NOTE:: + Due to limitations in our documentation system, we only show the changelog since the last release and the time this documentation is built. + For the changelog of previous releases, use the version selection in the hosted version of these docs at `zrepl.github.io `_. + + -0.0.1 ------ - -* Initial release +W diff --git a/docs/configuration.rst b/docs/configuration.rst index 98b7174..ffba719 100644 --- a/docs/configuration.rst +++ b/docs/configuration.rst @@ -10,7 +10,7 @@ Configuration configuration/preface configuration/jobs configuration/transports - configuration/map_filter_syntax + configuration/filter_syntax configuration/prune configuration/logging configuration/monitoring diff --git a/docs/configuration/filter_syntax.rst b/docs/configuration/filter_syntax.rst new file mode 100644 index 0000000..ea6bf29 --- /dev/null +++ b/docs/configuration/filter_syntax.rst @@ -0,0 +1,67 @@ +.. include:: ../global.rst.inc + +.. _pattern-filter: + +Filter Syntax +============= + +For :ref:`source jobs ` and :ref:`push jobs `, a filesystem filter must be defined (field ``filesystems``). +A filter takes a filesystem path (in the ZFS filesystem hierarchy) as parameter and returns ``true`` (pass) or ``false`` (block). + +A filter is specified as a **YAML dictionary** with patterns as keys and booleans as values. +The following rules determine which result is chosen for a given filesystem path: + +* More specific path patterns win over less specific ones +* Non-wildcard patterns (full path patterns) win over *subtree wildcards* (`<` at end of pattern) +* If the path in question does not match any pattern, the result is ``false``. + +The **subtree wildcard** ``<`` means "the dataset left of ``<`` and all its children". + +.. TIP:: + You can try out patterns for a configured job using the ``zrepl test`` subcommand. + +Examples +-------- + +Full Access +~~~~~~~~~~~ + +The following configuration will allow access to all filesystems. + +:: + + jobs: + - type: source + filesystems: { + "<": true, + } + ... + + +Fine-grained +~~~~~~~~~~~~ + +The following configuration demonstrates all rules presented above. + +:: + + jobs: + - type: source + filesystems: { + "tank<": true, # rule 1 + "tank/foo<": false, # rule 2 + "tank/foo/bar": true, # rule 3 + } + ... + + +Which rule applies to given path, and what is the result? + +:: + + tank/foo/bar/loo => 2 false + tank/bar => 1 true + tank/foo/bar => 3 true + zroot => NONE false + tank/var/log => 1 true + diff --git a/docs/configuration/jobs.rst b/docs/configuration/jobs.rst index 1854bd5..cdbafba 100644 --- a/docs/configuration/jobs.rst +++ b/docs/configuration/jobs.rst @@ -1,20 +1,106 @@ .. include:: ../global.rst.inc -.. |patient| replace:: :ref:`patient ` -.. |serve-transport| replace:: :ref:`serve transport` -.. |connect-transport| replace:: :ref:`connect transport` -.. |mapping| replace:: :ref:`mapping ` -.. |filter| replace:: :ref:`filter ` -.. |prune| replace:: :ref:`prune ` +.. |serve-transport| replace:: :ref:`serve specification` +.. |connect-transport| replace:: :ref:`connect specification` +.. |snapshotting-spec| replace:: :ref:`snapshotting specification ` +.. |pruning-spec| replace:: :ref:`pruning specification ` +.. |filter-spec| replace:: :ref:`filter specification` .. _job: -Job Types -========= +Job Types & Replication +======================= + +Overview & Terminology +---------------------- A *job* is the unit of activity tracked by the zrepl daemon and configured in the |mainconfig|. Every job has a unique ``name``, a ``type`` and type-dependent fields which are documented on this page. -Check out the :ref:`tutorial` and :sampleconf:`/` for examples on how job types are actually used. + +Replication always happens between a pair of jobs: one is the **active side**, and one the **passive side**. +The active side executes the replication logic whereas the passive side responds to requests after checking the active side's permissions. +For communication, the active side connects to the passive side using a :ref:`transport ` and starts issuing remote procedure calls (RPCs). + +The following table shows how different job types can be combined to achieve both push and pull mode setups: + ++-----------------------+--------------+----------------------------------+-----------------------------------------------+ +| Setup name | active side | passive side | use case | ++=======================+==============+==================================+===============================================+ +| Push mode | ``push`` | ``sink`` | * Laptop backup | +| | | | * NAS behind NAT to offsite | ++-----------------------+--------------+----------------------------------+-----------------------------------------------+ +| Pull mode | ``pull`` | ``source`` | * Central backup-server for many nodes | +| | | | * Remote server to NAS behind NAT | ++-----------------------+--------------+----------------------------------+-----------------------------------------------+ +| Local replication | | ``push`` + ``sink`` in one config | * Backup FreeBSD boot pool | +| | | with :ref:`local transport ` | | ++-----------------------+--------------+----------------------------------+-----------------------------------------------+ + +How the Active Side Works +~~~~~~~~~~~~~~~~~~~~~~~~~ + +The active side (:ref:`push ` and :ref:`pull ` job) executes the replication and pruning logic: + +* Wakeup because of finished snapshotting (``push`` job) or pull interval ticker (``pull`` job). +* Connect to the corresponding passive side using a :ref:`transport ` and instantiate an RPC client. +* Replicate data from the sending to the receiving side. +* Prune on sender & receiver. + +.. TIP:: + The progress of the active side can be watched live using the ``zrepl status`` subcommand. + +How the Passive Side Works +~~~~~~~~~~~~~~~~~~~~~~~~~~ + +The passive side (:ref:`sink ` and :ref:`source `) waits for connections from the corresponding active side, +using the transport listener type specified in the ``serve`` field of the job configuration. +Each transport listener provides a client's identity to the passive side job. +It uses the client identity for access control: + +* The ``sink`` job only allows pushes to those ZFS filesystems to the active side that are located below ``root_fs/${client_identity}``. +* The ``source`` job has a whitelist of client identities that are allowed pull access. + +.. TIP:: + The implementation of the ``sink`` job requires that the connecting client identities be a valid ZFS filesystem name components. + +How Replication Works +~~~~~~~~~~~~~~~~~~~~~ + +One of the major design goals of the replication module is to avoid any duplication of the nontrivial logic. +As such, the code works on abstract senders and receiver **endpoints**, where typically one will be implemented by a local program object and the other is an RPC client instance. +Regardless of push- or pull-style setup, the logic executes on the active side, i.e. in the ``push`` or ``pull`` job. + +The following steps take place during replication and can be monitored using the ``zrepl status`` subcommand: + +* Plan the replication: + + * Compare sender and receiver filesystem snapshots + * Build the **replication plan** + + * Per filesystem, compute a diff between sender and receiver snapshots + * Build a list of replication steps + + * If possible, use incremental sends (``zfs send -i``) + * Otherwise, use full send of most recent snapshot on sender + * Give up on filesystems that cannot be replicated without data loss + + * Retry on errors that are likely temporary (i.e. network failures). + * Give up on filesystems where a permanent error was received over RPC. + +* Execute the plan + + * Perform replication steps in the following order: + Among all filesystems with pending replication steps, pick the filesystem whose next replication step's snapshot is the oldest. + * After a successful replication step, update the replication cursor bookmark (see below) + +The idea behind the execution order of replication steps is that if the sender snapshots all filesystems simultaneously at fixed intervals, the receiver will have all filesystems snapshotted at time ``T1`` before the first snapshot at ``T2 = T1 + $interval`` is replicated. + +.. _replication-cursor-bookmark: + +The **replication cursor bookmark** ``#zrepl_replication_cursor`` is kept per filesystem on the sending side of a replication setup: +It is a bookmark of the most recent successfully replicated snapshot to the receiving side. +It is is used by the :ref:`not_replicated ` keep rule to identify all snapshots that have not yet been replicated to the receiving side. +Regardless of whether that keep rule is used, the bookmark ensures that replication can always continue incrementally. .. ATTENTION:: @@ -22,12 +108,133 @@ Check out the :ref:`tutorial` and :sampleconf:`/` for examples on how job types Whe receiving a filesystem, it is never mounted (`-u` flag) and `mountpoint=none` is set. This is temporary and being worked on :issue:`24`. + +.. _job-snapshotting-spec: + +Taking Snaphots +--------------- + +The ``push`` and ``source`` jobs can automatically take periodic snapshots of the filesystems matched by the ``filesystems`` filter field. +The snapshot names are composed of a user-defined prefix followed by a UTC date formatted like ``20060102_150405_000``. +We use UTC because it will avoid name conflicts when switching time zones or between summer and winter time. + +For ``push`` jobs, replication is automatically triggered after all filesystems have been snapshotted. + +:: + + jobs: + - type: push + filesystems: { + "<": true, + "tmp": false + } + snapshotting: + type: periodic + prefix: zrepl_ + interval: 10m + ... + +There is also a ``manual`` snapshotting type, which covers the following use cases: + +* Existing infrastructure for automatic snapshots: you only want to use zrepl for replication. +* Run scripts before and after taking snapshots (like locking database tables). + We are working on better integration for this use case: see :issue:`74`. + +Note that you will have to trigger replication manually using the ``zrepl wakeup JOB`` subcommand in that case. + +:: + + jobs: + - type: push + filesystems: { + "<": true, + "tmp": false + } + snapshotting: + type: manual + ... + + +.. _job-push: + +Job Type ``push`` +----------------- + +.. list-table:: + :widths: 20 80 + :header-rows: 1 + + * - Parameter + - Comment + * - ``type`` + - = ``push`` + * - ``name`` + - unique name of the job + * - ``connect`` + - |connect-transport| + * - ``filesystems`` + - |filter-spec| for filesystems to be snapshotted and pushed to the sink + * - ``snapshotting`` + - |snapshotting-spec| + * - ``pruning`` + - |pruning-spec| + +Example config: :sampleconf:`/push.yml` + +.. _job-sink: + +Job Type ``sink`` +----------------- + +.. list-table:: + :widths: 20 80 + :header-rows: 1 + + * - Parameter + - Comment + * - ``type`` + - = ``sink`` + * - ``name`` + - unique name of the job + * - ``serve`` + - |serve-transport| + * - ``root_fs`` + - ZFS dataset path are received to + ``$root_fs/$client_identity`` + +Example config: :sampleconf:`/sink.yml` + +.. _job-pull: + +Job Type ``pull`` +----------------- + +.. list-table:: + :widths: 20 80 + :header-rows: 1 + + * - Parameter + - Comment + * - ``type`` + - = ``pull`` + * - ``name`` + - unique name of the job + * - ``connect`` + - |connect-transport| + * - ``root_fs`` + - ZFS dataset path are received to + ``$root_fs/$client_identity`` + * - ``interval`` + - Interval at which to pull from the source job + * - ``pruning`` + - |pruning-spec| + +Example config: :sampleconf:`/pull.yml` + .. _job-source: -Source Job ----------- - -Example: :sampleconf:`pullbackup/productionhost.yml`. +Job Type ``source`` +------------------- .. list-table:: :widths: 20 80 @@ -42,142 +249,19 @@ Example: :sampleconf:`pullbackup/productionhost.yml`. * - ``serve`` - |serve-transport| * - ``filesystems`` - - |filter| for filesystems to expose to client - * - ``snapshot_prefix`` - - prefix for ZFS snapshots taken by this job - * - ``interval`` - - snapshotting interval - * - ``prune`` - - |prune| for versions of filesytems in ``filesystems``, versions prefixed with ``snapshot_prefix`` + - |filter-spec| for filesystems to be snapshotted and exposed to connecting clients + * - ``snapshotting`` + - |snapshotting-spec| + +Example config: :sampleconf:`/source.yml` + +.. _replication-local: + +Local replication +----------------- + +If you have the need for local replication (most likely between two local storage pools), you can use the :ref:`local transport type ` to connect a local push job to a local sink job. + +Example config: :sampleconf:`/local.yml`. -- Snapshotting Task (every ``interval``, |patient|) - - - A snapshot of filesystems matched by ``filesystems`` is taken every ``interval`` with prefix ``snapshot_prefix``. - - A bookmark of that snapshot is created with the same name. - - The ``prune`` policy is evaluated for versions of filesystems matched by ``filesystems``, versions prefixed with ``snapshot_prefix``. - -- Serve Task - - - Wait for connections from pull job using ``serve``. - -A source job is the counterpart to a :ref:`job-pull`. - -Make sure you read the |prune| policy documentation. - -.. _job-pull: - -Pull Job --------- - -Example: :sampleconf:`pullbackup/backuphost.yml` - -.. list-table:: - :widths: 20 80 - :header-rows: 1 - - * - Parameter - - Comment - * - ``type`` - - = ``pull`` - * - ``name`` - - unqiue name of the job - * - ``connect`` - - |connect-transport| - * - ``interval`` - - Interval between pull attempts - * - ``mapping`` - - |mapping| for remote to local filesystems - * - ``snapshot_prefix`` - - prefix snapshots must match to be considered for replication & pruning - * - ``prune`` - - |prune| policy for versions of filesystems of local filesystems reachable by ``mapping``, versions prefixed with ``snapshot_prefix`` - -* Main Task (every ``interval``, |patient|) - - #. A connection to the remote source job is established using the strategy in ``connect`` - #. ``mapping`` maps filesystems presented by the remote side to local *target filesystems* - #. Those remote filesystems with a local *target filesystem* are replicated - - #. Only snapshots with prefix ``snapshot_prefix`` are replicated. - #. If possible, incremental replication takes place. - #. If the local target filesystem does not exist, the most recent snapshot is sent fully (non-incremental). - #. On conflicts, an error is logged but replication of other filesystems with mapping continues. - - #. The ``prune`` policy is evaluated for all *target filesystems* - -A pull job is the counterpart to a :ref:`job-source`. - -Make sure you read the |prune| policy documentation. - -.. _job-local: - -Local Job ---------- - -Example: :sampleconf:`localbackup/host1.yml` - -.. list-table:: - :widths: 20 80 - :header-rows: 1 - - * - Parameter - - Comment - * - ``type`` - - = ``local`` - * - ``name`` - - unqiue name of the job - * - ``mapping`` - - |mapping| from source to target filesystem (both local) - * - ``snapshot_prefix`` - - prefix for ZFS snapshots taken by this job - * - ``interval`` - - snapshotting & replication interval - * - ``prune_lhs`` - - pruning policy on left-hand-side (source) - * - ``prune_rhs`` - - pruning policy on right-hand-side (target) - -* Main Task (every ``interval``, |patient|) - - #. Evaluate ``mapping`` for local filesystems, those with a *target filesystem* are called *mapped filesystems*. - #. Snapshot *mapped filesystems* with ``snapshot_prefix``. - #. Bookmark the snapshot created above. - #. Replicate *mapped filesystems* to their respective *target filesystems*: - - #. Only snapshots with prefix ``snapshot_prefix`` are replicated. - #. If possible, incremental replication takes place. - #. If the *target filesystem* does not exist, the most recent snapshot is sent fully (non-incremental). - #. On conflicts, an error is logged but replication of other *mapped filesystems* continues. - - #. The ``prune_lhs`` policy is triggered for all *mapped filesystems* - #. The ``prune_rhs`` policy is triggered for all *target filesystems* - -A local job is combination of source & pull job executed on the same machine. - -Terminology ------------ - -task - - A job consists of one or more tasks and a task consists of one or more steps. - Some tasks may be periodic while others wait for an event to occur. - - -patient task - - .. _job-term-patient: - - A patient task is supposed to execute some task every `interval`. - We call the start of the task an *invocation*. - - * If the task completes in less than `interval`, the task is restarted at `last_invocation + interval`. - * Otherwise, a patient job - * logs a warning as soon as a task exceeds its configured `interval` - * waits for the last invocation to finish - * logs a warning with the effective task duration - * immediately starts a new invocation of the task - -filesystem version - - A snapshot or a bookmark. diff --git a/docs/configuration/logging.rst b/docs/configuration/logging.rst index 4be4991..f1a8466 100644 --- a/docs/configuration/logging.rst +++ b/docs/configuration/logging.rst @@ -8,18 +8,17 @@ Logging zrepl uses structured logging to provide users with easily processable log messages. Logging outlets are configured in the ``global`` section of the |mainconfig|. -Check out :sampleconf:`random/logging_and_monitoring.yml` for an example on how to configure multiple outlets: :: global: logging: - - outlet: OUTLET_TYPE + - type: OUTLET_TYPE level: MINIMUM_LEVEL format: FORMAT - - outlet: OUTLET_TYPE + - type: OUTLET_TYPE level: MINIMUM_LEVEL format: FORMAT @@ -45,7 +44,7 @@ By default, the following logging configuration is used global: logging: - - outlet: "stdout" + - type: "stdout" level: "warn" format: "human" @@ -93,8 +92,8 @@ Formats * - Format - Description * - ``human`` - - emphasizes context by putting job, task, step and other context variables into brackets - before the actual message, followed by remaining fields in logfmt style| + - prints job and subsystem into brackets before the actual message, + followed by remaining fields in logfmt style * - ``logfmt`` - `logfmt `_ output. zrepl uses `this Go package `_. * - ``json`` @@ -118,7 +117,7 @@ Outlets are the destination for log entries. * - Parameter - Comment - * - ``outlet`` + * - ``type`` - ``stdout`` * - ``level`` - minimum :ref:`log level ` @@ -126,9 +125,11 @@ Outlets are the destination for log entries. - output :ref:`format ` * - ``time`` - always include time in output (``true`` or ``false``) + * - ``color`` + - colorize output according to log level (``true`` or ``false``) Writes all log entries with minimum level ``level`` formatted by ``format`` to stdout. -If stdout is a tty, interactive usage is assumed and the current time is included in the output. +If stdout is a tty, interactive usage is assumed and both ``time`` and ``color`` are set to ``true``. Can only be specified once. @@ -140,7 +141,7 @@ Can only be specified once. * - Parameter - Comment - * - ``outlet`` + * - ``type`` - ``syslog`` * - ``level`` - minimum :ref:`log level ` @@ -163,7 +164,7 @@ Can only be specified once. * - Parameter - Comment - * - ``outlet`` + * - ``type`` - ``tcp`` * - ``level`` - minimum :ref:`log level ` @@ -179,11 +180,9 @@ Can only be specified once. - TLS config (see below) Establishes a TCP connection to ``address`` and sends log messages with minimum level ``level`` formatted by ``format``. - If ``tls`` is not specified, an unencrypted connection is established. - If ``tls`` is specified, the TCP connection is secured with TLS + Client Authentication. -This is particularly useful in combination with log aggregation services that run on an other machine. +The latter is particularly useful in combination with log aggregation services. .. list-table:: :widths: 10 90 diff --git a/docs/configuration/map_filter_syntax.rst b/docs/configuration/map_filter_syntax.rst deleted file mode 100644 index 3f6f44a..0000000 --- a/docs/configuration/map_filter_syntax.rst +++ /dev/null @@ -1,108 +0,0 @@ -.. include:: ../global.rst.inc - -Mapping & Filter Syntax -======================= - -For various job types, a filesystem ``mapping`` or ``filter`` needs to be -specified. - -Both have in common that they take a filesystem path (in the ZFS filesystem hierarchy)as parameters and return something. -Mappings return a *target filesystem* and filters return a *filter result*. - -The pattern syntax is the same for mappings and filters and is documented in the following section. - -Common Pattern Syntax ---------------------- - -A mapping / filter is specified as a **YAML dictionary** with patterns as keys and -results as values. -The following rules determine which result is chosen for a given filesystem path: - -* More specific path patterns win over less specific ones -* Non-wildcard patterns (full path patterns) win over *subtree wildcards* (`<` at end of pattern) - -The **subtree wildcard** ``<`` means "the dataset left of ``<`` and all its children". - -Example -~~~~~~~ - -:: - - # Rule number and its pattern - 1: tank< # tank and all its children - 2: tank/foo/bar # full path pattern (no wildcard) - 3: tank/foo< # tank/foo and all its children - - # Which rule applies to given path? - tank/foo/bar/loo => 3 - tank/bar => 1 - tank/foo/bar => 2 - zroot => NO MATCH - tank/var/log => 1 - -.. _pattern-mapping: - -Mappings --------- - -Mappings map a *source filesystem path* to a *target filesystem path*. -Per pattern, either a target filesystem path or ``"!"`` is specified as a result. - -* If no pattern matches, there exists no target filesystem (``NO MATCH``). -* If the result is a ``"!"``, there exists no target filesystem (``NO MATCH``). -* If the pattern is a non-wildcard pattern, the source path is mapped to the target path on the right. -* If the pattern ends with a *subtree wildcard* (``<``), the source path is **prefix-trimmed** with the path specified left of ``<``. - - * Note: this means that only for *wildcard-only* patterns (pattern= ``<`` ) is the source path simply appended to the target path. - -The example is from the :sampleconf:`localbackup/host1.yml` example config. - -:: - - jobs: - - name: mirror_local - type: local - mapping: { - "zroot/var/db<": "storage/backups/local/zroot/var/db", - "zroot/usr/home<": "storage/backups/local/zroot/usr/home", - "zroot/usr/home/paranoid": "!", #don't backup paranoid user - "zroot/poudriere/ports<": "!", #don't backup the ports trees - } - ... - - -:: - - zroot/var/db => storage/backups/local/zroot/var/db - zroot/var/db/a/child => storage/backups/local/zroot/var/db/a/child - zroot/usr/home => storage/backups/local/zroot/usr/home - zroot/usr/home/paranoid => NOT MAPPED - zroot/usr/home/bob => storage/backups/local/zroot/usr/home/bob - zroot/usr/src => NOT MAPPED - zroot/poudriere/ports/2017Q3 => NOT MAPPED - zroot/poudriere/ports/HEAD => NOT MAPPED - -.. TIP:: - - You can try out patterns for a configured job using the ``zrepl test`` subcommand. - -.. _pattern-filter: - -Filters -------- - -Valid filter results: ``ok`` or ``!``. - -The example below show the source job from the :ref:`tutorial `: -The corresponding pull job is allowed access to ``zroot/var/db``, ``zroot/usr/home`` + children except ``zroot/usr/home/paranoid``:: - - jobs: - - name: pull_backup - type: source - ... - filesystems: { - "zroot/var/db": "ok", - "zroot/usr/home<": "ok", - "zroot/usr/home/paranoid": "!", - } - ... diff --git a/docs/configuration/monitoring.rst b/docs/configuration/monitoring.rst index ac213e8..8974ee3 100644 --- a/docs/configuration/monitoring.rst +++ b/docs/configuration/monitoring.rst @@ -6,7 +6,6 @@ Monitoring ========== Monitoring endpoints are configured in the ``global.monitoring`` section of the |mainconfig|. -Check out :sampleconf:`random/logging_and_monitoring.yml` for examples. .. _monitoring-prometheus: @@ -17,7 +16,7 @@ zrepl can expose `Prometheus metrics `_ string for tcp, e.g. ``:9091`` or ``127.0.0.1:9091``. The Prometheues monitoring job appears in the ``zrepl control`` job list and may be specified **at most once**. -There is no stability guarantee on the exported metrics. +At the time of writing, there is no stability guarantee on the exported metrics. :: diff --git a/docs/configuration/preface.rst b/docs/configuration/preface.rst index 989f720..271bb6a 100644 --- a/docs/configuration/preface.rst +++ b/docs/configuration/preface.rst @@ -15,7 +15,7 @@ zrepl searches for its main configuration file in the following locations (in th * ``/etc/zrepl/zrepl.yml`` * ``/usr/local/etc/zrepl/zrepl.yml`` -The examples in the :ref:`tutorial` or the ``cmd/sampleconf`` directory should provide a good starting point. +The examples in the :ref:`tutorial` or the :sampleconf:`/` directory should provide a good starting point. ------------------- Runtime Directories diff --git a/docs/configuration/prune.rst b/docs/configuration/prune.rst index 0d5c181..15a6b5d 100644 --- a/docs/configuration/prune.rst +++ b/docs/configuration/prune.rst @@ -3,46 +3,96 @@ Pruning Policies ================ -In zrepl, *pruning* means *destroying filesystem versions by some policy* where filesystem versions are bookmarks and snapshots. +In zrepl, *pruning* means *destroying snapshots*. +Pruning must happen on both sides of a replication or the systems would inevitable run out of disk space at some point. -A *pruning policy* takes a list of filesystem versions and decides for each whether it should be kept or destroyed. +Typically, the requirements to temporal resolution and maximum retention time differ per side. +For example, when using zrepl to back up a busy database server, you will want high temporal resolution (snapshots every 10 min) for the last 24h in case of administrative disasters, but cannot afford to store them for much longer because you might have high turnover volume in the database. +On the receiving side, you may have more disk space available, or need to comply with other backup retention policies. -The job context defines which snapshots are even considered for pruning, for example through the ``snapshot_prefix`` variable. -Check the respective :ref:`job definition ` for details. +zrepl uses a set of **keep rules** to determine which snapshots shall be kept per filesystem. +**A snapshot that is not kept by any rule is destroyed.** +The keep rules are **evaluated on the active side** (:ref:`push ` or :ref:`pull job `) of the replication setup, for both active and passive side, after replication completed or was determined to have failed permanently. -Currently, the :ref:`prune-retention-grid` is the only supported pruning policy. +Example Configuration: + +:: + + jobs: + - type: push + name: ... + connect: ... + filesystems: { + "<": true, + "tmp": false + } + snapshotting: + type: periodic + prefix: zrepl_ + interval: 10m + pruning: + keep_sender: + - type: not_replicated + # make sure manually created snapshots by the administrator are kept + - type: regex + regex: "^manual_.*" + - type: grid + grid: 1x1h(keep=all) | 24x1h | 14x1d + regex: "^zrepl_.*" + keep_receiver: + - type: grid + grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d + regex: "^zrepl_.*" + # manually created snapshots will be kept forever on receiver .. TIP:: You can perform a dry-run of a job's pruning policy using the ``zrepl test`` subcommand. -.. _prune-retention-grid: +.. ATTENTION:: -Retention Grid --------------- + It is currently not possible to define pruning on a source job. + The source job creates snapshots, which means that extended replication downtime will fill up the source's zpool with snapshots, since pruning is directed by the corresponding active side (pull job). + If this is a potential risk for you, consider using :ref:`push mode `. + + +.. _prune-keep-not-replicated: + +Policy ``not_replicated`` +------------------------- + +:: + + jobs: + - type: push + pruning: + keep_sender: + - type: not_replicated + ... + +``not_replicated`` keeps all snapshots that have not been replicated to the receiving side. +It only makes sense to specify this rule on a sender (source or push job). +The state required to evaluate this rule is stored in the :ref:`replication cursor bookmark ` on the sending side. + +.. _prune-keep-retention-grid: + +Policy ``grid`` +--------------- :: jobs: - - name: pull_app-srv - type: pull - ... - prune: - policy: grid - grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d - │ │ + - type: pull + pruning: + keep_receiver: + - type: grid + regex: "^zrepl_.*" + grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d + │ │ └─ one hour interval - │ - └─ 24 adjacent one-hour intervals - - - name: pull_backup - type: source - interval: 10m - prune: - policy: grid - grid: 1x1d(keep=all) - keep_bookmarks: 144 - + │ + └─ 24 adjacent one-hour intervals + ... The retention grid can be thought of as a time-based sieve: The ``grid`` field specifies a list of adjacent time intervals: @@ -51,16 +101,13 @@ All intervals to its right describe time intervals further in the past. Each interval carries a maximum number of snapshots to keep. It is secified via ``(keep=N)``, where ``N`` is either ``all`` (all snapshots are kept) or a positive integer. -The default value is **1**. - -Bookmarks are not affected by the above. -Instead, the ``keep_bookmarks`` field specifies the number of bookmarks to be kept per filesystem. -You only need to specify ``keep_bookmarks`` at the source-side of a replication setup since the destination side does not receive bookmarks. -You can specify ``all`` as a value to keep all bookmarks, but be warned that you should install some other way to prune unneeded ones then (see below). +The default value is **keep=1**. The following procedure happens during pruning: -#. The list of snapshots eligible for pruning is sorted by ``creation`` +#. The list of snapshots is filtered by the regular expression in ``regex``. + Only snapshots names that match the regex are considered for this rule, all others are not affected. +#. The filtered list of snapshots is sorted by ``creation`` #. The left edge of the first interval is aligned to the ``creation`` date of the youngest snapshot #. A list of buckets is created, one for each interval #. The list of snapshots is split up into the buckets. @@ -69,16 +116,42 @@ The following procedure happens during pruning: #. the contained snapshot list is sorted by creation. #. snapshots from the list, oldest first, are destroyed until the specified ``keep`` count is reached. #. all remaining snapshots on the list are kept. -#. The list of bookmarks eligible for pruning is sorted by ``createtxg`` and the most recent ``keep_bookmarks`` bookmarks are kept. -.. _replication-downtime: -.. ATTENTION:: +.. _prune-keep-last-n: + +Policy ``last_n`` +----------------- + +:: + + jobs: + - type: push + pruning: + keep_receiver: + - type: last_n + count: 10 + ... + +``last_n`` keeps the last ``count`` snapshots (last = youngest = most recent creation date). + +.. _prune-keep-regex: + +Policy ``regex`` +---------------- + +:: + + jobs: + - type: push + pruning: + keep_receiver: + - type: regex + regex: "^(zrepl|manual)_.*" + ... + +``regex`` keeps all snapshots whose names are matched by the regular expressionin ``regex``. +Like all other regular expression fields in prune policies, zrepl uses Go's `regexp.Regexp `_ Perl-compatible regular expressions (`Syntax `_). - Be aware that ``keep_bookmarks x interval`` (interval of the job level) controls the **maximum allowable replication downtime** between source and destination. - If replication does not work for whatever reason, source and destination will eventually run out of sync because the source will continue pruning snapshots. - The only recovery in that case is full replication, which may not always be viable due to disk space or traffic constraints. - Further note that while bookmarks consume a constant amount of disk space, listing them requires temporary dynamic **kernel memory** proportional to the number of bookmarks. - Thus, do not use ``all`` or an inappropriately high value without good reason. diff --git a/docs/configuration/transports.rst b/docs/configuration/transports.rst index e1c512a..05e4222 100644 --- a/docs/configuration/transports.rst +++ b/docs/configuration/transports.rst @@ -5,42 +5,156 @@ Transports ========== -A transport provides an authenticated `io.ReadWriteCloser `_ to the RPC layer. -(An ``io.ReadWriteCloser`` is essentially a bidirectional reliable communication channel.) +The zrepl RPC layer uses **transports** to establish a single, bidirectional data stream between an active and passive job. +On the passive (serving) side, the transport also provides the **client identity** to the upper layers: +this string is used for access control and separation of filesystem sub-trees in :ref:`sink jobs `. +Transports are specified in the ``connect`` or ``serve`` section of a job definition. -Currently, only the ``ssh+stdinserver`` transport is supported. +.. ATTENTION:: + + The **client identities must be valid ZFS dataset path components** + because the :ref:`sink job ` uses ``${root_fs}/${client_identity}`` to determine the client's subtree. + +.. _transport-tcp: + +``tcp`` Transport +----------------- + +The ``tcp`` transport uses plain TCP, which means that the data is **not encrypted** on the wire. +Clients are identified by their IPv4 or IPv6 addresses, and the client identity is established through a mapping on the server. + +This transport may also be used in conjunction with network-layer encryption and/or VPN tunnels to provide encryption on the wire. +To make the IP-based client authentication effective, such solutions should provide authenticated IP addresses. +Some options to consider: + +* `WireGuard `_: Linux-focussed, in-kernel TLS +* `OpenVPN `_: Cross-platform VPN, uses tun on \*nix +* `IPSec `_: Properly standardized, in-kernel network-layer VPN +* `spiped `_: think of it as an encrypted pipe between two servers +* SSH + + * `sshuttle `_: VPN-like solution, but using SSH + * `SSH port forwarding `_: Systemd user unit & make it start before the zrepl service. + +Serve +~~~~~ + +:: + + jobs: + - type: sink + serve: + type: tcp + listen: ":8888" + clients: { + "192.168.122.123" : "mysql01" + "192.168.122.123" : "mx01" + } + ... + +Connect +~~~~~~~ + +:: + + jobs: + - type: push + connect: + type: tcp + address: "10.23.42.23:8888" + dial_timeout: # optional, default 10s + ... + +.. _transport-tcp+tlsclientauth: + +``tls`` Transport +----------------- + +The ``tls`` transport uses TCP + TLS with client authentication using client certificates. +The client identity is the common name (CN) presented in the client certificate. +It is recommended to set up a dedicated CA infrastructure for this transport, e.g. using OpenVPN's `EasyRSA `_. + +The implementation uses `Go's TLS library `_. +Since Go binaries are statically linked, you or your distribution need to recompile zrepl when vulnerabilities in that library are disclosed. + +All file paths are resolved relative to the zrepl daemon's working directory. +Specify absolute paths if you are unsure what directory that is (or find out from your init system). + +Serve +~~~~~ + +:: + + jobs: + - type: sink + root_fs: "pool2/backup_laptops" + serve: + type: tls + listen: ":8888" + ca: /etc/zrepl/ca.crt + cert: /etc/zrepl/prod.crt + key: /etc/zrepl/prod.key + client_cns: + - "laptop1" + - "homeserver" + +The ``ca`` field specified the certificate authority used to validate client certificates. +The ``client_cns`` list specifies a list of accepted client common names (which are also the client identities for this transport). + +Connect +~~~~~~~ + +:: + + jobs: + - type: pull + connect: + type: tls + address: "server1.foo.bar:8888" + ca: /etc/zrepl/ca.crt + cert: /etc/zrepl/backupserver.crt + key: /etc/zrepl/backupserver.key + server_cn: "server1" + dial_timeout: # optional, default 10s + +The ``ca`` field specifies the CA which signed the server's certificate (``serve.cert``). +The ``server_cn`` specifies the expected common name (CN) of the server's certificate. +It overrides the hostname specified in ``address``. +The connection fails if either do not match. .. _transport-ssh+stdinserver: ``ssh+stdinserver`` Transport ----------------------------- -The way the ``ssh+stdinserver`` transport works is inspired by `git shell `_ and `Borg Backup `_. +``ssh+stdinserver`` is inspired by `git shell `_ and `Borg Backup `_. It is provided by the Go package ``github.com/problame/go-netssh``. -The config excerpts are taken from the :ref:`tutorial` which you should complete before reading further. .. _transport-ssh+stdinserver-serve: -Serve Mode -~~~~~~~~~~ +Serve +~~~~~ :: jobs: - - name: pull_backup - type: source + - type: source serve: type: stdinserver - client_identity: backup-srv.example.com + client_identities: + - "client1" + - "client2" ... -The serving job opens a UNIX socket named after ``client_identity`` in the runtime directory, e.g. ``/var/run/zrepl/stdinserver/backup-srv.example.com``. +First of all, note that ``type=stdinserver`` in this case: +Currently, only ``connect.type=ssh+stdinserver`` can connect to a ``serve.type=stdinserver``, but we want to keep that option open for future extensions. -On the same machine, the ``zrepl stdinserver $client_identity`` command connects to that socket. -For example, ``zrepl stdinserver backup-srv.example.com`` connects to the UNIX socket ``/var/run/zrepl/stdinserver/backup-srv.example.com``. +The serving job opens a UNIX socket named after ``client_identity`` in the runtime directory. +In our example above, that is ``/var/run/zrepl/stdinserver/client1`` and ``/var/run/zrepl/stdinserver/client2``. +On the same machine, the ``zrepl stdinserver $client_identity`` command connects to ``/var/run/zrepl/stdinserver/$client_identity``. It then passes its stdin and stdout file descriptors to the zrepl daemon via *cmsg(3)*. -zrepl daemon in turn combines them into an ``io.ReadWriteCloser``: +zrepl daemon in turn combines them into an object implementing ``net.Conn``: a ``Write()`` turns into a write to stdout, a ``Read()`` turns into a read from stdin. Interactive use of the ``stdinserver`` subcommand does not make much sense. @@ -54,8 +168,8 @@ This can be achieved with an entry in the ``authorized_keys`` file of the servin # for older OpenSSH versions command="zrepl stdinserver CLIENT_IDENTITY",no-port-forwarding,no-X11-forwarding,no-pty,no-agent-forwarding,no-user-rc CLIENT_SSH_KEY -* CLIENT_IDENTITY is substituted with ``backup-srv.example.com`` in our example -* CLIENT_SSH_KEY is substituted with the public part of the SSH keypair specified in the ``connect`` directive on the connecting host. +* CLIENT_IDENTITY is substituted with an entry from ``client_identities`` in our example +* CLIENT_SSH_KEY is substituted with the public part of the SSH keypair specified in the ``connect.identity_file`` directive on the connecting host. .. NOTE:: @@ -64,24 +178,24 @@ This can be achieved with an entry in the ``authorized_keys`` file of the servin To recap, this is of how client authentication works with the ``ssh+stdinserver`` transport: -* Connections to the ``client_identity`` UNIX socket are blindly trusted by zrepl daemon. -* Thus, the runtime directory must be private to the zrepl user (checked by zrepl daemon) +* Connections to the ``/var/run/zrepl/stdinserver/${client_identity}`` UNIX socket are blindly trusted by zrepl daemon. + The connection client identity is the name of the socket, i.e. ``${client_identity}``. +* Thus, the runtime directory must be private to the zrepl user (this is checked by zrepl daemon) * The admin of the host with the serving zrepl daemon controls the ``authorized_keys`` file. * Thus, the administrator controls the mapping ``PUBKEY -> CLIENT_IDENTITY``. .. _transport-ssh+stdinserver-connect: -Connect Mode -~~~~~~~~~~~~ +Connect +~~~~~~~ :: jobs: - - name: pull_app-srv - type: pull + - type: pull connect: type: ssh+stdinserver - host: app-srv.example.com + host: prod.example.com user: root port: 22 identity_file: /etc/zrepl/ssh/identity @@ -102,15 +216,46 @@ The connecting zrepl daemon #. The remote user, host and port correspond to those configured. #. Further options can be specified using the ``options`` field, which appends each entry in the list to the command line using ``-o $entry``. -#. Wraps the pipe ends in an ``io.ReadWriteCloser`` and uses it for RPC. - +#. Wraps the pipe ends in a ``net.Conn`` and returns it to the RPC layer. As discussed in the section above, the connecting zrepl daemon expects that ``zrepl stdinserver $client_identity`` is executed automatically via an ``authorized_keys`` file entry. -The ``known_hosts`` file used by the ssh command must contain an entry for the serving host, e.g., ``app-srv.example.com`` in the example above. +The ``known_hosts`` file used by the ssh command must contain an entry for ``connect.host`` prior to starting zrepl. +Thus, run the following on the pulling host's command line (substituting ``connect.host``): + +:: + + ssh -i /etc/zrepl/ssh/identity root@prod.example.com .. NOTE:: The environment variables of the underlying SSH process are cleared. ``$SSH_AUTH_SOCK`` will not be available. It is suggested to create a separate, unencrypted SSH key solely for that purpose. + +.. _transport-local: + +``local`` Transport +------------------- + +The local transport can be used to implement :ref:`local replication `, i.e., push replication between a push and sink job defined in the same configuration file. + +The ``listener_name`` is analogous to a hostname and must match between ``serve`` and ``connect``. +The ``client_identity`` is used by the sink as documented above. + +:: + + jobs: + - type: sink + serve: + type: local + listener_name: localsink + ... + + - type: push + connect: + type: local + listener_name: localsink + client_identity: local_backup + ... + diff --git a/docs/implementation.rst b/docs/implementation.rst index bc9b1f9..4e514d2 100644 --- a/docs/implementation.rst +++ b/docs/implementation.rst @@ -5,9 +5,12 @@ Implementation Overview .. WARNING:: - Incomplete / under construction + Incomplete and possibly outdated. + Check out the :ref:`talks about zrepl ` at various conferences for up-to-date material. + Alternatively, have a `look at the source code `_ ;) The following design aspects may convince you that ``zrepl`` is superior to a hacked-together shell script solution. +Also check out the :ref:`talks about zrepl ` at various conferences. Testability & Performance ------------------------- @@ -28,7 +31,7 @@ While it is tempting to just issue a few ``ssh remote 'zfs send ...' | zfs recv` * The snapshot streams need to be compatible. * Communication is still unidirectional. Thus, you will most likely - * either not take advantage of features such as *compressed send & recv* + * either not take advantage of advanced replication features such as *compressed send & recv* * or issue additional ``ssh`` commands in advance to figure out what features are supported on the other side. * Advanced logic in shell scripts is ugly to read, poorly testable and a pain to maintain. @@ -36,25 +39,21 @@ While it is tempting to just issue a few ``ssh remote 'zfs send ...' | zfs recv` zrepl takes a different approach: * Define an RPC protocol. -* Establish an encrypted, authenticated, bidirectional communication channel... -* ... with zrepl running at both ends of it. +* Establish an encrypted, authenticated, bidirectional communication channel. +* Run daemons on both sides of the setup and let them talk to each other. This has several obvious benefits: * No blank root shell access is given to the other side. -* Instead, an *authenticated* peer can *request* filesystem lists, snapshot streams, etc. -* Requests are then checked against job-specific ACLs, limiting a client to the filesystems it is actually allowed to replicate. -* The :ref:`transport mechanism ` is decoupled from the remaining logic, keeping it extensible. +* An *authenticated* peer *requests* filesystem lists, snapshot streams, etc. +* The server decides which filesystems it exposes to which peers. +* The :ref:`transport mechanism ` is decoupled from the remaining logic, which allows us to painlessly offer multiple transport mechanisms. Protocol Implementation ~~~~~~~~~~~~~~~~~~~~~~~ -zrepl implements its own RPC protocol. -This is mostly due to the fact that existing solutions do not provide efficient means to transport large amounts of data. - -Package `github.com/zrepl/zrepl/rpc `_ builds a special-case handling around returning an ``io.Reader`` as part of a unary RPC call. - -Measurements show only a single memory-to-memory copy of a snapshot stream is made using ``github.com/zrepl/zrepl/rpc``, and there is still potential for further optimizations. +zrepl uses a custom RPC protol because, at the time of writing, existing solutions like gRPC do not provide efficient means to transport large amounts of data, whose size is unknown at send time (= zfs send streams). +The package used is `github.com/problame/go-streamrpc `_. Logging & Transparency ---------------------- diff --git a/docs/index.rst b/docs/index.rst index d7de850..6fe0e7e 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -8,10 +8,15 @@ zrepl - ZFS replication ----------------------- -.. ATTENTION:: - zrepl as well as this documentation is still under active development. - It is neither feature complete nor is there a stability guarantee on the configuration format. - Use & test at your own risk ;) +**zrepl** is a one-stop, integrated solution for ZFS replication. + +.. raw:: html + +
+ + + +
Getting started ~~~~~~~~~~~~~~~ @@ -21,42 +26,47 @@ The :ref:`10 minutes tutorial setup ` gives you a first impression. Main Features ~~~~~~~~~~~~~ -* Filesystem Replication +* **Filesystem replication** - * [x] Local & Remote - * [x] Pull mode - * [ ] Push mode - * [x] Access control checks when pulling datasets - * [x] :ref:`Flexible mapping ` rules - * [x] Bookmarks support - * [ ] Feature-negotiation for + * [x] Pull & Push mode + * [x] Multiple transport :ref:`transports `: TCP, TCP + TLS client auth, SSH - * Resumable `send & receive` - * Compressed `send & receive` - * Raw encrypted `send & receive` (as soon as it is available) + * Advanced replication features -* Automatic snapshot creation + * [ ] Resumable send & receive + * [ ] Compressed send & receive + * [ ] Raw encrypted send & receive - * [x] Ensure fixed time interval between snapshots +* **Automatic snapshot management** -* Automatic snapshot :ref:`pruning ` + * [x] Periodic filesystem snapshots + * [x] Flexible :ref:`pruning rule system ` - * [x] Age-based fading (grandfathering scheme) + * [x] Age-based fading (grandfathering scheme) + * [x] Bookmarks to avoid divergence between sender and receiver -* Logging \& Monitoring +* **Sophisticated Monitoring & Logging** - * Detailed & structured :ref:`logging ` + * [x] Live progress reporting via `zrepl status` :ref:`subcommand ` + * [x] Comprehensive, structured :ref:`logging ` * ``human``, ``logfmt`` and ``json`` formatting * stdout, syslog and TCP (+TLS client auth) outlets - * Prometheus :ref:`monitoring ` endpoint + * [x] Prometheus :ref:`monitoring ` endpoint -* Maintainable implementation in Go +* **Maintainable implementation in Go** * [x] Cross platform * [x] Type safe & testable code + +.. ATTENTION:: + zrepl as well as this documentation is still under active development. + There is no stability guarantee on the RPC protocol or configuration format, + but we do our best to document breaking changes in the :ref:`changelog`. + + Contributing ~~~~~~~~~~~~ @@ -88,6 +98,7 @@ Table of Contents configuration usage implementation + pr changelog GitHub Repository & Issue Tracker - pr + diff --git a/docs/pr.rst b/docs/pr.rst index 8d40bec..fcb64d7 100644 --- a/docs/pr.rst +++ b/docs/pr.rst @@ -1,8 +1,17 @@ + +.. _pr-talks: + Talks & Presentations ===================== -* Talk at EuroBSDCon2017 FreeBSD DevSummit ( - `Slides `_, - `Event `_ +* Talk at OpenZFS Developer Summit 2018 of pre-release 0.1 ( + `25min Recording `__ , + `Slides `__ , + `Event `__ + ) +* Talk at EuroBSDCon2017 FreeBSD DevSummit with live demo of zrepl 0.0.3 ( + `55min Recording `__, + `Slides `__, + `Event `__ ) diff --git a/docs/sphinxconf/conf.py b/docs/sphinxconf/conf.py index 6d13b08..b270434 100644 --- a/docs/sphinxconf/conf.py +++ b/docs/sphinxconf/conf.py @@ -101,6 +101,17 @@ html_static_path = ['../_static'] html_logo = '../_static/zrepl.svg' +html_context = { + # https://github.com/rtfd/sphinx_rtd_theme/issues/205 + # Add 'Edit on Github' link instead of 'View page source' + "display_github": True, + "github_user": "zrepl", + "github_repo": "zrepl", + "github_version": "master", + "conf_py_path": "/docs/", + "source_suffix": source_suffix, +} + # -- Options for HTMLHelp output ------------------------------------------ # Output file base name for HTML help builder. @@ -162,7 +173,7 @@ texinfo_documents = [ # http://www.sphinx-doc.org/en/stable/ext/extlinks.html extlinks = { 'issue':('https://github.com/zrepl/zrepl/issues/%s', 'issue #'), - 'sampleconf':('https://github.com/zrepl/zrepl/blob/master/cmd/sampleconf/%s', 'cmd/sampleconf/'), + 'sampleconf':('https://github.com/zrepl/zrepl/blob/master/config/samples%s', 'config/samples'), 'commit':('https://github.com/zrepl/zrepl/commit/%s', 'commit '), } diff --git a/docs/tutorial.rst b/docs/tutorial.rst index fa769ed..db83e22 100644 --- a/docs/tutorial.rst +++ b/docs/tutorial.rst @@ -9,164 +9,120 @@ Tutorial This tutorial shows how zrepl can be used to implement a ZFS-based pull backup. We assume the following scenario: -* Production server ``app-srv`` with filesystems to back up: +* Production server ``prod`` with filesystems to back up: * ``zroot/var/db`` * ``zroot/usr/home`` and all its child filesystems * **except** ``zroot/usr/home/paranoid`` belonging to a user doing backups themselves -* Backup server ``backup-srv`` with +* Backup server ``backups`` with - * Filesystem ``storage/zrepl/pull/app-srv`` + children dedicated to backups of ``app-srv`` + * Filesystem ``storage/zrepl/pull/prod`` + children dedicated to backups of ``prod`` Our backup solution should fulfill the following requirements: -* Periodically snapshot the filesystems on ``app-srv`` *every 10 minutes* -* Incrementally replicate these snapshots to ``storage/zrepl/pull/app-srv/*`` on ``backup-srv`` -* Keep only very few snapshots on ``app-srv`` to save disk space -* Keep a fading history (24 hourly, 30 daily, 6 monthly) of snapshots on ``backup-srv`` +* Periodically snapshot the filesystems on ``prod`` *every 10 minutes* +* Incrementally replicate these snapshots to ``storage/zrepl/pull/prod/*`` on ``backups`` +* Keep only very few snapshots on ``prod`` to save disk space +* Keep a fading history (24 hourly, 30 daily, 6 monthly) of snapshots on ``backups`` Analysis -------- We can model this situation as two jobs: -* A **source job** on ``app-srv`` +* A **source job** on ``prod`` * Creates the snapshots - * Keeps a short history of snapshots to enable incremental replication to ``backup-srv`` - * Accepts connections from ``backup-srv`` + * Keeps a short history of snapshots to enable incremental replication to ``backups`` + * Accepts connections from ``backups`` -* A **pull job** on ``backup-srv`` +* A **pull job** on ``backups`` - * Connects to the ``zrepl daemon`` process on ``app-srv`` - * Pulls the snapshots to ``storage/zrepl/pull/app-srv/*`` - * Fades out snapshots in ``storage/zrepl/pull/app-srv/*`` as they age + * Connects to the ``zrepl daemon`` process on ``prod`` + * Pulls the snapshots to ``storage/zrepl/pull/prod/*`` + * Fades out snapshots in ``storage/zrepl/pull/prod/*`` as they age Why doesn't the **pull job** create the snapshots before pulling? -As is the case with all distributed systems, the link between ``app-srv`` and ``backup-srv`` might be down for an hour or two. +As is the case with all distributed systems, the link between ``prod`` and ``backups`` might be down for an hour or two. We do not want to sacrifice our required backup resolution of 10 minute intervals for a temporary connection outage. -When the link comes up again, ``backup-srv`` will happily catch up the 12 snapshots taken by ``app-srv`` in the meantime, without -a gap in our backup history. +When the link comes up again, ``backups`` will catch up with the snapshots taken by ``prod`` in the meantime, without a gap in our backup history. Install zrepl ------------- Follow the :ref:`OS-specific installation instructions ` and come back here. -Configure ``backup-srv`` ------------------------- +Configure server ``backups`` +---------------------------- -We define a **pull job** named ``pull_app-srv`` in the |mainconfig| on host ``backup-srv``: :: +We define a **pull job** named ``pull_prod`` in ``/etc/zrepl/zrepl.yml`` or ``/usr/local/etc/zrepl/zrepl.yml`` on host ``backups`` : :: jobs: - - name: pull_app-srv + - name: pull_prod type: pull connect: - type: ssh+stdinserver - host: app-srv.example.com - user: root - port: 22 - identity_file: /etc/zrepl/ssh/identity + type: tcp + address: "192.168.2.20:2342" + root_fs: "storage/zrepl/pull/prod" interval: 10m - mapping: { - "<":"storage/zrepl/pull/app-srv" - } - snapshot_prefix: zrepl_pull_backup_ - prune: - policy: grid - grid: 1x1h(keep=all) | 24x1h | 35x1d | 6x30d + pruning: + keep_sender: + - type: not_replicated + - type: last_n + count: 10 + keep_receiver: + - type: grid + grid: 1x1h(keep=all) | 24x1h | 30x1d | 6x30d + regex: "^zrepl_" + interval: 10m -The ``connect`` section instructs the zrepl daemon to use the ``stdinserver`` transport: -``backup-srv`` will connect to the specified SSH server and expect ``zrepl stdinserver CLIENT_IDENTITY`` instead of the shell on the other side. +The ``connect`` section instructs the zrepl daemon to use plain TCP transport. +Check out the :ref:`transports ` section for alternatives that support encryption. -It uses the private key specified at ``connect.identity_file`` which we still need to create: :: +.. _tutorial-configure-prod: - cd /etc/zrepl - mkdir -p ssh - chmod 0700 ssh - ssh-keygen -t ed25519 -N '' -f /etc/zrepl/ssh/identity +Configure server ``prod`` +------------------------- -Note that most use cases do not benefit from separate keypairs per remote endpoint. -Thus, it is sufficient to create one keypair and use it for all ``connect`` directives on one host. - -zrepl uses ssh's default ``known_hosts`` file, which must contain a host identification entry for ``app-srv.example.com``. -If that entry does not already exist, we need to generate it. -Run the following command, compare the host fingerprints, and confirm with yes if they match. -You will not be able to get a shell with the identity file we just generated, which is fine. :: - - ssh -i /etc/zrepl/ssh/identity root@app-srv.example.com - -Learn more about :ref:`transport-ssh+stdinserver` transport and the :ref:`pull job ` format. - -.. _tutorial-configure-app-srv: - -Configure ``app-srv`` ---------------------- - -We define a corresponding **source job** named ``pull_backup`` in the |mainconfig| on host ``app-srv``: :: +We define a corresponding **source job** named ``source_backups`` in ``/etc/zrepl/zrepl.yml`` or ``/usr/local/etc/zrepl/zrepl.yml`` on host ``prod`` : :: jobs: - - name: pull_backup + - name: source_backups type: source serve: - type: stdinserver - client_identity: backup-srv.example.com + type: tcp + listen: ":2342" + clients: { + "192.168.2.10" : "backups" + } filesystems: { - "zroot/var/db": "ok", - "zroot/usr/home<": "ok", - "zroot/usr/home/paranoid": "!", + "zroot/var/db:": true, + "zroot/usr/home<": true, + "zroot/usr/home/paranoid": false } - snapshot_prefix: zrepl_pull_backup_ - interval: 10m - prune: - policy: grid - grid: 1x1d(keep=all) - keep_bookmarks: 144 + snapshotting: + type: periodic + prefix: zrepl_ + interval: 10m -The ``serve`` section corresponds to the ``connect`` section in the configuration of ``backup-srv``. - -We now want to authenticate ``backup-srv`` before allowing it to pull data. -This is done by limiting SSH connections from ``backup-srv`` to execute the ``stdinserver`` subcommand. - -Open ``/root/.ssh/authorized_keys`` and add either of the the following lines.:: - - # for OpenSSH >= 7.2 - command="zrepl stdinserver backup-srv.example.com",restrict CLIENT_SSH_KEY - # for older OpenSSH versions - command="zrepl stdinserver backup-srv.example.com",no-port-forwarding,no-X11-forwarding,no-pty,no-agent-forwarding,no-user-rc CLIENT_SSH_KEY - -.. ATTENTION:: - - Replace CLIENT_SSH_KEY with the contents of ``/etc/zrepl/ssh/identity.pub`` from ``app-srv``. - Mind the trailing ``.pub`` in the filename. - The entries **must** be on a single line, including the replaced CLIENT_SSH_KEY. - - -.. HINT:: - - You may need to adjust the ``PermitRootLogin`` option in ``/etc/ssh/sshd_config`` to ``forced-commands-only`` or higher for this to work. - Refer to sshd_config(5) for details. - -The argument ``backup-srv.example.com`` is the client identity of ``backup-srv`` as defined in ``jobs.serve.client_identity``. - -Again, both :ref:`transport-ssh+stdinserver` transport and the :ref:`job-source` format are documented. +The ``serve`` section whitelists ``backups``'s IP address ``192.168.2.10`` and assigns it the client identity ``backups`` which will show up in the logs. +Again, check the :ref:`docs for encrypted transports `. Apply Configuration Changes --------------------------- -We need to restart the zrepl daemon on **both** ``app-srv`` and ``backup-srv``. +We need to restart the zrepl daemon on **both** ``prod`` and ``backups``. This is :ref:`OS-specific `. Watch it Work ------------- -Run ``zrepl control status`` to view the current activity of the configured jobs. -If a job encountered problems since it last left idle state, the output contains useful debug log. +Run ``zrepl status`` on ``prod`` to monitor the replication and pruning activity. Additionally, you can check the detailed structured logs of the `zrepl daemon` process and use GNU *watch* to view the snapshots present on both machines. @@ -175,7 +131,7 @@ If you like tmux, here is a handy script that works on FreeBSD: :: pkg install gnu-watch tmux tmux new-window tmux split-window "tail -f /var/log/zrepl.log" - tmux split-window "gnu-watch 'zfs list -t snapshot -o name,creation -s creation | grep zrepl_pull_backup_'" + tmux split-window "gnu-watch 'zfs list -t snapshot -o name,creation -s creation | grep zrepl_'" tmux select-layout tiled The Linux equivalent might look like this: :: @@ -183,7 +139,7 @@ The Linux equivalent might look like this: :: # make sure tmux is installed & let's assume you use systemd + journald tmux new-window tmux split-window "journalctl -f -u zrepl.service" - tmux split-window "watch 'zfs list -t snapshot -o name,creation -s creation | grep zrepl_pull_backup_'" + tmux split-window "watch 'zfs list -t snapshot -o name,creation -s creation | grep zrepl_'" tmux select-layout tiled Summary diff --git a/docs/usage.rst b/docs/usage.rst index 3beb7ba..3bddfab 100644 --- a/docs/usage.rst +++ b/docs/usage.rst @@ -10,8 +10,8 @@ CLI Overview .. NOTE:: - To avoid duplication, the zrepl binary is self-documenting: - invoke any subcommand at any level with the ``--help`` flag to get information on the subcommand, available flags, etc. + The zrepl binary is self-documenting: + run ``zrepl help`` for an overview of the available subcommands or ``zrepl SUBCOMMAND --help`` for information on available flags, etc. .. list-table:: :widths: 30 70 @@ -19,16 +19,18 @@ CLI Overview * - Subcommand - Description + * - ``zrepl help`` + - show subcommand overview * - ``zrepl daemon`` - run the daemon, required for all zrepl functionality - * - ``zrepl control`` - - control / query the daemon - * - ``zrepl control status`` - - show job activity / monitoring (``--format raw``) - * - ``zrepl test`` - - test configuration, try pattern syntax, dry run pruning policy, etc. + * - ``zrepl status`` + - show job activity, or with ``--raw`` for JSON output * - ``zrepl stdinserver`` - see :ref:`transport-ssh+stdinserver` + * - ``zrepl wakeup JOB`` + - manually trigger replication + pruning + * - ``zrepl configcheck`` + - check if config can be parsed without errors .. _usage-zrepl-daemon: From a97684923a2c7391bc94d23d419d1b03ea4e8990 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 11 Oct 2018 21:17:43 +0200 Subject: [PATCH 150/167] refactor: socketpair into utils package (useful elsewhere) --- daemon/serve/serve_local.go | 48 ++-------------------------------- util/socketpair/socketpair.go | 49 +++++++++++++++++++++++++++++++++++ 2 files changed, 51 insertions(+), 46 deletions(-) create mode 100644 util/socketpair/socketpair.go diff --git a/daemon/serve/serve_local.go b/daemon/serve/serve_local.go index 19498fc..f71ba70 100644 --- a/daemon/serve/serve_local.go +++ b/daemon/serve/serve_local.go @@ -4,9 +4,8 @@ import ( "context" "fmt" "github.com/zrepl/zrepl/config" - "golang.org/x/sys/unix" + "github.com/zrepl/zrepl/util/socketpair" "net" - "os" "sync" ) @@ -135,7 +134,7 @@ func (l *LocalListener) Accept(ctx context.Context) (AuthenticatedConn, error) { } getLogger(ctx).Debug("creating socketpair") - left, right, err := makeSocketpairConn() + left, right, err := socketpair.SocketPair() if err != nil { res := connectResult{nil, fmt.Errorf("server error: %s", err)} if respErr := respondToRequest(req, res); respErr != nil { @@ -161,49 +160,6 @@ func (l *LocalListener) Accept(ctx context.Context) (AuthenticatedConn, error) { return localConn{right, req.clientIdentity}, nil } -type fileConn struct { - net.Conn // net.FileConn - f *os.File -} - -func (c fileConn) Close() error { - if err := c.Conn.Close(); err != nil { - return err - } - if err := c.f.Close(); err != nil { - return err - } - return nil -} - -func makeSocketpairConn() (a, b net.Conn, err error) { - // don't use net.Pipe, as it doesn't implement things like lingering, which our code relies on - sockpair, err := unix.Socketpair(unix.AF_UNIX, unix.SOCK_STREAM, 0) - if err != nil { - return nil, nil, err - } - toConn := func(fd int) (net.Conn, error) { - f := os.NewFile(uintptr(fd), "fileconn") - if f == nil { - panic(fd) - } - c, err := net.FileConn(f) - if err != nil { - f.Close() - return nil, err - } - return fileConn{Conn: c, f: f}, nil - } - if a, err = toConn(sockpair[0]); err != nil { // shadowing - return nil, nil, err - } - if b, err = toConn(sockpair[1]); err != nil { // shadowing - a.Close() - return nil, nil, err - } - return a, b, nil -} - func (l *LocalListener) Close() error { // FIXME: make sure concurrent Accepts return with error, and further Accepts return that error, too // Example impl: for each accept, do context.WithCancel, and store the cancel in a list diff --git a/util/socketpair/socketpair.go b/util/socketpair/socketpair.go new file mode 100644 index 0000000..615c8f2 --- /dev/null +++ b/util/socketpair/socketpair.go @@ -0,0 +1,49 @@ +package socketpair + +import ( + "golang.org/x/sys/unix" + "net" + "os" +) +type fileConn struct { + net.Conn // net.FileConn + f *os.File +} + +func (c fileConn) Close() error { + if err := c.Conn.Close(); err != nil { + return err + } + if err := c.f.Close(); err != nil { + return err + } + return nil +} + +func SocketPair() (a, b net.Conn, err error) { + // don't use net.Pipe, as it doesn't implement things like lingering, which our code relies on + sockpair, err := unix.Socketpair(unix.AF_UNIX, unix.SOCK_STREAM, 0) + if err != nil { + return nil, nil, err + } + toConn := func(fd int) (net.Conn, error) { + f := os.NewFile(uintptr(fd), "fileconn") + if f == nil { + panic(fd) + } + c, err := net.FileConn(f) + if err != nil { + f.Close() + return nil, err + } + return fileConn{Conn: c, f: f}, nil + } + if a, err = toConn(sockpair[0]); err != nil { // shadowing + return nil, nil, err + } + if b, err = toConn(sockpair[1]); err != nil { // shadowing + a.Close() + return nil, nil, err + } + return a, b, nil +} From be962998bae4911cb20ff3dc434d990eac44070d Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 11 Oct 2018 21:20:55 +0200 Subject: [PATCH 151/167] move serve and connecter into transports package --- daemon/job/active.go | 2 +- daemon/job/passive.go | 2 +- daemon/logging/build_logging.go | 2 +- daemon/{ => transport}/connecter/connect_local.go | 2 +- daemon/{ => transport}/connecter/connect_ssh.go | 0 daemon/{ => transport}/connecter/connect_tcp.go | 0 daemon/{ => transport}/connecter/connect_tls.go | 0 daemon/{ => transport}/connecter/connecter.go | 0 daemon/{ => transport}/serve/serve.go | 0 daemon/{ => transport}/serve/serve_local.go | 0 daemon/{ => transport}/serve/serve_stdinserver.go | 0 daemon/{ => transport}/serve/serve_tcp.go | 0 daemon/{ => transport}/serve/serve_tls.go | 0 13 files changed, 4 insertions(+), 4 deletions(-) rename daemon/{ => transport}/connecter/connect_local.go (93%) rename daemon/{ => transport}/connecter/connect_ssh.go (100%) rename daemon/{ => transport}/connecter/connect_tcp.go (100%) rename daemon/{ => transport}/connecter/connect_tls.go (100%) rename daemon/{ => transport}/connecter/connecter.go (100%) rename daemon/{ => transport}/serve/serve.go (100%) rename daemon/{ => transport}/serve/serve_local.go (100%) rename daemon/{ => transport}/serve/serve_stdinserver.go (100%) rename daemon/{ => transport}/serve/serve_tcp.go (100%) rename daemon/{ => transport}/serve/serve_tls.go (100%) diff --git a/daemon/job/active.go b/daemon/job/active.go index edf24e9..791530e 100644 --- a/daemon/job/active.go +++ b/daemon/job/active.go @@ -6,7 +6,7 @@ import ( "github.com/problame/go-streamrpc" "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/config" - "github.com/zrepl/zrepl/daemon/connecter" + "github.com/zrepl/zrepl/daemon/transport/connecter" "github.com/zrepl/zrepl/daemon/filters" "github.com/zrepl/zrepl/daemon/pruner" "github.com/zrepl/zrepl/endpoint" diff --git a/daemon/job/passive.go b/daemon/job/passive.go index ac843b3..99071a8 100644 --- a/daemon/job/passive.go +++ b/daemon/job/passive.go @@ -8,7 +8,7 @@ import ( "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon/filters" "github.com/zrepl/zrepl/daemon/logging" - "github.com/zrepl/zrepl/daemon/serve" + "github.com/zrepl/zrepl/daemon/transport/serve" "github.com/zrepl/zrepl/daemon/snapper" "github.com/zrepl/zrepl/endpoint" "github.com/zrepl/zrepl/zfs" diff --git a/daemon/logging/build_logging.go b/daemon/logging/build_logging.go index 754fa2f..fcc1fa4 100644 --- a/daemon/logging/build_logging.go +++ b/daemon/logging/build_logging.go @@ -15,7 +15,7 @@ import ( "github.com/zrepl/zrepl/tlsconf" "os" "github.com/zrepl/zrepl/daemon/snapper" - "github.com/zrepl/zrepl/daemon/serve" + "github.com/zrepl/zrepl/daemon/transport/serve" ) func OutletsFromConfig(in config.LoggingOutletEnumList) (*logger.Outlets, error) { diff --git a/daemon/connecter/connect_local.go b/daemon/transport/connecter/connect_local.go similarity index 93% rename from daemon/connecter/connect_local.go rename to daemon/transport/connecter/connect_local.go index c70b285..45c3d68 100644 --- a/daemon/connecter/connect_local.go +++ b/daemon/transport/connecter/connect_local.go @@ -4,7 +4,7 @@ import ( "context" "fmt" "github.com/zrepl/zrepl/config" - "github.com/zrepl/zrepl/daemon/serve" + "github.com/zrepl/zrepl/daemon/transport/serve" "net" ) diff --git a/daemon/connecter/connect_ssh.go b/daemon/transport/connecter/connect_ssh.go similarity index 100% rename from daemon/connecter/connect_ssh.go rename to daemon/transport/connecter/connect_ssh.go diff --git a/daemon/connecter/connect_tcp.go b/daemon/transport/connecter/connect_tcp.go similarity index 100% rename from daemon/connecter/connect_tcp.go rename to daemon/transport/connecter/connect_tcp.go diff --git a/daemon/connecter/connect_tls.go b/daemon/transport/connecter/connect_tls.go similarity index 100% rename from daemon/connecter/connect_tls.go rename to daemon/transport/connecter/connect_tls.go diff --git a/daemon/connecter/connecter.go b/daemon/transport/connecter/connecter.go similarity index 100% rename from daemon/connecter/connecter.go rename to daemon/transport/connecter/connecter.go diff --git a/daemon/serve/serve.go b/daemon/transport/serve/serve.go similarity index 100% rename from daemon/serve/serve.go rename to daemon/transport/serve/serve.go diff --git a/daemon/serve/serve_local.go b/daemon/transport/serve/serve_local.go similarity index 100% rename from daemon/serve/serve_local.go rename to daemon/transport/serve/serve_local.go diff --git a/daemon/serve/serve_stdinserver.go b/daemon/transport/serve/serve_stdinserver.go similarity index 100% rename from daemon/serve/serve_stdinserver.go rename to daemon/transport/serve/serve_stdinserver.go diff --git a/daemon/serve/serve_tcp.go b/daemon/transport/serve/serve_tcp.go similarity index 100% rename from daemon/serve/serve_tcp.go rename to daemon/transport/serve/serve_tcp.go diff --git a/daemon/serve/serve_tls.go b/daemon/transport/serve/serve_tls.go similarity index 100% rename from daemon/serve/serve_tls.go rename to daemon/transport/serve/serve_tls.go From 1fb59c953adeb30560b97eeb03acc471f2b3a042 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 11 Oct 2018 21:18:44 +0200 Subject: [PATCH 152/167] implement transport protocol handshake (even before streamrpc handshake) --- daemon/transport/connecter/connecter.go | 30 ++++++ daemon/transport/handshake.go | 136 ++++++++++++++++++++++++ daemon/transport/handshake_test.go | 119 +++++++++++++++++++++ daemon/transport/serve/serve.go | 40 +++++++ 4 files changed, 325 insertions(+) create mode 100644 daemon/transport/handshake.go create mode 100644 daemon/transport/handshake_test.go diff --git a/daemon/transport/connecter/connecter.go b/daemon/transport/connecter/connecter.go index 3c0f0de..fa772a7 100644 --- a/daemon/transport/connecter/connecter.go +++ b/daemon/transport/connecter/connecter.go @@ -1,12 +1,39 @@ package connecter import ( + "context" "fmt" "github.com/problame/go-streamrpc" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon/streamrpcconfig" + "github.com/zrepl/zrepl/daemon/transport" + "net" + "time" ) + +type HandshakeConnecter struct { + connecter streamrpc.Connecter +} + +func (c HandshakeConnecter) Connect(ctx context.Context) (net.Conn, error) { + conn, err := c.connecter.Connect(ctx) + if err != nil { + return nil, err + } + dl, ok := ctx.Deadline() + if !ok { + dl = time.Now().Add(10 * time.Second) // FIXME constant + } + if err := transport.DoHandshakeCurrentVersion(conn, dl); err != nil { + conn.Close() + return nil, err + } + return conn, nil +} + + + func FromConfig(g *config.Global, in config.ConnectEnum) (*ClientFactory, error) { var ( connecter streamrpc.Connecter @@ -41,6 +68,9 @@ func FromConfig(g *config.Global, in config.ConnectEnum) (*ClientFactory, error) if err := config.Validate(); err != nil { return nil, err } + + connecter = HandshakeConnecter{connecter} + return &ClientFactory{connecter: connecter, config: &config}, nil } diff --git a/daemon/transport/handshake.go b/daemon/transport/handshake.go new file mode 100644 index 0000000..ecfd495 --- /dev/null +++ b/daemon/transport/handshake.go @@ -0,0 +1,136 @@ +package transport + +import ( + "bytes" + "fmt" + "io" + "net" + "strings" + "time" + "unicode/utf8" +) + +type HandshakeMessage struct { + ProtocolVersion int + Extensions []string +} + +func (m *HandshakeMessage) Encode() ([]byte, error) { + if m.ProtocolVersion <= 0 || m.ProtocolVersion > 9999 { + return nil, fmt.Errorf("protocol version must be in [1, 9999]") + } + if len(m.Extensions) >= 9999 { + return nil, fmt.Errorf("protocol only supports [0, 9999] extensions") + } + // EXTENSIONS is a count of subsequent \n separated lines that contain protocol extensions + var extensions strings.Builder + for i, ext := range m.Extensions { + if strings.ContainsAny(ext, "\n") { + return nil, fmt.Errorf("Extension #%d contains forbidden newline character", i) + } + if !utf8.ValidString(ext) { + return nil, fmt.Errorf("Extension #%d is not valid UTF-8", i) + } + extensions.WriteString(ext) + extensions.WriteString("\n") + } + withoutLen := fmt.Sprintf("ZREPL_ZFS_REPLICATION PROTOVERSION=%04d EXTENSIONS=%04d\n%s", + m.ProtocolVersion, len(m.Extensions), extensions.String()) + withLen := fmt.Sprintf("%010d %s", len(withoutLen), withoutLen) + return []byte(withLen), nil +} + +func (m *HandshakeMessage) DecodeReader(r io.Reader, maxLen int) error { + var lenAndSpace [11]byte + if _, err := io.ReadFull(r, lenAndSpace[:]); err != nil { + return err + } + if !utf8.Valid(lenAndSpace[:]) { + return fmt.Errorf("invalid start of handshake message: not valid UTF-8") + } + var followLen int + n, err := fmt.Sscanf(string(lenAndSpace[:]), "%010d ", &followLen) + if n != 1 || err != nil { + return fmt.Errorf("could not parse handshake message length") + } + if followLen > maxLen { + return fmt.Errorf("handshake message length exceeds max length (%d vs %d)", + followLen, maxLen) + } + + var buf bytes.Buffer + _, err = io.Copy(&buf, io.LimitReader(r, int64(followLen))) + if err != nil { + return err + } + + var ( + protoVersion, extensionCount int + ) + n, err = fmt.Fscanf(&buf, "ZREPL_ZFS_REPLICATION PROTOVERSION=%04d EXTENSIONS=%4d\n", + &protoVersion, &extensionCount) + if n != 2 || err != nil { + return fmt.Errorf("could not parse handshake message: %s", err) + } + if protoVersion < 1 { + return fmt.Errorf("invalid protocol version %q", protoVersion) + } + m.ProtocolVersion = protoVersion + + if extensionCount < 0 { + return fmt.Errorf("invalid extension count %q", extensionCount) + } + if extensionCount == 0 { + if buf.Len() != 0 { + return fmt.Errorf("unexpected data trailing after header") + } + m.Extensions = nil + return nil + } + s := buf.String() + if strings.Count(s, "\n") != extensionCount { + return fmt.Errorf("inconsistent extension count: found %d, header says %d", len(m.Extensions), extensionCount) + } + exts := strings.Split(s, "\n") + if exts[len(exts)-1] != "" { + return fmt.Errorf("unexpected data trailing after last extension newline") + } + m.Extensions = exts[0:len(exts)-1] + + return nil +} + +func DoHandshakeCurrentVersion(conn net.Conn, deadline time.Time) error { + // current protocol version is hardcoded here + return DoHandshakeVersion(conn, deadline, 1) +} + +func DoHandshakeVersion(conn net.Conn, deadline time.Time, version int) error { + ours := HandshakeMessage{ + ProtocolVersion: version, + Extensions: nil, + } + hsb, err := ours.Encode() + if err != nil { + return fmt.Errorf("could not encode protocol banner: %s", err) + } + + conn.SetDeadline(deadline) + _, err = io.Copy(conn, bytes.NewBuffer(hsb)) + if err != nil { + return fmt.Errorf("could not send protocol banner: %s", err) + } + + theirs := HandshakeMessage{} + if err := theirs.DecodeReader(conn, 16 * 4096); err != nil { // FIXME constant + return fmt.Errorf("could not decode protocol banner: %s", err) + } + + if theirs.ProtocolVersion != ours.ProtocolVersion { + return fmt.Errorf("protocol versions do not match: ours is %d, theirs is %d", + ours.ProtocolVersion, theirs.ProtocolVersion) + } + // ignore extensions, we don't use them + + return nil +} diff --git a/daemon/transport/handshake_test.go b/daemon/transport/handshake_test.go new file mode 100644 index 0000000..d1c72b4 --- /dev/null +++ b/daemon/transport/handshake_test.go @@ -0,0 +1,119 @@ +package transport + +import ( + "bytes" + "fmt" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/zrepl/zrepl/util/socketpair" + "io" + "strings" + "testing" + "time" +) + +func TestHandshakeMessage_Encode(t *testing.T) { + + msg := HandshakeMessage{ + ProtocolVersion: 2342, + } + + encB, err := msg.Encode() + require.NoError(t, err) + enc := string(encB) + t.Logf("enc: %s", enc) + + + + assert.False(t, strings.ContainsAny(enc[0:10], " ")) + assert.True(t, enc[10] == ' ') + + var ( + headerlen, protoversion, extensionCount int + ) + n, err := fmt.Sscanf(enc, "%010d ZREPL_ZFS_REPLICATION PROTOVERSION=%04d EXTENSIONS=%04d\n", + &headerlen, &protoversion, &extensionCount) + if n != 3 || (err != nil && err != io.EOF) { + t.Fatalf("%v %v", n, err) + } + + assert.Equal(t, 2342, protoversion) + assert.Equal(t, 0, extensionCount) + assert.Equal(t, len(enc)-11, headerlen) + +} + +func TestHandshakeMessage_Encode_InvalidProtocolVersion(t *testing.T) { + + for _, pv := range []int{-1, 0, 10000, 10001} { + t.Logf("testing invalid protocol version = %v", pv) + msg := HandshakeMessage{ + ProtocolVersion: pv, + } + b, err := msg.Encode() + assert.Error(t, err) + assert.Nil(t, b) + } + +} + +func TestHandshakeMessage_DecodeReader(t *testing.T) { + + in := HandshakeMessage{ + 2342, + []string{"foo", "bar 2342"}, + } + + enc, err := in.Encode() + require.NoError(t, err) + + out := HandshakeMessage{} + err = out.DecodeReader(bytes.NewReader([]byte(enc)), 4 * 4096) + assert.NoError(t, err) + assert.Equal(t, 2342, out.ProtocolVersion) + assert.Equal(t, 2, len(out.Extensions)) + assert.Equal(t, "foo", out.Extensions[0]) + assert.Equal(t, "bar 2342", out.Extensions[1]) + +} + +func TestDoHandshakeVersion_ErrorOnDifferentVersions(t *testing.T) { + srv, client, err := socketpair.SocketPair() + if err != nil { + t.Fatal(err) + } + defer srv.Close() + defer client.Close() + + srvErrCh := make(chan error) + go func() { + srvErrCh <- DoHandshakeVersion(srv, time.Now().Add(2*time.Second), 1) + }() + err = DoHandshakeVersion(client, time.Now().Add(2*time.Second), 2) + t.Log(err) + assert.Error(t, err) + assert.True(t, strings.Contains(err.Error(), "version")) + + srvErr := <-srvErrCh + t.Log(srvErr) + assert.Error(t, srvErr) + assert.True(t, strings.Contains(srvErr.Error(), "version")) +} + +func TestDoHandshakeCurrentVersion(t *testing.T) { + srv, client, err := socketpair.SocketPair() + if err != nil { + t.Fatal(err) + } + defer srv.Close() + defer client.Close() + + srvErrCh := make(chan error) + go func() { + srvErrCh <- DoHandshakeVersion(srv, time.Now().Add(2*time.Second), 1) + }() + err = DoHandshakeVersion(client, time.Now().Add(2*time.Second), 1) + assert.Nil(t, err) + assert.Nil(t, <-srvErrCh) + +} diff --git a/daemon/transport/serve/serve.go b/daemon/transport/serve/serve.go index 15e6ba3..c1b3bb1 100644 --- a/daemon/transport/serve/serve.go +++ b/daemon/transport/serve/serve.go @@ -3,12 +3,14 @@ package serve import ( "github.com/pkg/errors" "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon/transport" "net" "github.com/zrepl/zrepl/daemon/streamrpcconfig" "github.com/problame/go-streamrpc" "context" "github.com/zrepl/zrepl/logger" "github.com/zrepl/zrepl/zfs" + "time" ) type contextKey int @@ -71,6 +73,42 @@ type ListenerFactory interface { Listen() (AuthenticatedListener, error) } +type HandshakeListenerFactory struct { + lf ListenerFactory +} + +func (lf HandshakeListenerFactory) Listen() (AuthenticatedListener, error) { + l, err := lf.lf.Listen() + if err != nil { + return nil, err + } + return HandshakeListener{l}, nil +} + +type HandshakeListener struct { + l AuthenticatedListener +} + +func (l HandshakeListener) Addr() (net.Addr) { return l.l.Addr() } + +func (l HandshakeListener) Close() error { return l.l.Close() } + +func (l HandshakeListener) Accept(ctx context.Context) (AuthenticatedConn, error) { + conn, err := l.l.Accept(ctx) + if err != nil { + return nil, err + } + dl, ok := ctx.Deadline() + if !ok { + dl = time.Now().Add(10*time.Second) // FIXME constant + } + if err := transport.DoHandshakeCurrentVersion(conn, dl); err != nil { + conn.Close() + return nil, err + } + return conn, nil +} + func FromConfig(g *config.Global, in config.ServeEnum) (lf ListenerFactory, conf *streamrpc.ConnConfig, _ error) { var ( @@ -100,6 +138,8 @@ func FromConfig(g *config.Global, in config.ServeEnum) (lf ListenerFactory, conf return nil, nil, rpcErr } + lf = HandshakeListenerFactory{lf} + return lf, conf, nil } From f9d24d15ed63b401d30db2379ce27ac31718463b Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 12 Oct 2018 12:44:40 +0200 Subject: [PATCH 153/167] move wakup mechanism into separate package --- daemon/daemon.go | 9 +++++---- daemon/job/active.go | 3 ++- daemon/job/job.go | 27 --------------------------- daemon/job/wakeup/wakeup.go | 35 +++++++++++++++++++++++++++++++++++ 4 files changed, 42 insertions(+), 32 deletions(-) create mode 100644 daemon/job/wakeup/wakeup.go diff --git a/daemon/daemon.go b/daemon/daemon.go index 8ada3c9..41cf667 100644 --- a/daemon/daemon.go +++ b/daemon/daemon.go @@ -7,6 +7,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon/job" + "github.com/zrepl/zrepl/daemon/job/wakeup" "github.com/zrepl/zrepl/daemon/logging" "github.com/zrepl/zrepl/logger" "github.com/zrepl/zrepl/version" @@ -100,13 +101,13 @@ type jobs struct { // m protects all fields below it m sync.RWMutex - wakeups map[string]job.WakeupFunc // by Job.Name + wakeups map[string]wakeup.Func // by Job.Name jobs map[string]job.Job } func newJobs() *jobs { return &jobs{ - wakeups: make(map[string]job.WakeupFunc), + wakeups: make(map[string]wakeup.Func), jobs: make(map[string]job.Job), } } @@ -193,8 +194,8 @@ func (s *jobs) start(ctx context.Context, j job.Job, internal bool) { s.jobs[jobName] = j ctx = job.WithLogger(ctx, jobLog) - ctx, wakeupChan := job.WithWakeup(ctx) - s.wakeups[jobName] = wakeupChan + ctx, wakeup := wakeup.Context(ctx) + s.wakeups[jobName] = wakeup s.wg.Add(1) go func() { diff --git a/daemon/job/active.go b/daemon/job/active.go index 791530e..b50216d 100644 --- a/daemon/job/active.go +++ b/daemon/job/active.go @@ -6,6 +6,7 @@ import ( "github.com/problame/go-streamrpc" "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon/job/wakeup" "github.com/zrepl/zrepl/daemon/transport/connecter" "github.com/zrepl/zrepl/daemon/filters" "github.com/zrepl/zrepl/daemon/pruner" @@ -233,7 +234,7 @@ outer: log.WithError(ctx.Err()).Info("context") break outer - case <-WaitWakeup(ctx): + case <-wakeup.Wait(ctx): case <-periodicDone: } invocationCount++ diff --git a/daemon/job/job.go b/daemon/job/job.go index 85334a5..5b3a684 100644 --- a/daemon/job/job.go +++ b/daemon/job/job.go @@ -3,7 +3,6 @@ package job import ( "context" "encoding/json" - "errors" "fmt" "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/logger" @@ -15,7 +14,6 @@ type contextKey int const ( contextKeyLog contextKey = iota - contextKeyWakeup ) func GetLogger(ctx context.Context) Logger { @@ -29,22 +27,6 @@ func WithLogger(ctx context.Context, l Logger) context.Context { return context.WithValue(ctx, contextKeyLog, l) } -type WakeupFunc func() error - -var AlreadyWokenUp = errors.New("already woken up") - -func WithWakeup(ctx context.Context) (context.Context, WakeupFunc) { - wc := make(chan struct{}) - wuf := func() error { - select { - case wc <- struct{}{}: - return nil - default: - return AlreadyWokenUp - } - } - return context.WithValue(ctx, contextKeyWakeup, wc), wuf -} type Job interface { Name() string @@ -119,12 +101,3 @@ func (s *Status) UnmarshalJSON(in []byte) (err error) { } return err } - -func WaitWakeup(ctx context.Context) <-chan struct{} { - wc, ok := ctx.Value(contextKeyWakeup).(chan struct{}) - if !ok { - wc = make(chan struct{}) - } - return wc -} - diff --git a/daemon/job/wakeup/wakeup.go b/daemon/job/wakeup/wakeup.go new file mode 100644 index 0000000..a099b53 --- /dev/null +++ b/daemon/job/wakeup/wakeup.go @@ -0,0 +1,35 @@ +package wakeup + +import ( + "context" + "errors" +) + +type contextKey int + +const contextKeyWakeup contextKey = iota + +func Wait(ctx context.Context) <-chan struct{} { + wc, ok := ctx.Value(contextKeyWakeup).(chan struct{}) + if !ok { + wc = make(chan struct{}) + } + return wc +} + +type Func func() error + +var AlreadyWokenUp = errors.New("already woken up") + +func Context(ctx context.Context) (context.Context, Func) { + wc := make(chan struct{}) + wuf := func() error { + select { + case wc <- struct{}{}: + return nil + default: + return AlreadyWokenUp + } + } + return context.WithValue(ctx, contextKeyWakeup, wc), wuf +} From d17ecc3b5ce59aa38d89ac786b36b31281f5ac88 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 12 Oct 2018 12:45:37 +0200 Subject: [PATCH 154/167] replication/fsrep: report Pending[0] problem as fsrep problem in RetryWait state --- replication/fsrep/fsfsm.go | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index a8c836e..2397309 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -312,6 +312,13 @@ func (fsr *Replication) Report() *Report { for i := range fsr.pending { rep.Pending[i] = fsr.pending[i].Report() } + + if fsr.state&RetryWait != 0 { + if len(rep.Pending) != 0 { // should always be true for RetryWait == true? + rep.Problem = rep.Pending[0].Problem + } + } + return &rep } @@ -509,10 +516,15 @@ func (s *ReplicationStep) Report() *StepReport { if s.byteCounter != nil { bytes = s.byteCounter.Bytes() } + problem := "" + if s.err != nil { + problem = s.err.Error() + } rep := StepReport{ From: from, To: s.to.RelName(), Status: s.state, + Problem: problem, Bytes: bytes, ExpectedBytes: s.expectedSize, } From cb83a26c90bb1c42d04f8b0282042d58f862a934 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 12 Oct 2018 13:12:28 +0200 Subject: [PATCH 155/167] replication: wakeup + retry handling: make wakeups work in retry wait states - handle wakeups in Planning state - fsrep.Replication yields immediately in RetryWait - once the queue only contains fsrep.Replication in retryWait: transition replication.Replication into WorkingWait state - handle wakeups in WorkingWait state, too --- replication/fsrep/fsfsm.go | 22 ++++++------- replication/internal/queue/queue.go | 13 +++----- replication/mainfsm.go | 48 ++++++++++++++++++----------- 3 files changed, 46 insertions(+), 37 deletions(-) diff --git a/replication/fsrep/fsfsm.go b/replication/fsrep/fsfsm.go index 2397309..0297dc9 100644 --- a/replication/fsrep/fsfsm.go +++ b/replication/fsrep/fsfsm.go @@ -191,7 +191,7 @@ type ReplicationStep struct { expectedSize int64 // 0 means no size estimate present / possible } -func (f *Replication) TakeStep(ctx context.Context, sender Sender, receiver Receiver) (post State, nextStepDate time.Time) { +func (f *Replication) TakeStep(ctx context.Context, sender Sender, receiver Receiver) (post State, nextStepDate, retryWaitUntil time.Time) { var u updater = func(fu func(*Replication)) State { f.lock.Lock() @@ -213,6 +213,7 @@ func (f *Replication) TakeStep(ctx context.Context, sender Sender, receiver Rece return } nextStepDate = f.pending[0].to.SnapshotTime() + retryWaitUntil = f.retryWaitUntil }) getLogger(ctx). @@ -221,7 +222,13 @@ func (f *Replication) TakeStep(ctx context.Context, sender Sender, receiver Rece WithField("duration", delta). Debug("fsr step taken") - return post, nextStepDate + return post, nextStepDate, retryWaitUntil +} + +func (f *Replication) RetryWaitUntil() time.Time { + f.lock.Lock() + defer f.lock.Unlock() + return f.retryWaitUntil } type updater func(func(fsr *Replication)) State @@ -275,15 +282,8 @@ func stateRetryWait(ctx context.Context, sender Sender, receiver Receiver, u upd u(func(f *Replication) { sleepUntil = f.retryWaitUntil }) - t := time.NewTimer(sleepUntil.Sub(time.Now())) - defer t.Stop() - select { - case <-ctx.Done(): - return u(func(f *Replication) { - f.state = PermanentError - f.err = ctx.Err() - }).fsrsf() - case <-t.C: + if time.Now().Before(sleepUntil) { + return u(nil).fsrsf() } return u(func(f *Replication) { f.state = Ready diff --git a/replication/internal/queue/queue.go b/replication/internal/queue/queue.go index a3d7c0c..323460b 100644 --- a/replication/internal/queue/queue.go +++ b/replication/internal/queue/queue.go @@ -8,11 +8,12 @@ import ( ) type replicationQueueItem struct { - retriesSinceLastError int // duplicates fsr.state to avoid accessing and locking fsr state State // duplicates fsr.current.nextStepDate to avoid accessing & locking fsr nextStepDate time.Time + // duplicates fsr.retryWaitUntil to avoid accessing & locking fsr + retryWaitUntil time.Time fsr *Replication } @@ -42,7 +43,7 @@ var lessmap = map[State]lessmapEntry{ RetryWait: { prio: 1, less: func(a, b *replicationQueueItem) bool { - return a.retriesSinceLastError < b.retriesSinceLastError + return a.retryWaitUntil.Before(b.retryWaitUntil) }, }, } @@ -113,12 +114,8 @@ func (h ReplicationQueueItemHandle) GetFSReplication() *Replication { return h.i.fsr } -func (h ReplicationQueueItemHandle) Update(newState State, nextStepDate time.Time) { +func (h ReplicationQueueItemHandle) Update(newState State, nextStepDate, retryWaitUntil time.Time) { h.i.state = newState h.i.nextStepDate = nextStepDate - if h.i.state&Ready != 0 { - h.i.retriesSinceLastError = 0 - } else if h.i.state&RetryWait != 0 { - h.i.retriesSinceLastError++ - } + h.i.retryWaitUntil = retryWaitUntil } diff --git a/replication/mainfsm.go b/replication/mainfsm.go index d876491..15137b0 100644 --- a/replication/mainfsm.go +++ b/replication/mainfsm.go @@ -7,6 +7,7 @@ import ( "errors" "fmt" "github.com/prometheus/client_golang/prometheus" + "github.com/zrepl/zrepl/daemon/job/wakeup" "math/bits" "sync" "time" @@ -186,6 +187,8 @@ func resolveConflict(conflict error) (path []*pdu.FilesystemVersion, msg string) return nil, "no automated way to handle conflict type" } +var PlanningRetryInterval = 10 * time.Second // FIXME make constant onfigurable + func statePlanning(ctx context.Context, sender Sender, receiver Receiver, u updater) state { log := getLogger(ctx) @@ -193,7 +196,9 @@ func statePlanning(ctx context.Context, sender Sender, receiver Receiver, u upda log.Info("start planning") handlePlanningError := func(err error) state { + // FIXME classify error as temporary or permanent / max retry counter return u(func(r *Replication) { + r.sleepUntil = time.Now().Add(PlanningRetryInterval) r.planningError = err r.state = PlanningError }).rsf() @@ -301,15 +306,12 @@ func statePlanning(ctx context.Context, sender Sender, receiver Receiver, u upda }).rsf() } -var RetrySleepDuration = 10 * time.Second // FIXME make constant onfigurable - func statePlanningError(ctx context.Context, sender Sender, receiver Receiver, u updater) state { - - sleepUntil := time.Now().Add(RetrySleepDuration) + var sleepUntil time.Time u(func(r *Replication) { - r.sleepUntil = sleepUntil + sleepUntil = r.sleepUntil }) - t := time.NewTimer(RetrySleepDuration) + t := time.NewTimer(sleepUntil.Sub(time.Now())) getLogger(ctx).WithField("until", sleepUntil).Info("retry wait after planning error") defer t.Stop() select { @@ -319,10 +321,11 @@ func statePlanningError(ctx context.Context, sender Sender, receiver Receiver, u r.contextError = ctx.Err() }).rsf() case <-t.C: - return u(func(r *Replication) { - r.state = Planning - }).rsf() + case <-wakeup.Wait(ctx): } + return u(func(r *Replication) { + r.state = Planning + }).rsf() } func stateWorking(ctx context.Context, sender Sender, receiver Receiver, u updater) state { @@ -342,21 +345,28 @@ func stateWorking(ctx context.Context, sender Sender, receiver Receiver, u updat return rsfNext } - state, nextStepDate := active.GetFSReplication().TakeStep(ctx, sender, receiver) + retryWaitUntil := active.GetFSReplication().RetryWaitUntil() + if retryWaitUntil.After(time.Now()) { + return u(func(r *Replication) { + r.sleepUntil = retryWaitUntil + r.state = WorkingWait + }).rsf() + } + state, nextStepDate, retryWaitUntil := active.GetFSReplication().TakeStep(ctx, sender, receiver) return u(func(r *Replication) { - active.Update(state, nextStepDate) + active.Update(state, nextStepDate, retryWaitUntil) r.active = nil }).rsf() } func stateWorkingWait(ctx context.Context, sender Sender, receiver Receiver, u updater) state { - sleepUntil := time.Now().Add(RetrySleepDuration) + var sleepUntil time.Time u(func(r *Replication) { - r.sleepUntil = sleepUntil + sleepUntil = r.sleepUntil }) - t := time.NewTimer(RetrySleepDuration) - getLogger(ctx).WithField("until", sleepUntil).Info("retry wait after send/recv error") + t := time.NewTimer(PlanningRetryInterval) + getLogger(ctx).WithField("until", sleepUntil).Info("retry wait because no filesystems are ready") defer t.Stop() select { case <-ctx.Done(): @@ -364,11 +374,13 @@ func stateWorkingWait(ctx context.Context, sender Sender, receiver Receiver, u u r.state = ContextDone r.contextError = ctx.Err() }).rsf() + case <-t.C: - return u(func(r *Replication) { - r.state = Working - }).rsf() + case <-wakeup.Wait(ctx): } + return u(func(r *Replication) { + r.state = Working + }).rsf() } // Report provides a summary of the progress of the Replication, From 9bb7b19c930287ecf29f730acb7c10c75c1e7ec9 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 12 Oct 2018 15:29:07 +0200 Subject: [PATCH 156/167] pruner: handle replication cursor being older than any snapshot correctly --- daemon/pruner/pruner.go | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/daemon/pruner/pruner.go b/daemon/pruner/pruner.go index ebdfe1c..c2f3382 100644 --- a/daemon/pruner/pruner.go +++ b/daemon/pruner/pruner.go @@ -402,7 +402,17 @@ fsloop: sort.Slice(tfsvs, func(i, j int) bool { return tfsvs[i].CreateTXG < tfsvs[j].CreateTXG }) - preCursor := true + + haveCursorSnapshot := false + for _, tfsv := range tfsvs { + if tfsv.Type != pdu.FilesystemVersion_Snapshot { + continue + } + if tfsv.Guid == rc.GetGuid() { + haveCursorSnapshot = true + } + } + preCursor := haveCursorSnapshot for _, tfsv := range tfsvs { if tfsv.Type != pdu.FilesystemVersion_Snapshot { continue @@ -413,6 +423,7 @@ fsloop: l.WithError(pfs.err).Error("") continue fsloop } + // note that we cannot use CreateTXG because target and receiver could be on different pools atCursor := tfsv.Guid == rc.GetGuid() preCursor = preCursor && !atCursor pfs.snaps = append(pfs.snaps, snapshot{ From 025fbda984f512e93c85e058f21cd6bccfec5b43 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 12 Oct 2018 16:00:37 +0200 Subject: [PATCH 157/167] client/status: only show progress bar in non-planning states --- client/status.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/client/status.go b/client/status.go index df1b793..a8417df 100644 --- a/client/status.go +++ b/client/status.go @@ -261,7 +261,9 @@ func (t *tui) renderReplicationReport(rep *replication.Report) { if rep.SleepUntil.After(time.Now()) { t.printf("Sleeping until %s (%s left)\n", rep.SleepUntil, rep.SleepUntil.Sub(time.Now())) } - { // Progress: [---------------] + if rep.Status != replication.Planning.String() && + rep.Status != replication.PlanningError.String() { + // Progress: [---------------] sumUpFSRep := func(rep *fsrep.Report) (transferred, total int64) { for _, s := range rep.Pending { transferred += s.Bytes From 89e0103abdc9c321f6e44e846374159f05179581 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 12 Oct 2018 20:50:30 +0200 Subject: [PATCH 158/167] move wakeup subcommand into signal subcommand and add reset subcommand --- client/{wakeup.go => signal.go} | 12 ++++++----- daemon/control.go | 16 ++++++++++++--- daemon/daemon.go | 16 +++++++++++++++ daemon/job/active.go | 16 +++++++++++++++ daemon/job/reset/reset.go | 35 +++++++++++++++++++++++++++++++++ main.go | 10 +++++----- 6 files changed, 92 insertions(+), 13 deletions(-) rename client/{wakeup.go => signal.go} (52%) create mode 100644 daemon/job/reset/reset.go diff --git a/client/wakeup.go b/client/signal.go similarity index 52% rename from client/wakeup.go rename to client/signal.go index d6a224f..273ea18 100644 --- a/client/wakeup.go +++ b/client/signal.go @@ -6,9 +6,9 @@ import ( "github.com/zrepl/zrepl/daemon" ) -func RunWakeup(config *config.Config, args []string) error { - if len(args) != 1 { - return errors.Errorf("Expected 1 argument: job") +func RunSignal(config *config.Config, args []string) error { + if len(args) != 2 { + return errors.Errorf("Expected 2 arguments: [wakeup|reset] JOB") } httpc, err := controlHttpClient(config.Global.Control.SockPath) @@ -16,11 +16,13 @@ func RunWakeup(config *config.Config, args []string) error { return err } - err = jsonRequestResponse(httpc, daemon.ControlJobEndpointWakeup, + err = jsonRequestResponse(httpc, daemon.ControlJobEndpointSignal, struct { Name string + Op string }{ - Name: args[0], + Name: args[1], + Op: args[0], }, struct{}{}, ) diff --git a/daemon/control.go b/daemon/control.go index 0be8a0d..9163dbc 100644 --- a/daemon/control.go +++ b/daemon/control.go @@ -4,6 +4,7 @@ import ( "bytes" "context" "encoding/json" + "fmt" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/daemon/job" @@ -65,7 +66,7 @@ const ( ControlJobEndpointPProf string = "/debug/pprof" ControlJobEndpointVersion string = "/version" ControlJobEndpointStatus string = "/status" - ControlJobEndpointWakeup string = "/wakeup" + ControlJobEndpointSignal string = "/signal" ) func (j *controlJob) Run(ctx context.Context) { @@ -104,17 +105,26 @@ func (j *controlJob) Run(ctx context.Context) { return s, nil }}}) - mux.Handle(ControlJobEndpointWakeup, + mux.Handle(ControlJobEndpointSignal, requestLogger{log: log, handler: jsonRequestResponder{func(decoder jsonDecoder) (interface{}, error) { type reqT struct { Name string + Op string } var req reqT if decoder(&req) != nil { return nil, errors.Errorf("decode failed") } - err := j.jobs.wakeup(req.Name) + var err error + switch req.Op { + case "wakeup": + err = j.jobs.wakeup(req.Name) + case "reset": + err = j.jobs.reset(req.Name) + default: + err = fmt.Errorf("operation %q is invalid", req.Op) + } return struct{}{}, err }}}) diff --git a/daemon/daemon.go b/daemon/daemon.go index 41cf667..9f0e185 100644 --- a/daemon/daemon.go +++ b/daemon/daemon.go @@ -7,6 +7,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon/job" + "github.com/zrepl/zrepl/daemon/job/reset" "github.com/zrepl/zrepl/daemon/job/wakeup" "github.com/zrepl/zrepl/daemon/logging" "github.com/zrepl/zrepl/logger" @@ -102,12 +103,14 @@ type jobs struct { // m protects all fields below it m sync.RWMutex wakeups map[string]wakeup.Func // by Job.Name + resets map[string]reset.Func // by Job.Name jobs map[string]job.Job } func newJobs() *jobs { return &jobs{ wakeups: make(map[string]wakeup.Func), + resets: make(map[string]reset.Func), jobs: make(map[string]job.Job), } } @@ -163,6 +166,17 @@ func (s *jobs) wakeup(job string) error { return wu() } +func (s *jobs) reset(job string) error { + s.m.RLock() + defer s.m.RUnlock() + + wu, ok := s.resets[job] + if !ok { + return errors.Errorf("Job %s does not exist", job) + } + return wu() +} + const ( jobNamePrometheus = "_prometheus" jobNameControl = "_control" @@ -195,7 +209,9 @@ func (s *jobs) start(ctx context.Context, j job.Job, internal bool) { s.jobs[jobName] = j ctx = job.WithLogger(ctx, jobLog) ctx, wakeup := wakeup.Context(ctx) + ctx, resetFunc := reset.Context(ctx) s.wakeups[jobName] = wakeup + s.resets[jobName] = resetFunc s.wg.Add(1) go func() { diff --git a/daemon/job/active.go b/daemon/job/active.go index b50216d..0d78abb 100644 --- a/daemon/job/active.go +++ b/daemon/job/active.go @@ -6,6 +6,7 @@ import ( "github.com/problame/go-streamrpc" "github.com/prometheus/client_golang/prometheus" "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon/job/reset" "github.com/zrepl/zrepl/daemon/job/wakeup" "github.com/zrepl/zrepl/daemon/transport/connecter" "github.com/zrepl/zrepl/daemon/filters" @@ -248,6 +249,21 @@ func (j *ActiveSide) do(ctx context.Context) { log := GetLogger(ctx) ctx = logging.WithSubsystemLoggers(ctx, log) + // allow cancellation of an invocation (this function) + ctx, cancelThisRun := context.WithCancel(ctx) + defer cancelThisRun() + runDone := make(chan struct{}) + defer close(runDone) + go func() { + select { + case <-runDone: + case <-reset.Wait(ctx): + log.Info("reset received, cancelling current invocation") + cancelThisRun() + case <-ctx.Done(): + } + }() + client, err := j.clientFactory.NewClient() if err != nil { log.WithError(err).Error("factory cannot instantiate streamrpc client") diff --git a/daemon/job/reset/reset.go b/daemon/job/reset/reset.go new file mode 100644 index 0000000..b7322ed --- /dev/null +++ b/daemon/job/reset/reset.go @@ -0,0 +1,35 @@ +package reset + +import ( + "context" + "errors" +) + +type contextKey int + +const contextKeyReset contextKey = iota + +func Wait(ctx context.Context) <-chan struct{} { + wc, ok := ctx.Value(contextKeyReset).(chan struct{}) + if !ok { + wc = make(chan struct{}) + } + return wc +} + +type Func func() error + +var AlreadyReset = errors.New("already reset") + +func Context(ctx context.Context) (context.Context, Func) { + wc := make(chan struct{}) + wuf := func() error { + select { + case wc <- struct{}{}: + return nil + default: + return AlreadyReset + } + } + return context.WithValue(ctx, contextKeyReset, wc), wuf +} diff --git a/main.go b/main.go index d23aae8..7c112a6 100644 --- a/main.go +++ b/main.go @@ -29,15 +29,15 @@ var daemonCmd = &cobra.Command{ }, } -var wakeupCmd = &cobra.Command{ - Use: "wakeup JOB", - Short: "trigger replication and subsequent pruning for a job", +var signalCmd = &cobra.Command{ + Use: "signal [wakeup|reset] JOB", + Short: "wake up a job from wait state or abort its current invocation", RunE: func(cmd *cobra.Command, args []string) error { conf, err := config.ParseConfig(rootArgs.configFile) if err != nil { return err } - return client.RunWakeup(conf, args) + return client.RunSignal(conf, args) }, } @@ -153,7 +153,7 @@ func init() { //cobra.OnInitialize(initConfig) rootCmd.PersistentFlags().StringVar(&rootArgs.configFile, "config", "", "config file path") rootCmd.AddCommand(daemonCmd) - rootCmd.AddCommand(wakeupCmd) + rootCmd.AddCommand(signalCmd) statusCmd.Flags().BoolVar(&statusCmdFlags.Raw, "raw", false, "dump raw status description from zrepl daemon") rootCmd.AddCommand(statusCmd) rootCmd.AddCommand(stdinserverCmd) From af3d96dab84188d956a2c74c47d4c74d3655e833 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 12 Oct 2018 22:10:49 +0200 Subject: [PATCH 159/167] use enumer generate tool for state strings --- client/status.go | 20 +++++++-- daemon/pruner/pruner.go | 2 +- daemon/pruner/state_enumer.go | 76 +++++++++++++++++++++++++++++++++++ daemon/pruner/state_string.go | 35 ---------------- lazy.sh | 3 +- replication/mainfsm.go | 2 +- replication/state_enumer.go | 76 +++++++++++++++++++++++++++++++++++ replication/state_string.go | 35 ---------------- 8 files changed, 172 insertions(+), 77 deletions(-) create mode 100644 daemon/pruner/state_enumer.go delete mode 100644 daemon/pruner/state_string.go create mode 100644 replication/state_enumer.go delete mode 100644 replication/state_string.go diff --git a/client/status.go b/client/status.go index a8417df..d56773e 100644 --- a/client/status.go +++ b/client/status.go @@ -252,7 +252,13 @@ func (t *tui) renderReplicationReport(rep *replication.Report) { return all[i].Filesystem < all[j].Filesystem }) - t.printf("Status: %s", rep.Status) + state, err := replication.StateString(rep.Status) + if err != nil { + t.printf("Status: %q (parse error: %q)\n", rep.Status, err) + return + } + + t.printf("Status: %s", state) t.newline() if rep.Problem != "" { t.printf("Problem: %s", rep.Problem) @@ -261,8 +267,8 @@ func (t *tui) renderReplicationReport(rep *replication.Report) { if rep.SleepUntil.After(time.Now()) { t.printf("Sleeping until %s (%s left)\n", rep.SleepUntil, rep.SleepUntil.Sub(time.Now())) } - if rep.Status != replication.Planning.String() && - rep.Status != replication.PlanningError.String() { + + if state != replication.Planning && state != replication.PlanningError { // Progress: [---------------] sumUpFSRep := func(rep *fsrep.Report) (transferred, total int64) { for _, s := range rep.Pending { @@ -304,7 +310,13 @@ func (t *tui) renderPrunerReport(r *pruner.Report) { return } - t.printf("Status: %s", r.State) + state, err := pruner.StateString(r.State) + if err != nil { + t.printf("Status: %q (parse error: %q)\n", r.State, err) + return + } + + t.printf("Status: %s", state) t.newline() if r.Error != "" { diff --git a/daemon/pruner/pruner.go b/daemon/pruner/pruner.go index c2f3382..cddd687 100644 --- a/daemon/pruner/pruner.go +++ b/daemon/pruner/pruner.go @@ -151,7 +151,7 @@ func (f *PrunerFactory) BuildReceiverPruner(ctx context.Context, target Target, return p } -//go:generate stringer -type=State +//go:generate enumer -type=State type State int const ( diff --git a/daemon/pruner/state_enumer.go b/daemon/pruner/state_enumer.go new file mode 100644 index 0000000..8c396ab --- /dev/null +++ b/daemon/pruner/state_enumer.go @@ -0,0 +1,76 @@ +// Code generated by "enumer -type=State"; DO NOT EDIT. + +package pruner + +import ( + "fmt" +) + +const ( + _StateName_0 = "PlanPlanWait" + _StateName_1 = "Exec" + _StateName_2 = "ExecWait" + _StateName_3 = "ErrPerm" + _StateName_4 = "Done" +) + +var ( + _StateIndex_0 = [...]uint8{0, 4, 12} + _StateIndex_1 = [...]uint8{0, 4} + _StateIndex_2 = [...]uint8{0, 8} + _StateIndex_3 = [...]uint8{0, 7} + _StateIndex_4 = [...]uint8{0, 4} +) + +func (i State) String() string { + switch { + case 1 <= i && i <= 2: + i -= 1 + return _StateName_0[_StateIndex_0[i]:_StateIndex_0[i+1]] + case i == 4: + return _StateName_1 + case i == 8: + return _StateName_2 + case i == 16: + return _StateName_3 + case i == 32: + return _StateName_4 + default: + return fmt.Sprintf("State(%d)", i) + } +} + +var _StateValues = []State{1, 2, 4, 8, 16, 32} + +var _StateNameToValueMap = map[string]State{ + _StateName_0[0:4]: 1, + _StateName_0[4:12]: 2, + _StateName_1[0:4]: 4, + _StateName_2[0:8]: 8, + _StateName_3[0:7]: 16, + _StateName_4[0:4]: 32, +} + +// StateString retrieves an enum value from the enum constants string name. +// Throws an error if the param is not part of the enum. +func StateString(s string) (State, error) { + if val, ok := _StateNameToValueMap[s]; ok { + return val, nil + } + return 0, fmt.Errorf("%s does not belong to State values", s) +} + +// StateValues returns all values of the enum +func StateValues() []State { + return _StateValues +} + +// IsAState returns "true" if the value is listed in the enum definition. "false" otherwise +func (i State) IsAState() bool { + for _, v := range _StateValues { + if i == v { + return true + } + } + return false +} diff --git a/daemon/pruner/state_string.go b/daemon/pruner/state_string.go deleted file mode 100644 index 68e04e1..0000000 --- a/daemon/pruner/state_string.go +++ /dev/null @@ -1,35 +0,0 @@ -// Code generated by "stringer -type=State"; DO NOT EDIT. - -package pruner - -import "strconv" - -const ( - _State_name_0 = "PlanPlanWait" - _State_name_1 = "Exec" - _State_name_2 = "ExecWait" - _State_name_3 = "ErrPerm" - _State_name_4 = "Done" -) - -var ( - _State_index_0 = [...]uint8{0, 4, 12} -) - -func (i State) String() string { - switch { - case 1 <= i && i <= 2: - i -= 1 - return _State_name_0[_State_index_0[i]:_State_index_0[i+1]] - case i == 4: - return _State_name_1 - case i == 8: - return _State_name_2 - case i == 16: - return _State_name_3 - case i == 32: - return _State_name_4 - default: - return "State(" + strconv.FormatInt(int64(i), 10) + ")" - } -} diff --git a/lazy.sh b/lazy.sh index a8c7316..434f66b 100755 --- a/lazy.sh +++ b/lazy.sh @@ -30,7 +30,8 @@ builddep() { go get -u golang.org/x/tools/cmd/stringer go get -u github.com/golang/dep/cmd/dep go get -u github.com/golang/protobuf/protoc-gen-go - if ! type stringer || ! type dep || ! type protoc-gen-go; then + go get -u github.com/alvaroloes/enumer + if ! type stringer || ! type dep || ! type protoc-gen-go; || ! type enumer then echo "Installed dependencies but can't find them in \$PATH, adjust it to contain \$GOPATH/bin" 1>&2 exit 1 fi diff --git a/replication/mainfsm.go b/replication/mainfsm.go index 15137b0..086fda1 100644 --- a/replication/mainfsm.go +++ b/replication/mainfsm.go @@ -18,7 +18,7 @@ import ( "github.com/zrepl/zrepl/replication/pdu" ) -//go:generate stringer -type=State +//go:generate enumer -type=State type State uint const ( diff --git a/replication/state_enumer.go b/replication/state_enumer.go new file mode 100644 index 0000000..7343d69 --- /dev/null +++ b/replication/state_enumer.go @@ -0,0 +1,76 @@ +// Code generated by "enumer -type=State"; DO NOT EDIT. + +package replication + +import ( + "fmt" +) + +const ( + _StateName_0 = "PlanningPlanningError" + _StateName_1 = "Working" + _StateName_2 = "WorkingWait" + _StateName_3 = "Completed" + _StateName_4 = "ContextDone" +) + +var ( + _StateIndex_0 = [...]uint8{0, 8, 21} + _StateIndex_1 = [...]uint8{0, 7} + _StateIndex_2 = [...]uint8{0, 11} + _StateIndex_3 = [...]uint8{0, 9} + _StateIndex_4 = [...]uint8{0, 11} +) + +func (i State) String() string { + switch { + case 1 <= i && i <= 2: + i -= 1 + return _StateName_0[_StateIndex_0[i]:_StateIndex_0[i+1]] + case i == 4: + return _StateName_1 + case i == 8: + return _StateName_2 + case i == 16: + return _StateName_3 + case i == 32: + return _StateName_4 + default: + return fmt.Sprintf("State(%d)", i) + } +} + +var _StateValues = []State{1, 2, 4, 8, 16, 32} + +var _StateNameToValueMap = map[string]State{ + _StateName_0[0:8]: 1, + _StateName_0[8:21]: 2, + _StateName_1[0:7]: 4, + _StateName_2[0:11]: 8, + _StateName_3[0:9]: 16, + _StateName_4[0:11]: 32, +} + +// StateString retrieves an enum value from the enum constants string name. +// Throws an error if the param is not part of the enum. +func StateString(s string) (State, error) { + if val, ok := _StateNameToValueMap[s]; ok { + return val, nil + } + return 0, fmt.Errorf("%s does not belong to State values", s) +} + +// StateValues returns all values of the enum +func StateValues() []State { + return _StateValues +} + +// IsAState returns "true" if the value is listed in the enum definition. "false" otherwise +func (i State) IsAState() bool { + for _, v := range _StateValues { + if i == v { + return true + } + } + return false +} diff --git a/replication/state_string.go b/replication/state_string.go deleted file mode 100644 index a4514d2..0000000 --- a/replication/state_string.go +++ /dev/null @@ -1,35 +0,0 @@ -// Code generated by "stringer -type=State"; DO NOT EDIT. - -package replication - -import "strconv" - -const ( - _State_name_0 = "PlanningPlanningError" - _State_name_1 = "Working" - _State_name_2 = "WorkingWait" - _State_name_3 = "Completed" - _State_name_4 = "ContextDone" -) - -var ( - _State_index_0 = [...]uint8{0, 8, 21} -) - -func (i State) String() string { - switch { - case 1 <= i && i <= 2: - i -= 1 - return _State_name_0[_State_index_0[i]:_State_index_0[i+1]] - case i == 4: - return _State_name_1 - case i == 8: - return _State_name_2 - case i == 16: - return _State_name_3 - case i == 32: - return _State_name_4 - default: - return "State(" + strconv.FormatInt(int64(i), 10) + ")" - } -} From d584e1ac541c43de40dbcb7137fbcd89c54f4ec4 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 12 Oct 2018 22:15:07 +0200 Subject: [PATCH 160/167] daemon/job/active: fix race in updateTasks If concurrent updates strictly modify *different* members of the tasks struct, the copying + lock-drop still constitutes a race condition: The last updater always wins and sets tasks to its copy + changes. This eliminates the other updater's changes. --- daemon/job/active.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/daemon/job/active.go b/daemon/job/active.go index 0d78abb..482b368 100644 --- a/daemon/job/active.go +++ b/daemon/job/active.go @@ -43,16 +43,14 @@ type activeSideTasks struct { func (a *ActiveSide) updateTasks(u func(*activeSideTasks)) activeSideTasks { a.tasksMtx.Lock() + defer a.tasksMtx.Unlock() var copy activeSideTasks copy = a.tasks - a.tasksMtx.Unlock() if u == nil { return copy } u(©) - a.tasksMtx.Lock() a.tasks = copy - a.tasksMtx.Unlock() return copy } From a85abe8bae7a542c6fe092719c66ed592dbe30b4 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 12 Oct 2018 22:16:11 +0200 Subject: [PATCH 161/167] client/status: improve hiding of data if current state makes it obsolete --- client/status.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/client/status.go b/client/status.go index d56773e..7ecb986 100644 --- a/client/status.go +++ b/client/status.go @@ -264,7 +264,8 @@ func (t *tui) renderReplicationReport(rep *replication.Report) { t.printf("Problem: %s", rep.Problem) t.newline() } - if rep.SleepUntil.After(time.Now()) { + if rep.SleepUntil.After(time.Now()) && + state & ^(replication.ContextDone|replication.Completed) != 0 { t.printf("Sleeping until %s (%s left)\n", rep.SleepUntil, rep.SleepUntil.Sub(time.Now())) } @@ -338,7 +339,10 @@ func (t *tui) renderPrunerReport(r *pruner.Report) { all = append(all, commonFS{&r.Completed[i], true}) } - if r.State == pruner.Plan.String() { + switch state { + case pruner.Plan: fallthrough + case pruner.PlanWait: fallthrough + case pruner.ErrPerm: return } From aeb87ffbcf86233b59bfdb78dfacae071b09cee6 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 12 Oct 2018 22:46:34 +0200 Subject: [PATCH 162/167] daemon/job/active: push mode: awful hack for handling of concurrent snapshots + stale remote operation We have the problem that there are legitimate use cases where a user does not want their machine to fill up with snapshots, even if it means unreplicated must be destroyed. This can be expressed by *not* configuring the keep rule `not_replicated` for the snapshot-creating side. This commit only addresses push mode because we don't support pruning in the source job. We adivse users in the docs to use push mode if they have above use case, so this is fine - at least for 0.1. Ideally, the replication.Replication would communicate to the pruner which snapshots are currently part of the replication plan, and then we'd need some conflict resolution to determine whether it's more important to destroy the snapshots or to replicate them (destroy should win?). However, we don't have the infrastructure for this yet (we could parse the replication report, but that's just ugly). And we want to get 0.1 out, so showtime for a dirty hack: We start replication, and ideally, replication and pruning is done before new snapshot have been taken. If so: great. However, what happens if snapshots have been taken and we are not done with replication and / or pruning? * If replicatoin is making progress according to its state, let it run. This covers the *important* situation of initial replication, where replication may easily take longer than a single snapshotting interval. * If replication is in an error state, cancel it through context cancellation. * As with the pruner below, the main problem here is that status output will only contain "context cancelled" after the cancellation, instead of showing the reason why it was cancelled. Not nice, but oh well, the logs provide enough detail for this niche situation... * If we are past replication, we're still pruning * Leave the local (send-side) pruning alone. Again, we only implement this hack for push, so we know sender is local, and it will only fail hard, not retry. * If the remote (receiver-side) pruner is in an error state, cancel it through context cancellation. * Otherwise, let it run. Note that every time we "let it run", we tolerate a temporary excess of snapshots, but given sufficiently aggressive timeouts and the assumption that the snapshot interval is much greater than the timeouts, this is not a significant problem in practice. --- daemon/job/active.go | 133 ++++++++++++++++++++++++++++++++++++++++--- 1 file changed, 124 insertions(+), 9 deletions(-) diff --git a/daemon/job/active.go b/daemon/job/active.go index 482b368..2822eba 100644 --- a/daemon/job/active.go +++ b/daemon/job/active.go @@ -238,11 +238,11 @@ outer: } invocationCount++ invLog := log.WithField("invocation", invocationCount) - j.do(WithLogger(ctx, invLog)) + j.do(WithLogger(ctx, invLog), periodicDone) } } -func (j *ActiveSide) do(ctx context.Context) { +func (j *ActiveSide) do(ctx context.Context, periodicWakeup <-chan struct{}) { log := GetLogger(ctx) ctx = logging.WithSubsystemLoggers(ctx, log) @@ -277,14 +277,129 @@ func (j *ActiveSide) do(ctx context.Context) { }) log.Info("start replication") - tasks.replication.Drive(ctx, sender, receiver) + replicationDone := make(chan struct{}) + replicationCtx, replicationCancel := context.WithCancel(ctx) + defer replicationCancel() + go func() { + tasks.replication.Drive(replicationCtx, sender, receiver) + close(replicationDone) + }() + outer: + for { + select { + case <-replicationDone: + // fine! + break outer + case <-periodicWakeup: + // Replication took longer than the periodic interval. + // + // For pull jobs, this isn't so bad because nothing changes on the active side + // if replication doesn't go forward. + // + // For push jobs, this means snapshots were taken. + // We need to invoke the pruner now, because otherwise an infinitely stuck replication + // will cause this side to fill up with snapshots. + // + // However, there are cases where replication progresses and just takes longer, + // and we don't want these situations be interrupted by a prune, which will require + // re-planning and starting over (think of initial replication as an example). + // + // Therefore, we prohibit pruning of snapshots that are part of the current replication plan. + // If there is no such plan, we kill the replication. - tasks = j.updateTasks(func(tasks *activeSideTasks) { - tasks.prunerSender = j.prunerFactory.BuildSenderPruner(ctx, sender, sender) - tasks.prunerReceiver = j.prunerFactory.BuildReceiverPruner(ctx, receiver, sender) - }) + if j.mode.Type() == TypePush { + + rep := tasks.replication.Report() + state, err := replication.StateString(rep.Status) + if err != nil { + panic(err) + } + + switch state { + case replication.Planning: + fallthrough + case replication.PlanningError: + fallthrough + case replication.WorkingWait: + log.WithField("repl_state", state.String()). + Info("cancelling replication after new snapshots invalidated its current state") + replicationCancel() + log.Info("waiting for replication to stop") + <-replicationDone // no need to wait for ctx.Done, replication is already bound to global cancel + break outer + default: + log.WithField("repl_state", state.String()). + Warn("new snapshots while replication is running and making progress") + } + + } + + } + } + + + var pruningWg sync.WaitGroup log.Info("start pruning sender") - tasks.prunerSender.Prune() + pruningWg.Add(1) + go func() { + defer pruningWg.Done() + tasks := j.updateTasks(func(tasks *activeSideTasks) { + tasks.prunerSender = j.prunerFactory.BuildSenderPruner(ctx, sender, sender) + }) + tasks.prunerSender.Prune() + // FIXME no need to do the cancellation dance with sender, we know it's local for push + // FIXME and we don't worry about pull ATM + }() log.Info("start pruning receiver") - tasks.prunerReceiver.Prune() + pruningWg.Add(1) + go func() { + defer pruningWg.Done() + + receiverPrunerCtx, receiverPrunerCancel := context.WithCancel(ctx) + defer receiverPrunerCancel() + tasks := j.updateTasks(func(tasks *activeSideTasks) { + tasks.prunerReceiver = j.prunerFactory.BuildReceiverPruner(receiverPrunerCtx, receiver, sender) + }) + receiverPrunerDone := make(chan struct{}) + go func() { + defer close(receiverPrunerDone) + tasks.prunerReceiver.Prune() + }() + + outer: + for { + select { + case <-receiverPrunerDone: + // fine! + break outer + case <-periodicWakeup: + // see comments for similar apporach with replication above + if j.mode.Type() == TypePush { + rep := tasks.prunerReceiver.Report() + state, err := pruner.StateString(rep.State) + if err != nil { + panic(err) + } + switch state { + case pruner.PlanWait: + fallthrough + case pruner.ExecWait: + log.WithField("pruner_state", state.String()). + Info("cancelling failing prune on receiver because new snapshots were taken on sender") + receiverPrunerCancel() + log.Info("waiting for receiver pruner to stop") + <-receiverPrunerDone + break outer + default: + log.WithField("pruner_state", state.String()). + Warn("new snapshots while prune on receiver is still running") + } + } + } + } + + }() + + pruningWg.Wait() // if pruners handle ctx cancellation correctly, we don't need to wait for it here + } From 5c3c83b2cb2c203d461ff28b39390cce82f161df Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 13 Oct 2018 15:07:50 +0200 Subject: [PATCH 163/167] cli: refactor to allow definition of subcommands next to their implementation --- cli/cli.go | 105 ++++++++++++++++++++++++++ client/configcheck.go | 36 ++++++++- client/pprof.go | 37 ++++++++- client/signal.go | 11 ++- client/status.go | 26 +++++-- client/stdinserver.go | 14 +++- client/version.go | 39 +++++++--- daemon/main.go | 9 +++ main.go | 169 +++--------------------------------------- 9 files changed, 255 insertions(+), 191 deletions(-) create mode 100644 cli/cli.go diff --git a/cli/cli.go b/cli/cli.go new file mode 100644 index 0000000..5e646d1 --- /dev/null +++ b/cli/cli.go @@ -0,0 +1,105 @@ +package cli + +import ( + "fmt" + "github.com/spf13/cobra" + "github.com/spf13/pflag" + "github.com/zrepl/zrepl/config" + "os" +) + +var rootArgs struct { + configPath string +} + +var rootCmd = &cobra.Command{ + Use: "zrepl", + Short: "One-stop ZFS replication solution", +} + +var bashcompCmd = &cobra.Command{ + Use: "bashcomp path/to/out/file", + Short: "generate bash completions", + Run: func(cmd *cobra.Command, args []string) { + if len(args) != 1 { + fmt.Fprintf(os.Stderr, "specify exactly one positional agument\n") + cmd.Usage() + os.Exit(1) + } + if err := rootCmd.GenBashCompletionFile(args[0]); err != nil { + fmt.Fprintf(os.Stderr, "error generating bash completion: %s", err) + os.Exit(1) + } + }, + Hidden: true, +} + +func init() { + rootCmd.PersistentFlags().StringVar(&rootArgs.configPath, "config", "", "config file path") + rootCmd.AddCommand(bashcompCmd) +} + +type Subcommand struct { + Use string + Short string + NoRequireConfig bool + Run func(subcommand *Subcommand, args []string) error + SetupFlags func(f *pflag.FlagSet) + + config *config.Config + configErr error +} + +func (s *Subcommand) ConfigParsingError() error { + return s.configErr +} + +func (s *Subcommand) Config() *config.Config { + if !s.NoRequireConfig && s.config == nil { + panic("command that requires config is running and has no config set") + } + return s.config +} + +func (s *Subcommand) run(cmd *cobra.Command, args []string) { + s.tryParseConfig() + err := s.Run(s, args) + if err != nil { + fmt.Fprintf(os.Stderr, "%s\n", err) + os.Exit(1) + } +} + +func (s *Subcommand) tryParseConfig() { + config, err := config.ParseConfig(rootArgs.configPath) + s.configErr = err + if err != nil { + if s.NoRequireConfig { + // doesn't matter + return + } else { + fmt.Fprintf(os.Stderr, "could not parse config: %s\n", err) + os.Exit(1) + } + } + s.config = config +} + +func AddSubcommand(s *Subcommand) { + cmd := cobra.Command{ + Use: s.Use, + Short: s.Short, + Run: s.run, + } + if s.SetupFlags != nil { + s.SetupFlags(cmd.Flags()) + } + rootCmd.AddCommand(&cmd) +} + + +func Run() { + if err := rootCmd.Execute(); err != nil { + os.Exit(1) + } +} \ No newline at end of file diff --git a/client/configcheck.go b/client/configcheck.go index b50ac80..34c1b13 100644 --- a/client/configcheck.go +++ b/client/configcheck.go @@ -1,8 +1,36 @@ package client -import "github.com/zrepl/zrepl/config" +import ( + "encoding/json" + "github.com/kr/pretty" + "github.com/spf13/pflag" + "github.com/zrepl/yaml-config" + "github.com/zrepl/zrepl/cli" + "os" +) -func RunConfigcheck(conf *config.Config, args []string) error { - // TODO: do the 'build' steps, e.g. build the jobs and see if that fails - return nil +var configcheckArgs struct { + format string } + +var ConfigcheckCmd = &cli.Subcommand{ + Use: "configcheck", + Short: "check if config can be parsed without errors", + SetupFlags: func(f *pflag.FlagSet) { + f.StringVar(&configcheckArgs.format, "format", "", "dump parsed config object [pretty|yaml|json]") + }, + Run: func(subcommand *cli.Subcommand, args []string) error { + switch configcheckArgs.format { + case "pretty": + _, err := pretty.Println(subcommand.Config()) + return err + case "json": + return json.NewEncoder(os.Stdout).Encode(subcommand.Config()) + case "yaml": + return yaml.NewEncoder(os.Stdout).Encode(subcommand.Config()) + default: // no output + } + return nil + }, +} + diff --git a/client/pprof.go b/client/pprof.go index 7afb4cb..29cd4e5 100644 --- a/client/pprof.go +++ b/client/pprof.go @@ -1,17 +1,48 @@ package client import ( + "errors" + "github.com/zrepl/zrepl/cli" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon" "log" "os" ) -type PProfArgs struct { +var pprofArgs struct { daemon.PprofServerControlMsg } -func RunPProf(conf *config.Config, args PProfArgs) { +var PprofCmd = &cli.Subcommand{ + Use: "pprof off | [on TCP_LISTEN_ADDRESS]", + Short: "start a http server exposing go-tool-compatible profiling endpoints at TCP_LISTEN_ADDRESS", + Run: func(subcommand *cli.Subcommand, args []string) error { + if len(args) < 1 { + goto enargs + } + switch args[0] { + case "on": + pprofArgs.Run = true + if len(args) != 2 { + return errors.New("must specify TCP_LISTEN_ADDRESS as second positional argument") + } + pprofArgs.HttpListenAddress = args[1] + case "off": + if len(args) != 1 { + goto enargs + } + pprofArgs.Run = false + } + + RunPProf(subcommand.Config()) + return nil + enargs: + return errors.New("invalid number of positional arguments") + + }, +} + +func RunPProf(conf *config.Config) { log := log.New(os.Stderr, "", 0) die := func() { @@ -26,7 +57,7 @@ func RunPProf(conf *config.Config, args PProfArgs) { log.Printf("error creating http client: %s", err) die() } - err = jsonRequestResponse(httpc, daemon.ControlJobEndpointPProf, args.PprofServerControlMsg, struct{}{}) + err = jsonRequestResponse(httpc, daemon.ControlJobEndpointPProf, pprofArgs.PprofServerControlMsg, struct{}{}) if err != nil { log.Printf("error sending control message: %s", err) die() diff --git a/client/signal.go b/client/signal.go index 273ea18..701849e 100644 --- a/client/signal.go +++ b/client/signal.go @@ -2,11 +2,20 @@ package client import ( "github.com/pkg/errors" + "github.com/zrepl/zrepl/cli" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon" ) -func RunSignal(config *config.Config, args []string) error { +var SignalCmd = &cli.Subcommand{ + Use: "signal [wakeup|reset] JOB", + Short: "wake up a job from wait state or abort its current invocation", + Run: func(subcommand *cli.Subcommand, args []string) error { + return runSignalCmd(subcommand.Config(), args) + }, +} + +func runSignalCmd(config *config.Config, args []string) error { if len(args) != 2 { return errors.Errorf("Expected 2 arguments: [wakeup|reset] JOB") } diff --git a/client/status.go b/client/status.go index 7ecb986..7e715e9 100644 --- a/client/status.go +++ b/client/status.go @@ -4,21 +4,22 @@ import ( "fmt" "github.com/nsf/termbox-go" "github.com/pkg/errors" + "github.com/spf13/pflag" "github.com/zrepl/yaml-config" - "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/cli" "github.com/zrepl/zrepl/daemon" "github.com/zrepl/zrepl/daemon/job" "github.com/zrepl/zrepl/daemon/pruner" "github.com/zrepl/zrepl/replication" "github.com/zrepl/zrepl/replication/fsrep" + "io" "math" + "net/http" + "os" "sort" "strings" "sync" "time" - "io" - "os" - "net/http" ) type tui struct { @@ -73,17 +74,26 @@ func (t *tui) addIndent(indent int) { t.moveLine(0, 0) } -type StatusFlags struct { +var statusFlags struct { Raw bool } -func RunStatus(flags StatusFlags, config *config.Config, args []string) error { - httpc, err := controlHttpClient(config.Global.Control.SockPath) +var StatusCmd = &cli.Subcommand{ + Use: "status", + Short: "show job activity or dump as JSON for monitoring", + SetupFlags: func(f *pflag.FlagSet) { + f.BoolVar(&statusFlags.Raw, "raw", false, "dump raw status description from zrepl daemon") + }, + Run: runStatus, +} + +func runStatus(s *cli.Subcommand, args []string) error { + httpc, err := controlHttpClient(s.Config().Global.Control.SockPath) if err != nil { return err } - if flags.Raw { + if statusFlags.Raw { resp, err := httpc.Get("http://unix"+daemon.ControlJobEndpointStatus) if err != nil { return err diff --git a/client/stdinserver.go b/client/stdinserver.go index 9d47a25..5db5520 100644 --- a/client/stdinserver.go +++ b/client/stdinserver.go @@ -1,18 +1,26 @@ package client import ( + "github.com/zrepl/zrepl/cli" "os" "context" + "errors" "github.com/problame/go-netssh" + "github.com/zrepl/zrepl/config" "log" "path" - "github.com/zrepl/zrepl/config" - "errors" ) +var StdinserverCmd = &cli.Subcommand{ + Use: "stdinserver CLIENT_IDENTITY", + Short: "stdinserver transport mode (started from authorized_keys file as forced command)", + Run: func(subcommand *cli.Subcommand, args []string) error { + return runStdinserver(subcommand.Config(), args) + }, +} -func RunStdinserver(config *config.Config, args []string) error { +func runStdinserver(config *config.Config, args []string) error { // NOTE: the netssh proxying protocol requires exiting with non-zero status if anything goes wrong defer os.Exit(1) diff --git a/client/version.go b/client/version.go index e584a80..9dcec05 100644 --- a/client/version.go +++ b/client/version.go @@ -2,27 +2,39 @@ package client import ( "fmt" + "github.com/spf13/pflag" + "github.com/zrepl/zrepl/cli" "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon" "github.com/zrepl/zrepl/version" "os" ) -type VersionArgs struct { +var versionArgs struct { Show string Config *config.Config + ConfigErr error } -func RunVersion(args VersionArgs) { +var VersionCmd = &cli.Subcommand{ + Use: "version", + Short: "print version of zrepl binary and running daemon", + NoRequireConfig: true, + SetupFlags: func(f *pflag.FlagSet) { + f.StringVar(&versionArgs.Show, "show", "", "version info to show (client|daemon)") + }, + Run: func(subcommand *cli.Subcommand, args []string) error { + versionArgs.Config = subcommand.Config() + versionArgs.ConfigErr = subcommand.ConfigParsingError() + return runVersionCmd() + }, +} - die := func() { - fmt.Fprintf(os.Stderr, "exiting after error\n") - os.Exit(1) - } +func runVersionCmd() error { + args := versionArgs if args.Show != "daemon" && args.Show != "client" && args.Show != "" { - fmt.Fprintf(os.Stderr, "show flag must be 'client' or 'server' or be left empty") - die() + return fmt.Errorf("show flag must be 'client' or 'server' or be left empty") } var clientVersion, daemonVersion *version.ZreplVersionInformation @@ -32,17 +44,19 @@ func RunVersion(args VersionArgs) { } if args.Show == "daemon" || args.Show == "" { + if args.ConfigErr != nil { + return fmt.Errorf("config parsing error: %s", args.ConfigErr) + } + httpc, err := controlHttpClient(args.Config.Global.Control.SockPath) if err != nil { - fmt.Fprintf(os.Stderr, "server: error: %s\n", err) - die() + return fmt.Errorf("server: error: %s\n", err) } var info version.ZreplVersionInformation err = jsonRequestResponse(httpc, daemon.ControlJobEndpointVersion, "", &info) if err != nil { - fmt.Fprintf(os.Stderr, "server: error: %s\n", err) - die() + return fmt.Errorf("server: error: %s\n", err) } daemonVersion = &info fmt.Printf("server: %s\n", daemonVersion.String()) @@ -54,4 +68,5 @@ func RunVersion(args VersionArgs) { } } + return nil } diff --git a/daemon/main.go b/daemon/main.go index 2dd2590..fae8c12 100644 --- a/daemon/main.go +++ b/daemon/main.go @@ -1,7 +1,16 @@ package daemon import ( + "github.com/zrepl/zrepl/cli" "github.com/zrepl/zrepl/logger" ) type Logger = logger.Logger + +var DaemonCmd = &cli.Subcommand { + Use: "daemon", + Short: "run the zrepl daemon", + Run: func(subcommand *cli.Subcommand, args []string) error { + return Run(subcommand.Config()) + }, +} diff --git a/main.go b/main.go index 7c112a6..3b5ae45 100644 --- a/main.go +++ b/main.go @@ -2,172 +2,21 @@ package main import ( - "errors" - "github.com/spf13/cobra" + "github.com/zrepl/zrepl/cli" "github.com/zrepl/zrepl/client" - "github.com/zrepl/zrepl/config" "github.com/zrepl/zrepl/daemon" - "log" - "os" - "fmt" ) -var rootCmd = &cobra.Command{ - Use: "zrepl", - Short: "One-stop ZFS replication solution", -} - -var daemonCmd = &cobra.Command{ - Use: "daemon", - Short: "run the zrepl daemon", - RunE: func(cmd *cobra.Command, args []string) error { - conf, err := config.ParseConfig(rootArgs.configFile) - if err != nil { - return err - } - return daemon.Run(conf) - }, -} - -var signalCmd = &cobra.Command{ - Use: "signal [wakeup|reset] JOB", - Short: "wake up a job from wait state or abort its current invocation", - RunE: func(cmd *cobra.Command, args []string) error { - conf, err := config.ParseConfig(rootArgs.configFile) - if err != nil { - return err - } - return client.RunSignal(conf, args) - }, -} - -var statusCmdFlags client.StatusFlags - -var statusCmd = &cobra.Command{ - Use: "status", - Short: "show job activity or dump as JSON for monitoring", - RunE: func(cmd *cobra.Command, args []string) error { - conf, err := config.ParseConfig(rootArgs.configFile) - if err != nil { - return err - } - return client.RunStatus(statusCmdFlags, conf, args) - }, -} - -var stdinserverCmd = &cobra.Command{ - Use: "stdinserver CLIENT_IDENTITY", - Short: "stdinserver transport mode (started from authorized_keys file as forced command)", - RunE: func(cmd *cobra.Command, args []string) error { - conf, err := config.ParseConfig(rootArgs.configFile) - if err != nil { - return err - } - return client.RunStdinserver(conf, args) - }, -} - - -var bashcompCmd = &cobra.Command{ - Use: "bashcomp path/to/out/file", - Short: "generate bash completions", - Run: func(cmd *cobra.Command, args []string) { - if len(args) != 1 { - fmt.Fprintf(os.Stderr, "specify exactly one positional agument\n") - cmd.Usage() - os.Exit(1) - } - if err := rootCmd.GenBashCompletionFile(args[0]); err != nil { - fmt.Fprintf(os.Stderr, "error generating bash completion: %s", err) - os.Exit(1) - } - }, - Hidden: true, -} - -var configcheckCmd = &cobra.Command{ - Use: "configcheck", - Short: "check if config can be parsed without errors", - RunE: func(cmd *cobra.Command, args []string) error { - conf, err := config.ParseConfig(rootArgs.configFile) - if err != nil { - return err - } - return client.RunConfigcheck(conf, args) - }, -} - -var versionCmdArgs client.VersionArgs -var versionCmd = &cobra.Command{ - Use: "version", - Short: "print version of zrepl binary and running daemon", - Run: func(cmd *cobra.Command, args []string) { - conf, err := config.ParseConfig(rootArgs.configFile) - if err == nil { - versionCmdArgs.Config = conf - } - client.RunVersion(versionCmdArgs) - }, -} - -var pprofCmd = &cobra.Command{ - Use: "pprof off | [on TCP_LISTEN_ADDRESS]", - Short: "start a http server exposing go-tool-compatible profiling endpoints at TCP_LISTEN_ADDRESS", - RunE: func(cmd *cobra.Command, args []string) error { - conf, err := config.ParseConfig(rootArgs.configFile) - if err != nil { - return err - } - - var pprofCmdArgs client.PProfArgs - if cmd.Flags().NArg() < 1 { - goto enargs - } - switch cmd.Flags().Arg(0) { - case "on": - pprofCmdArgs.Run = true - if cmd.Flags().NArg() != 2 { - return errors.New("must specify TCP_LISTEN_ADDRESS as second positional argument") - } - pprofCmdArgs.HttpListenAddress = cmd.Flags().Arg(1) - case "off": - if cmd.Flags().NArg() != 1 { - goto enargs - } - pprofCmdArgs.Run = false - } - - client.RunPProf(conf, pprofCmdArgs) - return nil - enargs: - return errors.New("invalid number of positional arguments") - - }, -} - -var rootArgs struct { - configFile string -} - func init() { - //cobra.OnInitialize(initConfig) - rootCmd.PersistentFlags().StringVar(&rootArgs.configFile, "config", "", "config file path") - rootCmd.AddCommand(daemonCmd) - rootCmd.AddCommand(signalCmd) - statusCmd.Flags().BoolVar(&statusCmdFlags.Raw, "raw", false, "dump raw status description from zrepl daemon") - rootCmd.AddCommand(statusCmd) - rootCmd.AddCommand(stdinserverCmd) - rootCmd.AddCommand(bashcompCmd) - rootCmd.AddCommand(configcheckCmd) - versionCmd.Flags().StringVar(&versionCmdArgs.Show, "show", "", "version info to show (client|daemon)") - rootCmd.AddCommand(versionCmd) - rootCmd.AddCommand(pprofCmd) + cli.AddSubcommand(daemon.DaemonCmd) + cli.AddSubcommand(client.StatusCmd) + cli.AddSubcommand(client.SignalCmd) + cli.AddSubcommand(client.StdinserverCmd) + cli.AddSubcommand(client.ConfigcheckCmd) + cli.AddSubcommand(client.VersionCmd) + cli.AddSubcommand(client.PprofCmd) } func main() { - - if err := rootCmd.Execute(); err != nil { - log.Printf("error executing root command: %s", err) - os.Exit(1) - } + cli.Run() } From 63169c51b77705dabb8ad7a65011dc1521f49d3d Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 13 Oct 2018 15:53:52 +0200 Subject: [PATCH 164/167] add 'test filesystems' subcommand for testing filesystem filters --- cli/cli.go | 15 +++- client/testcmd.go | 108 +++++++++++++++++++++++++++ config/config.go | 22 ++++++ docs/configuration/filter_syntax.rst | 2 +- main.go | 1 + 5 files changed, 145 insertions(+), 3 deletions(-) create mode 100644 client/testcmd.go diff --git a/cli/cli.go b/cli/cli.go index 5e646d1..8904bdc 100644 --- a/cli/cli.go +++ b/cli/cli.go @@ -45,6 +45,7 @@ type Subcommand struct { NoRequireConfig bool Run func(subcommand *Subcommand, args []string) error SetupFlags func(f *pflag.FlagSet) + SetupSubcommands func() []*Subcommand config *config.Config configErr error @@ -86,15 +87,25 @@ func (s *Subcommand) tryParseConfig() { } func AddSubcommand(s *Subcommand) { + addSubcommandToCobraCmd(rootCmd, s) +} + +func addSubcommandToCobraCmd(c *cobra.Command, s *Subcommand) { cmd := cobra.Command{ Use: s.Use, Short: s.Short, - Run: s.run, + } + if s.SetupSubcommands == nil { + cmd.Run = s.run + } else { + for _, sub := range s.SetupSubcommands() { + addSubcommandToCobraCmd(&cmd, sub) + } } if s.SetupFlags != nil { s.SetupFlags(cmd.Flags()) } - rootCmd.AddCommand(&cmd) + c.AddCommand(&cmd) } diff --git a/client/testcmd.go b/client/testcmd.go new file mode 100644 index 0000000..6635db5 --- /dev/null +++ b/client/testcmd.go @@ -0,0 +1,108 @@ +package client + +import ( + "fmt" + "github.com/spf13/pflag" + "github.com/zrepl/zrepl/cli" + "github.com/zrepl/zrepl/config" + "github.com/zrepl/zrepl/daemon/filters" + "github.com/zrepl/zrepl/zfs" +) + +var TestCmd = &cli.Subcommand { + Use: "test", + SetupSubcommands: func() []*cli.Subcommand { + return []*cli.Subcommand{testFilter} + }, +} + +var testFilterArgs struct { + job string + all bool + input string +} + +var testFilter = &cli.Subcommand{ + Use: "filesystems --job JOB [--all | --input INPUT]", + Short: "test filesystems filter specified in push or source job", + SetupFlags: func(f *pflag.FlagSet) { + f.StringVar(&testFilterArgs.job, "job", "", "the name of the push or source job") + f.StringVar(&testFilterArgs.input, "input", "", "a filesystem name to test against the job's filters") + f.BoolVar(&testFilterArgs.all, "all", false, "test all local filesystems") + }, + Run: runTestFilterCmd, +} + +func runTestFilterCmd(subcommand *cli.Subcommand, args []string) error { + + if testFilterArgs.job == "" { + return fmt.Errorf("must specify --job flag") + } + if !(testFilterArgs.all != (testFilterArgs.input != "")) { // xor + return fmt.Errorf("must set one: --all or --input") + } + + conf := subcommand.Config() + + var confFilter config.FilesystemsFilter + job, err := conf.Job(testFilterArgs.job) + if err != nil { + return err + } + switch j := job.Ret.(type) { + case *config.SourceJob: confFilter = j.Filesystems + case *config.PushJob: confFilter = j.Filesystems + default: + return fmt.Errorf("job type %T does not have filesystems filter", j) + } + + f, err := filters.DatasetMapFilterFromConfig(confFilter) + if err != nil { + return fmt.Errorf("filter invalid: %s", err) + } + + var fsnames []string + if testFilterArgs.input != "" { + fsnames = []string{testFilterArgs.input} + } else { + out, err := zfs.ZFSList([]string{"name"}) + if err != nil { + return fmt.Errorf("could not list ZFS filesystems: %s", err) + } + for _, row := range out { + + fsnames = append(fsnames, row[0]) + } + } + + fspaths := make([]*zfs.DatasetPath, len(fsnames)) + for i, fsname := range fsnames { + path, err := zfs.NewDatasetPath(fsname) + if err != nil { + return err + } + fspaths[i] = path + } + + hadFilterErr := false + for _, in := range fspaths { + var res string + var errStr string + pass, err := f.Filter(in) + if err != nil { + res = "ERROR" + errStr = err.Error() + hadFilterErr = true + } else if pass { + res = "ACCEPT" + } else { + res = "REJECT" + } + fmt.Printf("%s\t%s\t%s\n", res, in.ToString(), errStr) + } + + if hadFilterErr { + return fmt.Errorf("filter errors occurred") + } + return nil +} \ No newline at end of file diff --git a/config/config.go b/config/config.go index 62d2cb3..1b14dba 100644 --- a/config/config.go +++ b/config/config.go @@ -17,10 +17,32 @@ type Config struct { Global *Global `yaml:"global,optional,fromdefaults"` } +func (c *Config) Job(name string) (*JobEnum, error) { + for _, j := range c.Jobs { + if j.Name() == name { + return &j, nil + } + } + return nil, fmt.Errorf("job %q not defined in config", name) +} + type JobEnum struct { Ret interface{} } +func (j JobEnum) Name() string { + var name string + switch v := j.Ret.(type) { + case *PushJob: name = v.Name + case *SinkJob: name = v.Name + case *PullJob: name = v.Name + case *SourceJob: name = v.Name + default: + panic(fmt.Sprintf("unknownn job type %T", v)) + } + return name +} + type ActiveJob struct { Type string `yaml:"type"` Name string `yaml:"name"` diff --git a/docs/configuration/filter_syntax.rst b/docs/configuration/filter_syntax.rst index ea6bf29..6657998 100644 --- a/docs/configuration/filter_syntax.rst +++ b/docs/configuration/filter_syntax.rst @@ -18,7 +18,7 @@ The following rules determine which result is chosen for a given filesystem path The **subtree wildcard** ``<`` means "the dataset left of ``<`` and all its children". .. TIP:: - You can try out patterns for a configured job using the ``zrepl test`` subcommand. + You can try out patterns for a configured job using the ``zrepl test filesystems`` subcommand for push and source jobs. Examples -------- diff --git a/main.go b/main.go index 3b5ae45..7d209b9 100644 --- a/main.go +++ b/main.go @@ -15,6 +15,7 @@ func init() { cli.AddSubcommand(client.ConfigcheckCmd) cli.AddSubcommand(client.VersionCmd) cli.AddSubcommand(client.PprofCmd) + cli.AddSubcommand(client.TestCmd) } func main() { From 92a1a6d2ca7b99de6ec2e426eb6093b596e622be Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 13 Oct 2018 15:55:16 +0200 Subject: [PATCH 165/167] docs: fix wrong subcommand for configcheck --- docs/configuration/preface.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration/preface.rst b/docs/configuration/preface.rst index 271bb6a..b184d44 100644 --- a/docs/configuration/preface.rst +++ b/docs/configuration/preface.rst @@ -35,5 +35,5 @@ Alternatively, for default settings, the following should to the trick. Validating ---------- -The config can be validated using the ``zrepl test`` subcommand. +The config can be validated using the ``zrepl configcheck`` subcommand. From f6cf23779fcfe74f2f60db104d963f5ff69f7c33 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 13 Oct 2018 15:55:54 +0200 Subject: [PATCH 166/167] docs: Remove stale TIP for dry-run zrepl test subcommand. Won't make it to 0.1 --- docs/configuration/prune.rst | 3 --- 1 file changed, 3 deletions(-) diff --git a/docs/configuration/prune.rst b/docs/configuration/prune.rst index 15a6b5d..7aaacad 100644 --- a/docs/configuration/prune.rst +++ b/docs/configuration/prune.rst @@ -45,9 +45,6 @@ Example Configuration: regex: "^zrepl_.*" # manually created snapshots will be kept forever on receiver -.. TIP:: - - You can perform a dry-run of a job's pruning policy using the ``zrepl test`` subcommand. .. ATTENTION:: From 87c89578894f20b4d23bd8e1f561fa9ec3370099 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 13 Oct 2018 16:00:13 +0200 Subject: [PATCH 167/167] build: fixup be962998bae4911cb20ff3dc434d990eac44070d: broken makefile --- Makefile | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/Makefile b/Makefile index f66192d..bcaec3c 100644 --- a/Makefile +++ b/Makefile @@ -5,15 +5,16 @@ ROOT := github.com/zrepl/zrepl SUBPKGS += client SUBPKGS += config SUBPKGS += daemon -SUBPKGS += daemon/connecter SUBPKGS += daemon/filters SUBPKGS += daemon/job SUBPKGS += daemon/logging SUBPKGS += daemon/nethelpers SUBPKGS += daemon/pruner -SUBPKGS += daemon/serve SUBPKGS += daemon/snapper SUBPKGS += daemon/streamrpcconfig +SUBPKGS += daemon/transport +SUBPKGS += daemon/transport/connecter +SUBPKGS += daemon/transport/serve SUBPKGS += endpoint SUBPKGS += logger SUBPKGS += pruning