diff --git a/planner/cascades/group.go b/planner/cascades/group.go new file mode 100644 index 0000000000000..c044ec78fea1a --- /dev/null +++ b/planner/cascades/group.go @@ -0,0 +1,72 @@ +// Copyright 2018 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cascades + +import ( + "container/list" + "fmt" +) + +// Group is short for expression group, which is used to store all the +// logically equivalent expressions. It's a set of GroupExpr. +type Group struct { + equivalents *list.List + fingerprints map[string]*list.Element + + explored bool + selfFingerprint string +} + +// NewGroup creates a new Group. +func NewGroup(e *GroupExpr) *Group { + g := &Group{ + equivalents: list.New(), + fingerprints: make(map[string]*list.Element), + } + g.Insert(e) + return g +} + +// FingerPrint returns the unique fingerprint of the group. +func (g *Group) FingerPrint() string { + if g.selfFingerprint == "" { + g.selfFingerprint = fmt.Sprintf("%p", g) + } + return g.selfFingerprint +} + +// Insert a nonexistent group expression. +func (g *Group) Insert(e *GroupExpr) bool { + if g.Exists(e) { + return false + } + newEquiv := g.equivalents.PushBack(e) + g.fingerprints[e.FingerPrint()] = newEquiv + return true +} + +// Delete an existing group expression. +func (g *Group) Delete(e *GroupExpr) { + fingerprint := e.FingerPrint() + if equiv, ok := g.fingerprints[fingerprint]; ok { + g.equivalents.Remove(equiv) + delete(g.fingerprints, fingerprint) + } +} + +// Exists checks whether a group expression existed in a Group. +func (g *Group) Exists(e *GroupExpr) bool { + _, ok := g.fingerprints[e.FingerPrint()] + return ok +} diff --git a/planner/cascades/group_expr.go b/planner/cascades/group_expr.go new file mode 100644 index 0000000000000..cda5bd08927dd --- /dev/null +++ b/planner/cascades/group_expr.go @@ -0,0 +1,53 @@ +// Copyright 2018 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cascades + +import ( + "fmt" + + plannercore "github.com/pingcap/tidb/planner/core" +) + +// GroupExpr is used to store all the logically equivalent expressions which +// have the same root operator. Different from a normal expression, the +// children of a group expression are expression Groups, not expressions. +// Another property of group expression is that the child group references will +// never be changed once the group expression is created. +type GroupExpr struct { + exprNode plannercore.LogicalPlan + children []*Group + explored bool + + selfFingerprint string +} + +// NewGroupExpr creates a GroupExpr based on a logical plan node. +func NewGroupExpr(node plannercore.LogicalPlan) *GroupExpr { + return &GroupExpr{ + exprNode: node, + children: nil, + explored: false, + } +} + +// FingerPrint gets the unique fingerprint of the group expression. +func (e *GroupExpr) FingerPrint() string { + if e.selfFingerprint == "" { + e.selfFingerprint = fmt.Sprintf("%v", e.exprNode.ID()) + for i := range e.children { + e.selfFingerprint += e.children[i].FingerPrint() + } + } + return e.selfFingerprint +} diff --git a/planner/cascades/group_expr_test.go b/planner/cascades/group_expr_test.go new file mode 100644 index 0000000000000..ca111959ee282 --- /dev/null +++ b/planner/cascades/group_expr_test.go @@ -0,0 +1,35 @@ +// Copyright 2018 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cascades + +import ( + . "github.com/pingcap/check" + plannercore "github.com/pingcap/tidb/planner/core" +) + +func (s *testCascadesSuite) TestNewGroupExpr(c *C) { + p := &plannercore.LogicalLimit{} + expr := NewGroupExpr(p) + c.Assert(expr.exprNode, Equals, p) + c.Assert(expr.children, IsNil) + c.Assert(expr.explored, IsFalse) +} + +func (s *testCascadesSuite) TestGroupExprFingerprint(c *C) { + p := &plannercore.LogicalLimit{} + expr := NewGroupExpr(p) + + // we haven't set the id of the created LogicalLimit, so the result is 0. + c.Assert(expr.FingerPrint(), Equals, "0") +} diff --git a/planner/cascades/group_test.go b/planner/cascades/group_test.go new file mode 100644 index 0000000000000..a5eab747aee27 --- /dev/null +++ b/planner/cascades/group_test.go @@ -0,0 +1,87 @@ +// Copyright 2018 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cascades + +import ( + "testing" + + . "github.com/pingcap/check" + plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/testleak" +) + +func TestT(t *testing.T) { + CustomVerboseFlag = true + TestingT(t) +} + +var _ = Suite(&testCascadesSuite{}) + +type testCascadesSuite struct { + sctx sessionctx.Context +} + +func (s *testCascadesSuite) SetUpSuite(c *C) { + testleak.BeforeTest() + s.sctx = mock.NewContext() +} + +func (s *testCascadesSuite) TearDownSuite(c *C) { + testleak.AfterTest(c)() +} + +func (s *testCascadesSuite) TestNewGroup(c *C) { + p := &plannercore.LogicalLimit{} + expr := NewGroupExpr(p) + g := NewGroup(expr) + + c.Assert(g.equivalents.Len(), Equals, 1) + c.Assert(g.equivalents.Front().Value.(*GroupExpr), Equals, expr) + c.Assert(len(g.fingerprints), Equals, 1) + c.Assert(g.explored, IsFalse) +} + +func (s *testCascadesSuite) TestGroupInsert(c *C) { + p := &plannercore.LogicalLimit{} + expr := NewGroupExpr(p) + g := NewGroup(expr) + c.Assert(g.Insert(expr), IsFalse) + expr.selfFingerprint = "1" + c.Assert(g.Insert(expr), IsTrue) +} + +func (s *testCascadesSuite) TestGroupDelete(c *C) { + p := &plannercore.LogicalLimit{} + expr := NewGroupExpr(p) + g := NewGroup(expr) + c.Assert(g.equivalents.Len(), Equals, 1) + + g.Delete(expr) + c.Assert(g.equivalents.Len(), Equals, 0) + + g.Delete(expr) + c.Assert(g.equivalents.Len(), Equals, 0) +} + +func (s *testCascadesSuite) TestGroupExists(c *C) { + p := &plannercore.LogicalLimit{} + expr := NewGroupExpr(p) + g := NewGroup(expr) + c.Assert(g.Exists(expr), IsTrue) + + g.Delete(expr) + c.Assert(g.Exists(expr), IsFalse) +}