diff --git a/dumpling/export/block_allow_list_test.go b/dumpling/export/block_allow_list_test.go index 13246989bd411..ee618854462fa 100644 --- a/dumpling/export/block_allow_list_test.go +++ b/dumpling/export/block_allow_list_test.go @@ -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) { diff --git a/dumpling/export/config.go b/dumpling/export/config.go index 593a9cdea3643..933dedb81bc43 100644 --- a/dumpling/export/config.go +++ b/dumpling/export/config.go @@ -20,11 +20,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 ( diff --git a/dumpling/export/config_test.go b/dumpling/export/config_test.go index 036235bba771e..5a29031b8c0b0 100644 --- a/dumpling/export/config_test.go +++ b/dumpling/export/config_test.go @@ -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) { diff --git a/dumpling/export/consistency.go b/dumpling/export/consistency.go index b951307b42312..63bc33ef35d44 100644 --- a/dumpling/export/consistency.go +++ b/dumpling/export/consistency.go @@ -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" ) diff --git a/dumpling/export/consistency_test.go b/dumpling/export/consistency_test.go index c2e7bf08abf49..b0c574f67cfba 100644 --- a/dumpling/export/consistency_test.go +++ b/dumpling/export/consistency_test.go @@ -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) { diff --git a/dumpling/export/dump.go b/dumpling/export/dump.go index 101a1dcf804ca..7cf20eb790270 100755 --- a/dumpling/export/dump.go +++ b/dumpling/export/dump.go @@ -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" @@ -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 @@ -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)) } }() @@ -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) { @@ -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()) @@ -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) } @@ -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), @@ -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()), @@ -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) @@ -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) } @@ -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}) @@ -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)) } }() } @@ -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 } @@ -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 } @@ -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 @@ -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) @@ -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() { diff --git a/dumpling/export/dump_test.go b/dumpling/export/dump_test.go index f6f638fe493b1..cdf2deb2a51b3 100644 --- a/dumpling/export/dump_test.go +++ b/dumpling/export/dump_test.go @@ -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) { diff --git a/dumpling/export/http_handler.go b/dumpling/export/http_handler.go index fd11a4bb4b5c6..91332736f17f2 100644 --- a/dumpling/export/http_handler.go +++ b/dumpling/export/http_handler.go @@ -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 @@ -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)) } } diff --git a/dumpling/export/ir.go b/dumpling/export/ir.go index 4b98019605e9c..b74d7dc1dfef3 100644 --- a/dumpling/export/ir.go +++ b/dumpling/export/ir.go @@ -8,6 +8,7 @@ import ( "strings" "github.com/pingcap/errors" + tcontext "github.com/pingcap/tidb/dumpling/context" ) diff --git a/dumpling/export/ir_impl.go b/dumpling/export/ir_impl.go index bd1916aaa4fd0..cd8272c3772a6 100644 --- a/dumpling/export/ir_impl.go +++ b/dumpling/export/ir_impl.go @@ -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. diff --git a/dumpling/export/main_test.go b/dumpling/export/main_test.go index 14d98c9606fa8..8ea24d1c51e82 100644 --- a/dumpling/export/main_test.go +++ b/dumpling/export/main_test.go @@ -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 diff --git a/dumpling/export/metadata.go b/dumpling/export/metadata.go index 5b57be24ad071..7e70344268b62 100644 --- a/dumpling/export/metadata.go +++ b/dumpling/export/metadata.go @@ -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 { @@ -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(>idSet) 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 != "" { diff --git a/dumpling/export/metadata_test.go b/dumpling/export/metadata_test.go index b37fabe312904..2c3e8a48140a7 100644 --- a/dumpling/export/metadata_test.go +++ b/dumpling/export/metadata_test.go @@ -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" ) diff --git a/dumpling/export/prepare.go b/dumpling/export/prepare.go index a2b1aa2ab0a6b..f9036ec32ea98 100644 --- a/dumpling/export/prepare.go +++ b/dumpling/export/prepare.go @@ -10,6 +10,7 @@ import ( "text/template" "github.com/pingcap/errors" + tcontext "github.com/pingcap/tidb/dumpling/context" ) diff --git a/dumpling/export/prepare_test.go b/dumpling/export/prepare_test.go index de3df7563c336..c142bec5582b8 100644 --- a/dumpling/export/prepare_test.go +++ b/dumpling/export/prepare_test.go @@ -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) { diff --git a/dumpling/export/retry.go b/dumpling/export/retry.go index e6091605def72..13f4931d72a0c 100644 --- a/dumpling/export/retry.go +++ b/dumpling/export/retry.go @@ -9,8 +9,9 @@ import ( "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" "github.com/pingcap/tidb-tools/pkg/dbutil" - tcontext "github.com/pingcap/tidb/dumpling/context" "go.uber.org/zap" + + tcontext "github.com/pingcap/tidb/dumpling/context" ) const ( @@ -83,7 +84,7 @@ func (b *lockTablesBackoffer) NextBackoff(err error) time.Duration { b.attempt-- db, table, err := getTableFromMySQLError(mysqlErr.Message) if err != nil { - b.tctx.L().Error("retry lock tables meet error", zap.Error(err)) + b.tctx.L().Error("fail to retry lock tables", zap.Error(err)) b.attempt = 0 return 0 } diff --git a/dumpling/export/sql.go b/dumpling/export/sql.go index 91ce082f2a58e..9467a5c258f67 100644 --- a/dumpling/export/sql.go +++ b/dumpling/export/sql.go @@ -14,13 +14,13 @@ import ( "strings" "github.com/go-sql-driver/mysql" - "github.com/pingcap/errors" + "go.uber.org/zap" + tcontext "github.com/pingcap/tidb/dumpling/context" "github.com/pingcap/tidb/dumpling/log" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/store/helper" - "go.uber.org/zap" ) const ( @@ -264,7 +264,7 @@ func ListAllDatabasesTables(tctx *tcontext.Context, db *sql.Conn, databaseNames if engine == "" && (comment == "" || comment == TableTypeViewStr) { tableType = TableTypeView } else if engine == "" { - tctx.L().Warn("Invalid table without engine found", zap.String("database", schema), zap.String("table", table)) + tctx.L().Warn("invalid table without engine found", zap.String("database", schema), zap.String("table", table)) continue } if _, ok := selectedTableType[tableType]; !ok { @@ -1049,7 +1049,7 @@ func estimateCount(tctx *tcontext.Context, dbName, tableName string, db *sql.Con func detectEstimateRows(tctx *tcontext.Context, db *sql.Conn, query string, fieldNames []string) uint64 { rows, err := db.QueryContext(tctx, query) if err != nil { - tctx.L().Warn("can't detect estimate rows from db", + tctx.L().Info("can't estimate rows from db", zap.String("query", query), log.ShortError(err)) return 0 } @@ -1057,13 +1057,13 @@ func detectEstimateRows(tctx *tcontext.Context, db *sql.Conn, query string, fiel rows.Next() columns, err := rows.Columns() if err != nil { - tctx.L().Warn("can't get columns from db", + tctx.L().Info("can't get columns when estimate rows from db", zap.String("query", query), log.ShortError(err)) return 0 } err = rows.Err() if err != nil { - tctx.L().Warn("rows meet some error during the query", + tctx.L().Info("rows meet some error during the query when estimate rows", zap.String("query", query), log.ShortError(err)) return 0 } @@ -1084,17 +1084,18 @@ found: } err = rows.Scan(addr...) if err != nil || fieldIndex < 0 { - tctx.L().Warn("can't get estimate count from db", - zap.String("query", query), log.ShortError(err)) + tctx.L().Info("can't estimate rows from db", + zap.String("query", query), zap.Int("fieldIndex", fieldIndex), log.ShortError(err)) return 0 } estRows, err := strconv.ParseFloat(oneRow[fieldIndex].String, 64) if err != nil { - tctx.L().Warn("can't get parse rows from db", + tctx.L().Info("can't get parse estimate rows from db", zap.String("query", query), log.ShortError(err)) return 0 } + return uint64(estRows) } diff --git a/dumpling/export/sql_test.go b/dumpling/export/sql_test.go index 056f0d5ca15ff..15b6671c04f19 100644 --- a/dumpling/export/sql_test.go +++ b/dumpling/export/sql_test.go @@ -16,11 +16,10 @@ import ( "strings" "testing" - "github.com/stretchr/testify/require" - "github.com/DATA-DOG/go-sqlmock" "github.com/coreos/go-semver/semver" "github.com/pingcap/errors" + "github.com/stretchr/testify/require" tcontext "github.com/pingcap/tidb/dumpling/context" ) diff --git a/dumpling/export/status.go b/dumpling/export/status.go index 5b2bfa41c2112..6eb1cbeac25fc 100644 --- a/dumpling/export/status.go +++ b/dumpling/export/status.go @@ -6,10 +6,10 @@ import ( "fmt" "time" - tcontext "github.com/pingcap/tidb/dumpling/context" - "github.com/docker/go-units" "go.uber.org/zap" + + tcontext "github.com/pingcap/tidb/dumpling/context" ) const logProgressTick = 2 * time.Minute diff --git a/dumpling/export/test_util.go b/dumpling/export/test_util.go index 739a431b2230e..67d7f62f9fcfb 100644 --- a/dumpling/export/test_util.go +++ b/dumpling/export/test_util.go @@ -9,6 +9,7 @@ import ( "fmt" "github.com/DATA-DOG/go-sqlmock" + tcontext "github.com/pingcap/tidb/dumpling/context" ) diff --git a/dumpling/export/util.go b/dumpling/export/util.go index dc1f98dbaa775..56320767ef475 100644 --- a/dumpling/export/util.go +++ b/dumpling/export/util.go @@ -10,8 +10,9 @@ import ( "time" "github.com/pingcap/errors" - tcontext "github.com/pingcap/tidb/dumpling/context" "go.etcd.io/etcd/clientv3" + + tcontext "github.com/pingcap/tidb/dumpling/context" ) const tidbServerInformationPath = "/tidb/server/info" diff --git a/dumpling/export/writer.go b/dumpling/export/writer.go index b8c1f23d00f7f..7790ece4489c2 100644 --- a/dumpling/export/writer.go +++ b/dumpling/export/writer.go @@ -10,10 +10,11 @@ import ( "text/template" "github.com/pingcap/errors" + "go.uber.org/zap" + "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/utils" tcontext "github.com/pingcap/tidb/dumpling/context" - "go.uber.org/zap" ) // Writer is the abstraction that keep pulling data from database and write to files. @@ -73,7 +74,7 @@ func (w *Writer) run(taskStream <-chan Task) error { for { select { case <-w.tctx.Done(): - w.tctx.L().Warn("context has been done, the writer will exit", + w.tctx.L().Info("context has been done, the writer will exit", zap.Int64("writer ID", w.id)) return nil case task, ok := <-taskStream: @@ -226,7 +227,7 @@ func (w *Writer) tryToWriteTableData(tctx *tcontext.Context, meta TableMeta, ir } } if !somethingIsWritten { - tctx.L().Warn("no data written in table chunk", + tctx.L().Info("no data written in table chunk", zap.String("database", meta.DatabaseName()), zap.String("table", meta.TableName()), zap.Int("chunkIdx", curChkIdx)) diff --git a/dumpling/export/writer_serial_test.go b/dumpling/export/writer_serial_test.go index 8657c9c21eaa2..a606784a1bd6b 100644 --- a/dumpling/export/writer_serial_test.go +++ b/dumpling/export/writer_serial_test.go @@ -8,9 +8,9 @@ import ( "strings" "testing" + "github.com/pingcap/errors" "github.com/stretchr/testify/require" - "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/storage" tcontext "github.com/pingcap/tidb/dumpling/context" ) diff --git a/dumpling/export/writer_test.go b/dumpling/export/writer_test.go index c7a4b9371b8cf..d78d9f7bed78f 100644 --- a/dumpling/export/writer_test.go +++ b/dumpling/export/writer_test.go @@ -12,8 +12,9 @@ import ( "testing" "github.com/DATA-DOG/go-sqlmock" - tcontext "github.com/pingcap/tidb/dumpling/context" "github.com/stretchr/testify/require" + + tcontext "github.com/pingcap/tidb/dumpling/context" ) func TestWriteDatabaseMeta(t *testing.T) { diff --git a/dumpling/export/writer_util.go b/dumpling/export/writer_util.go index 4f3c32269839a..2ccbefad372f9 100755 --- a/dumpling/export/writer_util.go +++ b/dumpling/export/writer_util.go @@ -14,12 +14,13 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/summary" tcontext "github.com/pingcap/tidb/dumpling/context" "github.com/pingcap/tidb/dumpling/log" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" ) const lengthLimit = 1048576 @@ -402,8 +403,8 @@ func write(tctx *tcontext.Context, writer storage.ExternalFileWriter, str string if outputLength >= 200 { outputLength = 200 } - tctx.L().Error("writing failed", - zap.String("string", str[:outputLength]), + tctx.L().Warn("fail to write", + zap.String("heading 200 characters", str[:outputLength]), zap.Error(err)) } return errors.Trace(err) @@ -417,12 +418,11 @@ func writeBytes(tctx *tcontext.Context, writer storage.ExternalFileWriter, p []b if outputLength >= 200 { outputLength = 200 } - tctx.L().Error("writing failed", - zap.ByteString("string", p[:outputLength]), - zap.String("writer", fmt.Sprintf("%#v", writer)), + tctx.L().Warn("fail to write", + zap.ByteString("heading 200 characters", p[:outputLength]), zap.Error(err)) if strings.Contains(err.Error(), "Part number must be an integer between 1 and 10000") { - err = errors.Annotate(err, "work around: dump file exceeding 50GB, please specify -F=256MB -r=200000 to avoid this problem") + err = errors.Annotate(err, "workaround: dump file exceeding 50GB, please specify -F=256MB -r=200000 to avoid this problem") } } return errors.Trace(err) @@ -433,7 +433,7 @@ func buildFileWriter(tctx *tcontext.Context, s storage.ExternalStorage, fileName fullPath := path.Join(s.URI(), fileName) writer, err := storage.WithCompression(s, compressType).Create(tctx, fileName) if err != nil { - tctx.L().Error("open file failed", + tctx.L().Warn("fail to open file", zap.String("path", fullPath), zap.Error(err)) return nil, nil, errors.Trace(err) @@ -445,7 +445,7 @@ func buildFileWriter(tctx *tcontext.Context, s storage.ExternalStorage, fileName return } err = errors.Trace(err) - tctx.L().Error("close file failed", + tctx.L().Warn("fail to close file", zap.String("path", fullPath), zap.Error(err)) } @@ -462,7 +462,7 @@ func buildInterceptFileWriter(pCtx *tcontext.Context, s storage.ExternalStorage, // which will cause a context canceled error when closing gcs's Writer w, err := storage.WithCompression(s, compressType).Create(pCtx, fileName) if err != nil { - pCtx.L().Error("open file failed", + pCtx.L().Warn("fail to open file", zap.String("path", fullPath), zap.Error(err)) return newWriterError(err) @@ -481,7 +481,7 @@ func buildInterceptFileWriter(pCtx *tcontext.Context, s storage.ExternalStorage, pCtx.L().Debug("tear down lazy file writer...", zap.String("path", fullPath)) err := writer.Close(ctx) if err != nil { - pCtx.L().Error("close file failed", + pCtx.L().Warn("fail to close file", zap.String("path", fullPath), zap.Error(err)) }