Skip to content

Commit

Permalink
Merge branch 'master' of github.com:pingcap/tidb into addWarning4bind…
Browse files Browse the repository at this point in the history
…Cache
  • Loading branch information
Reminiscent committed Mar 15, 2022
2 parents 3691299 + b83af27 commit 5525271
Show file tree
Hide file tree
Showing 61 changed files with 3,142 additions and 915 deletions.
15 changes: 8 additions & 7 deletions br/pkg/lightning/mydump/loader.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,8 @@ import (
"strings"

"github.com/pingcap/errors"
regexprrouter "github.com/pingcap/tidb-tools/pkg/regexpr-router"
filter "github.com/pingcap/tidb-tools/pkg/table-filter"
router "github.com/pingcap/tidb-tools/pkg/table-router"
"github.com/pingcap/tidb/br/pkg/lightning/common"
"github.com/pingcap/tidb/br/pkg/lightning/config"
"github.com/pingcap/tidb/br/pkg/lightning/log"
Expand Down Expand Up @@ -91,10 +91,11 @@ func (m *MDTableMeta) GetSchema(ctx context.Context, store storage.ExternalStora
Mydumper File Loader
*/
type MDLoader struct {
store storage.ExternalStorage
dbs []*MDDatabaseMeta
filter filter.Filter
router *router.Table
store storage.ExternalStorage
dbs []*MDDatabaseMeta
filter filter.Filter
// router *router.Table
router *regexprrouter.RouteTable
fileRouter FileRouter
charSet string
}
Expand Down Expand Up @@ -123,15 +124,15 @@ func NewMyDumpLoader(ctx context.Context, cfg *config.Config) (*MDLoader, error)
}

func NewMyDumpLoaderWithStore(ctx context.Context, cfg *config.Config, store storage.ExternalStorage) (*MDLoader, error) {
var r *router.Table
var r *regexprrouter.RouteTable
var err error

if len(cfg.Routes) > 0 && len(cfg.Mydumper.FileRouters) > 0 {
return nil, common.ErrInvalidConfig.GenWithStack("table route is deprecated, can't config both [routes] and [mydumper.files]")
}

if len(cfg.Routes) > 0 {
r, err = router.NewTableRouter(cfg.Mydumper.CaseSensitive, cfg.Routes)
r, err = regexprrouter.NewRegExprRouter(cfg.Mydumper.CaseSensitive, cfg.Routes)
if err != nil {
return nil, common.ErrInvalidConfig.Wrap(err).GenWithStack("invalid table route rule")
}
Expand Down
53 changes: 53 additions & 0 deletions br/pkg/lightning/mydump/loader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -363,6 +363,16 @@ func TestRouter(t *testing.T) {
TargetSchema: "v",
TargetTable: "vv",
},
{
SchemaPattern: "~.*regexpr[1-9]+",
TablePattern: "~.*regexprtable",
TargetSchema: "downstream_db",
TargetTable: "downstream_table",
},
{
SchemaPattern: "~.bdb.*",
TargetSchema: "db",
},
}

/*
Expand All @@ -386,6 +396,12 @@ func TestRouter(t *testing.T) {
e0-schema-create.sql
e0.f0-schema.sql
e0.f0-schema-view.sql
test_regexpr1-schema-create.sql
test_regexpr1.test_regexprtable-schema.sql
test_regexpr1.test_regexprtable.1.sql
zbdb-schema-create.sql
zbdb.table-schema.sql
zbdb.table.1.sql
*/

s.touch(t, "a0-schema-create.sql")
Expand All @@ -412,6 +428,14 @@ func TestRouter(t *testing.T) {
s.touch(t, "e0.f0-schema.sql")
s.touch(t, "e0.f0-schema-view.sql")

s.touch(t, "test_regexpr1-schema-create.sql")
s.touch(t, "test_regexpr1.test_regexprtable-schema.sql")
s.touch(t, "test_regexpr1.test_regexprtable.1.sql")

s.touch(t, "zbdb-schema-create.sql")
s.touch(t, "zbdb.table-schema.sql")
s.touch(t, "zbdb.table.1.sql")

mdl, err := md.NewMyDumpLoader(context.Background(), s.cfg)
require.NoError(t, err)
require.Equal(t, []*md.MDDatabaseMeta{
Expand Down Expand Up @@ -505,6 +529,35 @@ func TestRouter(t *testing.T) {
},
},
},

{
Name: "downstream_db",
SchemaFile: md.FileInfo{TableName: filter.Table{Schema: "downstream_db", Name: ""}, FileMeta: md.SourceFileMeta{Path: "test_regexpr1-schema-create.sql", Type: md.SourceTypeSchemaSchema}},
Tables: []*md.MDTableMeta{
{
DB: "downstream_db",
Name: "downstream_table",
SchemaFile: md.FileInfo{TableName: filter.Table{Schema: "downstream_db", Name: "downstream_table"}, FileMeta: md.SourceFileMeta{Path: "test_regexpr1.test_regexprtable-schema.sql", Type: md.SourceTypeTableSchema}},
DataFiles: []md.FileInfo{{TableName: filter.Table{Schema: "downstream_db", Name: "downstream_table"}, FileMeta: md.SourceFileMeta{Path: "test_regexpr1.test_regexprtable.1.sql", Type: md.SourceTypeSQL, SortKey: "1"}}},
IndexRatio: 0.0,
IsRowOrdered: true,
},
},
},
{
Name: "db",
SchemaFile: md.FileInfo{TableName: filter.Table{Schema: "db", Name: ""}, FileMeta: md.SourceFileMeta{Path: "zbdb-schema-create.sql", Type: md.SourceTypeSchemaSchema}},
Tables: []*md.MDTableMeta{
{
DB: "db",
Name: "table",
SchemaFile: md.FileInfo{TableName: filter.Table{Schema: "db", Name: "table"}, FileMeta: md.SourceFileMeta{Path: "zbdb.table-schema.sql", Type: md.SourceTypeTableSchema}},
DataFiles: []md.FileInfo{{TableName: filter.Table{Schema: "db", Name: "table"}, FileMeta: md.SourceFileMeta{Path: "zbdb.table.1.sql", Type: md.SourceTypeSQL, SortKey: "1"}}},
IndexRatio: 0.0,
IsRowOrdered: true,
},
},
},
}, mdl.GetDatabases())
}

