Skip to content

Commit

Permalink
*: Support SHOW PLACEMENT for placement policies (#27531)
Browse files Browse the repository at this point in the history
  • Loading branch information
lcwangchao authored Aug 31, 2021
1 parent 8fd6193 commit 376e85b
Show file tree
Hide file tree
Showing 10 changed files with 307 additions and 84 deletions.
2 changes: 2 additions & 0 deletions executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -215,6 +215,8 @@ func (e *ShowExec) fetchAll(ctx context.Context) error {
return e.fetchShowBRIE(ast.BRIEKindRestore)
case ast.ShowPlacementLabels:
return e.fetchShowPlacementLabels(ctx)
case ast.ShowPlacement:
return e.fetchShowPlacement(ctx)
}
return nil
}
Expand Down
21 changes: 21 additions & 0 deletions executor/show_placement.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,3 +127,24 @@ func (e *ShowExec) fetchShowPlacementLabels(ctx context.Context) error {

return nil
}

func (e *ShowExec) fetchShowPlacement(_ context.Context) error {
err := e.fetchAllPlacementPolicies()
if err != nil {
return err
}

return nil
}

func (e *ShowExec) fetchAllPlacementPolicies() error {
policies := e.is.AllPlacementPolicies()
sort.Slice(policies, func(i, j int) bool { return policies[i].Name.O < policies[j].Name.O })
for _, policy := range policies {
name := policy.Name
settings := policy.PlacementSettings
e.appendRow([]interface{}{"POLICY " + name.String(), settings.String(), "SCHEDULED"})
}

return nil
}
85 changes: 85 additions & 0 deletions executor/show_placement_labels_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
// 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 executor

import (
gjson "encoding/json"

. "github.com/pingcap/check"
"github.com/pingcap/tidb/store/helper"
"github.com/pingcap/tidb/types/json"
)

var _ = SerialSuites(&testShowPlacementLabelSuit{})

type testShowPlacementLabelSuit struct {
}

func (s *testShowPlacementLabelSuit) TestShowPlacementLabelsBuilder(c *C) {
cases := []struct {
stores [][]*helper.StoreLabel
expects [][]interface{}
}{
{
stores: nil,
expects: nil,
},
{
stores: [][]*helper.StoreLabel{
{{Key: "zone", Value: "z1"}, {Key: "rack", Value: "r3"}, {Key: "host", Value: "h1"}},
{{Key: "zone", Value: "z1"}, {Key: "rack", Value: "r1"}, {Key: "host", Value: "h2"}},
{{Key: "zone", Value: "z1"}, {Key: "rack", Value: "r2"}, {Key: "host", Value: "h2"}},
{{Key: "zone", Value: "z2"}, {Key: "rack", Value: "r1"}, {Key: "host", Value: "h2"}},
nil,
{{Key: "k1", Value: "v1"}},
},
expects: [][]interface{}{
{"host", []string{"h1", "h2"}},
{"k1", []string{"v1"}},
{"rack", []string{"r1", "r2", "r3"}},
{"zone", []string{"z1", "z2"}},
},
},
}

b := &showPlacementLabelsResultBuilder{}
toBinaryJSON := func(obj interface{}) (bj json.BinaryJSON) {
d, err := gjson.Marshal(obj)
c.Assert(err, IsNil)
err = bj.UnmarshalJSON(d)
c.Assert(err, IsNil)
return
}

for _, ca := range cases {
for _, store := range ca.stores {
bj := toBinaryJSON(store)
err := b.AppendStoreLabels(bj)
c.Assert(err, IsNil)
}

rows, err := b.BuildRows()
c.Assert(err, IsNil)
c.Assert(len(rows), Equals, len(ca.expects))
for idx, expect := range ca.expects {
row := rows[idx]
bj := toBinaryJSON(expect[1])

c.Assert(row[0].(string), Equals, expect[0].(string))
c.Assert(row[1].(json.BinaryJSON).TypeCode, Equals, bj.TypeCode)
c.Assert(row[1].(json.BinaryJSON).Value, BytesEquals, bj.Value)
}
}
}
99 changes: 40 additions & 59 deletions executor/show_placement_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,74 +12,55 @@
// See the License for the specific language governing permissions and
// limitations under the License.

package executor
package executor_test

