zfs: userrefs, platformtests for ListFilesystemVersions and ListMapping (likely needs fixup from next commit)

This commit is contained in:
Christian Schwarz
2020-04-05 19:12:20 +02:00
parent e82aea5862
commit b16a9ede46
12 changed files with 381 additions and 90 deletions

View File

@@ -211,17 +211,15 @@ func doMigrateReplicationCursorFS(ctx context.Context, v1CursorJobs []job.Job, f
} }
fmt.Printf("identified owning job %q\n", owningJob.Name()) fmt.Printf("identified owning job %q\n", owningJob.Name())
versions, err := zfs.ZFSListFilesystemVersions(fs, nil) bookmarks, err := zfs.ZFSListFilesystemVersions(fs, zfs.ListFilesystemVersionsOptions{
Types: zfs.Bookmarks,
})
if err != nil { if err != nil {
return errors.Wrapf(err, "list filesystem versions of %q", fs.ToString()) return errors.Wrapf(err, "list filesystem versions of %q", fs.ToString())
} }
var oldCursor *zfs.FilesystemVersion var oldCursor *zfs.FilesystemVersion
for i, fsv := range versions { for i, fsv := range bookmarks {
if fsv.Type != zfs.Bookmark {
continue
}
_, _, err := endpoint.ParseReplicationCursorBookmarkName(fsv.ToAbsPath(fs)) _, _, err := endpoint.ParseReplicationCursorBookmarkName(fsv.ToAbsPath(fs))
if err != endpoint.ErrV1ReplicationCursor { if err != endpoint.ErrV1ReplicationCursor {
continue continue
@@ -232,7 +230,7 @@ func doMigrateReplicationCursorFS(ctx context.Context, v1CursorJobs []job.Job, f
return errors.Wrap(err, "multiple filesystem versions identified as v1 replication cursors") return errors.Wrap(err, "multiple filesystem versions identified as v1 replication cursors")
} }
oldCursor = &versions[i] oldCursor = &bookmarks[i]
} }

View File

@@ -1,41 +0,0 @@
package filters
import (
"strings"
"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
}
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
}

View File

