Skip to content

Commit

Permalink
*: log the caller of "terror.Log()" (pingcap#4729)
Browse files Browse the repository at this point in the history
  • Loading branch information
zz-jason authored and hanfei1991 committed Oct 10, 2017
1 parent a8db31d commit 3e1d036
Show file tree
Hide file tree
Showing 48 changed files with 121 additions and 114 deletions.
7 changes: 4 additions & 3 deletions cmd/benchkv/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
_ "github.com/go-sql-driver/mysql"

log "github.com/Sirupsen/logrus"
"github.com/juju/errors"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/terror"
Expand Down Expand Up @@ -78,7 +79,7 @@ func Init() {

go func() {
err1 := http.ListenAndServe(":9191", nil)
terror.Log(err1)
terror.Log(errors.Trace(err1))
}()
}

Expand All @@ -100,7 +101,7 @@ func batchRW(value []byte) {
}
key := fmt.Sprintf("key_%d", k)
err = txn.Set([]byte(key), value)
terror.Log(err)
terror.Log(errors.Trace(err))
err = txn.Commit()
if err != nil {
txnRolledbackCounter.WithLabelValues("txn").Inc()
Expand All @@ -127,7 +128,7 @@ func main() {

defer terror.Call(resp.Body.Close)
text, err1 := ioutil.ReadAll(resp.Body)
terror.Log(err1)
terror.Log(errors.Trace(err1))

fmt.Println(string(text))

Expand Down
2 changes: 1 addition & 1 deletion cmd/benchraw/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ func main() {
log.SetLevel(log.WarnLevel)
go func() {
err := http.ListenAndServe(":9191", nil)
terror.Log(err)
terror.Log(errors.Trace(err))
}()

value := make([]byte, *valueSize)
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -330,7 +330,7 @@ func (d *ddl) Stop() error {
func (d *ddl) start(ctx goctx.Context) {
d.quitCh = make(chan struct{})
err := d.ownerManager.CampaignOwner(ctx)
terror.Log(err)
terror.Log(errors.Trace(err))
d.wait.Add(1)
go d.onDDLWorker()

Expand Down
2 changes: 1 addition & 1 deletion ddl/delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,7 @@ func (dr *delRange) startEmulator() {
return
}
err := dr.doDelRangeWork()
terror.Log(err)
terror.Log(errors.Trace(err))
}
}

Expand Down
4 changes: 2 additions & 2 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -354,10 +354,10 @@ func (do *Domain) loadSchemaInLoop(lease time.Duration) {

// Close closes the Domain and release its resource.
func (do *Domain) Close() {
terror.Log(do.ddl.Stop())
terror.Log(errors.Trace(do.ddl.Stop()))
close(do.exit)
if do.etcdClient != nil {
terror.Log(do.etcdClient.Close())
terror.Log(errors.Trace(do.etcdClient.Close()))
}
do.sysSessionPool.Close()
do.wg.Wait()
Expand Down
2 changes: 1 addition & 1 deletion executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -184,7 +184,7 @@ func (a *statement) handleNoDelayExecutor(e Executor, ctx context.Context, pi pr
if pi != nil {
pi.SetProcessInfo("")
}
terror.Log(e.Close())
terror.Log(errors.Trace(e.Close()))
a.logSlowQuery()
}()
for {
Expand Down
6 changes: 3 additions & 3 deletions executor/distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -515,7 +515,7 @@ func (e *XSelectIndexExec) nextForSingleRead() (Row, error) {
}
if rowData == nil {
// Finish current partial result and get the next one.
terror.Log(e.partialResult.Close())
terror.Log(errors.Trace(e.partialResult.Close()))
e.partialResult = nil
continue
}
Expand Down Expand Up @@ -646,7 +646,7 @@ func (e *XSelectIndexExec) fetchHandles(idxResult distsql.SelectResult, ch chan<
defer func() {
close(ch)
close(workCh)
terror.Log(idxResult.Close())
terror.Log(errors.Trace(idxResult.Close()))
}()

lookupConcurrencyLimit := e.ctx.GetSessionVars().IndexLookupConcurrency
Expand Down Expand Up @@ -1040,7 +1040,7 @@ func (e *XSelectTableExec) Next() (Row, error) {
}
if rowData == nil {
// Finish the current partial result and get the next one.
terror.Log(e.partialResult.Close())
terror.Log(errors.Trace(e.partialResult.Close()))
e.partialResult = nil
continue
}
Expand Down
4 changes: 2 additions & 2 deletions executor/join.go
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,7 @@ func (e *HashJoinExec) fetchBigExec() {
for _, cn := range e.bigTableResultCh {
close(cn)
}
terror.Log(e.bigExec.Close())
terror.Log(errors.Trace(e.bigExec.Close()))
e.wg.Done()
}()
curBatchSize := 1
Expand Down Expand Up @@ -217,7 +217,7 @@ func (e *HashJoinExec) prepare() error {
return errors.Trace(err)
}
if row == nil {
terror.Log(e.smallExec.Close())
terror.Log(errors.Trace(e.smallExec.Close()))
break
}

Expand Down
2 changes: 1 addition & 1 deletion executor/merge_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -272,7 +272,7 @@ func (e *MergeJoinExec) Close() error {

lErr := e.leftRowBlock.reader.Close()
if lErr != nil {
terror.Log(e.rightRowBlock.reader.Close())
terror.Log(errors.Trace(e.rightRowBlock.reader.Close()))
return errors.Trace(lErr)
}
rErr := e.rightRowBlock.reader.Close()
Expand Down
4 changes: 2 additions & 2 deletions executor/new_distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ func (e *TableReaderExecutor) Next() (Row, error) {
if rowData == nil {
// Finish the current partial result and get the next one.
err = e.partialResult.Close()
terror.Log(err)
terror.Log(errors.Trace(err))
e.partialResult = nil
continue
}
Expand Down Expand Up @@ -230,7 +230,7 @@ func (e *IndexReaderExecutor) Next() (Row, error) {
if rowData == nil {
// Finish the current partial result and get the next one.
err = e.partialResult.Close()
terror.Log(err)
terror.Log(errors.Trace(err))
e.partialResult = nil
continue
}
Expand Down
2 changes: 1 addition & 1 deletion executor/set.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,7 @@ func (e *SetExecutor) executeSet() error {
return errors.Trace(err)
}
valStr, err := value.ToString()
terror.Log(err)
terror.Log(errors.Trace(err))
log.Infof("[%d] set system variable %s = %s", sessionVars.ConnectionID, name, valStr)
}

Expand Down
4 changes: 2 additions & 2 deletions expression/aggregation/avg.go
Original file line number Diff line number Diff line change
Expand Up @@ -91,9 +91,9 @@ func (af *avgFunction) GetResult(ctx *AggEvaluateContext) (d types.Datum) {
y := types.NewDecFromInt(ctx.Count)
to := new(types.MyDecimal)
err := types.DecimalDiv(x, y, to, types.DivFracIncr)
terror.Log(err)
terror.Log(errors.Trace(err))
err = to.Round(to, ctx.Value.Frac()+types.DivFracIncr, types.ModeHalfEven)
terror.Log(err)
terror.Log(errors.Trace(err))
d.SetMysqlDecimal(to)
}
return
Expand Down
2 changes: 1 addition & 1 deletion expression/builtin_cast.go
Original file line number Diff line number Diff line change
Expand Up @@ -1315,7 +1315,7 @@ func BuildCastFunction(ctx context.Context, expr Expression, tp *types.FieldType
fc = &castAsStringFunctionClass{baseFunctionClass{ast.Cast, 1, 1}, tp}
}
f, err := fc.getFunction(ctx, []Expression{expr})
terror.Log(err)
terror.Log(errors.Trace(err))
res = &ScalarFunction{
FuncName: model.NewCIStr(ast.Cast),
RetType: tp,
Expand Down
4 changes: 2 additions & 2 deletions expression/builtin_time.go
Original file line number Diff line number Diff line change
Expand Up @@ -2897,7 +2897,7 @@ func (b *builtinUnixTimestampCurrentSig) evalInt(row []types.Datum) (int64, bool
return 0, true, errors.Trace(err)
}
intVal, err := dec.ToInt()
terror.Log(err)
terror.Log(errors.Trace(err))
return intVal, false, nil
}

Expand All @@ -2922,7 +2922,7 @@ func (b *builtinUnixTimestampIntSig) evalInt(row []types.Datum) (int64, bool, er
return 0, true, errors.Trace(err)
}
intVal, err := dec.ToInt()
terror.Log(err)
terror.Log(errors.Trace(err))
return intVal, false, nil
}

Expand Down
2 changes: 1 addition & 1 deletion expression/constant.go
Original file line number Diff line number Diff line change
Expand Up @@ -172,7 +172,7 @@ func (c *Constant) Decorrelate(_ *Schema) Expression {
// HashCode implements Expression interface.
func (c *Constant) HashCode() []byte {
bytes, err := codec.EncodeValue(nil, c.Value)
terror.Log(err)
terror.Log(errors.Trace(err))
return bytes
}

Expand Down
3 changes: 2 additions & 1 deletion expression/constant_propagation.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ package expression

import (
log "github.com/Sirupsen/logrus"
"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/mysql"
Expand Down Expand Up @@ -176,7 +177,7 @@ func (s *propagateConstantSolver) pickNewEQConds(visited []bool) (retMapper map[
if col == nil {
if con, ok = cond.(*Constant); ok {
value, err := EvalBool([]Expression{con}, nil, s.ctx)
terror.Log(err)
terror.Log(errors.Trace(err))
if !value {
s.setConds2ConstFalse()
return nil
Expand Down
3 changes: 2 additions & 1 deletion expression/expr_to_pb.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"time"

log "github.com/Sirupsen/logrus"
"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/mysql"
Expand Down Expand Up @@ -125,7 +126,7 @@ func (pc PbConverter) constantToPBExpr(con *Constant) *tipb.Expr {
t := d.GetMysqlTime()
if t.Type == mysql.TypeTimestamp && loc != time.UTC {
err := t.ConvertTimeZone(loc, time.UTC)
terror.Log(err)
terror.Log(errors.Trace(err))
}
v, err := t.ToPackedUint()
if err != nil {
Expand Down
2 changes: 1 addition & 1 deletion expression/expression.go
Original file line number Diff line number Diff line change
Expand Up @@ -293,7 +293,7 @@ func ColumnInfos2ColumnsWithDBName(dbName, tblName model.CIStr, colInfos []*mode
func NewValuesFunc(offset int, retTp *types.FieldType, ctx context.Context) *ScalarFunction {
fc := &valuesFunctionClass{baseFunctionClass{ast.Values, 0, 0}, offset, retTp}
bt, err := fc.getFunction(ctx, nil)
terror.Log(err)
terror.Log(errors.Trace(err))
return &ScalarFunction{
FuncName: model.NewCIStr(ast.Values),
RetType: retTp,
Expand Down
2 changes: 1 addition & 1 deletion expression/helper.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ func GetTimeValue(ctx context.Context, v interface{}, tp byte, fsp int) (d types
}
} else if upperX == types.ZeroDatetimeStr {
value, err = types.ParseTimeFromNum(0, tp, fsp)
terror.Log(err)
terror.Log(errors.Trace(err))
} else {
value, err = types.ParseTime(x, tp, fsp)
if err != nil {
Expand Down
6 changes: 3 additions & 3 deletions expression/scalar_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ func NewFunction(ctx context.Context, funcName string, retType *types.FieldType,
// NewFunctionInternal is similar to NewFunction, but do not returns error, should only be used internally.
func NewFunctionInternal(ctx context.Context, funcName string, retType *types.FieldType, args ...Expression) Expression {
expr, err := NewFunction(ctx, funcName, retType, args...)
terror.Log(err)
terror.Log(errors.Trace(err))
return expr
}

Expand Down Expand Up @@ -257,14 +257,14 @@ func (sf *ScalarFunction) EvalJSON(row []types.Datum, sc *variable.StatementCont
func (sf *ScalarFunction) HashCode() []byte {
v := make([]types.Datum, 0, len(sf.GetArgs())+1)
bytes, err := codec.EncodeValue(nil, types.NewStringDatum(sf.FuncName.L))
terror.Log(err)
terror.Log(errors.Trace(err))
v = append(v, types.NewBytesDatum(bytes))
for _, arg := range sf.GetArgs() {
v = append(v, types.NewBytesDatum(arg.HashCode()))
}
bytes = bytes[:0]
bytes, err = codec.EncodeValue(bytes, v...)
terror.Log(err)
terror.Log(errors.Trace(err))
return bytes
}

Expand Down
4 changes: 2 additions & 2 deletions expression/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -178,9 +178,9 @@ func timeZone2Duration(tz string) time.Duration {

i := strings.Index(tz, ":")
h, err := strconv.Atoi(tz[1:i])
terror.Log(err)
terror.Log(errors.Trace(err))
m, err := strconv.Atoi(tz[i+1:])
terror.Log(err)
terror.Log(errors.Trace(err))
return time.Duration(sign) * (time.Duration(h)*time.Hour + time.Duration(m)*time.Minute)
}

Expand Down
6 changes: 3 additions & 3 deletions kv/txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,20 +45,20 @@ func RunInNewTxn(store Storage, retryable bool, f func(txn Transaction) error) e
if retryable && IsRetryableError(err) {
log.Warnf("[kv] Retry txn %v original txn %v err %v", txn, originalTxnTS, err)
err1 := txn.Rollback()
terror.Log(err1)
terror.Log(errors.Trace(err1))
continue
}
if err != nil {
err1 := txn.Rollback()
terror.Log(err1)
terror.Log(errors.Trace(err1))
return errors.Trace(err)
}

err = txn.Commit()
if retryable && IsRetryableError(err) {
log.Warnf("[kv] Retry txn %v original txn %v err %v", txn, originalTxnTS, err)
err1 := txn.Rollback()
terror.Log(err1)
terror.Log(errors.Trace(err1))
BackOff(i)
continue
}
Expand Down
5 changes: 3 additions & 2 deletions plan/eliminate_projection.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
package plan

import (
"github.com/juju/errors"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/terror"
Expand Down Expand Up @@ -84,7 +85,7 @@ func doPhysicalProjectionElimination(p PhysicalPlan) PhysicalPlan {
}
child := p.Children()[0]
err := RemovePlan(p)
terror.Log(err)
terror.Log(errors.Trace(err))
return child.(PhysicalPlan)
}

Expand Down Expand Up @@ -166,7 +167,7 @@ func (pe *projectionEliminater) eliminate(p LogicalPlan, replace map[string]*exp
replace[string(col.HashCode())] = exprs[i].(*expression.Column)
}
err := RemovePlan(p)
terror.Log(err)
terror.Log(errors.Trace(err))
return child.(LogicalPlan)
}

Expand Down
2 changes: 1 addition & 1 deletion plan/physical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1237,7 +1237,7 @@ func (p *Selection) makeScanController() *physicalPlanInfo {
*col.Data = expression.One.Value
}
newCond, err := expression.SubstituteCorCol2Constant(cond)
terror.Log(err)
terror.Log(errors.Trace(err))
corColConds = append(corColConds, newCond)
}
if p.controllerStatus == controlTableScan {
Expand Down
4 changes: 2 additions & 2 deletions plan/plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -516,7 +516,7 @@ func (p *basePlan) ReplaceParent(parent, newPar Plan) error {
return nil
}
}
return SystemInternalErrorType.Gen("ReplaceParent Failed!")
return SystemInternalErrorType.Gen("ReplaceParent Failed: parent \"%s\" not found", parent.ExplainID())
}

// ReplaceChild means replace a child with another one.
Expand All @@ -527,7 +527,7 @@ func (p *basePlan) ReplaceChild(child, newChild Plan) error {
return nil
}
}
return SystemInternalErrorType.Gen("ReplaceChildren Failed!")
return SystemInternalErrorType.Gen("ReplaceChildren Failed: child \"%s\" not found", child.ExplainID())
}

// Parents implements Plan Parents interface.
Expand Down
Loading

0 comments on commit 3e1d036

Please sign in to comment.