Skip to content

Commit

Permalink
Merge branch 'master' into issue-29103
Browse files Browse the repository at this point in the history
  • Loading branch information
Mini256 committed Dec 20, 2021
2 parents fa5f208 + b9d9f19 commit 56c570d
Show file tree
Hide file tree
Showing 48 changed files with 1,250 additions and 1,081 deletions.
1 change: 1 addition & 0 deletions .codecov.yml
Original file line number Diff line number Diff line change
Expand Up @@ -43,4 +43,5 @@ ignore:
- "executor/seqtest/.*"
- "metrics/.*"
- "expression/generator/.*"
- "br/pkg/mock/.*"

10 changes: 8 additions & 2 deletions br/pkg/lightning/backend/local/local.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,8 @@ const (
gRPCBackOffMaxDelay = 10 * time.Minute

// See: https://github.com/tikv/tikv/blob/e030a0aae9622f3774df89c62f21b2171a72a69e/etc/config-template.toml#L360
regionMaxKeyCount = 1_440_000
// lower the max-key-count to avoid tikv trigger region auto split
regionMaxKeyCount = 1_280_000
defaultRegionSplitSize = 96 * units.MiB

propRangeIndex = "tikv.range_index"
Expand Down Expand Up @@ -782,7 +783,12 @@ func (local *local) WriteToTiKV(
size := int64(0)
totalCount := int64(0)
firstLoop := true
regionMaxSize := regionSplitSize * 4 / 3
// if region-split-size <= 96MiB, we bump the threshold a bit to avoid too many retry split
// because the range-properties is not 100% accurate
regionMaxSize := regionSplitSize
if regionSplitSize <= defaultRegionSplitSize {
regionMaxSize = regionSplitSize * 4 / 3
}

for iter.First(); iter.Valid(); iter.Next() {
size += int64(len(iter.Key()) + len(iter.Value()))
Expand Down
4 changes: 2 additions & 2 deletions br/pkg/lightning/restore/table_restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -998,8 +998,8 @@ func estimateCompactionThreshold(cp *checkpoints.TableCheckpoint, factor int64)
threshold := totalRawFileSize / 512
threshold = utils.NextPowerOfTwo(threshold)
if threshold < compactionLowerThreshold {
// disable compaction if threshold is smaller than lower bound
threshold = 0
// too may small SST files will cause inaccuracy of region range estimation,
threshold = compactionLowerThreshold
} else if threshold > compactionUpperThreshold {
threshold = compactionUpperThreshold
}
Expand Down
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
9 changes: 5 additions & 4 deletions cmd/explaintest/r/new_character_set_builtin.result
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
set @@sql_mode = '';
drop table if exists t;
create table t (a char(20) charset utf8mb4, b char(20) charset gbk, c binary(20));
insert into t values ('一二三', '一二三', '一二三');
Expand Down Expand Up @@ -244,17 +245,17 @@ insert into t values ('65'), ('123456'), ('123456789');
select char(a using gbk), char(a using utf8), char(a) from t;
char(a using gbk) char(a using utf8) char(a)
A A A
釦 �@ �@
NULL [� [�
釦  �@
[ [ [�
select char(12345678 using gbk);
char(12345678 using gbk)
糰N
set @@tidb_enable_vectorized_expression = true;
select char(a using gbk), char(a using utf8), char(a) from t;
char(a using gbk) char(a using utf8) char(a)
A A A
釦 �@ �@
NULL [� [�
釦  �@
[ [ [�
select char(12345678 using gbk);
char(12345678 using gbk)
糰N
Expand Down
1 change: 1 addition & 0 deletions cmd/explaintest/t/new_character_set_builtin.test
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
set @@sql_mode = '';
-- test for builtin function hex(), length(), ascii(), octet_length()
drop table if exists t;
create table t (a char(20) charset utf8mb4, b char(20) charset gbk, c binary(20));
Expand Down
2 changes: 2 additions & 0 deletions ddl/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -310,4 +310,6 @@ var (
errDependentByFunctionalIndex = dbterror.ClassDDL.NewStd(mysql.ErrDependentByFunctionalIndex)
// errFunctionalIndexOnBlob when the expression of expression index returns blob or text.
errFunctionalIndexOnBlob = dbterror.ClassDDL.NewStd(mysql.ErrFunctionalIndexOnBlob)
// ErrIncompatibleTiFlashAndPlacement when placement and tiflash replica options are set at the same time
ErrIncompatibleTiFlashAndPlacement = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("Placement and tiflash replica options cannot be set at the same time", nil))
)
16 changes: 16 additions & 0 deletions ddl/placement_policy.go
Original file line number Diff line number Diff line change
Expand Up @@ -381,3 +381,19 @@ func checkPlacementPolicyNotUsedByTable(tblInfo *model.TableInfo, policy *model.

return nil
}

func tableHasPlacementSettings(tblInfo *model.TableInfo) bool {
if tblInfo.DirectPlacementOpts != nil || tblInfo.PlacementPolicyRef != nil {
return true
}

if tblInfo.Partition != nil {
for _, def := range tblInfo.Partition.Definitions {
if def.DirectPlacementOpts != nil || def.PlacementPolicyRef != nil {
return true
}
}
}

return false
}
105 changes: 105 additions & 0 deletions ddl/placement_sql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (

. "github.com/pingcap/check"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/ddl/placement"
mysql "github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/parser/model"
Expand Down Expand Up @@ -472,3 +473,107 @@ func (s *testDBSuite6) TestEnablePlacementCheck(c *C) {
tk.MustGetErrCode("create table m (c int) partition by range (c) (partition p1 values less than (200) followers=2);", mysql.ErrUnsupportedDDLOperation)
tk.MustGetErrCode("alter table t partition p1 placement policy=\"placement_x\";", mysql.ErrUnsupportedDDLOperation)
}

func (s *testDBSuite6) TestPlacementTiflashCheck(c *C) {
tk := testkit.NewTestKit(c, s.store)
se, err := session.CreateSession4Test(s.store)
c.Assert(err, IsNil)
_, err = se.Execute(context.Background(), "set @@global.tidb_enable_alter_placement=1")
c.Assert(err, IsNil)

c.Assert(failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`), IsNil)
defer func() {
err := failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount")
c.Assert(err, IsNil)
}()

tk.MustExec("use test")
tk.MustExec("drop placement policy if exists p1")
tk.MustExec("drop table if exists tp")

tk.MustExec("create placement policy p1 primary_region='r1' regions='r1'")
defer tk.MustExec("drop placement policy if exists p1")

tk.MustExec(`CREATE TABLE tp (id INT) PARTITION BY RANGE (id) (
PARTITION p0 VALUES LESS THAN (100),
PARTITION p1 VALUES LESS THAN (1000)
)`)
defer tk.MustExec("drop table if exists tp")
tk.MustExec("alter table tp set tiflash replica 1")

err = tk.ExecToErr("alter table tp placement policy p1")
c.Assert(ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err), IsTrue)
err = tk.ExecToErr("alter table tp primary_region='r2' regions='r2'")
c.Assert(ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err), IsTrue)
err = tk.ExecToErr("alter table tp partition p0 placement policy p1")
c.Assert(ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err), IsTrue)
err = tk.ExecToErr("alter table tp partition p0 primary_region='r2' regions='r2'")
c.Assert(ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err), IsTrue)
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" +
"PARTITION BY RANGE (`id`)\n" +
"(PARTITION `p0` VALUES LESS THAN (100),\n" +
" PARTITION `p1` VALUES LESS THAN (1000))"))

tk.MustExec("drop table tp")
tk.MustExec(`CREATE TABLE tp (id INT) placement policy p1 PARTITION BY RANGE (id) (
PARTITION p0 VALUES LESS THAN (100),
PARTITION p1 VALUES LESS THAN (1000)
)`)
err = tk.ExecToErr("alter table tp set tiflash replica 1")
c.Assert(ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err), IsTrue)
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`p1` */\n" +
"PARTITION BY RANGE (`id`)\n" +
"(PARTITION `p0` VALUES LESS THAN (100),\n" +
" PARTITION `p1` VALUES LESS THAN (1000))"))

tk.MustExec("drop table tp")
tk.MustExec(`CREATE TABLE tp (id INT) PARTITION BY RANGE (id) (
PARTITION p0 VALUES LESS THAN (100) placement policy p1 ,
PARTITION p1 VALUES LESS THAN (1000)
)`)
err = tk.ExecToErr("alter table tp set tiflash replica 1")
c.Assert(ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err), IsTrue)
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" +
"PARTITION BY RANGE (`id`)\n" +
"(PARTITION `p0` VALUES LESS THAN (100) /*T![placement] PLACEMENT POLICY=`p1` */,\n" +
" PARTITION `p1` VALUES LESS THAN (1000))"))

tk.MustExec("drop table tp")
tk.MustExec(`CREATE TABLE tp (id INT) primary_region='r2' regions='r2' PARTITION BY RANGE (id) (
PARTITION p0 VALUES LESS THAN (100),
PARTITION p1 VALUES LESS THAN (1000)
)`)
err = tk.ExecToErr("alter table tp set tiflash replica 1")
c.Assert(ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err), IsTrue)
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PRIMARY_REGION=\"r2\" REGIONS=\"r2\" */\n" +
"PARTITION BY RANGE (`id`)\n" +
"(PARTITION `p0` VALUES LESS THAN (100),\n" +
" PARTITION `p1` VALUES LESS THAN (1000))"))

tk.MustExec("drop table tp")
tk.MustExec(`CREATE TABLE tp (id INT) PARTITION BY RANGE (id) (
PARTITION p0 VALUES LESS THAN (100) primary_region='r3' regions='r3',
PARTITION p1 VALUES LESS THAN (1000)
)`)
err = tk.ExecToErr("alter table tp set tiflash replica 1")
c.Assert(ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err), IsTrue)
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" +
"PARTITION BY RANGE (`id`)\n" +
"(PARTITION `p0` VALUES LESS THAN (100) /*T![placement] PRIMARY_REGION=\"r3\" REGIONS=\"r3\" */,\n" +
" PARTITION `p1` VALUES LESS THAN (1000))"))
}
12 changes: 12 additions & 0 deletions ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -954,6 +954,10 @@ func (w *worker) onSetTableFlashReplica(t *meta.Meta, job *model.Job) (ver int64
return ver, errors.Trace(err)
}

if replicaInfo.Count > 0 && tableHasPlacementSettings(tblInfo) {
return ver, errors.Trace(ErrIncompatibleTiFlashAndPlacement)
}

// Ban setting replica count for tables in system database.
if tidb_util.IsMemOrSysDB(job.SchemaName) {
return ver, errors.Trace(errUnsupportedAlterReplicaForSysTable)
Expand Down Expand Up @@ -1274,6 +1278,10 @@ func onAlterTablePartitionPlacement(t *meta.Meta, job *model.Job) (ver int64, er
return 0, err
}

if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Count > 0 {
return 0, errors.Trace(ErrIncompatibleTiFlashAndPlacement)
}

ptInfo := tblInfo.GetPartitionInfo()
var partitionDef *model.PartitionDefinition
definitions := ptInfo.Definitions
Expand Down Expand Up @@ -1341,6 +1349,10 @@ func onAlterTablePlacement(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
return 0, err
}

if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Count > 0 {
return 0, errors.Trace(ErrIncompatibleTiFlashAndPlacement)
}

if _, err = checkPlacementPolicyRefValidAndCanNonValidJob(t, job, policyRefInfo); err != nil {
return 0, errors.Trace(err)
}
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
Loading

0 comments on commit 56c570d

Please sign in to comment.