Skip to content

Commit

Permalink
ddl: add slow operations log to add indices, make trouble shooting ea…
Browse files Browse the repository at this point in the history
…sier. (#7060) (#7083)
  • Loading branch information
winkyao authored Jul 19, 2018
1 parent 8438bad commit f2e84fc
Show file tree
Hide file tree
Showing 5 changed files with 48 additions and 0 deletions.
19 changes: 19 additions & 0 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package ddl
import (
"context"
"math"
"sync/atomic"
"time"

"github.com/juju/errors"
Expand All @@ -26,6 +27,7 @@ import (
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
Expand Down Expand Up @@ -504,8 +506,13 @@ func (w *addIndexWorker) fetchRowColVals(txn kv.Transaction, taskRange reorgInde
w.idxRecords = w.idxRecords[:0]
startTime := time.Now()
handleOutOfRange := false
oprStartTime := time.Now()
err := iterateSnapshotRows(w.sessCtx.GetStore(), w.table, txn.StartTS(), taskRange.startHandle,
func(handle int64, recordKey kv.Key, rawRow []byte) (bool, error) {
oprEndTime := time.Now()
w.logSlowOperations(oprEndTime.Sub(oprStartTime), "iterateSnapshotRows in fetchRowColVals", 0)
oprStartTime = oprEndTime

if !taskRange.endIncluded {
handleOutOfRange = handle >= taskRange.endHandle
} else {
Expand Down Expand Up @@ -534,13 +541,24 @@ func (w *addIndexWorker) fetchRowColVals(txn kv.Transaction, taskRange reorgInde
return w.idxRecords, handleOutOfRange, errors.Trace(err)
}

func (w *addIndexWorker) logSlowOperations(elapsed time.Duration, slowMsg string, threshold uint32) {
if threshold == 0 {
threshold = atomic.LoadUint32(&variable.DDLSlowOprThreshold)
}

if elapsed >= time.Duration(threshold)*time.Millisecond {
log.Infof("[ddl-reorg][SLOW-OPERATIONS] elapsed time: %v, message: %v", elapsed, slowMsg)
}
}

// backfillIndexInTxn will backfill table index in a transaction, lock corresponding rowKey, if the value of rowKey is changed,
// indicate that index columns values may changed, index is not allowed to be added, so the txn will rollback and retry.
// backfillIndexInTxn will add w.batchCnt indices once, default value of w.batchCnt is 128.
// TODO: make w.batchCnt can be modified by system variable.
func (w *addIndexWorker) backfillIndexInTxn(handleRange reorgIndexTask) (nextHandle int64, addedCount, scanCount int, errInTxn error) {
addedCount = 0
scanCount = 0
oprStartTime := time.Now()
errInTxn = kv.RunInNewTxn(w.sessCtx.GetStore(), true, func(txn kv.Transaction) error {
txn.SetOption(kv.Priority, kv.PriorityLow)
idxRecords, handleOutOfRange, err := w.fetchRowColVals(txn, handleRange)
Expand Down Expand Up @@ -576,6 +594,7 @@ func (w *addIndexWorker) backfillIndexInTxn(handleRange reorgIndexTask) (nextHan
}
return nil
})
w.logSlowOperations(time.Since(oprStartTime), "backfillIndexInTxn", 3000)

return
}
Expand Down
8 changes: 8 additions & 0 deletions docs/tidb_http_api.md
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,14 @@ timezone.*
curl -x POST -d "log_level=info" http://{TiDBIP}:10080/settings
```

1. Change TiDB DDL slow log threshold

The unit is millisecond.

```shell
curl -x POST -d "ddl_slow_threshold=300" http://{TiDBIP}:10080/settings
```

1. 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
Expand Down
11 changes: 11 additions & 0 deletions server/http_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -548,6 +548,17 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) {
return
}
}
if ddlSlowThreshold := req.Form.Get("ddl_slow_threshold"); ddlSlowThreshold != "" {
threshold, err1 := strconv.Atoi(ddlSlowThreshold)
if err1 != nil {
writeError(w, err1)
return
}
if threshold > 0 {
atomic.StoreUint32(&variable.DDLSlowOprThreshold, uint32(threshold))
}
}

} else {
writeData(w, config.GetGlobalConfig())
}
Expand Down
8 changes: 8 additions & 0 deletions server/http_handler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -605,4 +605,12 @@ func (ts *HTTPHandlerTestSuite) TestPostSettings(c *C) {
c.Assert(atomic.LoadUint32(&variable.ProcessGeneralLog), Equals, uint32(0))
c.Assert(log.GetLevel(), Equals, log.InfoLevel)
c.Assert(config.GetGlobalConfig().Log.Level, Equals, "info")

// test ddl_slow_threshold
form = make(url.Values)
form.Set("ddl_slow_threshold", "200")
resp, err = http.PostForm("http://127.0.0.1:10090/settings", form)
c.Assert(err, IsNil)
c.Assert(resp.StatusCode, Equals, http.StatusOK)
c.Assert(atomic.LoadUint32(&variable.DDLSlowOprThreshold), Equals, uint32(200))
}
2 changes: 2 additions & 0 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,4 +190,6 @@ const (
// Process global variables.
var (
ProcessGeneralLog uint32
// DDLSlowOprThreshold is the threshold for ddl slow operations, uint is millisecond.
DDLSlowOprThreshold uint32 = 300
)

0 comments on commit f2e84fc

Please sign in to comment.