new features: {resumable,encrypted,hold-protected} send-recv, last-received-hold

- **Resumable Send & Recv Support**
  No knobs required, automatically used where supported.
- **Hold-Protected Send & Recv**
  Automatic ZFS holds to ensure that we can always resume a replication step.
- **Encrypted Send & Recv Support** for OpenZFS native encryption.
  Configurable at the job level, i.e., for all filesystems a job is responsible for.
- **Receive-side hold on last received dataset**
  The counterpart to the replication cursor bookmark on the send-side.
  Ensures that incremental replication will always be possible between a sender and receiver.

Design Doc
----------

`replication/design.md` doc describes how we use ZFS holds and bookmarks to ensure that a single replication step is always resumable.

The replication algorithm described in the design doc introduces the notion of job IDs (please read the details on this design doc).
We reuse the job names for job IDs and use `JobID` type to ensure that a job name can be embedded into hold tags, bookmark names, etc.
This might BREAK CONFIG on upgrade.

Protocol Version Bump
---------------------

This commit makes backwards-incompatible changes to the replication/pdu protobufs.
Thus, bump the version number used in the protocol handshake.

Replication Cursor Format Change
--------------------------------

The new replication cursor bookmark format is: `#zrepl_CURSOR_G_${this.GUID}_J_${jobid}`
Including the GUID enables transaction-safe moving-forward of the cursor.
Including the job id enables that multiple sending jobs can send the same filesystem without interfering.
The `zrepl migrate replication-cursor:v1-v2` subcommand can be used to safely destroy old-format cursors once zrepl has created new-format cursors.

Changes in This Commit
----------------------

- package zfs
  - infrastructure for holds
  - infrastructure for resume token decoding
  - implement a variant of OpenZFS's `entity_namecheck` and use it for validation in new code
  - ZFSSendArgs to specify a ZFS send operation
    - validation code protects against malicious resume tokens by checking that the token encodes the same send parameters that the send-side would use if no resume token were available (i.e. same filesystem, `fromguid`, `toguid`)
  - RecvOptions support for `recv -s` flag
  - convert a bunch of ZFS operations to be idempotent
    - achieved through more differentiated error message scraping / additional pre-/post-checks

- package replication/pdu
  - add field for encryption to send request messages
  - add fields for resume handling to send & recv request messages
  - receive requests now contain `FilesystemVersion To` in addition to the filesystem into which the stream should be `recv`d into
    - can use `zfs recv $root_fs/$client_id/path/to/dataset@${To.Name}`, which enables additional validation after recv (i.e. whether `To.Guid` matched what we received in the stream)
    - used to set `last-received-hold`
- package replication/logic
  - introduce `PlannerPolicy` struct, currently only used to configure whether encrypted sends should be requested from the sender
  - integrate encryption and resume token support into `Step` struct

- package endpoint
  - move the concepts that endpoint builds on top of ZFS to a single file `endpoint/endpoint_zfs.go`
    - step-holds + step-bookmarks
    - last-received-hold
    - new replication cursor + old replication cursor compat code
  - adjust `endpoint/endpoint.go` handlers for
    - encryption
    - resumability
    - new replication cursor
    - last-received-hold

- client subcommand `zrepl holds list`: list all holds and hold-like bookmarks that zrepl thinks belong to it
- client subcommand `zrepl migrate replication-cursor:v1-v2`
This commit is contained in:
Christian Schwarz 2019-09-11 17:19:17 +02:00
parent 9a4763ceee
commit 58c08c855f
72 changed files with 5445 additions and 818 deletions

View File

@ -144,9 +144,19 @@ format:
ZREPL_PLATFORMTEST_POOLNAME := zreplplatformtest
ZREPL_PLATFORMTEST_IMAGEPATH := /tmp/zreplplatformtest.pool.img
ZREPL_PLATFORMTEST_MOUNTPOINT := /tmp/zreplplatformtest.pool
ZREPL_PLATFORMTEST_ZFS_LOG := /tmp/zreplplatformtest.zfs.log
# ZREPL_PLATFORMTEST_STOP_AND_KEEP := -failure.stop-and-keep-pool
ZREPL_PLATFORMTEST_ARGS :=
platformtest: # do not track dependency on platformtest-bin to allow build of platformtest outside of test VM
"$(ARTIFACTDIR)/platformtest-$(ZREPL_TARGET_TUPLE)" -poolname "$(ZREPL_PLATFORMTEST_POOLNAME)" -imagepath "$(ZREPL_PLATFORMTEST_IMAGEPATH)" $(ZREPL_PLATFORMTEST_ARGS)
rm -f "$(ZREPL_PLATFORMTEST_ZFS_LOG)"
platformtest/logmockzfs/logzfsenv "$(ZREPL_PLATFORMTEST_ZFS_LOG)" `which zfs` \
"$(ARTIFACTDIR)/platformtest-$(ZREPL_TARGET_TUPLE)" \
-poolname "$(ZREPL_PLATFORMTEST_POOLNAME)" \
-imagepath "$(ZREPL_PLATFORMTEST_IMAGEPATH)" \
-mountpoint "$(ZREPL_PLATFORMTEST_MOUNTPOINT)" \
$(ZREPL_PLATFORMTEST_STOP_AND_KEEP) \
$(ZREPL_PLATFORMTEST_ARGS)
##################### NOARCH #####################
.PHONY: noarch $(ARTIFACTDIR)/bash_completion $(ARTIFACTDIR)/go_env.txt docs docs-clean

87
client/holds.go Normal file
View File

@ -0,0 +1,87 @@
package client
import (
"context"
"encoding/json"
"fmt"
"os"
"strings"
"github.com/pkg/errors"
"github.com/zrepl/zrepl/cli"
"github.com/zrepl/zrepl/daemon/filters"
"github.com/zrepl/zrepl/endpoint"
"github.com/zrepl/zrepl/zfs"
)
var (
HoldsCmd = &cli.Subcommand{
Use: "holds",
Short: "manage holds & step bookmarks",
SetupSubcommands: func() []*cli.Subcommand {
return holdsList
},
}
)
var holdsList = []*cli.Subcommand{
&cli.Subcommand{
Use: "list [FSFILTER]",
Run: doHoldsList,
NoRequireConfig: true,
Short: `
FSFILTER SYNTAX:
representation of a 'filesystems' filter statement on the command line
`,
},
}
func fsfilterFromCliArg(arg string) (zfs.DatasetFilter, error) {
mappings := strings.Split(arg, ",")
f := filters.NewDatasetMapFilter(len(mappings), true)
for _, m := range mappings {
thisMappingErr := fmt.Errorf("expecting comma-separated list of <dataset-pattern>:<ok|!> pairs, got %q", m)
lhsrhs := strings.SplitN(m, ":", 2)
if len(lhsrhs) != 2 {
return nil, thisMappingErr
}
err := f.Add(lhsrhs[0], lhsrhs[1])
if err != nil {
return nil, fmt.Errorf("%s: %s", thisMappingErr, err)
}
}
return f.AsFilter(), nil
}
func doHoldsList(sc *cli.Subcommand, args []string) error {
var err error
ctx := context.Background()
if len(args) > 1 {
return errors.New("this subcommand takes at most one argument")
}
var filter zfs.DatasetFilter
if len(args) == 0 {
filter = zfs.NoFilter()
} else {
filter, err = fsfilterFromCliArg(args[0])
if err != nil {
return errors.Wrap(err, "cannot parse filesystem filter args")
}
}
listing, err := endpoint.ListZFSHoldsAndBookmarks(ctx, filter)
if err != nil {
return err // context clear by invocation of command
}
enc := json.NewEncoder(os.Stdout)
enc.SetIndent(" ", " ")
if err := enc.Encode(listing); err != nil {
panic(err)
}
return nil
}

View File