Expand Down
20 changes: 10 additions & 10 deletions cmd/explaintest/r/imdbload.result
Original file line number Diff line number Diff line change
Expand Up @@ -350,21 +350,21 @@ CE_trace

explain select * from keyword where ((phonetic_code = 'R1652') and (keyword > 'ecg-monitor' and keyword < 'killers'));
id estRows task access object operator info
IndexLookUp_11 10129.18 root
├─IndexRangeScan_8(Build) 10129.18 cop[tikv] table:keyword, index:itest(phonetic_code, keyword) range:("R1652" "ecg-monitor","R1652" "killers"), keep order:false
└─Selection_10(Probe) 10129.18 cop[tikv] gt(imdbload.keyword.keyword, "ecg-monitor"), lt(imdbload.keyword.keyword, "killers")
└─TableRowIDScan_9 10129.18 cop[tikv] table:keyword keep order:false
IndexLookUp_11 901.00 root
├─IndexRangeScan_8(Build) 901.00 cop[tikv] table:keyword, index:itest(phonetic_code, keyword) range:("R1652" "ecg-monitor","R1652" "killers"), keep order:false
└─Selection_10(Probe) 901.00 cop[tikv] gt(imdbload.keyword.keyword, "ecg-monitor"), lt(imdbload.keyword.keyword, "killers")
└─TableRowIDScan_9 901.00 cop[tikv] table:keyword keep order:false
trace plan target = 'estimation' select * from keyword where ((phonetic_code = 'R1652') and (keyword > 'ecg-monitor' and keyword < 'killers'));
CE_trace
[{"table_name":"keyword","type":"Column Stats-Point","expr":"((phonetic_code = 'R1652'))","row_count":23480},{"table_name":"keyword","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":236627},{"table_name":"keyword","type":"Column Stats-Range","expr":"((keyword > 'ecg-monitor' and keyword < 'killers'))","row_count":44075},{"table_name":"keyword","type":"Index Stats-Point","expr":"((phonetic_code = 'R1652'))","row_count":23480},{"table_name":"keyword","type":"Index Stats-Range","expr":"((keyword > 'ecg-monitor' and keyword < 'killers'))","row_count":44036},{"table_name":"keyword","type":"Index Stats-Range","expr":"((keyword >= 'ecg-m' and keyword <= 'kille'))","row_count":44036},{"table_name":"keyword","type":"Index Stats-Range","expr":"((phonetic_code = 'R1652') and (keyword > 'ecg-monitor' and keyword < 'killers'))","row_count":10129},{"table_name":"keyword","type":"Table Stats-Expression-CNF","expr":"`and`(`eq`(imdbload.keyword.phonetic_code, 'R1652'), `and`(`gt`(imdbload.keyword.keyword, 'ecg-monitor'), `lt`(imdbload.keyword.keyword, 'killers')))","row_count":10129}]
[{"table_name":"keyword","type":"Column Stats-Point","expr":"((phonetic_code = 'R1652'))","row_count":23480},{"table_name":"keyword","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":236627},{"table_name":"keyword","type":"Column Stats-Range","expr":"((keyword > 'ecg-monitor' and keyword < 'killers'))","row_count":44075},{"table_name":"keyword","type":"Index Stats-Point","expr":"((phonetic_code = 'R1652'))","row_count":23480},{"table_name":"keyword","type":"Index Stats-Range","expr":"((keyword > 'ecg-monitor' and keyword < 'killers'))","row_count":44036},{"table_name":"keyword","type":"Index Stats-Range","expr":"((keyword >= 'ecg-m' and keyword <= 'kille'))","row_count":44036},{"table_name":"keyword","type":"Index Stats-Range","expr":"((phonetic_code = 'R1652') and (keyword > 'ecg-monitor' and keyword < 'killers'))","row_count":901},{"table_name":"keyword","type":"Table Stats-Expression-CNF","expr":"`and`(`eq`(imdbload.keyword.phonetic_code, 'R1652'), `and`(`gt`(imdbload.keyword.keyword, 'ecg-monitor'), `lt`(imdbload.keyword.keyword, 'killers')))","row_count":901}]

