Skip to content

Commit

Permalink
Merge branch 'master' into time
Browse files Browse the repository at this point in the history
  • Loading branch information
alivxxx authored Sep 5, 2018
2 parents 499a11a + 15e709c commit d34ba1e
Show file tree
Hide file tree
Showing 34 changed files with 432 additions and 118 deletions.
3 changes: 1 addition & 2 deletions cmd/explaintest/r/explain_easy.result
Original file line number Diff line number Diff line change
Expand Up @@ -50,8 +50,7 @@ IndexJoin_11 4166.67 root left outer join, inner:IndexLookUp_10, outer key:test.
└─TableScan_9 10.00 cop table:t2, keep order:false, stats:pseudo
explain update t1 set t1.c2 = 2 where t1.c1 = 1;
id count task operator info
TableReader_5 1.00 root data:TableScan_4
└─TableScan_4 1.00 cop table:t1, range:[1,1], keep order:false, stats:pseudo
Point_Get_1 1.00 root table:t1, handle:1
explain delete from t1 where t1.c2 = 1;
id count task operator info
IndexLookUp_9 10.00 root
Expand Down
3 changes: 1 addition & 2 deletions cmd/explaintest/r/explain_easy_stats.result
Original file line number Diff line number Diff line change
Expand Up @@ -56,8 +56,7 @@ Projection_6 2481.25 root test.t1.c1, test.t1.c2, test.t1.c3, test.t2.c1, test.t
└─TableScan_20 1985.00 cop table:t2, keep order:false
explain update t1 set t1.c2 = 2 where t1.c1 = 1;
id count task operator info
TableReader_5 1.00 root data:TableScan_4
└─TableScan_4 1.00 cop table:t1, range:[1,1], keep order:false
Point_Get_1 1.00 root table:t1, handle:1
explain delete from t1 where t1.c2 = 1;
id count task operator info
IndexLookUp_9 0.00 root
Expand Down
31 changes: 26 additions & 5 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1695,6 +1695,22 @@ func (s *testDBSuite) TestCreateTableWithPartition(c *C) {
partition p1 values less than (to_seconds('2005-01-01')));`)
s.tk.MustQuery("show create table t26").Check(
testkit.Rows("t26 CREATE TABLE `t26` (\n `a` date DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin\nPARTITION BY RANGE ( to_seconds(`a`) ) (\n PARTITION p0 VALUES LESS THAN (63240134400),\n PARTITION p1 VALUES LESS THAN (63271756800)\n)"))
s.tk.MustExec(`create table t27 (a bigint unsigned not null)
partition by range(a) (
partition p0 values less than (10),
partition p1 values less than (100),
partition p2 values less than (1000),
partition p3 values less than (18446744073709551000),
partition p4 values less than (18446744073709551614)
);`)
s.tk.MustExec(`create table t28 (a bigint unsigned not null)
partition by range(a) (
partition p0 values less than (10),
partition p1 values less than (100),
partition p2 values less than (1000),
partition p3 values less than (18446744073709551000 + 1),
partition p4 values less than (18446744073709551000 + 10)
);`)
}

func (s *testDBSuite) TestTableDDLWithFloatType(c *C) {
Expand Down Expand Up @@ -2117,24 +2133,29 @@ func (s *testDBSuite) TestRebaseAutoID(c *C) {
s.testErrorCode(c, "alter table tidb.test2 add column b int auto_increment key, auto_increment=10;", tmysql.ErrUnknown)
}

func (s *testDBSuite) TestYearTypeCreateTable(c *C) {
func (s *testDBSuite) TestZeroFillCreateTable(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
s.tk.MustExec("use test")
s.tk.MustExec("drop table if exists abc;")
s.tk.MustExec("create table abc(y year, x int, primary key(y));")
s.tk.MustExec("create table abc(y year, z tinyint(10) zerofill, primary key(y));")
is := s.dom.InfoSchema()
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("abc"))
c.Assert(err, IsNil)
var yearCol *model.ColumnInfo
var yearCol, zCol *model.ColumnInfo
for _, col := range tbl.Meta().Columns {
if col.Name.String() == "y" {
yearCol = col
break
}
if col.Name.String() == "z" {
zCol = col
}
}
c.Assert(yearCol, NotNil)
c.Assert(yearCol.Tp, Equals, mysql.TypeYear)
c.Assert(mysql.HasUnsignedFlag(yearCol.Flag), IsFalse)
c.Assert(mysql.HasUnsignedFlag(yearCol.Flag), IsTrue)

c.Assert(zCol, NotNil)
c.Assert(mysql.HasUnsignedFlag(zCol.Flag), IsTrue)
}

func (s *testDBSuite) TestCheckColumnDefaultValue(c *C) {
Expand Down
8 changes: 7 additions & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -396,6 +396,12 @@ func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef, o
col.Flag &= ^mysql.BinaryFlag
col.Flag |= mysql.ZerofillFlag
}
// If you specify ZEROFILL for a numeric column, MySQL automatically adds the UNSIGNED attribute to the column.
// See https://dev.mysql.com/doc/refman/5.7/en/numeric-type-overview.html for more details.
// But some types like bit and year, won't show its unsigned flag in `show create table`.
if mysql.HasZerofillFlag(col.Flag) {
col.Flag |= mysql.UnsignedFlag
}
err := checkPriKeyConstraint(col, hasDefaultValue, hasNullFlag, outPriKeyConstraint)
if err != nil {
return nil, nil, errors.Trace(err)
Expand Down Expand Up @@ -906,7 +912,7 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err e
return errors.Trace(err)
}

if err = checkCreatePartitionValue(ctx, tbInfo, pi); err != nil {
if err = checkCreatePartitionValue(ctx, tbInfo, pi, cols); err != nil {
return errors.Trace(err)
}

Expand Down
8 changes: 6 additions & 2 deletions ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,21 +62,24 @@ type worker struct {
quitCh chan struct{}
wg sync.WaitGroup

reorgCtx *reorgCtx // reorgCtx is used for reorganization.
sessPool *sessionPool // sessPool is used to new sessions to execute SQL in ddl package.
reorgCtx *reorgCtx // reorgCtx is used for reorganization.
delRangeManager delRangeManager
}

func newWorker(tp workerType, store kv.Storage, ctxPool *pools.ResourcePool) *worker {
sessPool := &sessionPool{resPool: ctxPool}
worker := &worker{
id: atomic.AddInt32(&ddlWorkerID, 1),
tp: tp,
ddlJobCh: make(chan struct{}, 1),
quitCh: make(chan struct{}),
reorgCtx: &reorgCtx{notifyCancelReorgJob: 0},
sessPool: sessPool,
}

if ctxPool != nil {
worker.delRangeManager = newDelRangeManager(store, ctxPool)
worker.delRangeManager = newDelRangeManager(store, sessPool)
log.Infof("[ddl] start delRangeManager OK, with emulator: %t", !store.SupportDeleteRange())
} else {
worker.delRangeManager = newMockDelRangeManager()
Expand Down Expand Up @@ -104,6 +107,7 @@ func (w *worker) String() string {
func (w *worker) close() {
close(w.quitCh)
w.delRangeManager.clear()
w.sessPool.close()
w.wg.Wait()
log.Infof("[ddl-%s] close DDL worker", w)
}
Expand Down
21 changes: 21 additions & 0 deletions ddl/ddl_worker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/admin"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tidb/util/testleak"
"golang.org/x/net/context"
)
Expand Down Expand Up @@ -710,3 +711,23 @@ func (s *testDDLSuite) TestParallelDDL(c *C) {
tc = &TestDDLCallback{}
d.SetHook(tc)
}

func (s *testDDLSuite) TestDDLPackageExecuteSQL(c *C) {
store := testCreateStore(c, "test_run_sql")
defer store.Close()

RunWorker = true
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
testCheckOwner(c, d, true)
defer d.Stop()
worker := d.generalWorker()
c.Assert(worker, NotNil)

// In test environment, worker.ctxPool will be nil, and get will return mock.Context.
// We just test that can use it to call sqlexec.SQLExecutor.Execute.
sess, err := worker.sessPool.get()
c.Assert(err, IsNil)
defer worker.sessPool.put(sess)
se := sess.(sqlexec.SQLExecutor)
_, _ = se.Execute(context.Background(), "create table t(a int);")
}
24 changes: 8 additions & 16 deletions ddl/delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,9 @@ import (
"sync"

"github.com/juju/errors"
"github.com/ngaut/pools"
"github.com/pingcap/tidb/ddl/util"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/tablecodec"
Expand All @@ -51,7 +49,7 @@ type delRangeManager interface {

type delRange struct {
store kv.Storage
ctxPool *pools.ResourcePool
sessPool *sessionPool
storeSupport bool
emulatorCh chan struct{}
keys []kv.Key
Expand All @@ -61,10 +59,10 @@ type delRange struct {
}

// newDelRangeManager returns a delRangeManager.
func newDelRangeManager(store kv.Storage, ctxPool *pools.ResourcePool) delRangeManager {
func newDelRangeManager(store kv.Storage, sessPool *sessionPool) delRangeManager {
dr := &delRange{
store: store,
ctxPool: ctxPool,
sessPool: sessPool,
storeSupport: store.SupportDeleteRange(),
quitCh: make(chan struct{}),
}
Expand All @@ -77,14 +75,11 @@ func newDelRangeManager(store kv.Storage, ctxPool *pools.ResourcePool) delRangeM

// addDelRangeJob implements delRangeManager interface.
func (dr *delRange) addDelRangeJob(job *model.Job) error {
resource, err := dr.ctxPool.Get()
ctx, err := dr.sessPool.get()
if err != nil {
return errors.Trace(err)
}
defer dr.ctxPool.Put(resource)
ctx := resource.(sessionctx.Context)
ctx.GetSessionVars().SetStatusFlag(mysql.ServerStatusAutocommit, true)
ctx.GetSessionVars().InRestrictedSQL = true
defer dr.sessPool.put(ctx)

err = insertJobIntoDeleteRangeTable(ctx, job)
if err != nil {
Expand All @@ -110,7 +105,7 @@ func (dr *delRange) clear() {
log.Infof("[ddl] closing delRange session pool")
close(dr.quitCh)
dr.wait.Wait()
dr.ctxPool.Close()
dr.sessPool.close()
}

// startEmulator is only used for those storage engines which don't support
Expand All @@ -131,15 +126,12 @@ func (dr *delRange) startEmulator() {
}

func (dr *delRange) doDelRangeWork() error {
resource, err := dr.ctxPool.Get()
ctx, err := dr.sessPool.get()
if err != nil {
log.Errorf("[ddl] delRange emulator get session fail: %s", err)
return errors.Trace(err)
}
defer dr.ctxPool.Put(resource)
ctx := resource.(sessionctx.Context)
ctx.GetSessionVars().SetStatusFlag(mysql.ServerStatusAutocommit, true)
ctx.GetSessionVars().InRestrictedSQL = true
defer dr.sessPool.put(ctx)

ranges, err := util.LoadDeleteRanges(ctx, math.MaxInt64)
if err != nil {
Expand Down
67 changes: 48 additions & 19 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ func checkPartitionFuncType(ctx sessionctx.Context, s *ast.CreateTableStmt, cols

// checkCreatePartitionValue checks whether `less than value` is strictly increasing for each partition.
// Side effect: it may simplify the partition range definition from a constant expression to an integer.
func checkCreatePartitionValue(ctx sessionctx.Context, tblInfo *model.TableInfo, pi *model.PartitionInfo) error {
func checkCreatePartitionValue(ctx sessionctx.Context, tblInfo *model.TableInfo, pi *model.PartitionInfo, cols []*table.Column) error {
defs := pi.Definitions
if len(defs) <= 1 {
return nil
Expand All @@ -187,13 +187,14 @@ func checkCreatePartitionValue(ctx sessionctx.Context, tblInfo *model.TableInfo,
if strings.EqualFold(defs[len(defs)-1].LessThan[0], partitionMaxValue) {
defs = defs[:len(defs)-1]
}
var prevRangeValue int64
isUnsignedBigint := isRangePartitionColUnsignedBigint(cols, pi)
var prevRangeValue interface{}
for i := 0; i < len(defs); i++ {
if strings.EqualFold(defs[i].LessThan[0], partitionMaxValue) {
return errors.Trace(ErrPartitionMaxvalue)
}

currentRangeValue, fromExpr, err := getRangeValue(ctx, tblInfo, defs[i].LessThan[0])
currentRangeValue, fromExpr, err := getRangeValue(ctx, tblInfo, defs[i].LessThan[0], isUnsignedBigint)
if err != nil {
return errors.Trace(err)
}
Expand All @@ -207,8 +208,14 @@ func checkCreatePartitionValue(ctx sessionctx.Context, tblInfo *model.TableInfo,
continue
}

if currentRangeValue <= prevRangeValue {
return errors.Trace(ErrRangeNotIncreasing)
if isUnsignedBigint {
if currentRangeValue.(uint64) <= prevRangeValue.(uint64) {
return errors.Trace(ErrRangeNotIncreasing)
}
} else {
if currentRangeValue.(int64) <= prevRangeValue.(int64) {
return errors.Trace(ErrRangeNotIncreasing)
}
}
prevRangeValue = currentRangeValue
}
Expand All @@ -217,23 +224,34 @@ func checkCreatePartitionValue(ctx sessionctx.Context, tblInfo *model.TableInfo,

// getRangeValue gets an integer from the range value string.
// The returned boolean value indicates whether the input string is a constant expression.
func getRangeValue(ctx sessionctx.Context, tblInfo *model.TableInfo, str string) (int64, bool, error) {

if value, err := strconv.ParseInt(str, 10, 64); err == nil {
return value, false, nil
}
func getRangeValue(ctx sessionctx.Context, tblInfo *model.TableInfo, str string, unsignedBigint bool) (interface{}, bool, error) {
// Unsigned bigint was converted to uint64 handle.
if unsignedBigint {
if value, err := strconv.ParseUint(str, 10, 64); err == nil {
return value, false, nil
}

// The range value maybe not an integer, it could be a constant expression.
// For example, the following two cases are the same:
// PARTITION p0 VALUES LESS THAN (TO_SECONDS('2004-01-01'))
// PARTITION p0 VALUES LESS THAN (63340531200)
if e, err1 := expression.ParseSimpleExprWithTableInfo(ctx, str, tblInfo); err1 == nil {
res, isNull, err2 := e.EvalInt(ctx, chunk.Row{})
if err2 == nil && isNull == false {
return res, true, nil
if e, err1 := expression.ParseSimpleExprWithTableInfo(ctx, str, tblInfo); err1 == nil {
res, isNull, err2 := e.EvalInt(ctx, chunk.Row{})
if err2 == nil && isNull == false {
return uint64(res), true, nil
}
}
} else {
if value, err := strconv.ParseInt(str, 10, 64); err == nil {
return value, false, nil
}
// The range value maybe not an integer, it could be a constant expression.
// For example, the following two cases are the same:
// PARTITION p0 VALUES LESS THAN (TO_SECONDS('2004-01-01'))
// PARTITION p0 VALUES LESS THAN (63340531200)
if e, err1 := expression.ParseSimpleExprWithTableInfo(ctx, str, tblInfo); err1 == nil {
res, isNull, err2 := e.EvalInt(ctx, chunk.Row{})
if err2 == nil && isNull == false {
return res, true, nil
}
}
}

return 0, false, ErrNotAllowedTypeInPartition.GenByArgs(str)
}

Expand Down Expand Up @@ -388,3 +406,14 @@ func checkConstraintIncludePartKey(partkeys []string, constraints map[string]str
}
return true
}

// isRangePartitionColUnsignedBigint returns true if the partitioning key column type is unsigned bigint type.
func isRangePartitionColUnsignedBigint(cols []*table.Column, pi *model.PartitionInfo) bool {
for _, col := range cols {
isUnsigned := col.Tp == mysql.TypeLonglong && mysql.HasUnsignedFlag(col.Flag)
if isUnsigned && strings.Contains(strings.ToLower(pi.Expr), col.Name.L) {
return true
}
}
return false
}
Loading

0 comments on commit d34ba1e

Please sign in to comment.