Skip to content

Commit ddf9096

Browse files
craig[bot]kev-caospilchen
committed
144808: backup: fix wrong URIs/localities on skip compaction chains r=msbutler a=kev-cao The current tests for `ValidateEndTimeAndTruncate` only check that the correct number of URIs and localities are returned, but do not check that the correct values are returned. This allowed a bug to slip by where if compactions were being skipped, the wrong URIs and localities would be returned. This patch fixes the bug and updates the tests to be more specific. Epic: None Release note: None 145341: sql/rls: exempt foreign key propagation from RLS enforcement r=spilchen a=spilchen Previously, RLS policies were applied during foreign key propagation, triggered by ON DELETE or ON UPDATE actions specified in foreign key definitions. This behavior was incorrect because foreign key maintenance operations must not be blocked by RLS to ensure data integrity. Postgres exempts such operations, and we now match that behavior. Fixes #145333 Epic: CRDB-11724 Release note: none Co-authored-by: Kevin Cao <[email protected]> Co-authored-by: Matt Spilchen <[email protected]>
3 parents 5be8397 + a465edf + 5ad8f5a commit ddf9096

File tree

9 files changed

+466
-23
lines changed

9 files changed

+466
-23
lines changed

pkg/backup/backupinfo/BUILD.bazel

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -38,7 +38,6 @@ go_library(
3838
"//pkg/sql/sem/tree",
3939
"//pkg/sql/stats",
4040
"//pkg/storage",
41-
"//pkg/util",
4241
"//pkg/util/bulk",
4342
"//pkg/util/ctxgroup",
4443
"//pkg/util/encoding",
@@ -81,6 +80,7 @@ go_test(
8180
"//pkg/sql/isql",
8281
"//pkg/testutils/serverutils",
8382
"//pkg/testutils/testcluster",
83+
"//pkg/util",
8484
"//pkg/util/bulk",
8585
"//pkg/util/hlc",
8686
"//pkg/util/leaktest",

pkg/backup/backupinfo/manifest_handling.go

Lines changed: 21 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,6 @@ import (
4242
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
4343
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
4444
"github.com/cockroachdb/cockroach/pkg/sql/stats"
45-
"github.com/cockroachdb/cockroach/pkg/util"
4645
"github.com/cockroachdb/cockroach/pkg/util/bulk"
4746
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
4847
"github.com/cockroachdb/cockroach/pkg/util/encoding"
@@ -890,11 +889,8 @@ func ValidateEndTimeAndTruncate(
890889
includeCompacted bool,
891890
) ([]string, []backuppb.BackupManifest, []jobspb.RestoreDetails_BackupLocalityInfo, error) {
892891
if !includeCompacted {
893-
mainBackupManifests = util.Filter(
894-
mainBackupManifests,
895-
func(manifest backuppb.BackupManifest) bool {
896-
return !manifest.IsCompacted
897-
},
892+
defaultURIs, mainBackupManifests, localityInfo = skipCompactedBackups(
893+
defaultURIs, mainBackupManifests, localityInfo,
898894
)
899895
}
900896

@@ -965,6 +961,25 @@ func ValidateEndTimeAndTruncate(
965961
)
966962
}
967963

964+
// skipCompactedBackups removes any compacted backups from the list of
965+
// backups and returns the updated list of URIs, manifests, and locality info.
966+
//
967+
// NOTE: This function modifies the underlying memory of the slices passed in.
968+
func skipCompactedBackups(
969+
defaultURIs []string,
970+
manifests []backuppb.BackupManifest,
971+
localityInfo []jobspb.RestoreDetails_BackupLocalityInfo,
972+
) ([]string, []backuppb.BackupManifest, []jobspb.RestoreDetails_BackupLocalityInfo) {
973+
for i := len(manifests) - 1; i >= 0; i-- {
974+
if manifests[i].IsCompacted {
975+
defaultURIs = slices.Delete(defaultURIs, i, i+1)
976+
manifests = slices.Delete(manifests, i, i+1)
977+
localityInfo = slices.Delete(localityInfo, i, i+1)
978+
}
979+
}
980+
return defaultURIs, manifests, localityInfo
981+
}
982+
968983
// validateContinuity checks that the backups are continuous.
969984
func validateContinuity(manifests []backuppb.BackupManifest) error {
970985
if len(manifests) == 0 {

pkg/backup/backupinfo/manifest_handling_test.go

Lines changed: 35 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,9 @@ package backupinfo_test
88
import (
99
"context"
1010
"fmt"
11+
"slices"
1112
"sort"
13+
"strconv"
1214
"testing"
1315

1416
"github.com/cockroachdb/cockroach/pkg/backup/backupinfo"
@@ -24,6 +26,7 @@ import (
2426
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
2527
"github.com/cockroachdb/cockroach/pkg/sql/isql"
2628
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
29+
"github.com/cockroachdb/cockroach/pkg/util"
2730
"github.com/cockroachdb/cockroach/pkg/util/bulk"
2831
"github.com/cockroachdb/cockroach/pkg/util/hlc"
2932
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
@@ -549,10 +552,32 @@ func TestValidateEndTimeAndTruncate(t *testing.T) {
549552
},
550553
} {
551554
t.Run(tc.name, func(t *testing.T) {
555+
// Using the expected start and end times, create a slice containing the
556+
// expected expectedOrder of the manifests based on their original indexes. For
557+
// example, [2, 0, 1] means that the expected output should should have
558+
// the third input manifest first, the first input manifest second, and the
559+
// second input manifest last.
560+
expectedOrder := util.Map(tc.expected, func(ts []int) string {
561+
return strconv.Itoa(slices.IndexFunc(tc.manifests, func(m backuppb.BackupManifest) bool {
562+
return m.StartTime.WallTime == int64(ts[0]) && m.EndTime.WallTime == int64(ts[1])
563+
}))
564+
})
565+
// Create URIs and locality info in a way that we can track their
566+
// resulting order after being truncated. We set input URIs to their
567+
// stringified index, and locality info maps to a map containing just
568+
// their index. We can later use the expectedOrder to check that the
569+
// output maps to the expected indexes.
570+
inputURIs := make([]string, len(tc.manifests))
571+
inputLocs := make([]jobspb.RestoreDetails_BackupLocalityInfo, len(tc.manifests))
572+
for i := range tc.manifests {
573+
index := strconv.Itoa(i)
574+
inputURIs[i] = index
575+
inputLocs[i].URIsByOriginalLocalityKV = make(map[string]string)
576+
inputLocs[i].URIsByOriginalLocalityKV[index] = index
577+
}
578+
552579
uris, res, locs, err := backupinfo.ValidateEndTimeAndTruncate(
553-
make([]string, len(tc.manifests)),
554-
tc.manifests,
555-
make([]jobspb.RestoreDetails_BackupLocalityInfo, len(tc.manifests)),
580+
inputURIs, tc.manifests, inputLocs,
556581
hlc.Timestamp{WallTime: int64(tc.endTime)},
557582
false, /* includeSkipped */
558583
tc.includeCompacted,
@@ -561,9 +586,14 @@ func TestValidateEndTimeAndTruncate(t *testing.T) {
561586
require.ErrorContains(t, err, tc.err)
562587
return
563588
}
564-
require.Equal(t, len(tc.expected), len(uris))
565-
require.Equal(t, len(tc.expected), len(locs))
566589
require.Equal(t, len(tc.expected), len(res))
590+
require.Equal(t, expectedOrder, uris)
591+
require.Len(t, locs, len(tc.expected))
592+
for idx, rank := range expectedOrder {
593+
_, ok := locs[idx].URIsByOriginalLocalityKV[rank]
594+
require.True(t, ok)
595+
}
596+
567597
for i := range tc.expected {
568598
actual := []int{int(res[i].StartTime.WallTime), int(res[i].EndTime.WallTime)}
569599
require.Equal(t, tc.expected[i], actual)

pkg/backup/compaction_test.go

Lines changed: 51 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -699,6 +699,48 @@ func TestCompactionCheckpointing(t *testing.T) {
699699
validateCompactedBackupForTables(t, db, []string{"bank"}, "'nodelocal://1/backup'", start, end, 2)
700700
}
701701

702+
func TestToggleCompactionForRestore(t *testing.T) {
703+
defer leaktest.AfterTest(t)()
704+
defer log.Scope(t).Close(t)
705+
706+
tc, db, _, cleanup := backupRestoreTestSetup(
707+
t, singleNode, 1 /* numAccounts */, InitManualReplication,
708+
)
709+
defer cleanup()
710+
start := getTime(t)
711+
db.Exec(t, fmt.Sprintf("BACKUP INTO 'nodelocal://1/backup' AS OF SYSTEM TIME '%d'", start))
712+
db.Exec(t, "BACKUP INTO LATEST IN 'nodelocal://1/backup'")
713+
end := getTime(t)
714+
db.Exec(t, fmt.Sprintf("BACKUP INTO LATEST IN 'nodelocal://1/backup' AS OF SYSTEM TIME '%d'", end))
715+
var backupPath string
716+
db.QueryRow(t, "SHOW BACKUPS IN 'nodelocal://1/backup'").Scan(&backupPath)
717+
var compactionID jobspb.JobID
718+
db.QueryRow(
719+
t,
720+
fmt.Sprintf(
721+
`SELECT crdb_internal.backup_compaction(
722+
'BACKUP INTO LATEST IN ''nodelocal://1/backup''',
723+
'%s', %d::DECIMAL, %d::DECIMAL
724+
)`,
725+
backupPath, start, end,
726+
),
727+
).Scan(&compactionID)
728+
waitForSuccessfulJob(t, tc, compactionID)
729+
730+
var compRestoreID, classicRestoreID jobspb.JobID
731+
var unused any
732+
db.QueryRow(
733+
t, "RESTORE DATABASE data FROM LATEST IN 'nodelocal://1/backup' WITH new_db_name = 'data1'",
734+
).Scan(&compRestoreID, &unused, &unused, &unused)
735+
db.Exec(t, "SET CLUSTER SETTING restore.compacted_backups.enabled = false")
736+
db.QueryRow(
737+
t, "RESTORE DATABASE data FROM LATEST IN 'nodelocal://1/backup' WITH new_db_name = 'data2'",
738+
).Scan(&classicRestoreID, &unused, &unused, &unused)
739+
740+
require.Equal(t, 2, getNumBackupsInRestore(t, db, compRestoreID))
741+
require.Equal(t, 3, getNumBackupsInRestore(t, db, classicRestoreID))
742+
}
743+
702744
// validateCompactedBackupForTables is a wrapper around
703745
// validateCompactedBackupForTablesWithOpts that passes in no options.
704746
func validateCompactedBackupForTables(
@@ -747,7 +789,13 @@ func validateCompactedBackupForTablesWithOpts(
747789
restoredRows := db.QueryStr(t, "SELECT * FROM "+table)
748790
require.Equal(t, originalRows, restoredRows, "table %s", table)
749791
}
750-
// Check that the number of backups used in the restore is correct.
792+
793+
require.Equal(t, numBackups, getNumBackupsInRestore(t, db, restoreJobID))
794+
}
795+
796+
// getNumBackupsInRestore returns the number of backups used in the restore
797+
func getNumBackupsInRestore(t *testing.T, db *sqlutils.SQLRunner, jobID jobspb.JobID) int {
798+
t.Helper()
751799
var detailsStr string
752800
db.QueryRow(t, `SELECT crdb_internal.pb_to_json(
753801
'cockroach.sql.jobs.jobspb.Payload',
@@ -756,10 +804,10 @@ func validateCompactedBackupForTablesWithOpts(
756804
FROM system.job_info
757805
WHERE job_id = $1
758806
AND info_key = 'legacy_payload';
759-
`, restoreJobID).Scan(&detailsStr)
807+
`, jobID).Scan(&detailsStr)
760808
var details jobspb.RestoreDetails
761809
require.NoError(t, json.Unmarshal([]byte(detailsStr), &details))
762-
require.Equal(t, numBackups, len(details.URIs))
810+
return len(details.URIs)
763811
}
764812

765813
// ensureBackupExists ensures that a backup exists that spans the given start and end times.

pkg/sql/logictest/testdata/logic_test/row_level_security

Lines changed: 86 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1955,6 +1955,92 @@ DROP TABLE parent;
19551955
statement ok
19561956
DROP USER fk_user;
19571957

1958+
# Test FK propagation with RLS enabled on child table.
1959+
subtest fk_cascade
1960+
1961+
statement ok
1962+
CREATE TABLE customers (
1963+
id INT PRIMARY KEY,
1964+
name TEXT
1965+
);
1966+
1967+
statement ok
1968+
CREATE TABLE orders (
1969+
id INT PRIMARY KEY,
1970+
customer_id INT REFERENCES customers(id) ON UPDATE CASCADE ON DELETE SET NULL
1971+
);
1972+
1973+
statement ok
1974+
ALTER TABLE orders ENABLE ROW LEVEL SECURITY;
1975+
1976+
statement ok
1977+
INSERT INTO customers VALUES (1, 'bob');
1978+
1979+
statement ok
1980+
INSERT INTO orders VALUES (1000, 1), (1001, 1);
1981+
1982+
statement ok
1983+
CREATE USER u1;
1984+
1985+
statement ok
1986+
GRANT ALL ON orders, customers TO u1;
1987+
1988+
statement ok
1989+
SET ROLE u1;
1990+
1991+
# Verify u1 cannot ready anything from orders
1992+
query II
1993+
SELECT id, customer_id FROM orders
1994+
----
1995+
1996+
# Update the customer ID. This should succeed and cascade to orders.
1997+
query IT
1998+
UPDATE customers SET id = 2 WHERE id = 1 RETURNING id, name
1999+
----
2000+
2 bob
2001+
2002+
statement ok
2003+
RESET ROLE
2004+
2005+
query II
2006+
SELECT id, customer_id FROM orders ORDER BY id
2007+
----
2008+
1000 2
2009+
1001 2
2010+
2011+
statement ok
2012+
SET ROLE u1;
2013+
2014+
# Delete the customer. This should set customer_id in orders to NULL.
2015+
query IT
2016+
DELETE FROM customers WHERE id = 2 RETURNING id, name
2017+
----
2018+
2 bob
2019+
2020+
# Try to validate oders as u1, but invisible due to RLS
2021+
query IT
2022+
SELECT id, customer_id FROM orders ORDER BY id
2023+
----
2024+
2025+
statement ok
2026+
RESET ROLE;
2027+
2028+
# validate as the root user, should see the cascaded update
2029+
query II
2030+
SELECT id, customer_id FROM orders ORDER BY id
2031+
----
2032+
1000 NULL
2033+
1001 NULL
2034+
2035+
statement ok
2036+
DROP TABLE orders;
2037+
2038+
statement ok
2039+
DROP TABLE customers;
2040+
2041+
statement ok
2042+
DROP USER u1;
2043+
19582044
# Ensure CHECK constraints can work alongside RLS policies
19592045
subtest check_constraint
19602046

pkg/sql/opt/optbuilder/fk_cascade.go

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -527,7 +527,8 @@ func (cb *onDeleteSetBuilder) Build(
527527
// against the parent we are cascading from. Need to investigate in which
528528
// cases this is safe (e.g. other cascades could have messed with the parent
529529
// table in the meantime).
530-
mb.buildUpdate(nil /* returning */)
530+
// The exempt policy is used for RLS to maintain data integrity.
531+
mb.buildUpdate(nil /* returning */, cat.PolicyScopeExempt)
531532
return mb.outScope.expr
532533
})
533534
}
@@ -783,7 +784,8 @@ func (cb *onUpdateCascadeBuilder) Build(
783784
// Cascades can fire triggers on the child table.
784785
mb.buildRowLevelBeforeTriggers(tree.TriggerEventUpdate, true /* cascade */)
785786

786-
mb.buildUpdate(nil /* returning */)
787+
// The exempt policy is used for RLS to maintain data integrity.
788+
mb.buildUpdate(nil /* returning */, cat.PolicyScopeExempt)
787789
return mb.outScope.expr
788790
})
789791
}

pkg/sql/opt/optbuilder/select.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -771,7 +771,8 @@ func (b *Builder) buildScan(
771771
}
772772

773773
// Apply any filters required to enforce RLS policies. This must be done
774-
// after projecting out virtual columns, in case any policies reference them.
774+
// after adding projections for virtual columns, in case any policies
775+
// reference them.
775776
b.addRowLevelSecurityFilter(tabMeta, outScope, policyCommandScope)
776777

777778
if b.trackSchemaDeps {

0 commit comments

Comments
 (0)