Skip to content
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

ddl_puller.go(ticdc): fix DDLs are ignored when schema versions are out of order #11733

Merged
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 1 addition & 4 deletions cdc/puller/ddl_puller.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,6 @@ type ddlJobPullerImpl struct {
kvStorage tidbkv.Storage
schemaStorage entry.SchemaStorage
resolvedTs uint64
schemaVersion int64
filter filter.Filter
// ddlTableInfo is initialized when receive the first concurrent DDL job.
ddlTableInfo *entry.DDLTableInfo
Expand Down Expand Up @@ -317,8 +316,7 @@ func (p *ddlJobPullerImpl) handleJob(job *timodel.Job) (skip bool, err error) {
return false, nil
}

if job.BinlogInfo.FinishedTS <= p.getResolvedTs() ||
job.BinlogInfo.SchemaVersion <= p.schemaVersion {
if job.BinlogInfo.FinishedTS <= p.getResolvedTs() {
log.Info("ddl job finishedTs less than puller resolvedTs,"+
"discard the ddl job",
zap.String("namespace", p.changefeedID.Namespace),
Expand Down Expand Up @@ -480,7 +478,6 @@ func (p *ddlJobPullerImpl) handleJob(job *timodel.Job) (skip bool, err error) {
errors.Trace(err), job.Query, job.StartTS, job.StartTS)
}
p.setResolvedTs(job.BinlogInfo.FinishedTS)
p.schemaVersion = job.BinlogInfo.SchemaVersion

return p.checkIneligibleTableDDL(snap, job)
}
Expand Down
75 changes: 75 additions & 0 deletions cdc/puller/ddl_puller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -546,6 +546,81 @@ func TestHandleJob(t *testing.T) {
require.NoError(t, err)
require.True(t, skip)
}

// Test finishedTs less than resolvedTs, the job will be skipped
{
recordResolvedTs := ddlJobPullerImpl.getResolvedTs()
// set a fake resolvedTs for test
fakeResolvedTs := uint64(100)
ddlJobPullerImpl.setResolvedTs(fakeResolvedTs)
// mock a job
job := &timodel.Job{
Type: timodel.ActionCreateTable,
BinlogInfo: &timodel.HistoryInfo{
FinishedTS: fakeResolvedTs - 1,
},
}
skip, err := ddlJobPullerImpl.handleJob(job)
require.NoError(t, err)
require.True(t, skip)
// reset resolvedTs
ddlJobPullerImpl.setResolvedTs(recordResolvedTs)
}
}

func TestSchemaOutOfOrder(t *testing.T) {
ddlJobPuller, helper := newMockDDLJobPuller(t, true)
defer helper.Close()
startTs := uint64(10)
ddlJobPullerImpl := ddlJobPuller.(*ddlJobPullerImpl)
ddlJobPullerImpl.setResolvedTs(startTs)

// Create database
{
job := helper.DDL2Job("create database testschema")
skip, err := ddlJobPullerImpl.handleJob(job)
require.NoError(t, err)
require.False(t, skip)
}

// Create table
{
job := helper.DDL2Job("create table testschema.t1(id int primary key)")
skip, err := ddlJobPullerImpl.handleJob(job)
require.NoError(t, err)
require.False(t, skip)

job = helper.DDL2Job("create table testschema.t2(id int primary key)")
skip, err = ddlJobPullerImpl.handleJob(job)
require.NoError(t, err)
require.False(t, skip)
}

// Test schema versions out of order
{
recordResolvedTs := ddlJobPullerImpl.getResolvedTs()
// set a fake resolvedTs for test
fakeResolvedTs := uint64(100)
ddlJobPullerImpl.setResolvedTs(fakeResolvedTs)
// mock two jobs
job1 := helper.DDL2Job("alter table testschema.t1 add column x int")
job2 := helper.DDL2Job("alter table testschema.t1 add column y int")
job1.BinlogInfo.SchemaVersion = 1
job1.BinlogInfo.FinishedTS = fakeResolvedTs + 2
job2.BinlogInfo.SchemaVersion = 2
job2.BinlogInfo.FinishedTS = fakeResolvedTs + 1

skip, err := ddlJobPullerImpl.handleJob(job2)
require.NoError(t, err)
require.False(t, skip)

skip, err = ddlJobPullerImpl.handleJob(job1)
require.NoError(t, err)
require.False(t, skip)

// reset resolvedTs
ddlJobPullerImpl.setResolvedTs(recordResolvedTs)
}
}

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