Skip to content

Commit

Permalink
*: Cleanup UseNewPlanner (pingcap#1614)
Browse files Browse the repository at this point in the history
  • Loading branch information
shenli authored Aug 21, 2016
1 parent 3752e70 commit 27ae3a2
Show file tree
Hide file tree
Showing 7 changed files with 12 additions and 82 deletions.
33 changes: 0 additions & 33 deletions bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,6 @@ import (

"github.com/ngaut/log"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/plan"
)

var smallCount = 100
Expand Down Expand Up @@ -218,22 +217,6 @@ func BenchmarkJoin(b *testing.B) {
}
}

func BenchmarkNewJoin(b *testing.B) {
b.StopTimer()
se := prepareBenchSession()
prepareJoinBenchData(se, "int", "%v", smallCount)
b.StartTimer()
plan.UseNewPlanner = true
for i := 0; i < b.N; i++ {
rs, err := se.Execute("select * from t a join t b on a.col = b.col")
if err != nil {
b.Fatal(err)
}
readResult(rs[0], 100)
}
plan.UseNewPlanner = false
}

func BenchmarkJoinLimit(b *testing.B) {
b.StopTimer()
se := prepareBenchSession()
Expand All @@ -247,19 +230,3 @@ func BenchmarkJoinLimit(b *testing.B) {
readResult(rs[0], 1)
}
}

func BenchmarkNewJoinLimit(b *testing.B) {
b.StopTimer()
se := prepareBenchSession()
prepareJoinBenchData(se, "int", "%v", smallCount)
b.StartTimer()
plan.UseNewPlanner = true
for i := 0; i < b.N; i++ {
rs, err := se.Execute("select * from t a join t b on a.col = b.col limit 1")
if err != nil {
b.Fatal(err)
}
readResult(rs[0], 1)
}
plan.UseNewPlanner = false
}
2 changes: 1 addition & 1 deletion executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ type recordSet struct {
}

