github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/simple/avro.go (about)

     1  // Copyright 2023 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package simple
    15  
    16  import (
    17  	"sort"
    18  	"sync"
    19  	"time"
    20  
    21  	"github.com/pingcap/tidb/pkg/parser/mysql"
    22  	"github.com/pingcap/tidb/pkg/parser/types"
    23  	"github.com/pingcap/tiflow/cdc/model"
    24  )
    25  
    26  func newTableSchemaMap(tableInfo *model.TableInfo) interface{} {
    27  	pkInIndexes := false
    28  	indexesSchema := make([]interface{}, 0, len(tableInfo.Indices))
    29  	for _, idx := range tableInfo.Indices {
    30  		index := map[string]interface{}{
    31  			"name":     idx.Name.O,
    32  			"unique":   idx.Unique,
    33  			"primary":  idx.Primary,
    34  			"nullable": false,
    35  		}
    36  		columns := make([]string, 0, len(idx.Columns))
    37  		for _, col := range idx.Columns {
    38  			columns = append(columns, col.Name.O)
    39  			colInfo := tableInfo.Columns[col.Offset]
    40  			// An index is not null when all columns of are not null
    41  			if !mysql.HasNotNullFlag(colInfo.GetFlag()) {
    42  				index["nullable"] = true
    43  			}
    44  		}
    45  		index["columns"] = columns
    46  		if idx.Primary {
    47  			pkInIndexes = true
    48  		}
    49  		indexesSchema = append(indexesSchema, index)
    50  	}
    51  
    52  	// sometimes the primary key is not in the index, we need to find it manually.
    53  	if !pkInIndexes {
    54  		pkColumns := tableInfo.GetPrimaryKeyColumnNames()
    55  		if len(pkColumns) != 0 {
    56  			index := map[string]interface{}{
    57  				"name":     "primary",
    58  				"nullable": false,
    59  				"primary":  true,
    60  				"unique":   true,
    61  				"columns":  pkColumns,
    62  			}
    63  			indexesSchema = append(indexesSchema, index)
    64  		}
    65  	}
    66  
    67  	sort.SliceStable(tableInfo.Columns, func(i, j int) bool {
    68  		return tableInfo.Columns[i].ID < tableInfo.Columns[j].ID
    69  	})
    70  
    71  	columnsSchema := make([]interface{}, 0, len(tableInfo.Columns))
    72  	for _, col := range tableInfo.Columns {
    73  		mysqlType := map[string]interface{}{
    74  			"mysqlType": types.TypeToStr(col.GetType(), col.GetCharset()),
    75  			"charset":   col.GetCharset(),
    76  			"collate":   col.GetCollate(),
    77  			"length":    col.GetFlen(),
    78  		}
    79  
    80  		switch col.GetType() {
    81  		case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong,
    82  			mysql.TypeFloat, mysql.TypeDouble, mysql.TypeBit, mysql.TypeYear:
    83  			mysqlType["unsigned"] = map[string]interface{}{
    84  				"boolean": mysql.HasUnsignedFlag(col.GetFlag()),
    85  			}
    86  			mysqlType["zerofill"] = map[string]interface{}{
    87  				"boolean": mysql.HasZerofillFlag(col.GetFlag()),
    88  			}
    89  		case mysql.TypeEnum, mysql.TypeSet:
    90  			mysqlType["elements"] = map[string]interface{}{
    91  				"array": col.GetElems(),
    92  			}
    93  		case mysql.TypeNewDecimal:
    94  			mysqlType["decimal"] = map[string]interface{}{
    95  				"int": col.GetDecimal(),
    96  			}
    97  		default:
    98  		}
    99  
   100  		column := map[string]interface{}{
   101  			"name":     col.Name.O,
   102  			"dataType": mysqlType,
   103  			"nullable": !mysql.HasNotNullFlag(col.GetFlag()),
   104  			"default":  nil,
   105  		}
   106  		defaultValue := model.GetColumnDefaultValue(col)
   107  		if defaultValue != nil {
   108  			// according to TiDB source code, the default value is converted to string if not nil.
   109  			column["default"] = map[string]interface{}{
   110  				"string": defaultValue,
   111  			}
   112  		}
   113  
   114  		columnsSchema = append(columnsSchema, column)
   115  	}
   116  
   117  	result := map[string]interface{}{
   118  		"database": tableInfo.TableName.Schema,
   119  		"table":    tableInfo.TableName.Table,
   120  		"tableID":  tableInfo.ID,
   121  		"version":  int64(tableInfo.UpdateTS),
   122  		"columns":  columnsSchema,
   123  		"indexes":  indexesSchema,
   124  	}
   125  
   126  	return result
   127  }
   128  
   129  func newResolvedMessageMap(ts uint64) map[string]interface{} {
   130  	watermark := map[string]interface{}{
   131  		"version":  defaultVersion,
   132  		"type":     string(MessageTypeWatermark),
   133  		"commitTs": int64(ts),
   134  		"buildTs":  time.Now().UnixMilli(),
   135  	}
   136  	watermark = map[string]interface{}{
   137  		"com.pingcap.simple.avro.Watermark": watermark,
   138  	}
   139  
   140  	payload := map[string]interface{}{
   141  		"type":    string(MessageTypeWatermark),
   142  		"payload": watermark,
   143  	}
   144  
   145  	return map[string]interface{}{
   146  		"com.pingcap.simple.avro.Message": payload,
   147  	}
   148  }
   149  
   150  func newBootstrapMessageMap(tableInfo *model.TableInfo) map[string]interface{} {
   151  	m := map[string]interface{}{
   152  		"version":     defaultVersion,
   153  		"type":        string(MessageTypeBootstrap),
   154  		"tableSchema": newTableSchemaMap(tableInfo),
   155  		"buildTs":     time.Now().UnixMilli(),
   156  	}
   157  
   158  	m = map[string]interface{}{
   159  		"com.pingcap.simple.avro.Bootstrap": m,
   160  	}
   161  
   162  	payload := map[string]interface{}{
   163  		"type":    string(MessageTypeBootstrap),
   164  		"payload": m,
   165  	}
   166  
   167  	return map[string]interface{}{
   168  		"com.pingcap.simple.avro.Message": payload,
   169  	}
   170  }
   171  
   172  func newDDLMessageMap(ddl *model.DDLEvent) map[string]interface{} {
   173  	result := map[string]interface{}{
   174  		"version":  defaultVersion,
   175  		"type":     string(getDDLType(ddl.Type)),
   176  		"sql":      ddl.Query,
   177  		"commitTs": int64(ddl.CommitTs),
   178  		"buildTs":  time.Now().UnixMilli(),
   179  	}
   180  
   181  	if ddl.TableInfo != nil && ddl.TableInfo.TableInfo != nil {
   182  		tableSchema := newTableSchemaMap(ddl.TableInfo)
   183  		result["tableSchema"] = map[string]interface{}{
   184  			"com.pingcap.simple.avro.TableSchema": tableSchema,
   185  		}
   186  	}
   187  	if ddl.PreTableInfo != nil && ddl.PreTableInfo.TableInfo != nil {
   188  		tableSchema := newTableSchemaMap(ddl.PreTableInfo)
   189  		result["preTableSchema"] = map[string]interface{}{
   190  			"com.pingcap.simple.avro.TableSchema": tableSchema,
   191  		}
   192  	}
   193  
   194  	result = map[string]interface{}{
   195  		"com.pingcap.simple.avro.DDL": result,
   196  	}
   197  	payload := map[string]interface{}{
   198  		"type":    string(MessageTypeDDL),
   199  		"payload": result,
   200  	}
   201  	return map[string]interface{}{
   202  		"com.pingcap.simple.avro.Message": payload,
   203  	}
   204  }
   205  
   206  var (
   207  	// genericMapPool return holder for each column and checksum
   208  	genericMapPool = sync.Pool{
   209  		New: func() any {
   210  			return make(map[string]interface{})
   211  		},
   212  	}
   213  	// rowMapPool return map for each row
   214  	rowMapPool = sync.Pool{
   215  		New: func() any {
   216  			return make(map[string]interface{})
   217  		},
   218  	}
   219  
   220  	dmlMessagePayloadPool = sync.Pool{
   221  		New: func() any {
   222  			return make(map[string]interface{})
   223  		},
   224  	}
   225  
   226  	// dmlMessagePool return a map for the dml message
   227  	dmlMessagePool = sync.Pool{
   228  		New: func() any {
   229  			return make(map[string]interface{})
   230  		},
   231  	}
   232  
   233  	messageHolderPool = sync.Pool{
   234  		New: func() any {
   235  			return make(map[string]interface{})
   236  		},
   237  	}
   238  )
   239  
   240  func (a *avroMarshaller) newDMLMessageMap(
   241  	event *model.RowChangedEvent,
   242  	onlyHandleKey bool,
   243  	claimCheckFileName string,
   244  ) map[string]interface{} {
   245  	dmlMessagePayload := dmlMessagePayloadPool.Get().(map[string]interface{})
   246  	dmlMessagePayload["version"] = defaultVersion
   247  	dmlMessagePayload["database"] = event.TableInfo.GetSchemaName()
   248  	dmlMessagePayload["table"] = event.TableInfo.GetTableName()
   249  	dmlMessagePayload["tableID"] = event.TableInfo.ID
   250  	dmlMessagePayload["commitTs"] = int64(event.CommitTs)
   251  	dmlMessagePayload["buildTs"] = time.Now().UnixMilli()
   252  	dmlMessagePayload["schemaVersion"] = int64(event.TableInfo.UpdateTS)
   253  
   254  	if !a.config.LargeMessageHandle.Disabled() && onlyHandleKey {
   255  		dmlMessagePayload["handleKeyOnly"] = map[string]interface{}{
   256  			"boolean": true,
   257  		}
   258  	}
   259  
   260  	if a.config.LargeMessageHandle.EnableClaimCheck() && claimCheckFileName != "" {
   261  		dmlMessagePayload["claimCheckLocation"] = map[string]interface{}{
   262  			"string": claimCheckFileName,
   263  		}
   264  	}
   265  
   266  	if a.config.EnableRowChecksum && event.Checksum != nil {
   267  		cc := map[string]interface{}{
   268  			"version":   event.Checksum.Version,
   269  			"corrupted": event.Checksum.Corrupted,
   270  			"current":   int64(event.Checksum.Current),
   271  			"previous":  int64(event.Checksum.Previous),
   272  		}
   273  
   274  		holder := genericMapPool.Get().(map[string]interface{})
   275  		holder["com.pingcap.simple.avro.Checksum"] = cc
   276  		dmlMessagePayload["checksum"] = holder
   277  	}
   278  
   279  	if event.IsInsert() {
   280  		data := a.collectColumns(event.Columns, event.TableInfo, onlyHandleKey)
   281  		dmlMessagePayload["data"] = data
   282  		dmlMessagePayload["type"] = string(DMLTypeInsert)
   283  	} else if event.IsDelete() {
   284  		old := a.collectColumns(event.PreColumns, event.TableInfo, onlyHandleKey)
   285  		dmlMessagePayload["old"] = old
   286  		dmlMessagePayload["type"] = string(DMLTypeDelete)
   287  	} else if event.IsUpdate() {
   288  		data := a.collectColumns(event.Columns, event.TableInfo, onlyHandleKey)
   289  		dmlMessagePayload["data"] = data
   290  		old := a.collectColumns(event.PreColumns, event.TableInfo, onlyHandleKey)
   291  		dmlMessagePayload["old"] = old
   292  		dmlMessagePayload["type"] = string(DMLTypeUpdate)
   293  	}
   294  
   295  	dmlMessagePayload = map[string]interface{}{
   296  		"com.pingcap.simple.avro.DML": dmlMessagePayload,
   297  	}
   298  
   299  	dmlMessage := dmlMessagePool.Get().(map[string]interface{})
   300  	dmlMessage["type"] = string(MessageTypeDML)
   301  	dmlMessage["payload"] = dmlMessagePayload
   302  
   303  	messageHolder := messageHolderPool.Get().(map[string]interface{})
   304  	messageHolder["com.pingcap.simple.avro.Message"] = dmlMessage
   305  
   306  	return messageHolder
   307  }
   308  
   309  func recycleMap(m map[string]interface{}) {
   310  	dmlMessage := m["com.pingcap.simple.avro.Message"].(map[string]interface{})
   311  	dml := dmlMessage["payload"].(map[string]interface{})["com.pingcap.simple.avro.DML"].(map[string]interface{})
   312  
   313  	checksum := dml["checksum"]
   314  	if checksum != nil {
   315  		checksum := checksum.(map[string]interface{})
   316  		clear(checksum)
   317  		genericMapPool.Put(checksum)
   318  	}
   319  
   320  	dataMap := dml["data"]
   321  	if dataMap != nil {
   322  		dataMap := dataMap.(map[string]interface{})["map"].(map[string]interface{})
   323  		for _, col := range dataMap {
   324  			colMap := col.(map[string]interface{})
   325  			clear(colMap)
   326  			genericMapPool.Put(col)
   327  		}
   328  		clear(dataMap)
   329  		rowMapPool.Put(dataMap)
   330  	}
   331  
   332  	oldDataMap := dml["old"]
   333  	if oldDataMap != nil {
   334  		oldDataMap := oldDataMap.(map[string]interface{})["map"].(map[string]interface{})
   335  		for _, col := range oldDataMap {
   336  			colMap := col.(map[string]interface{})
   337  			clear(colMap)
   338  			genericMapPool.Put(col)
   339  		}
   340  		clear(oldDataMap)
   341  		rowMapPool.Put(oldDataMap)
   342  	}
   343  
   344  	clear(dml)
   345  	dmlMessagePayloadPool.Put(dml)
   346  
   347  	clear(dmlMessage)
   348  	dmlMessagePool.Put(dmlMessage)
   349  
   350  	clear(m)
   351  	messageHolderPool.Put(m)
   352  }
   353  
   354  func (a *avroMarshaller) collectColumns(
   355  	columns []*model.ColumnData, tableInfo *model.TableInfo, onlyHandleKey bool,
   356  ) map[string]interface{} {
   357  	result := rowMapPool.Get().(map[string]interface{})
   358  	for _, col := range columns {
   359  		if col != nil {
   360  			colFlag := tableInfo.ForceGetColumnFlagType(col.ColumnID)
   361  			if onlyHandleKey && !colFlag.IsHandleKey() {
   362  				continue
   363  			}
   364  			colInfo := tableInfo.ForceGetColumnInfo(col.ColumnID)
   365  			value, avroType := a.encodeValue4Avro(col.Value, &colInfo.FieldType)
   366  			holder := genericMapPool.Get().(map[string]interface{})
   367  			holder[avroType] = value
   368  			result[colInfo.Name.O] = holder
   369  		}
   370  	}
   371  	return map[string]interface{}{
   372  		"map": result,
   373  	}
   374  }
   375  
   376  func newTableSchemaFromAvroNative(native map[string]interface{}) *TableSchema {
   377  	rawColumns := native["columns"].([]interface{})
   378  	columns := make([]*columnSchema, 0, len(rawColumns))
   379  	for _, raw := range rawColumns {
   380  		raw := raw.(map[string]interface{})
   381  		rawDataType := raw["dataType"].(map[string]interface{})
   382  
   383  		var (
   384  			decimal  int
   385  			elements []string
   386  			unsigned bool
   387  			zerofill bool
   388  		)
   389  
   390  		if rawDataType["elements"] != nil {
   391  			rawElements := rawDataType["elements"].(map[string]interface{})["array"].([]interface{})
   392  			for _, rawElement := range rawElements {
   393  				elements = append(elements, rawElement.(string))
   394  			}
   395  		}
   396  		if rawDataType["decimal"] != nil {
   397  			decimal = int(rawDataType["decimal"].(map[string]interface{})["int"].(int32))
   398  		}
   399  		if rawDataType["unsigned"] != nil {
   400  			unsigned = rawDataType["unsigned"].(map[string]interface{})["boolean"].(bool)
   401  		}
   402  		if rawDataType["zerofill"] != nil {
   403  			zerofill = rawDataType["zerofill"].(map[string]interface{})["boolean"].(bool)
   404  		}
   405  
   406  		dt := dataType{
   407  			MySQLType: rawDataType["mysqlType"].(string),
   408  			Charset:   rawDataType["charset"].(string),
   409  			Collate:   rawDataType["collate"].(string),
   410  			Length:    int(rawDataType["length"].(int64)),
   411  			Decimal:   decimal,
   412  			Elements:  elements,
   413  			Unsigned:  unsigned,
   414  			Zerofill:  zerofill,
   415  		}
   416  
   417  		var defaultValue interface{}
   418  		rawDefault := raw["default"]
   419  		switch v := rawDefault.(type) {
   420  		case nil:
   421  		case map[string]interface{}:
   422  			defaultValue = v["string"].(string)
   423  		}
   424  
   425  		column := &columnSchema{
   426  			Name:     raw["name"].(string),
   427  			Nullable: raw["nullable"].(bool),
   428  			Default:  defaultValue,
   429  			DataType: dt,
   430  		}
   431  		columns = append(columns, column)
   432  	}
   433  
   434  	rawIndexes := native["indexes"].([]interface{})
   435  	indexes := make([]*IndexSchema, 0, len(rawIndexes))
   436  	for _, raw := range rawIndexes {
   437  		raw := raw.(map[string]interface{})
   438  		rawColumns := raw["columns"].([]interface{})
   439  		keyColumns := make([]string, 0, len(rawColumns))
   440  		for _, rawColumn := range rawColumns {
   441  			keyColumns = append(keyColumns, rawColumn.(string))
   442  		}
   443  		index := &IndexSchema{
   444  			Name:     raw["name"].(string),
   445  			Unique:   raw["unique"].(bool),
   446  			Primary:  raw["primary"].(bool),
   447  			Nullable: raw["nullable"].(bool),
   448  			Columns:  keyColumns,
   449  		}
   450  		indexes = append(indexes, index)
   451  	}
   452  	return &TableSchema{
   453  		Schema:  native["database"].(string),
   454  		Table:   native["table"].(string),
   455  		TableID: native["tableID"].(int64),
   456  		Version: uint64(native["version"].(int64)),
   457  		Columns: columns,
   458  		Indexes: indexes,
   459  	}
   460  }
   461  
   462  func newMessageFromAvroNative(native interface{}, m *message) {
   463  	rawValues := native.(map[string]interface{})["com.pingcap.simple.avro.Message"].(map[string]interface{})
   464  	rawPayload := rawValues["payload"].(map[string]interface{})
   465  
   466  	rawMessage := rawPayload["com.pingcap.simple.avro.Watermark"]
   467  	if rawMessage != nil {
   468  		rawValues = rawMessage.(map[string]interface{})
   469  		m.Version = int(rawValues["version"].(int32))
   470  		m.Type = MessageTypeWatermark
   471  		m.CommitTs = uint64(rawValues["commitTs"].(int64))
   472  		m.BuildTs = rawValues["buildTs"].(int64)
   473  		return
   474  	}
   475  
   476  	rawMessage = rawPayload["com.pingcap.simple.avro.Bootstrap"]
   477  	if rawMessage != nil {
   478  		rawValues = rawMessage.(map[string]interface{})
   479  		m.Version = int(rawValues["version"].(int32))
   480  		m.Type = MessageTypeBootstrap
   481  		m.BuildTs = rawValues["buildTs"].(int64)
   482  		m.TableSchema = newTableSchemaFromAvroNative(rawValues["tableSchema"].(map[string]interface{}))
   483  		return
   484  	}
   485  
   486  	rawMessage = rawPayload["com.pingcap.simple.avro.DDL"]
   487  	if rawMessage != nil {
   488  		rawValues = rawMessage.(map[string]interface{})
   489  		m.Version = int(rawValues["version"].(int32))
   490  		m.Type = MessageType(rawValues["type"].(string))
   491  		m.SQL = rawValues["sql"].(string)
   492  		m.CommitTs = uint64(rawValues["commitTs"].(int64))
   493  		m.BuildTs = rawValues["buildTs"].(int64)
   494  
   495  		rawTableSchemaValues := rawValues["tableSchema"]
   496  		if rawTableSchemaValues != nil {
   497  			rawTableSchema := rawTableSchemaValues.(map[string]interface{})
   498  			rawTableSchema = rawTableSchema["com.pingcap.simple.avro.TableSchema"].(map[string]interface{})
   499  			m.TableSchema = newTableSchemaFromAvroNative(rawTableSchema)
   500  		}
   501  
   502  		rawPreTableSchemaValue := rawValues["preTableSchema"]
   503  		if rawPreTableSchemaValue != nil {
   504  			rawPreTableSchema := rawPreTableSchemaValue.(map[string]interface{})
   505  			rawPreTableSchema = rawPreTableSchema["com.pingcap.simple.avro.TableSchema"].(map[string]interface{})
   506  			m.PreTableSchema = newTableSchemaFromAvroNative(rawPreTableSchema)
   507  		}
   508  		return
   509  	}
   510  
   511  	rawValues = rawPayload["com.pingcap.simple.avro.DML"].(map[string]interface{})
   512  	m.Type = MessageType(rawValues["type"].(string))
   513  	m.Version = int(rawValues["version"].(int32))
   514  	m.CommitTs = uint64(rawValues["commitTs"].(int64))
   515  	m.BuildTs = rawValues["buildTs"].(int64)
   516  	m.Schema = rawValues["database"].(string)
   517  	m.Table = rawValues["table"].(string)
   518  	m.TableID = rawValues["tableID"].(int64)
   519  	m.SchemaVersion = uint64(rawValues["schemaVersion"].(int64))
   520  
   521  	if rawValues["handleKeyOnly"] != nil {
   522  		m.HandleKeyOnly = rawValues["handleKeyOnly"].(map[string]interface{})["boolean"].(bool)
   523  	}
   524  	if rawValues["claimCheckLocation"] != nil {
   525  		m.ClaimCheckLocation = rawValues["claimCheckLocation"].(map[string]interface{})["string"].(string)
   526  	}
   527  
   528  	m.Checksum = newChecksum(rawValues)
   529  	m.Data = newDataMap(rawValues["data"])
   530  	m.Old = newDataMap(rawValues["old"])
   531  }
   532  
   533  func newChecksum(raw map[string]interface{}) *checksum {
   534  	rawValue := raw["checksum"]
   535  	if rawValue == nil {
   536  		return nil
   537  	}
   538  	rawChecksum := rawValue.(map[string]interface{})
   539  	rawChecksum = rawChecksum["com.pingcap.simple.avro.Checksum"].(map[string]interface{})
   540  	return &checksum{
   541  		Version:   int(rawChecksum["version"].(int32)),
   542  		Corrupted: rawChecksum["corrupted"].(bool),
   543  		Current:   uint32(rawChecksum["current"].(int64)),
   544  		Previous:  uint32(rawChecksum["previous"].(int64)),
   545  	}
   546  }
   547  
   548  func newDataMap(rawValues interface{}) map[string]interface{} {
   549  	if rawValues == nil {
   550  		return nil
   551  	}
   552  	data := make(map[string]interface{})
   553  	rawDataMap := rawValues.(map[string]interface{})["map"].(map[string]interface{})
   554  	for key, value := range rawDataMap {
   555  		if value == nil {
   556  			data[key] = nil
   557  			continue
   558  		}
   559  		valueMap := value.(map[string]interface{})
   560  		for _, v := range valueMap {
   561  			data[key] = v
   562  		}
   563  	}
   564  	return data
   565  }