diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel index 56863d4679d9..977aea08e369 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel @@ -13,6 +13,8 @@ go_test( "//pkg/ccl/partitionccl", "//pkg/ccl/utilccl", "//pkg/config/zonepb", + "//pkg/jobs", + "//pkg/jobs/jobspb", "//pkg/jobs/jobsprotectedts", "//pkg/kv", "//pkg/kv/kvserver/protectedts", diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go index 84d106f5ca13..09daf1fa428a 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go @@ -19,10 +19,13 @@ import ( _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/partitionccl" "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/jobs/jobsprotectedts" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster" @@ -208,9 +211,37 @@ func TestDataDriven(t *testing.T) { var protectTS int d.ScanArgs(t, "record-id", &recordID) d.ScanArgs(t, "ts", &protectTS) - target := spanconfigtestutils.ParseProtectionTarget(t, d.Input) jobID := tenant.JobsRegistry().MakeJobID() + if d.HasArg("mk-job-with-type") { + var mkJobWithType string + d.ScanArgs(t, "mk-job-with-type", &mkJobWithType) + var rec jobs.Record + switch mkJobWithType { + case "backup": + rec = jobs.Record{ + Description: "backup", + Username: security.RootUserName(), + Details: jobspb.BackupDetails{}, + Progress: jobspb.BackupProgress{}, + } + default: + rec = jobs.Record{ + Description: "cdc", + Username: security.RootUserName(), + Details: jobspb.ChangefeedDetails{}, + Progress: jobspb.ChangefeedProgress{}, + } + } + require.NoError(t, tenant.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + if _, err = tenant.JobsRegistry().CreateJobWithTxn(ctx, rec, jobID, txn); err != nil { + return err + } + return nil + })) + } + target := spanconfigtestutils.ParseProtectionTarget(t, d.Input) + require.NoError(t, tenant.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { require.Len(t, recordID, 1, diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/exclude_data_from_backup b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/exclude_data_from_backup index 5e549f408456..9f8a6396e54f 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/exclude_data_from_backup +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/exclude_data_from_backup @@ -41,12 +41,32 @@ translate database=db table=t2 ---- /Table/10{7-8} range default +# Write a protection record as a backup. +protect record-id=1 ts=1 mk-job-with-type=backup +descs 104 +---- + +# Write another protection record as a non-backup user. +protect record-id=2 ts=2 mk-job-with-type=cdc +descs 104 +---- + +# Translate to ensure that the ProtectionPolicy is set with +# `ignore_if_excluded_from_backup` for the record written by the backup only. +translate database=db +---- +/Table/10{6-7} protection_policies=[{ts: 1,ignore_if_excluded_from_backup: true} {ts: 2}] exclude_data_from_backup=true +/Table/10{7-8} protection_policies=[{ts: 1,ignore_if_excluded_from_backup: true} {ts: 2}] + # Alter table t1 to unmark its data ephemeral. exec-sql ALTER TABLE db.t1 SET (exclude_data_from_backup = false); ---- +release record-id=1 +---- + translate database=db ---- -/Table/10{6-7} range default -/Table/10{7-8} range default +/Table/10{6-7} protection_policies=[{ts: 2}] +/Table/10{7-8} protection_policies=[{ts: 2}] diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts index 5e6401befaf9..900dbe74c55f 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts @@ -28,7 +28,7 @@ descs 106 translate database=db ---- -/Table/10{6-7} num_replicas=7 num_voters=5 pts=[1] +/Table/10{6-7} num_replicas=7 num_voters=5 protection_policies=[{ts: 1}] /Table/10{7-8} num_replicas=7 # Write a protected timestamp on db, so we should see it on both t1 and t2. @@ -38,8 +38,8 @@ descs 104 translate database=db ---- -/Table/10{6-7} num_replicas=7 num_voters=5 pts=[1 2] -/Table/10{7-8} num_replicas=7 pts=[2] +/Table/10{6-7} num_replicas=7 num_voters=5 protection_policies=[{ts: 1} {ts: 2}] +/Table/10{7-8} num_replicas=7 protection_policies=[{ts: 2}] # Release the protected timestamp on table t1 release record-id=1 @@ -47,8 +47,8 @@ release record-id=1 translate database=db ---- -/Table/10{6-7} num_replicas=7 num_voters=5 pts=[2] -/Table/10{7-8} num_replicas=7 pts=[2] +/Table/10{6-7} num_replicas=7 num_voters=5 protection_policies=[{ts: 2}] +/Table/10{7-8} num_replicas=7 protection_policies=[{ts: 2}] # Release the protected timestamp on database db release record-id=2 @@ -71,7 +71,7 @@ descs 106 translate database=db ---- -/Table/106{-/2} num_replicas=7 num_voters=5 pts=[3] -/Table/106/{2-3} ttl_seconds=1 num_replicas=7 num_voters=5 pts=[3] -/Table/10{6/3-7} num_replicas=7 num_voters=5 pts=[3] +/Table/106{-/2} num_replicas=7 num_voters=5 protection_policies=[{ts: 3}] +/Table/106/{2-3} ttl_seconds=1 num_replicas=7 num_voters=5 protection_policies=[{ts: 3}] +/Table/10{6/3-7} num_replicas=7 num_voters=5 protection_policies=[{ts: 3}] /Table/10{7-8} num_replicas=7 diff --git a/pkg/jobs/jobsprotectedts/jobs_protected_ts.go b/pkg/jobs/jobsprotectedts/jobs_protected_ts.go index 5da0493f0651..b04fbf9adb6e 100644 --- a/pkg/jobs/jobsprotectedts/jobs_protected_ts.go +++ b/pkg/jobs/jobsprotectedts/jobs_protected_ts.go @@ -50,8 +50,38 @@ func GetMetaType(metaType MetaType) string { return metaTypes[metaType] } +// IsRecordWrittenByJobType returns true if the job implied with this value of +// meta is of type jobType. This method can be used to filter records written by +// a particular job type. +func IsRecordWrittenByJobType( + ctx context.Context, + jr *jobs.Registry, + txn *kv.Txn, + meta []byte, + metaType MetaType, + jobType jobspb.Type, +) (bool, error) { + switch metaType { + case Jobs: + jobID, err := decodeID(meta) + if err != nil { + return false, err + } + j, err := jr.LoadJobWithTxn(ctx, jobspb.JobID(jobID), txn) + if jobs.HasJobNotFoundError(err) { + return false, nil + } + if err != nil { + return false, err + } + payload := j.Payload() + return payload.Type() == jobType, nil + } + return false, nil +} + // MakeStatusFunc returns a function which determines whether the job or -// schedule implied with this value of meta should be removed by the reconciler. +// schedule implied with this value of meta should be removed by the ptreconciler. func MakeStatusFunc( jr *jobs.Registry, ie sqlutil.InternalExecutor, metaType MetaType, ) ptreconcile.StatusFunc { diff --git a/pkg/kv/kvserver/protectedts/protectedts.go b/pkg/kv/kvserver/protectedts/protectedts.go index 9124d1bd8968..10349f7e9aa9 100644 --- a/pkg/kv/kvserver/protectedts/protectedts.go +++ b/pkg/kv/kvserver/protectedts/protectedts.go @@ -92,10 +92,10 @@ type Storage interface { // passed txn remains safe for future use. Release(context.Context, *kv.Txn, uuid.UUID) error - // GetMetadata retreives the metadata with the provided Txn. + // GetMetadata retrieves the metadata with the provided Txn. GetMetadata(context.Context, *kv.Txn) (ptpb.Metadata, error) - // GetState retreives the entire state of protectedts.Storage with the + // GetState retrieves the entire state of protectedts.Storage with the // provided Txn. GetState(context.Context, *kv.Txn) (ptpb.State, error) diff --git a/pkg/roachpb/span_config.go b/pkg/roachpb/span_config.go index ad3ebc198a27..698234da8eab 100644 --- a/pkg/roachpb/span_config.go +++ b/pkg/roachpb/span_config.go @@ -96,6 +96,18 @@ func (c ConstraintsConjunction) String() string { return sb.String() } +// String implements the stringer interface. +func (p ProtectionPolicy) String() string { + var sb strings.Builder + sb.WriteString(fmt.Sprintf("{ts: %d", int(p.ProtectedTimestamp.WallTime))) + if p.IgnoreIfExcludedFromBackup { + sb.WriteString(fmt.Sprintf(",ignore_if_excluded_from_backup: %t", + p.IgnoreIfExcludedFromBackup)) + } + sb.WriteString("}") + return sb.String() +} + // TestingDefaultSpanConfig exports the default span config for testing purposes. func TestingDefaultSpanConfig() SpanConfig { return SpanConfig{ diff --git a/pkg/roachpb/span_config.proto b/pkg/roachpb/span_config.proto index b326da31cc2f..360d57a330e3 100644 --- a/pkg/roachpb/span_config.proto +++ b/pkg/roachpb/span_config.proto @@ -43,11 +43,22 @@ message GCPolicy { // applies over a given span. message ProtectionPolicy { option (gogoproto.equal) = true; + option (gogoproto.goproto_stringer) = false; option (gogoproto.populate) = true; // ProtectedTimestamp is a timestamp above which GC should not run, regardless // of the GC TTL. util.hlc.Timestamp protected_timestamp = 1 [(gogoproto.nullable) = false]; + + // IgnoreIfExcludedFromBackup is set to true if the ProtectionPolicy can be + // ignored if the span it applies to has been marked to be excluded from + // backups i.e. the applied SpanConfig has `exclude_data_from_backup = true`. + // This field is currently only set to true when a protected timestamp record + // has been written by a backup job. This is to ensure that ProtectionPolicies + // written by non-backup users (CDC, streaming) on spans marked as + // `exclude_data_from_backup` are still respected when making GC decisions on + // the span. + bool ignore_if_excluded_from_backup = 2; } // Constraint constrains the stores that a replica can be stored on. It diff --git a/pkg/spanconfig/spanconfigsqltranslator/BUILD.bazel b/pkg/spanconfig/spanconfigsqltranslator/BUILD.bazel index cf62bf4147d8..f8496c5e44f8 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/BUILD.bazel +++ b/pkg/spanconfig/spanconfigsqltranslator/BUILD.bazel @@ -10,6 +10,9 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/config/zonepb", + "//pkg/jobs", + "//pkg/jobs/jobspb", + "//pkg/jobs/jobsprotectedts", "//pkg/keys", "//pkg/kv", "//pkg/kv/kvserver/protectedts/ptpb", @@ -28,16 +31,25 @@ go_library( go_test( name = "spanconfigsqltranslator_test", srcs = [ + "main_test.go", "protectedts_state_reader_test.go", "sqltranslator_test.go", ], data = glob(["testdata/**"]), embed = [":spanconfigsqltranslator"], deps = [ + "//pkg/base", + "//pkg/jobs", "//pkg/jobs/jobsprotectedts", + "//pkg/kv", "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/roachpb", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", "//pkg/sql/catalog/descpb", + "//pkg/testutils/serverutils", + "//pkg/testutils/testcluster", "//pkg/util/hlc", "//pkg/util/leaktest", "//pkg/util/log", diff --git a/pkg/spanconfig/spanconfigsqltranslator/main_test.go b/pkg/spanconfig/spanconfigsqltranslator/main_test.go new file mode 100644 index 000000000000..d6c5b5e90537 --- /dev/null +++ b/pkg/spanconfig/spanconfigsqltranslator/main_test.go @@ -0,0 +1,31 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package spanconfigsqltranslator_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" +) + +func TestMain(m *testing.M) { + security.SetAssetLoader(securitytest.EmbeddedAssets) + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} + +//go:generate ../../util/leaktest/add-leaktest.sh *_test.go diff --git a/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader.go b/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader.go index 10b16c7b0015..bdcb566730ef 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader.go +++ b/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader.go @@ -13,9 +13,14 @@ package spanconfigsqltranslator import ( "context" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/jobs/jobsprotectedts" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/errors" ) // protectedTimestampStateReader provides a target specific view of the @@ -31,15 +36,15 @@ type protectedTimestampStateReader struct { // protected timestamp records given the supplied ptpb.State. The ptpb.State is // the transactional state of the `system.protected_ts_records` table. func newProtectedTimestampStateReader( - _ context.Context, ptsState ptpb.State, -) *protectedTimestampStateReader { + ctx context.Context, jr *jobs.Registry, txn *kv.Txn, ptsState ptpb.State, +) (*protectedTimestampStateReader, error) { reader := &protectedTimestampStateReader{ schemaObjectProtections: make(map[descpb.ID][]roachpb.ProtectionPolicy), tenantProtections: make([]tenantProtectedTimestamps, 0), clusterProtections: make([]roachpb.ProtectionPolicy, 0), } - reader.loadProtectedTimestampRecords(ptsState) - return reader + err := reader.loadProtectedTimestampRecords(ctx, jr, txn, ptsState) + return reader, err } // getProtectionPoliciesForCluster returns all the protected timestamps that @@ -70,22 +75,32 @@ func (p *protectedTimestampStateReader) getProtectionPoliciesForSchemaObject( return p.schemaObjectProtections[descID] } -func (p *protectedTimestampStateReader) loadProtectedTimestampRecords(ptsState ptpb.State) { +func (p *protectedTimestampStateReader) loadProtectedTimestampRecords( + ctx context.Context, jr *jobs.Registry, txn *kv.Txn, ptsState ptpb.State, +) error { tenantProtections := make(map[roachpb.TenantID][]roachpb.ProtectionPolicy) for _, record := range ptsState.Records { + // TODO(adityamaru): When schedules start writing protected timestamp + // records extend this logic to also include those records. + isBackupProtectedTimestampRecord, err := jobsprotectedts.IsRecordWrittenByJobType(ctx, jr, txn, + record.Meta, jobsprotectedts.Jobs, jobspb.TypeBackup) + if err != nil { + return errors.Wrap(err, "failed to check if record was written by a BACKUP") + } + protectionPolicy := roachpb.ProtectionPolicy{ + ProtectedTimestamp: record.Timestamp, + IgnoreIfExcludedFromBackup: isBackupProtectedTimestampRecord, + } switch t := record.Target.GetUnion().(type) { case *ptpb.Target_Cluster: - p.clusterProtections = append(p.clusterProtections, - roachpb.ProtectionPolicy{ProtectedTimestamp: record.Timestamp}) + p.clusterProtections = append(p.clusterProtections, protectionPolicy) case *ptpb.Target_Tenants: for _, tenID := range t.Tenants.IDs { - tenantProtections[tenID] = append(tenantProtections[tenID], - roachpb.ProtectionPolicy{ProtectedTimestamp: record.Timestamp}) + tenantProtections[tenID] = append(tenantProtections[tenID], protectionPolicy) } case *ptpb.Target_SchemaObjects: for _, descID := range t.SchemaObjects.IDs { - p.schemaObjectProtections[descID] = append(p.schemaObjectProtections[descID], - roachpb.ProtectionPolicy{ProtectedTimestamp: record.Timestamp}) + p.schemaObjectProtections[descID] = append(p.schemaObjectProtections[descID], protectionPolicy) } } } @@ -94,4 +109,5 @@ func (p *protectedTimestampStateReader) loadProtectedTimestampRecords(ptsState p p.tenantProtections = append(p.tenantProtections, tenantProtectedTimestamps{tenantID: tenID, protections: tenantProtections}) } + return nil } diff --git a/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader_test.go b/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader_test.go index 0ff58332dc29..ef974ab1c41c 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader_test.go +++ b/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader_test.go @@ -16,10 +16,14 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobsprotectedts" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -31,6 +35,10 @@ func TestProtectedTimestampStateReader(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + s, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(context.Background()) + jr := s.JobRegistry().(*jobs.Registry) + mkRecordAndAddToState := func(state *ptpb.State, ts hlc.Timestamp, target *ptpb.Target) { recordID := uuid.MakeV4() rec := jobsprotectedts.MakeRecord(recordID, int64(1), ts, nil, /* deprecatedSpans */ @@ -64,7 +72,14 @@ func TestProtectedTimestampStateReader(t *testing.T) { protectTenants(state, ts(5), []roachpb.TenantID{roachpb.MakeTenantID(2)}) protectTenants(state, ts(6), []roachpb.TenantID{roachpb.MakeTenantID(2)}) - ptsStateReader := newProtectedTimestampStateReader(context.Background(), *state) + ctx := context.Background() + var ptsStateReader *protectedTimestampStateReader + require.NoError(t, s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + var err error + ptsStateReader, err = newProtectedTimestampStateReader(context.Background(), jr, txn, *state) + require.NoError(t, err) + return nil + })) clusterTimestamps := ptsStateReader.getProtectionPoliciesForCluster() require.Len(t, clusterTimestamps, 1) require.Equal(t, []roachpb.ProtectionPolicy{{ProtectedTimestamp: ts(3)}}, clusterTimestamps) diff --git a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go index 2918547edb92..acdb3e6933d6 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go +++ b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go @@ -82,7 +82,10 @@ func (s *SQLTranslator) Translate( if err != nil { return errors.Wrap(err, "failed to get protected timestamp state") } - ptsStateReader := newProtectedTimestampStateReader(ctx, ptsState) + ptsStateReader, err := newProtectedTimestampStateReader(ctx, s.execCfg.JobRegistry, txn, ptsState) + if err != nil { + return errors.Wrap(err, "failed to create a ProtectedTimestampStateReader") + } // For every ID we want to translate, first expand it to descendant leaf // IDs that have span configurations associated for them. We also diff --git a/pkg/spanconfig/spanconfigtestutils/utils.go b/pkg/spanconfig/spanconfigtestutils/utils.go index 5ae42e7a398e..eab37c50713c 100644 --- a/pkg/spanconfig/spanconfigtestutils/utils.go +++ b/pkg/spanconfig/spanconfigtestutils/utils.go @@ -331,11 +331,11 @@ func PrintSpanConfigDiffedAgainstDefaults(conf roachpb.SpanConfig) string { rhs := conf.GCPolicy.ProtectionPolicies[j].ProtectedTimestamp return lhs.Less(rhs) }) - timestamps := make([]string, 0, len(conf.GCPolicy.ProtectionPolicies)) - for _, pts := range conf.GCPolicy.ProtectionPolicies { - timestamps = append(timestamps, strconv.Itoa(int(pts.ProtectedTimestamp.WallTime))) + protectionPolicies := make([]string, 0, len(conf.GCPolicy.ProtectionPolicies)) + for _, pp := range conf.GCPolicy.ProtectionPolicies { + protectionPolicies = append(protectionPolicies, pp.String()) } - diffs = append(diffs, fmt.Sprintf("pts=[%s]", strings.Join(timestamps, " "))) + diffs = append(diffs, fmt.Sprintf("protection_policies=[%s]", strings.Join(protectionPolicies, " "))) } if conf.ExcludeDataFromBackup != defaultConf.ExcludeDataFromBackup { diffs = append(diffs, fmt.Sprintf("exclude_data_from_backup=%v", conf.ExcludeDataFromBackup))