func (a *recordSet) Fields() ([]*ast.ResultField, error) {
if plan.UseNewPlanner && len(a.fields) == 0 {
if len(a.fields) == 0 {
for _, col := range a.schema {
rf := &ast.ResultField{
ColumnAsName: col.ColName,
Expand Down
20 changes: 6 additions & 14 deletions executor/executor_write.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/plan"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/terror"
Expand Down Expand Up @@ -305,19 +304,12 @@ func (e *DeleteExec) Next() (*Row, error) {
tblNames := make(map[string]string)
if e.IsMultiTable {
// Delete from multiple tables should consider table ident list.
if !plan.UseNewPlanner {
fs := e.SelectExec.Fields()
for _, f := range fs {
if len(f.TableAsName.L) > 0 {
tblNames[f.TableAsName.L] = f.TableName.Name.L
} else {
tblNames[f.TableName.Name.L] = f.TableName.Name.L
}
}
} else {
schema := e.SelectExec.Schema()
for _, s := range schema {
tblNames[s.TblName.L] = s.TblName.L
fs := e.SelectExec.Fields()
for _, f := range fs {
if len(f.TableAsName.L) > 0 {
tblNames[f.TableAsName.L] = f.TableName.Name.L
} else {
tblNames[f.TableName.Name.L] = f.TableName.Name.L
}
}
if len(tblNames) != 0 {
Expand Down
16 changes: 5 additions & 11 deletions interpreter/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,18 +27,16 @@ import (
"github.com/ngaut/log"
"github.com/peterh/liner"
"github.com/pingcap/tidb"
"github.com/pingcap/tidb/plan"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util/printer"
)

var (
logLevel = flag.String("L", "error", "log level")
store = flag.String("store", "goleveldb", "the name for the registered storage, e.g. memory, goleveldb, boltdb")
dbPath = flag.String("dbpath", "test", "db path")
dbName = flag.String("dbname", "test", "default db name")
lease = flag.Int("lease", 1, "schema lease seconds, very dangerous to change only if you know what you do")
useNewPlan = flag.Int("newplan", 1, "If use new planner.")
logLevel = flag.String("L", "error", "log level")
store = flag.String("store", "goleveldb", "the name for the registered storage, e.g. memory, goleveldb, boltdb")
dbPath = flag.String("dbpath", "test", "db path")
dbName = flag.String("dbname", "test", "default db name")
lease = flag.Int("lease", 1, "schema lease seconds, very dangerous to change only if you know what you do")

line *liner.State
historyPath = "/tmp/tidb_interpreter"
Expand Down Expand Up @@ -180,10 +178,6 @@ func main() {

tidb.SetSchemaLease(time.Duration(*lease) * time.Second)

if *useNewPlan == 0 {
plan.UseNewPlanner = false
}

// use test as default DB.
mdb, err := sql.Open(tidb.DriverName, *store+"://"+*dbPath+"/"+*dbName)
if err != nil {
Expand Down
3 changes: 0 additions & 3 deletions plan/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,6 @@ import (
"github.com/pingcap/tidb/util/types"
)

// UseNewPlanner means if use the new planner.
var UseNewPlanner = true

type idAllocator struct {
id int
}
Expand Down
14 changes: 0 additions & 14 deletions plan/plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -98,7 +98,6 @@ func newMockResolve(node ast.Node) error {
}

func (s *testPlanSuite) TestPredicatePushDown(c *C) {
UseNewPlanner = true
defer testleak.AfterTest(c)()
cases := []struct {
sql string
Expand Down Expand Up @@ -205,11 +204,9 @@ func (s *testPlanSuite) TestPredicatePushDown(c *C) {
c.Assert(err, IsNil)
c.Assert(ToString(p), Equals, ca.best, Commentf("for %s", ca.sql))
}
UseNewPlanner = false
}

func (s *testPlanSuite) TestJoinReOrder(c *C) {
UseNewPlanner = true
defer testleak.AfterTest(c)()
cases := []struct {
sql string
Expand Down Expand Up @@ -263,11 +260,9 @@ func (s *testPlanSuite) TestJoinReOrder(c *C) {
p = res.p.PushLimit(nil)
c.Assert(ToString(p), Equals, ca.best, Commentf("for %s", ca.sql))
}
UseNewPlanner = false
}

func (s *testPlanSuite) TestCBO(c *C) {
UseNewPlanner = true
defer testleak.AfterTest(c)()
cases := []struct {
sql string
Expand Down Expand Up @@ -353,11 +348,9 @@ func (s *testPlanSuite) TestCBO(c *C) {
p = res.p.PushLimit(nil)
c.Assert(ToString(p), Equals, ca.best, Commentf("for %s", ca.sql))
}
UseNewPlanner = false
}

func (s *testPlanSuite) TestRefine(c *C) {
UseNewPlanner = true
defer testleak.AfterTest(c)()
cases := []struct {
sql string
Expand Down Expand Up @@ -517,11 +510,9 @@ func (s *testPlanSuite) TestRefine(c *C) {
np := res.p.PushLimit(nil)
c.Assert(ToString(np), Equals, ca.best, Commentf("for %s", ca.sql))
}
UseNewPlanner = false
}

func (s *testPlanSuite) TestColumnPruning(c *C) {
UseNewPlanner = true
defer testleak.AfterTest(c)()
cases := []struct {
sql string
Expand Down Expand Up @@ -653,7 +644,6 @@ func (s *testPlanSuite) TestColumnPruning(c *C) {
c.Assert(err, IsNil)
check(p, c, ca.ans, comment)
}
UseNewPlanner = false
}

func (s *testPlanSuite) TestAllocID(c *C) {
Expand All @@ -667,7 +657,6 @@ func (s *testPlanSuite) TestAllocID(c *C) {
}

func (s *testPlanSuite) TestNewRangeBuilder(c *C) {
UseNewPlanner = true
defer testleak.AfterTest(c)()
rb := &rangeBuilder{}

Expand Down Expand Up @@ -850,7 +839,6 @@ func (s *testPlanSuite) TestNewRangeBuilder(c *C) {
got := fmt.Sprintf("%v", result)
c.Assert(got, Equals, ca.resultStr, Commentf("different for expr %s", ca.exprStr))
}
UseNewPlanner = false
}

func (s *testPlanSuite) TestTableScanWithOrder(c *C) {
Expand Down Expand Up @@ -886,7 +874,6 @@ func (s *testPlanSuite) TestTableScanWithOrder(c *C) {
}

func (s *testPlanSuite) TestConstantFolding(c *C) {
UseNewPlanner = true
defer testleak.AfterTest(c)()

cases := []struct {
Expand Down Expand Up @@ -956,7 +943,6 @@ func (s *testPlanSuite) TestConstantFolding(c *C) {

c.Assert(expression.ComposeCNFCondition(selection.Conditions).ToString(), Equals, ca.resultStr, Commentf("different for expr %s", ca.exprStr))
}
UseNewPlanner = false
}

func (s *testPlanSuite) TestCoveringIndex(c *C) {
Expand Down
6 changes: 0 additions & 6 deletions tidb-server/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@ import (
"github.com/pingcap/tidb"
"github.com/pingcap/tidb/metric"
"github.com/pingcap/tidb/perfschema"
"github.com/pingcap/tidb/plan"
"github.com/pingcap/tidb/server"
"github.com/pingcap/tidb/store/localstore/boltdb"
"github.com/pingcap/tidb/store/tikv"
Expand All @@ -46,7 +45,6 @@ var (
socket = flag.String("socket", "", "The socket file to use for connection.")
enablePS = flag.Bool("perfschema", false, "If enable performance schema.")
reportStatus = flag.Bool("report-status", true, "If enable status report HTTP service.")
useNewPlan = flag.Bool("newplan", true, "If use new planner.")
logFile = flag.String("log-file", "", "log file path")
)

Expand Down Expand Up @@ -94,10 +92,6 @@ func main() {
perfschema.EnablePerfSchema()
}

if !*useNewPlan {
plan.UseNewPlanner = false
}

// Create a session to load information schema.
se, err := tidb.CreateSession(store)
if err != nil {
Expand Down

0 comments on commit 27ae3a2

Please sign in to comment.