mirror of
https://github.com/zrepl/zrepl.git
synced 2025-04-15 15:09:00 +02:00
pruner: improve cancellation + error handling strategy
Pruner now backs off as soon as there is an error, making that error the Error field in the pruner report. The error is also stored in the specific *fs that failed, and we maintain an error counter per *fs to de-prioritize those fs that failed. Like with replication, the de-prioritization on errors is to avoid ' getting stuck' with an individual filesystem until the watchdog hits.
This commit is contained in:
parent
50c1549865
commit
438f950be3
@ -391,8 +391,8 @@ func (t *tui) renderPrunerReport(r *pruner.Report) {
|
|||||||
for _, fs := range all {
|
for _, fs := range all {
|
||||||
t.write(rightPad(fs.Filesystem, maxFSname, " "))
|
t.write(rightPad(fs.Filesystem, maxFSname, " "))
|
||||||
t.write(" ")
|
t.write(" ")
|
||||||
if fs.Error != "" {
|
if fs.LastError != "" {
|
||||||
t.printf("ERROR: %s\n", fs.Error) // whitespace is padding
|
t.printf("ERROR (%d): %s\n", fs.LastError) // whitespace is padding
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -13,6 +13,7 @@ import (
|
|||||||
"github.com/zrepl/zrepl/util/watchdog"
|
"github.com/zrepl/zrepl/util/watchdog"
|
||||||
"net"
|
"net"
|
||||||
"sort"
|
"sort"
|
||||||
|
"strings"
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
)
|
)
|
||||||
@ -69,8 +70,7 @@ type Pruner struct {
|
|||||||
err error
|
err error
|
||||||
|
|
||||||
// State Exec
|
// State Exec
|
||||||
prunePending []*fs
|
execQueue *execQueue
|
||||||
pruneCompleted []*fs
|
|
||||||
}
|
}
|
||||||
|
|
||||||
type PrunerFactory struct {
|
type PrunerFactory struct {
|
||||||
@ -223,7 +223,8 @@ type Report struct {
|
|||||||
type FSReport struct {
|
type FSReport struct {
|
||||||
Filesystem string
|
Filesystem string
|
||||||
SnapshotList, DestroyList []SnapshotReport
|
SnapshotList, DestroyList []SnapshotReport
|
||||||
Error string
|
ErrorCount int
|
||||||
|
LastError string
|
||||||
}
|
}
|
||||||
|
|
||||||
type SnapshotReport struct {
|
type SnapshotReport struct {
|
||||||
@ -247,17 +248,8 @@ func (p *Pruner) Report() *Report {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if p.state & Plan|PlanWait == 0 {
|
if p.execQueue != nil {
|
||||||
return &r
|
r.Pending, r.Completed = p.execQueue.Report()
|
||||||
}
|
|
||||||
|
|
||||||
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
|
return &r
|
||||||
@ -289,14 +281,11 @@ type fs struct {
|
|||||||
destroyList []pruning.Snapshot
|
destroyList []pruning.Snapshot
|
||||||
|
|
||||||
mtx sync.RWMutex
|
mtx sync.RWMutex
|
||||||
// for Plan
|
|
||||||
err error
|
|
||||||
}
|
|
||||||
|
|
||||||
func (f *fs) Update(err error) {
|
// only during Exec state, also used by execQueue
|
||||||
f.mtx.Lock()
|
execErrLast error
|
||||||
defer f.mtx.Unlock()
|
execErrCount int
|
||||||
f.err = err
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (f *fs) Report() FSReport {
|
func (f *fs) Report() FSReport {
|
||||||
@ -305,8 +294,9 @@ func (f *fs) Report() FSReport {
|
|||||||
|
|
||||||
r := FSReport{}
|
r := FSReport{}
|
||||||
r.Filesystem = f.path
|
r.Filesystem = f.path
|
||||||
if f.err != nil {
|
r.ErrorCount = f.execErrCount
|
||||||
r.Error = f.err.Error()
|
if f.execErrLast != nil {
|
||||||
|
r.LastError = f.execErrLast.Error()
|
||||||
}
|
}
|
||||||
|
|
||||||
r.SnapshotList = make([]SnapshotReport, len(f.snaps))
|
r.SnapshotList = make([]SnapshotReport, len(f.snaps))
|
||||||
@ -379,7 +369,6 @@ func statePlan(a *args, u updater) state {
|
|||||||
}
|
}
|
||||||
|
|
||||||
pfss := make([]*fs, len(tfss))
|
pfss := make([]*fs, len(tfss))
|
||||||
fsloop:
|
|
||||||
for i, tfs := range tfss {
|
for i, tfs := range tfss {
|
||||||
|
|
||||||
l := GetLogger(ctx).WithField("fs", tfs.Path)
|
l := GetLogger(ctx).WithField("fs", tfs.Path)
|
||||||
@ -394,12 +383,8 @@ fsloop:
|
|||||||
tfsvs, err := target.ListFilesystemVersions(ctx, tfs.Path)
|
tfsvs, err := target.ListFilesystemVersions(ctx, tfs.Path)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
l.WithError(err).Error("cannot list filesystem versions")
|
l.WithError(err).Error("cannot list filesystem versions")
|
||||||
if shouldRetry(err) {
|
|
||||||
return onErr(u, err)
|
return onErr(u, err)
|
||||||
}
|
}
|
||||||
pfs.err = err
|
|
||||||
continue fsloop
|
|
||||||
}
|
|
||||||
pfs.snaps = make([]pruning.Snapshot, 0, len(tfsvs))
|
pfs.snaps = make([]pruning.Snapshot, 0, len(tfsvs))
|
||||||
|
|
||||||
rcReq := &pdu.ReplicationCursorReq{
|
rcReq := &pdu.ReplicationCursorReq{
|
||||||
@ -411,16 +396,11 @@ fsloop:
|
|||||||
rc, err := receiver.ReplicationCursor(ctx, rcReq)
|
rc, err := receiver.ReplicationCursor(ctx, rcReq)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
l.WithError(err).Error("cannot get replication cursor")
|
l.WithError(err).Error("cannot get replication cursor")
|
||||||
if shouldRetry(err) {
|
|
||||||
return onErr(u, err)
|
return onErr(u, err)
|
||||||
}
|
}
|
||||||
pfs.err = err
|
|
||||||
continue fsloop
|
|
||||||
}
|
|
||||||
if rc.GetError() != "" {
|
if rc.GetError() != "" {
|
||||||
l.WithField("reqErr", rc.GetError()).Error("cannot get replication cursor")
|
l.WithField("reqErr", rc.GetError()).Error("cannot get replication cursor")
|
||||||
pfs.err = fmt.Errorf("%s", rc.GetError())
|
return onErr(u, err)
|
||||||
continue fsloop
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -445,9 +425,11 @@ fsloop:
|
|||||||
}
|
}
|
||||||
creation, err := tfsv.CreationAsTime()
|
creation, err := tfsv.CreationAsTime()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
pfs.err = fmt.Errorf("%s%s has invalid creation date: %s", tfs, tfsv.RelName(), err)
|
err := fmt.Errorf("%s%s has invalid creation date: %s", tfs, tfsv.RelName(), err)
|
||||||
l.WithError(pfs.err).Error("")
|
l.WithError(err).
|
||||||
continue fsloop
|
WithField("tfsv", tfsv.RelName()).
|
||||||
|
Error("error with fileesystem version")
|
||||||
|
return onErr(u, err)
|
||||||
}
|
}
|
||||||
// note that we cannot use CreateTXG because target and receiver could be on different pools
|
// note that we cannot use CreateTXG because target and receiver could be on different pools
|
||||||
atCursor := tfsv.Guid == rc.GetGuid()
|
atCursor := tfsv.Guid == rc.GetGuid()
|
||||||
@ -459,9 +441,9 @@ fsloop:
|
|||||||
})
|
})
|
||||||
}
|
}
|
||||||
if preCursor {
|
if preCursor {
|
||||||
pfs.err = fmt.Errorf("replication cursor not found in prune target filesystem versions")
|
err := fmt.Errorf("replication cursor not found in prune target filesystem versions")
|
||||||
l.WithError(pfs.err).Error("")
|
l.Error(err.Error())
|
||||||
continue fsloop
|
return onErr(u, err)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Apply prune rules
|
// Apply prune rules
|
||||||
@ -471,12 +453,9 @@ fsloop:
|
|||||||
|
|
||||||
return u(func(pruner *Pruner) {
|
return u(func(pruner *Pruner) {
|
||||||
pruner.Progress.MadeProgress()
|
pruner.Progress.MadeProgress()
|
||||||
|
pruner.execQueue = newExecQueue(len(pfss))
|
||||||
for _, pfs := range pfss {
|
for _, pfs := range pfss {
|
||||||
if pfs.err != nil {
|
pruner.execQueue.Put(pfs, nil, false)
|
||||||
pruner.pruneCompleted = append(pruner.pruneCompleted, pfs)
|
|
||||||
} else {
|
|
||||||
pruner.prunePending = append(pruner.prunePending, pfs)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
pruner.state = Exec
|
pruner.state = Exec
|
||||||
}).statefunc()
|
}).statefunc()
|
||||||
@ -486,17 +465,15 @@ func stateExec(a *args, u updater) state {
|
|||||||
|
|
||||||
var pfs *fs
|
var pfs *fs
|
||||||
state := u(func(pruner *Pruner) {
|
state := u(func(pruner *Pruner) {
|
||||||
if len(pruner.prunePending) == 0 {
|
pfs = pruner.execQueue.Pop()
|
||||||
|
if pfs == nil {
|
||||||
nextState := Done
|
nextState := Done
|
||||||
for _, pfs := range pruner.pruneCompleted {
|
if pruner.execQueue.HasCompletedFSWithErrors() {
|
||||||
if pfs.err != nil {
|
|
||||||
nextState = ErrPerm
|
nextState = ErrPerm
|
||||||
}
|
}
|
||||||
}
|
|
||||||
pruner.state = nextState
|
pruner.state = nextState
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
pfs = pruner.prunePending[0]
|
|
||||||
})
|
})
|
||||||
if state != Exec {
|
if state != Exec {
|
||||||
return state.statefunc()
|
return state.statefunc()
|
||||||
@ -510,22 +487,62 @@ func stateExec(a *args, u updater) state {
|
|||||||
WithField("destroy_snap", destroyList[i].Name).
|
WithField("destroy_snap", destroyList[i].Name).
|
||||||
Debug("policy destroys snapshot")
|
Debug("policy destroys snapshot")
|
||||||
}
|
}
|
||||||
pfs.Update(nil)
|
|
||||||
req := pdu.DestroySnapshotsReq{
|
req := pdu.DestroySnapshotsReq{
|
||||||
Filesystem: pfs.path,
|
Filesystem: pfs.path,
|
||||||
Snapshots: destroyList,
|
Snapshots: destroyList,
|
||||||
}
|
}
|
||||||
_, err := a.target.DestroySnapshots(a.ctx, &req)
|
GetLogger(a.ctx).WithField("fs", pfs.path).Debug("destroying snapshots")
|
||||||
pfs.Update(err)
|
res, err := a.target.DestroySnapshots(a.ctx, &req)
|
||||||
if err != nil && shouldRetry(err) {
|
if err != nil {
|
||||||
|
u(func(pruner *Pruner) {
|
||||||
|
pruner.execQueue.Put(pfs, err, false)
|
||||||
|
})
|
||||||
|
return onErr(u, err)
|
||||||
|
}
|
||||||
|
// check if all snapshots were destroyed
|
||||||
|
destroyResults := make(map[string]*pdu.DestroySnapshotRes)
|
||||||
|
for _, fsres := range res.Results {
|
||||||
|
destroyResults[fsres.Snapshot.Name] = fsres
|
||||||
|
}
|
||||||
|
err = nil
|
||||||
|
destroyFails := make([]*pdu.DestroySnapshotRes, 0)
|
||||||
|
for _, reqDestroy := range destroyList {
|
||||||
|
res, ok := destroyResults[reqDestroy.Name]
|
||||||
|
if !ok {
|
||||||
|
err = fmt.Errorf("missing destroy-result for %s", reqDestroy.RelName())
|
||||||
|
break
|
||||||
|
} else if res.Error != "" {
|
||||||
|
destroyFails = append(destroyFails, res)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if err == nil && len(destroyFails) > 0 {
|
||||||
|
names := make([]string, len(destroyFails))
|
||||||
|
pairs := make([]string, len(destroyFails))
|
||||||
|
allSame := true
|
||||||
|
lastMsg := destroyFails[0].Error
|
||||||
|
for i := 0; i < len(destroyFails); i++{
|
||||||
|
allSame = allSame && destroyFails[i].Error == lastMsg
|
||||||
|
relname := destroyFails[i].Snapshot.RelName()
|
||||||
|
names[i] = relname
|
||||||
|
pairs[i] = fmt.Sprintf("(%s: %s)", relname, destroyFails[i].Error)
|
||||||
|
}
|
||||||
|
if allSame {
|
||||||
|
err = fmt.Errorf("destroys failed %s: %s",
|
||||||
|
strings.Join(names, ", "), lastMsg)
|
||||||
|
} else {
|
||||||
|
err = fmt.Errorf("destroys failed: %s", strings.Join(pairs, ", "))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
u(func(pruner *Pruner) {
|
||||||
|
pruner.execQueue.Put(pfs, err, err == nil)
|
||||||
|
})
|
||||||
|
if err != nil {
|
||||||
|
GetLogger(a.ctx).WithError(err).Error("target could not destroy snapshots")
|
||||||
return onErr(u, err)
|
return onErr(u, err)
|
||||||
}
|
}
|
||||||
// if it's not retryable, treat is like as being done
|
|
||||||
|
|
||||||
return u(func(pruner *Pruner) {
|
return u(func(pruner *Pruner) {
|
||||||
pruner.Progress.MadeProgress()
|
pruner.Progress.MadeProgress()
|
||||||
pruner.pruneCompleted = append(pruner.pruneCompleted, pfs)
|
|
||||||
pruner.prunePending = pruner.prunePending[1:]
|
|
||||||
}).statefunc()
|
}).statefunc()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
89
daemon/pruner/pruner_queue.go
Normal file
89
daemon/pruner/pruner_queue.go
Normal file
@ -0,0 +1,89 @@
|
|||||||
|
package pruner
|
||||||
|
|
||||||
|
import (
|
||||||
|
"sort"
|
||||||
|
"strings"
|
||||||
|
"sync"
|
||||||
|
)
|
||||||
|
|
||||||
|
type execQueue struct {
|
||||||
|
mtx sync.Mutex
|
||||||
|
pending, completed []*fs
|
||||||
|
}
|
||||||
|
|
||||||
|
func newExecQueue(cap int) *execQueue {
|
||||||
|
q := execQueue{
|
||||||
|
pending: make([]*fs, 0, cap),
|
||||||
|
completed: make([]*fs, 0, cap),
|
||||||
|
}
|
||||||
|
return &q
|
||||||
|
}
|
||||||
|
|
||||||
|
func (q *execQueue) Report() (pending, completed []FSReport) {
|
||||||
|
q.mtx.Lock()
|
||||||
|
defer q.mtx.Unlock()
|
||||||
|
|
||||||
|
pending = make([]FSReport, len(q.pending))
|
||||||
|
for i, fs := range q.pending {
|
||||||
|
pending[i] = fs.Report()
|
||||||
|
}
|
||||||
|
completed = make([]FSReport, len(q.completed))
|
||||||
|
for i, fs := range q.completed {
|
||||||
|
completed[i] = fs.Report()
|
||||||
|
}
|
||||||
|
|
||||||
|
return pending, completed
|
||||||
|
}
|
||||||
|
|
||||||
|
func (q *execQueue) HasCompletedFSWithErrors() bool {
|
||||||
|
q.mtx.Lock()
|
||||||
|
defer q.mtx.Unlock()
|
||||||
|
for _, fs := range q.completed {
|
||||||
|
if fs.execErrLast != nil {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
func (q *execQueue) Pop() *fs {
|
||||||
|
if len(q.pending) == 0 {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
fs := q.pending[0]
|
||||||
|
q.pending = q.pending[1:]
|
||||||
|
return fs
|
||||||
|
}
|
||||||
|
|
||||||
|
func(q *execQueue) Put(fs *fs, err error, done bool) {
|
||||||
|
fs.mtx.Lock()
|
||||||
|
fs.execErrLast = err
|
||||||
|
if err != nil {
|
||||||
|
fs.execErrCount++
|
||||||
|
}
|
||||||
|
if done || (err != nil && !shouldRetry(fs.execErrLast)) {
|
||||||
|
fs.mtx.Unlock()
|
||||||
|
q.mtx.Lock()
|
||||||
|
q.completed = append(q.completed, fs)
|
||||||
|
q.mtx.Unlock()
|
||||||
|
return
|
||||||
|
}
|
||||||
|
fs.mtx.Unlock()
|
||||||
|
|
||||||
|
q.mtx.Lock()
|
||||||
|
// inefficient priority q
|
||||||
|
q.pending = append(q.pending, fs)
|
||||||
|
sort.SliceStable(q.pending, func(i, j int) bool {
|
||||||
|
q.pending[i].mtx.Lock()
|
||||||
|
defer q.pending[i].mtx.Unlock()
|
||||||
|
q.pending[j].mtx.Lock()
|
||||||
|
defer q.pending[j].mtx.Unlock()
|
||||||
|
if q.pending[i].execErrCount != q.pending[j].execErrCount {
|
||||||
|
return q.pending[i].execErrCount < q.pending[j].execErrCount
|
||||||
|
}
|
||||||
|
return strings.Compare(q.pending[i].path, q.pending[j].path) == -1
|
||||||
|
})
|
||||||
|
q.mtx.Unlock()
|
||||||
|
|
||||||
|
|
||||||
|
}
|
@ -133,16 +133,14 @@ func TestPruner_Prune(t *testing.T) {
|
|||||||
},
|
},
|
||||||
listVersionsErrs: map[string][]error{
|
listVersionsErrs: map[string][]error{
|
||||||
"zroot/foo": {
|
"zroot/foo": {
|
||||||
stubNetErr{msg: "fakeerror1", temporary: true}, // should be classified as temporaty
|
stubNetErr{msg: "fakeerror1", temporary: true},
|
||||||
stubNetErr{msg: "fakeerror2", temporary: true,},
|
stubNetErr{msg: "fakeerror2", temporary: true,},
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
destroyErrs: map[string][]error{
|
destroyErrs: map[string][]error{
|
||||||
"zroot/foo": {
|
"zroot/baz": {
|
||||||
fmt.Errorf("permanent error"),
|
stubNetErr{msg: "fakeerror3", temporary: true}, // first error puts it back in the queue
|
||||||
},
|
stubNetErr{msg:"permanent error"}, // so it will be last when pruner gives up due to permanent err
|
||||||
"zroot/bar": {
|
|
||||||
stubNetErr{msg: "fakeerror3", temporary: true},
|
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
destroyed: make(map[string][]string),
|
destroyed: make(map[string][]string),
|
||||||
@ -178,9 +176,6 @@ func TestPruner_Prune(t *testing.T) {
|
|||||||
"zroot/foo": {
|
"zroot/foo": {
|
||||||
stubNetErr{msg: "fakeerror4", temporary: true},
|
stubNetErr{msg: "fakeerror4", temporary: true},
|
||||||
},
|
},
|
||||||
"zroot/baz": {
|
|
||||||
fmt.Errorf("permanent error2"),
|
|
||||||
},
|
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -199,9 +194,8 @@ func TestPruner_Prune(t *testing.T) {
|
|||||||
p.Prune()
|
p.Prune()
|
||||||
|
|
||||||
exp := map[string][]string{
|
exp := map[string][]string{
|
||||||
|
"zroot/foo": {"drop_c"},
|
||||||
"zroot/bar": {"drop_g"},
|
"zroot/bar": {"drop_g"},
|
||||||
// drop_c is prohibited by failing destroy
|
|
||||||
// drop_i is prohibiteed by failing ReplicationCursor call
|
|
||||||
}
|
}
|
||||||
|
|
||||||
assert.Equal(t, exp, target.destroyed)
|
assert.Equal(t, exp, target.destroyed)
|
||||||
|
Loading…
Reference in New Issue
Block a user