@ -4,9 +4,13 @@ import (
"context"
"fmt"
"github.com/fatih/color"
"github.com/kr/pretty"
"github.com/pkg/errors"
"github.com/spf13/pflag"
"github.com/zrepl/zrepl/daemon/job"
"github.com/zrepl/zrepl/endpoint"
"github.com/zrepl/zrepl/zfs"
"github.com/zrepl/zrepl/cli"
@ -31,6 +35,13 @@ var migrations = []*cli.Subcommand{
f.BoolVar(&migratePlaceholder0_1Args.dryRun, "dry-run", false, "dry run")
},
},
&cli.Subcommand{
Use: "replication-cursor:v1-v2",
Run: doMigrateReplicationCursor,
SetupFlags: func(f *pflag.FlagSet) {
f.BoolVar(&migrateReplicationCursorArgs.dryRun, "dry-run", false, "dry run")
},
},
}
var migratePlaceholder0_1Args struct {
@ -102,3 +113,161 @@ func doMigratePlaceholder0_1(sc *cli.Subcommand, args []string) error {
return nil
}
var migrateReplicationCursorArgs struct {
dryRun bool
}
var bold = color.New(color.Bold)
var succ = color.New(color.FgGreen)
var fail = color.New(color.FgRed)
var migrateReplicationCursorSkipSentinel = fmt.Errorf("skipping this filesystem")
func doMigrateReplicationCursor(sc *cli.Subcommand, args []string) error {
if len(args) != 0 {
return fmt.Errorf("migration does not take arguments, got %v", args)
}
cfg := sc.Config()
jobs, err := job.JobsFromConfig(cfg)
if err != nil {
fmt.Printf("cannot parse config:\n%s\n\n", err)
fmt.Printf("NOTE: this migration was released together with a change in job name requirements.\n")
return fmt.Errorf("exiting migration after error")
}
ctx := context.Background()
v1cursorJobs := make([]job.Job, 0, len(cfg.Jobs))
for i, j := range cfg.Jobs {
if jobs[i].Name() != j.Name() {
panic("implementation error")
}
switch j.Ret.(type) {
case *config.PushJob:
v1cursorJobs = append(v1cursorJobs, jobs[i])
case *config.SourceJob:
v1cursorJobs = append(v1cursorJobs, jobs[i])
default:
fmt.Printf("ignoring job %q (%d/%d, type %T), not supposed to create v1 replication cursors\n", j.Name(), i, len(cfg.Jobs), j.Ret)
continue
}
}
// scan all filesystems for v1 replication cursors
fss, err := zfs.ZFSListMapping(ctx, zfs.NoFilter())
if err != nil {
return errors.Wrap(err, "list filesystems")
}
var hadError bool
for _, fs := range fss {
bold.Printf("INSPECT FILESYTEM %q\n", fs.ToString())
err := doMigrateReplicationCursorFS(ctx, v1cursorJobs, fs)
if err == migrateReplicationCursorSkipSentinel {
bold.Printf("FILESYSTEM SKIPPED\n")
} else if err != nil {
hadError = true
fail.Printf("MIGRATION FAILED: %s\n", err)
} else {
succ.Printf("FILESYSTEM %q COMPLETE\n", fs.ToString())
}
}
if hadError {
fail.Printf("\n\none or more filesystems could not be migrated, please inspect output and or re-run migration")
return errors.Errorf("")
}
return nil
}
func doMigrateReplicationCursorFS(ctx context.Context, v1CursorJobs []job.Job, fs *zfs.DatasetPath) error {
var owningJob job.Job = nil
for _, job := range v1CursorJobs {
conf := job.SenderConfig()
if conf == nil {
continue
}
pass, err := conf.FSF.Filter(fs)
if err != nil {
return errors.Wrapf(err, "filesystem filter error in job %q for fs %q", job.Name(), fs.ToString())
}
if !pass {
continue
}
if owningJob != nil {
return errors.Errorf("jobs %q and %q both match %q\ncannot attribute replication cursor to either one", owningJob.Name(), job.Name(), fs)
}
owningJob = job
}
if owningJob == nil {
fmt.Printf("no job's Filesystems filter matches\n")
return migrateReplicationCursorSkipSentinel
}
fmt.Printf("identified owning job %q\n", owningJob.Name())
versions, err := zfs.ZFSListFilesystemVersions(fs, nil)
if err != nil {
return errors.Wrapf(err, "list filesystem versions of %q", fs.ToString())
}
var oldCursor *zfs.FilesystemVersion
for i, fsv := range versions {
if fsv.Type != zfs.Bookmark {
continue
}
_, _, err := endpoint.ParseReplicationCursorBookmarkName(fsv.ToAbsPath(fs))
if err != endpoint.ErrV1ReplicationCursor {
continue
}
if oldCursor != nil {
fmt.Printf("unexpected v1 replication cursor candidate: %q", fsv.ToAbsPath(fs))
return errors.Wrap(err, "multiple filesystem versions identified as v1 replication cursors")
}
oldCursor = &versions[i]
}
if oldCursor == nil {
bold.Printf("no v1 replication cursor found for filesystem %q\n", fs.ToString())
return migrateReplicationCursorSkipSentinel
}
fmt.Printf("found v1 replication cursor:\n%s\n", pretty.Sprint(oldCursor))
mostRecentNew, err := endpoint.GetMostRecentReplicationCursorOfJob(ctx, fs.ToString(), owningJob.SenderConfig().JobID)
if err != nil {
return errors.Wrapf(err, "get most recent v2 replication cursor")
}
if mostRecentNew == nil {
return errors.Errorf("no v2 replication cursor found for job %q on filesystem %q", owningJob.SenderConfig().JobID, fs.ToString())
}
fmt.Printf("most recent v2 replication cursor:\n%#v", oldCursor)
if !(mostRecentNew.CreateTXG >= oldCursor.CreateTXG) {
return errors.Errorf("v1 replication cursor createtxg is higher than v2 cursor's, skipping this filesystem")
}
fmt.Printf("determined that v2 cursor is bookmark of same or newer version than v1 cursor\n")
fmt.Printf("destroying v1 cursor %q\n", oldCursor.ToAbsPath(fs))
if migrateReplicationCursorArgs.dryRun {
succ.Printf("DRY RUN\n")
} else {
if err := zfs.ZFSDestroyFilesystemVersion(fs, oldCursor); err != nil {
return err
}
}
return nil
}

View File

@ -751,8 +751,17 @@ func (t *tui) printFilesystemStatus(rep *report.FilesystemReport, active bool, m
if nextStep.IsIncremental() {
next = fmt.Sprintf("next: %s => %s", nextStep.Info.From, nextStep.Info.To)
} else {
next = fmt.Sprintf("next: %s (full)", nextStep.Info.To)
next = fmt.Sprintf("next: full send %s", nextStep.Info.To)
}
attribs := []string{}
if nextStep.Info.Resumed {
attribs = append(attribs, "resumed")
}
attribs = append(attribs, fmt.Sprintf("encrypted=%s", nextStep.Info.Encrypted))
next += fmt.Sprintf(" (%s)", strings.Join(attribs, ", "))
} else {
next = "" // individual FSes may still be in planning state
}

View File

@ -1,7 +1,10 @@
package client
import (
"context"
"encoding/json"
"fmt"
"os"
"github.com/pkg/errors"
"github.com/spf13/pflag"
@ -15,7 +18,7 @@ import (
var TestCmd = &cli.Subcommand{
Use: "test",
SetupSubcommands: func() []*cli.Subcommand {
return []*cli.Subcommand{testFilter, testPlaceholder}
return []*cli.Subcommand{testFilter, testPlaceholder, testDecodeResumeToken}
},
}
@ -176,3 +179,32 @@ func runTestPlaceholder(subcommand *cli.Subcommand, args []string) error {
}
return nil
}
var testDecodeResumeTokenArgs struct {
token string
}
var testDecodeResumeToken = &cli.Subcommand{
Use: "decoderesumetoken --token TOKEN",
Short: "decode resume token",
SetupFlags: func(f *pflag.FlagSet) {
f.StringVar(&testDecodeResumeTokenArgs.token, "token", "", "the resume token obtained from the receive_resume_token property")
},
Run: runTestDecodeResumeTokenCmd,
}
func runTestDecodeResumeTokenCmd(subcommand *cli.Subcommand, args []string) error {
if testDecodeResumeTokenArgs.token == "" {
return fmt.Errorf("token argument must be specified")
}
token, err := zfs.ParseResumeToken(context.Background(), testDecodeResumeTokenArgs.token)
if err != nil {
return err
}
enc := json.NewEncoder(os.Stdout)
enc.SetIndent("", " ")
if err := enc.Encode(&token); err != nil {
panic(err)
}
return nil
}

View File

@ -75,16 +75,42 @@ type SnapJob struct {
Filesystems FilesystemsFilter `yaml:"filesystems"`
}
type SendOptions struct {
Encrypted bool `yaml:"encrypted"`
}
var _ yaml.Defaulter = (*SendOptions)(nil)
func (l *SendOptions) SetDefault() {
*l = SendOptions{Encrypted: false}
}
type RecvOptions struct {
// Note: we cannot enforce encrypted recv as the ZFS cli doesn't provide a mechanism for it
// Encrypted bool `yaml:"may_encrypted"`
// Future:
// Reencrypt bool `yaml:"reencrypt"`
}
var _ yaml.Defaulter = (*RecvOptions)(nil)
func (l *RecvOptions) SetDefault() {
*l = RecvOptions{}
}
type PushJob struct {
ActiveJob `yaml:",inline"`
Snapshotting SnapshottingEnum `yaml:"snapshotting"`
Filesystems FilesystemsFilter `yaml:"filesystems"`
Send *SendOptions `yaml:"send,fromdefaults,optional"`
}
type PullJob struct {
ActiveJob `yaml:",inline"`
RootFS string `yaml:"root_fs"`
Interval PositiveDurationOrManual `yaml:"interval"`
Recv *RecvOptions `yaml:"recv,fromdefaults,optional"`
}
type PositiveDurationOrManual struct {
@ -121,12 +147,14 @@ func (i *PositiveDurationOrManual) UnmarshalYAML(u func(interface{}, bool) error
type SinkJob struct {
PassiveJob `yaml:",inline"`
RootFS string `yaml:"root_fs"`
Recv *RecvOptions `yaml:"recv,optional,fromdefaults"`
}
type SourceJob struct {
PassiveJob `yaml:",inline"`
Snapshotting SnapshottingEnum `yaml:"snapshotting"`
Filesystems FilesystemsFilter `yaml:"filesystems"`
Send *SendOptions `yaml:"send,optional,fromdefaults"`
}
type FilesystemsFilter map[string]bool

View File

@ -0,0 +1 @@
package config

View File

@ -0,0 +1,74 @@
package config
import (
"fmt"
"testing"
"github.com/stretchr/testify/assert"
)
func TestSendOptions(t *testing.T) {
tmpl := `
jobs:
- name: foo
type: push
connect:
type: local
listener_name: foo
client_identity: bar
filesystems: {"<": true}
%s
snapshotting:
type: manual
pruning:
keep_sender:
- type: last_n
count: 10
keep_receiver:
- type: last_n
count: 10
`
encrypted_false := `
send:
encrypted: false
`
encrypted_true := `
send:
encrypted: true
`
encrypted_unspecified := `
send: {}
`
send_not_specified := `
`
fill := func(s string) string { return fmt.Sprintf(tmpl, s) }
var c *Config
t.Run("encrypted_false", func(t *testing.T) {
c = testValidConfig(t, fill(encrypted_false))
encrypted := c.Jobs[0].Ret.(*PushJob).Send.Encrypted
assert.Equal(t, false, encrypted)
})
t.Run("encrypted_true", func(t *testing.T) {
c = testValidConfig(t, fill(encrypted_true))
encrypted := c.Jobs[0].Ret.(*PushJob).Send.Encrypted
assert.Equal(t, true, encrypted)
})
t.Run("encrypted_unspecified", func(t *testing.T) {
c, err := testConfig(t, fill(encrypted_unspecified))
assert.Error(t, err)
assert.Nil(t, c)
})
t.Run("send_not_specified", func(t *testing.T) {
c, err := testConfig(t, fill(send_not_specified))
assert.NoError(t, err)
assert.NotNil(t, c)
})
}

View File

@ -10,6 +10,8 @@ jobs:
address: "backup-server.foo.bar:8888"
snapshotting:
type: manual
send:
encrypted: false
pruning:
keep_sender:
- type: not_replicated

View File

@ -15,6 +15,7 @@ import (
"github.com/zrepl/zrepl/daemon/job"
"github.com/zrepl/zrepl/daemon/nethelpers"
"github.com/zrepl/zrepl/endpoint"
"github.com/zrepl/zrepl/logger"
"github.com/zrepl/zrepl/util/envconst"
"github.com/zrepl/zrepl/version"
@ -44,6 +45,8 @@ func (j *controlJob) Status() *job.Status { return &job.Status{Type: job.TypeInt
func (j *controlJob) OwnedDatasetSubtreeRoot() (p *zfs.DatasetPath, ok bool) { return nil, false }
func (j *controlJob) SenderConfig() *endpoint.SenderConfig { return nil }
var promControl struct {
requestBegin *prometheus.CounterVec
requestFinished *prometheus.HistogramVec

View File

@ -1,11 +1,11 @@
package filters
import (
"testing"
"github.com/zrepl/zrepl/zfs"
)
import "testing"
func TestDatasetMapFilter(t *testing.T) {
type testCase struct {

View File

@ -2,6 +2,7 @@ package job
import (
"context"
"fmt"
"sync"
"time"
@ -28,7 +29,7 @@ import (
type ActiveSide struct {
mode activeMode
name string
name endpoint.JobID
connecter transport.Connecter
prunerFactory *pruner.PrunerFactory
@ -82,6 +83,7 @@ type activeMode interface {
DisconnectEndpoints()
SenderReceiver() (logic.Sender, logic.Receiver)
Type() Type
PlannerPolicy() logic.PlannerPolicy
RunPeriodic(ctx context.Context, wakeUpCommon chan<- struct{})
SnapperReport() *snapper.Report
ResetConnectBackoff()
@ -91,7 +93,8 @@ type modePush struct {
setupMtx sync.Mutex
sender *endpoint.Sender
receiver *rpc.Client
fsfilter endpoint.FSFilter
senderConfig *endpoint.SenderConfig
plannerPolicy *logic.PlannerPolicy
snapper *snapper.PeriodicOrManual
}
@ -101,7 +104,7 @@ func (m *modePush) ConnectEndpoints(loggers rpc.Loggers, connecter transport.Con
if m.receiver != nil || m.sender != nil {
panic("inconsistent use of ConnectEndpoints and DisconnectEndpoints")
}
m.sender = endpoint.NewSender(m.fsfilter)
m.sender = endpoint.NewSender(*m.senderConfig)
m.receiver = rpc.NewClient(connecter, loggers)
}
@ -121,6 +124,8 @@ func (m *modePush) SenderReceiver() (logic.Sender, logic.Receiver) {
func (m *modePush) Type() Type { return TypePush }
func (m *modePush) PlannerPolicy() logic.PlannerPolicy { return *m.plannerPolicy }
func (m *modePush) RunPeriodic(ctx context.Context, wakeUpCommon chan<- struct{}) {
m.snapper.Run(ctx, wakeUpCommon)
}
@ -137,13 +142,22 @@ func (m *modePush) ResetConnectBackoff() {
}
}
func modePushFromConfig(g *config.Global, in *config.PushJob) (*modePush, error) {
func modePushFromConfig(g *config.Global, in *config.PushJob, jobID endpoint.JobID) (*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
m.senderConfig = &endpoint.SenderConfig{
FSF: fsf,
Encrypt: &zfs.NilBool{B: in.Send.Encrypted},
JobID: jobID,
}
m.plannerPolicy = &logic.PlannerPolicy{
EncryptedSend: logic.TriFromBool(in.Send.Encrypted),
}
if m.snapper, err = snapper.FromConfig(g, fsf, in.Snapshotting); err != nil {
return nil, errors.Wrap(err, "cannot build snapper")
@ -155,8 +169,10 @@ func modePushFromConfig(g *config.Global, in *config.PushJob) (*modePush, error)
type modePull struct {
setupMtx sync.Mutex
receiver *endpoint.Receiver
receiverConfig endpoint.ReceiverConfig
sender *rpc.Client
rootFS *zfs.DatasetPath
plannerPolicy *logic.PlannerPolicy
interval config.PositiveDurationOrManual
}
@ -166,7 +182,7 @@ func (m *modePull) ConnectEndpoints(loggers rpc.Loggers, connecter transport.Con
if m.receiver != nil || m.sender != nil {
panic("inconsistent use of ConnectEndpoints and DisconnectEndpoints")
}
m.receiver = endpoint.NewReceiver(m.rootFS, false)
m.receiver = endpoint.NewReceiver(m.receiverConfig)
m.sender = rpc.NewClient(connecter, loggers)
}
@ -186,6 +202,8 @@ func (m *modePull) SenderReceiver() (logic.Sender, logic.Receiver) {
func (*modePull) Type() Type { return TypePull }
func (m *modePull) PlannerPolicy() logic.PlannerPolicy { return *m.plannerPolicy }
func (m *modePull) RunPeriodic(ctx context.Context, wakeUpCommon chan<- struct{}) {
if m.interval.Manual {
GetLogger(ctx).Info("manual pull configured, periodic pull disabled")
@ -223,7 +241,7 @@ func (m *modePull) ResetConnectBackoff() {
}
}
func modePullFromConfig(g *config.Global, in *config.PullJob) (m *modePull, err error) {
func modePullFromConfig(g *config.Global, in *config.PullJob, jobID endpoint.JobID) (m *modePull, err error) {
m = &modePull{}
m.interval = in.Interval
@ -235,26 +253,56 @@ func modePullFromConfig(g *config.Global, in *config.PullJob) (m *modePull, err
return nil, errors.New("RootFS must not be empty") // duplicates error check of receiver
}
m.plannerPolicy = &logic.PlannerPolicy{
EncryptedSend: logic.DontCare,
}
m.receiverConfig = endpoint.ReceiverConfig{
JobID: jobID,
RootWithoutClientComponent: m.rootFS,
AppendClientIdentity: false, // !
UpdateLastReceivedHold: true,
}
if err := m.receiverConfig.Validate(); err != nil {
return nil, errors.Wrap(err, "cannot build receiver config")
}
return m, nil
}
func activeSide(g *config.Global, in *config.ActiveJob, mode activeMode) (j *ActiveSide, err error) {
func activeSide(g *config.Global, in *config.ActiveJob, configJob interface{}) (j *ActiveSide, err error) {
j = &ActiveSide{}
j.name, err = endpoint.MakeJobID(in.Name)
if err != nil {
return nil, errors.Wrap(err, "invalid job name")
}
switch v := configJob.(type) {
case *config.PushJob:
j.mode, err = modePushFromConfig(g, v, j.name) // shadow
case *config.PullJob:
j.mode, err = modePullFromConfig(g, v, j.name) // shadow
default:
panic(fmt.Sprintf("implementation error: unknown job type %T", v))
}
if err != nil {
return nil, err // no wrapping required
}
j = &ActiveSide{mode: mode}
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},
ConstLabels: prometheus.Labels{"zrepl_job": j.name.String()},
}, []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},
ConstLabels: prometheus.Labels{"zrepl_job": j.name.String()},
}, []string{"filesystem"})
j.connecter, err = fromconfig.ConnecterFromConfig(g, in.Connect)
@ -267,7 +315,7 @@ func activeSide(g *config.Global, in *config.ActiveJob, mode activeMode) (j *Act
Subsystem: "pruning",
Name: "time",
Help: "seconds spent in pruner",
ConstLabels: prometheus.Labels{"zrepl_job": j.name},
ConstLabels: prometheus.Labels{"zrepl_job": j.name.String()},
}, []string{"prune_side"})
j.prunerFactory, err = pruner.NewPrunerFactory(in.Pruning, j.promPruneSecs)
if err != nil {
@ -283,7 +331,7 @@ func (j *ActiveSide) RegisterMetrics(registerer prometheus.Registerer) {
registerer.MustRegister(j.promBytesReplicated)
}
func (j *ActiveSide) Name() string { return j.name }
func (j *ActiveSide) Name() string { return j.name.String() }
type ActiveSideStatus struct {
Replication *report.Report
@ -318,6 +366,15 @@ func (j *ActiveSide) OwnedDatasetSubtreeRoot() (rfs *zfs.DatasetPath, ok bool) {
return pull.rootFS.Copy(), true
}
func (j *ActiveSide) SenderConfig() *endpoint.SenderConfig {
push, ok := j.mode.(*modePush)
if !ok {
_ = j.mode.(*modePull) // make sure we didn't introduce a new job type
return nil
}
return push.senderConfig
}
func (j *ActiveSide) Run(ctx context.Context) {
log := GetLogger(ctx)
ctx = logging.WithSubsystemLoggers(ctx, log)
@ -383,7 +440,7 @@ func (j *ActiveSide) do(ctx context.Context) {
*tasks = activeSideTasks{}
tasks.replicationCancel = repCancel
tasks.replicationReport, repWait = replication.Do(
ctx, logic.NewPlanner(j.promRepStateSecs, j.promBytesReplicated, sender, receiver),
ctx, logic.NewPlanner(j.promRepStateSecs, j.promBytesReplicated, sender, receiver, j.mode.PlannerPolicy()),
)
tasks.state = ActiveSideReplicating
})

View File

@ -48,20 +48,12 @@ func buildJob(c *config.Global, in config.JobEnum) (j Job, err error) {
// FIXME prettify this
switch v := in.Ret.(type) {
case *config.SinkJob:
m, err := modeSinkFromConfig(c, v)
if err != nil {
return cannotBuildJob(err, v.Name)
}
j, err = passiveSideFromConfig(c, &v.PassiveJob, m)
j, err = passiveSideFromConfig(c, &v.PassiveJob, v)
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)
j, err = passiveSideFromConfig(c, &v.PassiveJob, v)
if err != nil {
return cannotBuildJob(err, v.Name)
}
@ -71,20 +63,12 @@ func buildJob(c *config.Global, in config.JobEnum) (j Job, err error) {
return cannotBuildJob(err, v.Name)
}
case *config.PushJob:
m, err := modePushFromConfig(c, v)
if err != nil {
return cannotBuildJob(err, v.Name)
}
j, err = activeSide(c, &v.ActiveJob, m)
j, err = activeSide(c, &v.ActiveJob, v)
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)
j, err = activeSide(c, &v.ActiveJob, v)
if err != nil {
return cannotBuildJob(err, v.Name)
}

View File

@ -1,9 +1,13 @@
package job
import (
"fmt"
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/zrepl/zrepl/config"
)
func TestValidateReceivingSidesDoNotOverlap(t *testing.T) {
@ -41,3 +45,66 @@ func TestValidateReceivingSidesDoNotOverlap(t *testing.T) {
}
}
}
func TestJobIDErrorHandling(t *testing.T) {
tmpl := `
jobs:
- name: %s
type: push
connect:
type: local
listener_name: foo
client_identity: bar
filesystems: {"<": true}
snapshotting:
type: manual
pruning:
keep_sender:
- type: last_n
count: 10
keep_receiver:
- type: last_n
count: 10
`
fill := func(s string) string { return fmt.Sprintf(tmpl, s) }
type Case struct {
jobName string
valid bool
}
cases := []Case{
{"validjobname", true},
{"valid with spaces", true},
{"invalid\twith\ttabs", false},
{"invalid#withdelimiter", false},
{"invalid@withdelimiter", false},
{"withnewline\\nmiddle", false},
{"withnewline\\n", false},
{"withslash/", false},
{"withslash/inthemiddle", false},
{"/", false},
}
for i := range cases {
t.Run(cases[i].jobName, func(t *testing.T) {
c := cases[i]
conf, err := config.ParseConfigBytes([]byte(fill(c.jobName)))
require.NoError(t, err, "not expecting yaml-config to know about job ids")
require.NotNil(t, conf)
jobs, err := JobsFromConfig(conf)
if c.valid {
assert.NoError(t, err)
require.Len(t, jobs, 1)
assert.Equal(t, c.jobName, jobs[0].Name())
} else {
t.Logf("error: %s", err)
assert.Error(t, err)
assert.Nil(t, jobs)
}
})
}
}

View File

@ -7,6 +7,7 @@ import (
"github.com/prometheus/client_golang/prometheus"
"github.com/zrepl/zrepl/endpoint"
"github.com/zrepl/zrepl/logger"
"github.com/zrepl/zrepl/zfs"
)
@ -38,6 +39,7 @@ type Job interface {
// Jobs that return a subtree of the dataset hierarchy
// must return the root of that subtree as rfs and ok = true
OwnedDatasetSubtreeRoot() (rfs *zfs.DatasetPath, ok bool)
SenderConfig() *endpoint.SenderConfig
}
type Type string

View File

@ -20,7 +20,7 @@ import (
type PassiveSide struct {
mode passiveMode
name string
name endpoint.JobID
listen transport.AuthenticatedListenerFactory
}
@ -32,43 +32,56 @@ type passiveMode interface {
}
type modeSink struct {
rootDataset *zfs.DatasetPath
receiverConfig endpoint.ReceiverConfig
}
func (m *modeSink) Type() Type { return TypeSink }
func (m *modeSink) Handler() rpc.Handler {
return endpoint.NewReceiver(m.rootDataset, true)
return endpoint.NewReceiver(m.receiverConfig)
}
func (m *modeSink) RunPeriodic(_ context.Context) {}
func (m *modeSink) SnapperReport() *snapper.Report { return nil }
func modeSinkFromConfig(g *config.Global, in *config.SinkJob) (m *modeSink, err error) {
func modeSinkFromConfig(g *config.Global, in *config.SinkJob, jobID endpoint.JobID) (m *modeSink, err error) {
m = &modeSink{}
m.rootDataset, err = zfs.NewDatasetPath(in.RootFS)
rootDataset, err := zfs.NewDatasetPath(in.RootFS)
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
m.receiverConfig = endpoint.ReceiverConfig{
JobID: jobID,
RootWithoutClientComponent: rootDataset,
AppendClientIdentity: true, // !
UpdateLastReceivedHold: true,
}
if err := m.receiverConfig.Validate(); err != nil {
return nil, errors.Wrap(err, "cannot build receiver config")
}
return m, nil
}
type modeSource struct {
fsfilter zfs.DatasetFilter
senderConfig *endpoint.SenderConfig
snapper *snapper.PeriodicOrManual
}
func modeSourceFromConfig(g *config.Global, in *config.SourceJob) (m *modeSource, err error) {
func modeSourceFromConfig(g *config.Global, in *config.SourceJob, jobID endpoint.JobID) (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
m.senderConfig = &endpoint.SenderConfig{
FSF: fsf,
Encrypt: &zfs.NilBool{B: in.Send.Encrypted},
JobID: jobID,
}
if m.snapper, err = snapper.FromConfig(g, fsf, in.Snapshotting); err != nil {
return nil, errors.Wrap(err, "cannot build snapper")
@ -80,7 +93,7 @@ func modeSourceFromConfig(g *config.Global, in *config.SourceJob) (m *modeSource
func (m *modeSource) Type() Type { return TypeSource }
func (m *modeSource) Handler() rpc.Handler {
return endpoint.NewSender(m.fsfilter)
return endpoint.NewSender(*m.senderConfig)
}
func (m *modeSource) RunPeriodic(ctx context.Context) {
@ -91,9 +104,25 @@ func (m *modeSource) SnapperReport() *snapper.Report {
return m.snapper.Report()
}
func passiveSideFromConfig(g *config.Global, in *config.PassiveJob, mode passiveMode) (s *PassiveSide, err error) {
func passiveSideFromConfig(g *config.Global, in *config.PassiveJob, configJob interface{}) (s *PassiveSide, err error) {
s = &PassiveSide{}
s.name, err = endpoint.MakeJobID(in.Name)
if err != nil {
return nil, errors.Wrap(err, "invalid job name")
}
switch v := configJob.(type) {
case *config.SinkJob:
s.mode, err = modeSinkFromConfig(g, v, s.name) // shadow
case *config.SourceJob:
s.mode, err = modeSourceFromConfig(g, v, s.name) // shadow
}
if err != nil {
return nil, err // no wrapping necessary
}
s = &PassiveSide{mode: mode, name: in.Name}
if s.listen, err = fromconfig.ListenerFactoryFromConfig(g, in.Serve); err != nil {
return nil, errors.Wrap(err, "cannot build listener factory")
}
@ -101,7 +130,7 @@ func passiveSideFromConfig(g *config.Global, in *config.PassiveJob, mode passive
return s, nil
}
func (j *PassiveSide) Name() string { return j.name }
func (j *PassiveSide) Name() string { return j.name.String() }
type PassiveStatus struct {
Snapper *snapper.Report
@ -120,7 +149,16 @@ func (j *PassiveSide) OwnedDatasetSubtreeRoot() (rfs *zfs.DatasetPath, ok bool)
_ = j.mode.(*modeSource) // make sure we didn't introduce a new job type
return nil, false
}
return sink.rootDataset.Copy(), true
return sink.receiverConfig.RootWithoutClientComponent.Copy(), true
}
func (j *PassiveSide) SenderConfig() *endpoint.SenderConfig {
source, ok := j.mode.(*modeSource)
if !ok {
_ = j.mode.(*modeSink) // make sure we didn't introduce a new job type
return nil
}
return source.senderConfig
}
func (*PassiveSide) RegisterMetrics(registerer prometheus.Registerer) {}

View File

@ -2,7 +2,6 @@ package job
import (
"context"
"fmt"
"sort"
"github.com/pkg/errors"
@ -20,7 +19,7 @@ import (
)
type SnapJob struct {
name string
name endpoint.JobID
fsfilter zfs.DatasetFilter
snapper *snapper.PeriodicOrManual
@ -31,7 +30,7 @@ type SnapJob struct {
pruner *pruner.Pruner
}
func (j *SnapJob) Name() string { return j.name }
func (j *SnapJob) Name() string { return j.name.String() }
func (j *SnapJob) Type() Type { return TypeSnap }
@ -46,13 +45,16 @@ func snapJobFromConfig(g *config.Global, in *config.SnapJob) (j *SnapJob, err er
if j.snapper, err = snapper.FromConfig(g, fsf, in.Snapshotting); err != nil {
return nil, errors.Wrap(err, "cannot build snapper")
}
j.name = in.Name
j.name, err = endpoint.MakeJobID(in.Name)
if err != nil {
return nil, errors.Wrap(err, "invalid job name")
}
j.promPruneSecs = prometheus.NewHistogramVec(prometheus.HistogramOpts{
Namespace: "zrepl",
Subsystem: "pruning",
Name: "time",
Help: "seconds spent in pruner",
ConstLabels: prometheus.Labels{"zrepl_job": j.name},
ConstLabels: prometheus.Labels{"zrepl_job": j.name.String()},
}, []string{"prune_side"})
j.prunerFactory, err = pruner.NewLocalPrunerFactory(in.Pruning, j.promPruneSecs)
if err != nil {
@ -84,6 +86,8 @@ func (j *SnapJob) OwnedDatasetSubtreeRoot() (rfs *zfs.DatasetPath, ok bool) {
return nil, false
}
func (j *SnapJob) SenderConfig() *endpoint.SenderConfig { return nil }
func (j *SnapJob) Run(ctx context.Context) {
log := GetLogger(ctx)
ctx = logging.WithSubsystemLoggers(ctx, log)
@ -133,9 +137,6 @@ type alwaysUpToDateReplicationCursorHistory struct {
var _ pruner.History = (*alwaysUpToDateReplicationCursorHistory)(nil)
func (h alwaysUpToDateReplicationCursorHistory) ReplicationCursor(ctx context.Context, req *pdu.ReplicationCursorReq) (*pdu.ReplicationCursorRes, error) {
if req.GetGet() == nil {
return nil, fmt.Errorf("unsupported ReplicationCursor request: SnapJob only supports GETting a (faked) cursor")
}
fsvReq := &pdu.ListFilesystemVersionsReq{
Filesystem: req.GetFilesystem(),
}
@ -162,7 +163,12 @@ func (h alwaysUpToDateReplicationCursorHistory) ListFilesystems(ctx context.Cont
func (j *SnapJob) doPrune(ctx context.Context) {
log := GetLogger(ctx)
ctx = logging.WithSubsystemLoggers(ctx, log)
sender := endpoint.NewSender(j.fsfilter)
sender := endpoint.NewSender(endpoint.SenderConfig{
JobID: j.name,
FSF: j.fsfilter,
// FIXME encryption setting is irrelevant for SnapJob because the endpoint is only used as pruner.Target
Encrypt: &zfs.NilBool{B: true},
})
j.pruner = j.prunerFactory.BuildLocalPruner(ctx, sender, alwaysUpToDateReplicationCursorHistory{sender})
log.Info("start pruning")
j.pruner.Prune()

View File

@ -10,6 +10,7 @@ import (
"github.com/zrepl/zrepl/config"
"github.com/zrepl/zrepl/daemon/job"
"github.com/zrepl/zrepl/endpoint"
"github.com/zrepl/zrepl/logger"
"github.com/zrepl/zrepl/rpc/dataconn/frameconn"
"github.com/zrepl/zrepl/util/tcpsock"
@ -48,6 +49,8 @@ func (j *prometheusJob) Status() *job.Status { return &job.Status{Type: job.Type
func (j *prometheusJob) OwnedDatasetSubtreeRoot() (p *zfs.DatasetPath, ok bool) { return nil, false }
func (j *prometheusJob) SenderConfig() *endpoint.SenderConfig { return nil }
func (j *prometheusJob) RegisterMetrics(registerer prometheus.Registerer) {}
func (j *prometheusJob) Run(ctx context.Context) {

View File

@ -412,9 +412,6 @@ tfss_loop:
rcReq := &pdu.ReplicationCursorReq{
Filesystem: tfs.Path,
Op: &pdu.ReplicationCursorReq_Get{
Get: &pdu.ReplicationCursorReq_GetOp{},
},
}
rc, err := receiver.ReplicationCursor(ctx, rcReq)
if err != nil {

View File

@ -27,12 +27,42 @@ We use the following annotations for classifying changes:
* |bugfix| Change that fixes a bug, no regressions or incompatibilities expected.
* |docs| Change to the documentation.
0.2.2 (unreleased)
------------------
0.3
---
This is a big one! Headlining features:
* **Resumable Send & Recv Support**
No knobs required, automatically used where supported.
* **Hold-Protected Send & Recv**
Automatic ZFS holds to ensure that we can always resume a replication step.
* **Encrypted Send & Recv Support** for OpenZFS native encryption.
:ref:`Configurable <job-send-options>` at the job level, i.e., for all filesystems a job is responsible for.
* **Receive-side hold on last received dataset**
The counterpart to the replication cursor bookmark on the send-side.
Ensures that incremental replication will always be possible between a sender and receiver.
Actual changelog:
* |break_config| **more restrictive job names than in prior zrepl versions**
Starting with this version, job names are going to be embedded into ZFS holds and bookmark names.
* |break| |mig| replication cursor representation changed
* zrepl now manages the :ref:`replication cursor bookmark <replication-cursor-and-last-received-hold>` per job-filesystem tuple instead of a single replication cursor per filesystem.
In the future, this will permit multiple sending jobs to send from the same filesystems.
* ZFS does not allow bookmark renaming, thus we cannot migrate the old replication cursors.
* zrepl 0.3 will automatically create cursors in the new format for new replications, and warn if it still finds ones in the old format.
* Run ``zrepl migrate replication-cursor:v1-v2`` to safely destroy old-format cursors.
The migration will ensure that only those old-format cursors are destroyed that have been superseeded by new-format cursors.
* |bugfix| missing logger context vars in control connection handlers
* |bugfix| improved error messages on ``zfs send`` errors
* |feature| New option ``listen_freebind`` (tcp, tls, prometheus listener)
* |bugfix| |docs| snapshotting: clarify sync-up behavior and warn about filesystems
that will not be snapshotted until the sync-up phase is over
* |docs| Document new replication features in the :ref:`config overview <overview-how-replication-works>` and :repomasterlink:`replication/design.md`.
* **[MAINTAINER NOTICE]** New platform tests in this version, please make sure you run them for your distro!
0.2.1
-----

View File

@ -11,6 +11,7 @@ Configuration
configuration/jobs
configuration/transports
configuration/filter_syntax
configuration/sendrecvoptions
configuration/snapshotting
configuration/prune
configuration/logging

View File

@ -24,6 +24,8 @@ Job Type ``push``
- |connect-transport|
* - ``filesystems``
- |filter-spec| for filesystems to be snapshotted and pushed to the sink
* - ``send``
- |send-options|
* - ``snapshotting``
- |snapshotting-spec|
* - ``pruning``
@ -101,6 +103,8 @@ Job Type ``source``
- |serve-transport|
* - ``filesystems``
- |filter-spec| for filesystems to be snapshotted and exposed to connecting clients
* - ``send``
- |send-options|
* - ``snapshotting``
- |snapshotting-spec|

View File

@ -90,6 +90,8 @@ It uses the client identity for access control:
.. TIP::
The implementation of the ``sink`` job requires that the connecting client identities be a valid ZFS filesystem name components.
.. _overview-how-replication-works:
How Replication Works
---------------------
@ -107,9 +109,8 @@ The following steps take place during replication and can be monitored using the
* Per filesystem, compute a diff between sender and receiver snapshots
* Build a list of replication steps
* If possible, use incremental sends (``zfs send -i``)
* If possible, use incremental and resumable 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.
@ -119,17 +120,23 @@ The following steps take place during replication and can be monitored using the
* 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.
* Create placeholder filesystems on the receiving side to mirror the dataset paths on the sender to ``root_fs/${client_identity}``.
* After a successful replication step, update the replication cursor bookmark (see below).
* Aquire send-side step-holds on the step's `from` and `to` snapshots.
* Perform the replication step.
* Move the **replication cursor** bookmark on the sending side (see below).
* Move the **last-received-hold** on the receiving side (see below).
* Release the send-side step-holds.
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:
.. _replication-cursor-and-last-received-hold:
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 <prune-keep-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.
Note that there is only one cursor bookmark per filesystem, which prohibits multiple jobs to replicate the same filesystem (:ref:`see below<jobs-multiple-jobs>`).
**Replication cursor** bookmark and **last-received-hold** are managed by zrepl to ensure that future replications can always be done incrementally:
the replication cursor is a send-side bookmark of the most recent successfully replicated snapshot,
and the last-received-hold is a hold of that snapshot on the receiving side.
The replication cursor has the format ``#zrepl_CUSOR_G_<GUID>_J_<JOBNAME>``.
The last-received-hold tag has the format ``#zrepl_last_received_J_<JOBNAME>``.
Encoding the job name in the names ensures that multiple sending jobs can replicate the same filesystem to different receivers without interference.
The ``zrepl holds list`` provides a listing of all bookmarks and holds managed by zrepl.
.. _replication-placeholder-property:
@ -144,9 +151,13 @@ The ``zrepl test placeholder`` command can be used to check whether a filesystem
.. ATTENTION::
Currently, zrepl does not replicate filesystem properties.
Whe receiving a filesystem, it is never mounted (`-u` flag) and `mountpoint=none` is set.
When receiving a filesystem, it is never mounted (`-u` flag) and `mountpoint=none` is set.
This is temporary and being worked on :issue:`24`.
.. NOTE::
More details can be found in the design document :repomasterlink:`replication/design.md`.
.. _jobs-multiple-jobs:
@ -171,7 +182,7 @@ No Overlapping
Jobs run independently of each other.
If two jobs match the same filesystem with their ``filesystems`` filter, they will operate on that filesystem independently and potentially in parallel.
For example, if job A prunes snapshots that job B is planning to replicate, the replication will fail because B asssumed the snapshot to still be present.
More subtle race conditions can occur with the :ref:`replication cursor bookmark <replication-cursor-bookmark>`, which currently only exists once per filesystem.
However, the next replication attempt will re-examine the situation from scratch and should work.
N push jobs to 1 sink
~~~~~~~~~~~~~~~~~~~~~

View File

@ -66,7 +66,7 @@ Policy ``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 <replication-cursor-bookmark>` on the sending side.
The state required to evaluate this rule is stored in the :ref:`replication cursor bookmark <replication-cursor-and-last-received-hold>` on the sending side.
.. _prune-keep-retention-grid:

View File

@ -0,0 +1,45 @@
.. include:: ../global.rst.inc
Send & Recv Options
===================
.. _job-send-options:
Send Options
~~~~~~~~~~~~
::
jobs:
- type: push
filesystems: ...
send:
encrypted: true
...
:ref:`Source<job-source>` and :ref:`push<job-push>` jobs have an optional ``send`` configuration section.
``encryption`` option
---------------------
The ``encryption`` variable controls whether the matched filesystems are sent as `OpenZFS native encryption <http://open-zfs.org/wiki/ZFS-Native_Encryption>`_ raw sends.
More specificially, if ``encryption=true``, zrepl
* checks for any of the filesystems matched by ``filesystems`` whether the ZFS ``encryption`` property indicates that the filesystem is actually encrypted with ZFS native encryption and
* invokes the ``zfs send`` subcommand with the ``-w`` option (raw sends) and
* expects the receiving side to support OpenZFS native encryption (recv will fail otherwise)
Filesystems matched by ``filesystems`` that are not encrypted are not sent and will cause error log messages.
If ``encryption=false``, zrepl expects that filesystems matching ``filesystems`` are not encrypted or have loaded encryption keys.
.. _job-recv-options:
Recv Options
~~~~~~~~~~~~
:ref:`Sink<job-sink>` and :ref:`pull<job-pull>` jobs have an optional ``recv`` configuration section.
However, there are currently no variables to configure there.

View File

@ -21,6 +21,8 @@
.. |serve-transport| replace:: :ref:`serve specification<transport>`
.. |connect-transport| replace:: :ref:`connect specification<transport>`
.. |send-options| replace:: :ref:`send options<job-send-options>`, e.g. for encrypted sends
.. |recv-options| replace:: :ref:`recv options<job-recv-options>`
.. |snapshotting-spec| replace:: :ref:`snapshotting specification <job-snapshotting-spec>`
.. |pruning-spec| replace:: :ref:`pruning specification <prune>`
.. |filter-spec| replace:: :ref:`filter specification<pattern-filter>`

View File

@ -57,9 +57,11 @@ Main Features
* Advanced replication features
* [x] Automatic retries for temporary network errors
* [ ] Resumable send & receive
* [x] Automatic resumable send & receive
* [x] Automatic ZFS holds during send & receive
* [x] Automatic bookmark \& hold management for guaranteed incremental send & recv
* [x] Encrypted raw send & receive to untrusted receivers (OpenZFS native encryption)
* [ ] Compressed send & receive
* [ ] Raw encrypted send & receive
* **Automatic snapshot management**
@ -83,6 +85,7 @@ Main Features
* **Maintainable implementation in Go**
* [x] Cross platform
* [x] Dynamic feature checking
* [x] Type safe & testable code

View File

@ -175,4 +175,3 @@ Congratulations, you have a working push backup. Where to go next?
* Read more about :ref:`configuration format, options & job types <configuration_toc>`
* Configure :ref:`logging <logging>` \& :ref:`monitoring <monitoring>`.

View File

@ -5,6 +5,7 @@ import (
"context"
"fmt"
"path"
"sync"
"github.com/pkg/errors"
@ -15,13 +16,39 @@ import (
"github.com/zrepl/zrepl/zfs"
)
type SenderConfig struct {
FSF zfs.DatasetFilter
Encrypt *zfs.NilBool
JobID JobID
}
func (c *SenderConfig) Validate() error {
c.JobID.MustValidate()
if err := c.Encrypt.Validate(); err != nil {
return errors.Wrap(err, "`Encrypt` field invalid")
}
if _, err := StepHoldTag(c.JobID); err != nil {
return fmt.Errorf("JobID cannot be used for hold tag: %s", err)
}
return nil
}
// Sender implements replication.ReplicationEndpoint for a sending side
type Sender struct {
FSFilter zfs.DatasetFilter
encrypt *zfs.NilBool
jobId JobID
}
func NewSender(fsf zfs.DatasetFilter) *Sender {
return &Sender{FSFilter: fsf}
func NewSender(conf SenderConfig) *Sender {
if err := conf.Validate(); err != nil {
panic("invalid config" + err.Error())
}
return &Sender{
FSFilter: conf.FSF,
encrypt: conf.Encrypt,
jobId: conf.JobID,
}
}
func (s *Sender) filterCheckFS(fs string) (*zfs.DatasetPath, error) {
@ -49,10 +76,15 @@ func (s *Sender) ListFilesystems(ctx context.Context, r *pdu.ListFilesystemReq)
}
rfss := make([]*pdu.Filesystem, len(fss))
for i := range fss {
encEnabled, err := zfs.ZFSGetEncryptionEnabled(ctx, fss[i].ToString())
if err != nil {
return nil, errors.Wrap(err, "cannot get filesystem encryption status")
}
rfss[i] = &pdu.Filesystem{
Path: fss[i].ToString(),
// FIXME: not supporting ResumeToken yet
// ResumeToken does not make sense from Sender
IsPlaceholder: false, // sender FSs are never placeholders
IsEncrypted: encEnabled,
}
}
res := &pdu.ListFilesystemRes{Filesystems: rfss}
@ -77,13 +109,86 @@ func (s *Sender) ListFilesystemVersions(ctx context.Context, r *pdu.ListFilesyst
}
func (p *Sender) HintMostRecentCommonAncestor(ctx context.Context, r *pdu.HintMostRecentCommonAncestorReq) (*pdu.HintMostRecentCommonAncestorRes, error) {
var err error
fs := r.GetFilesystem()
mostRecent, err := sendArgsFromPDUAndValidateExists(ctx, fs, r.GetSenderVersion())
if err != nil {
msg := "HintMostRecentCommonAncestor rpc with nonexistent most recent version"
getLogger(ctx).WithField("fs", fs).WithField("hinted_most_recent", fmt.Sprintf("%#v", mostRecent)).
Warn(msg)
return nil, errors.Wrap(err, msg)
}
// move replication cursor to this position
_, err = MoveReplicationCursor(ctx, fs, mostRecent, p.jobId)
if err == zfs.ErrBookmarkCloningNotSupported {
getLogger(ctx).Debug("not creating replication cursor from bookmark because ZFS does not support it")
// fallthrough
} else if err != nil {
return nil, errors.Wrap(err, "cannot set replication cursor to hinted version")
}
// cleanup previous steps
if err := ReleaseStepAll(ctx, fs, mostRecent, p.jobId); err != nil {
return nil, errors.Wrap(err, "cannot cleanup prior invocation's step holds and bookmarks")
}
return &pdu.HintMostRecentCommonAncestorRes{}, nil
}
var maxConcurrentZFSSendSemaphore = semaphore.New(envconst.Int64("ZREPL_ENDPOINT_MAX_CONCURRENT_SEND", 10))
func uncheckedSendArgsFromPDU(fsv *pdu.FilesystemVersion) *zfs.ZFSSendArgVersion {
if fsv == nil {
return nil
}
return &zfs.ZFSSendArgVersion{RelName: fsv.GetRelName(), GUID: fsv.Guid}
}
func sendArgsFromPDUAndValidateExists(ctx context.Context, fs string, fsv *pdu.FilesystemVersion) (*zfs.ZFSSendArgVersion, error) {
v := uncheckedSendArgsFromPDU(fsv)
if v == nil {
return nil, errors.New("must not be nil")
}
if err := v.ValidateExists(ctx, fs); err != nil {
return nil, err
}
return v, nil
}
func (s *Sender) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, zfs.StreamCopier, error) {
_, err := s.filterCheckFS(r.Filesystem)
if err != nil {
return nil, nil, err
}
switch r.Encrypted {
case pdu.Tri_DontCare:
// use s.encrypt setting
// ok, fallthrough outer
case pdu.Tri_False:
if s.encrypt.B {
return nil, nil, errors.New("only encrytped sends allowed (send -w + encryption!= off), but unencrytped send requested")
}
// fallthrough outer
case pdu.Tri_True:
if !s.encrypt.B {
return nil, nil, errors.New("only unencrypted sends allowed, but encrypted send requested")
}
// fallthrough outer
default:
return nil, nil, fmt.Errorf("unknown pdu.Tri variant %q", r.Encrypted)
}
sendArgs := zfs.ZFSSendArgs{
FS: r.Filesystem,
From: uncheckedSendArgsFromPDU(r.GetFrom()), // validated by zfs.ZFSSendDry / zfs.ZFSSend
To: uncheckedSendArgsFromPDU(r.GetTo()), // validated by zfs.ZFSSendDry / zfs.ZFSSend
Encrypted: s.encrypt,
ResumeToken: r.ResumeToken, // nil or not nil, depending on decoding success
}
getLogger(ctx).Debug("acquire concurrent send semaphore")
// TODO use try-acquire and fail with resource-exhaustion rpc status
@ -95,28 +200,154 @@ func (s *Sender) Send(ctx context.Context, r *pdu.SendReq) (*pdu.SendRes, zfs.St
}
defer guard.Release()
si, err := zfs.ZFSSendDry(r.Filesystem, r.From, r.To, "")
si, err := zfs.ZFSSendDry(ctx, sendArgs)
if err != nil {
return nil, nil, err
return nil, nil, errors.Wrap(err, "zfs send dry failed")
}
// From now on, assume that sendArgs has been validated by ZFSSendDry
// (because validation invovles shelling out, it's actually a little expensive)
var expSize int64 = 0 // protocol says 0 means no estimate
if si.SizeEstimate != -1 { // but si returns -1 for no size estimate
expSize = si.SizeEstimate
}
res := &pdu.SendRes{ExpectedSize: expSize}
res := &pdu.SendRes{
ExpectedSize: expSize,
UsedResumeToken: r.ResumeToken != "",
}
if r.DryRun {
return res, nil, nil
}
streamCopier, err := zfs.ZFSSend(ctx, r.Filesystem, r.From, r.To, "")
// update replication cursor
if sendArgs.From != nil {
// For all but the first replication, this should always be a no-op because SendCompleted already moved the cursor
_, err = MoveReplicationCursor(ctx, sendArgs.FS, sendArgs.From, s.jobId)
if err == zfs.ErrBookmarkCloningNotSupported {
getLogger(ctx).Debug("not creating replication cursor from bookmark because ZFS does not support it")
// fallthrough
} else if err != nil {
return nil, nil, errors.Wrap(err, "cannot set replication cursor to `from` version before starting send")
}
}
// make sure `From` doesn't go away in order to make this step resumable
if sendArgs.From != nil {
err := HoldStep(ctx, sendArgs.FS, sendArgs.From, s.jobId)
if err == zfs.ErrBookmarkCloningNotSupported {
getLogger(ctx).Debug("not creating step bookmark because ZFS does not support it")
// fallthrough
} else if err != nil {
return nil, nil, errors.Wrap(err, "cannot create step bookmark")
}
}
// make sure `To` doesn't go away in order to make this step resumable
err = HoldStep(ctx, sendArgs.FS, sendArgs.To, s.jobId)
if err != nil {
return nil, nil, err
return nil, nil, errors.Wrapf(err, "cannot hold `to` version %q before starting send", sendArgs.To.RelName)
}
// step holds & replication cursor released / moved forward in s.SendCompleted => s.moveCursorAndReleaseSendHolds
streamCopier, err := zfs.ZFSSend(ctx, sendArgs)
if err != nil {
return nil, nil, errors.Wrap(err, "zfs send failed")
}
return res, streamCopier, nil
}
func (p *Sender) SendCompleted(ctx context.Context, r *pdu.SendCompletedReq) (*pdu.SendCompletedRes, error) {
orig := r.GetOriginalReq() // may be nil, always use proto getters
fs := orig.GetFilesystem()
var err error
var from *zfs.ZFSSendArgVersion
if orig.GetFrom() != nil {
from, err = sendArgsFromPDUAndValidateExists(ctx, fs, orig.GetFrom()) // no shadow
if err != nil {
return nil, errors.Wrap(err, "validate `from` exists")
}
}
to, err := sendArgsFromPDUAndValidateExists(ctx, fs, orig.GetTo())
if err != nil {
return nil, errors.Wrap(err, "validate `to` exists")
}
log := getLogger(ctx).WithField("to_guid", to.GUID).
WithField("fs", fs).
WithField("to", to.RelName)
if from != nil {
log = log.WithField("from", from.RelName).WithField("from_guid", from.GUID)
}
log.Debug("move replication cursor to most recent common version")
destroyedCursors, err := MoveReplicationCursor(ctx, fs, to, p.jobId)
if err != nil {
if err == zfs.ErrBookmarkCloningNotSupported {
log.Debug("not setting replication cursor, bookmark cloning not supported")
} else {
msg := "cannot move replication cursor, keeping hold on `to` until successful"
log.WithError(err).Error(msg)
err = errors.Wrap(err, msg)
// it is correct to not release the hold if we can't move the cursor!
return &pdu.SendCompletedRes{}, err
}
} else {
log.Info("successfully moved replication cursor")
}
// kick off releasing of step holds / bookmarks
// if we fail to release them, don't bother the caller:
// they are merely an implementation detail on the sender for better resumability
var wg sync.WaitGroup
wg.Add(2)
go func() {
defer wg.Done()
log.Debug("release step-hold of or step-bookmark on `to`")
err = ReleaseStep(ctx, fs, to, p.jobId)
if err != nil {
log.WithError(err).Error("cannot release step-holds on or destroy step-bookmark of `to`")
} else {
log.Info("successfully released step-holds on or destroyed step-bookmark of `to`")
}
}()
go func() {
defer wg.Done()
if from == nil {
return
}
log.Debug("release step-hold of or step-bookmark on `from`")
err := ReleaseStep(ctx, fs, from, p.jobId)
if err != nil {
if dne, ok := err.(*zfs.DatasetDoesNotExist); ok {
// If bookmark cloning is not supported, `from` might be the old replication cursor
// and thus have already been destroyed by MoveReplicationCursor above
// In that case, nonexistence of `from` is not an error, otherwise it is.
fsp, err := zfs.NewDatasetPath(fs)
if err != nil {
panic(err) // fs has been validated multiple times above
}
for _, fsv := range destroyedCursors {
if fsv.ToAbsPath(fsp) == dne.Path {
log.Info("`from` was a replication cursor and has already been destroyed")
return
}
}
// fallthrough
}
log.WithError(err).Error("cannot release step-holds on or destroy step-bookmark of `from`")
} else {
log.Info("successfully released step-holds on or destroyed step-bookmark of `from`")
}
}()
wg.Wait()
return &pdu.SendCompletedRes{}, nil
}
func (p *Sender) DestroySnapshots(ctx context.Context, req *pdu.DestroySnapshotsReq) (*pdu.DestroySnapshotsRes, error) {
dp, err := p.filterCheckFS(req.Filesystem)
if err != nil {
@ -146,9 +377,7 @@ func (p *Sender) ReplicationCursor(ctx context.Context, req *pdu.ReplicationCurs
return nil, err
}
switch op := req.Op.(type) {
case *pdu.ReplicationCursorReq_Get:
cursor, err := zfs.ZFSGetReplicationCursor(dp)
cursor, err := GetMostRecentReplicationCursorOfJob(ctx, dp.ToString(), p.jobId)
if err != nil {
return nil, err
}
@ -156,15 +385,6 @@ func (p *Sender) ReplicationCursor(ctx context.Context, req *pdu.ReplicationCurs
return &pdu.ReplicationCursorRes{Result: &pdu.ReplicationCursorRes_Notexist{Notexist: true}}, 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 {
return nil, err
}
return &pdu.ReplicationCursorRes{Result: &pdu.ReplicationCursorRes_Guid{Guid: guid}}, nil
default:
return nil, errors.Errorf("unknown op %T", op)
}
}
func (p *Sender) Receive(ctx context.Context, r *pdu.ReceiveReq, receive zfs.StreamCopier) (*pdu.ReceiveRes, error) {
@ -183,21 +403,41 @@ type FSMap interface { // FIXME unused
AsFilter() FSFilter
}
type ReceiverConfig struct {
JobID JobID
RootWithoutClientComponent *zfs.DatasetPath // TODO use
AppendClientIdentity bool
UpdateLastReceivedHold bool
}
func (c *ReceiverConfig) copyIn() {
c.RootWithoutClientComponent = c.RootWithoutClientComponent.Copy()
}
func (c *ReceiverConfig) Validate() error {
c.JobID.MustValidate()
if c.RootWithoutClientComponent.Length() <= 0 {
return errors.New("RootWithoutClientComponent must not be an empty dataset path")
}
return nil
}
// Receiver implements replication.ReplicationEndpoint for a receiving side
type Receiver struct {
rootWithoutClientComponent *zfs.DatasetPath
appendClientIdentity bool
conf ReceiverConfig // validated
recvParentCreationMtx *chainlock.L
}
func NewReceiver(rootDataset *zfs.DatasetPath, appendClientIdentity bool) *Receiver {
if rootDataset.Length() <= 0 {
panic(fmt.Sprintf("root dataset must not be an empty path: %v", rootDataset))
func NewReceiver(config ReceiverConfig) *Receiver {
config.copyIn()
if err := config.Validate(); err != nil {
panic(err)
}
return &Receiver{
rootWithoutClientComponent: rootDataset.Copy(),
appendClientIdentity: appendClientIdentity,
conf: config,
recvParentCreationMtx: chainlock.New(),
}
}
@ -221,8 +461,8 @@ func clientRoot(rootFS *zfs.DatasetPath, clientIdentity string) (*zfs.DatasetPat
}
func (s *Receiver) clientRootFromCtx(ctx context.Context) *zfs.DatasetPath {
if !s.appendClientIdentity {
return s.rootWithoutClientComponent.Copy()
if !s.conf.AppendClientIdentity {
return s.conf.RootWithoutClientComponent.Copy()
}
clientIdentity, ok := ctx.Value(ClientIdentityKey).(string)
@ -230,7 +470,7 @@ func (s *Receiver) clientRootFromCtx(ctx context.Context) *zfs.DatasetPath {
panic(fmt.Sprintf("ClientIdentityKey context value must be set"))
}
clientRoot, err := clientRoot(s.rootWithoutClientComponent, clientIdentity)
clientRoot, err := clientRoot(s.conf.RootWithoutClientComponent, clientIdentity)
if err != nil {
panic(fmt.Sprintf("ClientIdentityContextKey must have been validated before invoking Receiver: %s", err))
}
@ -282,8 +522,27 @@ func (s *Receiver) ListFilesystems(ctx context.Context, req *pdu.ListFilesystemR
err := errors.Errorf("inconsistent placeholder state: filesystem %q must exist in this context", a.ToString())
return nil, err
}
token, err := zfs.ZFSGetReceiveResumeTokenOrEmptyStringIfNotSupported(ctx, a)
if err != nil {
l.WithError(err).Error("cannot get receive resume token")
return nil, err
}
encEnabled, err := zfs.ZFSGetEncryptionEnabled(ctx, a.ToString())
if err != nil {
l.WithError(err).Error("cannot get encryption enabled status")
return nil, err
}
l.WithField("receive_resume_token", token).Debug("receive resume token")
a.TrimPrefix(root)
fss = append(fss, &pdu.Filesystem{Path: a.ToString(), IsPlaceholder: ph.IsPlaceholder})
fs := &pdu.Filesystem{
Path: a.ToString(),
IsPlaceholder: ph.IsPlaceholder,
ResumeToken: token,
IsEncrypted: encEnabled,
}
fss = append(fss, fs)
}
if len(fss) == 0 {
getLogger(ctx).Debug("no filesystems found")
@ -344,7 +603,18 @@ func (s *Receiver) Receive(ctx context.Context, req *pdu.ReceiveReq, receive zfs
root := s.clientRootFromCtx(ctx)
lp, err := subroot{root}.MapToLocal(req.Filesystem)
if err != nil {
return nil, err
return nil, errors.Wrap(err, "`Filesystem` invalid")
}
to := uncheckedSendArgsFromPDU(req.GetTo())
if to == nil {
return nil, errors.New("`To` must not be nil")
}
if err := to.ValidateInMemory(lp.ToString()); err != nil {
return nil, errors.Wrap(err, "`To` invalid")
}
if !to.IsSnapshot() {
return nil, errors.New("`To` must be a snapshot")
}
// create placeholder parent filesystems as appropriate
@ -380,11 +650,11 @@ func (s *Receiver) Receive(ctx context.Context, req *pdu.ReceiveReq, receive zfs
}
if !ph.FSExists {
if s.rootWithoutClientComponent.HasPrefix(v.Path) {
if s.conf.RootWithoutClientComponent.HasPrefix(v.Path) {
if v.Path.Length() == 1 {
visitErr = fmt.Errorf("pool %q not imported", v.Path.ToString())
} else {
visitErr = fmt.Errorf("root_fs %q does not exist", s.rootWithoutClientComponent.ToString())
visitErr = fmt.Errorf("root_fs %q does not exist", s.conf.RootWithoutClientComponent.ToString())
}
getLogger(ctx).WithError(visitErr).Error("placeholders are only created automatically below root_fs")
return false
@ -422,6 +692,17 @@ func (s *Receiver) Receive(ctx context.Context, req *pdu.ReceiveReq, receive zfs
}
}
if req.ClearResumeToken && ph.FSExists {
if err := zfs.ZFSRecvClearResumeToken(lp.ToString()); err != nil {
return nil, errors.Wrap(err, "cannot clear resume token")
}
}
recvOpts.SavePartialRecvState, err = zfs.ResumeRecvSupported(ctx, lp)
if err != nil {
return nil, errors.Wrap(err, "cannot determine whether we can use resumable send & recv")
}
getLogger(ctx).Debug("acquire concurrent recv semaphore")
// TODO use try-acquire and fail with resource-exhaustion rpc status
// => would require handling on the client-side
@ -434,13 +715,30 @@ func (s *Receiver) Receive(ctx context.Context, req *pdu.ReceiveReq, receive zfs
getLogger(ctx).WithField("opts", fmt.Sprintf("%#v", recvOpts)).Debug("start receive command")
if err := zfs.ZFSRecv(ctx, lp.ToString(), receive, recvOpts); err != nil {
snapFullPath := to.FullPath(lp.ToString())
if err := zfs.ZFSRecv(ctx, lp.ToString(), to, receive, recvOpts); err != nil {
getLogger(ctx).
WithError(err).
WithField("opts", recvOpts).
Error("zfs receive failed")
return nil, err
}
// validate that we actually received what the sender claimed
if err := to.ValidateExists(ctx, lp.ToString()); err != nil {
msg := "receive request's `To` version does not match what we received in the stream"
getLogger(ctx).WithError(err).WithField("snap", snapFullPath).Error(msg)
getLogger(ctx).Error("aborting recv request, but keeping received snapshot for inspection")
return nil, errors.Wrap(err, msg)
}
if s.conf.UpdateLastReceivedHold {
getLogger(ctx).Debug("move last-received-hold")
if err := MoveLastReceivedHold(ctx, lp.ToString(), *to, s.conf.JobID); err != nil {
return nil, errors.Wrap(err, "cannot move last-received-hold")
}
}
return &pdu.ReceiveRes{}, nil
}
@ -453,6 +751,19 @@ func (s *Receiver) DestroySnapshots(ctx context.Context, req *pdu.DestroySnapsho
return doDestroySnapshots(ctx, lp, req.Snapshots)
}
func (p *Receiver) HintMostRecentCommonAncestor(ctx context.Context, r *pdu.HintMostRecentCommonAncestorReq) (*pdu.HintMostRecentCommonAncestorRes, error) {
// we don't move last-received-hold as part of this hint
// because that wouldn't give us any benefit wrt resumability.
//
// Other reason: the replication logic that issues this RPC would require refactoring
// to include the receiver's FilesystemVersion in the request)
return &pdu.HintMostRecentCommonAncestorRes{}, nil
}
func (p *Receiver) SendCompleted(context.Context, *pdu.SendCompletedReq) (*pdu.SendCompletedRes, error) {
return &pdu.SendCompletedRes{}, nil
}
func doDestroySnapshots(ctx context.Context, lp *zfs.DatasetPath, snaps []*pdu.FilesystemVersion) (*pdu.DestroySnapshotsRes, error) {
reqs := make([]*zfs.DestroySnapOp, len(snaps))
ress := make([]*pdu.DestroySnapshotRes, len(snaps))

503
endpoint/endpoint_zfs.go Normal file
View File

@ -0,0 +1,503 @@
package endpoint
import (
"context"
"fmt"
"regexp"
"sort"
"github.com/kr/pretty"
"github.com/pkg/errors"
"github.com/zrepl/zrepl/zfs"
)
var stepHoldTagRE = regexp.MustCompile("^zrepl_STEP_J_(.+)")
func StepHoldTag(jobid JobID) (string, error) {
return stepHoldTagImpl(jobid.String())
}
func stepHoldTagImpl(jobid string) (string, error) {
t := fmt.Sprintf("zrepl_STEP_J_%s", jobid)
if err := zfs.ValidHoldTag(t); err != nil {
return "", err
}
return t, nil
}
// err != nil always means that the bookmark is not a step bookmark
func ParseStepHoldTag(tag string) (JobID, error) {
match := stepHoldTagRE.FindStringSubmatch(tag)
if match == nil {
return JobID{}, fmt.Errorf("parse hold tag: match regex %q", stepHoldTagRE)
}
jobID, err := MakeJobID(match[1])
if err != nil {
return JobID{}, errors.Wrap(err, "parse hold tag: invalid job id field")
}
return jobID, nil
}
const stepBookmarkNamePrefix = "zrepl_STEP"
// v must be validated by caller
func StepBookmarkName(fs string, guid uint64, id JobID) (string, error) {
return stepBookmarkNameImpl(fs, guid, id.String())
}
func stepBookmarkNameImpl(fs string, guid uint64, jobid string) (string, error) {
return makeJobAndGuidBookmarkName(stepBookmarkNamePrefix, fs, guid, jobid)
}
// name is the full bookmark name, including dataset path
//
// err != nil always means that the bookmark is not a step bookmark
func ParseStepBookmarkName(fullname string) (guid uint64, jobID JobID, err error) {
guid, jobID, err = parseJobAndGuidBookmarkName(fullname, stepBookmarkNamePrefix)
if err != nil {
err = errors.Wrap(err, "parse step bookmark name") // no shadow!
}
return guid, jobID, err
}
const replicationCursorBookmarkNamePrefix = "zrepl_CURSOR"
func ReplicationCursorBookmarkName(fs string, guid uint64, id JobID) (string, error) {
return replicationCursorBookmarkNameImpl(fs, guid, id.String())
}
func replicationCursorBookmarkNameImpl(fs string, guid uint64, jobid string) (string, error) {
return makeJobAndGuidBookmarkName(replicationCursorBookmarkNamePrefix, fs, guid, jobid)
}
var ErrV1ReplicationCursor = fmt.Errorf("bookmark name is a v1-replication cursor")
//err != nil always means that the bookmark is not a valid replication bookmark
//
// Returns ErrV1ReplicationCursor as error if the bookmark is a v1 replication cursor
func ParseReplicationCursorBookmarkName(fullname string) (uint64, JobID, error) {
// check for legacy cursors
{
if err := zfs.EntityNamecheck(fullname, zfs.EntityTypeBookmark); err != nil {
return 0, JobID{}, errors.Wrap(err, "parse replication cursor bookmark name")
}
_, _, name, err := zfs.DecomposeVersionString(fullname)
if err != nil {
return 0, JobID{}, errors.Wrap(err, "parse replication cursor bookmark name: decompose version string")
}
const V1ReplicationCursorBookmarkName = "zrepl_replication_cursor"
if name == V1ReplicationCursorBookmarkName {
return 0, JobID{}, ErrV1ReplicationCursor
}
}
guid, jobID, err := parseJobAndGuidBookmarkName(fullname, replicationCursorBookmarkNamePrefix)
if err != nil {
err = errors.Wrap(err, "parse replication cursor bookmark name") // no shadow
}
return guid, jobID, err
}
// may return nil for both values, indicating there is no cursor
func GetMostRecentReplicationCursorOfJob(ctx context.Context, fs string, jobID JobID) (*zfs.FilesystemVersion, error) {
fsp, err := zfs.NewDatasetPath(fs)
if err != nil {
return nil, err
}
candidates, err := GetReplicationCursors(ctx, fsp, jobID)
if err != nil || len(candidates) == 0 {
return nil, err
}
sort.Slice(candidates, func(i, j int) bool {
return candidates[i].CreateTXG < candidates[j].CreateTXG
})
mostRecent := candidates[len(candidates)-1]
return &mostRecent, nil
}
func GetReplicationCursors(ctx context.Context, fs *zfs.DatasetPath, jobID JobID) ([]zfs.FilesystemVersion, error) {
listOut := &ListHoldsAndBookmarksOutput{}
if err := listZFSHoldsAndBookmarksImplFS(ctx, listOut, fs); err != nil {
return nil, errors.Wrap(err, "get replication cursor: list bookmarks and holds")
}
if len(listOut.V1ReplicationCursors) > 0 {
getLogger(ctx).WithField("bookmark", pretty.Sprint(listOut.V1ReplicationCursors)).
Warn("found v1-replication cursor bookmarks, consider running migration 'replication-cursor:v1-v2' after successful replication with this zrepl version")
}
candidates := make([]zfs.FilesystemVersion, 0)
for _, v := range listOut.ReplicationCursorBookmarks {
zv := zfs.ZFSSendArgVersion{
RelName: "#" + v.Name,
GUID: v.Guid,
}
if err := zv.ValidateExists(ctx, v.FS); err != nil {
getLogger(ctx).WithError(err).WithField("bookmark", zv.FullPath(v.FS)).
Error("found invalid replication cursor bookmark")
continue
}
candidates = append(candidates, v.v)
}
return candidates, nil
}
// `target` is validated before replication cursor is set. if validation fails, the cursor is not moved.
//
// returns ErrBookmarkCloningNotSupported if version is a bookmark and bookmarking bookmarks is not supported by ZFS
func MoveReplicationCursor(ctx context.Context, fs string, target *zfs.ZFSSendArgVersion, jobID JobID) (destroyedCursors []zfs.FilesystemVersion, err error) {
if !target.IsSnapshot() {
return nil, zfs.ErrBookmarkCloningNotSupported
}
snapProps, err := target.ValidateExistsAndGetCheckedProps(ctx, fs)
if err != nil {
return nil, errors.Wrapf(err, "invalid replication cursor target %q (guid=%v)", target.RelName, target.GUID)
}
bookmarkname, err := ReplicationCursorBookmarkName(fs, snapProps.Guid, jobID)
if err != nil {
return nil, errors.Wrap(err, "determine replication cursor name")
}
// idempotently create bookmark (guid is encoded in it, hence we'll most likely add a new one
// cleanup the old one afterwards
err = zfs.ZFSBookmark(fs, *target, bookmarkname)
if err != nil {
if err == zfs.ErrBookmarkCloningNotSupported {
return nil, err // TODO go1.13 use wrapping
}
return nil, errors.Wrapf(err, "cannot create bookmark")
}
destroyedCursors, err = DestroyObsoleteReplicationCursors(ctx, fs, target, jobID)
if err != nil {
return nil, errors.Wrap(err, "destroy obsolete replication cursors")
}
return destroyedCursors, nil
}
func DestroyObsoleteReplicationCursors(ctx context.Context, fs string, target *zfs.ZFSSendArgVersion, jobID JobID) (destroyed []zfs.FilesystemVersion, err error) {
return destroyBookmarksOlderThan(ctx, fs, target, jobID, func(shortname string) (accept bool) {
_, parsedID, err := ParseReplicationCursorBookmarkName(fs + "#" + shortname)
return err == nil && parsedID == jobID
})
}
// idempotently hold / step-bookmark `version`
//
// returns ErrBookmarkCloningNotSupported if version is a bookmark and bookmarking bookmarks is not supported by ZFS
func HoldStep(ctx context.Context, fs string, v *zfs.ZFSSendArgVersion, jobID JobID) error {
if err := v.ValidateExists(ctx, fs); err != nil {
return err
}
if v.IsSnapshot() {
tag, err := StepHoldTag(jobID)
if err != nil {
return errors.Wrap(err, "step hold tag")
}
if err := zfs.ZFSHold(ctx, fs, *v, tag); err != nil {
return errors.Wrap(err, "step hold: zfs")
}
return nil
}
v.MustBeBookmark()
bmname, err := StepBookmarkName(fs, v.GUID, jobID)
if err != nil {
return errors.Wrap(err, "create step bookmark: determine bookmark name")
}
// idempotently create bookmark
err = zfs.ZFSBookmark(fs, *v, bmname)
if err != nil {
if err == zfs.ErrBookmarkCloningNotSupported {
// TODO we could actually try to find a local snapshot that has the requested GUID
// however, the replication algorithm prefers snapshots anyways, so this quest
// is most likely not going to be successful. Also, there's the possibility that
// the caller might want to filter what snapshots are eligibile, and this would
// complicate things even further.
return err // TODO go1.13 use wrapping
}
return errors.Wrap(err, "create step bookmark: zfs")
}
return nil
}
// idempotently release the step-hold on v if v is a snapshot
// or idempotently destroy the step-bookmark of v if v is a bookmark
//
// note that this operation leaves v itself untouched, unless v is the step-bookmark itself, in which case v is destroyed
//
// returns an instance of *zfs.DatasetDoesNotExist if `v` does not exist
func ReleaseStep(ctx context.Context, fs string, v *zfs.ZFSSendArgVersion, jobID JobID) error {
if err := v.ValidateExists(ctx, fs); err != nil {
return err
}
if v.IsSnapshot() {
tag, err := StepHoldTag(jobID)
if err != nil {
return errors.Wrap(err, "step release tag")
}
if err := zfs.ZFSRelease(ctx, tag, v.FullPath(fs)); err != nil {
return errors.Wrap(err, "step release: zfs")
}
return nil
}
v.MustBeBookmark()
bmname, err := StepBookmarkName(fs, v.GUID, jobID)
if err != nil {
return errors.Wrap(err, "step release: determine bookmark name")
}
// idempotently destroy bookmark
if err := zfs.ZFSDestroyIdempotent(bmname); err != nil {
return errors.Wrap(err, "step release: bookmark destroy: zfs")
}
return nil
}
// release {step holds, step bookmarks} earlier and including `mostRecent`
func ReleaseStepAll(ctx context.Context, fs string, mostRecent *zfs.ZFSSendArgVersion, jobID JobID) error {
if err := mostRecent.ValidateInMemory(fs); err != nil {
return err
}
tag, err := StepHoldTag(jobID)
if err != nil {
return errors.Wrap(err, "step release all: tag")
}
err = zfs.ZFSReleaseAllOlderAndIncludingGUID(ctx, fs, mostRecent.GUID, tag)
if err != nil {
return errors.Wrapf(err, "step release all: release holds older and including %q", mostRecent.FullPath(fs))
}
_, err = destroyBookmarksOlderThan(ctx, fs, mostRecent, jobID, func(shortname string) bool {
_, parsedId, parseErr := ParseStepBookmarkName(fs + "#" + shortname)
return parseErr == nil && parsedId == jobID
})
if err != nil {
return errors.Wrapf(err, "step release all: destroy bookmarks older than %q", mostRecent.FullPath(fs))
}
return nil
}
var lastReceivedHoldTagRE = regexp.MustCompile("^zrepl_last_received_J_(.+)$")
// err != nil always means that the bookmark is not a step bookmark
func ParseLastReceivedHoldTag(tag string) (JobID, error) {
match := lastReceivedHoldTagRE.FindStringSubmatch(tag)
if match == nil {
return JobID{}, errors.Errorf("parse last-received-hold tag: does not match regex %s", lastReceivedHoldTagRE.String())
}
jobId, err := MakeJobID(match[1])
if err != nil {
return JobID{}, errors.Wrap(err, "parse last-received-hold tag: invalid job id field")
}
return jobId, nil
}
func LastReceivedHoldTag(jobID JobID) (string, error) {
return lastReceivedHoldImpl(jobID.String())
}
func lastReceivedHoldImpl(jobid string) (string, error) {
tag := fmt.Sprintf("zrepl_last_received_J_%s", jobid)
if err := zfs.ValidHoldTag(tag); err != nil {
return "", err
}
return tag, nil
}
func MoveLastReceivedHold(ctx context.Context, fs string, to zfs.ZFSSendArgVersion, jobID JobID) error {
if err := to.ValidateExists(ctx, fs); err != nil {
return err
}
if err := zfs.EntityNamecheck(to.FullPath(fs), zfs.EntityTypeSnapshot); err != nil {
return err
}
tag, err := LastReceivedHoldTag(jobID)
if err != nil {
return errors.Wrap(err, "last-received-hold: hold tag")
}
// we never want to be without a hold
// => hold new one before releasing old hold
err = zfs.ZFSHold(ctx, fs, to, tag)
if err != nil {
return errors.Wrap(err, "last-received-hold: hold newly received")
}
err = zfs.ZFSReleaseAllOlderThanGUID(ctx, fs, to.GUID, tag)
if err != nil {
return errors.Wrap(err, "last-received-hold: release older holds")
}
return nil
}
type ListHoldsAndBookmarksOutputBookmarkV1ReplicationCursor struct {
FS string
Name string
}
type ListHoldsAndBookmarksOutput struct {
StepBookmarks []*ListHoldsAndBookmarksOutputBookmark
StepHolds []*ListHoldsAndBookmarksOutputHold
ReplicationCursorBookmarks []*ListHoldsAndBookmarksOutputBookmark
V1ReplicationCursors []*ListHoldsAndBookmarksOutputBookmarkV1ReplicationCursor
LastReceivedHolds []*ListHoldsAndBookmarksOutputHold
}
type ListHoldsAndBookmarksOutputBookmark struct {
FS, Name string
Guid uint64
JobID JobID
v zfs.FilesystemVersion
}
type ListHoldsAndBookmarksOutputHold struct {
FS string
Snap string
SnapGuid uint64
SnapCreateTXG uint64
Tag string
JobID JobID
}
// List all holds and bookmarks managed by endpoint
func ListZFSHoldsAndBookmarks(ctx context.Context, fsfilter zfs.DatasetFilter) (*ListHoldsAndBookmarksOutput, error) {
// initialize all fields so that JSON serializion of output looks pretty (see client/holds.go)
// however, listZFSHoldsAndBookmarksImplFS shouldn't rely on it
out := &ListHoldsAndBookmarksOutput{
StepBookmarks: make([]*ListHoldsAndBookmarksOutputBookmark, 0),
StepHolds: make([]*ListHoldsAndBookmarksOutputHold, 0),
ReplicationCursorBookmarks: make([]*ListHoldsAndBookmarksOutputBookmark, 0),
V1ReplicationCursors: make([]*ListHoldsAndBookmarksOutputBookmarkV1ReplicationCursor, 0),
LastReceivedHolds: make([]*ListHoldsAndBookmarksOutputHold, 0),
}
fss, err := zfs.ZFSListMapping(ctx, fsfilter)
if err != nil {
return nil, errors.Wrap(err, "list filesystems")
}
for _, fs := range fss {
err := listZFSHoldsAndBookmarksImplFS(ctx, out, fs)
if err != nil {
return nil, errors.Wrapf(err, "list holds and bookmarks on %q", fs.ToString())
}
}
return out, nil
}
func listZFSHoldsAndBookmarksImplFS(ctx context.Context, out *ListHoldsAndBookmarksOutput, fs *zfs.DatasetPath) error {
fsvs, err := zfs.ZFSListFilesystemVersions(fs, nil)
if err != nil {
return errors.Wrapf(err, "list filesystem versions of %q", fs)
}
for _, v := range fsvs {
switch v.Type {
case zfs.Bookmark:
listZFSHoldsAndBookmarksImplTryParseBookmark(ctx, out, fs, v)
case zfs.Snapshot:
holds, err := zfs.ZFSHolds(ctx, fs.ToString(), v.Name)
if err != nil {
return errors.Wrapf(err, "get holds of %q", v.ToAbsPath(fs))
}
for _, tag := range holds {
listZFSHoldsAndBookmarksImplSnapshotTryParseHold(ctx, out, fs, v, tag)
}
default:
continue
}
}
return nil
}
// pure function, err != nil always indicates parsing error
func listZFSHoldsAndBookmarksImplTryParseBookmark(ctx context.Context, out *ListHoldsAndBookmarksOutput, fs *zfs.DatasetPath, v zfs.FilesystemVersion) {
var err error
if v.Type != zfs.Bookmark {
panic("impl error")
}
fullname := v.ToAbsPath(fs)
bm := &ListHoldsAndBookmarksOutputBookmark{
FS: fs.ToString(), Name: v.Name, v: v,
}
bm.Guid, bm.JobID, err = ParseStepBookmarkName(fullname)
if err == nil {
out.StepBookmarks = append(out.StepBookmarks, bm)
return
}
bm.Guid, bm.JobID, err = ParseReplicationCursorBookmarkName(fullname)
if err == nil {
out.ReplicationCursorBookmarks = append(out.ReplicationCursorBookmarks, bm)
return
} else if err == ErrV1ReplicationCursor {
v1rc := &ListHoldsAndBookmarksOutputBookmarkV1ReplicationCursor{
FS: fs.ToString(), Name: v.Name,
}
out.V1ReplicationCursors = append(out.V1ReplicationCursors, v1rc)
return
}
}
// pure function, err != nil always indicates parsing error
func listZFSHoldsAndBookmarksImplSnapshotTryParseHold(ctx context.Context, out *ListHoldsAndBookmarksOutput, fs *zfs.DatasetPath, v zfs.FilesystemVersion, holdTag string) {
var err error
if v.Type != zfs.Snapshot {
panic("impl error")
}
hold := &ListHoldsAndBookmarksOutputHold{
FS: fs.ToString(),
Snap: v.Name,
SnapGuid: v.Guid,
SnapCreateTXG: v.CreateTXG,
Tag: holdTag,
}
hold.JobID, err = ParseStepHoldTag(holdTag)
if err == nil {
out.StepHolds = append(out.StepHolds, hold)
return
}
hold.JobID, err = ParseLastReceivedHoldTag(holdTag)
if err == nil {
out.LastReceivedHolds = append(out.LastReceivedHolds, hold)
return
}
}

View File

@ -0,0 +1,109 @@
package endpoint
import (
"context"
"fmt"
"regexp"
"strconv"
"github.com/pkg/errors"
"github.com/zrepl/zrepl/zfs"
)
// returns the short name (no fs# prefix)
func makeJobAndGuidBookmarkName(prefix string, fs string, guid uint64, jobid string) (string, error) {
bmname := fmt.Sprintf(prefix+"_G_%016x_J_%s", guid, jobid)
if err := zfs.EntityNamecheck(fmt.Sprintf("%s#%s", fs, bmname), zfs.EntityTypeBookmark); err != nil {
return "", err
}
return bmname, nil
}
var jobAndGuidBookmarkRE = regexp.MustCompile(`(.+)_G_([0-9a-f]{16})_J_(.+)$`)
func parseJobAndGuidBookmarkName(fullname string, prefix string) (guid uint64, jobID JobID, _ error) {
if len(prefix) == 0 {
panic("prefix must not be empty")
}
if err := zfs.EntityNamecheck(fullname, zfs.EntityTypeBookmark); err != nil {
return 0, JobID{}, err
}
_, _, name, err := zfs.DecomposeVersionString(fullname)
if err != nil {
return 0, JobID{}, errors.Wrap(err, "decompose bookmark name")
}
match := jobAndGuidBookmarkRE.FindStringSubmatch(name)
if match == nil {
return 0, JobID{}, errors.Errorf("bookmark name does not match regex %q", jobAndGuidBookmarkRE.String())
}
if match[1] != prefix {
return 0, JobID{}, errors.Errorf("prefix component does not match: expected %q, got %q", prefix, match[1])
}
guid, err = strconv.ParseUint(match[2], 16, 64)
if err != nil {
return 0, JobID{}, errors.Wrapf(err, "parse guid component: %q", match[2])
}
jobID, err = MakeJobID(match[3])
if err != nil {
return 0, JobID{}, errors.Wrapf(err, "parse jobid component: %q", match[3])
}
return guid, jobID, nil
}
func destroyBookmarksOlderThan(ctx context.Context, fs string, mostRecent *zfs.ZFSSendArgVersion, jobID JobID, filter func(shortname string) (accept bool)) (destroyed []zfs.FilesystemVersion, err error) {
if filter == nil {
panic(filter)
}
fsp, err := zfs.NewDatasetPath(fs)
if err != nil {
return nil, errors.Wrap(err, "invalid filesystem path")
}
mostRecentProps, err := mostRecent.ValidateExistsAndGetCheckedProps(ctx, fs)
if err != nil {
return nil, errors.Wrap(err, "validate most recent version argument")
}
stepBookmarks, err := zfs.ZFSListFilesystemVersions(fsp, zfs.FilterFromClosure(
func(t zfs.VersionType, name string) (accept bool, err error) {
if t != zfs.Bookmark {
return false, nil
}
return filter(name), nil
}))
if err != nil {
return nil, errors.Wrap(err, "list bookmarks")
}
// cut off all bookmarks prior to mostRecent's CreateTXG
var destroy []zfs.FilesystemVersion
for _, v := range stepBookmarks {
if v.Type != zfs.Bookmark {
panic("implementation error")
}
if !filter(v.Name) {
panic("inconsistent filter result")
}
if v.CreateTXG < mostRecentProps.CreateTXG {
destroy = append(destroy, v)
}
}
// FIXME use batch destroy, must adopt code to handle bookmarks
for _, v := range destroy {
if err := zfs.ZFSDestroyIdempotent(v.ToAbsPath(fsp)); err != nil {
return nil, errors.Wrap(err, "destroy bookmark")
}
}
return destroy, nil
}

View File

@ -0,0 +1,51 @@
package endpoint
import (
"testing"
"github.com/stretchr/testify/assert"
)
func TestParseJobAndGuidBookmarkName(t *testing.T) {
type Case struct {
input string
expectErr bool
guid uint64
jobid string
}
cases := []Case{
{
`p1/sync#zrepl_CURSOR_G_932f3a7089080ce2_J_push with legitimate name`,
false, 0x932f3a7089080ce2, "push with legitimate name",
},
{
input: `p1/sync#zrepl_CURSOR_G_932f3a7089_J_push with legitimate name`,
expectErr: true,
},
{
input: `p1/sync#zrepl_CURSOR_G_932f3a7089080ce2_J_push with il\tlegitimate name`,
expectErr: true,
},
{
input: `p1/sync#otherprefix_G_932f3a7089080ce2_J_push with legitimate name`,
expectErr: true,
},
}
for i := range cases {
t.Run(cases[i].input, func(t *testing.T) {
guid, jobid, err := parseJobAndGuidBookmarkName(cases[i].input, replicationCursorBookmarkNamePrefix)
if cases[i].expectErr {
assert.Error(t, err)
} else {
assert.NoError(t, err)
assert.Equal(t, cases[i].guid, guid)
assert.Equal(t, MustMakeJobID(cases[i].jobid), jobid)
}
})
}
}

78
endpoint/jobid.go Normal file
View File

@ -0,0 +1,78 @@
package endpoint
import (
"encoding/json"
"fmt"
"github.com/pkg/errors"
"github.com/zrepl/zrepl/zfs"
)
// An instance of this type returned by MakeJobID guarantees
// that that instance's JobID.String() can be used in a ZFS dataset name and hold tag.
type JobID struct {
jid string
}
func MakeJobID(s string) (JobID, error) {
if len(s) == 0 {
return JobID{}, fmt.Errorf("must not be empty string")
}
if err := zfs.ComponentNamecheck(s); err != nil {
return JobID{}, errors.Wrap(err, "muse be usable as a dataset path component")
}
if _, err := stepBookmarkNameImpl("pool/ds", 0xface601d, s); err != nil {
// note that this might still fail due to total maximum name length, but we can't enforce that
return JobID{}, errors.Wrap(err, "must be usable for a step bookmark")
}
if _, err := stepHoldTagImpl(s); err != nil {
return JobID{}, errors.Wrap(err, "must be usable for a step hold tag")
}
if _, err := lastReceivedHoldImpl(s); err != nil {
return JobID{}, errors.Wrap(err, "must be usabel as a last-recieved-hold tag")
}
// FIXME replication cursor bookmark name
_, err := zfs.NewDatasetPath(s)
if err != nil {
return JobID{}, fmt.Errorf("must be usable in a ZFS dataset path: %s", err)
}
return JobID{s}, nil
}
func MustMakeJobID(s string) JobID {
jid, err := MakeJobID(s)
if err != nil {
panic(err)
}
return jid
}
func (j JobID) expectInitialized() {
if j.jid == "" {
panic("use of unitialized JobID")
}
}
func (j JobID) String() string {
j.expectInitialized()
return j.jid
}
var _ json.Marshaler = JobID{}
var _ json.Unmarshaler = (*JobID)(nil)
func (j JobID) MarshalJSON() ([]byte, error) { return json.Marshal(j.jid) }
func (j *JobID) UnmarshalJSON(b []byte) error {
return json.Unmarshal(b, &j.jid)
}
func (j JobID) MustValidate() { j.expectInitialized() }

2
go.mod
View File

@ -29,7 +29,7 @@ require (
github.com/stretchr/testify v1.4.0
github.com/yudai/gojsondiff v0.0.0-20170107030110-7b1b7adf999d
github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82 // go1.12 thinks it needs this
github.com/zrepl/yaml-config v0.0.0-20190928121844-af7ca3f8448f
github.com/zrepl/yaml-config v0.0.0-20191220194647-cbb6b0cf4bdd
golang.org/x/net v0.0.0-20190613194153-d28f0bde5980
golang.org/x/sync v0.0.0-20190423024810-112230192c58
golang.org/x/sys v0.0.0-20191026070338-33540a1f6037

2
go.sum
View File

@ -281,6 +281,8 @@ github.com/yudai/pp v2.0.1+incompatible h1:Q4//iY4pNF6yPLZIigmvcl7k/bPgrcTPIFIcm
github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc=
github.com/zrepl/yaml-config v0.0.0-20190928121844-af7ca3f8448f h1:3MuiGfgMHCSwKUcsuI7ODbi50j+evTB7SsoOBMNC5Fk=
github.com/zrepl/yaml-config v0.0.0-20190928121844-af7ca3f8448f/go.mod h1:JmNwisZzOvW4GfpfLvhZ+gtyKLsIiA+WC+wNKJGJaFg=
github.com/zrepl/yaml-config v0.0.0-20191220194647-cbb6b0cf4bdd h1:SSo67WLS+99QESvbW8Meibz7zCrxshP71U9dH5KOCXM=
github.com/zrepl/yaml-config v0.0.0-20191220194647-cbb6b0cf4bdd/go.mod h1:JmNwisZzOvW4GfpfLvhZ+gtyKLsIiA+WC+wNKJGJaFg=
github.com/zrepl/zrepl v0.2.0/go.mod h1:M3Zv2IGSO8iYpUjsZD6ayZ2LHy7zyMfzet9XatKOrZ8=
golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=

View File

@ -17,6 +17,7 @@ func init() {
cli.AddSubcommand(client.PprofCmd)
cli.AddSubcommand(client.TestCmd)
cli.AddSubcommand(client.MigrateCmd)
cli.AddSubcommand(client.HoldsCmd)
}
func main() {

View File

@ -43,8 +43,8 @@ func doMain() error {
flag.StringVar(&args.createArgs.PoolName, "poolname", "", "")
flag.StringVar(&args.createArgs.ImagePath, "imagepath", "", "")
flag.Int64Var(&args.createArgs.ImageSize, "imagesize", 100*(1<<20), "")
flag.StringVar(&args.createArgs.Mountpoint, "mountpoint", "none", "")
flag.Int64Var(&args.createArgs.ImageSize, "imagesize", 200*(1<<20), "")
flag.StringVar(&args.createArgs.Mountpoint, "mountpoint", "", "")
flag.BoolVar(&args.stopAndKeepPoolOnFail, "failure.stop-and-keep-pool", false, "if a test case fails, stop test execution and keep pool as it was when the test failed")
flag.StringVar(&args.run, "run", "", "")
flag.Parse()

View File

@ -20,6 +20,11 @@ var SkipNowSentinel = fmt.Errorf("platformtest: SkipNow called on context")
var _ assert.TestingT = (*Context)(nil)
var _ require.TestingT = (*Context)(nil)
func (c *Context) Logf(format string, args ...interface{}) {
msg := fmt.Sprintf(format, args...)
GetLog(c).Info(msg)
}
func (c *Context) Errorf(format string, args ...interface{}) {
GetLog(c).Printf(format, args...)
}

View File

@ -5,6 +5,7 @@ import (
"bytes"
"context"
"fmt"
"io/ioutil"
"os"
"os/exec"
"strings"
@ -48,6 +49,7 @@ func (o *DestroyRootOp) Run(ctx context.Context, e Execer) error {
type FSOp struct {
Op Op
Path string
Encrypted bool // only for Op=Add
}
func (o *FSOp) Run(ctx context.Context, e Execer) error {
@ -57,7 +59,22 @@ func (o *FSOp) Run(ctx context.Context, e Execer) error {
case AssertNotExists:
return e.RunExpectFailureNoOutput(ctx, "zfs", "get", "-H", "name", o.Path)
case Add:
return e.RunExpectSuccessNoOutput(ctx, "zfs", "create", o.Path)
opts := []string{"create"}
if o.Encrypted {
const passphraseFilePath = "/tmp/zreplplatformtest.encryption.passphrase"
const passphrase = "foobar2342"
err := ioutil.WriteFile(passphraseFilePath, []byte(passphrase), 0600)
if err != nil {
panic(err)
}
opts = append(opts,
"-o", "encryption=on",
"-o", "keylocation=file:///"+passphraseFilePath,
"-o", "keyformat=passphrase",
)
}
opts = append(opts, o.Path)
return e.RunExpectSuccessNoOutput(ctx, "zfs", opts...)
case Del:
return e.RunExpectSuccessNoOutput(ctx, "zfs", "destroy", o.Path)
default:
@ -249,7 +266,25 @@ nextLine:
if strings.ContainsAny(comps.Text(), "@") {
stmts = append(stmts, &SnapOp{Op: op, Path: fmt.Sprintf("%s/%s", rootds, comps.Text())})
} else {
stmts = append(stmts, &FSOp{Op: op, Path: fmt.Sprintf("%s/%s", rootds, comps.Text())})
// FS
fs := comps.Text()
var encrypted bool = false
if op == Add {
if comps.Scan() {
t := comps.Text()
switch t {
case "encrypted":
encrypted = true
default:
panic(fmt.Sprintf("unexpected token %q", t))
}
}
}
stmts = append(stmts, &FSOp{
Op: op,
Path: fmt.Sprintf("%s/%s", rootds, fs),
Encrypted: encrypted,
})
}
if comps.Scan() {

View File

@ -2,6 +2,7 @@ package platformtest
import (
"context"
"fmt"
"os"
"path/filepath"
@ -29,8 +30,8 @@ func (a ZpoolCreateArgs) Validate() error {
if a.ImageSize < minImageSize {
return errors.Errorf("ImageSize must be > %v, got %v", minImageSize, a.ImageSize)
}
if a.Mountpoint != "none" {
return errors.Errorf("Mountpoint must be \"none\"")
if a.Mountpoint == "" || a.Mountpoint[0] != '/' {
return errors.Errorf("Mountpoint must be an absolute path to a directory")
}
if a.PoolName == "" {
return errors.Errorf("PoolName must not be emtpy")
@ -69,7 +70,10 @@ func CreateOrReplaceZpool(ctx context.Context, e Execer, args ZpoolCreateArgs) (
image.Close()
// create the pool
err = e.RunExpectSuccessNoOutput(ctx, "zpool", "create", "-f", "-O", "mountpoint=none", args.PoolName, args.ImagePath)
err = e.RunExpectSuccessNoOutput(ctx, "zpool", "create", "-f",
"-O", fmt.Sprintf("mountpoint=%s", args.Mountpoint),
args.PoolName, args.ImagePath,
)
if err != nil {
return nil, errors.Wrap(err, "zpool create")
}

View File

@ -0,0 +1,154 @@
package tests
import (
"fmt"
"github.com/stretchr/testify/require"
"github.com/zrepl/zrepl/platformtest"
"github.com/zrepl/zrepl/zfs"
)
type rollupReleaseExpectTags struct {
Snap string
Holds map[string]bool
}
func rollupReleaseTest(ctx *platformtest.Context, cb func(fs string) []rollupReleaseExpectTags) {
platformtest.Run(ctx, platformtest.PanicErr, ctx.RootDataset, `
DESTROYROOT
CREATEROOT
+ "foo bar"
+ "foo bar@1"
+ "foo bar@2"
+ "foo bar@3"
+ "foo bar@4"
+ "foo bar@5"
+ "foo bar@6"
R zfs hold zrepl_platformtest "${ROOTDS}/foo bar@1"
R zfs hold zrepl_platformtest_2 "${ROOTDS}/foo bar@2"
R zfs hold zrepl_platformtest "${ROOTDS}/foo bar@3"
R zfs hold zrepl_platformtest "${ROOTDS}/foo bar@5"
R zfs hold zrepl_platformtest "${ROOTDS}/foo bar@6"
R zfs bookmark "${ROOTDS}/foo bar@5" "${ROOTDS}/foo bar#5"
`)
fs := fmt.Sprintf("%s/foo bar", ctx.RootDataset)
expTags := cb(fs)
for _, exp := range expTags {
holds, err := zfs.ZFSHolds(ctx, fs, exp.Snap)
if err != nil {
panic(err)
}
for _, h := range holds {
if e, ok := exp.Holds[h]; !ok || !e {
panic(fmt.Sprintf("tag %q on snap %q not expected", h, exp.Snap))
}
}
}
}
func RollupReleaseIncluding(ctx *platformtest.Context) {
rollupReleaseTest(ctx, func(fs string) []rollupReleaseExpectTags {
guid5, err := zfs.ZFSGetGUID(fs, "@5")
require.NoError(ctx, err)
err = zfs.ZFSReleaseAllOlderAndIncludingGUID(ctx, fs, guid5, "zrepl_platformtest")
require.NoError(ctx, err)
return []rollupReleaseExpectTags{
{"1", map[string]bool{}},
{"2", map[string]bool{"zrepl_platformtest_2": true}},
{"3", map[string]bool{}},
{"4", map[string]bool{}},
{"5", map[string]bool{}},
{"6", map[string]bool{"zrepl_platformtest": true}},
}
})
}
func RollupReleaseExcluding(ctx *platformtest.Context) {
rollupReleaseTest(ctx, func(fs string) []rollupReleaseExpectTags {
guid5, err := zfs.ZFSGetGUID(fs, "@5")
require.NoError(ctx, err)
err = zfs.ZFSReleaseAllOlderThanGUID(ctx, fs, guid5, "zrepl_platformtest")
require.NoError(ctx, err)
return []rollupReleaseExpectTags{
{"1", map[string]bool{}},
{"2", map[string]bool{"zrepl_platformtest_2": true}},
{"3", map[string]bool{}},
{"4", map[string]bool{}},
{"5", map[string]bool{"zrepl_platformtest": true}},
{"6", map[string]bool{"zrepl_platformtest": true}},
}
})
}
func RollupReleaseMostRecentIsBookmarkWithoutSnapshot(ctx *platformtest.Context) {
rollupReleaseTest(ctx, func(fs string) []rollupReleaseExpectTags {
guid5, err := zfs.ZFSGetGUID(fs, "#5")
require.NoError(ctx, err)
err = zfs.ZFSRelease(ctx, "zrepl_platformtest", fs+"@5")
require.NoError(ctx, err)
err = zfs.ZFSDestroy(fs + "@5")
require.NoError(ctx, err)
err = zfs.ZFSReleaseAllOlderAndIncludingGUID(ctx, fs, guid5, "zrepl_platformtest")
require.NoError(ctx, err)
return []rollupReleaseExpectTags{
{"1", map[string]bool{}},
{"2", map[string]bool{"zrepl_platformtest_2": true}},
{"3", map[string]bool{}},
{"4", map[string]bool{}},
// {"5", map[string]bool{}}, doesn't exist
{"6", map[string]bool{"zrepl_platformtest": true}},
}
})
}
func RollupReleaseMostRecentIsBookmarkAndSnapshotStillExists(ctx *platformtest.Context) {
rollupReleaseTest(ctx, func(fs string) []rollupReleaseExpectTags {
guid5, err := zfs.ZFSGetGUID(fs, "#5")
require.NoError(ctx, err)
err = zfs.ZFSReleaseAllOlderAndIncludingGUID(ctx, fs, guid5, "zrepl_platformtest")
require.NoError(ctx, err)
return []rollupReleaseExpectTags{
{"1", map[string]bool{}},
{"2", map[string]bool{"zrepl_platformtest_2": true}},
{"3", map[string]bool{}},
{"4", map[string]bool{}},
{"5", map[string]bool{}},
{"6", map[string]bool{"zrepl_platformtest": true}},
}
})
}
func RollupReleaseMostRecentDoesntExist(ctx *platformtest.Context) {
rollupReleaseTest(ctx, func(fs string) []rollupReleaseExpectTags {
const nonexistentGuid = 0 // let's take our chances...
err := zfs.ZFSReleaseAllOlderAndIncludingGUID(ctx, fs, nonexistentGuid, "zrepl_platformtest")
require.Error(ctx, err)
require.Contains(ctx, err.Error(), "cannot find snapshot or bookmark with guid 0")
return []rollupReleaseExpectTags{
{"1", map[string]bool{"zrepl_platformtest": true}},
{"2", map[string]bool{"zrepl_platformtest_2": true}},
{"3", map[string]bool{"zrepl_platformtest": true}},
{"4", map[string]bool{"zrepl_platformtest": true}},
{"5", map[string]bool{"zrepl_platformtest": true}},
{"6", map[string]bool{"zrepl_platformtest": true}},
}
})
}

View File

@ -0,0 +1,139 @@
package tests
import (
"io"
"math/rand"
"os"
"path"
"strings"
"github.com/stretchr/testify/require"
"github.com/zrepl/zrepl/platformtest"
"github.com/zrepl/zrepl/util/limitio"
"github.com/zrepl/zrepl/zfs"
)
func sendArgVersion(fs, relName string) zfs.ZFSSendArgVersion {
guid, err := zfs.ZFSGetGUID(fs, relName)
if err != nil {
panic(err)
}
return zfs.ZFSSendArgVersion{
RelName: relName,
GUID: guid,
}
}
func mustDatasetPath(fs string) *zfs.DatasetPath {
p, err := zfs.NewDatasetPath(fs)
if err != nil {
panic(err)
}
return p
}
func mustSnapshot(snap string) {
if err := zfs.EntityNamecheck(snap, zfs.EntityTypeSnapshot); err != nil {
panic(err)
}
comps := strings.Split(snap, "@")
if len(comps) != 2 {
panic(comps)
}
err := zfs.ZFSSnapshot(mustDatasetPath(comps[0]), comps[1], false)
if err != nil {
panic(err)
}
}
func mustGetProps(entity string) zfs.ZFSPropCreateTxgAndGuidProps {
props, err := zfs.ZFSGetCreateTXGAndGuid(entity)
check(err)
return props
}
func check(err error) {
if err != nil {
panic(err)
}
}
var dummyDataRand = rand.New(rand.NewSource(99))
func writeDummyData(path string, numBytes int64) {
r := io.LimitReader(dummyDataRand, numBytes)
d, err := os.OpenFile(path, os.O_CREATE|os.O_TRUNC|os.O_WRONLY, 0666)
check(err)
defer d.Close()
_, err = io.Copy(d, r)
check(err)
}
type dummySnapshotSituation struct {
sendFS string
dummyDataLen int64
snapA *zfs.ZFSSendArgVersion
snapB *zfs.ZFSSendArgVersion
}
type resumeSituation struct {
sendArgs zfs.ZFSSendArgs
recvOpts zfs.RecvOptions
sendErr, recvErr error
recvErrDecoded *zfs.RecvFailedWithResumeTokenErr
}
func makeDummyDataSnapshots(ctx *platformtest.Context, sendFS string) (situation dummySnapshotSituation) {
situation.sendFS = sendFS
sendFSMount, err := zfs.ZFSGetMountpoint(sendFS)
require.NoError(ctx, err)
require.True(ctx, sendFSMount.Mounted)
const dummyLen = int64(10 * (1 << 20))
situation.dummyDataLen = dummyLen
writeDummyData(path.Join(sendFSMount.Mountpoint, "dummy_data"), dummyLen)
mustSnapshot(sendFS + "@a snapshot")
snapA := sendArgVersion(sendFS, "@a snapshot")
situation.snapA = &snapA
writeDummyData(path.Join(sendFSMount.Mountpoint, "dummy_data"), dummyLen)
mustSnapshot(sendFS + "@b snapshot")
snapB := sendArgVersion(sendFS, "@b snapshot")
situation.snapB = &snapB
return situation
}
func makeResumeSituation(ctx *platformtest.Context, src dummySnapshotSituation, recvFS string, sendArgs zfs.ZFSSendArgs, recvOptions zfs.RecvOptions) *resumeSituation {
situation := &resumeSituation{}
situation.sendArgs = sendArgs
situation.recvOpts = recvOptions
require.True(ctx, recvOptions.SavePartialRecvState, "this method would be pointeless otherwise")
require.Equal(ctx, sendArgs.FS, src.sendFS)
copier, err := zfs.ZFSSend(ctx, sendArgs)
situation.sendErr = err
if err != nil {
return situation
}
limitedCopier := zfs.NewReadCloserCopier(limitio.ReadCloser(copier, src.dummyDataLen/2))
defer limitedCopier.Close()
require.NotNil(ctx, sendArgs.To)
err = zfs.ZFSRecv(ctx, recvFS, sendArgs.To, limitedCopier, recvOptions)
situation.recvErr = err
ctx.Logf("zfs recv exit with %T %s", err, err)
require.NotNil(ctx, err)
resumeErr, ok := err.(*zfs.RecvFailedWithResumeTokenErr)
require.True(ctx, ok)
situation.recvErrDecoded = resumeErr
return situation
}

View File

@ -0,0 +1,59 @@
package tests
import (
"fmt"
"github.com/zrepl/zrepl/platformtest"
"github.com/zrepl/zrepl/zfs"
)
func IdempotentBookmark(ctx *platformtest.Context) {
platformtest.Run(ctx, platformtest.PanicErr, ctx.RootDataset, `
DESTROYROOT
CREATEROOT
+ "foo bar"
+ "foo bar@a snap"
+ "foo bar@another snap"
`)
fs := fmt.Sprintf("%s/foo bar", ctx.RootDataset)
asnap := sendArgVersion(fs, "@a snap")
anotherSnap := sendArgVersion(fs, "@another snap")
err := zfs.ZFSBookmark(fs, asnap, "a bookmark")
if err != nil {
panic(err)
}
// do it again, should be idempotent
err = zfs.ZFSBookmark(fs, asnap, "a bookmark")
if err != nil {
panic(err)
}
// should fail for another snapshot
err = zfs.ZFSBookmark(fs, anotherSnap, "a bookmark")
if err == nil {
panic(err)
}
if _, ok := err.(*zfs.BookmarkExists); !ok {
panic(fmt.Sprintf("has type %T", err))
}
// destroy the snapshot
if err := zfs.ZFSDestroy(fmt.Sprintf("%s@a snap", fs)); err != nil {
panic(err)
}
// do it again, should fail with special error type
err = zfs.ZFSBookmark(fs, asnap, "a bookmark")
if err == nil {
panic(err)
}
if _, ok := err.(*zfs.DatasetDoesNotExist); !ok {
panic(fmt.Sprintf("has type %T", err))
}
}

View File

@ -0,0 +1,75 @@
package tests
import (
"fmt"
"log"
"github.com/zrepl/zrepl/platformtest"
"github.com/zrepl/zrepl/zfs"
)
func IdempotentDestroy(ctx *platformtest.Context) {
platformtest.Run(ctx, platformtest.PanicErr, ctx.RootDataset, `
DESTROYROOT
CREATEROOT
+ "foo bar"
+ "foo bar@a snap"
`)
fs := fmt.Sprintf("%s/foo bar", ctx.RootDataset)
asnap := sendArgVersion(fs, "@a snap")
err := zfs.ZFSBookmark(fs, asnap, "a bookmark")
if err != nil {
panic(err)
}
type testCase struct {
description, path string
}
cases := []testCase{
{"snapshot", fmt.Sprintf("%s@a snap", fs)},
{"bookmark", fmt.Sprintf("%s#a bookmark", fs)},
{"filesystem", fs},
}
for i := range cases {
func() {
c := cases[i]
log.Printf("SUBBEGIN testing idempotent destroy %q for path %q", c.description, c.path)
log.Println("destroy existing")
err = zfs.ZFSDestroy(c.path)
if err != nil {
panic(err)
}
log.Println("destroy again, non-idempotently, must error")
err = zfs.ZFSDestroy(c.path)
if _, ok := err.(*zfs.DatasetDoesNotExist); !ok {
panic(fmt.Sprintf("%T: %s", err, err))
}
log.Println("destroy again, idempotently, must not error")
err = zfs.ZFSDestroyIdempotent(c.path)
if err != nil {
panic(err)
}
log.Println("SUBEND")
}()
}
// also test idempotent destroy for cases where the parent dataset does not exist
err = zfs.ZFSDestroyIdempotent(fmt.Sprintf("%s/not foo bar@nonexistent snapshot", ctx.RootDataset))
if err != nil {
panic(err)
}
err = zfs.ZFSDestroyIdempotent(fmt.Sprintf("%s/not foo bar#nonexistent bookmark", ctx.RootDataset))
if err != nil {
panic(err)
}
}

View File

@ -0,0 +1,47 @@
package tests
import (
"fmt"
"github.com/zrepl/zrepl/platformtest"
"github.com/zrepl/zrepl/zfs"
)
func IdempotentHold(ctx *platformtest.Context) {
platformtest.Run(ctx, platformtest.PanicErr, ctx.RootDataset, `
DESTROYROOT
CREATEROOT
+ "foo bar"
+ "foo bar@1"
`)
defer platformtest.Run(ctx, platformtest.PanicErr, ctx.RootDataset, `
R zfs release zrepl_platformtest "${ROOTDS}/foo bar@1"
- "foo bar@1"
- "foo bar"
`)
fs := fmt.Sprintf("%s/foo bar", ctx.RootDataset)
v1 := sendArgVersion(fs, "@1")
tag := "zrepl_platformtest"
err := zfs.ZFSHold(ctx, fs, v1, tag)
if err != nil {
panic(err)
}
err = zfs.ZFSHold(ctx, fs, v1, tag)
if err != nil {
panic(err)
}
vnonexistent := zfs.ZFSSendArgVersion{
RelName: "@nonexistent",
GUID: 0xbadf00d,
}
err = zfs.ZFSHold(ctx, fs, vnonexistent, tag)
if err == nil {
panic("still expecting error for nonexistent snapshot")
}
}

View File

@ -3,6 +3,9 @@ package tests
import (
"fmt"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/zrepl/zrepl/endpoint"
"github.com/zrepl/zrepl/platformtest"
"github.com/zrepl/zrepl/zfs"
)
@ -13,25 +16,35 @@ func ReplicationCursor(ctx *platformtest.Context) {
CREATEROOT
+ "foo bar"
+ "foo bar@1 with space"
R zfs bookmark "${ROOTDS}/foo bar@1 with space" "${ROOTDS}/foo bar#1 with space"
+ "foo bar@2 with space"
R zfs bookmark "${ROOTDS}/foo bar@1 with space" "${ROOTDS}/foo bar#2 with space"
+ "foo bar@3 with space"
R zfs bookmark "${ROOTDS}/foo bar@1 with space" "${ROOTDS}/foo bar#3 with space"
`)
jobid := endpoint.MustMakeJobID("zreplplatformtest")
ds, err := zfs.NewDatasetPath(ctx.RootDataset + "/foo bar")
if err != nil {
panic(err)
}
guid, err := zfs.ZFSSetReplicationCursor(ds, "1 with space")
fs := ds.ToString()
snap := sendArgVersion(fs, "@1 with space")
destroyed, err := endpoint.MoveReplicationCursor(ctx, fs, &snap, jobid)
if err != nil {
panic(err)
}
snapProps, err := zfs.ZFSGetCreateTXGAndGuid(ds.ToString() + "@1 with space")
assert.Empty(ctx, destroyed)
snapProps, err := zfs.ZFSGetCreateTXGAndGuid(snap.FullPath(fs))
if err != nil {
panic(err)
}
if guid != snapProps.Guid {
panic(fmt.Sprintf("guids to not match: %v != %v", guid, snapProps.Guid))
}
bm, err := zfs.ZFSGetReplicationCursor(ds)
bm, err := endpoint.GetMostRecentReplicationCursorOfJob(ctx, fs, jobid)
if err != nil {
panic(err)
}
@ -41,22 +54,15 @@ func ReplicationCursor(ctx *platformtest.Context) {
if bm.Guid != snapProps.Guid {
panic(fmt.Sprintf("guids do not match: %v != %v", bm.Guid, snapProps.Guid))
}
if bm.Guid != guid {
panic(fmt.Sprintf("guids do not match: %v != %v", bm.Guid, guid))
}
// test nonexistent
err = zfs.ZFSDestroyFilesystemVersion(ds, bm)
if err != nil {
panic(err)
}
bm2, err := zfs.ZFSGetReplicationCursor(ds)
if bm2 != nil {
panic(fmt.Sprintf("expecting no replication cursor after deleting it, got %v", bm))
}
if err != nil {
panic(fmt.Sprintf("expecting no error for getting nonexistent replication cursor, bot %v", err))
}
// try moving
cursor1BookmarkName, err := endpoint.ReplicationCursorBookmarkName(fs, snap.GUID, jobid)
require.NoError(ctx, err)
// TODO test moving the replication cursor
snap2 := sendArgVersion(fs, "@2 with space")
destroyed, err = endpoint.MoveReplicationCursor(ctx, fs, &snap2, jobid)
require.NoError(ctx, err)
require.Equal(ctx, 1, len(destroyed))
require.Equal(ctx, zfs.Bookmark, destroyed[0].Type)
require.Equal(ctx, cursor1BookmarkName, destroyed[0].Name)
}

View File

@ -0,0 +1,64 @@
package tests
import (
"fmt"
"github.com/stretchr/testify/require"
"github.com/zrepl/zrepl/platformtest"
"github.com/zrepl/zrepl/zfs"
)
func ResumableRecvAndTokenHandling(ctx *platformtest.Context) {
platformtest.Run(ctx, platformtest.PanicErr, ctx.RootDataset, `
DESTROYROOT
CREATEROOT
+ "send er"
`)
sendFS := fmt.Sprintf("%s/send er", ctx.RootDataset)
recvFS := fmt.Sprintf("%s/recv er", ctx.RootDataset)
supported, err := zfs.ResumeRecvSupported(ctx, mustDatasetPath(sendFS))
check(err)
src := makeDummyDataSnapshots(ctx, sendFS)
s := makeResumeSituation(ctx, src, recvFS, zfs.ZFSSendArgs{
FS: sendFS,
To: src.snapA,
Encrypted: &zfs.NilBool{B: false},
ResumeToken: "",
}, zfs.RecvOptions{
RollbackAndForceRecv: false, // doesnt' exist yet
SavePartialRecvState: true,
})
if !supported {
_, ok := s.recvErr.(*zfs.ErrRecvResumeNotSupported)
require.True(ctx, ok)
// we know that support on sendFS implies support on recvFS
// => asser that if we don't support resumed recv, the method returns ""
tok, err := zfs.ZFSGetReceiveResumeTokenOrEmptyStringIfNotSupported(ctx, mustDatasetPath(recvFS))
check(err)
require.Equal(ctx, "", tok)
return // nothing more to test for recv that doesn't support -s
}
getTokenRaw, err := zfs.ZFSGetReceiveResumeTokenOrEmptyStringIfNotSupported(ctx, mustDatasetPath(recvFS))
check(err)
require.NotEmpty(ctx, getTokenRaw)
decodedToken, err := zfs.ParseResumeToken(ctx, getTokenRaw)
check(err)
require.True(ctx, decodedToken.HasToGUID)
require.Equal(ctx, s.sendArgs.To.GUID, decodedToken.ToGUID)
recvErr := s.recvErr.(*zfs.RecvFailedWithResumeTokenErr)
require.Equal(ctx, recvErr.ResumeTokenRaw, getTokenRaw)
require.Equal(ctx, recvErr.ResumeTokenParsed, decodedToken)
}

View File

@ -0,0 +1,105 @@
package tests
import (
"github.com/stretchr/testify/require"
"github.com/zrepl/zrepl/platformtest"
"github.com/zrepl/zrepl/zfs"
)
type resumeTokenTest struct {
Msg string
Token string
ExpectToken *zfs.ResumeToken
ExpectError error
}
func (rtt *resumeTokenTest) Test(t *platformtest.Context) {
resumeSendSupported, err := zfs.ResumeSendSupported()
if err != nil {
t.Errorf("cannot determine whether resume supported: %T %s", err, err)
t.FailNow()
return
}
res, err := zfs.ParseResumeToken(t, rtt.Token)
// if decoding is not supported, don't bother with the expectations
if !resumeSendSupported {
require.Error(t, err)
require.Equal(t, zfs.ResumeTokenDecodingNotSupported, err)
return
}
if rtt.ExpectError != nil {
require.EqualValues(t, rtt.ExpectError, err)
return
}
if rtt.ExpectToken != nil {
require.Nil(t, err)
require.EqualValues(t, rtt.ExpectToken, res)
return
}
}
func ResumeTokenParsing(ctx *platformtest.Context) {
// cases generated using resumeTokensGenerate.bash on ZoL 0.8.1
cases := []resumeTokenTest{
{
Msg: "zreplplatformtest/dst/full",
Token: "1-b338b54f3-c0-789c636064000310a500c4ec50360710e72765a52697303030419460caa7a515a796806474e0f26c48f2499525a9c540ba42430fabfe92fcf4d2cc140686c88a76d578ae45530c90e439c1f27989b9a90c0c5545a905390539892569f945b940234bf48b8b921d12c1660200c61a1aba",
ExpectToken: &zfs.ResumeToken{
HasToGUID: true,
ToGUID: 0x94a20a5f25877859,
ToName: "zreplplatformtest/src@a",
},
},
{
Msg: "zreplplatformtest/dst/full_raw",
Token: "1-e3f40c323-f8-789c636064000310a500c4ec50360710e72765a52697303030419460caa7a515a796806474e0f26c48f2499525a9c540da454f0fabfe92fcf4d2cc140686c88a76d578ae45530c90e439c1f27989b9a90c0c5545a905390539892569f945b940234bf48b8b921d12c1e6713320dc9f9c9f5b50945a5c9c9f0d119380ba07265f94580e936200004ff12141",
ExpectToken: &zfs.ResumeToken{
HasToGUID: true,
ToGUID: 0x94a20a5f25877859,
ToName: "zreplplatformtest/src@a",
HasCompressOK: true, CompressOK: true,
HasRawOk: true, RawOK: true,
},
},
{
Msg: "zreplplatformtest/dst/inc",
Token: "1-eadabb296-e8-789c636064000310a501c49c50360710a715e5e7a69766a63040416445bb6a3cd7a2290a40363b92bafca4acd4e412060626a83a0cf9b4b4e2d412908c0e5c9e0d493ea9b224b518483ba8ea61d55f920f714515bf9b3fc3c396ef0648f29c60f9bcc4dc54a07c516a414e414e62495a7e512ed0c812fde2a2648724b09900d43e2191",
ExpectToken: &zfs.ResumeToken{
HasFromGUID: true, FromGUID: 0x94a20a5f25877859,
HasToGUID: true, ToGUID: 0xf784e1004f460f7a,
ToName: "zreplplatformtest/src@b",
},
},
{
Msg: "zreplplatformtest/dst/inc_raw",
Token: "1-1164f8d409-120-789c636064000310a501c49c50360710a715e5e7a69766a63040416445bb6a3cd7a2290a40363b92bafca4acd4e412060626a83a0cf9b4b4e2d412908c0e5c9e0d493ea9b224b51848f368eb61d55f920f714515bf9b3fc3c396ef0648f29c60f9bcc4dc54a07c516a414e414e62495a7e512ed0c812fde2a2648724b079dc0c087f26e7e71614a51617e76743c424a0ee81c9172596c3a41800dd2c2818",
ExpectToken: &zfs.ResumeToken{
HasFromGUID: true, FromGUID: 0x94a20a5f25877859,
HasToGUID: true, ToGUID: 0xf784e1004f460f7a,
ToName: "zreplplatformtest/src@b",
HasCompressOK: true, CompressOK: true,
HasRawOk: true, RawOK: true,
},
},
// manual test csaes
{
Msg: "corrupted",
Token: "1-1164f8d409-120-badf00d064000310a501c49c50360710a715e5e7a69766a63040416445bb6a3cd7a2290a40363b92bafca4acd4e412060626a83a0cf9b4b4e2d412908c0e5c9e0d493ea9b224b51848f368eb61d55f920f714515bf9b3fc3c396ef0648f29c60f9bcc4dc54a07c516a414e414e62495a7e512ed0c812fde2a2648724b079dc0c087f26e7e71614a51617e76743c424a0ee81c9172596c3a41800dd2c2818",
ExpectError: zfs.ResumeTokenCorruptError,
},
}
for _, test := range cases {
ctx.Logf("BEGIN SUBTEST: %s", test.Msg)
test.Test(ctx)
ctx.Logf("COMPLETE SUBTEST: %s", test.Msg)
}
}

View File

@ -0,0 +1,57 @@
#!/usr/bin/env bash
set -euo pipefail
set -x
POOLNAME="zreplplatformtest"
POOLIMG="$1"
if zpool status "$POOLNAME"; then
exit 1
fi
if [ -e "$POOLIMG" ]; then
exit 1
fi
fallocate -l 500M "$POOLIMG"
zpool create "$POOLNAME" "$POOLIMG"
SRC_DATASET="$POOLNAME/src"
DST_ROOT="$POOLNAME/dst"
keylocation="$(mktemp /tmp/ZREPL_PLATFORMTEST_GENERATE_TOKENS_KEYFILE_XXX)"
echo "foobar123" > "$keylocation"
zfs create -o encryption=on -o keylocation="file:///$keylocation" -o keyformat=passphrase "$SRC_DATASET"
rm "$keylocation"
SRC_MOUNT="$(zfs get -H -o value mountpoint "$SRC_DATASET")"
test -d "$SRC_MOUNT"
A="$SRC_DATASET"@a
B="$SRC_DATASET"@b
dd if=/dev/urandom of="$SRC_MOUNT"/dummy_data bs=1M count=5
zfs snapshot "$A"
dd if=/dev/urandom of="$SRC_MOUNT"/dummy_data bs=1M count=5
zfs snapshot "$B"
zfs create "$DST_ROOT"
cutoff="dd bs=1024 count=3K"
set +e
zfs send "$A" | $cutoff | zfs recv -s "$DST_ROOT"/full
zfs send "$A" | zfs recv "$DST_ROOT"/inc
zfs send -i "$A" "$B" | $cutoff | zfs recv -s "$DST_ROOT"/inc
zfs send -w "$A" | $cutoff | zfs recv -s "$DST_ROOT"/full_raw
zfs send -w "$A" | zfs recv "$DST_ROOT"/inc_raw
zfs send -w -i "$A" "$B" | $cutoff | zfs recv -s "$DST_ROOT"/inc_raw
set -e
TOKENS="$(zfs list -H -o name,receive_resume_token -r "$DST_ROOT")"
echo "$TOKENS" | awk -e '//{ if ($2 == "-") { } else { system("zfs send -nvt " + $2);} }'
zpool destroy "$POOLNAME"
rm "$POOLIMG"

View File

@ -0,0 +1,208 @@
package tests
import (
"fmt"
"github.com/stretchr/testify/require"
"github.com/zrepl/zrepl/platformtest"
"github.com/zrepl/zrepl/zfs"
)
func SendArgsValidationEncryptedSendOfUnencryptedDatasetForbidden(ctx *platformtest.Context) {
supported, err := zfs.EncryptionCLISupported(ctx)
check(err)
expectNotSupportedErr := !supported
platformtest.Run(ctx, platformtest.PanicErr, ctx.RootDataset, `
DESTROYROOT
CREATEROOT
+ "send er"
+ "send er@a snap"
`)
fs := fmt.Sprintf("%s/send er", ctx.RootDataset)
props := mustGetProps(fs + "@a snap")
sendArgs := zfs.ZFSSendArgs{
FS: fs,
To: &zfs.ZFSSendArgVersion{
RelName: "@a snap",
GUID: props.Guid,
},
Encrypted: &zfs.NilBool{B: true},
ResumeToken: "",
}
stream, err := zfs.ZFSSend(ctx, sendArgs)
if err == nil {
defer stream.Close()
}
if expectNotSupportedErr {
require.Error(ctx, err)
require.Equal(ctx, zfs.ErrEncryptedSendNotSupported, err)
return
}
require.Error(ctx, err)
ctx.Logf("send err: %T %s", err, err)
validationErr, ok := err.(*zfs.ZFSSendArgsValidationError)
require.True(ctx, ok)
require.True(ctx, validationErr.What == zfs.ZFSSendArgsEncryptedSendRequestedButFSUnencrypted)
}
func SendArgsValidationResumeTokenEncryptionMismatchForbidden(ctx *platformtest.Context) {
supported, err := zfs.EncryptionCLISupported(ctx)
check(err)
if !supported {
ctx.SkipNow()
}
supported, err = zfs.ResumeSendSupported()
check(err)
if !supported {
ctx.SkipNow()
}
platformtest.Run(ctx, platformtest.PanicErr, ctx.RootDataset, `
DESTROYROOT
CREATEROOT
+ "send er" encrypted
`)
sendFS := fmt.Sprintf("%s/send er", ctx.RootDataset)
unencRecvFS := fmt.Sprintf("%s/unenc recv", ctx.RootDataset)
encRecvFS := fmt.Sprintf("%s/enc recv", ctx.RootDataset)
src := makeDummyDataSnapshots(ctx, sendFS)
unencS := makeResumeSituation(ctx, src, unencRecvFS, zfs.ZFSSendArgs{
FS: sendFS,
To: src.snapA,
Encrypted: &zfs.NilBool{B: false}, // !
}, zfs.RecvOptions{
RollbackAndForceRecv: false,
SavePartialRecvState: true,
})
encS := makeResumeSituation(ctx, src, encRecvFS, zfs.ZFSSendArgs{
FS: sendFS,
To: src.snapA,
Encrypted: &zfs.NilBool{B: true}, // !
}, zfs.RecvOptions{
RollbackAndForceRecv: false,
SavePartialRecvState: true,
})
// threat model: use of a crafted resume token that requests an unencrypted send
// but send args require encrypted send
{
var maliciousSend zfs.ZFSSendArgs = encS.sendArgs
maliciousSend.ResumeToken = unencS.recvErrDecoded.ResumeTokenRaw
stream, err := zfs.ZFSSend(ctx, maliciousSend)
if err == nil {
defer stream.Close()
}
require.Nil(ctx, stream)
require.Error(ctx, err)
ctx.Logf("send err: %T %s", err, err)
validationErr, ok := err.(*zfs.ZFSSendArgsValidationError)
require.True(ctx, ok)
require.Equal(ctx, validationErr.What, zfs.ZFSSendArgsResumeTokenMismatch)
ctx.Logf("%s", validationErr)
mismatchError, ok := validationErr.Msg.(*zfs.ZFSSendArgsResumeTokenMismatchError)
require.True(ctx, ok)
require.Equal(ctx, mismatchError.What, zfs.ZFSSendArgsResumeTokenMismatchEncryptionNotSet)
}
// threat model: use of a crafted resume token that requests an encryped send
// but send args require unencrypted send
{
var maliciousSend zfs.ZFSSendArgs = unencS.sendArgs
maliciousSend.ResumeToken = encS.recvErrDecoded.ResumeTokenRaw
stream, err := zfs.ZFSSend(ctx, maliciousSend)
if err == nil {
defer stream.Close()
}
require.Nil(ctx, stream)
require.Error(ctx, err)
ctx.Logf("send err: %T %s", err, err)
validationErr, ok := err.(*zfs.ZFSSendArgsValidationError)
require.True(ctx, ok)
require.Equal(ctx, validationErr.What, zfs.ZFSSendArgsResumeTokenMismatch)
ctx.Logf("%s", validationErr)
mismatchError, ok := validationErr.Msg.(*zfs.ZFSSendArgsResumeTokenMismatchError)
require.True(ctx, ok)
require.Equal(ctx, mismatchError.What, zfs.ZFSSendArgsResumeTokenMismatchEncryptionSet)
}
}
func SendArgsValidationResumeTokenDifferentFilesystemForbidden(ctx *platformtest.Context) {
supported, err := zfs.EncryptionCLISupported(ctx)
check(err)
if !supported {
ctx.SkipNow()
}
supported, err = zfs.ResumeSendSupported()
check(err)
if !supported {
ctx.SkipNow()
}
platformtest.Run(ctx, platformtest.PanicErr, ctx.RootDataset, `
DESTROYROOT
CREATEROOT
+ "send er1"
+ "send er2"
`)
sendFS1 := fmt.Sprintf("%s/send er1", ctx.RootDataset)
sendFS2 := fmt.Sprintf("%s/send er2", ctx.RootDataset)
recvFS := fmt.Sprintf("%s/unenc recv", ctx.RootDataset)
src1 := makeDummyDataSnapshots(ctx, sendFS1)
src2 := makeDummyDataSnapshots(ctx, sendFS2)
rs := makeResumeSituation(ctx, src1, recvFS, zfs.ZFSSendArgs{
FS: sendFS1,
To: src1.snapA,
Encrypted: &zfs.NilBool{B: false},
}, zfs.RecvOptions{
RollbackAndForceRecv: false,
SavePartialRecvState: true,
})
// threat model: forged resume token tries to steal a full send of snapA on fs2 by
// presenting a resume token for full send of snapA on fs1
var maliciousSend zfs.ZFSSendArgs = zfs.ZFSSendArgs{
FS: sendFS2,
To: &zfs.ZFSSendArgVersion{
RelName: src2.snapA.RelName,
GUID: src2.snapA.GUID,
},
Encrypted: &zfs.NilBool{B: false},
ResumeToken: rs.recvErrDecoded.ResumeTokenRaw,
}
stream, err := zfs.ZFSSend(ctx, maliciousSend)
if err == nil {
defer stream.Close()
}
require.Nil(ctx, stream)
require.Error(ctx, err)
ctx.Logf("send err: %T %s", err, err)
validationErr, ok := err.(*zfs.ZFSSendArgsValidationError)
require.True(ctx, ok)
require.Equal(ctx, validationErr.What, zfs.ZFSSendArgsResumeTokenMismatch)
ctx.Logf("%s", validationErr)
mismatchError, ok := validationErr.Msg.(*zfs.ZFSSendArgsResumeTokenMismatchError)
require.True(ctx, ok)
require.Equal(ctx, mismatchError.What, zfs.ZFSSendArgsResumeTokenMismatchFilesystem)
}

View File

@ -18,4 +18,17 @@ var Cases = []Case{
UndestroyableSnapshotParsing,
GetNonexistent,
ReplicationCursor,
RollupReleaseIncluding,
RollupReleaseExcluding,
RollupReleaseMostRecentIsBookmarkWithoutSnapshot,
RollupReleaseMostRecentIsBookmarkAndSnapshotStillExists,
RollupReleaseMostRecentDoesntExist,
IdempotentHold,
IdempotentBookmark,
IdempotentDestroy,
ResumeTokenParsing,
ResumableRecvAndTokenHandling,
SendArgsValidationEncryptedSendOfUnencryptedDatasetForbidden,
SendArgsValidationResumeTokenEncryptionMismatchForbidden,
SendArgsValidationResumeTokenDifferentFilesystemForbidden,
}

342
replication/design.md Normal file
View File

@ -0,0 +1,342 @@
The goal of this document is to describe what **logical steps** zrepl takes to replicate ZFS filesystems.
Note that the actual code executing these steps is spread over the `endpoint.{Sender,Receiver}` RPC endpoint implementations. The code in `replication/{driver,logic}` (mostly `logic.Step.doReplication`) drives the replication and invokes the `endpoint` methods (locally or via RPC).
Hence, when trying to map algorithm to implementation, use the code in package `replication` as the entrypoint and follow the RPC calls.
* The [Single Replication Step](#zrepl-algo-single-step) algorithm is implemented as described
* step holds are implemented as described
* step bookmarks rely on bookmark cloning, which is WIP in OpenZFS (see respective TODO comments)
* the feature to hold receive-side `to` immediately after replication is used to move the `last-received-hold` forward
* this is a little more than what we allow in the algorithm (we only allow specifying a hold tag whereas moving the `last-received-hold` is a little more complex)
* the algorithm's sender-side callback is used move the `zrepl_CURSOR` bookmark to point to `to`
* The `zrepl_CURSOR` bookmark and the `last-received-hold` ensure that future replication steps can always be done using incremental replication:
* Both reference / hold the last successfully received snapshot `to`.
* Thus, `to` or `#zrepl_CURSOR_G_${to.GUID}_J_${jobid}` can always be used for a future incremental replication step `to => future-to`:
* incremental send will be possible because `to` doesn't go away because we claim ownership of the `#zrepl_CURSOR` namespace
* incremental recv will be possible iff
* `to` will still be present on the recv-side because of `last-received-hold`
* the recv-side fs doesn't get newer snapshots than `to` in the meantime
* guaranteed because the zrepl model of the receiver assumes ownership of the filesystems it receives into
* if that assumption is broken, future replication attempts will fail with a conflict
* The [Algorithm for Planning and Executing Replication of an Filesystems](#zrepl-algo-filesystem) is a design draft and not used
* We also have [Notes on Planning and Executing Replication of Multiple Filesystems](#zrepl-algo-multiple-filesystems-notes)
---
<a id="zrepl-algo-single-step"></a>
## Algorithm for a Single Replication Step
The algorithm described below describes how a single replication step must be implemented.
A *replication step* is a full send of a snapshot `to` for initial replication, or an incremental send (`from => to`) for incremental replication.
The algorithm **ensures resumability** of the replication step in presence of
* uncoordinated or unsynchronized destruction of the filesystem, snapshots, or bookmarks involved in the replication step
* network failures at any time
* other instances of this algorithm executing the same step in parallel (e.g. concurrent replication to different destinations)
To accomplish this goal, the algorithm **assumes ownersip of parts of the ZFS hold tag namespace and the bookmark namespace**:
* holds with prefix `zrepl_STEP` on any snapshot are reserved for zrepl
* bookmarks with prefix `zrepl_STEP` are reserved for zrepl
Resumability of a step cannot be guaranteed if these sub-namespaces are manipulated through software other than zrepl.
Note that the algorithm **does not ensure** that a replication *plan*, which describes a *set* of replication steps, can be carried out successfully.
If that is desirable, additional measures outside of this algorithm must be taken.
---
### Definitions:
#### Step Completion & Invariants
The replication step (full `to` send or `from => to` send) is *complete* iff the algorithm ran to completion without errors or a permanent non-network error is reported by sender or receiver.
Specifically, the algorithm may be invoked with the same `from` and `to` arguments, and potentially a `resume_token`, after a temporary (like network-related) failure:
**Unless permanent errors occur, repeated invocations of the algorithm with updated resume token will converge monotonically (but not strictly monotonically) toward completion.**
Note that the mere existence of `to` on the receiving side does not constitue completion, since there may still be post-recv actions to be performed on sender and receiver.
#### Job and Job ID
This algorithm supports that *multiple* instance of it run in parallel on the *same* step (full `to` / `from => to` pair).
An exemplary use case for this feature are concurrent replication jobs that replicate the same dataset to different receivers.
**We require that all parallel invocations of this algorithm provide different and unique `jobid`s.**
Violation of `jobid` uniqueness across parallel jobs may result in interference between instances of this algorithm, resulting in potential compromise of resumability.
After a step is *completed*, `jobid` is guaranteed to not be encoded in on-disk state.
Before a step is completed, there is no such guarantee.
Changing the `jobid` before step completion may compromise resumability and may leak the underlying ZFS holds or step bookmarks (i.e. zrepl won't clean them up)
Note the definition of *complete* above.
#### Step Bookmark
A step bookmark is our equivalent of ZFS holds, but for bookmarks.<br/>
A step bookmark is a ZFS bookmark whose name matches the following regex:
```
STEP_BOOKMARK = #zrepl_STEP_bm_G_([0-9a-f]+)_J_(.+)
```
- capture group `1` must be the guid of `zfs get guid ${STEP_BOOKMARK}`, encoded hexadecimal (without leading `0x`) and fixed-length (i.e. padded with leading zeroes)
- capture group `2` must be equal to the `jobid`
### Algorithm
INPUT:
* `jobid`
* `from`: snapshot or bookmark: may be nil for full send
* `to`: snapshot, must never be nil
* `resume_token` (may be nil)
* (`from` and `to` must be on the same filesystem)
#### Prepare-Phase
Send-side: make sure `to` and `from` don't go away
- hold `to` using `idempotent_hold(to, zrepl_STEP_J_${jobid})`
- make sure `from` doesn't go away:
- if `from` is a snapshot: hold `from` using `idempotent_hold(from, zrepl_STEP_J_${jobid})`
- else `idempotent_step_bookmark(from)` (`from` is a bookmark)
- PERMANENT ERROR if this fails (e.g. because `from` is a bookmark whose snapshot no longer exists and we don't have bookmark copying yet (ZFS feature is in development) )
- Why? we must assume the given bookmark is externally managed (i.e. not in the )
- this means the bookmark is externally created bookmark and cannot be trusted to persist until the replication step succeeds
- Maybe provide an override-knob for this behavior
Recv-side: no-op
- `from` cannot go away once we received enough data for the step to be resumable:
```text
# do a partial incremental recv @a=>@b (i.e. recv with -s, and abort the incremental recv in the middle)
# try destroying the incremental source on the receiving side
zfs destroy p1/recv@a
cannot destroy 'p1/recv@a': snapshot has dependent clones
use '-R' to destroy the following datasets:
p1/recv/%recv
# => doesn't work, because zfs recv is implemented as a `clone` internally, that's exactly what we want
```
- if recv-side `to` exists, goto cleaup-phase (no replication to do)
- `to` cannot be destroyed while being received, because it isn't visible as a snapshot yet (it isn't yet one after all)
#### Replication Phase
Attempt the replication step:
start `zfs send` and pipe its output into `zfs recv` until an error occurs or `recv` returns without error.
Let us now think about interferences that could occur during this phase, and ensure that none of them compromise the goals of this algorithm, i.e., monotonic convergence toward step completion using resumable send & recv.
**Safety from External Pruning**
We are safe from pruning during the replication step because we have guarantees that no external action will destroy send-side `from` and `to`, and recv-side `to` (for both snapshot and bookmark `from`s)<br/>
**Safety In Presence of Network Failures During Replication**
Network failures during replication can be recovered from using resumable send & recv:
- Network failure before the receive-side could stored data:
- Send-side `from` and `to` are guaranteed to be still present due to zfs holds
- recv-side `from` may no longer exist because we don't `hold` it explicitly
- if that is the case, ERROR OUT, step can never complete
- If the step planning algorithm does not include the step, for example because a snapshot filter configuration was changed by the user inbetween which hides `from` or `to` from the second planning attempt: **tough luck, we're leaking all holds**
- Network failure during the replication
- send-side `from` and `to` are still present due to zfs holds
- recv-side `from` is still present because the partial receive state prevents its destruction (see prepare-phase)
- if recv-side hasa resume token, the resume token will continue to work on the sender because `from`s and `to` are still present
- Network failure at the end of the replication step stream transmission
- Variant A: failure from the sender's perspective, success from the receiver's perspective
- receive-side `to` doesn't have a hold and could be destroyed anytime
- receive-side `from` doesn't have a hold and could be destroyed anytime
- thus, when the step is invoked again, pattern match `(from_exists, to_exists)`
- `(true, true)`: prepare-phase will early-exit
- `(false,true)`: prepare-phase will error out bc. `from` does not exist
- `(true, false)`: entire step will be re-done
- FIXME monotonicity requirement does not hold
- `(false, false)`: prepare-phase will error out bc. `from` does not exist
- Variant B: success from the sender's perspective, failure from the receiver's perspective
- No idea how this would happen except for bugs in error reporting in the replication protocol
- Misclassification by the sender, most likely broken error handling in the sender or replication protocol
- => the sender will release holds and move the replication cursor while the receiver won't => tough luck
If the RPC used for `zfs recv` returned without error, this phase is done.
(Although the *replication step* (this algorithm) is not yet *complete*, see definition of complete).
#### Cleanup-Phase
At the end of this phase, all intermediate state we built up to support the resumable replication of the step is destroyed.
However, consumers of this algorithm might want to take advantage of the fact that we currently still have holds / step bookmarks.
##### Recv-side: Optionally hold `to` with a caller-defined tag
Users of the algorithm might want to depend on `to` being available on the receiving side for a longer duration than the lifetime of the current step's algorithm invocation.
For reasons explained in the next paragraph, we cannot guarantee that we have a `hold` when `recv` exists. If that were the case, we could take our time to provide a generalized callback to the user of the algorithm, and have them do whatever they want with `to` while we guarantee that `to` doesn't go away through the hold. But that functionality isn't available, so we only provide a fixed operation right after receive: **take a `hold` with a tag of the algorithm user's choice**. That's what's needed to guarantee that a replication plan, consisting of multiple consecutive steps, can be carried out without a receive-side prune job interfering by destroying `to`. Technically, this step is racy, i.e., it could happen that `to` is destroyed between `recv` and `hold`. But this is a) unlikely and b) not fatal because we can detect that hold fails because the 'dataset does not exist` and re-do the entire transmission since we still hold send-side `from` and `to`, i.e. we just lose a lot of work in rare cases.
So why can't we have a `hold` at the time `recv` exits?
It seems like [`zfs send --holds`](https://github.com/zfsonlinux/zfs/commit/9c5e88b1ded19cb4b19b9d767d5c71b34c189540) could be used to send the send-side's holds to the receive-side. But that flag always sends all holds, and `--holds` is implemented in user-space libzfs, i.e., doesn't happen in the same txg as the recv completion. Thus, the race window is in fact only smaller (unless we oversaw some synchronization in userland zfs).
But if we're willing to entertain the idea a little further, we still hit the problem that `--holds` sends _all_ holds, whereas our goal is to _temporarily_ have >= 1 hold that we own until the callback is done, and then release all of the received holds so that no holds created by us exist after this algorithm completes.
Since there could be concurrent `zfs hold` invocations on the sender and receiver while this algorithm runs, and because `--holds` doesn't provide info about which holds were sent, we cannot correctly destroy _exactly_ those holds that we received.
##### Send-side: callback to consumer, then cleanup
We can provide the algorithm user with a generic callback because we have holds / step bookmarks for `to` and `from`, respectively.
Example use case for the sender-side callback is the replication cursor, see algorithm for filesystem replication below.
After the callback is done: cleanup holds & step bookmark:
- `idempotent_release(to, zrepl_STEP_J_${jobid})`
- make sure `from` can now go away:
- if `from` is a snapshot: `idempotent_release(from, zrepl_STEP_J_${jobid})`
- else `idempotent_step_bookmark_destroy(from)`
- if `from` fulfills the properties of a step bookmark: destroy it
- otherwise: it's a bookmark not created by this algorithm that happened to be used for replication: leave it alone
- that can only happen if user used the override knob
Note that "make sure `from` can now go away" is the inverse of "Send-side: make sure `to` and `from` don't go away". Those are relatively complex operations and should be implemented in the same file next to each other to ease maintainability.
---
### Notes & Pseudo-APIs used in the algorithm
- `idempotent_hold(snapshot s, string tag)` like zfs hold, but doesn't error if hold already exists
- `idempotent_release(snapshot s, string tag)` like zfs hold, but doesn't error if hold already exists
- `idempotent_step_bookmark(snapshot_or_bookmark b)` creates a *step bookmark* of b
- determine step bookmark name N
- if `N` already exists, verify it's a correct step bookmark => DONE or ERROR
- if `b` is a snapshot, issue `zfs bookmark`
- if `b` is a bookmark:
- if bookmark cloning supported, use it to duplicate the bookmark
- else ERROR OUT, with an error that upper layers can identify as such, so that they are able to ignore the fact that we couldn't create a step bookmark
- `idempotent_destroy(bookmark #b_$GUID, of a snapshot s)` must atomically check that `$GUID == s.guid` before destroying s
- `idempotent_bookmark(snapshot s, $GUID, name #b_$GUID)` must atomically check that `$GUID == s.guid` at the time of bookmark creation
- `idempotent_destroy(snapshot s)` must atomically check that zrepl's `s.guid` matches the current `guid` of the snapshot (i.e. destroy by guid)
<a id="zrepl-algo-filesystem"></a>
## Algorithm for Planning and Executing Replication of a Filesystems (planned, not implemented yet)
This algorithm describes how a filesystem or zvol is replicated in zrepl.
The algorithm is invoked with a `jobid`, `sender`, `receiver`, a sender-side `filesystem path`.
It builds a diff between the sender and receiver filesystem bookmarks+snapshots and determines whether a replication conflict exists or whether fast-forward replication using full or incremental sends is possible.
In case of conflict, the algorithm errors out with a conflict description that can be used to manually or automatically resolve the conflict.
Otherwise, the algorithm builds a list of replication steps that are then worked on sequentially by the "Algorithm for a Single Replication Step".
The algorithm ensures that a plan can be executed exactly as planned by aquiring appropriate zfs holds.
The algorithm can be configured to retry a plan when encountering non-permanent errors (e.g. network errors).
However, permanent errors result in the plan being cancelled.
Regardless of whether a plan can be executed to completion or is cancelled, the algorithm guarantees that leftover artifacts (i.e. holds) of its invocation are cleaned up.
However, since network failure can occur at any point, there might be stale holds on sending or receiving side after a crash or other error.
These will be cleaned up on a subsequent invocation of this algorithm with the same `jobid`.
The algorithm is fit to be executed in parallel from the same sender to different receivers.
To be clear: replicating in parallel from the same sender to different receivers is supported.
But one instance of the algorithm assumes ownership of the `filesystem path` on the receiver.
The algorithm reserves the following sub-namespaces:
* zfs hold: `zrepl_FS`
* bookmark names: `zrepl_FS`
### Definitions
#### ZFS Filesystem Path
We refer to the name of a ZFS filesystem or volume as a *filesystem path*, sometimes just *path*.
The name includes the pool name.
#### Sender and Receiver
Sender and Receiver are passed as RPC stubs to the algorithm.
One of those RPC stubs will typically be local, i.e., call methods on a struct in the same address space.
The other RPC stub may invoke actual calls over the network (unless local replication is used).
The algorithm does not make any assumption about which object is local or an actual stub.
This design decouples the replication logic (described in this algorithm) from the question which side (sender or receiver) initiates the replication.
### Algorithm
**Cleanup Previous Invocations With Same `jobid`** by scanning the filesystem's list of snapshots and step bookmarks, and destroying any which was created by this algorithm when it was invoked with `jobid`.<br/>
TODO HOW
**Build a fast-forward list of replication steps** `STEPS`.
`STEPS` may contain an optional initial full send, and subsequent incremental send steps.
`STEPS[0]` may also have a resume token.
If fast-forward is not possible, produce a conflict description and ERROR OUT.<br/>
TODOs:
- make it configurable what snapshots are included in the list (i.e. every one we see, only most recent, at least one every X hours, ...)
**Ensure that we will be able to carry out all steps** by aquiring holds or fsstep bookmarks on the sending side
- `idempotent_hold([s.to for s in STEPS], zrepl_FS_J_${jobid})`
- `if STEPS[0].from != nil: idempotent_FSSTEP_bookmark(STEPS[0].from, zrepl_FSSTEP_bm_G_${STEPS[0].from.guid}_J_${jobid})`
**Determine which steps have not been completed (`uncompleted_steps`)** (we might be in an second invocation of this algorithm after a network failure and some steps might already be done):
- `res_tok := receiver.ResumeToken(fs)`
- `rmrfsv := receiver.MostRecentFilesystemVersion(fs)`
- if `res_tok != nil`: ensure that `res_tok` has a correspondinng step in `STEPS`, otherwise ERROR OUT
- if `rmrfsv != nil`: ensure that `res_tok` has a correspondinng step in `STEPS`, otherwise ERROR OUT
- if `(res_token != nil && rmrfsv != nil)`: ensure that `res_tok` is the subsequent step to the one we found for `rmrfsv`
- if both are nil, we are at the beginning, `uncompleted_steps = STEPS` and goto next block
- `rstep := if res_tok != nil { res_tok } else { rmrfsv }`
- `uncompleted_steps := STEPS[find_step_idx(STEPS, rstep).expect("must exist, checked above"):]`
- Note that we do not explicitly check for the completion of prior replication steps.
All we care about is what needs to be done from `rstep`.
- This is intentional and necessary because we cummutatively release all holds and step bookmarks made for steps that preceed a just-completed step (see next paragraph)
**Execute uncompleted steps**<br/>
Invoke the "Algorithm for a Single Replication Step" for each step in `uncompleted_steps`.
Remember to pass the resume token if one exists.
In the wind-down phase of each replication step `from => to`, while the per-step algorithm still holds send-side `from` and `to`, as well as recv-side `to`:
- Sending side: **Idempotently move the replication cursor to `to`.**
- Why: replication cursor tracks the sender's last known replication state, outlives the current plan, but is required to guarantee that future invocations of this algorithm find an incremental path.
- Impl for 'atomic' move: have two cursors (fixes [#177](https://github.com/zrepl/zrepl/issues/177))
- Idempotently cursor-bookmark `to` using `idempotent_bookmark(to, to.guid, #zrepl_replication_cursor_G_${to.guid}_J_${jobid})`
- Idempotently destroy old cursor-bookmark of `from` `idempotent_destroy(#zrepl_replication_cursor_G_${from.guid}_J_${jobid}, from)`
- If `from` is a snapshot, release the hold on it using `idempotent_release(from, zrepl_J_${jobid})`
- FIXME: resumability if we crash inbetween those operations (or scrap it and wait for channel programs to bring atomicity)
- Receiving side: **`idempotent_hold(to, zrepl_FS_J_${jobid})` because its going to be the next step's `from`**
- As discussed in the section on the per-step algorithm, this is a feature provided to us by the per-step algorithm.
- Receiving side + Sending side (order doesn't matter): Make sure all holds & step bookmarks made by this plan on already replicated snapshots are released / destroyed:
- `idempotent_release_prior_and_including(from, zrepl_FS_TODO)`
- `idempotent_step_bookmark_destroy_prior_and_including(from, zrepl_FS_TODO)`
**Cleanup** receiving and sending side (order doesn't matter)
- `idempotent_release_prior_and_including(STEPS[-1].to, zrepl_FS_TODO)`
- `idempotent_step_bookmark_destroy_prior_and_including(STEPS[-1].from, zrepl_FS_TODO)`
### Notes
- `idempotent_FSSTEP_bookmark` is like `idempotent_STEP_bookmark`, but with prefix `zrepl_FSSTEP` instead of `zrepl_STEP`
<a id="zrepl-algo-multiple-filesystems-notes"></a>
## Notes on Planning and Executing Replication of Multiple Filesystems
### The RPC Interfaces Uses Send-side Filesystem Names to Identify a Filesystem
The model of the receiver includes the assumption that it will `Receive` all snapshot streams sent to it into filesystems with paths prefixed with `root_fs`.
This is useful for separating filesystem path namespaces for multiple clients.
The receiver hides this prefixing in its RPC interface, i.e., when responding to `ListFilesystems` rpcs, the prefix is removed before sending the response.
This behavior is useful to achieve symmetry in this algorithm: we do not need to take the prefixing into consideration when computing diffs.
For the receiver, it has the advantage that `Receive` rpc can enforce the namespacing and need not trust this algorithm to apply it correctly.
The receiver is also allowed (and may need to) do implement other filtering / namespace transformations.
For example, when only `foo/a/b` has been received, but not `foo` nor `foo/a`, the receiver must not include the latter two in its `ListFilesystems` response.
The current zrepl receiver endpoint implementation uses the `zrepl:placeholder` property to mark filesystems `foo` and `foo/a` as placeholders, and filters out such filesystems in the `ListFilesystems` response.
Another approach would be to have a flat list of received filesystems per sender, and have a separate table that associates send-side names and receive-side filesystem paths.
Similarly, the sender is allowed to filter the output of its RPC interface to hide certain filesystems or snapshots.
#### Consequences of the Above Design
Namespace filtering and transformations of filesystem paths on sender and receiver are user-configurable.
Both ends of the replication setup have their own config, and do not coordinate config changes.
This results in a number of challenges:
- A sender might change its filter to allow additional filesystems to be replicated.
For example, where `foo/a/b` was initially allowed, the new filter might allow `foo` and `foo/a` as well.
If the receiver uses the `zrepl:placeholder` approach as sketched out above, this means that the receiver will need to replace the placeholder filesystems `$root_fs/foo` and `$root_fs/foo/a` with the incoming full sends of `foo` and `foo/a`.
- Send-side renames cannot be handled efficiently because send-side rename effectively changes the filesystem identity because we use its name.

View File

@ -23,6 +23,32 @@ var _ = math.Inf
// proto package needs to be updated.
const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
type Tri int32
const (
Tri_DontCare Tri = 0
Tri_False Tri = 1
Tri_True Tri = 2
)
var Tri_name = map[int32]string{
0: "DontCare",
1: "False",
2: "True",
}
var Tri_value = map[string]int32{
"DontCare": 0,
"False": 1,
"True": 2,
}
func (x Tri) String() string {
return proto.EnumName(Tri_name, int32(x))
}
func (Tri) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_pdu_0f43b713cd3bf056, []int{0}
}
type FilesystemVersion_VersionType int32
const (
@ -43,7 +69,7 @@ func (x FilesystemVersion_VersionType) String() string {
return proto.EnumName(FilesystemVersion_VersionType_name, int32(x))
}
func (FilesystemVersion_VersionType) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_pdu_83b7e2a28d820622, []int{5, 0}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{5, 0}
}
type ListFilesystemReq struct {
@ -56,7 +82,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_83b7e2a28d820622, []int{0}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{0}
}
func (m *ListFilesystemReq) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ListFilesystemReq.Unmarshal(m, b)
@ -87,7 +113,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_83b7e2a28d820622, []int{1}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{1}
}
func (m *ListFilesystemRes) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ListFilesystemRes.Unmarshal(m, b)
@ -118,6 +144,7 @@ type Filesystem struct {
Path string `protobuf:"bytes,1,opt,name=Path,proto3" json:"Path,omitempty"`
ResumeToken string `protobuf:"bytes,2,opt,name=ResumeToken,proto3" json:"ResumeToken,omitempty"`
IsPlaceholder bool `protobuf:"varint,3,opt,name=IsPlaceholder,proto3" json:"IsPlaceholder,omitempty"`
IsEncrypted bool `protobuf:"varint,4,opt,name=IsEncrypted,proto3" json:"IsEncrypted,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -127,7 +154,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_83b7e2a28d820622, []int{2}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{2}
}
func (m *Filesystem) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Filesystem.Unmarshal(m, b)
@ -168,6 +195,13 @@ func (m *Filesystem) GetIsPlaceholder() bool {
return false
}
func (m *Filesystem) GetIsEncrypted() bool {
if m != nil {
return m.IsEncrypted
}
return false
}
type ListFilesystemVersionsReq struct {
Filesystem string `protobuf:"bytes,1,opt,name=Filesystem,proto3" json:"Filesystem,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
@ -179,7 +213,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_83b7e2a28d820622, []int{3}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{3}
}
func (m *ListFilesystemVersionsReq) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ListFilesystemVersionsReq.Unmarshal(m, b)
@ -217,7 +251,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_83b7e2a28d820622, []int{4}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{4}
}
func (m *ListFilesystemVersionsRes) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ListFilesystemVersionsRes.Unmarshal(m, b)
@ -259,7 +293,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_83b7e2a28d820622, []int{5}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{5}
}
func (m *FilesystemVersion) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_FilesystemVersion.Unmarshal(m, b)
@ -316,20 +350,18 @@ func (m *FilesystemVersion) GetCreation() string {
type SendReq struct {
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,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
// 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.
// May be empty / null to request a full transfer of To
From *FilesystemVersion `protobuf:"bytes,2,opt,name=From,proto3" json:"From,omitempty"`
To *FilesystemVersion `protobuf:"bytes,3,opt,name=To,proto3" json:"To,omitempty"`
// If ResumeToken is not empty, the resume token that CAN be used for 'zfs
// send' by the sender. The sender MUST indicate use of ResumeToken in the
// reply message 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,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"`
Encrypted Tri `protobuf:"varint,5,opt,name=Encrypted,proto3,enum=Tri" json:"Encrypted,omitempty"`
DryRun bool `protobuf:"varint,6,opt,name=DryRun,proto3" json:"DryRun,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -339,7 +371,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_83b7e2a28d820622, []int{6}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{6}
}
func (m *SendReq) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SendReq.Unmarshal(m, b)
@ -366,18 +398,18 @@ func (m *SendReq) GetFilesystem() string {
return ""
}
func (m *SendReq) GetFrom() string {
func (m *SendReq) GetFrom() *FilesystemVersion {
if m != nil {
return m.From
}
return ""
return nil
}
func (m *SendReq) GetTo() string {
func (m *SendReq) GetTo() *FilesystemVersion {
if m != nil {
return m.To
}
return ""
return nil
}
func (m *SendReq) GetResumeToken() string {
@ -387,18 +419,11 @@ func (m *SendReq) GetResumeToken() string {
return ""
}
func (m *SendReq) GetCompress() bool {
func (m *SendReq) GetEncrypted() Tri {
if m != nil {
return m.Compress
return m.Encrypted
}
return false
}
func (m *SendReq) GetDedup() bool {
if m != nil {
return m.Dedup
}
return false
return Tri_DontCare
}
func (m *SendReq) GetDryRun() bool {
@ -420,7 +445,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_83b7e2a28d820622, []int{7}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{7}
}
func (m *Property) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Property.Unmarshal(m, b)
@ -456,6 +481,7 @@ func (m *Property) GetValue() string {
type SendRes struct {
// Whether the resume token provided in the request has been used or not.
// If the SendReq.ResumeToken == "", this field has no meaning.
UsedResumeToken bool `protobuf:"varint,2,opt,name=UsedResumeToken,proto3" json:"UsedResumeToken,omitempty"`
// Expected stream size determined by dry run, not exact.
// 0 indicates that for the given SendReq, no size estimate could be made.
@ -470,7 +496,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_83b7e2a28d820622, []int{8}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{8}
}
func (m *SendRes) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SendRes.Unmarshal(m, b)
@ -511,10 +537,80 @@ func (m *SendRes) GetProperties() []*Property {
return nil
}
type SendCompletedReq struct {
OriginalReq *SendReq `protobuf:"bytes,2,opt,name=OriginalReq,proto3" json:"OriginalReq,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *SendCompletedReq) Reset() { *m = SendCompletedReq{} }
func (m *SendCompletedReq) String() string { return proto.CompactTextString(m) }
func (*SendCompletedReq) ProtoMessage() {}
func (*SendCompletedReq) Descriptor() ([]byte, []int) {
return fileDescriptor_pdu_0f43b713cd3bf056, []int{9}
}
func (m *SendCompletedReq) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SendCompletedReq.Unmarshal(m, b)
}
func (m *SendCompletedReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_SendCompletedReq.Marshal(b, m, deterministic)
}
func (dst *SendCompletedReq) XXX_Merge(src proto.Message) {
xxx_messageInfo_SendCompletedReq.Merge(dst, src)
}
func (m *SendCompletedReq) XXX_Size() int {
return xxx_messageInfo_SendCompletedReq.Size(m)
}
func (m *SendCompletedReq) XXX_DiscardUnknown() {
xxx_messageInfo_SendCompletedReq.DiscardUnknown(m)
}
var xxx_messageInfo_SendCompletedReq proto.InternalMessageInfo
func (m *SendCompletedReq) GetOriginalReq() *SendReq {
if m != nil {
return m.OriginalReq
}
return nil
}
type SendCompletedRes struct {
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *SendCompletedRes) Reset() { *m = SendCompletedRes{} }
func (m *SendCompletedRes) String() string { return proto.CompactTextString(m) }
func (*SendCompletedRes) ProtoMessage() {}
func (*SendCompletedRes) Descriptor() ([]byte, []int) {
return fileDescriptor_pdu_0f43b713cd3bf056, []int{10}
}
func (m *SendCompletedRes) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SendCompletedRes.Unmarshal(m, b)
}
func (m *SendCompletedRes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_SendCompletedRes.Marshal(b, m, deterministic)
}
func (dst *SendCompletedRes) XXX_Merge(src proto.Message) {
xxx_messageInfo_SendCompletedRes.Merge(dst, src)
}
func (m *SendCompletedRes) XXX_Size() int {
return xxx_messageInfo_SendCompletedRes.Size(m)
}
func (m *SendCompletedRes) XXX_DiscardUnknown() {
xxx_messageInfo_SendCompletedRes.DiscardUnknown(m)
}
var xxx_messageInfo_SendCompletedRes proto.InternalMessageInfo
type ReceiveReq struct {
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,proto3" json:"ClearResumeToken,omitempty"`
To *FilesystemVersion `protobuf:"bytes,2,opt,name=To,proto3" json:"To,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,3,opt,name=ClearResumeToken,proto3" json:"ClearResumeToken,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -524,7 +620,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_83b7e2a28d820622, []int{9}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{11}
}
func (m *ReceiveReq) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ReceiveReq.Unmarshal(m, b)
@ -551,6 +647,13 @@ func (m *ReceiveReq) GetFilesystem() string {
return ""
}
func (m *ReceiveReq) GetTo() *FilesystemVersion {
if m != nil {
return m.To
}
return nil
}
func (m *ReceiveReq) GetClearResumeToken() bool {
if m != nil {
return m.ClearResumeToken
@ -568,7 +671,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_83b7e2a28d820622, []int{10}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{12}
}
func (m *ReceiveRes) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ReceiveRes.Unmarshal(m, b)
@ -601,7 +704,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_83b7e2a28d820622, []int{11}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{13}
}
func (m *DestroySnapshotsReq) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DestroySnapshotsReq.Unmarshal(m, b)
@ -647,7 +750,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_83b7e2a28d820622, []int{12}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{14}
}
func (m *DestroySnapshotRes) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DestroySnapshotRes.Unmarshal(m, b)
@ -692,7 +795,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_83b7e2a28d820622, []int{13}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{15}
}
func (m *DestroySnapshotsRes) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DestroySnapshotsRes.Unmarshal(m, b)
@ -721,10 +824,6 @@ func (m *DestroySnapshotsRes) GetResults() []*DestroySnapshotRes {
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:"-"`
@ -734,7 +833,7 @@ 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_83b7e2a28d820622, []int{14}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{16}
}
func (m *ReplicationCursorReq) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ReplicationCursorReq.Unmarshal(m, b)
@ -761,185 +860,6 @@ func (m *ReplicationCursorReq) GetFilesystem() string {
return ""
}
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_83b7e2a28d820622, []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_83b7e2a28d820622, []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 ""
}
type ReplicationCursorRes struct {
// Types that are valid to be assigned to Result:
// *ReplicationCursorRes_Guid
@ -954,7 +874,7 @@ 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_83b7e2a28d820622, []int{15}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{17}
}
func (m *ReplicationCursorRes) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ReplicationCursorRes.Unmarshal(m, b)
@ -1090,7 +1010,7 @@ func (m *PingReq) Reset() { *m = PingReq{} }
func (m *PingReq) String() string { return proto.CompactTextString(m) }
func (*PingReq) ProtoMessage() {}
func (*PingReq) Descriptor() ([]byte, []int) {
return fileDescriptor_pdu_83b7e2a28d820622, []int{16}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{18}
}
func (m *PingReq) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_PingReq.Unmarshal(m, b)
@ -1129,7 +1049,7 @@ func (m *PingRes) Reset() { *m = PingRes{} }
func (m *PingRes) String() string { return proto.CompactTextString(m) }
func (*PingRes) ProtoMessage() {}
func (*PingRes) Descriptor() ([]byte, []int) {
return fileDescriptor_pdu_83b7e2a28d820622, []int{17}
return fileDescriptor_pdu_0f43b713cd3bf056, []int{19}
}
func (m *PingRes) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_PingRes.Unmarshal(m, b)
@ -1156,6 +1076,82 @@ func (m *PingRes) GetEcho() string {
return ""
}
type HintMostRecentCommonAncestorReq struct {
Filesystem string `protobuf:"bytes,1,opt,name=Filesystem,proto3" json:"Filesystem,omitempty"`
SenderVersion *FilesystemVersion `protobuf:"bytes,2,opt,name=SenderVersion,proto3" json:"SenderVersion,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *HintMostRecentCommonAncestorReq) Reset() { *m = HintMostRecentCommonAncestorReq{} }
func (m *HintMostRecentCommonAncestorReq) String() string { return proto.CompactTextString(m) }
func (*HintMostRecentCommonAncestorReq) ProtoMessage() {}
func (*HintMostRecentCommonAncestorReq) Descriptor() ([]byte, []int) {
return fileDescriptor_pdu_0f43b713cd3bf056, []int{20}
}
func (m *HintMostRecentCommonAncestorReq) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_HintMostRecentCommonAncestorReq.Unmarshal(m, b)
}
func (m *HintMostRecentCommonAncestorReq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_HintMostRecentCommonAncestorReq.Marshal(b, m, deterministic)
}
func (dst *HintMostRecentCommonAncestorReq) XXX_Merge(src proto.Message) {
xxx_messageInfo_HintMostRecentCommonAncestorReq.Merge(dst, src)
}
func (m *HintMostRecentCommonAncestorReq) XXX_Size() int {
return xxx_messageInfo_HintMostRecentCommonAncestorReq.Size(m)
}
func (m *HintMostRecentCommonAncestorReq) XXX_DiscardUnknown() {
xxx_messageInfo_HintMostRecentCommonAncestorReq.DiscardUnknown(m)
}
var xxx_messageInfo_HintMostRecentCommonAncestorReq proto.InternalMessageInfo
func (m *HintMostRecentCommonAncestorReq) GetFilesystem() string {
if m != nil {
return m.Filesystem
}
return ""
}
func (m *HintMostRecentCommonAncestorReq) GetSenderVersion() *FilesystemVersion {
if m != nil {
return m.SenderVersion
}
return nil
}
type HintMostRecentCommonAncestorRes struct {
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *HintMostRecentCommonAncestorRes) Reset() { *m = HintMostRecentCommonAncestorRes{} }
func (m *HintMostRecentCommonAncestorRes) String() string { return proto.CompactTextString(m) }
func (*HintMostRecentCommonAncestorRes) ProtoMessage() {}
func (*HintMostRecentCommonAncestorRes) Descriptor() ([]byte, []int) {
return fileDescriptor_pdu_0f43b713cd3bf056, []int{21}
}
func (m *HintMostRecentCommonAncestorRes) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_HintMostRecentCommonAncestorRes.Unmarshal(m, b)
}
func (m *HintMostRecentCommonAncestorRes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_HintMostRecentCommonAncestorRes.Marshal(b, m, deterministic)
}
func (dst *HintMostRecentCommonAncestorRes) XXX_Merge(src proto.Message) {
xxx_messageInfo_HintMostRecentCommonAncestorRes.Merge(dst, src)
}
func (m *HintMostRecentCommonAncestorRes) XXX_Size() int {
return xxx_messageInfo_HintMostRecentCommonAncestorRes.Size(m)
}
func (m *HintMostRecentCommonAncestorRes) XXX_DiscardUnknown() {
xxx_messageInfo_HintMostRecentCommonAncestorRes.DiscardUnknown(m)
}
var xxx_messageInfo_HintMostRecentCommonAncestorRes proto.InternalMessageInfo
func init() {
proto.RegisterType((*ListFilesystemReq)(nil), "ListFilesystemReq")
proto.RegisterType((*ListFilesystemRes)(nil), "ListFilesystemRes")
@ -1166,17 +1162,20 @@ func init() {
proto.RegisterType((*SendReq)(nil), "SendReq")
proto.RegisterType((*Property)(nil), "Property")
proto.RegisterType((*SendRes)(nil), "SendRes")
proto.RegisterType((*SendCompletedReq)(nil), "SendCompletedReq")
proto.RegisterType((*SendCompletedRes)(nil), "SendCompletedRes")
proto.RegisterType((*ReceiveReq)(nil), "ReceiveReq")
proto.RegisterType((*ReceiveRes)(nil), "ReceiveRes")
proto.RegisterType((*DestroySnapshotsReq)(nil), "DestroySnapshotsReq")
proto.RegisterType((*DestroySnapshotRes)(nil), "DestroySnapshotRes")
proto.RegisterType((*DestroySnapshotsRes)(nil), "DestroySnapshotsRes")
proto.RegisterType((*ReplicationCursorReq)(nil), "ReplicationCursorReq")
proto.RegisterType((*ReplicationCursorReq_GetOp)(nil), "ReplicationCursorReq.GetOp")
proto.RegisterType((*ReplicationCursorReq_SetOp)(nil), "ReplicationCursorReq.SetOp")
proto.RegisterType((*ReplicationCursorRes)(nil), "ReplicationCursorRes")
proto.RegisterType((*PingReq)(nil), "PingReq")
proto.RegisterType((*PingRes)(nil), "PingRes")
proto.RegisterType((*HintMostRecentCommonAncestorReq)(nil), "HintMostRecentCommonAncestorReq")
proto.RegisterType((*HintMostRecentCommonAncestorRes)(nil), "HintMostRecentCommonAncestorRes")
proto.RegisterEnum("Tri", Tri_name, Tri_value)
proto.RegisterEnum("FilesystemVersion_VersionType", FilesystemVersion_VersionType_name, FilesystemVersion_VersionType_value)
}
@ -1197,6 +1196,8 @@ type ReplicationClient interface {
ListFilesystemVersions(ctx context.Context, in *ListFilesystemVersionsReq, opts ...grpc.CallOption) (*ListFilesystemVersionsRes, error)
DestroySnapshots(ctx context.Context, in *DestroySnapshotsReq, opts ...grpc.CallOption) (*DestroySnapshotsRes, error)
ReplicationCursor(ctx context.Context, in *ReplicationCursorReq, opts ...grpc.CallOption) (*ReplicationCursorRes, error)
SendCompleted(ctx context.Context, in *SendCompletedReq, opts ...grpc.CallOption) (*SendCompletedRes, error)
HintMostRecentCommonAncestor(ctx context.Context, in *HintMostRecentCommonAncestorReq, opts ...grpc.CallOption) (*HintMostRecentCommonAncestorRes, error)
}
type replicationClient struct {
@ -1252,6 +1253,24 @@ func (c *replicationClient) ReplicationCursor(ctx context.Context, in *Replicati
return out, nil
}
func (c *replicationClient) SendCompleted(ctx context.Context, in *SendCompletedReq, opts ...grpc.CallOption) (*SendCompletedRes, error) {
out := new(SendCompletedRes)
err := c.cc.Invoke(ctx, "/Replication/SendCompleted", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *replicationClient) HintMostRecentCommonAncestor(ctx context.Context, in *HintMostRecentCommonAncestorReq, opts ...grpc.CallOption) (*HintMostRecentCommonAncestorRes, error) {
out := new(HintMostRecentCommonAncestorRes)
err := c.cc.Invoke(ctx, "/Replication/HintMostRecentCommonAncestor", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
// ReplicationServer is the server API for Replication service.
type ReplicationServer interface {
Ping(context.Context, *PingReq) (*PingRes, error)
@ -1259,6 +1278,8 @@ type ReplicationServer interface {
ListFilesystemVersions(context.Context, *ListFilesystemVersionsReq) (*ListFilesystemVersionsRes, error)
DestroySnapshots(context.Context, *DestroySnapshotsReq) (*DestroySnapshotsRes, error)
ReplicationCursor(context.Context, *ReplicationCursorReq) (*ReplicationCursorRes, error)
SendCompleted(context.Context, *SendCompletedReq) (*SendCompletedRes, error)
HintMostRecentCommonAncestor(context.Context, *HintMostRecentCommonAncestorReq) (*HintMostRecentCommonAncestorRes, error)
}
func RegisterReplicationServer(s *grpc.Server, srv ReplicationServer) {
@ -1355,6 +1376,42 @@ func _Replication_ReplicationCursor_Handler(srv interface{}, ctx context.Context
return interceptor(ctx, in, info, handler)
}
func _Replication_SendCompleted_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(SendCompletedReq)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(ReplicationServer).SendCompleted(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/Replication/SendCompleted",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(ReplicationServer).SendCompleted(ctx, req.(*SendCompletedReq))
}
return interceptor(ctx, in, info, handler)
}
func _Replication_HintMostRecentCommonAncestor_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(HintMostRecentCommonAncestorReq)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(ReplicationServer).HintMostRecentCommonAncestor(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/Replication/HintMostRecentCommonAncestor",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(ReplicationServer).HintMostRecentCommonAncestor(ctx, req.(*HintMostRecentCommonAncestorReq))
}
return interceptor(ctx, in, info, handler)
}
var _Replication_serviceDesc = grpc.ServiceDesc{
ServiceName: "Replication",
HandlerType: (*ReplicationServer)(nil),
@ -1379,63 +1436,77 @@ var _Replication_serviceDesc = grpc.ServiceDesc{
MethodName: "ReplicationCursor",
Handler: _Replication_ReplicationCursor_Handler,
},
{
MethodName: "SendCompleted",
Handler: _Replication_SendCompleted_Handler,
},
{
MethodName: "HintMostRecentCommonAncestor",
Handler: _Replication_HintMostRecentCommonAncestor_Handler,
},
},
Streams: []grpc.StreamDesc{},
Metadata: "pdu.proto",
}
func init() { proto.RegisterFile("pdu.proto", fileDescriptor_pdu_83b7e2a28d820622) }
func init() { proto.RegisterFile("pdu.proto", fileDescriptor_pdu_0f43b713cd3bf056) }
var fileDescriptor_pdu_83b7e2a28d820622 = []byte{
// 785 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0xd1, 0x8e, 0xe3, 0x34,
0x14, 0x9d, 0xb4, 0x69, 0x9b, 0xde, 0x0e, 0xbb, 0x1d, 0x4f, 0x59, 0x85, 0x00, 0xab, 0xca, 0xcb,
0x43, 0x17, 0x89, 0x80, 0x0a, 0x2f, 0x08, 0x09, 0x89, 0x4e, 0x67, 0x67, 0x10, 0xb0, 0x54, 0x6e,
0x59, 0xad, 0xf6, 0x2d, 0x34, 0x57, 0x6d, 0x34, 0x6d, 0x9d, 0xb5, 0x13, 0xb4, 0xe5, 0x91, 0xbf,
0x9a, 0x7f, 0xe0, 0x91, 0x0f, 0x42, 0x76, 0xe3, 0x34, 0x6d, 0x52, 0xa9, 0x4f, 0xf1, 0x39, 0xf7,
0xda, 0x3e, 0xf7, 0xd8, 0xd7, 0x81, 0x76, 0x1c, 0xa6, 0x7e, 0x2c, 0x78, 0xc2, 0xe9, 0x35, 0x5c,
0xfd, 0x1a, 0xc9, 0xe4, 0x55, 0xb4, 0x42, 0xb9, 0x95, 0x09, 0xae, 0x19, 0xbe, 0xa7, 0xa3, 0x32,
0x29, 0xc9, 0x57, 0xd0, 0xd9, 0x13, 0xd2, 0xb5, 0xfa, 0xf5, 0x41, 0x67, 0xd8, 0xf1, 0x0b, 0x49,
0xc5, 0x38, 0x5d, 0x02, 0xec, 0x21, 0x21, 0x60, 0x4f, 0x82, 0x64, 0xe9, 0x5a, 0x7d, 0x6b, 0xd0,
0x66, 0x7a, 0x4c, 0xfa, 0xd0, 0x61, 0x28, 0xd3, 0x35, 0xce, 0xf8, 0x03, 0x6e, 0xdc, 0x9a, 0x0e,
0x15, 0x29, 0xf2, 0x05, 0x7c, 0xf4, 0xb3, 0x9c, 0xac, 0x82, 0x39, 0x2e, 0xf9, 0x2a, 0x44, 0xe1,
0xd6, 0xfb, 0xd6, 0xc0, 0x61, 0x87, 0x24, 0xfd, 0x01, 0x3e, 0x39, 0x54, 0xfb, 0x06, 0x85, 0x8c,
0xf8, 0x46, 0x32, 0x7c, 0x4f, 0x9e, 0x17, 0x65, 0x64, 0xdb, 0x17, 0x18, 0xfa, 0xcb, 0xe9, 0xc9,
0x92, 0xf8, 0xe0, 0x18, 0x98, 0xd5, 0x4b, 0xfc, 0x52, 0x26, 0xcb, 0x73, 0xe8, 0x7f, 0x16, 0x5c,
0x95, 0xe2, 0x64, 0x08, 0xf6, 0x6c, 0x1b, 0xa3, 0xde, 0xfc, 0xc9, 0xf0, 0x79, 0x79, 0x05, 0x3f,
0xfb, 0xaa, 0x2c, 0xa6, 0x73, 0x95, 0x5f, 0xaf, 0x83, 0x35, 0x66, 0xa6, 0xe8, 0xb1, 0xe2, 0xee,
0xd2, 0x28, 0xd4, 0x26, 0xd8, 0x4c, 0x8f, 0xc9, 0x67, 0xd0, 0xbe, 0x11, 0x18, 0x24, 0x38, 0x7b,
0x7b, 0xe7, 0xda, 0x3a, 0xb0, 0x27, 0x88, 0x07, 0x8e, 0x06, 0x11, 0xdf, 0xb8, 0x0d, 0xbd, 0x52,
0x8e, 0xe9, 0x4b, 0xe8, 0x14, 0xb6, 0x25, 0x97, 0xe0, 0x4c, 0x37, 0x41, 0x2c, 0x97, 0x3c, 0xe9,
0x5e, 0x28, 0x34, 0xe2, 0xfc, 0x61, 0x1d, 0x88, 0x87, 0xae, 0x45, 0x1f, 0x2d, 0x68, 0x4d, 0x71,
0x13, 0x9e, 0xe1, 0xa7, 0x12, 0xf9, 0x4a, 0xf0, 0xb5, 0x11, 0xae, 0xc6, 0xe4, 0x09, 0xd4, 0x66,
0x5c, 0xcb, 0x6e, 0xb3, 0xda, 0x8c, 0x1f, 0x1f, 0xbc, 0x5d, 0x3e, 0x78, 0x25, 0x9c, 0xaf, 0x63,
0x81, 0x52, 0x6a, 0xe1, 0x0e, 0xcb, 0x31, 0xe9, 0x41, 0x63, 0x8c, 0x61, 0x1a, 0xbb, 0x4d, 0x1d,
0xd8, 0x01, 0xf2, 0x0c, 0x9a, 0x63, 0xb1, 0x65, 0xe9, 0xc6, 0x6d, 0x69, 0x3a, 0x43, 0xf4, 0x3b,
0x70, 0x26, 0x82, 0xc7, 0x28, 0x92, 0x6d, 0x6e, 0xaa, 0x55, 0x30, 0xb5, 0x07, 0x8d, 0x37, 0xc1,
0x2a, 0x35, 0x4e, 0xef, 0x00, 0xfd, 0x27, 0xaf, 0x58, 0x92, 0x01, 0x3c, 0xfd, 0x43, 0x62, 0x78,
0x7c, 0x55, 0x1d, 0x76, 0x4c, 0x13, 0x0a, 0x97, 0xb7, 0x1f, 0x62, 0x9c, 0x27, 0x18, 0x4e, 0xa3,
0xbf, 0x51, 0x57, 0x5c, 0x67, 0x07, 0x1c, 0x79, 0x09, 0x90, 0xe9, 0x89, 0x50, 0xba, 0xb6, 0xbe,
0x54, 0x6d, 0xdf, 0x48, 0x64, 0x85, 0x20, 0x7d, 0x0b, 0xc0, 0x70, 0x8e, 0xd1, 0x5f, 0x78, 0x8e,
0xf1, 0x5f, 0x42, 0xf7, 0x66, 0x85, 0x81, 0x28, 0xeb, 0x2c, 0xf1, 0xf4, 0xb2, 0xb0, 0xb2, 0xa4,
0x0b, 0xb8, 0x1e, 0xa3, 0x4c, 0x04, 0xdf, 0x9a, 0x1b, 0x70, 0x4e, 0xe7, 0x90, 0x6f, 0xa0, 0x9d,
0xe7, 0xbb, 0xb5, 0x93, 0xdd, 0xb1, 0x4f, 0xa2, 0xef, 0x80, 0x1c, 0x6d, 0x94, 0x35, 0x99, 0x81,
0x7a, 0x97, 0x13, 0x4d, 0x66, 0x72, 0xd4, 0x89, 0xdd, 0x0a, 0xc1, 0x85, 0x39, 0x31, 0x0d, 0xe8,
0xb8, 0xaa, 0x08, 0xf5, 0x68, 0xb5, 0x54, 0xe1, 0xab, 0xc4, 0x34, 0xf0, 0xb5, 0x5f, 0x96, 0xc0,
0x4c, 0x0e, 0xfd, 0xd7, 0x82, 0x1e, 0xc3, 0x78, 0x15, 0xcd, 0x75, 0x93, 0xdc, 0xa4, 0x42, 0x72,
0x71, 0x8e, 0x19, 0x5f, 0x43, 0x7d, 0x81, 0x89, 0x96, 0xd4, 0x19, 0x7e, 0xea, 0x57, 0xad, 0xe1,
0xdf, 0x61, 0xf2, 0x7b, 0x7c, 0x7f, 0xc1, 0x54, 0xa6, 0x9a, 0x20, 0x31, 0xd1, 0x57, 0xe4, 0xe4,
0x84, 0xa9, 0x99, 0x20, 0x31, 0xf1, 0x5a, 0xd0, 0xd0, 0x0b, 0x78, 0x2f, 0xa0, 0xa1, 0x03, 0xaa,
0x49, 0x72, 0xe3, 0x76, 0x5e, 0xe4, 0x78, 0x64, 0x43, 0x8d, 0xc7, 0x74, 0x56, 0x59, 0x8d, 0x6a,
0xa1, 0xdd, 0x4b, 0xa2, 0xea, 0xb0, 0xef, 0x2f, 0xf2, 0xb7, 0xc4, 0x79, 0xcd, 0x13, 0xfc, 0x10,
0xc9, 0xdd, 0x7a, 0xce, 0xfd, 0x05, 0xcb, 0x99, 0x91, 0x03, 0xcd, 0x9d, 0x4b, 0xf4, 0x05, 0xb4,
0x26, 0xd1, 0x66, 0xa1, 0x6c, 0x71, 0xa1, 0xf5, 0x1b, 0x4a, 0x19, 0x2c, 0x4c, 0x53, 0x19, 0x48,
0x3f, 0x37, 0x49, 0x52, 0xb5, 0xdd, 0xed, 0x7c, 0xc9, 0x4d, 0xdb, 0xa9, 0xf1, 0xf0, 0xb1, 0xa6,
0xde, 0x80, 0x5c, 0x1a, 0xf1, 0xc0, 0x56, 0xe9, 0xc4, 0xf1, 0xb3, 0xa5, 0x3d, 0x33, 0x92, 0xe4,
0x7b, 0x78, 0x7a, 0xf8, 0x44, 0x4b, 0x42, 0xfc, 0xd2, 0x4f, 0xcb, 0x2b, 0x73, 0x92, 0x4c, 0xe0,
0x59, 0xf5, 0xeb, 0x4e, 0x3c, 0xff, 0xe4, 0x3f, 0xc3, 0x3b, 0x1d, 0x93, 0xe4, 0x47, 0xe8, 0x1e,
0xdf, 0x33, 0xd2, 0xf3, 0x2b, 0xfa, 0xc7, 0xab, 0x62, 0x25, 0xf9, 0x09, 0xae, 0x4a, 0x47, 0x42,
0x3e, 0xae, 0x3c, 0x7f, 0xaf, 0x92, 0x96, 0xa3, 0xc6, 0xbb, 0x7a, 0x1c, 0xa6, 0x7f, 0x36, 0xf5,
0x0f, 0xfc, 0xdb, 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0x37, 0x0e, 0xf2, 0xe4, 0xcd, 0x07, 0x00,
0x00,
var fileDescriptor_pdu_0f43b713cd3bf056 = []byte{
// 892 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x56, 0xdf, 0x6f, 0xdb, 0x36,
0x10, 0x8e, 0x6c, 0x39, 0x91, 0xcf, 0xe9, 0xea, 0x5c, 0xb2, 0x42, 0x13, 0xba, 0xce, 0xe3, 0x86,
0xc1, 0x0d, 0x30, 0x61, 0xc8, 0x7e, 0x60, 0xc3, 0x80, 0x02, 0x8d, 0x93, 0x34, 0xc5, 0xd6, 0xce,
0x60, 0xbc, 0x62, 0xe8, 0x9b, 0x6a, 0x1f, 0x12, 0x21, 0xb2, 0xa8, 0x90, 0xf4, 0x50, 0x6f, 0x7b,
0xda, 0xe3, 0xfe, 0xbd, 0xe5, 0x0f, 0x2a, 0x44, 0x4b, 0xb6, 0x6c, 0xc9, 0x89, 0x9f, 0xcc, 0xfb,
0x78, 0x14, 0xef, 0xbe, 0xfb, 0xee, 0x68, 0x68, 0x26, 0xa3, 0x89, 0x9f, 0x48, 0xa1, 0x05, 0xdb,
0x87, 0xbd, 0x5f, 0x43, 0xa5, 0xcf, 0xc2, 0x88, 0xd4, 0x54, 0x69, 0x1a, 0x73, 0xba, 0x61, 0xc7,
0x65, 0x50, 0xe1, 0xd7, 0xd0, 0x5a, 0x00, 0xca, 0xb5, 0x3a, 0xf5, 0x6e, 0xeb, 0xa8, 0xe5, 0x17,
0x9c, 0x8a, 0xfb, 0xec, 0x3f, 0x0b, 0x60, 0x61, 0x23, 0x82, 0xdd, 0x0f, 0xf4, 0x95, 0x6b, 0x75,
0xac, 0x6e, 0x93, 0x9b, 0x35, 0x76, 0xa0, 0xc5, 0x49, 0x4d, 0xc6, 0x34, 0x10, 0xd7, 0x14, 0xbb,
0x35, 0xb3, 0x55, 0x84, 0xf0, 0x4b, 0x78, 0xf0, 0x52, 0xf5, 0xa3, 0x60, 0x48, 0x57, 0x22, 0x1a,
0x91, 0x74, 0xeb, 0x1d, 0xab, 0xeb, 0xf0, 0x65, 0x30, 0xfd, 0xce, 0x4b, 0x75, 0x1a, 0x0f, 0xe5,
0x34, 0xd1, 0x34, 0x72, 0x6d, 0xe3, 0x53, 0x84, 0xd8, 0xcf, 0xf0, 0xc9, 0x72, 0x42, 0x6f, 0x48,
0xaa, 0x50, 0xc4, 0x8a, 0xd3, 0x0d, 0x3e, 0x29, 0x06, 0x9a, 0x05, 0x58, 0x40, 0xd8, 0x2f, 0xeb,
0x0f, 0x2b, 0xf4, 0xc1, 0xc9, 0xcd, 0x8c, 0x12, 0xf4, 0x4b, 0x9e, 0x7c, 0xee, 0xc3, 0x6e, 0x2d,
0xd8, 0x2b, 0xed, 0xe3, 0x11, 0xd8, 0x83, 0x69, 0x42, 0xe6, 0xf2, 0x8f, 0x8e, 0x9e, 0x94, 0xbf,
0xe0, 0x67, 0xbf, 0xa9, 0x17, 0x37, 0xbe, 0x29, 0xa3, 0xaf, 0x83, 0x31, 0x65, 0xb4, 0x99, 0x75,
0x8a, 0xbd, 0x98, 0x84, 0x23, 0x43, 0x93, 0xcd, 0xcd, 0x1a, 0x1f, 0x43, 0xb3, 0x27, 0x29, 0xd0,
0x34, 0xf8, 0xe3, 0x85, 0xe1, 0xc6, 0xe6, 0x0b, 0x00, 0x3d, 0x70, 0x8c, 0x11, 0x8a, 0xd8, 0x6d,
0x98, 0x2f, 0xcd, 0x6d, 0xf6, 0x14, 0x5a, 0x85, 0x6b, 0x71, 0x17, 0x9c, 0x8b, 0x38, 0x48, 0xd4,
0x95, 0xd0, 0xed, 0xad, 0xd4, 0x3a, 0x16, 0xe2, 0x7a, 0x1c, 0xc8, 0xeb, 0xb6, 0xc5, 0xfe, 0xb7,
0x60, 0xe7, 0x82, 0xe2, 0xd1, 0x06, 0x7c, 0xe2, 0x57, 0x60, 0x9f, 0x49, 0x31, 0x36, 0x81, 0x57,
0xd3, 0x65, 0xf6, 0x91, 0x41, 0x6d, 0x20, 0x4c, 0x2a, 0xd5, 0x5e, 0xb5, 0x81, 0x58, 0x95, 0x90,
0x5d, 0x96, 0x10, 0x83, 0xe6, 0x42, 0x1a, 0x0d, 0xc3, 0xaf, 0xed, 0x0f, 0x64, 0xc8, 0x17, 0x30,
0x3e, 0x82, 0xed, 0x13, 0x39, 0xe5, 0x93, 0xd8, 0xdd, 0x36, 0xda, 0xc9, 0x2c, 0xf6, 0x1d, 0x38,
0x7d, 0x29, 0x12, 0x92, 0x7a, 0x3a, 0xa7, 0xdb, 0x2a, 0xd0, 0x7d, 0x00, 0x8d, 0x37, 0x41, 0x34,
0xc9, 0x6b, 0x30, 0x33, 0xd8, 0xbf, 0x73, 0x2e, 0x14, 0x76, 0xe1, 0xe1, 0xef, 0x8a, 0x46, 0xab,
0x32, 0x77, 0xf8, 0x2a, 0x8c, 0x0c, 0x76, 0x4f, 0xdf, 0x27, 0x34, 0xd4, 0x34, 0xba, 0x08, 0xff,
0x22, 0x93, 0x77, 0x9d, 0x2f, 0x61, 0xf8, 0x14, 0x20, 0x8b, 0x27, 0x24, 0xe5, 0xda, 0x46, 0x6e,
0x4d, 0x3f, 0x0f, 0x91, 0x17, 0x36, 0xd9, 0x33, 0x68, 0xa7, 0x31, 0xf4, 0xc4, 0x38, 0x89, 0x48,
0x93, 0x29, 0xcc, 0x21, 0xb4, 0x7e, 0x93, 0xe1, 0x65, 0x18, 0x07, 0x11, 0xa7, 0x9b, 0x8c, 0x7f,
0xc7, 0xcf, 0xea, 0xc6, 0x8b, 0x9b, 0x0c, 0x4b, 0xe7, 0x15, 0xfb, 0x07, 0x80, 0xd3, 0x90, 0xc2,
0x3f, 0x69, 0x93, 0x32, 0xcf, 0xca, 0x57, 0xbb, 0xb3, 0x7c, 0x87, 0xd0, 0xee, 0x45, 0x14, 0xc8,
0x22, 0x3f, 0xb3, 0x16, 0x2f, 0xe1, 0x6c, 0xb7, 0x70, 0xbb, 0x62, 0x97, 0xb0, 0x7f, 0x42, 0x4a,
0x4b, 0x31, 0xcd, 0x35, 0xb9, 0x49, 0x2f, 0xe3, 0x37, 0xd0, 0x9c, 0xfb, 0xbb, 0xb5, 0xb5, 0xfd,
0xba, 0x70, 0x62, 0x6f, 0x01, 0x57, 0x2e, 0xca, 0xda, 0x3e, 0x37, 0xcd, 0x2d, 0x6b, 0xda, 0x3e,
0xf7, 0x49, 0x95, 0x72, 0x2a, 0xa5, 0x90, 0xb9, 0x52, 0x8c, 0xc1, 0x4e, 0xaa, 0x92, 0x48, 0x27,
0xed, 0x4e, 0x9a, 0x78, 0xa4, 0xf3, 0x91, 0xb2, 0xef, 0x97, 0x43, 0xe0, 0xb9, 0x0f, 0xfb, 0x01,
0x0e, 0x38, 0x25, 0x51, 0x38, 0x34, 0x5d, 0xdb, 0x9b, 0x48, 0x25, 0xe4, 0x26, 0x73, 0x6d, 0x50,
0x79, 0x4e, 0xe1, 0x41, 0x36, 0x44, 0xd2, 0x13, 0xf6, 0xf9, 0xd6, 0x7c, 0x8c, 0x38, 0xaf, 0x85,
0xa6, 0xf7, 0xa1, 0xd2, 0x33, 0x09, 0x9f, 0x6f, 0xf1, 0x39, 0x72, 0xec, 0xc0, 0xf6, 0x2c, 0x1c,
0xf6, 0x05, 0xec, 0xf4, 0xc3, 0xf8, 0x32, 0x0d, 0xc0, 0x85, 0x9d, 0x57, 0xa4, 0x54, 0x70, 0x99,
0x77, 0x4d, 0x6e, 0xb2, 0x4f, 0x73, 0x27, 0x95, 0xf6, 0xd5, 0xe9, 0xf0, 0x4a, 0xe4, 0x7d, 0x95,
0xae, 0xd9, 0xdf, 0xf0, 0xd9, 0x79, 0x18, 0xeb, 0x57, 0x42, 0xe9, 0xb4, 0xe4, 0xb1, 0xee, 0x89,
0xf1, 0x58, 0xc4, 0xcf, 0xe3, 0x21, 0x29, 0xbd, 0x51, 0x72, 0xf8, 0x23, 0x3c, 0x48, 0xf5, 0x4b,
0x32, 0xab, 0xc5, 0x1d, 0x42, 0x5c, 0x76, 0x64, 0x9f, 0xdf, 0x77, 0xb9, 0x3a, 0xec, 0x42, 0x7d,
0x20, 0xc3, 0x74, 0x04, 0x9e, 0x88, 0x58, 0xf7, 0x02, 0x49, 0xed, 0x2d, 0x6c, 0x42, 0xe3, 0x2c,
0x88, 0x14, 0xb5, 0x2d, 0x74, 0xc0, 0x1e, 0xc8, 0x09, 0xb5, 0x6b, 0x47, 0xb7, 0xf5, 0x74, 0x40,
0xcd, 0x49, 0x46, 0x0f, 0xec, 0x34, 0x71, 0x74, 0xfc, 0x8c, 0x24, 0x2f, 0x5f, 0x29, 0xfc, 0x09,
0x1e, 0x2e, 0xbf, 0x33, 0x0a, 0xd1, 0x2f, 0x3d, 0xce, 0x5e, 0x19, 0x53, 0xd8, 0x87, 0x47, 0xd5,
0x4f, 0x14, 0x7a, 0xfe, 0xda, 0x87, 0xcf, 0x5b, 0xbf, 0xa7, 0xf0, 0x19, 0xb4, 0x57, 0xa5, 0x89,
0x07, 0x7e, 0x45, 0xcb, 0x79, 0x55, 0xa8, 0xc2, 0xe7, 0xb0, 0x57, 0x12, 0x17, 0x7e, 0xec, 0x57,
0x09, 0xd5, 0xab, 0x84, 0x15, 0x7e, 0x3f, 0x2b, 0xe1, 0x7c, 0x04, 0xe1, 0x9e, 0xbf, 0x3a, 0xd2,
0xbc, 0x12, 0xa4, 0xf0, 0x1d, 0x3c, 0xbe, 0xab, 0x7e, 0xd8, 0xf1, 0xef, 0xd1, 0x96, 0x77, 0x9f,
0x87, 0x3a, 0x6e, 0xbc, 0xad, 0x27, 0xa3, 0xc9, 0xbb, 0x6d, 0xf3, 0x1f, 0xea, 0xdb, 0x0f, 0x01,
0x00, 0x00, 0xff, 0xff, 0xad, 0x4e, 0x98, 0x29, 0x50, 0x09, 0x00, 0x00,
}

View File

@ -4,31 +4,29 @@ option go_package = "pdu";
service Replication {
rpc Ping(PingReq) returns (PingRes);
rpc ListFilesystems(ListFilesystemReq) returns (ListFilesystemRes);
rpc ListFilesystemVersions (ListFilesystemVersionsReq) returns (ListFilesystemVersionsRes);
rpc ListFilesystemVersions(ListFilesystemVersionsReq)
returns (ListFilesystemVersionsRes);
rpc DestroySnapshots(DestroySnapshotsReq) returns (DestroySnapshotsRes);
rpc ReplicationCursor(ReplicationCursorReq) returns (ReplicationCursorRes);
rpc SendCompleted(SendCompletedReq) returns (SendCompletedRes);
rpc HintMostRecentCommonAncestor(HintMostRecentCommonAncestorReq) returns (HintMostRecentCommonAncestorRes);
// for Send and Recv, see package rpc
}
message ListFilesystemReq {}
message ListFilesystemRes {
repeated Filesystem Filesystems = 1;
}
message ListFilesystemRes { repeated Filesystem Filesystems = 1; }
message Filesystem {
string Path = 1;
string ResumeToken = 2;
bool IsPlaceholder = 3;
bool IsEncrypted = 4;
}
message ListFilesystemVersionsReq {
string Filesystem = 1;
}
message ListFilesystemVersionsReq { string Filesystem = 1; }
message ListFilesystemVersionsRes {
repeated FilesystemVersion Versions = 1;
}
message ListFilesystemVersionsRes { repeated FilesystemVersion Versions = 1; }
message FilesystemVersion {
enum VersionType {
@ -42,25 +40,28 @@ message FilesystemVersion {
string Creation = 5; // RFC 3339
}
enum Tri {
DontCare = 0;
False = 1;
True = 2;
}
message SendReq {
string Filesystem = 1;
string From = 2;
// May be empty / null to request a full transfer of From
string To = 3;
// May be empty / null to request a full transfer of To
FilesystemVersion From = 2;
FilesystemVersion 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.
// If ResumeToken is not empty, the resume token that CAN be used for 'zfs
// send' by the sender. The sender MUST indicate use of ResumeToken in the
// reply message 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;
Tri Encrypted = 5;
bool DryRun = 7;
bool DryRun = 6;
}
message Property {
@ -70,6 +71,7 @@ message Property {
message SendRes {
// Whether the resume token provided in the request has been used or not.
// If the SendReq.ResumeToken == "", this field has no meaning.
bool UsedResumeToken = 2;
// Expected stream size determined by dry run, not exact.
@ -79,11 +81,19 @@ message SendRes {
repeated Property Properties = 4;
}
message ReceiveReq {
string Filesystem = 1; // FIXME should be snapshot name, we can enforce that on recv
message SendCompletedReq {
SendReq OriginalReq = 2;
}
// If true, the receiver should clear the resume token before perfoming the zfs recv of the stream in the request
bool ClearResumeToken = 2;
message SendCompletedRes {}
message ReceiveReq {
string Filesystem = 1;
FilesystemVersion To = 2;
// If true, the receiver should clear the resume token before perfoming the
// zfs recv of the stream in the request
bool ClearResumeToken = 3;
}
message ReceiveRes {}
@ -99,21 +109,9 @@ message DestroySnapshotRes {
string Error = 2;
}
message DestroySnapshotsRes {
repeated DestroySnapshotRes Results = 1;
}
message DestroySnapshotsRes { repeated DestroySnapshotRes Results = 1; }
message ReplicationCursorReq {
string Filesystem = 1;
message GetOp {}
message SetOp {
string Snapshot = 2;
}
oneof op {
GetOp get = 2;
SetOp set = 3;
}
}
message ReplicationCursorReq { string Filesystem = 1; }
message ReplicationCursorRes {
oneof Result {
@ -122,11 +120,15 @@ message ReplicationCursorRes {
}
}
message PingReq {
string Message = 1;
}
message PingReq { string Message = 1; }
message PingRes {
// Echo must be PingReq.Message
string Echo = 1;
}
message HintMostRecentCommonAncestorReq {
string Filesystem = 1;
FilesystemVersion SenderVersion = 2;
}
message HintMostRecentCommonAncestorRes {}

View File

@ -7,6 +7,14 @@ import (
"github.com/zrepl/zrepl/zfs"
)
func (v *FilesystemVersion) GetRelName() string {
zv, err := v.ZFSFilesystemVersion()
if err != nil {
return ""
}
return zv.String()
}
func (v *FilesystemVersion) RelName() string {
zv, err := v.ZFSFilesystemVersion()
if err != nil {

View File

@ -30,6 +30,7 @@ type Endpoint interface {
ListFilesystemVersions(ctx context.Context, req *pdu.ListFilesystemVersionsReq) (*pdu.ListFilesystemVersionsRes, error)
DestroySnapshots(ctx context.Context, req *pdu.DestroySnapshotsReq) (*pdu.DestroySnapshotsRes, error)
WaitForConnectivity(ctx context.Context) error
HintMostRecentCommonAncestor(context.Context, *pdu.HintMostRecentCommonAncestorReq) (*pdu.HintMostRecentCommonAncestorRes, error)
}
type Sender interface {
@ -38,6 +39,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, zfs.StreamCopier, error)
SendCompleted(ctx context.Context, r *pdu.SendCompletedReq) (*pdu.SendCompletedRes, error)
ReplicationCursor(ctx context.Context, req *pdu.ReplicationCursorReq) (*pdu.ReplicationCursorRes, error)
}
@ -48,9 +50,14 @@ type Receiver interface {
Receive(ctx context.Context, req *pdu.ReceiveReq, receive zfs.StreamCopier) (*pdu.ReceiveRes, error)
}
type PlannerPolicy struct {
EncryptedSend tri // all sends must be encrypted (send -w, and encryption!=off)
}
type Planner struct {
sender Sender
receiver Receiver
policy PlannerPolicy
promSecsPerState *prometheus.HistogramVec // labels: state
promBytesReplicated *prometheus.CounterVec // labels: filesystem
@ -109,9 +116,10 @@ func (p *Planner) WaitForConnectivity(ctx context.Context) error {
type Filesystem struct {
sender Sender
receiver Receiver
policy PlannerPolicy
Path string // compat
receiverFS *pdu.Filesystem
receiverFS, senderFS *pdu.Filesystem // receiverFS may be nil, senderFS never nil
promBytesReplicated prometheus.Counter // compat
sizeEstimateRequestSem *semaphore.S
@ -146,7 +154,9 @@ type Step struct {
receiver Receiver
parent *Filesystem
from, to *pdu.FilesystemVersion // compat
from, to *pdu.FilesystemVersion // from may be nil, indicating full send
encrypt tri
resumeToken string // empty means no resume token shall be used
expectedSize int64 // 0 means no size estimate present / possible
@ -186,23 +196,36 @@ func (s *Step) ReportInfo() *report.StepInfo {
}
s.byteCounterMtx.Unlock()
// FIXME stick to zfs convention of from and to
from := ""
if s.from != nil {
from = s.from.RelName()
}
var encrypted report.EncryptedEnum
switch s.encrypt {
case DontCare:
encrypted = report.EncryptedSenderDependent
case True:
encrypted = report.EncryptedTrue
case False:
encrypted = report.EncryptedFalse
default:
panic(fmt.Sprintf("unknown variant %s", s.encrypt))
}
return &report.StepInfo{
From: from,
To: s.to.RelName(),
Resumed: s.resumeToken != "",
Encrypted: encrypted,
BytesExpected: s.expectedSize,
BytesReplicated: byteCounter,
}
}
func NewPlanner(secsPerState *prometheus.HistogramVec, bytesReplicated *prometheus.CounterVec, sender Sender, receiver Receiver) *Planner {
func NewPlanner(secsPerState *prometheus.HistogramVec, bytesReplicated *prometheus.CounterVec, sender Sender, receiver Receiver, policy PlannerPolicy) *Planner {
return &Planner{
sender: sender,
receiver: receiver,
policy: policy,
promSecsPerState: secsPerState,
promBytesReplicated: bytesReplicated,
}
@ -265,7 +288,9 @@ func (p *Planner) doPlanning(ctx context.Context) ([]*Filesystem, error) {
q = append(q, &Filesystem{
sender: p.sender,
receiver: p.receiver,
policy: p.policy,
Path: fs.Path,
senderFS: fs,
receiverFS: receiverFS,
promBytesReplicated: ctr,
sizeEstimateRequestSem: sizeEstimateRequestSem,
@ -281,6 +306,10 @@ func (fs *Filesystem) doPlanning(ctx context.Context) ([]*Step, error) {
log.Debug("assessing filesystem")
if fs.policy.EncryptedSend == True && !fs.senderFS.GetIsEncrypted() {
return nil, fmt.Errorf("sender filesystem is not encrypted but policy mandates encrypted send")
}
sfsvsres, err := fs.sender.ListFilesystemVersions(ctx, &pdu.ListFilesystemVersionsReq{Filesystem: fs.Path})
if err != nil {
log.WithError(err).Error("cannot get remote filesystem versions")
@ -306,6 +335,150 @@ func (fs *Filesystem) doPlanning(ctx context.Context) ([]*Step, error) {
rfsvs = []*pdu.FilesystemVersion{}
}
var resumeToken *zfs.ResumeToken
var resumeTokenRaw string
if fs.receiverFS != nil && fs.receiverFS.ResumeToken != "" {
resumeTokenRaw = fs.receiverFS.ResumeToken // shadow
log.WithField("receiverFS.ResumeToken", resumeTokenRaw).Debug("decode receiver fs resume token")
resumeToken, err = zfs.ParseResumeToken(ctx, resumeTokenRaw) // shadow
if err != nil {
// TODO in theory, we could do replication without resume token, but that would mean that
// we need to discard the resumable state on the receiver's side.
// Would be easy by setting UsedResumeToken=false in the RecvReq ...
// FIXME / CHECK semantics UsedResumeToken if SendReq.ResumeToken == ""
log.WithError(err).Error("cannot decode resume token, aborting")
return nil, err
}
log.WithField("token", resumeToken).Debug("decode resume token")
}
// give both sides a hint about how far the replication got
// This serves as a cummulative variant of SendCompleted and can be useful
// for example to release stale holds from an earlier (interrupted) replication.
// TODO FIXME: enqueue this as a replication step instead of doing it here during planning
// then again, the step should run regardless of planning success
// so maybe a separate phase before PLANNING, then?
path, conflict := IncrementalPath(rfsvs, sfsvs)
var sender_mrca *pdu.FilesystemVersion // from sfsvs
if conflict == nil && len(path) > 0 {
sender_mrca = path[0] // shadow
}
if sender_mrca != nil {
var wg sync.WaitGroup
doHint := func(ep Endpoint, name string) {
defer wg.Done()
log := log.WithField("to_side", name).
WithField("sender_mrca", sender_mrca.String())
log.Debug("hint most recent common ancestor")
hint := &pdu.HintMostRecentCommonAncestorReq{
Filesystem: fs.Path,
SenderVersion: sender_mrca,
}
_, err := ep.HintMostRecentCommonAncestor(ctx, hint)
if err != nil {
log.WithError(err).Error("error hinting most recent common ancestor")
}
}
wg.Add(2)
go doHint(fs.sender, "sender")
go doHint(fs.receiver, "receiver")
wg.Wait()
} else {
log.Debug("cannot identify most recent common ancestor, skipping hint")
}
var steps []*Step
// build the list of replication steps
//
// prefer to resume any started replication instead of starting over with a normal IncrementalPath
//
// look for the step encoded in the resume token in the sender's version
// if we find that step:
// 1. use it as first step (including resume token)
// 2. compute subsequent steps by computing incremental path from the token.To version on
// ...
// that's actually equivalent to simply cutting off earlier versions from rfsvs and sfsvs
if resumeToken != nil {
sfsvs := SortVersionListByCreateTXGThenBookmarkLTSnapshot(sfsvs)
var fromVersion, toVersion *pdu.FilesystemVersion
var toVersionIdx int
for idx, sfsv := range sfsvs {
if resumeToken.HasFromGUID && sfsv.Guid == resumeToken.FromGUID {
if fromVersion != nil && fromVersion.Type == pdu.FilesystemVersion_Snapshot {
// prefer snapshots over bookmarks for size estimation
} else {
fromVersion = sfsv
}
}
if resumeToken.HasToGUID && sfsv.Guid == resumeToken.ToGUID && sfsv.Type == pdu.FilesystemVersion_Snapshot {
// `toversion` must always be a snapshot
toVersion, toVersionIdx = sfsv, idx
}
}
encryptionMatches := false
switch fs.policy.EncryptedSend {
case True:
encryptionMatches = resumeToken.RawOK && resumeToken.CompressOK
case False:
encryptionMatches = !resumeToken.RawOK && !resumeToken.CompressOK
case DontCare:
encryptionMatches = true
}
log.WithField("fromVersion", fromVersion).
WithField("toVersion", toVersion).
WithField("encryptionMatches", encryptionMatches).
Debug("result of resume-token-matching to sender's versions")
if !encryptionMatches {
return nil, fmt.Errorf("resume token `rawok`=%v and `compressok`=%v are incompatible with encryption policy=%v", resumeToken.RawOK, resumeToken.CompressOK, fs.policy.EncryptedSend)
} else if toVersion == nil {
return nil, fmt.Errorf("resume token `toguid` = %v not found on sender (`toname` = %q)", resumeToken.ToGUID, resumeToken.ToName)
} else if fromVersion == toVersion {
return nil, fmt.Errorf("resume token `fromguid` and `toguid` match same version on sener")
}
// fromVersion may be nil, toVersion is no nil, encryption matches
// good to go this one step!
resumeStep := &Step{
parent: fs,
sender: fs.sender,
receiver: fs.receiver,
from: fromVersion,
to: toVersion,
encrypt: fs.policy.EncryptedSend,
resumeToken: resumeTokenRaw,
}
// by definition, the resume token _must_ be the receiver's most recent version, if they have any
// don't bother checking, zfs recv will produce an error if above assumption is wrong
//
// thus, subsequent steps are just incrementals on the sender's remaining _snapshots_ (not bookmarks)
var remainingSFSVs []*pdu.FilesystemVersion
for _, sfsv := range sfsvs[toVersionIdx:] {
if sfsv.Type == pdu.FilesystemVersion_Snapshot {
remainingSFSVs = append(remainingSFSVs, sfsv)
}
}
steps = make([]*Step, 0, len(remainingSFSVs)) // shadow
steps = append(steps, resumeStep)
for i := 0; i < len(remainingSFSVs)-1; i++ {
steps = append(steps, &Step{
parent: fs,
sender: fs.sender,
receiver: fs.receiver,
from: remainingSFSVs[i],
to: remainingSFSVs[i+1],
encrypt: fs.policy.EncryptedSend,
})
}
} else { // resumeToken == nil
path, conflict := IncrementalPath(rfsvs, sfsvs)
if conflict != nil {
var msg string
@ -322,15 +495,16 @@ func (fs *Filesystem) doPlanning(ctx context.Context) ([]*Step, error) {
return nil, conflict
}
steps := make([]*Step, 0, len(path))
// FIXME unify struct declarations => initializer?
steps = make([]*Step, 0, len(path)) // shadow
if len(path) == 1 {
steps = append(steps, &Step{
parent: fs,
sender: fs.sender,
receiver: fs.receiver,
from: nil,
to: path[0],
encrypt: fs.policy.EncryptedSend,
})
} else {
for i := 0; i < len(path)-1; i++ {
@ -338,11 +512,18 @@ func (fs *Filesystem) doPlanning(ctx context.Context) ([]*Step, error) {
parent: fs,
sender: fs.sender,
receiver: fs.receiver,
from: path[i],
to: path[i+1],
encrypt: fs.policy.EncryptedSend,
})
}
}
}
if len(steps) == 0 {
log.Info("planning determined that no replication steps are required")
}
log.Debug("compute send size estimate")
errs := make(chan error, len(steps))
@ -389,29 +570,6 @@ func (fs *Filesystem) doPlanning(ctx context.Context) ([]*Step, error) {
return steps, nil
}
// type FilesystemsReplicationFailedError struct {
// FilesystemsWithError []*fsrep.Replication
// }
// func (e FilesystemsReplicationFailedError) Error() string {
// allSame := true
// lastErr := e.FilesystemsWithError[0].Err().Error()
// for _, fs := range e.FilesystemsWithError {
// fsErr := fs.Err().Error()
// allSame = allSame && lastErr == fsErr
// }
// fsstr := "multiple filesystems"
// if len(e.FilesystemsWithError) == 1 {
// fsstr = fmt.Sprintf("filesystem %s", e.FilesystemsWithError[0].FS())
// }
// errorStr := lastErr
// if !allSame {
// errorStr = "multiple different errors"
// }
// return fmt.Sprintf("%s could not be replicated: %s", fsstr, errorStr)
// }
func (s *Step) updateSizeEstimate(ctx context.Context) error {
log := getLogger(ctx)
@ -430,20 +588,14 @@ func (s *Step) updateSizeEstimate(ctx context.Context) error {
func (s *Step) buildSendRequest(dryRun bool) (sr *pdu.SendReq) {
fs := s.parent.Path
if s.from == nil {
sr = &pdu.SendReq{
Filesystem: fs,
To: s.to.RelName(),
From: s.from, // may be nil
To: s.to,
Encrypted: s.encrypt.ToPDU(),
ResumeToken: s.resumeToken,
DryRun: dryRun,
}
} else {
sr = &pdu.SendReq{
Filesystem: fs,
From: s.from.RelName(),
To: s.to.RelName(),
DryRun: dryRun,
}
}
return sr
}
@ -478,6 +630,7 @@ func (s *Step) doReplication(ctx context.Context) error {
rr := &pdu.ReceiveReq{
Filesystem: fs,
To: sr.GetTo(),
ClearResumeToken: !sres.UsedResumeToken,
}
log.Debug("initiate receive request")
@ -486,6 +639,7 @@ func (s *Step) doReplication(ctx context.Context) error {
log.
WithError(err).
WithField("errType", fmt.Sprintf("%T", err)).
WithField("rr", fmt.Sprintf("%v", rr)).
Error("receive request failed (might also be error on sender)")
// This failure could be due to
// - an unexpected exit of ZFS on the sending side
@ -495,22 +649,12 @@ func (s *Step) doReplication(ctx context.Context) error {
}
log.Debug("receive finished")
log.Debug("advance replication cursor")
req := &pdu.ReplicationCursorReq{
Filesystem: fs,
Op: &pdu.ReplicationCursorReq_Set{
Set: &pdu.ReplicationCursorReq_SetOp{
Snapshot: s.to.GetName(),
},
},
}
_, err = s.sender.ReplicationCursor(ctx, req)
log.Debug("tell sender replication completed")
_, err = s.sender.SendCompleted(ctx, &pdu.SendCompletedReq{
OriginalReq: sr,
})
if err != nil {
log.WithError(err).Error("error advancing replication cursor")
// If this fails and replication planning restarts, the diff algorithm will find
// that cursor out of place. This is not a problem because then, it would just use another FS
// However, we FIXME have no means to just update the cursor in a
// second replication attempt right after this one where we don't have new snaps yet
log.WithError(err).Error("error telling sender that replication completed successfully")
return err
}

View File

@ -0,0 +1,46 @@
package logic
import (
"fmt"
"github.com/zrepl/zrepl/replication/logic/pdu"
)
type tri int
const (
DontCare = 0x0
False = 0x1
True = 0x2
)
func (t tri) String() string {
switch t {
case DontCare:
return "dontcare"
case False:
return "false"
case True:
return "true"
}
panic(fmt.Sprintf("unknown variant %v", int(t)))
}
func (t tri) ToPDU() pdu.Tri {
switch t {
case DontCare:
return pdu.Tri_DontCare
case False:
return pdu.Tri_False
case True:
return pdu.Tri_True
}
panic(fmt.Sprintf("unknown variant %v", int(t)))
}
func TriFromBool(b bool) tri {
if b {
return True
}
return False
}

View File

@ -85,8 +85,18 @@ type StepReport struct {
Info *StepInfo
}
type EncryptedEnum string
const (
EncryptedTrue EncryptedEnum = "yes"
EncryptedFalse EncryptedEnum = "no"
EncryptedSenderDependent EncryptedEnum = "sender-dependent"
)
type StepInfo struct {
From, To string
Resumed bool
Encrypted EncryptedEnum
BytesExpected int64
BytesReplicated int64
}
@ -150,5 +160,5 @@ func (f *FilesystemReport) NextStep() *StepReport {
}
func (f *StepReport) IsIncremental() bool {
return f.Info.From != "" // FIXME change to ZFS semantics (To != "")
return f.Info.From != ""
}

View File

@ -116,6 +116,14 @@ func (c *Client) ReplicationCursor(ctx context.Context, in *pdu.ReplicationCurso
return c.controlClient.ReplicationCursor(ctx, in)
}
func (c *Client) SendCompleted(ctx context.Context, in *pdu.SendCompletedReq) (*pdu.SendCompletedRes, error) {
return c.controlClient.SendCompleted(ctx, in)
}
func (c *Client) HintMostRecentCommonAncestor(ctx context.Context, in *pdu.HintMostRecentCommonAncestorReq) (*pdu.HintMostRecentCommonAncestorRes, error) {
return c.controlClient.HintMostRecentCommonAncestor(ctx, in)
}
func (c *Client) WaitForConnectivity(ctx context.Context) error {
ctx, cancel := context.WithCancel(ctx)
defer cancel()

View File

@ -152,7 +152,7 @@ func (m *HandshakeMessage) DecodeReader(r io.Reader, maxLen int) error {
func DoHandshakeCurrentVersion(conn net.Conn, deadline time.Time) *HandshakeError {
// current protocol version is hardcoded here
return DoHandshakeVersion(conn, deadline, 1)
return DoHandshakeVersion(conn, deadline, 2)
}
const HandshakeMessageMaxLen = 16 * 4096

36
util/limitio/limitio.go Normal file
View File

@ -0,0 +1,36 @@
package limitio
import "io"
type readCloser struct {
read int64
limit int64
r io.ReadCloser
}
func ReadCloser(rc io.ReadCloser, limit int64) io.ReadCloser {
return &readCloser{0, limit, rc}
}
var _ io.ReadCloser = (*readCloser)(nil)
func (r *readCloser) Read(b []byte) (int, error) {
if len(b) == 0 {
return 0, nil
}
if r.read == r.limit {
return 0, io.EOF
}
if r.read+int64(len(b)) >= r.limit {
b = b[:int(r.limit-r.read)]
}
readN, err := r.r.Read(b)
r.read += int64(readN)
return readN, err
}
func (r *readCloser) Close() error { return r.r.Close() }

View File

@ -0,0 +1,44 @@
package limitio
import (
"bytes"
"io"
"testing"
"github.com/stretchr/testify/assert"
)
type mockRC struct {
r io.Reader
closed bool
}
func newMockRC(r io.Reader) *mockRC { return &mockRC{r, false} }
func (m mockRC) Read(b []byte) (int, error) {
if m.closed {
return 0, io.ErrClosedPipe
}
return m.r.Read(b)
}
func (m *mockRC) Close() error {
m.closed = true
return nil
}
func TestReadCloser(t *testing.T) {
foobarReader := bytes.NewReader([]byte("foobar2342"))
mock := newMockRC(foobarReader)
limited := ReadCloser(mock, 6)
var buf [20]byte
n, err := limited.Read(buf[:])
assert.NoError(t, err)
assert.Equal(t, 6, n)
assert.Equal(t, buf[:n], []byte("foobar"))
n, err = limited.Read(buf[:])
assert.Equal(t, 0, n)
assert.Equal(t, io.EOF, err)
limited.Close()
assert.True(t, mock.closed)
}

71
zfs/encryption.go Normal file
View File

@ -0,0 +1,71 @@
package zfs
import (
"context"
"fmt"
"os/exec"
"strings"
"sync"
"github.com/pkg/errors"
"github.com/zrepl/zrepl/util/envconst"
)
var encryptionCLISupport struct {
once sync.Once
supported bool
err error
}
func EncryptionCLISupported(ctx context.Context) (bool, error) {
encryptionCLISupport.once.Do(func() {
// "feature discovery"
cmd := exec.Command("zfs", "load-key")
output, err := cmd.CombinedOutput()
if ee, ok := err.(*exec.ExitError); !ok || ok && !ee.Exited() {
encryptionCLISupport.err = errors.Wrap(err, "native encryption cli support feature check failed")
}
def := strings.Contains(string(output), "load-key") && strings.Contains(string(output), "keylocation")
encryptionCLISupport.supported = envconst.Bool("ZREPL_EXPERIMENTAL_ZFS_ENCRYPTION_CLI_SUPPORTED", def)
debug("encryption cli feature check complete %#v", &encryptionCLISupport)
})
return encryptionCLISupport.supported, encryptionCLISupport.err
}
// returns false, nil if encryption is not supported
func ZFSGetEncryptionEnabled(ctx context.Context, fs string) (enabled bool, err error) {
defer func(e *error) {
if *e != nil {
*e = fmt.Errorf("zfs get encryption enabled fs=%q: %s", fs, *e)
}
}(&err)
if supp, err := EncryptionCLISupported(ctx); err != nil {
return false, err
} else if !supp {
return false, nil
}
if err := validateZFSFilesystem(fs); err != nil {
return false, err
}
props, err := zfsGet(fs, []string{"encryption"}, sourceAny)
if err != nil {
return false, errors.Wrap(err, "cannot get `encryption` property")
}
val := props.Get("encryption")
switch val {
case "":
panic("zfs get should return a value for `encryption`")
case "-":
return false, errors.New("`encryption` property should never be \"-\"")
case "off":
return false, nil
default:
// we don't want to hardcode the cipher list, and we checked for != 'off'
// ==> assume any other value means encryption is enabled
// TODO add test to OpenZFS test suite
return true, nil
}
}

307
zfs/holds.go Normal file
View File

@ -0,0 +1,307 @@
package zfs
import (
"bufio"
"bytes"
"context"
"fmt"
"os"
"os/exec"
"sort"
"strconv"
"strings"
"syscall"
"github.com/pkg/errors"
"github.com/zrepl/zrepl/util/envconst"
)
// no need for feature tests, holds have been around forever
func validateNotEmpty(field, s string) error {
if s == "" {
return fmt.Errorf("`%s` must not be empty", field)
}
return nil
}
// returned err != nil is guaranteed to represent invalid hold tag
func ValidHoldTag(tag string) error {
maxlen := envconst.Int("ZREPL_ZFS_MAX_HOLD_TAG_LEN", 256-1) // 256 include NULL byte, from module/zfs/dsl_userhold.c
if len(tag) > maxlen {
return fmt.Errorf("hold tag %q exceeds max length of %d", tag, maxlen)
}
return nil
}
// Idemptotent: does not return an error if the tag already exists
func ZFSHold(ctx context.Context, fs string, v ZFSSendArgVersion, tag string) error {
if err := v.ValidateInMemory(fs); err != nil {
return errors.Wrap(err, "invalid version")
}
if !v.IsSnapshot() {
return errors.Errorf("can only hold snapshots, got %s", v.RelName)
}
if err := validateNotEmpty("tag", tag); err != nil {
return err
}
fullPath := v.FullPath(fs)
output, err := exec.CommandContext(ctx, "zfs", "hold", tag, fullPath).CombinedOutput()
if err != nil {
if bytes.Contains(output, []byte("tag already exists on this dataset")) {
goto success
}
return &ZFSError{output, errors.Wrapf(err, "cannot hold %q", fullPath)}
}
success:
return nil
}
func ZFSHolds(ctx context.Context, fs, snap string) ([]string, error) {
if err := validateZFSFilesystem(fs); err != nil {
return nil, errors.Wrap(err, "`fs` is not a valid filesystem path")
}
if snap == "" {
return nil, fmt.Errorf("`snap` must not be empty")
}
dp := fmt.Sprintf("%s@%s", fs, snap)
output, err := exec.CommandContext(ctx, "zfs", "holds", "-H", dp).CombinedOutput()
if err != nil {
return nil, &ZFSError{output, errors.Wrap(err, "zfs holds failed")}
}
scan := bufio.NewScanner(bytes.NewReader(output))
var tags []string
for scan.Scan() {
// NAME TAG TIMESTAMP
comps := strings.SplitN(scan.Text(), "\t", 3)
if len(comps) != 3 {
return nil, fmt.Errorf("zfs holds: unexpected output\n%s", output)
}
if comps[0] != dp {
return nil, fmt.Errorf("zfs holds: unexpected output: expecting %q as first component, got %q\n%s", dp, comps[0], output)
}
tags = append(tags, comps[1])
}
return tags, nil
}
// Idempotent: if the hold doesn't exist, this is not an error
func ZFSRelease(ctx context.Context, tag string, snaps ...string) error {
cumLens := make([]int, len(snaps))
for i := 1; i < len(snaps); i++ {
cumLens[i] = cumLens[i-1] + len(snaps[i])
}
maxInvocationLen := 12 * os.Getpagesize()
var noSuchTagLines, otherLines []string
for i := 0; i < len(snaps); {
var j = i
for ; j < len(snaps); j++ {
if cumLens[j]-cumLens[i] > maxInvocationLen {
break
}
}
args := []string{"release", tag}
args = append(args, snaps[i:j]...)
output, err := exec.CommandContext(ctx, "zfs", args...).CombinedOutput()
if pe, ok := err.(*os.PathError); err != nil && ok && pe.Err == syscall.E2BIG {
maxInvocationLen = maxInvocationLen / 2
continue
}
maxInvocationLen = maxInvocationLen + os.Getpagesize()
i = j
// even if release fails for datasets where there's no hold with the tag
// the hold is still released on datasets which have a hold with the tag
// FIXME verify this in a platformtest
// => screen-scrape
scan := bufio.NewScanner(bytes.NewReader(output))
for scan.Scan() {
line := scan.Text()
if strings.Contains(line, "no such tag on this dataset") {
noSuchTagLines = append(noSuchTagLines, line)
} else {
otherLines = append(otherLines, line)
}
}
}
if debugEnabled {
debug("zfs release: no such tag lines=%v otherLines=%v", noSuchTagLines, otherLines)
}
if len(otherLines) > 0 {
return fmt.Errorf("unknown zfs error while releasing hold with tag %q: unidentified stderr lines\n%s", tag, strings.Join(otherLines, "\n"))
}
return nil
}
// Idempotent: if the hold doesn't exist, this is not an error
func ZFSReleaseAllOlderAndIncludingGUID(ctx context.Context, fs string, snapOrBookmarkGuid uint64, tag string) error {
return doZFSReleaseAllOlderAndIncOrExcludingGUID(ctx, fs, snapOrBookmarkGuid, tag, true)
}
// Idempotent: if the hold doesn't exist, this is not an error
func ZFSReleaseAllOlderThanGUID(ctx context.Context, fs string, snapOrBookmarkGuid uint64, tag string) error {
return doZFSReleaseAllOlderAndIncOrExcludingGUID(ctx, fs, snapOrBookmarkGuid, tag, false)
}
type zfsReleaseAllOlderAndIncOrExcludingGUIDZFSListLine struct {
entityType EntityType
name string
createtxg uint64
guid uint64
userrefs uint64 // always 0 for bookmarks
}
func doZFSReleaseAllOlderAndIncOrExcludingGUID(ctx context.Context, fs string, snapOrBookmarkGuid uint64, tag string, includeGuid bool) error {
// TODO channel program support still unreleased but
// might be a huge performance improvement
// https://github.com/zfsonlinux/zfs/pull/7902/files
if err := validateZFSFilesystem(fs); err != nil {
return errors.Wrap(err, "`fs` is not a valid filesystem path")
}
if tag == "" {
return fmt.Errorf("`tag` must not be empty`")
}
output, err := exec.CommandContext(ctx,
"zfs", "list", "-o", "type,name,createtxg,guid,userrefs",
"-H", "-t", "snapshot,bookmark", "-r", "-d", "1", fs).CombinedOutput()
if err != nil {
return &ZFSError{output, errors.Wrap(err, "cannot list snapshots and their userrefs")}
}
lines, err := doZFSReleaseAllOlderAndIncOrExcludingGUIDParseListOutput(output)
if err != nil {
return errors.Wrap(err, "unexpected ZFS output")
}
releaseSnaps, err := doZFSReleaseAllOlderAndIncOrExcludingGUIDFindSnapshots(snapOrBookmarkGuid, includeGuid, lines)
if err != nil {
return err
}
if len(releaseSnaps) == 0 {
return nil
}
return ZFSRelease(ctx, tag, releaseSnaps...)
}
func doZFSReleaseAllOlderAndIncOrExcludingGUIDParseListOutput(output []byte) ([]zfsReleaseAllOlderAndIncOrExcludingGUIDZFSListLine, error) {
scan := bufio.NewScanner(bytes.NewReader(output))
var lines []zfsReleaseAllOlderAndIncOrExcludingGUIDZFSListLine
for scan.Scan() {
const numCols = 5
comps := strings.SplitN(scan.Text(), "\t", numCols)
if len(comps) != numCols {
return nil, fmt.Errorf("not %d columns\n%s", numCols, output)
}
dstype := comps[0]
name := comps[1]
var entityType EntityType
switch dstype {
case "snapshot":
entityType = EntityTypeSnapshot
case "bookmark":
entityType = EntityTypeBookmark
default:
return nil, fmt.Errorf("column 0 is %q, expecting \"snapshot\" or \"bookmark\"", dstype)
}
createtxg, err := strconv.ParseUint(comps[2], 10, 64)
if err != nil {
return nil, fmt.Errorf("cannot parse createtxg %q: %s\n%s", comps[2], err, output)
}
guid, err := strconv.ParseUint(comps[3], 10, 64)
if err != nil {
return nil, fmt.Errorf("cannot parse guid %q: %s\n%s", comps[3], err, output)
}
var userrefs uint64
switch entityType {
case EntityTypeBookmark:
if comps[4] != "-" {
return nil, fmt.Errorf("entity type \"bookmark\" should have userrefs=\"-\", got %q", comps[4])
}
userrefs = 0
case EntityTypeSnapshot:
userrefs, err = strconv.ParseUint(comps[4], 10, 64) // shadow
if err != nil {
return nil, fmt.Errorf("cannot parse userrefs %q: %s\n%s", comps[4], err, output)
}
default:
panic(entityType)
}
lines = append(lines, zfsReleaseAllOlderAndIncOrExcludingGUIDZFSListLine{
entityType: entityType,
name: name,
createtxg: createtxg,
guid: guid,
userrefs: userrefs,
})
}
return lines, nil
}
func doZFSReleaseAllOlderAndIncOrExcludingGUIDFindSnapshots(snapOrBookmarkGuid uint64, includeGuid bool, lines []zfsReleaseAllOlderAndIncOrExcludingGUIDZFSListLine) (releaseSnaps []string, err error) {
// sort lines by createtxg,(snap < bookmark)
// we cannot do this using zfs list -s because `type` is not a
sort.Slice(lines, func(i, j int) (less bool) {
if lines[i].createtxg == lines[j].createtxg {
iET := func(t EntityType) int {
switch t {
case EntityTypeSnapshot:
return 0
case EntityTypeBookmark:
return 1
default:
panic("unepxected entity type " + t.String())
}
}
return iET(lines[i].entityType) < iET(lines[j].entityType)
}
return lines[i].createtxg < lines[j].createtxg
})
// iterate over snapshots oldest to newest and collect snapshots that have holds and
// are older than (inclusive or exclusive, depends on includeGuid) a snapshot or bookmark
// with snapOrBookmarkGuid
foundGuid := false
for _, line := range lines {
if line.guid == snapOrBookmarkGuid {
foundGuid = true
}
if line.userrefs > 0 {
if !foundGuid || (foundGuid && includeGuid) {
// only snapshots have userrefs > 0, no need to check entityType
releaseSnaps = append(releaseSnaps, line.name)
}
}
if foundGuid {
// The secondary key in sorting (snap < bookmark) guarantees that we
// A) either found the snapshot with snapOrBoomkarkGuid
// B) or no snapshot with snapGuid exists, but one or more bookmarks of it exists
// In the case of A, we already added the snapshot to releaseSnaps if includeGuid requests it,
// and can ignore possible subsequent bookmarks of the snapshot.
// In the case of B, there is nothing to add to releaseSnaps.
break
}
}
if !foundGuid {
return nil, fmt.Errorf("cannot find snapshot or bookmark with guid %v", snapOrBookmarkGuid)
}
return releaseSnaps, nil
}

38
zfs/holds_test.go Normal file
View File

@ -0,0 +1,38 @@
package zfs
import (
"testing"
"github.com/kr/pretty"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
func TestDoZFSReleaseAllOlderAndIncOrExcludingGUIDFindSnapshots(t *testing.T) {
// what we test here: sort bookmark #3 before @3
// => assert that the function doesn't stop at the first guid match
// (which might be a bookmark, depending on zfs list ordering)
// but instead considers the entire stride of boomarks and snapshots with that guid
//
// also, throw in unordered createtxg for good measure
list, err := doZFSReleaseAllOlderAndIncOrExcludingGUIDParseListOutput(
[]byte("snapshot\tfoo@1\t1\t1013001\t1\n" +
"snapshot\tfoo@2\t2\t2013002\t1\n" +
"bookmark\tfoo#3\t3\t7013003\t-\n" +
"snapshot\tfoo@6\t6\t5013006\t1\n" +
"snapshot\tfoo@3\t3\t7013003\t1\n" +
"snapshot\tfoo@4\t3\t6013004\t1\n" +
""),
)
require.NoError(t, err)
t.Log(pretty.Sprint(list))
require.Equal(t, 6, len(list))
require.Equal(t, EntityTypeBookmark, list[2].entityType)
releaseSnaps, err := doZFSReleaseAllOlderAndIncOrExcludingGUIDFindSnapshots(7013003, true, list)
t.Logf("releasedSnaps = %#v", releaseSnaps)
assert.NoError(t, err)
assert.Equal(t, []string{"foo@1", "foo@2", "foo@3"}, releaseSnaps)
}

193
zfs/namecheck.go Normal file
View File

@ -0,0 +1,193 @@
package zfs
import (
"bytes"
"fmt"
"regexp"
"strings"
"unicode"
)
const MaxDatasetNameLen = 256 - 1
type EntityType string
const (
EntityTypeFilesystem EntityType = "filesystem"
EntityTypeVolume EntityType = "volume"
EntityTypeSnapshot EntityType = "snapshot"
EntityTypeBookmark EntityType = "bookmark"
)
func (e EntityType) Validate() error {
switch e {
case EntityTypeFilesystem:
return nil
case EntityTypeVolume:
return nil
case EntityTypeSnapshot:
return nil
case EntityTypeBookmark:
return nil
default:
return fmt.Errorf("invalid entity type %q", string(e))
}
}
func (e EntityType) MustValidate() {
if err := e.Validate(); err != nil {
panic(err)
}
}
func (e EntityType) String() string {
e.MustValidate()
return string(e)
}
var componentValidChar = regexp.MustCompile(`^[0-9a-zA-Z-_\.: ]+$`)
// From module/zcommon/zfs_namecheck.c
//
// Snapshot names must be made up of alphanumeric characters plus the following
// characters:
//
// [-_.: ]
//
func ComponentNamecheck(datasetPathComponent string) error {
if len(datasetPathComponent) == 0 {
return fmt.Errorf("path component must not be empty")
}
if len(datasetPathComponent) > MaxDatasetNameLen {
return fmt.Errorf("path component must not be longer than %d chars", MaxDatasetNameLen)
}
if !(isASCII(datasetPathComponent)) {
return fmt.Errorf("path component must be ASCII")
}
if !componentValidChar.MatchString(datasetPathComponent) {
return fmt.Errorf("path component must only contain alphanumeric chars and any in %q", "-_.: ")
}
if datasetPathComponent == "." || datasetPathComponent == ".." {
return fmt.Errorf("path component must not be '%s'", datasetPathComponent)
}
return nil
}
type PathValidationError struct {
path string
entityType EntityType
msg string
}
func (e *PathValidationError) Path() string { return e.path }
func (e *PathValidationError) Error() string {
return fmt.Sprintf("invalid %s %q: %s", e.entityType, e.path, e.msg)
}
// combines
//
// lib/libzfs/libzfs_dataset.c: zfs_validate_name
// module/zcommon/zfs_namecheck.c: entity_namecheck
//
// The '%' character is not allowed because it's reserved for zfs-internal use
func EntityNamecheck(path string, t EntityType) (err *PathValidationError) {
pve := func(msg string) *PathValidationError {
return &PathValidationError{path: path, entityType: t, msg: msg}
}
t.MustValidate()
// delimiter checks
if t != EntityTypeSnapshot && strings.Contains(path, "@") {
return pve("snapshot delimiter '@' is not expected here")
}
if t == EntityTypeSnapshot && !strings.Contains(path, "@") {
return pve("missing '@' delimiter in snapshot name")
}
if t != EntityTypeBookmark && strings.Contains(path, "#") {
return pve("bookmark delimiter '#' is not expected here")
}
if t == EntityTypeBookmark && !strings.Contains(path, "#") {
return pve("missing '#' delimiter in bookmark name")
}
// EntityTypeVolume and EntityTypeFilesystem are already covered above
if strings.Contains(path, "%") {
return pve("invalid character '%' in name")
}
// mimic module/zcommon/zfs_namecheck.c: entity_namecheck
if len(path) > MaxDatasetNameLen {
return pve("name too long")
}
if len(path) == 0 {
return pve("must not be empty")
}
if !isASCII(path) {
return pve("must be ASCII")
}
slashComps := bytes.Split([]byte(path), []byte("/"))
bookmarkOrSnapshotDelims := 0
for compI, comp := range slashComps {
snapCount := bytes.Count(comp, []byte("@"))
bookCount := bytes.Count(comp, []byte("#"))
if !(snapCount*bookCount == 0) {
panic("implementation error: delimiter checks before this loop must ensure this cannot happen")
}
bookmarkOrSnapshotDelims += snapCount + bookCount
if bookmarkOrSnapshotDelims > 1 {
return pve("multiple delimiters '@' or '#' are not allowed")
}
if bookmarkOrSnapshotDelims == 1 && compI != len(slashComps)-1 {
return pve("snapshot or bookmark must not contain '/'")
}
if bookmarkOrSnapshotDelims == 0 {
// hot path, all but last component
if err := ComponentNamecheck(string(comp)); err != nil {
return pve(err.Error())
}
continue
}
subComps := bytes.FieldsFunc(comp, func(r rune) bool {
return r == '#' || r == '@'
})
if len(subComps) > 2 {
panic("implementation error: delimiter checks above should ensure a single bookmark or snapshot delimiter per component")
}
if len(subComps) != 2 {
return pve("empty component, bookmark or snapshot name not allowed")
}
for _, comp := range subComps {
if err := ComponentNamecheck(string(comp)); err != nil {
return pve(err.Error())
}
}
}
return nil
}
func isASCII(s string) bool {
for i := 0; i < len(s); i++ {
if s[i] > unicode.MaxASCII {
return false
}
}
return true
}

64
zfs/namecheck_test.go Normal file
View File

@ -0,0 +1,64 @@
package zfs
import (
"strings"
"testing"
)
func TestEntityNamecheck(t *testing.T) {
type testcase struct {
input string
entityType EntityType
ok bool
}
tcs := []testcase{
{"/", EntityTypeFilesystem, false},
{"/foo", EntityTypeFilesystem, false},
{"/foo@bar", EntityTypeSnapshot, false},
{"foo", EntityTypeBookmark, false},
{"foo", EntityTypeSnapshot, false},
{"foo@bar", EntityTypeBookmark, false},
{"foo#bar", EntityTypeSnapshot, false},
{"foo#book", EntityTypeBookmark, true},
{"foo#book@bar", EntityTypeBookmark, false},
{"foo/book@bar", EntityTypeSnapshot, true},
{"foo/book#bar", EntityTypeBookmark, true},
{"foo/for%idden", EntityTypeFilesystem, false},
{"foo/bår", EntityTypeFilesystem, false},
{"", EntityTypeFilesystem, false},
{"foo/bar@", EntityTypeSnapshot, false},
{"foo/bar#", EntityTypeBookmark, false},
{"foo/bar#@blah", EntityTypeBookmark, false},
{"foo bar/baz bar@blah foo", EntityTypeSnapshot, true},
{"foo bar/baz bar@#lah foo", EntityTypeSnapshot, false},
{"foo bar/baz bar@@lah foo", EntityTypeSnapshot, false},
{"foo bar/baz bar##lah foo", EntityTypeBookmark, false},
{"foo bar/baz@blah/foo", EntityTypeSnapshot, false},
{"foo bar/baz@blah/foo", EntityTypeFilesystem, false},
{"foo/b\tr@ba\tz", EntityTypeSnapshot, false},
{"foo/b\tr@baz", EntityTypeSnapshot, false},
{"foo/bar@ba\tz", EntityTypeSnapshot, false},
{"foo/./bar", EntityTypeFilesystem, false},
{"foo/../bar", EntityTypeFilesystem, false},
{"foo/bar@..", EntityTypeFilesystem, false},
{"foo/bar@.", EntityTypeFilesystem, false},
{strings.Repeat("a", MaxDatasetNameLen), EntityTypeFilesystem, true},
{strings.Repeat("a", MaxDatasetNameLen) + "a", EntityTypeFilesystem, false},
{strings.Repeat("a", MaxDatasetNameLen-2) + "/a", EntityTypeFilesystem, true},
{strings.Repeat("a", MaxDatasetNameLen-4) + "/a@b", EntityTypeSnapshot, true},
{strings.Repeat("a", MaxDatasetNameLen) + "/a@b", EntityTypeSnapshot, false},
}
for idx := range tcs {
t.Run(tcs[idx].input, func(t *testing.T) {
tc := tcs[idx]
err := EntityNamecheck(tc.input, tc.entityType)
if !((err == nil && tc.ok) || (err != nil && !tc.ok)) {
t.Errorf("expecting ok=%v but got err=%v", tc.ok, err)
}
})
}
}

View File

@ -1,50 +0,0 @@
package zfs
import (
"fmt"
"github.com/pkg/errors"
)
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 nil, err
}
for _, v := range versions {
if v.Type == Bookmark && v.Name == ReplicationCursorBookmarkName {
return &v, nil
}
}
return nil, nil
}
func ZFSSetReplicationCursor(fs *DatasetPath, snapname string) (guid uint64, err error) {
snapPath := fmt.Sprintf("%s@%s", fs.ToString(), snapname)
debug("replication cursor: snap path %q", snapPath)
snapProps, err := ZFSGetCreateTXGAndGuid(snapPath)
if err != nil {
return 0, errors.Wrapf(err, "get properties of %q", snapPath)
}
bookmarkPath := fmt.Sprintf("%s#%s", fs.ToString(), ReplicationCursorBookmarkName)
propsBookmark, err := ZFSGetCreateTXGAndGuid(bookmarkPath)
_, bookmarkNotExistErr := err.(*DatasetDoesNotExist)
if err != nil && !bookmarkNotExistErr {
return 0, errors.Wrap(err, "zfs: replication cursor: get bookmark txg")
}
if err == nil {
if snapProps.CreateTXG < propsBookmark.CreateTXG {
return 0, errors.New("zfs: 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, errors.Wrap(err, "zfs: replication cursor: destroy current cursor")
}
}
if err := ZFSBookmark(fs, snapname, ReplicationCursorBookmarkName); err != nil {
return 0, errors.Wrapf(err, "zfs: replication cursor: create bookmark")
}
return snapProps.Guid, nil
}

View File

@ -2,17 +2,26 @@ package zfs
import (
"context"
"errors"
"fmt"
"os/exec"
"regexp"
"strconv"
"strings"
"sync"
"time"
"github.com/pkg/errors"
"github.com/zrepl/zrepl/util/envconst"
)
// NOTE: Update ZFSSendARgs.Validate when changning fields (potentially SECURITY SENSITIVE)
type ResumeToken struct {
HasFromGUID, HasToGUID bool
FromGUID, ToGUID uint64
// no support for other fields
ToName string
HasCompressOK, CompressOK bool
HasRawOk, RawOK bool
}
var resumeTokenNVListRE = regexp.MustCompile(`\t(\S+) = (.*)`)
@ -23,11 +32,134 @@ 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")
var resumeSendSupportedCheck struct {
once sync.Once
supported bool
err error
}
func ResumeSendSupported() (bool, error) {
resumeSendSupportedCheck.once.Do(func() {
// "feature discovery"
cmd := exec.Command("zfs", "send")
output, err := cmd.CombinedOutput()
if ee, ok := err.(*exec.ExitError); !ok || ok && !ee.Exited() {
resumeSendSupportedCheck.err = errors.Wrap(err, "resumable send cli support feature check failed")
}
def := strings.Contains(string(output), "receive_resume_token")
resumeSendSupportedCheck.supported = envconst.Bool("ZREPL_EXPERIMENTAL_ZFS_SEND_RESUME_SUPPORTED", def)
debug("resume send feature check complete %#v", &resumeSendSupportedCheck)
})
return resumeSendSupportedCheck.supported, resumeSendSupportedCheck.err
}
var resumeRecvPoolSupportRecheckTimeout = envconst.Duration("ZREPL_ZFS_RESUME_RECV_POOL_SUPPORT_RECHECK_TIMEOUT", 30*time.Second)
type resumeRecvPoolSupportedResult struct {
lastCheck time.Time
supported bool
err error
}
var resumeRecvSupportedCheck struct {
mtx sync.RWMutex
flagSupport struct {
checked bool
supported bool
err error
}
poolSupported map[string]resumeRecvPoolSupportedResult
}
// fs == nil only checks for CLI support
func ResumeRecvSupported(ctx context.Context, fs *DatasetPath) (bool, error) {
sup := &resumeRecvSupportedCheck
sup.mtx.RLock()
defer sup.mtx.RUnlock()
upgradeWhile := func(cb func()) {
sup.mtx.RUnlock()
defer sup.mtx.RLock()
sup.mtx.Lock()
defer sup.mtx.Unlock()
cb()
}
if !sup.flagSupport.checked {
output, err := exec.CommandContext(ctx, "zfs", "receive").CombinedOutput()
upgradeWhile(func() {
sup.flagSupport.checked = true
if ee, ok := err.(*exec.ExitError); err != nil && (!ok || ok && !ee.Exited()) {
sup.flagSupport.err = err
} else {
sup.flagSupport.supported = strings.Contains(string(output), "-A <filesystem|volume>")
}
debug("resume recv cli flag feature check result: %#v", sup.flagSupport)
})
// fallthrough
}
if sup.flagSupport.err != nil {
return false, errors.Wrap(sup.flagSupport.err, "zfs recv feature check for resumable send & recv failed")
} else if !sup.flagSupport.supported || fs == nil {
return sup.flagSupport.supported, nil
}
// Flag is supported and pool-support is request
// Now check for pool support
pool, err := fs.Pool()
if err != nil {
return false, errors.Wrap(err, "resume recv check requires pool of dataset")
}
if sup.poolSupported == nil {
upgradeWhile(func() {
sup.poolSupported = make(map[string]resumeRecvPoolSupportedResult)
})
}
var poolSup resumeRecvPoolSupportedResult
var ok bool
if poolSup, ok = sup.poolSupported[pool]; !ok || // shadow
(!poolSup.supported && time.Since(poolSup.lastCheck) > resumeRecvPoolSupportRecheckTimeout) {
output, err := exec.CommandContext(ctx, "zpool", "get", "-H", "-p", "-o", "value", "feature@extensible_dataset", pool).CombinedOutput()
if err != nil {
debug("resume recv pool support check result: %#v", sup.flagSupport)
poolSup.supported = false
poolSup.err = err
} else {
poolSup.err = nil
o := strings.TrimSpace(string(output))
poolSup.supported = o == "active" || o == "enabled"
}
poolSup.lastCheck = time.Now()
// we take the lock late, so two updaters might check simultaneously, but that shouldn't hurt
upgradeWhile(func() {
sup.poolSupported[pool] = poolSup
})
// fallthrough
}
if poolSup.err != nil {
return false, errors.Wrapf(poolSup.err, "pool %q check for feature@extensible_dataset feature failed", pool)
}
return poolSup.supported, nil
}
// 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) {
if supported, err := ResumeSendSupported(); err != nil {
return nil, err
} else if !supported {
return nil, ResumeTokenDecodingNotSupported
}
// Example resume tokens:
//
// From a non-incremental send
@ -48,8 +180,6 @@ 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, 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 {
@ -94,6 +224,20 @@ func ParseResumeToken(ctx context.Context, token string) (*ResumeToken, error) {
return nil, ResumeTokenParsingError
}
rt.HasToGUID = true
case "toname":
rt.ToName = val
case "rawok":
rt.HasRawOk = true
rt.RawOK, err = strconv.ParseBool(val)
if err != nil {
return nil, ResumeTokenParsingError
}
case "compressok":
rt.HasCompressOK = true
rt.CompressOK, err = strconv.ParseBool(val)
if err != nil {
return nil, ResumeTokenParsingError
}
}
}
@ -105,17 +249,35 @@ func ParseResumeToken(ctx context.Context, token string) (*ResumeToken, error) {
}
func ZFSGetReceiveResumeToken(fs *DatasetPath) (string, error) {
// if string is empty and err == nil, the feature is not supported
func ZFSGetReceiveResumeTokenOrEmptyStringIfNotSupported(ctx context.Context, fs *DatasetPath) (string, error) {
if supported, err := ResumeRecvSupported(ctx, fs); err != nil {
return "", errors.Wrap(err, "cannot determine zfs recv resume support")
} else if !supported {
return "", nil
}
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]
res := props.Get(prop_receive_resume_token)
debug("%q receive_resume_token=%q", fs.ToString(), res)
if res == "-" {
return "", nil
} else {
return res, nil
}
}
func (t *ResumeToken) ToNameSplit() (fs *DatasetPath, snapName string, err error) {
comps := strings.SplitN(t.ToName, "@", 2)
if len(comps) != 2 {
return nil, "", fmt.Errorf("resume token field `toname` does not contain @: %q", t.ToName)
}
dp, err := NewDatasetPath(comps[0])
if err != nil {
return nil, "", errors.Wrap(err, "resume token field `toname` dataset path invalid")
}
return dp, comps[1], nil
}

View File

@ -1,68 +0,0 @@
package zfs_test
import (
"context"
"testing"
"github.com/stretchr/testify/assert"
"github.com/zrepl/zrepl/zfs"
)
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) {
t.SkipNow() // FIXME not compatible with docker
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)
}
}

View File

@ -88,6 +88,19 @@ type FilesystemVersionFilter interface {
Filter(t VersionType, name string) (accept bool, err error)
}
type closureFilesystemVersionFilter struct {
cb func(t VersionType, name string) (accept bool, err error)
}
func (f *closureFilesystemVersionFilter) Filter(t VersionType, name string) (accept bool, err error) {
return f.cb(t, name)
}
func FilterFromClosure(cb func(t VersionType, name string) (accept bool, err error)) FilesystemVersionFilter {
return &closureFilesystemVersionFilter{cb}
}
// returned versions are sorted by createtxg
func ZFSListFilesystemVersions(fs *DatasetPath, filter FilesystemVersionFilter) (res []FilesystemVersion, err error) {
listResults := make(chan ZFSListResult)

View File

@ -3,20 +3,19 @@ package zfs
import (
"bufio"
"bytes"
"context"
"encoding/json"
"fmt"
"io"
"os"
"os/exec"
"regexp"
"sort"
"strconv"
"strings"
"sync"
"time"
"context"
"regexp"
"strconv"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
@ -113,6 +112,14 @@ func (p *DatasetPath) UnmarshalJSON(b []byte) error {
return json.Unmarshal(b, &p.comps)
}
func (p *DatasetPath) Pool() (string, error) {
if len(p.comps) < 1 {
return "", fmt.Errorf("dataset path does not have a pool component")
}
return p.comps[0], nil
}
func NewDatasetPath(s string) (p *DatasetPath, err error) {
p = &DatasetPath{}
if s == "" {
@ -287,17 +294,7 @@ func ZFSListChan(ctx context.Context, out chan ZFSListResult, properties []strin
}
}
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
}
// FIXME replace with EntityNamecheck
func validateZFSFilesystem(fs string) error {
if len(fs) < 1 {
return errors.New("filesystem path must have length > 0")
@ -305,31 +302,40 @@ func validateZFSFilesystem(fs string) error {
return nil
}
func absVersion(fs, v string) (full string, err error) {
// v must not be nil and be already validated
func absVersion(fs string, v *ZFSSendArgVersion) (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
return fmt.Sprintf("%s%s", fs, v.RelName), nil
}
func buildCommonSendArgs(fs string, from, to string, token string) ([]string, error) {
// tok is allowed to be nil
// a must already be validated
//
// SECURITY SENSITIVE because Raw must be handled correctly
func (a ZFSSendArgs) buildCommonSendArgs() ([]string, error) {
args := make([]string, 0, 3)
if token != "" {
args = append(args, "-t", token)
// ResumeToken takes precedence, we assume that it has been validated to reflect
// what is described by the other fields in ZFSSendArgs
if a.ResumeToken != "" {
args = append(args, "-t", a.ResumeToken)
return args, nil
}
toV, err := absVersion(fs, to)
if a.Encrypted.B {
args = append(args, "-w")
}
toV, err := absVersion(a.FS, a.To)
if err != nil {
return nil, err
}
fromV := ""
if from != "" {
fromV, err = absVersion(fs, from)
if a.From != nil {
fromV, err = absVersion(a.FS, a.From)
if err != nil {
return nil, err
}
@ -513,17 +519,317 @@ func (s *sendStream) killAndWait(precedingReadErr error) error {
return s.opErr
}
// NOTE: When updating this struct, make sure to update funcs Validate ValidateCorrespondsToResumeToken
type ZFSSendArgVersion struct {
RelName string
GUID uint64
}
func (v ZFSSendArgVersion) ValidateInMemory(fs string) error {
if fs == "" {
panic(fs)
}
if len(v.RelName) == 0 {
return errors.New("`RelName` must not be empty")
}
var et EntityType
switch v.RelName[0] {
case '@':
et = EntityTypeSnapshot
case '#':
et = EntityTypeBookmark
default:
return fmt.Errorf("`RelName` field must start with @ or #, got %q", v.RelName)
}
full := v.fullPathUnchecked(fs)
if err := EntityNamecheck(full, et); err != nil {
return err
}
return nil
}
func (v ZFSSendArgVersion) mustValidateInMemory(fs string) {
if err := v.ValidateInMemory(fs); err != nil {
panic(err)
}
}
// fs must be not empty
func (a ZFSSendArgVersion) ValidateExistsAndGetCheckedProps(ctx context.Context, fs string) (ZFSPropCreateTxgAndGuidProps, error) {
if err := a.ValidateInMemory(fs); err != nil {
return ZFSPropCreateTxgAndGuidProps{}, nil
}
realProps, err := ZFSGetCreateTXGAndGuid(a.FullPath(fs))
if err != nil {
return ZFSPropCreateTxgAndGuidProps{}, err
}
if realProps.Guid != a.GUID {
return ZFSPropCreateTxgAndGuidProps{}, fmt.Errorf("`GUID` field does not match real dataset's GUID: %q != %q", realProps.Guid, a.GUID)
}
return realProps, nil
}
func (a ZFSSendArgVersion) ValidateExists(ctx context.Context, fs string) error {
_, err := a.ValidateExistsAndGetCheckedProps(ctx, fs)
return err
}
func (v ZFSSendArgVersion) FullPath(fs string) string {
v.mustValidateInMemory(fs)
return v.fullPathUnchecked(fs)
}
func (v ZFSSendArgVersion) fullPathUnchecked(fs string) string {
return fmt.Sprintf("%s%s", fs, v.RelName)
}
func (v ZFSSendArgVersion) IsSnapshot() bool {
v.mustValidateInMemory("unimportant")
return v.RelName[0] == '@'
}
func (v ZFSSendArgVersion) MustBeBookmark() {
v.mustValidateInMemory("unimportant")
if v.RelName[0] != '#' {
panic(fmt.Sprintf("must be bookmark, got %q", v.RelName))
}
}
type NilBool struct{ B bool }
func (n *NilBool) Validate() error {
if n == nil {
return fmt.Errorf("must explicitly set `true` or `false`")
}
return nil
}
func (n *NilBool) String() string {
if n == nil {
return "unset"
}
return fmt.Sprintf("%v", n.B)
}
// When updating this struct, check Validate and ValidateCorrespondsToResumeToken (Potentiall SECURITY SENSITIVE)
type ZFSSendArgs struct {
FS string
From, To *ZFSSendArgVersion // From may be nil
Encrypted *NilBool
// Prefereed if not empty
ResumeToken string // if not nil, must match what is specified in From, To (covered by ValidateCorrespondsToResumeToken)
}
type zfsSendArgsValidationContext struct {
encEnabled *NilBool
}
type ZFSSendArgsValidationErrorCode int
const (
ZFSSendArgsGenericValidationError ZFSSendArgsValidationErrorCode = 1 + iota
ZFSSendArgsEncryptedSendRequestedButFSUnencrypted
ZFSSendArgsFSEncryptionCheckFail
ZFSSendArgsResumeTokenMismatch
)
type ZFSSendArgsValidationError struct {
Args ZFSSendArgs
What ZFSSendArgsValidationErrorCode
Msg error
}
func newValidationError(sendArgs ZFSSendArgs, what ZFSSendArgsValidationErrorCode, cause error) *ZFSSendArgsValidationError {
return &ZFSSendArgsValidationError{sendArgs, what, cause}
}
func newGenericValidationError(sendArgs ZFSSendArgs, cause error) *ZFSSendArgsValidationError {
return &ZFSSendArgsValidationError{sendArgs, ZFSSendArgsGenericValidationError, cause}
}
func (e ZFSSendArgsValidationError) Error() string {
return e.Msg.Error()
}
// - Recursively call Validate on each field.
// - Make sure that if ResumeToken != "", it reflects the same operation as the other paramters would.
//
// This function is not pure because GUIDs are checked against the local host's datasets.
func (a ZFSSendArgs) Validate(ctx context.Context) error {
if dp, err := NewDatasetPath(a.FS); err != nil || dp.Length() == 0 {
return newGenericValidationError(a, fmt.Errorf("`FS` must be a valid non-zero dataset path"))
}
if a.To == nil {
return newGenericValidationError(a, fmt.Errorf("`To` must not be nil"))
}
if err := a.To.ValidateExists(ctx, a.FS); err != nil {
return newGenericValidationError(a, errors.Wrap(err, "`To` invalid"))
}
if a.From != nil {
if err := a.From.ValidateExists(ctx, a.FS); err != nil {
return newGenericValidationError(a, errors.Wrap(err, "`From` invalid"))
}
// falthrough
}
if err := a.Encrypted.Validate(); err != nil {
return newGenericValidationError(a, errors.Wrap(err, "`Raw` invalid"))
}
valCtx := &zfsSendArgsValidationContext{}
fsEncrypted, err := ZFSGetEncryptionEnabled(ctx, a.FS)
if err != nil {
return newValidationError(a, ZFSSendArgsFSEncryptionCheckFail,
errors.Wrapf(err, "cannot check whether filesystem %q is encrypted", a.FS))
}
valCtx.encEnabled = &NilBool{fsEncrypted}
if a.Encrypted.B && !fsEncrypted {
return newValidationError(a, ZFSSendArgsEncryptedSendRequestedButFSUnencrypted,
errors.Errorf("encrypted send requested, but filesystem %q is not encrypted", a.FS))
}
if a.ResumeToken != "" {
if err := a.validateCorrespondsToResumeToken(ctx, valCtx); err != nil {
return newValidationError(a, ZFSSendArgsResumeTokenMismatch, err)
}
}
return nil
}
type ZFSSendArgsResumeTokenMismatchError struct {
What ZFSSendArgsResumeTokenMismatchErrorCode
Err error
}
func (e *ZFSSendArgsResumeTokenMismatchError) Error() string { return e.Err.Error() }
type ZFSSendArgsResumeTokenMismatchErrorCode int
// The format is ZFSSendArgsResumeTokenMismatch+WhatIsWrongInToken
const (
ZFSSendArgsResumeTokenMismatchGeneric ZFSSendArgsResumeTokenMismatchErrorCode = 1 + iota
ZFSSendArgsResumeTokenMismatchEncryptionNotSet // encryption not set in token but required by send args
ZFSSendArgsResumeTokenMismatchEncryptionSet // encryption not set in token but not required by send args
ZFSSendArgsResumeTokenMismatchFilesystem
)
func (c ZFSSendArgsResumeTokenMismatchErrorCode) fmt(format string, args ...interface{}) *ZFSSendArgsResumeTokenMismatchError {
return &ZFSSendArgsResumeTokenMismatchError{
What: c,
Err: fmt.Errorf(format, args...),
}
}
// This is SECURITY SENSITIVE and requires exhaustive checking of both side's values
// An attacker requesting a Send with a crafted ResumeToken may encode different parameters in the resume token than expected:
// for example, they may specify another file system (e.g. the filesystem with secret data) or request unencrypted send instead of encrypted raw send.
func (a ZFSSendArgs) validateCorrespondsToResumeToken(ctx context.Context, valCtx *zfsSendArgsValidationContext) error {
if a.ResumeToken == "" {
return nil // nothing to do
}
debug("decoding resume token %q", a.ResumeToken)
t, err := ParseResumeToken(ctx, a.ResumeToken)
debug("decode resumee token result: %#v %T %v", t, err, err)
if err != nil {
return err
}
tokenFS, _, err := t.ToNameSplit()
if err != nil {
return err
}
gen := ZFSSendArgsResumeTokenMismatchGeneric
if a.FS != tokenFS.ToString() {
return ZFSSendArgsResumeTokenMismatchFilesystem.fmt(
"filesystem in resume token field `toname` = %q does not match expected value %q", tokenFS.ToString(), a.FS)
}
if (a.From != nil) != t.HasFromGUID { // existence must be same
if t.HasFromGUID {
return gen.fmt("resume token not expected to be incremental, but `fromguid` = %q", t.FromGUID)
} else {
return gen.fmt("resume token expected to be incremental, but `fromguid` not present")
}
} else if t.HasFromGUID { // if exists (which is same, we checked above), they must match
if t.FromGUID != a.From.GUID {
return gen.fmt("resume token `fromguid` != expected: %q != %q", t.FromGUID, a.From.GUID)
}
} else {
_ = struct{}{} // both empty, ok
}
// To must never be empty
if !t.HasToGUID {
return gen.fmt("resume token does not have `toguid`")
}
if t.ToGUID != a.To.GUID { // a.To != nil because Validate checks for that
return gen.fmt("resume token `toguid` != expected: %q != %q", t.ToGUID, a.To.GUID)
}
if a.Encrypted.B {
if !(t.RawOK && t.CompressOK) {
return ZFSSendArgsResumeTokenMismatchEncryptionNotSet.fmt(
"resume token must have `rawok` and `compressok` = true but got %v %v", t.RawOK, t.CompressOK)
}
// fallthrough
} else {
if t.RawOK || t.CompressOK {
return ZFSSendArgsResumeTokenMismatchEncryptionSet.fmt(
"resume token must not have `rawok` or `compressok` set but got %v %v", t.RawOK, t.CompressOK)
}
// fallthrough
}
return nil
}
var zfsSendStderrCaptureMaxSize = envconst.Int("ZREPL_ZFS_SEND_STDERR_MAX_CAPTURE_SIZE", 1<<15)
var ErrEncryptedSendNotSupported = fmt.Errorf("raw sends which are required for encrypted zfs send are not supported")
// if token != "", then send -t token is used
// otherwise send [-i from] to is used
// (if from is "" a full ZFS send is done)
func ZFSSend(ctx context.Context, fs string, from, to string, token string) (*ReadCloserCopier, error) {
//
// Returns ErrEncryptedSendNotSupported if encrypted send is requested but not supported by CLI
func ZFSSend(ctx context.Context, sendArgs ZFSSendArgs) (*ReadCloserCopier, error) {
args := make([]string, 0)
args = append(args, "send")
sargs, err := buildCommonSendArgs(fs, from, to, token)
// pre-validation of sendArgs for plain ErrEncryptedSendNotSupported error
// TODO go1.13: push this down to sendArgs.Validate
if encryptedSendValid := sendArgs.Encrypted.Validate(); encryptedSendValid == nil && sendArgs.Encrypted.B {
supported, err := EncryptionCLISupported(ctx)
if err != nil {
return nil, errors.Wrap(err, "cannot determine CLI native encryption support")
}
if !supported {
return nil, ErrEncryptedSendNotSupported
}
}
if err := sendArgs.Validate(ctx); err != nil {
return nil, err // do not wrap, part of API, tested by platformtest
}
sargs, err := sendArgs.buildCommonSendArgs()
if err != nil {
return nil, err
}
@ -651,28 +957,32 @@ func (s *DrySendInfo) unmarshalInfoLine(l string) (regexMatched bool, err error)
return true, nil
}
// from may be "", in which case a full ZFS send is done
// to may be "", in which case a full ZFS send is done
// May return BookmarkSizeEstimationNotSupported as err if from is a bookmark.
func ZFSSendDry(fs string, from, to string, token string) (_ *DrySendInfo, err error) {
func ZFSSendDry(ctx context.Context, sendArgs ZFSSendArgs) (_ *DrySendInfo, err error) {
if strings.Contains(from, "#") {
if err := sendArgs.Validate(ctx); err != nil {
return nil, errors.Wrap(err, "cannot validate send args")
}
if sendArgs.From != nil && strings.Contains(sendArgs.From.RelName, "#") {
/* 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
*/
fromAbs, err := absVersion(fs, from)
fromAbs, err := absVersion(sendArgs.FS, sendArgs.From)
if err != nil {
return nil, fmt.Errorf("error building abs version for 'from': %s", err)
}
toAbs, err := absVersion(fs, to)
toAbs, err := absVersion(sendArgs.FS, sendArgs.To)
if err != nil {
return nil, fmt.Errorf("error building abs version for 'to': %s", err)
}
return &DrySendInfo{
Type: DrySendTypeIncremental,
Filesystem: fs,
Filesystem: sendArgs.FS,
From: fromAbs,
To: toAbs,
SizeEstimate: -1}, nil
@ -680,7 +990,7 @@ func ZFSSendDry(fs string, from, to string, token string) (_ *DrySendInfo, err e
args := make([]string, 0)
args = append(args, "send", "-n", "-v", "-P")
sargs, err := buildCommonSendArgs(fs, from, to, token)
sargs, err := sendArgs.buildCommonSendArgs()
if err != nil {
return nil, err
}
@ -689,7 +999,7 @@ func ZFSSendDry(fs string, from, to string, token string) (_ *DrySendInfo, err e
cmd := exec.Command(ZFS_BINARY, args...)
output, err := cmd.CombinedOutput()
if err != nil {
return nil, err
return nil, &ZFSError{output, err}
}
var si DrySendInfo
if err := si.unmarshalZFSOutput(output); err != nil {
@ -720,13 +1030,35 @@ type RecvOptions struct {
// Rollback to the oldest snapshot, destroy it, then perform `recv -F`.
// Note that this doesn't change property values, i.e. an existing local property value will be kept.
RollbackAndForceRecv bool
// Set -s flag used for resumable send & recv
SavePartialRecvState bool
}
func ZFSRecv(ctx context.Context, fs string, streamCopier StreamCopier, opts RecvOptions) (err error) {
if err := validateZFSFilesystem(fs); err != nil {
return err
type ErrRecvResumeNotSupported struct {
FS string
CheckErr error
}
func (e *ErrRecvResumeNotSupported) Error() string {
var buf strings.Builder
fmt.Fprintf(&buf, "zfs resumable recv into %q: ", e.FS)
if e.CheckErr != nil {
fmt.Fprint(&buf, e.CheckErr.Error())
} else {
fmt.Fprintf(&buf, "not supported by ZFS or pool")
}
return buf.String()
}
func ZFSRecv(ctx context.Context, fs string, v *ZFSSendArgVersion, streamCopier StreamCopier, opts RecvOptions) (err error) {
if err := v.ValidateInMemory(fs); err != nil {
return errors.Wrap(err, "invalid version")
}
if !v.IsSnapshot() {
return errors.New("must receive into a snapshot")
}
fsdp, err := NewDatasetPath(fs)
if err != nil {
return err
@ -773,7 +1105,13 @@ func ZFSRecv(ctx context.Context, fs string, streamCopier StreamCopier, opts Rec
if opts.RollbackAndForceRecv {
args = append(args, "-F")
}
args = append(args, fs)
if opts.SavePartialRecvState {
if supported, err := ResumeRecvSupported(ctx, fsdp); err != nil || !supported {
return &ErrRecvResumeNotSupported{FS: fs, CheckErr: err}
}
args = append(args, "-s")
}
args = append(args, v.FullPath(fs))
ctx, cancelCmd := context.WithCancel(ctx)
defer cancelCmd()
@ -816,14 +1154,18 @@ func ZFSRecv(ctx context.Context, fs string, streamCopier StreamCopier, opts Rec
copierErrChan <- streamCopier.WriteStreamTo(stdinWriter)
stdinWriter.Close()
}()
waitErrChan := make(chan *ZFSError)
waitErrChan := make(chan error)
go func() {
defer close(waitErrChan)
if err = cmd.Wait(); err != nil {
if rtErr := tryRecvErrorWithResumeToken(ctx, stderr.String()); rtErr != nil {
waitErrChan <- rtErr
} else {
waitErrChan <- &ZFSError{
Stderr: stderr.Bytes(),
WaitErr: err,
}
}
return
}
}()
@ -846,6 +1188,33 @@ func ZFSRecv(ctx context.Context, fs string, streamCopier StreamCopier, opts Rec
return copierErr // if it's not a write error, the copier error is more interesting
}
type RecvFailedWithResumeTokenErr struct {
Msg string
ResumeTokenRaw string
ResumeTokenParsed *ResumeToken
}
var recvErrorResumeTokenRE = regexp.MustCompile(`A resuming stream can be generated on the sending system by running:\s+zfs send -t\s(\S+)`)
func tryRecvErrorWithResumeToken(ctx context.Context, stderr string) *RecvFailedWithResumeTokenErr {
if match := recvErrorResumeTokenRE.FindStringSubmatch(stderr); match != nil {
parsed, err := ParseResumeToken(ctx, match[1])
if err != nil {
return nil
}
return &RecvFailedWithResumeTokenErr{
Msg: stderr,
ResumeTokenRaw: match[1],
ResumeTokenParsed: parsed,
}
}
return nil
}
func (e *RecvFailedWithResumeTokenErr) Error() string {
return fmt.Sprintf("receive failed, resume token available: %s\n%#v", e.ResumeTokenRaw, e.ResumeTokenParsed)
}
type ClearResumeTokenError struct {
ZFSOutput []byte
CmdError error
@ -934,6 +1303,55 @@ func ZFSGet(fs *DatasetPath, props []string) (*ZFSProperties, error) {
return zfsGet(fs.ToString(), props, sourceAny)
}
// The returned error includes requested filesystem and version as quoted strings in its error message
func ZFSGetGUID(fs string, version string) (g uint64, err error) {
defer func(e *error) {
if *e != nil {
*e = fmt.Errorf("zfs get guid fs=%q version=%q: %s", fs, version, *e)
}
}(&err)
if err := validateZFSFilesystem(fs); err != nil {
return 0, err
}
if len(version) == 0 {
return 0, errors.New("version must have non-zero length")
}
if strings.IndexAny(version[0:1], "@#") != 0 {
return 0, errors.New("version does not start with @ or #")
}
path := fmt.Sprintf("%s%s", fs, version)
props, err := zfsGet(path, []string{"guid"}, sourceAny) // always local
if err != nil {
return 0, err
}
return strconv.ParseUint(props.Get("guid"), 10, 64)
}
type GetMountpointOutput struct {
Mounted bool
Mountpoint string
}
func ZFSGetMountpoint(fs string) (*GetMountpointOutput, error) {
if err := EntityNamecheck(fs, EntityTypeFilesystem); err != nil {
return nil, err
}
props, err := zfsGet(fs, []string{"mountpoint", "mounted"}, sourceAny)
if err != nil {
return nil, err
}
o := &GetMountpointOutput{}
o.Mounted = props.Get("mounted") == "yes"
o.Mountpoint = props.Get("mountpoint")
if o.Mountpoint == "none" {
o.Mountpoint = ""
}
if o.Mounted && o.Mountpoint == "" {
panic("unexpected zfs get output")
}
return o, nil
}
func ZFSGetRawAnySource(path string, props []string) (*ZFSProperties, error) {
return zfsGet(path, props, sourceAny)
}
@ -946,6 +1364,15 @@ type DatasetDoesNotExist struct {
func (d *DatasetDoesNotExist) Error() string { return fmt.Sprintf("dataset %q does not exist", d.Path) }
func tryDatasetDoesNotExist(expectPath string, stderr []byte) error {
if sm := zfsGetDatasetDoesNotExistRegexp.FindSubmatch(stderr); sm != nil {
if string(sm[1]) == expectPath {
return &DatasetDoesNotExist{expectPath}
}
}
return nil
}
type zfsPropertySource uint
const (
@ -993,10 +1420,8 @@ func zfsGet(path string, props []string, allowedSources zfsPropertySource) (*ZFS
if exitErr, ok := err.(*exec.ExitError); ok {
if exitErr.Exited() {
// screen-scrape output
if sm := zfsGetDatasetDoesNotExistRegexp.FindSubmatch(exitErr.Stderr); sm != nil {
if string(sm[1]) == path {
return nil, &DatasetDoesNotExist{path}
}
if ddne := tryDatasetDoesNotExist(path, exitErr.Stderr); ddne != nil {
return nil, ddne
}
}
return nil, &ZFSError{
@ -1090,6 +1515,10 @@ func (e *DestroySnapshotsError) Error() string {
var destroySnapshotsErrorRegexp = regexp.MustCompile(`^cannot destroy snapshot ([^@]+)@(.+): (.*)$`) // yes, datasets can contain `:`
var destroyOneOrMoreSnapshotsNoneExistedErrorRegexp = regexp.MustCompile(`^could not find any snapshots to destroy; check snapshot names.`)
var destroyBookmarkDoesNotExist = regexp.MustCompile(`^bookmark '([^']+)' does not exist`)
func tryParseDestroySnapshotsError(arg string, stderr []byte) *DestroySnapshotsError {
argComps := strings.SplitN(arg, "@", 2)
@ -1161,7 +1590,14 @@ func ZFSDestroy(arg string) (err error) {
Stderr: stderr.Bytes(),
WaitErr: err,
}
if dserr := tryParseDestroySnapshotsError(arg, stderr.Bytes()); dserr != nil {
if destroyOneOrMoreSnapshotsNoneExistedErrorRegexp.Match(stderr.Bytes()) {
err = &DatasetDoesNotExist{arg}
} else if match := destroyBookmarkDoesNotExist.FindStringSubmatch(stderr.String()); match != nil && match[1] == arg {
err = &DatasetDoesNotExist{arg}
} else if dsNotExistErr := tryDatasetDoesNotExist(filesystem, stderr.Bytes()); dsNotExistErr != nil {
err = dsNotExistErr
} else if dserr := tryParseDestroySnapshotsError(arg, stderr.Bytes()); dserr != nil {
err = dserr
}
@ -1171,12 +1607,12 @@ func ZFSDestroy(arg string) (err error) {
}
func zfsBuildSnapName(fs *DatasetPath, name string) string { // TODO defensive
return fmt.Sprintf("%s@%s", fs.ToString(), name)
func ZFSDestroyIdempotent(path string) error {
err := ZFSDestroy(path)
if _, ok := err.(*DatasetDoesNotExist); ok {
return nil
}
func zfsBuildBookmarkName(fs *DatasetPath, name string) string { // TODO defensive
return fmt.Sprintf("%s#%s", fs.ToString(), name)
return err
}
func ZFSSnapshot(fs *DatasetPath, name string, recursive bool) (err error) {
@ -1184,7 +1620,10 @@ func ZFSSnapshot(fs *DatasetPath, name string, recursive bool) (err error) {
promTimer := prometheus.NewTimer(prom.ZFSSnapshotDuration.WithLabelValues(fs.ToString()))
defer promTimer.ObserveDuration()
snapname := zfsBuildSnapName(fs, name)
snapname := fmt.Sprintf("%s@%s", fs.ToString(), name)
if err := EntityNamecheck(snapname, EntityTypeSnapshot); err != nil {
return errors.Wrap(err, "zfs snapshot")
}
cmd := exec.Command(ZFS_BINARY, "snapshot", snapname)
stderr := bytes.NewBuffer(make([]byte, 0, 1024))
@ -1205,13 +1644,46 @@ func ZFSSnapshot(fs *DatasetPath, name string, recursive bool) (err error) {
}
func ZFSBookmark(fs *DatasetPath, snapshot, bookmark string) (err error) {
var zfsBookmarkExistsRegex = regexp.MustCompile("^cannot create bookmark '[^']+': bookmark exists")
promTimer := prometheus.NewTimer(prom.ZFSBookmarkDuration.WithLabelValues(fs.ToString()))
type BookmarkExists struct {
zfsMsg string
fs, bookmark string
bookmarkOrigin ZFSSendArgVersion
bookGuid uint64
}
func (e *BookmarkExists) Error() string {
return fmt.Sprintf("bookmark %s (guid=%v) with #%s: bookmark #%s exists but has different guid (%v)",
e.bookmarkOrigin.FullPath(e.fs), e.bookmarkOrigin.GUID, e.bookmark, e.bookmark, e.bookGuid,
)
}
var ErrBookmarkCloningNotSupported = fmt.Errorf("bookmark cloning feature is not yet supported by ZFS")
// idempotently create bookmark of the given version v
//
// v must be validated by the caller
//
// does not destroy an existing bookmark, returns
//
func ZFSBookmark(fs string, v ZFSSendArgVersion, bookmark string) (err error) {
promTimer := prometheus.NewTimer(prom.ZFSBookmarkDuration.WithLabelValues(fs))
defer promTimer.ObserveDuration()
snapname := zfsBuildSnapName(fs, snapshot)
bookmarkname := zfsBuildBookmarkName(fs, bookmark)
if !v.IsSnapshot() {
return ErrBookmarkCloningNotSupported // TODO This is work in progress: https://github.com/zfsonlinux/zfs/pull/9571
}
snapname := v.FullPath(fs)
if err := EntityNamecheck(snapname, EntityTypeSnapshot); err != nil {
return err
}
bookmarkname := fmt.Sprintf("%s#%s", fs, bookmark)
if err := EntityNamecheck(bookmarkname, EntityTypeBookmark); err != nil {
return err
}
debug("bookmark: %q %q", snapname, bookmarkname)
@ -1225,13 +1697,37 @@ func ZFSBookmark(fs *DatasetPath, snapshot, bookmark string) (err error) {
}
if err = cmd.Wait(); err != nil {
err = &ZFSError{
if ddne := tryDatasetDoesNotExist(snapname, stderr.Bytes()); err != nil {
return ddne
} else if zfsBookmarkExistsRegex.Match(stderr.Bytes()) {
// check if this was idempotent
bookGuid, err := ZFSGetGUID(fs, "#"+bookmark)
if err != nil {
return errors.Wrap(err, "bookmark idempotency check") // guid error expressive enough
}
if v.GUID == bookGuid {
debug("bookmark: %q %q was idempotent: {snap,book}guid %d == %d", snapname, bookmarkname, v.GUID, bookGuid)
return nil
}
return &BookmarkExists{
fs: fs, bookmarkOrigin: v, bookmark: bookmark,
zfsMsg: stderr.String(),
bookGuid: bookGuid,
}
} else {
return &ZFSError{
Stderr: stderr.Bytes(),
WaitErr: err,
}
}
return
}
return nil
}