Skip to content

Commit

Permalink
*: support visible database/table
Browse files Browse the repository at this point in the history
  • Loading branch information
zimulala committed Dec 17, 2019
1 parent cbf7415 commit 6480527
Show file tree
Hide file tree
Showing 11 changed files with 281 additions and 27 deletions.
26 changes: 26 additions & 0 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,32 @@ type testIntegrationSuite3 struct{ *testIntegrationSuite }
type testIntegrationSuite4 struct{ *testIntegrationSuite }
type testIntegrationSuite5 struct{ *testIntegrationSuite }

func (s *testIntegrationSuite5) TestInvisibleDatabaseOrTable(c *C) {
tk := testkit.NewTestKit(c, s.store)

tk.MustExec("use test")
tk.MustExec(("create table t(a int, b int) invisible"))
tk.MustGetErrCode(("create table t(a int, b int) invisible"), mysql.ErrTableExists)
tk.MustGetErrCode(("create table t(a int, b int) visible"), mysql.ErrTableExists)
tk.MustGetErrCode(("create table t(a int, b int)"), mysql.ErrTableExists)
tk.MustGetErrCode(("alter table t add column c int"), mysql.ErrInvalidDDLState)
tk.MustGetErrCode(("insert into t values(1, 2)"), mysql.ErrSchemaState)
tk.MustQuery("show create table t").Check(testkit.Rows(
"t CREATE TABLE `t` (\n" +
" `a` int(11) DEFAULT NULL,\n" +
" `b` int(11) DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin/*!90001 INVISIBLE */",
))
tk.MustExec("alter table t visible")
tk.MustExec("insert into t values(1, 2)")
tk.MustQuery("show create table t").Check(testkit.Rows(
"t CREATE TABLE `t` (\n" +
" `a` int(11) DEFAULT NULL,\n" +
" `b` int(11) DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin",
))
}

