-
Notifications
You must be signed in to change notification settings - Fork 5.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
importinto/lightning: do remote checksum via sql #44803
Changes from 1 commit
5bce3a4
b926642
f83f431
9b70a6b
8c3b8e8
cba363c
202dbbf
7ca8cbc
905fc93
61a9586
88d63ae
5047150
f6600df
f5802af
066f8b8
7afcd5d
ae058d9
01156f6
bd7de93
f2ee878
8720abd
f4ba13f
c1bb4d8
d30922f
7c7686e
1983eb0
a068fcb
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -16,14 +16,25 @@ package importinto | |
|
||
import ( | ||
"context" | ||
"strconv" | ||
"time" | ||
|
||
"github.com/pingcap/errors" | ||
"github.com/pingcap/failpoint" | ||
"github.com/pingcap/tidb/br/pkg/lightning/backend/local" | ||
"github.com/pingcap/tidb/br/pkg/lightning/common" | ||
"github.com/pingcap/tidb/br/pkg/lightning/config" | ||
verify "github.com/pingcap/tidb/br/pkg/lightning/verification" | ||
"github.com/pingcap/tidb/disttask/framework/proto" | ||
"github.com/pingcap/tidb/disttask/framework/scheduler" | ||
"github.com/pingcap/tidb/disttask/framework/storage" | ||
"github.com/pingcap/tidb/executor/importer" | ||
"github.com/pingcap/tidb/sessionctx" | ||
"github.com/pingcap/tidb/sessionctx/variable" | ||
"github.com/pingcap/tidb/util/chunk" | ||
"github.com/pingcap/tidb/util/logutil" | ||
"github.com/pingcap/tidb/util/mathutil" | ||
tikvstore "github.com/tikv/client-go/v2/kv" | ||
"go.uber.org/zap" | ||
) | ||
|
||
|
@@ -73,6 +84,115 @@ func (e *postProcessMinimalTaskExecutor) Run(ctx context.Context) error { | |
return postProcess(ctx, mTask.taskMeta, &mTask.meta, mTask.logger) | ||
} | ||
|
||
// postProcess does the post-processing for the task. | ||
func postProcess(ctx context.Context, taskMeta *TaskMeta, subtaskMeta *PostProcessStepMeta, logger *zap.Logger) (err error) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Need to add more unit tests for this function or checksum via sql |
||
failpoint.Inject("syncBeforePostProcess", func() { | ||
TestSyncChan <- struct{}{} | ||
<-TestSyncChan | ||
}) | ||
|
||
logger.Info("post process") | ||
|
||
// TODO: create table indexes depends on the option. | ||
// create table indexes even if the post process is failed. | ||
// defer func() { | ||
// err2 := createTableIndexes(ctx, globalTaskManager, taskMeta, logger) | ||
// err = multierr.Append(err, err2) | ||
// }() | ||
|
||
return verifyChecksum(ctx, taskMeta, subtaskMeta, logger) | ||
} | ||
|
||
func verifyChecksum(ctx context.Context, taskMeta *TaskMeta, subtaskMeta *PostProcessStepMeta, logger *zap.Logger) error { | ||
if taskMeta.Plan.Checksum == config.OpLevelOff { | ||
return nil | ||
} | ||
localChecksum := verify.MakeKVChecksum(subtaskMeta.Checksum.Size, subtaskMeta.Checksum.KVs, subtaskMeta.Checksum.Sum) | ||
logger.Info("local checksum", zap.Object("checksum", &localChecksum)) | ||
|
||
failpoint.Inject("waitCtxDone", func() { | ||
<-ctx.Done() | ||
}) | ||
|
||
globalTaskManager, err := storage.GetTaskManager() | ||
if err != nil { | ||
return err | ||
} | ||
remoteChecksum, err := checksumTable(ctx, globalTaskManager, taskMeta, logger) | ||
if err != nil { | ||
return err | ||
} | ||
if !remoteChecksum.IsEqual(&localChecksum) { | ||
err2 := common.ErrChecksumMismatch.GenWithStackByArgs( | ||
remoteChecksum.Checksum, localChecksum.Sum(), | ||
remoteChecksum.TotalKVs, localChecksum.SumKVS(), | ||
remoteChecksum.TotalBytes, localChecksum.SumSize(), | ||
) | ||
if taskMeta.Plan.Checksum == config.OpLevelOptional { | ||
logger.Warn("verify checksum failed, but checksum is optional, will skip it", zap.Error(err2)) | ||
err2 = nil | ||
} | ||
return err2 | ||
} | ||
logger.Info("checksum pass", zap.Object("local", &localChecksum)) | ||
return nil | ||
} | ||
|
||
func checksumTable(ctx context.Context, executor storage.SessionExecutor, taskMeta *TaskMeta, logger *zap.Logger) (*local.RemoteChecksum, error) { | ||
var ( | ||
tableName = common.UniqueTable(taskMeta.Plan.DBName, taskMeta.Plan.TableInfo.Name.L) | ||
sql = "ADMIN CHECKSUM TABLE " + tableName | ||
remoteChecksum *local.RemoteChecksum | ||
maxErrorRetryCount = 3 | ||
distSQLScanConcurrency int | ||
rs []chunk.Row | ||
execErr error | ||
) | ||
|
||
err := executor.WithNewSession(func(se sessionctx.Context) error { | ||
if err := se.GetSessionVars().SetSystemVar(variable.TiDBBackOffWeight, strconv.Itoa(3*tikvstore.DefBackOffWeight)); err != nil { | ||
return err | ||
} | ||
distSQLScanConcurrency = se.GetSessionVars().DistSQLScanConcurrency() | ||
|
||
for i := 0; i < maxErrorRetryCount; i++ { | ||
rs, execErr = storage.ExecSQL(ctx, se, sql) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. put in explicit txn? |
||
if execErr == nil { | ||
if len(rs) < 1 { | ||
return errors.New("empty checksum result") | ||
} | ||
// ADMIN CHECKSUM TABLE <schema>.<table> example. | ||
// mysql> admin checksum table test.t; | ||
// +---------+------------+---------------------+-----------+-------------+ | ||
// | Db_name | Table_name | Checksum_crc64_xor | Total_kvs | Total_bytes | | ||
// +---------+------------+---------------------+-----------+-------------+ | ||
// | test | t | 8520875019404689597 | 7296873 | 357601387 | | ||
// +---------+------------+------------- | ||
remoteChecksum = &local.RemoteChecksum{ | ||
Schema: rs[0].GetString(0), | ||
Table: rs[0].GetString(1), | ||
Checksum: rs[0].GetUint64(2), | ||
TotalKVs: rs[0].GetUint64(3), | ||
TotalBytes: rs[0].GetUint64(4), | ||
} | ||
return nil | ||
} | ||
if !common.IsRetryableError(execErr) { | ||
return execErr | ||
} | ||
|
||
logger.Warn("remote checksum failed", zap.String("sql", sql), zap.Error(execErr), | ||
zap.Int("concurrency", distSQLScanConcurrency), zap.Int("retry", i)) | ||
if distSQLScanConcurrency > local.MinDistSQLScanConcurrency { | ||
distSQLScanConcurrency = mathutil.Max(distSQLScanConcurrency/2, local.MinDistSQLScanConcurrency) | ||
se.GetSessionVars().SetDistSQLScanConcurrency(distSQLScanConcurrency) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. revert it back after execute? the session taken from resource pool |
||
} | ||
} | ||
return execErr | ||
}) | ||
return remoteChecksum, err | ||
} | ||
|
||
func init() { | ||
scheduler.RegisterSubtaskExectorConstructor(proto.ImportInto, StepImport, | ||
// The order of the subtask executors is the same as the order of the subtasks. | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How about add a hidden switch for checksum via sql or tikv ?