-
Notifications
You must be signed in to change notification settings - Fork 288
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
consumer(ticdc): canal-json decoder cache table info for each message to avoid allocate memory on each new message #11895
Changes from all commits
f9cbca3
64f991c
c5ca16d
ad3c830
e5eedae
ffc3c3b
3ffb1d7
9a7d6c8
74cc7ea
981de3a
8e6a322
1cf6afe
2683725
df4d5cf
10e21dc
1e536cc
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 |
---|---|---|
|
@@ -26,6 +26,9 @@ import ( | |
"github.com/pingcap/errors" | ||
"github.com/pingcap/log" | ||
"github.com/pingcap/tidb/br/pkg/storage" | ||
timodel "github.com/pingcap/tidb/pkg/meta/model" | ||
pmodel "github.com/pingcap/tidb/pkg/parser/model" | ||
"github.com/pingcap/tidb/pkg/parser/mysql" | ||
"github.com/pingcap/tiflow/cdc/model" | ||
cerror "github.com/pingcap/tiflow/pkg/errors" | ||
"github.com/pingcap/tiflow/pkg/sink/codec" | ||
|
@@ -37,6 +40,11 @@ import ( | |
"golang.org/x/text/encoding/charmap" | ||
) | ||
|
||
type tableKey struct { | ||
schema string | ||
table string | ||
} | ||
|
||
// batchDecoder decodes the byte into the original message. | ||
type batchDecoder struct { | ||
data []byte | ||
|
@@ -48,6 +56,8 @@ type batchDecoder struct { | |
|
||
upstreamTiDB *sql.DB | ||
bytesDecoder *encoding.Decoder | ||
|
||
tableInfoCache map[tableKey]*model.TableInfo | ||
} | ||
|
||
// NewBatchDecoder return a decoder for canal-json | ||
|
@@ -72,10 +82,11 @@ func NewBatchDecoder( | |
} | ||
|
||
return &batchDecoder{ | ||
config: codecConfig, | ||
storage: externalStorage, | ||
upstreamTiDB: db, | ||
bytesDecoder: charmap.ISO8859_1.NewDecoder(), | ||
config: codecConfig, | ||
storage: externalStorage, | ||
upstreamTiDB: db, | ||
bytesDecoder: charmap.ISO8859_1.NewDecoder(), | ||
tableInfoCache: make(map[tableKey]*model.TableInfo), | ||
}, nil | ||
} | ||
|
||
|
@@ -263,6 +274,52 @@ func (b *batchDecoder) assembleHandleKeyOnlyRowChangedEvent( | |
return b.NextRowChangedEvent() | ||
} | ||
|
||
func setColumnInfos( | ||
tableInfo *timodel.TableInfo, | ||
rawColumns map[string]interface{}, | ||
mysqlType map[string]string, | ||
pkNames map[string]struct{}, | ||
) { | ||
mockColumnID := int64(100) | ||
for name := range rawColumns { | ||
columnInfo := new(timodel.ColumnInfo) | ||
columnInfo.ID = mockColumnID | ||
columnInfo.Name = pmodel.NewCIStr(name) | ||
if utils.IsBinaryMySQLType(mysqlType[name]) { | ||
columnInfo.AddFlag(mysql.BinaryFlag) | ||
} | ||
if _, isPK := pkNames[name]; isPK { | ||
columnInfo.AddFlag(mysql.PriKeyFlag) | ||
} | ||
tableInfo.Columns = append(tableInfo.Columns, columnInfo) | ||
mockColumnID++ | ||
} | ||
} | ||
|
||
func setIndexes( | ||
tableInfo *timodel.TableInfo, | ||
pkNames map[string]struct{}, | ||
) { | ||
indexColumns := make([]*timodel.IndexColumn, 0, len(pkNames)) | ||
for idx, col := range tableInfo.Columns { | ||
name := col.Name.O | ||
if _, ok := pkNames[name]; ok { | ||
indexColumns = append(indexColumns, &timodel.IndexColumn{ | ||
Name: pmodel.NewCIStr(name), | ||
Offset: idx, | ||
}) | ||
} | ||
} | ||
indexInfo := &timodel.IndexInfo{ | ||
ID: 1, | ||
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. Please follow the previous behavior. https://github.com/pingcap/tiflow/blob/master/cdc/model/sink.go#L917-L920 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. there is only primary key, and should 1, it's the previous behavior. |
||
Name: pmodel.NewCIStr("primary"), | ||
Columns: indexColumns, | ||
Unique: true, | ||
Primary: true, | ||
} | ||
tableInfo.Indices = append(tableInfo.Indices, indexInfo) | ||
} | ||
|
||
// NextRowChangedEvent implements the RowEventDecoder interface | ||
// `HasNext` should be called before this. | ||
func (b *batchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { | ||
|
@@ -282,7 +339,7 @@ func (b *batchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { | |
} | ||
} | ||
|
||
result, err := canalJSONMessage2RowChange(b.msg) | ||
result, err := b.canalJSONMessage2RowChange() | ||
if err != nil { | ||
return nil, err | ||
} | ||
|
@@ -299,6 +356,17 @@ func (b *batchDecoder) NextDDLEvent() (*model.DDLEvent, error) { | |
} | ||
|
||
result := canalJSONMessage2DDLEvent(b.msg) | ||
|
||
schema := *b.msg.getSchema() | ||
table := *b.msg.getTable() | ||
// if receive a table level DDL, just remove the table info to trigger create a new one. | ||
if schema != "" && table != "" { | ||
cacheKey := tableKey{ | ||
schema: schema, | ||
table: table, | ||
} | ||
delete(b.tableInfoCache, cacheKey) | ||
} | ||
b.msg = nil | ||
return result, 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.
cached item is deleted if receive table level DDL, and recreate a new table info.