Skip to content

Commit

Permalink
Merge branch 'err-mgr' of ssh://github.com/glorv/tidb into err-mgr
Browse files Browse the repository at this point in the history
  • Loading branch information
glorv committed Nov 29, 2021
2 parents 3332896 + 45e92fc commit da21e07
Show file tree
Hide file tree
Showing 53 changed files with 1,094 additions and 241 deletions.
24 changes: 16 additions & 8 deletions br/pkg/lightning/restore/check_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,6 @@ import (
"github.com/pingcap/tidb/br/pkg/lightning/mydump"
"github.com/pingcap/tidb/br/pkg/lightning/verification"
"github.com/pingcap/tidb/br/pkg/storage"
"github.com/pingcap/tidb/br/pkg/version"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/table"
Expand Down Expand Up @@ -179,6 +178,15 @@ func (rc *Controller) ClusterIsAvailable(ctx context.Context) error {
return nil
}

func isTiFlash(store *api.MetaStore) bool {
for _, label := range store.Labels {
if label.Key == "engine" && label.Value == "tiflash" {
return true
}
}
return false
}

func (rc *Controller) checkEmptyRegion(ctx context.Context) error {
passed := true
message := "Cluster doesn't have too many empty regions"
Expand Down Expand Up @@ -206,7 +214,7 @@ func (rc *Controller) checkEmptyRegion(ctx context.Context) error {
}
}
for _, store := range storeInfo.Stores {
stores[store.Store.Id] = store
stores[store.Store.StoreID] = store
}
tableCount := 0
for _, db := range rc.dbMetas {
Expand All @@ -224,10 +232,10 @@ func (rc *Controller) checkEmptyRegion(ctx context.Context) error {
)
for storeID, regionCnt := range regions {
if store, ok := stores[storeID]; ok {
if store.Store.State != metapb.StoreState_Up {
if metapb.StoreState(metapb.StoreState_value[store.Store.StateName]) != metapb.StoreState_Up {
continue
}
if version.IsTiFlash(store.Store.Store) {
if isTiFlash(store.Store) {
continue
}
if regionCnt > errorThrehold {
Expand Down Expand Up @@ -269,10 +277,10 @@ func (rc *Controller) checkRegionDistribution(ctx context.Context) error {
}
stores := make([]*api.StoreInfo, 0, len(result.Stores))
for _, store := range result.Stores {
if store.Store.State != metapb.StoreState_Up {
if metapb.StoreState(metapb.StoreState_value[store.Store.StateName]) != metapb.StoreState_Up {
continue
}
if version.IsTiFlash(store.Store.Store) {
if isTiFlash(store.Store) {
continue
}
stores = append(stores, store)
Expand Down Expand Up @@ -302,11 +310,11 @@ func (rc *Controller) checkRegionDistribution(ctx context.Context) error {
passed = false
message = fmt.Sprintf("Region distribution is unbalanced, the ratio of the regions count of the store(%v) "+
"with least regions(%v) to the store(%v) with most regions(%v) is %v, but we expect it must not be less than %v",
minStore.Store.Id, minStore.Status.RegionCount, maxStore.Store.Id, maxStore.Status.RegionCount, ratio, errorRegionCntMinMaxRatio)
minStore.Store.StoreID, minStore.Status.RegionCount, maxStore.Store.StoreID, maxStore.Status.RegionCount, ratio, errorRegionCntMinMaxRatio)
} else if ratio < warnRegionCntMinMaxRatio {
message = fmt.Sprintf("Region distribution is unbalanced, the ratio of the regions count of the store(%v) "+
"with least regions(%v) to the store(%v) with most regions(%v) is %v, but we expect it should not be less than %v",
minStore.Store.Id, minStore.Status.RegionCount, maxStore.Store.Id, maxStore.Status.RegionCount, ratio, warnRegionCntMinMaxRatio)
minStore.Store.StoreID, minStore.Status.RegionCount, maxStore.Store.StoreID, maxStore.Status.RegionCount, ratio, warnRegionCntMinMaxRatio)
}
return nil
}
Expand Down
20 changes: 10 additions & 10 deletions br/pkg/lightning/restore/restore_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1978,7 +1978,7 @@ func (s *tableRestoreSuite) TestCheckClusterRegion(c *C) {
testCases := []testCase{
{
stores: api.StoresInfo{Stores: []*api.StoreInfo{
{Store: &api.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &api.StoreStatus{RegionCount: 200}},
{Store: &api.MetaStore{StoreID: 1}, Status: &api.StoreStatus{RegionCount: 200}},
}},
emptyRegions: api.RegionsInfo{
Regions: append([]api.RegionInfo(nil), makeRegions(100, 1)...),
Expand All @@ -1989,9 +1989,9 @@ func (s *tableRestoreSuite) TestCheckClusterRegion(c *C) {
},
{
stores: api.StoresInfo{Stores: []*api.StoreInfo{
{Store: &api.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &api.StoreStatus{RegionCount: 2000}},
{Store: &api.MetaStore{Store: &metapb.Store{Id: 2}}, Status: &api.StoreStatus{RegionCount: 3100}},
{Store: &api.MetaStore{Store: &metapb.Store{Id: 3}}, Status: &api.StoreStatus{RegionCount: 2500}},
{Store: &api.MetaStore{StoreID: 1}, Status: &api.StoreStatus{RegionCount: 2000}},
{Store: &api.MetaStore{StoreID: 2}, Status: &api.StoreStatus{RegionCount: 3100}},
{Store: &api.MetaStore{StoreID: 3}, Status: &api.StoreStatus{RegionCount: 2500}},
}},
emptyRegions: api.RegionsInfo{
Regions: append(append(append([]api.RegionInfo(nil),
Expand All @@ -2009,19 +2009,19 @@ func (s *tableRestoreSuite) TestCheckClusterRegion(c *C) {
},
{
stores: api.StoresInfo{Stores: []*api.StoreInfo{
{Store: &api.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &api.StoreStatus{RegionCount: 1200}},
{Store: &api.MetaStore{Store: &metapb.Store{Id: 2}}, Status: &api.StoreStatus{RegionCount: 3000}},
{Store: &api.MetaStore{Store: &metapb.Store{Id: 3}}, Status: &api.StoreStatus{RegionCount: 2500}},
{Store: &api.MetaStore{StoreID: 1}, Status: &api.StoreStatus{RegionCount: 1200}},
{Store: &api.MetaStore{StoreID: 2}, Status: &api.StoreStatus{RegionCount: 3000}},
{Store: &api.MetaStore{StoreID: 3}, Status: &api.StoreStatus{RegionCount: 2500}},
}},
expectMsgs: []string{".*Region distribution is unbalanced.*but we expect it must not be less than 0.5.*"},
expectResult: false,
expectErrorCnt: 1,
},
{
stores: api.StoresInfo{Stores: []*api.StoreInfo{
{Store: &api.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &api.StoreStatus{RegionCount: 0}},
{Store: &api.MetaStore{Store: &metapb.Store{Id: 2}}, Status: &api.StoreStatus{RegionCount: 2800}},
{Store: &api.MetaStore{Store: &metapb.Store{Id: 3}}, Status: &api.StoreStatus{RegionCount: 2500}},
{Store: &api.MetaStore{StoreID: 1}, Status: &api.StoreStatus{RegionCount: 0}},
{Store: &api.MetaStore{StoreID: 2}, Status: &api.StoreStatus{RegionCount: 2800}},
{Store: &api.MetaStore{StoreID: 3}, Status: &api.StoreStatus{RegionCount: 2500}},
}},
expectMsgs: []string{".*Region distribution is unbalanced.*but we expect it must not be less than 0.5.*"},
expectResult: false,
Expand Down
3 changes: 2 additions & 1 deletion ddl/column_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,8 @@ func (s *testColumnChangeSuite) TestColumnChange(c *C) {
c.Assert(err, IsNil)
}()
// create table t (c1 int, c2 int);
tblInfo := testTableInfo(c, d, "t", 2)
tblInfo, err := testTableInfo(d, "t", 2)
c.Assert(err, IsNil)
ctx := testNewContext(d)
err = ctx.NewTxn(context.Background())
c.Assert(err, IsNil)
Expand Down
15 changes: 10 additions & 5 deletions ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -197,7 +197,8 @@ func (s *testColumnSuite) TestColumnBasic(c *C) {
c.Assert(err, IsNil)
}()

tblInfo := testTableInfo(c, d, "t1", 3)
tblInfo, err := testTableInfo(d, "t1", 3)
c.Assert(err, IsNil)
ctx := testNewContext(d)

testCreateTable(c, ctx, d, s.dbInfo, tblInfo)
Expand Down Expand Up @@ -843,7 +844,8 @@ func (s *testColumnSuite) TestAddColumn(c *C) {
WithLease(testLease),
)
c.Assert(err, IsNil)
tblInfo := testTableInfo(c, d, "t", 3)
tblInfo, err := testTableInfo(d, "t", 3)
c.Assert(err, IsNil)
ctx := testNewContext(d)

err = ctx.NewTxn(context.Background())
Expand Down Expand Up @@ -931,7 +933,8 @@ func (s *testColumnSuite) TestAddColumns(c *C) {
WithLease(testLease),
)
c.Assert(err, IsNil)
tblInfo := testTableInfo(c, d, "t", 3)
tblInfo, err := testTableInfo(d, "t", 3)
c.Assert(err, IsNil)
ctx := testNewContext(d)

err = ctx.NewTxn(context.Background())
Expand Down Expand Up @@ -1016,7 +1019,8 @@ func (s *testColumnSuite) TestDropColumn(c *C) {
WithLease(testLease),
)
c.Assert(err, IsNil)
tblInfo := testTableInfo(c, d, "t2", 4)
tblInfo, err := testTableInfo(d, "t2", 4)
c.Assert(err, IsNil)
ctx := testNewContext(d)

err = ctx.NewTxn(context.Background())
Expand Down Expand Up @@ -1092,7 +1096,8 @@ func (s *testColumnSuite) TestDropColumns(c *C) {
WithLease(testLease),
)
c.Assert(err, IsNil)
tblInfo := testTableInfo(c, d, "t2", 4)
tblInfo, err := testTableInfo(d, "t2", 4)
c.Assert(err, IsNil)
ctx := testNewContext(d)

err = ctx.NewTxn(context.Background())
Expand Down
97 changes: 97 additions & 0 deletions ddl/db_cache_serial_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
// Copyright 2021 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package ddl_test

import (
"testing"
"time"

"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/testkit"
"github.com/stretchr/testify/require"
)

func TestAlterTableCache(t *testing.T) {
store, err := mockstore.NewMockStore()
require.NoError(t, err)
session.SetSchemaLease(600 * time.Millisecond)
session.DisableStats4Test()
dom, err := session.BootstrapSession(store)
require.NoError(t, err)

dom.SetStatsUpdating(true)

clean := func() {
dom.Close()
err := store.Close()
require.NoError(t, err)
}
defer clean()
tk := testkit.NewTestKit(t, store)
tk2 := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("drop table if exists t1")
tk2.MustExec("use test")
/* Test of cache table */
tk.MustExec("create table t1 ( n int auto_increment primary key)")
tk.MustGetErrCode("alter table t1 ca", errno.ErrParse)
tk.MustGetErrCode("alter table t2 cache", errno.ErrNoSuchTable)
tk.MustExec("alter table t1 cache")
checkTableCacheStatus(t, tk.Session(), "test", "t1", model.TableCacheStatusEnable)
tk.MustExec("drop table if exists t1")
/*Test can't skip schema checker*/
tk.MustExec("drop table if exists t1,t2")
tk.MustExec("CREATE TABLE t1 (a int)")
tk.MustExec("CREATE TABLE t2 (a int)")
tk.MustExec("begin")
tk.MustExec("insert into t1 set a=1;")
tk2.MustExec("alter table t1 cache;")
_, err = tk.Exec("commit")
require.True(t, terror.ErrorEqual(domain.ErrInfoSchemaChanged, err))
/* Test can skip schema checker */
tk.MustExec("begin")
tk.MustExec("drop table if exists t1")
tk.MustExec("CREATE TABLE t1 (a int)")
tk.MustExec("insert into t1 set a=2;")
tk2.MustExec("alter table t2 cache")
tk.MustExec("commit")
// Test if a table is not exists
tk.MustExec("drop table if exists t")
tk.MustGetErrCode("alter table t cache", errno.ErrNoSuchTable)
tk.MustExec("create table t (a int)")
tk.MustExec("alter table t cache")
// Multiple alter cache is okay
tk.MustExec("alter table t cache")
tk.MustExec("alter table t cache")
// Test a temporary table
tk.MustExec("drop table if exists t")
tk.MustExec("create temporary table t (id int primary key auto_increment, u int unique, v int)")
tk.MustExec("drop table if exists tmp1")
// local temporary table alter is not supported
tk.MustGetErrCode("alter table t cache", errno.ErrUnsupportedDDLOperation)
// test global temporary table
tk.MustExec("create global temporary table tmp1 " +
"(id int not null primary key, code int not null, value int default null, unique key code(code))" +
"on commit delete rows")
tk.MustGetErrMsg("alter table tmp1 cache", ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache").Error())

}
Loading

0 comments on commit da21e07

Please sign in to comment.