Skip to content

Commit

Permalink
Merge branch 'master' into pre-check
Browse files Browse the repository at this point in the history
  • Loading branch information
Defined2014 committed Aug 23, 2022
2 parents b01e4c4 + cfd4ddd commit a65ad21
Show file tree
Hide file tree
Showing 112 changed files with 2,269 additions and 642 deletions.
21 changes: 12 additions & 9 deletions .bazelrc
Original file line number Diff line number Diff line change
@@ -1,24 +1,27 @@
startup --host_jvm_args=-Xmx8g
startup --unlimit_coredumps

run:ci --color=yes

build --announce_rc
build --java_language_version=17
build --java_runtime_version=17
build --tool_java_language_version=17
build --tool_java_runtime_version=17
build --incompatible_strict_action_env --incompatible_enable_cc_toolchain_resolution
build:ci --color=yes
run:ci --color=yes

build:ci --experimental_remote_cache_compression
build:release --workspace_status_command=./build/print-workspace-status.sh --stamp
build:release --config=ci
build --incompatible_strict_action_env --incompatible_enable_cc_toolchain_resolution
build:ci --experimental_remote_cache_compression
test:ci --color=yes
test:ci --verbose_failures
test:ci --test_env=GO_TEST_WRAP_TESTV=1 --test_verbose_timeout_warnings
test:ci --test_env=TZ=Asia/Shanghai --test_output=errors --experimental_ui_max_stdouterr_bytes=104857600

build:race --config=ci
build:race --@io_bazel_rules_go//go/config:race --test_env=GORACE=halt_on_error=1 --test_sharding_strategy=disabled

test --test_env=TZ=Asia/Shanghai
test --test_output=errors --test_summary=testcase
test:ci --color=yes
test:ci --verbose_failures --test_verbose_timeout_warnings
test:ci --test_env=GO_TEST_WRAP_TESTV=1
test:ci --experimental_ui_max_stdouterr_bytes=104857600
test:race --test_timeout=1200,6000,18000,72000

