diff --git a/docs/tidb_http_api.md b/docs/tidb_http_api.md index b6fa14b6d43d5..dbdeb664f292f 100644 --- a/docs/tidb_http_api.md +++ b/docs/tidb_http_api.md @@ -234,6 +234,7 @@ } } } +<<<<<<< HEAD ``` *Hint: The meaning of the MVCC operation type:* @@ -252,6 +253,268 @@ ``` 1. Get MVCC Information of the first key in the table with a specified start ts +======= + +15. Scatter regions of the specified table, add a `scatter-range` scheduler for the PD and the range is same as the table range. + + ```shell + curl http://{TiDBIP}:10080/tables/{db}/{table}/scatter + ``` + + *Hint: On a partitioned table, use the `table(partition)` pattern as the table name, `test(p1)` for example.* + + **Note**: The `scatter-range` scheduler may conflict with the global scheduler, do not use it for long periods on the larger table. + +16. Stop scatter the regions, disable the `scatter-range` scheduler for the specified table. + + ```shell + curl http://{TiDBIP}:10080/tables/{db}/{table}/stop-scatter + ``` + + *Hint: On a partitioned table, use the `table(partition)` pattern as the table name, `test(p1)` for example.* + +17. Get TiDB server settings + + ```shell + curl http://{TiDBIP}:10080/settings + ``` + +18. Get TiDB server information. + + ```shell + curl http://{TiDBIP}:10080/info + ``` + + ```shell + $curl http://127.0.0.1:10080/info + { + "ddl_id": "f7e73ed5-63b4-4cb4-ba7c-42b32dc74e77", + "git_hash": "f572e33854e1c0f942f031e9656d0004f99995c6", + "ip": "", + "is_owner": true, + "lease": "45s", + "listening_port": 4000, + "status_port": 10080, + "version": "5.7.25-TiDB-v2.1.0-rc.3-355-gf572e3385-dirty" + } + ``` + +19. Get TiDB cluster all servers information. + + ```shell + curl http://{TiDBIP}:10080/info/all + ``` + + ```shell + $curl http://127.0.0.1:10080/info/all + { + "servers_num": 2, + "owner_id": "29a65ec0-d931-4f9e-a212-338eaeffab96", + "is_all_server_version_consistent": true, + "all_servers_info": { + "29a65ec0-d931-4f9e-a212-338eaeffab96": { + "version": "5.7.25-TiDB-v4.0.0-alpha-669-g8f2a09a52-dirty", + "git_hash": "8f2a09a52fdcaf9d9bfd775d2c6023f363dc121e", + "ddl_id": "29a65ec0-d931-4f9e-a212-338eaeffab96", + "ip": "", + "listening_port": 4000, + "status_port": 10080, + "lease": "45s", + "binlog_status": "Off" + }, + "cd13c9eb-c3ee-4887-af9b-e64f3162d92c": { + "version": "5.7.25-TiDB-v4.0.0-alpha-669-g8f2a09a52-dirty", + "git_hash": "8f2a09a52fdcaf9d9bfd775d2c6023f363dc121e", + "ddl_id": "cd13c9eb-c3ee-4887-af9b-e64f3162d92c", + "ip": "", + "listening_port": 4001, + "status_port": 10081, + "lease": "45s", + "binlog_status": "Off" + } + } + } + ``` + +20. Enable/Disable TiDB server general log + + ```shell + curl -X POST -d "tidb_general_log=1" http://{TiDBIP}:10080/settings + curl -X POST -d "tidb_general_log=0" http://{TiDBIP}:10080/settings + ``` + +21. Change TiDB server log level + + ```shell + curl -X POST -d "log_level=debug" http://{TiDBIP}:10080/settings + curl -X POST -d "log_level=info" http://{TiDBIP}:10080/settings + ``` + +22. Change TiDB DDL slow log threshold + + The unit is millisecond. + + ```shell + curl -X POST -d "ddl_slow_threshold=300" http://{TiDBIP}:10080/settings + ``` + +23. Get the column value by an encoded row and some information that can be obtained from a column of the table schema information. + + Argument example: rowBin=base64_encoded_row_value + + ```shell + curl http://{TiDBIP}:10080/tables/{colID}/{colFlag}/{colLen}?rowBin={val} + ``` + + *Hint: For the column which field type is timezone dependent, e.g. `timestamp`, convert its value to UTC timezone.* + +24. Resign the ddl owner, let tidb start a new ddl owner election. + + ```shell + curl -X POST http://{TiDBIP}:10080/ddl/owner/resign + ``` + + **Note**: If you request a TiDB that is not ddl owner, the response will be `This node is not a ddl owner, can't be resigned.` + +25. Get the TiDB DDL job history information. + + ```shell + curl http://{TiDBIP}:10080/ddl/history + ``` + + **Note**: When the DDL history is very very long, system table may containg too many jobs. This interface will get a maximum of 2048 history ddl jobs by default. If you want get more jobs, consider adding `start_job_id` and `limit`. + +26. Get count {number} TiDB DDL job history information. + + ```shell + curl http://{TiDBIP}:10080/ddl/history?limit={number} + ``` + +27. Get count {number} TiDB DDL job history information, start with job {id} + + ```shell + curl http://{TIDBIP}:10080/ddl/history?start_job_id={id}&limit={number} + ``` + +28. Download TiDB debug info + + ```shell + curl http://{TiDBIP}:10080/debug/zip?seconds=60 --output debug.zip + ``` + + zip file will include: + + - Go heap pprof(after GC) + - Go cpu pprof(10s) + - Go mutex pprof + - Full goroutine + - TiDB config and version + + Param: + + - seconds: profile time(s), default is 10s. + +29. Get statistics data of specified table. + + ```shell + curl http://{TiDBIP}:10080/stats/dump/{db}/{table} + ``` + +30. Get statistics data of specific table and timestamp. + + ```shell + curl http://{TiDBIP}:10080/stats/dump/{db}/{table}/{yyyyMMddHHmmss} + ``` + + ```shell + curl http://{TiDBIP}:10080/stats/dump/{db}/{table}/{yyyy-MM-dd HH:mm:ss} + ``` + +31. Resume the binlog writing when Pump is recovered. + + ```shell + curl http://{TiDBIP}:10080/binlog/recover + ``` + + Return value: + + - timeout, return status code: 400, message: `timeout` + - If it returns normally, status code: 200, message example: + + ```text + { +   "Skipped": false, +   "SkippedCommitterCounter": 0 + } + ``` + + `Skipped`: false indicates that the current binlog is not in the skipped state, otherwise, it is in the skipped state + `SkippedCommitterCounter`: Represents how many transactions are currently being committed in the skipped state. By default, the API will return after waiting until all skipped-binlog transactions are committed. If this value is greater than 0, it means that you need to wait until them are committed . + + Param: + + - op=nowait: return after binlog status is recoverd, do not wait until the skipped-binlog transactions are committed. + - op=reset: reset `SkippedCommitterCounter` to 0 to avoid the problem that `SkippedCommitterCounter` is not cleared due to some unusual cases. + - op=status: Get the current status of binlog recovery. + +32. Enable/disable async commit feature + + ```shell + curl -X POST -d "tidb_enable_async_commit=1" http://{TiDBIP}:10080/settings + curl -X POST -d "tidb_enable_async_commit=0" http://{TiDBIP}:10080/settings + ``` + +33. Enable/disable one-phase commit feature + + ```shell + curl -X POST -d "tidb_enable_1pc=1" http://{TiDBIP}:10080/settings + curl -X POST -d "tidb_enable_1pc=0" http://{TiDBIP}:10080/settings + ``` + +34. Enable/disable the mutation checker + + ```shell + curl -X POST -d "tidb_enable_mutation_checker=1" http://{TiDBIP}:10080/settings + curl -X POST -d "tidb_enable_mutation_checker=0" http://{TiDBIP}:10080/settings + ``` + +35. Get/Set the size of the Ballast Object + + ```shell + # get current size of the ballast object + curl -v http://{TiDBIP}:10080/debug/ballast-object-sz + # reset the size of the ballast object (2GB in this example) + curl -v -X POST -d "2147483648" http://{TiDBIP}:10080/debug/ballast-object-sz + ``` + +36. Set deadlock history table capacity + + ```shell + curl -X POST -d "deadlock_history_capacity={number}" http://{TiDBIP}:10080/settings + ``` + +37. Set whether deadlock history (`DEADLOCKS`) collect retryable deadlocks + + ```shell + curl -X POST -d "deadlock_history_collect_retryable={bool_val}" http://{TiDBIP}:10080/settings + ``` + +38. Set transaction_id to digest mapping minimum duration threshold, only transactions which last longer than this threshold will be collected into `TRX_SUMMARY`. + + ```shell + curl -X POST -d "transaction_id_digest_min_duration={number}" http://{TiDBIP}:10080/settings + ``` + + Unit of duration here is ms. + +39. Set transaction summary table (`TRX_SUMMARY`) capacity + + ```shell + curl -X POST -d "transaction_summary_capacity={number}" http://{TiDBIP}:10080/settings + ``` + +40. The commands are used to handle smooth upgrade mode(refer to the [TiDB Smooth Upgrade](https://github.com/pingcap/docs/blob/4aa0b1d5078617cc06bd1957c5c93e86efb4668d/smooth-upgrade-tidb.md) for details) operations. We can send these upgrade operations to the cluster. The operations here include `start`, `finish` and `show`. +>>>>>>> f2856e3521f (ddl: limit the count of getting ddlhistory jobs (#55590)) ```shell curl http://{TiDBIP}:10080/mvcc/txn/{startTS}/{db}/{table} diff --git a/pkg/ddl/ddl_history.go b/pkg/ddl/ddl_history.go new file mode 100644 index 0000000000000..1455f39a2db76 --- /dev/null +++ b/pkg/ddl/ddl_history.go @@ -0,0 +1,183 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl + +import ( + "cmp" + "context" + "fmt" + "slices" + "strconv" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/ddl/logutil" + sess "github.com/pingcap/tidb/pkg/ddl/session" + "github.com/pingcap/tidb/pkg/ddl/util" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessiontxn" + "go.uber.org/zap" +) + +// DefNumHistoryJobs is default value of the default number of history job +const ( + DefNumHistoryJobs = 10 + batchNumHistoryJobs = 128 + // DefNumGetDDLHistoryJobs is the max count for getting the ddl history once. + DefNumGetDDLHistoryJobs = 2048 +) + +// AddHistoryDDLJob record the history job. +func AddHistoryDDLJob(ctx context.Context, sess *sess.Session, t *meta.Meta, job *model.Job, updateRawArgs bool) error { + err := addHistoryDDLJob2Table(ctx, sess, job, updateRawArgs) + if err != nil { + logutil.DDLLogger().Info("failed to add DDL job to history table", zap.Error(err)) + } + // we always add history DDL job to job list at this moment. + return t.AddHistoryDDLJob(job, updateRawArgs) +} + +// addHistoryDDLJob2Table adds DDL job to history table. +func addHistoryDDLJob2Table(ctx context.Context, sess *sess.Session, job *model.Job, updateRawArgs bool) error { + b, err := job.Encode(updateRawArgs) + if err != nil { + return err + } + _, err = sess.Execute(ctx, + fmt.Sprintf("insert ignore into mysql.tidb_ddl_history(job_id, job_meta, db_name, table_name, schema_ids, table_ids, create_time) values (%d, %s, %s, %s, %s, %s, %v)", + job.ID, util.WrapKey2String(b), strconv.Quote(job.SchemaName), strconv.Quote(job.TableName), + strconv.Quote(strconv.FormatInt(job.SchemaID, 10)), + strconv.Quote(strconv.FormatInt(job.TableID, 10)), + strconv.Quote(model.TSConvert2Time(job.StartTS).String()), + ), + "insert_history") + return errors.Trace(err) +} + +// GetHistoryJobByID return history DDL job by ID. +func GetHistoryJobByID(sess sessionctx.Context, id int64) (*model.Job, error) { + err := sessiontxn.NewTxn(context.Background(), sess) + if err != nil { + return nil, err + } + defer func() { + // we can ignore the commit error because this txn is readonly. + _ = sess.CommitTxn(context.Background()) + }() + txn, err := sess.Txn(true) + if err != nil { + return nil, err + } + t := meta.NewMeta(txn) + job, err := t.GetHistoryDDLJob(id) + return job, errors.Trace(err) +} + +// GetLastNHistoryDDLJobs returns the DDL history jobs and an error. +// The maximum count of history jobs is num. +func GetLastNHistoryDDLJobs(t *meta.Meta, maxNumJobs int) ([]*model.Job, error) { + iterator, err := GetLastHistoryDDLJobsIterator(t) + if err != nil { + return nil, errors.Trace(err) + } + return iterator.GetLastJobs(maxNumJobs, nil) +} + +// IterHistoryDDLJobs iterates history DDL jobs until the `finishFn` return true or error. +func IterHistoryDDLJobs(txn kv.Transaction, finishFn func([]*model.Job) (bool, error)) error { + txnMeta := meta.NewMeta(txn) + iter, err := GetLastHistoryDDLJobsIterator(txnMeta) + if err != nil { + return err + } + cacheJobs := make([]*model.Job, 0, DefNumHistoryJobs) + for { + cacheJobs, err = iter.GetLastJobs(DefNumHistoryJobs, cacheJobs) + if err != nil || len(cacheJobs) == 0 { + return err + } + finish, err := finishFn(cacheJobs) + if err != nil || finish { + return err + } + } +} + +// GetLastHistoryDDLJobsIterator gets latest N history DDL jobs iterator. +func GetLastHistoryDDLJobsIterator(m *meta.Meta) (meta.LastJobIterator, error) { + return m.GetLastHistoryDDLJobsIterator() +} + +// GetAllHistoryDDLJobs get all the done DDL jobs. +func GetAllHistoryDDLJobs(m *meta.Meta) ([]*model.Job, error) { + iterator, err := GetLastHistoryDDLJobsIterator(m) + if err != nil { + return nil, errors.Trace(err) + } + allJobs := make([]*model.Job, 0, batchNumHistoryJobs) + for { + jobs, err := iterator.GetLastJobs(batchNumHistoryJobs, nil) + if err != nil { + return nil, errors.Trace(err) + } + allJobs = append(allJobs, jobs...) + if len(jobs) < batchNumHistoryJobs { + break + } + } + // sort job. + slices.SortFunc(allJobs, func(i, j *model.Job) int { + return cmp.Compare(i.ID, j.ID) + }) + return allJobs, nil +} + +// ScanHistoryDDLJobs get some of the done DDL jobs. +// When the DDL history is quite large, GetAllHistoryDDLJobs() API can't work well, because it makes the server OOM. +// The result is in descending order by job ID. +func ScanHistoryDDLJobs(m *meta.Meta, startJobID int64, limit int) ([]*model.Job, error) { + var iter meta.LastJobIterator + var err error + + if startJobID == 0 { + // if 'start_job_id' == 0 and 'limit' == 0(default value), get the last 1024 ddl history job by defaultly. + if limit == 0 { + limit = DefNumGetDDLHistoryJobs + + failpoint.Inject("history-ddl-jobs-limit", func(val failpoint.Value) { + injectLimit, ok := val.(int) + if ok { + logutil.DDLLogger().Info("failpoint history-ddl-jobs-limit", zap.Int("limit", injectLimit)) + limit = injectLimit + } + }) + } + iter, err = m.GetLastHistoryDDLJobsIterator() + } else { + // if 'start_job_id' > 0, it must set value to 'limit' + if limit == 0 { + return nil, errors.New("when 'start_job_id' is specified, it must work with a 'limit'") + } + iter, err = m.GetHistoryDDLJobsIterator(startJobID) + } + if err != nil { + return nil, errors.Trace(err) + } + + return iter.GetLastJobs(limit, nil) +} diff --git a/pkg/ddl/ddl_history_test.go b/pkg/ddl/ddl_history_test.go new file mode 100644 index 0000000000000..d47dcc7b0cf24 --- /dev/null +++ b/pkg/ddl/ddl_history_test.go @@ -0,0 +1,138 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Copyright 2013 The ql Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSES/QL-LICENSE file. + +package ddl_test + +import ( + "context" + "testing" + + "github.com/ngaut/pools" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/ddl" + "github.com/pingcap/tidb/pkg/ddl/session" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/stretchr/testify/require" +) + +func TestDDLHistoryBasic(t *testing.T) { + var ( + ddlHistoryJobCount = 0 + ) + + store := testkit.CreateMockStore(t) + rs := pools.NewResourcePool(func() (pools.Resource, error) { + newTk := testkit.NewTestKit(t, store) + return newTk.Session(), nil + }, 8, 8, 0) + sessPool := session.NewSessionPool(rs) + sessCtx, err := sessPool.Get() + require.NoError(t, err) + sess := session.NewSession(sessCtx) + + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnLightning) + err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { + t := meta.NewMeta(txn) + return ddl.AddHistoryDDLJob(context.Background(), sess, t, &model.Job{ + ID: 1, + }, false) + }) + + require.NoError(t, err) + + err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { + t := meta.NewMeta(txn) + return ddl.AddHistoryDDLJob(context.Background(), sess, t, &model.Job{ + ID: 2, + }, false) + }) + + require.NoError(t, err) + + job, err := ddl.GetHistoryJobByID(sessCtx, 1) + require.NoError(t, err) + require.Equal(t, int64(1), job.ID) + + err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { + m := meta.NewMeta(txn) + jobs, err := ddl.GetLastNHistoryDDLJobs(m, 2) + require.NoError(t, err) + require.Equal(t, 2, len(jobs)) + return nil + }) + + require.NoError(t, err) + + err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { + m := meta.NewMeta(txn) + jobs, err := ddl.GetAllHistoryDDLJobs(m) + require.NoError(t, err) + ddlHistoryJobCount = len(jobs) + return nil + }) + + require.NoError(t, err) + + err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { + m := meta.NewMeta(txn) + jobs, err := ddl.ScanHistoryDDLJobs(m, 2, 2) + require.NoError(t, err) + require.Equal(t, 2, len(jobs)) + require.Equal(t, int64(2), jobs[0].ID) + require.Equal(t, int64(1), jobs[1].ID) + return nil + }) + + require.NoError(t, err) + + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/history-ddl-jobs-limit", "return(128)")) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/history-ddl-jobs-limit")) + }() + + err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { + m := meta.NewMeta(txn) + jobs, err := ddl.ScanHistoryDDLJobs(m, 0, 0) + require.NoError(t, err) + if ddlHistoryJobCount <= 128 { + require.Equal(t, ddlHistoryJobCount, len(jobs)) + } else { + require.Equal(t, 128, len(jobs)) + } + require.True(t, len(jobs) > 2) + require.Equal(t, int64(2), jobs[ddlHistoryJobCount-2].ID) + require.Equal(t, int64(1), jobs[ddlHistoryJobCount-1].ID) + return nil + }) + + require.NoError(t, err) +} + +func TestScanHistoryDDLJobsWithErrorLimit(t *testing.T) { + var ( + m = &meta.Meta{} + startJobID int64 = 10 + limit = 0 + ) + + _, err := ddl.ScanHistoryDDLJobs(m, startJobID, limit) + require.ErrorContains(t, err, "when 'start_job_id' is specified, it must work with a 'limit'") +} diff --git a/server/http_handler.go b/server/http_handler.go index 99a88293fe0cf..4c4a75d2a0b8a 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -1296,10 +1296,20 @@ func (h tableHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } // ServeHTTP handles request of ddl jobs history. +<<<<<<< HEAD:server/http_handler.go func (h ddlHistoryJobHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { var jobID, limitID int var err error if jobValue := req.FormValue(qJobID); len(jobValue) > 0 { +======= +func (h DDLHistoryJobHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { + var ( + jobID = 0 + limitID = 0 + err error + ) + if jobValue := req.FormValue(handler.JobID); len(jobValue) > 0 { +>>>>>>> f2856e3521f (ddl: limit the count of getting ddlhistory jobs (#55590)):pkg/server/handler/tikvhandler/tikv_handler.go jobID, err = strconv.Atoi(jobValue) if err != nil { writeError(w, err) @@ -1316,8 +1326,14 @@ func (h ddlHistoryJobHandler) ServeHTTP(w http.ResponseWriter, req *http.Request writeError(w, err) return } +<<<<<<< HEAD:server/http_handler.go if limitID < 1 { writeError(w, errors.New("ddl history limit must be greater than 0")) +======= + if limitID < 1 || limitID > ddl.DefNumGetDDLHistoryJobs { + handler.WriteError(w, + errors.Errorf("ddl history limit must be greater than 0 and less than or equal to %v", ddl.DefNumGetDDLHistoryJobs)) +>>>>>>> f2856e3521f (ddl: limit the count of getting ddlhistory jobs (#55590)):pkg/server/handler/tikvhandler/tikv_handler.go return } } @@ -1337,11 +1353,7 @@ func (h ddlHistoryJobHandler) getHistoryDDL(jobID, limit int) (jobs []*model.Job } txnMeta := meta.NewMeta(txn) - if jobID == 0 && limit == 0 { - jobs, err = ddl.GetAllHistoryDDLJobs(txnMeta) - } else { - jobs, err = ddl.ScanHistoryDDLJobs(txnMeta, int64(jobID), limit) - } + jobs, err = ddl.ScanHistoryDDLJobs(txnMeta, int64(jobID), limit) if err != nil { return nil, errors.Trace(err) } diff --git a/server/http_handler_test.go b/server/http_handler_test.go index de9cdf655943b..dfa40e4ac9b54 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -16,6 +16,7 @@ package server import ( "bytes" + "cmp" "context" "crypto/tls" "crypto/x509" @@ -989,6 +990,11 @@ func TestAllHistory(t *testing.T) { data, err := ddl.GetAllHistoryDDLJobs(txnMeta) require.NoError(t, err) err = decoder.Decode(&jobs) + require.True(t, len(jobs) < ddl.DefNumGetDDLHistoryJobs) + // sort job. + slices.SortFunc(jobs, func(i, j *model.Job) int { + return cmp.Compare(i.ID, j.ID) + }) require.NoError(t, err) require.NoError(t, resp.Body.Close())