Skip to content

Commit

Permalink
dumpling: optimize dumpling log level (#29341)
Browse files Browse the repository at this point in the history
  • Loading branch information
lichunzhu authored Nov 4, 2021
1 parent 5016767 commit fb5b3a0
Show file tree
Hide file tree
Showing 25 changed files with 116 additions and 79 deletions.
3 changes: 2 additions & 1 deletion dumpling/export/block_allow_list_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,8 +8,9 @@ import (

"github.com/pingcap/tidb-tools/pkg/filter"
tf "github.com/pingcap/tidb-tools/pkg/table-filter"
tcontext "github.com/pingcap/tidb/dumpling/context"
"github.com/stretchr/testify/require"

tcontext "github.com/pingcap/tidb/dumpling/context"
)

func TestFilterTables(t *testing.T) {
Expand Down
5 changes: 3 additions & 2 deletions dumpling/export/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,12 @@ import (
"github.com/pingcap/errors"
filter "github.com/pingcap/tidb-tools/pkg/table-filter"
"github.com/pingcap/tidb-tools/pkg/utils"
"github.com/pingcap/tidb/br/pkg/storage"
tcontext "github.com/pingcap/tidb/dumpling/context"
"github.com/prometheus/client_golang/prometheus"
"github.com/spf13/pflag"
"go.uber.org/zap"

"github.com/pingcap/tidb/br/pkg/storage"
tcontext "github.com/pingcap/tidb/dumpling/context"
)

const (
Expand Down
3 changes: 2 additions & 1 deletion dumpling/export/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5,8 +5,9 @@ package export
import (
"testing"

tcontext "github.com/pingcap/tidb/dumpling/context"
"github.com/stretchr/testify/require"

tcontext "github.com/pingcap/tidb/dumpling/context"
)

func TestCreateExternalStorage(t *testing.T) {
Expand Down
1 change: 1 addition & 0 deletions dumpling/export/consistency.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@ import (
"database/sql"

"github.com/pingcap/errors"

"github.com/pingcap/tidb/br/pkg/utils"
tcontext "github.com/pingcap/tidb/dumpling/context"
)
Expand Down
6 changes: 3 additions & 3 deletions dumpling/export/consistency_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,11 +7,11 @@ import (
"errors"
"testing"

tcontext "github.com/pingcap/tidb/dumpling/context"
"github.com/stretchr/testify/require"

"github.com/DATA-DOG/go-sqlmock"
"github.com/go-sql-driver/mysql"
"github.com/stretchr/testify/require"

tcontext "github.com/pingcap/tidb/dumpling/context"
)

func TestConsistencyController(t *testing.T) {
Expand Down
76 changes: 48 additions & 28 deletions dumpling/export/dump.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,10 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
pclog "github.com/pingcap/log"
pd "github.com/tikv/pd/client"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"

"github.com/pingcap/tidb/br/pkg/storage"
"github.com/pingcap/tidb/br/pkg/summary"
"github.com/pingcap/tidb/dumpling/cli"
Expand All @@ -27,13 +31,12 @@ import (
"github.com/pingcap/tidb/store/helper"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/util/codec"
pd "github.com/tikv/pd/client"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
)

var openDBFunc = sql.Open

var emptyHandleValsErr = errors.New("empty handleVals for TiDB table")

// Dumper is the dump progress structure
type Dumper struct {
tctx *tcontext.Context
Expand Down Expand Up @@ -122,7 +125,7 @@ func (d *Dumper) Dump() (dumpErr error) {
defer func() {
err = conCtrl.TearDown(tctx)
if err != nil {
tctx.L().Error("fail to tear down consistency controller", zap.Error(err))
tctx.L().Warn("fail to tear down consistency controller", zap.Error(err))
}
}()

Expand Down Expand Up @@ -150,7 +153,7 @@ func (d *Dumper) Dump() (dumpErr error) {
}
}
if err = d.renewSelectTableRegionFuncForLowerTiDB(tctx); err != nil {
tctx.L().Error("fail to update select table region info for TiDB", zap.Error(err))
tctx.L().Info("cannot update select table region info for TiDB", zap.Error(err))
}

rebuildConn := func(conn *sql.Conn) (*sql.Conn, error) {
Expand Down Expand Up @@ -468,9 +471,10 @@ func (d *Dumper) concurrentDumpTable(tctx *tcontext.Context, conn *sql.Conn, met
// don't retry on context error and successful tasks
if err2 := errors.Cause(err); err2 == nil || err2 == context.DeadlineExceeded || err2 == context.Canceled {
return err
} else if err2 != emptyHandleValsErr {
tctx.L().Info("fallback to concurrent dump tables using rows due to some problem. This won't influence the whole dump process",
zap.String("database", db), zap.String("table", tbl), log.ShortError(err))
}
tctx.L().Warn("fallback to concurrent dump tables using rows due to tidb error",
zap.String("database", db), zap.String("table", tbl), log.ShortError(err))
}

orderByClause, err := buildOrderByClause(conf, conn, db, tbl, meta.HasImplicitRowID())
Expand All @@ -481,7 +485,7 @@ func (d *Dumper) concurrentDumpTable(tctx *tcontext.Context, conn *sql.Conn, met
field, err := pickupPossibleField(meta, conn)
if err != nil || field == "" {
// skip split chunk logic if not found proper field
tctx.L().Warn("fallback to sequential dump due to no proper field",
tctx.L().Info("fallback to sequential dump due to no proper field. This won't influence the whole dump process",
zap.String("database", db), zap.String("table", tbl), log.ShortError(err))
return d.dumpWholeTableDirectly(tctx, meta, taskChan, "", orderByClause, 0, 1)
}
Expand All @@ -493,7 +497,7 @@ func (d *Dumper) concurrentDumpTable(tctx *tcontext.Context, conn *sql.Conn, met
zap.Uint64("estimateCount", count))
if count < conf.Rows {
// skip chunk logic if estimates are low
tctx.L().Warn("skip concurrent dump due to estimate count < rows",
tctx.L().Info("fallback to sequential dump due to estimate count < rows. This won't influence the whole dump process",
zap.Uint64("estimate count", count),
zap.Uint64("conf.rows", conf.Rows),
zap.String("database", db),
Expand All @@ -503,7 +507,9 @@ func (d *Dumper) concurrentDumpTable(tctx *tcontext.Context, conn *sql.Conn, met

min, max, err := d.selectMinAndMaxIntValue(conn, db, tbl, field)
if err != nil {
return err
tctx.L().Info("fallback to sequential dump due to cannot get bounding values. This won't influence the whole dump process",
log.ShortError(err))
return d.dumpWholeTableDirectly(tctx, meta, taskChan, "", orderByClause, 0, 1)
}
tctx.L().Debug("get int bounding values",
zap.String("lower", min.String()),
Expand Down Expand Up @@ -571,13 +577,11 @@ func (d *Dumper) selectMinAndMaxIntValue(conn *sql.Conn, db, tbl, field string)
row := conn.QueryRowContext(tctx, query)
err := row.Scan(&smin, &smax)
if err != nil {
tctx.L().Error("split chunks - get max min failed", zap.String("query", query), zap.Error(err))
return zero, zero, errors.Trace(err)
return zero, zero, errors.Annotatef(err, "can't get min/max values to split chunks, query: %s", query)
}
if !smax.Valid || !smin.Valid {
// found no data
tctx.L().Warn("no data to dump", zap.String("database", db), zap.String("table", tbl))
return zero, zero, nil
return zero, zero, errors.Errorf("no invalid min/max value found in query %s", query)
}

max := new(big.Int)
Expand Down Expand Up @@ -666,7 +670,7 @@ func (d *Dumper) sendConcurrentDumpTiDBTasks(tctx *tcontext.Context,
if len(handleVals) == 0 {
if partition == "" {
// return error to make outside function try using rows method to dump data
return errors.Errorf("empty handleVals for TiDB table `%s`.`%s`", escapeString(db), escapeString(tbl))
return errors.Annotatef(emptyHandleValsErr, "table: `%s`.`%s`", escapeString(db), escapeString(tbl))
}
return d.dumpWholeTableDirectly(tctx, meta, taskChan, partition, buildOrderByClauseString(handleColNames), startChunkIdx, totalChunk)
}
Expand Down Expand Up @@ -797,7 +801,7 @@ func selectTiDBTableRegion(tctx *tcontext.Context, conn *sql.Conn, meta TableMet
}
pkVal, err2 := extractTiDBRowIDFromDecodedKey(tidbRowID, decodedKey.String)
if err2 != nil {
logger.Debug("fail to extract pkVal from decoded start key",
logger.Debug("cannot extract pkVal from decoded start key",
zap.Int("rowID", rowID), zap.String("startKey", startKey.String), zap.String("decodedKey", decodedKey.String), zap.Error(err2))
} else {
pkVals = append(pkVals, []string{pkVal})
Expand Down Expand Up @@ -1026,7 +1030,7 @@ func startHTTPService(d *Dumper) error {
go func() {
err := startDumplingService(d.tctx, conf.StatusAddr)
if err != nil {
d.L().Warn("meet error when stopping dumpling http service", log.ShortError(err))
d.L().Info("meet error when stopping dumpling http service", log.ShortError(err))
}
}()
}
Expand Down Expand Up @@ -1083,17 +1087,17 @@ func tidbSetPDClientForGC(d *Dumper) error {
}
pdAddrs, err := GetPdAddrs(tctx, pool)
if err != nil {
tctx.L().Warn("meet error while fetching pd addrs", log.ShortError(err))
tctx.L().Info("meet some problem while fetching pd addrs. This won't affect dump process", log.ShortError(err))
return nil
}
if len(pdAddrs) > 0 {
doPdGC, err := checkSameCluster(tctx, pool, pdAddrs)
if err != nil {
tctx.L().Warn("meet error while check whether fetched pd addr and TiDB belong to one cluster", log.ShortError(err), zap.Strings("pdAddrs", pdAddrs))
tctx.L().Info("meet error while check whether fetched pd addr and TiDB belong to one cluster. This won't affect dump process", log.ShortError(err), zap.Strings("pdAddrs", pdAddrs))
} else if doPdGC {
pdClient, err := pd.NewClientWithContext(tctx, pdAddrs, pd.SecurityOption{})
if err != nil {
tctx.L().Warn("create pd client to control GC failed", log.ShortError(err), zap.Strings("pdAddrs", pdAddrs))
tctx.L().Info("create pd client to control GC failed. This won't affect dump process", log.ShortError(err), zap.Strings("pdAddrs", pdAddrs))
}
d.tidbPDClientForGC = pdClient
}
Expand All @@ -1106,20 +1110,28 @@ func tidbGetSnapshot(d *Dumper) error {
conf, doPdGC := d.conf, d.tidbPDClientForGC != nil
consistency := conf.Consistency
pool, tctx := d.dbHandle, d.tctx
if conf.Snapshot == "" && (doPdGC || consistency == "snapshot") {
snapshotConsistency := consistency == "snapshot"
if conf.Snapshot == "" && (doPdGC || snapshotConsistency) {
conn, err := pool.Conn(tctx)
if err != nil {
tctx.L().Warn("cannot get snapshot from TiDB", log.ShortError(err))
return nil
tctx.L().Warn("fail to open connection to get snapshot from TiDB", log.ShortError(err))
// for consistency snapshot, we must get a snapshot here, or we will dump inconsistent data, but for other consistency we can ignore this error.
if !snapshotConsistency {
err = nil
}
return err
}
snapshot, err := getSnapshot(conn)
_ = conn.Close()
if err != nil {
tctx.L().Warn("cannot get snapshot from TiDB", log.ShortError(err))
return nil
tctx.L().Warn("fail to get snapshot from TiDB", log.ShortError(err))
// for consistency snapshot, we must get a snapshot here, or we will dump inconsistent data, but for other consistency we can ignore this error.
if !snapshotConsistency {
err = nil
}
return err
}
conf.Snapshot = snapshot
return nil
}
return nil
}
Expand Down Expand Up @@ -1190,7 +1202,7 @@ func setSessionParam(d *Dumper) error {
if consistency == consistencyTypeSnapshot {
conf.ServerInfo.HasTiKV, err = CheckTiDBWithTiKV(pool)
if err != nil {
d.L().Warn("fail to check whether TiDB has TiKV", log.ShortError(err))
d.L().Info("cannot check whether TiDB has TiKV, will apply tidb_snapshot by default. This won't affect dump process", log.ShortError(err))
}
if conf.ServerInfo.HasTiKV {
sessionParam["tidb_snapshot"] = snapshot
Expand All @@ -1211,6 +1223,14 @@ func (d *Dumper) renewSelectTableRegionFuncForLowerTiDB(tctx *tcontext.Context)
tctx.L().Debug("no need to build region info because database is not TiDB 3.x")
return nil
}
// for TiDB v3.0+, the original selectTiDBTableRegionFunc will always fail,
// because TiDB v3.0 doesn't have `tidb_decode_key` function nor `DB_NAME`,`TABLE_NAME` columns in `INFORMATION_SCHEMA.TIKV_REGION_STATUS`.
// reference: https://github.com/pingcap/tidb/blob/c497d5c/dumpling/export/dump.go#L775
// To avoid this function continuously returning errors and confusing users because we fail to init this function at first,
// selectTiDBTableRegionFunc is set to always return an ignorable error at first.
d.selectTiDBTableRegionFunc = func(_ *tcontext.Context, _ *sql.Conn, meta TableMeta) (pkFields []string, pkVals [][]string, err error) {
return nil, nil, errors.Annotatef(emptyHandleValsErr, "table: `%s`.`%s`", escapeString(meta.DatabaseName()), escapeString(meta.TableName()))
}
dbHandle, err := openDBFunc("mysql", conf.GetDSN(""))
if err != nil {
return errors.Trace(err)
Expand Down Expand Up @@ -1254,7 +1274,7 @@ func (d *Dumper) renewSelectTableRegionFuncForLowerTiDB(tctx *tcontext.Context)
// Try to decode it as a record key.
tableID, handle, err := tablecodec.DecodeRecordKey(key)
if err != nil {
d.L().Debug("fail to decode region start key", zap.Error(err), zap.String("key", region.StartKey), zap.Int64("tableID", tableID))
d.L().Debug("cannot decode region start key", zap.Error(err), zap.String("key", region.StartKey), zap.Int64("tableID", tableID))
continue
}
if handle.IsInt() {
Expand Down
3 changes: 2 additions & 1 deletion dumpling/export/dump_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,9 +10,10 @@ import (

"github.com/DATA-DOG/go-sqlmock"
"github.com/pingcap/errors"
tcontext "github.com/pingcap/tidb/dumpling/context"
"github.com/stretchr/testify/require"
"golang.org/x/sync/errgroup"

tcontext "github.com/pingcap/tidb/dumpling/context"
)

func TestDumpBlock(t *testing.T) {
Expand Down
7 changes: 4 additions & 3 deletions dumpling/export/http_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,10 +10,11 @@ import (
"time"

"github.com/pingcap/errors"
tcontext "github.com/pingcap/tidb/dumpling/context"
"github.com/pingcap/tidb/dumpling/log"
"github.com/prometheus/client_golang/prometheus/promhttp"
"github.com/soheilhy/cmux"

tcontext "github.com/pingcap/tidb/dumpling/context"
"github.com/pingcap/tidb/dumpling/log"
)

var cmuxReadTimeout = 10 * time.Second
Expand All @@ -34,7 +35,7 @@ func startHTTPServer(tctx *tcontext.Context, lis net.Listener) {
err := httpServer.Serve(lis)
err = errors.Cause(err)
if err != nil && !isErrNetClosing(err) && err != http.ErrServerClosed {
tctx.L().Warn("dumpling http handler return with error", log.ShortError(err))
tctx.L().Info("dumpling http handler return with error", log.ShortError(err))
}
}

Expand Down
1 change: 1 addition & 0 deletions dumpling/export/ir.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ import (
"strings"

"github.com/pingcap/errors"

tcontext "github.com/pingcap/tidb/dumpling/context"
)

Expand Down
3 changes: 2 additions & 1 deletion dumpling/export/ir_impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,8 +7,9 @@ import (
"strings"

"github.com/pingcap/errors"
tcontext "github.com/pingcap/tidb/dumpling/context"
"go.uber.org/zap"

tcontext "github.com/pingcap/tidb/dumpling/context"
)

// rowIter implements the SQLRowIter interface.
Expand Down
3 changes: 2 additions & 1 deletion dumpling/export/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@ import (
"os"
"testing"

"github.com/pingcap/tidb/dumpling/log"
"github.com/prometheus/client_golang/prometheus"
"github.com/stretchr/testify/require"
"go.uber.org/goleak"

"github.com/pingcap/tidb/dumpling/log"
)

var appLogger log.Logger
Expand Down
5 changes: 3 additions & 2 deletions dumpling/export/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,9 +11,10 @@ import (
"time"

"github.com/pingcap/errors"
"go.uber.org/zap"

"github.com/pingcap/tidb/br/pkg/storage"
tcontext "github.com/pingcap/tidb/dumpling/context"
"go.uber.org/zap"
)

type globalMetadata struct {
Expand Down Expand Up @@ -135,7 +136,7 @@ func recordGlobalMetaData(tctx *tcontext.Context, db *sql.Conn, buffer *bytes.Bu
var gtidSet string
err = db.QueryRowContext(context.Background(), "SELECT @@global.gtid_binlog_pos").Scan(&gtidSet)
if err != nil {
tctx.L().Error("fail to get gtid for mariaDB", zap.Error(err))
tctx.L().Warn("fail to get gtid for mariaDB", zap.Error(err))
}

if logFile != "" {
Expand Down
2 changes: 1 addition & 1 deletion dumpling/export/metadata_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,9 +9,9 @@ import (
"os"
"testing"

"github.com/DATA-DOG/go-sqlmock"
"github.com/stretchr/testify/require"

"github.com/DATA-DOG/go-sqlmock"
"github.com/pingcap/tidb/br/pkg/storage"
tcontext "github.com/pingcap/tidb/dumpling/context"
)
Expand Down
1 change: 1 addition & 0 deletions dumpling/export/prepare.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import (
"text/template"

"github.com/pingcap/errors"

tcontext "github.com/pingcap/tidb/dumpling/context"
)

Expand Down
4 changes: 2 additions & 2 deletions dumpling/export/prepare_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,10 +8,10 @@ import (
"strings"
"testing"

tcontext "github.com/pingcap/tidb/dumpling/context"
"github.com/DATA-DOG/go-sqlmock"
"github.com/stretchr/testify/require"

"github.com/DATA-DOG/go-sqlmock"
tcontext "github.com/pingcap/tidb/dumpling/context"
)

func TestPrepareDumpingDatabases(t *testing.T) {
Expand Down
Loading

0 comments on commit fb5b3a0

Please sign in to comment.