You can not select more than 25 topics
Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
247 lines
8.7 KiB
247 lines
8.7 KiB
// Copyright 2017 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 handle_test
|
|
|
|
import (
|
|
. "github.com/pingcap/check"
|
|
"github.com/pingcap/parser/model"
|
|
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
|
"github.com/pingcap/tidb/types"
|
|
"github.com/pingcap/tidb/util/testkit"
|
|
)
|
|
|
|
func (s *testStatsSuite) TestDDLAfterLoad(c *C) {
|
|
defer cleanEnv(c, s.store, s.do)
|
|
testKit := testkit.NewTestKit(c, s.store)
|
|
testKit.MustExec("use test")
|
|
testKit.MustExec("create table t (c1 int, c2 int)")
|
|
testKit.MustExec("analyze table t")
|
|
do := s.do
|
|
is := do.InfoSchema()
|
|
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
|
|
c.Assert(err, IsNil)
|
|
tableInfo := tbl.Meta()
|
|
statsTbl := do.StatsHandle().GetTableStats(tableInfo)
|
|
c.Assert(statsTbl.Pseudo, IsFalse)
|
|
recordCount := 1000
|
|
for i := 0; i < recordCount; i++ {
|
|
testKit.MustExec("insert into t values (?, ?)", i, i+1)
|
|
}
|
|
testKit.MustExec("analyze table t")
|
|
statsTbl = do.StatsHandle().GetTableStats(tableInfo)
|
|
c.Assert(statsTbl.Pseudo, IsFalse)
|
|
// add column
|
|
testKit.MustExec("alter table t add column c10 int")
|
|
is = do.InfoSchema()
|
|
tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
|
|
c.Assert(err, IsNil)
|
|
tableInfo = tbl.Meta()
|
|
|
|
sc := new(stmtctx.StatementContext)
|
|
count := statsTbl.ColumnGreaterRowCount(sc, types.NewDatum(recordCount+1), tableInfo.Columns[0].ID)
|
|
c.Assert(count, Equals, 0.0)
|
|
count = statsTbl.ColumnGreaterRowCount(sc, types.NewDatum(recordCount+1), tableInfo.Columns[2].ID)
|
|
c.Assert(int(count), Equals, 333)
|
|
}
|
|
|
|
func (s *testStatsSuite) TestDDLTable(c *C) {
|
|
defer cleanEnv(c, s.store, s.do)
|
|
testKit := testkit.NewTestKit(c, s.store)
|
|
testKit.MustExec("use test")
|
|
testKit.MustExec("create table t (c1 int, c2 int)")
|
|
do := s.do
|
|
is := do.InfoSchema()
|
|
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
|
|
c.Assert(err, IsNil)
|
|
tableInfo := tbl.Meta()
|
|
h := do.StatsHandle()
|
|
err = h.HandleDDLEvent(<-h.DDLEventCh())
|
|
c.Assert(err, IsNil)
|
|
c.Assert(h.Update(is), IsNil)
|
|
statsTbl := h.GetTableStats(tableInfo)
|
|
c.Assert(statsTbl.Pseudo, IsFalse)
|
|
|
|
testKit.MustExec("create table t1 (c1 int, c2 int, index idx(c1))")
|
|
is = do.InfoSchema()
|
|
tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1"))
|
|
c.Assert(err, IsNil)
|
|
tableInfo = tbl.Meta()
|
|
err = h.HandleDDLEvent(<-h.DDLEventCh())
|
|
c.Assert(err, IsNil)
|
|
c.Assert(h.Update(is), IsNil)
|
|
statsTbl = h.GetTableStats(tableInfo)
|
|
c.Assert(statsTbl.Pseudo, IsFalse)
|
|
|
|
testKit.MustExec("truncate table t1")
|
|
is = do.InfoSchema()
|
|
tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1"))
|
|
c.Assert(err, IsNil)
|
|
tableInfo = tbl.Meta()
|
|
err = h.HandleDDLEvent(<-h.DDLEventCh())
|
|
c.Assert(err, IsNil)
|
|
c.Assert(h.Update(is), IsNil)
|
|
statsTbl = h.GetTableStats(tableInfo)
|
|
c.Assert(statsTbl.Pseudo, IsFalse)
|
|
}
|
|
|
|
func (s *testStatsSuite) TestDDLHistogram(c *C) {
|
|
defer cleanEnv(c, s.store, s.do)
|
|
testKit := testkit.NewTestKit(c, s.store)
|
|
do := s.do
|
|
h := do.StatsHandle()
|
|
|
|
testKit.MustExec("use test")
|
|
testKit.MustExec("create table t (c1 int, c2 int)")
|
|
<-h.DDLEventCh()
|
|
testKit.MustExec("insert into t values(1,2),(3,4)")
|
|
testKit.MustExec("analyze table t")
|
|
|
|
testKit.MustExec("alter table t add column c_null int")
|
|
err := h.HandleDDLEvent(<-h.DDLEventCh())
|
|
c.Assert(err, IsNil)
|
|
is := do.InfoSchema()
|
|
c.Assert(h.Update(is), IsNil)
|
|
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
|
|
c.Assert(err, IsNil)
|
|
tableInfo := tbl.Meta()
|
|
statsTbl := do.StatsHandle().GetTableStats(tableInfo)
|
|
c.Assert(statsTbl.Pseudo, IsFalse)
|
|
c.Check(statsTbl.Columns[tableInfo.Columns[2].ID].NullCount, Equals, int64(2))
|
|
c.Check(statsTbl.Columns[tableInfo.Columns[2].ID].NDV, Equals, int64(0))
|
|
|
|
testKit.MustExec("alter table t add column c3 int NOT NULL")
|
|
err = h.HandleDDLEvent(<-h.DDLEventCh())
|
|
c.Assert(err, IsNil)
|
|
is = do.InfoSchema()
|
|
c.Assert(h.Update(is), IsNil)
|
|
tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
|
|
c.Assert(err, IsNil)
|
|
tableInfo = tbl.Meta()
|
|
statsTbl = do.StatsHandle().GetTableStats(tableInfo)
|
|
c.Assert(statsTbl.Pseudo, IsFalse)
|
|
sc := new(stmtctx.StatementContext)
|
|
count, err := statsTbl.ColumnEqualRowCount(sc, types.NewIntDatum(0), tableInfo.Columns[3].ID)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(count, Equals, float64(2))
|
|
count, err = statsTbl.ColumnEqualRowCount(sc, types.NewIntDatum(1), tableInfo.Columns[3].ID)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(count, Equals, float64(0))
|
|
|
|
testKit.MustExec("alter table t add column c4 datetime NOT NULL default CURRENT_TIMESTAMP")
|
|
err = h.HandleDDLEvent(<-h.DDLEventCh())
|
|
c.Assert(err, IsNil)
|
|
is = do.InfoSchema()
|
|
c.Assert(h.Update(is), IsNil)
|
|
tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
|
|
c.Assert(err, IsNil)
|
|
tableInfo = tbl.Meta()
|
|
statsTbl = do.StatsHandle().GetTableStats(tableInfo)
|
|
// If we don't use original default value, we will get a pseudo table.
|
|
c.Assert(statsTbl.Pseudo, IsFalse)
|
|
|
|
testKit.MustExec("alter table t add column c5 varchar(15) DEFAULT '123'")
|
|
err = h.HandleDDLEvent(<-h.DDLEventCh())
|
|
c.Assert(err, IsNil)
|
|
is = do.InfoSchema()
|
|
c.Assert(h.Update(is), IsNil)
|
|
tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
|
|
c.Assert(err, IsNil)
|
|
tableInfo = tbl.Meta()
|
|
statsTbl = do.StatsHandle().GetTableStats(tableInfo)
|
|
c.Assert(statsTbl.Pseudo, IsFalse)
|
|
c.Check(statsTbl.Columns[tableInfo.Columns[5].ID].AvgColSize(statsTbl.Count, false), Equals, 3.0)
|
|
|
|
testKit.MustExec("create index i on t(c2, c1)")
|
|
testKit.MustExec("analyze table t")
|
|
rs := testKit.MustQuery("select count(*) from mysql.stats_histograms where table_id = ? and hist_id = 1 and is_index =1", tableInfo.ID)
|
|
rs.Check(testkit.Rows("1"))
|
|
rs = testKit.MustQuery("select count(*) from mysql.stats_buckets where table_id = ? and hist_id = 1 and is_index = 1", tableInfo.ID)
|
|
rs.Check(testkit.Rows("2"))
|
|
}
|
|
|
|
func (s *testStatsSuite) TestDDLPartition(c *C) {
|
|
defer cleanEnv(c, s.store, s.do)
|
|
testKit := testkit.NewTestKit(c, s.store)
|
|
testKit.MustExec("use test")
|
|
testKit.MustExec("drop table if exists t")
|
|
createTable := `CREATE TABLE t (a int, b int, primary key(a), index idx(b))
|
|
PARTITION BY RANGE ( a ) (
|
|
PARTITION p0 VALUES LESS THAN (6),
|
|
PARTITION p1 VALUES LESS THAN (11),
|
|
PARTITION p2 VALUES LESS THAN (16),
|
|
PARTITION p3 VALUES LESS THAN (21)
|
|
)`
|
|
testKit.MustExec(createTable)
|
|
do := s.do
|
|
is := do.InfoSchema()
|
|
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
|
|
c.Assert(err, IsNil)
|
|
tableInfo := tbl.Meta()
|
|
h := do.StatsHandle()
|
|
err = h.HandleDDLEvent(<-h.DDLEventCh())
|
|
c.Assert(err, IsNil)
|
|
c.Assert(h.Update(is), IsNil)
|
|
pi := tableInfo.GetPartitionInfo()
|
|
for _, def := range pi.Definitions {
|
|
statsTbl := h.GetPartitionStats(tableInfo, def.ID)
|
|
c.Assert(statsTbl.Pseudo, IsFalse)
|
|
}
|
|
|
|
testKit.MustExec("insert into t values (1,2),(6,2),(11,2),(16,2)")
|
|
testKit.MustExec("analyze table t")
|
|
testKit.MustExec("alter table t add column c varchar(15) DEFAULT '123'")
|
|
err = h.HandleDDLEvent(<-h.DDLEventCh())
|
|
c.Assert(err, IsNil)
|
|
is = do.InfoSchema()
|
|
c.Assert(h.Update(is), IsNil)
|
|
tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
|
|
c.Assert(err, IsNil)
|
|
tableInfo = tbl.Meta()
|
|
pi = tableInfo.GetPartitionInfo()
|
|
for _, def := range pi.Definitions {
|
|
statsTbl := h.GetPartitionStats(tableInfo, def.ID)
|
|
c.Assert(statsTbl.Pseudo, IsFalse)
|
|
c.Check(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count, false), Equals, 3.0)
|
|
}
|
|
|
|
addPartition := "alter table t add partition (partition p4 values less than (26))"
|
|
testKit.MustExec(addPartition)
|
|
is = s.do.InfoSchema()
|
|
tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
|
|
c.Assert(err, IsNil)
|
|
tableInfo = tbl.Meta()
|
|
err = h.HandleDDLEvent(<-h.DDLEventCh())
|
|
c.Assert(err, IsNil)
|
|
c.Assert(h.Update(is), IsNil)
|
|
pi = tableInfo.GetPartitionInfo()
|
|
for _, def := range pi.Definitions {
|
|
statsTbl := h.GetPartitionStats(tableInfo, def.ID)
|
|
c.Assert(statsTbl.Pseudo, IsFalse)
|
|
}
|
|
|
|
truncatePartition := "alter table t truncate partition p4"
|
|
testKit.MustExec(truncatePartition)
|
|
is = s.do.InfoSchema()
|
|
tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
|
|
c.Assert(err, IsNil)
|
|
tableInfo = tbl.Meta()
|
|
err = h.HandleDDLEvent(<-h.DDLEventCh())
|
|
c.Assert(err, IsNil)
|
|
c.Assert(h.Update(is), IsNil)
|
|
pi = tableInfo.GetPartitionInfo()
|
|
for _, def := range pi.Definitions {
|
|
statsTbl := h.GetPartitionStats(tableInfo, def.ID)
|
|
c.Assert(statsTbl.Pseudo, IsFalse)
|
|
}
|
|
}
|
|
|