explain select * from cast_info where (nr_order is null) and (person_role_id = 2) and (note >= '(key set pa: Florida');
id estRows task access object operator info
IndexLookUp_11 935147.84 root
├─IndexRangeScan_8(Build) 935147.84 cop[tikv] table:cast_info, index:itest2(nr_order, person_role_id, note) range:[NULL 2 "(key set pa: Florida",NULL 2 +inf], keep order:false
└─Selection_10(Probe) 935147.84 cop[tikv] ge(imdbload.cast_info.note, "(key set pa: Florida")
└─TableRowIDScan_9 935147.84 cop[tikv] table:cast_info keep order:false
IndexLookUp_11 144633.00 root
├─IndexRangeScan_8(Build) 144633.00 cop[tikv] table:cast_info, index:itest2(nr_order, person_role_id, note) range:[NULL 2 "(key set pa: Florida",NULL 2 +inf], keep order:false
└─Selection_10(Probe) 144633.00 cop[tikv] ge(imdbload.cast_info.note, "(key set pa: Florida")
└─TableRowIDScan_9 144633.00 cop[tikv] table:cast_info keep order:false
trace plan target = 'estimation' select * from cast_info where (nr_order is null) and (person_role_id = 2) and (note >= '(key set pa: Florida');
CE_trace
[{"table_name":"cast_info","type":"Column Stats-Point","expr":"((nr_order is null))","row_count":45995275},{"table_name":"cast_info","type":"Column Stats-Point","expr":"((person_role_id = 2))","row_count":2089611},{"table_name":"cast_info","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":63475835},{"table_name":"cast_info","type":"Column Stats-Range","expr":"((note >= '(key set pa: Florida' and true))","row_count":14934328},{"table_name":"cast_info","type":"Index Stats-Point","expr":"((person_role_id = 2))","row_count":2089611},{"table_name":"cast_info","type":"Index Stats-Range","expr":"((nr_order is null) and (person_role_id = 2) and (note >= '(key set pa: Florida' and true))","row_count":935147},{"table_name":"cast_info","type":"Table Stats-Expression-CNF","expr":"`and`(`isnull`(imdbload.cast_info.nr_order), `and`(`eq`(imdbload.cast_info.person_role_id, 2), `ge`(imdbload.cast_info.note, '(key set pa: Florida')))","row_count":935147},{"table_name":"cast_info","type":"Table Stats-Expression-CNF","expr":"`eq`(imdbload.cast_info.person_role_id, 2)","row_count":2089611}]
[{"table_name":"cast_info","type":"Column Stats-Point","expr":"((nr_order is null))","row_count":45995275},{"table_name":"cast_info","type":"Column Stats-Point","expr":"((person_role_id = 2))","row_count":2089611},{"table_name":"cast_info","type":"Column Stats-Range","expr":"((id >= -9223372036854775808 and id <= 9223372036854775807))","row_count":63475835},{"table_name":"cast_info","type":"Column Stats-Range","expr":"((note >= '(key set pa: Florida' and true))","row_count":14934328},{"table_name":"cast_info","type":"Index Stats-Point","expr":"((person_role_id = 2))","row_count":2089611},{"table_name":"cast_info","type":"Index Stats-Range","expr":"((nr_order is null) and (person_role_id = 2) and (note >= '(key set pa: Florida' and true))","row_count":144633},{"table_name":"cast_info","type":"Table Stats-Expression-CNF","expr":"`and`(`isnull`(imdbload.cast_info.nr_order), `and`(`eq`(imdbload.cast_info.person_role_id, 2), `ge`(imdbload.cast_info.note, '(key set pa: Florida')))","row_count":144633},{"table_name":"cast_info","type":"Table Stats-Expression-CNF","expr":"`eq`(imdbload.cast_info.person_role_id, 2)","row_count":2089611}]