func (s *testIntegrationSuite5) TestNoZeroDateMode(c *C) {
tk := testkit.NewTestKit(c, s.store)

Expand Down
8 changes: 7 additions & 1 deletion ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -183,6 +183,10 @@ var (
ErrPrimaryCantHaveNull = terror.ClassDDL.New(mysql.ErrPrimaryCantHaveNull, mysql.MySQLErrName[mysql.ErrPrimaryCantHaveNull])
// ErrErrorOnRename returns error for wrong database name in alter table rename
ErrErrorOnRename = terror.ClassDDL.New(mysql.ErrErrorOnRename, mysql.MySQLErrName[mysql.ErrErrorOnRename])
// ErrAlterVisibility returns an error when changing database/table visibility is not supported.
ErrAlterVisibility = terror.ClassDDL.New(mysql.ErrAlterVisibility, mysql.MySQLErrName[mysql.ErrAlterVisibility])
// ErrSchemaState returns an error for wrong schema state.
ErrSchemaState = terror.ClassDDL.New(mysql.ErrSchemaState, mysql.MySQLErrName[mysql.ErrSchemaState])

// ErrNotAllowedTypeInPartition returns not allowed type error when creating table partition with unsupported expression type.
ErrNotAllowedTypeInPartition = terror.ClassDDL.New(mysql.ErrFieldTypeNotAllowedAsPartitionField, mysql.MySQLErrName[mysql.ErrFieldTypeNotAllowedAsPartitionField])
Expand Down Expand Up @@ -227,7 +231,7 @@ var (

// DDL is responsible for updating schema in data store and maintaining in-memory InfoSchema cache.
type DDL interface {
CreateSchema(ctx sessionctx.Context, name model.CIStr, charsetInfo *ast.CharsetOpt) error
CreateSchema(ctx sessionctx.Context, name model.CIStr, charsetInfo *ast.CharsetOpt, isVisible bool) error
AlterSchema(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) error
DropSchema(ctx sessionctx.Context, schema model.CIStr) error
CreateTable(ctx sessionctx.Context, stmt *ast.CreateTableStmt) error
Expand Down Expand Up @@ -727,6 +731,8 @@ func init() {
mysql.ErrWrongTableName: mysql.ErrWrongTableName,
mysql.ErrWrongTypeColumnValue: mysql.ErrWrongTypeColumnValue,
mysql.WarnDataTruncated: mysql.WarnDataTruncated,
mysql.ErrAlterVisibility: mysql.ErrAlterVisibility,
mysql.ErrSchemaState: mysql.ErrSchemaState,
}
terror.ErrClassToMySQLCodes[terror.ClassDDL] = ddlMySQLErrCodes
}
97 changes: 92 additions & 5 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ import (
"go.uber.org/zap"
)

func (d *ddl) CreateSchema(ctx sessionctx.Context, schema model.CIStr, charsetInfo *ast.CharsetOpt) (err error) {
func (d *ddl) CreateSchema(ctx sessionctx.Context, schema model.CIStr, charsetInfo *ast.CharsetOpt, isVisible bool) (err error) {
is := d.GetInfoSchemaWithInterceptor(ctx)
_, ok := is.SchemaByName(schema)
if ok {
Expand Down Expand Up @@ -84,6 +84,9 @@ func (d *ddl) CreateSchema(ctx sessionctx.Context, schema model.CIStr, charsetIn
} else {
dbInfo.Charset, dbInfo.Collate = charset.GetDefaultCharsetAndCollate()
}
if !isVisible {
dbInfo.State = model.StateDeleteOnly
}

job := &model.Job{
SchemaID: schemaID,
Expand All @@ -100,6 +103,8 @@ func (d *ddl) CreateSchema(ctx sessionctx.Context, schema model.CIStr, charsetIn
func (d *ddl) AlterSchema(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) (err error) {
// Resolve target charset and collation from options.
var toCharset, toCollate string
isVisible := true
isChangingVisibility := false
for _, val := range stmt.Options {
switch val.Tp {
case ast.DatabaseOptionCharset:
Expand All @@ -119,6 +124,13 @@ func (d *ddl) AlterSchema(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) (
return ErrConflictingDeclarations.GenWithStackByArgs(toCharset, info.CharsetName)
}
toCollate = info.Name
case ast.DatabaseOptionVisibility:
if val.UintValue == uint64(ast.VisibilityOptionVisible) {
isVisible = true
} else if val.UintValue == uint64(ast.VisibilityOptionInvisible) {
isVisible = false
}
isChangingVisibility = true
}
}
if toCollate == "" {
Expand All @@ -143,13 +155,28 @@ func (d *ddl) AlterSchema(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) (
return errors.Trace(err)
}

jobTp := model.ActionModifySchemaCharsetAndCollate
jobArgs := []interface{}{toCharset, toCollate}
// Change schema visibility.
if isChangingVisibility {
if !isVisible {
return ErrAlterVisibility.GenWithStackByArgs("schema", "invisible is unsupported")
}
if dbInfo.State != model.StateDeleteOnly {
return ErrAlterVisibility.GenWithStackByArgs("schema",
fmt.Sprintf("visible, schema %s state %s changing visibility is unsupported", dbInfo.Name, dbInfo.State))
}
jobTp = model.ActionAlterSchemaVisibility
jobArgs = []interface{}{toCharset, toCollate, isVisible}
}

// Do the DDL job.
job := &model.Job{
SchemaID: dbInfo.ID,
SchemaName: dbInfo.Name.L,
Type: model.ActionModifySchemaCharsetAndCollate,
Type: jobTp,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{toCharset, toCollate},
Args: jobArgs,
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
Expand Down Expand Up @@ -1394,12 +1421,13 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err e
if err != nil {
return errors.Trace(err)
}
originalState := tbInfo.State
tbInfo.State = model.StatePublic
err = checkTableInfoValid(tbInfo)
if err != nil {
return err
}
tbInfo.State = model.StateNone
tbInfo.State = originalState

job := &model.Job{
SchemaID: schema.ID,
Expand Down Expand Up @@ -1767,6 +1795,7 @@ func resolveDefaultTableCharsetAndCollation(tbInfo *model.TableInfo, dbCharset,

// handleTableOptions updates tableInfo according to table options.
func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) error {
isVisible := true
for _, op := range options {
switch op.Tp {
case ast.TableOptionAutoIncrement:
Expand All @@ -1786,11 +1815,20 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err
tbInfo.MaxShardRowIDBits = tbInfo.ShardRowIDBits
case ast.TableOptionPreSplitRegion:
tbInfo.PreSplitRegions = op.UintValue
case ast.TableOptionVisibility:
if op.UintValue == uint64(ast.VisibilityOptionVisible) {
isVisible = true
} else if op.UintValue == uint64(ast.VisibilityOptionInvisible) {
isVisible = false
}
}
}
if tbInfo.PreSplitRegions > tbInfo.ShardRowIDBits {
tbInfo.PreSplitRegions = tbInfo.ShardRowIDBits
}
if !isVisible {
tbInfo.State = model.StateDeleteOnly
}
return nil
}

Expand Down Expand Up @@ -1977,11 +2015,11 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A
err = d.AlterTableCharsetAndCollate(ctx, ident, toCharset, toCollate)
handledCharsetOrCollate = true
}

if err != nil {
return errors.Trace(err)
}
}
err = d.AlterVisibility(ctx, ident, spec)
case ast.AlterTableSetTiFlashReplica:
err = d.AlterTableSetTiFlashReplica(ctx, ident, spec.TiFlashReplica)
default:
Expand Down Expand Up @@ -2925,6 +2963,55 @@ func (d *ddl) AlterTableComment(ctx sessionctx.Context, ident ast.Ident, spec *a
return errors.Trace(err)
}

func (d *ddl) AlterVisibility(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error {
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema)
}

tb, err := is.TableByName(ident.Schema, ident.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name))
}

isVisible := true
var isChangingVisibility bool
for _, opt := range spec.Options {
if opt.Tp == ast.TableOptionVisibility {
if opt.UintValue == uint64(ast.VisibilityOptionVisible) {
isVisible = true
} else if opt.UintValue == uint64(ast.VisibilityOptionInvisible) {
isVisible = false
}
isChangingVisibility = true
}
}
tblInfo := tb.Meta()
if isChangingVisibility {
if !isVisible {
return ErrAlterVisibility.GenWithStackByArgs("table", "invisible is unsupported")
}
if tblInfo.State != model.StateDeleteOnly {
return ErrAlterVisibility.GenWithStackByArgs("table",
fmt.Sprintf("visible, table %s state %s changing visibility is unsupported", tblInfo.Name, tblInfo.State))
}
}

job := &model.Job{
SchemaID: schema.ID,
TableID: tblInfo.ID,
SchemaName: schema.Name.L,
Type: model.ActionAlterTableVisibility,
BinlogInfo: &model.HistoryInfo{},
Args: nil,
}

err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}

// AlterTableCharset changes the table charset and collate.
func (d *ddl) AlterTableCharsetAndCollate(ctx sessionctx.Context, ident ast.Ident, toCharset, toCollate string) error {
// use the last one.
Expand Down
6 changes: 5 additions & 1 deletion ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -522,7 +522,9 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
case model.ActionCreateSchema:
ver, err = onCreateSchema(d, t, job)
case model.ActionModifySchemaCharsetAndCollate:
ver, err = onModifySchemaCharsetAndCollate(t, job)
ver, err = onModifySchemaOptions(t, job)
case model.ActionAlterSchemaVisibility:
ver, err = onModifySchemaOptions(t, job)
case model.ActionDropSchema:
ver, err = onDropSchema(t, job)
case model.ActionCreateTable:
Expand Down Expand Up @@ -581,6 +583,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
ver, err = onSetTableFlashReplica(t, job)
case model.ActionUpdateTiFlashReplicaStatus:
ver, err = onUpdateFlashReplicaStatus(t, job)
case model.ActionAlterTableVisibility:
ver, err = onAlterTableVisibility(t, job)
default:
// Invalid job, cancel it.
job.State = model.JobStateCancelled
Expand Down
15 changes: 13 additions & 2 deletions ddl/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ func onCreateSchema(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error
}

dbInfo.ID = schemaID
dbInfo.State = model.StateNone

err := checkSchemaNotExists(d, t, schemaID, dbInfo)
if err != nil {
Expand All @@ -57,6 +56,15 @@ func onCreateSchema(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error
// Finish this job.
job.FinishDBJob(model.JobStateDone, model.StatePublic, ver, dbInfo)
return ver, nil
case model.StateDeleteOnly:
// Create a invisible schema.
err = t.CreateDatabase(dbInfo)
if err != nil {
return ver, errors.Trace(err)
}
// Finish this job.
job.FinishDBJob(model.JobStateDone, model.StateDeleteOnly, ver, dbInfo)
return ver, nil
default:
// We can't enter here.
return ver, errors.Errorf("invalid db state %v", dbInfo.State)
Expand Down Expand Up @@ -109,7 +117,7 @@ func checkSchemaNotExistsFromStore(t *meta.Meta, schemaID int64, dbInfo *model.D
return nil
}

func onModifySchemaCharsetAndCollate(t *meta.Meta, job *model.Job) (ver int64, _ error) {
func onModifySchemaOptions(t *meta.Meta, job *model.Job) (ver int64, _ error) {
var toCharset, toCollate string
if err := job.DecodeArgs(&toCharset, &toCollate); err != nil {
job.State = model.JobStateCancelled
Expand All @@ -121,6 +129,9 @@ func onModifySchemaCharsetAndCollate(t *meta.Meta, job *model.Job) (ver int64, _
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
if job.Type == model.ActionAlterSchemaVisibility {
dbInfo.State = model.StatePublic
}

if dbInfo.Charset == toCharset && dbInfo.Collate == toCollate {
job.FinishDBJob(model.JobStateDone, model.StatePublic, ver, dbInfo)
Expand Down
31 changes: 30 additions & 1 deletion ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,6 @@ func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error)
return ver, errors.Trace(err)
}

tbInfo.State = model.StateNone
err := checkTableNotExists(d, t, schemaID, tbInfo.Name.L)
if err != nil {
if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableExists.Equal(err) {
Expand All @@ -78,6 +77,17 @@ func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error)
job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo)
asyncNotifyEvent(d, &util.Event{Tp: model.ActionCreateTable, TableInfo: tbInfo})
return ver, nil
case model.StateDeleteOnly:
// Create a invisible table.
tbInfo.UpdateTS = t.StartTS
err = createTableOrViewWithCheck(t, job, schemaID, tbInfo)
if err != nil {
return ver, errors.Trace(err)
}
// Finish this job.
job.FinishTableJob(model.JobStateDone, model.StateDeleteOnly, ver, tbInfo)
asyncNotifyEvent(d, &util.Event{Tp: model.ActionCreateTable, TableInfo: tbInfo})
return ver, nil
default:
return ver, ErrInvalidDDLState.GenWithStackByArgs("table", tbInfo.State)
}
Expand Down Expand Up @@ -647,6 +657,25 @@ func onModifyTableComment(t *meta.Meta, job *model.Job) (ver int64, _ error) {
return ver, nil
}

func onAlterTableVisibility(t *meta.Meta, job *model.Job) (ver int64, _ error) {
tblInfo, err := checkTableExistAndCancelNonExistJob(t, job, job.SchemaID)
if err != nil {
return ver, errors.Trace(err)
}
if tblInfo.State != model.StateDeleteOnly {
job.State = model.JobStateCancelled
return ver, ErrInvalidDDLState.GenWithStack("table %s is not in public, but %s", tblInfo.Name, tblInfo.State)
}

tblInfo.State = model.StatePublic
ver, err = updateVersionAndTableInfo(t, job, tblInfo, true)
if err != nil {
return ver, errors.Trace(err)
}
job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo)
return ver, nil
}

func onModifyTableCharsetAndCollate(t *meta.Meta, job *model.Job) (ver int64, _ error) {
var toCharset, toCollate string
if err := job.DecodeArgs(&toCharset, &toCollate); err != nil {
Expand Down
9 changes: 8 additions & 1 deletion executor/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -157,6 +157,7 @@ func (e *DDLExec) executeRenameTable(s *ast.RenameTableStmt) error {

func (e *DDLExec) executeCreateDatabase(s *ast.CreateDatabaseStmt) error {
var opt *ast.CharsetOpt
isVisible := true
if len(s.Options) != 0 {
opt = &ast.CharsetOpt{}
for _, val := range s.Options {
Expand All @@ -165,10 +166,16 @@ func (e *DDLExec) executeCreateDatabase(s *ast.CreateDatabaseStmt) error {
opt.Chs = val.Value
case ast.DatabaseOptionCollate:
opt.Col = val.Value
case ast.DatabaseOptionVisibility:
if val.UintValue == uint64(ast.VisibilityOptionVisible) {
isVisible = true
} else if val.UintValue == uint64(ast.VisibilityOptionInvisible) {
isVisible = false
}
}
}
}
err := domain.GetDomain(e.ctx).DDL().CreateSchema(e.ctx, model.NewCIStr(s.Name), opt)
err := domain.GetDomain(e.ctx).DDL().CreateSchema(e.ctx, model.NewCIStr(s.Name), opt, isVisible)
if err != nil {
if infoschema.ErrDatabaseExists.Equal(err) && s.IfNotExists {
err = nil
Expand Down
Loading

0 comments on commit 6480527

Please sign in to comment.