import (
gjson "encoding/json"

. "github.com/pingcap/check"
"github.com/pingcap/tidb/store/helper"
"github.com/pingcap/tidb/types/json"
"github.com/pingcap/tidb/util/testkit"
)

var _ = SerialSuites(&testShowPlacementSuit{})
func (s *testSuite5) TestShowPlacement(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("drop placement policy if exists p1")

type testShowPlacementSuit struct {
}
tk.MustExec("create placement policy pa1 " +
"PRIMARY_REGION=\"cn-east-1\" " +
"REGIONS=\"cn-east-1,cn-east-2\"" +
"SCHEDULE=\"EVEN\"")
defer tk.MustExec("drop placement policy pa1")

tk.MustExec("create placement policy pa2 " +
"LEADER_CONSTRAINTS=\"[+region=us-east-1]\" " +
"FOLLOWER_CONSTRAINTS=\"[+region=us-east-2]\" " +
"FOLLOWERS=3")
defer tk.MustExec("drop placement policy pa2")

func (s *testPartitionSuite) TestShowPlacementLabelsBuilder(c *C) {
cases := []struct {
stores [][]*helper.StoreLabel
expects [][]interface{}
}{
{
stores: nil,
expects: nil,
},
{
stores: [][]*helper.StoreLabel{
{{Key: "zone", Value: "z1"}, {Key: "rack", Value: "r3"}, {Key: "host", Value: "h1"}},
{{Key: "zone", Value: "z1"}, {Key: "rack", Value: "r1"}, {Key: "host", Value: "h2"}},
{{Key: "zone", Value: "z1"}, {Key: "rack", Value: "r2"}, {Key: "host", Value: "h2"}},
{{Key: "zone", Value: "z2"}, {Key: "rack", Value: "r1"}, {Key: "host", Value: "h2"}},
nil,
{{Key: "k1", Value: "v1"}},
},
expects: [][]interface{}{
{"host", []string{"h1", "h2"}},
{"k1", []string{"v1"}},
{"rack", []string{"r1", "r2", "r3"}},
{"zone", []string{"z1", "z2"}},
},
},
}
tk.MustExec("create placement policy pb1 " +
"VOTER_CONSTRAINTS=\"[+region=bj]\" " +
"LEARNER_CONSTRAINTS=\"[+region=sh]\" " +
"CONSTRAINTS=\"[+disk=ssd]\"" +
"VOTERS=5 " +
"LEARNERS=3")
defer tk.MustExec("drop placement policy pb1")

b := &showPlacementLabelsResultBuilder{}
toBinaryJSON := func(obj interface{}) (bj json.BinaryJSON) {
d, err := gjson.Marshal(obj)
c.Assert(err, IsNil)
err = bj.UnmarshalJSON(d)
c.Assert(err, IsNil)
return
}
tk.MustQuery("show placement").Check(testkit.Rows(
"POLICY pa1 PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\" SCHEDULED",
"POLICY pa2 LEADER_CONSTRAINTS=\"[+region=us-east-1]\" FOLLOWERS=3 FOLLOWER_CONSTRAINTS=\"[+region=us-east-2]\" SCHEDULED",
"POLICY pb1 CONSTRAINTS=\"[+disk=ssd]\" VOTERS=5 VOTER_CONSTRAINTS=\"[+region=bj]\" LEARNERS=3 LEARNER_CONSTRAINTS=\"[+region=sh]\" SCHEDULED",
))

for _, ca := range cases {
for _, store := range ca.stores {
bj := toBinaryJSON(store)
err := b.AppendStoreLabels(bj)
c.Assert(err, IsNil)
}
tk.MustQuery("show placement like 'POLICY%'").Check(testkit.Rows(
"POLICY pa1 PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\" SCHEDULED",
"POLICY pa2 LEADER_CONSTRAINTS=\"[+region=us-east-1]\" FOLLOWERS=3 FOLLOWER_CONSTRAINTS=\"[+region=us-east-2]\" SCHEDULED",
"POLICY pb1 CONSTRAINTS=\"[+disk=ssd]\" VOTERS=5 VOTER_CONSTRAINTS=\"[+region=bj]\" LEARNERS=3 LEARNER_CONSTRAINTS=\"[+region=sh]\" SCHEDULED",
))

rows, err := b.BuildRows()
c.Assert(err, IsNil)
c.Assert(len(rows), Equals, len(ca.expects))
for idx, expect := range ca.expects {
row := rows[idx]
bj := toBinaryJSON(expect[1])
tk.MustQuery("show placement like 'POLICY pa%'").Check(testkit.Rows(
"POLICY pa1 PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\" SCHEDULED",
"POLICY pa2 LEADER_CONSTRAINTS=\"[+region=us-east-1]\" FOLLOWERS=3 FOLLOWER_CONSTRAINTS=\"[+region=us-east-2]\" SCHEDULED",
))

c.Assert(row[0].(string), Equals, expect[0].(string))
c.Assert(row[1].(json.BinaryJSON).TypeCode, Equals, bj.TypeCode)
c.Assert(row[1].(json.BinaryJSON).Value, BytesEquals, bj.Value)
}
}
tk.MustQuery("show placement where Target='POLICY pb1'").Check(testkit.Rows(
"POLICY pb1 CONSTRAINTS=\"[+disk=ssd]\" VOTERS=5 VOTER_CONSTRAINTS=\"[+region=bj]\" LEARNERS=3 LEARNER_CONSTRAINTS=\"[+region=sh]\" SCHEDULED",
))
}
2 changes: 1 addition & 1 deletion infoschema/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -537,7 +537,7 @@ func (b *Builder) copyPoliciesMap(oldIS *infoSchema) {
is := b.is
is.policyMutex.Lock()
defer is.policyMutex.Unlock()
for _, v := range oldIS.PlacementPolicies() {
for _, v := range oldIS.AllPlacementPolicies() {
is.policyMap[v.Name.L] = v
}
}
Expand Down
5 changes: 4 additions & 1 deletion infoschema/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,8 @@ type InfoSchema interface {
SetBundle(*placement.Bundle)
// RuleBundles will return a copy of all rule bundles.
RuleBundles() []*placement.Bundle
// AllPlacementPolicies returns all placement policies
AllPlacementPolicies() []*placementpolicy.PolicyInfo
}

type sortedTables []table.Table
Expand Down Expand Up @@ -373,7 +375,8 @@ func (is *infoSchema) PolicyByName(name model.CIStr) (*placementpolicy.PolicyInf
return t, r
}

func (is *infoSchema) PlacementPolicies() []*placementpolicy.PolicyInfo {
// AllPlacementPolicies returns all placement policies
func (is *infoSchema) AllPlacementPolicies() []*placementpolicy.PolicyInfo {
is.policyMutex.RLock()
defer is.policyMutex.RUnlock()
policies := make([]*placementpolicy.PolicyInfo, 0, len(is.policyMap))
Expand Down
20 changes: 11 additions & 9 deletions meta/meta_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,15 +52,17 @@ func TestPlacementPolicy(t *testing.T) {

// test the meta storage of placemnt policy.
policy := &placementpolicy.PolicyInfo{
Name: model.NewCIStr("aa"),
PrimaryRegion: "my primary",
Regions: "my regions",
Learners: 1,
Followers: 2,
Voters: 3,
Schedule: "even",
Constraints: "+disk=ssd",
LearnerConstraints: "+zone=shanghai",
Name: model.NewCIStr("aa"),
PlacementSettings: placementpolicy.PlacementSettings{
PrimaryRegion: "my primary",
Regions: "my regions",
Learners: 1,
Followers: 2,
Voters: 3,
Schedule: "even",
Constraints: "+disk=ssd",
LearnerConstraints: "+zone=shanghai",
},
}
err = m.CreatePolicy(policy)
require.NoError(t, err)
Expand Down
3 changes: 3 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -4130,6 +4130,9 @@ func buildShowSchema(s *ast.ShowStmt, isView bool, isSequence bool) (schema *exp
case ast.ShowPlacementLabels:
names = []string{"Key", "Values"}
ftypes = []byte{mysql.TypeVarchar, mysql.TypeJSON}
case ast.ShowPlacement:
names = []string{"Target", "Placement", "Scheduling_state"}
ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar}
}

schema = expression.NewSchema(make([]*expression.Column, 0, len(names))...)
Expand Down
Loading

0 comments on commit 376e85b

Please sign in to comment.