Skip to content

Commit

Permalink
infoschema: supply ut to verify special attributes' correctness (ping…
Browse files Browse the repository at this point in the history
  • Loading branch information
fzzf678 committed Aug 22, 2024
1 parent 48989e7 commit 162cb15
Show file tree
Hide file tree
Showing 4 changed files with 229 additions and 2 deletions.
2 changes: 1 addition & 1 deletion pkg/infoschema/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,7 @@ go_test(
],
embed = [":infoschema"],
flaky = True,
shard_count = 22,
shard_count = 23,
deps = [
"//pkg/ddl/placement",
"//pkg/domain",
Expand Down
127 changes: 127 additions & 0 deletions pkg/infoschema/infoschema_v2_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/pingcap/tidb/pkg/infoschema/internal"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/meta"
"github.com/pingcap/tidb/pkg/meta/autoid"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -326,3 +327,129 @@ func TestBundles(t *testing.T) {
require.True(t, ok)
require.Equal(t, policyInfo, getPolicyInfo)
}

func updateTableSpecialAttribute(t *testing.T, dbInfo *model.DBInfo, tblInfo *model.TableInfo, builder *Builder, r autoid.Requirement,
actionType model.ActionType, ver int64, filter specialAttributeFilter, add bool) *model.TableInfo {
internal.UpdateTable(t, r.Store(), dbInfo, tblInfo)
txn, err := r.Store().Begin()
require.NoError(t, err)
_, err = builder.ApplyDiff(meta.NewMeta(txn), &model.SchemaDiff{Type: actionType, Version: ver, SchemaID: dbInfo.ID, TableID: tblInfo.ID})
require.NoError(t, err)
is := builder.Build(math.MaxUint64)
tblInfoRes := is.ListTablesWithSpecialAttribute(filter)
if add {
// add special attribute
require.Equal(t, 1, len(tblInfoRes))
require.Equal(t, 1, len(tblInfoRes[0].TableInfos))
return tblInfoRes[0].TableInfos[0]
}
require.Equal(t, 0, len(tblInfoRes))
return nil
}

func TestSpecialAttributeCorrectnessInSchemaChange(t *testing.T) {
r := internal.CreateAutoIDRequirement(t)
defer func() {
r.Store().Close()
}()

schemaName := model.NewCIStr("testDB")
tableName := model.NewCIStr("testTable")
builder := NewBuilder(r, nil, NewData(), variable.SchemaCacheSize.Load() > 0)
err := builder.InitWithDBInfos(nil, nil, nil, 1)
require.NoError(t, err)
is := builder.Build(math.MaxUint64)
require.Equal(t, 2, len(is.AllSchemas()))

// create database
dbInfo := internal.MockDBInfo(t, r.Store(), schemaName.O)
internal.AddDB(t, r.Store(), dbInfo)
txn, err := r.Store().Begin()
require.NoError(t, err)
_, err = builder.ApplyDiff(meta.NewMeta(txn), &model.SchemaDiff{Type: model.ActionCreateSchema, Version: 1, SchemaID: dbInfo.ID})
require.NoError(t, err)
is = builder.Build(math.MaxUint64)
require.Equal(t, 3, len(is.AllSchemas()))
require.NoError(t, txn.Rollback())

// create table
tblInfo := internal.MockTableInfo(t, r.Store(), tableName.O)
internal.AddTable(t, r.Store(), dbInfo, tblInfo)
txn, err = r.Store().Begin()
require.NoError(t, err)
_, err = builder.ApplyDiff(meta.NewMeta(txn), &model.SchemaDiff{Type: model.ActionCreateTable, Version: 2, SchemaID: dbInfo.ID, TableID: tblInfo.ID})
require.NoError(t, err)
is = builder.Build(math.MaxUint64)
tblInfos, err := is.SchemaTableInfos(context.Background(), dbInfo.Name)
require.NoError(t, err)
require.Equal(t, 1, len(tblInfos))
require.NoError(t, txn.Rollback())

// tests partition info correctness in schema change
tblInfo.Partition = &model.PartitionInfo{
Expr: "aa+1",
Columns: []model.CIStr{
model.NewCIStr("aa"),
},
Definitions: []model.PartitionDefinition{
{ID: 1, Name: model.NewCIStr("p1")},
{ID: 2, Name: model.NewCIStr("p2")},
},
Enable: true,
DDLState: model.StatePublic,
}
// add partition
tblInfo1 := updateTableSpecialAttribute(t, dbInfo, tblInfo, builder, r, model.ActionAddTablePartition, 3, PartitionAttribute, true)
require.Equal(t, tblInfo.Partition, tblInfo1.Partition)
// drop partition
tblInfo.Partition.Definitions = tblInfo.Partition.Definitions[:1]
tblInfo1 = updateTableSpecialAttribute(t, dbInfo, tblInfo, builder, r, model.ActionDropTablePartition, 4, PartitionAttribute, true)
require.Equal(t, tblInfo.Partition, tblInfo1.Partition)

// test placement policy correctness in schema change
tblInfo.PlacementPolicyRef = &model.PolicyRefInfo{
ID: 1,
Name: model.NewCIStr("p3"),
}
tblInfo1 = updateTableSpecialAttribute(t, dbInfo, tblInfo, builder, r, model.ActionAlterTablePlacement, 5, PlacementPolicyAttribute, true)
require.Equal(t, tblInfo.PlacementPolicyRef, tblInfo1.PlacementPolicyRef)
tblInfo.PlacementPolicyRef = nil
updateTableSpecialAttribute(t, dbInfo, tblInfo, builder, r, model.ActionAlterTablePlacement, 6, PlacementPolicyAttribute, false)

// test tiflash replica correctness in schema change
tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{
Count: 1,
Available: true,
LocationLabels: []string{"zone"},
}
tblInfo1 = updateTableSpecialAttribute(t, dbInfo, tblInfo, builder, r, model.ActionSetTiFlashReplica, 7, TiFlashAttribute, true)
require.Equal(t, tblInfo.TiFlashReplica, tblInfo1.TiFlashReplica)
tblInfo.TiFlashReplica = nil
updateTableSpecialAttribute(t, dbInfo, tblInfo, builder, r, model.ActionSetTiFlashReplica, 8, TiFlashAttribute, false)

// test table lock correctness in schema change
tblInfo.Lock = &model.TableLockInfo{
Tp: model.TableLockRead,
State: model.TableLockStatePublic,
TS: 1,
}
tblInfo1 = updateTableSpecialAttribute(t, dbInfo, tblInfo, builder, r, model.ActionLockTable, 9, TableLockAttribute, true)
require.Equal(t, tblInfo.Lock, tblInfo1.Lock)
tblInfo.Lock = nil
updateTableSpecialAttribute(t, dbInfo, tblInfo, builder, r, model.ActionUnlockTable, 10, TableLockAttribute, false)

// test foreign key correctness in schema change
tblInfo.ForeignKeys = []*model.FKInfo{{
ID: 1,
Name: model.NewCIStr("fk_1"),
RefSchema: model.NewCIStr("t"),
RefTable: model.NewCIStr("t"),
RefCols: []model.CIStr{model.NewCIStr("a")},
Cols: []model.CIStr{model.NewCIStr("t_a")},
State: model.StateWriteOnly,
}}
tblInfo1 = updateTableSpecialAttribute(t, dbInfo, tblInfo, builder, r, model.ActionAddForeignKey, 11, ForeignKeysAttribute, true)
require.Equal(t, tblInfo.ForeignKeys, tblInfo1.ForeignKeys)
tblInfo.ForeignKeys = nil
updateTableSpecialAttribute(t, dbInfo, tblInfo, builder, r, model.ActionDropForeignKey, 12, ForeignKeysAttribute, false)
}
4 changes: 3 additions & 1 deletion pkg/meta/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -34,14 +34,16 @@ go_test(
],
embed = [":meta"],
flaky = True,
shard_count = 14,
shard_count = 15,
deps = [
"//pkg/ddl",
"//pkg/infoschema",
"//pkg/kv",
"//pkg/parser",
"//pkg/parser/ast",
"//pkg/parser/model",
"//pkg/planner/core",
"//pkg/session",
"//pkg/store/mockstore",
"//pkg/testkit/testsetup",
"//pkg/util",
Expand Down
98 changes: 98 additions & 0 deletions pkg/meta/meta_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,14 @@ import (

"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/ddl"
"github.com/pingcap/tidb/pkg/infoschema"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/meta"
"github.com/pingcap/tidb/pkg/parser"
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/parser/model"
_ "github.com/pingcap/tidb/pkg/planner/core"
"github.com/pingcap/tidb/pkg/session"
"github.com/pingcap/tidb/pkg/store/mockstore"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/intest"
Expand Down Expand Up @@ -856,3 +858,99 @@ func benchFastJSONTableNameInfo(b *testing.B, sql string) {
intest.Assert(tbInfo.Name.L == "t")
}
}

func TestSpecialAttributeCorrectnessAfterBootstrap(t *testing.T) {
store, err := mockstore.NewMockStore()
require.NoError(t, err)
defer func() {
require.NoError(t, store.Close())
}()

// create database
dbInfo := &model.DBInfo{
ID: 10001,
Name: model.NewCIStr("sc"),
State: model.StatePublic,
}

// create table with special attributes
tblInfo := &model.TableInfo{
ID: 10002,
Name: model.NewCIStr("cs"),
State: model.StatePublic,
Partition: &model.PartitionInfo{
Definitions: []model.PartitionDefinition{
{ID: 11, Name: model.NewCIStr("p1")},
{ID: 22, Name: model.NewCIStr("p2")},
},
Enable: true,
},
ForeignKeys: []*model.FKInfo{{
ID: 1,
Name: model.NewCIStr("fk"),
RefTable: model.NewCIStr("t"),
RefCols: []model.CIStr{model.NewCIStr("a")},
Cols: []model.CIStr{model.NewCIStr("t_a")},
}},
TiFlashReplica: &model.TiFlashReplicaInfo{
Count: 0,
LocationLabels: []string{"a,b,c"},
Available: true,
},
Lock: &model.TableLockInfo{
Tp: model.TableLockRead,
State: model.TableLockStatePreLock,
TS: 0,
},
PlacementPolicyRef: &model.PolicyRefInfo{
ID: 1,
Name: model.NewCIStr("r1"),
},
TTLInfo: &model.TTLInfo{
IntervalExprStr: "1",
IntervalTimeUnit: int(ast.TimeUnitDay),
Enable: true,
JobInterval: "1h",
},
}

ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL)
err = kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error {
err := meta.NewMeta(txn).CreateDatabase(dbInfo)
require.NoError(t, err)
err = meta.NewMeta(txn).CreateTableOrView(dbInfo.ID, tblInfo)
require.NoError(t, err)
return errors.Trace(err)
})
require.NoError(t, err)

// bootstrap
dom, err := session.BootstrapSession(store)
require.NoError(t, err)
defer dom.Close()

// verify partition info correctness
tblInfoRes := dom.InfoSchema().ListTablesWithSpecialAttribute(infoschema.PartitionAttribute)
require.Equal(t, len(tblInfoRes[0].TableInfos), 1)
require.Equal(t, tblInfo.Partition, tblInfoRes[0].TableInfos[0].Partition)
// foreign key info
tblInfoRes = dom.InfoSchema().ListTablesWithSpecialAttribute(infoschema.ForeignKeysAttribute)
require.Equal(t, len(tblInfoRes[0].TableInfos), 1)
require.Equal(t, tblInfo.ForeignKeys, tblInfoRes[0].TableInfos[0].ForeignKeys)
// tiflash replica info
tblInfoRes = dom.InfoSchema().ListTablesWithSpecialAttribute(infoschema.TiFlashAttribute)
require.Equal(t, len(tblInfoRes[0].TableInfos), 1)
require.Equal(t, tblInfo.TiFlashReplica, tblInfoRes[0].TableInfos[0].TiFlashReplica)
// lock info
tblInfoRes = dom.InfoSchema().ListTablesWithSpecialAttribute(infoschema.TableLockAttribute)
require.Equal(t, len(tblInfoRes[0].TableInfos), 1)
require.Equal(t, tblInfo.Lock, tblInfoRes[0].TableInfos[0].Lock)
// placement policy
tblInfoRes = dom.InfoSchema().ListTablesWithSpecialAttribute(infoschema.PlacementPolicyAttribute)
require.Equal(t, len(tblInfoRes[0].TableInfos), 1)
require.Equal(t, tblInfo.PlacementPolicyRef, tblInfoRes[0].TableInfos[0].PlacementPolicyRef)
// ttl info
tblInfoRes = dom.InfoSchema().ListTablesWithSpecialAttribute(infoschema.TTLAttribute)
require.Equal(t, len(tblInfoRes[0].TableInfos), 1)
require.Equal(t, tblInfo.TTLInfo, tblInfoRes[0].TableInfos[0].TTLInfo)
}

0 comments on commit 162cb15

Please sign in to comment.