-
Notifications
You must be signed in to change notification settings - Fork 286
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
pkg/filter(ticdc): ignore ddl in ddl_manager #10518
Changes from 7 commits
b502fbe
27988d5
65f9f3c
ada55c6
e59f5ed
93f22ee
2661127
96c7b2a
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 |
---|---|---|
|
@@ -111,6 +111,7 @@ type ddlManager struct { | |
redoMetaManager redo.MetaManager | ||
// ddlSink is used to ddlSink DDL events to the downstream | ||
ddlSink DDLSink | ||
filter filter.Filter | ||
|
||
// pendingDDLs store the pending DDL events of all tables | ||
// the DDL events in the same table are ordered by commitTs. | ||
|
@@ -136,6 +137,7 @@ func newDDLManager( | |
startTs model.Ts, | ||
checkpointTs model.Ts, | ||
ddlSink DDLSink, | ||
filter filter.Filter, | ||
ddlPuller puller.DDLPuller, | ||
schema entry.SchemaStorage, | ||
redoManager redo.DDLManager, | ||
|
@@ -152,6 +154,7 @@ func newDDLManager( | |
return &ddlManager{ | ||
changfeedID: changefeedID, | ||
ddlSink: ddlSink, | ||
filter: filter, | ||
ddlPuller: ddlPuller, | ||
schema: schema, | ||
redoDDLManager: redoManager, | ||
|
@@ -208,7 +211,7 @@ func (m *ddlManager) tick( | |
|
||
log.Info("handle a ddl job", | ||
zap.String("namespace", m.changfeedID.Namespace), | ||
zap.String("ID", m.changfeedID.ID), | ||
zap.String("changefeed", m.changfeedID.ID), | ||
zap.Int64("tableID", job.TableID), | ||
zap.Int64("jobID", job.ID), | ||
zap.String("query", job.Query), | ||
|
@@ -244,8 +247,14 @@ func (m *ddlManager) tick( | |
// Send DDL events to redo log. | ||
if m.redoDDLManager.Enabled() { | ||
for _, event := range events { | ||
err := m.redoDDLManager.EmitDDLEvent(ctx, event) | ||
skip, _, err := m.shouldSkipDDL(event) | ||
if err != nil { | ||
return nil, nil, errors.Trace(err) | ||
} | ||
if skip { | ||
continue | ||
} | ||
if err := m.redoDDLManager.EmitDDLEvent(ctx, event); err != nil { | ||
return nil, nil, err | ||
} | ||
} | ||
|
@@ -327,29 +336,34 @@ func (m *ddlManager) shouldExecDDL(nextDDL *model.DDLEvent) bool { | |
return checkpointReachBarrier && redoCheckpointReachBarrier && redoDDLResolvedTsExceedBarrier | ||
} | ||
|
||
func (m *ddlManager) shouldSkipDDL(ddl *model.DDLEvent) (bool, string, error) { | ||
ignored, err := m.filter.ShouldIgnoreDDLEvent(ddl) | ||
if err != nil { | ||
return false, "", errors.Trace(err) | ||
} | ||
if ignored { | ||
return true, "ddl is ignored by event filter rule, skip it", nil | ||
} | ||
|
||
// In a BDR mode cluster, TiCDC can receive DDLs from all roles of TiDB. | ||
// However, CDC only executes the DDLs from the TiDB that has BDRRolePrimary role. | ||
if m.BDRMode && ddl.BDRRole != string(ast.BDRRolePrimary) { | ||
return true, "changefeed is in BDRMode and the DDL is not executed by Primary Cluster, skip it", nil | ||
} | ||
return false, "", nil | ||
} | ||
|
||
// executeDDL executes ddlManager.executingDDL. | ||
func (m *ddlManager) executeDDL(ctx context.Context) error { | ||
if m.executingDDL == nil { | ||
return nil | ||
} | ||
|
||
// In a BDR mode cluster, TiCDC can receive DDLs from all roles of TiDB. | ||
// However, CDC only executes the DDLs from the TiDB that has BDRRolePrimary role. | ||
if m.BDRMode && m.executingDDL.BDRRole != string(ast.BDRRolePrimary) { | ||
log.Info("changefeed is in BDRMode and "+ | ||
"the DDL is not executed by Primary Cluster, skip it", | ||
zap.String("namespace", m.changfeedID.Namespace), | ||
zap.String("ID", m.changfeedID.ID), | ||
zap.Any("ddlEvent", m.executingDDL), | ||
zap.String("bdrRole", m.executingDDL.BDRRole)) | ||
tableName := m.executingDDL.TableInfo.TableName | ||
// Set it to nil first to accelerate GC. | ||
m.pendingDDLs[tableName][0] = nil | ||
m.pendingDDLs[tableName] = m.pendingDDLs[tableName][1:] | ||
m.schema.DoGC(m.executingDDL.CommitTs - 1) | ||
m.justSentDDL = m.executingDDL | ||
m.executingDDL = nil | ||
m.cleanCache() | ||
skip, cleanMsg, err := m.shouldSkipDDL(m.executingDDL) | ||
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. Why do we decide whether to skip this ddl here, but not in where we append ddls into pendingDDLs in "ddlManager.tick"? 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. Because all ddls must be cached in pengingDDLs until |
||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
if skip { | ||
m.cleanCache(cleanMsg) | ||
return nil | ||
} | ||
|
||
|
@@ -371,23 +385,10 @@ func (m *ddlManager) executeDDL(ctx context.Context) error { | |
|
||
done, err := m.ddlSink.emitDDLEvent(ctx, m.executingDDL) | ||
if err != nil { | ||
return err | ||
return errors.Trace(err) | ||
} | ||
if done { | ||
tableName := m.executingDDL.TableInfo.TableName | ||
log.Info("execute a ddl event successfully", | ||
zap.String("ddl", m.executingDDL.Query), | ||
zap.String("namespace", m.executingDDL.BDRRole), | ||
zap.Uint64("commitTs", m.executingDDL.CommitTs), | ||
zap.Stringer("table", tableName), | ||
) | ||
// Set it to nil first to accelerate GC. | ||
m.pendingDDLs[tableName][0] = nil | ||
m.pendingDDLs[tableName] = m.pendingDDLs[tableName][1:] | ||
m.schema.DoGC(m.executingDDL.CommitTs - 1) | ||
m.justSentDDL = m.executingDDL | ||
m.executingDDL = nil | ||
m.cleanCache() | ||
m.cleanCache("execute a ddl event successfully") | ||
} | ||
return nil | ||
} | ||
|
@@ -555,9 +556,22 @@ func (m *ddlManager) getSnapshotTs() (ts uint64) { | |
} | ||
|
||
// cleanCache cleans the tableInfoCache and physicalTablesCache. | ||
// It should be called after a DDL is applied to schema or a DDL | ||
// is sent to downstream successfully. | ||
func (m *ddlManager) cleanCache() { | ||
// It should be called after a DDL is skipped or sent to downstream successfully. | ||
func (m *ddlManager) cleanCache(msg string) { | ||
tableName := m.executingDDL.TableInfo.TableName | ||
log.Info(msg, zap.String("ddl", m.executingDDL.Query), | ||
zap.String("namespace", m.changfeedID.Namespace), | ||
zap.String("changefeed", m.changfeedID.ID), | ||
zap.String("bdrRole", m.executingDDL.BDRRole), | ||
zap.Any("ddlEvent", m.executingDDL)) | ||
|
||
// Set it to nil first to accelerate GC. | ||
m.pendingDDLs[tableName][0] = nil | ||
m.pendingDDLs[tableName] = m.pendingDDLs[tableName][1:] | ||
m.schema.DoGC(m.executingDDL.CommitTs - 1) | ||
m.justSentDDL = m.executingDDL | ||
m.executingDDL = nil | ||
|
||
m.tableInfoCache = nil | ||
m.physicalTablesCache = nil | ||
} | ||
|
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.
the second returned value looks used to print log, can we just print the log here, instead of return it ?
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.
If redo is enabled, shouldSkipDDL will be called twice, so we use the return value to avoid printing duplicate logs.