diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 33195366346a..40e4f6abcfe4 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -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'`) @@ -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"}`, }, }, ) @@ -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"}`, }, }, ) @@ -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"}`, }, }, ) @@ -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'`) @@ -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"}`, }, }, ) @@ -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'`) @@ -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"}`, }, }, ) @@ -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'`) @@ -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"}`, }, }, ) diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 863b6d193365..669b337667b9 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -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( @@ -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( @@ -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) } } @@ -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 { diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-tenants b/pkg/ccl/backupccl/testdata/backup-restore/restore-tenants index a302c03775fa..4d492962e230 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-tenants +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-tenants @@ -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' @@ -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'; @@ -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"} diff --git a/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go b/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go index 96643875d496..2bd2cc885ad8 100644 --- a/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go +++ b/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go @@ -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)) @@ -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 { diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go index cffcf3403c0e..ed509befce0a 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go @@ -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) }) diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go index 9c7debecc3ab..31225fd2d02b 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go @@ -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, }, diff --git a/pkg/ccl/streamingccl/streamingest/testdata/simple b/pkg/ccl/streamingccl/streamingest/testdata/simple index e893017ab55d..85878eae177d 100644 --- a/pkg/ccl/streamingccl/streamingest/testdata/simple +++ b/pkg/ccl/streamingccl/streamingest/testdata/simple @@ -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 diff --git a/pkg/ccl/testccl/sqlccl/tenant_gc_test.go b/pkg/ccl/testccl/sqlccl/tenant_gc_test.go index 210b32896bff..8a86c8fc5413 100644 --- a/pkg/ccl/testccl/sqlccl/tenant_gc_test.go +++ b/pkg/ccl/testccl/sqlccl/tenant_gc_test.go @@ -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}, ) })) diff --git a/pkg/sql/catalog/bootstrap/metadata.go b/pkg/sql/catalog/bootstrap/metadata.go index 64cfda8445f4..101e3b9d60cf 100644 --- a/pkg/sql/catalog/bootstrap/metadata.go +++ b/pkg/sql/catalog/bootstrap/metadata.go @@ -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 { diff --git a/pkg/sql/catalog/descpb/tenant.proto b/pkg/sql/catalog/descpb/tenant.proto index 56dfe77a640c..98c0ddfb3ace 100644 --- a/pkg/sql/catalog/descpb/tenant.proto +++ b/pkg/sql/catalog/descpb/tenant.proto @@ -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"]; diff --git a/pkg/sql/gcjob_test/gc_job_test.go b/pkg/sql/gcjob_test/gc_job_test.go index 3d50388fa13d..4b95b05b6c94 100644 --- a/pkg/sql/gcjob_test/gc_job_test.go +++ b/pkg/sql/gcjob_test/gc_job_test.go @@ -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, ) @@ -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`, ) }) diff --git a/pkg/sql/logictest/testdata/logic_test/tenant b/pkg/sql/logictest/testdata/logic_test/tenant index af2ee17092f0..03988c096db2 100644 --- a/pkg/sql/logictest/testdata/logic_test/tenant +++ b/pkg/sql/logictest/testdata/logic_test/tenant @@ -38,49 +38,49 @@ FROM system.tenants ORDER BY id ---- id active name crdb_internal.pb_to_json -1 true system {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"} -2 true tenant-one {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "2", "name": "tenant-one", "state": "ACTIVE", "tenantReplicationJobId": "0"} -3 true two {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "3", "name": "two", "state": "ACTIVE", "tenantReplicationJobId": "0"} -4 true three {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "4", "name": "three", "state": "ACTIVE", "tenantReplicationJobId": "0"} +1 true system {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"} +2 true tenant-one {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "2", "name": "tenant-one", "tenantReplicationJobId": "0"} +3 true two {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "3", "name": "two", "tenantReplicationJobId": "0"} +4 true three {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "4", "name": "three", "tenantReplicationJobId": "0"} query ITT colnames SHOW TENANT system ---- id name status -1 system ACTIVE +1 system READY query ITT colnames SHOW TENANT "tenant-one" ---- id name status -2 tenant-one ACTIVE +2 tenant-one READY query ITT colnames SHOW TENANT "two" ---- id name status -3 two ACTIVE +3 two READY query ITT colnames SHOW TENANT two ---- id name status -3 two ACTIVE +3 two READY query ITT colnames SHOW TENANT three ---- id name status -4 three ACTIVE +4 three READY query ITT colnames SHOW TENANTS ---- id name status -1 system ACTIVE -2 tenant-one ACTIVE -3 two ACTIVE -4 three ACTIVE +1 system READY +2 tenant-one READY +3 two READY +4 three READY statement error tenant name cannot be empty ALTER TENANT [4] RENAME TO "" @@ -100,8 +100,8 @@ ALTER TENANT blux RENAME TO 'blix' query ITT colnames SELECT * FROM [SHOW TENANTS] WHERE id = 4 ---- -id name status -4 blix ACTIVE +id name status +4 blix READY statement ok ALTER TENANT blix RENAME TO three @@ -109,8 +109,8 @@ ALTER TENANT blix RENAME TO three query ITT colnames SELECT * FROM [SHOW TENANTS] WHERE id = 4 ---- -id name status -4 three ACTIVE +id name status +4 three READY statement error tenant "seven" does not exist SHOW TENANT seven @@ -158,7 +158,7 @@ FROM system.tenants WHERE name = 'four' ORDER BY id ---- id active name crdb_internal.pb_to_json -5 true four {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "5", "name": "four", "state": "ACTIVE", "tenantReplicationJobId": "0"} +5 true four {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "5", "name": "four", "tenantReplicationJobId": "0"} statement ok DROP TENANT four @@ -232,14 +232,14 @@ FROM system.tenants ORDER BY id ---- id active name crdb_internal.pb_to_json -1 true system {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"} -2 true tenant-one {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "2", "name": "tenant-one", "state": "ACTIVE", "tenantReplicationJobId": "0"} -3 true two {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "3", "name": "two", "state": "ACTIVE", "tenantReplicationJobId": "0"} -4 true three {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "4", "name": "three", "state": "ACTIVE", "tenantReplicationJobId": "0"} -5 false NULL {"capabilities": {"canAdminSplit": false}, "droppedName": "four", "id": "5", "name": "", "state": "DROP", "tenantReplicationJobId": "0"} -6 false NULL {"capabilities": {"canAdminSplit": false}, "droppedName": "five-requiring-quotes", "id": "6", "name": "", "state": "DROP", "tenantReplicationJobId": "0"} -7 false NULL {"capabilities": {"canAdminSplit": false}, "droppedName": "to-be-reclaimed", "id": "7", "name": "", "state": "DROP", "tenantReplicationJobId": "0"} -8 true to-be-reclaimed {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "8", "name": "to-be-reclaimed", "state": "ACTIVE", "tenantReplicationJobId": "0"} +1 true system {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"} +2 true tenant-one {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "2", "name": "tenant-one", "tenantReplicationJobId": "0"} +3 true two {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "3", "name": "two", "tenantReplicationJobId": "0"} +4 true three {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "4", "name": "three", "tenantReplicationJobId": "0"} +5 false NULL {"capabilities": {"canAdminSplit": false}, "dataState": "DROP", "droppedName": "four", "id": "5", "name": "", "tenantReplicationJobId": "0"} +6 false NULL {"capabilities": {"canAdminSplit": false}, "dataState": "DROP", "droppedName": "five-requiring-quotes", "id": "6", "name": "", "tenantReplicationJobId": "0"} +7 false NULL {"capabilities": {"canAdminSplit": false}, "dataState": "DROP", "droppedName": "to-be-reclaimed", "id": "7", "name": "", "tenantReplicationJobId": "0"} +8 true to-be-reclaimed {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "8", "name": "to-be-reclaimed", "tenantReplicationJobId": "0"} # More valid tenant names. statement ok @@ -250,12 +250,12 @@ CREATE TENANT "hello-100" query ITT colnames SHOW TENANTS ---- -id name status -1 system ACTIVE -2 tenant-one ACTIVE -3 two ACTIVE -4 three ACTIVE -8 to-be-reclaimed ACTIVE -9 1 ACTIVE -10 a-b ACTIVE -11 hello-100 ACTIVE +id name status +1 system READY +2 tenant-one READY +3 two READY +4 three READY +8 to-be-reclaimed READY +9 1 READY +10 a-b READY +11 hello-100 READY diff --git a/pkg/sql/logictest/testdata/logic_test/tenant_builtins b/pkg/sql/logictest/testdata/logic_test/tenant_builtins index 89df728b0f10..ad4355432286 100644 --- a/pkg/sql/logictest/testdata/logic_test/tenant_builtins +++ b/pkg/sql/logictest/testdata/logic_test/tenant_builtins @@ -37,10 +37,10 @@ FROM system.tenants ORDER BY id ---- id active name crdb_internal.pb_to_json -1 true system {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"} -2 true tenant-number-eleven {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "2", "name": "tenant-number-eleven", "state": "ACTIVE", "tenantReplicationJobId": "0"} -5 true tenant-5 {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "5", "name": "tenant-5", "state": "ACTIVE", "tenantReplicationJobId": "0"} -10 true tenant-number-ten {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-number-ten", "state": "ACTIVE", "tenantReplicationJobId": "0"} +1 true system {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"} +2 true tenant-number-eleven {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "2", "name": "tenant-number-eleven", "tenantReplicationJobId": "0"} +5 true tenant-5 {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "5", "name": "tenant-5", "tenantReplicationJobId": "0"} +10 true tenant-number-ten {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "10", "name": "tenant-number-ten", "tenantReplicationJobId": "0"} # Check we can add a name where none existed before. statement ok @@ -93,7 +93,7 @@ id active name # Garbage collect a non-drop tenant fails. -query error tenant 5 is not in state DROP +query error tenant 5 is not in data state DROP SELECT crdb_internal.gc_tenant(5) # Note this just marks the tenant as dropped but does not call GC. @@ -107,10 +107,10 @@ FROM system.tenants ORDER BY id ---- id active name crdb_internal.pb_to_json -1 true system {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"} -2 true tenant-number-eleven {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "2", "name": "tenant-number-eleven", "state": "ACTIVE", "tenantReplicationJobId": "0"} -5 false NULL {"capabilities": {"canAdminSplit": false}, "droppedName": "my-new-tenant-name", "id": "5", "name": "", "state": "DROP", "tenantReplicationJobId": "0"} -10 true tenant-number-ten {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-number-ten", "state": "ACTIVE", "tenantReplicationJobId": "0"} +1 true system {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "1", "name": "system", "tenantReplicationJobId": "0"} +2 true tenant-number-eleven {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "2", "name": "tenant-number-eleven", "tenantReplicationJobId": "0"} +5 false NULL {"capabilities": {"canAdminSplit": false}, "dataState": "DROP", "droppedName": "my-new-tenant-name", "id": "5", "name": "", "tenantReplicationJobId": "0"} +10 true tenant-number-ten {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "10", "name": "tenant-number-ten", "tenantReplicationJobId": "0"} # Try to recreate an existing tenant. @@ -217,9 +217,9 @@ FROM system.tenants ORDER BY id ---- id active crdb_internal.pb_to_json -1 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"} -2 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "2", "name": "tenant-number-eleven", "state": "ACTIVE", "tenantReplicationJobId": "0"} -10 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-number-ten", "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": "tenant-number-eleven", "tenantReplicationJobId": "0"} +10 true {"capabilities": {"canAdminSplit": false}, "dataState": "READY", "droppedName": "", "id": "10", "name": "tenant-number-ten", "tenantReplicationJobId": "0"} query error tenant resource limits require a CCL binary SELECT crdb_internal.update_tenant_resource_limits(10, 1000, 100, 0, now(), 0) diff --git a/pkg/sql/show_tenant.go b/pkg/sql/show_tenant.go index fd087c95a231..76a4ad635e38 100644 --- a/pkg/sql/show_tenant.go +++ b/pkg/sql/show_tenant.go @@ -176,11 +176,11 @@ func (n *showTenantNode) getTenantValues( if n.withReplication { return nil, errors.Newf("tenant %q does not have an active replication job", tenantInfo.Name) } - values.tenantStatus = tenantStatus(values.tenantInfo.State.String()) + values.tenantStatus = tenantStatus(values.tenantInfo.DataState.String()) return &values, nil } - switch values.tenantInfo.State { + switch values.tenantInfo.DataState { case descpb.TenantInfo_ADD: // There is a replication job, we need to get the job info and the // replication stats in order to generate the exact tenant status. @@ -205,10 +205,10 @@ func (n *showTenantNode) getTenantValues( } values.tenantStatus = getTenantStatus(job.Status(), values.replicationInfo) - case descpb.TenantInfo_ACTIVE, descpb.TenantInfo_DROP: - values.tenantStatus = tenantStatus(values.tenantInfo.State.String()) + case descpb.TenantInfo_READY, descpb.TenantInfo_DROP: + values.tenantStatus = tenantStatus(values.tenantInfo.DataState.String()) default: - return nil, errors.Newf("tenant %q state is unknown: %s", tenantInfo.Name, values.tenantInfo.State.String()) + return nil, errors.Newf("tenant %q state is unknown: %s", tenantInfo.Name, values.tenantInfo.DataState.String()) } return &values, nil } diff --git a/pkg/sql/tenant_accessors.go b/pkg/sql/tenant_accessors.go index de4bfdae051d..57d4c461d70d 100644 --- a/pkg/sql/tenant_accessors.go +++ b/pkg/sql/tenant_accessors.go @@ -57,7 +57,7 @@ func GetAllNonDropTenantIDs(ctx context.Context, txn isql.Txn) ([]roachpb.Tenant ctx, "get-tenant-ids", txn.KV(), ` SELECT id FROM system.tenants - WHERE crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true)->>'state' != 'DROP' + WHERE crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true)->>'dataState' != 'DROP' ORDER BY id `) if err != nil { diff --git a/pkg/sql/tenant_creation.go b/pkg/sql/tenant_creation.go index a76d53d197ec..09bd75dee58a 100644 --- a/pkg/sql/tenant_creation.go +++ b/pkg/sql/tenant_creation.go @@ -90,9 +90,9 @@ func (p *planner) createTenantInternal( TenantInfo: descpb.TenantInfo{ ID: tenantID, // We synchronously initialize the tenant's keyspace below, so - // we can skip the ADD state and go straight to an ACTIVE state. - State: descpb.TenantInfo_ACTIVE, - Name: name, + // we can skip the ADD state and go straight to the READY state. + DataState: descpb.TenantInfo_READY, + Name: name, }, } @@ -254,7 +254,7 @@ func CreateTenantRecord( } } - active := info.State == descpb.TenantInfo_ACTIVE + active := info.DataState == descpb.TenantInfo_READY infoBytes, err := protoutil.Marshal(&info.TenantInfo) if err != nil { return roachpb.TenantID{}, err diff --git a/pkg/sql/tenant_deletion.go b/pkg/sql/tenant_deletion.go index 74ec422b116b..23c4c1821751 100644 --- a/pkg/sql/tenant_deletion.go +++ b/pkg/sql/tenant_deletion.go @@ -77,8 +77,8 @@ func dropTenantInternal( return err } - if info.State == descpb.TenantInfo_DROP { - return errors.Errorf("tenant %d is already in state DROP", tenID) + if info.DataState == descpb.TenantInfo_DROP { + return errors.Errorf("tenant %d is already in data state DROP", tenID) } // Mark the tenant as dropping. @@ -97,7 +97,7 @@ func dropTenantInternal( // TODO(ssd): We may want to implement a job that waits out // any running sql pods before enqueing the GC job. - info.State = descpb.TenantInfo_DROP + info.DataState = descpb.TenantInfo_DROP info.DroppedName = info.Name info.Name = "" if err := UpdateTenantRecord(ctx, settings, txn, info); err != nil { diff --git a/pkg/sql/tenant_gc.go b/pkg/sql/tenant_gc.go index 49aba9cb4180..f11a2c050a70 100644 --- a/pkg/sql/tenant_gc.go +++ b/pkg/sql/tenant_gc.go @@ -106,7 +106,7 @@ func GCTenantSync(ctx context.Context, execCfg *ExecutorConfig, info *descpb.Ten // clearTenant deletes the tenant's data. func clearTenant(ctx context.Context, execCfg *ExecutorConfig, info *descpb.TenantInfo) error { // Confirm tenant is ready to be cleared. - if info.State != descpb.TenantInfo_DROP { + if info.DataState != descpb.TenantInfo_DROP { return errors.Errorf("tenant %d is not in state DROP", info.ID) } @@ -145,8 +145,8 @@ func (p *planner) GCTenant(ctx context.Context, tenID uint64) error { } // Confirm tenant is ready to be cleared. - if info.State != descpb.TenantInfo_DROP { - return errors.Errorf("tenant %d is not in state DROP", info.ID) + if info.DataState != descpb.TenantInfo_DROP { + return errors.Errorf("tenant %d is not in data state DROP", info.ID) } _, err = createGCTenantJob( diff --git a/pkg/sql/tenant_update.go b/pkg/sql/tenant_update.go index e37bcaa5175c..1efbacf7c921 100644 --- a/pkg/sql/tenant_update.go +++ b/pkg/sql/tenant_update.go @@ -39,7 +39,7 @@ func UpdateTenantRecord( } tenID := info.ID - active := info.State == descpb.TenantInfo_ACTIVE + active := info.DataState == descpb.TenantInfo_READY infoBytes, err := protoutil.Marshal(info) if err != nil { return err @@ -58,11 +58,11 @@ func UpdateTenantRecord( } func validateTenantInfo(info *descpb.TenantInfo) error { - if info.TenantReplicationJobID != 0 && info.State == descpb.TenantInfo_ACTIVE { - return errors.Newf("tenant in state %v with replication job ID %d", info.State, info.TenantReplicationJobID) + if info.TenantReplicationJobID != 0 && info.DataState == descpb.TenantInfo_READY { + return errors.Newf("tenant in data state %v with replication job ID %d", info.DataState, info.TenantReplicationJobID) } - if info.DroppedName != "" && info.State != descpb.TenantInfo_DROP { - return errors.Newf("tenant in state %v with dropped name %q", info.State, info.DroppedName) + if info.DroppedName != "" && info.DataState != descpb.TenantInfo_DROP { + return errors.Newf("tenant in data state %v with dropped name %q", info.DataState, info.DroppedName) } return nil } @@ -125,7 +125,7 @@ func ActivateTenant( } // Mark the tenant as active. - info.State = descpb.TenantInfo_ACTIVE + info.DataState = descpb.TenantInfo_READY if err := UpdateTenantRecord(ctx, settings, txn, info); err != nil { return errors.Wrap(err, "activating tenant") } diff --git a/pkg/upgrade/upgrades/tenant_table_migration.go b/pkg/upgrade/upgrades/tenant_table_migration.go index 1844d86cf367..64762797d4db 100644 --- a/pkg/upgrade/upgrades/tenant_table_migration.go +++ b/pkg/upgrade/upgrades/tenant_table_migration.go @@ -32,7 +32,7 @@ CREATE UNIQUE INDEX tenants_name_idx ON system.public.tenants (name ASC) const addSystemTenantEntry = ` UPSERT INTO system.public.tenants (id, active, info) -VALUES (1, true, crdb_internal.json_to_pb('cockroach.sql.sqlbase.TenantInfo', '{"id":1,"state":0,"name":"` + catconstants.SystemTenantName + `"}')) +VALUES (1, true, crdb_internal.json_to_pb('cockroach.sql.sqlbase.TenantInfo', '{"id":1,"dataState":"READY","name":"` + catconstants.SystemTenantName + `"}')) ` func addTenantNameColumnAndSystemTenantEntry(