try-import /data/bazel
4 changes: 2 additions & 2 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -3366,8 +3366,8 @@ def go_deps():
name = "com_github_tikv_client_go_v2",
build_file_proto_mode = "disable_global",
importpath = "github.com/tikv/client-go/v2",
sum = "h1:WFR3seA8YtBhDn47YJSW1P1/lwBIXsk0vALnRVuaL/M=",
version = "v2.0.1-0.20220815094724-025596b7a20a",
sum = "h1:/nr7P8uzJQ7u3wPEBHCokrsVmuDvi/1x/zI/ydk5n8U=",
version = "v2.0.1-0.20220818084834-0d0ae0dcfb1f",
)
go_repository(
name = "com_github_tikv_pd_client",
Expand Down
1 change: 1 addition & 0 deletions br/pkg/lightning/backend/local/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ go_library(
"//parser/mysql",
"//table",
"//tablecodec",
"//types",
"//util/codec",
"//util/engine",
"//util/hack",
Expand Down
5 changes: 5 additions & 0 deletions br/pkg/lightning/backend/local/local.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ import (
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/engine"
"github.com/pingcap/tidb/util/mathutil"
Expand Down Expand Up @@ -1693,6 +1694,10 @@ func (local *local) ResolveDuplicateRows(ctx context.Context, tbl table.Table, t
if err == nil {
return nil
}
if types.ErrBadNumber.Equal(err) {
logger.Warn("delete duplicate rows encounter error", log.ShortError(err))
return common.ErrResolveDuplicateRows.Wrap(err).GenWithStackByArgs(tableName)
}
if log.IsContextCanceledError(err) {
return err
}
Expand Down
27 changes: 14 additions & 13 deletions br/pkg/lightning/common/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,19 +81,20 @@ var (
ErrKVIngestFailed = errors.Normalize("ingest tikv failed", errors.RFCCodeText("Lightning:KV:ErrKVIngestFailed"))
ErrKVRaftProposalDropped = errors.Normalize("raft proposal dropped", errors.RFCCodeText("Lightning:KV:ErrKVRaftProposalDropped"))

ErrUnknownBackend = errors.Normalize("unknown backend %s", errors.RFCCodeText("Lightning:Restore:ErrUnknownBackend"))
ErrCheckLocalFile = errors.Normalize("cannot find local file for table: %s engineDir: %s", errors.RFCCodeText("Lightning:Restore:ErrCheckLocalFile"))
ErrOpenDuplicateDB = errors.Normalize("open duplicate db error", errors.RFCCodeText("Lightning:Restore:ErrOpenDuplicateDB"))
ErrSchemaNotExists = errors.Normalize("table `%s`.`%s` schema not found", errors.RFCCodeText("Lightning:Restore:ErrSchemaNotExists"))
ErrInvalidSchemaStmt = errors.Normalize("invalid schema statement: '%s'", errors.RFCCodeText("Lightning:Restore:ErrInvalidSchemaStmt"))
ErrCreateSchema = errors.Normalize("create schema failed, table: %s, stmt: %s", errors.RFCCodeText("Lightning:Restore:ErrCreateSchema"))
ErrUnknownColumns = errors.Normalize("unknown columns in header (%s) for table %s", errors.RFCCodeText("Lightning:Restore:ErrUnknownColumns"))
ErrChecksumMismatch = errors.Normalize("checksum mismatched remote vs local => (checksum: %d vs %d) (total_kvs: %d vs %d) (total_bytes:%d vs %d)", errors.RFCCodeText("Lighting:Restore:ErrChecksumMismatch"))
ErrRestoreTable = errors.Normalize("restore table %s failed", errors.RFCCodeText("Lightning:Restore:ErrRestoreTable"))
ErrEncodeKV = errors.Normalize("encode kv error in file %s at offset %d", errors.RFCCodeText("Lightning:Restore:ErrEncodeKV"))
ErrAllocTableRowIDs = errors.Normalize("allocate table row id error", errors.RFCCodeText("Lightning:Restore:ErrAllocTableRowIDs"))
ErrInvalidMetaStatus = errors.Normalize("invalid meta status: '%s'", errors.RFCCodeText("Lightning:Restore:ErrInvalidMetaStatus"))
ErrTableIsChecksuming = errors.Normalize("table '%s' is checksuming", errors.RFCCodeText("Lightning:Restore:ErrTableIsChecksuming"))
ErrUnknownBackend = errors.Normalize("unknown backend %s", errors.RFCCodeText("Lightning:Restore:ErrUnknownBackend"))
ErrCheckLocalFile = errors.Normalize("cannot find local file for table: %s engineDir: %s", errors.RFCCodeText("Lightning:Restore:ErrCheckLocalFile"))
ErrOpenDuplicateDB = errors.Normalize("open duplicate db error", errors.RFCCodeText("Lightning:Restore:ErrOpenDuplicateDB"))
ErrSchemaNotExists = errors.Normalize("table `%s`.`%s` schema not found", errors.RFCCodeText("Lightning:Restore:ErrSchemaNotExists"))
ErrInvalidSchemaStmt = errors.Normalize("invalid schema statement: '%s'", errors.RFCCodeText("Lightning:Restore:ErrInvalidSchemaStmt"))
ErrCreateSchema = errors.Normalize("create schema failed, table: %s, stmt: %s", errors.RFCCodeText("Lightning:Restore:ErrCreateSchema"))
ErrUnknownColumns = errors.Normalize("unknown columns in header (%s) for table %s", errors.RFCCodeText("Lightning:Restore:ErrUnknownColumns"))
ErrChecksumMismatch = errors.Normalize("checksum mismatched remote vs local => (checksum: %d vs %d) (total_kvs: %d vs %d) (total_bytes:%d vs %d)", errors.RFCCodeText("Lighting:Restore:ErrChecksumMismatch"))
ErrRestoreTable = errors.Normalize("restore table %s failed", errors.RFCCodeText("Lightning:Restore:ErrRestoreTable"))
ErrEncodeKV = errors.Normalize("encode kv error in file %s at offset %d", errors.RFCCodeText("Lightning:Restore:ErrEncodeKV"))
ErrAllocTableRowIDs = errors.Normalize("allocate table row id error", errors.RFCCodeText("Lightning:Restore:ErrAllocTableRowIDs"))
ErrInvalidMetaStatus = errors.Normalize("invalid meta status: '%s'", errors.RFCCodeText("Lightning:Restore:ErrInvalidMetaStatus"))
ErrTableIsChecksuming = errors.Normalize("table '%s' is checksuming", errors.RFCCodeText("Lightning:Restore:ErrTableIsChecksuming"))
ErrResolveDuplicateRows = errors.Normalize("resolve duplicate rows error on table '%s'", errors.RFCCodeText("Lightning:Restore:ErrResolveDuplicateRows"))
)

type withStack struct {
Expand Down
10 changes: 4 additions & 6 deletions build/nogo_config.json
Original file line number Diff line number Diff line change
Expand Up @@ -159,6 +159,7 @@
"ddl/backfilling.go": "ddl/backfilling.go",
"ddl/column.go": "ddl/column.go",
"ddl/index.go": "ddl/index.go",
"ddl/lightning/": "ddl/lightning/",
"server/conn.go": "server/conn.go",
"server/conn_stmt.go": "server/conn_stmt.go",
"server/conn_test.go": "server/conn_test.go",
Expand Down Expand Up @@ -293,11 +294,11 @@
"ddl/backfilling.go": "ddl/backfilling.go",
"ddl/column.go": "ddl/column.go",
"ddl/index.go": "ddl/index.go",
"ddl/lightning/": "ddl/lightning/",
"expression/builtin_cast.go": "expression/builtin_cast code",
"server/conn.go": "server/conn.go",
"server/conn_stmt.go": "server/conn_stmt.go",
"server/conn_test.go": "server/conn_test.go",
"ddl/index.go": "ddl/index code",
"planner/core/rule_partition_eliminate.go": "planner/core/rule_partition_eliminate code",
"distsql/": "ignore distsql code",
"dumpling/export": "dumpling/export code",
Expand Down Expand Up @@ -650,15 +651,12 @@
"only_files": {
"types/json/binary_functions.go": "types/json/binary_functions.go",
"types/json/binary_test.go": "types/json/binary_test.go",
"ddl/backfilling.go": "ddl/backfilling.go",
"ddl/column.go": "ddl/column.go",
"ddl/index.go": "ddl/index.go",
"ddl/": "enable to ddl",
"expression/builtin_cast.go": "enable expression/builtin_cast.go",
"planner/core/plan.go": "planner/core/plan.go",
"server/conn.go": "server/conn.go",
"server/conn_stmt.go": "server/conn_stmt.go",
"server/conn_test.go": "server/conn_test.go",
"ddl/": "enable to ddl"
"server/conn_test.go": "server/conn_test.go"
}
},
"SA2000": {
Expand Down
2 changes: 2 additions & 0 deletions cmd/explaintest/r/clustered_index.result
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
set @@tidb_enable_outer_join_reorder=true;
drop database if exists with_cluster_index;
create database with_cluster_index;
drop database if exists wout_cluster_index;
Expand Down Expand Up @@ -127,3 +128,4 @@ StreamAgg_17 1.00 root funcs:count(Column#9)->Column#7
└─IndexReader_18 1.00 root index:StreamAgg_9
└─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#9
└─IndexRangeScan_16 109.70 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[803163,+inf], keep order:false
set @@tidb_enable_outer_join_reorder=false;
16 changes: 8 additions & 8 deletions cmd/explaintest/r/collation_agg_func_disabled.result
Original file line number Diff line number Diff line change
Expand Up @@ -263,13 +263,13 @@ min(d)
desc format='brief' select min(d collate utf8mb4_bin) from tt;
id estRows task access object operator info
StreamAgg 1.00 root funcs:min(Column#8)->Column#6
└─Projection 1.00 root cast(collation_agg_func.tt.d, json BINARY)->Column#8
└─Projection 1.00 root cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#8
└─Projection 1.00 root collation_agg_func.tt.d
└─TopN 1.00 root Column#7, offset:0, count:1
└─Projection 1.00 root collation_agg_func.tt.d, cast(collation_agg_func.tt.d, json BINARY)->Column#7
└─Projection 1.00 root collation_agg_func.tt.d, cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#7
└─TableReader 1.00 root data:TopN
└─TopN 1.00 cop[tikv] cast(collation_agg_func.tt.d, json BINARY), offset:0, count:1
└─Selection 8000.00 cop[tikv] not(isnull(cast(cast(collation_agg_func.tt.d, json BINARY), var_string(4294967295))))
└─TopN 1.00 cop[tikv] cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), offset:0, count:1
└─Selection 8000.00 cop[tikv] not(isnull(cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)))
└─TableFullScan 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo
select min(d collate utf8mb4_bin) from tt;
min(d collate utf8mb4_bin)
Expand All @@ -288,13 +288,13 @@ max(d)
desc format='brief' select max(d collate utf8mb4_bin) from tt;
id estRows task access object operator info
StreamAgg 1.00 root funcs:max(Column#8)->Column#6
└─Projection 1.00 root cast(collation_agg_func.tt.d, json BINARY)->Column#8
└─Projection 1.00 root cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#8
└─Projection 1.00 root collation_agg_func.tt.d
└─TopN 1.00 root Column#7:desc, offset:0, count:1
└─Projection 1.00 root collation_agg_func.tt.d, cast(collation_agg_func.tt.d, json BINARY)->Column#7
└─Projection 1.00 root collation_agg_func.tt.d, cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#7
└─TableReader 1.00 root data:TopN
└─TopN 1.00 cop[tikv] cast(collation_agg_func.tt.d, json BINARY):desc, offset:0, count:1
└─Selection 8000.00 cop[tikv] not(isnull(cast(cast(collation_agg_func.tt.d, json BINARY), var_string(4294967295))))
└─TopN 1.00 cop[tikv] cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin):desc, offset:0, count:1
└─Selection 8000.00 cop[tikv] not(isnull(cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)))
└─TableFullScan 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo
select max(d collate utf8mb4_bin) from tt;
max(d collate utf8mb4_bin)
Expand Down
28 changes: 24 additions & 4 deletions cmd/explaintest/r/collation_agg_func_enabled.result
Original file line number Diff line number Diff line change
Expand Up @@ -258,9 +258,19 @@ select min(d) from tt;
min(d)
{"A": "A"}
desc format='brief' select min(d collate utf8mb4_bin) from tt;
Error 1253: COLLATION 'utf8mb4_bin' is not valid for CHARACTER SET 'binary'
id estRows task access object operator info
StreamAgg 1.00 root funcs:min(Column#8)->Column#6
└─Projection 1.00 root cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#8
└─Projection 1.00 root collation_agg_func.tt.d
└─TopN 1.00 root Column#7, offset:0, count:1
└─Projection 1.00 root collation_agg_func.tt.d, cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#7
└─TableReader 1.00 root data:TopN
└─TopN 1.00 cop[tikv] cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin), offset:0, count:1
└─Selection 8000.00 cop[tikv] not(isnull(cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)))
└─TableFullScan 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo
select min(d collate utf8mb4_bin) from tt;
Error 1253: COLLATION 'utf8mb4_bin' is not valid for CHARACTER SET 'binary'
min(d collate utf8mb4_bin)
{"A": "A"}
desc format='brief' select max(d) from tt;
id estRows task access object operator info
StreamAgg 1.00 root funcs:max(collation_agg_func.tt.d)->Column#6
Expand All @@ -273,8 +283,18 @@ select max(d) from tt;
max(d)
{"c": "c"}
desc format='brief' select max(d collate utf8mb4_bin) from tt;
Error 1253: COLLATION 'utf8mb4_bin' is not valid for CHARACTER SET 'binary'
id estRows task access object operator info
StreamAgg 1.00 root funcs:max(Column#8)->Column#6
└─Projection 1.00 root cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#8
└─Projection 1.00 root collation_agg_func.tt.d
└─TopN 1.00 root Column#7:desc, offset:0, count:1
└─Projection 1.00 root collation_agg_func.tt.d, cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#7
└─TableReader 1.00 root data:TopN
└─TopN 1.00 cop[tikv] cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin):desc, offset:0, count:1
└─Selection 8000.00 cop[tikv] not(isnull(cast(collation_agg_func.tt.d, longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)))
└─TableFullScan 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo
select max(d collate utf8mb4_bin) from tt;
Error 1253: COLLATION 'utf8mb4_bin' is not valid for CHARACTER SET 'binary'
max(d collate utf8mb4_bin)
{"c": "c"}
drop database collation_agg_func;
use test
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
2 changes: 2 additions & 0 deletions cmd/explaintest/r/select.result
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
set @@tidb_enable_outer_join_reorder=true;
DROP TABLE IF EXISTS t;
CREATE TABLE t (
c1 int,
Expand Down Expand Up @@ -654,3 +655,4 @@ create table t3(a char(10), primary key (a));
insert into t3 values ('a');
select * from t3 where a > 0x80;
Error 1105: Cannot convert string '\x80' from binary to utf8mb4
set @@tidb_enable_outer_join_reorder=false;
2 changes: 2 additions & 0 deletions cmd/explaintest/t/clustered_index.test
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
set @@tidb_enable_outer_join_reorder=true;
drop database if exists with_cluster_index;
create database with_cluster_index;
drop database if exists wout_cluster_index;
Expand Down Expand Up @@ -53,3 +54,4 @@ explain select count(*) from wout_cluster_index.tbl_0 where col_0 <= 0 ;

explain select count(*) from with_cluster_index.tbl_0 where col_0 >= 803163 ;
explain select count(*) from wout_cluster_index.tbl_0 where col_0 >= 803163 ;
set @@tidb_enable_outer_join_reorder=false;
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;

2 changes: 2 additions & 0 deletions cmd/explaintest/t/select.test
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
set @@tidb_enable_outer_join_reorder=true;
DROP TABLE IF EXISTS t;

CREATE TABLE t (
Expand Down Expand Up @@ -279,3 +280,4 @@ create table t3(a char(10), primary key (a));
insert into t3 values ('a');
--error 1105
select * from t3 where a > 0x80;
set @@tidb_enable_outer_join_reorder=false;
Loading

0 comments on commit a65ad21

Please sign in to comment.