Skip to content

Commit

Permalink
*: support table encryption (pingcap#1)
Browse files Browse the repository at this point in the history
  • Loading branch information
CbcWestwolf authored Feb 24, 2023
1 parent 3fd392a commit dfbef74
Show file tree
Hide file tree
Showing 15 changed files with 134 additions and 6 deletions.
19 changes: 19 additions & 0 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")) }()
Expand Down
46 changes: 46 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -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
}
Expand Down Expand Up @@ -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)
Expand Down
2 changes: 2 additions & 0 deletions ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down
2 changes: 1 addition & 1 deletion ddl/multi_schema_change.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
11 changes: 11 additions & 0 deletions ddl/schematracker/dm_tracker.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
26 changes: 26 additions & 0 deletions ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
13 changes: 9 additions & 4 deletions executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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
Expand Down
4 changes: 4 additions & 0 deletions executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
}
Expand Down
1 change: 1 addition & 0 deletions parser/ast/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -2760,6 +2760,7 @@ type AlterTableSpec struct {
Statistics *StatisticsSpec
AttributesSpec *AttributesSpec
StatsOptionsSpec *StatsOptionsSpec
TableEncrption bool
}

type TiFlashReplicaSpec struct {
Expand Down
3 changes: 3 additions & 0 deletions parser/model/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,8 @@ const (

ActionAlterTTLInfo ActionType = 65
ActionAlterTTLRemove ActionType = 67

ActionAlterEncryptionOption ActionType = 68
)

var actionMap = map[ActionType]string{
Expand Down Expand Up @@ -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",
Expand Down
2 changes: 2 additions & 0 deletions parser/model/model.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
1 change: 0 additions & 1 deletion parser/parser.y
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
1 change: 1 addition & 0 deletions privilege/privileges/privileges.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
7 changes: 7 additions & 0 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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"
)
2 changes: 2 additions & 0 deletions util/dbterror/ddl_terror.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
)

0 comments on commit dfbef74

Please sign in to comment.