From 28ed15f9e2c906667a22ab300884b96707520e7c Mon Sep 17 00:00:00 2001 From: Fan Yang Date: Sun, 29 Sep 2024 19:09:11 +0800 Subject: [PATCH] feat: add an Arrow-based, columnar binlog buffer (#91) --- .github/workflows/go.yml | 2 +- backend/connpool.go | 19 +- binlog/README.md | 1 + binlog/constants.go | 152 +++ binlog/events.go | 37 + binlog/rbr.go | 981 ++++++++++++++++++ binlogreplication/binlog_file_utils.go | 2 +- binlogreplication/binlog_position_store.go | 31 +- binlogreplication/binlog_replica_applier.go | 406 +++++--- .../binlog_replica_controller.go | 11 +- .../binlog_replica_event_producer.go | 4 +- binlogreplication/binlog_replica_filtering.go | 2 +- binlogreplication/binlog_replication_test.go | 4 +- binlogreplication/writer.go | 39 +- catalog/index.go | 7 + catalog/provider.go | 30 +- charset/charset.go | 73 +- go.mod | 64 +- go.sum | 133 ++- harness/duck_harness.go | 2 +- main.go | 5 +- myarrow/appender.go | 100 ++ myarrow/schema.go | 129 +++ replica/appender.go | 86 ++ replica/controller.go | 245 +++++ replica/delta.go | 101 ++ replica/replication.go | 86 ++ replication.go => replica/updater.go | 320 +++--- 28 files changed, 2579 insertions(+), 493 deletions(-) create mode 100644 binlog/README.md create mode 100644 binlog/constants.go create mode 100644 binlog/events.go create mode 100644 binlog/rbr.go create mode 100644 myarrow/appender.go create mode 100644 myarrow/schema.go create mode 100644 replica/appender.go create mode 100644 replica/controller.go create mode 100644 replica/delta.go create mode 100644 replica/replication.go rename replication.go => replica/updater.go (52%) diff --git a/.github/workflows/go.yml b/.github/workflows/go.yml index 3aa0f13e..18333dde 100644 --- a/.github/workflows/go.yml +++ b/.github/workflows/go.yml @@ -19,7 +19,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v5 with: - go-version: '1.23' + go-version: '1.22' - name: Set up Python uses: actions/setup-python@v5 diff --git a/backend/connpool.go b/backend/connpool.go index 10f95195..0d9c312f 100644 --- a/backend/connpool.go +++ b/backend/connpool.go @@ -21,23 +21,30 @@ import ( "github.com/apecloud/myduckserver/catalog" "github.com/dolthub/go-mysql-server/sql" + "github.com/marcboeker/go-duckdb" "github.com/sirupsen/logrus" ) type ConnectionPool struct { *stdsql.DB - catalog string - conns sync.Map // concurrent-safe map[uint32]*stdsql.Conn - txns sync.Map // concurrent-safe map[uint32]*stdsql.Tx + connector *duckdb.Connector + catalog string + conns sync.Map // concurrent-safe map[uint32]*stdsql.Conn + txns sync.Map // concurrent-safe map[uint32]*stdsql.Tx } -func NewConnectionPool(catalog string, db *stdsql.DB) *ConnectionPool { +func NewConnectionPool(catalog string, connector *duckdb.Connector, db *stdsql.DB) *ConnectionPool { return &ConnectionPool{ - DB: db, - catalog: catalog, + DB: db, + connector: connector, + catalog: catalog, } } +func (p *ConnectionPool) Connector() *duckdb.Connector { + return p.connector +} + func (p *ConnectionPool) GetConn(ctx context.Context, id uint32) (*stdsql.Conn, error) { var conn *stdsql.Conn entry, ok := p.conns.Load(id) diff --git a/binlog/README.md b/binlog/README.md new file mode 100644 index 00000000..d9470e14 --- /dev/null +++ b/binlog/README.md @@ -0,0 +1 @@ +The code in this directory was copied and modified from [the Vitess project](https://github.com/vitessio/vitess) (as of 2024-09-19, https://github.com/vitessio/vitess/blob/main/go/mysql/binlog/). The original code is licensed under the Apache License, Version 2.0. The modifications are also licensed under the Apache License, Version 2.0. The goal is to bypass unnecessary string conversion and memory allocation in the original code. \ No newline at end of file diff --git a/binlog/constants.go b/binlog/constants.go new file mode 100644 index 00000000..e68e5e29 --- /dev/null +++ b/binlog/constants.go @@ -0,0 +1,152 @@ +/* +Copyright 2023 The Vitess Authors. + +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, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package binlog + +// This is the data type for a field. +// Values taken from include/mysql/mysql_com.h +const ( + // TypeDecimal is MYSQL_TYPE_DECIMAL. It is deprecated. + TypeDecimal = 0 + + // TypeTiny is MYSQL_TYPE_TINY + TypeTiny = 1 + + // TypeShort is MYSQL_TYPE_SHORT + TypeShort = 2 + + // TypeLong is MYSQL_TYPE_LONG + TypeLong = 3 + + // TypeFloat is MYSQL_TYPE_FLOAT + TypeFloat = 4 + + // TypeDouble is MYSQL_TYPE_DOUBLE + TypeDouble = 5 + + // TypeNull is MYSQL_TYPE_NULL + TypeNull = 6 + + // TypeTimestamp is MYSQL_TYPE_TIMESTAMP + TypeTimestamp = 7 + + // TypeLongLong is MYSQL_TYPE_LONGLONG + TypeLongLong = 8 + + // TypeInt24 is MYSQL_TYPE_INT24 + TypeInt24 = 9 + + // TypeDate is MYSQL_TYPE_DATE + TypeDate = 10 + + // TypeTime is MYSQL_TYPE_TIME + TypeTime = 11 + + // TypeDateTime is MYSQL_TYPE_DATETIME + TypeDateTime = 12 + + // TypeYear is MYSQL_TYPE_YEAR + TypeYear = 13 + + // TypeNewDate is MYSQL_TYPE_NEWDATE + TypeNewDate = 14 + + // TypeVarchar is MYSQL_TYPE_VARCHAR + TypeVarchar = 15 + + // TypeBit is MYSQL_TYPE_BIT + TypeBit = 16 + + // TypeTimestamp2 is MYSQL_TYPE_TIMESTAMP2 + TypeTimestamp2 = 17 + + // TypeDateTime2 is MYSQL_TYPE_DATETIME2 + TypeDateTime2 = 18 + + // TypeTime2 is MYSQL_TYPE_TIME2 + TypeTime2 = 19 + + // TypeVector is MYSQL_TYPE_VECTOR + TypeVector = 242 + + // TypeJSON is MYSQL_TYPE_JSON + TypeJSON = 245 + + // TypeNewDecimal is MYSQL_TYPE_NEWDECIMAL + TypeNewDecimal = 246 + + // TypeEnum is MYSQL_TYPE_ENUM + TypeEnum = 247 + + // TypeSet is MYSQL_TYPE_SET + TypeSet = 248 + + // TypeTinyBlob is MYSQL_TYPE_TINY_BLOB + TypeTinyBlob = 249 + + // TypeMediumBlob is MYSQL_TYPE_MEDIUM_BLOB + TypeMediumBlob = 250 + + // TypeLongBlob is MYSQL_TYPE_LONG_BLOB + TypeLongBlob = 251 + + // TypeBlob is MYSQL_TYPE_BLOB + TypeBlob = 252 + + // TypeVarString is MYSQL_TYPE_VAR_STRING + TypeVarString = 253 + + // TypeString is MYSQL_TYPE_STRING + TypeString = 254 + + // TypeGeometry is MYSQL_TYPE_GEOMETRY + TypeGeometry = 255 +) + +var TypeNames = map[byte]string{ + TypeDecimal: "DECIMAL", + TypeTiny: "TINY", + TypeShort: "SHORT", + TypeLong: "LONG", + TypeFloat: "FLOAT", + TypeDouble: "DOUBLE", + TypeNull: "NULL", + TypeTimestamp: "TIMESTAMP", + TypeLongLong: "LONGLONG", + TypeInt24: "INT24", + TypeDate: "DATE", + TypeTime: "TIME", + TypeDateTime: "DATETIME", + TypeYear: "YEAR", + TypeNewDate: "NEWDATE", + TypeVarchar: "VARCHAR", + TypeBit: "BIT", + TypeTimestamp2: "TIMESTAMP2", + TypeDateTime2: "DATETIME2", + TypeTime2: "TIME2", + TypeVector: "VECTOR", + TypeJSON: "JSON", + TypeNewDecimal: "NEWDECIMAL", + TypeEnum: "ENUM", + TypeSet: "SET", + TypeTinyBlob: "TINY_BLOB", + TypeMediumBlob: "MEDIUM_BLOB", + TypeLongBlob: "LONG_BLOB", + TypeBlob: "BLOB", + TypeVarString: "VAR_STRING", + TypeString: "STRING", + TypeGeometry: "GEOMETRY", +} diff --git a/binlog/events.go b/binlog/events.go new file mode 100644 index 00000000..43887c54 --- /dev/null +++ b/binlog/events.go @@ -0,0 +1,37 @@ +// Copyright 2024-2025 ApeCloud, Ltd. +// +// 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package binlog + +type RowEventType int8 + +const ( + // IMPORTANT: The order of these values is important. + // We translate UPDATE to DELETE + INSERT, so DELETE should come first. + DeleteRowEvent RowEventType = iota + UpdateRowEvent + InsertRowEvent +) + +func (e RowEventType) String() string { + switch e { + case DeleteRowEvent: + return "DELETE" + case UpdateRowEvent: + return "UPDATE" + case InsertRowEvent: + return "INSERT" + default: + return "UNKNOWN" + } +} diff --git a/binlog/rbr.go b/binlog/rbr.go new file mode 100644 index 00000000..8cf45eaf --- /dev/null +++ b/binlog/rbr.go @@ -0,0 +1,981 @@ +/* +Copyright 2023 The Vitess Authors. + +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, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package binlog + +import ( + "encoding/binary" + "math" + "math/big" + "time" + + "github.com/apache/arrow/go/v17/arrow" + "github.com/apache/arrow/go/v17/arrow/array" + "github.com/apache/arrow/go/v17/arrow/decimal128" + "github.com/apache/arrow/go/v17/arrow/decimal256" + "github.com/apecloud/myduckserver/charset" + "github.com/cockroachdb/apd/v3" + "github.com/dolthub/go-mysql-server/sql" + vtbinlog "vitess.io/vitess/go/mysql/binlog" + "vitess.io/vitess/go/sqltypes" + querypb "vitess.io/vitess/go/vt/proto/query" + "vitess.io/vitess/go/vt/proto/vtrpc" + "vitess.io/vitess/go/vt/vterrors" +) + +// ZeroTimestamp is the special value 0 for a timestamp. +var ZeroTimestamp = []byte("0000-00-00 00:00:00") + +var dig2bytes = []int{0, 1, 1, 2, 2, 3, 3, 4, 4, 4} + +var powerOf10s = [20]uint64{ + 1, + 10, + 100, + 1000, + 10000, + 1_00000, + 10_00000, + 100_00000, + 1000_00000, + 10000_00000, + 1_00000_00000, + 10_00000_00000, + 100_00000_00000, + 1000_00000_00000, + 10000_00000_00000, + 1_00000_00000_00000, + 10_00000_00000_00000, + 100_00000_00000_00000, + 1000_00000_00000_00000, + 10000_00000_00000_00000, +} + +// CellLength returns the new position after the field with the given +// type is read. +func CellLength(data []byte, pos int, typ byte, metadata uint16) (int, error) { + switch typ { + case TypeNull: + return 0, nil + case TypeTiny, TypeYear: + return 1, nil + case TypeShort: + return 2, nil + case TypeInt24: + return 3, nil + case TypeLong, TypeFloat, TypeTimestamp: + return 4, nil + case TypeLongLong, TypeDouble: + return 8, nil + case TypeDate, TypeTime, TypeNewDate: + return 3, nil + case TypeDateTime: + return 8, nil + case TypeVarchar, TypeVarString: + // Length is encoded in 1 or 2 bytes. + if metadata > 255 { + l := int(uint64(data[pos]) | + uint64(data[pos+1])<<8) + return l + 2, nil + } + l := int(data[pos]) + return l + 1, nil + case TypeBit: + // bitmap length is in metadata, as: + // upper 8 bits: bytes length + // lower 8 bits: bit length + nbits := ((metadata >> 8) * 8) + (metadata & 0xFF) + return (int(nbits) + 7) / 8, nil + case TypeTimestamp2: + // metadata has number of decimals. One byte encodes + // two decimals. + return 4 + (int(metadata)+1)/2, nil + case TypeDateTime2: + // metadata has number of decimals. One byte encodes + // two decimals. + return 5 + (int(metadata)+1)/2, nil + case TypeTime2: + // metadata has number of decimals. One byte encodes + // two decimals. + return 3 + (int(metadata)+1)/2, nil + case TypeNewDecimal: + precision := int(metadata >> 8) + scale := int(metadata & 0xff) + // Example: + // NNNNNNNNNNNN.MMMMMM + // 12 bytes 6 bytes + // precision is 18 + // scale is 6 + // storage is done by groups of 9 digits: + // - 32 bits are used to store groups of 9 digits. + // - any leftover digit is stored in: + // - 1 byte for 1 and 2 digits + // - 2 bytes for 3 and 4 digits + // - 3 bytes for 5 and 6 digits + // - 4 bytes for 7 and 8 digits (would also work for 9) + // both sides of the dot are stored separately. + // In this example, we'd have: + // - 2 bytes to store the first 3 full digits. + // - 4 bytes to store the next 9 full digits. + // - 3 bytes to store the 6 fractional digits. + intg := precision - scale + intg0 := intg / 9 + frac0 := scale / 9 + intg0x := intg - intg0*9 + frac0x := scale - frac0*9 + return intg0*4 + dig2bytes[intg0x] + frac0*4 + dig2bytes[frac0x], nil + case TypeEnum, TypeSet: + return int(metadata & 0xff), nil + case TypeJSON, TypeTinyBlob, TypeMediumBlob, TypeLongBlob, TypeBlob, TypeGeometry, TypeVector: + // Of the Blobs, only TypeBlob is used in binary logs, + // but supports others just in case. + switch metadata { + case 1: + return 1 + int(uint32(data[pos])), nil + case 2: + return 2 + int(uint32(data[pos])| + uint32(data[pos+1])<<8), nil + case 3: + return 3 + int(uint32(data[pos])| + uint32(data[pos+1])<<8| + uint32(data[pos+2])<<16), nil + case 4: + return 4 + int(uint32(data[pos])| + uint32(data[pos+1])<<8| + uint32(data[pos+2])<<16| + uint32(data[pos+3])<<24), nil + default: + return 0, vterrors.Errorf(vtrpc.Code_INTERNAL, "unsupported blob/geometry metadata value %v (data: %v pos: %v)", metadata, data, pos) + } + case TypeString: + // This may do String, Enum, and Set. The type is in + // metadata. If it's a string, then there will be more bits. + // This will give us the maximum length of the field. + t := metadata >> 8 + if t == TypeEnum || t == TypeSet { + return int(metadata & 0xff), nil + } + max := int((((metadata >> 4) & 0x300) ^ 0x300) + (metadata & 0xff)) + // Length is encoded in 1 or 2 bytes. + if max > 255 { + l := int(uint64(data[pos]) | + uint64(data[pos+1])<<8) + return l + 2, nil + } + l := int(data[pos]) + return l + 1, nil + + default: + return 0, vterrors.Errorf(vtrpc.Code_INTERNAL, "unsupported type %v (data: %v pos: %v)", typ, data, pos) + } +} + +// CellValue returns the data for a cell as a sqltypes.Value, and how +// many bytes it takes. It uses source type in querypb.Type and vitess type +// byte to determine general shared aspects of types and the querypb.Field to +// determine other info specifically about its underlying column (SQL column +// type, column length, charset, etc) +func CellValue(data []byte, pos int, typ byte, metadata uint16, column *sql.Column, builder array.Builder) (int, error) { + // logrus.Infof("CellValue: binlog type: %s, column: %v, type: %v, builder: %T", TypeNames[typ], column.Name, column.Type, builder) + ftype := querypb.Type(column.Type.Type()) + switch typ { + case TypeTiny: + if sqltypes.IsSigned(ftype) { + builder.(*array.Int8Builder).Append(int8(data[pos])) + } else { + builder.(*array.Uint8Builder).Append(data[pos]) + } + return 1, nil + case TypeYear: + val := data[pos] + if val == 0 { + builder.(*array.Uint16Builder).Append(0) + } else { + builder.(*array.Uint16Builder).Append(uint16(data[pos]) + 1900) + } + return 1, nil + case TypeShort: + val := binary.LittleEndian.Uint16(data[pos : pos+2]) + if sqltypes.IsSigned(ftype) { + builder.(*array.Int16Builder).Append(int16(val)) + } else { + builder.(*array.Uint16Builder).Append(val) + } + return 2, nil + case TypeInt24: + if sqltypes.IsSigned(ftype) && data[pos+2]&128 > 0 { + // Negative number, have to extend the sign. + val := int32(uint32(data[pos]) + + uint32(data[pos+1])<<8 + + uint32(data[pos+2])<<16 + + uint32(255)<<24) + builder.(*array.Int32Builder).Append(val) + } else { + // Positive number. + val := uint64(data[pos]) + + uint64(data[pos+1])<<8 + + uint64(data[pos+2])<<16 + switch builder := builder.(type) { + case *array.Int32Builder: + builder.Append(int32(val)) + case *array.Uint32Builder: + builder.Append(uint32(val)) + default: + return 0, vterrors.Errorf(vtrpc.Code_INTERNAL, "unexpected Arrow builder type %T", builder) + } + } + return 3, nil + case TypeLong: + val := binary.LittleEndian.Uint32(data[pos : pos+4]) + if sqltypes.IsSigned(ftype) { + builder.(*array.Int32Builder).Append(int32(val)) + } else { + builder.(*array.Uint32Builder).Append(val) + } + return 4, nil + case TypeFloat: + val := binary.LittleEndian.Uint32(data[pos : pos+4]) + fval := math.Float32frombits(val) + builder.(*array.Float32Builder).Append(fval) + return 4, nil + case TypeDouble: + val := binary.LittleEndian.Uint64(data[pos : pos+8]) + fval := math.Float64frombits(val) + builder.(*array.Float64Builder).Append(fval) + return 8, nil + case TypeTimestamp: + val := binary.LittleEndian.Uint32(data[pos : pos+4]) + builder.(*array.TimestampBuilder).AppendTime(time.Unix(int64(val), 0).UTC()) + return 4, nil + case TypeLongLong: + val := binary.LittleEndian.Uint64(data[pos : pos+8]) + if sqltypes.IsSigned(ftype) { + builder.(*array.Int64Builder).Append(int64(val)) + } else { + builder.(*array.Uint64Builder).Append(val) + } + return 8, nil + case TypeDate, TypeNewDate: + val := uint32(data[pos]) + + uint32(data[pos+1])<<8 + + uint32(data[pos+2])<<16 + day := val & 31 + month := val >> 5 & 15 + year := val >> 9 + t := time.Date(int(year), time.Month(month), int(day), 0, 0, 0, 0, time.UTC) + builder.(*array.Date32Builder).Append(arrow.Date32FromTime(t)) + return 3, nil + case TypeTime: + var hour, minute, second int32 + if data[pos+2]&128 > 0 { + // Negative number, have to extend the sign. + val := int32(uint32(data[pos]) + + uint32(data[pos+1])<<8 + + uint32(data[pos+2])<<16 + + uint32(255)<<24) + hour = val / 10000 + minute = -((val % 10000) / 100) + second = -(val % 100) + } else { + val := int32(data[pos]) + + int32(data[pos+1])<<8 + + int32(data[pos+2])<<16 + hour = val / 10000 + minute = (val % 10000) / 100 + second = val % 100 + } + duration := time.Duration(hour*3600+minute*60+second) * time.Second + builder.(*array.DurationBuilder).Append(arrow.Duration(duration.Microseconds())) + return 3, nil + case TypeDateTime: + val := binary.LittleEndian.Uint64(data[pos : pos+8]) + d := val / 1000000 + t := val % 1000000 + year := d / 10000 + month := (d % 10000) / 100 + day := d % 100 + hour := t / 10000 + minute := (t % 10000) / 100 + second := t % 100 + builder.(*array.TimestampBuilder).AppendTime(time.Date(int(year), time.Month(month), int(day), int(hour), int(minute), int(second), 0, time.UTC)) + return 8, nil + case TypeVarchar, TypeVarString: + // We trust that typ is compatible with the ftype + // Length is encoded in 1 or 2 bytes. + typeToUse := querypb.Type_VARCHAR + if ftype == querypb.Type_VARBINARY || ftype == querypb.Type_BINARY || ftype == querypb.Type_BLOB { + typeToUse = ftype + } + var ( + size int + src []byte + ) + if metadata > 255 { + l := int(uint64(data[pos]) | + uint64(data[pos+1])<<8) + size = l + 2 + src = data[pos+2 : pos+2+l] + } else { + l := int(data[pos]) + size = l + 1 + src = data[pos+1 : pos+1+l] + } + if typeToUse == querypb.Type_VARCHAR { + utf8str, err := charset.DecodeBytes(column.Type.(sql.StringType).CharacterSet(), src) + if err != nil { + return size, err + } + builder.(*array.StringBuilder).BinaryBuilder.Append(utf8str) + } else { + builder.(*array.BinaryBuilder).Append(src) + } + return size, nil + case TypeBit: + // The contents is just the bytes, quoted. + nbits := ((metadata >> 8) * 8) + (metadata & 0xFF) + l := (int(nbits) + 7) / 8 + var buf [8]byte + copy(buf[8-l:], data[pos:pos+l]) + builder.(*array.Uint64Builder).Append(binary.BigEndian.Uint64(buf[:])) + return l, nil + case TypeTimestamp2: + second := binary.BigEndian.Uint32(data[pos : pos+4]) + size := 4 + frac := 0 + mul := 0 + switch metadata { + case 1: + decimals := int(data[pos+4]) + frac = decimals / 10 + mul = 100000 + size = 5 + case 2: + decimals := int(data[pos+4]) + frac = decimals + mul = 10000 + size = 5 + case 3: + decimals := int(data[pos+4])<<8 + + int(data[pos+5]) + frac = decimals / 10 + mul = 1000 + size = 6 + case 4: + decimals := int(data[pos+4])<<8 + + int(data[pos+5]) + frac = decimals + mul = 100 + size = 6 + case 5: + decimals := int(data[pos+4])<<16 + + int(data[pos+5])<<8 + + int(data[pos+6]) + frac = decimals / 10 + mul = 10 + size = 7 + case 6: + decimals := int(data[pos+4])<<16 + + int(data[pos+5])<<8 + + int(data[pos+6]) + frac = decimals + mul = 1 + size = 7 + } + frac *= mul + t := time.Unix(int64(second), int64(frac*1000)).UTC() + builder.(*array.TimestampBuilder).AppendTime(t) + return size, nil + case TypeDateTime2: + ymdhms := (uint64(data[pos])<<32 | + uint64(data[pos+1])<<24 | + uint64(data[pos+2])<<16 | + uint64(data[pos+3])<<8 | + uint64(data[pos+4])) - uint64(0x8000000000) + ymd := ymdhms >> 17 + ym := ymd >> 5 + hms := ymdhms % (1 << 17) + + day := ymd % (1 << 5) + month := ym % 13 + year := ym / 13 + + second := hms % (1 << 6) + minute := (hms >> 6) % (1 << 6) + hour := hms >> 12 + + size := 5 + frac := 0 + mul := 0 + + switch metadata { + case 1: + decimals := int(data[pos+5]) + frac = decimals / 10 + mul = 100000 + size = 6 + case 2: + decimals := int(data[pos+5]) + frac = decimals + mul = 10000 + size = 6 + case 3: + decimals := int(data[pos+5])<<8 + + int(data[pos+6]) + frac = decimals / 10 + mul = 1000 + size = 7 + case 4: + decimals := int(data[pos+5])<<8 + + int(data[pos+6]) + frac = decimals + mul = 100 + size = 7 + case 5: + decimals := int(data[pos+5])<<16 + + int(data[pos+6])<<8 + + int(data[pos+7]) + frac = decimals / 10 + mul = 10 + size = 8 + case 6: + decimals := int(data[pos+5])<<16 + + int(data[pos+6])<<8 + + int(data[pos+7]) + frac = decimals + mul = 1 + size = 8 + } + frac *= mul + t := time.Date(int(year), time.Month(month), int(day), int(hour), int(minute), int(second), int(frac*1000), time.UTC) + builder.(*array.TimestampBuilder).AppendTime(t) + return size, nil + case TypeTime2: + hms := (int64(data[pos])<<16 | + int64(data[pos+1])<<8 | + int64(data[pos+2])) - 0x800000 + sign := 1 + if hms < 0 { + hms = -hms + sign = -1 + } + + frac := 0 + mul := 0 + switch metadata { + case 1: + frac = int(data[pos+3]) + if sign == -1 && frac != 0 { + hms-- + frac = 0x100 - frac + } + frac /= 10 + mul = 100000 + case 2: + frac = int(data[pos+3]) + if sign == -1 && frac != 0 { + hms-- + frac = 0x100 - frac + } + mul = 10000 + case 3: + frac = int(data[pos+3])<<8 | + int(data[pos+4]) + if sign == -1 && frac != 0 { + hms-- + frac = 0x10000 - frac + } + frac /= 10 + mul = 1000 + case 4: + frac = int(data[pos+3])<<8 | + int(data[pos+4]) + if sign == -1 && frac != 0 { + hms-- + frac = 0x10000 - frac + } + mul = 100 + case 5: + frac = int(data[pos+3])<<16 | + int(data[pos+4])<<8 | + int(data[pos+5]) + if sign == -1 && frac != 0 { + hms-- + frac = 0x1000000 - frac + } + frac /= 10 + mul = 10 + case 6: + frac = int(data[pos+3])<<16 | + int(data[pos+4])<<8 | + int(data[pos+5]) + if sign == -1 && frac != 0 { + hms-- + frac = 0x1000000 - frac + } + mul = 1 + } + frac *= mul + + hour := (hms >> 12) % (1 << 10) + minute := (hms >> 6) % (1 << 6) + second := hms % (1 << 6) + duration := time.Duration(hour*3600+minute*60+second)*time.Second + time.Duration(frac)*time.Microsecond + micros := int64(sign) * duration.Microseconds() + builder.(*array.DurationBuilder).Append(arrow.Duration(micros)) + return 3 + (int(metadata)+1)/2, nil + + case TypeNewDecimal: + precision := int(metadata >> 8) // total digits number + scale := int(metadata & 0xff) // number of fractional digits + intg := precision - scale // number of full digits + intg0 := intg / 9 // number of 32-bits digits + intg0x := intg - intg0*9 // leftover full digits + frac0 := scale / 9 // number of 32 bits fractionals + frac0x := scale - frac0*9 // leftover fractionals + + l := intg0*4 + dig2bytes[intg0x] + frac0*4 + dig2bytes[frac0x] + + // Copy the data so we can change it. Otherwise + // decoding is just too hard. + // Using a constant capacity to ensure stack allocation: + // https://github.com/golang/go/issues/27625 + d := make([]byte, l, 40) + copy(d, data[pos:pos+l]) + + // txt := &bytes.Buffer{} + + isNegative := (d[0] & 0x80) == 0 + d[0] ^= 0x80 // First bit is inverted. + if isNegative { + // Negative numbers are just inverted bytes. + // txt.WriteByte('-') + for i := range d { + d[i] ^= 0xff + } + } + + // the initial 128 bits are stack-allocated + var coeff apd.BigInt + + // first we have the leftover full digits + var val uint32 + switch dig2bytes[intg0x] { + case 0: + // nothing to do + case 1: + // one byte, up to two digits + val = uint32(d[0]) + case 2: + // two bytes, up to 4 digits + val = uint32(d[0])<<8 + + uint32(d[1]) + case 3: + // 3 bytes, up to 6 digits + val = uint32(d[0])<<16 + + uint32(d[1])<<8 + + uint32(d[2]) + case 4: + // 4 bytes, up to 8 digits (9 digits would be a full) + val = uint32(d[0])<<24 + + uint32(d[1])<<16 + + uint32(d[2])<<8 + + uint32(d[3]) + } + pos = dig2bytes[intg0x] + if val > 0 { + // txt.Write(strconv.AppendUint(nil, uint64(val), 10)) + coeff.SetUint64(uint64(val)) + } + + var multiplier, tmp apd.BigInt + multiplier.SetUint64(1_000_000_000) // 9 digits + + // now the full digits, 32 bits each, 9 digits + for range intg0 { + val = binary.BigEndian.Uint32(d[pos : pos+4]) + // fmt.Fprintf(txt, "%09d", val) + tmp.SetUint64(uint64(val)) + coeff.Mul(&coeff, &multiplier) + coeff.Add(&coeff, &tmp) + pos += 4 + } + + // now see if we have a fraction + if scale == 0 { + // When the field is a DECIMAL using a scale of 0, e.g. + // DECIMAL(5,0), a binlogged value of 0 is almost treated + // like the NULL byte and we get a 0 byte length value. + // In this case let's return the correct value of 0. + // if txt.Len() == 0 { + // txt.WriteRune('0') + // } + + // keep stack-allocated if possible + var bi big.Int + bi.SetBits(coeff.Bits()) + + switch b := builder.(type) { + case *array.Decimal128Builder: + num := decimal128.FromBigInt(&bi) + if isNegative { + num = num.Negate() + } + b.Append(num) + case *array.Decimal256Builder: + num := decimal256.FromBigInt(&bi) + if isNegative { + num = num.Negate() + } + b.Append(num) + default: + return l, vterrors.Errorf(vtrpc.Code_INTERNAL, "unexpected Arrow builder type: %T", builder) + } + return l, nil + } + + // txt.WriteByte('.') + fp := 0 + + // now the full fractional digits + for range frac0 { + val = binary.BigEndian.Uint32(d[pos : pos+4]) + // fmt.Fprintf(txt, "%09d", val) + tmp.SetUint64(uint64(val)) + coeff.Mul(&coeff, &multiplier) + coeff.Add(&coeff, &tmp) + fp += 9 + pos += 4 + } + + // then the partial fractional digits + switch dig2bytes[frac0x] { + case 0: + // Nothing to do + break + case 1: + // one byte, 1 or 2 digits + val = uint32(d[pos]) + if frac0x == 1 { + // fmt.Fprintf(txt, "%1d", val) + multiplier.SetUint64(10) + tmp.SetUint64(uint64(val)) + coeff.Mul(&coeff, &multiplier) + coeff.Add(&coeff, &tmp) + } else { + // fmt.Fprintf(txt, "%02d", val) + multiplier.SetUint64(100) + tmp.SetUint64(uint64(val)) + coeff.Mul(&coeff, &multiplier) + coeff.Add(&coeff, &tmp) + } + case 2: + // two bytes, 3 or 4 digits + val = uint32(d[pos])<<8 + + uint32(d[pos+1]) + if frac0x == 3 { + // fmt.Fprintf(txt, "%03d", val) + multiplier.SetUint64(1_000) + tmp.SetUint64(uint64(val)) + coeff.Mul(&coeff, &multiplier) + coeff.Add(&coeff, &tmp) + } else { + // fmt.Fprintf(txt, "%04d", val) + multiplier.SetUint64(10_000) + tmp.SetUint64(uint64(val)) + coeff.Mul(&coeff, &multiplier) + coeff.Add(&coeff, &tmp) + } + case 3: + // 3 bytes, 5 or 6 digits + val = uint32(d[pos])<<16 + + uint32(d[pos+1])<<8 + + uint32(d[pos+2]) + if frac0x == 5 { + // fmt.Fprintf(txt, "%05d", val) + multiplier.SetUint64(100_000) + tmp.SetUint64(uint64(val)) + coeff.Mul(&coeff, &multiplier) + coeff.Add(&coeff, &tmp) + } else { + // fmt.Fprintf(txt, "%06d", val) + multiplier.SetUint64(1_000_000) + tmp.SetUint64(uint64(val)) + coeff.Mul(&coeff, &multiplier) + coeff.Add(&coeff, &tmp) + } + case 4: + // 4 bytes, 7 or 8 digits (9 digits would be a full) + val = uint32(d[pos])<<24 + + uint32(d[pos+1])<<16 + + uint32(d[pos+2])<<8 + + uint32(d[pos+3]) + if frac0x == 7 { + // fmt.Fprintf(txt, "%07d", val) + multiplier.SetUint64(10_000_000) + tmp.SetUint64(uint64(val)) + coeff.Mul(&coeff, &multiplier) + coeff.Add(&coeff, &tmp) + } else { + // fmt.Fprintf(txt, "%08d", val) + multiplier.SetUint64(100_000_000) + tmp.SetUint64(uint64(val)) + coeff.Mul(&coeff, &multiplier) + coeff.Add(&coeff, &tmp) + } + } + fp += frac0x + + // Pad with zero digits if necessary: + // the arrow array shares a common scale for all values, + // so we need to ensure that the number of fractional digits is as expected. + desired := int(builder.Type().(arrow.DecimalType).GetScale()) + if fp < desired { + // Pad 19 zero digits at a time + multiplier.SetUint64(10000_00000_00000_00000) + for fp+19 < desired { + coeff.Mul(&coeff, &multiplier) + fp += 19 + } + // Add the remaining zero digits + multiplier.SetUint64(powerOf10s[desired-fp]) + coeff.Mul(&coeff, &multiplier) + fp = desired + } else if fp > desired { + return l, vterrors.Errorf(vtrpc.Code_INTERNAL, "unexpected fractional digits: %v > %v", fp, desired) + } + + // keep stack-allocated if possible + var bi big.Int + bi.SetBits(coeff.Bits()) + switch b := builder.(type) { + case *array.Decimal128Builder: + num := decimal128.FromBigInt(&bi) + if isNegative { + num = num.Negate() + } + b.Append(num) + case *array.Decimal256Builder: + num := decimal256.FromBigInt(&bi) + if isNegative { + num = num.Negate() + } + b.Append(num) + default: + return l, vterrors.Errorf(vtrpc.Code_INTERNAL, "unexpected Arrow builder type: %T", builder) + } + return l, nil + + case TypeEnum: + var idx int + l := int(metadata & 0xff) + switch l { + case 1: + // One byte storage. + idx = int(data[pos]) + case 2: + // Two bytes storage. + idx = int(binary.LittleEndian.Uint16(data[pos : pos+2])) + default: + return 0, vterrors.Errorf(vtrpc.Code_INTERNAL, "unexpected enum size: %v", metadata&0xff) + } + val, ok := column.Type.(sql.EnumType).At(idx) + if !ok { + return l, vterrors.Errorf(vtrpc.Code_INTERNAL, "enum value %v not found in %v", idx, column.Type) + } + builder.(*array.StringBuilder).Append(val) + return l, nil + + case TypeSet: + l := int(metadata & 0xff) + var val uint64 + for i := range l { + val += uint64(data[pos+i]) << (uint(i) * 8) + } + s, err := column.Type.(sql.SetType).BitsToString(val) + if err != nil { + return l, vterrors.Errorf(vtrpc.Code_INTERNAL, "invalid bit value %x for set %v", val, column.Type) + } + builder.(*array.StringBuilder).Append(s) + return l, nil + + case TypeJSON, TypeTinyBlob, TypeMediumBlob, TypeLongBlob, TypeBlob, TypeVector: + // Only TypeBlob and TypeVector is used in binary logs, + // but supports others just in case. + l := 0 + switch metadata { + case 1: + l = int(uint32(data[pos])) + case 2: + l = int(uint32(data[pos]) | + uint32(data[pos+1])<<8) + case 3: + l = int(uint32(data[pos]) | + uint32(data[pos+1])<<8 | + uint32(data[pos+2])<<16) + case 4: + l = int(uint32(data[pos]) | + uint32(data[pos+1])<<8 | + uint32(data[pos+2])<<16 | + uint32(data[pos+3])<<24) + default: + return 0, vterrors.Errorf(vtrpc.Code_INTERNAL, "unsupported blob metadata value %v (data: %v pos: %v)", metadata, data, pos) + } + pos += int(metadata) + + // For JSON, we parse the data, and emit SQL. + if typ == TypeJSON { + var err error + jsonData := data[pos : pos+l] + jsonVal, err := vtbinlog.ParseBinaryJSON(jsonData) + if err != nil { + panic(err) + } + var buf [64]byte + d := jsonVal.MarshalTo(buf[:0]) + builder.(*array.StringBuilder).BinaryBuilder.Append(d) + return l + int(metadata), nil + } + + // For blobs, we just copy the bytes. + switch builder := builder.(type) { + case *array.BinaryBuilder: + builder.Append(data[pos : pos+l]) + case *array.StringBuilder: + utf8str, err := charset.DecodeBytes(column.Type.(sql.StringType).CharacterSet(), data[pos:pos+l]) + if err != nil { + return l, vterrors.Errorf(vtrpc.Code_INTERNAL, "failed to decode string: %v", err) + } + builder.BinaryBuilder.Append(utf8str) + default: + return 0, vterrors.Errorf(vtrpc.Code_INTERNAL, "unexpected Arrow builder type: %T", builder) + } + return l + int(metadata), nil + + case TypeString: + // This may do String, Enum, and Set. The type is in + // metadata. If it's a string, then there will be more bits. + t := metadata >> 8 + if t == TypeEnum { + // We don't know the string values. So just use the + // numbers. + l := int(metadata & 0xff) + var idx int + switch metadata & 0xff { + case 1: + // One byte storage. + idx = int(data[pos]) + case 2: + // Two bytes storage. + idx = int(binary.LittleEndian.Uint16(data[pos : pos+2])) + default: + return 0, vterrors.Errorf(vtrpc.Code_INTERNAL, "unexpected enum size: %v", metadata&0xff) + } + str, ok := column.Type.(sql.EnumType).At(idx) + if !ok { + return l, vterrors.Errorf(vtrpc.Code_INTERNAL, "enum value %v not found in %v", data[pos], column.Type) + } + builder.(*array.StringBuilder).Append(str) + return l, nil + } + if t == TypeSet { + // We don't know the set values. So just use the + // numbers. + l := int(metadata & 0xff) + var val uint64 + for i := range l { + val += uint64(data[pos+i]) << (uint(i) * 8) + } + str, err := column.Type.(sql.SetType).BitsToString(val) + if err != nil { + return l, vterrors.Errorf(vtrpc.Code_INTERNAL, "invalid bit value %x for set %v", val, column.Type) + } + builder.(*array.StringBuilder).Append(str) + return l, nil + } + // This is a real string. The length is weird. + max := int((((metadata >> 4) & 0x300) ^ 0x300) + (metadata & 0xff)) + // Length is encoded in 1 or 2 bytes. + if max > 255 { + // This code path exists due to https://bugs.mysql.com/bug.php?id=37426. + // CHAR types need to allocate 3 bytes per char. So, the length for CHAR(255) + // cannot be represented in 1 byte. This also means that this rule does not + // apply to BINARY data. + l := int(uint64(data[pos]) | + uint64(data[pos+1])<<8) + utf8str, err := charset.DecodeBytes(column.Type.(sql.StringType).CharacterSet(), data[pos+2:pos+2+l]) + if err != nil { + return l + 2, vterrors.Errorf(vtrpc.Code_INTERNAL, "failed to decode string: %v", err) + } + builder.(*array.StringBuilder).BinaryBuilder.Append(utf8str) + return l + 2, nil + } + l := int(data[pos]) + mdata := data[pos+1 : pos+1+l] + if sqltypes.IsBinary(ftype) { + // For binary(n) column types, mysql pads the data on the right with nulls. However the binlog event contains + // the data without this padding. This causes several issues: + // * if a binary(n) column is part of the sharding key, the keyspace_id() returned during the copy phase + // (where the value is the result of a mysql query) is different from the one during replication + // (where the value is the one from the binlogs) + // * mysql where clause comparisons do not do the right thing without padding + // So for fixed length BINARY columns we right-pad it with nulls if necessary to match what MySQL returns. + // Because CHAR columns with a binary collation (e.g. utf8mb4_bin) have the same metadata as a BINARY column + // in binlog events, we also need to check for this case based on the underlying column type. + if l < max && ftype == querypb.Type_BINARY { + paddedData := make([]byte, max) + copy(paddedData[:l], mdata) + mdata = paddedData + } + if builder, ok := builder.(*array.BinaryBuilder); ok { + builder.Append(mdata) + return l + 1, nil + } // Otherwise, fall through to handle (VAR)CHAR/TEXT columns. + } + utf8str, err := charset.DecodeBytes(column.Type.(sql.StringType).CharacterSet(), mdata) + if err != nil { + return l + 1, vterrors.Errorf(vtrpc.Code_INTERNAL, "failed to decode string: %v", err) + } + builder.(*array.StringBuilder).BinaryBuilder.Append(utf8str) + return l + 1, nil + + case TypeGeometry: + l := 0 + switch metadata { + case 1: + l = int(uint32(data[pos])) + case 2: + l = int(uint32(data[pos]) | + uint32(data[pos+1])<<8) + case 3: + l = int(uint32(data[pos]) | + uint32(data[pos+1])<<8 | + uint32(data[pos+2])<<16) + case 4: + l = int(uint32(data[pos]) | + uint32(data[pos+1])<<8 | + uint32(data[pos+2])<<16 | + uint32(data[pos+3])<<24) + default: + return 0, vterrors.Errorf(vtrpc.Code_INTERNAL, "unsupported geometry metadata value %v (data: %v pos: %v)", metadata, data, pos) + } + pos += int(metadata) + builder.(*array.BinaryBuilder).Append(data[pos : pos+l]) + return l + int(metadata), nil + + default: + return 0, vterrors.Errorf(vtrpc.Code_INTERNAL, "unsupported type %v", typ) + } +} diff --git a/binlogreplication/binlog_file_utils.go b/binlogreplication/binlog_file_utils.go index 84f904ef..0d5de564 100644 --- a/binlogreplication/binlog_file_utils.go +++ b/binlogreplication/binlog_file_utils.go @@ -21,7 +21,7 @@ import ( "strconv" "strings" - "github.com/dolthub/vitess/go/mysql" + "vitess.io/vitess/go/mysql" ) // binlogFileMagicNumber holds the four bytes that start off every diff --git a/binlogreplication/binlog_position_store.go b/binlogreplication/binlog_position_store.go index cc63ad1e..793a40f8 100644 --- a/binlogreplication/binlog_position_store.go +++ b/binlogreplication/binlog_position_store.go @@ -25,7 +25,7 @@ import ( gms "github.com/dolthub/go-mysql-server" "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/vitess/go/mysql" + "vitess.io/vitess/go/mysql/replication" ) const binlogPositionDirectory = ".replica" @@ -45,33 +45,33 @@ type binlogPositionStore struct { // represents the set of GTIDs that have been successfully executed and applied on this replica. Currently only the // default binlog channel ("") is supported. If no .replica/binlog-position file is stored, this method returns a nil // mysql.Position and a nil error. If any errors are encountered, a nil mysql.Position and an error are returned. -func (store *binlogPositionStore) Load(engine *gms.Engine) (*mysql.Position, error) { +func (store *binlogPositionStore) Load(engine *gms.Engine) (pos replication.Position, err error) { store.mu.Lock() defer store.mu.Unlock() dir := filepath.Join(getDataDir(engine), binlogPositionDirectory) - _, err := os.Stat(dir) + _, err = os.Stat(dir) if err != nil && errors.Is(err, fs.ErrNotExist) { - return nil, nil + return pos, nil } else if err != nil { - return nil, err + return pos, err } _, err = os.Stat(filepath.Join(dir, binlogPositionFilename)) if err != nil && errors.Is(err, fs.ErrNotExist) { - return nil, nil + return pos, nil } else if err != nil { - return nil, err + return pos, err } filePath, err := filepath.Abs(filepath.Join(dir, binlogPositionFilename)) if err != nil { - return nil, err + return pos, err } bytes, err := os.ReadFile(filePath) if err != nil { - return nil, err + return pos, err } positionString := string(bytes) @@ -81,12 +81,7 @@ func (store *binlogPositionStore) Load(engine *gms.Engine) (*mysql.Position, err positionString = string(bytes[len(prefix):]) } - position, err := mysql.ParsePosition(mysqlFlavor, positionString) - if err != nil { - return nil, err - } - - return &position, nil + return replication.ParsePosition(mysqlFlavor, positionString) } // Save saves the specified |position| to disk in the .replica/binlog-position file at the root of the provider's @@ -95,8 +90,8 @@ func (store *binlogPositionStore) Load(engine *gms.Engine) (*mysql.Position, err // represents the set of GTIDs that have been successfully executed and applied on this replica. Currently only the // default binlog channel ("") is supported. If any errors are encountered persisting the position to disk, an // error is returned. -func (store *binlogPositionStore) Save(ctx *sql.Context, engine *gms.Engine, position *mysql.Position) error { - if position == nil { +func (store *binlogPositionStore) Save(ctx *sql.Context, engine *gms.Engine, position replication.Position) error { + if position.IsZero() { return fmt.Errorf("unable to save binlog position: nil position passed") } @@ -114,7 +109,7 @@ func (store *binlogPositionStore) Save(ctx *sql.Context, engine *gms.Engine, pos return err } - encodedPosition := mysql.EncodePosition(*position) + encodedPosition := replication.EncodePosition(position) return os.WriteFile(filePath, []byte(encodedPosition), 0666) } diff --git a/binlogreplication/binlog_replica_applier.go b/binlogreplication/binlog_replica_applier.go index e6ee713b..86849bfc 100644 --- a/binlogreplication/binlog_replica_applier.go +++ b/binlogreplication/binlog_replica_applier.go @@ -15,6 +15,7 @@ package binlogreplication import ( + "encoding/binary" "errors" "fmt" "io" @@ -23,17 +24,20 @@ import ( "sync/atomic" "time" + "github.com/apecloud/myduckserver/binlog" "github.com/apecloud/myduckserver/charset" gms "github.com/dolthub/go-mysql-server" "github.com/dolthub/go-mysql-server/sql" "github.com/dolthub/go-mysql-server/sql/binlogreplication" - "github.com/dolthub/go-mysql-server/sql/planbuilder" - "github.com/dolthub/go-mysql-server/sql/rowexec" "github.com/dolthub/go-mysql-server/sql/types" - "github.com/dolthub/vitess/go/mysql" - "github.com/dolthub/vitess/go/sqltypes" - vquery "github.com/dolthub/vitess/go/vt/proto/query" + doltvtmysql "github.com/dolthub/vitess/go/mysql" "github.com/sirupsen/logrus" + "vitess.io/vitess/go/mysql" + vbinlog "vitess.io/vitess/go/mysql/binlog" + "vitess.io/vitess/go/mysql/replication" + "vitess.io/vitess/go/mysql/sqlerror" + "vitess.io/vitess/go/sqltypes" + vquery "vitess.io/vitess/go/vt/proto/query" ) // positionStore is a singleton instance for loading/saving binlog position state to disk for durable storage. @@ -52,9 +56,9 @@ type binlogReplicaApplier struct { format *mysql.BinlogFormat tableMapsById map[uint64]*mysql.TableMap stopReplicationChan chan struct{} - currentGtid mysql.GTID + currentGtid replication.GTID replicationSourceUuid string - currentPosition *mysql.Position // successfully executed GTIDs + currentPosition replication.Position // successfully executed GTIDs filters *filterConfiguration running atomic.Bool engine *gms.Engine @@ -88,7 +92,7 @@ func (a *binlogReplicaApplier) Go(ctx *sql.Context) { a.running.Store(false) if err != nil { ctx.GetLogger().Errorf("unexpected error of type %T: '%v'", err, err.Error()) - MyBinlogReplicaController.setSqlError(mysql.ERUnknownError, err.Error()) + MyBinlogReplicaController.setSqlError(sqlerror.ERUnknownError, err.Error()) } }() } @@ -191,7 +195,7 @@ func (a *binlogReplicaApplier) startReplicationEventStream(ctx *sql.Context, con return err } - if position == nil { + if position.IsZero() { // If the positionStore doesn't have a record of executed GTIDs, check to see if the gtid_purged system // variable is set. If it holds a GTIDSet, then we use that as our starting position. As part of loading // a mysqldump onto a replica, gtid_purged will be set to indicate where to start replication. @@ -209,25 +213,25 @@ func (a *binlogReplicaApplier) startReplicationEventStream(ctx *sql.Context, con gtidPurged = gtidPurged[1:] } - purged, err := mysql.ParsePosition(mysqlFlavor, gtidPurged) + purged, err := replication.ParsePosition(mysqlFlavor, gtidPurged) if err != nil { return err } - position = &purged + position = purged } } - if position == nil { + if position.IsZero() { // If we still don't have any record of executed GTIDs, we create a GTIDSet with just one transaction ID // for the 0000 server ID. There doesn't seem to be a cleaner way of saying "start at the very beginning". // // Also... "starting position" is a bit of a misnomer – it's actually the processed GTIDs, which // indicate the NEXT GTID where replication should start, but it's not as direct as specifying // a starting position, like the Vitess function signature seems to suggest. - gtid := mysql.Mysql56GTID{ + gtid := replication.Mysql56GTID{ Sequence: 1, } - position = &mysql.Position{GTIDSet: gtid.GTIDSet()} + position = replication.Position{GTIDSet: gtid.GTIDSet()} } a.currentPosition = position @@ -246,7 +250,18 @@ func (a *binlogReplicaApplier) startReplicationEventStream(ctx *sql.Context, con return err } - return conn.SendBinlogDumpCommand(serverId, *position) + binlogFile := "" + if filePos, ok := position.GTIDSet.(replication.FilePosGTID); ok { + binlogFile = filePos.File + } + + ctx.GetLogger().WithFields(logrus.Fields{ + "serverId": serverId, + "binlogFile": binlogFile, + "position": position.String(), + }).Infoln("Sending binlog dump command to source") + + return conn.SendBinlogDumpCommand(serverId, binlogFile, position) } // replicaBinlogEventHandler runs a loop, processing binlog events until the applier's stop replication channel @@ -280,11 +295,11 @@ func (a *binlogReplicaApplier) replicaBinlogEventHandler(ctx *sql.Context) error err := a.processBinlogEvent(ctx, engine, event) if err != nil { ctx.GetLogger().Errorf("unexpected error of type %T: '%v'", err, err.Error()) - MyBinlogReplicaController.setSqlError(mysql.ERUnknownError, err.Error()) + MyBinlogReplicaController.setSqlError(sqlerror.ERUnknownError, err.Error()) } case err := <-eventProducer.ErrorChan(): - if sqlError, isSqlError := err.(*mysql.SQLError); isSqlError { + if sqlError, isSqlError := err.(*sqlerror.SQLError); isSqlError { badConnection := sqlError.Message == io.EOF.Error() || strings.HasPrefix(sqlError.Message, io.ErrUnexpectedEOF.Error()) if badConnection { @@ -302,7 +317,7 @@ func (a *binlogReplicaApplier) replicaBinlogEventHandler(ctx *sql.Context) error } else { // otherwise, log the error if it's something we don't expect and continue ctx.GetLogger().Errorf("unexpected error of type %T: '%v'", err, err.Error()) - MyBinlogReplicaController.setIoError(mysql.ERUnknownError, err.Error()) + MyBinlogReplicaController.setIoError(sqlerror.ERUnknownError, err.Error()) } case <-a.stopReplicationChan: @@ -332,7 +347,7 @@ func (a *binlogReplicaApplier) processBinlogEvent(ctx *sql.Context, engine *gms. if err != nil { msg := fmt.Sprintf("unable to strip checksum from binlog event: '%v'", err.Error()) ctx.GetLogger().Error(msg) - MyBinlogReplicaController.setSqlError(mysql.ERUnknownError, msg) + MyBinlogReplicaController.setSqlError(sqlerror.ERUnknownError, msg) } } @@ -359,12 +374,15 @@ func (a *binlogReplicaApplier) processBinlogEvent(ctx *sql.Context, engine *gms. if err != nil { return err } + + flags, mode := parseQueryEventVars(*a.format, event) + ctx.GetLogger().WithFields(logrus.Fields{ "database": query.Database, "charset": query.Charset, "query": query.SQL, - "options": fmt.Sprintf("0x%x", query.Options), - "sql_mode": fmt.Sprintf("0x%x", query.SqlMode), + "flags": fmt.Sprintf("0x%x", flags), + "sql_mode": fmt.Sprintf("0x%x", mode), }).Infoln("Received binlog event: Query") // When executing SQL statements sent from the primary, we can't be sure what database was modified unless we @@ -374,7 +392,7 @@ func (a *binlogReplicaApplier) processBinlogEvent(ctx *sql.Context, engine *gms. // avoid issues with correctness, at the cost of being slightly less efficient commitToAllDatabases = true - if query.Options&mysql.QFlagOptionAutoIsNull > 0 { + if flags&doltvtmysql.QFlagOptionAutoIsNull > 0 { ctx.GetLogger().Tracef("Setting sql_auto_is_null ON") ctx.SetSessionVariable(ctx, "sql_auto_is_null", 1) } else { @@ -382,7 +400,7 @@ func (a *binlogReplicaApplier) processBinlogEvent(ctx *sql.Context, engine *gms. ctx.SetSessionVariable(ctx, "sql_auto_is_null", 0) } - if query.Options&mysql.QFlagOptionNotAutocommit > 0 { + if flags&doltvtmysql.QFlagOptionNotAutocommit > 0 { ctx.GetLogger().Tracef("Setting autocommit=0") ctx.SetSessionVariable(ctx, "autocommit", 0) } else { @@ -390,7 +408,7 @@ func (a *binlogReplicaApplier) processBinlogEvent(ctx *sql.Context, engine *gms. ctx.SetSessionVariable(ctx, "autocommit", 1) } - if query.Options&mysql.QFlagOptionNoForeignKeyChecks > 0 { + if flags&doltvtmysql.QFlagOptionNoForeignKeyChecks > 0 { ctx.GetLogger().Tracef("Setting foreign_key_checks=0") ctx.SetSessionVariable(ctx, "foreign_key_checks", 0) } else { @@ -399,7 +417,7 @@ func (a *binlogReplicaApplier) processBinlogEvent(ctx *sql.Context, engine *gms. } // NOTE: unique_checks is not currently honored by Dolt - if query.Options&mysql.QFlagOptionRelaxedUniqueChecks > 0 { + if flags&doltvtmysql.QFlagOptionRelaxedUniqueChecks > 0 { ctx.GetLogger().Tracef("Setting unique_checks=0") ctx.SetSessionVariable(ctx, "unique_checks", 0) } else { @@ -407,13 +425,13 @@ func (a *binlogReplicaApplier) processBinlogEvent(ctx *sql.Context, engine *gms. ctx.SetSessionVariable(ctx, "unique_checks", 1) } - createCommit = strings.ToLower(query.SQL) != "begin" - // TODO(fan): Disable the transaction for now. - if createCommit { - if !(query.Database == "mysql" && strings.HasPrefix(query.SQL, "TRUNCATE TABLE")) { - ctx.SetCurrentDatabase(query.Database) - executeQueryWithEngine(ctx, engine, query.SQL) - } + createCommit = !strings.EqualFold(query.SQL, "begin") + // TODO(fan): Here we + // skip the transaction for now; + // skip the operations on `mysql.time_zone*` tables, which are not supported by go-mysql-server yet. + if createCommit && !(query.Database == "mysql" && strings.HasPrefix(query.SQL, "TRUNCATE TABLE time_zone")) { + ctx.SetCurrentDatabase(query.Database) + executeQueryWithEngine(ctx, engine, query.SQL) } case event.IsRotate(): @@ -509,7 +527,7 @@ func (a *binlogReplicaApplier) processBinlogEvent(ctx *sql.Context, engine *gms. if flags != 0 { msg := fmt.Sprintf("unsupported binlog protocol message: TableMap event with unsupported flags '%x'", flags) ctx.GetLogger().Errorf(msg) - MyBinlogReplicaController.setSqlError(mysql.ERUnknownError, msg) + MyBinlogReplicaController.setSqlError(sqlerror.ERUnknownError, msg) } a.tableMapsById[tableId] = tableMap } @@ -540,7 +558,7 @@ func (a *binlogReplicaApplier) processBinlogEvent(ctx *sql.Context, engine *gms. } if createCommit { - // TODO(fan): Disable transaction commit for now + // TODO(fan): Skip the transaction commit for now _ = commitToAllDatabases // var databasesToCommit []string // if commitToAllDatabases { @@ -569,18 +587,18 @@ func (a *binlogReplicaApplier) processBinlogEvent(ctx *sql.Context, engine *gms. // processRowEvent processes a WriteRows, DeleteRows, or UpdateRows binlog event and returns an error if any problems // were encountered. func (a *binlogReplicaApplier) processRowEvent(ctx *sql.Context, event mysql.BinlogEvent, engine *gms.Engine) error { - var eventType string + var eventName string switch { case event.IsDeleteRows(): - eventType = "DeleteRows" + eventName = "DeleteRows" case event.IsWriteRows(): - eventType = "WriteRows" + eventName = "WriteRows" case event.IsUpdateRows(): - eventType = "UpdateRows" + eventName = "UpdateRows" default: return fmt.Errorf("unsupported event type: %v", event) } - ctx.GetLogger().Tracef("Received binlog event: %s", eventType) + ctx.GetLogger().Tracef("Received binlog event: %s", eventName) tableId := event.TableID(*a.format) tableMap, ok := a.tableMapsById[tableId] @@ -604,7 +622,7 @@ func (a *binlogReplicaApplier) processRowEvent(ctx *sql.Context, event mysql.Bin ctx.GetLogger().WithFields(logrus.Fields{ "flags": fmt.Sprintf("%x", rows.Flags), - }).Tracef("Processing rows from %s event", eventType) + }).Tracef("Processing rows from %s event", eventName) flags := rows.Flags foreignKeyChecksDisabled := false @@ -619,45 +637,56 @@ func (a *binlogReplicaApplier) processRowEvent(ctx *sql.Context, event mysql.Bin if flags != 0 { msg := fmt.Sprintf("unsupported binlog protocol message: row event with unsupported flags '%x'", flags) ctx.GetLogger().Errorf(msg) - MyBinlogReplicaController.setSqlError(mysql.ERUnknownError, msg) + MyBinlogReplicaController.setSqlError(sqlerror.ERUnknownError, msg) } - schema, tableName, err := getTableSchema(ctx, engine, tableMap.Name, tableMap.Database) + pkSchema, tableName, err := getTableSchema(ctx, engine, tableMap.Name, tableMap.Database) if err != nil { return err } + schema := pkSchema.Schema + + fieldCount := len(schema) + if len(tableMap.Types) != fieldCount { + return fmt.Errorf("schema mismatch: expected %d fields, got %d from binlog", fieldCount, len(tableMap.Types)) + } - var typ EventType + var eventType binlog.RowEventType + var isRowFormat bool // all columns are present switch { case event.IsDeleteRows(): - typ = DeleteEvent - ctx.GetLogger().Tracef(" - Deleted Rows (table: %s)", tableMap.Name) + eventType = binlog.DeleteRowEvent + isRowFormat = rows.IdentifyColumns.BitCount() == fieldCount case event.IsUpdateRows(): - typ = UpdateEvent - ctx.GetLogger().Tracef(" - Updated Rows (table: %s)", tableMap.Name) + eventType = binlog.UpdateRowEvent + isRowFormat = rows.IdentifyColumns.BitCount() == fieldCount && rows.DataColumns.BitCount() == fieldCount case event.IsWriteRows(): - typ = InsertEvent - ctx.GetLogger().Tracef(" - Inserted Rows (table: %s)", tableMap.Name) + eventType = binlog.InsertRowEvent + isRowFormat = rows.DataColumns.BitCount() == fieldCount } + ctx.GetLogger().Tracef(" - %s Rows (db: %s, table: %s, row-format: %v)", eventType, tableMap.Database, tableName, isRowFormat) - tableWriter, err := a.tableWriterProvider.GetTableWriter( - ctx, engine, - tableMap.Database, tableName, - schema, - len(tableMap.Types), len(rows.Rows), - rows.IdentifyColumns, rows.DataColumns, - typ, - foreignKeyChecksDisabled, - ) - if err != nil { - return err + if isRowFormat && len(pkSchema.PkOrdinals) > 0 { + // --binlog-format=ROW & --binlog-row-image=full + return a.appendRowFormatChanges(ctx, engine, tableMap, tableName, schema, eventType, &rows) + } else { + return a.writeChanges(ctx, engine, tableMap, tableName, pkSchema, eventType, &rows, foreignKeyChecksDisabled) } +} +func (a *binlogReplicaApplier) writeChanges( + ctx *sql.Context, engine *gms.Engine, + tableMap *mysql.TableMap, tableName string, pkSchema sql.PrimaryKeySchema, + event binlog.RowEventType, rows *mysql.Rows, + foreignKeyChecksDisabled bool, +) error { identityRows := make([]sql.Row, 0, len(rows.Rows)) dataRows := make([]sql.Row, 0, len(rows.Rows)) for _, row := range rows.Rows { var identityRow, dataRow sql.Row + var err error + if len(row.Identify) > 0 { - identityRow, err = parseRow(ctx, engine, tableMap, schema, rows.IdentifyColumns, row.NullIdentifyColumns, row.Identify) + identityRow, err = parseRow(ctx, engine, tableMap, pkSchema.Schema, rows.IdentifyColumns, row.NullIdentifyColumns, row.Identify) if err != nil { return err } @@ -668,7 +697,7 @@ func (a *binlogReplicaApplier) processRowEvent(ctx *sql.Context, event mysql.Bin identityRows = append(identityRows, identityRow) if len(row.Data) > 0 { - dataRow, err = parseRow(ctx, engine, tableMap, schema, rows.DataColumns, row.NullColumns, row.Data) + dataRow, err = parseRow(ctx, engine, tableMap, pkSchema.Schema, rows.DataColumns, row.NullColumns, row.Data) if err != nil { return err } @@ -677,12 +706,26 @@ func (a *binlogReplicaApplier) processRowEvent(ctx *sql.Context, event mysql.Bin dataRows = append(dataRows, dataRow) } - switch { - case event.IsDeleteRows(): + tableWriter, err := a.tableWriterProvider.GetTableWriter( + ctx, engine, + tableMap.Database, tableName, + pkSchema, + len(tableMap.Types), len(rows.Rows), + rows.IdentifyColumns, rows.DataColumns, + event, + foreignKeyChecksDisabled, + ) + if err != nil { + return err + } + defer tableWriter.Rollback() + + switch event { + case binlog.DeleteRowEvent: err = tableWriter.Delete(ctx, identityRows) - case event.IsWriteRows(): + case binlog.InsertRowEvent: err = tableWriter.Insert(ctx, dataRows) - case event.IsUpdateRows(): + case binlog.UpdateRowEvent: err = tableWriter.Update(ctx, identityRows, dataRows) } if err != nil { @@ -692,33 +735,177 @@ func (a *binlogReplicaApplier) processRowEvent(ctx *sql.Context, event mysql.Bin ctx.GetLogger().WithFields(logrus.Fields{ "db": tableMap.Database, "table": tableName, - "event": eventType, + "event": event, "rows": len(rows.Rows), }).Infoln("processRowEvent") - return tableWriter.Close() + return tableWriter.Commit() +} + +func (a *binlogReplicaApplier) appendRowFormatChanges( + ctx *sql.Context, engine *gms.Engine, + tableMap *mysql.TableMap, tableName string, schema sql.Schema, + event binlog.RowEventType, rows *mysql.Rows, +) error { + appender, err := a.tableWriterProvider.GetDeltaAppender(ctx, engine, tableMap.Database, tableName, schema) + if err != nil { + return err + } + + var ( + fields = appender.Fields() + actions = appender.Action() + txnTags = appender.TxnTag() + txnServers = appender.TxnServer() + txnGroups = appender.TxnGroup() + txnSeqNumbers = appender.TxnSeqNumber() + + txnTag []byte + txnServer []byte + txnGroup []byte + txnSeq uint64 + ) + + switch gtid := a.currentGtid.(type) { + case replication.Mysql56GTID: + // TODO(fan): Add support for GTID tags in MySQL >=8.4 + txnServer = gtid.Server[:] + txnSeq = uint64(gtid.Sequence) + case replication.FilePosGTID: + txnGroup = []byte(gtid.File) + txnSeq = uint64(gtid.Pos) + case replication.MariadbGTID: + var domain, server [4]byte + binary.BigEndian.PutUint32(domain[:], gtid.Domain) + binary.BigEndian.PutUint32(server[:], gtid.Server) + txnTag = domain[:] + txnServer = server[:] + txnSeq = gtid.Sequence + } + + // The following code is a bit repetitive, but we want to avoid the overhead of function calls for each row. + + // Delete the before image + switch event { + case binlog.DeleteRowEvent, binlog.UpdateRowEvent: + for _, row := range rows.Rows { + actions.Append(int8(binlog.DeleteRowEvent)) + txnTags.Append(txnTag) + txnServers.Append(txnServer) + txnGroups.Append(txnGroup) + txnSeqNumbers.Append(txnSeq) + + pos := 0 + for i := range schema { + builder := fields[i] + + if row.NullIdentifyColumns.Bit(i) { + builder.AppendNull() + continue + } + + length, err := binlog.CellValue(row.Identify, pos, tableMap.Types[i], tableMap.Metadata[i], schema[i], builder) + if err != nil { + return err + } + pos += length + } + } + } + + // Insert the after image + switch event { + case binlog.InsertRowEvent, binlog.UpdateRowEvent: + for _, row := range rows.Rows { + actions.Append(int8(binlog.InsertRowEvent)) + txnTags.Append(txnTag) + txnServers.Append(txnServer) + txnGroups.Append(txnGroup) + txnSeqNumbers.Append(txnSeq) + + pos := 0 + for i := range schema { + builder := appender.Field(i) + + if row.NullColumns.Bit(i) { + builder.AppendNull() + continue + } + + length, err := binlog.CellValue(row.Data, pos, tableMap.Types[i], tableMap.Metadata[i], schema[i], builder) + if err != nil { + return err + } + pos += length + } + } + } + + // TODO(fan): Apparently this is not how the delta appender is supposed to be used. + // But let's make it work for now. + return a.tableWriterProvider.FlushDelta(ctx) } // // Helper functions // +// parseQueryEventVars parses the status variables block of a Query event and returns the flags and SQL mode. +// Copied from: Vitess's vitess/go/mysql/binlog_event_common.go +// See also: https://dev.mysql.com/doc/dev/mysql-server/latest/page_protocol_replication_binlog_event.html#sect_protocol_replication_event_query_03 +func parseQueryEventVars(format mysql.BinlogFormat, event mysql.BinlogEvent) (flags uint32, mode uint64) { + data := event.Bytes()[format.HeaderLength:] + const varPos = 4 + 4 + 1 + 2 + 2 + varsLen := int(binary.LittleEndian.Uint16(data[4+4+1+2 : 4+4+1+2+2])) + vars := data[varPos : varPos+varsLen] + +varsLoop: + for pos := 0; pos < len(vars); { + code := vars[pos] + pos++ + + switch code { + case mysql.QFlags2Code: + flags = binary.LittleEndian.Uint32(vars[pos : pos+4]) + pos += 4 + case mysql.QSQLModeCode: + mode = binary.LittleEndian.Uint64(vars[pos : pos+8]) + case mysql.QAutoIncrement: + pos += 4 + case mysql.QCatalog: + pos += 1 + int(vars[pos]) + 1 + case mysql.QCatalogNZCode: + pos += 1 + int(vars[pos]) + case mysql.QCharsetCode: + pos += 6 + default: + break varsLoop + } + } + + return +} + // getTableSchema returns a sql.Schema for the case-insensitive |tableName| in the database named // |databaseName|, along with the exact, case-sensitive table name. -func getTableSchema(ctx *sql.Context, engine *gms.Engine, tableName, databaseName string) (sql.Schema, string, error) { +func getTableSchema(ctx *sql.Context, engine *gms.Engine, tableName, databaseName string) (sql.PrimaryKeySchema, string, error) { database, err := engine.Analyzer.Catalog.Database(ctx, databaseName) if err != nil { - return nil, "", err + return sql.PrimaryKeySchema{}, "", err } table, ok, err := database.GetTableInsensitive(ctx, tableName) if err != nil { - return nil, "", err + return sql.PrimaryKeySchema{}, "", err } if !ok { - return nil, "", fmt.Errorf("unable to find table %q", tableName) + return sql.PrimaryKeySchema{}, "", fmt.Errorf("unable to find table %q", tableName) + } + + if pkTable, ok := table.(sql.PrimaryKeyTable); ok { + return pkTable.PrimaryKeySchema(), table.Name(), nil } - return table.Schema(), table.Name(), nil + return sql.NewPrimaryKeySchema(table.Schema()), table.Name(), nil } // parseRow parses the binary row data from a MySQL binlog event and converts it into a go-mysql-server Row using the @@ -731,7 +918,7 @@ func parseRow(ctx *sql.Context, engine *gms.Engine, tableMap *mysql.TableMap, sc for i, typ := range tableMap.Types { column := schema[i] - if columnsPresentBitmap.Bit(i) == false { + if !columnsPresentBitmap.Bit(i) { parsedRow = append(parsedRow, nil) continue } @@ -745,7 +932,11 @@ func parseRow(ctx *sql.Context, engine *gms.Engine, tableMap *mysql.TableMap, sc } } else { var length int - value, length, err = mysql.CellValue(data, pos, typ, tableMap.Metadata[i], getSignedType(column)) + value, length, err = vbinlog.CellValue(data, pos, typ, tableMap.Metadata[i], &vquery.Field{ + Name: column.Name, + Type: vquery.Type(column.Type.Type()), + ColumnType: column.Type.String(), + }) if err != nil { return nil, err } @@ -762,22 +953,6 @@ func parseRow(ctx *sql.Context, engine *gms.Engine, tableMap *mysql.TableMap, sc return parsedRow, nil } -// getSignedType returns a Vitess query.Type that can be used with the Vitess mysql.CellValue function to correctly -// parse the value of a signed or unsigned integer value. The mysql.TableMap structure provides information about the -// type, but it doesn't indicate if an integer type is signed or unsigned, so we have to look at the column type in the -// replica's schema and then choose any signed/unsigned query.Type to pass into mysql.CellValue to instruct it whether -// to treat a value as signed or unsigned – the actual type does not matter, only the signed/unsigned property. -func getSignedType(column *sql.Column) vquery.Type { - switch column.Type.Type() { - case vquery.Type_UINT8, vquery.Type_UINT16, vquery.Type_UINT24, vquery.Type_UINT32, vquery.Type_UINT64: - // For any unsigned integer value, we just need to return any unsigned numeric type to signal to Vitess to treat - // the value as unsigned. The actual type returned doesn't matter – only the signed/unsigned property is used. - return vquery.Type_UINT64 - default: - return vquery.Type_INT64 - } -} - // convertSqlTypesValues converts a sqltypes.Value instance (from vitess) into a sql.Type value (for go-mysql-server). func convertSqlTypesValue(ctx *sql.Context, engine *gms.Engine, value sqltypes.Value, column *sql.Column) (interface{}, error) { if value.IsNull() { @@ -823,8 +998,6 @@ func convertSqlTypesValue(ctx *sql.Context, engine *gms.Engine, value sqltypes.V // TODO: Consider moving this into DecimalType_.Convert; if DecimalType_.Convert handled trimming // leading/trailing whitespace, this special case for Decimal types wouldn't be needed. convertedValue, _, err = column.Type.Convert(strings.TrimSpace(value.ToString())) - case types.IsJSON(column.Type): - convertedValue, err = convertVitessJsonExpressionString(ctx, engine, value) case types.IsTimespan(column.Type): convertedValue, _, err = column.Type.Convert(value.ToString()) if err != nil { @@ -833,7 +1006,11 @@ func convertSqlTypesValue(ctx *sql.Context, engine *gms.Engine, value sqltypes.V convertedValue = convertedValue.(types.Timespan).String() default: convertedValue, _, err = column.Type.Convert(value.ToString()) - // logrus.WithField("column", column.Name).WithField("type", column.Type).Infof("Converting value[%s %v %s] to %v", value.Type(), value.Raw(), value.ToString(), convertedValue) + + // logrus.WithField("column", column.Name).WithField("type", column.Type).Infof( + // "Converting value[%s %v %s] to %v %T", + // value.Type(), value.Raw(), value.ToString(), convertedValue, convertedValue, + // ) } if err != nil { return nil, fmt.Errorf("unable to convert value %q, for column of type %T: %v", value.ToString(), column.Type, err.Error()) @@ -842,45 +1019,6 @@ func convertSqlTypesValue(ctx *sql.Context, engine *gms.Engine, value sqltypes.V return convertedValue, nil } -// convertVitessJsonExpressionString extracts a JSON value from the specified |value| instance, which Vitess has -// encoded as a SQL expression string. Vitess parses the binary JSON representation from an incoming binlog event, -// and converts it into an expression string containing JSON_OBJECT and JSON_ARRAY function calls. Because we don't -// have access to the raw JSON string or JSON bytes, we have to do extra work to translate from Vitess' SQL -// expression syntax into a raw JSON string value that we can pass to the storage layer. If Vitess kept around the -// raw string representation and returned it from value.ToString, this logic would not be necessary. -func convertVitessJsonExpressionString(ctx *sql.Context, engine *gms.Engine, value sqltypes.Value) (interface{}, error) { - if value.Type() != vquery.Type_EXPRESSION { - return nil, fmt.Errorf("invalid sqltypes.Value specified; expected a Value instance with an Expression type") - } - - strValue := value.String() - if strings.HasPrefix(strValue, "EXPRESSION(") { - strValue = strValue[len("EXPRESSION(") : len(strValue)-1] - } - - binder := planbuilder.New(ctx, engine.Analyzer.Catalog, engine.Parser) - node, _, _, qFlags, err := binder.Parse("SELECT "+strValue, false) - if err != nil { - return nil, err - } - - analyze, err := engine.Analyzer.Analyze(ctx, node, nil, qFlags) - if err != nil { - return nil, err - } - - rowIter, err := rowexec.DefaultBuilder.Build(ctx, analyze, nil) - if err != nil { - return nil, err - } - row, err := rowIter.Next(ctx) - if err != nil { - return nil, err - } - - return row[0], nil -} - func getAllUserDatabaseNames(ctx *sql.Context, engine *gms.Engine) []string { allDatabases := engine.Analyzer.Catalog.AllDatabases(ctx) userDatabaseNames := make([]string, 0, len(allDatabases)) @@ -941,7 +1079,7 @@ func executeQueryWithEngine(ctx *sql.Context, engine *gms.Engine, query string) "query": query, }).Errorf("Applying query failed") msg := fmt.Sprintf("Applying query failed: %v", err.Error()) - MyBinlogReplicaController.setSqlError(mysql.ERUnknownError, msg) + MyBinlogReplicaController.setSqlError(sqlerror.ERUnknownError, msg) } return } diff --git a/binlogreplication/binlog_replica_controller.go b/binlogreplication/binlog_replica_controller.go index 0c30e2d2..3a760792 100644 --- a/binlogreplication/binlog_replica_controller.go +++ b/binlogreplication/binlog_replica_controller.go @@ -22,6 +22,7 @@ import ( "time" "github.com/sirupsen/logrus" + "vitess.io/vitess/go/mysql/sqlerror" sqle "github.com/dolthub/go-mysql-server" "github.com/dolthub/go-mysql-server/sql" @@ -356,7 +357,7 @@ func (d *myBinlogReplicaController) GetReplicaStatus(ctx *sql.Context) (*binlogr copy.ReplicateDoTables = d.filters.getDoTables() copy.ReplicateIgnoreTables = d.filters.getIgnoreTables() - if d.applier.currentPosition != nil { + if !d.applier.currentPosition.IsZero() { copy.ExecutedGtidSet = d.applier.currentPosition.GTIDSet.String() copy.RetrievedGtidSet = copy.ExecutedGtidSet } @@ -405,7 +406,7 @@ func (d *myBinlogReplicaController) updateStatus(f func(status *binlogreplicatio } // setIoError updates the current replication status with the specific |errno| and |message| to describe an IO error. -func (d *myBinlogReplicaController) setIoError(errno uint, message string) { +func (d *myBinlogReplicaController) setIoError(errno sqlerror.ErrorCode, message string) { d.statusMutex.Lock() defer d.statusMutex.Unlock() @@ -416,12 +417,12 @@ func (d *myBinlogReplicaController) setIoError(errno uint, message string) { currentTime := time.Now() d.status.LastIoErrorTimestamp = ¤tTime - d.status.LastIoErrNumber = errno + d.status.LastIoErrNumber = uint(errno) d.status.LastIoError = message } // setSqlError updates the current replication status with the specific |errno| and |message| to describe an SQL error. -func (d *myBinlogReplicaController) setSqlError(errno uint, message string) { +func (d *myBinlogReplicaController) setSqlError(errno sqlerror.ErrorCode, message string) { d.statusMutex.Lock() defer d.statusMutex.Unlock() @@ -432,7 +433,7 @@ func (d *myBinlogReplicaController) setSqlError(errno uint, message string) { currentTime := time.Now() d.status.LastSqlErrorTimestamp = ¤tTime - d.status.LastSqlErrNumber = errno + d.status.LastSqlErrNumber = uint(errno) d.status.LastSqlError = message } diff --git a/binlogreplication/binlog_replica_event_producer.go b/binlogreplication/binlog_replica_event_producer.go index 34f45eb3..fdaf6e65 100644 --- a/binlogreplication/binlog_replica_event_producer.go +++ b/binlogreplication/binlog_replica_event_producer.go @@ -18,7 +18,7 @@ import ( "sync/atomic" "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/vitess/go/mysql" + "vitess.io/vitess/go/mysql" ) // binlogEventProducer is responsible for reading binlog events from an established connection and sending them back to @@ -70,7 +70,7 @@ func (p *binlogEventProducer) Go(_ *sql.Context) { // If this binlogEventProducer has been stopped while we were blocked waiting to read the next // binlog event, abort processing it and just return instead. - if p.IsRunning() == false { + if !p.IsRunning() { break } diff --git a/binlogreplication/binlog_replica_filtering.go b/binlogreplication/binlog_replica_filtering.go index 5f115bc6..46dd56bb 100644 --- a/binlogreplication/binlog_replica_filtering.go +++ b/binlogreplication/binlog_replica_filtering.go @@ -20,7 +20,7 @@ import ( "sync" "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/vitess/go/mysql" + "vitess.io/vitess/go/mysql" ) // filterConfiguration defines the binlog filtering rules applied on the replica. diff --git a/binlogreplication/binlog_replication_test.go b/binlogreplication/binlog_replication_test.go index 3b50524c..1c508e9c 100644 --- a/binlogreplication/binlog_replication_test.go +++ b/binlogreplication/binlog_replication_test.go @@ -121,7 +121,7 @@ func TestBinlogReplicationSanityCheck(t *testing.T) { // Insert/Update/Delete on the primary primaryDatabase.MustExec("insert into tableT values(100), (200)") waitForReplicaToCatchUp(t) - requireReplicaResults(t, "select * from db01.tableT", [][]any{{"100"}, {"200"}}) + requireReplicaResults(t, "select * from db01.tableT order by pk", [][]any{{"100"}, {"200"}}) primaryDatabase.MustExec("delete from tableT where pk = 100") waitForReplicaToCatchUp(t) requireReplicaResults(t, "select * from db01.tableT", [][]any{{"200"}}) @@ -237,7 +237,7 @@ func TestFlushLogs(t *testing.T) { primaryDatabase.MustExec("insert into t values (1), (2), (3);") waitForReplicaToCatchUp(t) - requireReplicaResults(t, "select * from db01.t;", [][]any{ + requireReplicaResults(t, "select * from db01.t order by pk;", [][]any{ {"1"}, {"2"}, {"3"}, }) } diff --git a/binlogreplication/writer.go b/binlogreplication/writer.go index 096b3205..b9c6acf7 100644 --- a/binlogreplication/writer.go +++ b/binlogreplication/writer.go @@ -1,24 +1,29 @@ package binlogreplication import ( + "github.com/apache/arrow/go/v17/arrow/array" + "github.com/apecloud/myduckserver/binlog" sqle "github.com/dolthub/go-mysql-server" "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/vitess/go/mysql" -) - -type EventType int - -const ( - InsertEvent EventType = iota - DeleteEvent - UpdateEvent + "vitess.io/vitess/go/mysql" ) type TableWriter interface { Insert(ctx *sql.Context, keyRows []sql.Row) error Delete(ctx *sql.Context, keyRows []sql.Row) error Update(ctx *sql.Context, keyRows []sql.Row, valueRows []sql.Row) error - Close() error + Commit() error + Rollback() error +} + +type DeltaAppender interface { + Field(i int) array.Builder + Fields() []array.Builder + Action() *array.Int8Builder + TxnTag() *array.BinaryDictionaryBuilder + TxnServer() *array.BinaryDictionaryBuilder + TxnGroup() *array.BinaryDictionaryBuilder + TxnSeqNumber() *array.Uint64Builder } type TableWriterProvider interface { @@ -26,10 +31,20 @@ type TableWriterProvider interface { GetTableWriter( ctx *sql.Context, engine *sqle.Engine, databaseName, tableName string, - schema sql.Schema, + schema sql.PrimaryKeySchema, columnCount, rowCount int, identifyColumns, dataColumns mysql.Bitmap, - eventType EventType, + eventType binlog.RowEventType, foreignKeyChecksDisabled bool, ) (TableWriter, error) + + // GetDeltaAppender returns an ArrowAppender for appending updates to the specified |table| in the specified |database|. + GetDeltaAppender( + ctx *sql.Context, engine *sqle.Engine, + databaseName, tableName string, + schema sql.Schema, + ) (DeltaAppender, error) + + // FlushDelta writes the accumulated changes to the database. + FlushDelta(ctx *sql.Context) error } diff --git a/catalog/index.go b/catalog/index.go index ecb842fb..08d76ed8 100644 --- a/catalog/index.go +++ b/catalog/index.go @@ -106,6 +106,13 @@ func (idx *Index) CanSupport(ranges ...sql.Range) bool { return true } +// CanSupportOrderBy returns whether this index can optimize ORDER BY a given expression type. +// Verifying that the expression's children match the index columns are done separately. +func (idx *Index) CanSupportOrderBy(expr sql.Expression) bool { + // Assuming true as default + return true +} + // PrefixLengths returns the prefix lengths for each column in this index func (idx *Index) PrefixLengths() []uint16 { return idx.PrefixLens diff --git a/catalog/provider.go b/catalog/provider.go index 0366cf59..ebe312f5 100644 --- a/catalog/provider.go +++ b/catalog/provider.go @@ -1,6 +1,7 @@ package catalog import ( + "context" "fmt" "path/filepath" "sort" @@ -10,6 +11,7 @@ import ( stdsql "database/sql" "github.com/dolthub/go-mysql-server/sql" + "github.com/marcboeker/go-duckdb" _ "github.com/marcboeker/go-duckdb" "github.com/apecloud/myduckserver/adapter" @@ -18,6 +20,7 @@ import ( type DatabaseProvider struct { mu *sync.RWMutex + connector *duckdb.Connector storage *stdsql.DB catalogName string dataDir string @@ -49,24 +52,28 @@ func NewDBProvider(dataDir, dbFile string) (*DatabaseProvider, error) { dsn = filepath.Join(dataDir, dbFile) } - storage, err := stdsql.Open("duckdb", dsn) + connector, err := duckdb.NewConnector(dsn, nil) if err != nil { return nil, err } - // install the json extension - _, err = storage.Exec("INSTALL json") - if err != nil { - return nil, err + storage := stdsql.OpenDB(connector) + + bootQueries := []string{ + "INSTALL arrow", + "LOAD arrow", } - // load the json extension - _, err = storage.Exec("LOAD json") - if err != nil { - return nil, err + for _, q := range bootQueries { + if _, err := storage.ExecContext(context.Background(), q); err != nil { + storage.Close() + connector.Close() + return nil, fmt.Errorf("failed to execute boot query %q: %v", q, err) + } } return &DatabaseProvider{ mu: &sync.RWMutex{}, + connector: connector, storage: storage, catalogName: name, dataDir: dataDir, @@ -75,9 +82,14 @@ func NewDBProvider(dataDir, dbFile string) (*DatabaseProvider, error) { } func (prov *DatabaseProvider) Close() error { + defer prov.connector.Close() return prov.storage.Close() } +func (prov *DatabaseProvider) Connector() *duckdb.Connector { + return prov.connector +} + func (prov *DatabaseProvider) Storage() *stdsql.DB { return prov.storage } diff --git a/charset/charset.go b/charset/charset.go index 2d1fc069..749d1288 100644 --- a/charset/charset.go +++ b/charset/charset.go @@ -5,6 +5,7 @@ import ( "fmt" "github.com/dolthub/go-mysql-server/sql" + "golang.org/x/text/encoding" "golang.org/x/text/encoding/charmap" "golang.org/x/text/encoding/simplifiedchinese" "golang.org/x/text/encoding/traditionalchinese" @@ -41,54 +42,72 @@ func IsSupportedNonUTF8(id sql.CharacterSetID) bool { return IsSupported(id) && !IsUTF8(id) } -func Decode(id sql.CharacterSetID, encoded string) (string, error) { +func getEncoding(id sql.CharacterSetID) (encoding.Encoding, error) { switch id { case sql.CharacterSet_Unspecified, sql.CharacterSet_ascii, sql.CharacterSet_utf8mb3, sql.CharacterSet_utf8mb4: - return encoded, nil + return encoding.Nop, nil case sql.CharacterSet_latin1: // https://dev.mysql.com/doc/refman/8.4/en/charset-we-sets.html // > MySQL's latin1 is the same as the Windows cp1252 character set. - return charmap.Windows1252.NewDecoder().String(encoded) + return charmap.Windows1252, nil case sql.CharacterSet_ucs2, sql.CharacterSet_utf16: // https://dev.mysql.com/doc/refman/8.4/en/charset-unicode-utf16.html - return unicode.UTF16(unicode.BigEndian, unicode.IgnoreBOM).NewDecoder().String(encoded) + return unicode.UTF16(unicode.BigEndian, unicode.IgnoreBOM), nil case sql.CharacterSet_utf16le: // https://dev.mysql.com/doc/refman/8.4/en/charset-unicode-utf16le.html - return unicode.UTF16(unicode.LittleEndian, unicode.IgnoreBOM).NewDecoder().String(encoded) + return unicode.UTF16(unicode.LittleEndian, unicode.IgnoreBOM), nil case sql.CharacterSet_utf32: // https://dev.mysql.com/doc/refman/8.4/en/charset-unicode-utf32.html - return utf32.UTF32(utf32.BigEndian, utf32.IgnoreBOM).NewDecoder().String(encoded) + return utf32.UTF32(utf32.BigEndian, utf32.IgnoreBOM), nil // https://dev.mysql.com/doc/refman/8.4/en/faqs-cjk.html case sql.CharacterSet_gb2312, sql.CharacterSet_gbk: - return simplifiedchinese.GBK.NewDecoder().String(encoded) + return simplifiedchinese.GBK, nil case sql.CharacterSet_gb18030: - return simplifiedchinese.GB18030.NewDecoder().String(encoded) + return simplifiedchinese.GB18030, nil case sql.CharacterSet_big5: - return traditionalchinese.Big5.NewDecoder().String(encoded) + return traditionalchinese.Big5, nil } - return encoded, fmt.Errorf("%s: %w", id, ErrUnsupported) + return encoding.Nop, fmt.Errorf("%s: %w", id, ErrUnsupported) } func Encode(id sql.CharacterSetID, utf8 string) (string, error) { - switch id { - case sql.CharacterSet_Unspecified, sql.CharacterSet_ascii, sql.CharacterSet_utf8mb3, sql.CharacterSet_utf8mb4: + en, err := getEncoding(id) + if err != nil { + return utf8, err + } else if en == encoding.Nop { // allocation-free fast path return utf8, nil - case sql.CharacterSet_latin1: - return charmap.Windows1252.NewEncoder().String(utf8) - case sql.CharacterSet_ucs2, sql.CharacterSet_utf16: - return unicode.UTF16(unicode.BigEndian, unicode.IgnoreBOM).NewEncoder().String(utf8) - case sql.CharacterSet_utf16le: - return unicode.UTF16(unicode.LittleEndian, unicode.IgnoreBOM).NewEncoder().String(utf8) - case sql.CharacterSet_utf32: - return utf32.UTF32(utf32.BigEndian, utf32.IgnoreBOM).NewEncoder().String(utf8) - case sql.CharacterSet_gb2312, sql.CharacterSet_gbk: - return simplifiedchinese.GBK.NewEncoder().String(utf8) - case sql.CharacterSet_gb18030: - return simplifiedchinese.GB18030.NewEncoder().String(utf8) - case sql.CharacterSet_big5: - return traditionalchinese.Big5.NewEncoder().String(utf8) } - return utf8, fmt.Errorf("%s: %w", id, ErrUnsupported) + return en.NewEncoder().String(utf8) +} + +func Decode(id sql.CharacterSetID, encoded string) (string, error) { + en, err := getEncoding(id) + if err != nil { + return encoded, err + } else if en == encoding.Nop { + return encoded, nil + } + return en.NewDecoder().String(encoded) +} + +func EncodeBytes(id sql.CharacterSetID, utf8 []byte) ([]byte, error) { + en, err := getEncoding(id) + if err != nil { + return utf8, err + } else if en == encoding.Nop { + return utf8, nil + } + return en.NewEncoder().Bytes(utf8) +} + +func DecodeBytes(id sql.CharacterSetID, encoded []byte) ([]byte, error) { + en, err := getEncoding(id) + if err != nil { + return encoded, err + } else if en == encoding.Nop { + return encoded, nil + } + return en.NewDecoder().Bytes(encoded) } diff --git a/go.mod b/go.mod index c5af6a0c..426c5238 100644 --- a/go.mod +++ b/go.mod @@ -1,68 +1,78 @@ module github.com/apecloud/myduckserver -go 1.23.1 +go 1.22.7 require ( github.com/Shopify/toxiproxy/v2 v2.9.0 - github.com/dolthub/go-mysql-server v0.18.2-0.20240815142344-761713e36043 - github.com/dolthub/vitess v0.0.0-20240807181005-71d735078e24 + github.com/apache/arrow/go/v17 v17.0.0 + github.com/cockroachdb/apd/v3 v3.2.1 + github.com/dolthub/go-mysql-server v0.18.2-0.20240926171723-77ed13c03196 + github.com/dolthub/vitess v0.0.0-20240919225659-2ad81685e772 github.com/go-sql-driver/mysql v1.8.1 github.com/jmoiron/sqlx v1.4.0 - github.com/marcboeker/go-duckdb v1.8.0 - github.com/prometheus/client_golang v1.19.0 + github.com/marcboeker/go-duckdb v1.8.1 + github.com/prometheus/client_golang v1.20.3 github.com/rs/zerolog v1.33.0 github.com/shopspring/decimal v1.3.1 github.com/sirupsen/logrus v1.8.1 github.com/stretchr/testify v1.9.0 - golang.org/x/text v0.16.0 + golang.org/x/text v0.18.0 gopkg.in/src-d/go-errors.v1 v1.0.0 + vitess.io/vitess v0.20.2 +) + +replace ( + github.com/dolthub/go-mysql-server v0.18.2-0.20240926171723-77ed13c03196 => github.com/fanyang01/go-mysql-server v0.0.0-20240927093603-e7d5b2c91bf7 + github.com/dolthub/vitess v0.0.0-20240919225659-2ad81685e772 => github.com/apecloud/dolt-vitess v0.0.0-20240927100428-4ba1490cf5da ) require ( filippo.io/edwards25519 v1.1.0 // indirect + github.com/AdaLogics/go-fuzz-headers v0.0.0-20230811130428-ced1acdcaa24 // indirect github.com/DATA-DOG/go-sqlmock v1.5.2 // indirect - github.com/apache/arrow/go/v17 v17.0.0 // indirect github.com/beorn7/perks v1.0.1 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect - github.com/davecgh/go-spew v1.1.1 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect + github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/dolthub/flatbuffers/v23 v23.3.3-dh.2 // indirect - github.com/dolthub/go-icu-regex v0.0.0-20230524105445-af7e7991c97e // indirect + github.com/dolthub/go-icu-regex v0.0.0-20240916130659-0118adc6b662 // indirect github.com/dolthub/jsonpath v0.0.2-0.20240227200619-19675ab05c71 // indirect github.com/dolthub/sqllogictest/go v0.0.0-20240618184124-ca47f9354216 // indirect github.com/go-kit/kit v0.10.0 // indirect github.com/goccy/go-json v0.10.3 // indirect github.com/gocraft/dbr/v2 v2.7.2 // indirect + github.com/golang/glog v1.2.2 // indirect github.com/google/flatbuffers v24.3.25+incompatible // indirect github.com/google/uuid v1.6.0 // indirect github.com/gorilla/mux v1.8.1 // indirect - github.com/hashicorp/golang-lru v0.5.4 // indirect + github.com/hashicorp/golang-lru v1.0.2 // indirect github.com/klauspost/compress v1.17.9 // indirect github.com/klauspost/cpuid/v2 v2.2.8 // indirect github.com/lestrrat-go/strftime v1.0.4 // indirect github.com/mattn/go-colorable v0.1.13 // indirect github.com/mattn/go-isatty v0.0.20 // indirect github.com/mitchellh/mapstructure v1.5.0 // indirect + github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/pierrec/lz4/v4 v4.1.21 // indirect + github.com/pires/go-proxyproto v0.7.0 // indirect github.com/pkg/errors v0.9.1 // indirect - github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_model v0.6.0 // indirect - github.com/prometheus/common v0.50.0 // indirect - github.com/prometheus/procfs v0.13.0 // indirect - github.com/rogpeppe/go-internal v1.11.0 // indirect + github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/common v0.59.1 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/rs/xid v1.5.0 // indirect + github.com/spf13/pflag v1.0.5 // indirect github.com/tetratelabs/wazero v1.1.0 // indirect github.com/zeebo/xxh3 v1.0.2 // indirect - go.opentelemetry.io/otel v1.7.0 // indirect - go.opentelemetry.io/otel/trace v1.7.0 // indirect - golang.org/x/crypto v0.23.0 // indirect - golang.org/x/exp v0.0.0-20240222234643-814bf88cf225 // indirect - golang.org/x/mod v0.18.0 // indirect - golang.org/x/sync v0.7.0 // indirect - golang.org/x/sys v0.21.0 // indirect - golang.org/x/tools v0.22.0 // indirect - golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240227224415-6ceb2ff114de // indirect - google.golang.org/grpc v1.63.2 // indirect + go.opentelemetry.io/otel v1.30.0 // indirect + go.opentelemetry.io/otel/trace v1.30.0 // indirect + golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 // indirect + golang.org/x/mod v0.21.0 // indirect + golang.org/x/sync v0.8.0 // indirect + golang.org/x/sys v0.25.0 // indirect + golang.org/x/tools v0.25.0 // indirect + golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1 // indirect + google.golang.org/grpc v1.66.2 // indirect google.golang.org/protobuf v1.34.2 // indirect gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index 6d05b938..228e3a0c 100644 --- a/go.sum +++ b/go.sum @@ -2,6 +2,8 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMT cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= filippo.io/edwards25519 v1.1.0 h1:FNf4tywRC1HmFuKW5xopWpigGjJKiJSV0Cqo0cJWDaA= filippo.io/edwards25519 v1.1.0/go.mod h1:BxyFTGdWcka3PhytdK4V28tE5sGfRvvvRV7EaN4VDT4= +github.com/AdaLogics/go-fuzz-headers v0.0.0-20230811130428-ced1acdcaa24 h1:bvDV9vkmnHYOMsOr4WLk+Vo07yKIzd94sVoIqshQ4bU= +github.com/AdaLogics/go-fuzz-headers v0.0.0-20230811130428-ced1acdcaa24/go.mod h1:8o94RPi1/7XTJvwPpRSzSUedZrtlirdB3r9Z20bi2f8= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/DATA-DOG/go-sqlmock v1.5.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= github.com/DATA-DOG/go-sqlmock v1.5.2 h1:OcvFkGmslmlZibjAjaHm3L//6LiuBgolP7OputlJIzU= @@ -22,6 +24,10 @@ github.com/apache/arrow/go/v17 v17.0.0 h1:RRR2bdqKcdbss9Gxy2NS/hK8i4LDMh23L6BbkN github.com/apache/arrow/go/v17 v17.0.0/go.mod h1:jR7QHkODl15PfYyjM2nU+yTLScZ/qfj7OSUZmJ8putc= github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= github.com/apache/thrift v0.13.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= +github.com/apecloud/dolt-vitess v0.0.0-20240919225659-2ad81685e772 h1:OgHbYQJXAEqDGjuRFMdELNBRoxNMDS+NbcU9umOZ7as= +github.com/apecloud/dolt-vitess v0.0.0-20240919225659-2ad81685e772/go.mod h1:uBvlRluuL+SbEWTCZ68o0xvsdYZER3CEG/35INdzfJM= +github.com/apecloud/dolt-vitess v0.0.0-20240927100428-4ba1490cf5da h1:+sOwYwbN/kZZd0Ggsz+ozKa6gdAUYz/bgVMJkoDmuMc= +github.com/apecloud/dolt-vitess v0.0.0-20240927100428-4ba1490cf5da/go.mod h1:uBvlRluuL+SbEWTCZ68o0xvsdYZER3CEG/35INdzfJM= github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= @@ -38,10 +44,12 @@ github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec/go.mod h1:jMjuTZXRI4dUb/I5gc9Hdhagfvm9+RyrPryS/auMzxE= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cockroachdb/apd/v3 v3.2.1 h1:U+8j7t0axsIgvQUqthuNm82HIrYXodOV2iWLWtEaIwg= +github.com/cockroachdb/apd/v3 v3.2.1/go.mod h1:klXJcjp+FffLTHlhIG69tezTDvdP065naDsHzKhYSqc= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= @@ -51,23 +59,28 @@ github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfc github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/denisenkom/go-mssqldb v0.10.0 h1:QykgLZBorFE95+gO3u9esLd0BmbvpWp0/waNNZfHBM8= github.com/denisenkom/go-mssqldb v0.10.0/go.mod h1:xbL0rPBG9cCiLr28tMa8zpbdarY27NDyej4t/EjAShU= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dolthub/flatbuffers/v23 v23.3.3-dh.2 h1:u3PMzfF8RkKd3lB9pZ2bfn0qEG+1Gms9599cr0REMww= github.com/dolthub/flatbuffers/v23 v23.3.3-dh.2/go.mod h1:mIEZOHnFx4ZMQeawhw9rhsj+0zwQj7adVsnBX7t+eKY= -github.com/dolthub/go-icu-regex v0.0.0-20230524105445-af7e7991c97e h1:kPsT4a47cw1+y/N5SSCkma7FhAPw7KeGmD6c9PBZW9Y= -github.com/dolthub/go-icu-regex v0.0.0-20230524105445-af7e7991c97e/go.mod h1:KPUcpx070QOfJK1gNe0zx4pA5sicIK1GMikIGLKC168= -github.com/dolthub/go-mysql-server v0.18.2-0.20240815142344-761713e36043 h1:KgrDVE4o4Y04XLnAs5BGv6I6z+Rd82FWntCbQEmbTKs= -github.com/dolthub/go-mysql-server v0.18.2-0.20240815142344-761713e36043/go.mod h1:nbdOzd0ceWONE80vbfwoRBjut7z3CIj69ZgDF/cKuaA= +github.com/dolthub/go-icu-regex v0.0.0-20240916130659-0118adc6b662 h1:aC17hZD6iwzBwwfO5M+3oBT5E5gGRiQPdn+vzpDXqIA= +github.com/dolthub/go-icu-regex v0.0.0-20240916130659-0118adc6b662/go.mod h1:KPUcpx070QOfJK1gNe0zx4pA5sicIK1GMikIGLKC168= +github.com/dolthub/go-mysql-server v0.18.1 h1:T+mTBfLrZPnOKvVx3iRx66f0oW+0saOnPa+O1OKUklQ= +github.com/dolthub/go-mysql-server v0.18.1/go.mod h1:8zjK76NDWRel1CFdg+DDzy/D5tdOeFOYKBcqf7IB+aA= +github.com/dolthub/go-mysql-server v0.18.2-0.20240923181307-5aacdb13e45a h1:rpCmZj332eiBbzsHsq3Sj5AWzl3Q7szDObwI49UqA8Y= +github.com/dolthub/go-mysql-server v0.18.2-0.20240923181307-5aacdb13e45a/go.mod h1:lGbU2bK+QNnlETdUjOOaE+UnlEUu31VaQOFKAFGyZN4= +github.com/dolthub/go-mysql-server v0.18.2-0.20240926171723-77ed13c03196 h1:H4bKFiOdjmhBrdjrNvYAuhfplpHM3aVFcbLXlGoD/Fc= +github.com/dolthub/go-mysql-server v0.18.2-0.20240926171723-77ed13c03196/go.mod h1:lGbU2bK+QNnlETdUjOOaE+UnlEUu31VaQOFKAFGyZN4= github.com/dolthub/jsonpath v0.0.2-0.20240227200619-19675ab05c71 h1:bMGS25NWAGTEtT5tOBsCuCrlYnLRKpbJVJkDbrTRhwQ= github.com/dolthub/jsonpath v0.0.2-0.20240227200619-19675ab05c71/go.mod h1:2/2zjLQ/JOOSbbSboojeg+cAwcRV0fDLzIiWch/lhqI= github.com/dolthub/sqllogictest/go v0.0.0-20240618184124-ca47f9354216 h1:JWkKRE4EHUcEVQCMRBej8DYxjYjRz/9MdF/NNQh0o70= github.com/dolthub/sqllogictest/go v0.0.0-20240618184124-ca47f9354216/go.mod h1:e/FIZVvT2IR53HBCAo41NjqgtEnjMJGKca3Y/dAmZaA= -github.com/dolthub/vitess v0.0.0-20240807181005-71d735078e24 h1:/zCd98CLZURqK85jQ+qRmEMx/dpXz85F1/Et7gqMGkk= -github.com/dolthub/vitess v0.0.0-20240807181005-71d735078e24/go.mod h1:uBvlRluuL+SbEWTCZ68o0xvsdYZER3CEG/35INdzfJM= +github.com/dolthub/vitess v0.0.0-20240919225659-2ad81685e772 h1:vDwBX7Lc8DnA8Zk0iRIu6slCw0GIUfYfFlYDYJQw8GQ= +github.com/dolthub/vitess v0.0.0-20240919225659-2ad81685e772/go.mod h1:uBvlRluuL+SbEWTCZ68o0xvsdYZER3CEG/35INdzfJM= github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/eapache/go-resiliency v1.1.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= @@ -76,6 +89,8 @@ github.com/edsrzf/mmap-go v1.0.0/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaB github.com/envoyproxy/go-control-plane v0.6.9/go.mod h1:SBwIajubJHhxtWwsL9s8ss4safvEdbitLhGGK48rN6g= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/fanyang01/go-mysql-server v0.0.0-20240927093603-e7d5b2c91bf7 h1:QSqlxTk6pjF/11KV2JsBNGLmIlMccdzpUkJmNp1lsTs= +github.com/fanyang01/go-mysql-server v0.0.0-20240927093603-e7d5b2c91bf7/go.mod h1:GYFkohqx2Nr8NNjcEwPV1XsALRl3/l0eKhMTVqUiPmM= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4= github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20= @@ -88,8 +103,6 @@ github.com/go-kit/kit v0.10.0/go.mod h1:xUsJbQ/Fp4kEt7AFgCuvyX4a71u8h9jB8tj/ORgO github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= -github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= -github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-sql-driver/mysql v1.4.1/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= @@ -110,6 +123,8 @@ github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zV github.com/golang-sql/civil v0.0.0-20190719163853-cb61b32ac6fe h1:lXe2qZdvpiX5WZkZR4hgp4KJVfY3nMkvmwbVkpv1rVY= github.com/golang-sql/civil v0.0.0-20190719163853-cb61b32ac6fe/go.mod h1:8vg3r2VgvsThLBIFL93Qb5yWzgyZWhEmBwUJWevAkK0= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/glog v1.2.2 h1:1+mZ9upx1Dh6FmUTFR1naJ77miKiXgALjWOZ3NVFPmY= +github.com/golang/glog v1.2.2/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= @@ -124,7 +139,6 @@ github.com/google/flatbuffers v24.3.25+incompatible/go.mod h1:1AeVuKshWv4vARoZat github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= @@ -158,8 +172,8 @@ github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09 github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/golang-lru v0.5.4 h1:YDjusn29QI/Das2iO9M0BHnIbxPeyuCHsjMW+lJfyTc= -github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= +github.com/hashicorp/golang-lru v1.0.2 h1:dV3g9Z/unq5DpblPpw+Oqcv4dU/1omnb4Ok8iPY6p1c= +github.com/hashicorp/golang-lru v1.0.2/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= @@ -194,6 +208,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= +github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= github.com/lestrrat-go/envload v0.0.0-20180220234015-a3eb8ddeffcc h1:RKf14vYWi2ttpEmkA4aQ3j4u9dStX2t4M8UM6qqNsG8= github.com/lestrrat-go/envload v0.0.0-20180220234015-a3eb8ddeffcc/go.mod h1:kopuH9ugFRkIXf3YoqHKyrJ9YfUFsckUU9S7B+XP+is= github.com/lestrrat-go/strftime v1.0.4 h1:T1Rb9EPkAhgxKqbcMIPguPq8glqXTA1koF8n9BHElA8= @@ -207,6 +223,8 @@ github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0U github.com/lyft/protoc-gen-validate v0.0.13/go.mod h1:XbGvPuh87YZc5TdIa2/I4pLk0QoUACkjt2znoq26NVQ= github.com/marcboeker/go-duckdb v1.8.0 h1:iOWv1wTL0JIMqpyns6hCf5XJJI4fY6lmJNk+itx5RRo= github.com/marcboeker/go-duckdb v1.8.0/go.mod h1:2oV8BZv88S16TKGKM+Lwd0g7DX84x0jMxjTInThC8Is= +github.com/marcboeker/go-duckdb v1.8.1 h1:jQjvsN49PNZC9IJLCIMjfD3lMO0QERKNYeZwhyVA8UY= +github.com/marcboeker/go-duckdb v1.8.1/go.mod h1:2oV8BZv88S16TKGKM+Lwd0g7DX84x0jMxjTInThC8Is= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= @@ -236,6 +254,8 @@ github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg= github.com/nats-io/jwt v0.3.2/go.mod h1:/euKqTS1ZD+zzjYrY7pseZrTtWQSjujC7xjPc8wL6eU= @@ -268,43 +288,46 @@ github.com/pierrec/lz4 v1.0.2-0.20190131084431-473cd7ce01a1/go.mod h1:3/3N9NVKO0 github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4/v4 v4.1.21 h1:yOVMLb6qSIDP67pl/5F7RepeKYu/VmTyEXvuMI5d9mQ= github.com/pierrec/lz4/v4 v4.1.21/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pires/go-proxyproto v0.7.0 h1:IukmRewDQFWC7kfnb66CSomk2q/seBuilHBYFwyq0Hs= +github.com/pires/go-proxyproto v0.7.0/go.mod h1:Vz/1JPY/OACxWGQNIRY2BeyDmpoaWmEP40O9LbuiFR4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= -github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod h1:p2iRAGwDERtqlqzRXnrOVns+ignqQo//hLXqYxZYVNs= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= -github.com/prometheus/client_golang v1.19.0 h1:ygXvpU1AoN1MhdzckN+PyD9QJOSD4x7kmXYlnfbA6JU= -github.com/prometheus/client_golang v1.19.0/go.mod h1:ZRM9uEAypZakd+q/x7+gmsvXdURP+DABIEIjnmDdp+k= +github.com/prometheus/client_golang v1.20.3 h1:oPksm4K8B+Vt35tUhw6GbSNSgVlVSBH0qELP/7u83l4= +github.com/prometheus/client_golang v1.20.3/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.6.0 h1:k1v3CzpSRUTrKMppY35TLwPvxHqBu0bYgxZzqGIgaos= -github.com/prometheus/client_model v0.6.0/go.mod h1:NTQHnmxFpouOD0DpvP4XujX3CdOAGQPoaGhyTchlyt8= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= github.com/prometheus/common v0.2.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= -github.com/prometheus/common v0.50.0 h1:YSZE6aa9+luNa2da6/Tik0q0A5AbR+U003TItK57CPQ= -github.com/prometheus/common v0.50.0/go.mod h1:wHFBCEVWVmHMUpg7pYcOm2QUR/ocQdYSJVQJKnHc3xQ= +github.com/prometheus/common v0.59.1 h1:LXb1quJHWm1P6wq/U824uxYi4Sg0oGvNeUm1z5dJoX0= +github.com/prometheus/common v0.59.1/go.mod h1:GpWM7dewqmVYcd7SmRaiWVe9SSqjf0UrwnYnpEZNuT0= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= -github.com/prometheus/procfs v0.13.0 h1:GqzLlQyfsPbaEHaQkO7tbDlriv/4o5Hudv6OXHGKX7o= -github.com/prometheus/procfs v0.13.0/go.mod h1:cd4PFCR54QLnGKPaKGA6l+cfuNXtht43ZKY6tow0Y1g= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= -github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= -github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= +github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= +github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= github.com/rs/xid v1.5.0 h1:mKX4bl4iPYJtEIxp6CYiUuLQ/8DYMoz0PUdtGgMFRVc= github.com/rs/xid v1.5.0/go.mod h1:trrq9SKmegXys3aeAKXMUTdJsYXVwGY3RLcfgqegfbg= github.com/rs/zerolog v1.33.0 h1:1cU2KZkvPxNyfgEmhHAz/1A9Bz+llsdYzklWFzgp0r8= @@ -326,6 +349,8 @@ github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4k github.com/sony/gobreaker v0.4.1/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= +github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= +github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/streadway/amqp v0.0.0-20190404075320-75d898a42a94/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/streadway/amqp v0.0.0-20190827072141-edfb9018d271/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/streadway/handy v0.0.0-20190108123426-d5acb3125c2a/go.mod h1:qNTQ5P5JnDBl6z3cMAg/SywNDC5ABu5ApDIw6lUbRmI= @@ -335,7 +360,6 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/tetratelabs/wazero v1.1.0 h1:EByoAhC+QcYpwSZJSs/aV0uokxPwBgKxfiokSUwAknQ= @@ -353,12 +377,18 @@ go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mI go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= go.opencensus.io v0.20.2/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opentelemetry.io/otel v1.7.0 h1:Z2lA3Tdch0iDcrhJXDIlC94XE+bxok1F9B+4Lz/lGsM= -go.opentelemetry.io/otel v1.7.0/go.mod h1:5BdUoMIz5WEs0vt0CUEMtSSaTSHBBVwrhnz7+nrD5xk= -go.opentelemetry.io/otel/trace v1.7.0 h1:O37Iogk1lEkMRXewVtZ1BBTVn5JEp8GrJvP92bJqC6o= -go.opentelemetry.io/otel/trace v1.7.0/go.mod h1:fzLSB9nqR2eXzxPXb2JW9IKE+ScyXA48yyE4TNvoHqU= +go.opentelemetry.io/otel v1.27.0 h1:9BZoF3yMK/O1AafMiQTVu0YDj5Ea4hPhxCs7sGva+cg= +go.opentelemetry.io/otel v1.27.0/go.mod h1:DMpAK8fzYRzs+bi3rS5REupisuqTheUlSZJ1WnZaPAQ= +go.opentelemetry.io/otel v1.30.0 h1:F2t8sK4qf1fAmY9ua4ohFS/K+FUuOPemHUIXHtktrts= +go.opentelemetry.io/otel v1.30.0/go.mod h1:tFw4Br9b7fOS+uEao81PJjVMjW/5fvNCbpsDIXqP0pc= +go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5/Rscw= +go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= +go.opentelemetry.io/otel/trace v1.30.0 h1:7UBkkYzeg3C7kQX8VAidWh2biiQbtAKjyIML8dQ9wmc= +go.opentelemetry.io/otel/trace v1.30.0/go.mod h1:5EyKqTzzmyqB9bwtCCq6pDLktPK6fmGf/Dph+8VI02o= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= +go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= +go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= @@ -374,8 +404,8 @@ golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.23.0 h1:dIJU/v2J8Mdglj/8rJ6UUOM3Zc9zLZxVZwwxMooUSAI= golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20240222234643-814bf88cf225 h1:LfspQV/FYTatPTr/3HzIcmiUFH7PGP+OQ6mgDYo3yuQ= -golang.org/x/exp v0.0.0-20240222234643-814bf88cf225/go.mod h1:CxmFvTBINI24O/j8iY7H1xHzx2i4OsyguNBmN/uPtqc= +golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 h1:e66Fs6Z+fZTbFBAxKfP3PALWBtpfqks2bwGcexMxgtk= +golang.org/x/exp v0.0.0-20240909161429-701f63a606c0/go.mod h1:2TbTHSBQa924w8M6Xs1QcRcFwyucIwBGpK1p2f1YFFY= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= @@ -383,8 +413,8 @@ golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHl golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= -golang.org/x/mod v0.18.0 h1:5+9lSbEzPSdWkH32vYPBwEpX8KwDbM52Ud9xBUvNlb0= -golang.org/x/mod v0.18.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/mod v0.21.0 h1:vvrHzRwRfVKSiLrG+d4FMl/Qi4ukBCE6kZlTUkDYRT0= +golang.org/x/mod v0.21.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -401,8 +431,8 @@ golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.26.0 h1:soB7SVo0PWrY4vPW/+ay0jKDNScG2X9wFeYlXIvJsOQ= -golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE= +golang.org/x/net v0.29.0 h1:5ORfpBpCs4HzDYoodCDBbwHzdR5UrLBZ3sOnUJmFoHo= +golang.org/x/net v0.29.0/go.mod h1:gLkgy8jTGERgjzMic6DS9+SP0ajcu6Xu3Orq/SpETg0= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -411,8 +441,8 @@ golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -433,12 +463,12 @@ golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.21.0 h1:rF+pYz3DAGSQAxAu1CbC7catZg4ebC4UIeIhKxBZvws= -golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= +golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= -golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -454,13 +484,12 @@ golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgw golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200103221440-774c71fcf114/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.22.0 h1:gqSGLZqv+AI9lIQzniJ0nZDRG5GBPsSi+DRNHWNz6yA= -golang.org/x/tools v0.22.0/go.mod h1:aCwcsjqvq7Yqt6TNyX7QMU2enbQ/Gt0bo6krSeEri+c= +golang.org/x/tools v0.25.0 h1:oFU9pkj/iJgs+0DT+VMHrx+oBKs/LJMV+Uvg78sl+fE= +golang.org/x/tools v0.25.0/go.mod h1:/vtpO8WL1N9cQC3FN5zPqb//fRXskFHbLKk4OW1Q7rg= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 h1:+cNy6SZtPcJQH3LJVLOSmiC7MMxXNOb3PU/VUEz+EhU= -golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= +golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da h1:noIWHXmPHxILtqtCOPIhSt0ABwskkZKjD3bXGnZGpNY= +golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= gonum.org/v1/gonum v0.15.0 h1:2lYxjRbTYyxkJxlhC+LvJIx3SsANPdRybu1tGj9/OrQ= gonum.org/v1/gonum v0.15.0/go.mod h1:xzZVBJBtS+Mz4q0Yl2LJTk+OxOg4jiXZ7qBoM0uISGo= google.golang.org/api v0.3.1/go.mod h1:6wY9I6uQWHQ8EM57III9mq/AjF+i8G65rmVagqKMtkk= @@ -473,8 +502,8 @@ google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRn google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190530194941-fb225487d101/go.mod h1:z3L6/3dTEVtUr6QSP8miRzeRqwQOioJ9I66odjN4I7s= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240227224415-6ceb2ff114de h1:cZGRis4/ot9uVm639a+rHCUaG0JJHEsdyzSQTMX+suY= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240227224415-6ceb2ff114de/go.mod h1:H4O17MA/PE9BsGx3w+a+W2VOLLD1Qf7oJneAoU6WktY= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1 h1:pPJltXNxVzT4pK9yD8vR9X75DaWYYmLGMsEvBfFQZzQ= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.0/go.mod h1:chYK+tFQF0nDUGJgXMSgLCQk3phJEuONr2DCgLDdAQM= @@ -484,8 +513,8 @@ google.golang.org/grpc v1.22.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyac google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.63.2 h1:MUeiw1B2maTVZthpU5xvASfTh3LDbxHd6IJ6QQVU+xM= -google.golang.org/grpc v1.63.2/go.mod h1:WAX/8DgncnokcFUldAxq7GeB5DXHDbMF+lLvDomNkRA= +google.golang.org/grpc v1.66.2 h1:3QdXkuq3Bkh7w+ywLdLvM56cmGvQHUMZpiCzt6Rqaoo= +google.golang.org/grpc v1.66.2/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= @@ -515,3 +544,5 @@ honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= +vitess.io/vitess v0.20.2 h1:Nc9yW99lHND5R7nObvoTbMMtwqoK4NBsJ+7U3T4jx1E= +vitess.io/vitess v0.20.2/go.mod h1:1eORqYx9TnhEzknEH9AnzAT/9nlAHEsqA8QZmzNGgVM= diff --git a/harness/duck_harness.go b/harness/duck_harness.go index d6651f3d..96372029 100644 --- a/harness/duck_harness.go +++ b/harness/duck_harness.go @@ -297,7 +297,7 @@ func (e *DuckTestEngine) Close() error { func NewEngine(t *testing.T, harness enginetest.Harness, dbProvider sql.DatabaseProvider, setupData []setup.SetupScript, statsProvider sql.StatsProvider, server bool) (enginetest.QueryEngine, error) { // Create the connection pool first, as it is needed by `NewEngineWithProvider` provider := dbProvider.(*catalog.DatabaseProvider) - pool := backend.NewConnectionPool(provider.CatalogName(), provider.Storage()) + pool := backend.NewConnectionPool(provider.CatalogName(), provider.Connector(), provider.Storage()) harness.(*DuckHarness).pool = pool e := enginetest.NewEngineWithProvider(t, harness, dbProvider) diff --git a/main.go b/main.go index b8864fa0..d87ae4ac 100644 --- a/main.go +++ b/main.go @@ -21,6 +21,7 @@ import ( "github.com/apecloud/myduckserver/backend" "github.com/apecloud/myduckserver/catalog" + "github.com/apecloud/myduckserver/replica" "github.com/apecloud/myduckserver/transpiler" sqle "github.com/dolthub/go-mysql-server" "github.com/dolthub/go-mysql-server/server" @@ -69,7 +70,7 @@ func main() { } defer provider.Close() - pool := backend.NewConnectionPool(provider.CatalogName(), provider.Storage()) + pool := backend.NewConnectionPool(provider.CatalogName(), provider.Connector(), provider.Storage()) engine := sqle.NewDefault(provider) @@ -80,7 +81,7 @@ func main() { logrus.Fatalln("Failed to set the persister:", err) } - registerReplicaController(provider, engine, pool) + replica.RegisterReplicaController(provider, engine, pool) config := server.Config{ Protocol: "tcp", diff --git a/myarrow/appender.go b/myarrow/appender.go new file mode 100644 index 00000000..ee38ff62 --- /dev/null +++ b/myarrow/appender.go @@ -0,0 +1,100 @@ +package myarrow + +import ( + "fmt" + "time" + + "github.com/apache/arrow/go/v17/arrow" + "github.com/apache/arrow/go/v17/arrow/array" + "github.com/apache/arrow/go/v17/arrow/memory" + "github.com/dolthub/go-mysql-server/sql" + "github.com/dolthub/go-mysql-server/sql/types" + "github.com/shopspring/decimal" +) + +type ArrowAppender struct { + *array.RecordBuilder +} + +func NewArrowAppender(schema sql.Schema, dictionary ...int) (ArrowAppender, error) { + pool := memory.NewGoAllocator() + arrowSchema, err := ToArrowSchema(schema, dictionary...) + if err != nil { + return ArrowAppender{}, err + } + return ArrowAppender{array.NewRecordBuilder(pool, arrowSchema)}, nil +} + +// Build creates a new arrow.Record from the memory buffers and resets the builder. +// The returned Record must be Release()'d after use. +func (a *ArrowAppender) Build() arrow.Record { + return a.RecordBuilder.NewRecord() +} + +func (a *ArrowAppender) Append(row sql.Row) error { + for i, b := range a.RecordBuilder.Fields() { + v := row[i] + if v == nil { + b.AppendNull() + continue + } + switch b.Type().ID() { + case arrow.UINT8: + b.(*array.Uint8Builder).Append(v.(uint8)) + case arrow.INT8: + b.(*array.Int8Builder).Append(v.(int8)) + case arrow.UINT16: + b.(*array.Uint16Builder).Append(v.(uint16)) + case arrow.INT16: + b.(*array.Int16Builder).Append(v.(int16)) + case arrow.UINT32: + b.(*array.Uint32Builder).Append(v.(uint32)) + case arrow.INT32: + b.(*array.Int32Builder).Append(v.(int32)) + case arrow.UINT64: + b.(*array.Uint64Builder).Append(v.(uint64)) + case arrow.INT64: + b.(*array.Int64Builder).Append(v.(int64)) + case arrow.FLOAT32: + b.(*array.Float32Builder).Append(v.(float32)) + case arrow.FLOAT64: + b.(*array.Float64Builder).Append(v.(float64)) + case arrow.STRING: + b.(*array.StringBuilder).Append(v.(string)) + case arrow.BINARY: + b.(*array.BinaryBuilder).Append(v.([]byte)) + case arrow.DECIMAL: + dv := v.(decimal.Decimal) + b.AppendValueFromString(dv.String()) + case arrow.TIMESTAMP: + tv := v.(time.Time) + at, err := arrow.TimestampFromTime(tv, b.Type().(*arrow.TimestampType).Unit) + if err != nil { + return err + } + b.(*array.TimestampBuilder).Append(at) + case arrow.DATE32: + tv := v.(time.Time) + b.(*array.Date32Builder).Append(arrow.Date32FromTime(tv)) + case arrow.DURATION: + sv := v.(string) + duration, err := types.Time.ConvertToTimeDuration(sv) + if err != nil { + return err + } + b.(*array.DurationBuilder).Append(arrow.Duration(duration.Microseconds())) + case arrow.DICTIONARY: + switch v := v.(type) { + case string: + b.(*array.BinaryDictionaryBuilder).AppendString(v) + case []byte: + b.(*array.BinaryDictionaryBuilder).Append(v) + default: + b.AppendValueFromString(fmt.Sprint(v)) + } + default: + b.AppendValueFromString(fmt.Sprint(v)) + } + } + return nil +} diff --git a/myarrow/schema.go b/myarrow/schema.go new file mode 100644 index 00000000..6a8e03a5 --- /dev/null +++ b/myarrow/schema.go @@ -0,0 +1,129 @@ +package myarrow + +import ( + "github.com/apache/arrow/go/v17/arrow" + "github.com/dolthub/go-mysql-server/sql" + "github.com/dolthub/vitess/go/vt/proto/query" +) + +func ToArrowSchema(s sql.Schema, dictionary ...int) (*arrow.Schema, error) { + fields := make([]arrow.Field, len(s)) + for i, col := range s { + at, err := ToArrowType(col.Type) + if err != nil { + return nil, err + } + fields[i] = arrow.Field{ + Name: col.Name, + Type: at, + Nullable: col.Nullable, + } + } + for _, i := range dictionary { + fields[i].Type = &arrow.DictionaryType{ + IndexType: arrow.PrimitiveTypes.Uint32, + ValueType: fields[i].Type, + } + } + return arrow.NewSchema(fields, nil), nil +} + +// ToArrowType translates the MySQL Type to Arrow Type. +func ToArrowType(t sql.Type) (arrow.DataType, error) { + at := toArrowType(t) + if at == nil { + return nil, sql.ErrInvalidType.New(t) + } + return at, nil +} + +func toArrowType(t sql.Type) arrow.DataType { + switch t.Type() { + case query.Type_UINT8: + return arrow.PrimitiveTypes.Uint8 + case query.Type_INT8: + return arrow.PrimitiveTypes.Int8 + case query.Type_UINT16: + return arrow.PrimitiveTypes.Uint16 + case query.Type_INT16: + return arrow.PrimitiveTypes.Int16 + case query.Type_UINT24: + return arrow.PrimitiveTypes.Uint32 + case query.Type_INT24: + return arrow.PrimitiveTypes.Int32 + case query.Type_UINT32: + return arrow.PrimitiveTypes.Uint32 + case query.Type_INT32: + return arrow.PrimitiveTypes.Int32 + case query.Type_UINT64: + return arrow.PrimitiveTypes.Uint64 + case query.Type_INT64: + return arrow.PrimitiveTypes.Int64 + case query.Type_FLOAT32: + return arrow.PrimitiveTypes.Float32 + case query.Type_FLOAT64: + return arrow.PrimitiveTypes.Float64 + case query.Type_TIMESTAMP: + switch t.(sql.DatetimeType).Precision() { + case 0: + return arrow.FixedWidthTypes.Timestamp_s + case 1, 2, 3: + return arrow.FixedWidthTypes.Timestamp_ms + default: + return arrow.FixedWidthTypes.Timestamp_us + } + case query.Type_DATE: + return arrow.FixedWidthTypes.Date32 + case query.Type_TIME: + return arrow.FixedWidthTypes.Duration_us + case query.Type_DATETIME: + var unit arrow.TimeUnit + switch t.(sql.DatetimeType).Precision() { + case 0: + unit = arrow.Second + case 1, 2, 3: + unit = arrow.Millisecond + default: + unit = arrow.Microsecond + } + return &arrow.TimestampType{Unit: unit, TimeZone: ""} + case query.Type_YEAR: + return arrow.PrimitiveTypes.Uint16 + case query.Type_DECIMAL: + dt := t.(sql.DecimalType) + if dt.Precision() > 38 { + return &arrow.Decimal256Type{ + Precision: int32(dt.Precision()), + Scale: int32(dt.Scale()), + } + } + return &arrow.Decimal128Type{ + Precision: int32(dt.Precision()), + Scale: int32(dt.Scale()), + } + case query.Type_TEXT: + return arrow.BinaryTypes.String + case query.Type_BLOB: + return arrow.BinaryTypes.Binary + case query.Type_VARCHAR: + return arrow.BinaryTypes.String + case query.Type_VARBINARY: + return arrow.BinaryTypes.Binary + case query.Type_CHAR: + return arrow.BinaryTypes.String + case query.Type_BINARY: + return arrow.BinaryTypes.Binary + case query.Type_BIT: + return arrow.PrimitiveTypes.Uint64 + case query.Type_ENUM: + return arrow.BinaryTypes.String + case query.Type_SET: + return arrow.BinaryTypes.String + case query.Type_JSON: + return arrow.BinaryTypes.String + case query.Type_GEOMETRY: + return arrow.BinaryTypes.Binary + default: + panic("unsupported data type") + } +} diff --git a/replica/appender.go b/replica/appender.go new file mode 100644 index 00000000..0a132def --- /dev/null +++ b/replica/appender.go @@ -0,0 +1,86 @@ +package replica + +import ( + "database/sql/driver" + + "github.com/apecloud/myduckserver/binlogreplication" + "github.com/dolthub/go-mysql-server/sql" + "github.com/marcboeker/go-duckdb" +) + +func (twp *tableWriterProvider) newTableAppender( + ctx *sql.Context, + databaseName, tableName string, + columnCount int, +) (*tableAppender, error) { + connector := twp.pool.Connector() + conn, err := connector.Connect(ctx.Context) + if err != nil { + connector.Close() + return nil, err + } + + txn, err := conn.(driver.ConnBeginTx).BeginTx(ctx.Context, driver.TxOptions{}) + if err != nil { + conn.Close() + connector.Close() + return nil, err + } + + appender, err := duckdb.NewAppenderFromConn(conn, databaseName, tableName) + if err != nil { + txn.Rollback() + conn.Close() + connector.Close() + return nil, err + } + + return &tableAppender{ + connector: connector, + conn: conn, + txn: txn, + appender: appender, + buffer: make([]driver.Value, columnCount), + }, nil +} + +type tableAppender struct { + connector *duckdb.Connector + conn driver.Conn + txn driver.Tx + appender *duckdb.Appender + buffer []driver.Value +} + +var _ binlogreplication.TableWriter = &tableAppender{} + +func (ta *tableAppender) Insert(ctx *sql.Context, rows []sql.Row) error { + for _, row := range rows { + for i, v := range row { + ta.buffer[i] = v + } + } + return ta.appender.AppendRow(ta.buffer...) +} + +func (ta *tableAppender) Delete(ctx *sql.Context, keyRows []sql.Row) error { + panic("not implemented") +} + +func (ta *tableAppender) Update(ctx *sql.Context, keyRows []sql.Row, valueRows []sql.Row) error { + panic("not implemented") +} + +func (ta *tableAppender) Commit() error { + defer ta.connector.Close() + defer ta.conn.Close() + defer ta.txn.Commit() + return ta.appender.Close() +} + +func (ta *tableAppender) Rollback() error { + defer ta.connector.Close() + defer ta.conn.Close() + defer ta.txn.Rollback() + return ta.appender.Close() +} diff --git a/replica/controller.go b/replica/controller.go new file mode 100644 index 00000000..bf3b5998 --- /dev/null +++ b/replica/controller.go @@ -0,0 +1,245 @@ +package replica + +import ( + "bytes" + "context" + stdsql "database/sql" + "fmt" + "strconv" + "strings" + "sync" + "unsafe" + + "github.com/apache/arrow/go/v17/arrow/ipc" + "github.com/apecloud/myduckserver/backend" + "github.com/apecloud/myduckserver/binlog" + "github.com/apecloud/myduckserver/binlogreplication" + "github.com/apecloud/myduckserver/catalog" + "github.com/dolthub/go-mysql-server/sql" + "github.com/dolthub/go-mysql-server/sql/types" + "github.com/sirupsen/logrus" +) + +type DeltaController struct { + mutex sync.Mutex + tables map[tableIdentifier]*deltaAppender + pool *backend.ConnectionPool +} + +func (c *DeltaController) GetDeltaAppender( + databaseName, tableName string, + schema sql.Schema, +) (binlogreplication.DeltaAppender, error) { + c.mutex.Lock() + defer c.mutex.Unlock() + + if c.tables == nil { + c.tables = make(map[tableIdentifier]*deltaAppender) + } + + id := tableIdentifier{databaseName, tableName} + appender, ok := c.tables[id] + if ok { + return appender, nil + } + appender, err := newDeltaAppender(schema) + if err != nil { + return nil, err + } + c.tables[id] = appender + return appender, nil +} + +// Flush writes the accumulated changes to the database. +func (c *DeltaController) Flush(ctx context.Context) error { + c.mutex.Lock() + defer c.mutex.Unlock() + + // Due to DuckDB's lack of support for atomic MERGE INTO, we have to do the following two steps separately: + // 1. Delete rows that are being updated. + // 2. Insert new rows. + // To guarantee the atomicity of the two steps, we have to wrap them in a transaction. + // Again, due to DuckDB's limitations of indexes, specifically over-eagerly unique constraint checking, + // if we do **DELETE then INSERT** in the same transaction, we would get + // a unique constraint violation error for INSERT, + // or data corruption for INSERT OR REPLACE|IGNORE INTO. + // + // This is a noteworthy pitfall and seems unlikely to be fixed in DuckDB in the near future, + // but we have to live with it. + // + // On the other hand, fortunately enough, **INSERT OR REPLACE then DELETE** in the same transaction works fine. + // + // The ultimate solution is to wait for DuckDB to improve its index handling. + // In the meantime, we could contribute a patch to DuckDB to support atomic MERGE INTO, + // which is another way to avoid the issue elegantly. + // + // See: + // https://duckdb.org/docs/sql/indexes.html#limitations-of-art-indexes + // https://github.com/duckdb/duckdb/issues/14133 + + tx, err := c.pool.Begin() + if err != nil { + return err + } + defer tx.Rollback() + + // Share the buffer among all tables. + buf := bytes.Buffer{} + + for table, appender := range c.tables { + if err := c.updateTable(ctx, tx, table, appender, &buf); err != nil { + return err + } + } + return tx.Commit() +} + +func (c *DeltaController) updateTable( + ctx context.Context, + tx *stdsql.Tx, + table tableIdentifier, + appender *deltaAppender, + buf *bytes.Buffer, +) error { + buf.Reset() + + schema := appender.BaseSchema() // schema of the base table + record := appender.Build() + defer record.Release() + + fmt.Println("record:", record) + + // TODO(fan): Switch to zero-copy Arrow ingestion once this PR is merged: + // https://github.com/marcboeker/go-duckdb/pull/283 + w := ipc.NewWriter(buf, ipc.WithSchema(record.Schema())) + if err := w.Write(record); err != nil { + panic(err) + } + if err := w.Close(); err != nil { + panic(err) + } + bytes := buf.Bytes() + size := len(bytes) + ptr := unsafe.Pointer(&bytes[0]) + ipcSQL := fmt.Sprintf( + " FROM scan_arrow_ipc([{ptr: %d::ubigint, size: %d::ubigint}])", + uintptr(ptr), size, + ) + + qualifiedTableName := catalog.ConnectIdentifiersANSI(table.dbName, table.tableName) + + pkColumns := make([]int, 0, 1) // Most tables have a single-column primary key + for i, col := range schema { + if col.PrimaryKey { + pkColumns = append(pkColumns, i) + } + } + pkList := catalog.QuoteIdentifierANSI(schema[pkColumns[0]].Name) + for _, i := range pkColumns[1:] { + pkList += ", " + catalog.QuoteIdentifierANSI(schema[i].Name) + } + + // Use the following SQL to get the latest view of the rows being updated. + // + // SELECT r[0] as action, ... + // FROM ( + // SELECT + // pk1, pk2, ..., + // LAST(ROW(*COLUMNS(*)) ORDER BY txn_group, txn_seq, action) AS r + // FROM delta + // GROUP BY pk1, pk2, ... + // ) + // + // Note that an update generates two rows: one for DELETE and one for INSERT. + // So the numeric value of DELETE action MUST be smaller than that of INSERT. + augmentedSchema := appender.Schema() + var builder strings.Builder + builder.Grow(512) + builder.WriteString("SELECT ") + builder.WriteString("r[1] AS ") + builder.WriteString(catalog.QuoteIdentifierANSI(augmentedSchema[0].Name)) + for i, col := range augmentedSchema[1:] { + builder.WriteString(", r[") + builder.WriteString(strconv.Itoa(i + 2)) + builder.WriteString("]") + if types.IsTimestampType(col.Type) { + builder.WriteString("::TIMESTAMP") + } + builder.WriteString(" AS ") + builder.WriteString(catalog.QuoteIdentifierANSI(col.Name)) + } + builder.WriteString(" FROM (SELECT ") + builder.WriteString(pkList) + builder.WriteString(", LAST(ROW(*COLUMNS(*)) ORDER BY txn_group, txn_seq, action) AS r") + builder.WriteString(ipcSQL) + builder.WriteString(" GROUP BY ") + builder.WriteString(pkList) + builder.WriteString(")") + condenseDeltaSQL := builder.String() + + var ( + result stdsql.Result + rowsAffected int64 + err error + ) + + // Create a temporary table to store the latest delta view. + result, err = tx.ExecContext(ctx, "CREATE OR REPLACE TEMP TABLE delta AS "+condenseDeltaSQL) + if err == nil { + rowsAffected, err = result.RowsAffected() + } + if err != nil { + return err + } + defer tx.ExecContext(ctx, "DROP TABLE IF EXISTS temp.main.delta") + + logrus.WithFields(logrus.Fields{ + "table": qualifiedTableName, + "rows": rowsAffected, + }).Infoln("Delta created") + + // Insert or replace new rows (action = INSERT) into the base table. + insertSQL := "INSERT OR REPLACE INTO " + + qualifiedTableName + + " SELECT * EXCLUDE (" + AugmentedColumnList + ") FROM temp.main.delta WHERE action = " + + strconv.Itoa(int(binlog.InsertRowEvent)) + result, err = tx.ExecContext(ctx, insertSQL) + if err == nil { + rowsAffected, err = result.RowsAffected() + } + if err != nil { + return err + } + + logrus.WithFields(logrus.Fields{ + "table": qualifiedTableName, + "rows": rowsAffected, + }).Infoln("Inserted") + + // Delete rows that have been deleted. + // The plan for `IN` is optimized to a SEMI JOIN, + // which is more efficient than ordinary INNER JOIN. + // DuckDB does not support multiple columns in `IN` clauses, + // so we need to handle this case separately using the `row()` function. + inTuple := pkList + if len(pkColumns) > 1 { + inTuple = "row(" + pkList + ")" + } + deleteSQL := "DELETE FROM " + qualifiedTableName + + " WHERE " + inTuple + " IN (SELECT " + inTuple + + "FROM temp.main.delta WHERE action = " + strconv.Itoa(int(binlog.DeleteRowEvent)) + ")" + result, err = tx.ExecContext(ctx, deleteSQL) + if err == nil { + rowsAffected, err = result.RowsAffected() + } + if err != nil { + return err + } + + logrus.WithFields(logrus.Fields{ + "table": qualifiedTableName, + "rows": rowsAffected, + }).Infoln("Deleted") + + return nil +} diff --git a/replica/delta.go b/replica/delta.go new file mode 100644 index 00000000..1fb62f38 --- /dev/null +++ b/replica/delta.go @@ -0,0 +1,101 @@ +package replica + +import ( + "github.com/apache/arrow/go/v17/arrow" + "github.com/apache/arrow/go/v17/arrow/array" + "github.com/apecloud/myduckserver/binlogreplication" + "github.com/apecloud/myduckserver/myarrow" + "github.com/dolthub/go-mysql-server/sql" + "github.com/dolthub/go-mysql-server/sql/types" +) + +const ( + AugmentedColumnList = "action, txn_tag, txn_server, txn_group, txn_seq" +) + +type tableIdentifier struct { + dbName, tableName string +} + +type deltaAppender struct { + schema sql.Schema + appender myarrow.ArrowAppender +} + +var _ binlogreplication.DeltaAppender = &deltaAppender{} + +// Create a new appender. +// Add action and GTID columns to the schema: +// +// https://mariadb.com/kb/en/gtid/ +// https://dev.mysql.com/doc/refman/9.0/en/replication-gtids-concepts.html +func newDeltaAppender(schema sql.Schema) (*deltaAppender, error) { + augmented := make(sql.Schema, 0, len(schema)+5) + augmented = append(augmented, &sql.Column{ + Name: "action", // delete = 0, update = 1, insert = 2 + Type: types.Int8, + }, &sql.Column{ + Name: "txn_tag", // GTID tag in MySQL>=8.4; GTID domain in MariaDB + Type: types.Text, + }, &sql.Column{ + Name: "txn_server", + Type: types.Blob, + }, &sql.Column{ + Name: "txn_group", // NULL for MySQL & MariaDB GTID; binlog file name for file position based replication + Type: types.Text, + }, &sql.Column{ + Name: "txn_seq", + Type: types.Uint64, + }) + augmented = append(augmented, schema...) + + appender, err := myarrow.NewArrowAppender(augmented, 1, 2, 3) + if err != nil { + return nil, err + } + + return &deltaAppender{ + schema: augmented, + appender: appender, + }, nil +} + +func (a *deltaAppender) Field(i int) array.Builder { + return a.appender.Field(i + 5) +} + +func (a *deltaAppender) Fields() []array.Builder { + return a.appender.Fields()[5:] +} + +func (a *deltaAppender) Schema() sql.Schema { + return a.schema +} + +func (a *deltaAppender) BaseSchema() sql.Schema { + return a.schema[5:] +} + +func (a *deltaAppender) Action() *array.Int8Builder { + return a.appender.Field(0).(*array.Int8Builder) +} + +func (a *deltaAppender) TxnTag() *array.BinaryDictionaryBuilder { + return a.appender.Field(1).(*array.BinaryDictionaryBuilder) +} + +func (a *deltaAppender) TxnServer() *array.BinaryDictionaryBuilder { + return a.appender.Field(2).(*array.BinaryDictionaryBuilder) +} + +func (a *deltaAppender) TxnGroup() *array.BinaryDictionaryBuilder { + return a.appender.Field(3).(*array.BinaryDictionaryBuilder) +} + +func (a *deltaAppender) TxnSeqNumber() *array.Uint64Builder { + return a.appender.Field(4).(*array.Uint64Builder) +} + +func (a *deltaAppender) Build() arrow.Record { + return a.appender.Build() +} diff --git a/replica/replication.go b/replica/replication.go new file mode 100644 index 00000000..6af7b000 --- /dev/null +++ b/replica/replication.go @@ -0,0 +1,86 @@ +// Copyright 2024-2025 ApeCloud, Ltd. +// +// 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package replica + +import ( + "context" + + sqle "github.com/dolthub/go-mysql-server" + "github.com/dolthub/go-mysql-server/memory" + "github.com/dolthub/go-mysql-server/sql" + "github.com/sirupsen/logrus" + "vitess.io/vitess/go/mysql" + + "github.com/apecloud/myduckserver/backend" + "github.com/apecloud/myduckserver/binlog" + "github.com/apecloud/myduckserver/binlogreplication" + "github.com/apecloud/myduckserver/catalog" +) + +// registerReplicaController registers the replica controller into the engine +// to handle the replication commands, such as START REPLICA, STOP REPLICA, etc. +func RegisterReplicaController(provider *catalog.DatabaseProvider, engine *sqle.Engine, pool *backend.ConnectionPool) { + replica := binlogreplication.MyBinlogReplicaController + replica.SetEngine(engine) + + session := backend.NewSession(memory.NewSession(sql.NewBaseSession(), provider), provider, pool) + ctx := sql.NewContext(context.Background(), sql.WithSession(session)) + ctx.SetCurrentDatabase("mysql") + replica.SetExecutionContext(ctx) + + twp := &tableWriterProvider{pool: pool} + twp.delta.pool = pool + replica.SetTableWriterProvider(twp) + + engine.Analyzer.Catalog.BinlogReplicaController = binlogreplication.MyBinlogReplicaController + + // If we're unable to restart replication, log an error, but don't prevent the server from starting up + if err := binlogreplication.MyBinlogReplicaController.AutoStart(ctx); err != nil { + logrus.Errorf("unable to restart replication: %s", err.Error()) + } +} + +type tableWriterProvider struct { + pool *backend.ConnectionPool + delta DeltaController +} + +var _ binlogreplication.TableWriterProvider = &tableWriterProvider{} + +func (twp *tableWriterProvider) GetTableWriter( + ctx *sql.Context, engine *sqle.Engine, + databaseName, tableName string, + schema sql.PrimaryKeySchema, + columnCount, rowCount int, + identifyColumns, dataColumns mysql.Bitmap, + eventType binlog.RowEventType, + foreignKeyChecksDisabled bool, +) (binlogreplication.TableWriter, error) { + // if eventType == binlogreplication.InsertEvent { + // return twp.newTableAppender(ctx, databaseName, tableName, columnCount) + // } + return twp.newTableUpdater(ctx, databaseName, tableName, schema, columnCount, rowCount, identifyColumns, dataColumns, eventType) +} + +func (twp *tableWriterProvider) GetDeltaAppender( + ctx *sql.Context, engine *sqle.Engine, + databaseName, tableName string, + schema sql.Schema, +) (binlogreplication.DeltaAppender, error) { + return twp.delta.GetDeltaAppender(databaseName, tableName, schema) +} + +func (twp *tableWriterProvider) FlushDelta(ctx *sql.Context) error { + return twp.delta.Flush(ctx) +} diff --git a/replication.go b/replica/updater.go similarity index 52% rename from replication.go rename to replica/updater.go index f198f111..42382a63 100644 --- a/replication.go +++ b/replica/updater.go @@ -1,139 +1,19 @@ -// Copyright 2024-2025 ApeCloud, Ltd. -// -// 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, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -package main +package replica import ( - "context" stdsql "database/sql" - "database/sql/driver" + "errors" "strings" - sqle "github.com/dolthub/go-mysql-server" - "github.com/dolthub/go-mysql-server/memory" - "github.com/dolthub/go-mysql-server/sql" - "github.com/dolthub/vitess/go/mysql" - "github.com/marcboeker/go-duckdb" - "github.com/sirupsen/logrus" - "github.com/apecloud/myduckserver/backend" + "github.com/apecloud/myduckserver/binlog" "github.com/apecloud/myduckserver/binlogreplication" - "github.com/apecloud/myduckserver/catalog" + "github.com/dolthub/go-mysql-server/sql" + "github.com/sirupsen/logrus" + "vitess.io/vitess/go/mysql" ) -// registerReplicaController registers the replica controller into the engine -// to handle the replication commands, such as START REPLICA, STOP REPLICA, etc. -func registerReplicaController(provider *catalog.DatabaseProvider, engine *sqle.Engine, pool *backend.ConnectionPool) { - replica := binlogreplication.MyBinlogReplicaController - replica.SetEngine(engine) - - session := backend.NewSession(memory.NewSession(sql.NewBaseSession(), provider), provider, pool) - ctx := sql.NewContext(context.Background(), sql.WithSession(session)) - ctx.SetCurrentDatabase("mysql") - replica.SetExecutionContext(ctx) - - replica.SetTableWriterProvider(&tableWriterProvider{pool}) - - engine.Analyzer.Catalog.BinlogReplicaController = binlogreplication.MyBinlogReplicaController - - // If we're unable to restart replication, log an error, but don't prevent the server from starting up - if err := binlogreplication.MyBinlogReplicaController.AutoStart(ctx); err != nil { - logrus.Errorf("unable to restart replication: %s", err.Error()) - } -} - -type tableWriterProvider struct { - pool *backend.ConnectionPool -} - -var _ binlogreplication.TableWriterProvider = &tableWriterProvider{} - -func (twp *tableWriterProvider) GetTableWriter( - ctx *sql.Context, engine *sqle.Engine, - databaseName, tableName string, - schema sql.Schema, - columnCount, rowCount int, - identifyColumns, dataColumns mysql.Bitmap, - eventType binlogreplication.EventType, - foreignKeyChecksDisabled bool, -) (binlogreplication.TableWriter, error) { - // if eventType == binlogreplication.InsertEvent { - // return twp.newTableAppender(ctx, databaseName, tableName, columnCount) - // } - return twp.newTableUpdater(ctx, databaseName, tableName, schema, columnCount, rowCount, identifyColumns, dataColumns, eventType) -} - -func (twp *tableWriterProvider) newTableAppender( - ctx *sql.Context, - databaseName, tableName string, - columnCount int, -) (*tableAppender, error) { - connector, err := duckdb.NewConnector(dbFilePath, nil) - if err != nil { - return nil, err - } - conn, err := connector.Connect(ctx.Context) - if err != nil { - connector.Close() - return nil, err - } - - appender, err := duckdb.NewAppenderFromConn(conn, databaseName, tableName) - if err != nil { - conn.Close() - connector.Close() - return nil, err - } - - return &tableAppender{ - connector: connector, - conn: conn, - appender: appender, - buffer: make([]driver.Value, columnCount), - }, nil -} - -type tableAppender struct { - connector *duckdb.Connector - conn driver.Conn - appender *duckdb.Appender - buffer []driver.Value -} - -var _ binlogreplication.TableWriter = &tableAppender{} - -func (ta *tableAppender) Insert(ctx *sql.Context, rows []sql.Row) error { - for _, row := range rows { - for i, v := range row { - ta.buffer[i] = v - } - } - return ta.appender.AppendRow(ta.buffer...) -} - -func (ta *tableAppender) Delete(ctx *sql.Context, keyRows []sql.Row) error { - panic("not implemented") -} - -func (ta *tableAppender) Update(ctx *sql.Context, keyRows []sql.Row, valueRows []sql.Row) error { - panic("not implemented") -} - -func (ta *tableAppender) Close() error { - defer ta.connector.Close() - defer ta.conn.Close() - return ta.appender.Close() -} +var ErrPartialPrimaryKeyUpdate = errors.New("primary key columns are (partially) updated but are not fully specified in the binlog") func isPkUpdate(schema sql.Schema, identifyColumns, dataColumns mysql.Bitmap) bool { for i, c := range schema { @@ -144,35 +24,41 @@ func isPkUpdate(schema sql.Schema, identifyColumns, dataColumns mysql.Bitmap) bo return false } +func getPrimaryKeyIndices(schema sql.Schema, columns mysql.Bitmap) []int { + var count int + var indices []int + for i, c := range schema { + set := columns.Count() > i && columns.Bit(i) + if c.PrimaryKey && !set { + return nil + } else if c.PrimaryKey && set { + indices = append(indices, count) + } + if set { + count++ + } + } + return indices +} + func (twp *tableWriterProvider) newTableUpdater( ctx *sql.Context, databaseName, tableName string, - schema sql.Schema, + pkSchema sql.PrimaryKeySchema, columnCount, rowCount int, identifyColumns, dataColumns mysql.Bitmap, - eventType binlogreplication.EventType, + eventType binlog.RowEventType, ) (*tableUpdater, error) { - tx, err := twp.pool.BeginTx(ctx, nil) - if err != nil { - return nil, err + schema := pkSchema.Schema + pkColumns := pkSchema.PkOrdinals + pkIndicesInIdentify := getPrimaryKeyIndices(schema, identifyColumns) + pkIndicesInData := getPrimaryKeyIndices(schema, dataColumns) + if len(pkIndicesInIdentify) == 0 && len(pkColumns) > 0 { + pkColumns = nil // disable primary key utilization } - - var identifyIndex int - var pkColumns []int - var pkIndices []int - for i, c := range schema { - identify := identifyColumns.Count() > i && identifyColumns.Bit(i) - if c.PrimaryKey && !identify { - pkColumns = nil - pkIndices = nil - break - } else if c.PrimaryKey && identify { - pkColumns = append(pkColumns, i) - pkIndices = append(pkIndices, identifyIndex) - } - if identify { - identifyIndex++ - } + pkSubSchema := make(sql.Schema, len(pkColumns)) + for i, idx := range pkColumns { + pkSubSchema[i] = schema[idx] } var ( @@ -180,42 +66,59 @@ func (twp *tableWriterProvider) newTableUpdater( paramCount int pkUpdate bool replace = false - reinsert string + cleanup string fullTableName = quoteIdentifier(databaseName) + "." + quoteIdentifier(tableName) keyCount, dataCount = identifyColumns.BitCount(), dataColumns.BitCount() ) switch eventType { - case binlogreplication.DeleteEvent: + case binlog.DeleteRowEvent: sql, paramCount = buildDeleteTemplate(fullTableName, columnCount, schema, pkColumns, identifyColumns) - case binlogreplication.UpdateEvent: + case binlog.UpdateRowEvent: pkUpdate = isPkUpdate(schema, identifyColumns, dataColumns) if pkUpdate { // If the primary key is being updated, we need to use DELETE + INSERT. + // // For example, if the primary has executed `UPDATE t SET pk = pk + 1;`, - // then both `UPDATE` and `REPLACE` will fail on the replica because the primary key is being updated: + // then both `UPDATE` and `INSERT OR REPLACE` will fail on the replica because the primary key is being updated: // - `UPDATE` will fail because of violation of the primary key constraint. // - `REPLACE` will fail because it will insert a new row but leave the old row unchanged. - sql, paramCount = buildDeleteTemplate(fullTableName, columnCount, schema, pkColumns, identifyColumns) - reinsert, _ = buildInsertTemplate(fullTableName, columnCount, false) + // + // However, `DELETE` then `INSERT` in the same transaction will still fail + // due to the over-eager unique constraint checking in DuckDB, just like the `UPDATE` case. + // + // The only way to work around this without breaking atomicity is to do `INSERT OR REPLACE` first, + // then `DELETE` the old row if the primary key has actually been modified. + // This requires the occurrence of the primary key columns in both the `identifyColumns` and `dataColumns`. + if len(pkIndicesInIdentify) == 0 || len(pkIndicesInData) == 0 { + return nil, ErrPartialPrimaryKeyUpdate + } + sql, paramCount = buildInsertTemplate(fullTableName, columnCount, true) + cleanup, _ = buildDeleteTemplate(fullTableName, columnCount, schema, pkColumns, identifyColumns) + replace = true } else if keyCount < columnCount || dataCount < columnCount { sql, paramCount = buildUpdateTemplate(fullTableName, columnCount, schema, pkColumns, identifyColumns, dataColumns) } else { sql, paramCount = buildInsertTemplate(fullTableName, columnCount, true) replace = true } - case binlogreplication.InsertEvent: + case binlog.InsertRowEvent: sql, paramCount = buildInsertTemplate(fullTableName, columnCount, false) } logrus.WithFields(logrus.Fields{ "sql": sql, "replace": replace, - "reinsert": reinsert, + "cleanup": cleanup, "keyCount": keyCount, "dataCount": dataCount, "pkUpdate": pkUpdate, }).Infoln("Creating table updater...") + tx, err := twp.pool.BeginTx(ctx, nil) + if err != nil { + return nil, err + } + stmt, err := tx.PrepareContext(ctx.Context, sql) if err != nil { tx.Rollback() @@ -227,9 +130,11 @@ func (twp *tableWriterProvider) newTableUpdater( tx: tx, stmt: stmt, replace: replace, - reinsert: reinsert, - pkIndices: pkIndices, + cleanup: cleanup, paramCount: paramCount, + + pkIndicesInIdentify: pkIndicesInIdentify, + pkIndicesInData: pkIndicesInData, }, nil } @@ -335,14 +240,18 @@ type tableUpdater struct { tx *stdsql.Tx stmt *stdsql.Stmt replace bool - reinsert string + cleanup string paramCount int - pkIndices []int + + pkSubSchema sql.Schema + pkIndicesInIdentify []int + pkIndicesInData []int } var _ binlogreplication.TableWriter = &tableUpdater{} func (tu *tableUpdater) Insert(ctx *sql.Context, rows []sql.Row) error { + defer tu.stmt.Close() for _, row := range rows { if _, err := tu.stmt.ExecContext(ctx.Context, row...); err != nil { return err @@ -352,11 +261,12 @@ func (tu *tableUpdater) Insert(ctx *sql.Context, rows []sql.Row) error { } func (tu *tableUpdater) Delete(ctx *sql.Context, keyRows []sql.Row) error { - buf := make(sql.Row, len(tu.pkIndices)) + defer tu.stmt.Close() + buf := make(sql.Row, len(tu.pkIndicesInIdentify)) for _, row := range keyRows { var keys sql.Row - if len(tu.pkIndices) > 0 { - for i, idx := range tu.pkIndices { + if len(tu.pkIndicesInIdentify) > 0 { + for i, idx := range tu.pkIndicesInIdentify { buf[i] = row[idx] } keys = buf @@ -371,34 +281,12 @@ func (tu *tableUpdater) Delete(ctx *sql.Context, keyRows []sql.Row) error { } func (tu *tableUpdater) Update(ctx *sql.Context, keyRows []sql.Row, valueRows []sql.Row) error { - if tu.replace { + if tu.replace && tu.cleanup == "" { return tu.Insert(ctx, valueRows) } - // https://duckdb.org/docs/sql/indexes#over-eager-unique-constraint-checking - if tu.reinsert != "" { - var err error - // DELETE - if err = tu.Delete(ctx, keyRows); err != nil { - return err - } - if err = tu.stmt.Close(); err != nil { - return err - } - if err = tu.tx.Commit(); err != nil { - return err - } - - tu.tx, err = tu.pool.BeginTx(ctx.Context, nil) - if err != nil { - return err - } - tu.stmt, err = tu.tx.PrepareContext(ctx.Context, tu.reinsert) - if err != nil { - return err - } - // INSERT - return tu.Insert(ctx, valueRows) + if tu.cleanup != "" { + return tu.doInsertThenDelete(ctx, keyRows, valueRows) } // UPDATE t SET col1 = ?, col2 = ? WHERE key1 = ? AND key2 = ? @@ -406,8 +294,8 @@ func (tu *tableUpdater) Update(ctx *sql.Context, keyRows []sql.Row, valueRows [] for i, values := range valueRows { keys := keyRows[i] copy(buf, values) - if len(tu.pkIndices) > 0 { - for j, idx := range tu.pkIndices { + if len(tu.pkIndicesInIdentify) > 0 { + for j, idx := range tu.pkIndicesInIdentify { buf[len(values)+j] = keys[idx] } } else { @@ -421,9 +309,53 @@ func (tu *tableUpdater) Update(ctx *sql.Context, keyRows []sql.Row, valueRows [] return nil } -func (tu *tableUpdater) Close() error { - defer tu.tx.Commit() - return tu.stmt.Close() +// https://duckdb.org/docs/sql/indexes#over-eager-unique-constraint-checking +// https://github.com/duckdb/duckdb/issues/14133 +func (tu *tableUpdater) doInsertThenDelete(ctx *sql.Context, beforeRows []sql.Row, afterRows []sql.Row) error { + var err error + + // INSERT OR REPLACE + if err = tu.Insert(ctx, afterRows); err != nil { + return err + } + + // DELETE if the primary key has actually been modified + stmt, err := tu.tx.PrepareContext(ctx.Context, tu.cleanup) + if err != nil { + return err + } + defer stmt.Close() + + beforeKey := make(sql.Row, len(tu.pkSubSchema)) + afterKey := make(sql.Row, len(tu.pkSubSchema)) + for i, before := range beforeRows { + after := afterRows[i] + for j, idx := range tu.pkIndicesInIdentify { + beforeKey[j] = before[idx] + } + for j, idx := range tu.pkIndicesInData { + afterKey[j] = after[idx] + } + if yes, err := beforeKey.Equals(afterKey, tu.pkSubSchema); err != nil { + return err + } else if yes { + // the row has already been deleted by the INSERT OR REPLACE statement + continue + } + if _, err := stmt.ExecContext(ctx.Context, beforeKey...); err != nil { + return err + } + } + + return nil +} + +func (tu *tableUpdater) Commit() error { + return tu.tx.Commit() +} + +func (tu *tableUpdater) Rollback() error { + return tu.tx.Rollback() } func quoteIdentifier(identifier string) string {