diff --git a/ddl/placement_policy_test.go b/ddl/placement_policy_test.go index 3e6dc3a97bcaf..248718d0179fa 100644 --- a/ddl/placement_policy_test.go +++ b/ddl/placement_policy_test.go @@ -203,12 +203,14 @@ func (s *testDBSuite6) TestAlterPlacementPolicy(c *C) { // test for normal cases tk.MustExec("alter placement policy x PRIMARY_REGION=\"bj\" REGIONS=\"bj,sh\"") tk.MustQuery("show placement where target='POLICY x'").Check(testkit.Rows("POLICY x PRIMARY_REGION=\"bj\" REGIONS=\"bj,sh\"")) + tk.MustQuery("select * from information_schema.placement_rules where policy_name = 'x'").Check(testkit.Rows("3 def x bj bj,sh 0 0")) tk.MustExec("alter placement policy x " + "PRIMARY_REGION=\"bj\" " + "REGIONS=\"bj\" " + "SCHEDULE=\"EVEN\"") tk.MustQuery("show placement where target='POLICY x'").Check(testkit.Rows("POLICY x PRIMARY_REGION=\"bj\" REGIONS=\"bj\" SCHEDULE=\"EVEN\"")) + tk.MustQuery("select * from INFORMATION_SCHEMA.PLACEMENT_RULES WHERE POLICY_NAME='x'").Check(testkit.Rows("3 def x bj bj EVEN 0 0")) tk.MustExec("alter placement policy x " + "LEADER_CONSTRAINTS=\"[+region=us-east-1]\" " + @@ -217,6 +219,9 @@ func (s *testDBSuite6) TestAlterPlacementPolicy(c *C) { tk.MustQuery("show placement where target='POLICY x'").Check( testkit.Rows("POLICY x LEADER_CONSTRAINTS=\"[+region=us-east-1]\" FOLLOWERS=3 FOLLOWER_CONSTRAINTS=\"[+region=us-east-2]\""), ) + tk.MustQuery("SELECT POLICY_NAME,LEADER_CONSTRAINTS,FOLLOWER_CONSTRAINTS,FOLLOWERS FROM information_schema.PLACEMENT_RULES WHERE POLICY_NAME = 'x'").Check( + testkit.Rows("x [+region=us-east-1] [+region=us-east-2] 3"), + ) tk.MustExec("alter placement policy x " + "VOTER_CONSTRAINTS=\"[+region=bj]\" " + @@ -227,11 +232,18 @@ func (s *testDBSuite6) TestAlterPlacementPolicy(c *C) { tk.MustQuery("show placement where target='POLICY x'").Check( testkit.Rows("POLICY x CONSTRAINTS=\"[+disk=ssd]\" VOTERS=5 VOTER_CONSTRAINTS=\"[+region=bj]\" LEARNERS=3 LEARNER_CONSTRAINTS=\"[+region=sh]\""), ) + tk.MustQuery("SELECT " + + "CATALOG_NAME,POLICY_NAME,SCHEMA_NAME,TABLE_NAME,PARTITION_NAME," + + "PRIMARY_REGION,REGIONS,CONSTRAINTS,LEADER_CONSTRAINTS,FOLLOWER_CONSTRAINTS,LEARNER_CONSTRAINTS," + + "SCHEDULE,FOLLOWERS,LEARNERS FROM INFORMATION_SCHEMA.placement_rules WHERE POLICY_NAME='x'").Check( + testkit.Rows("def x [+disk=ssd] [+region=sh] 0 3"), + ) // test alter not exist policies tk.MustExec("drop placement policy x") tk.MustGetErrCode("alter placement policy x REGIONS=\"bj,sh\"", mysql.ErrPlacementPolicyNotExists) tk.MustGetErrCode("alter placement policy x2 REGIONS=\"bj,sh\"", mysql.ErrPlacementPolicyNotExists) + tk.MustQuery("select * from INFORMATION_SCHEMA.PLACEMENT_RULES WHERE POLICY_NAME='x'").Check(testkit.Rows()) } func (s *testDBSuite6) TestCreateTableWithPlacementPolicy(c *C) { @@ -270,6 +282,7 @@ func (s *testDBSuite6) TestCreateTableWithPlacementPolicy(c *C) { c.Assert(policySetting.Schedule, Equals, "") } checkFunc(tbl.Meta().DirectPlacementOpts) + tk.MustQuery("SELECT * FROM information_schema.placement_rules WHERE TABLE_NAME = 't'").Check(testkit.Rows(" def test t cn-east-1 cn-east-1, cn-east-2 2 0")) tk.MustExec("drop table if exists t") // Direct placement option and placement policy can't co-exist. @@ -296,6 +309,7 @@ func (s *testDBSuite6) TestCreateTableWithPlacementPolicy(c *C) { c.Assert(tbl.Meta().PlacementPolicyRef, NotNil) c.Assert(tbl.Meta().PlacementPolicyRef.Name.L, Equals, "x") c.Assert(tbl.Meta().PlacementPolicyRef.ID != 0, Equals, true) + tk.MustQuery("SELECT * FROM information_schema.placement_rules WHERE TABLE_NAME = 't'").Check(testkit.Rows()) tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)" + @@ -319,6 +333,7 @@ func (s *testDBSuite6) TestCreateTableWithPlacementPolicy(c *C) { c.Assert(policySetting.Schedule, Equals, "") } checkFunc(tbl.Meta().DirectPlacementOpts) + tk.MustQuery("SELECT * FROM information_schema.placement_rules WHERE TABLE_NAME = 't'").Check(testkit.Rows(" def test t [+disk=ssd] 2 0")) tk.MustExec("drop table if exists t") tk.MustExec("drop placement policy if exists x") } @@ -510,6 +525,11 @@ func (s *testDBSuite6) TestAlterTablePartitionWithPlacementPolicy(c *C) { "PARTITION p1 VALUES LESS THAN (11)," + "PARTITION p2 VALUES LESS THAN (16)," + "PARTITION p3 VALUES LESS THAN (21));") + tk.MustQuery("SELECT " + + "CATALOG_NAME,POLICY_NAME,SCHEMA_NAME,TABLE_NAME,PARTITION_NAME," + + "PRIMARY_REGION,REGIONS,CONSTRAINTS,LEADER_CONSTRAINTS,FOLLOWER_CONSTRAINTS,LEARNER_CONSTRAINTS," + + "SCHEDULE,FOLLOWERS,LEARNERS FROM INFORMATION_SCHEMA.placement_rules WHERE table_NAME='t1'").Check( + testkit.Rows()) tk.MustExec("alter table t1 partition p0 " + "PRIMARY_REGION=\"cn-east-1\" " + @@ -535,6 +555,11 @@ func (s *testDBSuite6) TestAlterTablePartitionWithPlacementPolicy(c *C) { c.Assert(policySetting.Schedule, Equals, "") } checkFunc(ptDef.DirectPlacementOpts) + tk.MustQuery("SELECT " + + "CATALOG_NAME,POLICY_NAME,SCHEMA_NAME,TABLE_NAME,PARTITION_NAME," + + "PRIMARY_REGION,REGIONS,CONSTRAINTS,LEADER_CONSTRAINTS,FOLLOWER_CONSTRAINTS,LEARNER_CONSTRAINTS," + + "SCHEDULE,FOLLOWERS,LEARNERS FROM INFORMATION_SCHEMA.placement_rules WHERE TABLE_NAME='t1'").Check( + testkit.Rows("def test t1 p0 cn-east-1 cn-east-1, cn-east-2 2 0")) //Direct placement option and placement policy can't co-exist. _, err := tk.Exec("alter table t1 partition p0 " + diff --git a/ddl/placement_sql_test.go b/ddl/placement_sql_test.go index 2a37d26293bbe..c9b62cf552520 100644 --- a/ddl/placement_sql_test.go +++ b/ddl/placement_sql_test.go @@ -735,6 +735,7 @@ func (s *testDBSuite6) TestCreateSchemaWithPlacement(c *C) { tk.MustExec(`CREATE SCHEMA SchemaDirectPlacementTest PRIMARY_REGION='nl' REGIONS = "se,nz,nl" FOLLOWERS=3`) tk.MustQuery("SHOW CREATE SCHEMA schemadirectplacementtest").Check(testkit.Rows("SchemaDirectPlacementTest CREATE DATABASE `SchemaDirectPlacementTest` /*!40100 DEFAULT CHARACTER SET utf8mb4 */ /*T![placement] PRIMARY_REGION=\"nl\" REGIONS=\"se,nz,nl\" FOLLOWERS=3 */")) + tk.MustQuery("SELECT * FROM information_schema.placement_rules WHERE SCHEMA_NAME='SchemaDirectPlacementTest'").Check(testkit.Rows(" def SchemaDirectPlacementTest nl se,nz,nl 3 0")) tk.MustExec(`CREATE PLACEMENT POLICY PolicySchemaTest LEADER_CONSTRAINTS = "[+region=nl]" FOLLOWER_CONSTRAINTS="[+region=se]" FOLLOWERS=4 LEARNER_CONSTRAINTS="[+region=be]" LEARNERS=4`) tk.MustExec(`CREATE PLACEMENT POLICY PolicyTableTest LEADER_CONSTRAINTS = "[+region=tl]" FOLLOWER_CONSTRAINTS="[+region=tf]" FOLLOWERS=2 LEARNER_CONSTRAINTS="[+region=tle]" LEARNERS=1`) diff --git a/executor/builder.go b/executor/builder.go index 30fd402ee2466..b78a4df0c1dff 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1597,7 +1597,8 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo strings.ToLower(infoschema.TableClientErrorsSummaryGlobal), strings.ToLower(infoschema.TableClientErrorsSummaryByUser), strings.ToLower(infoschema.TableClientErrorsSummaryByHost), - strings.ToLower(infoschema.TableRegionLabel): + strings.ToLower(infoschema.TableRegionLabel), + strings.ToLower(infoschema.TablePlacementRules): return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), table: v.Table, diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index bb0caa1662f8c..423dc7823961d 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -166,6 +166,8 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex err = e.setDataForClientErrorsSummary(sctx, e.table.Name.O) case infoschema.TableRegionLabel: err = e.setDataForRegionLabel(sctx) + case infoschema.TablePlacementRules: + err = e.setDataFromPlacementRules(ctx, sctx, dbs) } if err != nil { return nil, err @@ -2829,6 +2831,135 @@ func (e *memtableRetriever) setDataForRegionLabel(ctx sessionctx.Context) error return nil } +func (e *memtableRetriever) setDataFromPlacementRules(ctx context.Context, sctx sessionctx.Context, schemas []*model.DBInfo) error { + checker := privilege.GetPrivilegeManager(sctx) + is := sctx.GetInfoSchema().(infoschema.InfoSchema) + var rows [][]types.Datum + + // Get global PLACEMENT POLICIES + // Currently no privileges needed for seeing global PLACEMENT POLICIES! + for _, policy := range is.AllPlacementPolicies() { + // Currently we skip converting syntactic sugar. We might revisit this decision still in the future + // I.e.: if PrimaryRegion or Regions are set, + // also convert them to LeaderConstraints and FollowerConstraints + // for better user experience searching for particular constraints + + row := types.MakeDatums( + policy.ID, + infoschema.CatalogVal, // CATALOG + policy.Name.O, // Policy Name + nil, // dbName, // SCHEMA + nil, // tbName, // TABLE + nil, // ptName, // PARTITION + policy.PlacementSettings.PrimaryRegion, + policy.PlacementSettings.Regions, + policy.PlacementSettings.Constraints, + policy.PlacementSettings.LeaderConstraints, + policy.PlacementSettings.FollowerConstraints, + policy.PlacementSettings.LearnerConstraints, + policy.PlacementSettings.Schedule, + policy.PlacementSettings.Followers, + policy.PlacementSettings.Learners, + ) + rows = append(rows, row) + } + + // Get DIRECT PLACEMENT from schemas/tables/partitions + for _, schema := range schemas { + // Traverse all schemas and all tables (and eventually all partitions) + // to extract any Direct Placement information on Schema/Table/Partition. + // Currently there is no filtering during traversal implemented for queries like + // SELECT * FROM placment_rules WHERE SCHEMA_NAME IN ('schema1', 'schema2') + // or SELECT * FROM placment_rules WHERE SCHEMA_NAME = 'schema1' AND TABLE_NAME = 'table1' + anyTablePriv := false + for _, table := range schema.Tables { + if table.IsView() { + continue + } + // TODO: Filter on table, to avoid iterating over every table if SELECT * FROM placment_rules WHERE TABLE_NAME IN ('t1', 't2') + // Any privilege on the schema or a table within the schema should allow showing the direct placement rules for that schema (on schema level) + if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { + continue + } + anyTablePriv = true + if partInfo := table.GetPartitionInfo(); partInfo != nil { + for _, pi := range partInfo.Definitions { + if pi.DirectPlacementOpts != nil { + record := types.MakeDatums( + nil, // PLACEMENT POLICY ID, null since direct placement + infoschema.CatalogVal, // CATALOG + nil, // PLACEMENT POLICY, null since direct placement + schema.Name.O, // SCHEMA + table.Name.O, // TABLE + pi.Name.O, // PARTITION + pi.DirectPlacementOpts.PrimaryRegion, + pi.DirectPlacementOpts.Regions, + pi.DirectPlacementOpts.Constraints, + pi.DirectPlacementOpts.LeaderConstraints, + pi.DirectPlacementOpts.FollowerConstraints, + pi.DirectPlacementOpts.LearnerConstraints, + pi.DirectPlacementOpts.Schedule, + pi.DirectPlacementOpts.Followers, + pi.DirectPlacementOpts.Learners, + ) + rows = append(rows, record) + } + } + } + if table.DirectPlacementOpts == nil { + continue + } + record := types.MakeDatums( + nil, // PLACEMENT POLICY ID, null since direct placement + infoschema.CatalogVal, // CATALOG + nil, // PLACEMENT POLICY, null since direct placement + schema.Name.O, // SCHEMA + table.Name.O, // TABLE + nil, // PARTITION + table.DirectPlacementOpts.PrimaryRegion, + table.DirectPlacementOpts.Regions, + table.DirectPlacementOpts.Constraints, + table.DirectPlacementOpts.LeaderConstraints, + table.DirectPlacementOpts.FollowerConstraints, + table.DirectPlacementOpts.LearnerConstraints, + table.DirectPlacementOpts.Schedule, + table.DirectPlacementOpts.Followers, + table.DirectPlacementOpts.Learners, + ) + rows = append(rows, record) + } + // Any privilege on global level, the schema or any table within that schema + // should allow showing the direct placement rules for that schema (on schema level) + if !anyTablePriv && checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, schema.Name.L, "", "", mysql.AllPrivMask) { + continue + } + if schema.DirectPlacementOpts == nil { + continue + } + record := types.MakeDatums( + nil, // PLACEMENT POLICY ID, null since direct placement + infoschema.CatalogVal, // CATALOG + nil, // PLACEMENT POLICY, null since direct placement + schema.Name.O, // SCHEMA + nil, // TABLE + nil, // PARTITION + schema.DirectPlacementOpts.PrimaryRegion, + schema.DirectPlacementOpts.Regions, + schema.DirectPlacementOpts.Constraints, + schema.DirectPlacementOpts.LeaderConstraints, + schema.DirectPlacementOpts.FollowerConstraints, + schema.DirectPlacementOpts.LearnerConstraints, + schema.DirectPlacementOpts.Schedule, + schema.DirectPlacementOpts.Followers, + schema.DirectPlacementOpts.Learners, + ) + rows = append(rows, record) + } + + e.rows = rows + return nil +} + func checkRule(rule *label.Rule) (dbName, tableName string, err error) { s := strings.Split(rule.ID, "/") if len(s) < 3 { diff --git a/infoschema/infoschema_test.go b/infoschema/infoschema_test.go index ffc321cc16b31..aba6617de3930 100644 --- a/infoschema/infoschema_test.go +++ b/infoschema/infoschema_test.go @@ -297,6 +297,7 @@ func TestInfoTables(t *testing.T) { "PROCESSLIST", "TIDB_TRX", "DEADLOCKS", + "PLACEMENT_RULES", } for _, tbl := range infoTables { tb, err1 := is.TableByName(util.InformationSchemaName, model.NewCIStr(tbl)) diff --git a/infoschema/tables.go b/infoschema/tables.go index 7911670b2eacd..8513c48bcb223 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -179,6 +179,8 @@ const ( TableDataLockWaits = "DATA_LOCK_WAITS" // TableRegionLabel is the string constant of region label table. TableRegionLabel = "REGION_LABEL" + // TablePlacementRules is the string constant of placement rules table. + TablePlacementRules = "PLACEMENT_RULES" ) const ( @@ -275,6 +277,7 @@ var tableIDMap = map[string]int64{ ClusterTableStatementsSummaryEvicted: autoid.InformationSchemaDBID + 76, TableRegionLabel: autoid.InformationSchemaDBID + 77, TableTiDBHotRegionsHistory: autoid.InformationSchemaDBID + 78, + TablePlacementRules: autoid.InformationSchemaDBID + 79, } type columnInfo struct { @@ -1455,6 +1458,24 @@ var tableRegionLabelCols = []columnInfo{ {name: "RANGES", tp: mysql.TypeBlob, size: types.UnspecifiedLength}, } +var tablePlacementRulesCols = []columnInfo{ + {name: "POLICY_ID", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, + {name: "CATALOG_NAME", tp: mysql.TypeVarchar, size: 512, flag: mysql.NotNullFlag}, + {name: "POLICY_NAME", tp: mysql.TypeVarchar, size: types.UnspecifiedLength}, // Catalog wide policy + {name: "SCHEMA_NAME", tp: mysql.TypeVarchar, size: types.UnspecifiedLength}, // System policy does not have a schema + {name: "TABLE_NAME", tp: mysql.TypeVarchar, size: types.UnspecifiedLength}, // Schema level rules does not have a table + {name: "PARTITION_NAME", tp: mysql.TypeVarchar, size: types.UnspecifiedLength}, // Table level rules does not have a partition + {name: "PRIMARY_REGION", tp: mysql.TypeVarchar, size: types.UnspecifiedLength, flag: mysql.NotNullFlag}, + {name: "REGIONS", tp: mysql.TypeVarchar, size: types.UnspecifiedLength, flag: mysql.NotNullFlag}, + {name: "CONSTRAINTS", tp: mysql.TypeVarchar, size: types.UnspecifiedLength, flag: mysql.NotNullFlag}, + {name: "LEADER_CONSTRAINTS", tp: mysql.TypeVarchar, size: types.UnspecifiedLength, flag: mysql.NotNullFlag}, + {name: "FOLLOWER_CONSTRAINTS", tp: mysql.TypeVarchar, size: types.UnspecifiedLength, flag: mysql.NotNullFlag}, + {name: "LEARNER_CONSTRAINTS", tp: mysql.TypeVarchar, size: types.UnspecifiedLength, flag: mysql.NotNullFlag}, + {name: "SCHEDULE", tp: mysql.TypeVarchar, size: 20, flag: mysql.NotNullFlag}, // EVEN or MAJORITY_IN_PRIMARY + {name: "FOLLOWERS", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, + {name: "LEARNERS", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, +} + // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. // The returned description string may be: // - "NOT_SHARDED": for tables that SHARD_ROW_ID_BITS is not specified. @@ -1839,6 +1860,7 @@ var tableNameToColumns = map[string][]columnInfo{ TableDeadlocks: tableDeadlocksCols, TableDataLockWaits: tableDataLockWaitsCols, TableRegionLabel: tableRegionLabelCols, + TablePlacementRules: tablePlacementRulesCols, } func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) { diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index dd1e2a9b04806..a12076c4ae815 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -2591,6 +2591,59 @@ func TestDBNameCaseSensitivityInTableLevel(t *testing.T) { mustExec(t, se, "grant select on metrics_schema.up to test_user;") } +func TestInformationSchemaPlacmentRulesPrivileges(t *testing.T) { + t.Parallel() + store, clean := newStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + + defer func() { + require.True(t, tk.Session().Auth(&auth.UserIdentity{ + Username: "root", + Hostname: "localhost", + }, nil, nil)) + tk.MustExec(`DROP SCHEMA IF EXISTS placment_rule_db`) + tk.MustExec(`DROP USER IF EXISTS placement_rule_user_scheam`) + tk.MustExec(`DROP USER IF EXISTS placement_rule_user_table`) + }() + tk.MustExec("CREATE DATABASE placement_rule_db") + tk.MustExec("USE placement_rule_db") + tk.MustExec(`CREATE TABLE placement_rule_table_se (a int) PRIMARY_REGION="se" REGIONS="se,nl"`) + tk.MustExec(`CREATE TABLE placement_rule_table_nl (a int) PRIMARY_REGION="nl" REGIONS="se,nl"`) + tk.MustQuery(`SELECT * FROM information_schema.placement_rules WHERE SCHEMA_NAME = "placement_rule_db"`).Sort().Check(testkit.Rows( + " def placement_rule_db placement_rule_table_nl nl se,nl 0 0", + " def placement_rule_db placement_rule_table_se se se,nl 0 0")) + tk.MustExec("CREATE USER placement_rule_user_schema") + tk.MustExec("CREATE USER placement_rule_user_table") + tk.MustExec("GRANT SELECT ON placement_rule_db.placement_rule_table_se TO placement_rule_user_table") + + require.True(t, tk.Session().Auth(&auth.UserIdentity{ + Username: "placement_rule_user_schema", + Hostname: "somehost", + }, nil, nil)) + tk.MustQuery(`SELECT * FROM information_schema.placement_rules WHERE SCHEMA_NAME = "placement_rule_db"`).Check(testkit.Rows()) + + require.True(t, tk.Session().Auth(&auth.UserIdentity{ + Username: "placement_rule_user_table", + Hostname: "somehost", + }, nil, nil)) + tk.MustQuery(`SELECT * FROM information_schema.placement_rules WHERE SCHEMA_NAME = "placement_rule_db"`).Check(testkit.Rows(" def placement_rule_db placement_rule_table_se se se,nl 0 0")) + + require.True(t, tk.Session().Auth(&auth.UserIdentity{ + Username: "root", + Hostname: "localhost", + }, nil, nil)) + tk.MustExec("GRANT SELECT ON placement_rule_db.* TO placement_rule_user_schema") + require.True(t, tk.Session().Auth(&auth.UserIdentity{ + Username: "placement_rule_user_schema", + Hostname: "somehost", + }, nil, nil)) + tk.MustQuery(`SELECT * FROM information_schema.placement_rules WHERE SCHEMA_NAME = "placement_rule_db"`).Sort().Check(testkit.Rows( + " def placement_rule_db placement_rule_table_nl nl se,nl 0 0", + " def placement_rule_db placement_rule_table_se se se,nl 0 0")) +} + func TestGrantCreateTmpTables(t *testing.T) { t.Parallel() store, clean := newStore(t)