Skip to content

Commit

Permalink
Merge branch 'master' of https://github.com/pingcap/tidb into issue-2…
Browse files Browse the repository at this point in the history
…3623_refine_arg_bug

Signed-off-by: guo-shaoge <shaoge1994@163.com>
  • Loading branch information
guo-shaoge committed Mar 29, 2021
2 parents f2154b3 + c32895f commit a2d8a15
Show file tree
Hide file tree
Showing 10 changed files with 65 additions and 10 deletions.
6 changes: 5 additions & 1 deletion executor/aggfuncs/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,9 +155,13 @@ func buildApproxPercentile(sctx sessionctx.Context, aggFuncDesc *aggregation.Agg

base := basePercentile{percent: int(percent), baseAggFunc: baseAggFunc{args: aggFuncDesc.Args, ordinal: ordinal}}

evalType := aggFuncDesc.Args[0].GetType().EvalType()
if aggFuncDesc.Args[0].GetType().Tp == mysql.TypeBit {
evalType = types.ETString // same as other aggregate function
}
switch aggFuncDesc.Mode {
case aggregation.CompleteMode, aggregation.Partial1Mode, aggregation.FinalMode:
switch aggFuncDesc.Args[0].GetType().EvalType() {
switch evalType {
case types.ETInt:
return &percentileOriginal4Int{base}
case types.ETReal:
Expand Down
2 changes: 1 addition & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -3863,7 +3863,7 @@ func NewRowDecoder(ctx sessionctx.Context, schema *expression.Schema, tbl *model
if len(pkCols) == 0 {
pkCols = tables.TryGetCommonPkColumnIds(tbl)
if len(pkCols) == 0 {
pkCols = []int64{0}
pkCols = []int64{-1}
}
}
defVal := func(i int, chk *chunk.Chunk) error {
Expand Down
4 changes: 2 additions & 2 deletions executor/index_lookup_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -532,8 +532,8 @@ func (iw *innerWorker) constructLookupContent(task *lookUpJoinTask) ([]*indexJoi
if iw.hasPrefixCol {
for i := range iw.outerCtx.keyCols {
// If it's a prefix column. Try to fix it.
if iw.colLens[i] != types.UnspecifiedLength {
ranger.CutDatumByPrefixLen(&dLookUpKey[i], iw.colLens[i], iw.rowTypes[iw.keyCols[i]])
if iw.colLens[iw.keyCols[i]] != types.UnspecifiedLength {
ranger.CutDatumByPrefixLen(&dLookUpKey[i], iw.colLens[iw.keyCols[i]], iw.rowTypes[iw.keyCols[i]])
}
}
// dLookUpKey is sorted and deduplicated at sortAndDedupLookUpContents.
Expand Down
25 changes: 25 additions & 0 deletions executor/index_lookup_join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -273,3 +273,28 @@ func (s *testSuite5) TestIssue19411(c *C) {
"2 2"))
tk.MustExec("commit")
}

func (s *testSuite5) TestIssue23653(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1 (c_int int, c_str varchar(40), primary key(c_str), unique key(c_int), unique key(c_str))")
tk.MustExec("create table t2 (c_int int, c_str varchar(40), primary key(c_int, c_str(4)), key(c_int), unique key(c_str))")
tk.MustExec("insert into t1 values (1, 'cool buck'), (2, 'reverent keller')")
tk.MustExec("insert into t2 select * from t1")
tk.MustQuery("select /*+ inl_join(t2) */ * from t1, t2 where t1.c_str = t2.c_str and t1.c_int = t2.c_int and t1.c_int = 2").Check(testkit.Rows(
"2 reverent keller 2 reverent keller"))
}

func (s *testSuite5) TestIssue23656(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1 (c_int int, c_str varchar(40), primary key(c_int, c_str(4)))")
tk.MustExec("create table t2 like t1")
tk.MustExec("insert into t1 values (1, 'clever jang'), (2, 'blissful aryabhata')")
tk.MustExec("insert into t2 select * from t1")
tk.MustQuery("select /*+ inl_join(t2) */ * from t1 join t2 on t1.c_str = t2.c_str where t1.c_int = t2.c_int;").Check(testkit.Rows(
"1 clever jang 1 clever jang",
"2 blissful aryabhata 2 blissful aryabhata"))
}
17 changes: 14 additions & 3 deletions executor/point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -475,6 +475,7 @@ func DecodeRowValToChunk(sctx sessionctx.Context, schema *expression.Schema, tbl
func decodeOldRowValToChunk(sctx sessionctx.Context, schema *expression.Schema, tblInfo *model.TableInfo, handle kv.Handle,
rowVal []byte, chk *chunk.Chunk) error {
pkCols := tables.TryGetCommonPkColumnIds(tblInfo)
prefixColIDs := tables.PrimaryPrefixColumnIDs(tblInfo)
colID2CutPos := make(map[int64]int, schema.Len())
for _, col := range schema.Columns {
if _, ok := colID2CutPos[col.ID]; !ok {
Expand All @@ -495,7 +496,7 @@ func decodeOldRowValToChunk(sctx sessionctx.Context, schema *expression.Schema,
chk.AppendNull(i)
continue
}
ok, err := tryDecodeFromHandle(tblInfo, i, col, handle, chk, decoder, pkCols)
ok, err := tryDecodeFromHandle(tblInfo, i, col, handle, chk, decoder, pkCols, prefixColIDs)
if err != nil {
return err
}
Expand All @@ -520,7 +521,8 @@ func decodeOldRowValToChunk(sctx sessionctx.Context, schema *expression.Schema,
return nil
}

func tryDecodeFromHandle(tblInfo *model.TableInfo, schemaColIdx int, col *expression.Column, handle kv.Handle, chk *chunk.Chunk, decoder *codec.Decoder, pkCols []int64) (bool, error) {
func tryDecodeFromHandle(tblInfo *model.TableInfo, schemaColIdx int, col *expression.Column, handle kv.Handle, chk *chunk.Chunk,
decoder *codec.Decoder, pkCols []int64, prefixColIDs []int64) (bool, error) {
if tblInfo.PKIsHandle && mysql.HasPriKeyFlag(col.RetType.Flag) {
chk.AppendInt64(schemaColIdx, handle.IntValue())
return true, nil
Expand All @@ -532,7 +534,7 @@ func tryDecodeFromHandle(tblInfo *model.TableInfo, schemaColIdx int, col *expres
// Try to decode common handle.
if mysql.HasPriKeyFlag(col.RetType.Flag) {
for i, hid := range pkCols {
if col.ID == hid {
if col.ID == hid && notPKPrefixCol(hid, prefixColIDs) {
_, err := decoder.DecodeOne(handle.EncodedCol(i), schemaColIdx, col.RetType)
if err != nil {
return false, errors.Trace(err)
Expand All @@ -544,6 +546,15 @@ func tryDecodeFromHandle(tblInfo *model.TableInfo, schemaColIdx int, col *expres
return false, nil
}

func notPKPrefixCol(colID int64, prefixColIDs []int64) bool {
for _, pCol := range prefixColIDs {
if pCol == colID {
return false
}
}
return true
}

func getColInfoByID(tbl *model.TableInfo, colID int64) *model.ColumnInfo {
for _, col := range tbl.Columns {
if col.ID == colID {
Expand Down
9 changes: 9 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8955,3 +8955,12 @@ func (s *testIntegrationSuite) TestIssue23623(c *C) {
tk.MustExec("insert into t1 values(-2147483648), (-2147483648), (null);")
tk.MustQuery("select count(*) from t1 where c1 > (select sum(c1) from t1);").Check(testkit.Rows("2"))
}

func (s *testIntegrationSuite) TestApproximatePercentile(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a bit(10))")
tk.MustExec("insert into t values(b'1111')")
tk.MustQuery("select approx_percentile(a, 10) from t").Check(testkit.Rows("<nil>"))
}
3 changes: 1 addition & 2 deletions planner/core/rule_column_pruning.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import (
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/expression/aggregation"
"github.com/pingcap/tidb/planner/util"
"github.com/pingcap/tidb/types"
)

type columnPruner struct {
Expand Down Expand Up @@ -112,7 +111,7 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column)
la.AggFuncs = []*aggregation.AggFuncDesc{one}
col := &expression.Column{
UniqueID: la.ctx.GetSessionVars().AllocPlanColumnID(),
RetType: types.NewFieldType(mysql.TypeLonglong),
RetType: one.RetTp,
}
la.schema.Columns = []*expression.Column{col}
}
Expand Down
7 changes: 7 additions & 0 deletions session/clustered_index_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -452,6 +452,13 @@ func (s *testClusteredSuite) TestClusteredWithOldRowFormat(c *C) {
tk.MustQuery("select cast(col_0 as char(20)) from t use index (`primary`);").Check(testkit.Rows("ddd"))
tk.MustQuery("select cast(col_0 as char(20)) from t use index (idx);").Check(testkit.Rows("ddd"))
tk.MustExec("admin check table t")

// Test for issue https://github.com/pingcap/tidb/issues/23646
tk.MustExec("drop table if exists txx")
tk.MustExec("create table txx(c1 varchar(100), c2 set('dav', 'aaa'), c3 varchar(100), primary key(c1(2), c2) clustered, unique key uk1(c2), index idx1(c2, c1, c3))")
tk.MustExec("insert into txx select 'AarTrNoAL', 'dav', '1'")
tk.MustExec("update txx set c3 = '10', c1 = 'BxTXbyKRFBGbcPmPR' where c2 in ('dav', 'dav')")
tk.MustExec("admin check table txx")
}

func (s *testClusteredSuite) TestIssue20002(c *C) {
Expand Down
1 change: 1 addition & 0 deletions store/tikv/region_request.go
Original file line number Diff line number Diff line change
Expand Up @@ -672,6 +672,7 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed
zap.Stringer("storeNotMatch", storeNotMatch),
zap.Stringer("ctx", ctx))
ctx.Store.markNeedCheck(s.regionCache.notifyCheckCh)
s.regionCache.InvalidateCachedRegion(ctx.Region)
return true, nil
}

Expand Down
1 change: 0 additions & 1 deletion store/tikv/region_request_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,6 @@ func (s *testRegionRequestToSingleStoreSuite) TestOnRegionError(c *C) {
c.Assert(err, NotNil)
c.Assert(resp, IsNil)
}()

}

func (s *testRegionRequestToThreeStoresSuite) TestStoreTokenLimit(c *C) {
Expand Down

0 comments on commit a2d8a15

Please sign in to comment.