Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
85282: backupccl: check crdb_internal.invalid_objects during backup test cleanup r=adityamaru a=msbutler

This patch adds a test helper function to easily check for invalid descriptors
during the cleanup of a test cluster created by any of the
backupRestoreTestSetup* helper funcs. This setup is used by many backup unit
tests (including all data driven tests), but a future PR should include this
clean up helper func in any backup unit test that doesn't include use
these helper funcs.

It's also worth noting this check would have caught #76764.

Informs #84757

Release note: none

87820: sql/schemachanger/*: optimize performance, primarily by adding support for containment r=ajwerner a=ajwerner

#### screl,scplan/rule: adopt containment

#### sql/schemachanger/rel: support inverted indexes and slice containment

This commit introduces some new concepts to rel:
* Slice attributes
* Inverted indexing of slice attributes
* Containment queries over slice attributes

One note is that the only support for containment queries is via an inverted
index. In the future, we could add direct containment evaluation such that if
we have a slice we could go iterate its members.

The basic idea is to introduce a slice membership type for the slice column
and to use the attribute in question to refer to the slice member value.

#### sql/schemachanger/rel: fix embedding support

#### sql/schemachanger/rel: optimize error checking
This change ends up making a huge difference. Otherwise, each subquery
invocation would lead to allocations which showed up in the profiles.

Fixes #86042

Release note: None

88712: sql: enable `IdempotentTombstone` for schema GC r=ajwerner a=erikgrinaker

`@ajwerner` Looks like we forgot to enable this. Let's get it in before the backport freeze.

---

This will avoid writing MVCC range tombstones across ranges if they don't contain any live data. This is particularly useful to avoid writing additional tombstones on retries.

Release note: None

Co-authored-by: Michael Butler <butler@cockroachlabs.com>
Co-authored-by: Andrew Werner <awerner32@gmail.com>
Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
  • Loading branch information
4 people committed Sep 26, 2022
4 parents eaa0f86 + 791e981 + 67a09d3 + 3dfb165 commit 18b15cf
Show file tree
Hide file tree
Showing 73 changed files with 1,010 additions and 341 deletions.
26 changes: 26 additions & 0 deletions pkg/ccl/backupccl/backuputils/testutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -172,3 +172,29 @@ func VerifyBackupRestoreStatementResult(

return nil
}

// CheckForInvalidDescriptors returns an error if there exists any descriptors in
// the crdb_internal.invalid_objects virtual table.
func CheckForInvalidDescriptors(t testing.TB, sqlDB *gosql.DB) {
// Ensure the connection to the database is still open.
if err := sqlDB.Ping(); err != nil {
t.Logf("Warning: Could not check for invalid descriptors: %v", err)
return
}
// Because crdb_internal.invalid_objects is a virtual table, by default, the
// query will take a lease on the database sqlDB is connected to and only run
// the query on the given database. The "" prefix prevents this lease
// acquisition and allows the query to fetch all descriptors in the cluster.
rows, err := sqlDB.Query(`SELECT id, obj_name, error FROM "".crdb_internal.invalid_objects`)
if err != nil {
t.Fatal(err)
}
invalidIDs, err := sqlutils.RowsToDataDrivenOutput(rows)
if err != nil {
t.Error(err)
}
if invalidIDs != "" {
t.Fatalf("the following descriptor ids are invalid\n%v", invalidIDs)
}
t.Log("no Invalid Descriptors")
}
11 changes: 8 additions & 3 deletions pkg/ccl/backupccl/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuputils"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
Expand Down Expand Up @@ -105,8 +106,12 @@ func newDatadrivenTestState() datadrivenTestState {
}
}

func (d *datadrivenTestState) cleanup(ctx context.Context) {
func (d *datadrivenTestState) cleanup(ctx context.Context, t *testing.T) {
// While the testCluster cleanupFns would close the dbConn and servers, close
// them manually to ensure all queries finish on tests that share these
// resources.
for _, db := range d.sqlDBs {
backuputils.CheckForInvalidDescriptors(t, db)
db.Close()
}
for _, s := range d.servers {
Expand Down Expand Up @@ -379,7 +384,7 @@ func TestDataDriven(t *testing.T) {
datadriven.Walk(t, testutils.TestDataPath(t, "backup-restore"), func(t *testing.T, path string) {
var lastCreatedServer string
ds := newDatadrivenTestState()
defer ds.cleanup(ctx)
defer ds.cleanup(ctx, t)
datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string {

execWithTagAndPausePoint := func(jobType jobspb.Type) string {
Expand Down Expand Up @@ -436,7 +441,7 @@ func TestDataDriven(t *testing.T) {
return ""

case "reset":
ds.cleanup(ctx)
ds.cleanup(ctx, t)
ds = newDatadrivenTestState()
return ""

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/restore_mid_schema_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -209,7 +209,7 @@ func verifyMidSchemaChange(
expNumSchemaChangeJobs := expectedSCJobCount(scName, majorVer)

synthesizedSchemaChangeJobs := sqlDB.QueryStr(t,
"SELECT description FROM crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE' AND description LIKE '%RESTORING%'")
`SELECT description FROM "".crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE' AND description LIKE '%RESTORING%'`)
require.Equal(t, expNumSchemaChangeJobs, len(synthesizedSchemaChangeJobs),
"Expected %d schema change jobs but found %v", expNumSchemaChangeJobs, synthesizedSchemaChangeJobs)

Expand Down
4 changes: 4 additions & 0 deletions pkg/ccl/backupccl/utils_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupbase"
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo"
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb"
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuputils"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
Expand Down Expand Up @@ -121,6 +122,7 @@ func backupRestoreTestSetupWithParams(
}

cleanupFn := func() {
backuputils.CheckForInvalidDescriptors(t, tc.Conns[0])
tc.Stopper().Stop(ctx) // cleans up in memory storage's auxiliary dirs
dirCleanupFn() // cleans up dir, which is the nodelocal:// storage
}
Expand Down Expand Up @@ -172,6 +174,7 @@ func backupRestoreTestSetupEmptyWithParams(
sqlDB = sqlutils.MakeSQLRunner(tc.Conns[0])

cleanupFn := func() {
backuputils.CheckForInvalidDescriptors(t, tc.Conns[0])
tc.Stopper().Stop(ctx) // cleans up in memory storage's auxiliary dirs
}

Expand All @@ -194,6 +197,7 @@ func createEmptyCluster(
sqlDB = sqlutils.MakeSQLRunner(tc.Conns[0])

cleanupFn := func() {
backuputils.CheckForInvalidDescriptors(t, tc.Conns[0])
tc.Stopper().Stop(ctx) // cleans up in memory storage's auxiliary dirs
dirCleanupFn() // cleans up dir, which is the nodelocal:// storage
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ Schema change plan for DROP DATABASE ‹multi_region_test_db› CASCADE;
│ │ ├── PUBLIC → TXN_DROPPED Database:{DescID: 104}
│ │ ├── PUBLIC → TXN_DROPPED Schema:{DescID: 106}
│ │ ├── PUBLIC → TXN_DROPPED EnumType:{DescID: 105}
│ │ ├── PUBLIC → TXN_DROPPED AliasType:{DescID: 107}
│ │ ├── PUBLIC → TXN_DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]}
│ │ ├── PUBLIC → TXN_DROPPED Table:{DescID: 108}
│ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 1}
│ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295}
Expand Down Expand Up @@ -59,7 +59,7 @@ Schema change plan for DROP DATABASE ‹multi_region_test_db› CASCADE;
│ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: admin}
│ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: public}
│ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: root}
│ │ ├── TXN_DROPPED → DROPPED AliasType:{DescID: 107}
│ │ ├── TXN_DROPPED → DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]}
│ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 107, ReferencedDescID: 106}
│ │ ├── PUBLIC → ABSENT Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104}
│ │ ├── PUBLIC → ABSENT Owner:{DescID: 108}
Expand Down Expand Up @@ -108,7 +108,7 @@ Schema change plan for DROP DATABASE ‹multi_region_test_db› CASCADE;
│ ├── DROPPED → ABSENT Database:{DescID: 104}
│ ├── DROPPED → ABSENT Schema:{DescID: 106}
│ ├── DROPPED → ABSENT EnumType:{DescID: 105}
│ ├── DROPPED → ABSENT AliasType:{DescID: 107}
│ ├── DROPPED → ABSENT AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]}
│ ├── DROPPED → ABSENT Table:{DescID: 108}
│ ├── DELETE_ONLY → ABSENT Column:{DescID: 108, ColumnID: 1}
│ ├── DELETE_ONLY → ABSENT Column:{DescID: 108, ColumnID: 4294967295}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,8 +35,8 @@ Schema change plan for DROP TABLE ‹multi_region_test_db›.‹public›.‹tab
│ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}
│ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 108, ColumnID: 2}
│ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_region, ColumnID: 2}
│ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}
│ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 108, ColumnID: 2}
│ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ReferencedTypeIDs: [105 107], ColumnFamilyID: 0, ColumnID: 2}
│ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [105 107], ColumnID: 2}
│ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 108, ColumnID: 4294967295}
│ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}
│ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE;
│ │ ├── • EnumType:{DescID: 105}
│ │ │ PUBLIC → TXN_DROPPED
│ │ │
│ │ ├── • AliasType:{DescID: 107}
│ │ ├── • AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]}
│ │ │ PUBLIC → TXN_DROPPED
│ │ │
│ │ ├── • Table:{DescID: 108}
Expand Down Expand Up @@ -293,13 +293,13 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE;
│ │ │ ├── • SameStagePrecedence dependency from DROPPED Database:{DescID: 104}
│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref"
│ │ │ │
│ │ │ └── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107}
│ │ │ └── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]}
│ │ │ rule: "descriptor drop right before dependent element removal"
│ │ │
│ │ ├── • Owner:{DescID: 107}
│ │ │ │ PUBLIC → ABSENT
│ │ │ │
│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107}
│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]}
│ │ │ │ rule: "descriptor drop right before dependent element removal"
│ │ │ │
│ │ │ └── • skip PUBLIC → ABSENT operations
Expand All @@ -308,7 +308,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE;
│ │ ├── • UserPrivileges:{DescID: 107, Name: admin}
│ │ │ │ PUBLIC → ABSENT
│ │ │ │
│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107}
│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]}
│ │ │ │ rule: "descriptor drop right before dependent element removal"
│ │ │ │
│ │ │ └── • skip PUBLIC → ABSENT operations
Expand All @@ -317,7 +317,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE;
│ │ ├── • UserPrivileges:{DescID: 107, Name: public}
│ │ │ │ PUBLIC → ABSENT
│ │ │ │
│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107}
│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]}
│ │ │ │ rule: "descriptor drop right before dependent element removal"
│ │ │ │
│ │ │ └── • skip PUBLIC → ABSENT operations
Expand All @@ -326,16 +326,16 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE;
│ │ ├── • UserPrivileges:{DescID: 107, Name: root}
│ │ │ │ PUBLIC → ABSENT
│ │ │ │
│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107}
│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]}
│ │ │ │ rule: "descriptor drop right before dependent element removal"
│ │ │ │
│ │ │ └── • skip PUBLIC → ABSENT operations
│ │ │ rule: "skip element removal ops on descriptor drop"
│ │ │
│ │ ├── • AliasType:{DescID: 107}
│ │ ├── • AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]}
│ │ │ │ TXN_DROPPED → DROPPED
│ │ │ │
│ │ │ └── • PreviousStagePrecedence dependency from TXN_DROPPED AliasType:{DescID: 107}
│ │ │ └── • PreviousStagePrecedence dependency from TXN_DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]}
│ │ │ rule: "descriptor TXN_DROPPED before DROPPED"
│ │ │
│ │ ├── • ObjectParent:{DescID: 107, ReferencedDescID: 106}
Expand All @@ -344,7 +344,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE;
│ │ │ ├── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 106}
│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref"
│ │ │ │
│ │ │ └── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107}
│ │ │ └── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]}
│ │ │ rule: "descriptor drop right before dependent element removal"
│ │ │
│ │ ├── • Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104}
Expand Down Expand Up @@ -661,10 +661,10 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE;
│ │ └── • PreviousTransactionPrecedence dependency from DROPPED EnumType:{DescID: 105}
│ │ rule: "descriptor DROPPED in transaction before removal"
│ │
│ ├── • AliasType:{DescID: 107}
│ ├── • AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]}
│ │ │ DROPPED → ABSENT
│ │ │
│ │ └── • PreviousTransactionPrecedence dependency from DROPPED AliasType:{DescID: 107}
│ │ └── • PreviousTransactionPrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]}
│ │ rule: "descriptor DROPPED in transaction before removal"
│ │
│ ├── • Table:{DescID: 108}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_
│ │ │ └── • skip PUBLIC → ABSENT operations
│ │ │ rule: "skip column dependents removal ops on relation drop"
│ │ │
│ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}
│ │ ├── • ColumnType:{DescID: 108, ReferencedTypeIDs: [105 107], ColumnFamilyID: 0, ColumnID: 2}
│ │ │ │ PUBLIC → ABSENT
│ │ │ │
│ │ │ ├── • SameStagePrecedence dependency from DROPPED Table:{DescID: 108}
Expand All @@ -173,10 +173,10 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_
│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 2}
│ │ │ │ rule: "column no longer public before dependents"
│ │ │ │
│ │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 108, ColumnID: 2}
│ │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [105 107], ColumnID: 2}
│ │ │ rule: "column type dependents removed right before column type"
│ │ │
│ │ ├── • ColumnDefaultExpression:{DescID: 108, ColumnID: 2}
│ │ ├── • ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [105 107], ColumnID: 2}
│ │ │ │ PUBLIC → ABSENT
│ │ │ │
│ │ │ ├── • SameStagePrecedence dependency from DROPPED Table:{DescID: 108}
Expand Down Expand Up @@ -402,10 +402,10 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_
│ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 108, Name: crdb_region, ColumnID: 2}
│ │ │ rule: "dependents removed before column"
│ │ │
│ │ ├── • Precedence dependency from ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}
│ │ ├── • Precedence dependency from ABSENT ColumnType:{DescID: 108, ReferencedTypeIDs: [105 107], ColumnFamilyID: 0, ColumnID: 2}
│ │ │ rule: "dependents removed before column"
│ │ │
│ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 108, ColumnID: 2}
│ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [105 107], ColumnID: 2}
│ │ │ rule: "dependents removed before column"
│ │ │
│ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/gcjob/table_garbage_collection.go
Original file line number Diff line number Diff line change
Expand Up @@ -254,6 +254,7 @@ func deleteAllSpanData(
EndKey: endKey.AsRawKey(),
},
UseRangeTombstone: true,
IdempotentTombstone: true,
UpdateRangeDeleteGCHint: true,
})
log.VEventf(ctx, 2, "delete range %s - %s", lastKey, endKey)
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/repair.go
Original file line number Diff line number Diff line change
Expand Up @@ -792,6 +792,7 @@ func (p *planner) ForceDeleteTableData(ctx context.Context, descID int64) error
b.AddRawRequest(&roachpb.DeleteRangeRequest{
RequestHeader: requestHeader,
UseRangeTombstone: true,
IdempotentTombstone: true,
UpdateRangeDeleteGCHint: true,
})
} else {
Expand Down
Loading

0 comments on commit 18b15cf

Please sign in to comment.