@@ -485,7 +485,10 @@ var findSyncPointFSNoFilesystemVersionsErr = fmt.Errorf("no filesystem versions"
func findSyncPointFSNextOptimalSnapshotTime(l Logger, now time.Time, interval time.Duration, prefix string, d *zfs.DatasetPath) (time.Time, error) { func findSyncPointFSNextOptimalSnapshotTime(l Logger, now time.Time, interval time.Duration, prefix string, d *zfs.DatasetPath) (time.Time, error) {
fsvs, err := zfs.ZFSListFilesystemVersions(d, filters.NewTypedPrefixFilter(prefix, zfs.Snapshot)) fsvs, err := zfs.ZFSListFilesystemVersions(d, zfs.ListFilesystemVersionsOptions{
Types: zfs.Snapshots,
ShortnamePrefix: prefix,
})
if err != nil { if err != nil {
return time.Time{}, errors.Wrap(err, "list filesystem versions") return time.Time{}, errors.Wrap(err, "list filesystem versions")
} }

View File

@@ -96,7 +96,7 @@ func (s *Sender) ListFilesystemVersions(ctx context.Context, r *pdu.ListFilesyst
if err != nil { if err != nil {
return nil, err return nil, err
} }
fsvs, err := zfs.ZFSListFilesystemVersions(lp, nil) fsvs, err := zfs.ZFSListFilesystemVersions(lp, zfs.ListFilesystemVersionsOptions{})
if err != nil { if err != nil {
return nil, err return nil, err
} }
@@ -130,6 +130,7 @@ func (p *Sender) HintMostRecentCommonAncestor(ctx context.Context, r *pdu.HintMo
TryReleaseStepStaleFS(ctx, fs, p.jobId) TryReleaseStepStaleFS(ctx, fs, p.jobId)
return &pdu.HintMostRecentCommonAncestorRes{}, nil return &pdu.HintMostRecentCommonAncestorRes{}, nil
} }
// we were hinted a specific common ancestor // we were hinted a specific common ancestor
mostRecentVersion, err := sendArgsFromPDUAndValidateExistsAndGetVersion(ctx, fs, r.GetSenderVersion()) mostRecentVersion, err := sendArgsFromPDUAndValidateExistsAndGetVersion(ctx, fs, r.GetSenderVersion())
@@ -588,8 +589,9 @@ func (s *Receiver) ListFilesystemVersions(ctx context.Context, req *pdu.ListFile
if err != nil { if err != nil {
return nil, err return nil, err
} }
// TODO share following code with sender
fsvs, err := zfs.ZFSListFilesystemVersions(lp, nil) fsvs, err := zfs.ZFSListFilesystemVersions(lp, zfs.ListFilesystemVersionsOptions{})
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@@ -24,6 +24,7 @@ type Stmt interface {
type Op string type Op string
const ( const (
Comment Op = "#"
AssertExists Op = "!E" AssertExists Op = "!E"
AssertNotExists Op = "!N" AssertNotExists Op = "!N"
Add Op = "+" Add Op = "+"
@@ -102,6 +103,26 @@ func (o *SnapOp) Run(ctx context.Context, e Execer) error {
} }
} }
type BookmarkOp struct {
Op Op
Existing string
Bookmark string
}
func (o *BookmarkOp) Run(ctx context.Context, e Execer) error {
switch o.Op {
case Add:
return e.RunExpectSuccessNoOutput(ctx, "zfs", "bookmark", o.Existing, o.Bookmark)
case Del:
if o.Existing != "" {
panic("existing must be empty for destroy, got " + o.Existing)
}
return e.RunExpectSuccessNoOutput(ctx, "zfs", "destroy", o.Bookmark)
default:
panic(o.Op)
}
}
type RunOp struct { type RunOp struct {
RootDS string RootDS string
Script string Script string
@@ -255,16 +276,26 @@ nextLine:
op = AssertExists op = AssertExists
case string(AssertNotExists): case string(AssertNotExists):
op = AssertNotExists op = AssertNotExists
case string(Comment):
op = Comment
continue
default: default:
return nil, &LineError{scan.Text(), fmt.Sprintf("invalid op %q", comps.Text())} return nil, &LineError{scan.Text(), fmt.Sprintf("invalid op %q", comps.Text())}
} }
// FS / SNAP // FS / SNAP / BOOKMARK
if err := expectMoreTokens(); err != nil { if err := expectMoreTokens(); err != nil {
return nil, err return nil, err
} }
if strings.ContainsAny(comps.Text(), "@") { if strings.ContainsAny(comps.Text(), "@") {
stmts = append(stmts, &SnapOp{Op: op, Path: fmt.Sprintf("%s/%s", rootds, comps.Text())}) stmts = append(stmts, &SnapOp{Op: op, Path: fmt.Sprintf("%s/%s", rootds, comps.Text())})
} else if strings.ContainsAny(comps.Text(), "#") {
bookmark := fmt.Sprintf("%s/%s", rootds, comps.Text())
if err := expectMoreTokens(); err != nil {
return nil, err
}
existing := fmt.Sprintf("%s/%s", rootds, comps.Text())
stmts = append(stmts, &BookmarkOp{Op: op, Existing: existing, Bookmark: bookmark})
} else { } else {
// FS // FS
fs := comps.Text() fs := comps.Text()

View File

@@ -5,6 +5,7 @@ import (
"math/rand" "math/rand"
"os" "os"
"path" "path"
"sort"
"strings" "strings"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
@@ -150,3 +151,26 @@ func makeResumeSituation(ctx *platformtest.Context, src dummySnapshotSituation,
return situation return situation
} }
func versionRelnamesSorted(versions []zfs.FilesystemVersion) []string {
var vstrs []string
for _, v := range versions {
vstrs = append(vstrs, v.RelName())
}
sort.Sort(sort.StringSlice(vstrs))
return vstrs
}
func datasetToStringSortedTrimPrefix(prefix *zfs.DatasetPath, paths []*zfs.DatasetPath) []string {
var pstrs []string
for _, p := range paths {
trimmed := p.Copy()
trimmed.TrimPrefix(prefix)
if trimmed.Length() == 0 {
continue
}
pstrs = append(pstrs, trimmed.ToString())
}
sort.Sort(sort.StringSlice(pstrs))
return pstrs
}

View File

@@ -0,0 +1,178 @@
package tests
import (
"fmt"
"sort"
"strings"
"github.com/stretchr/testify/require"
"github.com/zrepl/zrepl/platformtest"
"github.com/zrepl/zrepl/zfs"
)
func ListFilesystemVersionsTypeFilteringAndPrefix(t *platformtest.Context) {
platformtest.Run(t, platformtest.PanicErr, t.RootDataset, `
DESTROYROOT
CREATEROOT
+ "foo bar"
+ "foo bar@foo 1"
+ "foo bar#foo 1" "foo bar@foo 1"
+ "foo bar#bookfoo 1" "foo bar@foo 1"
+ "foo bar@foo 2"
+ "foo bar#foo 2" "foo bar@foo 2"
+ "foo bar#bookfoo 2" "foo bar@foo 2"
+ "foo bar@blup 1"
+ "foo bar#blup 1" "foo bar@blup 1"
+ "foo bar@ foo with leading whitespace"
# repeat the whole thing for a child dataset to make sure we disable recursion
+ "foo bar/child dataset"
+ "foo bar/child dataset@foo 1"
+ "foo bar/child dataset#foo 1" "foo bar/child dataset@foo 1"
+ "foo bar/child dataset#bookfoo 1" "foo bar/child dataset@foo 1"
+ "foo bar/child dataset@foo 2"
+ "foo bar/child dataset#foo 2" "foo bar/child dataset@foo 2"
+ "foo bar/child dataset#bookfoo 2" "foo bar/child dataset@foo 2"
+ "foo bar/child dataset@blup 1"
+ "foo bar/child dataset#blup 1" "foo bar/child dataset@blup 1"
+ "foo bar/child dataset@ foo with leading whitespace"
`)
fs := fmt.Sprintf("%s/foo bar", t.RootDataset)
// no options := all types
vs, err := zfs.ZFSListFilesystemVersions(mustDatasetPath(fs), zfs.ListFilesystemVersionsOptions{})
require.NoError(t, err)
require.Equal(t, []string{
"#blup 1", "#bookfoo 1", "#bookfoo 2", "#foo 1", "#foo 2",
"@ foo with leading whitespace", "@blup 1", "@foo 1", "@foo 2",
}, versionRelnamesSorted(vs))
// just snapshots
vs, err = zfs.ZFSListFilesystemVersions(mustDatasetPath(fs), zfs.ListFilesystemVersionsOptions{
Types: zfs.Snapshots,
})
require.NoError(t, err)
require.Equal(t, []string{"@ foo with leading whitespace", "@blup 1", "@foo 1", "@foo 2"}, versionRelnamesSorted(vs))
// just bookmarks
vs, err = zfs.ZFSListFilesystemVersions(mustDatasetPath(fs), zfs.ListFilesystemVersionsOptions{
Types: zfs.Bookmarks,
})
require.NoError(t, err)
require.Equal(t, []string{"#blup 1", "#bookfoo 1", "#bookfoo 2", "#foo 1", "#foo 2"}, versionRelnamesSorted(vs))
// just with prefix foo
vs, err = zfs.ZFSListFilesystemVersions(mustDatasetPath(fs), zfs.ListFilesystemVersionsOptions{
ShortnamePrefix: "foo",
})
require.NoError(t, err)
require.Equal(t, []string{"#foo 1", "#foo 2", "@foo 1", "@foo 2"}, versionRelnamesSorted(vs))
}
func ListFilesystemVersionsZeroExistIsNotAnError(t *platformtest.Context) {
platformtest.Run(t, platformtest.PanicErr, t.RootDataset, `
DESTROYROOT
CREATEROOT
+ "foo bar"
`)
fs := fmt.Sprintf("%s/foo bar", t.RootDataset)
vs, err := zfs.ZFSListFilesystemVersions(mustDatasetPath(fs), zfs.ListFilesystemVersionsOptions{})
require.Empty(t, vs)
require.NoError(t, err)
dsne, ok := err.(*zfs.DatasetDoesNotExist)
require.True(t, ok)
require.Equal(t, fs, dsne.Path)
}
func ListFilesystemVersionsFilesystemNotExist(t *platformtest.Context) {
platformtest.Run(t, platformtest.PanicErr, t.RootDataset, `
DESTROYROOT
CREATEROOT
`)
nonexistentFS := fmt.Sprintf("%s/not existent", t.RootDataset)
vs, err := zfs.ZFSListFilesystemVersions(mustDatasetPath(nonexistentFS), zfs.ListFilesystemVersionsOptions{})
require.Empty(t, vs)
require.Error(t, err)
t.Logf("err = %T\n%s", err, err)
dsne, ok := err.(*zfs.DatasetDoesNotExist)
require.True(t, ok)
require.Equal(t, nonexistentFS, dsne.Path)
}
func ListFilesystemVersionsUserrefs(t *platformtest.Context) {
platformtest.Run(t, platformtest.PanicErr, t.RootDataset, `
DESTROYROOT
CREATEROOT
+ "foo bar"
+ "foo bar@snap 1"
+ "foo bar#snap 1" "foo bar@snap 1"
+ "foo bar@snap 2"
+ "foo bar#snap 2" "foo bar@snap 2"
R zfs hold zrepl_platformtest "${ROOTDS}/foo bar@snap 2"
+ "foo bar@snap 3"
+ "foo bar#snap 3" "foo bar@snap 3"
R zfs hold zrepl_platformtest "${ROOTDS}/foo bar@snap 3"
R zfs hold zrepl_platformtest_second_hold "${ROOTDS}/foo bar@snap 3"
+ "foo bar@snap 4"
+ "foo bar#snap 4" "foo bar@snap 4"
+ "foo bar/child datset"
+ "foo bar/child datset@snap 1"
+ "foo bar/child datset#snap 1" "foo bar/child datset@snap 1"
+ "foo bar/child datset@snap 2"
+ "foo bar/child datset#snap 2" "foo bar/child datset@snap 2"
R zfs hold zrepl_platformtest "${ROOTDS}/foo bar/child datset@snap 2"
+ "foo bar/child datset@snap 3"
+ "foo bar/child datset#snap 3" "foo bar/child datset@snap 3"
R zfs hold zrepl_platformtest "${ROOTDS}/foo bar/child datset@snap 3"
R zfs hold zrepl_platformtest_second_hold "${ROOTDS}/foo bar/child datset@snap 3"
+ "foo bar/child datset@snap 4"
+ "foo bar/child datset#snap 4" "foo bar/child datset@snap 4"
`)
fs := fmt.Sprintf("%s/foo bar", t.RootDataset)
vs, err := zfs.ZFSListFilesystemVersions(mustDatasetPath(fs), zfs.ListFilesystemVersionsOptions{})
require.NoError(t, err)
type expectation struct {
relName string
userrefs zfs.OptionUint64
}
expect := []expectation{
{"#snap 1", zfs.OptionUint64{Valid: false}},
{"#snap 2", zfs.OptionUint64{Valid: false}},
{"#snap 3", zfs.OptionUint64{Valid: false}},
{"#snap 4", zfs.OptionUint64{Valid: false}},
{"@snap 1", zfs.OptionUint64{Value: 0, Valid: true}},
{"@snap 2", zfs.OptionUint64{Value: 1, Valid: true}},
{"@snap 3", zfs.OptionUint64{Value: 2, Valid: true}},
{"@snap 4", zfs.OptionUint64{Value: 0, Valid: true}},
}
sort.Slice(vs, func(i, j int) bool {
return strings.Compare(vs[i].RelName(), vs[j].RelName()) < 0
})
var expectRelNames []string
for _, e := range expect {
expectRelNames = append(expectRelNames, e.relName)
}
require.Equal(t, expectRelNames, versionRelnamesSorted(vs))
for i, e := range expect {
require.Equal(t, e.relName, vs[i].RelName())
require.Equal(t, e.userrefs, vs[i].UserRefs)
}
}

View File

@@ -0,0 +1,33 @@
package tests
import (
"strings"
"github.com/stretchr/testify/require"
"github.com/zrepl/zrepl/platformtest"
"github.com/zrepl/zrepl/zfs"
)
func ListFilesystemsNoFilter(t *platformtest.Context) {
platformtest.Run(t, platformtest.PanicErr, t.RootDataset, `
DESTROYROOT
CREATEROOT
R zfs create -V 10M "${ROOTDS}/bar baz"
+ "foo bar"
+ "foo bar/bar blup"
+ "foo bar/blah"
R zfs create -V 10M "${ROOTDS}/foo bar/blah/a volume"
`)
fss, err := zfs.ZFSListMapping(t, zfs.NoFilter())
require.NoError(t, err)
var onlyTestPool []*zfs.DatasetPath
for _, fs := range fss {
if strings.HasPrefix(fs.ToString(), t.RootDataset) {
onlyTestPool = append(onlyTestPool, fs)
}
}
onlyTestPoolStr := datasetToStringSortedTrimPrefix(mustDatasetPath(t.RootDataset), onlyTestPool)
require.Equal(t, []string{"bar baz", "foo bar", "foo bar/bar blup", "foo bar/blah", "foo bar/blah/a volume"}, onlyTestPoolStr)
}

View File

@@ -26,4 +26,9 @@ var Cases = []Case{
SendArgsValidationEncryptedSendOfUnencryptedDatasetForbidden, SendArgsValidationEncryptedSendOfUnencryptedDatasetForbidden,
SendArgsValidationResumeTokenEncryptionMismatchForbidden, SendArgsValidationResumeTokenEncryptionMismatchForbidden,
SendArgsValidationResumeTokenDifferentFilesystemForbidden, SendArgsValidationResumeTokenDifferentFilesystemForbidden,
ListFilesystemVersionsTypeFilteringAndPrefix,
ListFilesystemVersionsFilesystemNotExist,
ListFilesystemVersionsFilesystemNotExist,
ListFilesystemVersionsUserrefs,
ListFilesystemsNoFilter,
} }

View File

@@ -59,7 +59,7 @@ func ZFSListMappingProperties(ctx context.Context, filter DatasetFilter, propert
defer cancel() defer cancel()
rchan := make(chan ZFSListResult) rchan := make(chan ZFSListResult)
go ZFSListChan(ctx, rchan, properties, "-r", "-t", "filesystem,volume") go ZFSListChan(ctx, rchan, properties, nil, "-r", "-t", "filesystem,volume")
datasets = make([]ZFSListMappingPropertiesResult, 0) datasets = make([]ZFSListMappingPropertiesResult, 0)
for r := range rchan { for r := range rchan {

View File

@@ -20,6 +20,30 @@ const (
Snapshot VersionType = "snapshot" Snapshot VersionType = "snapshot"
) )
type VersionTypeSet map[VersionType]bool
var (
AllVersionTypes = VersionTypeSet{
Bookmark: true,
Snapshot: true,
}
Bookmarks = VersionTypeSet{
Bookmark: true,
}
Snapshots = VersionTypeSet{
Snapshot: true,
}
)
func (s VersionTypeSet) zfsListTFlagRepr() string {
var types []string
for t, _ := range s {
types = append(types, t.String())
}
return strings.Join(types, ",")
}
func (s VersionTypeSet) String() string { return s.zfsListTFlagRepr() }
func (t VersionType) DelimiterChar() string { func (t VersionType) DelimiterChar() string {
switch t { switch t {
case Bookmark: case Bookmark:
@@ -71,6 +95,14 @@ type FilesystemVersion struct {
// The time the dataset was created // The time the dataset was created
Creation time.Time Creation time.Time
// userrefs field (snapshots only)
UserRefs OptionUint64
}
type OptionUint64 struct {
Value uint64
Valid bool
} }
func (v FilesystemVersion) GetCreateTXG() uint64 { return v.CreateTXG } func (v FilesystemVersion) GetCreateTXG() uint64 { return v.CreateTXG }
@@ -105,7 +137,7 @@ func (v FilesystemVersion) ToSendArgVersion() ZFSSendArgVersion {
type ParseFilesystemVersionArgs struct { type ParseFilesystemVersionArgs struct {
fullname string fullname string
guid, createtxg, creation string guid, createtxg, creation, userrefs string
} }
func ParseFilesystemVersion(args ParseFilesystemVersionArgs) (v FilesystemVersion, err error) { func ParseFilesystemVersion(args ParseFilesystemVersionArgs) (v FilesystemVersion, err error) {
@@ -132,27 +164,49 @@ func ParseFilesystemVersion(args ParseFilesystemVersionArgs) (v FilesystemVersio
v.Creation = time.Unix(creationUnix, 0) v.Creation = time.Unix(creationUnix, 0)
} }
switch v.Type {
case Bookmark:
if args.userrefs != "-" {
return v, errors.Errorf("expecting %q for bookmark property userrefs, got %q", "-", args.userrefs)
}
v.UserRefs = OptionUint64{Valid: false}
case Snapshot:
if v.UserRefs.Value, err = strconv.ParseUint(args.userrefs, 10, 64); err != nil {
err = errors.Wrapf(err, "cannot parse userrefs %q", args.userrefs)
return v, err
}
v.UserRefs.Valid = true
default:
panic(v.Type)
}
return v, nil return v, nil
} }
type FilesystemVersionFilter interface { type ListFilesystemVersionsOptions struct {
Filter(t VersionType, name string) (accept bool, err error) // the prefix of the version name, without the delimiter char
// empty means any prefix matches
ShortnamePrefix string
// which types should be returned
// nil or len(0) means any prefix matches
Types VersionTypeSet
} }
type closureFilesystemVersionFilter struct { func (o *ListFilesystemVersionsOptions) typesFlagArgs() string {
cb func(t VersionType, name string) (accept bool, err error) if len(o.Types) == 0 {
return AllVersionTypes.zfsListTFlagRepr()
} else {
return o.Types.zfsListTFlagRepr()
}
} }
func (f *closureFilesystemVersionFilter) Filter(t VersionType, name string) (accept bool, err error) { func (o *ListFilesystemVersionsOptions) matches(v FilesystemVersion) bool {
return f.cb(t, name) return (len(o.Types) == 0 || o.Types[v.Type]) && strings.HasPrefix(v.Name, o.ShortnamePrefix)
} }
func FilterFromClosure(cb func(t VersionType, name string) (accept bool, err error)) FilesystemVersionFilter { // returned versions are sorted by createtxg FIXME drop sort by createtxg requirement
return &closureFilesystemVersionFilter{cb} func ZFSListFilesystemVersions(fs *DatasetPath, options ListFilesystemVersionsOptions) (res []FilesystemVersion, err error) {
}
// returned versions are sorted by createtxg
func ZFSListFilesystemVersions(fs *DatasetPath, filter FilesystemVersionFilter) (res []FilesystemVersion, err error) {
listResults := make(chan ZFSListResult) listResults := make(chan ZFSListResult)
promTimer := prometheus.NewTimer(prom.ZFSListFilesystemVersionDuration.WithLabelValues(fs.ToString())) promTimer := prometheus.NewTimer(prom.ZFSListFilesystemVersionDuration.WithLabelValues(fs.ToString()))
@@ -161,9 +215,10 @@ func ZFSListFilesystemVersions(fs *DatasetPath, filter FilesystemVersionFilter)
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
defer cancel() defer cancel()
go ZFSListChan(ctx, listResults, go ZFSListChan(ctx, listResults,
[]string{"name", "guid", "createtxg", "creation"}, []string{"name", "guid", "createtxg", "creation", "userrefs"},
fs,
"-r", "-d", "1", "-r", "-d", "1",
"-t", "bookmark,snapshot", "-t", options.typesFlagArgs(),
"-s", "createtxg", fs.ToString()) "-s", "createtxg", fs.ToString())
res = make([]FilesystemVersion, 0) res = make([]FilesystemVersion, 0)
@@ -182,21 +237,14 @@ func ZFSListFilesystemVersions(fs *DatasetPath, filter FilesystemVersionFilter)
guid: line[1], guid: line[1],
createtxg: line[2], createtxg: line[2],
creation: line[3], creation: line[3],
userrefs: line[4],
} }
v, err := ParseFilesystemVersion(args) v, err := ParseFilesystemVersion(args)
if err != nil { if err != nil {
return nil, err return nil, err
} }
accept := true if options.matches(v) {
if filter != nil {
accept, err = filter.Filter(v.Type, v.Name)
if err != nil {
err = fmt.Errorf("error executing filter: %s", err)
return nil, err
}
}
if accept {
res = append(res, v) res = append(res, v)
} }

View File

@@ -221,9 +221,12 @@ type ZFSListResult struct {
// If no error occurs, it is just closed. // If no error occurs, it is just closed.
// If the operation is cancelled via context, the channel is just closed. // If the operation is cancelled via context, the channel is just closed.
// //
// If notExistHint is not nil and zfs exits with an error,
// the stderr is attempted to be interpreted as a *DatasetDoesNotExist error.
//
// However, if callers do not drain `out` or cancel via `ctx`, the process will leak either running because // However, if callers do not drain `out` or cancel via `ctx`, the process will leak either running because
// IO is pending or as a zombie. // IO is pending or as a zombie.
func ZFSListChan(ctx context.Context, out chan ZFSListResult, properties []string, zfsArgs ...string) { func ZFSListChan(ctx context.Context, out chan ZFSListResult, properties []string, notExistHint *DatasetPath, zfsArgs ...string) {
defer close(out) defer close(out)
args := make([]string, 0, 4+len(zfsArgs)) args := make([]string, 0, 4+len(zfsArgs))
@@ -272,11 +275,21 @@ func ZFSListChan(ctx context.Context, out chan ZFSListResult, properties []strin
} }
} }
if err := cmd.Wait(); err != nil { if err := cmd.Wait(); err != nil {
if err, ok := err.(*exec.ExitError); ok { if _, ok := err.(*exec.ExitError); ok {
enotexist := func() error {
if notExistHint == nil {
return nil
}
return tryDatasetDoesNotExist(notExistHint.ToString(), stderrBuf.Bytes())
}
if err := enotexist(); err != nil {
sendResult(nil, err)
} else {
sendResult(nil, &ZFSError{ sendResult(nil, &ZFSError{
Stderr: stderrBuf.Bytes(), Stderr: stderrBuf.Bytes(),
WaitErr: err, WaitErr: err,
}) })
}
} else { } else {
sendResult(nil, &ZFSError{WaitErr: err}) sendResult(nil, &ZFSError{WaitErr: err})
} }
@@ -1075,15 +1088,12 @@ func ZFSRecv(ctx context.Context, fs string, v *ZFSSendArgVersion, streamCopier
// does not perform a rollback unless `send -R` was used (which we assume hasn't been the case) // does not perform a rollback unless `send -R` was used (which we assume hasn't been the case)
var snaps []FilesystemVersion var snaps []FilesystemVersion
{ {
vs, err := ZFSListFilesystemVersions(fsdp, nil) snaps, err := ZFSListFilesystemVersions(fsdp, ListFilesystemVersionsOptions{
Types: Snapshots,
})
if err != nil { if err != nil {
return fmt.Errorf("cannot list versions for rollback for forced receive: %s", err) return fmt.Errorf("cannot list versions for rollback for forced receive: %s", err)
} }
for _, v := range vs {
if v.Type == Snapshot {
snaps = append(snaps, v)
}
}
sort.Slice(snaps, func(i, j int) bool { sort.Slice(snaps, func(i, j int) bool {
return snaps[i].CreateTXG < snaps[j].CreateTXG return snaps[i].CreateTXG < snaps[j].CreateTXG
}) })