Skip to content

Commit

Permalink
planner: support the Group and GroupExpr for the cascades planner (#7917
Browse files Browse the repository at this point in the history
)
  • Loading branch information
zz-jason authored and XuHuaiyu committed Oct 18, 2018
1 parent cd1e2d1 commit 286a79b
Show file tree
Hide file tree
Showing 4 changed files with 247 additions and 0 deletions.
72 changes: 72 additions & 0 deletions planner/cascades/group.go
Original file line number Diff line number Diff line change
@@ -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
}
53 changes: 53 additions & 0 deletions planner/cascades/group_expr.go
Original file line number Diff line number Diff line change
@@ -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
}
35 changes: 35 additions & 0 deletions planner/cascades/group_expr_test.go
Original file line number Diff line number Diff line change
@@ -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")
}
87 changes: 87 additions & 0 deletions planner/cascades/group_test.go
Original file line number Diff line number Diff line change
@@ -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)
}

0 comments on commit 286a79b

Please sign in to comment.