-
Notifications
You must be signed in to change notification settings - Fork 5.9k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Skip optimizer and coprocessor for point select.
- Loading branch information
Showing
18 changed files
with
700 additions
and
77 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,181 @@ | ||
// Copyright 2018 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 executor | ||
|
||
import ( | ||
"github.com/juju/errors" | ||
"github.com/pingcap/tidb/expression" | ||
"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" | ||
"github.com/pingcap/tidb/table" | ||
"github.com/pingcap/tidb/table/tables" | ||
"github.com/pingcap/tidb/tablecodec" | ||
"github.com/pingcap/tidb/types" | ||
"github.com/pingcap/tidb/util/chunk" | ||
"github.com/pingcap/tidb/util/codec" | ||
"golang.org/x/net/context" | ||
) | ||
|
||
func (b *executorBuilder) buildPointSelect(p *plan.PointSelectPlan) Executor { | ||
return &PointSelectExecutor{ | ||
ctx: b.ctx, | ||
schema: p.Schema(), | ||
tblInfo: p.TblInfo, | ||
idxInfo: p.IndexInfo, | ||
idxVals: p.IndexValues, | ||
handle: p.Handle, | ||
startTS: b.getStartTS(), | ||
} | ||
} | ||
|
||
type PointSelectExecutor struct { | ||
ctx sessionctx.Context | ||
schema *expression.Schema | ||
tps []*types.FieldType | ||
tblInfo *model.TableInfo | ||
handle int64 | ||
idxInfo *model.IndexInfo | ||
idxVals []types.Datum | ||
startTS uint64 | ||
snapshot kv.Snapshot | ||
done bool | ||
} | ||
|
||
func (e *PointSelectExecutor) Open(context.Context) error { | ||
return nil | ||
} | ||
|
||
func (e *PointSelectExecutor) Close() error { | ||
return nil | ||
} | ||
|
||
func (e *PointSelectExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { | ||
chk.Reset() | ||
if e.done { | ||
return nil | ||
} | ||
e.done = true | ||
snapshot, err := e.ctx.GetStore().GetSnapshot(kv.Version{Ver: e.startTS}) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
e.snapshot = snapshot | ||
if e.idxInfo != nil { | ||
for i := range e.idxVals { | ||
colInfo := e.tblInfo.Columns[e.idxInfo.Columns[i].Offset] | ||
casted, err := table.CastValue(e.ctx, e.idxVals[i], colInfo) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
e.idxVals[i] = casted | ||
} | ||
encodedIdxVals, err := codec.EncodeKey(e.ctx.GetSessionVars().StmtCtx, nil, e.idxVals...) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
idxKey := tablecodec.EncodeIndexSeekKey(e.tblInfo.ID, e.idxInfo.ID, encodedIdxVals) | ||
handleVal, err := e.get(idxKey) | ||
if err != nil && !kv.ErrNotExist.Equal(err) { | ||
return errors.Trace(err) | ||
} | ||
if len(handleVal) == 0 { | ||
return nil | ||
} | ||
h, err := tables.DecodeHandle(handleVal) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
e.handle = h | ||
} | ||
key := tablecodec.EncodeRowKeyWithHandle(e.tblInfo.ID, e.handle) | ||
val, err := e.get(key) | ||
if err != nil && !kv.ErrNotExist.Equal(err) { | ||
return errors.Trace(err) | ||
} | ||
if len(val) == 0 { | ||
return nil | ||
} | ||
colIDs := make(map[int64]int) | ||
for i, col := range e.schema.Columns { | ||
colIDs[col.ID] = i | ||
} | ||
colVals, err := tablecodec.CutRowNew(val, colIDs) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
decoder := codec.NewDecoder(chk, e.ctx.GetSessionVars().GetTimeZone()) | ||
for id, offset := range colIDs { | ||
if e.tblInfo.PKIsHandle && mysql.HasPriKeyFlag(e.schema.Columns[offset].RetType.Flag) { | ||
chk.AppendInt64(offset, e.handle) | ||
continue | ||
} | ||
if id == model.ExtraHandleID { | ||
chk.AppendInt64(offset, e.handle) | ||
continue | ||
} | ||
colVal := colVals[offset] | ||
if len(colVal) == 0 { | ||
colInfo := getColInfoByID(e.tblInfo, id) | ||
d, err1 := table.GetColOriginDefaultValue(e.ctx, colInfo) | ||
if err1 != nil { | ||
return errors.Trace(err1) | ||
} | ||
chk.AppendDatum(offset, &d) | ||
continue | ||
} | ||
_, err = decoder.DecodeOne(colVals[offset], offset, e.schema.Columns[offset].RetType) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
} | ||
return nil | ||
} | ||
|
||
func (e *PointSelectExecutor) get(key kv.Key) (val []byte, err error) { | ||
txn := e.ctx.Txn() | ||
if txn != nil && txn.Valid() && !txn.IsReadOnly() { | ||
return txn.Get(key) | ||
} | ||
return e.snapshot.Get(key) | ||
} | ||
|
||
func getColInfoByID(tbl *model.TableInfo, colID int64) *model.ColumnInfo { | ||
for _, col := range tbl.Columns { | ||
if col.ID == colID { | ||
return col | ||
} | ||
} | ||
return nil | ||
} | ||
|
||
func (e *PointSelectExecutor) Schema() *expression.Schema { | ||
return e.schema | ||
} | ||
|
||
func (e *PointSelectExecutor) retTypes() []*types.FieldType { | ||
if e.tps == nil { | ||
e.tps = make([]*types.FieldType, e.schema.Len()) | ||
for i := range e.schema.Columns { | ||
e.tps[i] = e.schema.Columns[i].RetType | ||
} | ||
} | ||
return e.tps | ||
} | ||
|
||
func (e *PointSelectExecutor) newChunk() *chunk.Chunk { | ||
return chunk.NewChunkWithCapacity(e.retTypes(), 1) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.