Skip to content

Commit

Permalink
sql: rename TenantInfo.State to DataState, "ACTIVE" to "READY"
Browse files Browse the repository at this point in the history
We've discovered that we'd like to separate the readiness of the data
from the activation of the service. To emphasize this, this commit
renames the field "State" to "DataState".

Additionally, the state "ACTIVE" was confusing as it suggests that
something is running, whereas it merely marks the tenant data as ready
for use. So this commit also renames that state accordingly.

Release note: None
  • Loading branch information
knz committed Jan 21, 2023
1 parent b21379b commit 4d88e2d
Show file tree
Hide file tree
Showing 20 changed files with 137 additions and 130 deletions.
36 changes: 18 additions & 18 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6885,7 +6885,7 @@ func TestBackupRestoreTenant(t *testing.T) {
{`1`,
`true`,
`system`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"}`,
},
})
restoreDB.Exec(t, `RESTORE TENANT 10 FROM 'nodelocal://1/t10'`)
Expand All @@ -6896,13 +6896,13 @@ func TestBackupRestoreTenant(t *testing.T) {
`1`,
`true`,
`system`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"}`,
},
{
`10`,
`true`,
`tenant-10`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-10", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "10", "name": "tenant-10", "tenantReplicationJobId": "0"}`,
},
},
)
Expand Down Expand Up @@ -6936,13 +6936,13 @@ func TestBackupRestoreTenant(t *testing.T) {
`1`,
`true`,
`system`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"}`,
},
{
`10`,
`false`,
`NULL`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "tenant-10", "id": "10", "name": "", "state": "DROP", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "DROP", "droppedName": "tenant-10", "id": "10", "name": "", "tenantReplicationJobId": "0"}`,
},
},
)
Expand Down Expand Up @@ -6971,13 +6971,13 @@ func TestBackupRestoreTenant(t *testing.T) {
`1`,
`true`,
`system`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"}`,
},
{
`10`,
`true`,
`tenant-10`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-10", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "10", "name": "tenant-10", "tenantReplicationJobId": "0"}`,
},
},
)
Expand Down Expand Up @@ -7006,7 +7006,7 @@ func TestBackupRestoreTenant(t *testing.T) {
`1`,
`true`,
`system`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"}`,
},
})
restoreDB.Exec(t, `RESTORE TENANT 10 FROM 'nodelocal://1/t10'`)
Expand All @@ -7017,13 +7017,13 @@ func TestBackupRestoreTenant(t *testing.T) {
`1`,
`true`,
`system`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"}`,
},
{
`10`,
`true`,
`tenant-10`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-10", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "10", "name": "tenant-10", "tenantReplicationJobId": "0"}`,
},
},
)
Expand All @@ -7050,7 +7050,7 @@ func TestBackupRestoreTenant(t *testing.T) {
`1`,
`true`,
`system`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"}`,
},
})
restoreDB.Exec(t, `RESTORE TENANT 10 FROM 'nodelocal://1/clusterwide'`)
Expand All @@ -7061,13 +7061,13 @@ func TestBackupRestoreTenant(t *testing.T) {
`1`,
`true`,
`system`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"}`,
},
{
`10`,
`true`,
`tenant-10`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-10", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "10", "name": "tenant-10", "tenantReplicationJobId": "0"}`,
},
},
)
Expand Down Expand Up @@ -7105,7 +7105,7 @@ func TestBackupRestoreTenant(t *testing.T) {
`1`,
`true`,
`system`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"}`,
},
})
restoreDB.Exec(t, `RESTORE FROM 'nodelocal://1/clusterwide'`)
Expand All @@ -7115,25 +7115,25 @@ func TestBackupRestoreTenant(t *testing.T) {
{
`1`,
`true`, `system`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"}`,
},
{
`10`,
`true`,
`tenant-10`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-10", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "10", "name": "tenant-10", "tenantReplicationJobId": "0"}`,
},
{
`11`,
`true`,
`tenant-11`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "11", "name": "tenant-11", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "11", "name": "tenant-11", "tenantReplicationJobId": "0"}`,
},
{
`20`,
`true`,
`tenant-20`,
`{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "20", "name": "tenant-20", "state": "ACTIVE", "tenantReplicationJobId": "0"}`,
`{"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "20", "name": "tenant-20", "tenantReplicationJobId": "0"}`,
},
},
)
Expand Down
22 changes: 11 additions & 11 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -1199,17 +1199,17 @@ func createImportingDescriptors(
return err
}
for _, tenant := range details.Tenants {
switch tenant.State {
case descpb.TenantInfo_ACTIVE:
// If the tenant was backed up in an `ACTIVE` state then we create
// the restored record in an `ADDING` state and mark it `ACTIVE` at
switch tenant.DataState {
case descpb.TenantInfo_READY:
// If the tenant was backed up in the `READY` state then we create
// the restored record in an `ADD` state and mark it `READY` at
// the end of the restore.
tenant.State = descpb.TenantInfo_ADD
tenant.DataState = descpb.TenantInfo_ADD
case descpb.TenantInfo_DROP, descpb.TenantInfo_ADD:
// If the tenant was backed up in a `DROP` or `ADD` state then we must
// create the restored tenant record in that state as well.
default:
return errors.AssertionFailedf("unknown tenant state %v", tenant)
return errors.AssertionFailedf("unknown tenant data state %v", tenant)
}
spanConfigs := p.ExecCfg().SpanConfigKVAccessor.WithTxn(ctx, txn.KV())
if _, err := sql.CreateTenantRecord(
Expand Down Expand Up @@ -2191,9 +2191,9 @@ func (r *restoreResumer) publishDescriptors(
}

for _, tenant := range details.Tenants {
switch tenant.State {
case descpb.TenantInfo_ACTIVE:
// If the tenant was backed up in an `ACTIVE` state then we must activate
switch tenant.DataState {
case descpb.TenantInfo_READY:
// If the tenant was backed up in the `READY` state then we must activate
// the tenant as the final step of the restore. The tenant has already
// been created at an earlier stage in the restore in an `ADD` state.
if err := sql.ActivateTenant(
Expand All @@ -2205,7 +2205,7 @@ func (r *restoreResumer) publishDescriptors(
// If the tenant was backed up in a `DROP` or `ADD` state then we do not
// want to activate the tenant.
default:
return errors.AssertionFailedf("unknown tenant state %v", tenant)
return errors.AssertionFailedf("unknown tenant data state %v", tenant)
}
}

Expand Down Expand Up @@ -2325,7 +2325,7 @@ func (r *restoreResumer) OnFailOrCancel(
ctx context.Context, txn isql.Txn,
) error {
for _, tenant := range details.Tenants {
tenant.State = descpb.TenantInfo_DROP
tenant.DataState = descpb.TenantInfo_DROP
// This is already a job so no need to spin up a gc job for the tenant;
// instead just GC the data eagerly.
if err := sql.GCTenantSync(ctx, execCfg, &tenant.TenantInfo); err != nil {
Expand Down
20 changes: 10 additions & 10 deletions pkg/ccl/backupccl/testdata/backup-restore/restore-tenants
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,9 @@ DROP TENANT [5]
query-sql
SELECT id,active,crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) FROM system.tenants;
----
1 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}
5 false {"capabilities": {"canAdminSplit": false}, "droppedName": "tenant-5", "id": "5", "name": "", "state": "DROP", "tenantReplicationJobId": "0"}
6 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "6", "name": "tenant-6", "state": "ACTIVE", "tenantReplicationJobId": "0"}
1 true {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"}
5 false {"capabilities": {"canAdminSplit": false}, "dataState": "DROP", "droppedName": "tenant-5", "id": "5", "name": "", "tenantReplicationJobId": "0"}
6 true {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "6", "name": "tenant-6", "tenantReplicationJobId": "0"}

exec-sql
BACKUP INTO 'nodelocal://1/cluster'
Expand All @@ -49,9 +49,9 @@ RESTORE FROM LATEST IN 'nodelocal://1/cluster'
query-sql
SELECT id,active,crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) FROM system.tenants;
----
1 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}
5 false {"capabilities": {"canAdminSplit": false}, "droppedName": "tenant-5", "id": "5", "name": "tenant-5", "state": "DROP", "tenantReplicationJobId": "0"}
6 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "6", "name": "tenant-6", "state": "ACTIVE", "tenantReplicationJobId": "0"}
1 true {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"}
5 false {"capabilities": {"canAdminSplit": false}, "dataState": "DROP", "droppedName": "tenant-5", "id": "5", "name": "tenant-5", "tenantReplicationJobId": "0"}
6 true {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "6", "name": "tenant-6", "tenantReplicationJobId": "0"}

exec-sql expect-error-regex=(tenant 6 already exists)
RESTORE TENANT 6 FROM LATEST IN 'nodelocal://1/tenant6';
Expand All @@ -75,7 +75,7 @@ RESTORE TENANT 6 FROM LATEST IN 'nodelocal://1/tenant6' WITH tenant_name = 'newn
query-sql
SELECT id,active,crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) FROM system.tenants;
----
1 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}
2 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "2", "name": "newname", "state": "ACTIVE", "tenantReplicationJobId": "0"}
5 false {"capabilities": {"canAdminSplit": false}, "droppedName": "tenant-5", "id": "5", "name": "tenant-5", "state": "DROP", "tenantReplicationJobId": "0"}
6 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "6", "name": "tenant-6", "state": "ACTIVE", "tenantReplicationJobId": "0"}
1 true {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"}
2 true {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "2", "name": "newname", "tenantReplicationJobId": "0"}
5 false {"capabilities": {"canAdminSplit": false}, "dataState": "DROP", "droppedName": "tenant-5", "id": "5", "name": "tenant-5", "tenantReplicationJobId": "0"}
6 true {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "6", "name": "tenant-6", "tenantReplicationJobId": "0"}
Original file line number Diff line number Diff line change
Expand Up @@ -428,7 +428,7 @@ func TestTenantStreamingDropTenantCancelsStream(t *testing.T) {
}

// Set GC TTL low, so that the GC job completes quickly in the test.
c.DestSysSQL.Exec(t, "ALTER RANGE tenants CONFIGURE ZONE USING gc.Ttlseconds = 1;")
c.DestSysSQL.Exec(t, "ALTER RANGE tenants CONFIGURE ZONE USING gc.ttlseconds = 1;")
c.DestSysSQL.Exec(t, fmt.Sprintf("DROP TENANT %s", c.Args.DestTenantName))
jobutils.WaitForJobToCancel(c.T, c.DestSysSQL, jobspb.JobID(ingestionJobID))
jobutils.WaitForJobToCancel(c.T, c.SrcSysSQL, jobspb.JobID(producerJobID))
Expand Down Expand Up @@ -854,7 +854,7 @@ func TestTenantReplicationProtectedTimestampManagement(t *testing.T) {
}

// Set GC TTL low, so that the GC job completes quickly in the test.
c.DestSysSQL.Exec(t, "ALTER RANGE tenants CONFIGURE ZONE USING gc.Ttlseconds = 1;")
c.DestSysSQL.Exec(t, "ALTER RANGE tenants CONFIGURE ZONE USING gc.ttlseconds = 1;")
c.DestSysSQL.Exec(t, fmt.Sprintf("DROP TENANT %s", c.Args.DestTenantName))

if !completeReplication {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -553,7 +553,7 @@ func activateTenant(ctx context.Context, execCtx interface{}, newTenantID roachp
return err
}

info.State = descpb.TenantInfo_ACTIVE
info.DataState = descpb.TenantInfo_READY
info.TenantReplicationJobID = 0
return sql.UpdateTenantRecord(ctx, p.ExecCfg().Settings, txn, info)
})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ func ingestionPlanHook(
TenantInfo: descpb.TenantInfo{
// dstTenantID may be zero which will cause auto-allocation.
ID: dstTenantID,
State: descpb.TenantInfo_ADD,
DataState: descpb.TenantInfo_ADD,
Name: roachpb.TenantName(dstTenantName),
TenantReplicationJobID: jobID,
},
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/streamingccl/streamingest/testdata/simple
Original file line number Diff line number Diff line change
Expand Up @@ -28,13 +28,13 @@ IMPORT INTO d.x CSV DATA ('userfile:///dx/export*-n*.0.csv');
query-sql as=source-system
SHOW TENANTS
----
1 system ACTIVE
10 source ACTIVE
1 system READY
10 source READY

query-sql as=destination-system
SHOW TENANTS
----
1 system ACTIVE
1 system READY
2 destination REPLICATING

let $ts as=source-system
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/testccl/sqlccl/tenant_gc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,7 @@ func TestGCTenantRemovesSpanConfigs(t *testing.T) {
) error {
return sql.TestingUpdateTenantRecord(
ctx, ts.ClusterSettings(), txn,
&descpb.TenantInfo{ID: tenantID.ToUint64(), State: descpb.TenantInfo_DROP},
&descpb.TenantInfo{ID: tenantID.ToUint64(), DataState: descpb.TenantInfo_DROP},
)
}))

Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/catalog/bootstrap/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -561,9 +561,9 @@ func addSystemDatabaseToSchema(
// system tenant entry.
func addSystemTenantEntry(target *MetadataSchema) {
info := descpb.TenantInfo{
ID: roachpb.SystemTenantID.ToUint64(),
Name: catconstants.SystemTenantName,
State: descpb.TenantInfo_ACTIVE,
ID: roachpb.SystemTenantID.ToUint64(),
Name: catconstants.SystemTenantName,
DataState: descpb.TenantInfo_READY,
}
infoBytes, err := protoutil.Marshal(&info)
if err != nil {
Expand Down
23 changes: 15 additions & 8 deletions pkg/sql/catalog/descpb/tenant.proto
Original file line number Diff line number Diff line change
Expand Up @@ -22,19 +22,26 @@ import "multitenant/tenantcapabilities/tenantcapabilitiespb/capabilities.proto";
message TenantInfo {
option (gogoproto.equal) = true;

// The state of the tenant. Dictates whether SQL sessions for the tenant
// should be allowed, although this is currently not enforced.
enum State {
// Tenant is online and available for SQL sessions.
ACTIVE = 0;
// Tenant is being added. Not available for SQL sessions.
// The state of the tenant's logical keyspace.
enum DataState {
// Tenant data is ready and SQL servers can access it.
READY = 0;
// Tenant data is being added. Not available for SQL sessions.
ADD = 1;
// Tenant is being dropped. Not available for SQL sessions.
// Tenant data is being dropped. Not available for SQL sessions.
DROP = 2;
}

// ID is the internal numeric identifier of the tenant.
// It remains mostly invisible to clients.
optional uint64 id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "ID"];
optional State state = 2 [(gogoproto.nullable) = false];

// DataState is the state of the tenant's keyspace. Refer to the
// definition of DataState above for details.
optional DataState data_state = 2 [(gogoproto.nullable) = false];

// Name is the name of the tenant. Beware that the column name is "fixed"
// because it is referred to in a virtual column definition in system.tenants.
optional string name = 3 [
(gogoproto.nullable) = false,
(gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/roachpb.TenantName"];
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/gcjob_test/gc_job_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -453,7 +453,7 @@ func TestGCTenant(t *testing.T) {
txn,
execCfg.SpanConfigKVAccessor.WithTxn(ctx, txn.KV()),
&descpb.TenantInfoWithUsage{
TenantInfo: descpb.TenantInfo{ID: dropTenID, State: descpb.TenantInfo_DROP},
TenantInfo: descpb.TenantInfo{ID: dropTenID, DataState: descpb.TenantInfo_DROP},
},
execCfg.DefaultZoneConfig,
)
Expand Down Expand Up @@ -493,7 +493,7 @@ func TestGCTenant(t *testing.T) {
require.EqualError(
t,
gcClosure(dropTenID, progress),
`GC state for tenant id:11 state:DROP name:"tenant-11" dropped_name:"" tenant_replication_job_id:0 capabilities:<> is DELETED yet the tenant row still exists`,
`GC state for tenant id:11 data_state:DROP name:"tenant-11" dropped_name:"" tenant_replication_job_id:0 capabilities:<> is DELETED yet the tenant row still exists`,
)
})

Expand Down
Loading

0 comments on commit 4d88e2d

Please sign in to comment.