Skip to content

Commit

Permalink
Merge branch 'build-range-once' of https://github.com/xuyifangreeneye…
Browse files Browse the repository at this point in the history
…s/tidb into build-range-once
  • Loading branch information
xuyifangreeneyes committed Aug 24, 2022
2 parents afd056b + 2661e8b commit bb5f1c0
Show file tree
Hide file tree
Showing 30 changed files with 686 additions and 237 deletions.
1 change: 1 addition & 0 deletions bindinfo/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ go_test(
],
embed = [":bindinfo"],
flaky = True,
shard_count = 50,
deps = [
"//config",
"//domain",
Expand Down
25 changes: 25 additions & 0 deletions cmd/explaintest/r/explain_complex.result
Original file line number Diff line number Diff line change
Expand Up @@ -261,3 +261,28 @@ Sort 1.00 root test.org_department.left_value
└─TableReader(Probe) 9.99 root data:Selection
└─Selection 9.99 cop[tikv] eq(test.org_employee_position.status, 1000), not(isnull(test.org_employee_position.position_id))
└─TableFullScan 10000.00 cop[tikv] table:ep keep order:false, stats:pseudo
create table test.Tab_A (id int primary key,bid int,cid int,name varchar(20),type varchar(20),num int,amt decimal(11,2));
create table test.Tab_B (id int primary key,name varchar(20));
create table test.Tab_C (id int primary key,name varchar(20),amt decimal(11,2));
insert into test.Tab_A values(2,2,2,'A01','01',112,111);
insert into test.Tab_A values(4,4,4,'A02','02',112,111);
insert into test.Tab_B values(2,'B01');
insert into test.Tab_B values(4,'B02');
insert into test.Tab_C values(2,'C01',22);
insert into test.Tab_C values(4,'C01',5);
explain select Tab_A.name AAA,Tab_B.name BBB,Tab_A.amt Aamt, Tab_C.amt Bamt,IFNULL(Tab_C.amt, 0) FROM Tab_A left join Tab_B on Tab_A.bid=Tab_B.id left join Tab_C on Tab_A.cid=Tab_C.id and Tab_A.type='01' where Tab_A.num=112;
id estRows task access object operator info
Projection_8 15.62 root test.tab_a.name, test.tab_b.name, test.tab_a.amt, test.tab_c.amt, ifnull(test.tab_c.amt, 0)->Column#13
└─IndexJoin_13 15.62 root left outer join, inner:TableReader_10, outer key:test.tab_a.cid, inner key:test.tab_c.id, equal cond:eq(test.tab_a.cid, test.tab_c.id), left cond:eq(test.tab_a.type, "01")
├─IndexJoin_24(Build) 12.50 root left outer join, inner:TableReader_21, outer key:test.tab_a.bid, inner key:test.tab_b.id, equal cond:eq(test.tab_a.bid, test.tab_b.id)
│ ├─TableReader_33(Build) 10.00 root data:Selection_32
│ │ └─Selection_32 10.00 cop[tikv] eq(test.tab_a.num, 112)
│ │ └─TableFullScan_31 10000.00 cop[tikv] table:Tab_A keep order:false, stats:pseudo
│ └─TableReader_21(Probe) 1.00 root data:TableRangeScan_20
│ └─TableRangeScan_20 1.00 cop[tikv] table:Tab_B range: decided by [test.tab_a.bid], keep order:false, stats:pseudo
└─TableReader_10(Probe) 1.00 root data:TableRangeScan_9
└─TableRangeScan_9 1.00 cop[tikv] table:Tab_C range: decided by [test.tab_a.cid], keep order:false, stats:pseudo
select Tab_A.name AAA,Tab_B.name BBB,Tab_A.amt Aamt, Tab_C.amt Bamt,IFNULL(Tab_C.amt, 0) FROM Tab_A left join Tab_B on Tab_A.bid=Tab_B.id left join Tab_C on Tab_A.cid=Tab_C.id and Tab_A.type='01' where Tab_A.num=112;
AAA BBB Aamt Bamt IFNULL(Tab_C.amt, 0)
A01 B01 111.00 22.00 22.00
A02 B02 111.00 NULL 0
16 changes: 16 additions & 0 deletions cmd/explaintest/t/explain_complex.test
Original file line number Diff line number Diff line change
Expand Up @@ -174,3 +174,19 @@ CREATE TABLE org_position (
) ENGINE=InnoDB DEFAULT CHARSET=utf8;

explain format = 'brief' SELECT d.id, d.ctx, d.name, d.left_value, d.right_value, d.depth, d.leader_id, d.status, d.created_on, d.updated_on FROM org_department AS d LEFT JOIN org_position AS p ON p.department_id = d.id AND p.status = 1000 LEFT JOIN org_employee_position AS ep ON ep.position_id = p.id AND ep.status = 1000 WHERE (d.ctx = 1 AND (ep.user_id = 62 OR d.id = 20 OR d.id = 20) AND d.status = 1000) GROUP BY d.id ORDER BY d.left_value;

create table test.Tab_A (id int primary key,bid int,cid int,name varchar(20),type varchar(20),num int,amt decimal(11,2));
create table test.Tab_B (id int primary key,name varchar(20));
create table test.Tab_C (id int primary key,name varchar(20),amt decimal(11,2));

insert into test.Tab_A values(2,2,2,'A01','01',112,111);
insert into test.Tab_A values(4,4,4,'A02','02',112,111);
insert into test.Tab_B values(2,'B01');
insert into test.Tab_B values(4,'B02');
insert into test.Tab_C values(2,'C01',22);
insert into test.Tab_C values(4,'C01',5);

explain select Tab_A.name AAA,Tab_B.name BBB,Tab_A.amt Aamt, Tab_C.amt Bamt,IFNULL(Tab_C.amt, 0) FROM Tab_A left join Tab_B on Tab_A.bid=Tab_B.id left join Tab_C on Tab_A.cid=Tab_C.id and Tab_A.type='01' where Tab_A.num=112;

select Tab_A.name AAA,Tab_B.name BBB,Tab_A.amt Aamt, Tab_C.amt Bamt,IFNULL(Tab_C.amt, 0) FROM Tab_A left join Tab_B on Tab_A.bid=Tab_B.id left join Tab_C on Tab_A.cid=Tab_C.id and Tab_A.type='01' where Tab_A.num=112;

4 changes: 0 additions & 4 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -243,10 +243,6 @@ func alterTablePartitionBundles(t *meta.Meta, tblInfo *model.TableInfo, addingDe
p.Definitions = append(tblInfo.Partition.Definitions, addingDefinitions...)
tblInfo.Partition = &p

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

// bundle for table should be recomputed because it includes some default configs for partitions
tblBundle, err := placement.NewTableBundle(t, tblInfo)
if err != nil {
Expand Down
16 changes: 0 additions & 16 deletions ddl/placement_policy.go
Original file line number Diff line number Diff line change
Expand Up @@ -417,19 +417,3 @@ func checkPlacementPolicyNotUsedByTable(tblInfo *model.TableInfo, policy *model.

return nil
}

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

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

return false
}
80 changes: 65 additions & 15 deletions ddl/placement_sql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,11 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/domain/infosync"
mysql "github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/util/dbterror"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -578,48 +578,89 @@ func TestPlacementMode(t *testing.T) {
tk.MustQuery("show create database db2").Check(testkit.Rows("db2 CREATE DATABASE `db2` /*!40100 DEFAULT CHARACTER SET utf8mb4 */"))
}

func checkTiflashReplicaSet(t *testing.T, do *domain.Domain, db, tb string, cnt uint64) {
tbl, err := do.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(tb))
require.NoError(t, err)

tiflashReplica := tbl.Meta().TiFlashReplica
if cnt == 0 {
require.Nil(t, tiflashReplica)
return
}

CheckPlacementRule(infosync.GetMockTiFlash(), *infosync.MakeNewRule(tbl.Meta().ID, 1, nil))
require.NotNil(t, tiflashReplica)
require.Equal(t, cnt, tiflashReplica.Count)
}

func TestPlacementTiflashCheck(t *testing.T) {
store := testkit.CreateMockStore(t)
store, do := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`))
defer func() {
err := failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount")
require.NoError(t, err)
}()

tiflash := infosync.NewMockTiFlash()
infosync.SetMockTiFlash(tiflash)
defer func() {
tiflash.Lock()
tiflash.StatusServer.Close()
tiflash.Unlock()
}()

tk.MustExec("use test")
tk.MustExec("drop placement policy if exists p1")
tk.MustExec("drop placement policy if exists p2")
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 placement policy p2 primary_region='r2' regions='r1,r2'")
defer tk.MustExec("drop placement policy if exists p2")

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")
checkTiflashReplicaSet(t, do, "test", "tp", 0)
tk.MustExec("alter table tp set tiflash replica 1")

err := tk.ExecToErr("alter table tp placement policy p1")
require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err))
err = tk.ExecToErr("alter table tp partition p0 placement policy p1")
require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err))
tk.MustExec("alter table tp placement policy p1")
checkExistTableBundlesInPD(t, do, "test", "tp")
checkTiflashReplicaSet(t, do, "test", "tp", 1)
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" +
") 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("alter table tp partition p0 placement policy p2")
checkExistTableBundlesInPD(t, do, "test", "tp")
checkTiflashReplicaSet(t, do, "test", "tp", 1)
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) /*T![placement] PLACEMENT POLICY=`p2` */,\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")
require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err))
checkTiflashReplicaSet(t, do, "test", "tp", 0)

tk.MustExec("alter table tp set tiflash replica 1")
checkTiflashReplicaSet(t, do, "test", "tp", 1)
checkExistTableBundlesInPD(t, do, "test", "tp")
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
Expand All @@ -633,8 +674,11 @@ func TestPlacementTiflashCheck(t *testing.T) {
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")
require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err))
checkTiflashReplicaSet(t, do, "test", "tp", 0)

tk.MustExec("alter table tp set tiflash replica 1")
checkExistTableBundlesInPD(t, do, "test", "tp")
checkTiflashReplicaSet(t, do, "test", "tp", 1)
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
Expand All @@ -648,8 +692,11 @@ func TestPlacementTiflashCheck(t *testing.T) {
PARTITION p0 VALUES LESS THAN (100),
PARTITION p1 VALUES LESS THAN (1000)
)`)
err = tk.ExecToErr("alter table tp set tiflash replica 1")
require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err))
checkTiflashReplicaSet(t, do, "test", "tp", 0)