36 changes: 32 additions & 4 deletions ddl/attributes_sql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,18 +19,46 @@ import (
"fmt"
"math"
"testing"
"time"

"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/ddl/util"
"github.com/pingcap/tidb/domain/infosync"
"github.com/pingcap/tidb/store/gcworker"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/util/gcutil"
"github.com/stretchr/testify/require"
)

// MockGC is used to make GC work in the test environment.
func MockGC(tk *testkit.TestKit) (string, string, string, func()) {
originGC := util.IsEmulatorGCEnable()
resetGC := func() {
if originGC {
util.EmulatorGCEnable()
} else {
util.EmulatorGCDisable()
}
}

// disable emulator GC.
// Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl.
util.EmulatorGCDisable()
gcTimeFormat := "20060102-15:04:05 -0700 MST"
timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat)
timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(gcTimeFormat)
safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '')
ON DUPLICATE KEY
UPDATE variable_value = '%[1]s'`
// clear GC variables first.
tk.MustExec("delete from mysql.tidb where variable_name in ( 'tikv_gc_safe_point','tikv_gc_enable' )")
return timeBeforeDrop, timeAfterDrop, safePointSQL, resetGC
}

func TestAlterTableAttributes(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()

tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec(`create table alter_t (c int);`)
Expand Down Expand Up @@ -179,7 +207,7 @@ PARTITION BY RANGE (c) (
PARTITION p1 VALUES LESS THAN (11)
);`)

timeBeforeDrop, _, safePointSQL, resetGC := testkit.MockGC(tk)
timeBeforeDrop, _, safePointSQL, resetGC := MockGC(tk)
defer resetGC()

// Set GC safe point
Expand Down Expand Up @@ -222,7 +250,7 @@ PARTITION BY RANGE (c) (
PARTITION p1 VALUES LESS THAN (11)
);`)

timeBeforeDrop, _, safePointSQL, resetGC := testkit.MockGC(tk)
timeBeforeDrop, _, safePointSQL, resetGC := MockGC(tk)
defer resetGC()

// Set GC safe point
Expand Down Expand Up @@ -285,7 +313,7 @@ PARTITION BY RANGE (c) (
failpoint.Disable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed")
}()

timeBeforeDrop, _, safePointSQL, resetGC := testkit.MockGC(tk)
timeBeforeDrop, _, safePointSQL, resetGC := MockGC(tk)
defer resetGC()

// Set GC safe point
Expand Down Expand Up @@ -339,7 +367,7 @@ PARTITION BY RANGE (c) (
failpoint.Disable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed")
}()

timeBeforeDrop, _, safePointSQL, resetGC := testkit.MockGC(tk)
timeBeforeDrop, _, safePointSQL, resetGC := MockGC(tk)
defer resetGC()

// Set GC safe point
Expand Down
Loading

0 comments on commit 5525271

Please sign in to comment.