From 47174427bee677e4767c4c6d6325dafce47ea775 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Sat, 26 Aug 2023 01:45:34 +0800 Subject: [PATCH 1/7] *: use std/slices to replace exp/slices Signed-off-by: Weizhen Wang --- br/pkg/lightning/checkpoints/BUILD.bazel | 1 - br/pkg/lightning/checkpoints/checkpoints.go | 20 ++++++++--------- executor/inspection_result.go | 25 ++++++++++++--------- executor/union_scan.go | 19 ++++++++-------- expression/scalar_function.go | 6 ++--- parser/goyacc/BUILD.bazel | 1 - parser/goyacc/main.go | 2 +- statistics/cmsketch.go | 1 - 8 files changed, 37 insertions(+), 38 deletions(-) diff --git a/br/pkg/lightning/checkpoints/BUILD.bazel b/br/pkg/lightning/checkpoints/BUILD.bazel index 47c56d919d743..4c114e22beb48 100644 --- a/br/pkg/lightning/checkpoints/BUILD.bazel +++ b/br/pkg/lightning/checkpoints/BUILD.bazel @@ -26,7 +26,6 @@ go_library( "//util/sqlexec", "@com_github_joho_sqltocsv//:sqltocsv", "@com_github_pingcap_errors//:errors", - "@org_golang_x_exp//slices", "@org_uber_go_zap//:zap", ], ) diff --git a/br/pkg/lightning/checkpoints/checkpoints.go b/br/pkg/lightning/checkpoints/checkpoints.go index bf198ab666862..fe02edcc5b769 100644 --- a/br/pkg/lightning/checkpoints/checkpoints.go +++ b/br/pkg/lightning/checkpoints/checkpoints.go @@ -15,6 +15,7 @@ package checkpoints import ( + "cmp" "context" "database/sql" "encoding/json" @@ -22,6 +23,7 @@ import ( "io" "math" "path" + "slices" "sort" "strings" "sync" @@ -39,7 +41,6 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/mathutil" "go.uber.org/zap" - "golang.org/x/exp/slices" ) // CheckpointStatus is the status of a checkpoint. @@ -246,15 +247,11 @@ func (key *ChunkCheckpointKey) String() string { return fmt.Sprintf("%s:%d", key.Path, key.Offset) } -func (key *ChunkCheckpointKey) less(other *ChunkCheckpointKey) bool { - switch { - case key.Path < other.Path: - return true - case key.Path > other.Path: - return false - default: - return key.Offset < other.Offset +func (key *ChunkCheckpointKey) less(other *ChunkCheckpointKey) int { + if c := cmp.Compare(key.Path, other.Path); c != 0 { + return c } + return cmp.Compare(key.Offset, other.Offset) } // ChunkCheckpoint is the checkpoint for a chunk. @@ -440,8 +437,9 @@ func (cp *TableCheckpoint) Apply(cpd *TableCheckpointDiff) { for key, diff := range engineDiff.chunks { checkpointKey := key index := sort.Search(len(engine.Chunks), func(i int) bool { - return !engine.Chunks[i].Key.less(&checkpointKey) + return engine.Chunks[i].Key.less(&checkpointKey) > 0 }) + if index >= len(engine.Chunks) { continue } @@ -1351,7 +1349,7 @@ func (cpdb *FileCheckpointsDB) Get(_ context.Context, tableName string) (*TableC }) } - slices.SortFunc(engine.Chunks, func(i, j *ChunkCheckpoint) bool { + slices.SortFunc(engine.Chunks, func(i, j *ChunkCheckpoint) int { return i.Key.less(&j.Key) }) diff --git a/executor/inspection_result.go b/executor/inspection_result.go index a120bd22897ee..912263c3570f0 100644 --- a/executor/inspection_result.go +++ b/executor/inspection_result.go @@ -15,9 +15,11 @@ package executor import ( + "cmp" "context" "fmt" "math" + "slices" "strconv" "strings" @@ -34,7 +36,6 @@ import ( "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/size" "github.com/pingcap/tidb/util/sqlexec" - "golang.org/x/exp/slices" ) type ( @@ -169,20 +170,22 @@ func (e *inspectionResultRetriever) retrieve(ctx context.Context, sctx sessionct continue } // make result stable - slices.SortFunc(results, func(i, j inspectionResult) bool { - if i.degree != j.degree { - return i.degree > j.degree + slices.SortFunc(results, func(i, j inspectionResult) int { + if c := cmp.Compare(i.degree, j.degree); c != 0 { + return c } - if lhs, rhs := i.item, j.item; lhs != rhs { - return lhs < rhs + // lhs and rhs + if c := cmp.Compare(i.item, j.item); c != 0 { + return c } - if i.actual != j.actual { - return i.actual < j.actual + if c := cmp.Compare(i.actual, j.actual); c != 0 { + return c } - if lhs, rhs := i.tp, j.tp; lhs != rhs { - return lhs < rhs + // lhs and rhs + if c := cmp.Compare(i.tp, j.tp); c != 0 { + return c } - return i.instance < j.instance + return cmp.Compare(i.instance, j.instance) }) for _, result := range results { if len(result.instance) == 0 { diff --git a/executor/union_scan.go b/executor/union_scan.go index b6c8f3fe0f0e6..e065af8375a8a 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -209,10 +209,11 @@ func (us *UnionScanExec) getOneRow(ctx context.Context) ([]types.Datum, error) { } else if snapshotRow == nil { row = addedRow } else { - isSnapshotRow, err = us.compare(us.Ctx().GetSessionVars().StmtCtx, snapshotRow, addedRow) + isSnapshotRowInt, err := us.compare(us.Ctx().GetSessionVars().StmtCtx, snapshotRow, addedRow) if err != nil { return nil, err } + isSnapshotRow = isSnapshotRowInt < 0 if isSnapshotRow { row = snapshotRow } else { @@ -292,26 +293,26 @@ type compareExec struct { handleCols plannercore.HandleCols } -func (ce compareExec) compare(sctx *stmtctx.StatementContext, a, b []types.Datum) (ret bool, err error) { +func (ce compareExec) compare(sctx *stmtctx.StatementContext, a, b []types.Datum) (ret int, err error) { var cmp int for _, colOff := range ce.usedIndex { aColumn := a[colOff] bColumn := b[colOff] cmp, err = aColumn.Compare(sctx, &bColumn, ce.collators[colOff]) if err != nil { - return false, err + return 0, err } if cmp == 0 { continue } - if cmp > 0 && !ce.desc || cmp < 0 && ce.desc { - return false, nil + if ce.desc { + return -cmp, nil } - return true, nil + return cmp, nil } cmp, err = ce.handleCols.Compare(a, b, ce.collators) - if cmp > 0 && !ce.desc || cmp < 0 && ce.desc { - return false, err + if ce.desc { + return -cmp, err } - return true, err + return cmp, err } diff --git a/expression/scalar_function.go b/expression/scalar_function.go index 8564341a1e7f3..a3ce92059cfb0 100644 --- a/expression/scalar_function.go +++ b/expression/scalar_function.go @@ -17,7 +17,7 @@ package expression import ( "bytes" "fmt" - "sort" + "slices" "unsafe" "github.com/pingcap/errors" @@ -515,8 +515,8 @@ func simpleCanonicalizedHashCode(sf *ScalarFunction, sc *stmtctx.StatementContex // encode original function name. sf.canonicalhashcode = codec.EncodeCompactBytes(sf.canonicalhashcode, hack.Slice(sf.FuncName.L)) // reorder parameters hashcode, eg: a+b and b+a should has the same hashcode here. - sort.Slice(argsHashCode, func(i, j int) bool { - return bytes.Compare(argsHashCode[i], argsHashCode[j]) <= 0 + slices.SortFunc(argsHashCode, func(i, j []byte) int { + return bytes.Compare(i, j) }) for _, argCode := range argsHashCode { sf.canonicalhashcode = append(sf.canonicalhashcode, argCode...) diff --git a/parser/goyacc/BUILD.bazel b/parser/goyacc/BUILD.bazel index e8582c1c4a2ab..51d0991ed1b0c 100644 --- a/parser/goyacc/BUILD.bazel +++ b/parser/goyacc/BUILD.bazel @@ -14,7 +14,6 @@ go_library( "@com_github_cznic_sortutil//:sortutil", "@com_github_cznic_strutil//:strutil", "@com_github_pingcap_errors//:errors", - "@org_golang_x_exp//slices", "@org_modernc_parser//yacc", "@org_modernc_y//:y", ], diff --git a/parser/goyacc/main.go b/parser/goyacc/main.go index 9c587896a2af6..f691f4ecf418c 100644 --- a/parser/goyacc/main.go +++ b/parser/goyacc/main.go @@ -135,13 +135,13 @@ import ( "log" "os" "runtime" + "slices" "sort" "strings" "github.com/cznic/mathutil" "github.com/cznic/sortutil" "github.com/cznic/strutil" - "golang.org/x/exp/slices" parser "modernc.org/parser/yacc" "modernc.org/y" ) diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index c71111e447867..1c343db792e47 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -100,7 +100,6 @@ func newTopNHelper(sample [][]byte, numTop uint32) *topNHelper { } } sort.SliceStable(sorted, func(i, j int) bool { return sorted[i].cnt > sorted[j].cnt }) - failpoint.Inject("StabilizeV1AnalyzeTopN", func(val failpoint.Value) { if val.(bool) { // The earlier TopN entry will modify the CMSketch, therefore influence later TopN entry's row count. From faa6cc2894c072b89197405ff1d82b1602ae0a09 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Sat, 26 Aug 2023 01:51:53 +0800 Subject: [PATCH 2/7] update Signed-off-by: Weizhen Wang --- executor/mem_reader.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/executor/mem_reader.go b/executor/mem_reader.go index 8f2b56c5aad1f..964767c0b9460 100644 --- a/executor/mem_reader.go +++ b/executor/mem_reader.go @@ -17,6 +17,8 @@ package executor import ( "context" + "slices" + "github.com/pingcap/errors" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/expression" @@ -33,7 +35,6 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/tracing" - "golang.org/x/exp/slices" ) type memReader interface { @@ -143,7 +144,7 @@ func (m *memIndexReader) getMemRows(ctx context.Context) ([][]types.Datum, error } if m.keepOrder && m.table.GetPartitionInfo() != nil { - slices.SortFunc(m.addedRows, func(a, b []types.Datum) bool { + slices.SortFunc(m.addedRows, func(a, b []types.Datum) int { ret, err1 := m.compare(m.ctx.GetSessionVars().StmtCtx, a, b) if err1 != nil { err = err1 @@ -421,7 +422,7 @@ func (m *memTableReader) getMemRows(ctx context.Context) ([][]types.Datum, error } if m.keepOrder && m.table.GetPartitionInfo() != nil { - slices.SortFunc(m.addedRows, func(a, b []types.Datum) bool { + slices.SortFunc(m.addedRows, func(a, b []types.Datum) int { ret, err1 := m.compare(m.ctx.GetSessionVars().StmtCtx, a, b) if err1 != nil { err = err1 @@ -935,7 +936,7 @@ func (m *memIndexMergeReader) getMemRows(ctx context.Context) ([][]types.Datum, // Didn't set keepOrder = true for memTblReader, // In indexMerge, non-partitioned tables are also need reordered. if m.keepOrder { - slices.SortFunc(rows, func(a, b []types.Datum) bool { + slices.SortFunc(rows, func(a, b []types.Datum) int { ret, err1 := m.compare(m.ctx.GetSessionVars().StmtCtx, a, b) if err1 != nil { err = err1 From c28b750b27ad220142163221b3db349ef4e88cd4 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Sat, 26 Aug 2023 08:53:14 +0800 Subject: [PATCH 3/7] update Signed-off-by: Weizhen Wang --- executor/mem_reader.go | 1 - 1 file changed, 1 deletion(-) diff --git a/executor/mem_reader.go b/executor/mem_reader.go index 964767c0b9460..8bb7bf636064d 100644 --- a/executor/mem_reader.go +++ b/executor/mem_reader.go @@ -16,7 +16,6 @@ package executor import ( "context" - "slices" "github.com/pingcap/errors" From d23bce977b19f2793699f30b423f7d74dedf69e4 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Sat, 26 Aug 2023 08:58:57 +0800 Subject: [PATCH 4/7] update Signed-off-by: Weizhen Wang --- executor/BUILD.bazel | 1 - 1 file changed, 1 deletion(-) diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index 1296e90df8685..d0025b488579f 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -273,7 +273,6 @@ go_library( "@org_golang_google_grpc//credentials", "@org_golang_google_grpc//credentials/insecure", "@org_golang_google_grpc//status", - "@org_golang_x_exp//slices", "@org_golang_x_sync//errgroup", "@org_uber_go_atomic//:atomic", "@org_uber_go_zap//:zap", From c9f1872eb7f4e315c6fcbb26509dc129599965b4 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Sat, 26 Aug 2023 10:53:23 +0800 Subject: [PATCH 5/7] update Signed-off-by: Weizhen Wang --- br/pkg/lightning/checkpoints/checkpoints.go | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/br/pkg/lightning/checkpoints/checkpoints.go b/br/pkg/lightning/checkpoints/checkpoints.go index fe02edcc5b769..865cb6d4e759c 100644 --- a/br/pkg/lightning/checkpoints/checkpoints.go +++ b/br/pkg/lightning/checkpoints/checkpoints.go @@ -247,13 +247,24 @@ func (key *ChunkCheckpointKey) String() string { return fmt.Sprintf("%s:%d", key.Path, key.Offset) } -func (key *ChunkCheckpointKey) less(other *ChunkCheckpointKey) int { +func (key *ChunkCheckpointKey) compare(other *ChunkCheckpointKey) int { if c := cmp.Compare(key.Path, other.Path); c != 0 { return c } return cmp.Compare(key.Offset, other.Offset) } +func (key *ChunkCheckpointKey) less(other *ChunkCheckpointKey) bool { + switch { + case key.Path < other.Path: + return true + case key.Path > other.Path: + return false + default: + return key.Offset < other.Offset + } +} + // ChunkCheckpoint is the checkpoint for a chunk. type ChunkCheckpoint struct { Key ChunkCheckpointKey @@ -437,9 +448,8 @@ func (cp *TableCheckpoint) Apply(cpd *TableCheckpointDiff) { for key, diff := range engineDiff.chunks { checkpointKey := key index := sort.Search(len(engine.Chunks), func(i int) bool { - return engine.Chunks[i].Key.less(&checkpointKey) > 0 + return !engine.Chunks[i].Key.less(&checkpointKey) }) - if index >= len(engine.Chunks) { continue } @@ -1350,7 +1360,7 @@ func (cpdb *FileCheckpointsDB) Get(_ context.Context, tableName string) (*TableC } slices.SortFunc(engine.Chunks, func(i, j *ChunkCheckpoint) int { - return i.Key.less(&j.Key) + return i.Key.compare(&j.Key) }) cp.Engines[engineID] = engine From 42c309826690090af49bc6989bdec3354d6953ea Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Sat, 26 Aug 2023 12:15:47 +0800 Subject: [PATCH 6/7] update Signed-off-by: Weizhen Wang --- executor/inspection_result.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/inspection_result.go b/executor/inspection_result.go index 912263c3570f0..a0dbdb4b9afa3 100644 --- a/executor/inspection_result.go +++ b/executor/inspection_result.go @@ -172,7 +172,7 @@ func (e *inspectionResultRetriever) retrieve(ctx context.Context, sctx sessionct // make result stable slices.SortFunc(results, func(i, j inspectionResult) int { if c := cmp.Compare(i.degree, j.degree); c != 0 { - return c + return -c } // lhs and rhs if c := cmp.Compare(i.item, j.item); c != 0 { From 894486680e7b5d51c5dcc594381e366f4623774c Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Sat, 26 Aug 2023 19:31:30 +0800 Subject: [PATCH 7/7] update Signed-off-by: Weizhen Wang --- statistics/cmsketch.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index 1c343db792e47..508a6d90fac8e 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -37,7 +37,6 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/hack" - "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tipb/go-tipb" "github.com/twmb/murmur3" ) @@ -143,7 +142,7 @@ func NewCMSketchAndTopN(d, w int32, sample [][]byte, numTop uint32, rowCount uin helper := newTopNHelper(sample, numTop) // rowCount is not a accurate value when fast analyzing // In some cases, if user triggers fast analyze when rowCount is close to sampleSize, unexpected bahavior might happen. - rowCount = mathutil.Max(rowCount, uint64(len(sample))) + rowCount = max(rowCount, uint64(len(sample))) estimateNDV, scaleRatio := calculateEstimateNDV(helper, rowCount) defaultVal := calculateDefaultVal(helper, estimateNDV, scaleRatio, rowCount) c, t := buildCMSAndTopN(helper, d, w, scaleRatio, defaultVal) @@ -182,7 +181,7 @@ func calculateDefaultVal(helper *topNHelper, estimateNDV, scaleRatio, rowCount u return 1 } estimateRemainingCount := rowCount - (helper.sampleSize-helper.onlyOnceItems)*scaleRatio - return estimateRemainingCount / mathutil.Max(1, estimateNDV-sampleNDV+helper.onlyOnceItems) + return estimateRemainingCount / max(1, estimateNDV-sampleNDV+helper.onlyOnceItems) } // MemoryUsage returns the total memory usage of a CMSketch. @@ -386,7 +385,7 @@ func (c *CMSketch) MergeCMSketch4IncrementalAnalyze(rc *CMSketch, _ uint32) erro for i := range c.table { c.count = 0 for j := range c.table[i] { - c.table[i][j] = mathutil.Max(c.table[i][j], rc.table[i][j]) + c.table[i][j] = max(c.table[i][j], rc.table[i][j]) c.count += uint64(c.table[i][j]) } } @@ -521,7 +520,7 @@ func (c *CMSketch) GetWidthAndDepth() (width, depth int32) { // CalcDefaultValForAnalyze calculate the default value for Analyze. // The value of it is count / NDV in CMSketch. This means count and NDV are not include topN. func (c *CMSketch) CalcDefaultValForAnalyze(ndv uint64) { - c.defaultValue = c.count / mathutil.Max(1, ndv) + c.defaultValue = c.count / max(1, ndv) } // TopN stores most-common values, which is used to estimate point queries.