From 31167ed0fe6e609056e071f7dcabd4657e29660a Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 3 Aug 2022 09:43:58 +0800 Subject: [PATCH 01/18] *: set go 1.19 Signed-off-by: Weizhen Wang --- WORKSPACE | 2 +- 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/precheck_impl.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/streamhelper/advancer.go | 36 +-- br/pkg/streamhelper/advancer_daemon.go | 8 +- br/pkg/streamhelper/collector.go | 28 ++- br/pkg/streamhelper/tsheap.go | 11 +- br/pkg/task/stream.go | 2 +- br/pkg/utils/worker.go | 14 +- br/tests/br_key_locked/locker.go | 1 + build/linter/util/util.go | 1 - ddl/backfilling.go | 9 +- ddl/fail_test.go | 3 +- ddl/placement_policy_ddl_test.go | 2 +- ddl/reorg.go | 24 +- distsql/request_builder.go | 11 +- dumpling/export/writer_util.go | 5 +- executor/aggfuncs/func_ntile.go | 3 +- executor/aggregate.go | 75 +++--- executor/builder.go | 4 +- executor/compact_table.go | 10 +- executor/cte.go | 34 +-- 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/plan_replayer.go | 27 ++- executor/projection.go | 70 +++--- 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 +- meta/meta.go | 10 +- parser/ast/ddl.go | 10 +- parser/ast/dml.go | 32 ++- parser/ast/misc.go | 11 +- parser/auth/mysql_native_password.go | 27 ++- parser/format/format.go | 32 ++- parser/goyacc/main.go | 20 +- parser/model/model.go | 5 +- parser/test_driver/test_driver_mydecimal.go | 13 +- parser/types/etc.go | 4 +- 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/flat_plan.go | 5 +- planner/core/logical_plan_builder.go | 19 +- planner/core/partition_pruner_test.go | 19 +- planner/core/physical_plans.go | 13 +- planner/core/planbuilder.go | 2 +- planner/core/preprocess.go | 12 +- planner/core/rule_aggregation_push_down.go | 4 +- planner/core/rule_aggregation_skew_rewrite.go | 11 +- 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/session.go | 7 +- sessionctx/variable/sysvar.go | 4 +- sessiontxn/isolation/base.go | 3 +- sessiontxn/isolation/repeatable_read.go | 4 +- sessiontxn/staleread/main_test.go | 2 +- sessiontxn/staleread/provider_test.go | 2 +- statistics/builder.go | 3 +- statistics/cmsketch.go | 26 +- statistics/handle/handle_test.go | 2 +- statistics/handle/update.go | 10 +- statistics/histogram.go | 63 ++--- 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 | 229 +++++++++--------- types/time.go | 38 +-- util/chunk/chunk.go | 10 +- util/chunk/codec.go | 22 +- util/chunk/disk_test.go | 19 +- 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 | 27 ++- 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 +- 153 files changed, 1291 insertions(+), 1079 deletions(-) diff --git a/WORKSPACE b/WORKSPACE index 744ef2aed9801..f95edb1a9fd43 100644 --- a/WORKSPACE +++ b/WORKSPACE @@ -33,7 +33,7 @@ go_rules_dependencies() go_register_toolchains( nogo = "@//build:tidb_nogo", - version = "1.18.5", + version = "1.19", ) gazelle_dependencies() diff --git a/br/pkg/glue/console_glue.go b/br/pkg/glue/console_glue.go index 3b64faa14b56f..c69e24302e5ad 100644 --- a/br/pkg/glue/console_glue.go +++ b/br/pkg/glue/console_glue.go @@ -129,8 +129,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() { @@ -305,7 +307,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 6e56f5dae04c3..aebaba8456b4d 100644 --- a/br/pkg/lightning/backend/local/duplicate.go +++ b/br/pkg/lightning/backend/local/duplicate.go @@ -246,6 +246,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 6d6c6f6999cb0..deb1d46ccab46 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 5b22fbd942d60..75a1d9a6e9bcb 100644 --- a/br/pkg/lightning/backend/tidb/tidb.go +++ b/br/pkg/lightning/backend/tidb/tidb.go @@ -627,7 +627,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) @@ -655,10 +656,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 097c51dfdfe5f..775963a8549c4 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 4e0422ad7e7ff..d7b964d44dfad 100644 --- a/br/pkg/lightning/lightning.go +++ b/br/pkg/lightning/lightning.go @@ -257,11 +257,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 { @@ -309,10 +310,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 a16ad88de76c2..765c071351b64 100644 --- a/br/pkg/lightning/mydump/loader.go +++ b/br/pkg/lightning/mydump/loader.go @@ -100,7 +100,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/precheck_impl.go b/br/pkg/lightning/restore/precheck_impl.go index 910ecea780681..cbc5a92a1ec75 100644 --- a/br/pkg/lightning/restore/precheck_impl.go +++ b/br/pkg/lightning/restore/precheck_impl.go @@ -866,10 +866,10 @@ func (ci *csvHeaderCheckItem) GetCheckItemID() CheckItemID { } // Check tries 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 (ci *csvHeaderCheckItem) Check(ctx context.Context) (*CheckResult, error) { // if cfg set header = true 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 664f081e56db0..c761d53693364 100644 --- a/br/pkg/restore/db.go +++ b/br/pkg/restore/db.go @@ -171,7 +171,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 3cc635021d327..b726a5ec78729 100644 --- a/br/pkg/restore/split_test.go +++ b/br/pkg/restore/split_test.go @@ -320,8 +320,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(false) @@ -474,8 +475,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]*split.RegionInfo) bool { keys := [...]string{"", "aay", "bba", "bbf", "bbh", "bbj", "cca", "xxe", "xxz", ""} return validateRegionsExt(regions, keys[:], false) diff --git a/br/pkg/restore/stream_metas.go b/br/pkg/restore/stream_metas.go index b4c2c85e76d20..b96e28822f726 100644 --- a/br/pkg/restore/stream_metas.go +++ b/br/pkg/restore/stream_metas.go @@ -85,10 +85,12 @@ func (ms *StreamMetadataSet) CalculateShiftTS(startTS uint64) uint64 { } // 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 3792380d99f3c..d5aeb74b52cf0 100644 --- a/br/pkg/stream/stream_status.go +++ b/br/pkg/stream/stream_status.go @@ -255,7 +255,8 @@ type PDInfoProvider interface { // 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 PDInfoProvider) (float64, error) { c := mgr.GetPDClient() prefix := "http://" diff --git a/br/pkg/streamhelper/advancer.go b/br/pkg/streamhelper/advancer.go index 89abd721242c8..aa4f61042d1b2 100644 --- a/br/pkg/streamhelper/advancer.go +++ b/br/pkg/streamhelper/advancer.go @@ -29,23 +29,25 @@ import ( // CheckpointAdvancer is the central node for advancing the checkpoint of log backup. // It's a part of "checkpoint v3". // Generally, it scan the regions in the task range, collect checkpoints from tikvs. -// ┌──────┐ -// ┌────►│ TiKV │ -// │ └──────┘ -// │ -// │ -// ┌──────────┐GetLastFlushTSOfRegion│ ┌──────┐ -// │ Advancer ├──────────────────────┼────►│ TiKV │ -// └────┬─────┘ │ └──────┘ -// │ │ -// │ │ -// │ │ ┌──────┐ -// │ └────►│ TiKV │ -// │ └──────┘ -// │ -// │ UploadCheckpointV3 ┌──────────────────┐ -// └─────────────────────►│ PD │ -// └──────────────────┘ +/* + ┌──────┐ + ┌────►│ TiKV │ + │ └──────┘ + │ + │ + ┌──────────┐GetLastFlushTSOfRegion│ ┌──────┐ + │ Advancer ├──────────────────────┼────►│ TiKV │ + └────┬─────┘ │ └──────┘ + │ │ + │ │ + │ │ ┌──────┐ + │ └────►│ TiKV │ + │ └──────┘ + │ + │ UploadCheckpointV3 ┌──────────────────┐ + └─────────────────────►│ PD │ + └──────────────────┘ +*/ type CheckpointAdvancer struct { env Env diff --git a/br/pkg/streamhelper/advancer_daemon.go b/br/pkg/streamhelper/advancer_daemon.go index 909bdd85df3c6..8cb7552273dff 100644 --- a/br/pkg/streamhelper/advancer_daemon.go +++ b/br/pkg/streamhelper/advancer_daemon.go @@ -26,9 +26,11 @@ const ( // // ad := NewAdvancerDaemon(adv, mgr) // loop, err := ad.Begin(ctx) -// if err != nil { -// return err -// } +// +// if err != nil { +// return err +// } +// // loop() type AdvancerDaemon struct { adv *CheckpointAdvancer diff --git a/br/pkg/streamhelper/collector.go b/br/pkg/streamhelper/collector.go index d7c42adbfdda0..8e30c22804d39 100644 --- a/br/pkg/streamhelper/collector.go +++ b/br/pkg/streamhelper/collector.go @@ -221,19 +221,21 @@ type runningStoreCollector struct { // clusterCollector is the controller for collecting region checkpoints for the cluster. // It creates multi store collectors. -// ┌──────────────────────┐ Requesting ┌────────────┐ -// ┌─►│ StoreCollector[id=1] ├─────────────►│ TiKV[id=1] │ -// │ └──────────────────────┘ └────────────┘ -// │ -// │Owns -// ┌──────────────────┐ │ ┌──────────────────────┐ Requesting ┌────────────┐ -// │ ClusterCollector ├─────┼─►│ StoreCollector[id=4] ├─────────────►│ TiKV[id=4] │ -// └──────────────────┘ │ └──────────────────────┘ └────────────┘ -// │ -// │ -// │ ┌──────────────────────┐ Requesting ┌────────────┐ -// └─►│ StoreCollector[id=5] ├─────────────►│ TiKV[id=5] │ -// └──────────────────────┘ └────────────┘ +/* + ┌──────────────────────┐ Requesting ┌────────────┐ + ┌─►│ StoreCollector[id=1] ├─────────────►│ TiKV[id=1] │ + │ └──────────────────────┘ └────────────┘ + │ + │Owns + ┌──────────────────┐ │ ┌──────────────────────┐ Requesting ┌────────────┐ + │ ClusterCollector ├─────┼─►│ StoreCollector[id=4] ├─────────────►│ TiKV[id=4] │ + └──────────────────┘ │ └──────────────────────┘ └────────────┘ + │ + │ + │ ┌──────────────────────┐ Requesting ┌────────────┐ + └─►│ StoreCollector[id=5] ├─────────────►│ TiKV[id=5] │ + └──────────────────────┘ └────────────┘ +*/ type clusterCollector struct { mu sync.Mutex collectors map[uint64]runningStoreCollector diff --git a/br/pkg/streamhelper/tsheap.go b/br/pkg/streamhelper/tsheap.go index 75d674c56b18e..f4006e5e44c42 100644 --- a/br/pkg/streamhelper/tsheap.go +++ b/br/pkg/streamhelper/tsheap.go @@ -30,11 +30,12 @@ import ( // - is based on range (it only promises there won't be new committed txns in the range). // - the checkpoint of union of ranges is the minimal checkpoint of all ranges. // As an example: -// +----------------------------------+ -// ^-----------^ (Checkpoint = 42) -// ^---------------^ (Checkpoint = 76) -// ^-----------------------^ (Checkpoint = min(42, 76) = 42) -// +/* + +----------------------------------+ + ^-----------^ (Checkpoint = 42) + ^---------------^ (Checkpoint = 76) + ^-----------------------^ (Checkpoint = min(42, 76) = 42) +*/ // For calculating the global checkpoint, we can make a heap-like structure: // Checkpoint Ranges // 42 -> {[0, 8], [16, 100]} diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index 3c3d20967f58e..7b965cf2a64e4 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -425,7 +425,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/pkg/utils/worker.go b/br/pkg/utils/worker.go index cf80770d0ae67..acdb914c0fbfe 100644 --- a/br/pkg/utils/worker.go +++ b/br/pkg/utils/worker.go @@ -114,15 +114,15 @@ func (pool *WorkerPool) HasWorker() bool { // PanicToErr recovers when the execution get panicked, and set the error provided by the arg. // generally, this would be used with named return value and `defer`, like: // -// func foo() (err error) { -// defer utils.PanicToErr(&err) -// return maybePanic() -// } +// func foo() (err error) { +// defer utils.PanicToErr(&err) +// return maybePanic() +// } // // Before using this, there are some hints for reducing resource leakage or bugs: -// - If any of clean work (by `defer`) relies on the error (say, when error happens, rollback some operations.), please -// place `defer this` AFTER that. -// - All resources allocated should be freed by the `defer` syntax, or when panicking, they may not be recycled. +// - If any of clean work (by `defer`) relies on the error (say, when error happens, rollback some operations.), please +// place `defer this` AFTER that. +// - All resources allocated should be freed by the `defer` syntax, or when panicking, they may not be recycled. func PanicToErr(err *error) { item := recover() if item != nil { 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/build/linter/util/util.go b/build/linter/util/util.go index 7dbf016106f07..c26da13c859d7 100644 --- a/build/linter/util/util.go +++ b/build/linter/util/util.go @@ -27,7 +27,6 @@ import ( "honnef.co/go/tools/analysis/report" ) -// type skipType int const ( diff --git a/ddl/backfilling.go b/ddl/backfilling.go index 97653e8378aab..f45a75228c931 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -578,10 +578,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/fail_test.go b/ddl/fail_test.go index 570522e1303b4..39437b43a2b73 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 026fee08e2bdc..f5d1392018c84 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 f6d2e135352b3..84f36ca23217a 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -152,18 +152,18 @@ func (rc *reorgCtx) getRowCountAndKey() (int64, kv.Key, *meta.Element) { // 1: add index // 2: alter column type // 3: clean global index -// -// ddl goroutine >---------+ -// ^ | -// | | -// | | -// | | <---(doneCh)--- f() -// HandleDDLQueue(...) | <---(regular timeout) -// | | <---(ctx done) -// | | -// | | -// A more ddl round <-----+ -// +/* + ddl goroutine >---------+ + ^ | + | | + | | + | | <---(doneCh)--- f() + HandleDDLQueue(...) | <---(regular timeout) + | | <---(ctx done) + | | + | | + A more ddl round <-----+ +*/ // How can we cancel reorg job? // // The background reorg is continuously running except for several factors, for instances, ddl owner change, diff --git a/distsql/request_builder.go b/distsql/request_builder.go index f8beeb68cae68..70cfb8672dfc4 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 @@ -455,9 +456,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 c43a1d140cab4..978178c36c0f8 100644 --- a/dumpling/export/writer_util.go +++ b/dumpling/export/writer_util.go @@ -627,8 +627,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 534756cc31069..d33f5c8fcee5e 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -118,43 +118,44 @@ type AfFinalResult struct { // It is built from the Aggregate Plan. When Next() is called, it reads all the data from Src // 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----+ | -// inputCh +-+ | +-----------------+---+ -// | | | | -// ... +---+------------+ +----+-----------+ -// | | | partial worker | ...... | partial worker | -// +++ +--------------+-+ +-+--------------+ -// | ^ ^ -// | | | -// +----v---------+ +++ +-+ +++ -// | data fetcher | +------> | | | | ...... | | partialInputChs -// +--------------+ +-+ +-+ +-+ +/* + +-------------+ + | Main Thread | + +------+------+ + ^ + | + + + +-+- +-+ + | | ...... | | finalOutputCh + +++- +-+ + ^ + | + +---------------+ + | | + +--------------+ +--------------+ + | final worker | ...... | final worker | + +------------+-+ +-+------------+ + ^ ^ + | | + +-+ +-+ ...... +-+ + | | | | | | + ... ... ... partialOutputChs + | | | | | | + +++ +++ +++ + ^ ^ ^ + +-+ | | | + | | +--------o----+ | + inputCh +-+ | +-----------------+---+ + | | | | + ... +---+------------+ +----+-----------+ + | | | partial worker | ...... | partial worker | + +++ +--------------+-+ +-+--------------+ + | ^ ^ + | | | + +----v---------+ +++ +-+ +++ + | data fetcher | +------> | | | | ...... | | partialInputChs + +--------------+ +-+ +-+ +-+ +*/ type HashAggExec struct { baseExecutor diff --git a/executor/builder.go b/executor/builder.go index 10dfa54b78596..afea0f87b51a9 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2429,7 +2429,9 @@ func (b *executorBuilder) buildAnalyzeSamplingPushdown(task plannercore.AnalyzeC // From the paper "Random sampling for histogram construction: how much is enough?"'s Corollary 1 to Theorem 5, // for a table size n, histogram size k, maximum relative error in bin size f, and error probability gamma, // the minimum random sample size is -// r = 4 * k * ln(2*n/gamma) / f^2 +// +// r = 4 * k * ln(2*n/gamma) / f^2 +// // If we take f = 0.5, gamma = 0.01, n =1e6, we would got r = 305.82* k. // Since the there's log function over the table size n, the r grows slowly when the n increases. // If we take n = 1e12, a 300*k sample still gives <= 0.66 bin size error with probability 0.99. diff --git a/executor/compact_table.go b/executor/compact_table.go index 61a41e5c95b80..fe5c8f2116cec 100644 --- a/executor/compact_table.go +++ b/executor/compact_table.go @@ -213,14 +213,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 3e1d1db069315..84389f9439214 100644 --- a/executor/cte.go +++ b/executor/cte.go @@ -41,22 +41,24 @@ 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 -// other op other op +----------+ -// (seed) (recursive) | resTbl | -// ^ | | -// | +----------+ -// CTETableReaderExec -// ^ -// | read +----------+ -// +---------------+iterInTbl | -// | | -// +----------+ +/* + +----------+ + write |iterOutTbl| + CTEExec ------------------->| | + | +----+-----+ + ------------- | write + | | v + other op other op +----------+ + (seed) (recursive) | resTbl | + ^ | | + | +----------+ + CTETableReaderExec + ^ + | read +----------+ + +---------------+iterInTbl | + | | + +----------+ +*/ type CTEExec struct { baseExecutor diff --git a/executor/executor.go b/executor/executor.go index 29ca46324ae46..678eb3d39c0f2 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1019,6 +1019,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 { @@ -1708,21 +1709,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 a7f4ee6bb11ab..03be724d6573d 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 342aa88f955f3..bc29199a2c2b7 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -58,13 +58,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 4e63855d49101..1646b29580408 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -424,11 +424,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 01d124bbc23cd..1d3359e3aae26 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/plan_replayer.go b/executor/plan_replayer.go index 456ac90fa1fde..9eabcb38922e7 100644 --- a/executor/plan_replayer.go +++ b/executor/plan_replayer.go @@ -113,19 +113,20 @@ func (e *PlanReplayerSingleExec) Next(ctx context.Context, req *chunk.Chunk) err // dumpSingle will dump the information about a single sql. // The files will be organized into the following format: -// |-meta.txt -// |-schema.sql -// |-stats -// | |-stats1.json -// | |-stats2.json -// | |-.... -// |-config.toml -// |-variables.toml -// |-bindings.sql -// |-sqls.sql -// |_explain -// |-explain.txt -// +/* + |-meta.txt + |-schema.sql + |-stats + | |-stats1.json + | |-stats2.json + | |-.... + |-config.toml + |-variables.toml + |-bindings.sql + |-sqls.sql + |_explain + |-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 994128caff421..fc69763898260 100644 --- a/executor/projection.go +++ b/executor/projection.go @@ -148,31 +148,31 @@ func (e *ProjectionExec) open(ctx context.Context) error { // 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 | -// | ^ ^ | -// | | | | -// +------------------------------+ +----------------------+ -// +/* + +-----------+----------------------+--------------------------+ + | | | | + | +--------+---------+ +--------+---------+ +--------+---------+ + | | 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() { @@ -340,13 +340,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 @@ -408,11 +410,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 a667d5f11ca20..b47830839cae8 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 d89c1a71f0a92..efc56aa058d2a 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 532fcfa1f9f7c..36f75e1c1e921 100644 --- a/executor/write.go +++ b/executor/write.go @@ -48,8 +48,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 f02e45cc8bbac..924b96dc4773c 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -1430,8 +1430,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 @@ -1467,8 +1468,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 6ed67939220e0..02aab8ddf0ca5 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -6541,12 +6541,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 55f56702b8209..120987c53f17a 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 2acdf56c52899..a8e9e66998ec5 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -225,8 +225,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 bd6ba4a17ad08..277fa428aa222 100644 --- a/expression/util.go +++ b/expression/util.go @@ -140,9 +140,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. @@ -718,8 +719,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 1683afe782347..f5c0ec10f0fd6 100644 --- a/infoschema/perfschema/const.go +++ b/infoschema/perfschema/const.go @@ -284,7 +284,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 3fdedc090f0ab..c6a2ac321d4c9 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1521,10 +1521,11 @@ var tableVariablesInfoCols = []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 e285aca4c1e7c..484f42951cbb2 100644 --- a/kv/key.go +++ b/kv/key.go @@ -41,10 +41,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 c196166bc9c48..9710cf8065ef3 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 6309816c9f899..c25bc1df960e2 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -664,7 +664,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) { @@ -1064,9 +1066,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/meta/meta.go b/meta/meta.go index 66a6b7909aec4..3a994a6eb08e1 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -359,10 +359,12 @@ func (m *Meta) GetAutoIDAccessors(dbID, tableID int64) AutoIDAccessors { // GetSchemaVersionWithNonEmptyDiff gets current global schema version, if diff is nil, we should return version - 1. // Consider the following scenario: -// t1 t2 t3 t4 -// | | | -// update schema version | set diff -// stale read ts +// +// t1 t2 t3 t4 +// | | | +// update schema version | set diff +// stale read ts +// // At the first time, t2 reads the schema version v10, but the v10's diff is not set yet, so it loads v9 infoSchema. // But at t4 moment, v10's diff has been set and been cached in the memory, so stale read on t2 will get v10 schema from cache, // and inconsistency happen. diff --git a/parser/ast/ddl.go b/parser/ast/ddl.go index ce0c8e5f7a1c7..47ffc53ae65a3 100644 --- a/parser/ast/ddl.go +++ b/parser/ast/ddl.go @@ -670,10 +670,12 @@ const ( ) // IndexOption is the index options. -// KEY_BLOCK_SIZE [=] value -// | index_type -// | WITH PARSER parser_name -// | COMMENT 'string' +// +// KEY_BLOCK_SIZE [=] value +// | index_type +// | WITH PARSER parser_name +// | COMMENT 'string' +// // See http://dev.mysql.com/doc/refman/5.7/en/create-table.html type IndexOption struct { node diff --git a/parser/ast/dml.go b/parser/ast/dml.go index 99a76f2bc4409..525426946b0fd 100644 --- a/parser/ast/dml.go +++ b/parser/ast/dml.go @@ -93,25 +93,31 @@ func (*Join) resultSet() {} // NewCrossJoin builds a cross join without `on` or `using` clause. // If the right child is a join tree, we need to handle it differently to make the precedence get right. // Here is the example: t1 join t2 join t3 -// JOIN ON t2.a = t3.a -// t1 join / \ -// t2 t3 +// +// JOIN ON t2.a = t3.a +// t1 join / \ +// t2 t3 +// // (left) (right) // // We can not build it directly to: -// JOIN -// / \ -// t1 JOIN ON t2.a = t3.a -// / \ -// t2 t3 +// +// JOIN +// / \ +// t1 JOIN ON t2.a = t3.a +// / \ +// t2 t3 +// // The precedence would be t1 join (t2 join t3 on t2.a=t3.a), not (t1 join t2) join t3 on t2.a=t3.a // We need to find the left-most child of the right child, and build a cross join of the left-hand side // of the left child(t1), and the right hand side with the original left-most child of the right child(t2). -// JOIN t2.a = t3.a -// / \ -// JOIN t3 -// / \ -// t1 t2 +// +// JOIN t2.a = t3.a +// / \ +// JOIN t3 +// / \ +// t1 t2 +// // Besides, if the right handle side join tree's join type is right join and has explicit parentheses, we need to rewrite it to left join. // So t1 join t2 right join t3 would be rewrite to t1 join t3 left join t2. // If not, t1 join (t2 right join t3) would be (t1 join t2) right join t3. After rewrite the right join to left join. diff --git a/parser/ast/misc.go b/parser/ast/misc.go index 87b3712f942a3..1c1e1804814c0 100644 --- a/parser/ast/misc.go +++ b/parser/ast/misc.go @@ -1885,9 +1885,10 @@ type StatisticsSpec struct { // CreateStatisticsStmt is a statement to create extended statistics. // Examples: -// CREATE STATISTICS stats1 (cardinality) ON t(a, b, c); -// CREATE STATISTICS stats2 (dependency) ON t(a, b); -// CREATE STATISTICS stats3 (correlation) ON t(a, b); +// +// CREATE STATISTICS stats1 (cardinality) ON t(a, b, c); +// CREATE STATISTICS stats2 (dependency) ON t(a, b); +// CREATE STATISTICS stats3 (correlation) ON t(a, b); type CreateStatisticsStmt struct { stmtNode @@ -1955,7 +1956,8 @@ func (n *CreateStatisticsStmt) Accept(v Visitor) (Node, bool) { // DropStatisticsStmt is a statement to drop extended statistics. // Examples: -// DROP STATISTICS stats1; +// +// DROP STATISTICS stats1; type DropStatisticsStmt struct { stmtNode @@ -2087,6 +2089,7 @@ const ( ) // ShowSlow is used for the following command: +// // admin show slow top [ internal | all] N // admin show slow recent N type ShowSlow struct { diff --git a/parser/auth/mysql_native_password.go b/parser/auth/mysql_native_password.go index 05c6127c21991..2bfc1a8190667 100644 --- a/parser/auth/mysql_native_password.go +++ b/parser/auth/mysql_native_password.go @@ -25,19 +25,20 @@ import ( // CheckScrambledPassword check scrambled password received from client. // The new authentication is performed in following manner: -// SERVER: public_seed=create_random_string() -// send(public_seed) -// CLIENT: recv(public_seed) -// hash_stage1=sha1("password") -// hash_stage2=sha1(hash_stage1) -// reply=xor(hash_stage1, sha1(public_seed,hash_stage2) -// // this three steps are done in scramble() -// send(reply) -// SERVER: recv(reply) -// hash_stage1=xor(reply, sha1(public_seed,hash_stage2)) -// candidate_hash2=sha1(hash_stage1) -// check(candidate_hash2==hash_stage2) -// // this three steps are done in check_scramble() +// +// SERVER: public_seed=create_random_string() +// send(public_seed) +// CLIENT: recv(public_seed) +// hash_stage1=sha1("password") +// hash_stage2=sha1(hash_stage1) +// reply=xor(hash_stage1, sha1(public_seed,hash_stage2) +// // this three steps are done in scramble() +// send(reply) +// SERVER: recv(reply) +// hash_stage1=xor(reply, sha1(public_seed,hash_stage2)) +// candidate_hash2=sha1(hash_stage1) +// check(candidate_hash2==hash_stage2) +// // this three steps are done in check_scramble() func CheckScrambledPassword(salt, hpwd, auth []byte) bool { //nolint: gosec crypt := sha1.New() diff --git a/parser/format/format.go b/parser/format/format.go index 5c9c137c8fa27..adada122e255e 100644 --- a/parser/format/format.go +++ b/parser/format/format.go @@ -56,21 +56,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} } @@ -169,9 +176,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/parser/goyacc/main.go b/parser/goyacc/main.go index 93fc90efb3afe..cc7589773baa1 100644 --- a/parser/goyacc/main.go +++ b/parser/goyacc/main.go @@ -21,7 +21,7 @@ // Goyacc is a version of yacc generating Go parsers. // -// Usage +// # Usage // // Note: If no non flag arguments are given, goyacc reads standard input. // @@ -42,9 +42,7 @@ // -xegen examplesFile Generate a file suitable for -xe automatically from the grammar. // The file must not exist. ("") // -// -// -// Changelog +// # Changelog // // 2015-03-24: The search for a custom error message is now extended to include // also the last state that was shifted into, if any. This change resolves a @@ -70,7 +68,7 @@ // by parsing code fragments. If it returns true the parser exits immediately // with return value -1. // -// Overview +// # Overview // // The generated parser is reentrant and mostly backwards compatible with // parsers generated by go tool yacc[0]. yyParse expects to be given an @@ -104,7 +102,7 @@ // generated code. Setting it to distinct values allows multiple grammars to be // placed in a single package. // -// Differences wrt go tool yacc +// # Differences wrt go tool yacc // // - goyacc implements ideas from "Generating LR Syntax Error Messages from // Examples"[1]. Use the -xe flag to pass a name of the example file. For more @@ -115,14 +113,14 @@ // // - Minor changes in parser debug output. // -// Links +// # Links // // Referenced from elsewhere: // -// [0]: http://golang.org/cmd/yacc/ -// [1]: http://people.via.ecp.fr/~stilgar/doc/compilo/parser/Generating%20LR%20Syntax%20Error%20Messages.pdf -// [2]: http://godoc.org/github.com/cznic/y#hdr-Error_Examples -// [3]: http://www.gnu.org/software/bison/manual/html_node/Precedence-Only.html#Precedence-Only +// [0]: http://golang.org/cmd/yacc/ +// [1]: http://people.via.ecp.fr/~stilgar/doc/compilo/parser/Generating%20LR%20Syntax%20Error%20Messages.pdf +// [2]: http://godoc.org/github.com/cznic/y#hdr-Error_Examples +// [3]: http://www.gnu.org/software/bison/manual/html_node/Precedence-Only.html#Precedence-Only package main import ( diff --git a/parser/model/model.go b/parser/model/model.go index dca176e1f9a72..c4b6452782549 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -1029,8 +1029,9 @@ func (v *ViewCheckOption) String() string { } } -//revive:disable:exported // ViewInfo provides meta data describing a DB view. +// +//revive:disable:exported type ViewInfo struct { Algorithm ViewAlgorithm `json:"view_algorithm"` Definer *auth.UserIdentity `json:"view_definer"` @@ -1260,7 +1261,7 @@ func (i *IndexColumn) Clone() *IndexColumn { } // PrimaryKeyType is the type of primary key. -// Available values are 'clustered', 'nonclustered', and ''(default). +// Available values are 'clustered', 'nonclustered', and ”(default). type PrimaryKeyType int8 func (p PrimaryKeyType) String() string { diff --git a/parser/test_driver/test_driver_mydecimal.go b/parser/test_driver/test_driver_mydecimal.go index 9632cf6db5134..91bd04486689e 100644 --- a/parser/test_driver/test_driver_mydecimal.go +++ b/parser/test_driver/test_driver_mydecimal.go @@ -40,10 +40,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 @@ -102,11 +102,10 @@ func (d *MyDecimal) removeLeadingZeros() (wordIdx int, digitsInt 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) diff --git a/parser/types/etc.go b/parser/types/etc.go index 1fdfeaf05367f..2c07f57f35876 100644 --- a/parser/types/etc.go +++ b/parser/types/etc.go @@ -109,6 +109,7 @@ func TypeStr(tp byte) (r string) { // It is used for converting Text to Blob, // or converting Char to Binary. // Args: +// // tp: type enum // cs: charset func TypeToStr(tp byte, cs string) (r string) { @@ -126,7 +127,8 @@ func TypeToStr(tp byte, cs string) (r string) { // StrToType convert a string to type enum. // Args: -// ts: type string +// +// ts: type string func StrToType(ts string) (tp byte) { ts = strings.Replace(ts, "blob", "text", 1) ts = strings.Replace(ts, "binary", "char", 1) diff --git a/planner/cascades/optimize.go b/planner/cascades/optimize.go index 1934597bab573..29be0272e011e 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 930eac057a845..9f4325c5d28ac 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 (*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 0cf85917c1631..6888a09695a54 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -357,7 +357,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 18d8f6d2bd54a..929c5b210b839 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1327,11 +1327,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, _ []*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 231f5795deac2..784a9dae014a4 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 a6deded3078f2..bf48b3367bdc2 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1290,8 +1290,9 @@ func (ds *DataSource) buildIndexMergeTableScan(_ *property.PhysicalProperty, tab // `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) { @@ -1473,11 +1474,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/flat_plan.go b/planner/core/flat_plan.go index 3cffd1cfc8116..3058f0fd82e92 100644 --- a/planner/core/flat_plan.go +++ b/planner/core/flat_plan.go @@ -54,8 +54,9 @@ type FlatPlanTree []*FlatOperator // GetSelectPlan skips Insert, Delete and Update at the beginning of the FlatPlanTree. // Note: -// It returns a reference to the original FlatPlanTree, please avoid modifying the returned value. -// Since you get a part of the original slice, you need to adjust the FlatOperator.Depth and FlatOperator.ChildrenIdx when using them. +// +// It returns a reference to the original FlatPlanTree, please avoid modifying the returned value. +// Since you get a part of the original slice, you need to adjust the FlatOperator.Depth and FlatOperator.ChildrenIdx when using them. func (e FlatPlanTree) GetSelectPlan() FlatPlanTree { if len(e) == 0 { return nil diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 904470f4f7b61..4dbb7481fd18c 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -829,10 +829,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 { @@ -853,9 +853,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 { @@ -1801,7 +1802,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(_ 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 6cff71160b78f..d3db641a8efd9 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -422,14 +422,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 { @@ -638,7 +639,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 := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index a0abfef2d629d..ad6e3da458dd0 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1494,12 +1494,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/planbuilder.go b/planner/core/planbuilder.go index d15863661a6ae..bf5b7e738f040 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -105,7 +105,7 @@ type limitHintInfo struct { preferLimitToCop bool } -//MergeHintInfo ...one bool flag for cte +// MergeHintInfo ...one bool flag for cte type MergeHintInfo struct { preferMerge bool } diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index f39a1805bac80..c97114315a889 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -683,8 +683,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] { @@ -1658,9 +1660,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 bf3bbf4e81a1e..c9326929b550f 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -322,8 +322,8 @@ func (a *aggregationPushDownSolver) checkAllArgsColumn(fun *aggregation.AggFuncD } // 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_aggregation_skew_rewrite.go b/planner/core/rule_aggregation_skew_rewrite.go index 034347bd14725..f17c7e0bbb94e 100644 --- a/planner/core/rule_aggregation_skew_rewrite.go +++ b/planner/core/rule_aggregation_skew_rewrite.go @@ -28,11 +28,14 @@ type skewDistinctAggRewriter struct { } // skewDistinctAggRewriter will rewrite group distinct aggregate into 2 level aggregates, e.g.: -// select S_NATIONKEY as s, count(S_SUPPKEY), count(distinct S_NAME) from supplier group by s; +// +// select S_NATIONKEY as s, count(S_SUPPKEY), count(distinct S_NAME) from supplier group by s; +// // will be rewritten to -// select S_NATIONKEY as s, sum(c), count(S_NAME) from ( -// select S_NATIONKEY, S_NAME, count(S_SUPPKEY) c from supplier group by S_NATIONKEY, S_NAME -// ) as T group by s; +// +// select S_NATIONKEY as s, sum(c), count(S_NAME) from ( +// select S_NATIONKEY, S_NAME, count(S_SUPPKEY) c from supplier group by S_NATIONKEY, S_NAME +// ) as T group by s; // // If the group key is highly skewed and the distinct key has large number of distinct values // (a.k.a. high cardinality), the query execution will be slow. This rule may help to ease the diff --git a/planner/core/rule_join_elimination.go b/planner/core/rule_join_elimination.go index 3dc14300c7e74..a5eb238ec1bfa 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 041ca7eb7079d..dc13c59633146 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, _ []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 6b48f56020374..2274c9f8cf496 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 0af409cc52246..bd55c2a36a6b5 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -882,8 +882,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, @@ -896,7 +898,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 761736d0100e9..a182501cc05e1 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 67bd81b556d78..132af680b8be0 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -990,7 +990,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 f06e701902de8..6950255e32591 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -2088,9 +2088,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 dba8b622752db..0e76fba2126db 100644 --- a/session/bench_test.go +++ b/session/bench_test.go @@ -1832,7 +1832,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/session.go b/sessionctx/variable/session.go index 87d35064e6db7..9bd60476f8e3a 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -2903,9 +2903,10 @@ func (s *SessionVars) GetStrMatchDefaultSelectivity() float64 { // GetNegateStrMatchDefaultSelectivity means the default selectivity for not like and not regexp. // Note: -// 0 is a special value, which means the default selectivity is 0.9 and TopN assisted estimation is enabled. -// 0.8 (the default value) is also a special value. For backward compatibility, when the variable is set to 0.8, we -// keep the default selectivity of like/regexp and not like/regexp all 0.8. +// +// 0 is a special value, which means the default selectivity is 0.9 and TopN assisted estimation is enabled. +// 0.8 (the default value) is also a special value. For backward compatibility, when the variable is set to 0.8, we +// keep the default selectivity of like/regexp and not like/regexp all 0.8. func (s *SessionVars) GetNegateStrMatchDefaultSelectivity() float64 { if s.DefaultStrMatchSelectivity == DefTiDBDefaultStrMatchSelectivity { return DefTiDBDefaultStrMatchSelectivity diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index cbc4a95b0771e..80594f7d4e8a4 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -50,7 +50,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/sessiontxn/isolation/base.go b/sessiontxn/isolation/base.go index 7bded0b3d7baf..4a90f8f14c65c 100644 --- a/sessiontxn/isolation/base.go +++ b/sessiontxn/isolation/base.go @@ -40,6 +40,7 @@ import ( // - Provides some methods like `activateTxn` and `prepareTxn` to manage the inner transaction. // - Provides default methods `GetTxnInfoSchema`, `GetStmtReadTS` and `GetStmtForUpdateTS` and return the snapshot information schema or ts when `tidb_snapshot` is set. // - Provides other default methods like `Advise`, `OnStmtStart`, `OnStmtRetry` and `OnStmtErrorForNextAction` +// // The subclass can set some inner property of `baseTxnContextProvider` when it is constructed. // For example, `getStmtReadTSFunc` and `getStmtForUpdateTSFunc` should be set, and they will be called when `GetStmtReadTS` // or `GetStmtForUpdate` to get the timestamp that should be used by the corresponding isolation level. @@ -161,7 +162,7 @@ func (p *baseTxnContextProvider) GetReadReplicaScope() string { return kv.GlobalReplicaScope } -//GetStmtReadTS returns the read timestamp used by select statement (not for select ... for update) +// GetStmtReadTS returns the read timestamp used by select statement (not for select ... for update) func (p *baseTxnContextProvider) GetStmtReadTS() (uint64, error) { if _, err := p.ActivateTxn(); err != nil { return 0, err diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index 58fbc3d4cc065..18fa2ebd8608c 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -168,7 +168,9 @@ func (p *PessimisticRRTxnContextProvider) OnStmtErrorForNextAction(point session // AdviseOptimizeWithPlan optimizes for update point get related execution. // Use case: In for update point get related operations, we do not fetch ts from PD but use the last ts we fetched. -// We expect that the data that the point get acquires has not been changed. +// +// We expect that the data that the point get acquires has not been changed. +// // Benefit: Save the cost of acquiring ts from PD. // Drawbacks: If the data has been changed since the ts we used, we need to retry. // One exception is insert operation, when it has no select plan, we do not fetch the latest ts immediately. We only update ts diff --git a/sessiontxn/staleread/main_test.go b/sessiontxn/staleread/main_test.go index a60b5a95c9007..7ed95c5699c5c 100644 --- a/sessiontxn/staleread/main_test.go +++ b/sessiontxn/staleread/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/sessiontxn/staleread/provider_test.go b/sessiontxn/staleread/provider_test.go index e83343cf91dc0..6e8b6d8ec445e 100644 --- a/sessiontxn/staleread/provider_test.go +++ b/sessiontxn/staleread/provider_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/statistics/builder.go b/statistics/builder.go index f8ada420edc0a..c9ed700d7cbc0 100644 --- a/statistics/builder.go +++ b/statistics/builder.go @@ -405,7 +405,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 44ce8d224e7bf..848a10a653325 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -329,10 +329,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 { @@ -511,7 +513,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 @@ -718,13 +721,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 dfad8d1de9259..c2137802d0257 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -2239,7 +2239,7 @@ func TestIndexUsageInformation(t *testing.T) { )) } -//Functional Test:test batch insert +// Functional Test:test batch insert func TestIndexUsageInformationMultiIndex(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) session.SetIndexUsageSyncLease(1) diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 7f8f35a64e4df..89b2342f9d767 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -973,11 +973,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 b6bbc0a38a2ab..78db8f1b7259e 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -386,19 +386,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 { @@ -778,19 +781,20 @@ func (hg *Histogram) outOfRange(val types.Datum) bool { // The maximum row count it can get is the increaseCount. It reaches the maximum when out-of-range width reaches histogram range width. // 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│ │ │ \ -//────┴────┴───┴──┴─────────────┴───────────┴───── -// ▲ ▲ ▲ ▲ ▲ ▲ -// │ │ │ │ │ │ -// boundL │ │histL histR boundR -// │ │ -// lDatum rDatum +/* + /│ │\ + / │ │ \ + /x│ │◄─histogram─►│ \ + / xx│ │ range │ \ + / │xxx│ │ │ \ + / │xxx│ │ │ \ +────┴────┴───┴──┴─────────────┴───────────┴───── + ▲ ▲ ▲ ▲ ▲ ▲ + │ │ │ │ │ │ + boundL │ │histL histR boundR + │ │ + lDatum rDatum +*/ func (hg *Histogram) outOfRangeRowCount(lDatum, rDatum *types.Datum, increaseCount int64) float64 { if hg.Len() == 0 { return 0 @@ -945,7 +949,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 @@ -1298,10 +1303,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 c80af0b980c79..ae84aaa827a05 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 380887acf193f..ba847473fde85 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -635,10 +635,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 7cf36d958ab59..a4eec52f330c6 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 988835cbe1d8b..a76184d3a7a80 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 6a04b59d0d3f9..016e56ccea3ad 100644 --- a/table/tables/mutation_checker.go +++ b/table/tables/mutation_checker.go @@ -175,7 +175,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 89d3c02e4b559..62443b1e21b86 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -287,16 +287,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 0b06a15cd22b4..a91d497fdf831 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 389c8f51cb54b..da9f936cecd70 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 59509059a30bb..3a0cd42ed1c3e 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 1e28893318077..5a702d475c765 100644 --- a/types/datum.go +++ b/types/datum.go @@ -2365,14 +2365,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 3d374c81fcecc..4818828af09d0 100644 --- a/types/json/binary_functions.go +++ b/types/json/binary_functions.go @@ -208,8 +208,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 { @@ -1109,7 +1110,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 81a7714288410..d222eb204d9ef 100644 --- a/types/mydecimal.go +++ b/types/mydecimal.go @@ -137,7 +137,7 @@ func add(a, b, carry int32) (sum int32, newCarry int32) { // add2 adds a and b and carry, returns the sum and new carry. // It is only used in DecimalMul. -//nolint: revive +// nolint: revive func add2(a, b, carry int32) (int32, int32) { sum := int64(a) + int64(b) + int64(carry) if sum >= wordBase { @@ -195,10 +195,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 @@ -210,10 +210,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 @@ -312,11 +312,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) @@ -540,10 +539,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 { @@ -703,10 +702,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 @@ -752,14 +751,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 @@ -776,14 +776,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 @@ -801,18 +802,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 @@ -1137,78 +1140,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{}) @@ -1933,21 +1936,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) @@ -2087,26 +2090,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 c3608ee219a1c..3d997d940ddf4 100644 --- a/types/time.go +++ b/types/time.go @@ -613,18 +613,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() { @@ -834,12 +833,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 @@ -1753,9 +1753,9 @@ func matchDuration(str string, fsp int) (Duration, bool, 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 e302f56d31599..1dd2b4cff02e4 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 01a90dd9dc007..beb4996a681f5 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 f5ccc6da5138c..4e7f2f1e8a86d 100644 --- a/util/chunk/disk_test.go +++ b/util/chunk/disk_test.go @@ -358,15 +358,16 @@ func TestListInDiskWithChecksumAndEncryptReaderWithCacheNoFlush(t *testing.T) { // 4 B: checksum of this segment. // 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 -// +------+------------------------------------------+ +-----------------------------+ -// | | 1020B payload | | | -// |4Bytes| +---------+----------------------------+ | | | -// |checksum|8B collen| 1012B user data | | | 12B remained user data | -// | | +---------+----------------------------+ | | | -// | | | | | -// +------+------------------------------------------+ +-----------------------------+ +/* + Data in File Data in mem cache + +------+------------------------------------------+ +-----------------------------+ + | | 1020B payload | | | + |4Bytes| +---------+----------------------------+ | | | + |checksum|8B collen| 1012B user data | | | 12B remained user data | + | | +---------+----------------------------+ | | | + | | | | | + +------+------------------------------------------+ +-----------------------------+ +*/ func testReaderWithCache(t *testing.T) { testData := "0123456789" buf := bytes.NewBuffer(nil) diff --git a/util/chunk/iterator.go b/util/chunk/iterator.go index 7928f748fc4b4..2adebc5822c9f 100644 --- a/util/chunk/iterator.go +++ b/util/chunk/iterator.go @@ -43,9 +43,9 @@ func FreeIterator(it any) { // 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 bc6548c04b2ab..a5a2f3f45e3e4 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 d905809bca196..10a718975a7bd 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -1319,7 +1319,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 8b29bb41d7c37..89f8557ee09f7 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 db54f2c05c74a..2cbcd44d232a8 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -1028,12 +1028,12 @@ func (*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 8754dfbc353cb..85e65f0ff0e9a 100644 --- a/util/memory/tracker.go +++ b/util/memory/tracker.go @@ -103,8 +103,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 @@ -122,8 +123,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 4a076cca04ddd..0e28baa5f62fc 100644 --- a/util/misc.go +++ b/util/misc.go @@ -78,8 +78,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() @@ -97,10 +98,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) { //nolint: revive r := recover() diff --git a/util/mock/iter_test.go b/util/mock/iter_test.go index e99e9d17745cb..88c531f2119be 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 98d8733b8c111..8ce6aabde2e01 100644 --- a/util/processinfo.go +++ b/util/processinfo.go @@ -185,17 +185,19 @@ 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 -// +--+---------------------+--------------------------------------+------+ -// | | serverId | local connId |markup| -// |=0| (22b) | (40b) | =1 | -// +--+---------------------+--------------------------------------+------+ -// 32 bits version(coming soon): -// 31 1 0 -// +-----------------------------+------+ -// | ??? |markup| -// | ??? | =0 | -// +-----------------------------+------+ +/* + 63 62 41 40 1 0 + +--+---------------------+--------------------------------------+------+ + | | serverId | local connId |markup| + |=0| (22b) | (40b) | =1 | + +--+---------------------+--------------------------------------+------+ + 32 bits version(coming soon): + 31 1 0 + +-----------------------------+------+ + | ??? |markup| + | ??? | =0 | + +-----------------------------+------+ +*/ type GlobalConnID struct { ServerID uint64 LocalConnID uint64 @@ -251,7 +253,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 3d18126494f70..65973769eb610 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, @@ -911,7 +918,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) { @@ -977,8 +985,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, @@ -1081,12 +1090,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, @@ -1169,8 +1182,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 4481f739b6760..b9e83ea9a0cf9 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -492,14 +492,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 7b1f773ce3d46..7c9e9399c622a 100644 --- a/util/sqlexec/utils.go +++ b/util/sqlexec/utils.go @@ -233,7 +233,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 9dbdb33fed27d..a0f4c8590ab2c 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 2c49704241255..54020cc3084a2 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 de3cbb3206672..e3000370377c5 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 From 5873ecc004aae29bcf932f1ca8677d1371efbddf Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 3 Aug 2022 17:38:12 +0800 Subject: [PATCH 02/18] Update br/pkg/task/stream.go Co-authored-by: bb7133 --- br/pkg/task/stream.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index 7b965cf2a64e4..e9610e0c7c73a 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -425,7 +425,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, From ecd7ec02e2b07c547178f54a852f6aeb3360f643 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 3 Aug 2022 17:38:32 +0800 Subject: [PATCH 03/18] Update meta/autoid/autoid.go Co-authored-by: bb7133 --- meta/autoid/autoid.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index c25bc1df960e2..8c7930ce2f658 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -1066,7 +1066,7 @@ 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) | From 3140fa449b5b7ccf35b4d8c4b0313251fba329eb Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 4 Aug 2022 12:10:04 +0800 Subject: [PATCH 04/18] *: set go 1.19 Signed-off-by: Weizhen Wang --- table/tables/mutation_checker.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/table/tables/mutation_checker.go b/table/tables/mutation_checker.go index 016e56ccea3ad..15ae31a4bf240 100644 --- a/table/tables/mutation_checker.go +++ b/table/tables/mutation_checker.go @@ -174,9 +174,7 @@ 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. +// (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. func checkIndexKeys( sessVars *variable.SessionVars, t *TableCommon, rowToInsert, rowToRemove []types.Datum, indexMutations []mutation, indexIDToInfo map[int64]*model.IndexInfo, From ec7a006e96202ce7311a525edac422220957a604 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 4 Aug 2022 12:13:52 +0800 Subject: [PATCH 05/18] *: set go 1.19 Signed-off-by: Weizhen Wang --- planner/funcdep/fd_graph.go | 204 +++++++++++++++++++----------------- 1 file changed, 108 insertions(+), 96 deletions(-) diff --git a/planner/funcdep/fd_graph.go b/planner/funcdep/fd_graph.go index 4e85907240ae8..b4f622c62fd96 100644 --- a/planner/funcdep/fd_graph.go +++ b/planner/funcdep/fd_graph.go @@ -603,119 +603,131 @@ 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). -// -// <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. +// <1> strict/lax FD: join key filter conditions can not produce new strict/lax FD yet (knowledge: 1&2). // -// 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. +// <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. // -// 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() @@ -925,9 +937,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++ { From dca5261fb140bf77bdb9fa0fe35eec10b2ae00b5 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 4 Aug 2022 13:16:22 +0800 Subject: [PATCH 06/18] *: set go 1.19 Signed-off-by: Weizhen Wang --- planner/core/rule_predicate_push_down.go | 5 +---- planner/funcdep/fd_graph.go | 6 +++--- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index bd55c2a36a6b5..e4f7df6efd178 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -897,10 +897,7 @@ func (adder *exprPrefixAdder) addExprPrefix4CNFCond(conds []expression.Expressio // AddExprPrefix4DNFCond // 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` -// +// @param[in] condition the original condtion of the datasoure. e.g. `WHERE 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/funcdep/fd_graph.go b/planner/funcdep/fd_graph.go index b4f622c62fd96..861b02f79ee72 100644 --- a/planner/funcdep/fd_graph.go +++ b/planner/funcdep/fd_graph.go @@ -937,9 +937,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++ { From e30c312df3aac99255c3dd48c6a7161fb0fcfc6c Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 4 Aug 2022 13:43:44 +0800 Subject: [PATCH 07/18] *: set go 1.19 Signed-off-by: Weizhen Wang --- planner/core/rule_partition_processor.go | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 2274c9f8cf496..b24e64e7e402e 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -45,20 +45,18 @@ const FullRange = -1 // partitionProcessor rewrites the ast for table partition. // 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{} From 6d0f64793d2828a3b5e949e12eee99923bcd3d25 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Fri, 5 Aug 2022 00:40:04 +0800 Subject: [PATCH 08/18] update --- meta/meta.go | 51 +++++++++++++++------------------------------------ 1 file changed, 15 insertions(+), 36 deletions(-) diff --git a/meta/meta.go b/meta/meta.go index 3a994a6eb08e1..66b25cbe324e4 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -359,12 +359,12 @@ func (m *Meta) GetAutoIDAccessors(dbID, tableID int64) AutoIDAccessors { // GetSchemaVersionWithNonEmptyDiff gets current global schema version, if diff is nil, we should return version - 1. // Consider the following scenario: -// -// t1 t2 t3 t4 -// | | | -// update schema version | set diff -// stale read ts -// +/* +// t1 t2 t3 t4 +// | | | +// update schema version | set diff +// stale read ts +*/ // At the first time, t2 reads the schema version v10, but the v10's diff is not set yet, so it loads v9 infoSchema. // But at t4 moment, v10's diff has been set and been cached in the memory, so stale read on t2 will get v10 schema from cache, // and inconsistency happen. @@ -546,12 +546,17 @@ func (m *Meta) SetDDLTables() error { // CreateMySQLDatabaseIfNotExists creates mysql schema and return its DB ID. func (m *Meta) CreateMySQLDatabaseIfNotExists() (int64, error) { - id, err := m.GetSystemDBID() - if id != 0 || err != nil { - return id, err + dbs, err := m.ListDatabases() + if err != nil { + return 0, err + } + for _, db := range dbs { + if db.Name.L == mysql.SystemDB { + return db.ID, nil + } } - id, err = m.GenGlobalID() + id, err := m.GenGlobalID() if err != nil { return 0, errors.Trace(err) } @@ -566,20 +571,6 @@ func (m *Meta) CreateMySQLDatabaseIfNotExists() (int64, error) { return db.ID, err } -// GetSystemDBID gets the system DB ID. return (0, nil) indicates that the system DB does not exist. -func (m *Meta) GetSystemDBID() (int64, error) { - dbs, err := m.ListDatabases() - if err != nil { - return 0, err - } - for _, db := range dbs { - if db.Name.L == mysql.SystemDB { - return db.ID, nil - } - } - return 0, nil -} - // CheckDDLTableExists check if the tables related to concurrent DDL exists. func (m *Meta) CheckDDLTableExists() (bool, error) { v, err := m.txn.Get(mDDLTableVersion) @@ -1167,18 +1158,6 @@ func (m *Meta) GetLastHistoryDDLJobsIterator() (LastJobIterator, error) { }, nil } -// GetHistoryDDLJobsIterator gets the jobs iterator begin with startJobID. -func (m *Meta) GetHistoryDDLJobsIterator(startJobID int64) (LastJobIterator, error) { - field := m.jobIDKey(startJobID) - iter, err := structure.NewHashReverseIterBeginWithField(m.txn, mDDLJobHistoryKey, field) - if err != nil { - return nil, err - } - return &HLastJobIterator{ - iter: iter, - }, nil -} - // HLastJobIterator is the iterator for gets the latest history. type HLastJobIterator struct { iter *structure.ReverseHashIterator From e800a4bd814c6085bb891797fb807032983b56f2 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Fri, 5 Aug 2022 00:44:14 +0800 Subject: [PATCH 09/18] update --- planner/funcdep/fd_graph.go | 48 ++++++++++++++++++------------------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/planner/funcdep/fd_graph.go b/planner/funcdep/fd_graph.go index 861b02f79ee72..cbcd13b2897d3 100644 --- a/planner/funcdep/fd_graph.go +++ b/planner/funcdep/fd_graph.go @@ -635,10 +635,10 @@ func (s *FDSet) MakeCartesianProduct(rhs *FDSet) { // // <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 +// 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. @@ -657,11 +657,11 @@ func (s *FDSet) MakeCartesianProduct(rhs *FDSet) { // 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 +// 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 @@ -675,11 +675,11 @@ func (s *FDSet) MakeCartesianProduct(rhs *FDSet) { // 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 +// 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 @@ -691,11 +691,11 @@ func (s *FDSet) MakeCartesianProduct(rhs *FDSet) { // <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 +// 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. @@ -711,11 +711,11 @@ func (s *FDSet) MakeCartesianProduct(rhs *FDSet) { // <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 +// 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} From 7ceebad8fb938a624e606866c1d864d3b729ee46 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Fri, 5 Aug 2022 09:34:34 +0800 Subject: [PATCH 10/18] *: upgrade golangci-linter Signed-off-by: Weizhen Wang --- meta/meta.go | 39 ++++++++++++++++++++++++++++++--------- 1 file changed, 30 insertions(+), 9 deletions(-) diff --git a/meta/meta.go b/meta/meta.go index 66b25cbe324e4..807f63bf6c6ff 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -546,17 +546,12 @@ func (m *Meta) SetDDLTables() error { // CreateMySQLDatabaseIfNotExists creates mysql schema and return its DB ID. func (m *Meta) CreateMySQLDatabaseIfNotExists() (int64, error) { - dbs, err := m.ListDatabases() - if err != nil { - return 0, err - } - for _, db := range dbs { - if db.Name.L == mysql.SystemDB { - return db.ID, nil - } + id, err := m.GetSystemDBID() + if id != 0 || err != nil { + return id, err } - id, err := m.GenGlobalID() + id, err = m.GenGlobalID() if err != nil { return 0, errors.Trace(err) } @@ -571,6 +566,20 @@ func (m *Meta) CreateMySQLDatabaseIfNotExists() (int64, error) { return db.ID, err } +// GetSystemDBID gets the system DB ID. return (0, nil) indicates that the system DB does not exist. +func (m *Meta) GetSystemDBID() (int64, error) { + dbs, err := m.ListDatabases() + if err != nil { + return 0, err + } + for _, db := range dbs { + if db.Name.L == mysql.SystemDB { + return db.ID, nil + } + } + return 0, nil +} + // CheckDDLTableExists check if the tables related to concurrent DDL exists. func (m *Meta) CheckDDLTableExists() (bool, error) { v, err := m.txn.Get(mDDLTableVersion) @@ -1158,6 +1167,18 @@ func (m *Meta) GetLastHistoryDDLJobsIterator() (LastJobIterator, error) { }, nil } +// GetHistoryDDLJobsIterator gets the jobs iterator begin with startJobID. +func (m *Meta) GetHistoryDDLJobsIterator(startJobID int64) (LastJobIterator, error) { + field := m.jobIDKey(startJobID) + iter, err := structure.NewHashReverseIterBeginWithField(m.txn, mDDLJobHistoryKey, field) + if err != nil { + return nil, err + } + return &HLastJobIterator{ + iter: iter, + }, nil +} + // HLastJobIterator is the iterator for gets the latest history. type HLastJobIterator struct { iter *structure.ReverseHashIterator From e168ac77dd61b2c2322ae7da9b6d200db55cc988 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Sat, 6 Aug 2022 17:14:59 +0800 Subject: [PATCH 11/18] *: upgrade go1.19 Signed-off-by: Weizhen Wang --- go.mod | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 387d7840fe293..838c635f4d63c 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,6 @@ module github.com/pingcap/tidb -go 1.18 +go 1.19 require ( cloud.google.com/go/storage v1.21.0 From 4bf56879c9835d8346c18245ef6c3f63885cfc31 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Mon, 8 Aug 2022 10:54:28 +0800 Subject: [PATCH 12/18] build tidb upgrade go119 Signed-off-by: Weizhen Wang --- WORKSPACE | 2 +- go.mod | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/WORKSPACE b/WORKSPACE index f95edb1a9fd43..744ef2aed9801 100644 --- a/WORKSPACE +++ b/WORKSPACE @@ -33,7 +33,7 @@ go_rules_dependencies() go_register_toolchains( nogo = "@//build:tidb_nogo", - version = "1.19", + version = "1.18.5", ) gazelle_dependencies() diff --git a/go.mod b/go.mod index 838c635f4d63c..387d7840fe293 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,6 @@ module github.com/pingcap/tidb -go 1.19 +go 1.18 require ( cloud.google.com/go/storage v1.21.0 From 33b5aae76b6af649743c29caaefeca1292d89442 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Mon, 8 Aug 2022 19:02:45 +0800 Subject: [PATCH 13/18] recovery Signed-off-by: Weizhen Wang --- util/codec/codec.go | 3 +-- util/sqlexec/utils.go | 2 +- util/stringutil/string_util.go | 2 +- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/util/codec/codec.go b/util/codec/codec.go index 10a718975a7bd..d905809bca196 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -1319,8 +1319,7 @@ 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/sqlexec/utils.go b/util/sqlexec/utils.go index 7c9e9399c622a..7b1f773ce3d46 100644 --- a/util/sqlexec/utils.go +++ b/util/sqlexec/utils.go @@ -233,7 +233,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 a0f4c8590ab2c..9dbdb33fed27d 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 { From 705a9c97a012d7d81bdb8bfc32adfd198379bd8c Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Mon, 8 Aug 2022 19:08:55 +0800 Subject: [PATCH 14/18] fix Signed-off-by: Weizhen Wang --- util/codec/codec.go | 3 +-- util/sqlexec/utils.go | 5 ++++- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/util/codec/codec.go b/util/codec/codec.go index d905809bca196..307e511e183a9 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -1318,8 +1318,7 @@ 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. +// It is mostly the same as EncodeValue, but it doesn't contain truncation or verification logic in order to make the encoding lossless. func HashCode(b []byte, d types.Datum) []byte { switch d.Kind() { case types.KindInt64: diff --git a/util/sqlexec/utils.go b/util/sqlexec/utils.go index 7b1f773ce3d46..d61e5375378e1 100644 --- a/util/sqlexec/utils.go +++ b/util/sqlexec/utils.go @@ -233,7 +233,10 @@ 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...) From 4ef3b8712fe2f52d5b2c4936b17e0609eff82cce Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Mon, 8 Aug 2022 19:10:54 +0800 Subject: [PATCH 15/18] fix Signed-off-by: Weizhen Wang --- util/stringutil/string_util.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/util/stringutil/string_util.go b/util/stringutil/string_util.go index 9dbdb33fed27d..818f27436b079 100644 --- a/util/stringutil/string_util.go +++ b/util/stringutil/string_util.go @@ -323,7 +323,10 @@ 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 { From 219b5c41cad8b5b26e26fcbee157d1205b6e63c1 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Tue, 9 Aug 2022 11:29:55 +0800 Subject: [PATCH 16/18] fix Signed-off-by: Weizhen Wang --- parser/model/model.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parser/model/model.go b/parser/model/model.go index c4b6452782549..82c2d97cfeb5d 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -1261,7 +1261,7 @@ func (i *IndexColumn) Clone() *IndexColumn { } // PrimaryKeyType is the type of primary key. -// Available values are 'clustered', 'nonclustered', and ”(default). +// Available values are "clustered", "nonclustered", and ""(default). type PrimaryKeyType int8 func (p PrimaryKeyType) String() string { From c8e04e8174ca80783ef2c1a9b9b1afb9effb3591 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 10 Aug 2022 15:18:08 +0800 Subject: [PATCH 17/18] fix Signed-off-by: Weizhen Wang --- planner/funcdep/fd_graph.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/planner/funcdep/fd_graph.go b/planner/funcdep/fd_graph.go index cbcd13b2897d3..2d6bfb540dc06 100644 --- a/planner/funcdep/fd_graph.go +++ b/planner/funcdep/fd_graph.go @@ -635,10 +635,10 @@ func (s *FDSet) MakeCartesianProduct(rhs *FDSet) { // // <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 + 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. From c575f16dd4a5729ef8fa116d57252fdc085503a3 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 10 Aug 2022 15:18:51 +0800 Subject: [PATCH 18/18] fix Signed-off-by: Weizhen Wang --- ddl/partition.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/ddl/partition.go b/ddl/partition.go index a8d0b53ea363a..6463c9c243b40 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -890,10 +890,11 @@ func astIntValueExprFromStr(s string, unsigned bool) (ast.ExprNode, error) { // GeneratePartDefsFromInterval generates range partitions from INTERVAL partitioning. // Handles -// - CREATE TABLE: all partitions are generated -// - ALTER TABLE FIRST PARTITION (expr): Drops all partitions before the partition matching the expr (i.e. sets that partition as the new first partition) -// i.e. will return the partitions from old FIRST partition to (and including) new FIRST partition -// - ALTER TABLE LAST PARTITION (expr): Creates new partitions from (excluding) old LAST partition to (including) new LAST partition +// - CREATE TABLE: all partitions are generated +// - ALTER TABLE FIRST PARTITION (expr): Drops all partitions before the partition matching the expr (i.e. sets that partition as the new first partition) +// i.e. will return the partitions from old FIRST partition to (and including) new FIRST partition +// - ALTER TABLE LAST PARTITION (expr): Creates new partitions from (excluding) old LAST partition to (including) new LAST partition +// // partition definitions will be set on partitionOptions func GeneratePartDefsFromInterval(ctx sessionctx.Context, tp ast.AlterTableType, tbInfo *model.TableInfo, partitionOptions *ast.PartitionOptions) error { if partitionOptions == nil {