tk.MustExec("alter table tp set tiflash replica 1")
checkExistTableBundlesInPD(t, do, "test", "tp")
checkTiflashReplicaSet(t, do, "test", "tp", 1)
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
Expand All @@ -663,8 +710,11 @@ func TestPlacementTiflashCheck(t *testing.T) {
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")
require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err))
checkTiflashReplicaSet(t, do, "test", "tp", 0)

tk.MustExec("alter table tp set tiflash replica 1")
checkExistTableBundlesInPD(t, do, "test", "tp")
checkTiflashReplicaSet(t, do, "test", "tp", 1)
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
Expand Down
15 changes: 0 additions & 15 deletions ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -1108,11 +1108,6 @@ func (w *worker) onSetTableFlashReplica(d *ddlCtx, t *meta.Meta, job *model.Job)
return ver, errors.Trace(err)
}

if replicaInfo.Count > 0 && tableHasPlacementSettings(tblInfo) {
job.State = model.JobStateCancelled
return ver, errors.Trace(dbterror.ErrIncompatibleTiFlashAndPlacement)
}

// Ban setting replica count for tables in system database.
if tidb_util.IsMemOrSysDB(job.SchemaName) {
return ver, errors.Trace(dbterror.ErrUnsupportedTiFlashOperationForSysOrMemTable)
Expand Down Expand Up @@ -1473,11 +1468,6 @@ func onAlterTablePartitionPlacement(d *ddlCtx, t *meta.Meta, job *model.Job) (ve
return 0, err
}

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

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

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

if _, err = checkPlacementPolicyRefValidAndCanNonValidJob(t, job, policyRefInfo); err != nil {
return 0, errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -845,7 +845,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrAggregateInOrderNotSelect: mysql.Message("Expression #%d of ORDER BY clause is not in SELECT list, contains aggregate function; this is incompatible with %s", nil),
ErrInvalidJSONData: mysql.Message("Invalid JSON data provided to function %s: %s", nil),
ErrInvalidJSONText: mysql.Message("Invalid JSON text: %-.192s", []int{0}),
ErrInvalidJSONPath: mysql.Message("Invalid JSON path expression %s.", nil),
ErrInvalidJSONPath: mysql.Message("Invalid JSON path expression. The error is around character position %d.", []int{0}),
ErrInvalidJSONCharset: mysql.Message("Cannot create a JSON value from a string with CHARACTER SET '%s'.", nil),
ErrInvalidTypeForJSON: mysql.Message("Invalid data type for JSON data in argument %d to function %s; a JSON string or JSON type is required.", nil),
ErrInvalidJSONPathWildcard: mysql.Message("In this situation, path expressions may not contain the * and ** tokens.", nil),
Expand Down
2 changes: 1 addition & 1 deletion errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1588,7 +1588,7 @@ Invalid JSON text: %-.192s

["json:3143"]
error = '''
Invalid JSON path expression %s.
Invalid JSON path expression. The error is around character position %d.
'''

["json:3144"]
Expand Down
18 changes: 17 additions & 1 deletion expression/builtin_cast.go
Original file line number Diff line number Diff line change
Expand Up @@ -705,7 +705,23 @@ func (b *builtinCastStringAsJSONSig) evalJSON(row chunk.Row) (res json.BinaryJSO
if isNull || err != nil {
return res, isNull, err
}
if mysql.HasParseToJSONFlag(b.tp.GetFlag()) {

typ := b.args[0].GetType()
if types.IsBinaryStr(typ) {
buf := []byte(val)
if typ.GetType() == mysql.TypeString {
// the tailing zero should also be in the opaque json
buf = make([]byte, typ.GetFlen())
copy(buf, val)
}

res := json.CreateBinary(json.Opaque{
TypeCode: b.args[0].GetType().GetType(),
Buf: buf,
})

return res, false, err
} else if mysql.HasParseToJSONFlag(b.tp.GetFlag()) {
res, err = json.ParseBinaryFromString(val)
} else {
res = json.CreateBinary(val)
Expand Down
61 changes: 61 additions & 0 deletions expression/builtin_cast_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1550,3 +1550,64 @@ func TestCastConstAsDecimalFieldType(t *testing.T) {
require.Equal(t, tc.resultDecimal, expr.GetType().GetDecimal())
}
}

func TestCastBinaryStringAsJSONSig(t *testing.T) {
ctx := createContext(t)
sc := ctx.GetSessionVars().StmtCtx
originIgnoreTruncate := sc.IgnoreTruncate
sc.IgnoreTruncate = true
defer func() {
sc.IgnoreTruncate = originIgnoreTruncate
}()

// BINARY STRING will be converted to a JSON opaque
// and yield "base64:typeXX:<base64 encoded value>" finally
var tests = []struct {
str string
tp *types.FieldType
result json.BinaryJSON
resultStr string
}{
{
"a",
types.NewFieldTypeWithCollation(mysql.TypeVarString, charset.CollationBin, 4),
json.BinaryJSON{TypeCode: json.TypeCodeOpaque, Value: []byte{0xfd, 1, 'a'}},
`"base64:type253:YQ=="`,
},
{
"test",
types.NewFieldTypeWithCollation(mysql.TypeVarString, charset.CollationBin, 4),
json.BinaryJSON{TypeCode: json.TypeCodeOpaque, Value: []byte{0xfd, 4, 't', 'e', 's', 't'}},
`"base64:type253:dGVzdA=="`,
},
{
"a",
types.NewFieldTypeWithCollation(mysql.TypeString, charset.CollationBin, 4),
json.BinaryJSON{TypeCode: json.TypeCodeOpaque, Value: []byte{0xfe, 4, 'a', 0, 0, 0}},
`"base64:type254:YQAAAA=="`,
},
{
"a",
types.NewFieldTypeWithCollation(mysql.TypeBlob, charset.CollationBin, 4),
json.BinaryJSON{TypeCode: json.TypeCodeOpaque, Value: []byte{0xfc, 1, 'a'}},
`"base64:type252:YQ=="`,
},
}
for _, tt := range tests {
args := []Expression{&Column{RetType: tt.tp, Index: 0}}
tp := types.NewFieldType(mysql.TypeJSON)
tp.SetDecimal(types.DefaultFsp)
jsonFunc, err := newBaseBuiltinFunc(ctx, "", args, tp)
require.NoError(t, err)
sig := &builtinCastStringAsJSONSig{jsonFunc}

row := chunk.MutRowFromDatums(
[]types.Datum{types.NewCollationStringDatum(tt.str, charset.CollationBin)},
)
res, isNull, err := sig.evalJSON(row.ToRow())
require.NoError(t, err)
require.False(t, isNull)
require.Equal(t, tt.result, res)
require.Equal(t, tt.resultStr, res.String())
}
}
Loading

0 comments on commit bb5f1c0

Please sign in to comment.