*: add arguement StatementContext to Encode functions. (#5566)
This commit is contained in:
@ -77,16 +77,16 @@ func encodeRow(b []byte, row *comparableRow) ([]byte, error) {
|
||||
head = make([]byte, 8)
|
||||
body []byte
|
||||
)
|
||||
|
||||
body, err = codec.EncodeKey(body, row.key...)
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
body, err = codec.EncodeKey(sc, body, row.key...)
|
||||
if err != nil {
|
||||
return b, errors.Trace(err)
|
||||
}
|
||||
body, err = codec.EncodeKey(body, row.val...)
|
||||
body, err = codec.EncodeKey(sc, body, row.val...)
|
||||
if err != nil {
|
||||
return b, errors.Trace(err)
|
||||
}
|
||||
body, err = codec.EncodeKey(body, types.NewIntDatum(row.handle))
|
||||
body, err = codec.EncodeKey(sc, body, types.NewIntDatum(row.handle))
|
||||
if err != nil {
|
||||
return b, errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -25,6 +25,7 @@ import (
|
||||
"github.com/pingcap/tidb/meta"
|
||||
"github.com/pingcap/tidb/model"
|
||||
"github.com/pingcap/tidb/mysql"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/table"
|
||||
"github.com/pingcap/tidb/tablecodec"
|
||||
"github.com/pingcap/tidb/types"
|
||||
@ -315,6 +316,7 @@ func (d *ddl) addTableColumn(t table.Table, columnInfo *model.ColumnInfo, reorgI
|
||||
// This part of the column data rows is defaultSmallBatchCnt.
|
||||
func (d *ddl) backfillColumnInTxn(t table.Table, colMeta *columnMeta, handles []int64, txn kv.Transaction) (int64, error) {
|
||||
nextHandle := handles[0]
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.UTC}
|
||||
for _, handle := range handles {
|
||||
log.Debug("[ddl] backfill column...", handle)
|
||||
rowKey := t.RecordKey(handle)
|
||||
@ -344,7 +346,7 @@ func (d *ddl) backfillColumnInTxn(t table.Table, colMeta *columnMeta, handles []
|
||||
}
|
||||
newColumnIDs = append(newColumnIDs, colMeta.colID)
|
||||
newRow = append(newRow, colMeta.defaultVal)
|
||||
newRowVal, err := tablecodec.EncodeRow(newRow, newColumnIDs, time.UTC)
|
||||
newRowVal, err := tablecodec.EncodeRow(sc, newRow, newColumnIDs)
|
||||
if err != nil {
|
||||
return 0, errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -167,7 +167,7 @@ func (s *testIndexChangeSuite) TestIndexChange(c *C) {
|
||||
|
||||
func checkIndexExists(ctx context.Context, tbl table.Table, indexValue interface{}, handle int64, exists bool) error {
|
||||
idx := tbl.Indices()[0]
|
||||
doesExist, _, err := idx.Exist(ctx.Txn(), types.MakeDatums(indexValue), handle)
|
||||
doesExist, _, err := idx.Exist(ctx.GetSessionVars().StmtCtx, ctx.Txn(), types.MakeDatums(indexValue), handle)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -47,6 +47,7 @@ func (d *ddl) newContext() context.Context {
|
||||
c := mock.NewContext()
|
||||
c.Store = d.store
|
||||
c.GetSessionVars().SetStatusFlag(mysql.ServerStatusAutocommit, false)
|
||||
c.GetSessionVars().StmtCtx.TimeZone = time.UTC
|
||||
return c
|
||||
}
|
||||
|
||||
|
||||
@ -183,7 +183,7 @@ func (e *HashAggExec) getGroupKey(row types.Row) ([]byte, error) {
|
||||
}
|
||||
vals = append(vals, v)
|
||||
}
|
||||
bs, err := codec.EncodeValue([]byte{}, vals...)
|
||||
bs, err := codec.EncodeValue(e.sc, []byte{}, vals...)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -188,7 +188,7 @@ type AnalyzeIndexExec struct {
|
||||
func (e *AnalyzeIndexExec) open() error {
|
||||
idxRange := &ranger.NewRange{LowVal: []types.Datum{types.MinNotNullDatum()}, HighVal: []types.Datum{types.MaxValueDatum()}}
|
||||
var builder requestBuilder
|
||||
kvReq, err := builder.SetIndexRanges(e.tblInfo.ID, e.idxInfo.ID, []*ranger.NewRange{idxRange}).
|
||||
kvReq, err := builder.SetIndexRanges(e.ctx.GetSessionVars().StmtCtx, e.tblInfo.ID, e.idxInfo.ID, []*ranger.NewRange{idxRange}).
|
||||
SetAnalyzeRequest(e.analyzePB).
|
||||
SetKeepOrder(true).
|
||||
SetPriority(e.priority).
|
||||
@ -332,14 +332,15 @@ func (e *AnalyzeColumnsExec) buildStats() (hists []*statistics.Histogram, cms []
|
||||
if err != nil {
|
||||
return nil, nil, errors.Trace(err)
|
||||
}
|
||||
sc := e.ctx.GetSessionVars().StmtCtx
|
||||
if e.pkInfo != nil {
|
||||
pkHist, err = statistics.MergeHistograms(e.ctx.GetSessionVars().StmtCtx, pkHist, statistics.HistogramFromProto(resp.PkHist), maxBucketSize)
|
||||
pkHist, err = statistics.MergeHistograms(sc, pkHist, statistics.HistogramFromProto(resp.PkHist), maxBucketSize)
|
||||
if err != nil {
|
||||
return nil, nil, errors.Trace(err)
|
||||
}
|
||||
}
|
||||
for i, rc := range resp.Collectors {
|
||||
collectors[i].MergeSampleCollector(statistics.SampleCollectorFromProto(rc))
|
||||
collectors[i].MergeSampleCollector(sc, statistics.SampleCollectorFromProto(rc))
|
||||
}
|
||||
}
|
||||
timeZone := e.ctx.GetSessionVars().GetTimeZone()
|
||||
|
||||
@ -31,6 +31,7 @@ import (
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/model"
|
||||
"github.com/pingcap/tidb/plan"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/statistics"
|
||||
"github.com/pingcap/tidb/table"
|
||||
"github.com/pingcap/tidb/types"
|
||||
@ -1335,7 +1336,7 @@ func (builder *dataReaderBuilder) buildIndexReaderForIndexJoin(goCtx goctx.Conte
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
kvRanges, err := buildKvRangesForIndexJoin(e.tableID, e.index.ID, values, indexRanges, keyOff2IdxOff)
|
||||
kvRanges, err := buildKvRangesForIndexJoin(e.ctx.GetSessionVars().StmtCtx, e.tableID, e.index.ID, values, indexRanges, keyOff2IdxOff)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
@ -1349,7 +1350,7 @@ func (builder *dataReaderBuilder) buildIndexLookUpReaderForIndexJoin(goCtx goctx
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
kvRanges, err := buildKvRangesForIndexJoin(e.tableID, e.index.ID, values, indexRanges, keyOff2IdxOff)
|
||||
kvRanges, err := buildKvRangesForIndexJoin(e.ctx.GetSessionVars().StmtCtx, e.tableID, e.index.ID, values, indexRanges, keyOff2IdxOff)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
@ -1358,7 +1359,7 @@ func (builder *dataReaderBuilder) buildIndexLookUpReaderForIndexJoin(goCtx goctx
|
||||
}
|
||||
|
||||
// buildKvRangesForIndexJoin builds kv ranges for index join when the inner plan is index scan plan.
|
||||
func buildKvRangesForIndexJoin(tableID, indexID int64, keyDatums [][]types.Datum, indexRanges []*ranger.NewRange, keyOff2IdxOff []int) ([]kv.KeyRange, error) {
|
||||
func buildKvRangesForIndexJoin(sc *stmtctx.StatementContext, tableID, indexID int64, keyDatums [][]types.Datum, indexRanges []*ranger.NewRange, keyOff2IdxOff []int) ([]kv.KeyRange, error) {
|
||||
kvRanges := make([]kv.KeyRange, 0, len(indexRanges)*len(keyDatums))
|
||||
for _, val := range keyDatums {
|
||||
for _, ran := range indexRanges {
|
||||
@ -1367,7 +1368,7 @@ func buildKvRangesForIndexJoin(tableID, indexID int64, keyDatums [][]types.Datum
|
||||
ran.HighVal[idxOff] = val[keyOff]
|
||||
}
|
||||
}
|
||||
tmpKvRanges, err := indexRangesToKVRanges(tableID, indexID, indexRanges)
|
||||
tmpKvRanges, err := indexRangesToKVRanges(sc, tableID, indexID, indexRanges)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -130,34 +130,17 @@ func tableHandlesToKVRanges(tid int64, handles []int64) []kv.KeyRange {
|
||||
return krs
|
||||
}
|
||||
|
||||
// indexValuesToKVRanges will convert the index datums to kv ranges.
|
||||
func indexValuesToKVRanges(tid, idxID int64, values [][]types.Datum) ([]kv.KeyRange, error) {
|
||||
krs := make([]kv.KeyRange, 0, len(values))
|
||||
for _, vals := range values {
|
||||
// TODO: We don't process the case that equal key has different types.
|
||||
valKey, err := codec.EncodeKey(nil, vals...)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
valKeyNext := []byte(kv.Key(valKey).PrefixNext())
|
||||
rangeBeginKey := tablecodec.EncodeIndexSeekKey(tid, idxID, valKey)
|
||||
rangeEndKey := tablecodec.EncodeIndexSeekKey(tid, idxID, valKeyNext)
|
||||
krs = append(krs, kv.KeyRange{StartKey: rangeBeginKey, EndKey: rangeEndKey})
|
||||
}
|
||||
return krs, nil
|
||||
}
|
||||
|
||||
func indexRangesToKVRanges(tid, idxID int64, ranges []*ranger.NewRange) ([]kv.KeyRange, error) {
|
||||
func indexRangesToKVRanges(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.NewRange) ([]kv.KeyRange, error) {
|
||||
krs := make([]kv.KeyRange, 0, len(ranges))
|
||||
for _, ran := range ranges {
|
||||
low, err := codec.EncodeKey(nil, ran.LowVal...)
|
||||
low, err := codec.EncodeKey(sc, nil, ran.LowVal...)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
if ran.LowExclude {
|
||||
low = []byte(kv.Key(low).PrefixNext())
|
||||
}
|
||||
high, err := codec.EncodeKey(nil, ran.HighVal...)
|
||||
high, err := codec.EncodeKey(sc, nil, ran.HighVal...)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
@ -277,7 +260,7 @@ func setPBColumnsDefaultValue(ctx context.Context, pbColumns []*tipb.ColumnInfo,
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
pbColumns[i].DefaultVal, err = tablecodec.EncodeValue(d, ctx.GetSessionVars().GetTimeZone())
|
||||
pbColumns[i].DefaultVal, err = tablecodec.EncodeValue(ctx.GetSessionVars().StmtCtx, d)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
@ -488,7 +471,7 @@ func (e *IndexReaderExecutor) NextChunk(goCtx goctx.Context, chk *chunk.Chunk) e
|
||||
|
||||
// Open implements the Executor Open interface.
|
||||
func (e *IndexReaderExecutor) Open(goCtx goctx.Context) error {
|
||||
kvRanges, err := indexRangesToKVRanges(e.tableID, e.index.ID, e.ranges)
|
||||
kvRanges, err := indexRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, e.tableID, e.index.ID, e.ranges)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
@ -708,7 +691,7 @@ func (w *tableWorker) pickAndExecTask(goCtx goctx.Context) {
|
||||
|
||||
// Open implements the Executor Open interface.
|
||||
func (e *IndexLookUpExecutor) Open(goCtx goctx.Context) error {
|
||||
kvRanges, err := indexRangesToKVRanges(e.tableID, e.index.ID, e.ranges)
|
||||
kvRanges, err := indexRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, e.tableID, e.index.ID, e.ranges)
|
||||
if err != nil {
|
||||
e.feedback.Invalidate()
|
||||
return errors.Trace(err)
|
||||
@ -879,11 +862,11 @@ func (builder *requestBuilder) SetTableRanges(tid int64, tableRanges []ranger.In
|
||||
return builder
|
||||
}
|
||||
|
||||
func (builder *requestBuilder) SetIndexRanges(tid, idxID int64, ranges []*ranger.NewRange) *requestBuilder {
|
||||
func (builder *requestBuilder) SetIndexRanges(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.NewRange) *requestBuilder {
|
||||
if builder.err != nil {
|
||||
return builder
|
||||
}
|
||||
builder.Request.KeyRanges, builder.err = indexRangesToKVRanges(tid, idxID, ranges)
|
||||
builder.Request.KeyRanges, builder.err = indexRangesToKVRanges(sc, tid, idxID, ranges)
|
||||
return builder
|
||||
}
|
||||
|
||||
@ -892,14 +875,6 @@ func (builder *requestBuilder) SetTableHandles(tid int64, handles []int64) *requ
|
||||
return builder
|
||||
}
|
||||
|
||||
func (builder *requestBuilder) SetIndexValues(tid, idxID int64, values [][]types.Datum) *requestBuilder {
|
||||
if builder.err != nil {
|
||||
return builder
|
||||
}
|
||||
builder.Request.KeyRanges, builder.err = indexValuesToKVRanges(tid, idxID, values)
|
||||
return builder
|
||||
}
|
||||
|
||||
func (builder *requestBuilder) SetDAGRequest(dag *tipb.DAGRequest) *requestBuilder {
|
||||
if builder.err != nil {
|
||||
return builder
|
||||
|
||||
@ -377,7 +377,7 @@ func (e *CheckTableExec) run(goCtx goctx.Context) error {
|
||||
}
|
||||
for _, idx := range tb.Indices() {
|
||||
txn := e.ctx.Txn()
|
||||
err = admin.CompareIndexData(txn, tb, idx)
|
||||
err = admin.CompareIndexData(e.ctx.GetSessionVars().StmtCtx, txn, tb, idx)
|
||||
if err != nil {
|
||||
return errors.Errorf("%v err:%v", t.Name, err)
|
||||
}
|
||||
|
||||
@ -14,12 +14,15 @@
|
||||
package executor
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pingcap/tidb/ast"
|
||||
"github.com/pingcap/tidb/expression"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/model"
|
||||
"github.com/pingcap/tidb/mysql"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/tablecodec"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util"
|
||||
@ -167,8 +170,8 @@ func (s *testExecSuite) TestBuildKvRangesForIndexJoin(c *C) {
|
||||
joinKeyRows = append(joinKeyRows, generateDatumSlice(2, 3))
|
||||
|
||||
keyOff2IdxOff := []int{1, 3}
|
||||
|
||||
kvRanges, err := buildKvRangesForIndexJoin(0, 0, joinKeyRows, indexRanges, keyOff2IdxOff)
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
kvRanges, err := buildKvRangesForIndexJoin(sc, 0, 0, joinKeyRows, indexRanges, keyOff2IdxOff)
|
||||
c.Assert(err, IsNil)
|
||||
// Check the kvRanges is in order.
|
||||
for i, kvRange := range kvRanges {
|
||||
|
||||
@ -241,7 +241,7 @@ func (e *IndexLookUpJoin) constructRequestRows(outerRows []Row) ([][]types.Datum
|
||||
func (e *IndexLookUpJoin) constructJoinKeys(joinKeys [][]types.Datum) ([][]byte, error) {
|
||||
keys := make([][]byte, 0, len(joinKeys))
|
||||
for _, joinKey := range joinKeys {
|
||||
key, err := codec.EncodeKey(nil, joinKey...)
|
||||
key, err := codec.EncodeKey(e.ctx.GetSessionVars().StmtCtx, nil, joinKey...)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -21,6 +21,7 @@ import (
|
||||
"github.com/juju/errors"
|
||||
"github.com/pingcap/tidb/expression"
|
||||
"github.com/pingcap/tidb/plan"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/tablecodec"
|
||||
"github.com/pingcap/tidb/terror"
|
||||
"github.com/pingcap/tidb/types"
|
||||
@ -119,9 +120,9 @@ func makeJoinRow(a Row, b Row) Row {
|
||||
}
|
||||
|
||||
func (e *HashJoinExec) encodeRow(b []byte, row Row) ([]byte, error) {
|
||||
loc := e.ctx.GetSessionVars().GetTimeZone()
|
||||
sc := e.ctx.GetSessionVars().StmtCtx
|
||||
for _, datum := range row {
|
||||
tmp, err := tablecodec.EncodeValue(datum, loc)
|
||||
tmp, err := tablecodec.EncodeValue(sc, datum)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
@ -145,7 +146,7 @@ func (e *HashJoinExec) decodeRow(data []byte) (Row, error) {
|
||||
|
||||
// getJoinKey gets the hash key when given a row and hash columns.
|
||||
// It will return a boolean value representing if the hash key has null, a byte slice representing the result hash code.
|
||||
func getJoinKey(cols []*expression.Column, row Row, vals []types.Datum, bytes []byte) (bool, []byte, error) {
|
||||
func getJoinKey(sc *stmtctx.StatementContext, cols []*expression.Column, row Row, vals []types.Datum, bytes []byte) (bool, []byte, error) {
|
||||
var err error
|
||||
for i, col := range cols {
|
||||
vals[i], err = col.Eval(row)
|
||||
@ -159,7 +160,7 @@ func getJoinKey(cols []*expression.Column, row Row, vals []types.Datum, bytes []
|
||||
if len(vals) == 0 {
|
||||
return false, nil, nil
|
||||
}
|
||||
bytes, err = codec.HashValues(bytes, vals...)
|
||||
bytes, err = codec.HashValues(sc, bytes, vals...)
|
||||
return false, bytes, errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -230,7 +231,7 @@ func (e *HashJoinExec) prepare(goCtx goctx.Context) error {
|
||||
continue
|
||||
}
|
||||
|
||||
hasNull, joinKey, err := getJoinKey(e.innerKeys, innerRow, e.hashJoinBuffers[0].data, nil)
|
||||
hasNull, joinKey, err := getJoinKey(e.ctx.GetSessionVars().StmtCtx, e.innerKeys, innerRow, e.hashJoinBuffers[0].data, nil)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
@ -375,7 +376,7 @@ func (e *HashJoinExec) runJoinWorker(workerID int) {
|
||||
// If there are no matching rows and it is outer join, a null filled result row is created.
|
||||
func (e *HashJoinExec) joinOuterRow(workerID int, outerRow Row, resultBuffer *execResult) bool {
|
||||
buffer := e.hashJoinBuffers[workerID]
|
||||
hasNull, joinKey, err := getJoinKey(e.outerKeys, outerRow, buffer.data, buffer.bytes[:0:cap(buffer.bytes)])
|
||||
hasNull, joinKey, err := getJoinKey(e.ctx.GetSessionVars().StmtCtx, e.outerKeys, outerRow, buffer.data, buffer.bytes[:0:cap(buffer.bytes)])
|
||||
if err != nil {
|
||||
resultBuffer.err = errors.Trace(err)
|
||||
return false
|
||||
|
||||
@ -407,7 +407,7 @@ func (iw *innerWorker) constructDatumLookupKeys(task *lookUpJoinTask) ([][]types
|
||||
continue
|
||||
}
|
||||
keyBuf = keyBuf[:0]
|
||||
keyBuf, err = codec.EncodeKey(keyBuf, dLookUpKey...)
|
||||
keyBuf, err = codec.EncodeKey(iw.ctx.GetSessionVars().StmtCtx, keyBuf, dLookUpKey...)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
@ -524,7 +524,7 @@ func (iw *innerWorker) buildLookUpMap(task *lookUpJoinTask) error {
|
||||
for _, keyCol := range iw.keyCols {
|
||||
d := innerRow.GetDatum(keyCol, iw.rowTypes[keyCol])
|
||||
var err error
|
||||
keyBuf, err = codec.EncodeKey(keyBuf, d)
|
||||
keyBuf, err = codec.EncodeKey(iw.ctx.GetSessionVars().StmtCtx, keyBuf, d)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -267,7 +267,7 @@ func (af *aggFunction) updateSum(ctx *AggEvaluateContext, sc *stmtctx.StatementC
|
||||
return nil
|
||||
}
|
||||
if af.Distinct {
|
||||
d, err1 := ctx.DistinctChecker.Check([]types.Datum{value})
|
||||
d, err1 := ctx.DistinctChecker.Check(sc, []types.Datum{value})
|
||||
if err1 != nil {
|
||||
return errors.Trace(err1)
|
||||
}
|
||||
|
||||
@ -71,7 +71,7 @@ func (af *avgFunction) updateAvg(ctx *AggEvaluateContext, sc *stmtctx.StatementC
|
||||
return nil
|
||||
}
|
||||
if af.Distinct {
|
||||
d, err1 := ctx.DistinctChecker.Check([]types.Datum{value})
|
||||
d, err1 := ctx.DistinctChecker.Check(sc, []types.Datum{value})
|
||||
if err1 != nil {
|
||||
return errors.Trace(err1)
|
||||
}
|
||||
|
||||
@ -86,7 +86,7 @@ func (cf *concatFunction) Update(ctx *AggEvaluateContext, sc *stmtctx.StatementC
|
||||
datumBuf = append(datumBuf, value)
|
||||
}
|
||||
if cf.Distinct {
|
||||
d, err := ctx.DistinctChecker.Check(datumBuf)
|
||||
d, err := ctx.DistinctChecker.Check(sc, datumBuf)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -80,7 +80,7 @@ func (cf *countFunction) Update(ctx *AggEvaluateContext, sc *stmtctx.StatementCo
|
||||
}
|
||||
}
|
||||
if cf.Distinct {
|
||||
d, err := ctx.DistinctChecker.Check(datumBuf)
|
||||
d, err := ctx.DistinctChecker.Check(sc, datumBuf)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -22,10 +22,10 @@ func createDistinctChecker() *distinctChecker {
|
||||
}
|
||||
|
||||
// Check checks if values is distinct.
|
||||
func (d *distinctChecker) Check(values []types.Datum) (bool, error) {
|
||||
func (d *distinctChecker) Check(sc *stmtctx.StatementContext, values []types.Datum) (bool, error) {
|
||||
d.buf = d.buf[:0]
|
||||
var err error
|
||||
d.buf, err = codec.EncodeValue(d.buf, values...)
|
||||
d.buf, err = codec.EncodeValue(sc, d.buf, values...)
|
||||
if err != nil {
|
||||
return false, errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -1,7 +1,10 @@
|
||||
package aggregation
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/check"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/testleak"
|
||||
)
|
||||
@ -25,8 +28,9 @@ func (s *testUtilSuite) TestDistinct(c *check.C) {
|
||||
{[]interface{}{1, nil}, true},
|
||||
{[]interface{}{1, nil}, false},
|
||||
}
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
for _, tt := range tests {
|
||||
d, err := dc.Check(types.MakeDatums(tt.vals...))
|
||||
d, err := dc.Check(sc, types.MakeDatums(tt.vals...))
|
||||
c.Assert(err, check.IsNil)
|
||||
c.Assert(d, check.Equals, tt.expect)
|
||||
}
|
||||
|
||||
@ -21,10 +21,8 @@ import (
|
||||
"github.com/pingcap/tidb/context"
|
||||
"github.com/pingcap/tidb/mysql"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/terror"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/types/json"
|
||||
"github.com/pingcap/tidb/util/codec"
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
@ -316,17 +314,6 @@ func (c *Constant) Decorrelate(_ *Schema) Expression {
|
||||
return c
|
||||
}
|
||||
|
||||
// HashCode implements Expression interface.
|
||||
func (c *Constant) HashCode() []byte {
|
||||
_, err := c.Eval(nil)
|
||||
if err != nil {
|
||||
terror.Log(errors.Trace(err))
|
||||
}
|
||||
bytes, err := codec.EncodeValue(nil, c.Value)
|
||||
terror.Log(errors.Trace(err))
|
||||
return bytes
|
||||
}
|
||||
|
||||
// ResolveIndices implements Expression interface.
|
||||
func (c *Constant) ResolveIndices(_ *Schema) {
|
||||
}
|
||||
|
||||
@ -66,9 +66,6 @@ type Expression interface {
|
||||
// Clone copies an expression totally.
|
||||
Clone() Expression
|
||||
|
||||
// HashCode create the hashcode for expression
|
||||
HashCode() []byte
|
||||
|
||||
// Equal checks whether two expressions are equal.
|
||||
Equal(e Expression, ctx context.Context) bool
|
||||
|
||||
|
||||
@ -56,7 +56,6 @@ func (s *testEvaluatorSuite) TestConstant(c *C) {
|
||||
|
||||
c.Assert(Zero.IsCorrelated(), IsFalse)
|
||||
c.Assert(Zero.Decorrelate(nil).Equal(Zero, s.ctx), IsTrue)
|
||||
c.Assert(Zero.HashCode(), DeepEquals, []byte{0x8, 0x0})
|
||||
c.Assert(Zero.Equal(One, s.ctx), IsFalse)
|
||||
res, err := Zero.MarshalJSON()
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
@ -26,7 +26,6 @@ import (
|
||||
"github.com/pingcap/tidb/terror"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/types/json"
|
||||
"github.com/pingcap/tidb/util/codec"
|
||||
)
|
||||
|
||||
// ScalarFunction is the function that returns a value.
|
||||
@ -253,21 +252,6 @@ func (sf *ScalarFunction) EvalJSON(row types.Row, sc *stmtctx.StatementContext)
|
||||
return sf.Function.evalJSON(row)
|
||||
}
|
||||
|
||||
// HashCode implements Expression interface.
|
||||
func (sf *ScalarFunction) HashCode() []byte {
|
||||
v := make([]types.Datum, 0, len(sf.GetArgs())+1)
|
||||
bytes, err := codec.EncodeValue(nil, types.NewStringDatum(sf.FuncName.L))
|
||||
terror.Log(errors.Trace(err))
|
||||
v = append(v, types.NewBytesDatum(bytes))
|
||||
for _, arg := range sf.GetArgs() {
|
||||
v = append(v, types.NewBytesDatum(arg.HashCode()))
|
||||
}
|
||||
bytes = bytes[:0]
|
||||
bytes, err = codec.EncodeValue(bytes, v...)
|
||||
terror.Log(errors.Trace(err))
|
||||
return bytes
|
||||
}
|
||||
|
||||
// ResolveIndices implements Expression interface.
|
||||
func (sf *ScalarFunction) ResolveIndices(schema *Schema) {
|
||||
for _, arg := range sf.GetArgs() {
|
||||
|
||||
@ -38,7 +38,6 @@ func (s *testEvaluatorSuite) TestScalarFunction(c *C) {
|
||||
c.Assert(res, DeepEquals, []byte{0x22, 0x6c, 0x74, 0x28, 0x66, 0x65, 0x69, 0x2e, 0x68, 0x61, 0x6e, 0x2c, 0x20, 0x31, 0x29, 0x22})
|
||||
c.Assert(sf.IsCorrelated(), IsFalse)
|
||||
c.Assert(sf.Decorrelate(nil).Equal(sf, s.ctx), IsTrue)
|
||||
c.Assert(sf.HashCode(), DeepEquals, []byte{0x2, 0x8, 0x2, 0x4, 0x6c, 0x74, 0x2, 0x20, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x2, 0x12, 0x5, 0xbf, 0xf0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0})
|
||||
|
||||
sf = NewValuesFunc(0, types.NewFieldType(mysql.TypeLonglong), s.ctx)
|
||||
newSf, ok := sf.Clone().(*ScalarFunction)
|
||||
|
||||
@ -15,8 +15,10 @@ package kv
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"time"
|
||||
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/codec"
|
||||
"github.com/pingcap/tidb/util/testleak"
|
||||
@ -29,13 +31,14 @@ type testKeySuite struct {
|
||||
|
||||
func (s *testKeySuite) TestPartialNext(c *C) {
|
||||
defer testleak.AfterTest(c)()
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
// keyA represents a multi column index.
|
||||
keyA, err := codec.EncodeValue(nil, types.NewDatum("abc"), types.NewDatum("def"))
|
||||
keyA, err := codec.EncodeValue(sc, nil, types.NewDatum("abc"), types.NewDatum("def"))
|
||||
c.Check(err, IsNil)
|
||||
keyB, err := codec.EncodeValue(nil, types.NewDatum("abca"), types.NewDatum("def"))
|
||||
keyB, err := codec.EncodeValue(sc, nil, types.NewDatum("abca"), types.NewDatum("def"))
|
||||
|
||||
// We only use first column value to seek.
|
||||
seekKey, err := codec.EncodeValue(nil, types.NewDatum("abc"))
|
||||
seekKey, err := codec.EncodeValue(sc, nil, types.NewDatum("abc"))
|
||||
c.Check(err, IsNil)
|
||||
|
||||
nextKey := Key(seekKey).Next()
|
||||
|
||||
@ -154,7 +154,7 @@ func setPBColumnsDefaultValue(ctx context.Context, pbColumns []*tipb.ColumnInfo,
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
pbColumns[i].DefaultVal, err = tablecodec.EncodeValue(d, ctx.GetSessionVars().GetTimeZone())
|
||||
pbColumns[i].DefaultVal, err = tablecodec.EncodeValue(ctx.GetSessionVars().StmtCtx, d)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -867,7 +867,9 @@ func (t *regionHandlerTool) getMvccByStartTs(startTS uint64, startKey, endKey []
|
||||
}
|
||||
|
||||
func (t *regionHandlerTool) getMvccByIdxValue(idx table.Index, values url.Values, idxCols []*model.ColumnInfo, handleStr string) (*kvrpcpb.MvccGetByKeyResponse, error) {
|
||||
idxRow, err := t.formValue2DatumRow(values, idxCols)
|
||||
sc := new(stmtctx.StatementContext)
|
||||
sc.TimeZone = time.UTC
|
||||
idxRow, err := t.formValue2DatumRow(sc, values, idxCols)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
@ -875,7 +877,7 @@ func (t *regionHandlerTool) getMvccByIdxValue(idx table.Index, values url.Values
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
encodedKey, _, err := idx.GenIndexKey(idxRow, handle)
|
||||
encodedKey, _, err := idx.GenIndexKey(sc, idxRow, handle)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
@ -883,10 +885,8 @@ func (t *regionHandlerTool) getMvccByIdxValue(idx table.Index, values url.Values
|
||||
}
|
||||
|
||||
// formValue2DatumRow converts URL query string to a Datum Row.
|
||||
func (t *regionHandlerTool) formValue2DatumRow(values url.Values, idxCols []*model.ColumnInfo) ([]types.Datum, error) {
|
||||
func (t *regionHandlerTool) formValue2DatumRow(sc *stmtctx.StatementContext, values url.Values, idxCols []*model.ColumnInfo) ([]types.Datum, error) {
|
||||
data := make([]types.Datum, len(idxCols))
|
||||
sc := new(stmtctx.StatementContext)
|
||||
sc.TimeZone = time.UTC
|
||||
for i, col := range idxCols {
|
||||
colName := col.Name.String()
|
||||
vals, ok := values[colName]
|
||||
|
||||
@ -19,6 +19,7 @@ import (
|
||||
|
||||
"github.com/cznic/sortutil"
|
||||
"github.com/juju/errors"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/codec"
|
||||
"github.com/pingcap/tipb/go-tipb"
|
||||
@ -53,8 +54,8 @@ func (c *CMSketch) InsertBytes(bytes []byte) {
|
||||
}
|
||||
}
|
||||
|
||||
func (c *CMSketch) queryValue(val types.Datum) (uint32, error) {
|
||||
bytes, err := codec.EncodeValue(nil, val)
|
||||
func (c *CMSketch) queryValue(sc *stmtctx.StatementContext, val types.Datum) (uint32, error) {
|
||||
bytes, err := codec.EncodeValue(sc, nil, val)
|
||||
if err != nil {
|
||||
return 0, errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -15,15 +15,17 @@ package statistics
|
||||
|
||||
import (
|
||||
"math/rand"
|
||||
"time"
|
||||
|
||||
"github.com/juju/errors"
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/codec"
|
||||
)
|
||||
|
||||
func (c *CMSketch) insert(val *types.Datum) error {
|
||||
bytes, err := codec.EncodeValue(nil, *val)
|
||||
bytes, err := codec.EncodeValue(nil, nil, *val)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
@ -47,9 +49,10 @@ func buildCMSketchAndMap(d, w int32, seed int64, total, imax uint64, s float64)
|
||||
}
|
||||
|
||||
func averageAbsoluteError(cms *CMSketch, mp map[int64]uint32) (uint64, error) {
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
var total uint64
|
||||
for num, count := range mp {
|
||||
estimate, err := cms.queryValue(types.NewIntDatum(num))
|
||||
estimate, err := cms.queryValue(sc, types.NewIntDatum(num))
|
||||
if err != nil {
|
||||
return 0, errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -17,6 +17,7 @@ import (
|
||||
"hash"
|
||||
|
||||
"github.com/juju/errors"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/codec"
|
||||
"github.com/pingcap/tipb/go-tipb"
|
||||
@ -61,8 +62,8 @@ func (s *FMSketch) insertHashValue(hashVal uint64) {
|
||||
}
|
||||
|
||||
// InsertValue inserts a value into the FM sketch.
|
||||
func (s *FMSketch) InsertValue(value types.Datum) error {
|
||||
bytes, err := codec.EncodeValue(nil, value)
|
||||
func (s *FMSketch) InsertValue(sc *stmtctx.StatementContext, value types.Datum) error {
|
||||
bytes, err := codec.EncodeValue(sc, nil, value)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
@ -75,10 +76,10 @@ func (s *FMSketch) InsertValue(value types.Datum) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func buildFMSketch(values []types.Datum, maxSize int) (*FMSketch, int64, error) {
|
||||
func buildFMSketch(sc *stmtctx.StatementContext, values []types.Datum, maxSize int) (*FMSketch, int64, error) {
|
||||
s := NewFMSketch(maxSize)
|
||||
for _, value := range values {
|
||||
err := s.InsertValue(value)
|
||||
err := s.InsertValue(sc, value)
|
||||
if err != nil {
|
||||
return nil, 0, errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -14,20 +14,24 @@
|
||||
package statistics
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
)
|
||||
|
||||
func (s *testStatisticsSuite) TestSketch(c *C) {
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
maxSize := 1000
|
||||
sampleSketch, ndv, err := buildFMSketch(s.samples, maxSize)
|
||||
sampleSketch, ndv, err := buildFMSketch(sc, s.samples, maxSize)
|
||||
c.Check(err, IsNil)
|
||||
c.Check(ndv, Equals, int64(6232))
|
||||
|
||||
rcSketch, ndv, err := buildFMSketch(s.rc.(*recordSet).data, maxSize)
|
||||
rcSketch, ndv, err := buildFMSketch(sc, s.rc.(*recordSet).data, maxSize)
|
||||
c.Check(err, IsNil)
|
||||
c.Check(ndv, Equals, int64(73344))
|
||||
|
||||
pkSketch, ndv, err := buildFMSketch(s.pk.(*recordSet).data, maxSize)
|
||||
pkSketch, ndv, err := buildFMSketch(sc, s.pk.(*recordSet).data, maxSize)
|
||||
c.Check(err, IsNil)
|
||||
c.Check(ndv, Equals, int64(100480))
|
||||
|
||||
@ -45,8 +49,9 @@ func (s *testStatisticsSuite) TestSketch(c *C) {
|
||||
}
|
||||
|
||||
func (s *testStatisticsSuite) TestSketchProtoConversion(c *C) {
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
maxSize := 1000
|
||||
sampleSketch, ndv, err := buildFMSketch(s.samples, maxSize)
|
||||
sampleSketch, ndv, err := buildFMSketch(sc, s.samples, maxSize)
|
||||
c.Check(err, IsNil)
|
||||
c.Check(ndv, Equals, int64(6232))
|
||||
|
||||
|
||||
@ -548,7 +548,7 @@ func (c *Column) String() string {
|
||||
|
||||
func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum) (float64, error) {
|
||||
if c.CMSketch != nil {
|
||||
count, err := c.CMSketch.queryValue(val)
|
||||
count, err := c.CMSketch.queryValue(sc, val)
|
||||
return float64(count), errors.Trace(err)
|
||||
}
|
||||
count, err := c.Histogram.equalRowCount(sc, val)
|
||||
@ -659,11 +659,11 @@ func (idx *Index) equalRowCount(sc *stmtctx.StatementContext, b []byte) (float64
|
||||
func (idx *Index) getRowCount(sc *stmtctx.StatementContext, indexRanges []*ranger.NewRange) (float64, error) {
|
||||
totalCount := float64(0)
|
||||
for _, indexRange := range indexRanges {
|
||||
lb, err := codec.EncodeKey(nil, indexRange.LowVal...)
|
||||
lb, err := codec.EncodeKey(sc, nil, indexRange.LowVal...)
|
||||
if err != nil {
|
||||
return 0, errors.Trace(err)
|
||||
}
|
||||
rb, err := codec.EncodeKey(nil, indexRange.HighVal...)
|
||||
rb, err := codec.EncodeKey(sc, nil, indexRange.HighVal...)
|
||||
if err != nil {
|
||||
return 0, errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -39,7 +39,7 @@ type SampleCollector struct {
|
||||
}
|
||||
|
||||
// MergeSampleCollector merges two sample collectors.
|
||||
func (c *SampleCollector) MergeSampleCollector(rc *SampleCollector) {
|
||||
func (c *SampleCollector) MergeSampleCollector(sc *stmtctx.StatementContext, rc *SampleCollector) {
|
||||
c.NullCount += rc.NullCount
|
||||
c.Count += rc.Count
|
||||
c.FMSketch.mergeFMSketch(rc.FMSketch)
|
||||
@ -48,7 +48,7 @@ func (c *SampleCollector) MergeSampleCollector(rc *SampleCollector) {
|
||||
terror.Log(errors.Trace(err))
|
||||
}
|
||||
for _, val := range rc.Samples {
|
||||
err := c.collect(val)
|
||||
err := c.collect(sc, val)
|
||||
terror.Log(errors.Trace(err))
|
||||
}
|
||||
}
|
||||
@ -83,14 +83,14 @@ func SampleCollectorFromProto(collector *tipb.SampleCollector) *SampleCollector
|
||||
return s
|
||||
}
|
||||
|
||||
func (c *SampleCollector) collect(d types.Datum) error {
|
||||
func (c *SampleCollector) collect(sc *stmtctx.StatementContext, d types.Datum) error {
|
||||
if !c.IsMerger {
|
||||
if d.IsNull() {
|
||||
c.NullCount++
|
||||
return nil
|
||||
}
|
||||
c.Count++
|
||||
if err := c.FMSketch.InsertValue(d); err != nil {
|
||||
if err := c.FMSketch.InsertValue(sc, d); err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
if c.CMSketch != nil {
|
||||
@ -166,7 +166,7 @@ func (s SampleBuilder) CollectColumnStats() ([]*SampleCollector, *SortedBuilder,
|
||||
datums = datums[1:]
|
||||
}
|
||||
for i, val := range datums {
|
||||
err = collectors[i].collect(val)
|
||||
err = collectors[i].collect(s.Sc, val)
|
||||
if err != nil {
|
||||
return nil, nil, errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -14,10 +14,13 @@
|
||||
package statistics_test
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pingcap/tidb/ast"
|
||||
"github.com/pingcap/tidb/model"
|
||||
"github.com/pingcap/tidb/mysql"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/statistics"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/chunk"
|
||||
@ -134,12 +137,13 @@ func (s *testSampleSuite) TestMergeSampleCollector(c *C) {
|
||||
CMSketchDepth: 8,
|
||||
}
|
||||
s.rs.Close()
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
collectors, pkBuilder, err := builder.CollectColumnStats()
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(pkBuilder, IsNil)
|
||||
c.Assert(len(collectors), Equals, 2)
|
||||
collectors[0].IsMerger = true
|
||||
collectors[0].MergeSampleCollector(collectors[1])
|
||||
collectors[0].MergeSampleCollector(sc, collectors[1])
|
||||
c.Assert(collectors[0].FMSketch.NDV(), Equals, int64(9280))
|
||||
c.Assert(len(collectors[0].Samples), Equals, 1000)
|
||||
c.Assert(collectors[0].NullCount, Equals, int64(1000))
|
||||
|
||||
@ -18,6 +18,7 @@ import (
|
||||
"os"
|
||||
"runtime/pprof"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pingcap/tidb"
|
||||
@ -27,6 +28,7 @@ import (
|
||||
"github.com/pingcap/tidb/model"
|
||||
"github.com/pingcap/tidb/mysql"
|
||||
"github.com/pingcap/tidb/plan"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/statistics"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/codec"
|
||||
@ -64,6 +66,7 @@ func (s *testSelectivitySuite) generateIntDatum(dimension, num int) ([]types.Dat
|
||||
ret[i] = types.NewIntDatum(int64(i))
|
||||
}
|
||||
} else {
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
// In this way, we can guarantee the datum is in order.
|
||||
for i := 0; i < len; i++ {
|
||||
data := make([]types.Datum, dimension)
|
||||
@ -72,7 +75,7 @@ func (s *testSelectivitySuite) generateIntDatum(dimension, num int) ([]types.Dat
|
||||
data[dimension-k-1].SetInt64(int64(j % num))
|
||||
j = j / num
|
||||
}
|
||||
bytes, err := codec.EncodeKey(nil, data...)
|
||||
bytes, err := codec.EncodeKey(sc, nil, data...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
@ -16,6 +16,7 @@ package statistics
|
||||
import (
|
||||
"math"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/juju/errors"
|
||||
. "github.com/pingcap/check"
|
||||
@ -150,7 +151,8 @@ func (s *testStatisticsSuite) SetUpSuite(c *C) {
|
||||
}
|
||||
|
||||
func encodeKey(key types.Datum) types.Datum {
|
||||
buf, _ := codec.EncodeKey(nil, key)
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
buf, _ := codec.EncodeKey(sc, nil, key)
|
||||
return types.NewBytesDatum(buf)
|
||||
}
|
||||
|
||||
@ -187,7 +189,7 @@ func buildIndex(ctx context.Context, numBuckets, id int64, records ast.RecordSet
|
||||
break
|
||||
}
|
||||
datums := ast.RowToDatums(row, records.Fields())
|
||||
buf, err := codec.EncodeKey(nil, datums...)
|
||||
buf, err := codec.EncodeKey(ctx.GetSessionVars().StmtCtx, nil, datums...)
|
||||
if err != nil {
|
||||
return 0, nil, nil, errors.Trace(err)
|
||||
}
|
||||
@ -209,9 +211,9 @@ func checkRepeats(c *C, hg *Histogram) {
|
||||
|
||||
func (s *testStatisticsSuite) TestBuild(c *C) {
|
||||
bucketCount := int64(256)
|
||||
sketch, _, _ := buildFMSketch(s.rc.(*recordSet).data, 1000)
|
||||
ctx := mock.NewContext()
|
||||
sc := ctx.GetSessionVars().StmtCtx
|
||||
sketch, _, _ := buildFMSketch(sc, s.rc.(*recordSet).data, 1000)
|
||||
|
||||
collector := &SampleCollector{
|
||||
Count: s.count,
|
||||
@ -437,9 +439,9 @@ func buildCMSketch(values []types.Datum) *CMSketch {
|
||||
|
||||
func (s *testStatisticsSuite) TestColumnRange(c *C) {
|
||||
bucketCount := int64(256)
|
||||
sketch, _, _ := buildFMSketch(s.rc.(*recordSet).data, 1000)
|
||||
ctx := mock.NewContext()
|
||||
sc := ctx.GetSessionVars().StmtCtx
|
||||
sketch, _, _ := buildFMSketch(sc, s.rc.(*recordSet).data, 1000)
|
||||
|
||||
collector := &SampleCollector{
|
||||
Count: s.count,
|
||||
|
||||
@ -100,7 +100,7 @@ func (e *hashAggExec) Next(goCtx goctx.Context) (value [][]byte, err error) {
|
||||
for i, agg := range e.aggExprs {
|
||||
partialResults := agg.GetPartialResult(aggCtxs[i])
|
||||
for _, result := range partialResults {
|
||||
data, err := codec.EncodeValue(nil, result)
|
||||
data, err := codec.EncodeValue(e.evalCtx.sc, nil, result)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
@ -125,7 +125,7 @@ func (e *hashAggExec) getGroupKey() ([]byte, [][]byte, error) {
|
||||
if err != nil {
|
||||
return nil, nil, errors.Trace(err)
|
||||
}
|
||||
b, err := codec.EncodeValue(nil, v)
|
||||
b, err := codec.EncodeValue(e.evalCtx.sc, nil, v)
|
||||
if err != nil {
|
||||
return nil, nil, errors.Trace(err)
|
||||
}
|
||||
@ -214,7 +214,7 @@ func (e *streamAggExec) getPartialResult() ([][]byte, error) {
|
||||
for i, agg := range e.aggExprs {
|
||||
partialResults := agg.GetPartialResult(e.aggCtxs[i])
|
||||
for _, result := range partialResults {
|
||||
data, err := codec.EncodeValue(nil, result)
|
||||
data, err := codec.EncodeValue(e.evalCtx.sc, nil, result)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
@ -225,7 +225,7 @@ func (e *streamAggExec) getPartialResult() ([][]byte, error) {
|
||||
}
|
||||
e.currGroupByValues = e.currGroupByValues[:0]
|
||||
for _, d := range e.currGroupByRow {
|
||||
buf, err := codec.EncodeValue(nil, d)
|
||||
buf, err := codec.EncodeValue(e.evalCtx.sc, nil, d)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -115,8 +115,8 @@ type analyzeColumnsExec struct {
|
||||
|
||||
func (h *rpcHandler) handleAnalyzeColumnsReq(req *coprocessor.Request, analyzeReq *tipb.AnalyzeReq) (*coprocessor.Response, error) {
|
||||
sc := flagsToStatementContext(analyzeReq.Flags)
|
||||
timeZone := time.FixedZone("UTC", int(analyzeReq.TimeZoneOffset))
|
||||
evalCtx := &evalContext{sc: sc, timeZone: timeZone}
|
||||
sc.TimeZone = time.FixedZone("UTC", int(analyzeReq.TimeZoneOffset))
|
||||
evalCtx := &evalContext{sc: sc}
|
||||
columns := analyzeReq.ColReq.ColumnsInfo
|
||||
evalCtx.setColumnInfo(columns)
|
||||
e := &analyzeColumnsExec{
|
||||
|
||||
@ -22,6 +22,7 @@ import (
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/store/tikv"
|
||||
"github.com/pingcap/tidb/store/tikv/mocktikv"
|
||||
"github.com/pingcap/tidb/tablecodec"
|
||||
@ -54,15 +55,16 @@ func (s *testClusterSuite) TestClusterSplit(c *C) {
|
||||
idxID := int64(2)
|
||||
colID := int64(3)
|
||||
handle := int64(1)
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.UTC}
|
||||
for i := 0; i < 1000; i++ {
|
||||
rowKey := tablecodec.EncodeRowKeyWithHandle(tblID, handle)
|
||||
colValue := types.NewStringDatum(strconv.Itoa(int(handle)))
|
||||
// TODO: Should use session's TimeZone instead of UTC.
|
||||
rowValue, err1 := tablecodec.EncodeRow([]types.Datum{colValue}, []int64{colID}, time.UTC)
|
||||
rowValue, err1 := tablecodec.EncodeRow(sc, []types.Datum{colValue}, []int64{colID})
|
||||
c.Assert(err1, IsNil)
|
||||
txn.Set(rowKey, rowValue)
|
||||
|
||||
encodedIndexValue, err1 := codec.EncodeKey(nil, []types.Datum{colValue, types.NewIntDatum(handle)}...)
|
||||
encodedIndexValue, err1 := codec.EncodeKey(sc, nil, []types.Datum{colValue, types.NewIntDatum(handle)}...)
|
||||
c.Assert(err1, IsNil)
|
||||
idxKey := tablecodec.EncodeIndexSeekKey(tblID, idxID, encodedIndexValue)
|
||||
txn.Set(idxKey, []byte{'0'})
|
||||
|
||||
@ -102,11 +102,11 @@ func (h *rpcHandler) buildDAGExecutor(req *coprocessor.Request) (executor, *tipb
|
||||
return nil, nil, errors.Trace(err)
|
||||
}
|
||||
sc := flagsToStatementContext(dagReq.Flags)
|
||||
timeZone := time.FixedZone("UTC", int(dagReq.TimeZoneOffset))
|
||||
sc.TimeZone = time.FixedZone("UTC", int(dagReq.TimeZoneOffset))
|
||||
ctx := &dagContext{
|
||||
dagReq: dagReq,
|
||||
keyRanges: req.Ranges,
|
||||
evalCtx: &evalContext{sc: sc, timeZone: timeZone},
|
||||
evalCtx: &evalContext{sc: sc},
|
||||
}
|
||||
e, err := h.buildDAG(ctx, dagReq.Executors)
|
||||
if err != nil {
|
||||
@ -343,7 +343,6 @@ type evalContext struct {
|
||||
columnInfos []*tipb.ColumnInfo
|
||||
fieldTps []*types.FieldType
|
||||
sc *stmtctx.StatementContext
|
||||
timeZone *time.Location
|
||||
}
|
||||
|
||||
func (e *evalContext) setColumnInfo(cols []*tipb.ColumnInfo) {
|
||||
@ -363,7 +362,7 @@ func (e *evalContext) setColumnInfo(cols []*tipb.ColumnInfo) {
|
||||
func (e *evalContext) decodeRelatedColumnVals(relatedColOffsets []int, value [][]byte, row []types.Datum) error {
|
||||
var err error
|
||||
for _, offset := range relatedColOffsets {
|
||||
row[offset], err = tablecodec.DecodeColumnValue(value[offset], e.fieldTps[offset], e.timeZone)
|
||||
row[offset], err = tablecodec.DecodeColumnValue(value[offset], e.fieldTps[offset], e.sc.TimeZone)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -270,7 +270,7 @@ func (e *indexScanExec) decodeIndexKV(pair Pair) ([][]byte, error) {
|
||||
} else {
|
||||
handleDatum = types.NewIntDatum(handle)
|
||||
}
|
||||
handleBytes, err := codec.EncodeValue(b, handleDatum)
|
||||
handleBytes, err := codec.EncodeValue(nil, b, handleDatum)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
@ -543,7 +543,7 @@ func getRowData(columns []*tipb.ColumnInfo, colIDs map[int64]int, handle int64,
|
||||
} else {
|
||||
handleDatum = types.NewIntDatum(handle)
|
||||
}
|
||||
handleData, err1 := codec.EncodeValue(nil, handleDatum)
|
||||
handleData, err1 := codec.EncodeValue(nil, nil, handleDatum)
|
||||
if err1 != nil {
|
||||
return nil, errors.Trace(err1)
|
||||
}
|
||||
|
||||
@ -17,6 +17,7 @@ import (
|
||||
"github.com/pingcap/tidb/context"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/model"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/types"
|
||||
)
|
||||
|
||||
@ -33,15 +34,15 @@ type Index interface {
|
||||
// Create supports insert into statement.
|
||||
Create(ctx context.Context, rm kv.RetrieverMutator, indexedValues []types.Datum, h int64) (int64, error)
|
||||
// Delete supports delete from statement.
|
||||
Delete(m kv.Mutator, indexedValues []types.Datum, h int64) error
|
||||
Delete(sc *stmtctx.StatementContext, m kv.Mutator, indexedValues []types.Datum, h int64) error
|
||||
// Drop supports drop table, drop index statements.
|
||||
Drop(rm kv.RetrieverMutator) error
|
||||
// Exist supports check index exists or not.
|
||||
Exist(rm kv.RetrieverMutator, indexedValues []types.Datum, h int64) (bool, int64, error)
|
||||
Exist(sc *stmtctx.StatementContext, rm kv.RetrieverMutator, indexedValues []types.Datum, h int64) (bool, int64, error)
|
||||
// GenIndexKey generates an index key.
|
||||
GenIndexKey(indexedValues []types.Datum, h int64) (key []byte, distinct bool, err error)
|
||||
GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types.Datum, h int64) (key []byte, distinct bool, err error)
|
||||
// Seek supports where clause.
|
||||
Seek(r kv.Retriever, indexedValues []types.Datum) (iter IndexIterator, hit bool, err error)
|
||||
Seek(sc *stmtctx.StatementContext, r kv.Retriever, indexedValues []types.Datum) (iter IndexIterator, hit bool, err error)
|
||||
// SeekFirst supports aggregate min and ascend order by.
|
||||
SeekFirst(r kv.Retriever) (iter IndexIterator, err error)
|
||||
// FetchValues fetched index column values in a row.
|
||||
|
||||
@ -22,6 +22,7 @@ import (
|
||||
"github.com/pingcap/tidb/context"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/model"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/table"
|
||||
"github.com/pingcap/tidb/tablecodec"
|
||||
"github.com/pingcap/tidb/types"
|
||||
@ -131,7 +132,7 @@ func (c *index) Meta() *model.IndexInfo {
|
||||
|
||||
// GenIndexKey generates storage key for index values. Returned distinct indicates whether the
|
||||
// indexed values should be distinct in storage (i.e. whether handle is encoded in the key).
|
||||
func (c *index) GenIndexKey(indexedValues []types.Datum, h int64) (key []byte, distinct bool, err error) {
|
||||
func (c *index) GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types.Datum, h int64) (key []byte, distinct bool, err error) {
|
||||
if c.idxInfo.Unique {
|
||||
// See https://dev.mysql.com/doc/refman/5.7/en/create-index.html
|
||||
// A UNIQUE index creates a constraint such that all values in the index must be distinct.
|
||||
@ -165,9 +166,9 @@ func (c *index) GenIndexKey(indexedValues []types.Datum, h int64) (key []byte, d
|
||||
key = make([]byte, 0, len(c.prefix)+len(indexedValues)*9+9)
|
||||
}
|
||||
key = append(key, []byte(c.prefix)...)
|
||||
key, err = codec.EncodeKey(key, indexedValues...)
|
||||
key, err = codec.EncodeKey(sc, key, indexedValues...)
|
||||
if !distinct && err == nil {
|
||||
key, err = codec.EncodeKey(key, types.NewDatum(h))
|
||||
key, err = codec.EncodeKey(sc, key, types.NewDatum(h))
|
||||
}
|
||||
if err != nil {
|
||||
return nil, false, errors.Trace(err)
|
||||
@ -180,7 +181,7 @@ func (c *index) GenIndexKey(indexedValues []types.Datum, h int64) (key []byte, d
|
||||
// Create will return the existing entry's handle as the first return value, ErrKeyExists as the second return value.
|
||||
func (c *index) Create(ctx context.Context, rm kv.RetrieverMutator, indexedValues []types.Datum, h int64) (int64, error) {
|
||||
importData := ctx.GetSessionVars().ImportingData
|
||||
key, distinct, err := c.GenIndexKey(indexedValues, h)
|
||||
key, distinct, err := c.GenIndexKey(ctx.GetSessionVars().StmtCtx, indexedValues, h)
|
||||
if err != nil {
|
||||
return 0, errors.Trace(err)
|
||||
}
|
||||
@ -208,8 +209,8 @@ func (c *index) Create(ctx context.Context, rm kv.RetrieverMutator, indexedValue
|
||||
}
|
||||
|
||||
// Delete removes the entry for handle h and indexdValues from KV index.
|
||||
func (c *index) Delete(m kv.Mutator, indexedValues []types.Datum, h int64) error {
|
||||
key, _, err := c.GenIndexKey(indexedValues, h)
|
||||
func (c *index) Delete(sc *stmtctx.StatementContext, m kv.Mutator, indexedValues []types.Datum, h int64) error {
|
||||
key, _, err := c.GenIndexKey(sc, indexedValues, h)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
@ -243,8 +244,8 @@ func (c *index) Drop(rm kv.RetrieverMutator) error {
|
||||
}
|
||||
|
||||
// Seek searches KV index for the entry with indexedValues.
|
||||
func (c *index) Seek(r kv.Retriever, indexedValues []types.Datum) (iter table.IndexIterator, hit bool, err error) {
|
||||
key, _, err := c.GenIndexKey(indexedValues, 0)
|
||||
func (c *index) Seek(sc *stmtctx.StatementContext, r kv.Retriever, indexedValues []types.Datum) (iter table.IndexIterator, hit bool, err error) {
|
||||
key, _, err := c.GenIndexKey(sc, indexedValues, 0)
|
||||
if err != nil {
|
||||
return nil, false, errors.Trace(err)
|
||||
}
|
||||
@ -269,8 +270,8 @@ func (c *index) SeekFirst(r kv.Retriever) (iter table.IndexIterator, err error)
|
||||
return &indexIter{it: it, idx: c, prefix: c.prefix}, nil
|
||||
}
|
||||
|
||||
func (c *index) Exist(rm kv.RetrieverMutator, indexedValues []types.Datum, h int64) (bool, int64, error) {
|
||||
key, distinct, err := c.GenIndexKey(indexedValues, h)
|
||||
func (c *index) Exist(sc *stmtctx.StatementContext, rm kv.RetrieverMutator, indexedValues []types.Datum, h int64) (bool, int64, error) {
|
||||
key, distinct, err := c.GenIndexKey(sc, indexedValues, h)
|
||||
if err != nil {
|
||||
return false, 0, errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -15,10 +15,12 @@ package tables_test
|
||||
|
||||
import (
|
||||
"io"
|
||||
"time"
|
||||
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/model"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/store/tikv"
|
||||
"github.com/pingcap/tidb/table/tables"
|
||||
"github.com/pingcap/tidb/terror"
|
||||
@ -82,16 +84,16 @@ func (s *testIndexSuite) TestIndex(c *C) {
|
||||
c.Assert(getValues[1].GetInt64(), Equals, int64(2))
|
||||
c.Assert(h, Equals, int64(1))
|
||||
it.Close()
|
||||
|
||||
exist, _, err := index.Exist(txn, values, 100)
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
exist, _, err := index.Exist(sc, txn, values, 100)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(exist, IsFalse)
|
||||
|
||||
exist, _, err = index.Exist(txn, values, 1)
|
||||
exist, _, err = index.Exist(sc, txn, values, 1)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(exist, IsTrue)
|
||||
|
||||
err = index.Delete(txn, values, 1)
|
||||
err = index.Delete(sc, txn, values, 1)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
it, err = index.SeekFirst(txn)
|
||||
@ -107,14 +109,14 @@ func (s *testIndexSuite) TestIndex(c *C) {
|
||||
_, err = index.SeekFirst(txn)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
_, hit, err := index.Seek(txn, values)
|
||||
_, hit, err := index.Seek(sc, txn, values)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(hit, IsTrue)
|
||||
|
||||
err = index.Drop(txn)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
it, hit, err = index.Seek(txn, values)
|
||||
it, hit, err = index.Seek(sc, txn, values)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(hit, IsFalse)
|
||||
|
||||
@ -169,12 +171,12 @@ func (s *testIndexSuite) TestIndex(c *C) {
|
||||
c.Assert(h, Equals, int64(1))
|
||||
it.Close()
|
||||
|
||||
exist, h, err = index.Exist(txn, values, 1)
|
||||
exist, h, err = index.Exist(sc, txn, values, 1)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(h, Equals, int64(1))
|
||||
c.Assert(exist, IsTrue)
|
||||
|
||||
exist, h, err = index.Exist(txn, values, 2)
|
||||
exist, h, err = index.Exist(sc, txn, values, 2)
|
||||
c.Assert(err, NotNil)
|
||||
c.Assert(h, Equals, int64(1))
|
||||
c.Assert(exist, IsTrue)
|
||||
@ -211,7 +213,8 @@ func (s *testIndexSuite) TestCombineIndexSeek(c *C) {
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
index2 := tables.NewIndex(tblInfo, tblInfo.Indices[0])
|
||||
iter, hit, err := index2.Seek(txn, types.MakeDatums("abc", nil))
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
iter, hit, err := index2.Seek(sc, txn, types.MakeDatums("abc", nil))
|
||||
c.Assert(err, IsNil)
|
||||
defer iter.Close()
|
||||
c.Assert(hit, IsFalse)
|
||||
|
||||
@ -29,6 +29,7 @@ import (
|
||||
"github.com/pingcap/tidb/model"
|
||||
"github.com/pingcap/tidb/mysql"
|
||||
"github.com/pingcap/tidb/sessionctx/binloginfo"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/table"
|
||||
"github.com/pingcap/tidb/tablecodec"
|
||||
"github.com/pingcap/tidb/types"
|
||||
@ -254,7 +255,7 @@ func (t *Table) UpdateRecord(ctx context.Context, h int64, oldData, newData []ty
|
||||
}
|
||||
|
||||
key := t.RecordKey(h)
|
||||
value, err := tablecodec.EncodeRow(row, colIDs, ctx.GetSessionVars().GetTimeZone())
|
||||
value, err := tablecodec.EncodeRow(ctx.GetSessionVars().StmtCtx, row, colIDs)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
@ -283,7 +284,7 @@ func (t *Table) rebuildIndices(ctx context.Context, rm kv.RetrieverMutator, h in
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
if err = t.removeRowIndex(rm, h, oldVs, idx); err != nil {
|
||||
if err = t.removeRowIndex(ctx.GetSessionVars().StmtCtx, rm, h, oldVs, idx); err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
break
|
||||
@ -361,7 +362,7 @@ func (t *Table) AddRecord(ctx context.Context, r []types.Datum, skipHandleCheck
|
||||
}
|
||||
|
||||
key := t.RecordKey(recordID)
|
||||
value, err := tablecodec.EncodeRow(row, colIDs, ctx.GetSessionVars().GetTimeZone())
|
||||
value, err := tablecodec.EncodeRow(ctx.GetSessionVars().StmtCtx, row, colIDs)
|
||||
if err != nil {
|
||||
return 0, errors.Trace(err)
|
||||
}
|
||||
@ -521,11 +522,11 @@ func (t *Table) RemoveRecord(ctx context.Context, h int64, r []types.Datum) erro
|
||||
|
||||
func (t *Table) addInsertBinlog(ctx context.Context, h int64, row []types.Datum, colIDs []int64) error {
|
||||
mutation := t.getMutation(ctx)
|
||||
pk, err := codec.EncodeValue(nil, types.NewIntDatum(h))
|
||||
pk, err := codec.EncodeValue(ctx.GetSessionVars().StmtCtx, nil, types.NewIntDatum(h))
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
value, err := tablecodec.EncodeRow(row, colIDs, ctx.GetSessionVars().GetTimeZone())
|
||||
value, err := tablecodec.EncodeRow(ctx.GetSessionVars().StmtCtx, row, colIDs)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
@ -536,11 +537,11 @@ func (t *Table) addInsertBinlog(ctx context.Context, h int64, row []types.Datum,
|
||||
}
|
||||
|
||||
func (t *Table) addUpdateBinlog(ctx context.Context, oldRow, newRow []types.Datum, colIDs []int64) error {
|
||||
old, err := tablecodec.EncodeRow(oldRow, colIDs, ctx.GetSessionVars().GetTimeZone())
|
||||
old, err := tablecodec.EncodeRow(ctx.GetSessionVars().StmtCtx, oldRow, colIDs)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
newVal, err := tablecodec.EncodeRow(newRow, colIDs, ctx.GetSessionVars().GetTimeZone())
|
||||
newVal, err := tablecodec.EncodeRow(ctx.GetSessionVars().StmtCtx, newRow, colIDs)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
@ -552,7 +553,7 @@ func (t *Table) addUpdateBinlog(ctx context.Context, oldRow, newRow []types.Datu
|
||||
}
|
||||
|
||||
func (t *Table) addDeleteBinlog(ctx context.Context, r []types.Datum, colIDs []int64) error {
|
||||
data, err := tablecodec.EncodeRow(r, colIDs, ctx.GetSessionVars().GetTimeZone())
|
||||
data, err := tablecodec.EncodeRow(ctx.GetSessionVars().StmtCtx, r, colIDs)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
@ -579,7 +580,7 @@ func (t *Table) removeRowIndices(ctx context.Context, h int64, rec []types.Datum
|
||||
log.Infof("remove row index %v failed %v, txn %d, handle %d, data %v", v.Meta(), err, ctx.Txn().StartTS, h, rec)
|
||||
return errors.Trace(err)
|
||||
}
|
||||
if err = v.Delete(ctx.Txn(), vals, h); err != nil {
|
||||
if err = v.Delete(ctx.GetSessionVars().StmtCtx, ctx.Txn(), vals, h); err != nil {
|
||||
if v.Meta().State != model.StatePublic && kv.ErrNotExist.Equal(err) {
|
||||
// If the index is not in public state, we may have not created the index,
|
||||
// or already deleted the index, so skip ErrNotExist error.
|
||||
@ -593,8 +594,8 @@ func (t *Table) removeRowIndices(ctx context.Context, h int64, rec []types.Datum
|
||||
}
|
||||
|
||||
// removeRowIndex implements table.Table RemoveRowIndex interface.
|
||||
func (t *Table) removeRowIndex(rm kv.RetrieverMutator, h int64, vals []types.Datum, idx table.Index) error {
|
||||
if err := idx.Delete(rm, vals, h); err != nil {
|
||||
func (t *Table) removeRowIndex(sc *stmtctx.StatementContext, rm kv.RetrieverMutator, h int64, vals []types.Datum, idx table.Index) error {
|
||||
if err := idx.Delete(sc, rm, vals, h); err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
return nil
|
||||
|
||||
@ -22,6 +22,7 @@ import (
|
||||
"github.com/juju/errors"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/mysql"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/terror"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/codec"
|
||||
@ -157,18 +158,18 @@ func DecodeRowKey(key kv.Key) (int64, error) {
|
||||
}
|
||||
|
||||
// EncodeValue encodes a go value to bytes.
|
||||
func EncodeValue(raw types.Datum, loc *time.Location) ([]byte, error) {
|
||||
v, err := flatten(raw, loc)
|
||||
func EncodeValue(sc *stmtctx.StatementContext, raw types.Datum) ([]byte, error) {
|
||||
v, err := flatten(raw, sc.TimeZone)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
b, err := codec.EncodeValue(nil, v)
|
||||
b, err := codec.EncodeValue(sc, nil, v)
|
||||
return b, errors.Trace(err)
|
||||
}
|
||||
|
||||
// EncodeRow encode row data and column ids into a slice of byte.
|
||||
// Row layout: colID1, value1, colID2, value2, .....
|
||||
func EncodeRow(row []types.Datum, colIDs []int64, loc *time.Location) ([]byte, error) {
|
||||
func EncodeRow(sc *stmtctx.StatementContext, row []types.Datum, colIDs []int64) ([]byte, error) {
|
||||
if len(row) != len(colIDs) {
|
||||
return nil, errors.Errorf("EncodeRow error: data and columnID count not match %d vs %d", len(row), len(colIDs))
|
||||
}
|
||||
@ -177,7 +178,7 @@ func EncodeRow(row []types.Datum, colIDs []int64, loc *time.Location) ([]byte, e
|
||||
id := colIDs[i]
|
||||
idv := types.NewIntDatum(id)
|
||||
values[2*i] = idv
|
||||
fc, err := flatten(c, loc)
|
||||
fc, err := flatten(c, sc.TimeZone)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
@ -187,7 +188,7 @@ func EncodeRow(row []types.Datum, colIDs []int64, loc *time.Location) ([]byte, e
|
||||
// We could not set nil value into kv.
|
||||
return []byte{codec.NilFlag}, nil
|
||||
}
|
||||
return codec.EncodeValue(nil, values...)
|
||||
return codec.EncodeValue(sc, nil, values...)
|
||||
}
|
||||
|
||||
func flatten(data types.Datum, loc *time.Location) (types.Datum, error) {
|
||||
|
||||
@ -72,7 +72,8 @@ func (s *testTableCodecSuite) TestRowCodec(c *C) {
|
||||
for _, col := range cols {
|
||||
colIDs = append(colIDs, col.id)
|
||||
}
|
||||
bs, err := EncodeRow(row, colIDs, time.Local)
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
bs, err := EncodeRow(sc, row, colIDs)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(bs, NotNil)
|
||||
|
||||
@ -85,7 +86,6 @@ func (s *testTableCodecSuite) TestRowCodec(c *C) {
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(r, NotNil)
|
||||
c.Assert(r, HasLen, 3)
|
||||
sc := new(stmtctx.StatementContext)
|
||||
// Compare decoded row and original row
|
||||
for i, col := range cols {
|
||||
v, ok := r[col.id]
|
||||
@ -128,7 +128,7 @@ func (s *testTableCodecSuite) TestRowCodec(c *C) {
|
||||
}
|
||||
|
||||
// Make sure empty row return not nil value.
|
||||
bs, err = EncodeRow([]types.Datum{}, []int64{}, time.Local)
|
||||
bs, err = EncodeRow(sc, []types.Datum{}, []int64{})
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(bs, HasLen, 1)
|
||||
|
||||
@ -162,7 +162,8 @@ func (s *testTableCodecSuite) TestTimeCodec(c *C) {
|
||||
for _, col := range cols {
|
||||
colIDs = append(colIDs, col.id)
|
||||
}
|
||||
bs, err := EncodeRow(row, colIDs, time.Local)
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
bs, err := EncodeRow(sc, row, colIDs)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(bs, NotNil)
|
||||
|
||||
@ -175,7 +176,6 @@ func (s *testTableCodecSuite) TestTimeCodec(c *C) {
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(r, NotNil)
|
||||
c.Assert(r, HasLen, colLen)
|
||||
sc := new(stmtctx.StatementContext)
|
||||
// Compare decoded row and original row
|
||||
for i, col := range cols {
|
||||
v, ok := r[col.id]
|
||||
@ -200,19 +200,20 @@ func (s *testTableCodecSuite) TestCutRow(c *C) {
|
||||
row[1] = types.NewBytesDatum([]byte("abc"))
|
||||
row[2] = types.NewDecimalDatum(types.NewDecFromInt(1))
|
||||
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
data := make([][]byte, 3)
|
||||
data[0], err = EncodeValue(row[0], time.Local)
|
||||
data[0], err = EncodeValue(sc, row[0])
|
||||
c.Assert(err, IsNil)
|
||||
data[1], err = EncodeValue(row[1], time.Local)
|
||||
data[1], err = EncodeValue(sc, row[1])
|
||||
c.Assert(err, IsNil)
|
||||
data[2], err = EncodeValue(row[2], time.Local)
|
||||
data[2], err = EncodeValue(sc, row[2])
|
||||
c.Assert(err, IsNil)
|
||||
// Encode
|
||||
colIDs := make([]int64, 0, 3)
|
||||
for _, col := range cols {
|
||||
colIDs = append(colIDs, col.id)
|
||||
}
|
||||
bs, err := EncodeRow(row, colIDs, time.Local)
|
||||
bs, err := EncodeRow(sc, row, colIDs)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(bs, NotNil)
|
||||
|
||||
@ -235,7 +236,8 @@ func (s *testTableCodecSuite) TestCutKeyNew(c *C) {
|
||||
values := []types.Datum{types.NewIntDatum(1), types.NewBytesDatum([]byte("abc")), types.NewFloat64Datum(5.5)}
|
||||
handle := types.NewIntDatum(100)
|
||||
values = append(values, handle)
|
||||
encodedValue, err := codec.EncodeKey(nil, values...)
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
encodedValue, err := codec.EncodeKey(sc, nil, values...)
|
||||
c.Assert(err, IsNil)
|
||||
tableID := int64(4)
|
||||
indexID := int64(5)
|
||||
@ -257,7 +259,8 @@ func (s *testTableCodecSuite) TestCutKey(c *C) {
|
||||
values := []types.Datum{types.NewIntDatum(1), types.NewBytesDatum([]byte("abc")), types.NewFloat64Datum(5.5)}
|
||||
handle := types.NewIntDatum(100)
|
||||
values = append(values, handle)
|
||||
encodedValue, err := codec.EncodeKey(nil, values...)
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
encodedValue, err := codec.EncodeKey(sc, nil, values...)
|
||||
c.Assert(err, IsNil)
|
||||
tableID := int64(4)
|
||||
indexID := int64(5)
|
||||
|
||||
@ -23,6 +23,7 @@ import (
|
||||
"github.com/pingcap/tidb/meta"
|
||||
"github.com/pingcap/tidb/model"
|
||||
"github.com/pingcap/tidb/mysql"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/table"
|
||||
"github.com/pingcap/tidb/tablecodec"
|
||||
"github.com/pingcap/tidb/terror"
|
||||
@ -168,8 +169,8 @@ type RecordData struct {
|
||||
|
||||
// GetIndexRecordsCount returns the total number of the index records from startVals.
|
||||
// If startVals = nil, returns the total number of the index records.
|
||||
func GetIndexRecordsCount(txn kv.Transaction, kvIndex table.Index, startVals []types.Datum) (int64, error) {
|
||||
it, _, err := kvIndex.Seek(txn, startVals)
|
||||
func GetIndexRecordsCount(sc *stmtctx.StatementContext, txn kv.Transaction, kvIndex table.Index, startVals []types.Datum) (int64, error) {
|
||||
it, _, err := kvIndex.Seek(sc, txn, startVals)
|
||||
if err != nil {
|
||||
return 0, errors.Trace(err)
|
||||
}
|
||||
@ -193,9 +194,9 @@ func GetIndexRecordsCount(txn kv.Transaction, kvIndex table.Index, startVals []t
|
||||
// It returns data and the next startVals until it doesn't have data, then returns data is nil and
|
||||
// the next startVals is the values which can't get data. If startVals = nil and limit = -1,
|
||||
// it returns the index data of the whole.
|
||||
func ScanIndexData(txn kv.Transaction, kvIndex table.Index, startVals []types.Datum, limit int64) (
|
||||
func ScanIndexData(sc *stmtctx.StatementContext, txn kv.Transaction, kvIndex table.Index, startVals []types.Datum, limit int64) (
|
||||
[]*RecordData, []types.Datum, error) {
|
||||
it, _, err := kvIndex.Seek(txn, startVals)
|
||||
it, _, err := kvIndex.Seek(sc, txn, startVals)
|
||||
if err != nil {
|
||||
return nil, nil, errors.Trace(err)
|
||||
}
|
||||
@ -228,13 +229,13 @@ func ScanIndexData(txn kv.Transaction, kvIndex table.Index, startVals []types.Da
|
||||
// CompareIndexData compares index data one by one.
|
||||
// It returns nil if the data from the index is equal to the data from the table columns,
|
||||
// otherwise it returns an error with a different set of records.
|
||||
func CompareIndexData(txn kv.Transaction, t table.Table, idx table.Index) error {
|
||||
func CompareIndexData(sc *stmtctx.StatementContext, txn kv.Transaction, t table.Table, idx table.Index) error {
|
||||
err := checkIndexAndRecord(txn, t, idx)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
return checkRecordAndIndex(txn, t, idx)
|
||||
return checkRecordAndIndex(sc, txn, t, idx)
|
||||
}
|
||||
|
||||
func checkIndexAndRecord(txn kv.Transaction, t table.Table, idx table.Index) error {
|
||||
@ -275,7 +276,7 @@ func checkIndexAndRecord(txn kv.Transaction, t table.Table, idx table.Index) err
|
||||
return nil
|
||||
}
|
||||
|
||||
func checkRecordAndIndex(txn kv.Transaction, t table.Table, idx table.Index) error {
|
||||
func checkRecordAndIndex(sc *stmtctx.StatementContext, txn kv.Transaction, t table.Table, idx table.Index) error {
|
||||
cols := make([]*table.Column, len(idx.Meta().Columns))
|
||||
for i, col := range idx.Meta().Columns {
|
||||
cols[i] = t.Cols()[col.Offset]
|
||||
@ -283,7 +284,7 @@ func checkRecordAndIndex(txn kv.Transaction, t table.Table, idx table.Index) err
|
||||
|
||||
startKey := t.RecordKey(0)
|
||||
filterFunc := func(h1 int64, vals1 []types.Datum, cols []*table.Column) (bool, error) {
|
||||
isExist, h2, err := idx.Exist(txn, vals1, h1)
|
||||
isExist, h2, err := idx.Exist(sc, txn, vals1, h1)
|
||||
if kv.ErrKeyExists.Equal(err) {
|
||||
record1 := &RecordData{Handle: h1, Values: vals1}
|
||||
record2 := &RecordData{Handle: h2, Values: vals1}
|
||||
|
||||
@ -24,6 +24,7 @@ import (
|
||||
"github.com/pingcap/tidb/meta/autoid"
|
||||
"github.com/pingcap/tidb/model"
|
||||
"github.com/pingcap/tidb/mysql"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/store/tikv"
|
||||
"github.com/pingcap/tidb/table"
|
||||
"github.com/pingcap/tidb/table/tables"
|
||||
@ -313,16 +314,17 @@ func (s *testSuite) TestScan(c *C) {
|
||||
idxRow1 := &RecordData{Handle: int64(1), Values: types.MakeDatums(int64(10))}
|
||||
idxRow2 := &RecordData{Handle: int64(2), Values: types.MakeDatums(int64(20))}
|
||||
kvIndex := tables.NewIndex(tb.Meta(), indices[0].Meta())
|
||||
idxRows, nextVals, err := ScanIndexData(txn, kvIndex, idxRow1.Values, 2)
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
idxRows, nextVals, err := ScanIndexData(sc, txn, kvIndex, idxRow1.Values, 2)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(idxRows, DeepEquals, []*RecordData{idxRow1, idxRow2})
|
||||
idxRows, nextVals, err = ScanIndexData(txn, kvIndex, idxRow1.Values, 1)
|
||||
idxRows, nextVals, err = ScanIndexData(sc, txn, kvIndex, idxRow1.Values, 1)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(idxRows, DeepEquals, []*RecordData{idxRow1})
|
||||
idxRows, nextVals, err = ScanIndexData(txn, kvIndex, nextVals, 1)
|
||||
idxRows, nextVals, err = ScanIndexData(sc, txn, kvIndex, nextVals, 1)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(idxRows, DeepEquals, []*RecordData{idxRow2})
|
||||
idxRows, nextVals, err = ScanIndexData(txn, kvIndex, nextVals, 1)
|
||||
idxRows, nextVals, err = ScanIndexData(sc, txn, kvIndex, nextVals, 1)
|
||||
c.Assert(idxRows, IsNil)
|
||||
c.Assert(nextVals, DeepEquals, types.MakeDatums(nil))
|
||||
c.Assert(err, IsNil)
|
||||
@ -396,11 +398,11 @@ func (s *testSuite) testTableData(c *C, tb table.Table, rs []*RecordData) {
|
||||
func (s *testSuite) testIndex(c *C, tb table.Table, idx table.Index) {
|
||||
txn, err := s.store.Begin()
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
err = CompareIndexData(txn, tb, idx)
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
err = CompareIndexData(sc, txn, tb, idx)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
cnt, err := GetIndexRecordsCount(txn, idx, nil)
|
||||
cnt, err := GetIndexRecordsCount(sc, txn, idx, nil)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(cnt, Equals, int64(2))
|
||||
|
||||
@ -417,7 +419,7 @@ func (s *testSuite) testIndex(c *C, tb table.Table, idx table.Index) {
|
||||
|
||||
txn, err = s.store.Begin()
|
||||
c.Assert(err, IsNil)
|
||||
err = CompareIndexData(txn, tb, idx)
|
||||
err = CompareIndexData(sc, txn, tb, idx)
|
||||
c.Assert(err, NotNil)
|
||||
record1 := &RecordData{Handle: int64(3), Values: types.MakeDatums(int64(30))}
|
||||
diffMsg := newDiffRetError("index", record1, nil)
|
||||
@ -435,7 +437,7 @@ func (s *testSuite) testIndex(c *C, tb table.Table, idx table.Index) {
|
||||
|
||||
txn, err = s.store.Begin()
|
||||
c.Assert(err, IsNil)
|
||||
err = CompareIndexData(txn, tb, idx)
|
||||
err = CompareIndexData(sc, txn, tb, idx)
|
||||
c.Assert(err, NotNil)
|
||||
record2 := &RecordData{Handle: int64(3), Values: types.MakeDatums(int64(31))}
|
||||
diffMsg = newDiffRetError("index", record1, record2)
|
||||
@ -453,7 +455,7 @@ func (s *testSuite) testIndex(c *C, tb table.Table, idx table.Index) {
|
||||
|
||||
txn, err = s.store.Begin()
|
||||
c.Assert(err, IsNil)
|
||||
err = checkRecordAndIndex(txn, tb, idx)
|
||||
err = checkRecordAndIndex(sc, txn, tb, idx)
|
||||
c.Assert(err, NotNil)
|
||||
record2 = &RecordData{Handle: int64(5), Values: types.MakeDatums(int64(30))}
|
||||
diffMsg = newDiffRetError("index", record1, record2)
|
||||
@ -471,7 +473,7 @@ func (s *testSuite) testIndex(c *C, tb table.Table, idx table.Index) {
|
||||
|
||||
txn, err = s.store.Begin()
|
||||
c.Assert(err, IsNil)
|
||||
err = CompareIndexData(txn, tb, idx)
|
||||
err = CompareIndexData(sc, txn, tb, idx)
|
||||
c.Assert(err, NotNil)
|
||||
record1 = &RecordData{Handle: int64(4), Values: types.MakeDatums(int64(40))}
|
||||
diffMsg = newDiffRetError("index", record1, nil)
|
||||
@ -480,7 +482,7 @@ func (s *testSuite) testIndex(c *C, tb table.Table, idx table.Index) {
|
||||
// set data to:
|
||||
// index data (handle, data): (1, 10), (2, 20), (3, 30)
|
||||
// table data (handle, data): (1, 10), (2, 20), (3, 30), (4, 40)
|
||||
err = idx.Delete(txn, types.MakeDatums(int64(40)), 4)
|
||||
err = idx.Delete(sc, txn, types.MakeDatums(int64(40)), 4)
|
||||
c.Assert(err, IsNil)
|
||||
key = tablecodec.EncodeRowKey(tb.Meta().ID, codec.EncodeInt(nil, 4))
|
||||
setColValue(c, txn, key, types.NewDatum(int64(40)))
|
||||
@ -489,7 +491,7 @@ func (s *testSuite) testIndex(c *C, tb table.Table, idx table.Index) {
|
||||
|
||||
txn, err = s.store.Begin()
|
||||
c.Assert(err, IsNil)
|
||||
err = CompareIndexData(txn, tb, idx)
|
||||
err = CompareIndexData(sc, txn, tb, idx)
|
||||
c.Assert(err, NotNil)
|
||||
diffMsg = newDiffRetError("index", nil, record1)
|
||||
c.Assert(err.Error(), DeepEquals, diffMsg)
|
||||
@ -498,7 +500,8 @@ func (s *testSuite) testIndex(c *C, tb table.Table, idx table.Index) {
|
||||
func setColValue(c *C, txn kv.Transaction, key kv.Key, v types.Datum) {
|
||||
row := []types.Datum{v, {}}
|
||||
colIDs := []int64{2, 3}
|
||||
value, err := tablecodec.EncodeRow(row, colIDs, time.UTC)
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
value, err := tablecodec.EncodeRow(sc, row, colIDs)
|
||||
c.Assert(err, IsNil)
|
||||
err = txn.Set(key, value)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
@ -26,7 +26,7 @@ func composeEncodedData(size int) []byte {
|
||||
for i := 0; i < size; i++ {
|
||||
values = append(values, types.NewDatum(i))
|
||||
}
|
||||
bs, _ := EncodeValue(nil, values...)
|
||||
bs, _ := EncodeValue(nil, nil, values...)
|
||||
return bs
|
||||
}
|
||||
|
||||
|
||||
@ -19,6 +19,7 @@ import (
|
||||
|
||||
"github.com/juju/errors"
|
||||
"github.com/pingcap/tidb/mysql"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/terror"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/types/json"
|
||||
@ -43,7 +44,7 @@ const (
|
||||
|
||||
// encode will encode a datum and append it to a byte slice. If comparable is true, the encoded bytes can be sorted as it's original order.
|
||||
// If hash is true, the encoded bytes can be checked equal as it's original value.
|
||||
func encode(b []byte, vals []types.Datum, comparable bool, hash bool) ([]byte, error) {
|
||||
func encode(sc *stmtctx.StatementContext, b []byte, vals []types.Datum, comparable bool, hash bool) ([]byte, error) {
|
||||
for i, length := 0, len(vals); i < length; i++ {
|
||||
switch vals[i].Kind() {
|
||||
case types.KindInt64:
|
||||
@ -69,8 +70,8 @@ func encode(b []byte, vals []types.Datum, comparable bool, hash bool) ([]byte, e
|
||||
t := vals[i].GetMysqlTime()
|
||||
// Encoding timestamp need to consider timezone.
|
||||
// If it's not in UTC, transform to UTC first.
|
||||
if t.Type == mysql.TypeTimestamp && t.TimeZone != time.UTC {
|
||||
err := t.ConvertTimeZone(t.TimeZone, time.UTC)
|
||||
if t.Type == mysql.TypeTimestamp && sc.TimeZone != time.UTC {
|
||||
err := t.ConvertTimeZone(sc.TimeZone, time.UTC)
|
||||
if err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
@ -162,20 +163,20 @@ func encodeUnsignedInt(b []byte, v uint64, comparable bool) []byte {
|
||||
// EncodeKey appends the encoded values to byte slice b, returns the appended
|
||||
// slice. It guarantees the encoded value is in ascending order for comparison.
|
||||
// For Decimal type, datum must set datum's length and frac.
|
||||
func EncodeKey(b []byte, v ...types.Datum) ([]byte, error) {
|
||||
return encode(b, v, true, false)
|
||||
func EncodeKey(sc *stmtctx.StatementContext, b []byte, v ...types.Datum) ([]byte, error) {
|
||||
return encode(sc, b, v, true, false)
|
||||
}
|
||||
|
||||
// EncodeValue appends the encoded values to byte slice b, returning the appended
|
||||
// slice. It does not guarantee the order for comparison.
|
||||
func EncodeValue(b []byte, v ...types.Datum) ([]byte, error) {
|
||||
return encode(b, v, false, false)
|
||||
func EncodeValue(sc *stmtctx.StatementContext, b []byte, v ...types.Datum) ([]byte, error) {
|
||||
return encode(sc, b, v, false, false)
|
||||
}
|
||||
|
||||
// HashValues appends the encoded values to byte slice b, returning the appended
|
||||
// slice. If two datums are equal, they will generate the same bytes.
|
||||
func HashValues(b []byte, v ...types.Datum) ([]byte, error) {
|
||||
return encode(b, v, false, true)
|
||||
func HashValues(sc *stmtctx.StatementContext, b []byte, v ...types.Datum) ([]byte, error) {
|
||||
return encode(sc, b, v, false, true)
|
||||
}
|
||||
|
||||
// Decode decodes values from a byte slice generated with EncodeKey or EncodeValue
|
||||
|
||||
@ -78,16 +78,16 @@ func (s *testCodecSuite) TestCodecKey(c *C) {
|
||||
types.MakeDatums(uint64(1), uint64(1)),
|
||||
},
|
||||
}
|
||||
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
for i, t := range table {
|
||||
comment := Commentf("%d %v", i, t)
|
||||
b, err := EncodeKey(nil, t.Input...)
|
||||
b, err := EncodeKey(sc, nil, t.Input...)
|
||||
c.Assert(err, IsNil, comment)
|
||||
args, err := Decode(b, 1)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(args, DeepEquals, t.Expect)
|
||||
|
||||
b, err = EncodeValue(nil, t.Input...)
|
||||
b, err = EncodeValue(sc, nil, t.Input...)
|
||||
c.Assert(err, IsNil)
|
||||
args, err = Decode(b, 1)
|
||||
c.Assert(err, IsNil)
|
||||
@ -193,12 +193,12 @@ func (s *testCodecSuite) TestCodecKeyCompare(c *C) {
|
||||
-1,
|
||||
},
|
||||
}
|
||||
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
for _, t := range table {
|
||||
b1, err := EncodeKey(nil, t.Left...)
|
||||
b1, err := EncodeKey(sc, nil, t.Left...)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
b2, err := EncodeKey(nil, t.Right...)
|
||||
b2, err := EncodeKey(sc, nil, t.Right...)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
c.Assert(bytes.Compare(b1, b2), Equals, t.Expect, Commentf("%v - %v - %v - %v - %v", t.Left, t.Right, b1, b2, t.Expect))
|
||||
@ -524,11 +524,11 @@ func (s *testCodecSuite) TestTime(c *C) {
|
||||
"2011-01-01 00:00:00",
|
||||
"0001-01-01 00:00:00",
|
||||
}
|
||||
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
for _, t := range tbl {
|
||||
m := types.NewDatum(parseTime(c, t))
|
||||
|
||||
b, err := EncodeKey(nil, m)
|
||||
b, err := EncodeKey(sc, nil, m)
|
||||
c.Assert(err, IsNil)
|
||||
v, err := Decode(b, 1)
|
||||
c.Assert(err, IsNil)
|
||||
@ -553,9 +553,9 @@ func (s *testCodecSuite) TestTime(c *C) {
|
||||
m1 := types.NewDatum(parseTime(c, t.Arg1))
|
||||
m2 := types.NewDatum(parseTime(c, t.Arg2))
|
||||
|
||||
b1, err := EncodeKey(nil, m1)
|
||||
b1, err := EncodeKey(sc, nil, m1)
|
||||
c.Assert(err, IsNil)
|
||||
b2, err := EncodeKey(nil, m2)
|
||||
b2, err := EncodeKey(sc, nil, m2)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
ret := bytes.Compare(b1, b2)
|
||||
@ -570,11 +570,11 @@ func (s *testCodecSuite) TestDuration(c *C) {
|
||||
"00:00:00",
|
||||
"1 11:11:11",
|
||||
}
|
||||
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
for _, t := range tbl {
|
||||
m := parseDuration(c, t)
|
||||
|
||||
b, err := EncodeKey(nil, types.NewDatum(m))
|
||||
b, err := EncodeKey(sc, nil, types.NewDatum(m))
|
||||
c.Assert(err, IsNil)
|
||||
v, err := Decode(b, 1)
|
||||
c.Assert(err, IsNil)
|
||||
@ -596,9 +596,9 @@ func (s *testCodecSuite) TestDuration(c *C) {
|
||||
m1 := parseDuration(c, t.Arg1)
|
||||
m2 := parseDuration(c, t.Arg2)
|
||||
|
||||
b1, err := EncodeKey(nil, types.NewDatum(m1))
|
||||
b1, err := EncodeKey(sc, nil, types.NewDatum(m1))
|
||||
c.Assert(err, IsNil)
|
||||
b2, err := EncodeKey(nil, types.NewDatum(m2))
|
||||
b2, err := EncodeKey(sc, nil, types.NewDatum(m2))
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
ret := bytes.Compare(b1, b2)
|
||||
@ -624,12 +624,12 @@ func (s *testCodecSuite) TestDecimal(c *C) {
|
||||
"-12.340",
|
||||
"-0.1234",
|
||||
}
|
||||
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
for _, t := range tbl {
|
||||
dec := new(types.MyDecimal)
|
||||
err := dec.FromString([]byte(t))
|
||||
c.Assert(err, IsNil)
|
||||
b, err := EncodeKey(nil, types.NewDatum(dec))
|
||||
b, err := EncodeKey(sc, nil, types.NewDatum(dec))
|
||||
c.Assert(err, IsNil)
|
||||
v, err := Decode(b, 1)
|
||||
c.Assert(err, IsNil)
|
||||
@ -703,7 +703,6 @@ func (s *testCodecSuite) TestDecimal(c *C) {
|
||||
{uint64(math.MaxUint64), uint64(0), 1},
|
||||
{uint64(0), uint64(math.MaxUint64), -1},
|
||||
}
|
||||
sc := new(stmtctx.StatementContext)
|
||||
for _, t := range tblCmp {
|
||||
d1 := types.NewDatum(t.Arg1)
|
||||
dec1, err := d1.ToDecimal(sc)
|
||||
@ -718,9 +717,9 @@ func (s *testCodecSuite) TestDecimal(c *C) {
|
||||
d1.SetFrac(6)
|
||||
d2.SetLength(30)
|
||||
d2.SetFrac(6)
|
||||
b1, err := EncodeKey(nil, d1)
|
||||
b1, err := EncodeKey(sc, nil, d1)
|
||||
c.Assert(err, IsNil)
|
||||
b2, err := EncodeKey(nil, d2)
|
||||
b2, err := EncodeKey(sc, nil, d2)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
ret := bytes.Compare(b1, b2)
|
||||
@ -761,7 +760,7 @@ func (s *testCodecSuite) TestJSON(c *C) {
|
||||
}
|
||||
|
||||
buf := make([]byte, 0, 4096)
|
||||
buf, err := encode(buf, datums, false, false)
|
||||
buf, err := encode(nil, buf, datums, false, false)
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
datums1, err := Decode(buf, 2)
|
||||
@ -822,16 +821,17 @@ func (s *testCodecSuite) TestCut(c *C) {
|
||||
types.MakeDatums(types.NewDecFromInt(0), types.NewDecFromFloatForTest(-1.3)),
|
||||
},
|
||||
}
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
for i, t := range table {
|
||||
comment := Commentf("%d %v", i, t)
|
||||
b, err := EncodeKey(nil, t.Input...)
|
||||
b, err := EncodeKey(sc, nil, t.Input...)
|
||||
c.Assert(err, IsNil, comment)
|
||||
var d []byte
|
||||
for j, e := range t.Expect {
|
||||
d, b, err = CutOne(b)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(d, NotNil)
|
||||
ed, err1 := EncodeKey(nil, e)
|
||||
ed, err1 := EncodeKey(sc, nil, e)
|
||||
c.Assert(err1, IsNil)
|
||||
c.Assert(d, DeepEquals, ed, Commentf("%d:%d %#v", i, j, e))
|
||||
}
|
||||
@ -839,14 +839,14 @@ func (s *testCodecSuite) TestCut(c *C) {
|
||||
}
|
||||
for i, t := range table {
|
||||
comment := Commentf("%d %v", i, t)
|
||||
b, err := EncodeValue(nil, t.Input...)
|
||||
b, err := EncodeValue(sc, nil, t.Input...)
|
||||
c.Assert(err, IsNil, comment)
|
||||
var d []byte
|
||||
for j, e := range t.Expect {
|
||||
d, b, err = CutOne(b)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(d, NotNil)
|
||||
ed, err1 := EncodeValue(nil, e)
|
||||
ed, err1 := EncodeValue(sc, nil, e)
|
||||
c.Assert(err1, IsNil)
|
||||
c.Assert(d, DeepEquals, ed, Commentf("%d:%d %#v", i, j, e))
|
||||
}
|
||||
@ -855,15 +855,16 @@ func (s *testCodecSuite) TestCut(c *C) {
|
||||
}
|
||||
|
||||
func (s *testCodecSuite) TestSetRawValues(c *C) {
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
datums := types.MakeDatums(1, "abc", 1.1, []byte("def"))
|
||||
rowData, err := EncodeValue(nil, datums...)
|
||||
rowData, err := EncodeValue(sc, nil, datums...)
|
||||
c.Assert(err, IsNil)
|
||||
values := make([]types.Datum, 4)
|
||||
err = SetRawValues(rowData, values)
|
||||
c.Assert(err, IsNil)
|
||||
for i, rawVal := range values {
|
||||
c.Assert(rawVal.Kind(), Equals, types.KindRaw)
|
||||
encoded, err1 := EncodeValue(nil, datums[i])
|
||||
encoded, err1 := EncodeValue(sc, nil, datums[i])
|
||||
c.Assert(err1, IsNil)
|
||||
c.Assert(encoded, BytesEquals, rawVal.GetBytes())
|
||||
}
|
||||
@ -905,6 +906,7 @@ func (s *testCodecSuite) TestDecodeOneToChunk(c *C) {
|
||||
{json.CreateBinary("abc"), types.NewFieldType(mysql.TypeJSON)},
|
||||
{int64(1), types.NewFieldType(mysql.TypeYear)},
|
||||
}
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
|
||||
datums := make([]types.Datum, 0, len(table))
|
||||
tps := make([]*types.FieldType, 0, len(table))
|
||||
@ -915,7 +917,7 @@ func (s *testCodecSuite) TestDecodeOneToChunk(c *C) {
|
||||
chk := chunk.NewChunk(tps)
|
||||
rowCount := 3
|
||||
for rowIdx := 0; rowIdx < rowCount; rowIdx++ {
|
||||
encoded, err := EncodeValue(nil, datums...)
|
||||
encoded, err := EncodeValue(sc, nil, datums...)
|
||||
c.Assert(err, IsNil)
|
||||
for colIdx, t := range table {
|
||||
encoded, err = DecodeOneToChunk(encoded, chk, colIdx, t.tp, time.Local)
|
||||
@ -923,7 +925,6 @@ func (s *testCodecSuite) TestDecodeOneToChunk(c *C) {
|
||||
}
|
||||
}
|
||||
|
||||
sc := new(stmtctx.StatementContext)
|
||||
for colIdx, t := range table {
|
||||
for rowIdx := 0; rowIdx < rowCount; rowIdx++ {
|
||||
got := chk.GetRow(rowIdx).GetDatum(colIdx, t.tp)
|
||||
|
||||
@ -581,21 +581,21 @@ func (w *Worker) flushToFile() {
|
||||
return
|
||||
}
|
||||
defer terror.Call(outputFile.Close)
|
||||
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
for _, row := range w.buf {
|
||||
prevLen = len(outputByte)
|
||||
outputByte = append(outputByte, w.head...)
|
||||
outputByte, err = codec.EncodeKey(outputByte, row.key...)
|
||||
outputByte, err = codec.EncodeKey(sc, outputByte, row.key...)
|
||||
if err != nil {
|
||||
w.err = errors.Trace(err)
|
||||
return
|
||||
}
|
||||
outputByte, err = codec.EncodeKey(outputByte, row.val...)
|
||||
outputByte, err = codec.EncodeKey(sc, outputByte, row.val...)
|
||||
if err != nil {
|
||||
w.err = errors.Trace(err)
|
||||
return
|
||||
}
|
||||
outputByte, err = codec.EncodeKey(outputByte, types.NewIntDatum(row.handle))
|
||||
outputByte, err = codec.EncodeKey(sc, outputByte, types.NewIntDatum(row.handle))
|
||||
if err != nil {
|
||||
w.err = errors.Trace(err)
|
||||
return
|
||||
|
||||
@ -18,6 +18,7 @@ import (
|
||||
"fmt"
|
||||
"strconv"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/tidb/meta/autoid"
|
||||
|
||||
@ -26,6 +27,7 @@ import (
|
||||
"github.com/pingcap/tidb"
|
||||
"github.com/pingcap/tidb/domain"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/store/tikv"
|
||||
"github.com/pingcap/tidb/structure"
|
||||
"github.com/pingcap/tidb/tablecodec"
|
||||
@ -445,12 +447,13 @@ func (s *testKvEncoderSuite) TestSimpleKeyEncode(c *C) {
|
||||
handle := int64(1)
|
||||
expectRecordKey := tablecodec.EncodeRecordKey(tablePrefix, handle)
|
||||
|
||||
sc := &stmtctx.StatementContext{TimeZone: time.Local}
|
||||
indexPrefix := tablecodec.EncodeTableIndexPrefix(tableID, indexID)
|
||||
expectIdxKey := make([]byte, 0)
|
||||
expectIdxKey = append(expectIdxKey, []byte(indexPrefix)...)
|
||||
expectIdxKey, err = codec.EncodeKey(expectIdxKey, types.NewDatum([]byte("a")))
|
||||
expectIdxKey, err = codec.EncodeKey(sc, expectIdxKey, types.NewDatum([]byte("a")))
|
||||
c.Assert(err, IsNil)
|
||||
expectIdxKey, err = codec.EncodeKey(expectIdxKey, types.NewDatum(handle))
|
||||
expectIdxKey, err = codec.EncodeKey(sc, expectIdxKey, types.NewDatum(handle))
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
for _, row := range kvPairs {
|
||||
@ -487,7 +490,7 @@ func (s *testKvEncoderSuite) TestSimpleKeyEncode(c *C) {
|
||||
indexPrefix = tablecodec.EncodeTableIndexPrefix(tableID, indexID)
|
||||
expectIdxKey = []byte{}
|
||||
expectIdxKey = append(expectIdxKey, []byte(indexPrefix)...)
|
||||
expectIdxKey, err = codec.EncodeKey(expectIdxKey, types.NewDatum([]byte("a")))
|
||||
expectIdxKey, err = codec.EncodeKey(sc, expectIdxKey, types.NewDatum([]byte("a")))
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
for _, row := range kvPairs {
|
||||
|
||||
Reference in New Issue
Block a user