Skip to content

Commit

Permalink
Merge branch 'master' into br-increase-gcov
Browse files Browse the repository at this point in the history
  • Loading branch information
joccau authored Dec 20, 2021
2 parents e7b82a3 + e3c56b7 commit bb3391f
Show file tree
Hide file tree
Showing 25 changed files with 9,311 additions and 8,977 deletions.
1 change: 1 addition & 0 deletions br/pkg/restore/pipeline_items.go
Original file line number Diff line number Diff line change
Expand Up @@ -360,6 +360,7 @@ func (b *tikvSender) restoreWorker(ctx context.Context, ranges <-chan drainResul
eg.Go(func() error {
e := b.client.RestoreFiles(ectx, files, r.result.RewriteRules, b.updateCh)
if e != nil {
r.done()
return e
}
log.Info("restore batch done", rtree.ZapRanges(r.result.Ranges))
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/storage/local.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@ import (
)

const (
localDirPerm os.FileMode = 0o755
localDirPerm os.FileMode = 0o777
localFilePerm os.FileMode = 0o644
// LocalURIPrefix represents the local storage prefix.
LocalURIPrefix = "file://"
Expand Down
41 changes: 9 additions & 32 deletions docs/design/2021-08-18-charsets.md
Original file line number Diff line number Diff line change
Expand Up @@ -98,8 +98,10 @@ After receiving the non-utf-8 character set request, this solution will convert
### Collation

Add gbk_chinese_ci and gbk_bin collations. In addition, considering the performance, we can add the collation of utf8mb4 (gbk_utf8mb4_bin).
- To support gbk_chinese_ci and gbk_bin collations, it needs to turn on the `new_collations_enabled_on_first_bootstrap` switch.
- If `new_collations_enabled_on_first_bootstrap` is off, it only supports gbk_utf8mb4_bin which does not need to be converted to gbk charset before processing.
- Implement the Collator and WildcardPattern interface functions for each collation.
- gbk_chinese_ci and gbk_bin need to convert utf-8 to gbk encoding and then generate a sort key. gbk_utf8mb4_bin does not need to be converted to gbk code for processing.
- gbk_chinese_ci and gbk_bin need to convert utf-8 to gbk encoding and then generate a sort key.
- Implement the corresponding functions in the Coprocessor.

### DDL
Expand All @@ -119,43 +121,18 @@ Other behaviors that need to be dealt with:
#### Compatibility between TiDB versions

- Upgrade compatibility:
- Upgrades from versions below 4.0 do not support gbk or any character sets other than the original five (binary, ascii, latin1, utf8, utf8mb4).
- Upgrade from version 4.0 or higher
- There may be compatibility issues when performing non-utf-8-related operations during the rolling upgrade.
- The new version of the cluster is expected to have no compatibility issues when reading old data.
- There may be compatibility issues when performing operations during the rolling upgrade.
- The new version of the cluster is expected to have no compatibility issues when reading old data.
- Downgrade compatibility:
- Downgrade is not compatible. The index key uses the table of gbk_bin/gbk_chinese_ci. The lower version of TiDB will have problems when decoding, and it needs to be transcoded before downgrading.

#### Compatibility with MySQL

Illegal character related issue:
- Illegal character related issue:
- Due to the internal conversion of non-utf-8-related encoding to utf8 for processing, it is not fully compatible with MySQL in some cases in terms of illegal character processing. TiDB controls its behavior through sql_mode.

```sql
create table t3(a char(10) charset gbk);
insert into t3 values ('a');
// 0xcee5 is a valid gbk hex literal but invalid utf8mb4 hex literal.
select hex(concat(a, 0xcee5)) from t3;
-- mysql 61cee5
// 0xe4b880 is an invalid gbk hex literal but valid utf8mb4 hex literal.
select hex(concat(a, 0xe4b880)) from t3;
-- mysql 61e4b880 (test on mysql 5.7 and 8.0.22)
-- mysql returns "Cannot convert string '\x80' from binary to gbk" (test on mysql 8.0.25 and 8.0.26). TiDB will be compatible with this behavior.
// 0x80 is a hex literal that invalid for neither gbk nor utf8mb4.
select hex(concat(a, 0x80)) from t3;
-- mysql 6180 (test on mysql 5.7 and 8.0.22)
-- mysql returns "Cannot convert string '\x80' from binary to gbk" (test on mysql 8.0.25 and 8.0.26). TiDB will be compatible with this behavior.
set @@sql_mode = '';
insert into t3 values (0x80);
-- mysql gets a warning and insert null values (warning: "Incorrect string value: '\x80' for column 'a' at row 1")
set @@sql_mode = 'STRICT_TRANS_TABLES';
insert into t3 values (0x80);
-- mysql returns "Incorrect string value: '\x80' for column 'a' at row 1"
```
- Collation
- Fully support `gbk_bin` and `gbk_chinese_ci` only when the config `new_collations_enabled_on_first_bootstrap` is enabled. Otherwise, it only supports gbk_utf8mb4_bin.

#### Compatibility with other components

Expand Down
1 change: 1 addition & 0 deletions executor/aggregate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1483,6 +1483,7 @@ func TestAvgDecimal(t *testing.T) {
tk.MustExec("insert into td values (0,29815);")
tk.MustExec("insert into td values (10017,-32661);")
tk.MustQuery(" SELECT AVG( col_bigint / col_smallint) AS field1 FROM td;").Sort().Check(testkit.Rows("25769363061037.62077260"))
tk.MustQuery(" SELECT AVG(col_bigint) OVER (PARTITION BY col_smallint) as field2 FROM td where col_smallint = -23828;").Sort().Check(testkit.Rows("4.0000"))
tk.MustExec("drop table td;")
}

Expand Down
2 changes: 1 addition & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -4192,7 +4192,7 @@ func (b *executorBuilder) buildWindow(v *plannercore.PhysicalWindow) Executor {
partialResults := make([]aggfuncs.PartialResult, 0, len(v.WindowFuncDescs))
resultColIdx := v.Schema().Len() - len(v.WindowFuncDescs)
for _, desc := range v.WindowFuncDescs {
aggDesc, err := aggregation.NewAggFuncDesc(b.ctx, desc.Name, desc.Args, false)
aggDesc, err := aggregation.NewAggFuncDescForWindowFunc(b.ctx, desc, false)
if err != nil {
b.err = err
return nil
Expand Down
28 changes: 27 additions & 1 deletion executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ func (s *testSerialSuite1) TestSetVar(c *C) {
tk.MustQuery(`select @@global.low_priority_updates;`).Check(testkit.Rows("0"))
tk.MustExec(`set @@global.low_priority_updates="ON";`)
tk.MustQuery(`select @@global.low_priority_updates;`).Check(testkit.Rows("1"))
tk.MustExec(`set @@global.low_priority_updates=DEFAULT;`) // It will be set to compiled-in default value.
tk.MustExec(`set @@global.low_priority_updates=DEFAULT;`) // It will be set to default var value.
tk.MustQuery(`select @@global.low_priority_updates;`).Check(testkit.Rows("0"))
// For session
tk.MustQuery(`select @@session.low_priority_updates;`).Check(testkit.Rows("0"))
Expand Down Expand Up @@ -1387,6 +1387,32 @@ func (s *testSuite5) TestEnableNoopFunctionsVar(c *C) {

}

// https://github.com/pingcap/tidb/issues/29670
func (s *testSuite5) TestDefaultBehavior(c *C) {
tk := testkit.NewTestKit(c, s.store)

tk.MustQuery("SELECT @@default_storage_engine").Check(testkit.Rows("InnoDB"))
tk.MustExec("SET GLOBAL default_storage_engine = 'somethingweird'")
tk.MustExec("SET default_storage_engine = 'MyISAM'")
tk.MustQuery("SELECT @@default_storage_engine").Check(testkit.Rows("MyISAM"))
tk.MustExec("SET default_storage_engine = DEFAULT") // reads from global value
tk.MustQuery("SELECT @@default_storage_engine").Check(testkit.Rows("somethingweird"))
tk.MustExec("SET @@SESSION.default_storage_engine = @@GLOBAL.default_storage_engine") // example from MySQL manual
tk.MustQuery("SELECT @@default_storage_engine").Check(testkit.Rows("somethingweird"))
tk.MustExec("SET GLOBAL default_storage_engine = 'somethingweird2'")
tk.MustExec("SET default_storage_engine = @@GLOBAL.default_storage_engine") // variation of example
tk.MustQuery("SELECT @@default_storage_engine").Check(testkit.Rows("somethingweird2"))
tk.MustExec("SET default_storage_engine = DEFAULT") // restore default again for session global
tk.MustExec("SET GLOBAL default_storage_engine = DEFAULT") // restore default for global
tk.MustQuery("SELECT @@SESSION.default_storage_engine, @@GLOBAL.default_storage_engine").Check(testkit.Rows("somethingweird2 InnoDB"))

// Try sql_mode option which has validation
err := tk.ExecToErr("SET GLOBAL sql_mode = 'DEFAULT'") // illegal now
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, `ERROR 1231 (42000): Variable 'sql_mode' can't be set to the value of 'DEFAULT'`)
tk.MustExec("SET GLOBAL sql_mode = DEFAULT")
}

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

Expand Down
1 change: 1 addition & 0 deletions executor/tiflash_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -829,6 +829,7 @@ func (s *tiflashTestSuite) TestAvgOverflow(c *C) {
tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"")
tk.MustExec("set @@session.tidb_enforce_mpp=ON")
tk.MustQuery(" SELECT AVG( col_bigint / col_smallint) AS field1 FROM td;").Sort().Check(testkit.Rows("25769363061037.62077260"))
tk.MustQuery(" SELECT AVG(col_bigint) OVER (PARTITION BY col_smallint) as field2 FROM td where col_smallint = -23828;").Sort().Check(testkit.Rows("4.0000"))
tk.MustExec("drop table if exists td;")
}

Expand Down
9 changes: 9 additions & 0 deletions expression/aggregation/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ type AggFuncDesc struct {
}

// NewAggFuncDesc creates an aggregation function signature descriptor.
// this func cannot be called twice as the TypeInfer has changed the type of args in the first time.
func NewAggFuncDesc(ctx sessionctx.Context, name string, args []expression.Expression, hasDistinct bool) (*AggFuncDesc, error) {
b, err := newBaseFuncDesc(ctx, name, args)
if err != nil {
Expand All @@ -51,6 +52,14 @@ func NewAggFuncDesc(ctx sessionctx.Context, name string, args []expression.Expre
return &AggFuncDesc{baseFuncDesc: b, HasDistinct: hasDistinct}, nil
}

// NewAggFuncDescForWindowFunc creates an aggregation function from window functions, where baseFuncDesc may be ready.
func NewAggFuncDescForWindowFunc(ctx sessionctx.Context, Desc *WindowFuncDesc, hasDistinct bool) (*AggFuncDesc, error) {
if Desc.RetTp == nil { // safety check
return NewAggFuncDesc(ctx, Desc.Name, Desc.Args, hasDistinct)
}
return &AggFuncDesc{baseFuncDesc: baseFuncDesc{Desc.Name, Desc.Args, Desc.RetTp}, HasDistinct: hasDistinct}, nil
}

// String implements the fmt.Stringer interface.
func (a *AggFuncDesc) String() string {
buffer := bytes.NewBufferString(a.Name)
Expand Down
2 changes: 1 addition & 1 deletion expression/helper_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ func TestGetTimeValue(t *testing.T) {
require.Equal(t, "2012-12-12 00:00:00", timeValue.String())

sessionVars := ctx.GetSessionVars()
err = variable.SetSessionSystemVar(sessionVars, "timestamp", "default")
err = variable.SetSessionSystemVar(sessionVars, "timestamp", "0")
require.NoError(t, err)
v, err = GetTimeValue(ctx, "2012-12-12 00:00:00", mysql.TypeTimestamp, types.MinFsp)
require.NoError(t, err)
Expand Down
27 changes: 23 additions & 4 deletions parser/ast/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -1856,6 +1856,7 @@ const (
AdminShowTelemetry
AdminResetTelemetryID
AdminReloadStatistics
AdminFlushPlanCache
)

// HandleRange represents a range where handle value >= Begin and < End.
Expand All @@ -1864,6 +1865,15 @@ type HandleRange struct {
End int64
}

type StatementScope int

const (
StatementScopeNone StatementScope = iota
StatementScopeSession
StatementScopeInstance
StatementScopeGlobal
)

// ShowSlowType defines the type for SlowSlow statement.
type ShowSlowType int

Expand Down Expand Up @@ -1929,10 +1939,11 @@ type AdminStmt struct {
JobIDs []int64
JobNumber int64

HandleRanges []HandleRange
ShowSlow *ShowSlow
Plugins []string
Where ExprNode
HandleRanges []HandleRange
ShowSlow *ShowSlow
Plugins []string
Where ExprNode
StatementScope StatementScope
}

// Restore implements Node interface.
Expand Down Expand Up @@ -2070,6 +2081,14 @@ func (n *AdminStmt) Restore(ctx *format.RestoreCtx) error {
ctx.WriteKeyWord("RESET TELEMETRY_ID")
case AdminReloadStatistics:
ctx.WriteKeyWord("RELOAD STATS_EXTENDED")
case AdminFlushPlanCache:
if n.StatementScope == StatementScopeSession {
ctx.WriteKeyWord("FLUSH SESSION PLAN_CACHE")
} else if n.StatementScope == StatementScopeInstance {
ctx.WriteKeyWord("FLUSH INSTANCE PLAN_CACHE")
} else if n.StatementScope == StatementScopeGlobal {
ctx.WriteKeyWord("FLUSH GLOBAL PLAN_CACHE")
}
default:
return errors.New("Unsupported AdminStmt type")
}
Expand Down
1 change: 1 addition & 0 deletions parser/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -532,6 +532,7 @@ var tokenMap = map[string]int{
"PESSIMISTIC": pessimistic,
"PLACEMENT": placement,
"PLAN": plan,
"PLAN_CACHE": planCache,
"PLUGINS": plugins,
"POLICY": policy,
"POSITION": position,
Expand Down
Loading

0 comments on commit bb3391f

Please sign in to comment.