diff --git a/Makefile b/Makefile index d2bbdd11bb5e8..3890dce48fe98 100644 --- a/Makefile +++ b/Makefile @@ -159,23 +159,29 @@ else CGO_ENABLED=1 $(GOBUILD) -gcflags="all=-N -l" $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o '$(TARGET)' ./tidb-server endif +init-submodule: + git submodule init && git submodule update --force + enterprise-prepare: - git submodule init && git submodule update && cd extension/enterprise/generate && $(GO) generate -run genfile main.go + cd extension/enterprise/generate && $(GO) generate -run genfile main.go enterprise-clear: cd extension/enterprise/generate && $(GO) generate -run clear main.go -enterprise-docker: enterprise-prepare +enterprise-docker: init-submodule enterprise-prepare docker build -t "$(DOCKERPREFIX)tidb:latest" --build-arg 'GOPROXY=$(shell go env GOPROXY),' -f Dockerfile.enterprise . enterprise-server-build: ifeq ($(TARGET), "") - CGO_ENABLED=1 $(GOBUILD) -tags enterprise $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o bin/tidb-server tidb-server/main.go + CGO_ENABLED=1 $(GOBUILD) -tags enterprise $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG) $(EXTENSION_FLAG)' -o bin/tidb-server tidb-server/main.go else - CGO_ENABLED=1 $(GOBUILD) -tags enterprise $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o '$(TARGET)' tidb-server/main.go + CGO_ENABLED=1 $(GOBUILD) -tags enterprise $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG) $(EXTENSION_FLAG)' -o '$(TARGET)' tidb-server/main.go endif -enterprise-server: enterprise-prepare enterprise-server-build +enterprise-server: + @make init-submodule + @make enterprise-prepare + @make enterprise-server-build server_check: ifeq ($(TARGET), "") diff --git a/Makefile.common b/Makefile.common index f2ba132767617..66933c93fc256 100644 --- a/Makefile.common +++ b/Makefile.common @@ -66,6 +66,11 @@ LDFLAGS += -X "github.com/pingcap/tidb/util/versioninfo.TiDBGitHash=$(shell git LDFLAGS += -X "github.com/pingcap/tidb/util/versioninfo.TiDBGitBranch=$(shell git rev-parse --abbrev-ref HEAD)" LDFLAGS += -X "github.com/pingcap/tidb/util/versioninfo.TiDBEdition=$(TIDB_EDITION)" +EXTENSION_FLAG = +ifeq ($(shell if [ -a extension/enterprise/.git ]; then echo "true"; fi),true) + EXTENSION_FLAG += -X "github.com/pingcap/tidb/util/versioninfo.TiDBEnterpriseExtensionGitHash=$(shell cd extension/enterprise && git rev-parse HEAD)" +endif + TEST_LDFLAGS = -X "github.com/pingcap/tidb/config.checkBeforeDropLDFlag=1" COVERAGE_SERVER_LDFLAGS = -X "github.com/pingcap/tidb/tidb-server.isCoverageServer=1" diff --git a/br/pkg/lightning/mydump/BUILD.bazel b/br/pkg/lightning/mydump/BUILD.bazel index 529796e103c82..238ff5d4c7ef8 100644 --- a/br/pkg/lightning/mydump/BUILD.bazel +++ b/br/pkg/lightning/mydump/BUILD.bazel @@ -37,6 +37,7 @@ go_library( "@com_github_xitongsys_parquet_go//reader", "@com_github_xitongsys_parquet_go//source", "@org_golang_x_exp//slices", + "@org_golang_x_sync//errgroup", "@org_golang_x_text//encoding", "@org_golang_x_text//encoding/charmap", "@org_golang_x_text//encoding/simplifiedchinese", diff --git a/br/pkg/lightning/mydump/region.go b/br/pkg/lightning/mydump/region.go index 10d0f03689489..2f957f593302c 100644 --- a/br/pkg/lightning/mydump/region.go +++ b/br/pkg/lightning/mydump/region.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/util/mathutil" "go.uber.org/zap" + "golang.org/x/sync/errgroup" ) const ( @@ -217,84 +218,53 @@ func MakeTableRegions( start := time.Now() - execCtx, cancel := context.WithCancel(ctx) - defer cancel() - concurrency := mathutil.Max(cfg.Concurrency, 2) - fileChan := make(chan FileInfo, concurrency) - resultChan := make(chan fileRegionRes, concurrency) - var wg sync.WaitGroup - for i := 0; i < concurrency; i++ { - wg.Add(1) - go func() { - defer wg.Done() - for info := range fileChan { - var ( - regions []*TableRegion - sizes []float64 - err error - ) - dataFileSize := info.FileMeta.FileSize - if info.FileMeta.Type == SourceTypeParquet { - regions, sizes, err = makeParquetFileRegion(ctx, cfg, info) - } else if info.FileMeta.Type == SourceTypeCSV && cfg.StrictFormat && - info.FileMeta.Compression == CompressionNone && - dataFileSize > cfg.MaxChunkSize+cfg.MaxChunkSize/largeCSVLowerThresholdRation { - // If a csv file is overlarge, we need to split it into multiple regions. - // Note: We can only split a csv file whose format is strict. - // We increase the check threshold by 1/10 of the `max-region-size` because the source file size dumped by tools - // like dumpling might be slight exceed the threshold when it is equal `max-region-size`, so we can - // avoid split a lot of small chunks. - // If a csv file is compressed, we can't split it now because we can't get the exact size of a row. - regions, sizes, err = SplitLargeCSV(ctx, cfg, info) - } else { - regions, sizes, err = MakeSourceFileRegion(execCtx, cfg, info) - } - select { - case resultChan <- fileRegionRes{info: info, regions: regions, sizes: sizes, err: err}: - case <-ctx.Done(): - return - } - if err != nil { - log.FromContext(ctx).Error("make source file region error", zap.Error(err), zap.String("file_path", info.FileMeta.Path)) - break - } - } - }() - } + var fileRegionsMap sync.Map - go func() { - wg.Wait() - close(resultChan) - }() - - errChan := make(chan error, 1) + eg, egCtx := errgroup.WithContext(ctx) + eg.SetLimit(concurrency) meta := cfg.TableMeta - fileRegionsMap := make(map[string]fileRegionRes, len(meta.DataFiles)) - go func() { - for res := range resultChan { - if res.err != nil { - errChan <- res.err - return + for _, info := range meta.DataFiles { + info := info + eg.Go(func() error { + select { + case <-egCtx.Done(): + return nil + default: } - fileRegionsMap[res.info.FileMeta.Path] = res - } - errChan <- nil - }() - for _, dataFile := range meta.DataFiles { - select { - case fileChan <- dataFile: - case <-ctx.Done(): - close(fileChan) - return nil, ctx.Err() - case err := <-errChan: - return nil, err - } + var ( + regions []*TableRegion + sizes []float64 + err error + ) + dataFileSize := info.FileMeta.FileSize + if info.FileMeta.Type == SourceTypeParquet { + regions, sizes, err = makeParquetFileRegion(egCtx, cfg, info) + } else if info.FileMeta.Type == SourceTypeCSV && cfg.StrictFormat && + info.FileMeta.Compression == CompressionNone && + dataFileSize > cfg.MaxChunkSize+cfg.MaxChunkSize/largeCSVLowerThresholdRation { + // If a csv file is overlarge, we need to split it into multiple regions. + // Note: We can only split a csv file whose format is strict. + // We increase the check threshold by 1/10 of the `max-region-size` because the source file size dumped by tools + // like dumpling might be slight exceed the threshold when it is equal `max-region-size`, so we can + // avoid split a lot of small chunks. + // If a csv file is compressed, we can't split it now because we can't get the exact size of a row. + regions, sizes, err = SplitLargeCSV(egCtx, cfg, info) + } else { + regions, sizes, err = MakeSourceFileRegion(egCtx, cfg, info) + } + if err != nil { + log.FromContext(egCtx).Error("make source file region error", zap.Error(err), zap.String("file_path", info.FileMeta.Path)) + return err + } + result := fileRegionRes{info: info, regions: regions, sizes: sizes, err: err} + fileRegionsMap.Store(info.FileMeta.Path, result) + return nil + }) } - close(fileChan) - err := <-errChan - if err != nil { + + if err := eg.Wait(); err != nil { return nil, err } @@ -303,7 +273,12 @@ func MakeTableRegions( // rebase row-id for all chunk rowIDBase := int64(0) for _, dataFile := range meta.DataFiles { - fileRegionsRes := fileRegionsMap[dataFile.FileMeta.Path] + v, ok := fileRegionsMap.Load(dataFile.FileMeta.Path) + if !ok { + return nil, errors.Errorf("file %s not found in MakeTableRegions", dataFile.FileMeta.Path) + } + //nolint: forcetypeassert + fileRegionsRes := v.(fileRegionRes) for _, region := range fileRegionsRes.regions { region.Chunk.PrevRowIDMax += rowIDBase region.Chunk.RowIDMax += rowIDBase diff --git a/br/pkg/lightning/mydump/region_test.go b/br/pkg/lightning/mydump/region_test.go index 2633a6541e947..d947f1783242c 100644 --- a/br/pkg/lightning/mydump/region_test.go +++ b/br/pkg/lightning/mydump/region_test.go @@ -221,6 +221,13 @@ func TestMakeTableRegionsSplitLargeFile(t *testing.T) { assert.Equal(t, int64(0), regions[0].Chunk.Offset) assert.Equal(t, TableFileSizeINF, regions[0].Chunk.EndOffset) assert.Len(t, regions[0].Chunk.Columns, 0) + + // test canceled context will not panic + ctx, cancel := context.WithCancel(context.Background()) + cancel() + for i := 0; i < 20; i++ { + _, _ = MakeTableRegions(ctx, divideConfig) + } } func TestCompressedMakeSourceFileRegion(t *testing.T) { diff --git a/build/print-workspace-status.sh b/build/print-workspace-status.sh index dd2bd490812c7..f1909b8697828 100755 --- a/build/print-workspace-status.sh +++ b/build/print-workspace-status.sh @@ -27,6 +27,10 @@ TiDB_BUILD_UTCTIME=$(date -u '+%Y-%m-%d %H:%M:%S') TIDB_GIT_HASH=$(git rev-parse HEAD) TIDB_GIT_BRANCH=$(git rev-parse --abbrev-ref HEAD) TIDB_EDITION=${TIDB_EDITION:-Community} +TIDB_ENTERPRISE_EXTENSION_GIT_HASH="" +if [ -a "extension/enterprise/.git" ]; then + TIDB_ENTERPRISE_EXTENSION_GIT_HASH=$(cd extension/enterprise && git rev-parse HEAD) +fi cat <'$.zip' AS UNSIGNED ARRAY))); + +INSERT INTO t1 VALUES + +('{"id":1, "zip": [0,111,333]}'),('{"id":2, "zip": [123,456,0]}'), +('{"id":3, "zip": [123,123,111]}'), +('{"id":4, "zip": [456,567,222]}'), +('{"id":5, "zip": []}'); + +mysql> SELECT * FROM t1 WHERE 123 MEMBER OF (data->'$.zip'); ++-----------------------------------+ +| data | ++-----------------------------------+ +| {"id": 2, "zip": [123, 456, 0]} | +| {"id": 3, "zip": [123, 123, 111]} | ++-----------------------------------+ +2 rows in set (0.01 sec) +``` + +That is: N index records point to one row record (N: 1) +A common scenario involves having rows with associated tags and the need to efficiently query all data containing a specific tag. + +## Detailed Design + +### Overview + +- Using `cast(... as ... array)` to define a multi-valued index, which is essentially an expression index whose virtual column type is "array with type". The index is encoded in the same way as normal secondary indexes. +- The update of the multi-valued index behaves the same as the normal secondary index but the modification of one row may produce changes to multiple index records. If the type of array element is not satisfied the index definition, an error will be reported. +- Use `MEMBER OF`, `JSON_CONTAINS(subset)`, `JSON_OVERLAPS(intersection)` functions in where condition to using the multi-valued index. + +### Encoding + +The encoding of each index record is identical to the normal secondary index(see [TiDB Index Key/Value Format](https://docs.google.com/document/d/1Co5iMiaxitv3okJmLYLJxZYCNChcjzswJMRr-_45Eqg/edit) for more details). + +For string types, the encoding result in TiDB is collation-aware, we could use `binary` collation for strings(in MySQL it is `utf8mb4_0900_as_cs` and behaves almost the same as `binary`). + +- A row record may have multiple index records corresponding to it +``` +row ('pk1', [1, 1, 2]) +produces index records +1 -> 'pk1' +2 -> 'pk1' +``` + +- Multi-valued index can be a compound index +``` +row ('pk1', c1, [1, 1, 2], c2) +produces index records +(c1,1,c2) -> 'pk1' +(c1,2,c2) -> 'pk1' +``` + +- Multi-valued index can be a unique index + +### Parser + +New syntax: use `cast(... as... array)` to create the index. Add an `Array` field in `FuncCastExpr` indicate the use of this syntax. +```golang +type FuncCastExpr struct { + //... + Array bool +} +``` + +### Expression + +Use `JSONBinary` type as the return type of expression `cast(... as... array)`. In the `FieldType` structure, add an `array` field indicates whether the type is an array, and `tp` represents the type of elements in the array. +```golang +type FieldType struct { + // tp is type of the array elements + tp byte + // array indicates whether the type is an array + array bool +} +``` + +Implement new built-in functions `castAsTypedArrayFunctionSig`, `MEMBER OF` and `JSON_OVERLAPS`. + +### DML + +Data changes cause index changes, which are handled in the same way as normal secondary indexes. +- Insert: insert a new row record and add index records for each element in the array. +- Delete: delete the row record and delete index records for each element in the array. +- Update: delete the old index records and add new index records. + + +### DDL + +- Multi-valued index can be composite index, but only one JSON array can be used in the index definition. +- Multi-valued index is an expression index, so it has the same restrictions as expression index. +- Multi-valued index can be a unique index, but the uniqueness is not guaranteed within the same JSON array. + +## Planner + +### Column substitute + +The column in the where condition will substituted with the corresponding expression in the index definition if the query meet the following 3 requirements +- Where condition contains any of the 3 functions: `MEMBER OF`/`JSON_CONTAINS`/`JSON_OVERLAPS`. +- Functions' parameter type must consistent with multi-valued index definitions. +- The expression is consistent with multi-valued index definition. + +For example, the index definition is `create index idx on t((cast(data->'$.zip' as unsigned array)))`, the where condition is `where 123 member of (data->'$.zip')`, the column `data->'$.zip'` is substituted with `cast(data->'$.zip' as unsigned array)`. + +Index selection is the same as normal secondary index. + +### Build the operator + +For any of the 3 functions, we can use `IndexMerge` operator to fetch the data: +- MEMBER OF () +``` +IndexMerge + IndexRangeScan() + TableRowIDScan +``` + +- JSON_CONTAINS(, [, , , ...]) +``` +IndexMerge(AND) + IndexRangeScan() + IndexRangeScan() + IndexRangeScan() + ... + TableRowIDScan +``` + +- JSON_OVERLAPS(, [, , , ...]) +``` +IndexMerge(OR) + IndexRangeScan() + IndexRangeScan() + IndexRangeScan() + ... + TableRowIDScan +``` + +Each `IndexRangeScan` is a `PointGet` like operator. It will fetch the row record by the index record. Since different indexes could match the same primary key, we need to use `IndexMerge` to filter the duplicated row records. For `JSON_CONTAINS` we should use `AND` type of `IndexMerge`, because only the primary key that contained in all the `IndexRangeScan` can be filtered. For `JSON_OVERLAPS` we should use `OR` to filter the row records. + +If the multi-valued index is unique, it can be further optimized to `PointGet`. + +## Impacts & Risks + +### Limitations, and Characteristics +- Multi-valued index will only be used when the where condition contains any of the functions `MEMBER OF`/`JSON_CONTAINS`/`JSON_OVERLAPS`. So even if SQL contains hint, force index, use index, etc., it is not necessarily possible to force multi-valued index. +- `cast(... as ... array)` can only appear once in the composite index definition, and the casted column must be a JSON column. +- If multi-valued index is a unique index, then we have +```sql +-- Allowed: +INSERT INTO t1 VALUES('[1,1,2]'); +INSERT INTO t1 VALUES('[3,3,3,4,4,4]'); + +-- Disallowed, report dup-key error: +INSERT INTO t1 VALUES('[1,2]'); +INSERT INTO t1 VALUES('[2,3]'); +``` +- Nullability + - If the write data is an empty array, there will be no corresponding index record. Therefore: `not xxx` cannot use the index, because empty array data cannot access through the index. + - workaround: `select * from t where pk not in (select pk from t where a member of (xxx))`. + - If a column is null, add a null index record. + - Null is not allowed as an array item, trying to write will report an error. +- The type of multi-valued index cannot be `BINARY`, `JSON`, and `YEAR`. +- Multi-valued index cannot be a primary key or a foreign key. +- Storage Space and Performance: + - average number of array items * secondary index uses space. + - Compared to normal indexes, DML will produce more changes to multi-valued index records, so the Multi-valued index will bring more performance impact than normal indexes. +- All other limitations in expression index. diff --git a/docs/design/2023-04-15-ddl-pause-resume.md b/docs/design/2023-04-15-ddl-pause-resume.md new file mode 100644 index 0000000000000..d7f43e376431d --- /dev/null +++ b/docs/design/2023-04-15-ddl-pause-resume.md @@ -0,0 +1,106 @@ +# Pause/Resume DDL jobs + +- Author: [dhysum](https://github.com/dhysum) +- Tracking Issue: + - https://github.com/pingcap/tidb/issues/40041 + - https://github.com/pingcap/tidb/issues/18015 + +## Background and Benefits + +DDL jobs are running asynchronously in the background, which may take a very +long time to execute. The Admin could cancel the running DDL job if +necessary, for example, out of resource. + +As a DDL job may take a lot of effort, it could be a big waste to +just cancel and restart the job. It has several stages and steps back in the +yard to finish a DDL job, we may just pause it at some step and resume it right +from that place. + +Also, such feature will benefit the Upgrade of the TiDB Cluster. + +## Goal + +Add two commands to pause and resume a long-running DDL job. In particularly: + +1. `admin pause ddl jobs 3,5;` + + The jobs (here are job 3 and job 5) should be in Running or Queueing(Wait) + state. Other states should be rejected. + +2. `admin resume ddl jobs 3,5;` + + Only Paused jobs could be resumed. Other states should be rejected. + +## Architecture + +There is no change on the architecture, just following the current one. + +## Detail Design + +The whole design is simple: + +1. Add pause/resume in parser +2. Add builder in planner +3. Add builder in executor +4. Valid the job's state, and turn it to be Pausing, if `admin pause ...` +by end user + + 4.1 Valid the state first, only Running job could be paused, just like +`admin cancel ...` + + 4.2 Turn the state to be Pausing + + 4.3 Background worker will check the state, and turn the state to be Paused + , then just return and stay in current stage; And this is what is different + from `admin cancel ...` + + 4.4 Background worker will continue fetching the job and check the state, + and keep doing nothing until the state changed + + 4.5 Specially, the Reorg state could take a long time (maybe minutes, even + hours), we also need to check the state of the job, and stop the reorg + accordingly + +5. Validate the job's state, and turn it to be Queueing, if `admin resume ...` +by end user + + 5.1 After the job's state changes, the background worker should check the + job's state and continue the work + + 5.2 No other actions should be taken + +### State Machine + +![state-machine](./imgs/ddl_job_state_machine.png) + +## Usage + +1. Create an index + + `create index idx_usr on t_user(name);` + +2. In another session, show the DDL jobs + + `admin show ddl jobs 15;` + + It will show 15 DDL jobs, change the number if you want to see more. And + then, find the running one (job_id) you want to pause. + +3. Pause the job by + + `admin pause ddl jobs $job_id` ---- $job_id is what you get from upper step + +4. Resume the paused job + + Also, you can find the Paused jobs by (change the number if you want to see + more): + + `admin show ddl jobs 15;` + + Then, reumse it by: + + `admin resume ddl jobs $job_id` + +## Future Work + +None. diff --git a/docs/design/imgs/ddl_job_state_machine.png b/docs/design/imgs/ddl_job_state_machine.png new file mode 100644 index 0000000000000..4b155ad0d020e Binary files /dev/null and b/docs/design/imgs/ddl_job_state_machine.png differ diff --git a/executor/explain.go b/executor/explain.go index 5bbe4351a884b..ed55ebcc5c75b 100644 --- a/executor/explain.go +++ b/executor/explain.go @@ -16,6 +16,7 @@ package executor import ( "context" + "fmt" "os" "path/filepath" "runtime" @@ -324,7 +325,7 @@ type ruRuntimeStats struct { // String implements the RuntimeStats interface. func (e *ruRuntimeStats) String() string { if e.RURuntimeStats != nil { - return e.RURuntimeStats.String() + return fmt.Sprintf("RU:%f", e.RURuntimeStats.RRU()+e.RURuntimeStats.WRU()) } return "" } diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index 9e18cf8413882..11271a12698d8 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -74,7 +74,7 @@ func TestExplainFor(t *testing.T) { buf.WriteString(fmt.Sprintf("%v", v)) } } - require.Regexp(t, "TableReader_5 10000.00 0 root time:.*, loops:1,( RRU:.*, WRU:.*,)? cop_task: {num:.*, max:.*, proc_keys:.* rpc_num: 1, rpc_time:.*} data:TableFullScan_4 N/A N/A\n"+ + require.Regexp(t, "TableReader_5 10000.00 0 root time:.*, loops:1,( RU:.*,)? cop_task: {num:.*, max:.*, proc_keys:.* rpc_num: 1, rpc_time:.*} data:TableFullScan_4 N/A N/A\n"+ "└─TableFullScan_4 10000.00 0 cop.* table:t1 tikv_task:{time:.*, loops:0} keep order:false, stats:pseudo N/A N/A", buf.String()) } diff --git a/executor/importer/import.go b/executor/importer/import.go index 5a536c3752bfb..eb0241485485a 100644 --- a/executor/importer/import.go +++ b/executor/importer/import.go @@ -593,6 +593,7 @@ func (e *LoadDataController) initFieldMappings() []string { if len(e.ColumnsAndUserVars) == 0 { for _, v := range tableCols { + // Data for generated column is generated from the other rows rather than from the parsed data. fieldMapping := &FieldMapping{ Column: v, } @@ -641,12 +642,7 @@ func (e *LoadDataController) initLoadColumns(columnNames []string) error { return dbterror.ErrBadField.GenWithStackByArgs(missingColName, "field list") } - for _, col := range cols { - if !col.IsGenerated() { - // todo: should report error here, since in reorderColumns we report error if en(cols) != len(columnNames) - e.InsertColumns = append(e.InsertColumns, col) - } - } + e.InsertColumns = append(e.InsertColumns, cols...) // e.InsertColumns is appended according to the original tables' column sequence. // We have to reorder it to follow the use-specified column order which is shown in the columnNames. diff --git a/executor/load_data.go b/executor/load_data.go index 0c14652b221fc..5e6b0c69805cf 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -816,6 +816,12 @@ func (w *encodeWorker) parserData2TableData( continue } + // Don't set the value for generated columns. + if fieldMappings[i].Column.IsGenerated() { + row = append(row, types.NewDatum(nil)) + continue + } + row = append(row, parserData[i]) } for i := 0; i < len(w.controller.ColumnAssignments); i++ { diff --git a/executor/loadremotetest/one_csv_test.go b/executor/loadremotetest/one_csv_test.go index 4fc818360e283..c9d4edc54e9e2 100644 --- a/executor/loadremotetest/one_csv_test.go +++ b/executor/loadremotetest/one_csv_test.go @@ -266,3 +266,49 @@ mynull,"mynull" LINES TERMINATED BY '\n';`, gcsEndpoint) s.tk.MustMatchErrMsg(sql, `must specify FIELDS \[OPTIONALLY\] ENCLOSED BY`) } + +func (s *mockGCSSuite) TestLoadDataForGeneratedColumns() { + // For issue https://github.com/pingcap/tidb/issues/39885 + s.tk.MustExec("DROP DATABASE IF EXISTS load_csv;") + s.tk.MustExec("CREATE DATABASE load_csv;") + s.tk.MustExec("USE load_csv;") + s.tk.MustExec("set @@sql_mode = ''") + s.tk.MustExec(`CREATE TABLE load_csv.t_gen1 (a int, b int generated ALWAYS AS (a+1));`) + + s.server.CreateObject(fakestorage.Object{ + ObjectAttrs: fakestorage.ObjectAttrs{ + BucketName: "test-bucket", + Name: "generated_columns.csv", + }, + Content: []byte("1 2\n2 3"), + }) + + s.tk.MustExec(fmt.Sprintf("LOAD DATA INFILE 'gcs://test-bucket/generated_columns.csv?endpoint=%s' INTO TABLE load_csv.t_gen1", gcsEndpoint)) + s.tk.MustQuery("select * from t_gen1").Check(testkit.Rows("1 2", "2 3")) + s.tk.MustExec("delete from t_gen1") + + // Specify the column, this should also work. + s.tk.MustExec(fmt.Sprintf("LOAD DATA INFILE 'gcs://test-bucket/generated_columns.csv?endpoint=%s' INTO TABLE load_csv.t_gen1 (a)", gcsEndpoint)) + s.tk.MustQuery("select * from t_gen1").Check(testkit.Rows("1 2", "2 3")) + + // Swap the column and test again. + s.tk.MustExec(`create table t_gen2 (a int generated ALWAYS AS (b+1), b int);`) + s.tk.MustExec(fmt.Sprintf("LOAD DATA INFILE 'gcs://test-bucket/generated_columns.csv?endpoint=%s' INTO TABLE load_csv.t_gen2", gcsEndpoint)) + s.tk.MustQuery("select * from t_gen2").Check(testkit.Rows("3 2", "4 3")) + s.tk.MustExec(`delete from t_gen2`) + + // Specify the column b + s.tk.MustExec(fmt.Sprintf("LOAD DATA INFILE 'gcs://test-bucket/generated_columns.csv?endpoint=%s' INTO TABLE load_csv.t_gen2 (b)", gcsEndpoint)) + s.tk.MustQuery("show warnings").Check(testkit.Rows( + "Warning 1262 Row 1 was truncated; it contained more data than there were input columns", + "Warning 1262 Row 2 was truncated; it contained more data than there were input columns")) + s.tk.MustQuery("select * from t_gen2").Check(testkit.Rows("2 1", "3 2")) + s.tk.MustExec(`delete from t_gen2`) + + // Specify the column a + s.tk.MustExec(fmt.Sprintf("LOAD DATA INFILE 'gcs://test-bucket/generated_columns.csv?endpoint=%s' INTO TABLE load_csv.t_gen2 (a)", gcsEndpoint)) + s.tk.MustQuery("show warnings").Check(testkit.Rows( + "Warning 1262 Row 1 was truncated; it contained more data than there were input columns", + "Warning 1262 Row 2 was truncated; it contained more data than there were input columns")) + s.tk.MustQuery("select * from t_gen2").Check(testkit.Rows(" ", " ")) +} diff --git a/executor/tiflashtest/tiflash_test.go b/executor/tiflashtest/tiflash_test.go index c6384b0b99c70..57ce4e31eabf3 100644 --- a/executor/tiflashtest/tiflash_test.go +++ b/executor/tiflashtest/tiflash_test.go @@ -1424,15 +1424,15 @@ func TestDisaggregatedTiFlashQuery(t *testing.T) { "├─TableReader_15 3323.33 root MppVersion: 1, data:ExchangeSender_14", "│ └─ExchangeSender_14 3323.33 mpp[tiflash] ExchangeType: PassThrough", "│ └─Selection_13 3323.33 mpp[tiflash] lt(test.t1.c1, 2)", - "│ └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, partition:p0 keep order:false, stats:pseudo", + "│ └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, partition:p0 pushed down filter:empty, keep order:false, stats:pseudo", "├─TableReader_19 3323.33 root MppVersion: 1, data:ExchangeSender_18", "│ └─ExchangeSender_18 3323.33 mpp[tiflash] ExchangeType: PassThrough", "│ └─Selection_17 3323.33 mpp[tiflash] lt(test.t1.c1, 2)", - "│ └─TableFullScan_16 10000.00 mpp[tiflash] table:t1, partition:p1 keep order:false, stats:pseudo", + "│ └─TableFullScan_16 10000.00 mpp[tiflash] table:t1, partition:p1 pushed down filter:empty, keep order:false, stats:pseudo", "└─TableReader_23 3323.33 root MppVersion: 1, data:ExchangeSender_22", " └─ExchangeSender_22 3323.33 mpp[tiflash] ExchangeType: PassThrough", " └─Selection_21 3323.33 mpp[tiflash] lt(test.t1.c1, 2)", - " └─TableFullScan_20 10000.00 mpp[tiflash] table:t1, partition:p2 keep order:false, stats:pseudo")) + " └─TableFullScan_20 10000.00 mpp[tiflash] table:t1, partition:p2 pushed down filter:empty, keep order:false, stats:pseudo")) } func TestMPPMemoryTracker(t *testing.T) { diff --git a/planner/core/casetest/testdata/plan_suite_in.json b/planner/core/casetest/testdata/plan_suite_in.json index ccddea0fd76f0..4bcbb0c64fbee 100644 --- a/planner/core/casetest/testdata/plan_suite_in.json +++ b/planner/core/casetest/testdata/plan_suite_in.json @@ -159,6 +159,10 @@ "name": "TestMPPRightSemiJoin", "cases": [ "set @@session.tidb_allow_mpp=true", + "explain select * from t1 where exists (select * from t2 where t1.a=t2.b)", + "set @@session.tidb_prefer_broadcast_join_by_exchange_data_size=0", + "set @@session.tidb_broadcast_join_threshold_size=0", + "set @@session.tidb_broadcast_join_threshold_count=0", "explain select * from t1 where exists (select * from t2 where t1.a=t2.b)" ] }, diff --git a/planner/core/casetest/testdata/plan_suite_out.json b/planner/core/casetest/testdata/plan_suite_out.json index a089849913219..56a5973129503 100644 --- a/planner/core/casetest/testdata/plan_suite_out.json +++ b/planner/core/casetest/testdata/plan_suite_out.json @@ -1586,12 +1586,44 @@ "TableReader_34 0.80 root MppVersion: 1, data:ExchangeSender_33", "└─ExchangeSender_33 0.80 mpp[tiflash] ExchangeType: PassThrough", " └─HashJoin_32 0.80 mpp[tiflash] semi join, equal:[eq(test.t1.a, test.t2.b)]", + " ├─ExchangeReceiver_17(Build) 8.00 mpp[tiflash] ", + " │ └─ExchangeSender_16 8.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_15 8.00 mpp[tiflash] not(isnull(test.t2.b))", + " │ └─TableFullScan_14 8.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false", + " └─Selection_13(Probe) 1.00 mpp[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan_12 1.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false" + ], + "Warn": null + }, + { + "SQL": "set @@session.tidb_prefer_broadcast_join_by_exchange_data_size=0", + "Plan": null, + "Warn": null + }, + { + "SQL": "set @@session.tidb_broadcast_join_threshold_size=0", + "Plan": null, + "Warn": null + }, + { + "SQL": "set @@session.tidb_broadcast_join_threshold_count=0", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain select * from t1 where exists (select * from t2 where t1.a=t2.b)", + "Plan": [ + "TableReader_36 0.80 root MppVersion: 1, data:ExchangeSender_35", + "└─ExchangeSender_35 0.80 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_34 0.80 mpp[tiflash] semi join, equal:[eq(test.t1.a, test.t2.b)]", " ├─ExchangeReceiver_15(Build) 1.00 mpp[tiflash] ", - " │ └─ExchangeSender_14 1.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─ExchangeSender_14 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.a, collate: binary]", " │ └─Selection_13 1.00 mpp[tiflash] not(isnull(test.t1.a))", " │ └─TableFullScan_12 1.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false", - " └─Selection_17(Probe) 8.00 mpp[tiflash] not(isnull(test.t2.b))", - " └─TableFullScan_16 8.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false" + " └─ExchangeReceiver_19(Probe) 8.00 mpp[tiflash] ", + " └─ExchangeSender_18 8.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t2.b, collate: binary]", + " └─Selection_17 8.00 mpp[tiflash] not(isnull(test.t2.b))", + " └─TableFullScan_16 8.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false" ], "Warn": null } @@ -8311,7 +8343,7 @@ " │ └─ExchangeSender 1775.11 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.tbl_15.col_96, collate: binary], [name: test.tbl_15.col_91, collate: binary]", " │ └─HashAgg 1775.11 mpp[tiflash] group by:test.tbl_15.col_91, test.tbl_15.col_96, funcs:min(test.tbl_15.col_92)->Column#18, funcs:max(test.tbl_15.col_92)->Column#20", " │ └─Selection 2218.89 mpp[tiflash] gt(test.tbl_15.col_93, 7623.679908049186), ne(test.tbl_15.col_94, 2033-01-09 00:00:00.000000)", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p0 keep order:false, stats:pseudo", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p0 pushed down filter:empty, keep order:false, stats:pseudo", " ├─TableReader 1775.11 root MppVersion: 1, data:ExchangeSender", " │ └─ExchangeSender 1775.11 mpp[tiflash] ExchangeType: PassThrough", " │ └─HashAgg 1775.11 mpp[tiflash] group by:test.tbl_15.col_91, test.tbl_15.col_96, funcs:firstrow(test.tbl_15.col_96)->test.tbl_15.col_96, funcs:min(Column#30)->Column#15, funcs:firstrow(test.tbl_15.col_91)->test.tbl_15.col_91, funcs:max(Column#32)->Column#16, funcs:firstrow(test.tbl_15.col_96)->test.tbl_15.col_96, funcs:firstrow(test.tbl_15.col_91)->test.tbl_15.col_91", @@ -8319,7 +8351,7 @@ " │ └─ExchangeSender 1775.11 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.tbl_15.col_96, collate: binary], [name: test.tbl_15.col_91, collate: binary]", " │ └─HashAgg 1775.11 mpp[tiflash] group by:test.tbl_15.col_91, test.tbl_15.col_96, funcs:min(test.tbl_15.col_92)->Column#30, funcs:max(test.tbl_15.col_92)->Column#32", " │ └─Selection 2218.89 mpp[tiflash] gt(test.tbl_15.col_93, 7623.679908049186), ne(test.tbl_15.col_94, 2033-01-09 00:00:00.000000)", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p1 keep order:false, stats:pseudo", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p1 pushed down filter:empty, keep order:false, stats:pseudo", " ├─TableReader 1775.11 root MppVersion: 1, data:ExchangeSender", " │ └─ExchangeSender 1775.11 mpp[tiflash] ExchangeType: PassThrough", " │ └─HashAgg 1775.11 mpp[tiflash] group by:test.tbl_15.col_91, test.tbl_15.col_96, funcs:firstrow(test.tbl_15.col_96)->test.tbl_15.col_96, funcs:min(Column#42)->Column#15, funcs:firstrow(test.tbl_15.col_91)->test.tbl_15.col_91, funcs:max(Column#44)->Column#16, funcs:firstrow(test.tbl_15.col_96)->test.tbl_15.col_96, funcs:firstrow(test.tbl_15.col_91)->test.tbl_15.col_91", @@ -8327,7 +8359,7 @@ " │ └─ExchangeSender 1775.11 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.tbl_15.col_96, collate: binary], [name: test.tbl_15.col_91, collate: binary]", " │ └─HashAgg 1775.11 mpp[tiflash] group by:test.tbl_15.col_91, test.tbl_15.col_96, funcs:min(test.tbl_15.col_92)->Column#42, funcs:max(test.tbl_15.col_92)->Column#44", " │ └─Selection 2218.89 mpp[tiflash] gt(test.tbl_15.col_93, 7623.679908049186), ne(test.tbl_15.col_94, 2033-01-09 00:00:00.000000)", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p2 keep order:false, stats:pseudo", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p2 pushed down filter:empty, keep order:false, stats:pseudo", " └─TableReader 1775.11 root MppVersion: 1, data:ExchangeSender", " └─ExchangeSender 1775.11 mpp[tiflash] ExchangeType: PassThrough", " └─HashAgg 1775.11 mpp[tiflash] group by:test.tbl_15.col_91, test.tbl_15.col_96, funcs:firstrow(test.tbl_15.col_96)->test.tbl_15.col_96, funcs:min(Column#54)->Column#15, funcs:firstrow(test.tbl_15.col_91)->test.tbl_15.col_91, funcs:max(Column#56)->Column#16, funcs:firstrow(test.tbl_15.col_96)->test.tbl_15.col_96, funcs:firstrow(test.tbl_15.col_91)->test.tbl_15.col_91", @@ -8335,7 +8367,7 @@ " └─ExchangeSender 1775.11 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.tbl_15.col_96, collate: binary], [name: test.tbl_15.col_91, collate: binary]", " └─HashAgg 1775.11 mpp[tiflash] group by:test.tbl_15.col_91, test.tbl_15.col_96, funcs:min(test.tbl_15.col_92)->Column#54, funcs:max(test.tbl_15.col_92)->Column#56", " └─Selection 2218.89 mpp[tiflash] gt(test.tbl_15.col_93, 7623.679908049186), ne(test.tbl_15.col_94, 2033-01-09 00:00:00.000000)", - " └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p3 keep order:false, stats:pseudo" + " └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p3 pushed down filter:empty, keep order:false, stats:pseudo" ], "Warning": null }, @@ -8379,14 +8411,14 @@ " │ └─TableReader 39.97 root MppVersion: 1, data:ExchangeSender", " │ └─ExchangeSender 39.97 mpp[tiflash] ExchangeType: PassThrough", " │ └─Selection 39.97 mpp[tiflash] or(eq(test.tbl_16.col_100, 10672141), in(test.tbl_16.col_104, \"yfEG1t!*b\", \"C1*bqx_qyO\", \"vQ^yUpKHr&j#~\"))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_16, partition:p0 keep order:false, stats:pseudo", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_16, partition:p0 pushed down filter:empty, keep order:false, stats:pseudo", " └─StreamAgg 31.98 root group by:Column#26, funcs:count(Column#23)->Column#11, funcs:sum(Column#24)->Column#12, funcs:firstrow(Column#25)->test.tbl_16.col_100", " └─Projection 39.97 root test.tbl_16.col_100, cast(test.tbl_16.col_100, decimal(8,0) UNSIGNED BINARY)->Column#24, test.tbl_16.col_100, test.tbl_16.col_100", " └─Sort 39.97 root test.tbl_16.col_100", " └─TableReader 39.97 root MppVersion: 1, data:ExchangeSender", " └─ExchangeSender 39.97 mpp[tiflash] ExchangeType: PassThrough", " └─Selection 39.97 mpp[tiflash] or(eq(test.tbl_16.col_100, 10672141), in(test.tbl_16.col_104, \"yfEG1t!*b\", \"C1*bqx_qyO\", \"vQ^yUpKHr&j#~\"))", - " └─TableFullScan 10000.00 mpp[tiflash] table:tbl_16, partition:p1 keep order:false, stats:pseudo" + " └─TableFullScan 10000.00 mpp[tiflash] table:tbl_16, partition:p1 pushed down filter:empty, keep order:false, stats:pseudo" ], "Warning": null } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 31317b16bfb3f..c3fb82fde886e 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2398,7 +2398,7 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC preferredBuildIndex = 1 } } else if p.JoinType.IsSemiJoin() { - if !p.isNAAJ() && len(p.EqualConditions) > 0 && (p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin) { + if !useBCJ && !p.isNAAJ() && len(p.EqualConditions) > 0 && (p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin) { // TiFlash only supports Non-null_aware non-cross semi/anti_semi join to use both sides as build side preferredBuildIndex = 1 // MPPOuterJoinFixedBuildSide default value is false diff --git a/planner/core/explain.go b/planner/core/explain.go index a23b98909f4a3..916947e916895 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -20,7 +20,6 @@ import ( "strconv" "strings" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/kv" @@ -214,7 +213,7 @@ func (p *PhysicalTableScan) OperatorInfo(normalized bool) string { } } } - if !config.GetGlobalConfig().DisaggregatedTiFlash && p.ctx.GetSessionVars().EnableLateMaterialization && len(p.filterCondition) > 0 && p.StoreType == kv.TiFlash { + if p.ctx.GetSessionVars().EnableLateMaterialization && len(p.filterCondition) > 0 && p.StoreType == kv.TiFlash { buffer.WriteString("pushed down filter:") if len(p.lateMaterializationFilterCondition) > 0 { if normalized { diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index d5520e99d5a39..39ea9532ef724 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -562,7 +562,7 @@ func prunePhysicalColumnsInternal(sctx sessionctx.Context, plan PhysicalPlan) er // - Filter conditions that apply to the same column are either pushed down or not pushed down at all. func tryEnableLateMaterialization(sctx sessionctx.Context, plan PhysicalPlan) { // check if EnableLateMaterialization is set - if sctx.GetSessionVars().EnableLateMaterialization && !config.GetGlobalConfig().DisaggregatedTiFlash && !sctx.GetSessionVars().TiFlashFastScan { + if sctx.GetSessionVars().EnableLateMaterialization && !sctx.GetSessionVars().TiFlashFastScan { predicatePushDownToTableScan(sctx, plan) } if sctx.GetSessionVars().EnableLateMaterialization && sctx.GetSessionVars().TiFlashFastScan { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 68cc917eb9180..05c2bce8ef43a 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1870,10 +1870,6 @@ func NewSessionVars(hctx HookContext) *SessionVars { EnableLateMaterialization: DefTiDBOptEnableLateMaterialization, TiFlashComputeDispatchPolicy: tiflashcompute.DispatchPolicyConsistentHash, } - // Always disable late materialization for disaggregated TiFlash until it is supported. - if config.GetGlobalConfig().DisaggregatedTiFlash { - vars.EnableLateMaterialization = false - } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ indexLookupConcurrency: DefIndexLookupConcurrency, diff --git a/tidb-server/BUILD.bazel b/tidb-server/BUILD.bazel index 99239b523536e..4f6a543288eca 100644 --- a/tidb-server/BUILD.bazel +++ b/tidb-server/BUILD.bazel @@ -78,6 +78,7 @@ go_binary( "github.com/pingcap/tidb/parser/mysql.TiDBReleaseVersion": "{STABLE_TiDB_RELEASE_VERSION}", "github.com/pingcap/tidb/util/versioninfo.TiDBBuildTS": "{STABLE_TiDB_BUILD_UTCTIME}", "github.com/pingcap/tidb/util/versioninfo.TiDBGitHash": "{STABLE_TIDB_GIT_HASH}", + "github.com/pingcap/tidb/util/versioninfo.TiDBEnterpriseExtensionGitHash": "{STABLE_TIDB_ENTERPRISE_EXTENSION_GIT_HASH}", "github.com/pingcap/tidb/util/versioninfo.TiDBGitBranch": "{STABLE_TIDB_GIT_BRANCH}", "github.com/pingcap/tidb/util/versioninfo.TiDBEdition": "{STABLE_TIDB_EDITION}", }, diff --git a/util/printer/printer.go b/util/printer/printer.go index 084033b603ef2..5fcdaa4ec2dbd 100644 --- a/util/printer/printer.go +++ b/util/printer/printer.go @@ -31,7 +31,7 @@ import ( // PrintTiDBInfo prints the TiDB version information. func PrintTiDBInfo() { - logutil.BgLogger().Info("Welcome to TiDB.", + fields := []zap.Field{ zap.String("Release Version", mysql.TiDBReleaseVersion), zap.String("Edition", versioninfo.TiDBEdition), zap.String("Git Commit Hash", versioninfo.TiDBGitHash), @@ -40,7 +40,12 @@ func PrintTiDBInfo() { zap.String("GoVersion", buildVersion), zap.Bool("Race Enabled", israce.RaceEnabled), zap.Bool("Check Table Before Drop", config.CheckTableBeforeDrop), - zap.String("TiKV Min Version", versioninfo.TiKVMinVersion)) + zap.String("TiKV Min Version", versioninfo.TiKVMinVersion), + } + if versioninfo.TiDBEnterpriseExtensionGitHash != "" { + fields = append(fields, zap.String("Enterprise Extension Commit Hash", versioninfo.TiDBEnterpriseExtensionGitHash)) + } + logutil.BgLogger().Info("Welcome to TiDB.", fields...) configJSON, err := json.Marshal(config.GetGlobalConfig()) if err != nil { panic(err) @@ -50,6 +55,10 @@ func PrintTiDBInfo() { // GetTiDBInfo returns the git hash and build time of this tidb-server binary. func GetTiDBInfo() string { + enterpriseVersion := "" + if versioninfo.TiDBEnterpriseExtensionGitHash != "" { + enterpriseVersion = fmt.Sprintf("\nEnterprise Extension Commit Hash: %s", versioninfo.TiDBEnterpriseExtensionGitHash) + } return fmt.Sprintf("Release Version: %s\n"+ "Edition: %s\n"+ "Git Commit Hash: %s\n"+ @@ -59,7 +68,8 @@ func GetTiDBInfo() string { "Race Enabled: %v\n"+ "TiKV Min Version: %s\n"+ "Check Table Before Drop: %v\n"+ - "Store: %s", + "Store: %s"+ + "%s", mysql.TiDBReleaseVersion, versioninfo.TiDBEdition, versioninfo.TiDBGitHash, @@ -70,6 +80,7 @@ func GetTiDBInfo() string { versioninfo.TiKVMinVersion, config.CheckTableBeforeDrop, config.GetGlobalConfig().Store, + enterpriseVersion, ) } diff --git a/util/versioninfo/versioninfo.go b/util/versioninfo/versioninfo.go index 8c2b9b57d42e3..a282272fd940a 100644 --- a/util/versioninfo/versioninfo.go +++ b/util/versioninfo/versioninfo.go @@ -26,5 +26,6 @@ var ( TiDBGitBranch = "None" TiDBEdition = CommunityEdition // TiKVMinVersion is the minimum version of TiKV that can be compatible with the current TiDB. - TiKVMinVersion = "6.2.0-alpha" + TiKVMinVersion = "6.2.0-alpha" + TiDBEnterpriseExtensionGitHash = "" )