diff --git a/cdc/model/sink.go b/cdc/model/sink.go index eaa3740e505..9666d941a64 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -326,13 +326,35 @@ func (r *RowChangedEvent) HandleKeyColumns() []*Column { } if len(pkeyCols) == 0 { - // TODO redact the message - log.Panic("Cannot find handle key columns, bug?", zap.Reflect("event", r)) + log.Panic("Cannot find handle key columns.", zap.Any("event", r)) } return pkeyCols } +// PrimaryKeyColInfos returns the column(s) and colInfo(s) corresponding to the primary key(s) +func (r *RowChangedEvent) PrimaryKeyColInfos() ([]*Column, []rowcodec.ColInfo) { + pkeyCols := make([]*Column, 0) + pkeyColInfos := make([]rowcodec.ColInfo, 0) + + var cols []*Column + if r.IsDelete() { + cols = r.PreColumns + } else { + cols = r.Columns + } + + for i, col := range cols { + if col != nil && col.Flag.IsPrimaryKey() { + pkeyCols = append(pkeyCols, col) + pkeyColInfos = append(pkeyColInfos, r.ColInfos[i]) + } + } + + // It is okay not to have primary keys, so the empty array is an acceptable result + return pkeyCols, pkeyColInfos +} + // WithHandlePrimaryFlag set `HandleKeyFlag` and `PrimaryKeyFlag` func (r *RowChangedEvent) WithHandlePrimaryFlag(colNames map[string]struct{}) { for _, col := range r.Columns { diff --git a/cdc/sink/codec/avro.go b/cdc/sink/codec/avro.go index bc6c80c8f7f..116847c0714 100644 --- a/cdc/sink/codec/avro.go +++ b/cdc/sink/codec/avro.go @@ -18,11 +18,9 @@ import ( "context" "encoding/binary" "encoding/json" - "fmt" - "math" "math/big" "strconv" - "time" + "strings" "github.com/linkedin/goavro/v2" "github.com/pingcap/errors" @@ -30,20 +28,24 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/rowcodec" + "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/security" + "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" ) // AvroEventBatchEncoder converts the events to binary Avro data type AvroEventBatchEncoder struct { + namespace string keySchemaManager *AvroSchemaManager valueSchemaManager *AvroSchemaManager resultBuf []*MQMessage - tz *time.Location + enableTiDBExtension bool + decimalHandlingMode string + bigintUnsignedHandlingMode string } type avroEncodeResult struct { @@ -51,57 +53,36 @@ type avroEncodeResult struct { registryID int } -// newAvroEventBatchEncoder creates an AvroEventBatchEncoder -func newAvroEventBatchEncoder() *AvroEventBatchEncoder { - return &AvroEventBatchEncoder{ - valueSchemaManager: nil, - keySchemaManager: nil, - resultBuf: make([]*MQMessage, 0, 4096), - } -} - -// SetValueSchemaManager sets the value schema manager for an Avro encoder -func (a *AvroEventBatchEncoder) SetValueSchemaManager(manager *AvroSchemaManager) { - a.valueSchemaManager = manager -} - -// GetValueSchemaManager gets the value schema manager for an Avro encoder -func (a *AvroEventBatchEncoder) GetValueSchemaManager() *AvroSchemaManager { - return a.valueSchemaManager -} - -// SetKeySchemaManager sets the value schema manager for an Avro encoder -func (a *AvroEventBatchEncoder) SetKeySchemaManager(manager *AvroSchemaManager) { - a.keySchemaManager = manager -} - -// GetKeySchemaManager gets the value schema manager for an Avro encoder -func (a *AvroEventBatchEncoder) GetKeySchemaManager() *AvroSchemaManager { - return a.keySchemaManager -} - -// SetTimeZone sets the time-zone that is used to serialize Avro date-time types -func (a *AvroEventBatchEncoder) SetTimeZone(tz *time.Location) { - log.Debug("Setting Avro serializer timezone", zap.String("tz", tz.String())) - a.tz = tz -} - // AppendRowChangedEvent appends a row change event to the encoder // NOTE: the encoder can only store one RowChangedEvent! -func (a *AvroEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) error { - mqMessage := NewMQMessage(config.ProtocolAvro, nil, nil, e.CommitTs, model.MqMessageTypeRow, &e.Table.Schema, &e.Table.Table) +func (a *AvroEventBatchEncoder) AppendRowChangedEvent( + ctx context.Context, + topic string, + e *model.RowChangedEvent, +) error { + log.Debug("AppendRowChangedEvent", zap.Any("rowChangedEvent", e)) + mqMessage := NewMQMessage( + config.ProtocolAvro, + nil, + nil, + e.CommitTs, + model.MqMessageTypeRow, + &e.Table.Schema, + &e.Table.Table, + ) + topic = sanitizeTopic(topic) if !e.IsDelete() { - res, err := avroEncode(e.Table, a.valueSchemaManager, e.TableInfoVersion, e.Columns, e.ColInfos, a.tz) + res, err := a.avroEncode(ctx, e, topic, false) if err != nil { - log.Warn("AppendRowChangedEvent: avro encoding failed", zap.String("table", e.Table.String())) - return errors.Annotate(err, "AppendRowChangedEvent could not encode to Avro") + log.Error("AppendRowChangedEvent: avro encoding failed", zap.Error(err)) + return errors.Trace(err) } evlp, err := res.toEnvelope() if err != nil { - log.Warn("AppendRowChangedEvent: could not construct Avro envelope", zap.String("table", e.Table.String())) - return errors.Annotate(err, "AppendRowChangedEvent could not construct Avro envelope") + log.Error("AppendRowChangedEvent: could not construct Avro envelope", zap.Error(err)) + return errors.Trace(err) } mqMessage.Value = evlp @@ -109,21 +90,22 @@ func (a *AvroEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) mqMessage.Value = nil } - pkeyCols := e.HandleKeyColumns() - - res, err := avroEncode(e.Table, a.keySchemaManager, e.TableInfoVersion, pkeyCols, e.ColInfos, a.tz) + res, err := a.avroEncode(ctx, e, topic, true) if err != nil { - log.Warn("AppendRowChangedEvent: avro encoding failed", zap.String("table", e.Table.String())) - return errors.Annotate(err, "AppendRowChangedEvent could not encode to Avro") + log.Error("AppendRowChangedEvent: avro encoding failed", zap.Error(err)) + return errors.Trace(err) } - evlp, err := res.toEnvelope() - if err != nil { - log.Warn("AppendRowChangedEvent: could not construct Avro envelope", zap.String("table", e.Table.String())) - return errors.Annotate(err, "AppendRowChangedEvent could not construct Avro envelope") + if res != nil { + evlp, err := res.toEnvelope() + if err != nil { + log.Error("AppendRowChangedEvent: could not construct Avro envelope", zap.Error(err)) + return errors.Trace(err) + } + mqMessage.Key = evlp + } else { + mqMessage.Key = nil } - - mqMessage.Key = evlp mqMessage.IncRowsCount() a.resultBuf = append(a.resultBuf, mqMessage) @@ -160,30 +142,94 @@ func (a *AvroEventBatchEncoder) Size() int { return sum } -func avroEncode(table *model.TableName, manager *AvroSchemaManager, tableVersion uint64, cols []*model.Column, colInfos []rowcodec.ColInfo, tz *time.Location) (*avroEncodeResult, error) { +const ( + insertOperation = "c" + updateOperation = "u" +) + +func (a *AvroEventBatchEncoder) avroEncode( + ctx context.Context, + e *model.RowChangedEvent, + topic string, + isKey bool, +) (*avroEncodeResult, error) { + var ( + cols []*model.Column + colInfos []rowcodec.ColInfo + enableTiDBExtension bool + schemaManager *AvroSchemaManager + operation string + ) + if isKey { + cols, colInfos = e.PrimaryKeyColInfos() + enableTiDBExtension = false + schemaManager = a.keySchemaManager + } else { + cols = e.Columns + colInfos = e.ColInfos + enableTiDBExtension = a.enableTiDBExtension + schemaManager = a.valueSchemaManager + if e.IsInsert() { + operation = insertOperation + } else if e.IsUpdate() { + operation = updateOperation + } else { + log.Error("unknown operation", zap.Any("rowChangedEvent", e)) + return nil, cerror.ErrAvroEncodeFailed.GenWithStack("unknown operation") + } + } + + if len(cols) == 0 { + return nil, nil + } + + namespace := getAvroNamespace(a.namespace, e.Table) + schemaGen := func() (string, error) { - schema, err := ColumnInfoToAvroSchema(table.Table, cols) + schema, err := rowToAvroSchema( + namespace, + e.Table.Table, + cols, + colInfos, + enableTiDBExtension, + a.decimalHandlingMode, + a.bigintUnsignedHandlingMode, + ) if err != nil { - return "", errors.Annotate(err, "AvroEventBatchEncoder: generating schema failed") + log.Error("AvroEventBatchEncoder: generating schema failed", zap.Error(err)) + return "", errors.Trace(err) } return schema, nil } - // TODO pass ctx from the upper function. Need to modify the EventBatchEncoder interface. - avroCodec, registryID, err := manager.GetCachedOrRegister(context.Background(), *table, tableVersion, schemaGen) + avroCodec, registryID, err := schemaManager.GetCachedOrRegister( + ctx, + topic, + e.TableInfoVersion, + schemaGen, + ) if err != nil { - return nil, errors.Annotate(err, "AvroEventBatchEncoder: get-or-register failed") + return nil, errors.Trace(err) } - native, err := rowToAvroNativeData(cols, colInfos, tz) + native, err := rowToAvroData( + cols, + colInfos, + e.CommitTs, + operation, + enableTiDBExtension, + a.decimalHandlingMode, + a.bigintUnsignedHandlingMode, + ) if err != nil { - return nil, errors.Annotate(err, "AvroEventBatchEncoder: converting to native failed") + log.Error("AvroEventBatchEncoder: converting to native failed", zap.Error(err)) + return nil, errors.Trace(err) } bin, err := avroCodec.BinaryFromNative(nil, native) if err != nil { - return nil, errors.Annotate( - cerror.WrapError(cerror.ErrAvroEncodeToBinary, err), "AvroEventBatchEncoder: converting to Avro binary failed") + log.Error("AvroEventBatchEncoder: converting to Avro binary failed", zap.Error(err)) + return nil, cerror.WrapError(cerror.ErrAvroEncodeToBinary, err) } return &avroEncodeResult{ @@ -193,41 +239,155 @@ func avroEncode(table *model.TableName, manager *AvroSchemaManager, tableVersion } type avroSchemaTop struct { - Tp string `json:"type"` - Name string `json:"name"` - Fields []map[string]interface{} `json:"fields"` + Tp string `json:"type"` + Name string `json:"name"` + Namespace string `json:"namespace"` + Fields []map[string]interface{} `json:"fields"` } -type logicalType string +const ( + tidbType = "tidb_type" + tidbOp = "_tidb_op" + tidbCommitTs = "_tidb_commit_ts" + tidbPhysicalTime = "_tidb_commit_physical_time" +) -type avroLogicalType struct { - Type string `json:"type"` - LogicalType logicalType `json:"logicalType"` - Precision interface{} `json:"precision,omitempty"` - Scale interface{} `json:"scale,omitempty"` +var type2TiDBType = map[byte]string{ + mysql.TypeTiny: "INT", + mysql.TypeShort: "INT", + mysql.TypeInt24: "INT", + mysql.TypeLong: "INT", + mysql.TypeLonglong: "BIGINT", + mysql.TypeFloat: "FLOAT", + mysql.TypeDouble: "DOUBLE", + mysql.TypeBit: "BIT", + mysql.TypeNewDecimal: "DECIMAL", + mysql.TypeTinyBlob: "TEXT", + mysql.TypeMediumBlob: "TEXT", + mysql.TypeBlob: "TEXT", + mysql.TypeLongBlob: "TEXT", + mysql.TypeVarchar: "TEXT", + mysql.TypeVarString: "TEXT", + mysql.TypeString: "TEXT", + mysql.TypeEnum: "ENUM", + mysql.TypeSet: "SET", + mysql.TypeJSON: "JSON", + mysql.TypeDate: "DATE", + mysql.TypeDatetime: "DATETIME", + mysql.TypeTimestamp: "TIMESTAMP", + mysql.TypeDuration: "TIME", + mysql.TypeYear: "YEAR", +} + +func getTiDBTypeFromColumn(col *model.Column) string { + tt := type2TiDBType[col.Type] + if col.Flag.IsUnsigned() && (tt == "INT" || tt == "BIGINT") { + return tt + " UNSIGNED" + } + if col.Flag.IsBinary() && tt == "TEXT" { + return "BLOB" + } + return tt } const ( - timestampMillis logicalType = "timestamp-millis" - timeMillis logicalType = "time-millis" - decimalType logicalType = "decimal" + replacementChar = "_" + numberPrefix = "_" ) -// ColumnInfoToAvroSchema generates the Avro schema JSON for the corresponding columns -func ColumnInfoToAvroSchema(name string, columnInfo []*model.Column) (string, error) { +// sanitizeName escapes not permitted chars for avro +// debezium-core/src/main/java/io/debezium/schema/FieldNameSelector.java +// https://avro.apache.org/docs/current/spec.html#names +func sanitizeName(name string) string { + changed := false + var sb strings.Builder + for i, c := range name { + if i == 0 && (c >= '0' && c <= '9') { + sb.WriteString(numberPrefix) + sb.WriteRune(c) + changed = true + } else if !(c == '_' || + ('a' <= c && c <= 'z') || + ('A' <= c && c <= 'Z') || + ('0' <= c && c <= '9')) { + sb.WriteString(replacementChar) + changed = true + } else { + sb.WriteRune(c) + } + } + + sanitizedName := sb.String() + if changed { + log.Warn( + "Name is potentially not safe for serialization, replace it", + zap.String("name", name), + zap.String("replacedName", sanitizedName), + ) + } + return sanitizedName +} + +// sanitizeTopic escapes ".", it may has special meanings for sink connectors +func sanitizeTopic(name string) string { + return strings.ReplaceAll(name, ".", replacementChar) +} + +// https://github.com/debezium/debezium/blob/9f7ede0e0695f012c6c4e715e96aed85eecf6b5f \ +// /debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/antlr/ \ +// MySqlAntlrDdlParser.java#L374 +func escapeEnumAndSetOptions(option string) string { + option = strings.ReplaceAll(option, ",", "\\,") + option = strings.ReplaceAll(option, "\\'", "'") + option = strings.ReplaceAll(option, "''", "'") + return option +} + +func getAvroNamespace(namespace string, tableName *model.TableName) string { + return sanitizeName(namespace) + "." + sanitizeName(tableName.Schema) +} + +type avroSchema struct { + Type string `json:"type"` + // connect.parameters is designated field extracted by schema registry + Parameters map[string]string `json:"connect.parameters"` +} + +type avroLogicalTypeSchema struct { + avroSchema + LogicalType string `json:"logicalType"` + Precision interface{} `json:"precision,omitempty"` + Scale interface{} `json:"scale,omitempty"` +} + +func rowToAvroSchema( + namespace string, + name string, + columnInfo []*model.Column, + colInfos []rowcodec.ColInfo, + enableTiDBExtension bool, + decimalHandlingMode string, + bigintUnsignedHandlingMode string, +) (string, error) { top := avroSchemaTop{ - Tp: "record", - Name: name, - Fields: nil, + Tp: "record", + Name: sanitizeName(name), + Namespace: namespace, + Fields: nil, } - for _, col := range columnInfo { - avroType, err := getAvroDataTypeFromColumn(col) + for i, col := range columnInfo { + avroType, err := columnToAvroSchema( + col, + colInfos[i].Ft, + decimalHandlingMode, + bigintUnsignedHandlingMode, + ) if err != nil { return "", err } field := make(map[string]interface{}) - field["name"] = col.Name + field["name"] = sanitizeName(col.Name) if col.Flag.IsNullable() { field["type"] = []interface{}{"null", avroType} field["default"] = nil @@ -238,233 +398,262 @@ func ColumnInfoToAvroSchema(name string, columnInfo []*model.Column) (string, er top.Fields = append(top.Fields, field) } + if enableTiDBExtension { + top.Fields = append(top.Fields, + map[string]interface{}{ + "name": tidbOp, + "type": "string", + }, + map[string]interface{}{ + "name": tidbCommitTs, + "type": "long", + }, + map[string]interface{}{ + "name": tidbPhysicalTime, + "type": "long", + }, + ) + } + str, err := json.Marshal(&top) if err != nil { return "", cerror.WrapError(cerror.ErrAvroMarshalFailed, err) } - log.Debug("Avro Schema JSON generated", zap.ByteString("schema", str)) + log.Debug("rowToAvroSchema", zap.ByteString("schema", str)) return string(str), nil } -func rowToAvroNativeData(cols []*model.Column, colInfos []rowcodec.ColInfo, tz *time.Location) (interface{}, error) { +func rowToAvroData( + cols []*model.Column, + colInfos []rowcodec.ColInfo, + commitTs uint64, + operation string, + enableTiDBExtension bool, + decimalHandlingMode string, + bigintUnsignedHandlingMode string, +) (map[string]interface{}, error) { ret := make(map[string]interface{}, len(cols)) for i, col := range cols { if col == nil { continue } - data, str, err := columnToAvroNativeData(col, colInfos[i].Ft, tz) + data, str, err := columnToAvroData( + col, + colInfos[i].Ft, + decimalHandlingMode, + bigintUnsignedHandlingMode, + ) if err != nil { return nil, err } // https://pkg.go.dev/github.com/linkedin/goavro/v2#Union if col.Flag.IsNullable() { - ret[col.Name] = goavro.Union(str, data) + ret[sanitizeName(col.Name)] = goavro.Union(str, data) } else { - ret[col.Name] = data + ret[sanitizeName(col.Name)] = data } } - return ret, nil -} -func getAvroDataTypeFallback(v interface{}) (string, error) { - switch tp := v.(type) { - case bool: - return "boolean", nil - case []byte: - return "bytes", nil - case float64: - return "double", nil - case float32: - return "float", nil - case int64, uint64: - return "long", nil - case int, int32, uint32: - return "int", nil - case nil: - return "null", nil - case string: - return "string", nil - default: - log.Warn("getAvroDataTypeFallback: unknown type") - return "", cerror.ErrAvroUnknownType.GenWithStackByArgs(tp) + if enableTiDBExtension { + ret[tidbOp] = operation + ret[tidbCommitTs] = int64(commitTs) + ret[tidbPhysicalTime] = oracle.ExtractPhysical(commitTs) } -} -var unsignedLongAvroType = avroLogicalType{ - Type: "bytes", - LogicalType: decimalType, - Precision: 64, // enough to hold all values and is the default precision of confluent schema registry - Scale: 0, + log.Debug("rowToAvroData", zap.Any("data", ret)) + return ret, nil } -func getAvroDataTypeFromColumn(col *model.Column) (interface{}, error) { - log.Info("DEBUG: getAvroDataTypeFromColumn", zap.Reflect("col", col)) +func columnToAvroSchema( + col *model.Column, + ft *types.FieldType, + decimalHandlingMode string, + bigintUnsignedHandlingMode string, +) (interface{}, error) { + tt := getTiDBTypeFromColumn(col) switch col.Type { - case mysql.TypeFloat: - return "float", nil - case mysql.TypeDouble: - return "double", nil - case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString: - if col.Flag.IsBinary() { - return "bytes", nil + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24: + // BOOL/TINYINT/SMALLINT/MEDIUMINT + return avroSchema{ + Type: "int", + Parameters: map[string]string{tidbType: tt}, + }, nil + case mysql.TypeLong: // INT + if col.Flag.IsUnsigned() { + return avroSchema{ + Type: "long", + Parameters: map[string]string{tidbType: tt}, + }, nil } - return "string", nil - case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: - return avroLogicalType{ - Type: "long", - LogicalType: timestampMillis, + return avroSchema{ + Type: "int", + Parameters: map[string]string{tidbType: tt}, }, nil - case mysql.TypeDuration: - return avroLogicalType{ - Type: "int", - LogicalType: timeMillis, + case mysql.TypeLonglong: // BIGINT + if col.Flag.IsUnsigned() && bigintUnsignedHandlingMode == bigintUnsignedHandlingModeString { + return avroSchema{ + Type: "string", + Parameters: map[string]string{tidbType: tt}, + }, nil + } + return avroSchema{ + Type: "long", + Parameters: map[string]string{tidbType: tt}, + }, nil + case mysql.TypeFloat, mysql.TypeDouble: + return avroSchema{ + Type: "double", + Parameters: map[string]string{tidbType: tt}, }, nil - case mysql.TypeEnum: - return "string", nil - case mysql.TypeSet: - return "string", nil case mysql.TypeBit: - return unsignedLongAvroType, nil - case mysql.TypeNewDecimal: - return "string", nil - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24: - return "int", nil - case mysql.TypeLong: - if col.Flag.IsUnsigned() { - return "long", nil + displayFlen := ft.Flen + if displayFlen == -1 { + displayFlen, _ = mysql.GetDefaultFieldLengthAndDecimal(col.Type) } - return "int", nil - case mysql.TypeLonglong: - if col.Flag.IsUnsigned() { - return unsignedLongAvroType, nil + return avroSchema{ + Type: "bytes", + Parameters: map[string]string{ + tidbType: tt, + "length": strconv.Itoa(displayFlen), + }, + }, nil + case mysql.TypeNewDecimal: + if decimalHandlingMode == decimalHandlingModePrecise { + defaultFlen, defaultDecimal := mysql.GetDefaultFieldLengthAndDecimal(ft.Tp) + displayFlen, displayDecimal := ft.Flen, ft.Decimal + // length not specified, set it to system type default + if displayFlen == -1 { + displayFlen = defaultFlen + } + if displayDecimal == -1 { + displayDecimal = defaultDecimal + } + return avroLogicalTypeSchema{ + avroSchema: avroSchema{ + Type: "bytes", + Parameters: map[string]string{tidbType: tt}, + }, + LogicalType: "decimal", + Precision: displayFlen, + Scale: displayDecimal, + }, nil } - return "long", nil - case mysql.TypeNull: - return "null", nil - case mysql.TypeJSON: - return "string", nil - case mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: + // decimalHandlingMode == string + return avroSchema{ + Type: "string", + Parameters: map[string]string{tidbType: tt}, + }, nil + // TINYTEXT/MEDIUMTEXT/TEXT/LONGTEXT/CHAR/VARCHAR + // TINYBLOB/MEDIUMBLOB/BLOB/LONGBLOB/BINARY/VARBINARY + case mysql.TypeVarchar, + mysql.TypeString, + mysql.TypeVarString, + mysql.TypeTinyBlob, + mysql.TypeMediumBlob, + mysql.TypeLongBlob, + mysql.TypeBlob: if col.Flag.IsBinary() { - return "bytes", nil + return avroSchema{ + Type: "bytes", + Parameters: map[string]string{tidbType: tt}, + }, nil } - return "string", nil + return avroSchema{ + Type: "string", + Parameters: map[string]string{tidbType: tt}, + }, nil + case mysql.TypeEnum, mysql.TypeSet: + es := make([]string, 0, len(ft.Elems)) + for _, e := range ft.Elems { + e = escapeEnumAndSetOptions(e) + es = append(es, e) + } + return avroSchema{ + Type: "string", + Parameters: map[string]string{ + tidbType: tt, + "allowed": strings.Join(es, ","), + }, + }, nil + case mysql.TypeJSON: + return avroSchema{ + Type: "string", + Parameters: map[string]string{tidbType: tt}, + }, nil + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp, mysql.TypeDuration: + return avroSchema{ + Type: "string", + Parameters: map[string]string{tidbType: tt}, + }, nil case mysql.TypeYear: - return "long", nil + return avroSchema{ + Type: "int", + Parameters: map[string]string{tidbType: tt}, + }, nil default: - log.Panic("Unknown MySql type", zap.Reflect("mysqlType", col.Type)) - return "", errors.New("Unknown Mysql type") + log.Error("unknown mysql type", zap.Any("mysqlType", col.Type)) + return nil, cerror.ErrAvroEncodeFailed.GenWithStack("unknown mysql type") } } -var ( - zeroTimeStr = types.NewTime(types.ZeroCoreTime, mysql.TypeTimestamp, 0).String() - zeroDateStr = types.NewTime(types.ZeroCoreTime, mysql.TypeDate, 0).String() -) - -func columnToAvroNativeData(col *model.Column, ft *types.FieldType, tz *time.Location) (interface{}, string, error) { +func columnToAvroData( + col *model.Column, + ft *types.FieldType, + decimalHandlingMode string, + bigintUnsignedHandlingMode string, +) (interface{}, string, error) { if col.Value == nil { return nil, "null", nil } - handleUnsignedInt64 := func() (interface{}, string, error) { - var retVal interface{} - switch v := col.Value.(type) { - case uint64: - retVal = big.NewRat(0, 1).SetUint64(v) - case int64: - retVal = big.NewRat(0, 1).SetInt64(v) - } - return retVal, string("bytes." + decimalType), nil - } - switch col.Type { - case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate, mysql.TypeTimestamp: - // Refer to `unflatten` in cdc/entry/codec.go for why this piece of code is like this. - const fullType = "long." + timestampMillis - str := col.Value.(string) - - if (col.Type == mysql.TypeDate && str == zeroDateStr) || - (col.Type != mysql.TypeDate && str == zeroTimeStr) { - - return time.Time{}, string(fullType), nil - } - - var actualTz *time.Location - if col.Type != mysql.TypeTimestamp { - actualTz = time.UTC - } else { - actualTz = tz - } - - t, err := time.ParseInLocation(types.DateFormat, str, actualTz) - - if err == nil { - return t, string(fullType), nil - } - - t, err = time.ParseInLocation(types.TimeFormat, str, actualTz) - if err == nil { - return t, string(fullType), nil - } - - t, err = time.ParseInLocation(types.TimeFSPFormat, str, actualTz) - if err != nil { - return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) - } - return t, string(fullType), nil - case mysql.TypeDuration: - str := col.Value.(string) - var ( - hours int - minutes int - seconds int - frac string - ) - _, err := fmt.Sscanf(str, "%d:%d:%d.%s", &hours, &minutes, &seconds, &frac) - if err != nil { - _, err := fmt.Sscanf(str, "%d:%d:%d", &hours, &minutes, &seconds) - frac = "0" - - if err != nil { - return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) - } + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24: + if col.Flag.IsUnsigned() { + return int32(col.Value.(uint64)), "int", nil } - - fsp := len(frac) - fracInt, err := strconv.ParseInt(frac, 10, 32) - if err != nil { - return nil, "", err + return int32(col.Value.(int64)), "int", nil + case mysql.TypeLong: + if col.Flag.IsUnsigned() { + return int64(col.Value.(uint64)), "long", nil } - fracInt = int64(float64(fracInt) * math.Pow10(6-fsp)) - - d := types.NewDuration(hours, minutes, seconds, int(fracInt), fsp).Duration - const fullType = "int." + timeMillis - return d, string(fullType), nil - case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeTinyBlob, mysql.TypeBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: - if col.Flag.IsBinary() { - switch val := col.Value.(type) { - case string: - return []byte(val), "bytes", nil - case []byte: - return val, "bytes", nil - } - } else { - switch val := col.Value.(type) { - case string: - return val, "string", nil - case []byte: - return string(val), "string", nil + return int32(col.Value.(int64)), "int", nil + case mysql.TypeLonglong: + if col.Flag.IsUnsigned() { + if bigintUnsignedHandlingMode == bigintUnsignedHandlingModeLong { + return int64(col.Value.(uint64)), "long", nil } + // bigintUnsignedHandlingMode == "string" + return strconv.FormatUint(col.Value.(uint64), 10), "string", nil } - log.Panic("Avro could not process text-like type", zap.Reflect("col", col)) - return nil, "", errors.New("Unknown datum type") - case mysql.TypeYear: return col.Value.(int64), "long", nil - case mysql.TypeJSON: - return col.Value.(string), "string", nil + case mysql.TypeFloat, mysql.TypeDouble: + return col.Value.(float64), "double", nil + case mysql.TypeBit: + return []byte(types.NewBinaryLiteralFromUint(col.Value.(uint64), -1)), "bytes", nil case mysql.TypeNewDecimal: + if decimalHandlingMode == decimalHandlingModePrecise { + v, succ := new(big.Rat).SetString(col.Value.(string)) + if !succ { + return nil, "", cerror.ErrAvroEncodeFailed.GenWithStack( + "fail to encode Decimal value", + ) + } + return v, "bytes.decimal", nil + } + // decimalHandlingMode == "string" return col.Value.(string), "string", nil + case mysql.TypeVarchar, + mysql.TypeString, + mysql.TypeVarString, + mysql.TypeTinyBlob, + mysql.TypeBlob, + mysql.TypeMediumBlob, + mysql.TypeLongBlob: + if col.Flag.IsBinary() { + return col.Value, "bytes", nil + } + return string(col.Value.([]byte)), "string", nil case mysql.TypeEnum: enumVar, err := types.ParseEnumValue(ft.Elems, col.Value.(uint64)) if err != nil { @@ -477,36 +666,23 @@ func columnToAvroNativeData(col *model.Column, ft *types.FieldType, tz *time.Loc return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) } return setVar.Name, "string", nil - case mysql.TypeBit: - return handleUnsignedInt64() - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24: - if col.Flag.IsUnsigned() { - return int32(col.Value.(uint64)), "int", nil - } - return int32(col.Value.(int64)), "int", nil - case mysql.TypeLong: - if col.Flag.IsUnsigned() { - return int64(col.Value.(uint64)), "long", nil - } + case mysql.TypeJSON: + return col.Value.(string), "string", nil + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp, mysql.TypeDuration: + return col.Value.(string), "string", nil + case mysql.TypeYear: return col.Value.(int64), "int", nil - case mysql.TypeLonglong: - if col.Flag.IsUnsigned() { - return handleUnsignedInt64() - } - return col.Value.(int64), "long", nil - case mysql.TypeFloat: - return col.Value.(float64), "float", nil default: - avroType, err := getAvroDataTypeFallback(col.Value) - if err != nil { - return nil, "", err - } - return col.Value, avroType, nil + log.Error("unknown mysql type", zap.Any("mysqlType", col.Type)) + return nil, "", cerror.ErrAvroEncodeFailed.GenWithStack("unknown mysql type") } } const magicByte = uint8(0) +// confluent avro wire format, confluent avro is not same as apache avro +// https://rmoff.net/2020/07/03/why-json-isnt-the-same-as-json-schema-in-kafka-connect-converters \ +// -and-ksqldb-viewing-kafka-messages-bytes-as-hex/ func (r *avroEncodeResult) toEnvelope() ([]byte, error) { buf := new(bytes.Buffer) data := []interface{}{magicByte, int32(r.registryID), r.data} @@ -520,10 +696,10 @@ func (r *avroEncodeResult) toEnvelope() ([]byte, error) { } type avroEventBatchEncoderBuilder struct { + namespace string config *Config keySchemaManager *AvroSchemaManager valueSchemaManager *AvroSchemaManager - tz *time.Location } const ( @@ -531,32 +707,45 @@ const ( valueSchemaSuffix = "-value" ) -func newAvroEventBatchEncoderBuilder(credential *security.Credential, config *Config) (EncoderBuilder, error) { - ctx := context.Background() - keySchemaManager, err := NewAvroSchemaManager(ctx, credential, config.avroRegistry, keySchemaSuffix) +func newAvroEventBatchEncoderBuilder(ctx context.Context, config *Config) (EncoderBuilder, error) { + keySchemaManager, err := NewAvroSchemaManager( + ctx, + nil, + config.avroSchemaRegistry, + keySchemaSuffix, + ) if err != nil { return nil, errors.Trace(err) } - valueSchemaManager, err := NewAvroSchemaManager(ctx, credential, config.avroRegistry, valueSchemaSuffix) + valueSchemaManager, err := NewAvroSchemaManager( + ctx, + nil, + config.avroSchemaRegistry, + valueSchemaSuffix, + ) if err != nil { return nil, errors.Trace(err) } return &avroEventBatchEncoderBuilder{ + namespace: contextutil.ChangefeedIDFromCtx(ctx).Namespace, config: config, keySchemaManager: keySchemaManager, valueSchemaManager: valueSchemaManager, - tz: config.tz, }, nil } // Build an AvroEventBatchEncoder. func (b *avroEventBatchEncoderBuilder) Build() EventBatchEncoder { - encoder := newAvroEventBatchEncoder() - encoder.SetKeySchemaManager(b.keySchemaManager) - encoder.SetValueSchemaManager(b.valueSchemaManager) - encoder.SetTimeZone(b.tz) + encoder := &AvroEventBatchEncoder{} + encoder.namespace = b.namespace + encoder.keySchemaManager = b.keySchemaManager + encoder.valueSchemaManager = b.valueSchemaManager + encoder.resultBuf = make([]*MQMessage, 0, 4096) + encoder.enableTiDBExtension = b.config.enableTiDBExtension + encoder.decimalHandlingMode = b.config.avroDecimalHandlingMode + encoder.bigintUnsignedHandlingMode = b.config.avroBigintUnsignedHandlingMode return encoder } diff --git a/cdc/sink/codec/avro_test.go b/cdc/sink/codec/avro_test.go index 0f1de5f5e94..22265273f13 100644 --- a/cdc/sink/codec/avro_test.go +++ b/cdc/sink/codec/avro_test.go @@ -14,56 +14,60 @@ package codec import ( + "bytes" "context" "encoding/json" - "io" - "net/http" + "math/big" "testing" - "time" - "github.com/jarcoal/httpmock" "github.com/linkedin/goavro/v2" - "github.com/pingcap/check" - "github.com/pingcap/errors" - "github.com/pingcap/log" - model2 "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/puller" - "github.com/pingcap/tiflow/pkg/regionspan" - "github.com/pingcap/tiflow/pkg/security" - "github.com/pingcap/tiflow/pkg/util/testleak" - "go.uber.org/zap" + "github.com/stretchr/testify/require" ) -func Test(t *testing.T) { check.TestingT(t) } - -type avroBatchEncoderSuite struct { - encoder *AvroEventBatchEncoder -} - -var _ = check.Suite(&avroBatchEncoderSuite{}) - -func (s *avroBatchEncoderSuite) SetUpSuite(c *check.C) { - startAvroHTTPInterceptForTestingRegistry(c) - - keyManager, err := NewAvroSchemaManager(context.Background(), &security.Credential{}, "http://127.0.0.1:8081", "-key") - c.Assert(err, check.IsNil) - - valueManager, err := NewAvroSchemaManager(context.Background(), &security.Credential{}, "http://127.0.0.1:8081", "-value") - c.Assert(err, check.IsNil) +func setupEncoderAndSchemaRegistry( + enableTiDBExtension bool, + decimalHandlingMode string, + bigintUnsignedHandlingMode string, +) (*AvroEventBatchEncoder, error) { + startHTTPInterceptForTestingRegistry() + + keyManager, err := NewAvroSchemaManager( + context.Background(), + nil, + "http://127.0.0.1:8081", + "-key", + ) + if err != nil { + return nil, err + } - s.encoder = &AvroEventBatchEncoder{ - valueSchemaManager: valueManager, - keySchemaManager: keyManager, - resultBuf: make([]*MQMessage, 0, 4096), + valueManager, err := NewAvroSchemaManager( + context.Background(), + nil, + "http://127.0.0.1:8081", + "-value", + ) + if err != nil { + return nil, err } + + return &AvroEventBatchEncoder{ + namespace: model.DefaultNamespace, + valueSchemaManager: valueManager, + keySchemaManager: keyManager, + resultBuf: make([]*MQMessage, 0, 4096), + enableTiDBExtension: enableTiDBExtension, + decimalHandlingMode: decimalHandlingMode, + bigintUnsignedHandlingMode: bigintUnsignedHandlingMode, + }, nil } -func (s *avroBatchEncoderSuite) TearDownSuite(c *check.C) { - stopAvroHTTPInterceptForTestingRegistry() +func teardownEncoderAndSchemaRegistry() { + stopHTTPInterceptForTestingRegistry() } func setBinChsClnFlag(ft *types.FieldType) *types.FieldType { @@ -81,226 +85,760 @@ func setElems(ft *types.FieldType, elems []string) *types.FieldType { return ft } -func (s *avroBatchEncoderSuite) TestAvroEncodeOnly(c *check.C) { - defer testleak.AfterTest(c)() - - table := model.TableName{ - Schema: "testdb", - Table: "TestAvroEncodeOnly", - } +type avroTestColumnTuple struct { + col model.Column + colInfo rowcodec.ColInfo + expectedSchema interface{} + expectedData interface{} + expectedType string +} - cols := []*model.Column{ - {Name: "id", Value: int64(1), Type: mysql.TypeLong}, - {Name: "myint", Value: int64(2), Type: mysql.TypeLong}, - {Name: "mybool", Value: int64(1), Type: mysql.TypeTiny}, - {Name: "myfloat", Value: float64(3.14), Type: mysql.TypeFloat}, - {Name: "mybytes1", Value: []byte("Hello World"), Flag: model.BinaryFlag, Type: mysql.TypeBlob}, - {Name: "mybytes2", Value: []byte("Hello World"), Flag: model.BinaryFlag, Type: mysql.TypeMediumBlob}, - {Name: "mybytes3", Value: []byte("Hello World"), Flag: model.BinaryFlag, Type: mysql.TypeTinyBlob}, - {Name: "mybytes4", Value: []byte("Hello World"), Flag: model.BinaryFlag, Type: mysql.TypeLongBlob}, - {Name: "mybytes5", Value: []byte("Hello World"), Flag: model.BinaryFlag, Type: mysql.TypeVarString}, - {Name: "mybytes6", Value: []byte("Hello World"), Flag: model.BinaryFlag, Type: mysql.TypeString}, - {Name: "mybytes7", Value: []byte("Hello World"), Flag: model.BinaryFlag, Type: mysql.TypeVarchar}, - {Name: "mystring1", Value: "Hello World", Type: mysql.TypeBlob}, - {Name: "mystring2", Value: "Hello World", Type: mysql.TypeMediumBlob}, - {Name: "mystring3", Value: "Hello World", Type: mysql.TypeTinyBlob}, - {Name: "mystring4", Value: "Hello World", Type: mysql.TypeLongBlob}, - {Name: "mystring5", Value: "Hello World", Type: mysql.TypeVarString}, - {Name: "mystring6", Value: "Hello World", Type: mysql.TypeString}, - {Name: "mystring7", Value: "Hello World", Type: mysql.TypeVarchar}, - {Name: "myenum", Value: uint64(1), Type: mysql.TypeEnum}, - {Name: "myset", Value: uint64(1), Type: mysql.TypeSet}, - {Name: "ts", Value: time.Now().Format(types.TimeFSPFormat), Type: mysql.TypeTimestamp}, - {Name: "myjson", Value: "{\"foo\": \"bar\"}", Type: mysql.TypeJSON}, - } +var avroTestColumns = []*avroTestColumnTuple{ + { + model.Column{Name: "tiny", Value: int64(1), Type: mysql.TypeTiny}, + rowcodec.ColInfo{ + ID: 1, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeTiny), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT"}}, + int32(1), "int", + }, + { + model.Column{Name: "short", Value: int64(1), Type: mysql.TypeShort}, + rowcodec.ColInfo{ + ID: 2, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeShort), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT"}}, + int32(1), "int", + }, + { + model.Column{Name: "int24", Value: int64(1), Type: mysql.TypeInt24}, + rowcodec.ColInfo{ + ID: 3, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeInt24), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT"}}, + int32(1), "int", + }, + { + model.Column{Name: "long", Value: int64(1), Type: mysql.TypeLong}, + rowcodec.ColInfo{ + ID: 4, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeLong), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT"}}, + int32(1), "int", + }, + { + model.Column{Name: "longlong", Value: int64(1), Type: mysql.TypeLonglong}, + rowcodec.ColInfo{ + ID: 5, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeLonglong), + }, + avroSchema{Type: "long", Parameters: map[string]string{"tidb_type": "BIGINT"}}, + int64(1), "long", + }, + { + model.Column{ + Name: "tinyunsigned", + Value: uint64(1), + Type: mysql.TypeTiny, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 6, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setFlag(types.NewFieldType(mysql.TypeTiny), uint(model.UnsignedFlag)), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, + int32(1), "int", + }, + { + model.Column{ + Name: "shortunsigned", + Value: uint64(1), + Type: mysql.TypeShort, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 7, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setFlag(types.NewFieldType(mysql.TypeShort), uint(model.UnsignedFlag)), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, + int32(1), "int", + }, + { + model.Column{ + Name: "int24unsigned", + Value: uint64(1), + Type: mysql.TypeInt24, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 8, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setFlag(types.NewFieldType(mysql.TypeInt24), uint(model.UnsignedFlag)), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, + int32(1), "int", + }, + { + model.Column{ + Name: "longunsigned", + Value: uint64(1), + Type: mysql.TypeLong, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 9, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setFlag(types.NewFieldType(mysql.TypeLong), uint(model.UnsignedFlag)), + }, + avroSchema{Type: "long", Parameters: map[string]string{"tidb_type": "INT UNSIGNED"}}, + int64(1), "long", + }, + { + model.Column{ + Name: "longlongunsigned", + Value: uint64(1), + Type: mysql.TypeLonglong, + Flag: model.UnsignedFlag, + }, + rowcodec.ColInfo{ + ID: 10, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setFlag( + types.NewFieldType(mysql.TypeLonglong), + uint(model.UnsignedFlag), + ), + }, + avroSchema{Type: "long", Parameters: map[string]string{"tidb_type": "BIGINT UNSIGNED"}}, + int64(1), "long", + }, + { + model.Column{Name: "float", Value: float64(3.14), Type: mysql.TypeFloat}, + rowcodec.ColInfo{ + ID: 11, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeFloat), + }, + avroSchema{Type: "double", Parameters: map[string]string{"tidb_type": "FLOAT"}}, + float64(3.14), "double", + }, + { + model.Column{Name: "double", Value: float64(3.14), Type: mysql.TypeDouble}, + rowcodec.ColInfo{ + ID: 12, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeDouble), + }, + avroSchema{Type: "double", Parameters: map[string]string{"tidb_type": "DOUBLE"}}, + float64(3.14), "double", + }, + { + model.Column{Name: "bit", Value: uint64(683), Type: mysql.TypeBit}, + rowcodec.ColInfo{ + ID: 13, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeBit), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BIT", "length": "1"}}, + []byte("\x02\xab"), "bytes", + }, + { + model.Column{Name: "decimal", Value: "129012.1230000", Type: mysql.TypeNewDecimal}, + rowcodec.ColInfo{ + ID: 14, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeNewDecimal), + }, + avroLogicalTypeSchema{ + avroSchema: avroSchema{ + Type: "bytes", + Parameters: map[string]string{"tidb_type": "DECIMAL"}, + }, + LogicalType: "decimal", + Precision: 10, + Scale: 0, + }, + big.NewRat(129012123, 1000), "bytes.decimal", + }, + { + model.Column{Name: "tinytext", Value: []byte("hello world"), Type: mysql.TypeTinyBlob}, + rowcodec.ColInfo{ + ID: 15, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeBlob), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, + "hello world", "string", + }, + { + model.Column{Name: "mediumtext", Value: []byte("hello world"), Type: mysql.TypeMediumBlob}, + rowcodec.ColInfo{ + ID: 16, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeMediumBlob), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, + "hello world", "string", + }, + { + model.Column{Name: "text", Value: []byte("hello world"), Type: mysql.TypeBlob}, + rowcodec.ColInfo{ + ID: 17, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeBlob), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, + "hello world", "string", + }, + { + model.Column{Name: "longtext", Value: []byte("hello world"), Type: mysql.TypeLongBlob}, + rowcodec.ColInfo{ + ID: 18, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeLongBlob), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, + "hello world", "string", + }, + { + model.Column{Name: "varchar", Value: []byte("hello world"), Type: mysql.TypeVarchar}, + rowcodec.ColInfo{ + ID: 19, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeVarchar), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, + "hello world", "string", + }, + { + model.Column{Name: "varstring", Value: []byte("hello world"), Type: mysql.TypeVarString}, + rowcodec.ColInfo{ + ID: 20, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeVarString), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, + "hello world", "string", + }, + { + model.Column{Name: "string", Value: []byte("hello world"), Type: mysql.TypeString}, + rowcodec.ColInfo{ + ID: 21, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeString), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TEXT"}}, + "hello world", "string", + }, + { + model.Column{ + Name: "tinyblob", + Value: []byte("hello world"), + Type: mysql.TypeTinyBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 22, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeTinyBlob)), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, + []byte("hello world"), "bytes", + }, + { + model.Column{ + Name: "mediumblob", + Value: []byte("hello world"), + Type: mysql.TypeMediumBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 23, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeMediumBlob)), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, + []byte("hello world"), "bytes", + }, + { + model.Column{ + Name: "blob", + Value: []byte("hello world"), + Type: mysql.TypeBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 24, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeBlob)), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, + []byte("hello world"), "bytes", + }, + { + model.Column{ + Name: "longblob", + Value: []byte("hello world"), + Type: mysql.TypeLongBlob, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 25, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeLongBlob)), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, + []byte("hello world"), "bytes", + }, + { + model.Column{ + Name: "varbinary", + Value: []byte("hello world"), + Type: mysql.TypeVarchar, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 26, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeVarchar)), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, + []byte("hello world"), "bytes", + }, + { + model.Column{ + Name: "varbinary1", + Value: []byte("hello world"), + Type: mysql.TypeVarString, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 27, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeVarString)), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, + []byte("hello world"), "bytes", + }, + { + model.Column{ + Name: "binary", + Value: []byte("hello world"), + Type: mysql.TypeString, + Flag: model.BinaryFlag, + }, + rowcodec.ColInfo{ + ID: 28, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeString)), + }, + avroSchema{Type: "bytes", Parameters: map[string]string{"tidb_type": "BLOB"}}, + []byte("hello world"), "bytes", + }, + { + model.Column{Name: "enum", Value: uint64(1), Type: mysql.TypeEnum}, + rowcodec.ColInfo{ + ID: 29, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setElems(types.NewFieldType(mysql.TypeEnum), []string{"a,", "b"}), + }, + avroSchema{ + Type: "string", + Parameters: map[string]string{"tidb_type": "ENUM", "allowed": "a\\,,b"}, + }, + "a,", "string", + }, + { + model.Column{Name: "set", Value: uint64(1), Type: mysql.TypeSet}, + rowcodec.ColInfo{ + ID: 30, + IsPKHandle: false, + VirtualGenCol: false, + Ft: setElems(types.NewFieldType(mysql.TypeSet), []string{"a,", "b"}), + }, + avroSchema{ + Type: "string", + Parameters: map[string]string{"tidb_type": "SET", "allowed": "a\\,,b"}, + }, + "a,", "string", + }, + { + model.Column{Name: "json", Value: `{"key": "value"}`, Type: mysql.TypeJSON}, + rowcodec.ColInfo{ + ID: 31, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeJSON), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "JSON"}}, + `{"key": "value"}`, "string", + }, + { + model.Column{Name: "date", Value: "2000-01-01", Type: mysql.TypeDate}, + rowcodec.ColInfo{ + ID: 32, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeDate), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "DATE"}}, + "2000-01-01", "string", + }, + { + model.Column{Name: "datetime", Value: "2015-12-20 23:58:58", Type: mysql.TypeDatetime}, + rowcodec.ColInfo{ + ID: 33, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeDatetime), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "DATETIME"}}, + "2015-12-20 23:58:58", "string", + }, + { + model.Column{Name: "timestamp", Value: "1973-12-30 15:30:00", Type: mysql.TypeTimestamp}, + rowcodec.ColInfo{ + ID: 34, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeTimestamp), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TIMESTAMP"}}, + "1973-12-30 15:30:00", "string", + }, + { + model.Column{Name: "time", Value: "23:59:59", Type: mysql.TypeDuration}, + rowcodec.ColInfo{ + ID: 35, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeDuration), + }, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "TIME"}}, + "23:59:59", "string", + }, + { + model.Column{Name: "year", Value: int64(1970), Type: mysql.TypeYear}, + rowcodec.ColInfo{ + ID: 36, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeYear), + }, + avroSchema{Type: "int", Parameters: map[string]string{"tidb_type": "YEAR"}}, + int64(1970), "int", + }, +} - colInfos := []rowcodec.ColInfo{ - {ID: 1, IsPKHandle: true, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeLong)}, - {ID: 2, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeLong)}, - {ID: 3, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeTiny)}, - {ID: 4, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeFloat)}, - {ID: 5, IsPKHandle: false, VirtualGenCol: false, Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeBlob))}, - {ID: 6, IsPKHandle: false, VirtualGenCol: false, Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeMediumBlob))}, - {ID: 7, IsPKHandle: false, VirtualGenCol: false, Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeTinyBlob))}, - {ID: 8, IsPKHandle: false, VirtualGenCol: false, Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeLongBlob))}, - {ID: 9, IsPKHandle: false, VirtualGenCol: false, Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeVarString))}, - {ID: 10, IsPKHandle: false, VirtualGenCol: false, Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeString))}, - {ID: 11, IsPKHandle: false, VirtualGenCol: false, Ft: setBinChsClnFlag(types.NewFieldType(mysql.TypeVarchar))}, - {ID: 12, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeBlob)}, - {ID: 13, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeMediumBlob)}, - {ID: 14, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeTinyBlob)}, - {ID: 15, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeLongBlob)}, - {ID: 16, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeVarString)}, - {ID: 17, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeString)}, - {ID: 18, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeVarchar)}, - {ID: 19, IsPKHandle: false, VirtualGenCol: false, Ft: setElems(types.NewFieldType(mysql.TypeEnum), []string{"a", "b"})}, - {ID: 20, IsPKHandle: false, VirtualGenCol: false, Ft: setElems(types.NewFieldType(mysql.TypeSet), []string{"a", "b"})}, - {ID: 21, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeTimestamp)}, - {ID: 22, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeJSON)}, +func TestColumnToAvroSchema(t *testing.T) { + for _, v := range avroTestColumns { + schema, err := columnToAvroSchema(&v.col, v.colInfo.Ft, "precise", "long") + require.NoError(t, err) + require.Equal(t, v.expectedSchema, schema) + if v.col.Name == "decimal" { + schema, err := columnToAvroSchema(&v.col, v.colInfo.Ft, "string", "long") + require.NoError(t, err) + require.Equal( + t, + avroSchema{Type: "string", Parameters: map[string]string{"tidb_type": "DECIMAL"}}, + schema, + ) + } + if v.col.Name == "longlongunsigned" { + schema, err := columnToAvroSchema(&v.col, v.colInfo.Ft, "precise", "string") + require.NoError(t, err) + require.Equal( + t, + avroSchema{ + Type: "string", + Parameters: map[string]string{"tidb_type": "BIGINT UNSIGNED"}, + }, + schema, + ) + } } +} - schema, err := ColumnInfoToAvroSchema(table.Table, cols) - c.Assert(err, check.IsNil) - avroCodec, err := goavro.NewCodec(schema) - c.Assert(err, check.IsNil) - - r, err := avroEncode(&table, s.encoder.valueSchemaManager, 1, cols, colInfos, time.Local) - c.Assert(err, check.IsNil) - - res, _, err := avroCodec.NativeFromBinary(r.data) - c.Check(err, check.IsNil) - c.Check(res, check.NotNil) - for k, v := range res.(map[string]interface{}) { - if k == "myenum" || k == "myset" { - if vmap, ok := v.(map[string]interface{}); ok { - _, exists := vmap["string"] - c.Check(exists, check.IsTrue) - } +func TestColumnToAvroData(t *testing.T) { + t.Parallel() + + for _, v := range avroTestColumns { + data, str, err := columnToAvroData(&v.col, v.colInfo.Ft, "precise", "long") + require.NoError(t, err) + require.Equal(t, v.expectedData, data) + require.Equal(t, v.expectedType, str) + if v.col.Name == "decimal" { + data, str, err := columnToAvroData(&v.col, v.colInfo.Ft, "string", "long") + require.NoError(t, err) + require.Equal(t, "129012.1230000", data) + require.Equal(t, "string", str) + } + if v.col.Name == "longlongunsigned" { + data, str, err := columnToAvroData(&v.col, v.colInfo.Ft, "precise", "string") + require.NoError(t, err) + require.Equal(t, "1", data) + require.Equal(t, "string", str) } } +} - txt, err := avroCodec.TextualFromNative(nil, res) - c.Check(err, check.IsNil) - log.Info("TestAvroEncodeOnly", zap.ByteString("result", txt)) +func indentJSON(j string) string { + var buf bytes.Buffer + _ = json.Indent(&buf, []byte(j), "", " ") + return buf.String() } -func (s *avroBatchEncoderSuite) TestAvroNull(c *check.C) { - defer testleak.AfterTest(c)() +func TestRowToAvroSchema(t *testing.T) { + t.Parallel() table := model.TableName{ Schema: "testdb", - Table: "TestAvroNull", + Table: "rowtoavroschema", + } + namespace := getAvroNamespace(model.DefaultNamespace, &table) + var cols []*model.Column = make([]*model.Column, 0) + var colInfos []rowcodec.ColInfo = make([]rowcodec.ColInfo, 0) + + for _, v := range avroTestColumns { + cols = append(cols, &v.col) + colInfos = append(colInfos, v.colInfo) + colNew := v.col + colNew.Name = colNew.Name + "nullable" + colNew.Value = nil + colNew.Flag.SetIsNullable() + cols = append(cols, &colNew) + colInfos = append(colInfos, v.colInfo) } - cols := []*model.Column{ - {Name: "id", Value: int64(1), Flag: model.HandleKeyFlag, Type: mysql.TypeLong}, - {Name: "colNullable", Value: nil, Flag: model.NullableFlag, Type: mysql.TypeLong}, - {Name: "colNotnull", Value: int64(0), Type: mysql.TypeLong}, - {Name: "colNullable1", Value: int64(0), Flag: model.NullableFlag, Type: mysql.TypeLong}, + schema, err := rowToAvroSchema( + namespace, + table.Table, + cols, + colInfos, + false, + "precise", + "long", + ) + require.NoError(t, err) + require.Equal(t, expectedSchemaWithoutExtension, indentJSON(schema)) + _, err = goavro.NewCodec(schema) + require.NoError(t, err) + + schema, err = rowToAvroSchema( + namespace, + table.Table, + cols, + colInfos, + true, + "precise", + "long", + ) + require.NoError(t, err) + require.Equal(t, expectedSchemaWithExtension, indentJSON(schema)) + _, err = goavro.NewCodec(schema) + require.NoError(t, err) +} + +func TestRowToAvroData(t *testing.T) { + t.Parallel() + + var cols []*model.Column = make([]*model.Column, 0) + var colInfos []rowcodec.ColInfo = make([]rowcodec.ColInfo, 0) + + for _, v := range avroTestColumns { + cols = append(cols, &v.col) + colInfos = append(colInfos, v.colInfo) + colNew := v.col + colNew.Name = colNew.Name + "nullable" + colNew.Value = nil + colNew.Flag.SetIsNullable() + cols = append(cols, &colNew) + colInfos = append(colInfos, v.colInfo) } - colInfos := []rowcodec.ColInfo{ - {ID: 1, IsPKHandle: true, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeLong)}, - {ID: 2, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeLong)}, - { - ID: 3, IsPKHandle: false, VirtualGenCol: false, - Ft: setFlag(types.NewFieldType(mysql.TypeLong), uint(model.NullableFlag)), + data, err := rowToAvroData(cols, colInfos, 417318403368288260, "c", false, "precise", "long") + require.NoError(t, err) + _, exists := data["_tidb_commit_ts"] + require.False(t, exists) + _, exists = data["_tidb_op"] + require.False(t, exists) + _, exists = data["_tidb_commit_physical_time"] + require.False(t, exists) + + data, err = rowToAvroData(cols, colInfos, 417318403368288260, "c", true, "precise", "long") + require.NoError(t, err) + v, exists := data["_tidb_commit_ts"] + require.True(t, exists) + require.Equal(t, int64(417318403368288260), v.(int64)) + v, exists = data["_tidb_commit_physical_time"] + require.True(t, exists) + require.Equal(t, int64(1591943372224), v.(int64)) + v, exists = data["_tidb_op"] + require.True(t, exists) + require.Equal(t, "c", v.(string)) +} + +func TestAvroEncode(t *testing.T) { + encoder, err := setupEncoderAndSchemaRegistry(true, "precise", "long") + require.NoError(t, err) + defer teardownEncoderAndSchemaRegistry() + + var cols []*model.Column = make([]*model.Column, 0) + var colInfos []rowcodec.ColInfo = make([]rowcodec.ColInfo, 0) + + cols = append( + cols, + &model.Column{ + Name: "id", + Value: int64(1), + Type: mysql.TypeLong, + Flag: model.PrimaryKeyFlag, }, - {ID: 4, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeLong)}, + ) + colInfos = append( + colInfos, + rowcodec.ColInfo{ + ID: 1000, + IsPKHandle: false, + VirtualGenCol: false, + Ft: types.NewFieldType(mysql.TypeLong), + }, + ) + + for _, v := range avroTestColumns { + cols = append(cols, &v.col) + colInfos = append(colInfos, v.colInfo) + colNew := v.col + colNew.Name = colNew.Name + "nullable" + colNew.Value = nil + colNew.Flag.SetIsNullable() + cols = append(cols, &colNew) + colInfos = append(colInfos, v.colInfo) } - schema, err := ColumnInfoToAvroSchema(table.Table, cols) - c.Assert(err, check.IsNil) - var schemaObj avroSchemaTop - err = json.Unmarshal([]byte(schema), &schemaObj) - c.Assert(err, check.IsNil) - for _, v := range schemaObj.Fields { - if v["name"] == "colNullable" { - c.Assert(v["type"], check.DeepEquals, []interface{}{"null", "int"}) - } - if v["name"] == "colNotnull" { - c.Assert(v["type"], check.Equals, "int") - } + event := &model.RowChangedEvent{ + CommitTs: 417318403368288260, + Table: &model.TableName{ + Schema: "testdb", + Table: "avroencode", + }, + Columns: cols, + ColInfos: colInfos, } - native, err := rowToAvroNativeData(cols, colInfos, time.Local) - c.Assert(err, check.IsNil) - for k, v := range native.(map[string]interface{}) { - if k == "colNullable" { - c.Check(v, check.IsNil) - } - if k == "colNotnull" { - c.Assert(v, check.Equals, int64(0)) - } - if k == "colNullable1" { - c.Assert(v, check.DeepEquals, map[string]interface{}{"int": int64(0)}) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + keyCols, keyColInfos := event.PrimaryKeyColInfos() + namespace := getAvroNamespace(encoder.namespace, event.Table) + + keySchema, err := rowToAvroSchema( + namespace, + event.Table.Table, + keyCols, + keyColInfos, + false, + "precise", + "long", + ) + require.NoError(t, err) + avroKeyCodec, err := goavro.NewCodec(keySchema) + require.NoError(t, err) + + r, err := encoder.avroEncode(ctx, event, "default", true) + require.NoError(t, err) + res, _, err := avroKeyCodec.NativeFromBinary(r.data) + require.NoError(t, err) + require.NotNil(t, res) + for k := range res.(map[string]interface{}) { + if k == "_tidb_commit_ts" || k == "_tidb_op" || k == "_tidb_commit_physical_time" { + require.Fail(t, "key shall not include extension fields") } } - avroCodec, err := goavro.NewCodec(schema) - c.Assert(err, check.IsNil) - r, err := avroEncode(&table, s.encoder.valueSchemaManager, 1, cols, colInfos, time.Local) - c.Assert(err, check.IsNil) - - native, _, err = avroCodec.NativeFromBinary(r.data) - c.Check(err, check.IsNil) - c.Check(native, check.NotNil) - for k, v := range native.(map[string]interface{}) { - if k == "colNullable" { - c.Check(v, check.IsNil) - } - if k == "colNotnull" { - c.Assert(v.(int32), check.Equals, int32(0)) - } - if k == "colNullable1" { - c.Assert(v, check.DeepEquals, map[string]interface{}{"int": int32(0)}) + valueSchema, err := rowToAvroSchema( + namespace, + event.Table.Table, + cols, + colInfos, + true, + "precise", + "long", + ) + require.NoError(t, err) + avroValueCodec, err := goavro.NewCodec(valueSchema) + require.NoError(t, err) + + r, err = encoder.avroEncode(ctx, event, "default", false) + require.NoError(t, err) + res, _, err = avroValueCodec.NativeFromBinary(r.data) + require.NoError(t, err) + require.NotNil(t, res) + for k, v := range res.(map[string]interface{}) { + if k == "_tidb_op" { + require.Equal(t, "c", v.(string)) } } } -func (s *avroBatchEncoderSuite) TestAvroTimeZone(c *check.C) { - defer testleak.AfterTest(c)() +func TestAvroEnvelope(t *testing.T) { + t.Parallel() - table := model.TableName{ - Schema: "testdb", - Table: "TestAvroTimeZone", - } - - location, err := time.LoadLocation("UTC") - c.Check(err, check.IsNil) - - timestamp := time.Now() - cols := []*model.Column{ - {Name: "id", Value: int64(1), Type: mysql.TypeLong}, - {Name: "myint", Value: int64(2), Type: mysql.TypeLong}, - {Name: "mybool", Value: int64(1), Type: mysql.TypeTiny}, - {Name: "myfloat", Value: float64(3.14), Type: mysql.TypeFloat}, - {Name: "mystring", Value: []byte("Hello World"), Type: mysql.TypeBlob}, - {Name: "ts", Value: timestamp.In(location).Format(types.TimeFSPFormat), Type: mysql.TypeTimestamp}, - } - - colInfos := []rowcodec.ColInfo{ - {ID: 1, IsPKHandle: true, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeLong)}, - {ID: 2, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeLong)}, - {ID: 3, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeTiny)}, - {ID: 4, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeFloat)}, - {ID: 5, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeBlob)}, - {ID: 6, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeTimestamp)}, - } - - schema, err := ColumnInfoToAvroSchema(table.Table, cols) - c.Assert(err, check.IsNil) - avroCodec, err := goavro.NewCodec(schema) - c.Assert(err, check.IsNil) - - r, err := avroEncode(&table, s.encoder.valueSchemaManager, 1, cols, colInfos, location) - c.Assert(err, check.IsNil) - - res, _, err := avroCodec.NativeFromBinary(r.data) - c.Check(err, check.IsNil) - c.Check(res, check.NotNil) - actual := (res.(map[string]interface{}))["ts"].(time.Time) - c.Check(actual.Local().Sub(timestamp), check.LessEqual, time.Millisecond) -} - -func (s *avroBatchEncoderSuite) TestAvroEnvelope(c *check.C) { - defer testleak.AfterTest(c)() avroCodec, err := goavro.NewCodec(` { "type": "record", - "name": "TestAvroEnvelope", + "name": "testdb.avroenvelope", "fields" : [ {"name": "id", "type": "int", "default": 0} ] }`) - c.Assert(err, check.IsNil) + require.NoError(t, err) testNativeData := make(map[string]interface{}) testNativeData["id"] = 7 bin, err := avroCodec.BinaryFromNative(nil, testNativeData) - c.Check(err, check.IsNil) + require.NoError(t, err) res := avroEncodeResult{ data: bin, @@ -308,192 +846,52 @@ func (s *avroBatchEncoderSuite) TestAvroEnvelope(c *check.C) { } evlp, err := res.toEnvelope() - c.Check(err, check.IsNil) + require.NoError(t, err) - c.Assert(evlp[0], check.Equals, magicByte) - c.Assert(evlp[1:5], check.BytesEquals, []byte{0, 0, 0, 7}) + require.Equal(t, magicByte, evlp[0]) + require.Equal(t, []byte{0, 0, 0, 7}, evlp[1:5]) parsed, _, err := avroCodec.NativeFromBinary(evlp[5:]) - c.Assert(err, check.IsNil) - c.Assert(parsed, check.NotNil) + require.NoError(t, err) + require.NotNil(t, parsed) id, exists := parsed.(map[string]interface{})["id"] - c.Assert(exists, check.IsTrue) - c.Assert(id, check.Equals, int32(7)) + require.True(t, exists) + require.Equal(t, int32(7), id) } -func (s *avroBatchEncoderSuite) TestAvroEncode(c *check.C) { - defer testleak.AfterTest(c)() - testCaseUpdate := &model.RowChangedEvent{ - CommitTs: 417318403368288260, - Table: &model.TableName{ - Schema: "test", - Table: "person", - }, - Columns: []*model.Column{ - {Name: "id", Type: mysql.TypeLong, Flag: model.HandleKeyFlag, Value: int64(1)}, - {Name: "name", Type: mysql.TypeVarchar, Value: "Bob"}, - {Name: "tiny", Type: mysql.TypeTiny, Value: int64(255)}, - {Name: "utiny", Type: mysql.TypeTiny, Flag: model.UnsignedFlag, Value: uint64(100)}, - {Name: "comment", Type: mysql.TypeBlob, Value: []byte("测试")}, - }, - ColInfos: []rowcodec.ColInfo{ - {ID: 1, IsPKHandle: true, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeLong)}, - {ID: 2, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeVarchar)}, - {ID: 3, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeTiny)}, - {ID: 4, IsPKHandle: false, VirtualGenCol: false, Ft: setFlag(types.NewFieldType(mysql.TypeTiny), uint(model.UnsignedFlag))}, - {ID: 5, IsPKHandle: false, VirtualGenCol: false, Ft: types.NewFieldType(mysql.TypeBlob)}, - }, - } - - testCaseDdl := &model.DDLEvent{ - CommitTs: 417318403368288260, - TableInfo: &model.SimpleTableInfo{ - Schema: "test", Table: "person", - }, - Query: "create table person(id int, name varchar(32), tiny tinyint unsigned, comment text, primary key(id))", - Type: model2.ActionCreateTable, - } - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - pm := puller.NewMockPullerManager(c, true) - defer pm.TearDown() - pm.MustExec(testCaseDdl.Query) - ddlPlr := pm.CreatePuller(0, []regionspan.ComparableSpan{regionspan.ToComparableSpan(regionspan.GetDDLSpan())}) - go func() { - err := ddlPlr.Run(ctx) - if err != nil && errors.Cause(err) != context.Canceled { - c.Fail() - } - }() - - info := pm.GetTableInfo("test", "person") - testCaseDdl.TableInfo = new(model.SimpleTableInfo) - testCaseDdl.TableInfo.Schema = "test" - testCaseDdl.TableInfo.Table = "person" - testCaseDdl.TableInfo.ColumnInfo = make([]*model.ColumnInfo, len(info.Columns)) - for i, v := range info.Columns { - testCaseDdl.TableInfo.ColumnInfo[i] = new(model.ColumnInfo) - testCaseDdl.TableInfo.ColumnInfo[i].FromTiColumnInfo(v) - } - - _, err := s.encoder.EncodeDDLEvent(testCaseDdl) - c.Check(err, check.IsNil) - - err = s.encoder.AppendRowChangedEvent(testCaseUpdate) - c.Check(err, check.IsNil) -} - -func startAvroHTTPInterceptForTestingRegistry(c *check.C) { - httpmock.Activate() - - registry := mockRegistry{ - subjects: make(map[string]*mockRegistrySchema), - newID: 1, - } - - httpmock.RegisterResponder("GET", "http://127.0.0.1:8081", httpmock.NewStringResponder(200, "{}")) - - httpmock.RegisterResponder("POST", `=~^http://127.0.0.1:8081/subjects/(.+)/versions`, - func(req *http.Request) (*http.Response, error) { - subject, err := httpmock.GetSubmatch(req, 1) - if err != nil { - return nil, err - } - reqBody, err := io.ReadAll(req.Body) - if err != nil { - return nil, err - } - var reqData registerRequest - err = json.Unmarshal(reqBody, &reqData) - if err != nil { - return nil, err - } - - // c.Assert(reqData.SchemaType, check.Equals, "AVRO") - - var respData registerResponse - registry.mu.Lock() - item, exists := registry.subjects[subject] - if !exists { - item = &mockRegistrySchema{ - content: reqData.Schema, - version: 0, - ID: registry.newID, - } - registry.subjects[subject] = item - respData.ID = registry.newID - } else { - if item.content == reqData.Schema { - respData.ID = item.ID - } else { - item.content = reqData.Schema - item.version++ - item.ID = registry.newID - respData.ID = registry.newID - } - } - registry.newID++ - registry.mu.Unlock() - return httpmock.NewJsonResponse(200, &respData) - }) - - httpmock.RegisterResponder("GET", `=~^http://127.0.0.1:8081/subjects/(.+)/versions/latest`, - func(req *http.Request) (*http.Response, error) { - subject, err := httpmock.GetSubmatch(req, 1) - if err != nil { - return httpmock.NewStringResponse(500, "Internal Server Error"), err - } - - registry.mu.Lock() - item, exists := registry.subjects[subject] - registry.mu.Unlock() - if !exists { - return httpmock.NewStringResponse(404, ""), nil - } - - var respData lookupResponse - respData.Schema = item.content - respData.Name = subject - respData.RegistryID = item.ID - - return httpmock.NewJsonResponse(200, &respData) - }) - - httpmock.RegisterResponder("DELETE", `=~^http://127.0.0.1:8081/subjects/(.+)`, - func(req *http.Request) (*http.Response, error) { - subject, err := httpmock.GetSubmatch(req, 1) - if err != nil { - return nil, err - } - - registry.mu.Lock() - defer registry.mu.Unlock() - _, exists := registry.subjects[subject] - if !exists { - return httpmock.NewStringResponse(404, ""), nil - } - - delete(registry.subjects, subject) - return httpmock.NewStringResponse(200, ""), nil - }) - - failCounter := 0 - httpmock.RegisterResponder("POST", `=~^http://127.0.0.1:8081/may-fail`, - func(req *http.Request) (*http.Response, error) { - data, _ := io.ReadAll(req.Body) - c.Assert(len(data), check.Greater, 0) - c.Assert(int64(len(data)), check.Equals, req.ContentLength) - if failCounter < 3 { - failCounter++ - return httpmock.NewStringResponse(422, ""), nil - } - return httpmock.NewStringResponse(200, ""), nil - }) +func TestSanitizeName(t *testing.T) { + t.Parallel() + + require.Equal(t, "normalColumnName123", sanitizeName("normalColumnName123")) + require.Equal( + t, + "_1ColumnNameStartWithNumber", + sanitizeName("1ColumnNameStartWithNumber"), + ) + require.Equal(t, "A_B", sanitizeName("A.B")) + require.Equal(t, "columnNameWith__", sanitizeName("columnNameWith中文")) } -func stopAvroHTTPInterceptForTestingRegistry() { - httpmock.DeactivateAndReset() +func TestGetAvroNamespace(t *testing.T) { + t.Parallel() + + require.Equal( + t, + "normalNamespace.normalSchema", + getAvroNamespace( + "normalNamespace", + &model.TableName{Schema: "normalSchema", Table: "normalTable"}, + ), + ) + require.Equal( + t, + "_1Namespace._1Schema", + getAvroNamespace("1Namespace", &model.TableName{Schema: "1Schema", Table: "normalTable"}), + ) + require.Equal( + t, + "N_amespace.S_chema", + getAvroNamespace("N-amespace", &model.TableName{Schema: "S.chema", Table: "normalTable"}), + ) } diff --git a/cdc/sink/codec/avro_test_data.go b/cdc/sink/codec/avro_test_data.go new file mode 100644 index 00000000000..e2d33276b3f --- /dev/null +++ b/cdc/sink/codec/avro_test_data.go @@ -0,0 +1,1650 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package codec + +var expectedSchemaWithoutExtension = `{ + "type": "record", + "name": "rowtoavroschema", + "namespace": "default.testdb", + "fields": [ + { + "name": "tiny", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + }, + { + "default": null, + "name": "tinynullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + ] + }, + { + "name": "short", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + }, + { + "default": null, + "name": "shortnullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + ] + }, + { + "name": "int24", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + }, + { + "default": null, + "name": "int24nullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + ] + }, + { + "name": "long", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + }, + { + "default": null, + "name": "longnullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + ] + }, + { + "name": "longlong", + "type": { + "type": "long", + "connect.parameters": { + "tidb_type": "BIGINT" + } + } + }, + { + "default": null, + "name": "longlongnullable", + "type": [ + "null", + { + "type": "long", + "connect.parameters": { + "tidb_type": "BIGINT" + } + } + ] + }, + { + "name": "tinyunsigned", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + }, + { + "default": null, + "name": "tinyunsignednullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + ] + }, + { + "name": "shortunsigned", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + }, + { + "default": null, + "name": "shortunsignednullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + ] + }, + { + "name": "int24unsigned", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + }, + { + "default": null, + "name": "int24unsignednullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + ] + }, + { + "name": "longunsigned", + "type": { + "type": "long", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + }, + { + "default": null, + "name": "longunsignednullable", + "type": [ + "null", + { + "type": "long", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + ] + }, + { + "name": "longlongunsigned", + "type": { + "type": "long", + "connect.parameters": { + "tidb_type": "BIGINT UNSIGNED" + } + } + }, + { + "default": null, + "name": "longlongunsignednullable", + "type": [ + "null", + { + "type": "long", + "connect.parameters": { + "tidb_type": "BIGINT UNSIGNED" + } + } + ] + }, + { + "name": "float", + "type": { + "type": "double", + "connect.parameters": { + "tidb_type": "FLOAT" + } + } + }, + { + "default": null, + "name": "floatnullable", + "type": [ + "null", + { + "type": "double", + "connect.parameters": { + "tidb_type": "FLOAT" + } + } + ] + }, + { + "name": "double", + "type": { + "type": "double", + "connect.parameters": { + "tidb_type": "DOUBLE" + } + } + }, + { + "default": null, + "name": "doublenullable", + "type": [ + "null", + { + "type": "double", + "connect.parameters": { + "tidb_type": "DOUBLE" + } + } + ] + }, + { + "name": "bit", + "type": { + "type": "bytes", + "connect.parameters": { + "length": "1", + "tidb_type": "BIT" + } + } + }, + { + "default": null, + "name": "bitnullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "length": "1", + "tidb_type": "BIT" + } + } + ] + }, + { + "name": "decimal", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "DECIMAL" + }, + "logicalType": "decimal", + "precision": 10, + "scale": 0 + } + }, + { + "default": null, + "name": "decimalnullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "DECIMAL" + }, + "logicalType": "decimal", + "precision": 10, + "scale": 0 + } + ] + }, + { + "name": "tinytext", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + }, + { + "default": null, + "name": "tinytextnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + ] + }, + { + "name": "mediumtext", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + }, + { + "default": null, + "name": "mediumtextnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + ] + }, + { + "name": "text", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + }, + { + "default": null, + "name": "textnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + ] + }, + { + "name": "longtext", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + }, + { + "default": null, + "name": "longtextnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + ] + }, + { + "name": "varchar", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + }, + { + "default": null, + "name": "varcharnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + ] + }, + { + "name": "varstring", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + }, + { + "default": null, + "name": "varstringnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + ] + }, + { + "name": "string", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + }, + { + "default": null, + "name": "stringnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + ] + }, + { + "name": "tinyblob", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + }, + { + "default": null, + "name": "tinyblobnullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + ] + }, + { + "name": "mediumblob", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + }, + { + "default": null, + "name": "mediumblobnullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + ] + }, + { + "name": "blob", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + }, + { + "default": null, + "name": "blobnullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + ] + }, + { + "name": "longblob", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + }, + { + "default": null, + "name": "longblobnullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + ] + }, + { + "name": "varbinary", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + }, + { + "default": null, + "name": "varbinarynullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + ] + }, + { + "name": "varbinary1", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + }, + { + "default": null, + "name": "varbinary1nullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + ] + }, + { + "name": "binary", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + }, + { + "default": null, + "name": "binarynullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + ] + }, + { + "name": "enum", + "type": { + "type": "string", + "connect.parameters": { + "allowed": "a\\,,b", + "tidb_type": "ENUM" + } + } + }, + { + "default": null, + "name": "enumnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "allowed": "a\\,,b", + "tidb_type": "ENUM" + } + } + ] + }, + { + "name": "set", + "type": { + "type": "string", + "connect.parameters": { + "allowed": "a\\,,b", + "tidb_type": "SET" + } + } + }, + { + "default": null, + "name": "setnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "allowed": "a\\,,b", + "tidb_type": "SET" + } + } + ] + }, + { + "name": "json", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "JSON" + } + } + }, + { + "default": null, + "name": "jsonnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "JSON" + } + } + ] + }, + { + "name": "date", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "DATE" + } + } + }, + { + "default": null, + "name": "datenullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "DATE" + } + } + ] + }, + { + "name": "datetime", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "DATETIME" + } + } + }, + { + "default": null, + "name": "datetimenullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "DATETIME" + } + } + ] + }, + { + "name": "timestamp", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TIMESTAMP" + } + } + }, + { + "default": null, + "name": "timestampnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TIMESTAMP" + } + } + ] + }, + { + "name": "time", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TIME" + } + } + }, + { + "default": null, + "name": "timenullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TIME" + } + } + ] + }, + { + "name": "year", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "YEAR" + } + } + }, + { + "default": null, + "name": "yearnullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "YEAR" + } + } + ] + } + ] +}` + +var expectedSchemaWithExtension = `{ + "type": "record", + "name": "rowtoavroschema", + "namespace": "default.testdb", + "fields": [ + { + "name": "tiny", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + }, + { + "default": null, + "name": "tinynullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + ] + }, + { + "name": "short", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + }, + { + "default": null, + "name": "shortnullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + ] + }, + { + "name": "int24", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + }, + { + "default": null, + "name": "int24nullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + ] + }, + { + "name": "long", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + }, + { + "default": null, + "name": "longnullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "INT" + } + } + ] + }, + { + "name": "longlong", + "type": { + "type": "long", + "connect.parameters": { + "tidb_type": "BIGINT" + } + } + }, + { + "default": null, + "name": "longlongnullable", + "type": [ + "null", + { + "type": "long", + "connect.parameters": { + "tidb_type": "BIGINT" + } + } + ] + }, + { + "name": "tinyunsigned", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + }, + { + "default": null, + "name": "tinyunsignednullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + ] + }, + { + "name": "shortunsigned", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + }, + { + "default": null, + "name": "shortunsignednullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + ] + }, + { + "name": "int24unsigned", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + }, + { + "default": null, + "name": "int24unsignednullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + ] + }, + { + "name": "longunsigned", + "type": { + "type": "long", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + }, + { + "default": null, + "name": "longunsignednullable", + "type": [ + "null", + { + "type": "long", + "connect.parameters": { + "tidb_type": "INT UNSIGNED" + } + } + ] + }, + { + "name": "longlongunsigned", + "type": { + "type": "long", + "connect.parameters": { + "tidb_type": "BIGINT UNSIGNED" + } + } + }, + { + "default": null, + "name": "longlongunsignednullable", + "type": [ + "null", + { + "type": "long", + "connect.parameters": { + "tidb_type": "BIGINT UNSIGNED" + } + } + ] + }, + { + "name": "float", + "type": { + "type": "double", + "connect.parameters": { + "tidb_type": "FLOAT" + } + } + }, + { + "default": null, + "name": "floatnullable", + "type": [ + "null", + { + "type": "double", + "connect.parameters": { + "tidb_type": "FLOAT" + } + } + ] + }, + { + "name": "double", + "type": { + "type": "double", + "connect.parameters": { + "tidb_type": "DOUBLE" + } + } + }, + { + "default": null, + "name": "doublenullable", + "type": [ + "null", + { + "type": "double", + "connect.parameters": { + "tidb_type": "DOUBLE" + } + } + ] + }, + { + "name": "bit", + "type": { + "type": "bytes", + "connect.parameters": { + "length": "1", + "tidb_type": "BIT" + } + } + }, + { + "default": null, + "name": "bitnullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "length": "1", + "tidb_type": "BIT" + } + } + ] + }, + { + "name": "decimal", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "DECIMAL" + }, + "logicalType": "decimal", + "precision": 10, + "scale": 0 + } + }, + { + "default": null, + "name": "decimalnullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "DECIMAL" + }, + "logicalType": "decimal", + "precision": 10, + "scale": 0 + } + ] + }, + { + "name": "tinytext", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + }, + { + "default": null, + "name": "tinytextnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + ] + }, + { + "name": "mediumtext", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + }, + { + "default": null, + "name": "mediumtextnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + ] + }, + { + "name": "text", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + }, + { + "default": null, + "name": "textnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + ] + }, + { + "name": "longtext", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + }, + { + "default": null, + "name": "longtextnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + ] + }, + { + "name": "varchar", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + }, + { + "default": null, + "name": "varcharnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + ] + }, + { + "name": "varstring", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + }, + { + "default": null, + "name": "varstringnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + ] + }, + { + "name": "string", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + }, + { + "default": null, + "name": "stringnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TEXT" + } + } + ] + }, + { + "name": "tinyblob", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + }, + { + "default": null, + "name": "tinyblobnullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + ] + }, + { + "name": "mediumblob", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + }, + { + "default": null, + "name": "mediumblobnullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + ] + }, + { + "name": "blob", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + }, + { + "default": null, + "name": "blobnullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + ] + }, + { + "name": "longblob", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + }, + { + "default": null, + "name": "longblobnullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + ] + }, + { + "name": "varbinary", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + }, + { + "default": null, + "name": "varbinarynullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + ] + }, + { + "name": "varbinary1", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + }, + { + "default": null, + "name": "varbinary1nullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + ] + }, + { + "name": "binary", + "type": { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + }, + { + "default": null, + "name": "binarynullable", + "type": [ + "null", + { + "type": "bytes", + "connect.parameters": { + "tidb_type": "BLOB" + } + } + ] + }, + { + "name": "enum", + "type": { + "type": "string", + "connect.parameters": { + "allowed": "a\\,,b", + "tidb_type": "ENUM" + } + } + }, + { + "default": null, + "name": "enumnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "allowed": "a\\,,b", + "tidb_type": "ENUM" + } + } + ] + }, + { + "name": "set", + "type": { + "type": "string", + "connect.parameters": { + "allowed": "a\\,,b", + "tidb_type": "SET" + } + } + }, + { + "default": null, + "name": "setnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "allowed": "a\\,,b", + "tidb_type": "SET" + } + } + ] + }, + { + "name": "json", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "JSON" + } + } + }, + { + "default": null, + "name": "jsonnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "JSON" + } + } + ] + }, + { + "name": "date", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "DATE" + } + } + }, + { + "default": null, + "name": "datenullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "DATE" + } + } + ] + }, + { + "name": "datetime", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "DATETIME" + } + } + }, + { + "default": null, + "name": "datetimenullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "DATETIME" + } + } + ] + }, + { + "name": "timestamp", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TIMESTAMP" + } + } + }, + { + "default": null, + "name": "timestampnullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TIMESTAMP" + } + } + ] + }, + { + "name": "time", + "type": { + "type": "string", + "connect.parameters": { + "tidb_type": "TIME" + } + } + }, + { + "default": null, + "name": "timenullable", + "type": [ + "null", + { + "type": "string", + "connect.parameters": { + "tidb_type": "TIME" + } + } + ] + }, + { + "name": "year", + "type": { + "type": "int", + "connect.parameters": { + "tidb_type": "YEAR" + } + } + }, + { + "default": null, + "name": "yearnullable", + "type": [ + "null", + { + "type": "int", + "connect.parameters": { + "tidb_type": "YEAR" + } + } + ] + }, + { + "name": "_tidb_op", + "type": "string" + }, + { + "name": "_tidb_commit_ts", + "type": "long" + }, + { + "name": "_tidb_commit_physical_time", + "type": "long" + } + ] +}` diff --git a/cdc/sink/codec/canal.go b/cdc/sink/codec/canal.go index 5f730200a1e..689f13dbdc1 100644 --- a/cdc/sink/codec/canal.go +++ b/cdc/sink/codec/canal.go @@ -14,6 +14,7 @@ package codec import ( + "context" "fmt" "math" "reflect" @@ -419,7 +420,11 @@ func (d *CanalEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessage, e } // AppendRowChangedEvent implements the EventBatchEncoder interface -func (d *CanalEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) error { +func (d *CanalEventBatchEncoder) AppendRowChangedEvent( + ctx context.Context, + topic string, + e *model.RowChangedEvent, +) error { entry, err := d.entryBuilder.FromRowEvent(e) if err != nil { return errors.Trace(err) diff --git a/cdc/sink/codec/canal_flat.go b/cdc/sink/codec/canal_flat.go index 2ca79413bac..c5e4d880f66 100644 --- a/cdc/sink/codec/canal_flat.go +++ b/cdc/sink/codec/canal_flat.go @@ -14,6 +14,7 @@ package codec import ( + "context" "encoding/json" "sort" "strings" @@ -339,7 +340,11 @@ func (c *CanalFlatEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessag } // AppendRowChangedEvent implements the interface EventBatchEncoder -func (c *CanalFlatEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) error { +func (c *CanalFlatEventBatchEncoder) AppendRowChangedEvent( + ctx context.Context, + topic string, + e *model.RowChangedEvent, +) error { message, err := c.newFlatMessageForDML(e) if err != nil { return errors.Trace(err) diff --git a/cdc/sink/codec/canal_flat_test.go b/cdc/sink/codec/canal_flat_test.go index 32603bf7c8f..024c660943a 100644 --- a/cdc/sink/codec/canal_flat_test.go +++ b/cdc/sink/codec/canal_flat_test.go @@ -14,11 +14,11 @@ package codec import ( + "context" "encoding/json" "testing" mm "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/stretchr/testify/require" @@ -70,7 +70,7 @@ var testCaseDDL = &model.DDLEvent{ func TestBuildCanalFlatEventBatchEncoder(t *testing.T) { t.Parallel() - config := NewConfig(config.ProtocolCanalJSON, timeutil.SystemLocation()) + config := NewConfig(config.ProtocolCanalJSON) builder := &canalFlatEventBatchEncoderBuilder{config: config} encoder, ok := builder.Build().(*CanalFlatEventBatchEncoder) @@ -165,7 +165,7 @@ func TestNewCanalFlatEventBatchDecoder4RowMessage(t *testing.T) { encoder := &CanalFlatEventBatchEncoder{builder: NewCanalEntryBuilder(), enableTiDBExtension: encodeEnable} require.NotNil(t, encoder) - err := encoder.AppendRowChangedEvent(testCaseInsert) + err := encoder.AppendRowChangedEvent(context.Background(), "", testCaseInsert) require.Nil(t, err) mqMessages := encoder.Build() @@ -294,7 +294,7 @@ func TestBatching(t *testing.T) { for i := 1; i <= 1000; i++ { ts := uint64(i) updateCase.CommitTs = ts - err := encoder.AppendRowChangedEvent(&updateCase) + err := encoder.AppendRowChangedEvent(context.Background(), "", &updateCase) require.Nil(t, err) if i%100 == 0 { diff --git a/cdc/sink/codec/canal_test.go b/cdc/sink/codec/canal_test.go index 45dffd0cf3b..dd819e62249 100644 --- a/cdc/sink/codec/canal_test.go +++ b/cdc/sink/codec/canal_test.go @@ -14,6 +14,7 @@ package codec import ( + "context" "testing" "github.com/golang/protobuf/proto" // nolint:staticcheck @@ -85,7 +86,7 @@ func TestCanalEventBatchEncoder(t *testing.T) { for _, cs := range s.rowCases { encoder := NewCanalEventBatchEncoder() for _, row := range cs { - err := encoder.AppendRowChangedEvent(row) + err := encoder.AppendRowChangedEvent(context.Background(), "", row) require.Nil(t, err) } size := encoder.Size() diff --git a/cdc/sink/codec/codec_test.go b/cdc/sink/codec/codec_test.go index 757b0905d44..4880a78cc98 100644 --- a/cdc/sink/codec/codec_test.go +++ b/cdc/sink/codec/codec_test.go @@ -16,6 +16,7 @@ package codec import ( "bytes" "compress/zlib" + "context" "testing" "github.com/pingcap/tidb/parser/mysql" @@ -351,7 +352,7 @@ func codecEncodeRowChangedPB2(events []*model.RowChangedEvent) []byte { func codecEncodeRowCase(encoder EventBatchEncoder, events []*model.RowChangedEvent) ([]*MQMessage, error) { for _, event := range events { - err := encoder.AppendRowChangedEvent(event) + err := encoder.AppendRowChangedEvent(context.Background(), "", event) if err != nil { return nil, err } diff --git a/cdc/sink/codec/config.go b/cdc/sink/codec/config.go index d42c79d745a..9cd8773cda6 100644 --- a/cdc/sink/codec/config.go +++ b/cdc/sink/codec/config.go @@ -16,7 +16,6 @@ package codec import ( "net/url" "strconv" - "time" "github.com/pingcap/errors" "github.com/pingcap/tiflow/pkg/config" @@ -38,33 +37,44 @@ type Config struct { enableTiDBExtension bool // avro only - avroRegistry string - tz *time.Location + avroSchemaRegistry string + avroDecimalHandlingMode string + avroBigintUnsignedHandlingMode string } // NewConfig return a Config for codec -func NewConfig(protocol config.Protocol, tz *time.Location) *Config { +func NewConfig(protocol config.Protocol) *Config { return &Config{ protocol: protocol, maxMessageBytes: config.DefaultMaxMessageBytes, maxBatchSize: defaultMaxBatchSize, - enableTiDBExtension: false, - avroRegistry: "", - tz: tz, + enableTiDBExtension: false, + avroSchemaRegistry: "", + avroDecimalHandlingMode: "precise", + avroBigintUnsignedHandlingMode: "long", } } const ( - codecOPTEnableTiDBExtension = "enable-tidb-extension" - codecOPTMaxBatchSize = "max-batch-size" - codecOPTMaxMessageBytes = "max-message-bytes" - codecAvroRegistry = "registry" + codecOPTEnableTiDBExtension = "enable-tidb-extension" + codecOPTMaxBatchSize = "max-batch-size" + codecOPTMaxMessageBytes = "max-message-bytes" + codecOPTAvroDecimalHandlingMode = "avro-decimal-handling-mode" + codecOPTAvroBigintUnsignedHandlingMode = "avro-bigint-unsigned-handling-mode" + codecOPTAvroSchemaRegistry = "schema-registry" +) + +const ( + decimalHandlingModeString = "string" + decimalHandlingModePrecise = "precise" + bigintUnsignedHandlingModeString = "string" + bigintUnsignedHandlingModeLong = "long" ) // Apply fill the Config -func (c *Config) Apply(sinkURI *url.URL, opts map[string]string) error { +func (c *Config) Apply(sinkURI *url.URL, config *config.ReplicaConfig) error { params := sinkURI.Query() if s := params.Get(codecOPTEnableTiDBExtension); s != "" { b, err := strconv.ParseBool(s) @@ -90,8 +100,16 @@ func (c *Config) Apply(sinkURI *url.URL, opts map[string]string) error { c.maxMessageBytes = a } - if s, ok := opts[codecAvroRegistry]; ok { - c.avroRegistry = s + if s := params.Get(codecOPTAvroDecimalHandlingMode); s != "" { + c.avroDecimalHandlingMode = s + } + + if s := params.Get(codecOPTAvroBigintUnsignedHandlingMode); s != "" { + c.avroBigintUnsignedHandlingMode = s + } + + if config.SchemaRegistry != "" { + c.avroSchemaRegistry = config.SchemaRegistry } return nil @@ -105,26 +123,52 @@ func (c *Config) WithMaxMessageBytes(bytes int) *Config { // Validate the Config func (c *Config) Validate() error { - if c.protocol != config.ProtocolCanalJSON && c.enableTiDBExtension { - return cerror.ErrMQCodecInvalidConfig.GenWithStack(`enable-tidb-extension only support canal-json protocol`) + if c.enableTiDBExtension && + !(c.protocol == config.ProtocolCanalJSON || c.protocol == config.ProtocolAvro) { + return cerror.ErrMQCodecInvalidConfig.GenWithStack( + `enable-tidb-extension only supports canal-json/avro protocol`, + ) } if c.protocol == config.ProtocolAvro { - if c.avroRegistry == "" { - return cerror.ErrMQCodecInvalidConfig.GenWithStack(`Avro protocol requires parameter "registry"`) + if c.avroSchemaRegistry == "" { + return cerror.ErrMQCodecInvalidConfig.GenWithStack( + `Avro protocol requires parameter "%s"`, + codecOPTAvroSchemaRegistry, + ) + } + + if c.avroDecimalHandlingMode != decimalHandlingModePrecise && + c.avroDecimalHandlingMode != decimalHandlingModeString { + return cerror.ErrMQCodecInvalidConfig.GenWithStack( + `%s value could only be "%s" or "%s"`, + codecOPTAvroDecimalHandlingMode, + decimalHandlingModeString, + decimalHandlingModePrecise, + ) } - if c.tz == nil { - return cerror.ErrMQCodecInvalidConfig.GenWithStack("Avro protocol requires timezone to be set") + if c.avroBigintUnsignedHandlingMode != bigintUnsignedHandlingModeLong && + c.avroBigintUnsignedHandlingMode != bigintUnsignedHandlingModeString { + return cerror.ErrMQCodecInvalidConfig.GenWithStack( + `%s value could only be "%s" or "%s"`, + codecOPTAvroBigintUnsignedHandlingMode, + bigintUnsignedHandlingModeLong, + bigintUnsignedHandlingModeString, + ) } } if c.maxMessageBytes <= 0 { - return cerror.ErrMQCodecInvalidConfig.Wrap(errors.Errorf("invalid max-message-bytes %d", c.maxMessageBytes)) + return cerror.ErrMQCodecInvalidConfig.Wrap( + errors.Errorf("invalid max-message-bytes %d", c.maxMessageBytes), + ) } if c.maxBatchSize <= 0 { - return cerror.ErrMQCodecInvalidConfig.Wrap(errors.Errorf("invalid max-batch-size %d", c.maxBatchSize)) + return cerror.ErrMQCodecInvalidConfig.Wrap( + errors.Errorf("invalid max-batch-size %d", c.maxBatchSize), + ) } return nil diff --git a/cdc/sink/codec/config_test.go b/cdc/sink/codec/config_test.go index dce95afcd66..0ed057a137c 100644 --- a/cdc/sink/codec/config_test.go +++ b/cdc/sink/codec/config_test.go @@ -17,127 +17,191 @@ import ( "net/url" "testing" - "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tiflow/pkg/config" - cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/stretchr/testify/require" ) func TestNewConfig(t *testing.T) { t.Parallel() - c := NewConfig(config.ProtocolDefault, timeutil.SystemLocation()) - require.Equal(t, c.protocol, config.ProtocolDefault) - require.Equal(t, c.maxMessageBytes, config.DefaultMaxMessageBytes) - require.Equal(t, c.maxBatchSize, defaultMaxBatchSize) - require.Equal(t, c.enableTiDBExtension, false) - require.Equal(t, c.avroRegistry, "") + + c := NewConfig(config.ProtocolDefault) + require.Equal(t, config.ProtocolDefault, c.protocol) + require.Equal(t, config.DefaultMaxMessageBytes, c.maxMessageBytes) + require.Equal(t, defaultMaxBatchSize, c.maxBatchSize) + require.Equal(t, false, c.enableTiDBExtension) + require.Equal(t, "precise", c.avroDecimalHandlingMode) + require.Equal(t, "long", c.avroBigintUnsignedHandlingMode) + require.Equal(t, "", c.avroSchemaRegistry) } func TestConfigApplyValidate(t *testing.T) { t.Parallel() + + // enable-tidb-extension uri := "kafka://127.0.0.1:9092/abc?protocol=canal-json&enable-tidb-extension=true" sinkURI, err := url.Parse(uri) - require.Nil(t, err) + require.NoError(t, err) protocol := sinkURI.Query().Get("protocol") - require.Equal(t, protocol, "canal-json") + require.Equal(t, "canal-json", protocol) var p config.Protocol err = p.FromString(protocol) - require.Nil(t, err) + require.NoError(t, err) - c := NewConfig(p, timeutil.SystemLocation()) - require.Equal(t, c.protocol, config.ProtocolCanalJSON) + c := NewConfig(p) + require.Equal(t, config.ProtocolCanalJSON, c.protocol) - opts := make(map[string]string) - err = c.Apply(sinkURI, opts) - require.Nil(t, err) + replicaConfig := &config.ReplicaConfig{} + err = c.Apply(sinkURI, replicaConfig) + require.NoError(t, err) require.True(t, c.enableTiDBExtension) err = c.Validate() - require.Nil(t, err) + require.NoError(t, err) - // illegal enable-tidb-extension uri = "kafka://127.0.0.1:9092/abc?protocol=canal-json&enable-tidb-extension=a" sinkURI, err = url.Parse(uri) - require.Nil(t, err) - err = c.Apply(sinkURI, opts) - require.Error(t, err, "invalid syntax") + require.NoError(t, err) + err = c.Apply(sinkURI, replicaConfig) + require.ErrorContains(t, err, "invalid syntax") // Use enable-tidb-extension on other protocols uri = "kafka://127.0.0.1:9092/abc?protocol=open-protocol&enable-tidb-extension=true" sinkURI, err = url.Parse(uri) - require.Nil(t, err) + require.NoError(t, err) protocol = sinkURI.Query().Get("protocol") err = p.FromString(protocol) - require.Nil(t, err) + require.NoError(t, err) - c = NewConfig(p, timeutil.SystemLocation()) - err = c.Apply(sinkURI, opts) - require.Nil(t, err) + c = NewConfig(p) + err = c.Apply(sinkURI, replicaConfig) + require.NoError(t, err) require.True(t, c.enableTiDBExtension) err = c.Validate() - require.Error(t, err, "enable-tidb-extension only support canal-json protocol") + require.ErrorContains(t, err, "enable-tidb-extension only supports canal-json/avro protocol") // avro uri = "kafka://127.0.0.1:9092/abc?protocol=avro" sinkURI, err = url.Parse(uri) - require.Nil(t, err) + require.NoError(t, err) protocol = sinkURI.Query().Get("protocol") - require.Equal(t, protocol, "avro") + require.Equal(t, "avro", protocol) err = p.FromString(protocol) - require.Nil(t, err) - c = NewConfig(p, timeutil.SystemLocation()) - require.Equal(t, c.protocol, config.ProtocolAvro) - - err = c.Apply(sinkURI, opts) - require.Nil(t, err) - require.Equal(t, c.avroRegistry, "") - // `registry` not set + require.NoError(t, err) + c = NewConfig(p) + require.Equal(t, config.ProtocolAvro, c.protocol) + + err = c.Apply(sinkURI, replicaConfig) + require.NoError(t, err) + require.Equal(t, "", c.avroSchemaRegistry) + // `schema-registry` not set + err = c.Validate() + require.ErrorContains(t, err, `Avro protocol requires parameter "schema-registry"`) + + replicaConfig.SchemaRegistry = "this-is-a-uri" + err = c.Apply(sinkURI, replicaConfig) + require.NoError(t, err) + require.Equal(t, "this-is-a-uri", c.avroSchemaRegistry) err = c.Validate() - require.Error(t, err, `Avro protocol requires parameter "registry"`) + require.NoError(t, err) + + // avro-decimal-handling-mode + c = NewConfig(config.ProtocolAvro) + require.Equal(t, "precise", c.avroDecimalHandlingMode) + + uri = "kafka://127.0.0.1:9092/abc?protocol=avro&avro-decimal-handling-mode=string" + sinkURI, err = url.Parse(uri) + require.NoError(t, err) + + err = c.Apply(sinkURI, replicaConfig) + require.NoError(t, err) + require.Equal(t, "string", c.avroDecimalHandlingMode) + + err = c.Validate() + require.NoError(t, err) + + uri = "kafka://127.0.0.1:9092/abc?protocol=avro&avro-decimal-handling-mode=invalid" + sinkURI, err = url.Parse(uri) + require.NoError(t, err) + + err = c.Apply(sinkURI, replicaConfig) + require.NoError(t, err) + require.Equal(t, "invalid", c.avroDecimalHandlingMode) - opts["registry"] = "this-is-a-uri" - err = c.Apply(sinkURI, opts) - require.Nil(t, err) - require.Equal(t, c.avroRegistry, "this-is-a-uri") err = c.Validate() - require.Nil(t, err) + require.ErrorContains( + t, + err, + `avro-decimal-handling-mode value could only be "string" or "precise"`, + ) + + // avro-bigint-unsigned-handling-mode + c = NewConfig(config.ProtocolAvro) + require.Equal(t, "long", c.avroBigintUnsignedHandlingMode) + + uri = "kafka://127.0.0.1:9092/abc?protocol=avro&avro-bigint-unsigned-handling-mode=string" + sinkURI, err = url.Parse(uri) + require.NoError(t, err) + + err = c.Apply(sinkURI, replicaConfig) + require.NoError(t, err) + require.Equal(t, "string", c.avroBigintUnsignedHandlingMode) + + err = c.Validate() + require.NoError(t, err) + + uri = "kafka://127.0.0.1:9092/abc?protocol=avro&avro-bigint-unsigned-handling-mode=invalid" + sinkURI, err = url.Parse(uri) + require.NoError(t, err) + + err = c.Apply(sinkURI, replicaConfig) + require.NoError(t, err) + require.Equal(t, "invalid", c.avroBigintUnsignedHandlingMode) + + err = c.Validate() + require.ErrorContains( + t, + err, + `bigint-unsigned-handling-mode value could only be "long" or "string"`, + ) // Illegal max-message-bytes. uri = "kafka://127.0.0.1:9092/abc?kafka-version=2.6.0&max-message-bytes=a" sinkURI, err = url.Parse(uri) - require.Nil(t, err) - err = c.Apply(sinkURI, opts) - require.Error(t, err, "invalid syntax") + require.NoError(t, err) + err = c.Apply(sinkURI, replicaConfig) + require.ErrorContains(t, err, "invalid syntax") uri = "kafka://127.0.0.1:9092/abc?kafka-version=2.6.0&max-message-bytes=-1" sinkURI, err = url.Parse(uri) - require.Nil(t, err) - err = c.Apply(sinkURI, opts) - require.Nil(t, err) + require.NoError(t, err) + + c = NewConfig(config.ProtocolOpen) + err = c.Apply(sinkURI, replicaConfig) + require.NoError(t, err) err = c.Validate() - require.Error(t, err, cerror.ErrMQCodecInvalidConfig) + require.ErrorContains(t, err, "invalid max-message-bytes -1") // Illegal max-batch-size uri = "kafka://127.0.0.1:9092/abc?kafka-version=2.6.0&max-batch-size=a" sinkURI, err = url.Parse(uri) - require.Nil(t, err) - err = c.Apply(sinkURI, opts) - require.Error(t, err, "invalid syntax") + require.NoError(t, err) + err = c.Apply(sinkURI, replicaConfig) + require.ErrorContains(t, err, "invalid syntax") uri = "kafka://127.0.0.1:9092/abc?kafka-version=2.6.0&max-batch-size=-1" sinkURI, err = url.Parse(uri) - require.Nil(t, err) + require.NoError(t, err) - c = NewConfig(config.ProtocolOpen, timeutil.SystemLocation()) - err = c.Apply(sinkURI, opts) - require.Nil(t, err) + c = NewConfig(config.ProtocolOpen) + err = c.Apply(sinkURI, replicaConfig) + require.NoError(t, err) err = c.Validate() - require.Error(t, err, cerror.ErrMQCodecInvalidConfig) + require.ErrorContains(t, err, "invalid max-batch-size -1") } diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index 1e474dfadc7..4645cbece9d 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -14,6 +14,8 @@ package codec import ( + "context" + "github.com/pingcap/errors" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/codec/craft" @@ -50,7 +52,11 @@ func (e *CraftEventBatchEncoder) flush() { } // AppendRowChangedEvent implements the EventBatchEncoder interface -func (e *CraftEventBatchEncoder) AppendRowChangedEvent(ev *model.RowChangedEvent) error { +func (e *CraftEventBatchEncoder) AppendRowChangedEvent( + ctx context.Context, + topic string, + ev *model.RowChangedEvent, +) error { rows, size := e.rowChangedBuffer.AppendRowChangedEvent(ev) if size > e.maxMessageBytes || rows >= e.maxBatchSize { e.flush() diff --git a/cdc/sink/codec/craft_test.go b/cdc/sink/codec/craft_test.go index 9bd435245dd..ce1809f3ef0 100644 --- a/cdc/sink/codec/craft_test.go +++ b/cdc/sink/codec/craft_test.go @@ -14,10 +14,10 @@ package codec import ( + "context" "testing" "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/stretchr/testify/require" @@ -80,7 +80,7 @@ func testBatchCodec( for _, cs := range s.rowCases { events := 0 for _, row := range cs { - err := encoder.AppendRowChangedEvent(row) + err := encoder.AppendRowChangedEvent(context.Background(), "", row) events++ require.Nil(t, err) } @@ -121,7 +121,7 @@ func testBatchCodec( func TestCraftMaxMessageBytes(t *testing.T) { t.Parallel() - config := NewConfig(config.ProtocolCraft, timeutil.SystemLocation()).WithMaxMessageBytes(256) + config := NewConfig(config.ProtocolCraft).WithMaxMessageBytes(256) encoder := newCraftEventBatchEncoderBuilder(config).Build() testEvent := &model.RowChangedEvent{ @@ -131,7 +131,7 @@ func TestCraftMaxMessageBytes(t *testing.T) { } for i := 0; i < 10000; i++ { - err := encoder.AppendRowChangedEvent(testEvent) + err := encoder.AppendRowChangedEvent(context.Background(), "", testEvent) require.Nil(t, err) } @@ -143,7 +143,7 @@ func TestCraftMaxMessageBytes(t *testing.T) { func TestCraftMaxBatchSize(t *testing.T) { t.Parallel() - config := NewConfig(config.ProtocolCraft, timeutil.SystemLocation()).WithMaxMessageBytes(10485760) + config := NewConfig(config.ProtocolCraft).WithMaxMessageBytes(10485760) config.maxBatchSize = 64 encoder := newCraftEventBatchEncoderBuilder(config).Build() @@ -154,7 +154,7 @@ func TestCraftMaxBatchSize(t *testing.T) { } for i := 0; i < 10000; i++ { - err := encoder.AppendRowChangedEvent(testEvent) + err := encoder.AppendRowChangedEvent(context.Background(), "", testEvent) require.Nil(t, err) } @@ -183,14 +183,14 @@ func TestCraftMaxBatchSize(t *testing.T) { } func TestDefaultCraftEventBatchCodec(t *testing.T) { - config := NewConfig(config.ProtocolCraft, timeutil.SystemLocation()).WithMaxMessageBytes(8192) + config := NewConfig(config.ProtocolCraft).WithMaxMessageBytes(8192) config.maxBatchSize = 64 testBatchCodec(t, newCraftEventBatchEncoderBuilder(config), NewCraftEventBatchDecoder) } func TestBuildCraftEventBatchEncoder(t *testing.T) { t.Parallel() - config := NewConfig(config.ProtocolCraft, timeutil.SystemLocation()) + config := NewConfig(config.ProtocolCraft) builder := &craftEventBatchEncoderBuilder{config: config} encoder, ok := builder.Build().(*CraftEventBatchEncoder) diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index d2caeed3b83..4436be7d9f4 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -14,15 +14,14 @@ package codec import ( + "context" "encoding/binary" "time" - "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/security" + cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/tikv/client-go/v2/oracle" - "go.uber.org/zap" ) // EventBatchEncoder is an abstraction for events encoder @@ -30,8 +29,9 @@ type EventBatchEncoder interface { // EncodeCheckpointEvent appends a checkpoint event into the batch. // This event will be broadcast to all partitions to signal a global checkpoint. EncodeCheckpointEvent(ts uint64) (*MQMessage, error) - // AppendRowChangedEvent appends a row changed event into the batch - AppendRowChangedEvent(e *model.RowChangedEvent) error + // AppendRowChangedEvent appends the calling context, a row changed event and the dispatch + // topic into the batch + AppendRowChangedEvent(context.Context, string, *model.RowChangedEvent) error // EncodeDDLEvent appends a DDL event into the batch EncodeDDLEvent(e *model.DDLEvent) (*MQMessage, error) // Build builds the batch and returns the bytes of key and value. @@ -85,7 +85,15 @@ func (m *MQMessage) IncRowsCount() { } func newDDLMQMessage(proto config.Protocol, key, value []byte, event *model.DDLEvent) *MQMessage { - return NewMQMessage(proto, key, value, event.CommitTs, model.MqMessageTypeDDL, &event.TableInfo.Schema, &event.TableInfo.Table) + return NewMQMessage( + proto, + key, + value, + event.CommitTs, + model.MqMessageTypeDDL, + &event.TableInfo.Schema, + &event.TableInfo.Table, + ) } func newResolvedMQMessage(proto config.Protocol, key, value []byte, ts uint64) *MQMessage { @@ -94,7 +102,14 @@ func newResolvedMQMessage(proto config.Protocol, key, value []byte, ts uint64) * // NewMQMessage should be used when creating a MQMessage struct. // It copies the input byte slices to avoid any surprises in asynchronous MQ writes. -func NewMQMessage(proto config.Protocol, key []byte, value []byte, ts uint64, ty model.MqMessageType, schema, table *string) *MQMessage { +func NewMQMessage( + proto config.Protocol, + key []byte, + value []byte, + ts uint64, + ty model.MqMessageType, + schema, table *string, +) *MQMessage { ret := &MQMessage{ Key: nil, Value: nil, @@ -150,14 +165,14 @@ type EncoderBuilder interface { } // NewEventBatchEncoderBuilder returns an EncoderBuilder -func NewEventBatchEncoderBuilder(c *Config, credential *security.Credential) (EncoderBuilder, error) { +func NewEventBatchEncoderBuilder(ctx context.Context, c *Config) (EncoderBuilder, error) { switch c.protocol { case config.ProtocolDefault, config.ProtocolOpen: return newJSONEventBatchEncoderBuilder(c), nil case config.ProtocolCanal: return newCanalEventBatchEncoderBuilder(), nil case config.ProtocolAvro: - return newAvroEventBatchEncoderBuilder(credential, c) + return newAvroEventBatchEncoderBuilder(ctx, c) case config.ProtocolMaxwell: return newMaxwellEventBatchEncoderBuilder(), nil case config.ProtocolCanalJSON: @@ -165,7 +180,6 @@ func NewEventBatchEncoderBuilder(c *Config, credential *security.Credential) (En case config.ProtocolCraft: return newCraftEventBatchEncoderBuilder(c), nil default: - log.Warn("unknown codec protocol value of EventBatchEncoder, use open-protocol as the default", zap.Any("protocolValue", int(c.protocol))) - return newJSONEventBatchEncoderBuilder(c), nil + return nil, cerror.ErrMQSinkUnknownProtocol.GenWithStackByArgs(c.protocol) } } diff --git a/cdc/sink/codec/json.go b/cdc/sink/codec/json.go index 229dd4b6f94..92f0c7532cf 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -15,6 +15,7 @@ package codec import ( "bytes" + "context" "encoding/base64" "encoding/binary" "encoding/json" @@ -424,7 +425,11 @@ func (d *JSONEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessage, er } // AppendRowChangedEvent implements the EventBatchEncoder interface -func (d *JSONEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) error { +func (d *JSONEventBatchEncoder) AppendRowChangedEvent( + ctx context.Context, + topic string, + e *model.RowChangedEvent, +) error { keyMsg, valueMsg := rowEventToMqMessage(e) key, err := keyMsg.Encode() if err != nil { diff --git a/cdc/sink/codec/json_test.go b/cdc/sink/codec/json_test.go index 748f2153278..45733fa1896 100644 --- a/cdc/sink/codec/json_test.go +++ b/cdc/sink/codec/json_test.go @@ -14,11 +14,11 @@ package codec import ( + "context" "sort" "testing" "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/stretchr/testify/require" @@ -116,7 +116,7 @@ func (s *batchTester) testBatchCodec( encoder := encoderBuilder.Build() for _, row := range cs { - err := encoder.AppendRowChangedEvent(row) + err := encoder.AppendRowChangedEvent(context.Background(), "", row) require.Nil(t, err) } @@ -157,7 +157,7 @@ func (s *batchTester) testBatchCodec( func TestBuildJSONEventBatchEncoder(t *testing.T) { t.Parallel() - config := NewConfig(config.ProtocolOpen, timeutil.SystemLocation()) + config := NewConfig(config.ProtocolOpen) builder := &jsonEventBatchEncoderBuilder{config: config} encoder, ok := builder.Build().(*JSONEventBatchEncoder) require.True(t, ok) @@ -174,24 +174,26 @@ func TestMaxMessageBytes(t *testing.T) { Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, } + ctx := context.Background() + topic := "" // for a single message, the overhead is 36(maximumRecordOverhead) + 8(versionHea) = 44, just can hold it. a := 87 + 44 - config := NewConfig(config.ProtocolOpen, timeutil.SystemLocation()).WithMaxMessageBytes(a) + config := NewConfig(config.ProtocolOpen).WithMaxMessageBytes(a) encoder := newJSONEventBatchEncoderBuilder(config).Build() - err := encoder.AppendRowChangedEvent(testEvent) + err := encoder.AppendRowChangedEvent(ctx, topic, testEvent) require.Nil(t, err) // cannot hold a single message config = config.WithMaxMessageBytes(a - 1) encoder = newJSONEventBatchEncoderBuilder(config).Build() - err = encoder.AppendRowChangedEvent(testEvent) + err = encoder.AppendRowChangedEvent(ctx, topic, testEvent) require.NotNil(t, err) // make sure each batch's `Length` not greater than `max-message-bytes` config = config.WithMaxMessageBytes(256) encoder = newJSONEventBatchEncoderBuilder(config).Build() for i := 0; i < 10000; i++ { - err := encoder.AppendRowChangedEvent(testEvent) + err := encoder.AppendRowChangedEvent(ctx, topic, testEvent) require.Nil(t, err) } @@ -203,7 +205,7 @@ func TestMaxMessageBytes(t *testing.T) { func TestMaxBatchSize(t *testing.T) { t.Parallel() - config := NewConfig(config.ProtocolOpen, timeutil.SystemLocation()).WithMaxMessageBytes(1048576) + config := NewConfig(config.ProtocolOpen).WithMaxMessageBytes(1048576) config.maxBatchSize = 64 encoder := newJSONEventBatchEncoderBuilder(config).Build() @@ -214,7 +216,7 @@ func TestMaxBatchSize(t *testing.T) { } for i := 0; i < 10000; i++ { - err := encoder.AppendRowChangedEvent(testEvent) + err := encoder.AppendRowChangedEvent(context.Background(), "", testEvent) require.Nil(t, err) } @@ -243,7 +245,7 @@ func TestMaxBatchSize(t *testing.T) { } func TestDefaultEventBatchCodec(t *testing.T) { - config := NewConfig(config.ProtocolOpen, timeutil.SystemLocation()).WithMaxMessageBytes(8192) + config := NewConfig(config.ProtocolOpen).WithMaxMessageBytes(8192) config.maxBatchSize = 64 tester := NewDefaultBatchTester() tester.testBatchCodec(t, newJSONEventBatchEncoderBuilder(config), NewJSONEventBatchDecoder) diff --git a/cdc/sink/codec/maxwell.go b/cdc/sink/codec/maxwell.go index 7d991c7ef12..5a8f01a642f 100644 --- a/cdc/sink/codec/maxwell.go +++ b/cdc/sink/codec/maxwell.go @@ -15,6 +15,7 @@ package codec import ( "bytes" + "context" "encoding/binary" "encoding/json" @@ -163,7 +164,11 @@ func rowEventToMaxwellMessage(e *model.RowChangedEvent) (*mqMessageKey, *maxwell } // AppendRowChangedEvent implements the EventBatchEncoder interface -func (d *MaxwellEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) error { +func (d *MaxwellEventBatchEncoder) AppendRowChangedEvent( + ctx context.Context, + topic string, + e *model.RowChangedEvent, +) error { _, valueMsg := rowEventToMaxwellMessage(e) value, err := valueMsg.Encode() if err != nil { diff --git a/cdc/sink/codec/maxwell_test.go b/cdc/sink/codec/maxwell_test.go index f66caa20d78..21550d9ee6d 100644 --- a/cdc/sink/codec/maxwell_test.go +++ b/cdc/sink/codec/maxwell_test.go @@ -14,6 +14,7 @@ package codec import ( + "context" "testing" "github.com/pingcap/tiflow/cdc/model" @@ -32,7 +33,7 @@ func TestMaxwellEventBatchCodec(t *testing.T) { for _, cs := range rowCases { encoder := newEncoder() for _, row := range cs { - err := encoder.AppendRowChangedEvent(row) + err := encoder.AppendRowChangedEvent(context.Background(), "", row) require.Nil(t, err) } size := encoder.Size() diff --git a/cdc/sink/codec/schema_registry.go b/cdc/sink/codec/schema_registry.go index 3251346f579..c1baed6826b 100644 --- a/cdc/sink/codec/schema_registry.go +++ b/cdc/sink/codec/schema_registry.go @@ -20,7 +20,6 @@ import ( "io" "net/http" "net/url" - "regexp" "strings" "sync" "time" @@ -29,7 +28,6 @@ import ( "github.com/linkedin/goavro/v2" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/httputil" "github.com/pingcap/tiflow/pkg/security" @@ -43,7 +41,7 @@ type AvroSchemaManager struct { registryURL string subjectSuffix string - credential *security.Credential + credential *security.Credential // placeholder, currently always nil cacheRWLock sync.RWMutex cache map[string]*schemaCacheEntry @@ -71,68 +69,83 @@ type lookupResponse struct { Schema string `json:"schema"` } -// NewAvroSchemaManager creates a new AvroSchemaManager +// NewAvroSchemaManager creates a new AvroSchemaManager and test connectivity to the schema registry func NewAvroSchemaManager( ctx context.Context, credential *security.Credential, registryURL string, subjectSuffix string, ) (*AvroSchemaManager, error) { registryURL = strings.TrimRight(registryURL, "/") - // Test connectivity to the Schema Registry httpCli, err := httputil.NewClient(credential) if err != nil { return nil, errors.Trace(err) } resp, err := httpCli.Get(ctx, registryURL) if err != nil { - return nil, errors.Annotate( - cerror.WrapError(cerror.ErrAvroSchemaAPIError, err), "Test connection to Schema Registry failed") + log.Error("Test connection to Schema Registry failed", zap.Error(err)) + return nil, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) } defer resp.Body.Close() text, err := io.ReadAll(resp.Body) if err != nil { - return nil, errors.Annotate( - cerror.WrapError(cerror.ErrAvroSchemaAPIError, err), "Reading response from Schema Registry failed") + log.Error("Reading response from Schema Registry failed", zap.Error(err)) + return nil, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) } if string(text[:]) != "{}" { - return nil, cerror.ErrAvroSchemaAPIError.GenWithStack("Unexpected response from Schema Registry") + return nil, cerror.ErrAvroSchemaAPIError.GenWithStack( + "Unexpected response from Schema Registry", + ) } - log.Info("Successfully tested connectivity to Schema Registry", zap.String("registryURL", registryURL)) + log.Info( + "Successfully tested connectivity to Schema Registry", + zap.String("registryURL", registryURL), + ) return &AvroSchemaManager{ registryURL: registryURL, cache: make(map[string]*schemaCacheEntry, 1), subjectSuffix: subjectSuffix, - credential: credential, }, nil } -var regexRemoveSpaces = regexp.MustCompile(`\s`) - -// Register the latest schema for a table to the Registry, by passing in a Codec -// Returns the Schema's ID and err -func (m *AvroSchemaManager) Register(ctx context.Context, tableName model.TableName, codec *goavro.Codec) (int, error) { +// Register a schema in schema registry, no cache +func (m *AvroSchemaManager) Register( + ctx context.Context, + topicName string, + codec *goavro.Codec, +) (int, error) { // The Schema Registry expects the JSON to be without newline characters + buffer := new(bytes.Buffer) + err := json.Compact(buffer, []byte(codec.Schema())) + if err != nil { + log.Error("Could not compact schema", zap.Error(err)) + return 0, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) + } reqBody := registerRequest{ - Schema: regexRemoveSpaces.ReplaceAllString(codec.Schema(), ""), - // Commented out for compatibility with Confluent 5.4.x - // SchemaType: "AVRO", + Schema: buffer.String(), } payload, err := json.Marshal(&reqBody) if err != nil { - return 0, errors.Annotate( - cerror.WrapError(cerror.ErrAvroSchemaAPIError, err), "Could not marshal request to the Registry") + log.Error("Could not marshal request to the Registry", zap.Error(err)) + return 0, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) } - uri := m.registryURL + "/subjects/" + url.QueryEscape(m.tableNameToSchemaSubject(tableName)) + "/versions" - log.Debug("Registering schema", zap.String("uri", uri), zap.ByteString("payload", payload)) + uri := m.registryURL + "/subjects/" + url.QueryEscape( + m.topicNameToSchemaSubject(topicName), + ) + "/versions?normalize=true" + log.Info("Registering schema", zap.String("uri", uri), zap.ByteString("payload", payload)) req, err := http.NewRequestWithContext(ctx, "POST", uri, bytes.NewReader(payload)) if err != nil { - return 0, cerror.ErrAvroSchemaAPIError.GenWithStackByArgs() + log.Error("Failed to NewRequestWithContext", zap.Error(err)) + return 0, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) } - req.Header.Add("Accept", "application/vnd.schemaregistry.v1+json") - resp, err := httpRetry(ctx, m.credential, req, false) + req.Header.Add( + "Accept", + "application/vnd.schemaregistry.v1+json, application/vnd.schemaregistry+json, "+ + "application/json", + ) + resp, err := httpRetry(ctx, m.credential, req) if err != nil { return 0, err } @@ -140,28 +153,37 @@ func (m *AvroSchemaManager) Register(ctx context.Context, tableName model.TableN body, err := io.ReadAll(resp.Body) if err != nil { - return 0, errors.Annotate(err, "Failed to read response from Registry") + log.Error("Failed to read response from Registry", zap.Error(err)) + return 0, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) } if resp.StatusCode != 200 { - log.Warn("Failed to register schema to the Registry, HTTP error", + // https://docs.confluent.io/platform/current/schema-registry/develop/api.html \ + // #post--subjects-(string-%20subject)-versions + // 409 for incompatible schema + log.Error( + "Failed to register schema to the Registry, HTTP error", zap.Int("status", resp.StatusCode), zap.String("uri", uri), zap.ByteString("requestBody", payload), - zap.ByteString("responseBody", body)) - return 0, cerror.ErrAvroSchemaAPIError.GenWithStack("Failed to register schema to the Registry, HTTP error") + zap.ByteString("responseBody", body), + ) + return 0, cerror.ErrAvroSchemaAPIError.GenWithStackByArgs() } var jsonResp registerResponse err = json.Unmarshal(body, &jsonResp) if err != nil { - return 0, errors.Annotate( - cerror.WrapError(cerror.ErrAvroSchemaAPIError, err), "Failed to parse result from Registry") + log.Error("Failed to parse result from Registry", zap.Error(err)) + return 0, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) } if jsonResp.ID == 0 { - return 0, cerror.ErrAvroSchemaAPIError.GenWithStack("Illegal schema ID returned from Registry %d", jsonResp.ID) + return 0, cerror.ErrAvroSchemaAPIError.GenWithStack( + "Illegal schema ID returned from Registry %d", + jsonResp.ID, + ) } log.Info("Registered schema successfully", @@ -174,11 +196,16 @@ func (m *AvroSchemaManager) Register(ctx context.Context, tableName model.TableN // Lookup the latest schema and the Registry designated ID for that schema. // TiSchemaId is only used to trigger fetching from the Registry server. -// Calling this method with a tiSchemaID other than that used last time will invariably trigger a RESTful request to the Registry. +// Calling this method with a tiSchemaID other than that used last time will invariably trigger a +// RESTful request to the Registry. // Returns (codec, registry schema ID, error) // NOT USED for now, reserved for future use. -func (m *AvroSchemaManager) Lookup(ctx context.Context, tableName model.TableName, tiSchemaID uint64) (*goavro.Codec, int, error) { - key := m.tableNameToSchemaSubject(tableName) +func (m *AvroSchemaManager) Lookup( + ctx context.Context, + topicName string, + tiSchemaID uint64, +) (*goavro.Codec, int, error) { + key := m.topicNameToSchemaSubject(topicName) m.cacheRWLock.RLock() if entry, exists := m.cache[key]; exists && entry.tiSchemaID == tiSchemaID { log.Info("Avro schema lookup cache hit", @@ -194,17 +221,21 @@ func (m *AvroSchemaManager) Lookup(ctx context.Context, tableName model.TableNam zap.String("key", key), zap.Uint64("tiSchemaID", tiSchemaID)) - uri := m.registryURL + "/subjects/" + url.QueryEscape(m.tableNameToSchemaSubject(tableName)) + "/versions/latest" + uri := m.registryURL + "/subjects/" + url.QueryEscape(key) + "/versions/latest" log.Debug("Querying for latest schema", zap.String("uri", uri)) req, err := http.NewRequestWithContext(ctx, "GET", uri, nil) if err != nil { - return nil, 0, errors.Annotate( - cerror.WrapError(cerror.ErrAvroSchemaAPIError, err), "Error constructing request for Registry lookup") + log.Error("Error constructing request for Registry lookup", zap.Error(err)) + return nil, 0, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) } - req.Header.Add("Accept", "application/vnd.schemaregistry.v1+json, application/vnd.schemaregistry+json, application/json") + req.Header.Add( + "Accept", + "application/vnd.schemaregistry.v1+json, application/vnd.schemaregistry+json, "+ + "application/json", + ) - resp, err := httpRetry(ctx, m.credential, req, true) + resp, err := httpRetry(ctx, m.credential, req) if err != nil { return nil, 0, err } @@ -212,45 +243,47 @@ func (m *AvroSchemaManager) Lookup(ctx context.Context, tableName model.TableNam body, err := io.ReadAll(resp.Body) if err != nil { - return nil, 0, errors.Annotate( - cerror.WrapError(cerror.ErrAvroSchemaAPIError, err), "Failed to read response from Registry") + log.Error("Failed to parse result from Registry", zap.Error(err)) + return nil, 0, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) } if resp.StatusCode != 200 && resp.StatusCode != 404 { - log.Warn("Failed to query schema from the Registry, HTTP error", + log.Error("Failed to query schema from the Registry, HTTP error", zap.Int("status", resp.StatusCode), zap.String("uri", uri), - zap.ByteString("responseBody", body)) - return nil, 0, cerror.ErrAvroSchemaAPIError.GenWithStack("Failed to query schema from the Registry, HTTP error") + return nil, 0, cerror.ErrAvroSchemaAPIError.GenWithStack( + "Failed to query schema from the Registry, HTTP error", + ) } if resp.StatusCode == 404 { log.Warn("Specified schema not found in Registry", zap.String("key", key), zap.Uint64("tiSchemaID", tiSchemaID)) - - return nil, 0, cerror.ErrAvroSchemaAPIError.GenWithStackByArgs("Schema not found in Registry") + return nil, 0, cerror.ErrAvroSchemaAPIError.GenWithStackByArgs( + "Schema not found in Registry", + ) } var jsonResp lookupResponse err = json.Unmarshal(body, &jsonResp) if err != nil { - return nil, 0, errors.Annotate( - cerror.WrapError(cerror.ErrAvroSchemaAPIError, err), "Failed to parse result from Registry") + log.Error("Failed to parse result from Registry", zap.Error(err)) + return nil, 0, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) } cacheEntry := new(schemaCacheEntry) cacheEntry.codec, err = goavro.NewCodec(jsonResp.Schema) if err != nil { - return nil, 0, errors.Annotate( - cerror.WrapError(cerror.ErrAvroSchemaAPIError, err), "Creating Avro codec failed") + log.Error("Creating Avro codec failed", zap.Error(err)) + return nil, 0, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) } cacheEntry.registryID = jsonResp.RegistryID cacheEntry.tiSchemaID = tiSchemaID m.cacheRWLock.Lock() - m.cache[m.tableNameToSchemaSubject(tableName)] = cacheEntry + m.cache[key] = cacheEntry m.cacheRWLock.Unlock() log.Info("Avro schema lookup successful with cache miss", @@ -267,8 +300,15 @@ type SchemaGenerator func() (string, error) // GetCachedOrRegister checks if the suitable Avro schema has been cached. // If not, a new schema is generated, registered and cached. -func (m *AvroSchemaManager) GetCachedOrRegister(ctx context.Context, tableName model.TableName, tiSchemaID uint64, schemaGen SchemaGenerator) (*goavro.Codec, int, error) { - key := m.tableNameToSchemaSubject(tableName) +// Re-registering an existing schema shall return the same id(and version), so even if the +// cache is out-of-sync with schema registry, we could reload it. +func (m *AvroSchemaManager) GetCachedOrRegister( + ctx context.Context, + topicName string, + tiSchemaID uint64, + schemaGen SchemaGenerator, +) (*goavro.Codec, int, error) { + key := m.topicNameToSchemaSubject(topicName) m.cacheRWLock.RLock() if entry, exists := m.cache[key]; exists && entry.tiSchemaID == tiSchemaID { log.Debug("Avro schema GetCachedOrRegister cache hit", @@ -286,19 +326,19 @@ func (m *AvroSchemaManager) GetCachedOrRegister(ctx context.Context, tableName m schema, err := schemaGen() if err != nil { - return nil, 0, errors.Annotate(err, "GetCachedOrRegister: SchemaGen failed") + return nil, 0, err } codec, err := goavro.NewCodec(schema) if err != nil { - return nil, 0, errors.Annotate( - cerror.WrapError(cerror.ErrAvroSchemaAPIError, err), "GetCachedOrRegister: Could not make goavro codec") + log.Error("GetCachedOrRegister: Could not make goavro codec", zap.Error(err)) + return nil, 0, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) } - id, err := m.Register(ctx, tableName, codec) + id, err := m.Register(ctx, topicName, codec) if err != nil { - return nil, 0, errors.Annotate( - cerror.WrapError(cerror.ErrAvroSchemaAPIError, err), "GetCachedOrRegister: Could not register schema") + log.Error("GetCachedOrRegister: Could not register schema", zap.Error(err)) + return nil, 0, errors.Trace(err) } cacheEntry := new(schemaCacheEntry) @@ -307,7 +347,7 @@ func (m *AvroSchemaManager) GetCachedOrRegister(ctx context.Context, tableName m cacheEntry.tiSchemaID = tiSchemaID m.cacheRWLock.Lock() - m.cache[m.tableNameToSchemaSubject(tableName)] = cacheEntry + m.cache[key] = cacheEntry m.cacheRWLock.Unlock() log.Info("Avro schema GetCachedOrRegister successful with cache miss", @@ -321,15 +361,21 @@ func (m *AvroSchemaManager) GetCachedOrRegister(ctx context.Context, tableName m // ClearRegistry clears the Registry subject for the given table. Should be idempotent. // Exported for testing. // NOT USED for now, reserved for future use. -func (m *AvroSchemaManager) ClearRegistry(ctx context.Context, tableName model.TableName) error { - uri := m.registryURL + "/subjects/" + url.QueryEscape(m.tableNameToSchemaSubject(tableName)) +func (m *AvroSchemaManager) ClearRegistry(ctx context.Context, topicName string) error { + uri := m.registryURL + "/subjects/" + url.QueryEscape( + m.topicNameToSchemaSubject(topicName), + ) req, err := http.NewRequestWithContext(ctx, "DELETE", uri, nil) if err != nil { log.Error("Could not construct request for clearRegistry", zap.String("uri", uri)) return cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) } - req.Header.Add("Accept", "application/vnd.schemaregistry.v1+json, application/vnd.schemaregistry+json, application/json") - resp, err := httpRetry(ctx, m.credential, req, true) + req.Header.Add( + "Accept", + "application/vnd.schemaregistry.v1+json, application/vnd.schemaregistry+json, "+ + "application/json", + ) + resp, err := httpRetry(ctx, m.credential, req) if err != nil { return err } @@ -349,10 +395,17 @@ func (m *AvroSchemaManager) ClearRegistry(ctx context.Context, tableName model.T } log.Error("Error when clearing Registry", zap.Int("status", resp.StatusCode)) - return cerror.ErrAvroSchemaAPIError.GenWithStack("Error when clearing Registry, status = %d", resp.StatusCode) + return cerror.ErrAvroSchemaAPIError.GenWithStack( + "Error when clearing Registry, status = %d", + resp.StatusCode, + ) } -func httpRetry(ctx context.Context, credential *security.Credential, r *http.Request, allow404 bool) (*http.Response, error) { +func httpRetry( + ctx context.Context, + credential *security.Credential, + r *http.Request, +) (*http.Response, error) { var ( err error resp *http.Response @@ -369,6 +422,7 @@ func httpRetry(ctx context.Context, credential *security.Credential, r *http.Req } if err != nil { + log.Error("Failed to parse response", zap.Error(err)) return nil, cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) } for { @@ -382,7 +436,9 @@ func httpRetry(ctx context.Context, credential *security.Credential, r *http.Req goto checkCtx } - if resp.StatusCode >= 200 && resp.StatusCode < 300 || (resp.StatusCode == 404 && allow404) { + // retry 4xx codes like 409 & 422 has no meaning since it's non-recoverable + if resp.StatusCode >= 200 && resp.StatusCode < 300 || + (resp.StatusCode >= 400 && resp.StatusCode < 500) { break } log.Warn("HTTP server returned with error", zap.Int("status", resp.StatusCode)) @@ -393,7 +449,6 @@ func httpRetry(ctx context.Context, credential *security.Credential, r *http.Req select { case <-ctx.Done(): return nil, errors.New("HTTP retry cancelled") - default: } @@ -403,7 +458,7 @@ func httpRetry(ctx context.Context, credential *security.Credential, r *http.Req return resp, nil } -func (m *AvroSchemaManager) tableNameToSchemaSubject(tableName model.TableName) string { - // We should guarantee unique names for subjects - return tableName.Schema + "_" + tableName.Table + m.subjectSuffix +// TopicNameStrategy, ksqlDB only supports this +func (m *AvroSchemaManager) topicNameToSchemaSubject(topicName string) string { + return topicName + m.subjectSuffix } diff --git a/cdc/sink/codec/schema_registry_test.go b/cdc/sink/codec/schema_registry_test.go index 3e3e9852b6a..7dd7ed42faa 100644 --- a/cdc/sink/codec/schema_registry_test.go +++ b/cdc/sink/codec/schema_registry_test.go @@ -25,8 +25,6 @@ import ( "github.com/jarcoal/httpmock" "github.com/linkedin/goavro/v2" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/security" "github.com/stretchr/testify/require" ) @@ -42,7 +40,7 @@ type mockRegistrySchema struct { ID int } -func startHTTPInterceptForTestingRegistry(t *testing.T) { +func startHTTPInterceptForTestingRegistry() { httpmock.Activate() registry := mockRegistry{ @@ -50,7 +48,11 @@ func startHTTPInterceptForTestingRegistry(t *testing.T) { newID: 1, } - httpmock.RegisterResponder("GET", "http://127.0.0.1:8081", httpmock.NewStringResponder(200, "{}")) + httpmock.RegisterResponder( + "GET", + "http://127.0.0.1:8081", + httpmock.NewStringResponder(200, "{}"), + ) httpmock.RegisterResponder("POST", `=~^http://127.0.0.1:8081/subjects/(.+)/versions`, func(req *http.Request) (*http.Response, error) { @@ -68,15 +70,13 @@ func startHTTPInterceptForTestingRegistry(t *testing.T) { return nil, err } - // require.Equal(t, "AVRO", reqData.SchemaType) - var respData registerResponse registry.mu.Lock() item, exists := registry.subjects[subject] if !exists { item = &mockRegistrySchema{ content: reqData.Schema, - version: 0, + version: 1, ID: registry.newID, } registry.subjects[subject] = item @@ -127,24 +127,23 @@ func startHTTPInterceptForTestingRegistry(t *testing.T) { registry.mu.Lock() defer registry.mu.Unlock() - _, exists := registry.subjects[subject] + item, exists := registry.subjects[subject] if !exists { return httpmock.NewStringResponse(404, ""), nil } delete(registry.subjects, subject) - return httpmock.NewStringResponse(200, ""), nil + // simplify the response not returning all the versions + return httpmock.NewJsonResponse(200, []int{item.version}) }) failCounter := 0 httpmock.RegisterResponder("POST", `=~^http://127.0.0.1:8081/may-fail`, func(req *http.Request) (*http.Response, error) { - data, _ := io.ReadAll(req.Body) - require.Greater(t, len(data), 0) - require.Equal(t, req.ContentLength, int64(len(data))) + io.ReadAll(req.Body) if failCounter < 3 { failCounter++ - return httpmock.NewStringResponse(422, ""), nil + return httpmock.NewStringResponse(500, ""), nil } return httpmock.NewStringResponse(200, ""), nil }) @@ -161,21 +160,23 @@ func getTestingContext() context.Context { } func TestSchemaRegistry(t *testing.T) { - startHTTPInterceptForTestingRegistry(t) + startHTTPInterceptForTestingRegistry() defer stopHTTPInterceptForTestingRegistry() - table := model.TableName{ - Schema: "testdb", - Table: "test1", - } + manager, err := NewAvroSchemaManager( + getTestingContext(), + nil, + "http://127.0.0.1:8081", + "-value", + ) + require.NoError(t, err) - manager, err := NewAvroSchemaManager(getTestingContext(), &security.Credential{}, "http://127.0.0.1:8081", "-value") - require.Nil(t, err) + topic := "cdctest" - err = manager.ClearRegistry(getTestingContext(), table) - require.Nil(t, err) + err = manager.ClearRegistry(getTestingContext(), topic) + require.NoError(t, err) - _, _, err = manager.Lookup(getTestingContext(), table, 1) + _, _, err = manager.Lookup(getTestingContext(), topic, 1) require.Regexp(t, `.*not\sfound.*`, err) codec, err := goavro.NewCodec(`{ @@ -189,15 +190,15 @@ func TestSchemaRegistry(t *testing.T) { } ] }`) - require.Nil(t, err) + require.NoError(t, err) - _, err = manager.Register(getTestingContext(), table, codec) - require.Nil(t, err) + _, err = manager.Register(getTestingContext(), topic, codec) + require.NoError(t, err) var id int for i := 0; i < 2; i++ { - _, id, err = manager.Lookup(getTestingContext(), table, 1) - require.Nil(t, err) + _, id, err = manager.Lookup(getTestingContext(), topic, 1) + require.NoError(t, err) require.Greater(t, id, 0) } @@ -220,41 +221,46 @@ func TestSchemaRegistry(t *testing.T) { } ] }`) - require.Nil(t, err) - _, err = manager.Register(getTestingContext(), table, codec) - require.Nil(t, err) + require.NoError(t, err) + _, err = manager.Register(getTestingContext(), topic, codec) + require.NoError(t, err) - codec2, id2, err := manager.Lookup(getTestingContext(), table, 999) - require.Nil(t, err) + codec2, id2, err := manager.Lookup(getTestingContext(), topic, 999) + require.NoError(t, err) require.NotEqual(t, id, id2) - require.Equal(t, codec2.CanonicalSchema(), codec.CanonicalSchema()) + require.Equal(t, codec.CanonicalSchema(), codec2.CanonicalSchema()) } func TestSchemaRegistryBad(t *testing.T) { - startHTTPInterceptForTestingRegistry(t) + startHTTPInterceptForTestingRegistry() defer stopHTTPInterceptForTestingRegistry() - _, err := NewAvroSchemaManager(getTestingContext(), &security.Credential{}, "http://127.0.0.1:808", "-value") + _, err := NewAvroSchemaManager(getTestingContext(), nil, "http://127.0.0.1:808", "-value") require.NotNil(t, err) - _, err = NewAvroSchemaManager(getTestingContext(), &security.Credential{}, "https://127.0.0.1:8080", "-value") + _, err = NewAvroSchemaManager(getTestingContext(), nil, "https://127.0.0.1:8080", "-value") require.NotNil(t, err) } func TestSchemaRegistryIdempotent(t *testing.T) { - startHTTPInterceptForTestingRegistry(t) + startHTTPInterceptForTestingRegistry() defer stopHTTPInterceptForTestingRegistry() - table := model.TableName{ - Schema: "testdb", - Table: "test1", - } - manager, err := NewAvroSchemaManager(getTestingContext(), &security.Credential{}, "http://127.0.0.1:8081", "-value") - require.Nil(t, err) + manager, err := NewAvroSchemaManager( + getTestingContext(), + nil, + "http://127.0.0.1:8081", + "-value", + ) + require.NoError(t, err) + + topic := "cdctest" + for i := 0; i < 20; i++ { - err = manager.ClearRegistry(getTestingContext(), table) - require.Nil(t, err) + err = manager.ClearRegistry(getTestingContext(), topic) + require.NoError(t, err) } + codec, err := goavro.NewCodec(`{ "type": "record", "name": "test", @@ -274,28 +280,28 @@ func TestSchemaRegistryIdempotent(t *testing.T) { } ] }`) - require.Nil(t, err) + require.NoError(t, err) id := 0 for i := 0; i < 20; i++ { - id1, err := manager.Register(getTestingContext(), table, codec) - require.Nil(t, err) + id1, err := manager.Register(getTestingContext(), topic, codec) + require.NoError(t, err) require.True(t, id == 0 || id == id1) id = id1 } } func TestGetCachedOrRegister(t *testing.T) { - startHTTPInterceptForTestingRegistry(t) + startHTTPInterceptForTestingRegistry() defer stopHTTPInterceptForTestingRegistry() - table := model.TableName{ - Schema: "testdb", - Table: "test1", - } - - manager, err := NewAvroSchemaManager(getTestingContext(), &security.Credential{}, "http://127.0.0.1:8081", "-value") - require.Nil(t, err) + manager, err := NewAvroSchemaManager( + getTestingContext(), + nil, + "http://127.0.0.1:8081", + "-value", + ) + require.NoError(t, err) called := 0 // nolint:unparam @@ -304,7 +310,7 @@ func TestGetCachedOrRegister(t *testing.T) { called++ return `{ "type": "record", - "name": "test", + "name": "test1", "fields": [ { @@ -322,27 +328,28 @@ func TestGetCachedOrRegister(t *testing.T) { ] }`, nil } + topic := "cdctest" - codec, id, err := manager.GetCachedOrRegister(getTestingContext(), table, 1, schemaGen) - require.Nil(t, err) + codec, id, err := manager.GetCachedOrRegister(getTestingContext(), topic, 1, schemaGen) + require.NoError(t, err) require.Greater(t, id, 0) require.NotNil(t, codec) require.Equal(t, 1, called) - codec1, _, err := manager.GetCachedOrRegister(getTestingContext(), table, 1, schemaGen) - require.Nil(t, err) - require.Equal(t, codec, codec1) + codec1, _, err := manager.GetCachedOrRegister(getTestingContext(), topic, 1, schemaGen) + require.NoError(t, err) + require.True(t, codec == codec1) // check identity require.Equal(t, 1, called) - codec2, _, err := manager.GetCachedOrRegister(getTestingContext(), table, 2, schemaGen) - require.Nil(t, err) + codec2, _, err := manager.GetCachedOrRegister(getTestingContext(), topic, 2, schemaGen) + require.NoError(t, err) require.NotEqual(t, codec, codec2) require.Equal(t, 2, called) schemaGen = func() (string, error) { return `{ "type": "record", - "name": "test", + "name": "test1", "fields": [ { @@ -368,8 +375,13 @@ func TestGetCachedOrRegister(t *testing.T) { go func() { defer wg.Done() for j := 0; j < 100; j++ { - codec, id, err := manager.GetCachedOrRegister(getTestingContext(), table, uint64(finalI), schemaGen) - require.Nil(t, err) + codec, id, err := manager.GetCachedOrRegister( + getTestingContext(), + topic, + uint64(finalI), + schemaGen, + ) + require.NoError(t, err) require.Greater(t, id, 0) require.NotNil(t, codec) } @@ -379,7 +391,7 @@ func TestGetCachedOrRegister(t *testing.T) { } func TestHTTPRetry(t *testing.T) { - startHTTPInterceptForTestingRegistry(t) + startHTTPInterceptForTestingRegistry() defer stopHTTPInterceptForTestingRegistry() ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) @@ -388,9 +400,10 @@ func TestHTTPRetry(t *testing.T) { payload := []byte("test") req, err := http.NewRequestWithContext(ctx, "POST", "http://127.0.0.1:8081/may-fail", bytes.NewReader(payload)) - require.Nil(t, err) + require.NoError(t, err) - resp, err := httpRetry(ctx, nil, req, false) - require.Nil(t, err) + resp, err := httpRetry(ctx, nil, req) + require.NoError(t, err) + require.Equal(t, 200, resp.StatusCode) _ = resp.Body.Close() } diff --git a/cdc/sink/mq/dispatcher/event_router.go b/cdc/sink/mq/dispatcher/event_router.go index 5be2e9e4b36..6e25697ee1d 100644 --- a/cdc/sink/mq/dispatcher/event_router.go +++ b/cdc/sink/mq/dispatcher/event_router.go @@ -102,7 +102,7 @@ func NewEventRouter(cfg *config.ReplicaConfig, defaultTopic string) (*EventRoute } d := getPartitionDispatcher(ruleConfig, cfg.EnableOldValue) - t, err := getTopicDispatcher(ruleConfig, defaultTopic) + t, err := getTopicDispatcher(ruleConfig, defaultTopic, cfg.Sink.Protocol) if err != nil { return nil, err } @@ -112,6 +112,7 @@ func NewEventRouter(cfg *config.ReplicaConfig, defaultTopic string) (*EventRoute filter.Filter }{partitionDispatcher: d, topicDispatcher: t, Filter: f}) } + return &EventRouter{ defaultTopic: defaultTopic, rules: rules, @@ -249,7 +250,7 @@ func getPartitionDispatcher( // getTopicDispatcher returns the topic dispatcher for a specific topic rule (aka topic expression). func getTopicDispatcher( - ruleConfig *config.DispatchRule, defaultTopic string, + ruleConfig *config.DispatchRule, defaultTopic string, protocol string, ) (topic.Dispatcher, error) { if ruleConfig.TopicRule == "" { return topic.NewStaticTopicDispatcher(defaultTopic), nil @@ -257,9 +258,24 @@ func getTopicDispatcher( // check if this rule is a valid topic expression topicExpr := topic.Expression(ruleConfig.TopicRule) - err := topicExpr.Validate() - if err != nil { - return nil, err + + if protocol != "" { + var p config.Protocol + if err := p.FromString(protocol); err != nil { + return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) + } + + if p == config.ProtocolAvro { + err := topicExpr.ValidateForAvro() + if err != nil { + return nil, err + } + } else { + err := topicExpr.Validate() + if err != nil { + return nil, err + } + } } return topic.NewDynamicTopicDispatcher(topicExpr), nil } diff --git a/cdc/sink/mq/dispatcher/topic/expression.go b/cdc/sink/mq/dispatcher/topic/expression.go index bf91227886a..a6dde858a8d 100644 --- a/cdc/sink/mq/dispatcher/topic/expression.go +++ b/cdc/sink/mq/dispatcher/topic/expression.go @@ -30,6 +30,11 @@ var ( schemaRE = regexp.MustCompile(`\{schema\}`) // tableRE is used to match substring '{table}' in topic expression tableRE = regexp.MustCompile(`\{table\}`) + // avro has different topic name pattern requirements, '{schema}' and '{table}' placeholders + // are necessary + avroTopicNameRE = regexp.MustCompile( + `^[A-Za-z0-9\._\-]*\{schema\}[A-Za-z0-9\._\-]*\{table\}[A-Za-z0-9\._\-]*$`, + ) ) // The max length of kafka topic name is 249. @@ -52,6 +57,17 @@ func (e Expression) Validate() error { return nil } +// ValidateForAvro checks whether topic pattern is {schema}_{table}, the only allowed +func (e Expression) ValidateForAvro() error { + if ok := avroTopicNameRE.MatchString(string(e)); !ok { + return errors.ErrKafkaInvalidTopicExpression.GenWithStackByArgs( + "topic rule for Avro must contain {schema} and {table}", + ) + } + + return nil +} + // Substitute converts schema/table name in a topic expression to kafka topic name. // When doing conversion, the special characters other than [A-Za-z0-9\._\-] in schema/table // will be substituted for underscore '_'. diff --git a/cdc/sink/mq/mq.go b/cdc/sink/mq/mq.go index 7e42dab6325..07b70952a55 100644 --- a/cdc/sink/mq/mq.go +++ b/cdc/sink/mq/mq.go @@ -36,7 +36,6 @@ import ( "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/filter" - "github.com/pingcap/tiflow/pkg/security" "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" "golang.org/x/sync/errgroup" @@ -73,7 +72,6 @@ type mqSink struct { func newMqSink( ctx context.Context, - credential *security.Credential, topicManager manager.TopicManager, mqProducer producer.Producer, filter *filter.Filter, @@ -81,7 +79,7 @@ func newMqSink( replicaConfig *config.ReplicaConfig, encoderConfig *codec.Config, errCh chan error, ) (*mqSink, error) { - encoderBuilder, err := codec.NewEventBatchEncoderBuilder(encoderConfig, credential) + encoderBuilder, err := codec.NewEventBatchEncoderBuilder(ctx, encoderConfig) if err != nil { return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) } @@ -414,8 +412,8 @@ func NewKafkaSaramaSink(ctx context.Context, sinkURI *url.URL, return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) } - encoderConfig := codec.NewConfig(protocol, contextutil.TimezoneFromCtx(ctx)) - if err := encoderConfig.Apply(sinkURI, opts); err != nil { + encoderConfig := codec.NewConfig(protocol) + if err := encoderConfig.Apply(sinkURI, replicaConfig); err != nil { return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) } // always set encoder's `MaxMessageBytes` equal to producer's `MaxMessageBytes` @@ -454,7 +452,6 @@ func NewKafkaSaramaSink(ctx context.Context, sinkURI *url.URL, sink, err := newMqSink( ctx, - baseConfig.Credential, topicManager, sProducer, filter, @@ -487,8 +484,8 @@ func NewPulsarSink(ctx context.Context, sinkURI *url.URL, filter *filter.Filter, return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) } - encoderConfig := codec.NewConfig(protocol, contextutil.TimezoneFromCtx(ctx)) - if err := encoderConfig.Apply(sinkURI, opts); err != nil { + encoderConfig := codec.NewConfig(protocol) + if err := encoderConfig.Apply(sinkURI, replicaConfig); err != nil { return nil, errors.Trace(err) } // todo: set by pulsar producer's `max.message.bytes` @@ -501,15 +498,11 @@ func NewPulsarSink(ctx context.Context, sinkURI *url.URL, filter *filter.Filter, if err != nil { return nil, errors.Trace(err) } - // For now, it's a placeholder. Avro format have to make connection to Schema Registry, - // and it may need credential. - credential := &security.Credential{} fakeTopicManager := pulsarmanager.NewTopicManager( producer.GetPartitionNum(), ) sink, err := newMqSink( ctx, - credential, fakeTopicManager, producer, filter, diff --git a/cdc/sink/mq/mq_flush_worker.go b/cdc/sink/mq/mq_flush_worker.go index bf96cd5ab01..dbe3e9cfa75 100644 --- a/cdc/sink/mq/mq_flush_worker.go +++ b/cdc/sink/mq/mq_flush_worker.go @@ -154,7 +154,7 @@ func (w *flushWorker) asyncSend( ) error { for key, events := range partitionedRows { for _, event := range events { - err := w.encoder.AppendRowChangedEvent(event) + err := w.encoder.AppendRowChangedEvent(ctx, key.topic, event) if err != nil { return err } diff --git a/cdc/sink/mq/mq_flush_worker_test.go b/cdc/sink/mq/mq_flush_worker_test.go index a2a410beb76..70b1f320fec 100644 --- a/cdc/sink/mq/mq_flush_worker_test.go +++ b/cdc/sink/mq/mq_flush_worker_test.go @@ -19,12 +19,10 @@ import ( "testing" "github.com/pingcap/errors" - "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/codec" "github.com/pingcap/tiflow/cdc/sink/metrics" "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/security" "github.com/stretchr/testify/require" ) @@ -83,9 +81,8 @@ func NewMockProducer() *mockProducer { func newTestWorker() (*flushWorker, *mockProducer) { // 200 is about the size of a row change. - encoderConfig := codec.NewConfig(config.ProtocolOpen, timeutil.SystemLocation()). - WithMaxMessageBytes(200) - builder, err := codec.NewEventBatchEncoderBuilder(encoderConfig, &security.Credential{}) + encoderConfig := codec.NewConfig(config.ProtocolOpen).WithMaxMessageBytes(200) + builder, err := codec.NewEventBatchEncoderBuilder(context.Background(), encoderConfig) if err != nil { panic(err) } diff --git a/cdc/sink/mq/producer/kafka/config_test.go b/cdc/sink/mq/producer/kafka/config_test.go index 7b75079c614..a99084396c1 100644 --- a/cdc/sink/mq/producer/kafka/config_test.go +++ b/cdc/sink/mq/producer/kafka/config_test.go @@ -23,7 +23,6 @@ import ( "github.com/Shopify/sarama" "github.com/pingcap/errors" - "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/sink/codec" "github.com/pingcap/tiflow/pkg/config" @@ -403,8 +402,8 @@ func TestConfigurationCombinations(t *testing.T) { err = AdjustConfig(adminClient, baseConfig, saramaConfig, topic) require.Nil(t, err) - encoderConfig := codec.NewConfig(config.ProtocolOpen, timeutil.SystemLocation()) - err = encoderConfig.Apply(sinkURI, map[string]string{}) + encoderConfig := codec.NewConfig(config.ProtocolOpen) + err = encoderConfig.Apply(sinkURI, &config.ReplicaConfig{}) require.Nil(t, err) encoderConfig.WithMaxMessageBytes(saramaConfig.Producer.MaxMessageBytes) diff --git a/go.mod b/go.mod index 97a88836eff..9268b6ccac7 100644 --- a/go.mod +++ b/go.mod @@ -61,7 +61,7 @@ require ( github.com/soheilhy/cmux v0.1.5 github.com/spf13/cobra v1.4.0 github.com/spf13/pflag v1.0.5 - github.com/stretchr/testify v1.7.0 + github.com/stretchr/testify v1.7.1 github.com/swaggo/files v0.0.0-20190704085106-630677cd5c14 github.com/swaggo/gin-swagger v1.2.0 github.com/swaggo/swag v1.6.6-0.20200529100950-7c765ddd0476 diff --git a/go.sum b/go.sum index aa101316c4c..a53747e8b2a 100644 --- a/go.sum +++ b/go.sum @@ -1146,8 +1146,9 @@ github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81P github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.6.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.7.1 h1:5TQK59W5E3v0r2duFAb7P95B6hEeOyEnHRa8MjYSMTY= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= github.com/swaggo/files v0.0.0-20190704085106-630677cd5c14 h1:PyYN9JH5jY9j6av01SpfRMb+1DWg/i3MbGOKPxJ2wjM= github.com/swaggo/files v0.0.0-20190704085106-630677cd5c14/go.mod h1:gxQT6pBGRuIGunNf/+tSOB5OHvguWi8Tbt82WOkf35E= diff --git a/pkg/cmd/cli/cli_changefeed_create.go b/pkg/cmd/cli/cli_changefeed_create.go index 7b115513975..f2ca9042d40 100644 --- a/pkg/cmd/cli/cli_changefeed_create.go +++ b/pkg/cmd/cli/cli_changefeed_create.go @@ -113,6 +113,7 @@ type createChangefeedOptions struct { disableGCSafePointCheck bool startTs uint64 timezone string + schemaRegistry string cfg *config.ReplicaConfig } @@ -136,6 +137,8 @@ func (o *createChangefeedOptions) addFlags(cmd *cobra.Command) { cmd.PersistentFlags().BoolVarP(&o.disableGCSafePointCheck, "disable-gc-check", "", false, "Disable GC safe point check") cmd.PersistentFlags().Uint64Var(&o.startTs, "start-ts", 0, "Start ts of changefeed") cmd.PersistentFlags().StringVar(&o.timezone, "tz", "SYSTEM", "timezone used when checking sink uri (changefeed timezone is determined by cdc server)") + cmd.PersistentFlags(). + StringVar(&o.schemaRegistry, "schema-registry", "", "Avro Schema Registry URI") } // complete adapts from the command line args to the data and client required. @@ -268,6 +271,7 @@ func (o *createChangefeedOptions) completeCfg( // TODO(neil) enable ID bucket. } } + cfg.SchemaRegistry = o.schemaRegistry // Complete cfg. o.cfg = cfg diff --git a/pkg/config/config_test_data.go b/pkg/config/config_test_data.go index 74e3ae6c75e..cc085470bfc 100644 --- a/pkg/config/config_test_data.go +++ b/pkg/config/config_test_data.go @@ -181,7 +181,8 @@ const ( "max-log-size": 64, "flush-interval": 1000, "storage": "" - } + }, + "schema-registry": "" }` testCfgTestReplicaConfigMarshal2 = `{ diff --git a/pkg/config/mq_sink_protocol.go b/pkg/config/mq_sink_protocol.go index bac5ca08822..83e420a633d 100644 --- a/pkg/config/mq_sink_protocol.go +++ b/pkg/config/mq_sink_protocol.go @@ -47,6 +47,8 @@ func (p *Protocol) FromString(protocol string) error { *p = ProtocolCanal case "avro": *p = ProtocolAvro + case "flat-avro": + *p = ProtocolAvro case "maxwell": *p = ProtocolMaxwell case "canal-json": diff --git a/pkg/config/mq_sink_protocol_test.go b/pkg/config/mq_sink_protocol_test.go index f58d4d7959a..6522f9ee1f5 100644 --- a/pkg/config/mq_sink_protocol_test.go +++ b/pkg/config/mq_sink_protocol_test.go @@ -50,6 +50,10 @@ func TestFromString(t *testing.T) { protocol: "avro", expectedProtocolEnum: ProtocolAvro, }, + { + protocol: "flat-avro", + expectedProtocolEnum: ProtocolAvro, + }, { protocol: "craft", expectedProtocolEnum: ProtocolCraft, diff --git a/pkg/config/replica_config.go b/pkg/config/replica_config.go index 2952f216b8a..a9b87457213 100644 --- a/pkg/config/replica_config.go +++ b/pkg/config/replica_config.go @@ -46,6 +46,7 @@ var defaultReplicaConfig = &ReplicaConfig{ FlushIntervalInMs: 1000, Storage: "", }, + SchemaRegistry: "", } // ReplicaConfig represents some addition replication config for a changefeed @@ -61,6 +62,7 @@ type replicaConfig struct { Sink *SinkConfig `toml:"sink" json:"sink"` Cyclic *CyclicConfig `toml:"cyclic-replication" json:"cyclic-replication"` Consistent *ConsistentConfig `toml:"consistent" json:"consistent"` + SchemaRegistry string `toml:"schema-registry" json:"schema-registry"` } // Marshal returns the json marshal format of a ReplicationConfig diff --git a/pkg/config/replica_config_test.go b/pkg/config/replica_config_test.go index 817877379f9..84f08bb1714 100644 --- a/pkg/config/replica_config_test.go +++ b/pkg/config/replica_config_test.go @@ -21,7 +21,7 @@ import ( "github.com/stretchr/testify/require" ) -func mustIdentJSON(t *testing.T, j string) string { +func mustIndentJSON(t *testing.T, j string) string { var buf bytes.Buffer err := json.Indent(&buf, []byte(j), "", " ") require.Nil(t, err) @@ -44,7 +44,7 @@ func TestReplicaConfigMarshal(t *testing.T) { } b, err := conf.Marshal() require.Nil(t, err) - require.Equal(t, testCfgTestReplicaConfigMarshal1, mustIdentJSON(t, b)) + require.Equal(t, testCfgTestReplicaConfigMarshal1, mustIndentJSON(t, b)) conf2 := new(ReplicaConfig) err = conf2.Unmarshal([]byte(testCfgTestReplicaConfigMarshal2)) require.Nil(t, err) diff --git a/pkg/config/server_config_test.go b/pkg/config/server_config_test.go index 26dd8970ee7..d6a45f29a63 100644 --- a/pkg/config/server_config_test.go +++ b/pkg/config/server_config_test.go @@ -30,7 +30,7 @@ func TestServerConfigMarshal(t *testing.T) { b, err := conf.Marshal() require.Nil(t, err) - require.Equal(t, rawConfig, mustIdentJSON(t, b)) + require.Equal(t, rawConfig, mustIndentJSON(t, b)) conf2 := new(ServerConfig) err = conf2.Unmarshal([]byte(rawConfig)) require.Nil(t, err) diff --git a/tests/mq_protocol_tests/framework/avro/kafka_docker_env_test.go b/tests/mq_protocol_tests/framework/avro/kafka_docker_env_test.go index f0396e7065e..7fb117b92ef 100644 --- a/tests/mq_protocol_tests/framework/avro/kafka_docker_env_test.go +++ b/tests/mq_protocol_tests/framework/avro/kafka_docker_env_test.go @@ -52,9 +52,10 @@ func (t *dummyTask) Prepare(taskContext *framework.TaskContext) error { func (t *dummyTask) GetCDCProfile() *framework.CDCProfile { return &framework.CDCProfile{ - PDUri: framework.UpstreamPD, - SinkURI: "kafka://kafka:9092/testdb_test?protocol=avro", - Opts: map[string]string{"registry": "http://schema-registry:8081"}, + PDUri: framework.UpstreamPD, + SinkURI: "kafka://kafka:9092/testdb_test?protocol=avro" + + "&avro-decimal-handling-mode=string&avro-bigint-unsigned-handling-mode=string", + SchemaRegistry: "http://schema-registry:8081", } } diff --git a/tests/mq_protocol_tests/framework/avro/kafka_single_table.go b/tests/mq_protocol_tests/framework/avro/kafka_single_table.go index 5ab558ab62b..ce92bb73a39 100644 --- a/tests/mq_protocol_tests/framework/avro/kafka_single_table.go +++ b/tests/mq_protocol_tests/framework/avro/kafka_single_table.go @@ -41,9 +41,10 @@ func (a *SingleTableTask) Name() string { // GetCDCProfile implements Task func (a *SingleTableTask) GetCDCProfile() *framework.CDCProfile { return &framework.CDCProfile{ - PDUri: framework.UpstreamPD, - SinkURI: "kafka://kafka:9092/testdb_" + a.TableName + "?kafka-version=2.6.0&protocol=avro", - Opts: map[string]string{"registry": "http://schema-registry:8081"}, + PDUri: framework.UpstreamPD, + SinkURI: "kafka://kafka:9092/testdb_" + a.TableName + "?kafka-version=2.6.0" + + "&protocol=avro&avro-decimal-handling-mode=string&avro-bigint-unsigned-handling-mode=string", + SchemaRegistry: "http://schema-registry:8081", } } diff --git a/tests/mq_protocol_tests/framework/task.go b/tests/mq_protocol_tests/framework/task.go index c6ece257fa1..adfc3b228fe 100644 --- a/tests/mq_protocol_tests/framework/task.go +++ b/tests/mq_protocol_tests/framework/task.go @@ -44,10 +44,11 @@ type TaskContext struct { // CDCProfile represents the command line arguments used to create the changefeed type CDCProfile struct { - PDUri string - SinkURI string - ConfigFile string - Opts map[string]string + PDUri string + SinkURI string + ConfigFile string + SchemaRegistry string + Opts map[string]string } // CreateDB creates a database in both the upstream and the downstream @@ -99,6 +100,10 @@ func (p *CDCProfile) String() string { builder.WriteString(fmt.Sprintf("--config=%s ", strconv.Quote(p.ConfigFile))) } + if p.SchemaRegistry != "" { + builder.WriteString(fmt.Sprintf("--schema-registry=%s ", strconv.Quote(p.SchemaRegistry))) + } + if p.Opts == nil || len(p.Opts) == 0 { return builder.String() } diff --git a/tests/mq_protocol_tests/main.go b/tests/mq_protocol_tests/main.go index fcadc211965..de2f19e2569 100644 --- a/tests/mq_protocol_tests/main.go +++ b/tests/mq_protocol_tests/main.go @@ -33,7 +33,6 @@ var ( func testAvro() { env := avro.NewKafkaDockerEnv(*dockerComposeFile) - env.DockerComposeOperator.ExecEnv = []string{"CDC_TIME_ZONE=America/Los_Angeles"} task := &avro.SingleTableTask{TableName: "test"} testCases := []framework.Task{ cases.NewAlterCase(task), // this case is slow, so put it last