From 31adb084a4f54a964c21675a85a94d17141e65ae Mon Sep 17 00:00:00 2001 From: zyguan Date: Tue, 21 Mar 2023 00:35:32 +0800 Subject: [PATCH 1/3] txn: make kv pairs converted from lock invisible Signed-off-by: zyguan --- executor/batch_point_get.go | 20 ++-- executor/point_get.go | 10 +- store/driver/txn/txn_driver.go | 5 +- store/driver/txn/unionstore_driver.go | 96 +++++++++++++++++-- .../pessimistictest/pessimistic_test.go | 36 ++++++- 5 files changed, 142 insertions(+), 25 deletions(-) diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 0ce745d172e4a..939884c01b81e 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -450,15 +450,17 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { if !e.txn.Valid() { return kv.ErrInvalidTxn } - membuf := e.txn.GetMemBuffer() - for _, idxKey := range indexKeys { - handleVal := handleVals[string(idxKey)] - if len(handleVal) == 0 { - continue - } - err = membuf.Set(idxKey, handleVal) - if err != nil { - return err + membuf, ok := e.txn.GetMemBuffer().(interface{ ChangeLockIntoPut(kv.Key, []byte) error }) + if ok { + for _, idxKey := range indexKeys { + handleVal := handleVals[string(idxKey)] + if len(handleVal) == 0 { + continue + } + err = membuf.ChangeLockIntoPut(idxKey, handleVal) + if err != nil { + return err + } } } } diff --git a/executor/point_get.go b/executor/point_get.go index f33ba20b5dd5a..9a4a858a4fbd8 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -270,10 +270,12 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { if !e.txn.Valid() { return kv.ErrInvalidTxn } - memBuffer := e.txn.GetMemBuffer() - err = memBuffer.Set(e.idxKey, e.handleVal) - if err != nil { - return err + membuf, ok := e.txn.GetMemBuffer().(interface{ ChangeLockIntoPut(kv.Key, []byte) error }) + if ok { + err = membuf.ChangeLockIntoPut(e.idxKey, e.handleVal) + if err != nil { + return err + } } } } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index fc127be0013f2..20500e24feb34 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -41,6 +41,7 @@ import ( type tikvTxn struct { *tikv.KVTxn idxNameCache map[int64]*model.TableInfo + invisibleKeys map[string]struct{} snapshotInterceptor kv.SnapshotInterceptor // columnMapsCache is a cache used for the mutation checker columnMapsCache interface{} @@ -54,7 +55,7 @@ func NewTiKVTxn(txn *tikv.KVTxn) kv.Transaction { totalLimit := atomic.LoadUint64(&kv.TxnTotalSizeLimit) txn.GetUnionStore().SetEntrySizeLimit(entryLimit, totalLimit) - return &tikvTxn{txn, make(map[int64]*model.TableInfo), nil, nil} + return &tikvTxn{txn, make(map[int64]*model.TableInfo), make(map[string]struct{}), nil, nil} } func (txn *tikvTxn) GetTableInfo(id int64) *model.TableInfo { @@ -171,7 +172,7 @@ func (txn *tikvTxn) Set(k kv.Key, v []byte) error { } func (txn *tikvTxn) GetMemBuffer() kv.MemBuffer { - return newMemBuffer(txn.KVTxn.GetMemBuffer()) + return newMemBuffer(txn.KVTxn.GetMemBuffer(), txn.invisibleKeys) } func (txn *tikvTxn) SetOption(opt int, val interface{}) { diff --git a/store/driver/txn/unionstore_driver.go b/store/driver/txn/unionstore_driver.go index 6b2dcc283213e..ace3665cc7c83 100644 --- a/store/driver/txn/unionstore_driver.go +++ b/store/driver/txn/unionstore_driver.go @@ -19,20 +19,45 @@ import ( "github.com/pingcap/tidb/kv" derr "github.com/pingcap/tidb/store/driver/error" + tikverr "github.com/tikv/client-go/v2/error" tikvstore "github.com/tikv/client-go/v2/kv" "github.com/tikv/client-go/v2/tikv" + "github.com/tikv/client-go/v2/util" ) +type visibilityChecker interface { + isVisible(kv.Key) bool +} + // memBuffer wraps tikv.MemDB as kv.MemBuffer. type memBuffer struct { *tikv.MemDB + invisibleKeys map[string]struct{} } -func newMemBuffer(m *tikv.MemDB) kv.MemBuffer { +func newMemBuffer(m *tikv.MemDB, invisibleKeys map[string]struct{}) kv.MemBuffer { if m == nil { return nil } - return &memBuffer{MemDB: m} + return &memBuffer{MemDB: m, invisibleKeys: invisibleKeys} +} + +func (m *memBuffer) addInvisibleKey(k kv.Key) { + m.Lock() + m.invisibleKeys[util.String(k)] = struct{}{} + m.Unlock() +} + +func (m *memBuffer) delInvisibleKey(k kv.Key) { + m.Lock() + delete(m.invisibleKeys, util.String(k)) + m.Unlock() +} + +func (m *memBuffer) isVisible(k kv.Key) bool { + // shall be protected by MemBuffer.RLock + _, ok := m.invisibleKeys[util.String(k)] + return !ok } func (m *memBuffer) Size() int { @@ -40,20 +65,27 @@ func (m *memBuffer) Size() int { } func (m *memBuffer) Delete(k kv.Key) error { - return m.MemDB.Delete(k) + err := m.MemDB.Delete(k) + m.delInvisibleKey(k) + return derr.ToTiDBErr(err) } func (m *memBuffer) DeleteWithFlags(k kv.Key, ops ...kv.FlagsOp) error { err := m.MemDB.DeleteWithFlags(k, getTiKVFlagsOps(ops)...) + m.delInvisibleKey(k) return derr.ToTiDBErr(err) } func (m *memBuffer) Get(_ context.Context, key kv.Key) ([]byte, error) { + if !m.isVisible(key) { + return nil, kv.ErrNotExist + } data, err := m.MemDB.Get(key) return data, derr.ToTiDBErr(err) } func (m *memBuffer) GetFlags(key kv.Key) (kv.KeyFlags, error) { + // do not check `invisibleKeys` here since LockKeys may set flags on keys and those flags are always visible. data, err := m.MemDB.GetFlags(key) return getTiDBKeyFlags(data), derr.ToTiDBErr(err) } @@ -79,21 +111,32 @@ func (m *memBuffer) InspectStage(handle kv.StagingHandle, f func(kv.Key, kv.KeyF func (m *memBuffer) Set(key kv.Key, value []byte) error { err := m.MemDB.Set(key, value) + m.delInvisibleKey(key) return derr.ToTiDBErr(err) } func (m *memBuffer) SetWithFlags(key kv.Key, value []byte, ops ...kv.FlagsOp) error { err := m.MemDB.SetWithFlags(key, value, getTiKVFlagsOps(ops)...) + m.delInvisibleKey(key) return derr.ToTiDBErr(err) } +func (m *memBuffer) ChangeLockIntoPut(key kv.Key, value []byte) error { + // only change LOCK into PUT when the key does not existed, otherwise, we may mark a visible key as invisible. + if _, err := m.MemDB.Get(key); tikverr.IsErrNotFound(err) { + m.addInvisibleKey(key) + return derr.ToTiDBErr(m.MemDB.Set(key, value)) + } + return nil +} + // Iter creates an Iterator positioned on the first entry that k <= entry's key. // If such entry is not found, it returns an invalid Iterator with no error. // It yields only keys that < upperBound. If upperBound is nil, it means the upperBound is unbounded. // The Iterator must be Closed after use. func (m *memBuffer) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := m.MemDB.Iter(k, upperBound) - return &tikvIterator{Iterator: it}, derr.ToTiDBErr(err) + return newKVIterator(it, m), derr.ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -102,29 +145,33 @@ func (m *memBuffer) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { // TODO: Add lower bound limit func (m *memBuffer) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := m.MemDB.IterReverse(k) - return &tikvIterator{Iterator: it}, derr.ToTiDBErr(err) + return newKVIterator(it, m), derr.ToTiDBErr(err) } // SnapshotIter returns a Iterator for a snapshot of MemBuffer. func (m *memBuffer) SnapshotIter(k, upperbound kv.Key) kv.Iterator { it := m.MemDB.SnapshotIter(k, upperbound) - return &tikvIterator{Iterator: it} + return newKVIterator(it, m) } // SnapshotGetter returns a Getter for a snapshot of MemBuffer. func (m *memBuffer) SnapshotGetter() kv.Getter { - return newKVGetter(m.MemDB.SnapshotGetter()) + return newKVGetter(m.MemDB.SnapshotGetter(), m) } type tikvGetter struct { tikv.Getter + checker visibilityChecker } -func newKVGetter(getter tikv.Getter) kv.Getter { - return &tikvGetter{Getter: getter} +func newKVGetter(getter tikv.Getter, checker visibilityChecker) kv.Getter { + return &tikvGetter{getter, checker} } func (g *tikvGetter) Get(_ context.Context, k kv.Key) ([]byte, error) { + if !g.checker.isVisible(k) { + return nil, kv.ErrNotExist + } data, err := g.Getter.Get(k) return data, derr.ToTiDBErr(err) } @@ -132,12 +179,43 @@ func (g *tikvGetter) Get(_ context.Context, k kv.Key) ([]byte, error) { // tikvIterator wraps tikv.Iterator as kv.Iterator type tikvIterator struct { tikv.Iterator + checker visibilityChecker + initErr error +} + +func newKVIterator(iterator tikv.Iterator, checker visibilityChecker) kv.Iterator { + it := &tikvIterator{iterator, checker, nil} + if it.Valid() && !it.checker.isVisible(it.Key()) { + // skip first invisible key + it.initErr = it.Next() + } + return it } func (it *tikvIterator) Key() kv.Key { return kv.Key(it.Iterator.Key()) } +func (it *tikvIterator) Next() error { + if it.initErr != nil { + err := it.initErr + it.initErr = nil + return err + } + for { + err := it.Iterator.Next() + if err != nil { + return err + } + if !it.Valid() { + return nil + } + if it.checker.isVisible(it.Key()) { + return nil + } + } +} + func getTiDBKeyFlags(flag tikvstore.KeyFlags) kv.KeyFlags { var v kv.KeyFlags if flag.HasPresumeKeyNotExists() { diff --git a/tests/realtikvtest/pessimistictest/pessimistic_test.go b/tests/realtikvtest/pessimistictest/pessimistic_test.go index 096f6b7932edb..1c0c3122eea99 100644 --- a/tests/realtikvtest/pessimistictest/pessimistic_test.go +++ b/tests/realtikvtest/pessimistictest/pessimistic_test.go @@ -2864,7 +2864,7 @@ func TestChangeLockToPut(t *testing.T) { tk.MustExec("use test") tk2.MustExec("use test") - tk.MustExec("drop table if exists tk") + tk.MustExec("drop table if exists t1") tk.MustExec("create table t1(c1 varchar(20) key, c2 int, c3 int, unique key k1(c2), key k2(c3))") tk.MustExec(`insert into t1 values ("1", 1, 1), ("2", 2, 2), ("3", 3, 3)`) @@ -2912,6 +2912,40 @@ func TestChangeLockToPut(t *testing.T) { tk.MustExec("admin check table t1") } +func TestIssue28011(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + for _, tt := range []struct { + name string + lockQuery string + finalRows [][]interface{} + }{ + {"Update", "update t set b = 'x' where a = 'a'", testkit.Rows("a x", "b y", "c z")}, + {"BatchUpdate", "update t set b = 'x' where a in ('a', 'b', 'c')", testkit.Rows("a x", "b y", "c x")}, + {"SelectForUpdate", "select a from t where a = 'a' for update", testkit.Rows("a x", "b y", "c z")}, + {"BatchSelectForUpdate", "select a from t where a in ('a', 'b', 'c') for update", testkit.Rows("a x", "b y", "c z")}, + } { + t.Run(tt.name, func(t *testing.T) { + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a varchar(10) primary key nonclustered, b varchar(10))") + tk.MustExec("insert into t values ('a', 'x'), ('b', 'x'), ('c', 'z')") + tk.MustExec("begin") + tk.MustExec(tt.lockQuery) + tk.MustQuery("select a from t").Check(testkit.Rows("a", "b", "c")) + tk.MustExec("replace into t values ('b', 'y')") + tk.MustQuery("select a from t").Check(testkit.Rows("a", "b", "c")) + tk.MustQuery("select a, b from t order by a").Check(tt.finalRows) + tk.MustExec("commit") + tk.MustQuery("select a, b from t order by a").Check(tt.finalRows) + tk.MustExec("admin check table t") + }) + } +} + func createTable(part bool, columnNames []string, columnTypes []string) string { var str string str = "create table t(" From b789231d0e73d7be19031e94dc2003b2411619d9 Mon Sep 17 00:00:00 2001 From: zyguan Date: Tue, 21 Mar 2023 11:32:23 +0800 Subject: [PATCH 2/3] address the comment Signed-off-by: zyguan --- store/driver/txn/unionstore_driver.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/store/driver/txn/unionstore_driver.go b/store/driver/txn/unionstore_driver.go index ace3665cc7c83..f8c0ea5772ea7 100644 --- a/store/driver/txn/unionstore_driver.go +++ b/store/driver/txn/unionstore_driver.go @@ -26,7 +26,7 @@ import ( ) type visibilityChecker interface { - isVisible(kv.Key) bool + invisible(kv.Key) bool } // memBuffer wraps tikv.MemDB as kv.MemBuffer. @@ -54,10 +54,10 @@ func (m *memBuffer) delInvisibleKey(k kv.Key) { m.Unlock() } -func (m *memBuffer) isVisible(k kv.Key) bool { +func (m *memBuffer) invisible(k kv.Key) bool { // shall be protected by MemBuffer.RLock _, ok := m.invisibleKeys[util.String(k)] - return !ok + return ok } func (m *memBuffer) Size() int { @@ -77,7 +77,7 @@ func (m *memBuffer) DeleteWithFlags(k kv.Key, ops ...kv.FlagsOp) error { } func (m *memBuffer) Get(_ context.Context, key kv.Key) ([]byte, error) { - if !m.isVisible(key) { + if m.invisible(key) { return nil, kv.ErrNotExist } data, err := m.MemDB.Get(key) @@ -169,7 +169,7 @@ func newKVGetter(getter tikv.Getter, checker visibilityChecker) kv.Getter { } func (g *tikvGetter) Get(_ context.Context, k kv.Key) ([]byte, error) { - if !g.checker.isVisible(k) { + if g.checker.invisible(k) { return nil, kv.ErrNotExist } data, err := g.Getter.Get(k) @@ -185,7 +185,7 @@ type tikvIterator struct { func newKVIterator(iterator tikv.Iterator, checker visibilityChecker) kv.Iterator { it := &tikvIterator{iterator, checker, nil} - if it.Valid() && !it.checker.isVisible(it.Key()) { + if it.Valid() && it.checker.invisible(it.Key()) { // skip first invisible key it.initErr = it.Next() } @@ -210,7 +210,7 @@ func (it *tikvIterator) Next() error { if !it.Valid() { return nil } - if it.checker.isVisible(it.Key()) { + if !it.checker.invisible(it.Key()) { return nil } } From d8483a80389acabd6797feee8b492ab901d46e03 Mon Sep 17 00:00:00 2001 From: zyguan Date: Tue, 21 Mar 2023 18:49:39 +0800 Subject: [PATCH 3/3] fix check_dev Signed-off-by: zyguan --- br/pkg/glue/console_glue.go | 10 +- br/pkg/lightning/backend/local/duplicate.go | 1 + br/pkg/lightning/backend/noop/noop.go | 16 +- br/pkg/lightning/backend/tidb/tidb.go | 13 +- br/pkg/lightning/backend/tidb/tidb_test.go | 1 + br/pkg/lightning/lightning.go | 19 +- br/pkg/lightning/mydump/loader.go | 2 +- br/pkg/lightning/mydump/parser_generated.go | 2 +- br/pkg/lightning/mydump/region_test.go | 2 +- br/pkg/lightning/restore/check_info.go | 8 +- br/pkg/restore/db.go | 1 - br/pkg/restore/split_test.go | 10 +- br/pkg/restore/stream_metas.go | 10 +- br/pkg/stream/stream_status.go | 3 +- br/pkg/task/stream.go | 2 +- br/tests/br_key_locked/locker.go | 1 + ddl/backfilling.go | 9 +- ddl/ddl_api.go | 4 +- ddl/ddl_worker_test.go | 27 +-- ddl/fail_test.go | 3 +- ddl/placement_policy_ddl_test.go | 2 +- ddl/reorg.go | 18 +- distsql/request_builder.go | 11 +- dumpling/export/writer_util.go | 5 +- executor/aggfuncs/func_ntile.go | 3 +- executor/aggregate.go | 72 +++--- executor/compact_table.go | 10 +- executor/cte.go | 31 +-- executor/executor.go | 32 +-- executor/index_lookup_hash_join.go | 29 +-- executor/index_merge_reader.go | 14 +- executor/insert_common.go | 10 +- executor/joiner.go | 32 +-- executor/merge_join_test.go | 2 + executor/plan_replayer.go | 2 +- executor/projection.go | 98 ++++---- executor/replace.go | 8 +- executor/set_config.go | 5 +- executor/shuffle.go | 79 +++--- executor/sort.go | 12 +- executor/write.go | 4 +- expression/builtin_compare.go | 10 +- expression/builtin_time.go | 12 +- expression/constant_propagation.go | 14 +- expression/evaluator.go | 3 +- expression/expression.go | 5 +- expression/util.go | 12 +- infoschema/perfschema/const.go | 1 - infoschema/tables.go | 9 +- kv/key.go | 8 +- kv/mpp.go | 2 +- kv/txn.go | 3 +- meta/autoid/autoid.go | 10 +- planner/cascades/optimize.go | 12 +- planner/cascades/stringer.go | 5 +- planner/cascades/transformation_rules.go | 7 +- planner/core/cacheable_checker.go | 5 +- planner/core/common_plans.go | 3 +- planner/core/exhaust_physical_plans.go | 11 +- planner/core/expression_rewriter.go | 7 +- planner/core/find_best_task.go | 16 +- planner/core/find_best_task_test.go | 14 +- planner/core/logical_plan_builder.go | 19 +- planner/core/partition_pruner_test.go | 19 +- planner/core/physical_plans.go | 13 +- planner/core/preprocess.go | 12 +- planner/core/rule_aggregation_push_down.go | 4 +- planner/core/rule_join_elimination.go | 20 +- planner/core/rule_join_reorder_dp.go | 3 +- planner/core/rule_join_reorder_greedy.go | 6 +- planner/core/rule_partition_processor.go | 14 +- planner/core/rule_predicate_push_down.go | 10 +- planner/core/rule_result_reorder.go | 22 +- planner/core/task.go | 3 +- planner/funcdep/fd_graph.go | 189 +++++++-------- server/http_handler.go | 6 +- session/bench_test.go | 3 +- sessionctx/variable/sysvar.go | 4 +- statistics/builder.go | 3 +- statistics/cmsketch.go | 26 +- statistics/handle/handle_test.go | 2 +- statistics/handle/update.go | 12 +- statistics/histogram.go | 62 +++-- statistics/row_sampler.go | 24 +- statistics/selectivity.go | 6 +- store/copr/batch_coprocessor.go | 11 +- store/driver/txn/batch_getter_test.go | 2 +- store/driver/txn/driver_test.go | 2 +- store/mockstore/unistore/cluster.go | 16 +- .../unistore/cophandler/closure_exec.go | 3 +- store/store.go | 5 +- table/column.go | 7 +- table/tables/mutation_checker.go | 3 +- table/tables/partition.go | 21 +- tablecodec/tablecodec.go | 100 ++++---- .../pessimistictest/pessimistic_test.go | 1 + .../proto/go-binlog/secondary_binlog.pb.go | 34 +-- types/core_time.go | 6 +- types/datum.go | 17 +- types/etc.go | 1 + types/json/binary_functions.go | 9 +- types/json/binary_functions_test.go | 2 +- types/mydecimal.go | 227 +++++++++--------- types/time.go | 38 +-- util/chunk/chunk.go | 10 +- util/chunk/codec.go | 22 +- util/chunk/disk_test.go | 3 +- util/chunk/iterator.go | 6 +- util/codec/bytes.go | 18 +- util/codec/codec.go | 3 +- util/cteutil/storage.go | 12 +- util/disk/tracker.go | 4 +- util/errors.go | 2 +- util/execdetails/execdetails.go | 12 +- util/execdetails/main_test.go | 2 +- util/fastrand/random.go | 1 + util/format/format.go | 32 ++- util/memory/tracker.go | 10 +- util/misc.go | 14 +- util/mock/iter_test.go | 2 +- util/processinfo.go | 11 +- util/ranger/detacher.go | 49 ++-- util/ranger/ranger.go | 15 +- util/sqlexec/utils.go | 2 +- util/stringutil/string_util.go | 2 +- util/sys/linux/sys_test.go | 2 +- util/table-rule-selector/trie_selector.go | 8 +- util/timeutil/time.go | 3 +- util/topsql/collector/cpu.go | 25 +- util/watcher/watcher.go | 8 +- 130 files changed, 1124 insertions(+), 933 deletions(-) diff --git a/br/pkg/glue/console_glue.go b/br/pkg/glue/console_glue.go index 01c765567c62b..145de960aa8c3 100644 --- a/br/pkg/glue/console_glue.go +++ b/br/pkg/glue/console_glue.go @@ -91,8 +91,10 @@ func (t *Table) maxKeyLen() int { // Print prints the table. // The format would be like: -// Key1: -// Other: +// +// Key1: +// Other: +// // LongKey: // The format may change if the terminal size is small. func (t *Table) Print() { @@ -267,7 +269,9 @@ func (ps PrettyString) slicePointOf(s int) (realSlicePoint, endAt int) { // It is the abstraction of some subarea of the terminal, // you might imagine it as a panel in the tmux, but with infinity height. // For example, printing a frame with the width of 10 chars, and 4 chars offset left, would be like: -// v~~~~~~~~~~v Here is the "width of a frame". +// +// v~~~~~~~~~~v Here is the "width of a frame". +// // +--+----------+--+ // | Hello, wor | // | ld. | diff --git a/br/pkg/lightning/backend/local/duplicate.go b/br/pkg/lightning/backend/local/duplicate.go index 983ae33fcfd68..b0be0ddbfa9a5 100644 --- a/br/pkg/lightning/backend/local/duplicate.go +++ b/br/pkg/lightning/backend/local/duplicate.go @@ -245,6 +245,7 @@ type DupKVStream interface { // LocalDupKVStream implements the interface of DupKVStream. // It collects duplicate key-value pairs from a pebble.DB. +// //goland:noinspection GoNameStartsWithPackageName type LocalDupKVStream struct { iter Iter diff --git a/br/pkg/lightning/backend/noop/noop.go b/br/pkg/lightning/backend/noop/noop.go index 2ac3e2b346dbb..a403e3d6a1339 100644 --- a/br/pkg/lightning/backend/noop/noop.go +++ b/br/pkg/lightning/backend/noop/noop.go @@ -97,14 +97,14 @@ func (b noopBackend) CheckRequirements(context.Context, *backend.CheckCtx) error // name. The returned table info does not need to be precise if the encoder, // is not requiring them, but must at least fill in the following fields for // TablesFromMeta to succeed: -// - Name -// - State (must be model.StatePublic) -// - ID -// - Columns -// * Name -// * State (must be model.StatePublic) -// * Offset (must be 0, 1, 2, ...) -// - PKIsHandle (true = do not generate _tidb_rowid) +// - Name +// - State (must be model.StatePublic) +// - ID +// - Columns +// - Name +// - State (must be model.StatePublic) +// - Offset (must be 0, 1, 2, ...) +// - PKIsHandle (true = do not generate _tidb_rowid) func (b noopBackend) FetchRemoteTableModels(ctx context.Context, schemaName string) ([]*model.TableInfo, error) { return nil, nil } diff --git a/br/pkg/lightning/backend/tidb/tidb.go b/br/pkg/lightning/backend/tidb/tidb.go index 9ae70e7afef1d..50d2f1b956b14 100644 --- a/br/pkg/lightning/backend/tidb/tidb.go +++ b/br/pkg/lightning/backend/tidb/tidb.go @@ -472,7 +472,8 @@ type stmtTask struct { } // WriteBatchRowsToDB write rows in batch mode, which will insert multiple rows like this: -// insert into t1 values (111), (222), (333), (444); +// +// insert into t1 values (111), (222), (333), (444); func (be *tidbBackend) WriteBatchRowsToDB(ctx context.Context, tableName string, columnNames []string, r kv.Rows) error { rows := r.(tidbRows) insertStmt := be.checkAndBuildStmt(rows, tableName, columnNames) @@ -500,10 +501,12 @@ func (be *tidbBackend) checkAndBuildStmt(rows tidbRows, tableName string, column } // WriteRowsToDB write rows in row-by-row mode, which will insert multiple rows like this: -// insert into t1 values (111); -// insert into t1 values (222); -// insert into t1 values (333); -// insert into t1 values (444); +// +// insert into t1 values (111); +// insert into t1 values (222); +// insert into t1 values (333); +// insert into t1 values (444); +// // See more details in br#1366: https://github.com/pingcap/br/issues/1366 func (be *tidbBackend) WriteRowsToDB(ctx context.Context, tableName string, columnNames []string, r kv.Rows) error { rows := r.(tidbRows) diff --git a/br/pkg/lightning/backend/tidb/tidb_test.go b/br/pkg/lightning/backend/tidb/tidb_test.go index b7de642a26746..04a384d18df37 100644 --- a/br/pkg/lightning/backend/tidb/tidb_test.go +++ b/br/pkg/lightning/backend/tidb/tidb_test.go @@ -214,6 +214,7 @@ func TestWriteRowsErrorOnDup(t *testing.T) { } // TODO: temporarily disable this test before we fix strict mode +// //nolint:unused,deadcode func testStrictMode(t *testing.T) { s := createMysqlSuite(t) diff --git a/br/pkg/lightning/lightning.go b/br/pkg/lightning/lightning.go index dc389bebbc6ab..958a795997081 100644 --- a/br/pkg/lightning/lightning.go +++ b/br/pkg/lightning/lightning.go @@ -227,11 +227,12 @@ func (l *Lightning) goServe(statusAddr string, realAddrWriter io.Writer) error { } // RunOnce is used by binary lightning and host when using lightning as a library. -// - for binary lightning, taskCtx could be context.Background which means taskCtx wouldn't be canceled directly by its -// cancel function, but only by Lightning.Stop or HTTP DELETE using l.cancel. and glue could be nil to let lightning -// use a default glue later. -// - for lightning as a library, taskCtx could be a meaningful context that get canceled outside, and glue could be a -// caller implemented glue. +// - for binary lightning, taskCtx could be context.Background which means taskCtx wouldn't be canceled directly by its +// cancel function, but only by Lightning.Stop or HTTP DELETE using l.cancel. and glue could be nil to let lightning +// use a default glue later. +// - for lightning as a library, taskCtx could be a meaningful context that get canceled outside, and glue could be a +// caller implemented glue. +// // deprecated: use RunOnceWithOptions instead. func (l *Lightning) RunOnce(taskCtx context.Context, taskCfg *config.Config, glue glue.Glue) error { if err := taskCfg.Adjust(taskCtx); err != nil { @@ -270,10 +271,10 @@ func (l *Lightning) RunServer() error { } // RunOnceWithOptions is used by binary lightning and host when using lightning as a library. -// - for binary lightning, taskCtx could be context.Background which means taskCtx wouldn't be canceled directly by its -// cancel function, but only by Lightning.Stop or HTTP DELETE using l.cancel. No need to set Options -// - for lightning as a library, taskCtx could be a meaningful context that get canceled outside, and there Options may -// be used: +// - for binary lightning, taskCtx could be context.Background which means taskCtx wouldn't be canceled directly by its +// cancel function, but only by Lightning.Stop or HTTP DELETE using l.cancel. No need to set Options +// - for lightning as a library, taskCtx could be a meaningful context that get canceled outside, and there Options may +// be used: // - WithGlue: set a caller implemented glue. Otherwise, lightning will use a default glue later. // - WithDumpFileStorage: caller has opened an external storage for lightning. Otherwise, lightning will open a // storage by config diff --git a/br/pkg/lightning/mydump/loader.go b/br/pkg/lightning/mydump/loader.go index 09ef6229c45bc..bd48a25b22a17 100644 --- a/br/pkg/lightning/mydump/loader.go +++ b/br/pkg/lightning/mydump/loader.go @@ -88,7 +88,7 @@ func (m *MDTableMeta) GetSchema(ctx context.Context, store storage.ExternalStora } /* - Mydumper File Loader +Mydumper File Loader */ type MDLoader struct { store storage.ExternalStorage diff --git a/br/pkg/lightning/mydump/parser_generated.go b/br/pkg/lightning/mydump/parser_generated.go index bc58a9d181e0d..c803c0c4c2e40 100644 --- a/br/pkg/lightning/mydump/parser_generated.go +++ b/br/pkg/lightning/mydump/parser_generated.go @@ -32,7 +32,7 @@ import ( //.... lightning/mydump/parser.rl:137 -//.... tmp_parser.go:37 +// .... tmp_parser.go:37 const ( chunk_parser_start int = 21 chunk_parser_first_final int = 21 diff --git a/br/pkg/lightning/mydump/region_test.go b/br/pkg/lightning/mydump/region_test.go index a1dbb9f290a69..5c4bc1c7734b5 100644 --- a/br/pkg/lightning/mydump/region_test.go +++ b/br/pkg/lightning/mydump/region_test.go @@ -36,7 +36,7 @@ import ( // } /* - TODO : test with specified 'regionBlockSize' ... +TODO : test with specified 'regionBlockSize' ... */ func TestTableRegion(t *testing.T) { cfg := newConfigWithSourceDir("./examples") diff --git a/br/pkg/lightning/restore/check_info.go b/br/pkg/lightning/restore/check_info.go index 6d2504919ab99..41db33794e899 100644 --- a/br/pkg/lightning/restore/check_info.go +++ b/br/pkg/lightning/restore/check_info.go @@ -754,10 +754,10 @@ func (rc *Controller) SchemaIsValid(ctx context.Context, tableInfo *mydump.MDTab } // checkCSVHeader try to check whether the csv header config is consistent with the source csv files by: -// 1. pick one table with two CSV files and a unique/primary key -// 2. read the first row of those two CSV files -// 3. checks if the content of those first rows are compatible with the table schema, and whether the -// two rows are identical, to determine if the first rows are a header rows. +// 1. pick one table with two CSV files and a unique/primary key +// 2. read the first row of those two CSV files +// 3. checks if the content of those first rows are compatible with the table schema, and whether the +// two rows are identical, to determine if the first rows are a header rows. func (rc *Controller) checkCSVHeader(ctx context.Context, dbMetas []*mydump.MDDatabaseMeta) error { // if cfg set header = ture but source files actually contain not header, former SchemaCheck should // return error in this situation, so we need do it again. diff --git a/br/pkg/restore/db.go b/br/pkg/restore/db.go index 377b2ed5dd78f..e6e0a0c300e55 100644 --- a/br/pkg/restore/db.go +++ b/br/pkg/restore/db.go @@ -169,7 +169,6 @@ func (db *DB) CreateDatabase(ctx context.Context, schema *model.DBInfo) error { return errors.Trace(err) } -// func (db *DB) restoreSequence(ctx context.Context, table *metautil.Table) error { var restoreMetaSQL string var err error diff --git a/br/pkg/restore/split_test.go b/br/pkg/restore/split_test.go index 19a4b2a43f8e0..5e5224e2dc513 100644 --- a/br/pkg/restore/split_test.go +++ b/br/pkg/restore/split_test.go @@ -319,8 +319,9 @@ func TestScatterFinishInTime(t *testing.T) { // range: [aaa, aae), [aae, aaz), [ccd, ccf), [ccf, ccj) // rewrite rules: aa -> xx, cc -> bb // expected regions after split: -// [, aay), [aay, bba), [bba, bbf), [bbf, bbh), [bbh, bbj), -// [bbj, cca), [cca, xxe), [xxe, xxz), [xxz, ) +// +// [, aay), [aay, bba), [bba, bbf), [bbf, bbh), [bbh, bbj), +// [bbj, cca), [cca, xxe), [xxe, xxz), [xxz, ) func TestSplitAndScatter(t *testing.T) { t.Run("BatchScatter", func(t *testing.T) { client := initTestClient() @@ -448,8 +449,9 @@ func initRewriteRules() *restore.RewriteRules { } // expected regions after split: -// [, aay), [aay, bba), [bba, bbf), [bbf, bbh), [bbh, bbj), -// [bbj, cca), [cca, xxe), [xxe, xxz), [xxz, ) +// +// [, aay), [aay, bba), [bba, bbf), [bbf, bbh), [bbh, bbj), +// [bbj, cca), [cca, xxe), [xxe, xxz), [xxz, ) func validateRegions(regions map[uint64]*restore.RegionInfo) bool { keys := [...]string{"", "aay", "bba", "bbf", "bbh", "bbj", "cca", "xxe", "xxz", ""} if len(regions) != len(keys)-1 { diff --git a/br/pkg/restore/stream_metas.go b/br/pkg/restore/stream_metas.go index 716d994822850..74fd75075b7db 100644 --- a/br/pkg/restore/stream_metas.go +++ b/br/pkg/restore/stream_metas.go @@ -51,10 +51,12 @@ func (ms *StreamMetadataSet) iterateDataFiles(f func(d *backuppb.DataFileInfo) ( } // IterateFilesFullyBefore runs the function over all files contain data before the timestamp only. -// 0 before -// |------------------------------------------| -// |-file1---------------| <- File contains records in this TS range would be found. -// |-file2--------------| <- File contains any record out of this won't be found. +// +// 0 before +// |------------------------------------------| +// |-file1---------------| <- File contains records in this TS range would be found. +// |-file2--------------| <- File contains any record out of this won't be found. +// // This function would call the `f` over file1 only. func (ms *StreamMetadataSet) IterateFilesFullyBefore(before uint64, f func(d *backuppb.DataFileInfo) (shouldBreak bool)) { ms.iterateDataFiles(func(d *backuppb.DataFileInfo) (shouldBreak bool) { diff --git a/br/pkg/stream/stream_status.go b/br/pkg/stream/stream_status.go index 3e59c638714ce..8820402d02124 100644 --- a/br/pkg/stream/stream_status.go +++ b/br/pkg/stream/stream_status.go @@ -230,7 +230,8 @@ var logCountSumRe = regexp.MustCompile(`tikv_stream_handle_kv_batch_sum ([0-9]+) // MaybeQPS get a number like the QPS of last seconds for each store via the prometheus interface. // TODO: this is a temporary solution(aha, like in a Hackthon), -// we MUST find a better way for providing this information. +// +// we MUST find a better way for providing this information. func MaybeQPS(ctx context.Context, mgr *conn.Mgr) (float64, error) { c := mgr.GetPDClient() prefix := "http://" diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index d257eb868fb55..bd85db47da680 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -445,7 +445,7 @@ func (s *streamMgr) backupFullSchemas(ctx context.Context, g glue.Glue) error { return nil } -// RunStreamCommand run all kinds of `stream task`` +// RunStreamCommand run all kinds of `stream task“ func RunStreamCommand( ctx context.Context, g glue.Glue, diff --git a/br/tests/br_key_locked/locker.go b/br/tests/br_key_locked/locker.go index ec16a4b096e41..39e733fa1358c 100644 --- a/br/tests/br_key_locked/locker.go +++ b/br/tests/br_key_locked/locker.go @@ -185,6 +185,7 @@ type Locker struct { } // generateLocks sends Prewrite requests to TiKV to generate locks, without committing and rolling back. +// //nolint:gosec func (c *Locker) generateLocks(pctx context.Context) error { log.Info("genLock started") diff --git a/ddl/backfilling.go b/ddl/backfilling.go index b133ac12b1895..515223c89dd89 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -566,10 +566,11 @@ func setSessCtxLocation(sctx sessionctx.Context, info *reorgInfo) error { // The handle range is split from PD regions now. Each worker deal with a region table key range one time. // Each handle range by estimation, concurrent processing needs to perform after the handle range has been acquired. // The operation flow is as follows: -// 1. Open numbers of defaultWorkers goroutines. -// 2. Split table key range from PD regions. -// 3. Send tasks to running workers by workers's task channel. Each task deals with a region key ranges. -// 4. Wait all these running tasks finished, then continue to step 3, until all tasks is done. +// 1. Open numbers of defaultWorkers goroutines. +// 2. Split table key range from PD regions. +// 3. Send tasks to running workers by workers's task channel. Each task deals with a region key ranges. +// 4. Wait all these running tasks finished, then continue to step 3, until all tasks is done. +// // The above operations are completed in a transaction. // Finally, update the concurrent processing of the total number of rows, and store the completed handle value. func (w *worker) writePhysicalTableRecord(t table.PhysicalTable, bfWorkerType backfillWorkerType, indexInfo *model.IndexInfo, oldColInfo, colInfo *model.ColumnInfo, reorgInfo *reorgInfo) error { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 7c867b9b0b3d6..610f637a9121d 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -691,7 +691,9 @@ func ResolveCharsetCollation(charsetOpts ...ast.CharsetOpt) (string, string, err } // OverwriteCollationWithBinaryFlag is used to handle the case like -// CREATE TABLE t (a VARCHAR(255) BINARY) CHARSET utf8 COLLATE utf8_general_ci; +// +// CREATE TABLE t (a VARCHAR(255) BINARY) CHARSET utf8 COLLATE utf8_general_ci; +// // The 'BINARY' sets the column collation to *_bin according to the table charset. func OverwriteCollationWithBinaryFlag(colDef *ast.ColumnDef, chs, coll string) (newChs string, newColl string) { ignoreBinFlag := colDef.Tp.GetCharset() != "" && (colDef.Tp.GetCollate() != "" || containsColumnOption(colDef, ast.ColumnOptionCollate)) diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index f233c7c8f6635..ba459db351f27 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -1,17 +1,16 @@ -//// Copyright 2015 PingCAP, Inc. -//// -//// Licensed under the Apache License, Version 2.0 (the "License"); -//// you may not use this file except in compliance with the License. -//// You may obtain a copy of the License at -//// -//// http://www.apache.org/licenses/LICENSE-2.0 -//// -//// Unless required by applicable law or agreed to in writing, software -//// distributed under the License is distributed on an "AS IS" BASIS, -//// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -//// See the License for the specific language governing permissions and -//// limitations under the License. -// +// // Copyright 2015 PingCAP, Inc. +// // +// // Licensed under the Apache License, Version 2.0 (the "License"); +// // you may not use this file except in compliance with the License. +// // You may obtain a copy of the License at +// // +// // http://www.apache.org/licenses/LICENSE-2.0 +// // +// // Unless required by applicable law or agreed to in writing, software +// // distributed under the License is distributed on an "AS IS" BASIS, +// // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// // See the License for the specific language governing permissions and +// // limitations under the License. package ddl_test import ( diff --git a/ddl/fail_test.go b/ddl/fail_test.go index 741fa739ca289..b4b8666eade7e 100644 --- a/ddl/fail_test.go +++ b/ddl/fail_test.go @@ -4,14 +4,13 @@ // you may not use this file except in compliance with the License. // You may obtain a copy of the License at // -// http://www.apache.org/licenses/LICENSE-2.0 +// http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -// package ddl_test import ( diff --git a/ddl/placement_policy_ddl_test.go b/ddl/placement_policy_ddl_test.go index 6525ad6a2b092..812d7a39f36e9 100644 --- a/ddl/placement_policy_ddl_test.go +++ b/ddl/placement_policy_ddl_test.go @@ -4,7 +4,7 @@ // you may not use this file except in compliance with the License. // You may obtain a copy of the License at // -// http://www.apache.org/licenses/LICENSE-2.0 +// http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, diff --git a/ddl/reorg.go b/ddl/reorg.go index 6f8039c99c3e3..fbe0f7d079c12 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -167,14 +167,18 @@ func (rc *reorgCtx) clean() { // 3: clean global index // // ddl goroutine >---------+ -// ^ | -// | | -// | | -// | | <---(doneCh)--- f() +// +// ^ | +// | | +// | | +// | | <---(doneCh)--- f() +// // HandleDDLQueue(...) | <---(regular timeout) -// | | <---(ctx done) -// | | -// | | +// +// | | <---(ctx done) +// | | +// | | +// // A more ddl round <-----+ // // How can we cancel reorg job? diff --git a/distsql/request_builder.go b/distsql/request_builder.go index c8d7f0ebb3c0c..8c74d08915b14 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -286,8 +286,9 @@ func (builder *RequestBuilder) SetConcurrency(concurrency int) *RequestBuilder { } // SetTiDBServerID sets "TiDBServerID" for "kv.Request" -// ServerID is a unique id of TiDB instance among the cluster. -// See https://github.com/pingcap/tidb/blob/master/docs/design/2020-06-01-global-kill.md +// +// ServerID is a unique id of TiDB instance among the cluster. +// See https://github.com/pingcap/tidb/blob/master/docs/design/2020-06-01-global-kill.md func (builder *RequestBuilder) SetTiDBServerID(serverID uint64) *RequestBuilder { builder.Request.TiDBServerID = serverID return builder @@ -450,9 +451,9 @@ func encodeHandleKey(ran *ranger.Range) ([]byte, []byte) { // interpreted as an int64 variable. // // This function does the following: -// 1. split ranges into two groups as described above. -// 2. if there's a range that straddles the int64 boundary, split it into two ranges, which results in one smaller and -// one greater than MaxInt64. +// 1. split ranges into two groups as described above. +// 2. if there's a range that straddles the int64 boundary, split it into two ranges, which results in one smaller and +// one greater than MaxInt64. // // if `KeepOrder` is false, we merge the two groups of ranges into one group, to save an rpc call later // if `desc` is false, return signed ranges first, vice versa. diff --git a/dumpling/export/writer_util.go b/dumpling/export/writer_util.go index 73eb66b0a1da6..750721138a35a 100755 --- a/dumpling/export/writer_util.go +++ b/dumpling/export/writer_util.go @@ -607,8 +607,9 @@ func (f FileFormat) String() string { } // Extension returns the extension for specific format. -// text -> "sql" -// csv -> "csv" +// +// text -> "sql" +// csv -> "csv" func (f FileFormat) Extension() string { switch f { case FileFormatSQLText: diff --git a/executor/aggfuncs/func_ntile.go b/executor/aggfuncs/func_ntile.go index 059392b1cda41..795b4cdf99bfd 100644 --- a/executor/aggfuncs/func_ntile.go +++ b/executor/aggfuncs/func_ntile.go @@ -28,7 +28,8 @@ const ( // ntile divides the partition into n ranked groups and returns the group number a row belongs to. // e.g. We have 11 rows and n = 3. They will be divided into 3 groups. -// First 4 rows belongs to group 1. Following 4 rows belongs to group 2. The last 3 rows belongs to group 3. +// +// First 4 rows belongs to group 1. Following 4 rows belongs to group 2. The last 3 rows belongs to group 3. type ntile struct { n uint64 baseAggFunc diff --git a/executor/aggregate.go b/executor/aggregate.go index 9a0be36a5ccd4..8f66f41def76a 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -119,42 +119,44 @@ type AfFinalResult struct { // and updates all the items in PartialAggFuncs. // The parallel execution flow is as the following graph shows: // -// +-------------+ -// | Main Thread | -// +------+------+ -// ^ -// | -// + -// +-+- +-+ -// | | ...... | | finalOutputCh -// +++- +-+ -// ^ -// | -// +---------------+ -// | | -// +--------------+ +--------------+ -// | final worker | ...... | final worker | -// +------------+-+ +-+------------+ -// ^ ^ -// | | -// +-+ +-+ ...... +-+ -// | | | | | | -// ... ... ... partialOutputChs -// | | | | | | -// +++ +++ +++ -// ^ ^ ^ -// +-+ | | | -// | | +--------o----+ | +// +-------------+ +// | Main Thread | +// +------+------+ +// ^ +// | +// + +// +-+- +-+ +// | | ...... | | finalOutputCh +// +++- +-+ +// ^ +// | +// +---------------+ +// | | +// +--------------+ +--------------+ +// | final worker | ...... | final worker | +// +------------+-+ +-+------------+ +// ^ ^ +// | | +// +-+ +-+ ...... +-+ +// | | | | | | +// ... ... ... partialOutputChs +// | | | | | | +// +++ +++ +++ +// ^ ^ ^ +// +-+ | | | +// | | +--------o----+ | +// // inputCh +-+ | +-----------------+---+ -// | | | | -// ... +---+------------+ +----+-----------+ -// | | | partial worker | ...... | partial worker | -// +++ +--------------+-+ +-+--------------+ -// | ^ ^ -// | | | -// +----v---------+ +++ +-+ +++ -// | data fetcher | +------> | | | | ...... | | partialInputChs -// +--------------+ +-+ +-+ +-+ +// +// | | | | +// ... +---+------------+ +----+-----------+ +// | | | partial worker | ...... | partial worker | +// +++ +--------------+-+ +-+--------------+ +// | ^ ^ +// | | | +// +----v---------+ +++ +-+ +++ +// | data fetcher | +------> | | | | ...... | | partialInputChs +// +--------------+ +-+ +-+ +-+ type HashAggExec struct { baseExecutor diff --git a/executor/compact_table.go b/executor/compact_table.go index 10f1008488406..905303c7198fe 100644 --- a/executor/compact_table.go +++ b/executor/compact_table.go @@ -214,14 +214,18 @@ func (task *storeCompactTask) logProgressOptionally() { // // There are two kind of errors may be returned: // A. Error only cancel tasks related with this store, e.g. this store is down even after retry. -// The remaining partitions in this store should be cancelled. +// +// The remaining partitions in this store should be cancelled. +// // B. Error that should cancel tasks of other stores, e.g. CompactErrorCompactInProgress. -// The remaining partitions in this store should be cancelled, and tasks of other stores should also be cancelled. +// +// The remaining partitions in this store should be cancelled, and tasks of other stores should also be cancelled. // // During this function, some "problems" will cause it to early return, e.g. physical table not exist in this // store any more (maybe caused by DDL). No errors will be produced so that remaining partitions will continue // being compacted. -// Returns: (stopAllTasks, err) +// +// Returns: (stopAllTasks, err) func (task *storeCompactTask) compactOnePhysicalTable(physicalTableID int64) (bool, error) { var startKey []byte = nil for { // This loop is to compact incrementally for all data. Each RPC request will only compact a partial of data. diff --git a/executor/cte.go b/executor/cte.go index abce5fb7a4c3f..7e98064b1d8bd 100644 --- a/executor/cte.go +++ b/executor/cte.go @@ -41,22 +41,25 @@ var _ Executor = &CTEExec{} // which will be the input for new iteration. // At the end of each iteration, data in `iterOutTbl` will also be added into `resTbl`. // `resTbl` stores data of all iteration. -// +----------+ -// write |iterOutTbl| -// CTEExec ------------------->| | -// | +----+-----+ -// ------------- | write -// | | v +// +// +----------+ +// write |iterOutTbl| +// CTEExec ------------------->| | +// | +----+-----+ +// ------------- | write +// | | v +// // other op other op +----------+ // (seed) (recursive) | resTbl | -// ^ | | -// | +----------+ -// CTETableReaderExec -// ^ -// | read +----------+ -// +---------------+iterInTbl | -// | | -// +----------+ +// +// ^ | | +// | +----------+ +// CTETableReaderExec +// ^ +// | read +----------+ +// +---------------+iterInTbl | +// | | +// +----------+ type CTEExec struct { baseExecutor diff --git a/executor/executor.go b/executor/executor.go index 9aba519361175..c7bfb170d422d 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -889,6 +889,7 @@ func (e *CheckTableExec) checkTableRecord(ctx context.Context, idxOffset int) er // ShowSlowExec represents the executor of showing the slow queries. // It is build from the "admin show slow" statement: +// // admin show slow top [internal | all] N // admin show slow recent N type ShowSlowExec struct { @@ -1570,21 +1571,22 @@ func (e *MaxOneRowExec) Next(ctx context.Context, req *chunk.Chunk) error { // UnionExec pulls all it's children's result and returns to its parent directly. // A "resultPuller" is started for every child to pull result from that child and push it to the "resultPool", the used // "Chunk" is obtained from the corresponding "resourcePool". All resultPullers are running concurrently. -// +----------------+ -// +---> resourcePool 1 ---> | resultPuller 1 |-----+ -// | +----------------+ | -// | | -// | +----------------+ v -// +---> resourcePool 2 ---> | resultPuller 2 |-----> resultPool ---+ -// | +----------------+ ^ | -// | ...... | | -// | +----------------+ | | -// +---> resourcePool n ---> | resultPuller n |-----+ | -// | +----------------+ | -// | | -// | +-------------+ | -// |--------------------------| main thread | <---------------------+ -// +-------------+ +// +// +----------------+ +// +---> resourcePool 1 ---> | resultPuller 1 |-----+ +// | +----------------+ | +// | | +// | +----------------+ v +// +---> resourcePool 2 ---> | resultPuller 2 |-----> resultPool ---+ +// | +----------------+ ^ | +// | ...... | | +// | +----------------+ | | +// +---> resourcePool n ---> | resultPuller n |-----+ | +// | +----------------+ | +// | | +// | +-------------+ | +// |--------------------------| main thread | <---------------------+ +// +-------------+ type UnionExec struct { baseExecutor concurrency int diff --git a/executor/index_lookup_hash_join.go b/executor/index_lookup_hash_join.go index 4c20b9baf90cb..515c41523280e 100644 --- a/executor/index_lookup_hash_join.go +++ b/executor/index_lookup_hash_join.go @@ -39,11 +39,11 @@ import ( // numResChkHold indicates the number of resource chunks that an inner worker // holds at the same time. // It's used in 2 cases individually: -// 1. IndexMergeJoin -// 2. IndexNestedLoopHashJoin: -// It's used when IndexNestedLoopHashJoin.keepOuterOrder is true. -// Otherwise, there will be at most `concurrency` resource chunks throughout -// the execution of IndexNestedLoopHashJoin. +// 1. IndexMergeJoin +// 2. IndexNestedLoopHashJoin: +// It's used when IndexNestedLoopHashJoin.keepOuterOrder is true. +// Otherwise, there will be at most `concurrency` resource chunks throughout +// the execution of IndexNestedLoopHashJoin. const numResChkHold = 4 // IndexNestedLoopHashJoin employs one outer worker and N inner workers to @@ -53,10 +53,11 @@ const numResChkHold = 4 // 1. The outer worker reads N outer rows, builds a task and sends it to the // inner worker channel. // 2. The inner worker receives the tasks and does 3 things for every task: -// 1. builds hash table from the outer rows -// 2. builds key ranges from outer rows and fetches inner rows -// 3. probes the hash table and sends the join result to the main thread channel. -// Note: step 1 and step 2 runs concurrently. +// 1. builds hash table from the outer rows +// 2. builds key ranges from outer rows and fetches inner rows +// 3. probes the hash table and sends the join result to the main thread channel. +// Note: step 1 and step 2 runs concurrently. +// // 3. The main thread receives the join results. type IndexNestedLoopHashJoin struct { IndexLookUpJoin @@ -777,11 +778,11 @@ func (iw *indexHashJoinInnerWorker) collectMatchedInnerPtrs4OuterRows(ctx contex } // doJoinInOrder follows the following steps: -// 1. collect all the matched inner row ptrs for every outer row -// 2. do the join work -// 2.1 collect all the matched inner rows using the collected ptrs for every outer row -// 2.2 call tryToMatchInners for every outer row -// 2.3 call onMissMatch when no inner rows are matched +// 1. collect all the matched inner row ptrs for every outer row +// 2. do the join work +// 2.1 collect all the matched inner rows using the collected ptrs for every outer row +// 2.2 call tryToMatchInners for every outer row +// 2.3 call onMissMatch when no inner rows are matched func (iw *indexHashJoinInnerWorker) doJoinInOrder(ctx context.Context, task *indexHashJoinTask, joinResult *indexHashJoinResult, h hash.Hash64, resultCh chan *indexHashJoinResult) (err error) { defer func() { if err == nil && joinResult.chk != nil { diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index 660eaa71499b3..939826edd7170 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -57,13 +57,13 @@ var ( // // The execution flow is really like IndexLookUpReader. However, it uses multiple index scans // or table scans to get the handles: -// 1. use the partialTableWorkers and partialIndexWorkers to fetch the handles (a batch per time) -// and send them to the indexMergeProcessWorker. -// 2. indexMergeProcessWorker do the `Union` operation for a batch of handles it have got. -// For every handle in the batch: -// 1. check whether it has been accessed. -// 2. if not, record it and send it to the indexMergeTableScanWorker. -// 3. if accessed, just ignore it. +// 1. use the partialTableWorkers and partialIndexWorkers to fetch the handles (a batch per time) +// and send them to the indexMergeProcessWorker. +// 2. indexMergeProcessWorker do the `Union` operation for a batch of handles it have got. +// For every handle in the batch: +// 1. check whether it has been accessed. +// 2. if not, record it and send it to the indexMergeTableScanWorker. +// 3. if accessed, just ignore it. type IndexMergeReaderExecutor struct { baseExecutor diff --git a/executor/insert_common.go b/executor/insert_common.go index fa77665ab9be8..207b762400919 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -423,11 +423,11 @@ func (e *InsertValues) fastEvalRow(ctx context.Context, list []expression.Expres // setValueForRefColumn set some default values for the row to eval the row value with other columns, // it follows these rules: -// 1. for nullable and no default value column, use NULL. -// 2. for nullable and have default value column, use it's default value. -// 3. for not null column, use zero value even in strict mode. -// 4. for auto_increment column, use zero value. -// 5. for generated column, use NULL. +// 1. for nullable and no default value column, use NULL. +// 2. for nullable and have default value column, use it's default value. +// 3. for not null column, use zero value even in strict mode. +// 4. for auto_increment column, use zero value. +// 5. for generated column, use NULL. func (e *InsertValues) setValueForRefColumn(row []types.Datum, hasValue []bool) error { for i, c := range e.Table.Cols() { d, err := e.getColDefaultValue(i, c) diff --git a/executor/joiner.go b/executor/joiner.go index ecfab11f66822..3fd76560fe7a4 100644 --- a/executor/joiner.go +++ b/executor/joiner.go @@ -37,26 +37,26 @@ var ( // joiner is used to generate join results according to the join type. // A typical instruction flow is: // -// hasMatch, hasNull := false, false -// for innerIter.Current() != innerIter.End() { -// matched, isNull, err := j.tryToMatchInners(outer, innerIter, chk) -// // handle err -// hasMatch = hasMatch || matched -// hasNull = hasNull || isNull -// } -// if !hasMatch { -// j.onMissMatch(hasNull, outer, chk) -// } +// hasMatch, hasNull := false, false +// for innerIter.Current() != innerIter.End() { +// matched, isNull, err := j.tryToMatchInners(outer, innerIter, chk) +// // handle err +// hasMatch = hasMatch || matched +// hasNull = hasNull || isNull +// } +// if !hasMatch { +// j.onMissMatch(hasNull, outer, chk) +// } // // NOTE: This interface is **not** thread-safe. // TODO: unit test // for all join type -// 1. no filter, no inline projection -// 2. no filter, inline projection -// 3. no filter, inline projection to empty column -// 4. filter, no inline projection -// 5. filter, inline projection -// 6. filter, inline projection to empty column +// 1. no filter, no inline projection +// 2. no filter, inline projection +// 3. no filter, inline projection to empty column +// 4. filter, no inline projection +// 5. filter, inline projection +// 6. filter, inline projection to empty column type joiner interface { // tryToMatchInners tries to join an outer row with a batch of inner rows. When // 'inners.Len != 0' but all the joined rows are filtered, the outer row is diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index f1fa29b927a5c..a140809fd46d2 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -738,6 +738,7 @@ func TestMergeJoinDifferentTypes(t *testing.T) { } // TestVectorizedMergeJoin is used to test vectorized merge join with some corner cases. +// //nolint:gosimple // generates false positive fmt.Sprintf warnings which keep aligned func TestVectorizedMergeJoin(t *testing.T) { store, clean := testkit.CreateMockStore(t) @@ -856,6 +857,7 @@ func TestVectorizedMergeJoin(t *testing.T) { } // TestVectorizedShuffleMergeJoin is used to test vectorized shuffle merge join with some corner cases. +// //nolint:gosimple // generates false positive fmt.Sprintf warnings which keep aligned func TestVectorizedShuffleMergeJoin(t *testing.T) { store, clean := testkit.CreateMockStore(t) diff --git a/executor/plan_replayer.go b/executor/plan_replayer.go index 22520d5d3b722..11900e074773f 100644 --- a/executor/plan_replayer.go +++ b/executor/plan_replayer.go @@ -117,8 +117,8 @@ func (e *PlanReplayerSingleExec) Next(ctx context.Context, req *chunk.Chunk) err // |-bindings.sql // |-sqls.sql // |_explain -// |-explain.txt // +// |-explain.txt func (e *PlanReplayerSingleExec) dumpSingle(path string) (fileName string, err error) { // Create path err = os.MkdirAll(path, os.ModePerm) diff --git a/executor/projection.go b/executor/projection.go index f43d264ebc74a..af6aa27491c77 100644 --- a/executor/projection.go +++ b/executor/projection.go @@ -120,59 +120,65 @@ func (e *ProjectionExec) open(ctx context.Context) error { // // Here we explain the execution flow of the parallel projection implementation. // There are 3 main components: -// 1. "projectionInputFetcher": Fetch input "Chunk" from child. -// 2. "projectionWorker": Do the projection work. -// 3. "ProjectionExec.Next": Return result to parent. +// 1. "projectionInputFetcher": Fetch input "Chunk" from child. +// 2. "projectionWorker": Do the projection work. +// 3. "ProjectionExec.Next": Return result to parent. // // 1. "projectionInputFetcher" gets its input and output resources from its // "inputCh" and "outputCh" channel, once the input and output resources are // abtained, it fetches child's result into "input.chk" and: -// a. Dispatches this input to the worker specified in "input.targetWorker" -// b. Dispatches this output to the main thread: "ProjectionExec.Next" -// c. Dispatches this output to the worker specified in "input.targetWorker" +// +// a. Dispatches this input to the worker specified in "input.targetWorker" +// b. Dispatches this output to the main thread: "ProjectionExec.Next" +// c. Dispatches this output to the worker specified in "input.targetWorker" +// // It is finished and exited once: -// a. There is no more input from child. -// b. "ProjectionExec" close the "globalFinishCh" +// +// a. There is no more input from child. +// b. "ProjectionExec" close the "globalFinishCh" // // 2. "projectionWorker" gets its input and output resources from its // "inputCh" and "outputCh" channel, once the input and output resources are // abtained, it calculates the projection result use "input.chk" as the input // and "output.chk" as the output, once the calculation is done, it: -// a. Sends "nil" or error to "output.done" to mark this input is finished. -// b. Returns the "input" resource to "projectionInputFetcher.inputCh" +// +// a. Sends "nil" or error to "output.done" to mark this input is finished. +// b. Returns the "input" resource to "projectionInputFetcher.inputCh" +// // They are finished and exited once: -// a. "ProjectionExec" closes the "globalFinishCh" +// +// a. "ProjectionExec" closes the "globalFinishCh" // // 3. "ProjectionExec.Next" gets its output resources from its "outputCh" channel. // After receiving an output from "outputCh", it should wait to receive a "nil" // or error from "output.done" channel. Once a "nil" or error is received: -// a. Returns this output to its parent -// b. Returns the "output" resource to "projectionInputFetcher.outputCh" // -// +-----------+----------------------+--------------------------+ -// | | | | -// | +--------+---------+ +--------+---------+ +--------+---------+ -// | | projectionWorker | + projectionWorker | ... + projectionWorker | -// | +------------------+ +------------------+ +------------------+ -// | ^ ^ ^ ^ ^ ^ -// | | | | | | | -// | inputCh outputCh inputCh outputCh inputCh outputCh -// | ^ ^ ^ ^ ^ ^ -// | | | | | | | -// | | | -// | | +----------------->outputCh -// | | | | -// | | | v -// | +-------+-------+--------+ +---------------------+ -// | | projectionInputFetcher | | ProjectionExec.Next | -// | +------------------------+ +---------+-----------+ -// | ^ ^ | -// | | | | -// | inputCh outputCh | -// | ^ ^ | -// | | | | -// +------------------------------+ +----------------------+ +// a. Returns this output to its parent +// b. Returns the "output" resource to "projectionInputFetcher.outputCh" // +// +-----------+----------------------+--------------------------+ +// | | | | +// | +--------+---------+ +--------+---------+ +--------+---------+ +// | | projectionWorker | + projectionWorker | ... + projectionWorker | +// | +------------------+ +------------------+ +------------------+ +// | ^ ^ ^ ^ ^ ^ +// | | | | | | | +// | inputCh outputCh inputCh outputCh inputCh outputCh +// | ^ ^ ^ ^ ^ ^ +// | | | | | | | +// | | | +// | | +----------------->outputCh +// | | | | +// | | | v +// | +-------+-------+--------+ +---------------------+ +// | | projectionInputFetcher | | ProjectionExec.Next | +// | +------------------------+ +---------+-----------+ +// | ^ ^ | +// | | | | +// | inputCh outputCh | +// | ^ ^ | +// | | | | +// +------------------------------+ +----------------------+ func (e *ProjectionExec) Next(ctx context.Context, req *chunk.Chunk) error { req.GrowAndReset(e.maxChunkSize) if e.isUnparallelExec() { @@ -341,13 +347,15 @@ type projectionInputFetcher struct { // run gets projectionInputFetcher's input and output resources from its // "inputCh" and "outputCh" channel, once the input and output resources are // abtained, it fetches child's result into "input.chk" and: -// a. Dispatches this input to the worker specified in "input.targetWorker" -// b. Dispatches this output to the main thread: "ProjectionExec.Next" -// c. Dispatches this output to the worker specified in "input.targetWorker" +// +// a. Dispatches this input to the worker specified in "input.targetWorker" +// b. Dispatches this output to the main thread: "ProjectionExec.Next" +// c. Dispatches this output to the worker specified in "input.targetWorker" // // It is finished and exited once: -// a. There is no more input from child. -// b. "ProjectionExec" close the "globalFinishCh" +// +// a. There is no more input from child. +// b. "ProjectionExec" close the "globalFinishCh" func (f *projectionInputFetcher) run(ctx context.Context) { defer trace.StartRegion(ctx, "ProjectionFetcher").End() var output *projectionOutput @@ -409,11 +417,13 @@ type projectionWorker struct { // "inputCh" and "outputCh" channel, once the input and output resources are // abtained, it calculate the projection result use "input.chk" as the input // and "output.chk" as the output, once the calculation is done, it: -// a. Sends "nil" or error to "output.done" to mark this input is finished. -// b. Returns the "input" resource to "projectionInputFetcher.inputCh". +// +// a. Sends "nil" or error to "output.done" to mark this input is finished. +// b. Returns the "input" resource to "projectionInputFetcher.inputCh". // // It is finished and exited once: -// a. "ProjectionExec" closes the "globalFinishCh". +// +// a. "ProjectionExec" closes the "globalFinishCh". func (w *projectionWorker) run(ctx context.Context) { defer trace.StartRegion(ctx, "ProjectionWorker").End() var output *projectionOutput diff --git a/executor/replace.go b/executor/replace.go index 27fc50efcab0b..221cbf87b2504 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -163,10 +163,10 @@ func (e *ReplaceExec) replaceRow(ctx context.Context, r toBeCheckedRow) error { // removeIndexRow removes the row which has a duplicated key. // the return values: -// 1. bool: true when the row is unchanged. This means no need to remove, and then add the row. -// 2. bool: true when found the duplicated key. This only means that duplicated key was found, -// and the row was removed. -// 3. error: the error. +// 1. bool: true when the row is unchanged. This means no need to remove, and then add the row. +// 2. bool: true when found the duplicated key. This only means that duplicated key was found, +// and the row was removed. +// 3. error: the error. func (e *ReplaceExec) removeIndexRow(ctx context.Context, txn kv.Transaction, r toBeCheckedRow) (bool, bool, error) { for _, uk := range r.uniqueKeys { val, err := txn.Get(ctx, uk.newKey) diff --git a/executor/set_config.go b/executor/set_config.go index 6313c9152d57f..2b5931750aa3f 100644 --- a/executor/set_config.go +++ b/executor/set_config.go @@ -167,8 +167,9 @@ func isValidInstance(instance string) bool { // ConvertConfigItem2JSON converts the config item specified by key and val to json. // For example: -// set config x key="val" ==> {"key":"val"} -// set config x key=233 ==> {"key":233} +// +// set config x key="val" ==> {"key":"val"} +// set config x key=233 ==> {"key":233} func ConvertConfigItem2JSON(ctx sessionctx.Context, key string, val expression.Expression) (body string, err error) { if val == nil { return "", errors.Errorf("cannot set config to null") diff --git a/executor/shuffle.go b/executor/shuffle.go index 9143afd032f10..e886ac12f9156 100644 --- a/executor/shuffle.go +++ b/executor/shuffle.go @@ -30,50 +30,53 @@ import ( ) // ShuffleExec is the executor to run other executors in a parallel manner. +// // 1. It fetches chunks from M `DataSources` (value of M depends on the actual executor, e.g. M = 1 for WindowExec, M = 2 for MergeJoinExec). +// // 2. It splits tuples from each `DataSource` into N partitions (Only "split by hash" is implemented so far). +// // 3. It invokes N workers in parallel, each one has M `receiver` to receive partitions from `DataSources` -// 4. It assigns partitions received as input to each worker and executes child executors. -// 5. It collects outputs from each worker, then sends outputs to its parent. // -// +-------------+ -// +-------| Main Thread | -// | +------+------+ -// | ^ -// | | -// | + -// v +++ -// outputHolderCh | | outputCh (1 x Concurrency) -// v +++ -// | ^ -// | | -// | +-------+-------+ -// v | | -// +--------------+ +--------------+ -// +----- | worker | ....... | worker | worker (N Concurrency): child executor, eg. WindowExec (+SortExec) -// | +------------+-+ +-+------------+ -// | ^ ^ -// | | | -// | +-+ +-+ ...... +-+ -// | | | | | | | -// | ... ... ... inputCh (Concurrency x 1) -// v | | | | | | -// inputHolderCh +++ +++ +++ -// v ^ ^ ^ -// | | | | -// | +------o----+ | -// | | +-----------------+-----+ -// | | | -// | +---+------------+------------+----+-----------+ -// | | Partition Splitter | -// | +--------------+-+------------+-+--------------+ -// | ^ -// | | -// | +---------------v-----------------+ -// +----------> | fetch data from DataSource | -// +---------------------------------+ +// 4. It assigns partitions received as input to each worker and executes child executors. // +// 5. It collects outputs from each worker, then sends outputs to its parent. // +// +-------------+ +// +-------| Main Thread | +// | +------+------+ +// | ^ +// | | +// | + +// v +++ +// outputHolderCh | | outputCh (1 x Concurrency) +// v +++ +// | ^ +// | | +// | +-------+-------+ +// v | | +// +--------------+ +--------------+ +// +----- | worker | ....... | worker | worker (N Concurrency): child executor, eg. WindowExec (+SortExec) +// | +------------+-+ +-+------------+ +// | ^ ^ +// | | | +// | +-+ +-+ ...... +-+ +// | | | | | | | +// | ... ... ... inputCh (Concurrency x 1) +// v | | | | | | +// inputHolderCh +++ +++ +++ +// v ^ ^ ^ +// | | | | +// | +------o----+ | +// | | +-----------------+-----+ +// | | | +// | +---+------------+------------+----+-----------+ +// | | Partition Splitter | +// | +--------------+-+------------+-+--------------+ +// | ^ +// | | +// | +---------------v-----------------+ +// +----------> | fetch data from DataSource | +// +---------------------------------+ type ShuffleExec struct { baseExecutor concurrency int diff --git a/executor/sort.go b/executor/sort.go index 83b7bc59264af..3fb4b42376406 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -102,12 +102,12 @@ func (e *SortExec) Open(ctx context.Context) error { // Next implements the Executor Next interface. // Sort constructs the result following these step: -// 1. Read as mush as rows into memory. -// 2. If memory quota is triggered, sort these rows in memory and put them into disk as partition 1, then reset -// the memory quota trigger and return to step 1 -// 3. If memory quota is not triggered and child is consumed, sort these rows in memory as partition N. -// 4. Merge sort if the count of partitions is larger than 1. If there is only one partition in step 4, it works -// just like in-memory sort before. +// 1. Read as mush as rows into memory. +// 2. If memory quota is triggered, sort these rows in memory and put them into disk as partition 1, then reset +// the memory quota trigger and return to step 1 +// 3. If memory quota is not triggered and child is consumed, sort these rows in memory as partition N. +// 4. Merge sort if the count of partitions is larger than 1. If there is only one partition in step 4, it works +// just like in-memory sort before. func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { req.Reset() if !e.fetched { diff --git a/executor/write.go b/executor/write.go index 49707f6417bf5..7c9aba9331c4d 100644 --- a/executor/write.go +++ b/executor/write.go @@ -47,8 +47,8 @@ var ( // `modified` means which columns are really modified. It's used for secondary indices. // Length of `oldData` and `newData` equals to length of `t.WritableCols()`. // The return values: -// 1. changed (bool) : does the update really change the row values. e.g. update set i = 1 where i = 1; -// 2. err (error) : error in the update. +// 1. changed (bool) : does the update really change the row values. e.g. update set i = 1 where i = 1; +// 2. err (error) : error in the update. func updateRecord(ctx context.Context, sctx sessionctx.Context, h kv.Handle, oldData, newData []types.Datum, modified []bool, t table.Table, onDup bool, memTracker *memory.Tracker) (bool, error) { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index 5a5f5936ec072..1c72eaa012149 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -1429,8 +1429,9 @@ func isTemporalColumn(expr Expression) bool { // If isExceptional is true, ExecptionalVal is returned. Or, CorrectVal is returned. // CorrectVal: The computed result. If the constant can be converted to int without exception, return the val. Else return 'con'(the input). // ExceptionalVal : It is used to get more information to check whether 'int column [cmp] const' is true/false -// If the op == LT,LE,GT,GE and it gets an Overflow when converting, return inf/-inf. -// If the op == EQ,NullEQ and the constant can never be equal to the int column, return ‘con’(the input, a non-int constant). +// +// If the op == LT,LE,GT,GE and it gets an Overflow when converting, return inf/-inf. +// If the op == EQ,NullEQ and the constant can never be equal to the int column, return ‘con’(the input, a non-int constant). func tryToConvertConstantInt(ctx sessionctx.Context, targetFieldType *types.FieldType, con *Constant) (_ *Constant, isExceptional bool) { if con.GetType().EvalType() == types.ETInt { return con, false @@ -1466,8 +1467,9 @@ func tryToConvertConstantInt(ctx sessionctx.Context, targetFieldType *types.Fiel // If isExceptional is true, ExecptionalVal is returned. Or, CorrectVal is returned. // CorrectVal: The computed result. If the constant can be converted to int without exception, return the val. Else return 'con'(the input). // ExceptionalVal : It is used to get more information to check whether 'int column [cmp] const' is true/false -// If the op == LT,LE,GT,GE and it gets an Overflow when converting, return inf/-inf. -// If the op == EQ,NullEQ and the constant can never be equal to the int column, return ‘con’(the input, a non-int constant). +// +// If the op == LT,LE,GT,GE and it gets an Overflow when converting, return inf/-inf. +// If the op == EQ,NullEQ and the constant can never be equal to the int column, return ‘con’(the input, a non-int constant). func RefineComparedConstant(ctx sessionctx.Context, targetFieldType types.FieldType, con *Constant, op opcode.Op) (_ *Constant, isExceptional bool) { dt, err := con.Eval(chunk.Row{}) if err != nil { diff --git a/expression/builtin_time.go b/expression/builtin_time.go index d617c97219e71..0e808c4286c3f 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -7400,12 +7400,12 @@ func CalAppropriateTime(minTime, maxTime, minSafeTime time.Time) time.Time { } // For a SafeTS t and a time range [t1, t2]: -// 1. If t < t1, we will use t1 as the result, -// and with it, a read request may fail because it's an unreached SafeTS. -// 2. If t1 <= t <= t2, we will use t as the result, and with it, -// a read request won't fail. -// 2. If t2 < t, we will use t2 as the result, -// and with it, a read request won't fail because it's bigger than the latest SafeTS. +// 1. If t < t1, we will use t1 as the result, +// and with it, a read request may fail because it's an unreached SafeTS. +// 2. If t1 <= t <= t2, we will use t as the result, and with it, +// a read request won't fail. +// 2. If t2 < t, we will use t2 as the result, +// and with it, a read request won't fail because it's bigger than the latest SafeTS. func calAppropriateTime(minTime, maxTime, minSafeTime time.Time) time.Time { if minSafeTime.Before(minTime) || minSafeTime.After(maxTime) { logutil.BgLogger().Warn("calAppropriateTime", diff --git a/expression/constant_propagation.go b/expression/constant_propagation.go index c8d625c7b8be1..0524ea23356fb 100644 --- a/expression/constant_propagation.go +++ b/expression/constant_propagation.go @@ -114,14 +114,16 @@ func validEqualCond(ctx sessionctx.Context, cond Expression) (*Column, *Constant // tryToReplaceCond aims to replace all occurrences of column 'src' and try to replace it with 'tgt' in 'cond' // It returns -// bool: if a replacement happened -// bool: if 'cond' contains non-deterministic expression -// Expression: the replaced expression, or original 'cond' if the replacement didn't happen +// +// bool: if a replacement happened +// bool: if 'cond' contains non-deterministic expression +// Expression: the replaced expression, or original 'cond' if the replacement didn't happen // // For example: -// for 'a, b, a < 3', it returns 'true, false, b < 3' -// for 'a, b, sin(a) + cos(a) = 5', it returns 'true, false, returns sin(b) + cos(b) = 5' -// for 'a, b, cast(a) < rand()', it returns 'false, true, cast(a) < rand()' +// +// for 'a, b, a < 3', it returns 'true, false, b < 3' +// for 'a, b, sin(a) + cos(a) = 5', it returns 'true, false, returns sin(b) + cos(b) = 5' +// for 'a, b, cast(a) < rand()', it returns 'false, true, cast(a) < rand()' func tryToReplaceCond(ctx sessionctx.Context, src *Column, tgt *Column, cond Expression, nullAware bool) (bool, bool, Expression) { if src.RetType.GetType() != tgt.RetType.GetType() { return false, false, cond diff --git a/expression/evaluator.go b/expression/evaluator.go index d8cf7bdfed2e3..b9af0ddda00cc 100644 --- a/expression/evaluator.go +++ b/expression/evaluator.go @@ -25,7 +25,8 @@ type columnEvaluator struct { // run evaluates "Column" expressions. // NOTE: It should be called after all the other expressions are evaluated -// since it will change the content of the input Chunk. +// +// since it will change the content of the input Chunk. func (e *columnEvaluator) run(ctx sessionctx.Context, input, output *chunk.Chunk) error { for inputIdx, outputIdxes := range e.inputIdxToOutputIdxes { if err := output.SwapColumn(outputIdxes[0], input, inputIdx); err != nil { diff --git a/expression/expression.go b/expression/expression.go index 586b876719fd2..df2d431e4f269 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -224,8 +224,9 @@ func ExprNotNull(expr Expression) bool { // HandleOverflowOnSelection handles Overflow errors when evaluating selection filters. // We should ignore overflow errors when evaluating selection conditions: -// INSERT INTO t VALUES ("999999999999999999"); -// SELECT * FROM t WHERE v; +// +// INSERT INTO t VALUES ("999999999999999999"); +// SELECT * FROM t WHERE v; func HandleOverflowOnSelection(sc *stmtctx.StatementContext, val int64, err error) (int64, error) { if sc.InSelectStmt && err != nil && types.ErrOverflow.Equal(err) { return -1, nil diff --git a/expression/util.go b/expression/util.go index 40d4ab3f95774..6eac9f0d7726e 100644 --- a/expression/util.go +++ b/expression/util.go @@ -139,9 +139,10 @@ func ExtractCorColumns(expr Expression) (cols []*CorrelatedColumn) { // It's often observed that the pattern of the caller like this: // // cols := ExtractColumns(...) -// for _, col := range cols { -// if xxx(col) {...} -// } +// +// for _, col := range cols { +// if xxx(col) {...} +// } // // Provide an additional filter argument, this can be done in one step. // To avoid allocation for cols that not need. @@ -702,8 +703,9 @@ func ContainOuterNot(expr Expression) bool { // Input `not` means whether there is `not` outside `expr` // // eg. -// not(0+(t.a == 1 and t.b == 2)) returns true -// not(t.a) and not(t.b) returns false +// +// not(0+(t.a == 1 and t.b == 2)) returns true +// not(t.a) and not(t.b) returns false func containOuterNot(expr Expression, not bool) bool { if f, ok := expr.(*ScalarFunction); ok { switch f.FuncName.L { diff --git a/infoschema/perfschema/const.go b/infoschema/perfschema/const.go index 13ab10a699eeb..4c0414885caf1 100644 --- a/infoschema/perfschema/const.go +++ b/infoschema/perfschema/const.go @@ -283,7 +283,6 @@ const tableTransCurrent = "CREATE TABLE if not exists performance_schema." + tab "NESTING_EVENT_TYPE ENUM('TRANSACTION','STATEMENT','STAGE'));" // tableTransHistory contains the column name definitions for table events_transactions_history, same as MySQL. -// const tableTransHistory = "CREATE TABLE if not exists performance_schema." + tableNameEventsTransactionsHistory + " (" + "THREAD_ID BIGINT(20) UNSIGNED NOT NULL," + "EVENT_ID BIGINT(20) UNSIGNED NOT NULL," + diff --git a/infoschema/tables.go b/infoschema/tables.go index 765c0bfe6a02e..2a8a7c4fb6607 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1471,10 +1471,11 @@ var tablePlacementPoliciesCols = []columnInfo{ // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. // The returned description string may be: -// - "NOT_SHARDED": for tables that SHARD_ROW_ID_BITS is not specified. -// - "NOT_SHARDED(PK_IS_HANDLE)": for tables of which primary key is row id. -// - "PK_AUTO_RANDOM_BITS={bit_number}": for tables of which primary key is sharded row id. -// - "SHARD_BITS={bit_number}": for tables that with SHARD_ROW_ID_BITS. +// - "NOT_SHARDED": for tables that SHARD_ROW_ID_BITS is not specified. +// - "NOT_SHARDED(PK_IS_HANDLE)": for tables of which primary key is row id. +// - "PK_AUTO_RANDOM_BITS={bit_number}": for tables of which primary key is sharded row id. +// - "SHARD_BITS={bit_number}": for tables that with SHARD_ROW_ID_BITS. +// // The returned nil indicates that sharding information is not suitable for the table(for example, when the table is a View). // This function is exported for unit test. func GetShardingInfo(dbInfo *model.DBInfo, tableInfo *model.TableInfo) interface{} { diff --git a/kv/key.go b/kv/key.go index 3e68b5fc80dd7..d4227cff4b87f 100644 --- a/kv/key.go +++ b/kv/key.go @@ -40,10 +40,10 @@ func (k Key) Next() Key { // // Assume there are keys like: // -// rowkey1 -// rowkey1_column1 -// rowkey1_column2 -// rowKey2 +// rowkey1 +// rowkey1_column1 +// rowkey1_column2 +// rowKey2 // // If we seek 'rowkey1' Next, we will get 'rowkey1_column1'. // If we seek 'rowkey1' PrefixNext, we will get 'rowkey2'. diff --git a/kv/mpp.go b/kv/mpp.go index 012a182aacd1a..2e026a8a55a67 100644 --- a/kv/mpp.go +++ b/kv/mpp.go @@ -49,7 +49,7 @@ func (t *MPPTask) ToPB() *mpp.TaskMeta { return meta } -//MppTaskStates denotes the state of mpp tasks +// MppTaskStates denotes the state of mpp tasks type MppTaskStates uint8 const ( diff --git a/kv/txn.go b/kv/txn.go index dd51b7a2e56fe..d140cc3328b0e 100644 --- a/kv/txn.go +++ b/kv/txn.go @@ -81,7 +81,8 @@ func (ib *innerTxnStartTsBox) getMinStartTS(now time.Time, startTSLowerLimit uin // PrintLongTimeInternalTxn print the internal transaction information. // runByFunction true means the transaction is run by `RunInNewTxn`, -// false means the transaction is run by internal session. +// +// false means the transaction is run by internal session. func PrintLongTimeInternalTxn(now time.Time, startTS uint64, runByFunction bool) { if startTS > 0 { innerTxnStartTime := oracle.GetTimeFromTS(startTS) diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index 5d8745cf78697..964b5fe94ee9b 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -659,7 +659,9 @@ func CalcSequenceBatchSize(base, size, increment, offset, MIN, MAX int64) (int64 // SeekToFirstSequenceValue seeks to the next valid value (must be in range of [MIN, MAX]), // the bool indicates whether the first value is got. // The seeking formula is describe as below: -// nr := (base + increment - offset) / increment +// +// nr := (base + increment - offset) / increment +// // first := nr*increment + offset // Because formula computation will overflow Int64, so we transfer it to uint64 for distance computation. func SeekToFirstSequenceValue(base, increment, offset, MIN, MAX int64) (int64, bool) { @@ -1056,9 +1058,11 @@ func TestModifyBaseAndEndInjection(alloc Allocator, base, end int64) { // ShardIDLayout is used to calculate the bits length of different segments in auto id. // Generally, an auto id is consist of 3 segments: sign bit, shard bits and incremental bits. -// Take ``a BIGINT AUTO_INCREMENT PRIMARY KEY`` as an example, assume that the `shard_row_id_bits` = 5, +// Take “a BIGINT AUTO_INCREMENT PRIMARY KEY“ as an example, assume that the `shard_row_id_bits` = 5, // the layout is like -// | [sign_bit] (1 bit) | [shard_bits] (5 bits) | [incremental_bits] (64-1-5=58 bits) | +// +// | [sign_bit] (1 bit) | [shard_bits] (5 bits) | [incremental_bits] (64-1-5=58 bits) | +// // Please always use NewShardIDLayout() to instantiate. type ShardIDLayout struct { FieldType *types.FieldType diff --git a/planner/cascades/optimize.go b/planner/cascades/optimize.go index c1d73f21f8c89..1903daad28ba7 100644 --- a/planner/cascades/optimize.go +++ b/planner/cascades/optimize.go @@ -65,16 +65,16 @@ func (opt *Optimizer) GetImplementationRules(node plannercore.LogicalPlan) []Imp // FindBestPlan is the optimization entrance of the cascades planner. The // optimization is composed of 3 phases: preprocessing, exploration and implementation. // -//------------------------------------------------------------------------------ +// ------------------------------------------------------------------------------ // Phase 1: Preprocessing -//------------------------------------------------------------------------------ +// ------------------------------------------------------------------------------ // // The target of this phase is to preprocess the plan tree by some heuristic // rules which should always be beneficial, for example Column Pruning. // -//------------------------------------------------------------------------------ +// ------------------------------------------------------------------------------ // Phase 2: Exploration -//------------------------------------------------------------------------------ +// ------------------------------------------------------------------------------ // // The target of this phase is to explore all the logically equivalent // expressions by exploring all the equivalent group expressions of each group. @@ -86,9 +86,9 @@ func (opt *Optimizer) GetImplementationRules(node plannercore.LogicalPlan) []Imp // graph, where nodes are expressions and directed edges are the transformation // rules. // -//------------------------------------------------------------------------------ +// ------------------------------------------------------------------------------ // Phase 3: Implementation -//------------------------------------------------------------------------------ +// ------------------------------------------------------------------------------ // // The target of this phase is to search the best physical plan for a Group // which satisfies a certain required physical property. diff --git a/planner/cascades/stringer.go b/planner/cascades/stringer.go index 5bf84a7047422..99638eaaebe36 100644 --- a/planner/cascades/stringer.go +++ b/planner/cascades/stringer.go @@ -59,8 +59,9 @@ func toString(g *memo.Group, idMap map[*memo.Group]int, visited map[*memo.Group] // groupToString only stringifies a single Group. // Format: // Group#1 Column: [Column#1,Column#2,Column#13] Unique key: [] -// Selection_4 input:[Group#2], eq(Column#13, Column#2), gt(Column#1, 10) -// Projection_15 input:Group#3 Column#1, Column#2 +// +// Selection_4 input:[Group#2], eq(Column#13, Column#2), gt(Column#1, 10) +// Projection_15 input:Group#3 Column#1, Column#2 func groupToString(g *memo.Group, idMap map[*memo.Group]int) []string { schema := g.Prop.Schema colStrs := make([]string, 0, len(schema.Columns)) diff --git a/planner/cascades/transformation_rules.go b/planner/cascades/transformation_rules.go index 21dd9614fe90b..07803125258a3 100644 --- a/planner/cascades/transformation_rules.go +++ b/planner/cascades/transformation_rules.go @@ -240,9 +240,10 @@ func NewRulePushSelDownIndexScan() Transformation { // OnTransform implements Transformation interface. // It will transform `Selection -> IndexScan` to: -// `IndexScan(with a new access range)` or -// `Selection -> IndexScan(with a new access range)` -// or just keep the two GroupExprs unchanged. +// +// `IndexScan(with a new access range)` or +// `Selection -> IndexScan(with a new access range)` +// or just keep the two GroupExprs unchanged. func (r *PushSelDownIndexScan) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { sel := old.GetExpr().ExprNode.(*plannercore.LogicalSelection) is := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalIndexScan) diff --git a/planner/core/cacheable_checker.go b/planner/core/cacheable_checker.go index 3caec678692fb..90c0fcc3bbd83 100644 --- a/planner/core/cacheable_checker.go +++ b/planner/core/cacheable_checker.go @@ -52,8 +52,9 @@ func CacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.Info } // cacheableChecker checks whether a query's plan can be cached, querys that: -// 1. have ExistsSubqueryExpr, or -// 2. have VariableExpr +// 1. have ExistsSubqueryExpr, or +// 2. have VariableExpr +// // will not be cached currently. // NOTE: we can add more rules in the future. type cacheableChecker struct { diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 33a082880ca09..2f874bdbe2214 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -1035,7 +1035,8 @@ type Simple struct { } // PhysicalSimpleWrapper is a wrapper of `Simple` to implement physical plan interface. -// Used for simple statements executing in coprocessor. +// +// Used for simple statements executing in coprocessor. type PhysicalSimpleWrapper struct { basePhysicalPlan Inner Simple diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index cbbc770fdf985..fecce00ced731 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1326,11 +1326,12 @@ loopOtherConds: } // removeUselessEqAndInFunc removes the useless eq/in conditions. It's designed for the following case: -// t1 join t2 on t1.a=t2.a and t1.c=t2.c where t1.b > t2.b-10 and t1.b < t2.b+10 there's index(a, b, c) on t1. -// In this case the curIdxOff2KeyOff is [0 -1 1] and the notKeyEqAndIn is []. -// It's clearly that the column c cannot be used to access data. So we need to remove it and reset the IdxOff2KeyOff to -// [0 -1 -1]. -// So that we can use t1.a=t2.a and t1.b > t2.b-10 and t1.b < t2.b+10 to build ranges then access data. +// +// t1 join t2 on t1.a=t2.a and t1.c=t2.c where t1.b > t2.b-10 and t1.b < t2.b+10 there's index(a, b, c) on t1. +// In this case the curIdxOff2KeyOff is [0 -1 1] and the notKeyEqAndIn is []. +// It's clearly that the column c cannot be used to access data. So we need to remove it and reset the IdxOff2KeyOff to +// [0 -1 -1]. +// So that we can use t1.a=t2.a and t1.b > t2.b-10 and t1.b < t2.b+10 to build ranges then access data. func (ijHelper *indexJoinBuildHelper) removeUselessEqAndInFunc(idxCols []*expression.Column, notKeyEqAndIn []expression.Expression, outerJoinKeys []*expression.Column) (usefulEqAndIn, uselessOnes []expression.Expression) { ijHelper.curPossibleUsedKeys = make([]*expression.Column, 0, len(idxCols)) for idxColPos, notKeyColPos := 0, 0; idxColPos < len(idxCols); idxColPos++ { diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 6cdd36ae15efe..a698a2edfd685 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -264,9 +264,10 @@ func (er *expressionRewriter) ctxStackAppend(col expression.Expression, name *ty // 1. If op are EQ or NE or NullEQ, constructBinaryOpFunctions converts (a0,a1,a2) op (b0,b1,b2) to (a0 op b0) and (a1 op b1) and (a2 op b2) // 2. Else constructBinaryOpFunctions converts (a0,a1,a2) op (b0,b1,b2) to // `IF( a0 NE b0, a0 op b0, -// IF ( isNull(a0 NE b0), Null, -// IF ( a1 NE b1, a1 op b1, -// IF ( isNull(a1 NE b1), Null, a2 op b2))))` +// +// IF ( isNull(a0 NE b0), Null, +// IF ( a1 NE b1, a1 op b1, +// IF ( isNull(a1 NE b1), Null, a2 op b2))))` func (er *expressionRewriter) constructBinaryOpFunction(l expression.Expression, r expression.Expression, op string) (expression.Expression, error) { lLen, rLen := expression.GetRowLen(l), expression.GetRowLen(r) if lLen == 1 && rLen == 1 { diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 249cfdc1f69a6..20ba420fa898e 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1275,8 +1275,9 @@ func (ds *DataSource) buildIndexMergeTableScan(prop *property.PhysicalProperty, // `remaining`: exprs that can NOT be pushed to TiKV but can be pushed to other storage engines. // Why do we need this func? // IndexMerge only works on TiKV, so we need to find all exprs that cannot be pushed to TiKV, and add a new Selection above IndexMergeReader. -// But the new Selection should exclude the exprs that can NOT be pushed to ALL the storage engines. -// Because these exprs have already been put in another Selection(check rule_predicate_push_down). +// +// But the new Selection should exclude the exprs that can NOT be pushed to ALL the storage engines. +// Because these exprs have already been put in another Selection(check rule_predicate_push_down). func extractFiltersForIndexMerge(sc *stmtctx.StatementContext, client kv.Client, filters []expression.Expression) (pushed []expression.Expression, remaining []expression.Expression) { for _, expr := range filters { if expression.CanExprsPushDown(sc, []expression.Expression{expr}, client, kv.TiKV) { @@ -1457,11 +1458,12 @@ func (is *PhysicalIndexScan) getScanRowSize() float64 { // initSchema is used to set the schema of PhysicalIndexScan. Before calling this, // make sure the following field of PhysicalIndexScan are initialized: -// PhysicalIndexScan.Table *model.TableInfo -// PhysicalIndexScan.Index *model.IndexInfo -// PhysicalIndexScan.Index.Columns []*IndexColumn -// PhysicalIndexScan.IdxCols []*expression.Column -// PhysicalIndexScan.Columns []*model.ColumnInfo +// +// PhysicalIndexScan.Table *model.TableInfo +// PhysicalIndexScan.Index *model.IndexInfo +// PhysicalIndexScan.Index.Columns []*IndexColumn +// PhysicalIndexScan.IdxCols []*expression.Column +// PhysicalIndexScan.Columns []*model.ColumnInfo func (is *PhysicalIndexScan) initSchema(idxExprCols []*expression.Column, isDoubleRead bool) { indexCols := make([]*expression.Column, len(is.IdxCols), len(is.Index.Columns)+1) copy(indexCols, is.IdxCols) diff --git a/planner/core/find_best_task_test.go b/planner/core/find_best_task_test.go index bf39badb8a15f..1589b05fc3c86 100644 --- a/planner/core/find_best_task_test.go +++ b/planner/core/find_best_task_test.go @@ -48,13 +48,13 @@ func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty, planCoun // mockLogicalPlan4Test is a LogicalPlan which is used for unit test. // The basic assumption: -// 1. mockLogicalPlan4Test can generate tow kinds of physical plan: physicalPlan1 and -// physicalPlan2. physicalPlan1 can pass the property only when they are the same -// order; while physicalPlan2 cannot match any of the property(in other words, we can -// generate it only when then property is empty). -// 2. We have a hint for physicalPlan2. -// 3. If the property is empty, we still need to check `canGeneratePlan2` to decide -// whether it can generate physicalPlan2. +// 1. mockLogicalPlan4Test can generate tow kinds of physical plan: physicalPlan1 and +// physicalPlan2. physicalPlan1 can pass the property only when they are the same +// order; while physicalPlan2 cannot match any of the property(in other words, we can +// generate it only when then property is empty). +// 2. We have a hint for physicalPlan2. +// 3. If the property is empty, we still need to check `canGeneratePlan2` to decide +// whether it can generate physicalPlan2. type mockLogicalPlan4Test struct { baseLogicalPlan // hasHintForPlan2 indicates whether this mockPlan contains hint. diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index d6e2fa9d92b0c..4336c336ff95b 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -825,10 +825,10 @@ func (b *PlanBuilder) buildJoin(ctx context.Context, joinNode *ast.Join) (Logica // on the "USING" clause. // // According to the standard SQL, columns are ordered in the following way: -// 1. coalesced common columns of "leftPlan" and "rightPlan", in the order they -// appears in "leftPlan". -// 2. the rest columns in "leftPlan", in the order they appears in "leftPlan". -// 3. the rest columns in "rightPlan", in the order they appears in "rightPlan". +// 1. coalesced common columns of "leftPlan" and "rightPlan", in the order they +// appears in "leftPlan". +// 2. the rest columns in "leftPlan", in the order they appears in "leftPlan". +// 3. the rest columns in "rightPlan", in the order they appears in "rightPlan". func (b *PlanBuilder) buildUsingClause(p *LogicalJoin, leftPlan, rightPlan LogicalPlan, join *ast.Join) error { filter := make(map[string]bool, len(join.Using)) for _, col := range join.Using { @@ -849,9 +849,10 @@ func (b *PlanBuilder) buildUsingClause(p *LogicalJoin, leftPlan, rightPlan Logic // buildNaturalJoin builds natural join output schema. It finds out all the common columns // then using the same mechanism as buildUsingClause to eliminate redundant columns and build join conditions. // According to standard SQL, producing this display order: -// All the common columns -// Every column in the first (left) table that is not a common column -// Every column in the second (right) table that is not a common column +// +// All the common columns +// Every column in the first (left) table that is not a common column +// Every column in the second (right) table that is not a common column func (b *PlanBuilder) buildNaturalJoin(p *LogicalJoin, leftPlan, rightPlan LogicalPlan, join *ast.Join) error { err := b.coalesceCommonColumns(p, leftPlan, rightPlan, join.Tp, nil) if err != nil { @@ -1790,7 +1791,9 @@ func (b *PlanBuilder) buildUnion(ctx context.Context, selects []LogicalPlan, aft // divideUnionSelectPlans resolves union's select stmts to logical plans. // and divide result plans into "union-distinct" and "union-all" parts. // divide rule ref: -// https://dev.mysql.com/doc/refman/5.7/en/union.html +// +// https://dev.mysql.com/doc/refman/5.7/en/union.html +// // "Mixed UNION types are treated such that a DISTINCT union overrides any ALL union to its left." func (b *PlanBuilder) divideUnionSelectPlans(ctx context.Context, selects []LogicalPlan, setOprTypes []*ast.SetOprType) (distinctSelects []LogicalPlan, allSelects []LogicalPlan, err error) { firstUnionAllIdx := 0 diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index 45f9e1f8f5529..6fdf04ea5a873 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -424,14 +424,15 @@ type testTablePartitionInfo struct { } // getPartitionInfoFromPlan uses to extract table partition information from the plan tree string. Here is an example, the plan is like below: -// "Projection_7 80.00 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", -// "└─HashJoin_9 80.00 root CARTESIAN inner join", -// " ├─TableReader_12(Build) 8.00 root partition:p1 data:Selection_11", -// " │ └─Selection_11 8.00 cop[tikv] 1, eq(test_partition.t2.b, 6), in(test_partition.t2.a, 6, 7, 8)", -// " │ └─TableFullScan_10 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", -// " └─TableReader_15(Probe) 10.00 root partition:p0 data:Selection_14", -// " └─Selection_14 10.00 cop[tikv] 1, eq(test_partition.t1.a, 5)", -// " └─TableFullScan_13 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" +// +// "Projection_7 80.00 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", +// "└─HashJoin_9 80.00 root CARTESIAN inner join", +// " ├─TableReader_12(Build) 8.00 root partition:p1 data:Selection_11", +// " │ └─Selection_11 8.00 cop[tikv] 1, eq(test_partition.t2.b, 6), in(test_partition.t2.a, 6, 7, 8)", +// " │ └─TableFullScan_10 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", +// " └─TableReader_15(Probe) 10.00 root partition:p0 data:Selection_14", +// " └─Selection_14 10.00 cop[tikv] 1, eq(test_partition.t1.a, 5)", +// " └─TableFullScan_13 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" // // The return table partition info is: t1: p0; t2: p1 func getPartitionInfoFromPlan(plan []string) string { @@ -646,7 +647,7 @@ partition by range (a) ( tk.MustQuery("select * from t3 where not (a != 1)").Check(testkit.Rows("1")) } -//issue 22079 +// issue 22079 func TestRangePartitionPredicatePruner(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index c5c43df0a0a91..0b32c225c7c84 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1395,12 +1395,15 @@ func (p *PhysicalWindow) Clone() (PhysicalPlan, error) { // PhysicalShuffle represents a shuffle plan. // `Tails` and `DataSources` are the last plan within and the first plan following the "shuffle", respectively, -// to build the child executors chain. +// +// to build the child executors chain. +// // Take `Window` operator for example: -// Shuffle -> Window -> Sort -> DataSource, will be separated into: -// ==> Shuffle: for main thread -// ==> Window -> Sort(:Tail) -> shuffleWorker: for workers -// ==> DataSource: for `fetchDataAndSplit` thread +// +// Shuffle -> Window -> Sort -> DataSource, will be separated into: +// ==> Shuffle: for main thread +// ==> Window -> Sort(:Tail) -> shuffleWorker: for workers +// ==> DataSource: for `fetchDataAndSplit` thread type PhysicalShuffle struct { basePhysicalPlan diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 25b7818374338..eabcb22ed8815 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -705,8 +705,10 @@ func (p *preprocessor) checkAutoIncrement(stmt *ast.CreateTableStmt) { // checkSetOprSelectList checks union's selectList. // refer: https://dev.mysql.com/doc/refman/5.7/en/union.html -// https://mariadb.com/kb/en/intersect/ -// https://mariadb.com/kb/en/except/ +// +// https://mariadb.com/kb/en/intersect/ +// https://mariadb.com/kb/en/except/ +// // "To apply ORDER BY or LIMIT to an individual SELECT, place the clause inside the parentheses that enclose the SELECT." func (p *preprocessor) checkSetOprSelectList(stmt *ast.SetOprSelectList) { for _, sel := range stmt.Selects[:len(stmt.Selects)-1] { @@ -1689,9 +1691,9 @@ func (p *preprocessor) updateStateFromStaleReadProcessor() error { // ensureInfoSchema get the infoschema from the preprocessor. // there some situations: -// - the stmt specifies the schema version. -// - session variable -// - transaction context +// - the stmt specifies the schema version. +// - session variable +// - transaction context func (p *preprocessor) ensureInfoSchema() infoschema.InfoSchema { if p.InfoSchema != nil { return p.InfoSchema diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 496817d87c3d9..cae12d5b367fc 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -288,8 +288,8 @@ func (a *aggregationPushDownSolver) checkAnyCountAndSum(aggFuncs []*aggregation. } // TODO: -// 1. https://github.com/pingcap/tidb/issues/16355, push avg & distinct functions across join -// 2. remove this method and use splitPartialAgg instead for clean code. +// 1. https://github.com/pingcap/tidb/issues/16355, push avg & distinct functions across join +// 2. remove this method and use splitPartialAgg instead for clean code. func (a *aggregationPushDownSolver) makeNewAgg(ctx sessionctx.Context, aggFuncs []*aggregation.AggFuncDesc, gbyCols []*expression.Column, aggHints aggHintInfo, blockOffset int, nullGenerating bool) (*LogicalAggregation, error) { agg := LogicalAggregation{ diff --git a/planner/core/rule_join_elimination.go b/planner/core/rule_join_elimination.go index c7ed935b8dc99..80352f56e03d2 100644 --- a/planner/core/rule_join_elimination.go +++ b/planner/core/rule_join_elimination.go @@ -28,12 +28,12 @@ type outerJoinEliminator struct { } // tryToEliminateOuterJoin will eliminate outer join plan base on the following rules -// 1. outer join elimination: For example left outer join, if the parent only use the -// columns from left table and the join key of right table(the inner table) is a unique -// key of the right table. the left outer join can be eliminated. -// 2. outer join elimination with duplicate agnostic aggregate functions: For example left outer join. -// If the parent only use the columns from left table with 'distinct' label. The left outer join can -// be eliminated. +// 1. outer join elimination: For example left outer join, if the parent only use the +// columns from left table and the join key of right table(the inner table) is a unique +// key of the right table. the left outer join can be eliminated. +// 2. outer join elimination with duplicate agnostic aggregate functions: For example left outer join. +// If the parent only use the columns from left table with 'distinct' label. The left outer join can +// be eliminated. func (o *outerJoinEliminator) tryToEliminateOuterJoin(p *LogicalJoin, aggCols []*expression.Column, parentCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, bool, error) { var innerChildIdx int switch p.JoinType { @@ -153,10 +153,10 @@ func (o *outerJoinEliminator) isInnerJoinKeysContainIndex(innerPlan LogicalPlan, // It extracts all the columns from the duplicate agnostic aggregate functions. // The returned column set is nil if not all the aggregate functions are duplicate agnostic. // Only the following functions are considered to be duplicate agnostic: -// 1. MAX(arg) -// 2. MIN(arg) -// 3. FIRST_ROW(arg) -// 4. Other agg functions with DISTINCT flag, like SUM(DISTINCT arg) +// 1. MAX(arg) +// 2. MIN(arg) +// 3. FIRST_ROW(arg) +// 4. Other agg functions with DISTINCT flag, like SUM(DISTINCT arg) func GetDupAgnosticAggCols( p LogicalPlan, oldAggCols []*expression.Column, // Reuse the original buffer. diff --git a/planner/core/rule_join_reorder_dp.go b/planner/core/rule_join_reorder_dp.go index c91d74e1b7c28..71266328315f8 100644 --- a/planner/core/rule_join_reorder_dp.go +++ b/planner/core/rule_join_reorder_dp.go @@ -160,7 +160,8 @@ func (s *joinReorderDPSolver) bfsGraph(startNode int, visited []bool, adjacents // dpGraph is the core part of this algorithm. // It implements the traditional join reorder algorithm: DP by subset using the following formula: -// bestPlan[S:set of node] = the best one among Join(bestPlan[S1:subset of S], bestPlan[S2: S/S1]) +// +// bestPlan[S:set of node] = the best one among Join(bestPlan[S1:subset of S], bestPlan[S2: S/S1]) func (s *joinReorderDPSolver) dpGraph(visitID2NodeID, nodeID2VisitID []int, joinGroup []LogicalPlan, totalEqEdges []joinGroupEqEdge, totalNonEqEdges []joinGroupNonEqEdge, tracer *joinReorderTrace) (LogicalPlan, error) { nodeCnt := uint(len(visitID2NodeID)) diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index 309f4ac116bc4..4b648a9254960 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -33,8 +33,10 @@ type joinReorderGreedySolver struct { // current join tree. // // cumulative join cost = CumCount(lhs) + CumCount(rhs) + RowCount(join) -// For base node, its CumCount equals to the sum of the count of its subtree. -// See baseNodeCumCost for more details. +// +// For base node, its CumCount equals to the sum of the count of its subtree. +// See baseNodeCumCost for more details. +// // TODO: this formula can be changed to real physical cost in future. // // For the nodes and join trees which don't have a join equal condition to diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index aa0babe749cf1..3ab21cee1403a 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -47,15 +47,17 @@ const FullRange = -1 // Used by static partition prune mode. // // create table t (id int) partition by range (id) -// (partition p1 values less than (10), -// partition p2 values less than (20), -// partition p3 values less than (30)) +// +// (partition p1 values less than (10), +// partition p2 values less than (20), +// partition p3 values less than (30)) // // select * from t is equal to // select * from (union all -// select * from p1 where id < 10 -// select * from p2 where id < 20 -// select * from p3 where id < 30) +// +// select * from p1 where id < 10 +// select * from p2 where id < 20 +// select * from p3 where id < 30) // // partitionProcessor is here because it's easier to prune partition after predicate push down. type partitionProcessor struct{} diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index d0b9553d55085..f520b984c627f 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -879,8 +879,10 @@ func (adder *exprPrefixAdder) addExprPrefix4ShardIndex() ([]expression.Expressio // AddExprPrefix4CNFCond // add the prefix expression for CNF condition, e.g. `WHERE a = 1`, `WHERE a = 1 AND b = 10`, ...... // @param[in] conds the original condtion of the datasoure. e.g. `WHERE t1.a = 1 AND t1.b = 10 AND t2.a = 20`. -// if current datasource is `t1`, conds is {t1.a = 1, t1.b = 10}. if current datasource is -// `t2`, conds is {t2.a = 20} +// +// if current datasource is `t1`, conds is {t1.a = 1, t1.b = 10}. if current datasource is +// `t2`, conds is {t2.a = 20} +// // @return - the new condition after adding expression prefix func (adder *exprPrefixAdder) addExprPrefix4CNFCond(conds []expression.Expression) ([]expression.Expression, error) { newCondtionds, err := ranger.AddExpr4EqAndInCondition(adder.sctx, @@ -893,7 +895,9 @@ func (adder *exprPrefixAdder) addExprPrefix4CNFCond(conds []expression.Expressio // add the prefix expression for DNF condition, e.g. `WHERE a = 1 OR a = 10`, ...... // The condition returned is `WHERE (tidb_shard(a) = 214 AND a = 1) OR (tidb_shard(a) = 142 AND a = 10)` // @param[in] condition the original condtion of the datasoure. e.g. `WHERE a = 1 OR a = 10`. -// condtion is `a = 1 OR a = 10` +// +// condtion is `a = 1 OR a = 10` +// // @return - the new condition after adding expression prefix. It's still a LogicOr expression. func (adder *exprPrefixAdder) addExprPrefix4DNFCond(condition *expression.ScalarFunction) ([]expression.Expression, error) { var err error diff --git a/planner/core/rule_result_reorder.go b/planner/core/rule_result_reorder.go index 7ea7d73556b4d..a42943dc354df 100644 --- a/planner/core/rule_result_reorder.go +++ b/planner/core/rule_result_reorder.go @@ -22,17 +22,19 @@ import ( ) /* - resultReorder reorder query results. - NOTE: it's not a common rule for all queries, it's specially implemented for a few customers. +resultReorder reorder query results. +NOTE: it's not a common rule for all queries, it's specially implemented for a few customers. - Results of some queries are not ordered, for example: - create table t (a int); insert into t values (1), (2); select a from t; - In the case above, the result can be `1 2` or `2 1`, which is not ordered. - This rule reorders results by modifying or injecting a Sort operator: - 1. iterate the plan from the root, and ignore all input-order operators (Sel/Proj/Limit); - 2. when meeting the first non-input-order operator, - 2.1. if it's a Sort, update it by appending all output columns into its order-by list, - 2.2. otherwise, inject a new Sort upon this operator. +Results of some queries are not ordered, for example: + + create table t (a int); insert into t values (1), (2); select a from t; + +In the case above, the result can be `1 2` or `2 1`, which is not ordered. +This rule reorders results by modifying or injecting a Sort operator: + 1. iterate the plan from the root, and ignore all input-order operators (Sel/Proj/Limit); + 2. when meeting the first non-input-order operator, + 2.1. if it's a Sort, update it by appending all output columns into its order-by list, + 2.2. otherwise, inject a new Sort upon this operator. */ type resultReorder struct { } diff --git a/planner/core/task.go b/planner/core/task.go index df07ffd4ab91b..bec15641decd1 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -987,7 +987,8 @@ func (p *PhysicalTopN) getPushedDownTopN(childPlan PhysicalPlan) *PhysicalTopN { // canPushToIndexPlan checks if this TopN can be pushed to the index side of copTask. // It can be pushed to the index side when all columns used by ByItems are available from the index side and -// there's no prefix index column. +// +// there's no prefix index column. func (p *PhysicalTopN) canPushToIndexPlan(indexPlan PhysicalPlan, byItemCols []*expression.Column) bool { schema := indexPlan.Schema() for _, col := range byItemCols { diff --git a/planner/funcdep/fd_graph.go b/planner/funcdep/fd_graph.go index 4dee30d437a52..4e85907240ae8 100644 --- a/planner/funcdep/fd_graph.go +++ b/planner/funcdep/fd_graph.go @@ -603,118 +603,119 @@ func (s *FDSet) MakeCartesianProduct(rhs *FDSet) { // // Notification: // 1: the origin FD from the left side (rows-supplying) over the result of outer join filtered are preserved because -// it may be duplicated by multi-matching, but actually, they are the same left rows (don't violate FD definition). +// +// it may be duplicated by multi-matching, but actually, they are the same left rows (don't violate FD definition). // // 2: the origin FD from the right side (nulls-supplying) over the result of outer join filtered may not be valid anymore. // -// <1> strict FD may be wakened as a lax one. But if at least one non-NULL column is part of the determinant, the -// strict FD can be preserved. -// a b | c d e -// ------+---------------- -// 1 1 | 1 NULL 1 -// 1 2 | NULL NULL NULL -// 2 1 | NULL NULL NULL -// left join with (a,b) * (c,d,e) on (a=c and b=1), if there is a strict FD {d} -> {e} on the rhs. After supplied -// with null values, {d} -> {e} are degraded to a lax one {d} ~~> {e} as you see. the origin and supplied null value -// for d column determine different dependency. NULL -> 1 and NULL -> NULL which breaks strict FD definition. +// <1> strict FD may be wakened as a lax one. But if at least one non-NULL column is part of the determinant, the +// strict FD can be preserved. +// a b | c d e +// ------+---------------- +// 1 1 | 1 NULL 1 +// 1 2 | NULL NULL NULL +// 2 1 | NULL NULL NULL +// left join with (a,b) * (c,d,e) on (a=c and b=1), if there is a strict FD {d} -> {e} on the rhs. After supplied +// with null values, {d} -> {e} are degraded to a lax one {d} ~~> {e} as you see. the origin and supplied null value +// for d column determine different dependency. NULL -> 1 and NULL -> NULL which breaks strict FD definition. // -// If the determinant contains at least a not null column for example c here, FD like {c,d} -> {e} can survive -// after the left join. Because you can not find two same key, one from the origin rows and the other one from the -// supplied rows. +// If the determinant contains at least a not null column for example c here, FD like {c,d} -> {e} can survive +// after the left join. Because you can not find two same key, one from the origin rows and the other one from the +// supplied rows. // -// for lax FD, the supplied rows of null values don't affect lax FD itself. So we can keep it. +// for lax FD, the supplied rows of null values don't affect lax FD itself. So we can keep it. // -// <2> The FDSet should remove constant FD since null values may be substituted for some unmatched left rows. NULL is not a -// constant anymore. +// <2> The FDSet should remove constant FD since null values may be substituted for some unmatched left rows. NULL is not a +// constant anymore. // -// <3> equivalence FD should be removed since substituted null values are not equal to the other substituted null value. +// <3> equivalence FD should be removed since substituted null values are not equal to the other substituted null value. // // 3: the newly added FD from filters should take some consideration as below: // -// <1> strict/lax FD: join key filter conditions can not produce new strict/lax FD yet (knowledge: 1&2). +// <1> strict/lax FD: join key filter conditions can not produce new strict/lax FD yet (knowledge: 1&2). // -// <2> constant FD from the join conditions is only used for checking other FD. We cannot keep itself. -// a b | c d -// -------+--------- -// 1 1 | 1 1 -// 1 2 | NULL NULL -// left join with (a,b) * (c,d) on (a=c and d=1), some rhs rows will be substituted with null values, and FD on rhs -// {d=1} are lost. +// <2> constant FD from the join conditions is only used for checking other FD. We cannot keep itself. +// a b | c d +// -------+--------- +// 1 1 | 1 1 +// 1 2 | NULL NULL +// left join with (a,b) * (c,d) on (a=c and d=1), some rhs rows will be substituted with null values, and FD on rhs +// {d=1} are lost. // -// a b | c d -// -------+--------- -// 1 1 | 1 1 -// 1 2 | NULL NULL -// left join with (a,b) * (c,d) on (a=c and b=1), it only gives the pass to the first matching, lhs other rows are still -// kept and appended with null values. So the FD on rhs {b=1} are not applicable to lhs rows. +// a b | c d +// -------+--------- +// 1 1 | 1 1 +// 1 2 | NULL NULL +// left join with (a,b) * (c,d) on (a=c and b=1), it only gives the pass to the first matching, lhs other rows are still +// kept and appended with null values. So the FD on rhs {b=1} are not applicable to lhs rows. // -// above all: constant FD are lost +// above all: constant FD are lost // -// <3.1> equivalence FD: when the left join conditions only contain equivalence FD (EFD for short below) across left and right -// cols and no other `LEFT` condition on the (left-side cols except the cols in EFD's from) to filter the left join results. We can maintain the strict -// FD from EFD's `from` side to EFD's `to` side over the left join result. -// a b | c d e -// ------+---------------- -// 1 1 | 1 NULL 1 -// 1 2 | NULL NULL NULL -// 2 1 | NULL NULL NULL -// Neg eg: left join with (a,b) * (c,d,e) on (a=c and b=1), other b=1 will filter the result, causing the left row (1, 2) -// miss matched with right row (1, null 1) by a=c, consequently leading the left row appended as (1,2,null,null,null), which -// will break the FD: {a} -> {c} for key a=1 with different c=1/null. -// a b | c d e -// ------+---------------- -// 1 1 | NULL NULL NULL -// 2 1 | NULL NULL NULL -// Pos eg: if the filter is on EFD's `from` cols, it's ok. Let's say: (a,b) * (c,d,e) on (a=c and a=2), a=2 only won't leading -// same key a with matched c and mismatched NULL, neg case result is changed as above, so strict FD {a} -> {c} can exist. -// a b | c d e -// ------+---------------- -// 1 1 | 1 NULL 1 -// 1 2 | NULL NULL NULL -// 2 1 | NULL NULL NULL -// Neg eg: left join with (a,b) * (c,d,e) on (a=c and b=c), two EFD here, where b=c can also refuse some rows joined by a=c, -// consequently applying it with NULL as (1 2 | NULL NULL NULL), leading the same key a has different value 1/NULL. But -// macroscopically, we can combine {a,b} together as the strict FD's from side, so new FD {a,b} -> {c} is secured. For case -// of (a=c and b=ce), the FD is {a, b} -> {c, e} +// <3.1> equivalence FD: when the left join conditions only contain equivalence FD (EFD for short below) across left and right +// cols and no other `LEFT` condition on the (left-side cols except the cols in EFD's from) to filter the left join results. We can maintain the strict +// FD from EFD's `from` side to EFD's `to` side over the left join result. +// a b | c d e +// ------+---------------- +// 1 1 | 1 NULL 1 +// 1 2 | NULL NULL NULL +// 2 1 | NULL NULL NULL +// Neg eg: left join with (a,b) * (c,d,e) on (a=c and b=1), other b=1 will filter the result, causing the left row (1, 2) +// miss matched with right row (1, null 1) by a=c, consequently leading the left row appended as (1,2,null,null,null), which +// will break the FD: {a} -> {c} for key a=1 with different c=1/null. +// a b | c d e +// ------+---------------- +// 1 1 | NULL NULL NULL +// 2 1 | NULL NULL NULL +// Pos eg: if the filter is on EFD's `from` cols, it's ok. Let's say: (a,b) * (c,d,e) on (a=c and a=2), a=2 only won't leading +// same key a with matched c and mismatched NULL, neg case result is changed as above, so strict FD {a} -> {c} can exist. +// a b | c d e +// ------+---------------- +// 1 1 | 1 NULL 1 +// 1 2 | NULL NULL NULL +// 2 1 | NULL NULL NULL +// Neg eg: left join with (a,b) * (c,d,e) on (a=c and b=c), two EFD here, where b=c can also refuse some rows joined by a=c, +// consequently applying it with NULL as (1 2 | NULL NULL NULL), leading the same key a has different value 1/NULL. But +// macroscopically, we can combine {a,b} together as the strict FD's from side, so new FD {a,b} -> {c} is secured. For case +// of (a=c and b=ce), the FD is {a, b} -> {c, e} // -// conclusion: without this kind of limited left conditions to judge the join match, we can say: FD {a} -> {c} exists. +// conclusion: without this kind of limited left conditions to judge the join match, we can say: FD {a} -> {c} exists. // -// <3.2> equivalence FD: when the determinant and dependencies from an equivalence FD of join condition are each covering a strict -// FD of the left / right side. After joining, we can extend the left side strict FD's dependencies to all cols. -// a b | c d e -// ------+---------------- -// 1 1 | 1 NULL 1 -// 2 2 | NULL NULL NULL -// 3 1 | NULL NULL NULL -// left join with (a,b) * (c,d,e) on (a=c and b=1). Supposing that left `a` are strict Key and right `c` are strict Key too. -// Key means the strict FD can determine all cols from that table. -// case 1: left join matched -// one left row match one / multi rows from right side, since `c` is strict determine all cols from right table, so -// {a} == {b} --> {all cols in right}, according to the transitive rule of strict FD, we get {a} --> {all cols in right} -// case 2: left join miss match -// miss matched rows from left side are unique originally, even appended with NULL value from right side, they are still -// strictly determine themselves and even the all rows after left join. -// conclusion combined: -// If there is an equivalence covering both strict Key from the right and left, we can create a new strict FD: {columns of the left side of the join in the equivalence} -> {all columns after join}. +// <3.2> equivalence FD: when the determinant and dependencies from an equivalence FD of join condition are each covering a strict +// FD of the left / right side. After joining, we can extend the left side strict FD's dependencies to all cols. +// a b | c d e +// ------+---------------- +// 1 1 | 1 NULL 1 +// 2 2 | NULL NULL NULL +// 3 1 | NULL NULL NULL +// left join with (a,b) * (c,d,e) on (a=c and b=1). Supposing that left `a` are strict Key and right `c` are strict Key too. +// Key means the strict FD can determine all cols from that table. +// case 1: left join matched +// one left row match one / multi rows from right side, since `c` is strict determine all cols from right table, so +// {a} == {b} --> {all cols in right}, according to the transitive rule of strict FD, we get {a} --> {all cols in right} +// case 2: left join miss match +// miss matched rows from left side are unique originally, even appended with NULL value from right side, they are still +// strictly determine themselves and even the all rows after left join. +// conclusion combined: +// If there is an equivalence covering both strict Key from the right and left, we can create a new strict FD: {columns of the left side of the join in the equivalence} -> {all columns after join}. // -// <3.3> equivalence FD: let's see equivalence FD as double-directed strict FD from join equal conditions, and we only keep the -// rhs ~~> lhs. -// a b | c d e -// ------+---------------- -// 1 1 | 1 NULL 1 -// 1 2 | NULL NULL NULL -// 2 1 | NULL NULL NULL -// left join with (a,b) * (c,d,e) on (a=c and b=1). From the join equivalence condition can derive a new FD {ac} == {ac}. -// while since there are some supplied null value in the c column, we don't guarantee {ac} == {ac} yet, so do {a} -> {c} -// because two same determinant key {1} can point to different dependency {1} & {NULL}. But in return, FD like {c} -> {a} -// are degraded to the corresponding lax one. +// <3.3> equivalence FD: let's see equivalence FD as double-directed strict FD from join equal conditions, and we only keep the +// rhs ~~> lhs. +// a b | c d e +// ------+---------------- +// 1 1 | 1 NULL 1 +// 1 2 | NULL NULL NULL +// 2 1 | NULL NULL NULL +// left join with (a,b) * (c,d,e) on (a=c and b=1). From the join equivalence condition can derive a new FD {ac} == {ac}. +// while since there are some supplied null value in the c column, we don't guarantee {ac} == {ac} yet, so do {a} -> {c} +// because two same determinant key {1} can point to different dependency {1} & {NULL}. But in return, FD like {c} -> {a} +// are degraded to the corresponding lax one. // // 4: the new formed FD {left primary key, right primary key} -> {all columns} are preserved in spite of the null-supplied rows. // 5: There's no join key and no filters from the outer side. The join case is a cartesian product. In this case, -// the strict equivalence classes still exist. -// - If the right side has no row, we will supply null-extended rows, then the value of any column is NULL, and the equivalence class exists. -// - If the right side has rows, no row is filtered out after the filters since no row of the outer side is filtered out. Hence, the equivalence class remains. // +// the strict equivalence classes still exist. +// - If the right side has no row, we will supply null-extended rows, then the value of any column is NULL, and the equivalence class exists. +// - If the right side has rows, no row is filtered out after the filters since no row of the outer side is filtered out. Hence, the equivalence class remains. func (s *FDSet) MakeOuterJoin(innerFDs, filterFDs *FDSet, outerCols, innerCols FastIntSet, opt *ArgOpts) { // copy down the left PK and right PK before the s has changed for later usage. leftPK, ok1 := s.FindPrimaryKey() @@ -924,9 +925,9 @@ func (s *FDSet) AddFrom(fds *FDSet) { // MaxOneRow will regard every column in the fdSet as a constant. Since constant is stronger that strict FD, it will // take over all existed strict/lax FD, only keeping the equivalence. Because equivalence is stronger than constant. // -// f: {a}--> {b,c}, {abc} == {abc} -// cols: {a,c} -// result: {} --> {a,c}, {a,c} == {a,c} +// f: {a}--> {b,c}, {abc} == {abc} +// cols: {a,c} +// result: {} --> {a,c}, {a,c} == {a,c} func (s *FDSet) MaxOneRow(cols FastIntSet) { cnt := 0 for i := 0; i < len(s.fdEdges); i++ { diff --git a/server/http_handler.go b/server/http_handler.go index ec0106316fd92..f4476721a24c6 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -2054,9 +2054,9 @@ func (h *testHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } // Supported operations: -// * resolvelock?safepoint={uint64}&physical={bool}: -// * safepoint: resolve all locks whose timestamp is less than the safepoint. -// * physical: whether it uses physical(green GC) mode to scan locks. Default is true. +// - resolvelock?safepoint={uint64}&physical={bool}: +// - safepoint: resolve all locks whose timestamp is less than the safepoint. +// - physical: whether it uses physical(green GC) mode to scan locks. Default is true. func (h *testHandler) handleGC(op string, w http.ResponseWriter, req *http.Request) { if !atomic.CompareAndSwapUint32(&h.gcIsRunning, 0, 1) { writeError(w, errors.New("GC is running")) diff --git a/session/bench_test.go b/session/bench_test.go index c1164ec32de40..f07756f3221b3 100644 --- a/session/bench_test.go +++ b/session/bench_test.go @@ -1824,7 +1824,8 @@ func BenchmarkCompileExecutePreparedStmt(b *testing.B) { // TestBenchDaily collects the daily benchmark test result and generates a json output file. // The format of the json output is described by the BenchOutput. // Used by this command in the Makefile -// make bench-daily TO=xxx.json +// +// make bench-daily TO=xxx.json func TestBenchDaily(t *testing.T) { benchdaily.Run( BenchmarkPreparedPointGet, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index ac9e232f85dd8..363a84acbb230 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -45,7 +45,9 @@ import ( ) // All system variables declared here are ordered by their scopes, which follow the order of scopes below: -// [NONE, SESSION, INSTANCE, GLOBAL, GLOBAL & SESSION] +// +// [NONE, SESSION, INSTANCE, GLOBAL, GLOBAL & SESSION] +// // If you are adding a new system variable, please put it in the corresponding area. var defaultSysVars = []*SysVar{ /* The system variables below have NONE scope */ diff --git a/statistics/builder.go b/statistics/builder.go index fee2f55e0860d..b03be6004d18e 100644 --- a/statistics/builder.go +++ b/statistics/builder.go @@ -373,7 +373,8 @@ func BuildHistAndTopN( } // pruneTopNItem tries to prune the least common values in the top-n list if it is not significantly more common than the values not in the list. -// We assume that the ones not in the top-n list's selectivity is 1/remained_ndv which is the internal implementation of EqualRowCount +// +// We assume that the ones not in the top-n list's selectivity is 1/remained_ndv which is the internal implementation of EqualRowCount func pruneTopNItem(topns []TopNMeta, ndv, nullCount, sampleRows, totalRows int64) []TopNMeta { // If the sampleRows holds all rows, or NDV of samples equals to actual NDV, we just return the TopN directly. if sampleRows == totalRows || totalRows <= 1 || int64(len(topns)) >= ndv { diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index c39999e922320..70ec636dae817 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -345,10 +345,12 @@ func (c *CMSketch) MergeCMSketch(rc *CMSketch) error { // MergeCMSketch4IncrementalAnalyze merges two CM Sketch for incremental analyze. Since there is no value // that appears partially in `c` and `rc` for incremental analyze, it uses `max` to merge them. // Here is a simple proof: when we query from the CM sketch, we use the `min` to get the answer: -// (1): For values that only appears in `c, using `max` to merge them affects the `min` query result less than using `sum`; -// (2): For values that only appears in `rc`, it is the same as condition (1); -// (3): For values that appears both in `c` and `rc`, if they do not appear partially in `c` and `rc`, for example, -// if `v` appears 5 times in the table, it can appears 5 times in `c` and 3 times in `rc`, then `max` also gives the correct answer. +// +// (1): For values that only appears in `c, using `max` to merge them affects the `min` query result less than using `sum`; +// (2): For values that only appears in `rc`, it is the same as condition (1); +// (3): For values that appears both in `c` and `rc`, if they do not appear partially in `c` and `rc`, for example, +// if `v` appears 5 times in the table, it can appears 5 times in `c` and 3 times in `rc`, then `max` also gives the correct answer. +// // So in fact, if we can know the number of appearances of each value in the first place, it is better to use `max` to construct the CM sketch rather than `sum`. func (c *CMSketch) MergeCMSketch4IncrementalAnalyze(rc *CMSketch, numTopN uint32) error { if c.depth != rc.depth || c.width != rc.width { @@ -521,7 +523,8 @@ func (c *TopN) String() string { } // Num returns the ndv of the TopN. -// TopN is declared directly in Histogram. So the Len is occupied by the Histogram. We use Num instead. +// +// TopN is declared directly in Histogram. So the Len is occupied by the Histogram. We use Num instead. func (c *TopN) Num() int { if c == nil { return 0 @@ -707,13 +710,14 @@ func NewTopN(n int) *TopN { // MergePartTopN2GlobalTopN is used to merge the partition-level topN to global-level topN. // The input parameters: -// 1. `topNs` are the partition-level topNs to be merged. -// 2. `n` is the size of the global-level topN. Notice: This value can be 0 and has no default value, we must explicitly specify this value. -// 3. `hists` are the partition-level histograms. Some values not in topN may be placed in the histogram. We need it here to make the value in the global-level TopN more accurate. +// 1. `topNs` are the partition-level topNs to be merged. +// 2. `n` is the size of the global-level topN. Notice: This value can be 0 and has no default value, we must explicitly specify this value. +// 3. `hists` are the partition-level histograms. Some values not in topN may be placed in the histogram. We need it here to make the value in the global-level TopN more accurate. +// // The output parameters: -// 1. `*TopN` is the final global-level topN. -// 2. `[]TopNMeta` is the left topN value from the partition-level TopNs, but is not placed to global-level TopN. We should put them back to histogram latter. -// 3. `[]*Histogram` are the partition-level histograms which just delete some values when we merge the global-level topN. +// 1. `*TopN` is the final global-level topN. +// 2. `[]TopNMeta` is the left topN value from the partition-level TopNs, but is not placed to global-level TopN. We should put them back to histogram latter. +// 3. `[]*Histogram` are the partition-level histograms which just delete some values when we merge the global-level topN. func MergePartTopN2GlobalTopN(sc *stmtctx.StatementContext, version int, topNs []*TopN, n uint32, hists []*Histogram, isIndex bool) (*TopN, []TopNMeta, []*Histogram, error) { if checkEmptyTopNs(topNs) { return nil, nil, hists, nil diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index ce77cce0c9238..2c2720723448b 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -2257,7 +2257,7 @@ func TestIndexUsageInformation(t *testing.T) { )) } -//Functional Test:test batch insert +// Functional Test:test batch insert func TestIndexUsageInformationMultiIndex(t *testing.T) { store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() diff --git a/statistics/handle/update.go b/statistics/handle/update.go index ab5fae00c3dd8..f0a100cfa5622 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -340,7 +340,7 @@ func (h *Handle) sweepIdxUsageList() indexUsageMap { return mapper } -//batchInsertSize is the every insert values size limit.Used in such as DumpIndexUsageToKV,DumpColStatsUsageToKV +// batchInsertSize is the every insert values size limit.Used in such as DumpIndexUsageToKV,DumpColStatsUsageToKV const batchInsertSize = 10 // DumpIndexUsageToKV will dump in-memory index usage information to KV. @@ -969,11 +969,11 @@ func TableAnalyzed(tbl *statistics.Table) bool { } // NeedAnalyzeTable checks if we need to analyze the table: -// 1. If the table has never been analyzed, we need to analyze it when it has -// not been modified for a while. -// 2. If the table had been analyzed before, we need to analyze it when -// "tbl.ModifyCount/tbl.Count > autoAnalyzeRatio" and the current time is -// between `start` and `end`. +// 1. If the table has never been analyzed, we need to analyze it when it has +// not been modified for a while. +// 2. If the table had been analyzed before, we need to analyze it when +// "tbl.ModifyCount/tbl.Count > autoAnalyzeRatio" and the current time is +// between `start` and `end`. func NeedAnalyzeTable(tbl *statistics.Table, limit time.Duration, autoAnalyzeRatio float64) (bool, string) { analyzed := TableAnalyzed(tbl) if !analyzed { diff --git a/statistics/histogram.go b/statistics/histogram.go index 0c4764b8a90e6..d1d26b71df071 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -460,19 +460,22 @@ func (hg *Histogram) greaterRowCount(value types.Datum) float64 { // locateBucket locates where a value falls in the range of the Histogram. // Return value: -// exceed: if the value is larger than the upper bound of the last Bucket of the Histogram -// bucketIdx: assuming exceed if false, which Bucket does this value fall in (note: the range before a Bucket is also +// +// exceed: if the value is larger than the upper bound of the last Bucket of the Histogram +// bucketIdx: assuming exceed if false, which Bucket does this value fall in (note: the range before a Bucket is also // considered belong to this Bucket) -// inBucket: assuming exceed if false, whether this value falls in this Bucket, instead of falls between +// inBucket: assuming exceed if false, whether this value falls in this Bucket, instead of falls between // this Bucket and the previous Bucket. -// matchLastValue: assuming inBucket is true, if this value is the last value in this Bucket, which has a counter (Bucket.Repeat) +// matchLastValue: assuming inBucket is true, if this value is the last value in this Bucket, which has a counter (Bucket.Repeat) +// // Examples: -// val0 |<-[bkt0]->| |<-[bkt1]->val1(last value)| val2 |<--val3--[bkt2]->| |<-[bkt3]->| val4 -// locateBucket(val0): false, 0, false, false -// locateBucket(val1): false, 1, true, true -// locateBucket(val2): false, 2, false, false -// locateBucket(val3): false, 2, true, false -// locateBucket(val4): true, 3, false, false +// +// val0 |<-[bkt0]->| |<-[bkt1]->val1(last value)| val2 |<--val3--[bkt2]->| |<-[bkt3]->| val4 +// locateBucket(val0): false, 0, false, false +// locateBucket(val1): false, 1, true, true +// locateBucket(val2): false, 2, false, false +// locateBucket(val3): false, 2, true, false +// locateBucket(val4): true, 3, false, false func (hg *Histogram) locateBucket(value types.Datum) (exceed bool, bucketIdx int, inBucket, matchLastValue bool) { // Empty histogram if hg == nil || hg.Bounds.NumRows() == 0 { @@ -880,18 +883,22 @@ func (hg *Histogram) outOfRange(val types.Datum) bool { // As it shows below. To calculate the out-of-range row count, we need to calculate the percentage of the shaded area. // Note that we assume histL-boundL == histR-histL == boundR-histR here. // -// /│ │\ -// / │ │ \ -// /x│ │◄─histogram─►│ \ -// / xx│ │ range │ \ -// / │xxx│ │ │ \ -// / │xxx│ │ │ \ -//────┴────┴───┴──┴─────────────┴───────────┴───── -// ▲ ▲ ▲ ▲ ▲ ▲ -// │ │ │ │ │ │ +// /│ │\ +// / │ │ \ +// /x│ │◄─histogram─►│ \ +// / xx│ │ range │ \ +// / │xxx│ │ │ \ +// / │xxx│ │ │ \ +// +// ────┴────┴───┴──┴─────────────┴───────────┴───── +// +// ▲ ▲ ▲ ▲ ▲ ▲ +// │ │ │ │ │ │ +// // boundL │ │histL histR boundR -// │ │ -// lDatum rDatum +// +// │ │ +// lDatum rDatum func (hg *Histogram) outOfRangeRowCount(lDatum, rDatum *types.Datum, increaseCount int64) float64 { if hg.Len() == 0 { return 0 @@ -1599,7 +1606,8 @@ type countByRangeFunc = func(sessionctx.Context, int64, []*ranger.Range) (float6 // newHistogramBySelectivity fulfills the content of new histogram by the given selectivity result. // TODO: Datum is not efficient, try to avoid using it here. -// Also, there're redundant calculation with Selectivity(). We need to reduce it too. +// +// Also, there're redundant calculation with Selectivity(). We need to reduce it too. func newHistogramBySelectivity(sctx sessionctx.Context, histID int64, oldHist, newHist *Histogram, ranges []*ranger.Range, cntByRangeFunc countByRangeFunc) error { cntPerVal := int64(oldHist.AvgCountPerNotNullValue(int64(oldHist.TotalRowCount()))) var totCnt int64 @@ -2019,10 +2027,12 @@ func mergeBucketNDV(sc *stmtctx.StatementContext, left *bucket4Merging, right *b // mergeParitionBuckets merges buckets[l...r) to one global bucket. // global bucket: -// upper = buckets[r-1].upper -// count = sum of buckets[l...r).count -// repeat = sum of buckets[i] (buckets[i].upper == global bucket.upper && i in [l...r)) -// ndv = merge bucket ndv from r-1 to l by mergeBucketNDV +// +// upper = buckets[r-1].upper +// count = sum of buckets[l...r).count +// repeat = sum of buckets[i] (buckets[i].upper == global bucket.upper && i in [l...r)) +// ndv = merge bucket ndv from r-1 to l by mergeBucketNDV +// // Notice: lower is not calculated here. func mergePartitionBuckets(sc *stmtctx.StatementContext, buckets []*bucket4Merging) (*bucket4Merging, error) { if len(buckets) == 0 { diff --git a/statistics/row_sampler.go b/statistics/row_sampler.go index d091f2be818fb..87ec1e1d7eb1f 100644 --- a/statistics/row_sampler.go +++ b/statistics/row_sampler.go @@ -50,11 +50,13 @@ type baseCollector struct { // ReservoirRowSampleCollector collects the samples from the source and organize the samples by row. // It will maintain the following things: -// Row samples. -// FM sketches(To calculate the NDV). -// Null counts. -// The data sizes. -// The number of rows. +// +// Row samples. +// FM sketches(To calculate the NDV). +// Null counts. +// The data sizes. +// The number of rows. +// // It uses weighted reservoir sampling(A-Res) to do the sampling. type ReservoirRowSampleCollector struct { *baseCollector @@ -397,11 +399,13 @@ func RowSamplesToProto(samples WeightedRowSampleHeap) []*tipb.RowSample { // BernoulliRowSampleCollector collects the samples from the source and organize the sample by row. // It will maintain the following things: -// Row samples. -// FM sketches(To calculate the NDV). -// Null counts. -// The data sizes. -// The number of rows. +// +// Row samples. +// FM sketches(To calculate the NDV). +// Null counts. +// The data sizes. +// The number of rows. +// // It uses the bernoulli sampling to collect the data. type BernoulliRowSampleCollector struct { *baseCollector diff --git a/statistics/selectivity.go b/statistics/selectivity.go index 74e513ab2f696..df5d7180e7ba9 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -569,10 +569,12 @@ func CETraceExpr(sctx sessionctx.Context, tableID int64, tp string, expr express // It might be too tricky because it makes use of TiDB allowing using internal function name in SQL. // For example, you can write `eq`(a, 1), which is the same as a = 1. // We should have implemented this by first implementing a method to turn an expression to an AST -// then call astNode.Restore(), like the Constant case here. But for convenience, we use this trick for now. +// +// then call astNode.Restore(), like the Constant case here. But for convenience, we use this trick for now. // // It may be more appropriate to put this in expression package. But currently we only use it for CE trace, -// and it may not be general enough to handle all possible expressions. So we put it here for now. +// +// and it may not be general enough to handle all possible expressions. So we put it here for now. func ExprToString(e expression.Expression) (string, error) { switch expr := e.(type) { case *expression.ScalarFunction: diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 32f38a72f7576..1c9194076171a 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -282,11 +282,12 @@ func balanceBatchCopTaskWithContinuity(storeTaskMap map[uint64]*batchCopTask, ca } // balanceBatchCopTask balance the regions between available stores, the basic rule is -// 1. the first region of each original batch cop task belongs to its original store because some -// meta data(like the rpc context) in batchCopTask is related to it -// 2. for the remaining regions: -// if there is only 1 available store, then put the region to the related store -// otherwise, these region will be balance between TiFlash stores. +// 1. the first region of each original batch cop task belongs to its original store because some +// meta data(like the rpc context) in batchCopTask is related to it +// 2. for the remaining regions: +// if there is only 1 available store, then put the region to the related store +// otherwise, these region will be balance between TiFlash stores. +// // Currently, there are two balance strategies. // The first balance strategy: use a greedy algorithm to put it into the store with highest weight. This strategy only consider the region count between TiFlash stores. // diff --git a/store/driver/txn/batch_getter_test.go b/store/driver/txn/batch_getter_test.go index 2da439a49ed5c..f291390e2f92c 100644 --- a/store/driver/txn/batch_getter_test.go +++ b/store/driver/txn/batch_getter_test.go @@ -4,7 +4,7 @@ // you may not use this file except in compliance with the License. // You may obtain a copy of the License at // -// http://www.apache.org/licenses/LICENSE-2.0 +// http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, diff --git a/store/driver/txn/driver_test.go b/store/driver/txn/driver_test.go index 2a820e07bc03a..bb3f477eeda27 100644 --- a/store/driver/txn/driver_test.go +++ b/store/driver/txn/driver_test.go @@ -4,7 +4,7 @@ // you may not use this file except in compliance with the License. // You may obtain a copy of the License at // -// http://www.apache.org/licenses/LICENSE-2.0 +// http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, diff --git a/store/mockstore/unistore/cluster.go b/store/mockstore/unistore/cluster.go index 2d2843f2fe041..38e41db832e79 100644 --- a/store/mockstore/unistore/cluster.go +++ b/store/mockstore/unistore/cluster.go @@ -34,14 +34,14 @@ var _ testutils.Cluster = new(Cluster) // Cluster simulates a TiKV cluster. It focuses on management and the change of // meta data. A Cluster mainly includes following 3 kinds of meta data: -// 1) Region: A Region is a fragment of TiKV's data whose range is [start, end). -// The data of a Region is duplicated to multiple Peers and distributed in -// multiple Stores. -// 2) Peer: A Peer is a replica of a Region's data. All peers of a Region form -// a group, each group elects a Leader to provide services. -// 3) Store: A Store is a storage/service node. Try to think it as a TiKV server -// process. Only the store with request's Region's leader Peer could respond -// to client's request. +// 1. Region: A Region is a fragment of TiKV's data whose range is [start, end). +// The data of a Region is duplicated to multiple Peers and distributed in +// multiple Stores. +// 2. Peer: A Peer is a replica of a Region's data. All peers of a Region form +// a group, each group elects a Leader to provide services. +// 3. Store: A Store is a storage/service node. Try to think it as a TiKV server +// process. Only the store with request's Region's leader Peer could respond +// to client's request. type Cluster struct { *us.MockRegionManager diff --git a/store/mockstore/unistore/cophandler/closure_exec.go b/store/mockstore/unistore/cophandler/closure_exec.go index e7bac738bc67f..515fe6f6e1a32 100644 --- a/store/mockstore/unistore/cophandler/closure_exec.go +++ b/store/mockstore/unistore/cophandler/closure_exec.go @@ -162,7 +162,8 @@ func buildClosureExecutorFromExecutorList(dagCtx *dagContext, executors []*tipb. // buildClosureExecutor build a closureExecutor for the DAGRequest. // Currently the composition of executors are: -// tableScan|indexScan [selection] [topN | limit | agg] +// +// tableScan|indexScan [selection] [topN | limit | agg] func buildClosureExecutor(dagCtx *dagContext, dagReq *tipb.DAGRequest) (*closureExecutor, error) { scanExec, err := getScanExec(dagReq) if err != nil { diff --git a/store/store.go b/store/store.go index eca23fc5a5c83..d4b51f025d824 100644 --- a/store/store.go +++ b/store/store.go @@ -49,8 +49,9 @@ func Register(name string, driver kv.Driver) error { // The path must be a URL format 'engine://path?params' like the one for // session.Open() but with the dbname cut off. // Examples: -// goleveldb://relative/path -// boltdb:///absolute/path +// +// goleveldb://relative/path +// boltdb:///absolute/path // // The engine should be registered before creating storage. func New(path string) (kv.Storage, error) { diff --git a/table/column.go b/table/column.go index 91246fac9b8f0..821b7d8d4b368 100644 --- a/table/column.go +++ b/table/column.go @@ -197,9 +197,10 @@ func convertToIncorrectStringErr(err error, colName string) error { // handleZeroDatetime handles Timestamp/Datetime/Date zero date and invalid dates. // Currently only called from CastValue. // returns: -// value (possibly adjusted) -// boolean; true if break error/warning handling in CastValue and return what was returned from this -// error +// +// value (possibly adjusted) +// boolean; true if break error/warning handling in CastValue and return what was returned from this +// error func handleZeroDatetime(ctx sessionctx.Context, col *model.ColumnInfo, casted types.Datum, str string, tmIsInvalid bool) (types.Datum, bool, error) { sc := ctx.GetSessionVars().StmtCtx tm := casted.GetMysqlTime() diff --git a/table/tables/mutation_checker.go b/table/tables/mutation_checker.go index 3b37d8a99114d..4dc001848126a 100644 --- a/table/tables/mutation_checker.go +++ b/table/tables/mutation_checker.go @@ -171,7 +171,8 @@ func checkHandleConsistency(rowInsertion mutation, indexMutations []mutation, in // To check (1), we need // (a) {added indices} is a subset of {needed indices} => each index mutation is consistent with the input/row key/value // (b) {needed indices} is a subset of {added indices}. The check process would be exactly the same with how we generate -// the mutations, thus ignored. +// +// the mutations, thus ignored. func checkIndexKeys( sessVars *variable.SessionVars, t *TableCommon, rowToInsert, rowToRemove []types.Datum, indexMutations []mutation, indexIDToInfo map[int64]*model.IndexInfo, diff --git a/table/tables/partition.go b/table/tables/partition.go index 9f84333213a82..5c961094740f4 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -283,16 +283,17 @@ type ListPartitionGroup struct { // In partition p0, both value group0 (1,5) and group1 (1,6) are contain the column a which value is 1. // In partition p1, value group0 (1,7) contains the column a which value is 1. // So, the ListPartitionLocation of column a which value is 1 is: -// []ListPartitionGroup{ -// { -// PartIdx: 0, // `0` is the partition p0 index in all partitions. -// GroupIdxs: []int{0, 1} // `0,1` is the index of the value group0, group1. -// }, -// { -// PartIdx: 1, // `1` is the partition p1 index in all partitions. -// GroupIdxs: []int{0} // `0` is the index of the value group0. -// }, -// } +// +// []ListPartitionGroup{ +// { +// PartIdx: 0, // `0` is the partition p0 index in all partitions. +// GroupIdxs: []int{0, 1} // `0,1` is the index of the value group0, group1. +// }, +// { +// PartIdx: 1, // `1` is the partition p1 index in all partitions. +// GroupIdxs: []int{0} // `0` is the index of the value group0. +// }, +// } type ListPartitionLocation []ListPartitionGroup // IsEmpty returns true if the ListPartitionLocation is empty. diff --git a/tablecodec/tablecodec.go b/tablecodec/tablecodec.go index 90adccbc5fa64..f2a1ea71e63ca 100644 --- a/tablecodec/tablecodec.go +++ b/tablecodec/tablecodec.go @@ -896,8 +896,9 @@ func getIndexVersion(value []byte) int { } // DecodeIndexKV uses to decode index key values. -// `colsLen` is expected to be index columns count. -// `columns` is expected to be index columns + handle columns(if hdStatus is not HandleNotNeeded). +// +// `colsLen` is expected to be index columns count. +// `columns` is expected to be index columns + handle columns(if hdStatus is not HandleNotNeeded). func DecodeIndexKV(key, value []byte, colsLen int, hdStatus HandleStatus, columns []rowcodec.ColInfo) ([][]byte, error) { if len(value) <= MaxOldEncodeValueLen { return decodeIndexKvOldCollation(key, value, colsLen, hdStatus) @@ -1126,53 +1127,54 @@ func GenIndexKey(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, idxInfo // GenIndexValuePortal is the portal for generating index value. // Value layout: -// +-- IndexValueVersion0 (with restore data, or common handle, or index is global) -// | -// | Layout: TailLen | Options | Padding | [IntHandle] | [UntouchedFlag] -// | Length: 1 | len(options) | len(padding) | 8 | 1 -// | -// | TailLen: len(padding) + len(IntHandle) + len(UntouchedFlag) -// | Options: Encode some value for new features, such as common handle, new collations or global index. -// | See below for more information. -// | Padding: Ensure length of value always >= 10. (or >= 11 if UntouchedFlag exists.) -// | IntHandle: Only exists when table use int handles and index is unique. -// | UntouchedFlag: Only exists when index is untouched. -// | -// +-- Old Encoding (without restore data, integer handle, local) -// | -// | Layout: [Handle] | [UntouchedFlag] -// | Length: 8 | 1 -// | -// | Handle: Only exists in unique index. -// | UntouchedFlag: Only exists when index is untouched. -// | -// | If neither Handle nor UntouchedFlag exists, value will be one single byte '0' (i.e. []byte{'0'}). -// | Length of value <= 9, use to distinguish from the new encoding. -// | -// +-- IndexValueForClusteredIndexVersion1 -// | -// | Layout: TailLen | VersionFlag | Version | Options | [UntouchedFlag] -// | Length: 1 | 1 | 1 | len(options) | 1 -// | -// | TailLen: len(UntouchedFlag) -// | Options: Encode some value for new features, such as common handle, new collations or global index. -// | See below for more information. -// | UntouchedFlag: Only exists when index is untouched. -// | -// | Layout of Options: -// | -// | Segment: Common Handle | Global Index | New Collation -// | Layout: CHandle flag | CHandle Len | CHandle | PidFlag | PartitionID | restoreData -// | Length: 1 | 2 | len(CHandle) | 1 | 8 | len(restoreData) -// | -// | Common Handle Segment: Exists when unique index used common handles. -// | Global Index Segment: Exists when index is global. -// | New Collation Segment: Exists when new collation is used and index or handle contains non-binary string. -// | In v4.0, restored data contains all the index values. For example, (a int, b char(10)) and index (a, b). -// | The restored data contains both the values of a and b. -// | In v5.0, restored data contains only non-binary data(except for char and _bin). In the above example, the restored data contains only the value of b. -// | Besides, if the collation of b is _bin, then restored data is an integer indicate the spaces are truncated. Then we use sortKey -// | and the restored data together to restore original data. +// +// +-- IndexValueVersion0 (with restore data, or common handle, or index is global) +// | +// | Layout: TailLen | Options | Padding | [IntHandle] | [UntouchedFlag] +// | Length: 1 | len(options) | len(padding) | 8 | 1 +// | +// | TailLen: len(padding) + len(IntHandle) + len(UntouchedFlag) +// | Options: Encode some value for new features, such as common handle, new collations or global index. +// | See below for more information. +// | Padding: Ensure length of value always >= 10. (or >= 11 if UntouchedFlag exists.) +// | IntHandle: Only exists when table use int handles and index is unique. +// | UntouchedFlag: Only exists when index is untouched. +// | +// +-- Old Encoding (without restore data, integer handle, local) +// | +// | Layout: [Handle] | [UntouchedFlag] +// | Length: 8 | 1 +// | +// | Handle: Only exists in unique index. +// | UntouchedFlag: Only exists when index is untouched. +// | +// | If neither Handle nor UntouchedFlag exists, value will be one single byte '0' (i.e. []byte{'0'}). +// | Length of value <= 9, use to distinguish from the new encoding. +// | +// +-- IndexValueForClusteredIndexVersion1 +// | +// | Layout: TailLen | VersionFlag | Version | Options | [UntouchedFlag] +// | Length: 1 | 1 | 1 | len(options) | 1 +// | +// | TailLen: len(UntouchedFlag) +// | Options: Encode some value for new features, such as common handle, new collations or global index. +// | See below for more information. +// | UntouchedFlag: Only exists when index is untouched. +// | +// | Layout of Options: +// | +// | Segment: Common Handle | Global Index | New Collation +// | Layout: CHandle flag | CHandle Len | CHandle | PidFlag | PartitionID | restoreData +// | Length: 1 | 2 | len(CHandle) | 1 | 8 | len(restoreData) +// | +// | Common Handle Segment: Exists when unique index used common handles. +// | Global Index Segment: Exists when index is global. +// | New Collation Segment: Exists when new collation is used and index or handle contains non-binary string. +// | In v4.0, restored data contains all the index values. For example, (a int, b char(10)) and index (a, b). +// | The restored data contains both the values of a and b. +// | In v5.0, restored data contains only non-binary data(except for char and _bin). In the above example, the restored data contains only the value of b. +// | Besides, if the collation of b is _bin, then restored data is an integer indicate the spaces are truncated. Then we use sortKey +// | and the restored data together to restore original data. func GenIndexValuePortal(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, needRestoredData bool, distinct bool, untouched bool, indexedValues []types.Datum, h kv.Handle, partitionID int64, restoredData []types.Datum) ([]byte, error) { if tblInfo.IsCommonHandle && tblInfo.CommonHandleVersion == 1 { return GenIndexValueForClusteredIndexVersion1(sc, tblInfo, idxInfo, needRestoredData, distinct, untouched, indexedValues, h, partitionID, restoredData) diff --git a/tests/realtikvtest/pessimistictest/pessimistic_test.go b/tests/realtikvtest/pessimistictest/pessimistic_test.go index 1c0c3122eea99..49cd21a07f21f 100644 --- a/tests/realtikvtest/pessimistictest/pessimistic_test.go +++ b/tests/realtikvtest/pessimistictest/pessimistic_test.go @@ -58,6 +58,7 @@ func createAsyncCommitTestKit(t *testing.T, store kv.Storage) *testkit.TestKit { } // TODO: figure out a stable way to run Test1PCWithSchemaChange +// //nolint:unused func create1PCTestKit(t *testing.T, store kv.Storage) *testkit.TestKit { tk := testkit.NewTestKit(t, store) diff --git a/tidb-binlog/proto/go-binlog/secondary_binlog.pb.go b/tidb-binlog/proto/go-binlog/secondary_binlog.pb.go index 9cabc970b6f09..324ba6bbb96ec 100644 --- a/tidb-binlog/proto/go-binlog/secondary_binlog.pb.go +++ b/tidb-binlog/proto/go-binlog/secondary_binlog.pb.go @@ -2,21 +2,23 @@ // source: secondary_binlog.proto /* - Package slave_binlog is a generated protocol buffer package. - - It is generated from these files: - secondary_binlog.proto - - It has these top-level messages: - Column - ColumnInfo - Row - Table - Key - TableMutation - DMLData - DDLData - Binlog +Package slave_binlog is a generated protocol buffer package. + +It is generated from these files: + + secondary_binlog.proto + +It has these top-level messages: + + Column + ColumnInfo + Row + Table + Key + TableMutation + DMLData + DDLData + Binlog */ package slave_binlog @@ -350,7 +352,7 @@ func (m *Row) GetColumns() []*Column { return nil } -// Table contains mutations in a table. +// Table contains mutations in a table. type Table struct { SchemaName *string `protobuf:"bytes,1,opt,name=schema_name,json=schemaName" json:"schema_name,omitempty"` TableName *string `protobuf:"bytes,2,opt,name=table_name,json=tableName" json:"table_name,omitempty"` diff --git a/types/core_time.go b/types/core_time.go index 64820f98aaf79..f4ec6c0c9da13 100644 --- a/types/core_time.go +++ b/types/core_time.go @@ -277,8 +277,10 @@ func getFixDays(year, month, day int, ot gotime.Time) int { // compareTime compare two Time. // return: -// 0: if a == b -// 1: if a > b +// +// 0: if a == b +// 1: if a > b +// // -1: if a < b func compareTime(a, b CoreTime) int { ta := datetimeToUint64(a) diff --git a/types/datum.go b/types/datum.go index dd3a8f14e9a85..009a58f926b35 100644 --- a/types/datum.go +++ b/types/datum.go @@ -2308,14 +2308,17 @@ func getDatumBound(retType *FieldType, rType RoundingType) Datum { // ChangeReverseResultByUpperLowerBound is for expression's reverse evaluation. // Here is an example for what's effort for the function: CastRealAsInt(t.a), -// if the type of column `t.a` is mysql.TypeDouble, and there is a row that t.a == MaxFloat64 -// then the cast function will arrive a result MaxInt64. But when we do the reverse evaluation, -// if the result is MaxInt64, and the rounding type is ceiling. Then we should get the MaxFloat64 -// instead of float64(MaxInt64). +// +// if the type of column `t.a` is mysql.TypeDouble, and there is a row that t.a == MaxFloat64 +// then the cast function will arrive a result MaxInt64. But when we do the reverse evaluation, +// if the result is MaxInt64, and the rounding type is ceiling. Then we should get the MaxFloat64 +// instead of float64(MaxInt64). +// // Another example: cast(1.1 as signed) = 1, -// when we get the answer 1, we can only reversely evaluate 1.0 as the column value. So in this -// case, we should judge whether the rounding type are ceiling. If it is, then we should plus one for -// 1.0 and get the reverse result 2.0. +// +// when we get the answer 1, we can only reversely evaluate 1.0 as the column value. So in this +// case, we should judge whether the rounding type are ceiling. If it is, then we should plus one for +// 1.0 and get the reverse result 2.0. func ChangeReverseResultByUpperLowerBound( sc *stmtctx.StatementContext, retType *FieldType, diff --git a/types/etc.go b/types/etc.go index 4aa5576cfec25..56309d0c41ac4 100644 --- a/types/etc.go +++ b/types/etc.go @@ -162,6 +162,7 @@ func KindStr(kind byte) (r string) { // It is used for converting Text to Blob, // or converting Char to Binary. // Args: +// // tp: type enum // cs: charset var TypeToStr = ast.TypeToStr diff --git a/types/json/binary_functions.go b/types/json/binary_functions.go index 88770ae78c89f..e3ba88c7c8db1 100644 --- a/types/json/binary_functions.go +++ b/types/json/binary_functions.go @@ -207,8 +207,9 @@ func quoteString(s string) string { } // Extract receives several path expressions as arguments, matches them in bj, and returns: -// ret: target JSON matched any path expressions. maybe autowrapped as an array. -// found: true if any path expressions matched. +// +// ret: target JSON matched any path expressions. maybe autowrapped as an array. +// found: true if any path expressions matched. func (bj BinaryJSON) Extract(pathExprList []PathExpression) (ret BinaryJSON, found bool) { buf := make([]BinaryJSON, 0, 1) for _, pathExpr := range pathExprList { @@ -1105,7 +1106,9 @@ func (bj BinaryJSON) Search(containType string, search string, escape byte, path type extractCallbackFn func(fullpath PathExpression, bj BinaryJSON) (stop bool, err error) // extractToCallback callback alternative of extractTo -// would be more effective when walk through the whole JSON is unnecessary +// +// would be more effective when walk through the whole JSON is unnecessary +// // NOTICE: path [0] & [*] for JSON object other than array is INVALID, which is different from extractTo. func (bj BinaryJSON) extractToCallback(pathExpr PathExpression, callbackFn extractCallbackFn, fullpath PathExpression) (stop bool, err error) { if len(pathExpr.legs) == 0 { diff --git a/types/json/binary_functions_test.go b/types/json/binary_functions_test.go index 664564365e6ac..f8319c2d4fc41 100644 --- a/types/json/binary_functions_test.go +++ b/types/json/binary_functions_test.go @@ -4,7 +4,7 @@ // you may not use this file except in compliance with the License. // You may obtain a copy of the License at // -// http://www.apache.org/licenses/LICENSE-2.0 +// http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, diff --git a/types/mydecimal.go b/types/mydecimal.go index d1f158fa2a1bb..00a10aeccd46f 100644 --- a/types/mydecimal.go +++ b/types/mydecimal.go @@ -193,10 +193,10 @@ func fixWordCntError(wordsInt, wordsFrac int) (newWordsInt int, newWordsFrac int } /* - countLeadingZeroes returns the number of leading zeroes that can be removed from fraction. +countLeadingZeroes returns the number of leading zeroes that can be removed from fraction. - @param i start index - @param word value to compare against list of powers of 10 +@param i start index +@param word value to compare against list of powers of 10 */ func countLeadingZeroes(i int, word int32) int { leading := 0 @@ -208,10 +208,10 @@ func countLeadingZeroes(i int, word int32) int { } /* - countTrailingZeros returns the number of trailing zeroes that can be removed from fraction. +countTrailingZeros returns the number of trailing zeroes that can be removed from fraction. - @param i start index - @param word value to compare against list of powers of 10 +@param i start index +@param word value to compare against list of powers of 10 */ func countTrailingZeroes(i int, word int32) int { trailing := 0 @@ -310,11 +310,10 @@ func (d *MyDecimal) removeTrailingZeros() (lastWordIdx int, digitsFrac int) { // ToString converts decimal to its printable string representation without rounding. // -// RETURN VALUE -// -// str - result string -// errCode - eDecOK/eDecTruncate/eDecOverflow +// RETURN VALUE // +// str - result string +// errCode - eDecOK/eDecTruncate/eDecOverflow func (d *MyDecimal) ToString() (str []byte) { str = make([]byte, d.stringSize()) digitsFrac := int(d.digitsFrac) @@ -538,10 +537,10 @@ func (d *MyDecimal) FromString(str []byte) error { // shift < 0 means right shift. In fact it is multiplying on 10^shift. // // RETURN -// eDecOK OK -// eDecOverflow operation lead to overflow, number is untoched -// eDecTruncated number was rounded to fit into buffer // +// eDecOK OK +// eDecOverflow operation lead to overflow, number is untoched +// eDecTruncated number was rounded to fit into buffer func (d *MyDecimal) Shift(shift int) error { var err error if shift == 0 { @@ -701,10 +700,10 @@ func (d *MyDecimal) Shift(shift int) error { } /* - digitBounds returns bounds of decimal digits in the number. +digitBounds returns bounds of decimal digits in the number. - start - index (from 0 ) of first decimal digits. - end - index of position just after last decimal digit. + start - index (from 0 ) of first decimal digits. + end - index of position just after last decimal digit. */ func (d *MyDecimal) digitBounds() (start, end int) { var i int @@ -750,14 +749,15 @@ func (d *MyDecimal) digitBounds() (start, end int) { } /* - doMiniLeftShift does left shift for alignment of data in buffer. +doMiniLeftShift does left shift for alignment of data in buffer. + + shift number of decimal digits on which it should be shifted + beg/end bounds of decimal digits (see digitsBounds()) - shift number of decimal digits on which it should be shifted - beg/end bounds of decimal digits (see digitsBounds()) +NOTE - NOTE - Result fitting in the buffer should be garanted. - 'shift' have to be from 1 to digitsPerWord-1 (inclusive) + Result fitting in the buffer should be garanted. + 'shift' have to be from 1 to digitsPerWord-1 (inclusive) */ func (d *MyDecimal) doMiniLeftShift(shift, beg, end int) { bufFrom := beg / digitsPerWord @@ -774,14 +774,15 @@ func (d *MyDecimal) doMiniLeftShift(shift, beg, end int) { } /* - doMiniRightShift does right shift for alignment of data in buffer. +doMiniRightShift does right shift for alignment of data in buffer. + + shift number of decimal digits on which it should be shifted + beg/end bounds of decimal digits (see digitsBounds()) - shift number of decimal digits on which it should be shifted - beg/end bounds of decimal digits (see digitsBounds()) +NOTE - NOTE - Result fitting in the buffer should be garanted. - 'shift' have to be from 1 to digitsPerWord-1 (inclusive) + Result fitting in the buffer should be garanted. + 'shift' have to be from 1 to digitsPerWord-1 (inclusive) */ func (d *MyDecimal) doMiniRightShift(shift, beg, end int) { bufFrom := (end - 1) / digitsPerWord @@ -799,18 +800,20 @@ func (d *MyDecimal) doMiniRightShift(shift, beg, end int) { // Round rounds the decimal to "frac" digits. // -// to - result buffer. d == to is allowed -// frac - to what position after fraction point to round. can be negative! -// roundMode - round to nearest even or truncate -// ModeHalfUp rounds normally. -// ModeTruncate just truncates the decimal. +// to - result buffer. d == to is allowed +// frac - to what position after fraction point to round. can be negative! +// roundMode - round to nearest even or truncate +// ModeHalfUp rounds normally. +// ModeTruncate just truncates the decimal. // // NOTES -// frac can be negative ! -// one TRUNCATED error (line XXX below) isn't treated very logical :( +// +// frac can be negative ! +// one TRUNCATED error (line XXX below) isn't treated very logical :( // // RETURN VALUE -// nil/ErrTruncated/ErrOverflow +// +// nil/ErrTruncated/ErrOverflow func (d *MyDecimal) Round(to *MyDecimal, frac int, roundMode RoundMode) (err error) { // wordsFracTo is the number of fraction words in buffer. wordsFracTo := (frac + 1) / digitsPerWord @@ -1135,78 +1138,78 @@ ToBin converts decimal to its binary fixed-length representation two representations of the same length can be compared with memcmp with the correct -1/0/+1 result - PARAMS - precision/frac - if precision is 0, internal value of the decimal will be used, - then the encoded value is not memory comparable. + PARAMS + precision/frac - if precision is 0, internal value of the decimal will be used, + then the encoded value is not memory comparable. - NOTE - the buffer is assumed to be of the size DecimalBinSize(precision, frac) + NOTE + the buffer is assumed to be of the size DecimalBinSize(precision, frac) - RETURN VALUE - bin - binary value - errCode - eDecOK/eDecTruncate/eDecOverflow + RETURN VALUE + bin - binary value + errCode - eDecOK/eDecTruncate/eDecOverflow - DESCRIPTION - for storage decimal numbers are converted to the "binary" format. + DESCRIPTION + for storage decimal numbers are converted to the "binary" format. - This format has the following properties: - 1. length of the binary representation depends on the {precision, frac} - as provided by the caller and NOT on the digitsInt/digitsFrac of the decimal to - convert. - 2. binary representations of the same {precision, frac} can be compared - with memcmp - with the same result as DecimalCompare() of the original - decimals (not taking into account possible precision loss during - conversion). + This format has the following properties: + 1. length of the binary representation depends on the {precision, frac} + as provided by the caller and NOT on the digitsInt/digitsFrac of the decimal to + convert. + 2. binary representations of the same {precision, frac} can be compared + with memcmp - with the same result as DecimalCompare() of the original + decimals (not taking into account possible precision loss during + conversion). - This binary format is as follows: - 1. First the number is converted to have a requested precision and frac. - 2. Every full digitsPerWord digits of digitsInt part are stored in 4 bytes - as is - 3. The first digitsInt % digitesPerWord digits are stored in the reduced - number of bytes (enough bytes to store this number of digits - - see dig2bytes) - 4. same for frac - full word are stored as is, - the last frac % digitsPerWord digits - in the reduced number of bytes. - 5. If the number is negative - every byte is inversed. - 5. The very first bit of the resulting byte array is inverted (because - memcmp compares unsigned bytes, see property 2 above) + This binary format is as follows: + 1. First the number is converted to have a requested precision and frac. + 2. Every full digitsPerWord digits of digitsInt part are stored in 4 bytes + as is + 3. The first digitsInt % digitesPerWord digits are stored in the reduced + number of bytes (enough bytes to store this number of digits - + see dig2bytes) + 4. same for frac - full word are stored as is, + the last frac % digitsPerWord digits - in the reduced number of bytes. + 5. If the number is negative - every byte is inversed. + 5. The very first bit of the resulting byte array is inverted (because + memcmp compares unsigned bytes, see property 2 above) - Example: + Example: - 1234567890.1234 + 1234567890.1234 - internally is represented as 3 words + internally is represented as 3 words - 1 234567890 123400000 + 1 234567890 123400000 - (assuming we want a binary representation with precision=14, frac=4) - in hex it's + (assuming we want a binary representation with precision=14, frac=4) + in hex it's - 00-00-00-01 0D-FB-38-D2 07-5A-EF-40 + 00-00-00-01 0D-FB-38-D2 07-5A-EF-40 - now, middle word is full - it stores 9 decimal digits. It goes - into binary representation as is: + now, middle word is full - it stores 9 decimal digits. It goes + into binary representation as is: - ........... 0D-FB-38-D2 ............ + ........... 0D-FB-38-D2 ............ - First word has only one decimal digit. We can store one digit in - one byte, no need to waste four: + First word has only one decimal digit. We can store one digit in + one byte, no need to waste four: - 01 0D-FB-38-D2 ............ + 01 0D-FB-38-D2 ............ - now, last word. It's 123400000. We can store 1234 in two bytes: + now, last word. It's 123400000. We can store 1234 in two bytes: - 01 0D-FB-38-D2 04-D2 + 01 0D-FB-38-D2 04-D2 - So, we've packed 12 bytes number in 7 bytes. - And now we invert the highest bit to get the final result: + So, we've packed 12 bytes number in 7 bytes. + And now we invert the highest bit to get the final result: - 81 0D FB 38 D2 04 D2 + 81 0D FB 38 D2 04 D2 - And for -1234567890.1234 it would be + And for -1234567890.1234 it would be - 7E F2 04 C7 2D FB 2D + 7E F2 04 C7 2D FB 2D */ func (d *MyDecimal) ToBin(precision, frac int) ([]byte, error) { return d.WriteBin(precision, frac, []byte{}) @@ -1906,21 +1909,21 @@ func maxDecimal(precision, frac int, to *MyDecimal) { /* DecimalMul multiplies two decimals. - from1, from2 - factors - to - product + from1, from2 - factors + to - product - RETURN VALUE - E_DEC_OK/E_DEC_TRUNCATED/E_DEC_OVERFLOW; + RETURN VALUE + E_DEC_OK/E_DEC_TRUNCATED/E_DEC_OVERFLOW; - NOTES - in this implementation, with wordSize=4 we have digitsPerWord=9, - and 63-digit number will take only 7 words (basically a 7-digit - "base 999999999" number). Thus there's no need in fast multiplication - algorithms, 7-digit numbers can be multiplied with a naive O(n*n) - method. + NOTES + in this implementation, with wordSize=4 we have digitsPerWord=9, + and 63-digit number will take only 7 words (basically a 7-digit + "base 999999999" number). Thus there's no need in fast multiplication + algorithms, 7-digit numbers can be multiplied with a naive O(n*n) + method. - XXX if this library is to be used with huge numbers of thousands of - digits, fast multiplication must be implemented. + XXX if this library is to be used with huge numbers of thousands of + digits, fast multiplication must be implemented. */ func DecimalMul(from1, from2, to *MyDecimal) error { from1, from2, to = validateArgs(from1, from2, to) @@ -2060,26 +2063,26 @@ func DecimalDiv(from1, from2, to *MyDecimal, fracIncr int) error { /* DecimalMod does modulus of two decimals. - from1 - dividend - from2 - divisor - to - modulus + from1 - dividend + from2 - divisor + to - modulus - RETURN VALUE - E_DEC_OK/E_DEC_TRUNCATED/E_DEC_OVERFLOW/E_DEC_DIV_ZERO; + RETURN VALUE + E_DEC_OK/E_DEC_TRUNCATED/E_DEC_OVERFLOW/E_DEC_DIV_ZERO; - NOTES - see do_div_mod() + NOTES + see do_div_mod() - DESCRIPTION - the modulus R in R = M mod N + DESCRIPTION + the modulus R in R = M mod N - is defined as + is defined as - 0 <= |R| < |M| - sign R == sign M - R = M - k*N, where k is integer + 0 <= |R| < |M| + sign R == sign M + R = M - k*N, where k is integer - thus, there's no requirement for M or N to be integers + thus, there's no requirement for M or N to be integers */ func DecimalMod(from1, from2, to *MyDecimal) error { from1, from2, to = validateArgs(from1, from2, to) diff --git a/types/time.go b/types/time.go index d54394b642313..c99aba52358e9 100644 --- a/types/time.go +++ b/types/time.go @@ -596,18 +596,17 @@ func TruncateFrac(t gotime.Time, fsp int) (gotime.Time, error) { // ToPackedUint encodes Time to a packed uint64 value. // -// 1 bit 0 -// 17 bits year*13+month (year 0-9999, month 0-12) -// 5 bits day (0-31) -// 5 bits hour (0-23) -// 6 bits minute (0-59) -// 6 bits second (0-59) -// 24 bits microseconds (0-999999) +// 1 bit 0 +// 17 bits year*13+month (year 0-9999, month 0-12) +// 5 bits day (0-31) +// 5 bits hour (0-23) +// 6 bits minute (0-59) +// 6 bits second (0-59) +// 24 bits microseconds (0-999999) // -// Total: 64 bits = 8 bytes -// -// 0YYYYYYY.YYYYYYYY.YYdddddh.hhhhmmmm.mmssssss.ffffffff.ffffffff.ffffffff +// Total: 64 bits = 8 bytes // +// 0YYYYYYY.YYYYYYYY.YYdddddh.hhhhmmmm.mmssssss.ffffffff.ffffffff.ffffffff func (t Time) ToPackedUint() (uint64, error) { tm := t if t.IsZero() { @@ -817,12 +816,13 @@ var validIdxCombinations = map[int]struct { // empty string will be returned. // // Supported syntax: -// MySQL compatible: ((?P[-+])(?P[0-9]{2}):(?P[0-9]{2})){0,1}$, see -// https://dev.mysql.com/doc/refman/8.0/en/time-zone-support.html and https://dev.mysql.com/doc/refman/8.0/en/datetime.html -// the first link specified that timezone information should be in "[H]H:MM, prefixed with a + or -" while the -// second link specified that for string literal, "hour values less than than 10, a leading zero is required.". -// ISO-8601: Z|((((?P[-+])(?P[0-9]{2})(:(?P[0-9]{2}){0,1}){0,1})|((?P[0-9]{2}){0,1}){0,1}))$ -// see https://www.cl.cam.ac.uk/~mgk25/iso-time.html +// +// MySQL compatible: ((?P[-+])(?P[0-9]{2}):(?P[0-9]{2})){0,1}$, see +// https://dev.mysql.com/doc/refman/8.0/en/time-zone-support.html and https://dev.mysql.com/doc/refman/8.0/en/datetime.html +// the first link specified that timezone information should be in "[H]H:MM, prefixed with a + or -" while the +// second link specified that for string literal, "hour values less than than 10, a leading zero is required.". +// ISO-8601: Z|((((?P[-+])(?P[0-9]{2})(:(?P[0-9]{2}){0,1}){0,1})|((?P[0-9]{2}){0,1}){0,1}))$ +// see https://www.cl.cam.ac.uk/~mgk25/iso-time.html func GetTimezone(lit string) (idx int, tzSign, tzHour, tzSep, tzMinute string) { idx, zidx, sidx, spidx := -1, -1, -1, -1 // idx is for the position of the starting of the timezone information @@ -1723,9 +1723,9 @@ func matchDuration(str string, fsp int) (Duration, error) { } // canFallbackToDateTime return true -// 1. the string is failed to be parsed by `matchDuration` -// 2. the string is start with a series of digits whose length match the full format of DateTime literal (12, 14) -// or the string start with a date literal. +// 1. the string is failed to be parsed by `matchDuration` +// 2. the string is start with a series of digits whose length match the full format of DateTime literal (12, 14) +// or the string start with a date literal. func canFallbackToDateTime(str string) bool { digits, rest, err := parser.Digit(str, 1) if err != nil { diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index 39e91f9590311..1e6d05b21ae26 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -62,8 +62,9 @@ func NewChunkWithCapacity(fields []*types.FieldType, capacity int) *Chunk { } // New creates a new chunk. -// cap: the limit for the max number of rows. -// maxChunkSize: the max limit for the number of rows. +// +// cap: the limit for the max number of rows. +// maxChunkSize: the max limit for the number of rows. func New(fields []*types.FieldType, capacity, maxChunkSize int) *Chunk { chk := &Chunk{ columns: make([]*Column, 0, len(fields)), @@ -98,8 +99,9 @@ func renewWithCapacity(chk *Chunk, capacity, requiredRows int) *Chunk { // Renew creates a new Chunk based on an existing Chunk. The newly created Chunk // has the same data schema with the old Chunk. The capacity of the new Chunk // might be doubled based on the capacity of the old Chunk and the maxChunkSize. -// chk: old chunk(often used in previous call). -// maxChunkSize: the limit for the max number of rows. +// +// chk: old chunk(often used in previous call). +// maxChunkSize: the limit for the max number of rows. func Renew(chk *Chunk, maxChunkSize int) *Chunk { newCap := reCalcCapacity(chk, maxChunkSize) return renewWithCapacity(chk, newCap, maxChunkSize) diff --git a/util/chunk/codec.go b/util/chunk/codec.go index d7c30faec11dc..beaa409563ab5 100644 --- a/util/chunk/codec.go +++ b/util/chunk/codec.go @@ -233,17 +233,17 @@ func init() { // Decoder decodes the data returned from the coprocessor and stores the result in Chunk. // How Decoder works: -// 1. Initialization phase: Decode a whole input byte slice to Decoder.intermChk(intermediate chunk) using Codec.Decode. -// intermChk is introduced to simplify the implementation of decode phase. This phase uses pointer operations with -// less CPU and memory cost. -// 2. Decode phase: -// 2.1 Set the number of rows to be decoded to a value that is a multiple of 8 and greater than -// `chk.RequiredRows() - chk.NumRows()`. This reduces the overhead of copying the srcCol.nullBitMap into -// destCol.nullBitMap. -// 2.2 Append srcCol.offsets to destCol.offsets when the elements is of var-length type. And further adjust the -// offsets according to descCol.offsets[destCol.length]-srcCol.offsets[0]. -// 2.3 Append srcCol.nullBitMap to destCol.nullBitMap. -// 3. Go to step 1 when the input byte slice is consumed. +// 1. Initialization phase: Decode a whole input byte slice to Decoder.intermChk(intermediate chunk) using Codec.Decode. +// intermChk is introduced to simplify the implementation of decode phase. This phase uses pointer operations with +// less CPU and memory cost. +// 2. Decode phase: +// 2.1 Set the number of rows to be decoded to a value that is a multiple of 8 and greater than +// `chk.RequiredRows() - chk.NumRows()`. This reduces the overhead of copying the srcCol.nullBitMap into +// destCol.nullBitMap. +// 2.2 Append srcCol.offsets to destCol.offsets when the elements is of var-length type. And further adjust the +// offsets according to descCol.offsets[destCol.length]-srcCol.offsets[0]. +// 2.3 Append srcCol.nullBitMap to destCol.nullBitMap. +// 3. Go to step 1 when the input byte slice is consumed. type Decoder struct { intermChk *Chunk codec *Codec diff --git a/util/chunk/disk_test.go b/util/chunk/disk_test.go index ea056c48e9467..5c3d001b12ab4 100644 --- a/util/chunk/disk_test.go +++ b/util/chunk/disk_test.go @@ -272,7 +272,8 @@ func TestListInDiskWithChecksumAndEncrypt(t *testing.T) { // 8 B: all columns' length, in the following example, we will only have one column. // 1012 B: data in file. because max length of each segment is 1024, so we only have 1020B for user payload. // -// Data in File Data in mem cache +// Data in File Data in mem cache +// // +------+------------------------------------------+ +-----------------------------+ // | | 1020B payload | | | // |4Bytes| +---------+----------------------------+ | | | diff --git a/util/chunk/iterator.go b/util/chunk/iterator.go index ed9905b916a93..7cfd9001a439e 100644 --- a/util/chunk/iterator.go +++ b/util/chunk/iterator.go @@ -25,9 +25,9 @@ var ( // Iterator is used to iterate a number of rows. // -// for row := it.Begin(); row != it.End(); row = it.Next() { -// ... -// } +// for row := it.Begin(); row != it.End(); row = it.Next() { +// ... +// } type Iterator interface { // Begin resets the cursor of the iterator and returns the first Row. Begin() Row diff --git a/util/codec/bytes.go b/util/codec/bytes.go index c4f61442822fe..ab2b47f8079ff 100644 --- a/util/codec/bytes.go +++ b/util/codec/bytes.go @@ -34,14 +34,18 @@ var ( // EncodeBytes guarantees the encoded value is in ascending order for comparison, // encoding with the following rule: -// [group1][marker1]...[groupN][markerN] -// group is 8 bytes slice which is padding with 0. -// marker is `0xFF - padding 0 count` +// +// [group1][marker1]...[groupN][markerN] +// group is 8 bytes slice which is padding with 0. +// marker is `0xFF - padding 0 count` +// // For example: -// [] -> [0, 0, 0, 0, 0, 0, 0, 0, 247] -// [1, 2, 3] -> [1, 2, 3, 0, 0, 0, 0, 0, 250] -// [1, 2, 3, 0] -> [1, 2, 3, 0, 0, 0, 0, 0, 251] -// [1, 2, 3, 4, 5, 6, 7, 8] -> [1, 2, 3, 4, 5, 6, 7, 8, 255, 0, 0, 0, 0, 0, 0, 0, 0, 247] +// +// [] -> [0, 0, 0, 0, 0, 0, 0, 0, 247] +// [1, 2, 3] -> [1, 2, 3, 0, 0, 0, 0, 0, 250] +// [1, 2, 3, 0] -> [1, 2, 3, 0, 0, 0, 0, 0, 251] +// [1, 2, 3, 4, 5, 6, 7, 8] -> [1, 2, 3, 4, 5, 6, 7, 8, 255, 0, 0, 0, 0, 0, 0, 0, 0, 247] +// // Refer: https://github.com/facebook/mysql-5.6/wiki/MyRocks-record-format#memcomparable-format func EncodeBytes(b []byte, data []byte) []byte { // Allocate more space to avoid unnecessary slice growing. diff --git a/util/codec/codec.go b/util/codec/codec.go index 74e8a510ae1fd..07fea23e7fbef 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -1324,7 +1324,8 @@ func ConvertByCollationStr(str string, tp *types.FieldType) string { // HashCode encodes a Datum into a unique byte slice. // It is mostly the same as EncodeValue, but it doesn't contain truncation or verification logic in order -// to make the encoding lossless. +// +// to make the encoding lossless. func HashCode(b []byte, d types.Datum) []byte { switch d.Kind() { case types.KindInt64: diff --git a/util/cteutil/storage.go b/util/cteutil/storage.go index c84cbc5217ad3..5fdaf4424db8c 100644 --- a/util/cteutil/storage.go +++ b/util/cteutil/storage.go @@ -30,12 +30,12 @@ var _ Storage = &StorageRC{} // // Common usage as follows: // -// storage.Lock() -// if !storage.Done() { -// fill all data into storage -// } -// storage.UnLock() -// read data from storage +// storage.Lock() +// if !storage.Done() { +// fill all data into storage +// } +// storage.UnLock() +// read data from storage type Storage interface { // If is first called, will open underlying storage. Otherwise will add ref count by one. OpenAndRef() error diff --git a/util/disk/tracker.go b/util/disk/tracker.go index 57502ebeab7fc..e8740f3c070ae 100644 --- a/util/disk/tracker.go +++ b/util/disk/tracker.go @@ -22,8 +22,8 @@ import ( type Tracker = memory.Tracker // NewTracker creates a disk tracker. -// 1. "label" is the label used in the usage string. -// 2. "bytesLimit <= 0" means no limit. +// 1. "label" is the label used in the usage string. +// 2. "bytesLimit <= 0" means no limit. var NewTracker = memory.NewTracker // NewGlobalTrcaker creates a global disk tracker. diff --git a/util/errors.go b/util/errors.go index 7389177aec1a0..c85828be0b0bc 100644 --- a/util/errors.go +++ b/util/errors.go @@ -18,7 +18,7 @@ import ( "github.com/pingcap/errors" ) -//OriginError return original err +// OriginError return original err func OriginError(err error) error { for { e := errors.Cause(err) diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index ea1e4cda1e746..ab33b37fff610 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -954,12 +954,12 @@ func (e *RuntimeStatsWithCommit) formatBackoff(backoffTypes []string) string { // FormatDuration uses to format duration, this function will prune precision before format duration. // Pruning precision is for human readability. The prune rule is: -// 1. if the duration was less than 1us, return the original string. -// 2. readable value >=10, keep 1 decimal, otherwise, keep 2 decimal. such as: -// 9.412345ms -> 9.41ms -// 10.412345ms -> 10.4ms -// 5.999s -> 6s -// 100.45µs -> 100.5µs +// 1. if the duration was less than 1us, return the original string. +// 2. readable value >=10, keep 1 decimal, otherwise, keep 2 decimal. such as: +// 9.412345ms -> 9.41ms +// 10.412345ms -> 10.4ms +// 5.999s -> 6s +// 100.45µs -> 100.5µs func FormatDuration(d time.Duration) string { if d <= time.Microsecond { return d.String() diff --git a/util/execdetails/main_test.go b/util/execdetails/main_test.go index 0f3742206df1d..dd95111ca4830 100644 --- a/util/execdetails/main_test.go +++ b/util/execdetails/main_test.go @@ -4,7 +4,7 @@ // you may not use this file except in compliance with the License. // You may obtain a copy of the License at // -// http://www.apache.org/licenses/LICENSE-2.0 +// http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, diff --git a/util/fastrand/random.go b/util/fastrand/random.go index 90fa1f3f41748..33458a3d87678 100644 --- a/util/fastrand/random.go +++ b/util/fastrand/random.go @@ -49,6 +49,7 @@ func Buf(size int) []byte { } // Uint32 returns a lock free uint32 value. +// //go:linkname Uint32 runtime.fastrand func Uint32() uint32 diff --git a/util/format/format.go b/util/format/format.go index a6fd3c9d05940..fb307d593c605 100644 --- a/util/format/format.go +++ b/util/format/format.go @@ -57,21 +57,28 @@ var replace = map[rune]string{ // nest. The Formatter writes to io.Writer 'w' and inserts one 'indent' // string per current indent level value. // Behaviour of commands reaching negative indent levels is undefined. -// IndentFormatter(os.Stdout, "\t").Format("abc%d%%e%i\nx\ny\n%uz\n", 3) +// +// IndentFormatter(os.Stdout, "\t").Format("abc%d%%e%i\nx\ny\n%uz\n", 3) +// // output: -// abc3%e -// x -// y -// z +// +// abc3%e +// x +// y +// z +// // The Go quoted string literal form of the above is: -// "abc%%e\n\tx\n\tx\nz\n" +// +// "abc%%e\n\tx\n\tx\nz\n" +// // The commands can be scattered between separate invocations of Format(), // i.e. the formatter keeps track of the indent level and knows if it is // positioned on start of a line and should emit indentation(s). // The same output as above can be produced by e.g.: -// f := IndentFormatter(os.Stdout, " ") -// f.Format("abc%d%%e%i\nx\n", 3) -// f.Format("y\n%uz\n") +// +// f := IndentFormatter(os.Stdout, " ") +// f.Format("abc%d%%e%i\nx\n", 3) +// f.Format("y\n%uz\n") func IndentFormatter(w io.Writer, indent string) Formatter { return &indentFormatter{w, []byte(indent), 0, stBOL} } @@ -170,9 +177,12 @@ type flatFormatter indentFormatter // // The FlatFormatter is intended for flattening of normally nested structure textual representation to // a one top level structure per line form. -// FlatFormatter(os.Stdout, " ").Format("abc%d%%e%i\nx\ny\n%uz\n", 3) +// +// FlatFormatter(os.Stdout, " ").Format("abc%d%%e%i\nx\ny\n%uz\n", 3) +// // output in the form of a Go quoted string literal: -// "abc3%%e x y z\n" +// +// "abc3%%e x y z\n" func FlatFormatter(w io.Writer) Formatter { return (*flatFormatter)(IndentFormatter(w, "").(*indentFormatter)) } diff --git a/util/memory/tracker.go b/util/memory/tracker.go index 106ff210e83ed..d120d136ada5a 100644 --- a/util/memory/tracker.go +++ b/util/memory/tracker.go @@ -90,8 +90,9 @@ type bytesLimits struct { } // InitTracker initializes a memory tracker. -// 1. "label" is the label used in the usage string. -// 2. "bytesLimit <= 0" means no limit. +// 1. "label" is the label used in the usage string. +// 2. "bytesLimit <= 0" means no limit. +// // For the common tracker, isGlobal is default as false func InitTracker(t *Tracker, label int, bytesLimit int64, action ActionOnExceed) { t.mu.children = nil @@ -109,8 +110,9 @@ func InitTracker(t *Tracker, label int, bytesLimit int64, action ActionOnExceed) } // NewTracker creates a memory tracker. -// 1. "label" is the label used in the usage string. -// 2. "bytesLimit <= 0" means no limit. +// 1. "label" is the label used in the usage string. +// 2. "bytesLimit <= 0" means no limit. +// // For the common tracker, isGlobal is default as false func NewTracker(label int, bytesLimit int64) *Tracker { t := &Tracker{ diff --git a/util/misc.go b/util/misc.go index 9434ee3f98106..a8c66cb998db1 100644 --- a/util/misc.go +++ b/util/misc.go @@ -74,8 +74,9 @@ func RunWithRetry(retryCnt int, backoff uint64, f func() (bool, error)) (err err // WithRecovery wraps goroutine startup call with force recovery. // it will dump current goroutine stack into log if catch any recover result. -// exec: execute logic function. -// recoverFn: handler will be called after recover and before dump stack, passing `nil` means noop. +// +// exec: execute logic function. +// recoverFn: handler will be called after recover and before dump stack, passing `nil` means noop. func WithRecovery(exec func(), recoverFn func(r interface{})) { defer func() { r := recover() @@ -93,10 +94,11 @@ func WithRecovery(exec func(), recoverFn func(r interface{})) { // Recover includes operations such as recovering, clearing,and printing information. // It will dump current goroutine stack into log if catch any recover result. -// metricsLabel: The label of PanicCounter metrics. -// funcInfo: Some information for the panic function. -// recoverFn: Handler will be called after recover and before dump stack, passing `nil` means noop. -// quit: If this value is true, the current program exits after recovery. +// +// metricsLabel: The label of PanicCounter metrics. +// funcInfo: Some information for the panic function. +// recoverFn: Handler will be called after recover and before dump stack, passing `nil` means noop. +// quit: If this value is true, the current program exits after recovery. func Recover(metricsLabel, funcInfo string, recoverFn func(), quit bool) { r := recover() if r == nil { diff --git a/util/mock/iter_test.go b/util/mock/iter_test.go index 9e142973185ea..3a9d3ea9f0b2c 100644 --- a/util/mock/iter_test.go +++ b/util/mock/iter_test.go @@ -4,7 +4,7 @@ // you may not use this file except in compliance with the License. // You may obtain a copy of the License at // -// http://www.apache.org/licenses/LICENSE-2.0 +// http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, diff --git a/util/processinfo.go b/util/processinfo.go index 30b95aec100ae..e9f496f73f60a 100644 --- a/util/processinfo.go +++ b/util/processinfo.go @@ -184,13 +184,17 @@ type SessionManager interface { // GlobalConnID is the global connection ID, providing UNIQUE connection IDs across the whole TiDB cluster. // 64 bits version: -// 63 62 41 40 1 0 +// +// 63 62 41 40 1 0 +// // +--+---------------------+--------------------------------------+------+ // | | serverId | local connId |markup| // |=0| (22b) | (40b) | =1 | // +--+---------------------+--------------------------------------+------+ // 32 bits version(coming soon): -// 31 1 0 +// +// 31 1 0 +// // +-----------------------------+------+ // | ??? |markup| // | ??? | =0 | @@ -250,7 +254,8 @@ func (g *GlobalConnID) NextID() uint64 { } // ParseGlobalConnID parses an uint64 to GlobalConnID. -// `isTruncated` indicates that older versions of the client truncated the 64-bit GlobalConnID to 32-bit. +// +// `isTruncated` indicates that older versions of the client truncated the 64-bit GlobalConnID to 32-bit. func ParseGlobalConnID(id uint64) (g GlobalConnID, isTruncated bool, err error) { if id&0x80000000_00000000 > 0 { return GlobalConnID{}, false, errors.New("Unexpected connectionID excceeds int64") diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index 037a5402f048b..116cc687b9ce2 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -505,7 +505,9 @@ func extractValueInfo(expr expression.Expression) *valueInfo { // accesses: The condition will be used to build range. // filters: filters is the part that some access conditions need to be evaluate again since it's only the prefix part of char column. // newConditions: We'll simplify the given conditions if there're multiple in conditions or eq conditions on the same column. -// e.g. if there're a in (1, 2, 3) and a in (2, 3, 4). This two will be combined to a in (2, 3) and pushed to newConditions. +// +// e.g. if there're a in (1, 2, 3) and a in (2, 3, 4). This two will be combined to a in (2, 3) and pushed to newConditions. +// // columnValues: the constant column values for all index columns. columnValues[i] is nil if cols[i] is not constant. // bool: indicate whether there's nil range when merging eq and in conditions. func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Expression, cols []*expression.Column, @@ -882,14 +884,19 @@ func MergeDNFItems4Col(ctx sessionctx.Context, dnfItems []expression.Expression) // AddGcColumnCond add the `tidb_shard(x) = xxx` to the condition // @param[in] cols the columns of shard index, such as [tidb_shard(a), a, ...] // @param[in] accessCond the conditions relative to the index and arranged by the index column order. -// e.g. the index is uk(tidb_shard(a), a, b) and the where clause is -// `WHERE b = 1 AND a = 2 AND c = 3`, the param accessCond is {a = 2, b = 1} that is -// only relative to uk's columns. +// +// e.g. the index is uk(tidb_shard(a), a, b) and the where clause is +// `WHERE b = 1 AND a = 2 AND c = 3`, the param accessCond is {a = 2, b = 1} that is +// only relative to uk's columns. +// // @param[in] columnValues the values of index columns in param accessCond. if accessCond is {a = 2, b = 1}, -// columnValues is {2, 1}. if accessCond the "IN" function like `a IN (1, 2)`, columnValues -// is empty. +// +// columnValues is {2, 1}. if accessCond the "IN" function like `a IN (1, 2)`, columnValues +// is empty. +// // @retval - []expression.Expression the new conditions after adding `tidb_shard() = xxx` prefix -// error if error gernerated, return error +// +// error if error gernerated, return error func AddGcColumnCond(sctx sessionctx.Context, cols []*expression.Column, accessesCond []expression.Expression, @@ -912,7 +919,8 @@ func AddGcColumnCond(sctx sessionctx.Context, // AddGcColumn4InCond add the `tidb_shard(x) = xxx` for `IN` condition // For param explanation, please refer to the function `AddGcColumnCond`. // @retval - []expression.Expression the new conditions after adding `tidb_shard() = xxx` prefix -// error if error gernerated, return error +// +// error if error gernerated, return error func AddGcColumn4InCond(sctx sessionctx.Context, cols []*expression.Column, accessesCond []expression.Expression) ([]expression.Expression, error) { @@ -979,8 +987,9 @@ func AddGcColumn4InCond(sctx sessionctx.Context, // AddGcColumn4EqCond add the `tidb_shard(x) = xxx` prefix for equal condition // For param explanation, please refer to the function `AddGcColumnCond`. // @retval - []expression.Expression the new conditions after adding `tidb_shard() = xxx` prefix -// []*valueInfo the values of every columns in the returned new conditions -// error if error gernerated, return error +// +// []*valueInfo the values of every columns in the returned new conditions +// error if error gernerated, return error func AddGcColumn4EqCond(sctx sessionctx.Context, cols []*expression.Column, accessesCond []expression.Expression, @@ -1085,12 +1094,16 @@ func AddExpr4EqAndInCondition(sctx sessionctx.Context, conditions []expression.E // NeedAddGcColumn4ShardIndex check whether to add `tidb_shard(x) = xxx` // @param[in] cols the columns of shard index, such as [tidb_shard(a), a, ...] // @param[in] accessCond the conditions relative to the index and arranged by the index column order. -// e.g. the index is uk(tidb_shard(a), a, b) and the where clause is -// `WHERE b = 1 AND a = 2 AND c = 3`, the param accessCond is {a = 2, b = 1} that is -// only relative to uk's columns. +// +// e.g. the index is uk(tidb_shard(a), a, b) and the where clause is +// `WHERE b = 1 AND a = 2 AND c = 3`, the param accessCond is {a = 2, b = 1} that is +// only relative to uk's columns. +// // @param[in] columnValues the values of index columns in param accessCond. if accessCond is {a = 2, b = 1}, -// columnValues is {2, 1}. if accessCond the "IN" function like `a IN (1, 2)`, columnValues -// is empty. +// +// columnValues is {2, 1}. if accessCond the "IN" function like `a IN (1, 2)`, columnValues +// is empty. +// // @retval - return true if it needs to addr tidb_shard() prefix, ohterwise return false func NeedAddGcColumn4ShardIndex( cols []*expression.Column, @@ -1174,8 +1187,10 @@ func NeedAddColumn4EqCond(cols []*expression.Column, // (2) the first param of "IN" function should be a column not a expression like `a + b` // (3) the rest params of "IN" function all should be constant // (4) the first param of "IN" function should be the column in the expression of first index field. -// e.g. uk(tidb_shard(a), a). If the conditions is `WHERE b in (1, 2, 3)`, the first param of "IN" function -// is `b` that's not the column in `tidb_shard(a)`. +// +// e.g. uk(tidb_shard(a), a). If the conditions is `WHERE b in (1, 2, 3)`, the first param of "IN" function +// is `b` that's not the column in `tidb_shard(a)`. +// // @param sf "IN" function, e.g. `a IN (1, 2, 3)` func NeedAddColumn4InCond(cols []*expression.Column, accessCond []expression.Expression, sf *expression.ScalarFunction) bool { if len(cols) == 0 || len(accessCond) == 0 || sf == nil { diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index 551cd7fae176e..61e80cbc29ab4 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -490,14 +490,17 @@ func hasPrefix(lengths []int) bool { // change the exclude status of that point and return `true` to tell // that we need do a range merging since that interval may have intersection. // e.g. if the interval is (-inf -inf, a xxxxx), (a xxxxx, +inf +inf) and the length of the last column is 3, -// then we'll change it to (-inf -inf, a xxx], [a xxx, +inf +inf). You can see that this two interval intersect, -// so we need a merge operation. +// +// then we'll change it to (-inf -inf, a xxx], [a xxx, +inf +inf). You can see that this two interval intersect, +// so we need a merge operation. +// // Q: only checking the last column to decide whether the endpoint's exclude status needs to be reset is enough? // A: Yes, suppose that the interval is (-inf -inf, a xxxxx b) and only the second column needs to be cut. -// The result would be (-inf -inf, a xxx b) if the length of it is 3. Obviously we only need to care about the data -// whose the first two key is `a` and `xxx`. It read all data whose index value begins with `a` and `xxx` and the third -// value less than `b`, covering the values begin with `a` and `xxxxx` and the third value less than `b` perfectly. -// So in this case we don't need to reset its exclude status. The right endpoint case can be proved in the same way. +// +// The result would be (-inf -inf, a xxx b) if the length of it is 3. Obviously we only need to care about the data +// whose the first two key is `a` and `xxx`. It read all data whose index value begins with `a` and `xxx` and the third +// value less than `b`, covering the values begin with `a` and `xxxxx` and the third value less than `b` perfectly. +// So in this case we don't need to reset its exclude status. The right endpoint case can be proved in the same way. func fixPrefixColRange(ranges []*Range, lengths []int, tp []*types.FieldType) bool { var hasCut bool for _, ran := range ranges { diff --git a/util/sqlexec/utils.go b/util/sqlexec/utils.go index 8fa6bdd4d804a..8999c8010f3e6 100644 --- a/util/sqlexec/utils.go +++ b/util/sqlexec/utils.go @@ -225,7 +225,7 @@ func escapeSQL(sql string, args ...interface{}) ([]byte, error) { // 1. %?: automatic conversion by the type of arguments. E.g. []string -> ('s1','s2'..) // 2. %%: output % // 3. %n: for identifiers, for example ("use %n", db) -// But it does not prevent you from doing EscapeSQL("select '%?", ";SQL injection!;") => "select '';SQL injection!;'". +// But it does not prevent you from doing EscapeSQL("select '%?", ";SQL injection!;") => "select ”;SQL injection!;'". // It is still your responsibility to write safe SQL. func EscapeSQL(sql string, args ...interface{}) (string, error) { str, err := escapeSQL(sql, args...) diff --git a/util/stringutil/string_util.go b/util/stringutil/string_util.go index 1ab9a45cc8ad0..b42eb0ea9fb38 100644 --- a/util/stringutil/string_util.go +++ b/util/stringutil/string_util.go @@ -323,7 +323,7 @@ func (i StringerStr) String() string { } // Escape the identifier for pretty-printing. -// For instance, the identifier "foo `bar`" will become "`foo ``bar```". +// For instance, the identifier "foo `bar`" will become "`foo “bar```". // The sqlMode controls whether to escape with backquotes (`) or double quotes // (`"`) depending on whether mysql.ModeANSIQuotes is enabled. func Escape(str string, sqlMode mysql.SQLMode) string { diff --git a/util/sys/linux/sys_test.go b/util/sys/linux/sys_test.go index 7ce52ca8f123d..675e83db02c26 100644 --- a/util/sys/linux/sys_test.go +++ b/util/sys/linux/sys_test.go @@ -4,7 +4,7 @@ // you may not use this file except in compliance with the License. // You may obtain a copy of the License at // -// http://www.apache.org/licenses/LICENSE-2.0 +// http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, diff --git a/util/table-rule-selector/trie_selector.go b/util/table-rule-selector/trie_selector.go index 07cec79938ec9..5fc4a82cdcce8 100644 --- a/util/table-rule-selector/trie_selector.go +++ b/util/table-rule-selector/trie_selector.go @@ -21,10 +21,10 @@ import ( "github.com/pingcap/errors" ) -// 1. asterisk character (*, also called "star") matches zero or more characters, -// for example, doc* matches doc and document but not dodo; -// asterisk character must be the last character of wildcard word. -// 2. the question mark ? matches exactly one character +// 1. asterisk character (*, also called "star") matches zero or more characters, +// for example, doc* matches doc and document but not dodo; +// asterisk character must be the last character of wildcard word. +// 2. the question mark ? matches exactly one character const ( // asterisk [ * ] asterisk = '*' diff --git a/util/timeutil/time.go b/util/timeutil/time.go index f855cf9afde49..62d67bce117c6 100644 --- a/util/timeutil/time.go +++ b/util/timeutil/time.go @@ -154,7 +154,8 @@ func GetSystemTZ() (string, error) { // getLoc first trying to load location from a cache map. If nothing found in such map, then call // `time.LoadLocation` to get a timezone location. After trying both way, an error will be returned -// if valid Location is not found. +// +// if valid Location is not found. func (lm *locCache) getLoc(name string) (*time.Location, error) { if name == "System" { return time.Local, nil diff --git a/util/topsql/collector/cpu.go b/util/topsql/collector/cpu.go index eb7ba6183c3a6..d7131f2748b3b 100644 --- a/util/topsql/collector/cpu.go +++ b/util/topsql/collector/cpu.go @@ -214,22 +214,25 @@ type sqlStats struct { // tune use to adjust sql stats. Consider following situation: // The `sqlStats` maybe: -// plans: { -// "table_scan": 200ms, // The cpu time of the sql that plan with `table_scan` is 200ms. -// "index_scan": 300ms, // The cpu time of the sql that plan with `index_scan` is 300ms. -// }, -// total: 600ms, // The total cpu time of the sql is 600ms. +// +// plans: { +// "table_scan": 200ms, // The cpu time of the sql that plan with `table_scan` is 200ms. +// "index_scan": 300ms, // The cpu time of the sql that plan with `index_scan` is 300ms. +// }, +// total: 600ms, // The total cpu time of the sql is 600ms. +// // total_time - table_scan_time - index_scan_time = 100ms, and this 100ms means those sample data only contain the // sql_digest label, doesn't contain the plan_digest label. This is cause by the `pprof profile` is base on sample, // and the plan digest can only be set after optimizer generated execution plan. So the remain 100ms means the plan // optimizer takes time to generated plan. // After this tune function, the `sqlStats` become to: -// plans: { -// "" : 100ms, // 600 - 200 - 300 = 100ms, indicate the optimizer generated plan time cost. -// "table_scan": 200ms, -// "index_scan": 300ms, -// }, -// total: 600ms, +// +// plans: { +// "" : 100ms, // 600 - 200 - 300 = 100ms, indicate the optimizer generated plan time cost. +// "table_scan": 200ms, +// "index_scan": 300ms, +// }, +// total: 600ms, func (s *sqlStats) tune() { if len(s.plans) == 0 { s.plans[""] = s.total diff --git a/util/watcher/watcher.go b/util/watcher/watcher.go index f6b81683257e6..23a65350bfe13 100644 --- a/util/watcher/watcher.go +++ b/util/watcher/watcher.go @@ -34,10 +34,10 @@ var ( // Watcher watches for files or directory changes by polling // currently, if multi operations applied to one file or directory, only one event (with single Op) will be sent // the priority of Op is: -// 1. Modify -// 2. Chmod -// 3. Rename / Move -// 4. Create / Remove +// 1. Modify +// 2. Chmod +// 3. Rename / Move +// 4. Create / Remove type Watcher struct { Events chan Event Errors chan error