Skip to content

Commit

Permalink
Merge branch 'release-6.1' into cherry-pick-38802-to-release-6.1
Browse files Browse the repository at this point in the history
  • Loading branch information
winoros authored Nov 16, 2022
2 parents 37679ef + b2c62c6 commit 6c98a6f
Show file tree
Hide file tree
Showing 13 changed files with 168 additions and 69 deletions.
42 changes: 20 additions & 22 deletions executor/join.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"fmt"
"runtime/trace"
"strconv"
"sync"
"sync/atomic"
"time"

Expand Down Expand Up @@ -67,6 +66,8 @@ type HashJoinExec struct {

// closeCh add a lock for closing executor.
closeCh chan struct{}
worker util.WaitGroupWrapper
waiter util.WaitGroupWrapper
joinType plannercore.JoinType
requiredRows int64

Expand All @@ -89,9 +90,7 @@ type HashJoinExec struct {
prepared bool
isOuterJoin bool

// joinWorkerWaitGroup is for sync multiple join workers.
joinWorkerWaitGroup sync.WaitGroup
finished atomic.Value
finished atomic.Value

stats *hashJoinRuntimeStats
}
Expand Down Expand Up @@ -146,6 +145,7 @@ func (e *HashJoinExec) Close() error {
e.probeChkResourceCh = nil
e.joinChkResourceCh = nil
terror.Call(e.rowContainer.Close)
e.waiter.Wait()
}
e.outerMatchedStatus = e.outerMatchedStatus[:0]

Expand All @@ -159,6 +159,8 @@ func (e *HashJoinExec) Close() error {
// Open implements the Executor Open interface.
func (e *HashJoinExec) Open(ctx context.Context) error {
if err := e.baseExecutor.Open(ctx); err != nil {
e.closeCh = nil
e.prepared = false
return err
}
e.prepared = false
Expand All @@ -168,9 +170,10 @@ func (e *HashJoinExec) Open(ctx context.Context) error {
e.diskTracker = disk.NewTracker(e.id, -1)
e.diskTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.DiskTracker)

e.worker = util.WaitGroupWrapper{}
e.waiter = util.WaitGroupWrapper{}
e.closeCh = make(chan struct{})
e.finished.Store(false)
e.joinWorkerWaitGroup = sync.WaitGroup{}

if e.probeTypes == nil {
e.probeTypes = retTypes(e.probeSideExec)
Expand Down Expand Up @@ -264,13 +267,13 @@ func (e *HashJoinExec) wait4BuildSide() (emptyBuild bool, err error) {

// fetchBuildSideRows fetches all rows from build side executor, and append them
// to e.buildSideResult.
func (e *HashJoinExec) fetchBuildSideRows(ctx context.Context, chkCh chan<- *chunk.Chunk, doneCh <-chan struct{}) {
func (e *HashJoinExec) fetchBuildSideRows(ctx context.Context, chkCh chan<- *chunk.Chunk, errCh chan<- error, doneCh <-chan struct{}) {
defer close(chkCh)
var err error
failpoint.Inject("issue30289", func(val failpoint.Value) {
if val.(bool) {
err = errors.Errorf("issue30289 build return error")
e.buildFinished <- errors.Trace(err)
errCh <- errors.Trace(err)
return
}
})
Expand All @@ -281,7 +284,7 @@ func (e *HashJoinExec) fetchBuildSideRows(ctx context.Context, chkCh chan<- *chu
chk := chunk.NewChunkWithCapacity(e.buildSideExec.base().retFieldTypes, e.ctx.GetSessionVars().MaxChunkSize)
err = Next(ctx, e.buildSideExec, chk)
if err != nil {
e.buildFinished <- errors.Trace(err)
errCh <- errors.Trace(err)
return
}
failpoint.Inject("errorFetchBuildSideRowsMockOOMPanic", nil)
Expand Down Expand Up @@ -332,8 +335,7 @@ func (e *HashJoinExec) initializeForProbe() {

func (e *HashJoinExec) fetchAndProbeHashTable(ctx context.Context) {
e.initializeForProbe()
e.joinWorkerWaitGroup.Add(1)
go util.WithRecovery(func() {
e.worker.RunWithRecover(func() {
defer trace.StartRegion(ctx, "HashJoinProbeSideFetcher").End()
e.fetchProbeSideChunks(ctx)
}, e.handleProbeSideFetcherPanic)
Expand All @@ -344,14 +346,13 @@ func (e *HashJoinExec) fetchAndProbeHashTable(ctx context.Context) {
}

for i := uint(0); i < e.concurrency; i++ {
e.joinWorkerWaitGroup.Add(1)
workID := i
go util.WithRecovery(func() {
e.worker.RunWithRecover(func() {
defer trace.StartRegion(ctx, "HashJoinWorker").End()
e.runJoinWorker(workID, probeKeyColIdx)
}, e.handleJoinWorkerPanic)
}
go util.WithRecovery(e.waitJoinWorkersAndCloseResultChan, nil)
e.waiter.RunWithRecover(e.waitJoinWorkersAndCloseResultChan, nil)
}

func (e *HashJoinExec) handleProbeSideFetcherPanic(r interface{}) {
Expand All @@ -361,14 +362,12 @@ func (e *HashJoinExec) handleProbeSideFetcherPanic(r interface{}) {
if r != nil {
e.joinResultCh <- &hashjoinWorkerResult{err: errors.Errorf("%v", r)}
}
e.joinWorkerWaitGroup.Done()
}

func (e *HashJoinExec) handleJoinWorkerPanic(r interface{}) {
if r != nil {
e.joinResultCh <- &hashjoinWorkerResult{err: errors.Errorf("%v", r)}
}
e.joinWorkerWaitGroup.Done()
}

// Concurrently handling unmatched rows from the hash table
Expand Down Expand Up @@ -408,15 +407,14 @@ func (e *HashJoinExec) handleUnmatchedRowsFromHashTable(workerID uint) {
}

func (e *HashJoinExec) waitJoinWorkersAndCloseResultChan() {
e.joinWorkerWaitGroup.Wait()
e.worker.Wait()
if e.useOuterToBuild {
// Concurrently handling unmatched rows from the hash table at the tail
for i := uint(0); i < e.concurrency; i++ {
var workerID = i
e.joinWorkerWaitGroup.Add(1)
go util.WithRecovery(func() { e.handleUnmatchedRowsFromHashTable(workerID) }, e.handleJoinWorkerPanic)
e.worker.RunWithRecover(func() { e.handleUnmatchedRowsFromHashTable(workerID) }, e.handleJoinWorkerPanic)
}
e.joinWorkerWaitGroup.Wait()
e.worker.Wait()
}
close(e.joinResultCh)
}
Expand Down Expand Up @@ -682,7 +680,7 @@ func (e *HashJoinExec) Next(ctx context.Context, req *chunk.Chunk) (err error) {
e.rowContainerForProbe[i] = e.rowContainer.ShallowCopy()
}
}
go util.WithRecovery(func() {
e.worker.RunWithRecover(func() {
defer trace.StartRegion(ctx, "HashJoinHashTableBuilder").End()
e.fetchAndBuildHashTable(ctx)
}, e.handleFetchAndBuildHashTablePanic)
Expand Down Expand Up @@ -725,10 +723,10 @@ func (e *HashJoinExec) fetchAndBuildHashTable(ctx context.Context) {
buildSideResultCh := make(chan *chunk.Chunk, 1)
doneCh := make(chan struct{})
fetchBuildSideRowsOk := make(chan error, 1)
go util.WithRecovery(
e.worker.RunWithRecover(
func() {
defer trace.StartRegion(ctx, "HashJoinBuildSideFetcher").End()
e.fetchBuildSideRows(ctx, buildSideResultCh, doneCh)
e.fetchBuildSideRows(ctx, buildSideResultCh, fetchBuildSideRowsOk, doneCh)
},
func(r interface{}) {
if r != nil {
Expand Down
83 changes: 83 additions & 0 deletions executor/join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2827,6 +2827,89 @@ func TestIssue31129(t *testing.T) {
require.NoError(t, failpoint.Disable(fpName2))
}

func TestIssue37932(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk1 := testkit.NewTestKit(t, store)
tk2 := testkit.NewTestKit(t, store)
tk1.MustExec("use test")
tk2.MustExec("use test")
tk1.MustExec("create table tbl_1 ( col_1 set ( 'Alice','Bob','Charlie','David' ) not null default 'Alice' ,col_2 tinyint unsigned ,col_3 decimal ( 34 , 3 ) not null default 79 ,col_4 bigint unsigned not null ,col_5 bit ( 12 ) not null , unique key idx_1 ( col_2 ) ,unique key idx_2 ( col_2 ) ) charset utf8mb4 collate utf8mb4_bin ;")
tk1.MustExec("create table tbl_2 ( col_6 text ( 52 ) collate utf8_unicode_ci not null ,col_7 int unsigned not null ,col_8 blob ( 369 ) ,col_9 bit ( 51 ) ,col_10 decimal ( 38 , 16 ) , unique key idx_3 ( col_7 ) ,unique key idx_4 ( col_7 ) ) charset utf8 collate utf8_unicode_ci ;")
tk1.MustExec("create table tbl_3 ( col_11 set ( 'Alice','Bob','Charlie','David' ) not null ,col_12 bigint unsigned not null default 1678891638492596595 ,col_13 text ( 18 ) ,col_14 set ( 'Alice','Bob','Charlie','David' ) not null default 'Alice' ,col_15 mediumint , key idx_5 ( col_12 ) ,unique key idx_6 ( col_12 ) ) charset utf8mb4 collate utf8mb4_general_ci ;")
tk1.MustExec("create table tbl_4 ( col_16 set ( 'Alice','Bob','Charlie','David' ) not null ,col_17 tinyint unsigned ,col_18 int unsigned not null default 4279145838 ,col_19 varbinary ( 210 ) not null ,col_20 timestamp , primary key ( col_18 ) /*T![clustered_index] nonclustered */ ,key idx_8 ( col_19 ) ) charset utf8mb4 collate utf8mb4_unicode_ci ;")
tk1.MustExec("create table tbl_5 ( col_21 bigint ,col_22 set ( 'Alice','Bob','Charlie','David' ) ,col_23 blob ( 311 ) ,col_24 bigint unsigned not null default 3415443099312152509 ,col_25 time , unique key idx_9 ( col_21 ) ,unique key idx_10 ( col_21 ) ) charset gbk collate gbk_bin ;")
tk1.MustExec("insert into tbl_1 values ( 'Bob',null,0.04,2650749963804575036,4044 );")
tk1.MustExec("insert into tbl_1 values ( 'Alice',171,1838.2,6452757231340518222,1190 );")
tk1.MustExec("insert into tbl_1 values ( 'Bob',202,2.962,4304284252076747481,2112 );")
tk1.MustExec("insert into tbl_1 values ( 'David',155,32610.05,5899651588546531414,104 );")
tk1.MustExec("insert into tbl_1 values ( 'Charlie',52,4219.7,6151233689319516187,1246 );")
tk1.MustExec("insert into tbl_1 values ( 'Bob',55,3963.11,3614977408465893392,1188 );")
tk1.MustExec("insert into tbl_1 values ( 'Alice',203,72.01,1553550133494908281,1658 );")
tk1.MustExec("insert into tbl_1 values ( 'Bob',40,871.569,8114062926218465773,1397 );")
tk1.MustExec("insert into tbl_1 values ( 'Alice',165,7765,4481202107781982005,2089 );")
tk1.MustExec("insert into tbl_1 values ( 'David',79,7.02,993594504887208796,514 );")
tk1.MustExec("insert into tbl_2 values ( 'iB_%7c&q!6-gY4bkvg',2064909882,'dLN52t1YZSdJ',2251679806445488,32 );")
tk1.MustExec("insert into tbl_2 values ( 'h_',1478443689,'EqP+iN=',180492371752598,0.1 );")
tk1.MustExec("insert into tbl_2 values ( 'U@U&*WKfPzil=6YaDxp',4271201457,'QWuo24qkSSo',823931105457505,88514 );")
tk1.MustExec("insert into tbl_2 values ( 'FR4GA=',505128825,'RpEmV6ph5Z7',568030123046798,609381 );")
tk1.MustExec("insert into tbl_2 values ( '3GsU',166660047,'',1061132816887762,6.4605 );")
tk1.MustExec("insert into tbl_2 values ( 'BA4hPRD0lm*pbg#NE',3440634757,'7gUPe2',288001159469205,6664.9 );")
tk1.MustExec("insert into tbl_2 values ( '+z',2117152318,'WTkD(N',215697667226264,7.88 );")
tk1.MustExec("insert into tbl_2 values ( 'x@SPhy9lOomPa4LF',2881759652,'ETUXQQ0b4HnBSKgTWIU',153379720424625,null );")
tk1.MustExec("insert into tbl_2 values ( '',2075177391,'MPae!9%ufd',115899580476733,341.23 );")
tk1.MustExec("insert into tbl_2 values ( '~udi',1839363347,'iQj$$YsZc5ULTxG)yH',111454353417190,6.6 );")
tk1.MustExec("insert into tbl_3 values ( 'Alice',7032411265967085555,'P7*KBZ159','Alice',7516989 );")
tk1.MustExec("insert into tbl_3 values ( 'David',486417871670147038,'','Charlie',-2135446 );")
tk1.MustExec("insert into tbl_3 values ( 'Charlie',5784081664185069254,'7V_&YzKM~Q','Charlie',5583839 );")
tk1.MustExec("insert into tbl_3 values ( 'David',6346366522897598558,')Lp&$2)SC@','Bob',2522913 );")
tk1.MustExec("insert into tbl_3 values ( 'Charlie',224922711063053272,'gY','David',6624398 );")
tk1.MustExec("insert into tbl_3 values ( 'Alice',4678579167560495958,'fPIXY%R8WyY(=u&O','David',-3267160 );")
tk1.MustExec("insert into tbl_3 values ( 'David',8817108026311573677,'Cs0dZW*SPnKhV1','Alice',2359718 );")
tk1.MustExec("insert into tbl_3 values ( 'Bob',3177426155683033662,'o2=@zv2qQDhKUs)4y','Bob',-8091802 );")
tk1.MustExec("insert into tbl_3 values ( 'Bob',2543586640437235142,'hDa*CsOUzxmjf2m','Charlie',-8091935 );")
tk1.MustExec("insert into tbl_3 values ( 'Charlie',6204182067887668945,'DX-!=)dbGPQO','David',-1954600 );")
tk1.MustExec("insert into tbl_4 values ( 'David',167,576262750,'lX&x04W','2035-09-28' );")
tk1.MustExec("insert into tbl_4 values ( 'Charlie',236,2637776757,'92OhsL!w%7','2036-02-08' );")
tk1.MustExec("insert into tbl_4 values ( 'Bob',68,1077999933,'M0l','1997-09-16' );")
tk1.MustExec("insert into tbl_4 values ( 'Charlie',184,1280264753,'FhjkfeXsK1Q(','2030-03-16' );")
tk1.MustExec("insert into tbl_4 values ( 'Alice',10,2150711295,'Eqip)^tr*MoL','2032-07-02' );")
tk1.MustExec("insert into tbl_4 values ( 'Bob',108,2421602476,'Eul~~Df_Q8s&I3Y-7','2019-06-10' );")
tk1.MustExec("insert into tbl_4 values ( 'Alice',36,2811198561,'%XgRou0#iKtn*','2022-06-13' );")
tk1.MustExec("insert into tbl_4 values ( 'Charlie',115,330972286,'hKeJS','2000-11-15' );")
tk1.MustExec("insert into tbl_4 values ( 'Alice',6,2958326555,'c6+=1','2001-02-11' );")
tk1.MustExec("insert into tbl_4 values ( 'Alice',99,387404826,'figc(@9R*k3!QM_Vve','2036-02-17' );")
tk1.MustExec("insert into tbl_5 values ( -401358236474313609,'Charlie','4J$',701059766304691317,'08:19:10.00' );")
tk1.MustExec("insert into tbl_5 values ( 2759837898825557143,'Bob','E',5158554038674310466,'11:04:03.00' );")
tk1.MustExec("insert into tbl_5 values ( 273910054423832204,'Alice',null,8944547065167499612,'08:02:30.00' );")
tk1.MustExec("insert into tbl_5 values ( 2875669873527090798,'Alice','4^SpR84',4072881341903432150,'18:24:55.00' );")
tk1.MustExec("insert into tbl_5 values ( -8446590100588981557,'David','yBj8',8760380566452862549,'09:01:10.00' );")
tk1.MustExec("insert into tbl_5 values ( -1075861460175889441,'Charlie','ti11Pl0lJ',9139997565676405627,'08:30:14.00' );")
tk1.MustExec("insert into tbl_5 values ( 95663565223131772,'Alice','6$',8467839300407531400,'23:31:42.00' );")
tk1.MustExec("insert into tbl_5 values ( -5661709703968335255,'Charlie','',8122758569495329946,'19:36:24.00' );")
tk1.MustExec("insert into tbl_5 values ( 3338588216091909518,'Bob','',6558557574025196860,'15:22:56.00' );")
tk1.MustExec("insert into tbl_5 values ( 8918630521194612922,'David','I$w',5981981639362947650,'22:03:24.00' );")
tk1.MustExec("begin pessimistic;")
tk1.MustExec("insert ignore into tbl_1 set col_1 = 'David', col_2 = 110, col_3 = 37065, col_4 = 8164500960513474805, col_5 = 1264 on duplicate key update col_3 = 22151.5, col_4 = 6266058887081523571, col_5 = 3254, col_2 = 59, col_1 = 'Bob';")
tk1.MustExec("insert into tbl_4 (col_16,col_17,col_18,col_19,col_20) values ( 'Charlie',34,2499970462,'Z','1978-10-27' ) ,( 'David',217,1732485689,'*)~@@Q8ryi','2004-12-01' ) ,( 'Charlie',40,1360558255,'H(Y','1998-06-25' ) ,( 'Alice',108,2973455447,'%CcP4$','1979-03-28' ) ,( 'David',9,3835209932,'tdKXUzLmAzwFf$','2009-03-03' ) ,( 'David',68,163270003,'uimsclz@FQJN','1988-09-11' ) ,( 'Alice',76,297067264,'BzFF','1989-01-05' ) on duplicate key update col_16 = 'Charlie', col_17 = 14, col_18 = 4062155275, col_20 = '2002-03-07', col_19 = 'tmvchLzp*o8';")
tk2.MustExec("delete from tbl_3 where tbl_3.col_13 in ( null ,'' ,'g8EEzUU7LQ' ,'~fC3&B*cnOOx_' ,'%RF~AFto&x' ,'NlWkMWG^00' ,'e^4o2Ji^q_*Fa52Z' ) ;")
tk2.MustExec("delete from tbl_5 where not( tbl_5.col_21 between -1075861460175889441 and 3338588216091909518 ) ;")
tk1.MustExec("replace into tbl_1 (col_1,col_2,col_3,col_4,col_5) values ( 'Alice',83,8.33,4070808626051569664,455 ) ,( 'Alice',53,2.8,2763362085715461014,1912 ) ,( 'David',178,4242.8,962727993466011464,1844 ) ,( 'Alice',16,650054,5638988670318229867,565 ) ,( 'Alice',76,89783.1,3968605744540056024,2563 ) ,( 'Bob',120,0.89,1003144931151245839,2670 );")
tk1.MustExec("delete from tbl_5 where col_24 is null ;")
tk1.MustExec("delete from tbl_3 where tbl_3.col_11 in ( 'Alice' ,'Bob' ,'Alice' ) ;")
tk2.MustExec("insert into tbl_3 set col_11 = 'Bob', col_12 = 5701982550256146475, col_13 = 'Hhl)yCsQ2K3cfc^', col_14 = 'Alice', col_15 = -3718868 on duplicate key update col_15 = 7210750, col_12 = 6133680876296985245, col_14 = 'Alice', col_11 = 'David', col_13 = 'F+RMGE!_2^Cfr3Fw';")
tk2.MustExec("insert ignore into tbl_5 set col_21 = 2439343116426563397, col_22 = 'Charlie', col_23 = '~Spa2YzRFFom16XD', col_24 = 5571575017340582365, col_25 = '13:24:38.00' ;")
err := tk1.ExecToErr("update tbl_4 set tbl_4.col_20 = '2006-01-24' where tbl_4.col_18 in ( select col_11 from tbl_3 where IsNull( tbl_4.col_16 ) or not( tbl_4.col_19 in ( select col_3 from tbl_1 where tbl_4.col_16 between 'Alice' and 'David' and tbl_4.col_19 <= '%XgRou0#iKtn*' ) ) ) ;")
if err != nil {
print(err.Error())
if strings.Contains(err.Error(), "Truncated incorrect DOUBLE value") {
t.Log("Bug of truncated incorrect DOUBLE value has not been fixed, skipping")
return
}
}
require.NoError(t, err)
}

func TestOuterJoin(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
Expand Down
6 changes: 5 additions & 1 deletion expression/builtin_control.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,11 @@ func InferType4ControlFuncs(ctx sessionctx.Context, funcName string, lexp, rexp
}
flen := maxlen(lhsFlen, rhsFlen) + resultFieldType.GetDecimal() + 1 // account for -1 len fields
resultFieldType.SetFlen(mathutil.Min(flen, mysql.MaxDecimalWidth)) // make sure it doesn't overflow

} else if evalType == types.ETString {
lhsLen, rhsLen := lhs.GetFlen(), rhs.GetFlen()
if lhsLen != types.UnspecifiedLength && rhsLen != types.UnspecifiedLength {
resultFieldType.SetFlen(mathutil.Max(lhsLen, rhsLen))
}
} else {
resultFieldType.SetFlen(maxlen(lhs.GetFlen(), rhs.GetFlen()))
}
Expand Down
12 changes: 12 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7413,3 +7413,15 @@ func TestIssue38736(t *testing.T) {
// Also the filter is evaled as false.
tk.MustQuery("SELECT v0.c0 FROM v0 WHERE (v0.c0)NOT LIKE(BINARY v0.c0) or v0.c0 > 0").Check(testkit.Rows())
}

func TestIfNullParamMarker(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("create table t (c1 varchar(100), c2 varchar(128));")
tk.MustExec(`prepare pr1 from "insert into t values(ifnull(?,' '),ifnull(?,' '))";`)
tk.MustExec(`set @a='1',@b=repeat('x', 80);`)
// Should not report 'Data too long for column' error.
tk.MustExec(`execute pr1 using @a,@b;`)
}
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ require (
github.com/pingcap/failpoint v0.0.0-20220423142525-ae43b7f4e5c3
github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059
github.com/pingcap/kvproto v0.0.0-20220517085838-12e2f5a9d167
github.com/pingcap/log v1.1.0
github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c
github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4
github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e
github.com/pingcap/tipb v0.0.0-20220314125451-bfb5c2c55188
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -643,8 +643,8 @@ github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIf
github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8=
github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM=
github.com/pingcap/log v0.0.0-20211215031037-e024ba4eb0ee/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4=
github.com/pingcap/log v1.1.0 h1:ELiPxACz7vdo1qAvvaWJg1NrYFoY6gqAh/+Uo6aXdD8=
github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4=
github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c h1:crhkw6DD+07Bg1wYhW5Piw+kYNKZqFQqfC2puUf6gMI=
github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4=
github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 h1:HYbcxtnkN3s5tqrZ/z3eJS4j3Db8wMphEm1q10lY/TM=
github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4/go.mod h1:sDCsM39cGiv2vwunZkaFA917vVkqDTGSPbbV7z4Oops=
github.com/pingcap/tipb v0.0.0-20220314125451-bfb5c2c55188 h1:+46isFI9fR9R+nJVDMI55tCC/TCwp+bvVA4HLGEv1rY=
Expand Down
12 changes: 12 additions & 0 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6686,3 +6686,15 @@ func TestExplainAnalyzeDMLCommit(t *testing.T) {
require.NoError(t, err)
tk.MustQuery("select * from t").Check(testkit.Rows())
}

func TestAutoIncrementCheckWithCheckConstraint(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec(`CREATE TABLE t (
id INTEGER NOT NULL AUTO_INCREMENT,
CHECK (id IN (0, 1)),
KEY idx_autoinc_id (id)
)`)
}
4 changes: 4 additions & 0 deletions planner/core/preprocess.go
Original file line number Diff line number Diff line change
Expand Up @@ -629,6 +629,10 @@ func checkAutoIncrementOp(colDef *ast.ColumnDef, index int) (bool, error) {

func isConstraintKeyTp(constraints []*ast.Constraint, colDef *ast.ColumnDef) bool {
for _, c := range constraints {
// ignore constraint check
if c.Tp == ast.ConstraintCheck {
continue
}
if c.Keys[0].Expr != nil {
continue
}
Expand Down
2 changes: 1 addition & 1 deletion session/advisory_locks.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ func (a *advisoryLock) DecrReferences() {
a.referenceCount--
}

// References returns the current reference count for the advisory lock.
// ReferenceCount returns the current reference count for the advisory lock.
func (a *advisoryLock) ReferenceCount() int {
return a.referenceCount
}
Expand Down
Loading

0 comments on commit 6c98a6f

Please sign in to comment.