From dfbef74ffe0ac5bc998657e957d050d3fcb98d1a Mon Sep 17 00:00:00 2001 From: CbcWestwolf <1004626265@qq.com> Date: Fri, 24 Feb 2023 13:17:10 +0800 Subject: [PATCH] *: support table encryption (#1) --- ddl/db_test.go | 19 ++++++++++++ ddl/ddl_api.go | 46 ++++++++++++++++++++++++++++++ ddl/ddl_worker.go | 2 ++ ddl/multi_schema_change.go | 2 +- ddl/schematracker/dm_tracker.go | 11 +++++++ ddl/table.go | 26 +++++++++++++++++ executor/infoschema_reader.go | 13 ++++++--- executor/show.go | 4 +++ parser/ast/ddl.go | 1 + parser/model/ddl.go | 3 ++ parser/model/model.go | 2 ++ parser/parser.y | 1 - privilege/privileges/privileges.go | 1 + sessionctx/variable/sysvar.go | 7 +++++ util/dbterror/ddl_terror.go | 2 ++ 15 files changed, 134 insertions(+), 6 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 89d5a3cd86cd3..1b96fc4645e8b 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -690,6 +690,25 @@ func TestComment(t *testing.T) { tk.MustExec("drop table if exists ct, ct1") } +func TestTableEncryption(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + + tk.MustExec(`create table t (id int) encryption = 'Y'`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("t CREATE TABLE `t` (\n" + + " `id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin ENCRYPTION='Y'")) + tk.MustQuery(`select create_options from information_schema.tables where table_name = 't' and table_schema = 'test'`).Check(testkit.Rows("ENCRYPTION='Y'")) + + tk.MustExec(`alter table t encryption 'n'`) + tk.MustQuery(`show create table t`).Check(testkit.Rows("t CREATE TABLE `t` (\n" + + " `id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tk.MustQuery(`select create_options from information_schema.tables where table_name = 't' and table_schema = 'test'`).Check(testkit.Rows("")) +} + func TestRebaseAutoID(t *testing.T) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`)) defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange")) }() diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index c2afd27e45fd9..8c0d181e4056b 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3152,6 +3152,15 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err tbInfo.TTLInfo = ttlInfo handledTTLOrTTLEnable = true + case ast.TableOptionEncryption: + switch strings.ToUpper(op.StrValue) { + case "Y": + tbInfo.TableEncryption = true + case "N": + tbInfo.TableEncryption = false + default: + return errors.Trace(dbterror.ErrInvalidEncryptionOption) + } } } shardingBits := shardingBits(tbInfo) @@ -3498,6 +3507,16 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast err = d.AlterTableTTLInfoOrEnable(sctx, ident, ttlInfo, ttlEnable) handledTTLOrTTLEnable = true + case ast.TableOptionEncryption: + switch strings.ToUpper(opt.StrValue) { + case "Y": + spec.TableEncrption = true + case "N": + spec.TableEncrption = false + default: + return errors.Trace(dbterror.ErrInvalidEncryptionOption) + } + err = d.AlterTableEncryptionOption(sctx, ident, spec) default: err = dbterror.ErrUnsupportedAlterTableOption } @@ -5292,6 +5311,33 @@ func (d *ddl) AlterTableComment(ctx sessionctx.Context, ident ast.Ident, spec *a return errors.Trace(err) } +// AlterTableEncryptionOption updates the table encryption option. +func (d *ddl) AlterTableEncryptionOption(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + is := d.infoCache.GetLatest() + 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)) + } + job := &model.Job{ + SchemaID: schema.ID, + TableID: tb.Meta().ID, + SchemaName: schema.Name.L, + TableName: tb.Meta().Name.L, + Type: model.ActionAlterEncryptionOption, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{spec.TableEncrption}, + } + + err = d.DoDDLJob(ctx, job) + err = d.callHookOnChanged(job, err) + return errors.Trace(err) +} + // AlterTableAutoIDCache updates the table comment information. func (d *ddl) AlterTableAutoIDCache(ctx sessionctx.Context, ident ast.Ident, newCache int64) error { schema, tb, err := d.getSchemaAndTableByIdent(ctx, ident) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index a08d16f8e47b0..35b24b1ac0ffd 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -1302,6 +1302,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, ver, err = onTTLInfoChange(d, t, job) case model.ActionAlterTTLRemove: ver, err = onTTLInfoRemove(d, t, job) + case model.ActionAlterEncryptionOption: + ver, err = onModifyTableEncryptionOption(d, t, job) case model.ActionAddCheckConstraint: ver, err = w.onAddCheckConstraint(d, t, job) case model.ActionDropCheckConstraint: diff --git a/ddl/multi_schema_change.go b/ddl/multi_schema_change.go index ab306fe546932..3ae0a7a6c2db0 100644 --- a/ddl/multi_schema_change.go +++ b/ddl/multi_schema_change.go @@ -258,7 +258,7 @@ func fillMultiSchemaInfo(info *model.MultiSchemaInfo, job *model.Job) (err error case model.ActionAlterIndexVisibility: idxName := job.Args[0].(model.CIStr) info.AlterIndexes = append(info.AlterIndexes, idxName) - case model.ActionRebaseAutoID, model.ActionModifyTableComment, model.ActionModifyTableCharsetAndCollate: + case model.ActionRebaseAutoID, model.ActionModifyTableComment, model.ActionModifyTableCharsetAndCollate, model.ActionAlterEncryptionOption: case model.ActionAddForeignKey: fkInfo := job.Args[0].(*model.FKInfo) info.ForeignKeys = append(info.ForeignKeys, fkInfo.Name) diff --git a/ddl/schematracker/dm_tracker.go b/ddl/schematracker/dm_tracker.go index 75f8fa35b429d..284e76ec56233 100644 --- a/ddl/schematracker/dm_tracker.go +++ b/ddl/schematracker/dm_tracker.go @@ -1015,6 +1015,17 @@ func (d SchemaTracker) AlterTable(ctx context.Context, sctx sessionctx.Context, handledCharsetOrCollate = true case ast.TableOptionPlacementPolicy: case ast.TableOptionEngine: + case ast.TableOptionEncryption: + tblInfo = tblInfo.Clone() + switch strings.ToUpper(opt.StrValue) { + case "Y": + tblInfo.TableEncryption = true + case "N": + tblInfo.TableEncryption = false + default: + return errors.Trace(dbterror.ErrInvalidEncryptionOption) + } + _ = d.PutTable(ident.Schema, tblInfo) default: err = dbterror.ErrUnsupportedAlterTableOption } diff --git a/ddl/table.go b/ddl/table.go index 9e6fab762d3c5..f7aba7f59faf2 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -1168,6 +1168,32 @@ func onModifyTableComment(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ return ver, nil } +func onModifyTableEncryptionOption(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { + var encryption bool + if err := job.DecodeArgs(&encryption); err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + + if job.MultiSchemaInfo != nil && job.MultiSchemaInfo.Revertible { + job.MarkNonRevertible() + return ver, nil + } + + tblInfo.TableEncryption = encryption + ver, err = updateVersionAndTableInfo(d, 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(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { var toCharset, toCollate string var needsOverwriteCols bool diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 472220bb2dcc6..0dbb58ebd0a2c 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -671,18 +671,23 @@ func (e *memtableRetriever) setDataFromTables(ctx context.Context, sctx sessionc } createTime := types.NewTime(types.FromGoTime(table.GetUpdateTime().In(loc)), createTimeTp, types.DefaultFsp) - createOptions := "" + createOptionsStr := "" if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { continue } pkType := "NONCLUSTERED" if !table.IsView() { + var createOptions []string + if table.TableEncryption { + createOptions = append(createOptions, "ENCRYPTION='Y'") + } if table.GetPartitionInfo() != nil { - createOptions = "partitioned" + createOptions = append(createOptions, "partitioned") } else if table.TableCacheStatusType == model.TableCacheStatusEnable { - createOptions = "cached=on" + createOptions = append(createOptions, "cached=on") } + createOptionsStr = strings.Join(createOptions, " ") var autoIncID interface{} hasAutoIncID, _ := infoschema.HasAutoIncrementColumn(table) if hasAutoIncID { @@ -746,7 +751,7 @@ func (e *memtableRetriever) setDataFromTables(ctx context.Context, sctx sessionc nil, // CHECK_TIME collation, // TABLE_COLLATION nil, // CHECKSUM - createOptions, // CREATE_OPTIONS + createOptionsStr, // CREATE_OPTIONS table.Comment, // TABLE_COMMENT table.ID, // TIDB_TABLE_ID shardingInfo, // TIDB_ROW_ID_SHARDING_INFO diff --git a/executor/show.go b/executor/show.go index 0ae2b9b15a1e7..5538aa3457fa3 100644 --- a/executor/show.go +++ b/executor/show.go @@ -1225,6 +1225,10 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T buf.WriteString("*/") } + if tableInfo.TableEncryption { + fmt.Fprintf(buf, " ENCRYPTION='Y'") + } + if len(tableInfo.Comment) > 0 { fmt.Fprintf(buf, " COMMENT='%s'", format.OutputFormat(tableInfo.Comment)) } diff --git a/parser/ast/ddl.go b/parser/ast/ddl.go index 08d6b55e3e289..856f85c6df564 100644 --- a/parser/ast/ddl.go +++ b/parser/ast/ddl.go @@ -2760,6 +2760,7 @@ type AlterTableSpec struct { Statistics *StatisticsSpec AttributesSpec *AttributesSpec StatsOptionsSpec *StatsOptionsSpec + TableEncrption bool } type TiFlashReplicaSpec struct { diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 52e1534a503dd..35cdbf878de1d 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -102,6 +102,8 @@ const ( ActionAlterTTLInfo ActionType = 65 ActionAlterTTLRemove ActionType = 67 + + ActionAlterEncryptionOption ActionType = 68 ) var actionMap = map[ActionType]string{ @@ -167,6 +169,7 @@ var actionMap = map[ActionType]string{ ActionReorganizePartition: "alter table reorganize partition", ActionAlterTTLInfo: "alter table ttl", ActionAlterTTLRemove: "alter table no_ttl", + ActionAlterEncryptionOption: "alter table encryption", // `ActionAlterTableAlterPartition` is removed and will never be used. // Just left a tombstone here for compatibility. __DEPRECATED_ActionAlterTableAlterPartition: "alter partition", diff --git a/parser/model/model.go b/parser/model/model.go index ba7c46bcd6333..8cec47e225de1 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -552,6 +552,8 @@ type TableInfo struct { ExchangePartitionInfo *ExchangePartitionInfo `json:"exchange_partition_info"` TTLInfo *TTLInfo `json:"ttl_info"` + + TableEncryption bool `json:"encryption"` } // SepAutoInc decides whether _rowid and auto_increment id use separate allocator. diff --git a/parser/parser.y b/parser/parser.y index 514103f7196b6..cb7419458b4de 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -11778,7 +11778,6 @@ TableOption: } | "ENCRYPTION" EqOpt EncryptionOpt { - // Parse it but will ignore it $$ = &ast.TableOption{Tp: ast.TableOptionEncryption, StrValue: $3} } | "TTL" EqOpt Identifier '+' "INTERVAL" Literal TimeUnit diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index 9c1b013defb84..b26e689a6c41a 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -62,6 +62,7 @@ var dynamicPrivs = []string{ "RESTRICTED_USER_ADMIN", // User can not have their access revoked by SUPER users. "RESTRICTED_CONNECTION_ADMIN", // Can not be killed by PROCESS/CONNECTION_ADMIN privilege "RESTRICTED_REPLICA_WRITER_ADMIN", // Can write to the sever even when tidb_restriced_read_only is turned on. + "TABLE_ENCRYPTION_ADMIN", // Enables a user to override default encryption settings when table_encryption_privilege_check is enabled } var dynamicPrivLock sync.Mutex var defaultTokenLife = 15 * time.Minute diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 9c7f0654bb65b..30ca32e0734b6 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -582,6 +582,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: DisconnectOnExpiredPassword, Value: On, Type: TypeBool, ReadOnly: true, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return BoolToOnOff(!IsSandBoxModeEnabled.Load()), nil }}, + {Scope: ScopeGlobal, Name: TableEncryptionPrivilegeCheck, Value: Off, Type: TypeBool}, /* TiDB specific variables */ {Scope: ScopeGlobal, Name: TiDBTSOClientBatchMaxWaitTime, Value: strconv.FormatFloat(DefTiDBTSOClientBatchMaxWaitTime, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: 10, @@ -1162,6 +1163,7 @@ var defaultSysVars = []*SysVar{ }}, /* The system variables below have GLOBAL and SESSION scope */ + {Scope: ScopeGlobal | ScopeSession, Name: DefaultTableEncryption, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePlanReplayerCapture, Value: BoolToOnOff(false), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnablePlanReplayerCapture = TiDBOptOn(val) @@ -2568,4 +2570,9 @@ const ( ValidatePasswordSpecialCharCount = "validate_password.special_char_count" // ValidatePasswordDictionary specified the dictionary that validate_password uses for checking passwords. Each word is separated by semicolon (;). ValidatePasswordDictionary = "validate_password.dictionary" + // DefaultTableEncryption defines the default encryption setting applied to schemas when they are created without specifying an ENCRYPTION clause. + DefaultTableEncryption = "default_table_encryption" + // TableEncryptionPrivilegeCheck controls the TABLE_ENCRYPTION_ADMIN privilege check that occurs + // when creating or altering a table with an encryption setting that differs from the default schema encryption + TableEncryptionPrivilegeCheck = "table_encryption_privilege_check" ) diff --git a/util/dbterror/ddl_terror.go b/util/dbterror/ddl_terror.go index ecc7a9482018f..1f3daefb192d5 100644 --- a/util/dbterror/ddl_terror.go +++ b/util/dbterror/ddl_terror.go @@ -435,4 +435,6 @@ var ( ErrTempTableNotAllowedWithTTL = ClassDDL.NewStd(mysql.ErrTempTableNotAllowedWithTTL) // ErrUnsupportedTTLReferencedByFK returns when the TTL config is set for a table referenced by foreign key ErrUnsupportedTTLReferencedByFK = ClassDDL.NewStd(mysql.ErrUnsupportedTTLReferencedByFK) + // ErrInvalidEncryptionOption returns when en encryption option is invalid. + ErrInvalidEncryptionOption = ClassDDL.NewStd(mysql.ErrInvalidEncryptionOption) )