From e28d5b66c5b1a2c8bd6db90387237ae59f55e6c5 Mon Sep 17 00:00:00 2001 From: Jason Fulghum Date: Thu, 30 May 2024 15:16:04 -0700 Subject: [PATCH 01/14] First pass on buffering binlog events in a log file on disk --- go/cmd/dolt/commands/engine/sqlengine.go | 31 -- go/cmd/dolt/commands/sqlserver/server.go | 65 +++- .../binlog_primary_controller.go | 26 +- .../binlog_primary_log_manager.go | 282 ++++++++++++++++++ .../binlog_primary_streamer.go | 136 +++++++-- .../binlogreplication/binlog_primary_test.go | 10 +- .../sqle/binlogreplication/binlog_producer.go | 25 +- 7 files changed, 481 insertions(+), 94 deletions(-) create mode 100644 go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go diff --git a/go/cmd/dolt/commands/engine/sqlengine.go b/go/cmd/dolt/commands/engine/sqlengine.go index 645b6c74a55..5a6e8501f59 100644 --- a/go/cmd/dolt/commands/engine/sqlengine.go +++ b/go/cmd/dolt/commands/engine/sqlengine.go @@ -352,37 +352,6 @@ func configureBinlogPrimaryController(engine *gms.Engine) error { primaryController := dblr.NewDoltBinlogPrimaryController() engine.Analyzer.Catalog.BinlogPrimaryController = primaryController - _, logBinValue, ok := sql.SystemVariables.GetGlobal("log_bin") - if !ok { - return fmt.Errorf("unable to load @@log_bin system variable") - } - logBin, ok := logBinValue.(int8) - if !ok { - return fmt.Errorf("unexpected type for @@log_bin system variable: %T", logBinValue) - } - if logBin == 1 { - logrus.Debug("Enabling binary logging") - binlogProducer, err := dblr.NewBinlogProducer(primaryController.StreamerManager()) - if err != nil { - return err - } - doltdb.RegisterDatabaseUpdateListener(binlogProducer) - primaryController.BinlogProducer = binlogProducer - } - - _, logBinBranchValue, ok := sql.SystemVariables.GetGlobal("log_bin_branch") - if !ok { - return fmt.Errorf("unable to load @@log_bin_branch system variable") - } - logBinBranch, ok := logBinBranchValue.(string) - if !ok { - return fmt.Errorf("unexpected type for @@log_bin_branch system variable: %T", logBinBranchValue) - } - if logBinBranch != "" { - logrus.Debugf("Setting binary logging branch to %s", logBinBranch) - dblr.BinlogBranch = logBinBranch - } - return nil } diff --git a/go/cmd/dolt/commands/sqlserver/server.go b/go/cmd/dolt/commands/sqlserver/server.go index f505862dbd4..b5d1da54f58 100644 --- a/go/cmd/dolt/commands/sqlserver/server.go +++ b/go/cmd/dolt/commands/sqlserver/server.go @@ -285,21 +285,64 @@ func ConfigureServices( } controller.Register(InitSqlEngine) - // Persist any system variables that have a non-deterministic default value (i.e. @@server_uuid) - // We only do this on sql-server startup initially since we want to keep the persisted server_uuid - // in the configuration files for a sql-server, and not global for the whole host. - PersistNondeterministicSystemVarDefaults := &svcs.AnonService{ - InitF: func(ctx context.Context) error { - err := dsess.PersistSystemVarDefaults(dEnv) - if err != nil { - logrus.Errorf("unable to persist system variable defaults: %v", err) + InitBinlogging := &svcs.AnonService{ + InitF: func(context.Context) error { + primaryController1 := sqlEngine.GetUnderlyingEngine().Analyzer.Catalog.BinlogPrimaryController + + // TODO: rename to doltBinlogPrimaryController + primaryController, ok := primaryController1.(*binlogreplication.DoltBinlogPrimaryController) + if !ok { + panic("Found unexpected type of binlog controller!") + } + + _, logBinValue, ok := sql.SystemVariables.GetGlobal("log_bin") + if !ok { + return fmt.Errorf("unable to load @@log_bin system variable") + } + logBin, ok := logBinValue.(int8) + if !ok { + return fmt.Errorf("unexpected type for @@log_bin system variable: %T", logBinValue) + } + if logBin == 1 { + logrus.Debug("Enabling binary logging") + binlogProducer, err := binlogreplication.NewBinlogProducer(primaryController.StreamerManager()) + if err != nil { + return err + } + // NOTE: Hooks need to be applied AFTER this! + doltdb.RegisterDatabaseUpdateListener(binlogProducer) + primaryController.BinlogProducer = binlogProducer + + // TODO: This is a mess! How to clean this up! + provider := sqlEngine.GetUnderlyingEngine().Analyzer.Catalog.DbProvider + if doltProvider, ok := provider.(*sqle.DoltDatabaseProvider); ok { + doltProvider.InitDatabaseHooks = append(doltProvider.InitDatabaseHooks, binlogreplication.NewBinlogInitDatabaseHook(nil, doltdb.DatabaseUpdateListeners)) + doltProvider.DropDatabaseHooks = append(doltProvider.DropDatabaseHooks, binlogreplication.NewBinlogDropDatabaseHook(nil, doltdb.DatabaseUpdateListeners)) + } + + // TODO: How do we feed the binlogStream and binlogFormat to the log manager? + // Needs format to write the initial format event to the stream + // Needs binlogStream to write the initial format event + logManager := binlogreplication.NewLogManager(fs, binlogProducer.BinlogFormat(), binlogProducer.BinlogStream()) + primaryController.StreamerManager().LogManager(logManager) + } + + _, logBinBranchValue, ok := sql.SystemVariables.GetGlobal("log_bin_branch") + if !ok { + return fmt.Errorf("unable to load @@log_bin_branch system variable") + } + logBinBranch, ok := logBinBranchValue.(string) + if !ok { + return fmt.Errorf("unexpected type for @@log_bin_branch system variable: %T", logBinBranchValue) + } + if logBinBranch != "" { + logrus.Debugf("Setting binary logging branch to %s", logBinBranch) + binlogreplication.BinlogBranch = logBinBranch } - // Always return nil, because we don't want an invalid config value to prevent - // the server from starting up. return nil }, } - controller.Register(PersistNondeterministicSystemVarDefaults) + controller.Register(InitBinlogging) // Add superuser if specified user exists; add root superuser if no user specified and no existing privileges InitSuperUser := &svcs.AnonService{ diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go index 64f5da9614b..093e4500c4f 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go @@ -23,25 +23,25 @@ import ( "github.com/sirupsen/logrus" ) -// doltBinlogPrimaryController implements the binlogreplication.BinlogPrimaryController +// DoltBinlogPrimaryController implements the binlogreplication.BinlogPrimaryController // interface from GMS and is the main extension point where Dolt plugs in to GMS and // interprets commands and statements related to serving binlog events. -type doltBinlogPrimaryController struct { +type DoltBinlogPrimaryController struct { streamerManager *binlogStreamerManager BinlogProducer *binlogProducer } -var _ binlogreplication.BinlogPrimaryController = (*doltBinlogPrimaryController)(nil) +var _ binlogreplication.BinlogPrimaryController = (*DoltBinlogPrimaryController)(nil) -// NewDoltBinlogPrimaryController creates a new doltBinlogPrimaryController instance. -func NewDoltBinlogPrimaryController() *doltBinlogPrimaryController { - controller := doltBinlogPrimaryController{ +// NewDoltBinlogPrimaryController creates a new DoltBinlogPrimaryController instance. +func NewDoltBinlogPrimaryController() *DoltBinlogPrimaryController { + controller := DoltBinlogPrimaryController{ streamerManager: newBinlogStreamerManager(), } return &controller } -func (d *doltBinlogPrimaryController) StreamerManager() *binlogStreamerManager { +func (d *DoltBinlogPrimaryController) StreamerManager() *binlogStreamerManager { return d.streamerManager } @@ -52,7 +52,7 @@ func (d *doltBinlogPrimaryController) StreamerManager() *binlogStreamerManager { // to implement the ListReplicas method below. For now, this method is still useful to throw errors back to the // replica if bin logging isn't enabled, since errors returned from the BinlogDumpGtid method seem to be dropped // by the replica, instead of being displayed as an error. -func (d *doltBinlogPrimaryController) RegisterReplica(ctx *sql.Context, c *mysql.Conn, replicaHost string, replicaPort uint16) error { +func (d *DoltBinlogPrimaryController) RegisterReplica(ctx *sql.Context, c *mysql.Conn, replicaHost string, replicaPort uint16) error { if d.BinlogProducer == nil { return fmt.Errorf("no binlog currently being recorded; make sure the server is started with @@log_bin enabled") } @@ -61,7 +61,7 @@ func (d *doltBinlogPrimaryController) RegisterReplica(ctx *sql.Context, c *mysql } // BinlogDumpGtid implements the BinlogPrimaryController interface. -func (d *doltBinlogPrimaryController) BinlogDumpGtid(ctx *sql.Context, conn *mysql.Conn, gtidSet mysql.GTIDSet) error { +func (d *DoltBinlogPrimaryController) BinlogDumpGtid(ctx *sql.Context, conn *mysql.Conn, gtidSet mysql.GTIDSet) error { if d.BinlogProducer == nil { return fmt.Errorf("no binlog currently being recorded; make sure the server is started with @@log_bin enabled") } @@ -77,20 +77,20 @@ func (d *doltBinlogPrimaryController) BinlogDumpGtid(ctx *sql.Context, conn *mys } // ListReplicas implements the BinlogPrimaryController interface. -func (d *doltBinlogPrimaryController) ListReplicas(ctx *sql.Context) error { +func (d *DoltBinlogPrimaryController) ListReplicas(ctx *sql.Context) error { return fmt.Errorf("ListReplicas not implemented in Dolt yet") } // ListBinaryLogs implements the BinlogPrimaryController interface. -func (d *doltBinlogPrimaryController) ListBinaryLogs(_ *sql.Context) ([]binlogreplication.BinaryLogFileMetadata, error) { +func (d *DoltBinlogPrimaryController) ListBinaryLogs(_ *sql.Context) ([]binlogreplication.BinaryLogFileMetadata, error) { // TODO: No log file support yet, so just return an empty list return nil, nil } // GetBinaryLogStatus implements the BinlogPrimaryController interface. -func (d *doltBinlogPrimaryController) GetBinaryLogStatus(ctx *sql.Context) ([]binlogreplication.BinaryLogStatus, error) { +func (d *DoltBinlogPrimaryController) GetBinaryLogStatus(ctx *sql.Context) ([]binlogreplication.BinaryLogStatus, error) { return []binlogreplication.BinaryLogStatus{{ - File: binlogFilename, + File: d.streamerManager.logManager.currentBinlogFileName, Position: uint(d.BinlogProducer.binlogEventMeta.NextLogPosition), ExecutedGtids: d.BinlogProducer.currentGtidPosition(), }}, nil diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go new file mode 100644 index 00000000000..7681306909f --- /dev/null +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go @@ -0,0 +1,282 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// 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 binlogreplication + +import ( + "encoding/binary" + "fmt" + "github.com/dolthub/dolt/go/libraries/utils/filesys" + "os" + "path/filepath" + "strconv" + "strings" + + "github.com/dolthub/vitess/go/mysql" +) + +// maxBinlogSize is the maximum size of a binlog file, before the binlog writer rotates to a new binlog file. Once +// a binlog file reaches this size, or greater, the events from the next transaction should be written to a new file. +// Note that all events in a single transaction should be written to the same binlog file. +// This corresponds to the @@max_binlog_size system variable in MySQL: +// https://dev.mysql.com/doc/refman/8.0/en/replication-options-binary-log.html#sysvar_max_binlog_size +const maxBinlogSize = 1024 * 1024 * 1024 + +var binlogDirectory = filepath.Join(".dolt", "binlog") + +// binlogFileMagicNumber holds the four bytes that start off every +// MySQL binlog file and identify the file as a MySQL binlog. +var binlogFileMagicNumber = []byte{0xfe, 0x62, 0x69, 0x6e} + +type LogManager struct { + currentBinlogFile *os.File + currentBinlogFileName string + fs filesys.Filesys +} + +// NewLogManager creates a new LogManager instance where binlog files are stored in the .dolt/binlog directory +// underneath the specified |fs| filesystem. The |binlogFormat| and |binlogStream| are used to initialize the +// new binlog file. +func NewLogManager(fs filesys.Filesys, binlogFormat *mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) *LogManager { + // TODO: On server startup, we need to find the most recent binlog file, add a rotate event at the end (if necessary?), and start a new file. Documentation seems to indicate that a rotate event is added at the end of a binlog file, so that the streamer can jump to the next file, but I don't see this in our MySQL sample binlog files. Need to do more testing here. + + lm := &LogManager{ + fs: fs, + } + + // TODO: Could resolve the base dir for the binlog file directory here; would it help us avoid returning errors in other APIs? + + // Initialize binlog file storage (extract to function!) + err := fs.MkDirs(binlogDirectory) + if err != nil { + panic(err) + } + + // Initialize current binlog file + nextLogFilename, err := lm.nextLogFile() + if err != nil { + panic(err) + } + lm.currentBinlogFileName = nextLogFilename + + // Ugh... we need binlogFormat and binlogEventMeta in order to do this... + // Actually... Do we need binlogEventMeta, or could we fake it? We only need binlogEventMeta so that + // Vitess can call a function on that instance, and for the server Id. The position in the file + // should always be zero at this point, so maybe we could clean this up more? + err = lm.initializeCurrentLogFile(binlogFormat, binlogEventMeta) + if err != nil { + panic(err) + } + + return lm +} + +func (lm *LogManager) nextLogFile() (filename string, err error) { + mostRecentLogfile, err := lm.mostRecentLogFileForBranch(BinlogBranch) + if err != nil { + return "", err + } + + if mostRecentLogfile == "" { + return formatBinlogFilename(BinlogBranch, 1), nil + } else { + branch, sequence, err := parseBinlogFilename(mostRecentLogfile) + if err != nil { + return "", err + } + return formatBinlogFilename(branch, sequence+1), nil + } +} + +func (lm *LogManager) logFilesOnDisk() (files []string, err error) { + err = lm.fs.Iter(binlogDirectory, false, func(path string, size int64, isDir bool) (stop bool) { + base := filepath.Base(path) + if strings.HasPrefix(base, "binlog-") { + files = append(files, base) + } + + return false + }) + if err != nil { + return nil, err + } + + return files, nil +} + +func (lm *LogManager) logFilesOnDiskForBranch(branch string) (files []string, err error) { + branch = strings.ToLower(branch) + err = lm.fs.Iter(binlogDirectory, false, func(path string, size int64, isDir bool) (stop bool) { + base := filepath.Base(path) + if strings.HasPrefix(base, "binlog-"+branch) { + files = append(files, base) + } + + return false + }) + if err != nil { + return nil, err + } + + return files, nil +} + +func (lm *LogManager) mostRecentLogfile() (logFile string, err error) { + logFiles, err := lm.logFilesOnDisk() + if err != nil { + return "", err + } + + return logFiles[len(logFiles)-1], nil +} + +func (lm *LogManager) mostRecentLogFileForBranch(branch string) (logFile string, err error) { + logFiles, err := lm.logFilesOnDiskForBranch(branch) + if err != nil { + return "", err + } + + // TODO: This assumes the list comes back sorted by time or by filename + if len(logFiles) == 0 { + return "", nil + } else { + return logFiles[len(logFiles)-1], nil + } +} + +func (lm *LogManager) RotateLogFile() error { + // TODO: Handle logfile rotation: + // - Manual rotation from the FLUSH LOGS statement (Out of scope) + // - Automatically when the binary log file reaches the maximum size defined by the max_binlog_size configuration parameter. + // https://dev.mysql.com/doc/refman/8.0/en/replication-options-binary-log.html#sysvar_max_binlog_size + // - During a server shutdown or restart when the binary log is enabled. + return nil +} + +func (lm *LogManager) PurgeLogFiles() error { + // TODO: implement support for purging older binlog files + // This also requires setting gtid_purged + // https://dev.mysql.com/doc/refman/8.0/en/replication-options-gtids.html#sysvar_gtid_purged + // Need to test the case where the GTID requested is not + // available –has been executed, but has been purged + return nil +} + +func (lm *LogManager) initializeCurrentLogFile(binlogFormat *mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) error { + // Open the file in append mode + // TODO: we should probably create this file as soon as possible, like when we construct LogManager + // TODO: But, we should only construct log manager when binlogging is enabled + file, err := os.OpenFile(lm.currentBinlogFilepath(), os.O_APPEND|os.O_CREATE|os.O_WRONLY, 0644) + if err != nil { + return err + } + lm.currentBinlogFile = file + + // Write Magic Number + _, err = file.Write(binlogFileMagicNumber) + if err != nil { + return err + } + + // TODO: Do we need to do this? + binlogFilePosition := uint64(0) + binlogEventMeta.NextLogPosition = uint32(binlogFilePosition) + + // Write Format Event + binlogEvent := mysql.NewFormatDescriptionEvent(*binlogFormat, binlogEventMeta) + binlogEventMeta.NextLogPosition += binlogEvent.Length() + _, err = lm.currentBinlogFile.Write(binlogEvent.Bytes()) + return err +} + +// WriteEvents writes |binlogEvents| to the current binlog file. +func (lm *LogManager) WriteEvents(binlogEvents []mysql.BinlogEvent) error { + // Write to the file + rotateLogFile := false + for _, event := range binlogEvents { + nextPosition := binary.LittleEndian.Uint32(event.Bytes()[13 : 13+4]) + if nextPosition > maxBinlogSize { + rotateLogFile = true + } + + if _, err := lm.currentBinlogFile.Write(event.Bytes()); err != nil { + return err + } + } + + if rotateLogFile { + // TODO: We need to rotate after this transaction... (i.e. set of events) + // TODO: How do the streamers react when we rotate the binlog file? + // If a streamer is reading a file, it needs to continue reading + // NOTE: FormatDescription event should be the very first entry in the binlog file, + // and a Rotate event should be the very last entry in the (completed) binlog file. + // Streamers will read the rotate event and know what file to open next. + return lm.RotateLogFile() + } + + return nil +} + +func (lm *LogManager) resolveLogFile(filename string) (string, error) { + binlogBaseDir, err := lm.fs.Abs(binlogDirectory) + if err != nil { + return "", err + } + + // TODO: Should we make sure it exists? + return filepath.Join(binlogBaseDir, filename), nil +} + +func (lm *LogManager) currentBinlogFilepath() string { + logFile, err := lm.resolveLogFile(lm.currentBinlogFileName) + if err != nil { + // TODO: return an error, or handle this err somewhere else where we do return an error + panic(err) + } + + return logFile +} + +// formatBinlogFilename formats a binlog filename using the specified |branch| and |sequence| number. The +// returned filename will be of the form "binlog-main.000001". +func formatBinlogFilename(branch string, sequence int) string { + return fmt.Sprintf("binlog-%s.%06d", branch, sequence) +} + +// parseBinlogFilename parses a binlog filename, of the form "binlog-main.000001", into its branch and +// sequence number. +func parseBinlogFilename(filename string) (branch string, sequence int, err error) { + if !strings.HasPrefix(filename, "binlog-") { + return "", 0, fmt.Errorf("invalid binlog filename: %s; must start with 'binlog-'", filename) + } + + filename = strings.TrimPrefix(filename, "binlog-") + + splits := strings.Split(filename, ".") + if len(splits) != 2 { + return "", 0, fmt.Errorf( + "unable to parse binlog filename: %s; expected format 'binlog-branch.sequence'", filename) + } + + branch = splits[0] + sequenceString := splits[1] + + sequence, err = strconv.Atoi(sequenceString) + if err != nil { + return "", 0, fmt.Errorf( + "unable to parse binlog sequence number: %s; %s", sequenceString, err.Error()) + } + + return branch, sequence, nil +} diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go index 6ddc4abf61c..b6df41fbb15 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go @@ -15,7 +15,10 @@ package binlogreplication import ( + "encoding/binary" "fmt" + "io" + "os" "sync" "time" @@ -49,44 +52,130 @@ func newBinlogStreamer() *binlogStreamer { // and |binlogEventMeta| records the position of the stream. This method blocks until an error // is received over the stream (e.g. the connection closing) or the streamer is closed, // through it's quit channel. -func (streamer *binlogStreamer) startStream(ctx *sql.Context, conn *mysql.Conn, binlogFormat *mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) error { - if err := sendInitialEvents(ctx, conn, binlogFormat, &binlogEventMeta); err != nil { +func (streamer *binlogStreamer) startStream(ctx *sql.Context, conn *mysql.Conn, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata, logfile string) error { + logrus.Errorf("Starting stream... (connection ID: %d)", conn.ConnectionID) + + if err := sendInitialEvents(ctx, conn, binlogFormat, binlogEventMeta); err != nil { return err } + // TODO: Maybe we should just ask the LogManager to give us the file for reading? + file, err := os.Open(logfile) + if err != nil { + return err + } + buffer := make([]byte, len(binlogFileMagicNumber)) + bytesRead, err := file.Read(buffer) + if err != nil { + return err + } + if bytesRead != len(binlogFileMagicNumber) || string(buffer) != string(binlogFileMagicNumber) { + return fmt.Errorf("invalid magic number in binlog file!") + } + + defer file.Close() + for { - logrus.StandardLogger().Trace("binlog streamer is listening for messages") + logrus.Trace("binlog streamer is listening for messages") select { case <-streamer.quitChan: - logrus.StandardLogger().Trace("received message from streamer's quit channel") + logrus.Trace("received message from streamer's quit channel") streamer.ticker.Stop() return nil case <-streamer.ticker.C: - logrus.StandardLogger().Trace("sending binlog heartbeat") - if err := sendHeartbeat(conn, binlogFormat, binlogEventMeta); err != nil { + logrus.Trace("sending binlog heartbeat") + if err := sendHeartbeat(conn, binlogFormat, *binlogEventMeta); err != nil { return err } if err := conn.FlushBuffer(); err != nil { return fmt.Errorf("unable to flush binlog connection: %s", err.Error()) } - case events := <-streamer.eventChan: - // TODO: If an error occurs while sending an event, it would be nice to have a retry at this - // level. Technically the replica should be abel to automatically reconnect and restart - // the stream from the last GTID it executed successfully, but it would be better to - // avoid the extra work for the reconnection and restart if possible. - logrus.StandardLogger().Tracef("streaming %d binlog events", len(events)) - for _, event := range events { - if err := conn.WriteBinlogEvent(event, false); err != nil { + // TODO: Remove streamer.eventChan! + //case events := <-streamer.eventChan: + // // TODO: If an error occurs while sending an event, it would be nice to have a retry at this + // // level. Technically the replica should be abel to automatically reconnect and restart + // // the stream from the last GTID it executed successfully, but it would be better to + // // avoid the extra work for the reconnection and restart if possible. + // logrus.Tracef("streaming %d binlog events", len(events)) + // for _, event := range events { + // if err := conn.WriteBinlogEvent(event, false); err != nil { + // return err + // } + // } + // if err := conn.FlushBuffer(); err != nil { + // return fmt.Errorf("unable to flush binlog connection: %s", err.Error()) + // } + + default: + // TODO: Start with a simple polling approach, but we may need to change to + // inotify or something more efficient in the future. + logrus.Debug("checking file for new data...") + eof := false + for !eof { + headerBuffer := make([]byte, 4+1+4+4+4+2) + bytesRead, err := file.Read(headerBuffer) + if err != nil && err != io.EOF { return err } - binlogEventMeta.NextLogPosition += event.Length() - } - if err := conn.FlushBuffer(); err != nil { - return fmt.Errorf("unable to flush binlog connection: %s", err.Error()) + if err == io.EOF { + time.Sleep(100 * time.Millisecond) + continue + } + + // Event Header: + //timestamp := headerBuffer[0:4] + //eventType := headerBuffer[4] + //serverId := binary.LittleEndian.Uint32(headerBuffer[5:5+4]) + eventSize := binary.LittleEndian.Uint32(headerBuffer[9 : 9+4]) + + payloadBuffer := make([]byte, eventSize-uint32(len(headerBuffer))) + bytesRead, err = file.Read(payloadBuffer) + if err != nil && err != io.EOF { + return err + } + if err == io.EOF { + time.Sleep(100 * time.Millisecond) + continue + } + logrus.Errorf("read %d bytes from binlog file", bytesRead) + + if bytesRead > 0 { + binlogEvent := mysql.NewMysql56BinlogEvent(append(headerBuffer, payloadBuffer...)) + + if binlogEvent.IsRotate() { + // TODO: if this is a rotate event, then we need to switch to the new binlog file after + // we write this event out to the replica. + binlogEvent.IsRotate() + newLogfile := payloadBuffer[8:] + logrus.Errorf("Rotatating to new binlog file: %s", newLogfile) + // TODO: We need a way to convert the bare filename to a full path, in the right directory + } + + // Components of Log File support: + // - streamers streaming from log files + // - rotating log files (on startup, and on flush logs, and on size threshold) + // - purging log files (on request, and automatically, aging out) + // - looking up starting point in log file, based on GTID + // TOTAL TIME ESTIMATE: 15 days? + + nextlogposition := binary.LittleEndian.Uint32(binlogEvent.Bytes()[13:17]) + logrus.Errorf("Next log position: %d", nextlogposition) + + err := conn.WriteBinlogEvent(binlogEvent, false) + if err != nil { + return err + } + + err = conn.FlushBuffer() + if err != nil { + return err + } + } } + time.Sleep(500 * time.Millisecond) } } } @@ -98,6 +187,7 @@ type binlogStreamerManager struct { streamers []*binlogStreamer streamersMutex sync.Mutex quitChan chan struct{} + logManager *LogManager } // NewBinlogStreamerManager creates a new binlogStreamerManager instance. @@ -143,7 +233,7 @@ func (m *binlogStreamerManager) StartStream(ctx *sql.Context, conn *mysql.Conn, m.addStreamer(streamer) defer m.removeStreamer(streamer) - return streamer.startStream(ctx, conn, binlogFormat, binlogEventMeta) + return streamer.startStream(ctx, conn, binlogFormat, &binlogEventMeta, m.logManager.currentBinlogFilepath()) } // sendEvents sends |binlogEvents| to all the streams managed by this instance. @@ -175,11 +265,17 @@ func (m *binlogStreamerManager) removeStreamer(streamer *binlogStreamer) { } } +// LogManager sets the LogManager this streamer manager will work with to find +// and read from binlog files. +func (m *binlogStreamerManager) LogManager(manager *LogManager) { + m.logManager = manager +} + func sendHeartbeat(conn *mysql.Conn, binlogFormat *mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) error { binlogEventMeta.Timestamp = uint32(0) // Timestamp is zero for a heartbeat event logrus.WithField("log_position", binlogEventMeta.NextLogPosition).Tracef("sending heartbeat") - binlogEvent := mysql.NewHeartbeatEventWithLogFile(*binlogFormat, binlogEventMeta, binlogFilename) + binlogEvent := mysql.NewHeartbeatEvent(*binlogFormat, binlogEventMeta) return conn.WriteBinlogEvent(binlogEvent, false) } diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go index dc35c69ed77..8c892cc3a6b 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go @@ -187,7 +187,6 @@ func TestBinlogPrimary_ReplicaRestart(t *testing.T) { // Make a change while the replica is stopped to test that the server // doesn't error out when a registered replica is not available. - // NOTE: This won't be replicated until we start persisting the binlog to disk primaryDatabase.MustExec("insert into db01.t1 values (1, 'one');") // Restart the MySQL replica and reconnect to the Dolt primary @@ -201,17 +200,12 @@ func TestBinlogPrimary_ReplicaRestart(t *testing.T) { // Create another table and assert that it gets replicated primaryDatabase.MustExec("create table db01.t2 (pk int primary key, c1 varchar(255));") - // We can't use waitForReplicaToCatchUp here, because it won't get the statement - // that was executed while the replica was stopped. Once we support replaying binlog - // events from a log file, we can switch this to waitForReplicaToCatchUp(t) waitForReplicaToHaveLatestGtid(t) requireReplicaResults(t, "show tables;", [][]any{{"t1"}, {"t2"}}) - // Assert the executed GTID position now contains GTID #2 and GTID #4 - // (#1 isn't present, because it was executed before we turned on replication, - // and #3 isn't present, because it was executed while the replica was stopped) + // Assert the executed GTID position now contains all GTIDs status = queryReplicaStatus(t) - require.Equal(t, serverUuid+":1-2:4", status["Executed_Gtid_Set"]) + require.Equal(t, serverUuid+":1-4", status["Executed_Gtid_Set"]) } // TestBinlogPrimary_PrimaryRestart tests that a Dolt primary server can be restarted and that a replica diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go index 8fa1fa515b5..bec45409711 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go @@ -39,11 +39,6 @@ import ( // BinlogBranch specifies the branch used for generating binlog events. var BinlogBranch = "main" -// binlogFilename is the name of the filename used in binlog events. Note that -// currently, this doesn't map to a real file on disk yet, but the filename is -// still needed for binlog messages. -var binlogFilename = "binlog-" + BinlogBranch + ".000001" - // binlogProducer implements the doltdb.DatabaseUpdateListener interface so that it can listen for updates to Dolt // databases and generate binlog events describing them. Those binlog events are sent to the binlogStreamerManager, // which is responsible for delivering them to each connected replica. @@ -82,6 +77,17 @@ func NewBinlogProducer(streamerManager *binlogStreamerManager) (*binlogProducer, }, nil } +func (b *binlogProducer) BinlogFormat() *mysql.BinlogFormat { + return b.binlogFormat +} + +// TODO: It's kinda weird for us to expose BinlogStream here... This type doesn't seem fully necessary, and +// +// stream seems like a bad name. Might be a better way to shape this in Vitess. +func (b *binlogProducer) BinlogStream() mysql.BinlogEventMetadata { + return b.binlogEventMeta +} + // WorkingRootUpdated implements the doltdb.DatabaseUpdateListener interface. When a working root changes, // this function generates events for the binary log and sends them to all connected replicas. // @@ -136,8 +142,7 @@ func (b *binlogProducer) WorkingRootUpdated(ctx *sql.Context, databaseName strin binlogEvents = append(binlogEvents, b.newXIDEvent()) } - b.streamerManager.sendEvents(binlogEvents) - return nil + return b.streamerManager.logManager.WriteEvents(binlogEvents) } // DatabaseCreated implements the doltdb.DatabaseUpdateListener interface. @@ -156,8 +161,7 @@ func (b *binlogProducer) DatabaseCreated(ctx *sql.Context, databaseName string) createDatabaseStatement := fmt.Sprintf("create database `%s`;", databaseName) binlogEvents = append(binlogEvents, b.newQueryEvent(databaseName, createDatabaseStatement)) - b.streamerManager.sendEvents(binlogEvents) - return nil + return b.streamerManager.logManager.WriteEvents(binlogEvents) } // DatabaseDropped implements the doltdb.DatabaseUpdateListener interface. @@ -172,8 +176,7 @@ func (b *binlogProducer) DatabaseDropped(ctx *sql.Context, databaseName string) dropDatabaseStatement := fmt.Sprintf("drop database `%s`;", databaseName) binlogEvents = append(binlogEvents, b.newQueryEvent(databaseName, dropDatabaseStatement)) - b.streamerManager.sendEvents(binlogEvents) - return nil + return b.streamerManager.logManager.WriteEvents(binlogEvents) } // initializeGtidPosition loads the persisted GTID position from disk and initializes it From 97446fa41f744afb0b219a6375328ef88a5df602 Mon Sep 17 00:00:00 2001 From: Jason Fulghum Date: Mon, 17 Jun 2024 13:13:16 -0700 Subject: [PATCH 02/14] Changing the log manager to be responsible for setting the correct next log position in events written to file --- go/cmd/dolt/commands/engine/sqlengine.go | 3 - go/cmd/dolt/commands/sqlserver/server.go | 21 ++- .../binlog_primary_controller.go | 6 +- .../binlog_primary_log_manager.go | 171 +++++++++++++----- .../binlog_primary_streamer.go | 128 +++++++------ .../binlogreplication/binlog_primary_test.go | 28 ++- .../sqle/binlogreplication/binlog_producer.go | 54 ++---- .../binlog_replication_test.go | 6 +- 8 files changed, 255 insertions(+), 162 deletions(-) diff --git a/go/cmd/dolt/commands/engine/sqlengine.go b/go/cmd/dolt/commands/engine/sqlengine.go index 5a6e8501f59..59938bba290 100644 --- a/go/cmd/dolt/commands/engine/sqlengine.go +++ b/go/cmd/dolt/commands/engine/sqlengine.go @@ -35,7 +35,6 @@ import ( "github.com/dolthub/dolt/go/cmd/dolt/cli" "github.com/dolthub/dolt/go/libraries/doltcore/branch_control" "github.com/dolthub/dolt/go/libraries/doltcore/dconfig" - "github.com/dolthub/dolt/go/libraries/doltcore/doltdb" "github.com/dolthub/dolt/go/libraries/doltcore/env" "github.com/dolthub/dolt/go/libraries/doltcore/servercfg" dsqle "github.com/dolthub/dolt/go/libraries/doltcore/sqle" @@ -154,8 +153,6 @@ func NewSqlEngine( if err := configureBinlogPrimaryController(engine); err != nil { return nil, err } - pro.AddInitDatabaseHook(dblr.NewBinlogInitDatabaseHook(ctx, doltdb.DatabaseUpdateListeners)) - pro.AddDropDatabaseHook(dblr.NewBinlogDropDatabaseHook(ctx, doltdb.DatabaseUpdateListeners)) config.ClusterController.SetIsStandbyCallback(func(isStandby bool) { pro.SetIsStandby(isStandby) diff --git a/go/cmd/dolt/commands/sqlserver/server.go b/go/cmd/dolt/commands/sqlserver/server.go index b5d1da54f58..47ea2f413f3 100644 --- a/go/cmd/dolt/commands/sqlserver/server.go +++ b/go/cmd/dolt/commands/sqlserver/server.go @@ -287,10 +287,8 @@ func ConfigureServices( InitBinlogging := &svcs.AnonService{ InitF: func(context.Context) error { - primaryController1 := sqlEngine.GetUnderlyingEngine().Analyzer.Catalog.BinlogPrimaryController - - // TODO: rename to doltBinlogPrimaryController - primaryController, ok := primaryController1.(*binlogreplication.DoltBinlogPrimaryController) + primaryController := sqlEngine.GetUnderlyingEngine().Analyzer.Catalog.BinlogPrimaryController + doltBinlogPrimaryController, ok := primaryController.(*binlogreplication.DoltBinlogPrimaryController) if !ok { panic("Found unexpected type of binlog controller!") } @@ -305,26 +303,29 @@ func ConfigureServices( } if logBin == 1 { logrus.Debug("Enabling binary logging") - binlogProducer, err := binlogreplication.NewBinlogProducer(primaryController.StreamerManager()) + binlogProducer, err := binlogreplication.NewBinlogProducer(doltBinlogPrimaryController.StreamerManager()) if err != nil { return err } // NOTE: Hooks need to be applied AFTER this! doltdb.RegisterDatabaseUpdateListener(binlogProducer) - primaryController.BinlogProducer = binlogProducer + doltBinlogPrimaryController.BinlogProducer = binlogProducer // TODO: This is a mess! How to clean this up! provider := sqlEngine.GetUnderlyingEngine().Analyzer.Catalog.DbProvider if doltProvider, ok := provider.(*sqle.DoltDatabaseProvider); ok { - doltProvider.InitDatabaseHooks = append(doltProvider.InitDatabaseHooks, binlogreplication.NewBinlogInitDatabaseHook(nil, doltdb.DatabaseUpdateListeners)) - doltProvider.DropDatabaseHooks = append(doltProvider.DropDatabaseHooks, binlogreplication.NewBinlogDropDatabaseHook(nil, doltdb.DatabaseUpdateListeners)) + doltProvider.AddInitDatabaseHook(binlogreplication.NewBinlogInitDatabaseHook(nil, doltdb.DatabaseUpdateListeners)) + doltProvider.AddDropDatabaseHook(binlogreplication.NewBinlogDropDatabaseHook(nil, doltdb.DatabaseUpdateListeners)) } // TODO: How do we feed the binlogStream and binlogFormat to the log manager? // Needs format to write the initial format event to the stream // Needs binlogStream to write the initial format event - logManager := binlogreplication.NewLogManager(fs, binlogProducer.BinlogFormat(), binlogProducer.BinlogStream()) - primaryController.StreamerManager().LogManager(logManager) + logManager, err := binlogreplication.NewLogManager(fs, *binlogProducer.BinlogFormat(), binlogProducer.BinlogStream()) + if err != nil { + return err + } + doltBinlogPrimaryController.StreamerManager().LogManager(logManager) } _, logBinBranchValue, ok := sql.SystemVariables.GetGlobal("log_bin_branch") diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go index 093e4500c4f..c8c7e7f844c 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go @@ -61,12 +61,12 @@ func (d *DoltBinlogPrimaryController) RegisterReplica(ctx *sql.Context, c *mysql } // BinlogDumpGtid implements the BinlogPrimaryController interface. -func (d *DoltBinlogPrimaryController) BinlogDumpGtid(ctx *sql.Context, conn *mysql.Conn, gtidSet mysql.GTIDSet) error { +func (d *DoltBinlogPrimaryController) BinlogDumpGtid(ctx *sql.Context, conn *mysql.Conn, replicaExecutedGtids mysql.GTIDSet) error { if d.BinlogProducer == nil { return fmt.Errorf("no binlog currently being recorded; make sure the server is started with @@log_bin enabled") } - err := d.streamerManager.StartStream(ctx, conn, d.BinlogProducer.binlogFormat, d.BinlogProducer.binlogEventMeta) + err := d.streamerManager.StartStream(ctx, conn, replicaExecutedGtids, d.BinlogProducer.binlogFormat, d.BinlogProducer.binlogEventMeta) if err != nil { logrus.Warnf("exiting binlog streamer due to error: %s", err.Error()) } else { @@ -91,7 +91,7 @@ func (d *DoltBinlogPrimaryController) ListBinaryLogs(_ *sql.Context) ([]binlogre func (d *DoltBinlogPrimaryController) GetBinaryLogStatus(ctx *sql.Context) ([]binlogreplication.BinaryLogStatus, error) { return []binlogreplication.BinaryLogStatus{{ File: d.streamerManager.logManager.currentBinlogFileName, - Position: uint(d.BinlogProducer.binlogEventMeta.NextLogPosition), + Position: uint(d.streamerManager.logManager.currentPosition), ExecutedGtids: d.BinlogProducer.currentGtidPosition(), }}, nil } diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go index 7681306909f..3e8533aad33 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go @@ -17,21 +17,19 @@ package binlogreplication import ( "encoding/binary" "fmt" - "github.com/dolthub/dolt/go/libraries/utils/filesys" "os" "path/filepath" "strconv" "strings" + "sync" "github.com/dolthub/vitess/go/mysql" -) + "github.com/sirupsen/logrus" -// maxBinlogSize is the maximum size of a binlog file, before the binlog writer rotates to a new binlog file. Once -// a binlog file reaches this size, or greater, the events from the next transaction should be written to a new file. -// Note that all events in a single transaction should be written to the same binlog file. -// This corresponds to the @@max_binlog_size system variable in MySQL: -// https://dev.mysql.com/doc/refman/8.0/en/replication-options-binary-log.html#sysvar_max_binlog_size -const maxBinlogSize = 1024 * 1024 * 1024 + "github.com/dolthub/dolt/go/libraries/utils/filesys" + "github.com/dolthub/go-mysql-server/sql" + gmstypes "github.com/dolthub/go-mysql-server/sql/types" +) var binlogDirectory = filepath.Join(".dolt", "binlog") @@ -39,20 +37,29 @@ var binlogDirectory = filepath.Join(".dolt", "binlog") // MySQL binlog file and identify the file as a MySQL binlog. var binlogFileMagicNumber = []byte{0xfe, 0x62, 0x69, 0x6e} +// LogManager is responsible for the binary log files on disk, including actually writing events to the log files, +// rotating the log files, listing the available log files, and purging old log files. type LogManager struct { + mu *sync.Mutex currentBinlogFile *os.File currentBinlogFileName string + currentPosition int fs filesys.Filesys + binlogFormat mysql.BinlogFormat + binlogEventMeta mysql.BinlogEventMetadata } // NewLogManager creates a new LogManager instance where binlog files are stored in the .dolt/binlog directory // underneath the specified |fs| filesystem. The |binlogFormat| and |binlogStream| are used to initialize the // new binlog file. -func NewLogManager(fs filesys.Filesys, binlogFormat *mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) *LogManager { +func NewLogManager(fs filesys.Filesys, binlogFormat mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) (*LogManager, error) { // TODO: On server startup, we need to find the most recent binlog file, add a rotate event at the end (if necessary?), and start a new file. Documentation seems to indicate that a rotate event is added at the end of a binlog file, so that the streamer can jump to the next file, but I don't see this in our MySQL sample binlog files. Need to do more testing here. lm := &LogManager{ - fs: fs, + mu: &sync.Mutex{}, + fs: fs, + binlogFormat: binlogFormat, + binlogEventMeta: binlogEventMeta, } // TODO: Could resolve the base dir for the binlog file directory here; would it help us avoid returning errors in other APIs? @@ -60,28 +67,26 @@ func NewLogManager(fs filesys.Filesys, binlogFormat *mysql.BinlogFormat, binlogE // Initialize binlog file storage (extract to function!) err := fs.MkDirs(binlogDirectory) if err != nil { - panic(err) + return nil, err } // Initialize current binlog file nextLogFilename, err := lm.nextLogFile() if err != nil { - panic(err) + return nil, err } lm.currentBinlogFileName = nextLogFilename - // Ugh... we need binlogFormat and binlogEventMeta in order to do this... - // Actually... Do we need binlogEventMeta, or could we fake it? We only need binlogEventMeta so that - // Vitess can call a function on that instance, and for the server Id. The position in the file - // should always be zero at this point, so maybe we could clean this up more? - err = lm.initializeCurrentLogFile(binlogFormat, binlogEventMeta) - if err != nil { - panic(err) + if err = lm.initializeCurrentLogFile(binlogFormat, binlogEventMeta); err != nil { + return nil, err } - return lm + return lm, nil } +// nextLogFile returns the filename of the next bin log file in the current sequence. For example, if the +// current log file is "binlog-main.000008" the nextLogFile() would return "binlog-main.000009". Note that +// this function returns the file name only, not the full file path. func (lm *LogManager) nextLogFile() (filename string, err error) { mostRecentLogfile, err := lm.mostRecentLogFileForBranch(BinlogBranch) if err != nil { @@ -155,13 +160,31 @@ func (lm *LogManager) mostRecentLogFileForBranch(branch string) (logFile string, } } +// RotateLogFile rotates the current log file that is actively being written to. A new binlog file is created and +// initialized, including writing the first four bytes with the binlog magic number, and the old binlog file is closed. +// Rotation should occur when an administrator explicitly requests it with the `FLUSH LOGS` statement, during server +// shutdown or restart, or when the current binary log file size exceeds the maximum size defined by the +// @@max_binlog_size system variable. func (lm *LogManager) RotateLogFile() error { - // TODO: Handle logfile rotation: - // - Manual rotation from the FLUSH LOGS statement (Out of scope) - // - Automatically when the binary log file reaches the maximum size defined by the max_binlog_size configuration parameter. - // https://dev.mysql.com/doc/refman/8.0/en/replication-options-binary-log.html#sysvar_max_binlog_size - // - During a server shutdown or restart when the binary log is enabled. - return nil + nextLogFile, err := lm.nextLogFile() + if err != nil { + return err + } + logrus.Tracef("Rotating bin log file to: %s", nextLogFile) + + binlogEvent := mysql.NewRotateEvent(lm.binlogFormat, lm.binlogEventMeta, 0, nextLogFile) + if err = lm.writeEventsHelper(binlogEvent); err != nil { + return err + } + + // Close the current binlog file + if err = lm.currentBinlogFile.Close(); err != nil { + logrus.Errorf("error closing current binlog file before rotating to new file: %s", err.Error()) + } + + // Open and initialize a new binlog file + lm.currentBinlogFileName = nextLogFile + return lm.initializeCurrentLogFile(lm.binlogFormat, lm.binlogEventMeta) } func (lm *LogManager) PurgeLogFiles() error { @@ -173,54 +196,110 @@ func (lm *LogManager) PurgeLogFiles() error { return nil } -func (lm *LogManager) initializeCurrentLogFile(binlogFormat *mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) error { +// initializeCurrentLogFile creates and opens the current binlog file for append only writing, writes the first four +// bytes with the binlog magic numbers, then writes a Format Description event and a Previous GTIDs event. +func (lm *LogManager) initializeCurrentLogFile(binlogFormat mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) error { + logrus.Tracef("Initializing binlog file: %s", lm.currentBinlogFilepath()) + // Open the file in append mode - // TODO: we should probably create this file as soon as possible, like when we construct LogManager - // TODO: But, we should only construct log manager when binlogging is enabled file, err := os.OpenFile(lm.currentBinlogFilepath(), os.O_APPEND|os.O_CREATE|os.O_WRONLY, 0644) if err != nil { return err } lm.currentBinlogFile = file + lm.currentPosition = 0 // Write Magic Number _, err = file.Write(binlogFileMagicNumber) if err != nil { return err } + lm.currentPosition += len(binlogFileMagicNumber) - // TODO: Do we need to do this? - binlogFilePosition := uint64(0) - binlogEventMeta.NextLogPosition = uint32(binlogFilePosition) + // Write Format Description Event, the first event in each binlog file + binlogEvent := mysql.NewFormatDescriptionEvent(binlogFormat, binlogEventMeta) + if err = lm.writeEventsHelper(binlogEvent); err != nil { + return err + } + + // Write the Previous GTIDs event + // TODO: Instead of using the @@gtid_executed system variable, LogManager could keep track of which GTIDs + // it has seen logged and use that as the source of truth for the Previous GTIDs event. This would + // eliminate a race condition. In general, LogManager needs to track which GTIDs are represented in + // which log files better to support clients seeking to the right point in the stream. + _, rawValue, ok := sql.SystemVariables.GetGlobal("gtid_executed") + if !ok { + panic("unable to find @@gtid_executed system variable") + } + stringValue, ok := rawValue.(string) + if !ok { + panic(fmt.Sprintf("unexpected type for @@gtid_executed system variable: %T", rawValue)) + } - // Write Format Event - binlogEvent := mysql.NewFormatDescriptionEvent(*binlogFormat, binlogEventMeta) - binlogEventMeta.NextLogPosition += binlogEvent.Length() - _, err = lm.currentBinlogFile.Write(binlogEvent.Bytes()) - return err + gtidSet, err := mysql.ParseMysql56GTIDSet(stringValue) + if err != nil { + return err + } + return lm.writeEventsHelper(mysql.NewPreviousGtidsEvent(binlogFormat, binlogEventMeta, gtidSet.(mysql.Mysql56GTIDSet))) } -// WriteEvents writes |binlogEvents| to the current binlog file. -func (lm *LogManager) WriteEvents(binlogEvents []mysql.BinlogEvent) error { +// lookupMaxBinlogSize looks up the value of the @@max_binlog_size system variable and returns it, along with any +// errors encountered while looking it up. +func lookupMaxBinlogSize() (int, error) { + _, value, ok := sql.SystemVariables.GetGlobal("max_binlog_size") + if !ok { + return 0, fmt.Errorf("system variable @@max_binlog_size not found") + } + + intValue, _, err := gmstypes.Int32.Convert(value) + if err != nil { + return 0, err + } + return int(intValue.(int32)), nil +} + +// WriteEvents writes |binlogEvents| to the current binlog file. Access to write to the binary log is synchronized, +// so that only one thread can write to the log file at a time. +func (lm *LogManager) WriteEvents(binlogEvents ...mysql.BinlogEvent) error { + // synchronize on WriteEvents so that only one thread is writing to the log file at a time + lm.mu.Lock() + defer lm.mu.Unlock() + + return lm.writeEventsHelper(binlogEvents...) +} + +// writeEventsHelper writes |binlogEvents| to the current binlog file. This function is NOT synchronized, and is only +// intended to be used from code inside LogManager that needs to be called transitively from the WriteEvents method. +func (lm *LogManager) writeEventsHelper(binlogEvents ...mysql.BinlogEvent) error { + maxBinlogSize, err := lookupMaxBinlogSize() + if err != nil { + return err + } + // Write to the file rotateLogFile := false for _, event := range binlogEvents { - nextPosition := binary.LittleEndian.Uint32(event.Bytes()[13 : 13+4]) - if nextPosition > maxBinlogSize { + // NOTE: When we write the event to file, we need to ensure the next log position field + // is correct. That means we have to serialize the events going into the log file and + // we update their NextLogPosition field in the header to ensure it's correct. Because + // we change the packet, we must recompute the checksum. + nextPosition := lm.currentPosition + len(event.Bytes()) + binary.LittleEndian.PutUint32(event.Bytes()[13:13+4], uint32(nextPosition)) + mysql.UpdateChecksum(lm.binlogFormat, event) + + lm.currentPosition = nextPosition + if nextPosition > maxBinlogSize && !event.IsRotate() { rotateLogFile = true } + // Write the event to file if _, err := lm.currentBinlogFile.Write(event.Bytes()); err != nil { return err } } if rotateLogFile { - // TODO: We need to rotate after this transaction... (i.e. set of events) - // TODO: How do the streamers react when we rotate the binlog file? - // If a streamer is reading a file, it needs to continue reading - // NOTE: FormatDescription event should be the very first entry in the binlog file, - // and a Rotate event should be the very last entry in the (completed) binlog file. + // NOTE: Rotate event should be the very last entry in the (completed) binlog file. // Streamers will read the rotate event and know what file to open next. return lm.RotateLogFile() } diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go index b6df41fbb15..55afb1b450d 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go @@ -19,6 +19,7 @@ import ( "fmt" "io" "os" + "path/filepath" "sync" "time" @@ -32,17 +33,15 @@ import ( // It also sends heartbeat events to the replica over the same connection at // regular intervals. There is one streamer per connected replica. type binlogStreamer struct { - quitChan chan struct{} - eventChan chan []mysql.BinlogEvent - ticker *time.Ticker + quitChan chan struct{} + ticker *time.Ticker } // NewBinlogStreamer creates a new binlogStreamer instance. func newBinlogStreamer() *binlogStreamer { return &binlogStreamer{ - quitChan: make(chan struct{}), - eventChan: make(chan []mysql.BinlogEvent, 5), - ticker: time.NewTicker(30 * time.Second), + quitChan: make(chan struct{}), + ticker: time.NewTicker(30 * time.Second), } } @@ -52,26 +51,27 @@ func newBinlogStreamer() *binlogStreamer { // and |binlogEventMeta| records the position of the stream. This method blocks until an error // is received over the stream (e.g. the connection closing) or the streamer is closed, // through it's quit channel. -func (streamer *binlogStreamer) startStream(ctx *sql.Context, conn *mysql.Conn, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata, logfile string) error { - logrus.Errorf("Starting stream... (connection ID: %d)", conn.ConnectionID) +func (streamer *binlogStreamer) startStream(_ *sql.Context, conn *mysql.Conn, executedGtids mysql.GTIDSet, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata, logfile string) error { + logrus.Tracef("Starting binlog stream... (connection ID: %d)", conn.ConnectionID) if err := sendInitialEvents(ctx, conn, binlogFormat, binlogEventMeta); err != nil { return err } // TODO: Maybe we should just ask the LogManager to give us the file for reading? - file, err := os.Open(logfile) + file, err := openBinlogFileForReading(logfile) if err != nil { return err } - buffer := make([]byte, len(binlogFileMagicNumber)) - bytesRead, err := file.Read(buffer) - if err != nil { + + binlogEventMetaCopy := binlogEventMeta + binlogEventMetaCopy.NextLogPosition = 0 + + rotateEvent := mysql.NewFakeRotateEvent(*binlogFormat, binlogEventMetaCopy, filepath.Base(logfile)) + if err = conn.WriteBinlogEvent(rotateEvent, false); err != nil { return err } - if bytesRead != len(binlogFileMagicNumber) || string(buffer) != string(binlogFileMagicNumber) { - return fmt.Errorf("invalid magic number in binlog file!") - } + _ = conn.FlushBuffer() defer file.Close() @@ -93,27 +93,10 @@ func (streamer *binlogStreamer) startStream(ctx *sql.Context, conn *mysql.Conn, return fmt.Errorf("unable to flush binlog connection: %s", err.Error()) } - // TODO: Remove streamer.eventChan! - //case events := <-streamer.eventChan: - // // TODO: If an error occurs while sending an event, it would be nice to have a retry at this - // // level. Technically the replica should be abel to automatically reconnect and restart - // // the stream from the last GTID it executed successfully, but it would be better to - // // avoid the extra work for the reconnection and restart if possible. - // logrus.Tracef("streaming %d binlog events", len(events)) - // for _, event := range events { - // if err := conn.WriteBinlogEvent(event, false); err != nil { - // return err - // } - // } - // if err := conn.FlushBuffer(); err != nil { - // return fmt.Errorf("unable to flush binlog connection: %s", err.Error()) - // } - default: - // TODO: Start with a simple polling approach, but we may need to change to - // inotify or something more efficient in the future. logrus.Debug("checking file for new data...") eof := false + skippingGtids := false for !eof { headerBuffer := make([]byte, 4+1+4+4+4+2) bytesRead, err := file.Read(headerBuffer) @@ -121,6 +104,7 @@ func (streamer *binlogStreamer) startStream(ctx *sql.Context, conn *mysql.Conn, return err } if err == io.EOF { + logrus.Tracef("End of binlog file! Waiting for new events...") time.Sleep(100 * time.Millisecond) continue } @@ -137,40 +121,54 @@ func (streamer *binlogStreamer) startStream(ctx *sql.Context, conn *mysql.Conn, return err } if err == io.EOF { + logrus.Tracef("End of binlog file! Waiting for new events...") time.Sleep(100 * time.Millisecond) continue } - logrus.Errorf("read %d bytes from binlog file", bytesRead) if bytesRead > 0 { binlogEvent := mysql.NewMysql56BinlogEvent(append(headerBuffer, payloadBuffer...)) if binlogEvent.IsRotate() { - // TODO: if this is a rotate event, then we need to switch to the new binlog file after - // we write this event out to the replica. - binlogEvent.IsRotate() - newLogfile := payloadBuffer[8:] + newLogfile := string(payloadBuffer[8:(len(payloadBuffer) - 4)]) logrus.Errorf("Rotatating to new binlog file: %s", newLogfile) - // TODO: We need a way to convert the bare filename to a full path, in the right directory + + dir := filepath.Dir(logfile) + newLogfile = filepath.Join(dir, newLogfile) + + if err = file.Close(); err != nil { + logrus.Errorf("unable to close previous binlog file: %s", err.Error()) + } + + if file, err = openBinlogFileForReading(newLogfile); err != nil { + return err + } + + continue } - // Components of Log File support: - // - streamers streaming from log files - // - rotating log files (on startup, and on flush logs, and on size threshold) - // - purging log files (on request, and automatically, aging out) - // - looking up starting point in log file, based on GTID - // TOTAL TIME ESTIMATE: 15 days? + if binlogEvent.IsGTID() { + gtid, _, err := binlogEvent.GTID(*binlogFormat) + if err != nil { + return err + } + + // If the replica has already executed this GTID, then skip it. + if executedGtids.ContainsGTID(gtid) { + skippingGtids = true + } else { + skippingGtids = false + } + } - nextlogposition := binary.LittleEndian.Uint32(binlogEvent.Bytes()[13:17]) - logrus.Errorf("Next log position: %d", nextlogposition) + if skippingGtids { + continue + } - err := conn.WriteBinlogEvent(binlogEvent, false) - if err != nil { + if err := conn.WriteBinlogEvent(binlogEvent, false); err != nil { return err } - - err = conn.FlushBuffer() - if err != nil { + if err = conn.FlushBuffer(); err != nil { return err } } @@ -180,6 +178,28 @@ func (streamer *binlogStreamer) startStream(ctx *sql.Context, conn *mysql.Conn, } } +// openBinlogFileForReading opens the specified |logfile| for reading and reads the first four bytes to make sure they +// are the expected binlog file magic numbers. If any problems are encountered opening the file or reading the first +// four bytes, an error is returned. +func openBinlogFileForReading(logfile string) (*os.File, error) { + logrus.Errorf("Opening binlog file: %s", logfile) + + file, err := os.Open(logfile) + if err != nil { + return nil, err + } + buffer := make([]byte, len(binlogFileMagicNumber)) + bytesRead, err := file.Read(buffer) + if err != nil { + return nil, err + } + if bytesRead != len(binlogFileMagicNumber) || string(buffer) != string(binlogFileMagicNumber) { + return nil, fmt.Errorf("invalid magic number in binlog file!") + } + + return file, nil +} + // binlogStreamerManager manages a collection of binlogStreamers, one for reach connected replica, // and implements the doltdb.DatabaseUpdateListener interface to receive notifications of database // changes that need to be turned into binlog events and then sent to connected replicas. @@ -228,12 +248,12 @@ func (m *binlogStreamerManager) copyStreamers() []*binlogStreamer { // is closed, the streamer is sent a quit signal over its quit channel, or the streamer receives // errors while sending events over the connection. Note that this method blocks until the // streamer exits. -func (m *binlogStreamerManager) StartStream(ctx *sql.Context, conn *mysql.Conn, binlogFormat *mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) error { +func (m *binlogStreamerManager) StartStream(ctx *sql.Context, conn *mysql.Conn, executedGtids mysql.GTIDSet, binlogFormat *mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) error { streamer := newBinlogStreamer() m.addStreamer(streamer) defer m.removeStreamer(streamer) - return streamer.startStream(ctx, conn, binlogFormat, &binlogEventMeta, m.logManager.currentBinlogFilepath()) + return streamer.startStream(ctx, conn, executedGtids, binlogFormat, &binlogEventMeta, m.logManager.currentBinlogFilepath()) } // sendEvents sends |binlogEvents| to all the streams managed by this instance. diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go index 8c892cc3a6b..5fb3b06f14d 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go @@ -124,7 +124,33 @@ func TestBinlogPrimary(t *testing.T) { }) requirePrimaryResults(t, "SHOW BINARY LOG STATUS", [][]any{ - {"binlog-main.000001", "2226", "", "", uuid + ":1-3"}}) + {"binlog-main.000001", "2377", "", "", uuid + ":1-3"}}) +} + +// TestBinlogPrimary_Rotation tests how a Dolt primary server handles rotating the binary log file when the +// size threshold is reached. +func TestBinlogPrimary_Rotation(t *testing.T) { + defer teardown(t) + startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) + setupForDoltToMySqlReplication() + startReplication(t, doltPort) + + // Change the binlog rotation threshold on the primary to 10KB (instead of the default 1GB) + primaryDatabase.MustExec("SET @@GLOBAL.max_binlog_size = 10240;") + + // Generate enough data to trigger a logfile rotation + primaryDatabase.MustExec("create table t (n int);") + for i := range 100 { + primaryDatabase.MustExec(fmt.Sprintf("insert into t values (%d);", i)) + } + waitForReplicaToCatchUp(t) + requireReplicaResults(t, "SELECT MAX(n) FROM t;", [][]any{{"99"}}) + + // Check the binary log file status and ensure the file has been rotated + uuid := queryPrimaryServerUuid(t) + requirePrimaryResults(t, "show binary log status", [][]any{ + {"binlog-main.000003", "1027", "", "", uuid + ":1-102"}, + }) } // TestBinlogPrimary_Heartbeats tests that heartbeats sent from the primary to the replica are well-formed and diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go index bec45409711..2d1b3ffc8dc 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go @@ -142,7 +142,7 @@ func (b *binlogProducer) WorkingRootUpdated(ctx *sql.Context, databaseName strin binlogEvents = append(binlogEvents, b.newXIDEvent()) } - return b.streamerManager.logManager.WriteEvents(binlogEvents) + return b.streamerManager.logManager.WriteEvents(binlogEvents...) } // DatabaseCreated implements the doltdb.DatabaseUpdateListener interface. @@ -161,7 +161,7 @@ func (b *binlogProducer) DatabaseCreated(ctx *sql.Context, databaseName string) createDatabaseStatement := fmt.Sprintf("create database `%s`;", databaseName) binlogEvents = append(binlogEvents, b.newQueryEvent(databaseName, createDatabaseStatement)) - return b.streamerManager.logManager.WriteEvents(binlogEvents) + return b.streamerManager.logManager.WriteEvents(binlogEvents...) } // DatabaseDropped implements the doltdb.DatabaseUpdateListener interface. @@ -176,7 +176,7 @@ func (b *binlogProducer) DatabaseDropped(ctx *sql.Context, databaseName string) dropDatabaseStatement := fmt.Sprintf("drop database `%s`;", databaseName) binlogEvents = append(binlogEvents, b.newQueryEvent(databaseName, dropDatabaseStatement)) - return b.streamerManager.logManager.WriteEvents(binlogEvents) + return b.streamerManager.logManager.WriteEvents(binlogEvents...) } // initializeGtidPosition loads the persisted GTID position from disk and initializes it @@ -258,7 +258,6 @@ func (b *binlogProducer) createGtidEvent(ctx *sql.Context) (mysql.BinlogEvent, e gtid := mysql.Mysql56GTID{Server: sid, Sequence: b.gtidSequence} binlogEvent := mysql.NewMySQLGTIDEvent(*b.binlogFormat, b.binlogEventMeta, gtid, false) - b.binlogEventMeta.NextLogPosition += binlogEvent.Length() b.gtidSequence++ // Store the latest executed GTID to disk @@ -517,73 +516,43 @@ func (b *binlogProducer) currentGtidPosition() string { // newQueryEvent creates a new Query BinlogEvent for the specified |databaseName| and |query|, and updates the // stream's log position. func (b *binlogProducer) newQueryEvent(databaseName, query string) mysql.BinlogEvent { - b.mu.Lock() - defer b.mu.Unlock() - // TODO: Charset and SQL_MODE support - binlogEvent := mysql.NewQueryEvent(*b.binlogFormat, b.binlogEventMeta, mysql.Query{ + return mysql.NewQueryEvent(*b.binlogFormat, b.binlogEventMeta, mysql.Query{ Database: databaseName, Charset: nil, SQL: query, Options: 0, SqlMode: 0, }) - b.binlogEventMeta.NextLogPosition += binlogEvent.Length() - return binlogEvent } // newXIDEvent returns a new XID BinlogEvent and updates the stream's log position. func (b *binlogProducer) newXIDEvent() mysql.BinlogEvent { - b.mu.Lock() - defer b.mu.Unlock() - - binlogEvent := mysql.NewXIDEvent(*b.binlogFormat, b.binlogEventMeta) - b.binlogEventMeta.NextLogPosition += binlogEvent.Length() - return binlogEvent + return mysql.NewXIDEvent(*b.binlogFormat, b.binlogEventMeta) } // newTableMapEvent returns a new TableMap BinlogEvent for the specified |tableId| and |tableMap|, and updates the // stream's log position. func (b *binlogProducer) newTableMapEvent(tableId uint64, tableMap *mysql.TableMap) mysql.BinlogEvent { - b.mu.Lock() - defer b.mu.Unlock() - - binlogEvent := mysql.NewTableMapEvent(*b.binlogFormat, b.binlogEventMeta, tableId, tableMap) - b.binlogEventMeta.NextLogPosition += binlogEvent.Length() - return binlogEvent + return mysql.NewTableMapEvent(*b.binlogFormat, b.binlogEventMeta, tableId, tableMap) } // newWriteRowsEvent returns a new WriteRows BinlogEvent for the specified |tableId| and |rows|, and updates the // stream's log position. func (b *binlogProducer) newWriteRowsEvent(tableId uint64, rows mysql.Rows) mysql.BinlogEvent { - b.mu.Lock() - defer b.mu.Unlock() - - binlogEvent := mysql.NewWriteRowsEvent(*b.binlogFormat, b.binlogEventMeta, tableId, rows) - b.binlogEventMeta.NextLogPosition += binlogEvent.Length() - return binlogEvent + return mysql.NewWriteRowsEvent(*b.binlogFormat, b.binlogEventMeta, tableId, rows) } // newDeleteRowsEvent returns a new DeleteRows BinlogEvent for the specified |tableId| and |rows|, and updates the // stream's log position. func (b *binlogProducer) newDeleteRowsEvent(tableId uint64, rows mysql.Rows) mysql.BinlogEvent { - b.mu.Lock() - defer b.mu.Unlock() - - binlogEvent := mysql.NewDeleteRowsEvent(*b.binlogFormat, b.binlogEventMeta, tableId, rows) - b.binlogEventMeta.NextLogPosition += binlogEvent.Length() - return binlogEvent + return mysql.NewDeleteRowsEvent(*b.binlogFormat, b.binlogEventMeta, tableId, rows) } // newUpdateRowsEvent returns a new UpdateRows BinlogEvent for the specified |tableId| and |rows|, and updates the // stream's log position. func (b *binlogProducer) newUpdateRowsEvent(tableId uint64, rows mysql.Rows) mysql.BinlogEvent { - b.mu.Lock() - defer b.mu.Unlock() - - binlogEvent := mysql.NewUpdateRowsEvent(*b.binlogFormat, b.binlogEventMeta, tableId, rows) - b.binlogEventMeta.NextLogPosition += binlogEvent.Length() - return binlogEvent + return mysql.NewUpdateRowsEvent(*b.binlogFormat, b.binlogEventMeta, tableId, rows) } // extractRowCountAndDiffType uses |sch| and |diff| to determine how many changed rows this @@ -719,8 +688,7 @@ func createBinlogEventMetadata() (*mysql.BinlogEventMetadata, error) { } return &mysql.BinlogEventMetadata{ - ServerID: serverId, - NextLogPosition: 0, - Timestamp: uint32(time.Now().Unix()), + ServerID: serverId, + Timestamp: uint32(time.Now().Unix()), }, nil } diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_test.go index fff3e209d3b..80befb065dc 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_test.go @@ -455,10 +455,11 @@ func TestCharsetsAndCollations(t *testing.T) { // Test Helper Functions // -// waitForReplicaToCatchUp waits (up to 60s) for the replica to catch up with the primary database. The +// waitForReplicaToCatchUp waits (up to 30s) for the replica to catch up with the primary database. The // lag is measured by checking that gtid_executed is the same on the primary and replica. func waitForReplicaToCatchUp(t *testing.T) { - timeLimit := 60 * time.Second + timeLimit := 30 * time.Second + endTime := time.Now().Add(timeLimit) for time.Now().Before(endTime) { replicaGtid := queryGtid(t, replicaDatabase) @@ -474,6 +475,7 @@ func waitForReplicaToCatchUp(t *testing.T) { // Log some status of the replica, before failing the test outputShowReplicaStatus(t) + //outputReplicaApplierStatus(t) // NOTE: This can only run aginst MySQL, not Dolt t.Fatal("primary and replica did not synchronize within " + timeLimit.String()) } From 927a0ddcc39d89a19f37e2fda8583893f77002ca Mon Sep 17 00:00:00 2001 From: Jason Fulghum Date: Tue, 18 Jun 2024 17:28:11 -0700 Subject: [PATCH 03/14] Adding support for finding the right starting position in multiple log files --- .../binlog_primary_log_manager.go | 81 +++++-- .../binlog_primary_streamer.go | 213 ++++++++++++------ .../binlogreplication/binlog_primary_test.go | 65 ++++++ 3 files changed, 266 insertions(+), 93 deletions(-) diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go index 3e8533aad33..6809c2be5a0 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go @@ -47,14 +47,13 @@ type LogManager struct { fs filesys.Filesys binlogFormat mysql.BinlogFormat binlogEventMeta mysql.BinlogEventMetadata + binlogDirectory string } // NewLogManager creates a new LogManager instance where binlog files are stored in the .dolt/binlog directory // underneath the specified |fs| filesystem. The |binlogFormat| and |binlogStream| are used to initialize the // new binlog file. func NewLogManager(fs filesys.Filesys, binlogFormat mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) (*LogManager, error) { - // TODO: On server startup, we need to find the most recent binlog file, add a rotate event at the end (if necessary?), and start a new file. Documentation seems to indicate that a rotate event is added at the end of a binlog file, so that the streamer can jump to the next file, but I don't see this in our MySQL sample binlog files. Need to do more testing here. - lm := &LogManager{ mu: &sync.Mutex{}, fs: fs, @@ -64,26 +63,73 @@ func NewLogManager(fs filesys.Filesys, binlogFormat mysql.BinlogFormat, binlogEv // TODO: Could resolve the base dir for the binlog file directory here; would it help us avoid returning errors in other APIs? - // Initialize binlog file storage (extract to function!) - err := fs.MkDirs(binlogDirectory) + // Initialize binlog file storage directory + if err := fs.MkDirs(binlogDirectory); err != nil { + return nil, err + } + abs, err := fs.Abs(binlogDirectory) if err != nil { return nil, err } + lm.binlogDirectory = abs - // Initialize current binlog file - nextLogFilename, err := lm.nextLogFile() - if err != nil { + // Ensure the previous log file has a Rotate event that points to the new log file + if err := lm.addRotateEventToPreviousLogFile(); err != nil { return nil, err } - lm.currentBinlogFileName = nextLogFilename - if err = lm.initializeCurrentLogFile(binlogFormat, binlogEventMeta); err != nil { + // Initialize the new binlog file + if err := lm.createNewBinlogFile(); err != nil { return nil, err } return lm, nil } +// addRotateEventToPreviousLogFile finds the previous binlog file and appends a Rotate event that points to the +// next binlog file. This is necessary so that as streamers are reading from the binlog files, they have a pointer +// to follow to the next binlog file. +func (lm *LogManager) addRotateEventToPreviousLogFile() error { + // Find the most recent log file for the binlog branch + mostRecentLogfile, err := lm.mostRecentLogFileForBranch(BinlogBranch) + if err != nil { + return err + } + + // If there isn't a most recent log file, then there's nothing to do + if mostRecentLogfile == "" { + return nil + } + + // Open the log file and append a Rotate event + previousLogFile, err := os.OpenFile(filepath.Join(lm.binlogDirectory, mostRecentLogfile), os.O_WRONLY|os.O_APPEND, 0644) + if err != nil { + return err + } + defer previousLogFile.Close() + + nextLogFilename, err := lm.nextLogFile() + if err != nil { + return err + } + + rotateEvent := mysql.NewRotateEvent(lm.binlogFormat, lm.binlogEventMeta, 0, nextLogFilename) + _, err = previousLogFile.Write(rotateEvent.Bytes()) + return err +} + +// createNewBinlogFile creates a new binlog file and initializes it with the binlog magic number, a Format Description +// event, and a Previous GTIDs event. The new binlog file is opened for append only writing. +func (lm *LogManager) createNewBinlogFile() error { + nextLogFilename, err := lm.nextLogFile() + if err != nil { + return err + } + lm.currentBinlogFileName = nextLogFilename + + return lm.initializeCurrentLogFile(lm.binlogFormat, lm.binlogEventMeta) +} + // nextLogFile returns the filename of the next bin log file in the current sequence. For example, if the // current log file is "binlog-main.000008" the nextLogFile() would return "binlog-main.000009". Note that // this function returns the file name only, not the full file path. @@ -307,24 +353,13 @@ func (lm *LogManager) writeEventsHelper(binlogEvents ...mysql.BinlogEvent) error return nil } -func (lm *LogManager) resolveLogFile(filename string) (string, error) { - binlogBaseDir, err := lm.fs.Abs(binlogDirectory) - if err != nil { - return "", err - } - +func (lm *LogManager) resolveLogFile(filename string) string { // TODO: Should we make sure it exists? - return filepath.Join(binlogBaseDir, filename), nil + return filepath.Join(lm.binlogDirectory, filename) } func (lm *LogManager) currentBinlogFilepath() string { - logFile, err := lm.resolveLogFile(lm.currentBinlogFileName) - if err != nil { - // TODO: return an error, or handle this err somewhere else where we do return an error - panic(err) - } - - return logFile + return lm.resolveLogFile(lm.currentBinlogFileName) } // formatBinlogFilename formats a binlog filename using the specified |branch| and |sequence| number. The diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go index 55afb1b450d..8810bf2046e 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go @@ -51,14 +51,15 @@ func newBinlogStreamer() *binlogStreamer { // and |binlogEventMeta| records the position of the stream. This method blocks until an error // is received over the stream (e.g. the connection closing) or the streamer is closed, // through it's quit channel. -func (streamer *binlogStreamer) startStream(_ *sql.Context, conn *mysql.Conn, executedGtids mysql.GTIDSet, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata, logfile string) error { - logrus.Tracef("Starting binlog stream... (connection ID: %d)", conn.ConnectionID) +func (streamer *binlogStreamer) startStream(ctxs *sql.Context, conn *mysql.Conn, executedGtids mysql.GTIDSet, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata, logfile string) error { + logrus.WithField("connection_id", conn.ConnectionID). + WithField("executed_gtids", executedGtids.String()). + Trace("starting binlog stream") if err := sendInitialEvents(ctx, conn, binlogFormat, binlogEventMeta); err != nil { return err } - // TODO: Maybe we should just ask the LogManager to give us the file for reading? file, err := openBinlogFileForReading(logfile) if err != nil { return err @@ -95,83 +96,63 @@ func (streamer *binlogStreamer) startStream(_ *sql.Context, conn *mysql.Conn, ex default: logrus.Debug("checking file for new data...") - eof := false skippingGtids := false - for !eof { - headerBuffer := make([]byte, 4+1+4+4+4+2) - bytesRead, err := file.Read(headerBuffer) - if err != nil && err != io.EOF { - return err - } + // TODO: This for loop is nested in side a select block, inside another for loop + // We should be able to get rid of this inner for loop, otherwise the select + // block won't ever let us read from ticker or quitChan + for { + binlogEvent, err := readBinlogEventFromFile(file) if err == io.EOF { logrus.Tracef("End of binlog file! Waiting for new events...") - time.Sleep(100 * time.Millisecond) + time.Sleep(250 * time.Millisecond) continue - } - - // Event Header: - //timestamp := headerBuffer[0:4] - //eventType := headerBuffer[4] - //serverId := binary.LittleEndian.Uint32(headerBuffer[5:5+4]) - eventSize := binary.LittleEndian.Uint32(headerBuffer[9 : 9+4]) - - payloadBuffer := make([]byte, eventSize-uint32(len(headerBuffer))) - bytesRead, err = file.Read(payloadBuffer) - if err != nil && err != io.EOF { + } else if err != nil { return err } - if err == io.EOF { - logrus.Tracef("End of binlog file! Waiting for new events...") - time.Sleep(100 * time.Millisecond) - continue - } - if bytesRead > 0 { - binlogEvent := mysql.NewMysql56BinlogEvent(append(headerBuffer, payloadBuffer...)) + if binlogEvent.IsRotate() { + bytes := binlogEvent.Bytes() + newLogfile := string(bytes[19+8 : (len(bytes) - 4)]) + logrus.Errorf("Rotatating to new binlog file: %s", newLogfile) - if binlogEvent.IsRotate() { - newLogfile := string(payloadBuffer[8:(len(payloadBuffer) - 4)]) - logrus.Errorf("Rotatating to new binlog file: %s", newLogfile) + dir := filepath.Dir(logfile) + newLogfile = filepath.Join(dir, newLogfile) - dir := filepath.Dir(logfile) - newLogfile = filepath.Join(dir, newLogfile) - - if err = file.Close(); err != nil { - logrus.Errorf("unable to close previous binlog file: %s", err.Error()) - } - - if file, err = openBinlogFileForReading(newLogfile); err != nil { - return err - } - - continue + if err = file.Close(); err != nil { + logrus.Errorf("unable to close previous binlog file: %s", err.Error()) } - if binlogEvent.IsGTID() { - gtid, _, err := binlogEvent.GTID(*binlogFormat) - if err != nil { - return err - } - - // If the replica has already executed this GTID, then skip it. - if executedGtids.ContainsGTID(gtid) { - skippingGtids = true - } else { - skippingGtids = false - } + if file, err = openBinlogFileForReading(newLogfile); err != nil { + return err } - if skippingGtids { - continue - } + continue + } - if err := conn.WriteBinlogEvent(binlogEvent, false); err != nil { + if binlogEvent.IsGTID() { + gtid, _, err := binlogEvent.GTID(*binlogFormat) + if err != nil { return err } - if err = conn.FlushBuffer(); err != nil { - return err + + // If the replica has already executed this GTID, then skip it. + if executedGtids.ContainsGTID(gtid) { + skippingGtids = true + } else { + skippingGtids = false } } + + if skippingGtids { + continue + } + + if err := conn.WriteBinlogEvent(binlogEvent, false); err != nil { + return err + } + if err = conn.FlushBuffer(); err != nil { + return err + } } time.Sleep(500 * time.Millisecond) } @@ -182,8 +163,6 @@ func (streamer *binlogStreamer) startStream(_ *sql.Context, conn *mysql.Conn, ex // are the expected binlog file magic numbers. If any problems are encountered opening the file or reading the first // four bytes, an error is returned. func openBinlogFileForReading(logfile string) (*os.File, error) { - logrus.Errorf("Opening binlog file: %s", logfile) - file, err := os.Open(logfile) if err != nil { return nil, err @@ -253,14 +232,104 @@ func (m *binlogStreamerManager) StartStream(ctx *sql.Context, conn *mysql.Conn, m.addStreamer(streamer) defer m.removeStreamer(streamer) - return streamer.startStream(ctx, conn, executedGtids, binlogFormat, &binlogEventMeta, m.logManager.currentBinlogFilepath()) + // TODO: This may be a little tricky with branch switching... perhaps we should destroy the index when we + // switch branches and rebuild it as we create more logs? + // It seems like part of the contract with switching branches for replication is that the primary will + // invalidate/destroy all the previous binlog files? It doesn't seem safe to rely on them being + // correct after changing the branch back and forth. + + file, err := m.findLogFileForPosition(executedGtids) + if err != nil { + return err + } + + // TODO: We also need to handle cases where we are missing GTIDs that the replica doesn't have yet. In these + // cases we need to send errors back to the replica. + return streamer.startStream(ctx, conn, executedGtids, binlogFormat, binlogEventMeta, file) } -// sendEvents sends |binlogEvents| to all the streams managed by this instance. -func (m *binlogStreamerManager) sendEvents(binlogEvents []mysql.BinlogEvent) { - for _, streamer := range m.copyStreamers() { - logrus.StandardLogger().Tracef("queuing %d binlog events\n", len(binlogEvents)) - streamer.eventChan <- binlogEvents +// findLogFileForPosition searches through the binlog files on disk for the first log file that contains GTIDs that +// are not present in |executedGtids|. This is determined by reading the first GTID event from each log file and +// selecting the previous file when the first GTID not in |executedGtids| is found. +func (m *binlogStreamerManager) findLogFileForPosition(executedGtids mysql.GTIDSet) (string, error) { + files, err := m.logManager.logFilesOnDiskForBranch(BinlogBranch) + if err != nil { + return "", err + } + + for i, f := range files { + file, err := openBinlogFileForReading(filepath.Join(m.logManager.binlogDirectory, f)) + if err != nil { + return "", err + } + + binlogEvent, err := readFirstGtidEventFromFile(file) + file.Close() + if err == io.EOF { + continue + } else if err != nil { + return "", err + } + + if binlogEvent.IsGTID() { + gtid, _, err := binlogEvent.GTID(m.logManager.binlogFormat) + if err != nil { + return "", err + } + // If the first GTID in this file is contained in the set of GTIDs that the replica + // has already executed, then move on to check the next file. + if executedGtids.ContainsGTID(gtid) { + continue + } + + if i == 0 { + return m.logManager.currentBinlogFilepath(), nil + } + return filepath.Join(m.logManager.binlogDirectory, files[i-1]), nil + } + } + + // TODO: If we didn't find an unexecuted GTID in any of the files, just return + // the current log file. This is a bit of a hack, but it's enough to + // keep the following tests passing: + // TestBinlogPrimary_ChangeReplicationBranch + // TestBinlogPrimary_PrimaryRestart + // It might actually be better to just return the first available log file? + return m.logManager.currentBinlogFilepath(), nil +} + +func readBinlogEventFromFile(file *os.File) (mysql.BinlogEvent, error) { + headerBuffer := make([]byte, 4+1+4+4+4+2) + _, err := file.Read(headerBuffer) + if err != nil { + return nil, err + } + + // Event Header: + //timestamp := headerBuffer[0:4] + //eventType := headerBuffer[4] + //serverId := binary.LittleEndian.Uint32(headerBuffer[5:5+4]) + eventSize := binary.LittleEndian.Uint32(headerBuffer[9 : 9+4]) + + payloadBuffer := make([]byte, eventSize-uint32(len(headerBuffer))) + _, err = file.Read(payloadBuffer) + if err != nil { + return nil, err + } + + return mysql.NewMysql56BinlogEvent(append(headerBuffer, payloadBuffer...)), nil +} + +func readFirstGtidEventFromFile(file *os.File) (mysql.BinlogEvent, error) { + for { + binlogEvent, err := readBinlogEventFromFile(file) + if err != nil { + return nil, err + } + + if binlogEvent.IsGTID() { + return binlogEvent, nil + } } } @@ -277,6 +346,10 @@ func (m *binlogStreamerManager) removeStreamer(streamer *binlogStreamer) { m.streamersMutex.Lock() defer m.streamersMutex.Unlock() + if len(m.streamers) == 0 { + return + } + m.streamers = make([]*binlogStreamer, len(m.streamers)-1, 0) for _, element := range m.streamers { if element != streamer { diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go index 5fb3b06f14d..4f776dfd7f9 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go @@ -153,6 +153,71 @@ func TestBinlogPrimary_Rotation(t *testing.T) { }) } +// TestBinlogPrimary_ReplicaAndPrimaryRestart tests that a replica can disconnect and reconnect to the primary to +// restart the replication stream, even when the primary has been restarted and log files have rotated. +func TestBinlogPrimary_ReplicaAndPrimaryRestart(t *testing.T) { + defer teardown(t) + startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) + setupForDoltToMySqlReplication() + startReplication(t, doltPort) + + // Change the binlog rotation threshold on the primary to 10KB (instead of the default 1GB) so + // that log files will rotate more often + primaryDatabase.MustExec("SET @@GLOBAL.max_binlog_size = 10240;") + + // Create a table on the primary and assert that it gets replicated + primaryDatabase.MustExec("create table db01.t1 (pk int primary key, c1 varchar(255));") + waitForReplicaToCatchUp(t) + requireReplicaResults(t, "show tables;", [][]any{{"t1"}}) + + // Assert that the executed GTID position on the replica contains GTIDs 1 and 2 + serverUuid := queryPrimaryServerUuid(t) + status := queryReplicaStatus(t) + require.Equal(t, serverUuid+":1-2", status["Executed_Gtid_Set"]) + + // Stop the MySQL replica server and wait for a few seconds + stopMySqlServer(t) + time.Sleep(2_000 * time.Millisecond) + + // Generate enough data to trigger a logfile rotation + primaryDatabase.MustExec("create table t (n int);") + for i := range 100 { + primaryDatabase.MustExec(fmt.Sprintf("insert into t values (%d);", i)) + } + + // Stop the primary and restart it to test that it creates a new log file and + // applies a rotate event to the last log file + stopDoltSqlServer(t) + + // Restart the Dolt primary server + var err error + prevReplicaDatabase := replicaDatabase + doltPort, doltProcess, err = startDoltSqlServer(testDir, nil) + require.NoError(t, err) + primaryDatabase = replicaDatabase + replicaDatabase = prevReplicaDatabase + + // Generate more data on the primary after restarting + primaryDatabase.MustExec("use db01;") + for i := range 100 { + primaryDatabase.MustExec(fmt.Sprintf("insert into t values (%d);", i+100)) + } + + // Restart the MySQL replica and reconnect to the Dolt primary + prevPrimaryDatabase := primaryDatabase + mySqlPort, mySqlProcess, err = startMySqlServer(testDir) + require.NoError(t, err) + replicaDatabase = primaryDatabase + primaryDatabase = prevPrimaryDatabase + startReplication(t, doltPort) + waitForReplicaToCatchUp(t) + + // Assert the executed GTID position now contains all GTIDs + status = queryReplicaStatus(t) + require.Equal(t, serverUuid+":1-203", status["Executed_Gtid_Set"]) + requireReplicaResults(t, "SELECT MAX(n) FROM t;", [][]any{{"199"}}) +} + // TestBinlogPrimary_Heartbeats tests that heartbeats sent from the primary to the replica are well-formed and // don't cause the replica to close the stream. For example, if the nextLogPosition field in the heartbeat event // doesn't match up with the nextLogPosition from the previous event, then the replica will quit the connection. From 40f96739859df8576ae92ef4cd2482369198c93f Mon Sep 17 00:00:00 2001 From: Jason Fulghum Date: Wed, 26 Jun 2024 13:01:10 -0700 Subject: [PATCH 04/14] Implementing "show binary logs" support --- .../binlog_primary_controller.go | 22 +++++++++++++++++-- .../binlogreplication/binlog_primary_test.go | 8 ++++++- 2 files changed, 27 insertions(+), 3 deletions(-) diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go index c8c7e7f844c..c6743dcf456 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go @@ -16,6 +16,8 @@ package binlogreplication import ( "fmt" + "os" + "path/filepath" "github.com/dolthub/go-mysql-server/sql" "github.com/dolthub/go-mysql-server/sql/binlogreplication" @@ -83,8 +85,24 @@ func (d *DoltBinlogPrimaryController) ListReplicas(ctx *sql.Context) error { // ListBinaryLogs implements the BinlogPrimaryController interface. func (d *DoltBinlogPrimaryController) ListBinaryLogs(_ *sql.Context) ([]binlogreplication.BinaryLogFileMetadata, error) { - // TODO: No log file support yet, so just return an empty list - return nil, nil + logManager := d.streamerManager.logManager + logFiles, err := logManager.logFilesOnDiskForBranch(BinlogBranch) + if err != nil { + return nil, err + } + + logFileMetadata := make([]binlogreplication.BinaryLogFileMetadata, len(logFiles)) + for i, logFile := range logFiles { + fileStats, err := os.Stat(filepath.Join(logManager.binlogDirectory, logFile)) + if err != nil { + return nil, err + } + logFileMetadata[i] = binlogreplication.BinaryLogFileMetadata{ + Name: logFile, + Size: uint64(fileStats.Size()), + } + } + return logFileMetadata, nil } // GetBinaryLogStatus implements the BinlogPrimaryController interface. diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go index 4f776dfd7f9..2d3fbc0568f 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go @@ -148,9 +148,15 @@ func TestBinlogPrimary_Rotation(t *testing.T) { // Check the binary log file status and ensure the file has been rotated uuid := queryPrimaryServerUuid(t) - requirePrimaryResults(t, "show binary log status", [][]any{ + requirePrimaryResults(t, "show binary log status;", [][]any{ {"binlog-main.000003", "1027", "", "", uuid + ":1-102"}, }) + + requirePrimaryResults(t, "show binary logs;", [][]any{ + {"binlog-main.000001", "10318", "No"}, + {"binlog-main.000002", "10481", "No"}, + {"binlog-main.000003", "1027", "No"}, + }) } // TestBinlogPrimary_ReplicaAndPrimaryRestart tests that a replica can disconnect and reconnect to the primary to From 038c5a4caecf6bcf78e2f137fee317e7e7530927 Mon Sep 17 00:00:00 2001 From: Jason Fulghum Date: Wed, 10 Jul 2024 15:59:30 -0700 Subject: [PATCH 05/14] First pass on log purging support --- go/cmd/dolt/commands/sqlserver/server.go | 2 +- .../binlog_position_store.go | 6 +- .../binlog_primary_log_manager.go | 210 +++++++++++++++--- .../binlogreplication/binlog_primary_test.go | 82 +++++-- .../sqle/binlogreplication/binlog_producer.go | 26 ++- .../binlog_replica_applier.go | 5 +- .../doltcore/sqle/system_variables.go | 9 + 7 files changed, 273 insertions(+), 67 deletions(-) diff --git a/go/cmd/dolt/commands/sqlserver/server.go b/go/cmd/dolt/commands/sqlserver/server.go index 47ea2f413f3..257c7ab150e 100644 --- a/go/cmd/dolt/commands/sqlserver/server.go +++ b/go/cmd/dolt/commands/sqlserver/server.go @@ -303,7 +303,7 @@ func ConfigureServices( } if logBin == 1 { logrus.Debug("Enabling binary logging") - binlogProducer, err := binlogreplication.NewBinlogProducer(doltBinlogPrimaryController.StreamerManager()) + binlogProducer, err := binlogreplication.NewBinlogProducer(dEnv.FS, doltBinlogPrimaryController.StreamerManager()) if err != nil { return err } diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_position_store.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_position_store.go index 36b3b16bdbc..f96dcc32dfa 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_position_store.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_position_store.go @@ -25,6 +25,7 @@ import ( "github.com/dolthub/vitess/go/mysql" "github.com/dolthub/dolt/go/libraries/doltcore/sqle/dsess" + "github.com/dolthub/dolt/go/libraries/utils/filesys" ) const binlogPositionDirectory = ".doltcfg" @@ -44,13 +45,10 @@ 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 .doltcfg/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(ctx *sql.Context) (*mysql.Position, error) { +func (store *binlogPositionStore) Load(filesys filesys.Filesys) (*mysql.Position, error) { store.mu.Lock() defer store.mu.Unlock() - doltSession := dsess.DSessFromSess(ctx.Session) - filesys := doltSession.Provider().FileSystem() - doltDirExists, _ := filesys.Exists(binlogPositionDirectory) if !doltDirExists { return nil, nil diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go index 6809c2be5a0..08863bae049 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go @@ -17,11 +17,13 @@ package binlogreplication import ( "encoding/binary" "fmt" + "io" "os" "path/filepath" "strconv" "strings" "sync" + "time" "github.com/dolthub/vitess/go/mysql" "github.com/sirupsen/logrus" @@ -73,47 +75,152 @@ func NewLogManager(fs filesys.Filesys, binlogFormat mysql.BinlogFormat, binlogEv } lm.binlogDirectory = abs - // Ensure the previous log file has a Rotate event that points to the new log file - if err := lm.addRotateEventToPreviousLogFile(); err != nil { + // Initialize a new binlog file + if err := lm.createNewBinlogFile(); err != nil { return nil, err } - // Initialize the new binlog file - if err := lm.createNewBinlogFile(); err != nil { + // Purge any expired log files + if err := lm.purgeExpiredLogFiles(); err != nil { + return nil, err + } + + // Ensure the previous log file has a Rotate event that points to the new log file + if err := lm.addRotateEventToPreviousLogFile(); err != nil { return nil, err } return lm, nil } +// purgeExpiredLogFiles removes any binlog files that are older than @@binlog_expire_logs_seconds. This automatic +// binlog purging currently happens only on server startup. Eventually this should also be hooked into the `FLUSH LOGS` +// (or `FLUSH BINARY LOGS`) statement as well to match MySQL's behavior. +// +// When this method is called, it is expected that the new, current binlog file has already been initialized and that +// adding a Rotate event to the previous log has NOT occurred yet (otherwise adding the Rotate event would update the +// log file's last modified time and would not be purged). +func (lm *LogManager) purgeExpiredLogFiles() error { + expireLogsSeconds, err := lookupBinlogExpireLogsSeconds() + if expireLogsSeconds == 0 { + // If @@binlog_expire_logs_seconds is set to 0, then binlog files are never automatically expired + return nil + } + + purgeThresholdTime := time.Now().Add(-time.Duration(expireLogsSeconds) * time.Second) + + files, err := lm.logFilesOnDiskForBranch(BinlogBranch) + if err != nil { + return err + } + + for i := 0; i < len(files); i++ { + fullLogFilepath := filepath.Join(lm.binlogDirectory, files[i]) + stat, err := os.Stat(fullLogFilepath) + if err != nil { + return err + } + + if stat.ModTime().Before(purgeThresholdTime) { + logrus.Debugf("purging expired binlog file: %s", files[i]) + if err := os.Remove(fullLogFilepath); err != nil { + return err + } + } + } + + // TODO: This is where things get messy... + // This is where we find the oldest binlog file, and attempt to find the first GTID in it, so that we can + // set @@gtid_purged. + // This should mostly work, but not in the case where there is only one binlog file after purging (the + // current/new binlog file) and it doesn't have a GTID in it. In that case, we could default to + // @@gtid_executed. + // + // One of the problems here is that @@gtid_executed doesn't get set at startup, like it should. We currently need + // a *sql.Context to grab the DoltSession and pull out the location of the GTID position store/file. This happens + // the first time we process a GTID binlog event, but it needs to happen at startup. + // + // Another way to solve this issue would be to have an index file that tracks which GTIDs are present in each log + // file. I'd like to avoid this if possible, just because it adds more complexity and seems like we can get away + // without it right now. + + // Set @@gtid_purged if we deleted any files + files, err = lm.logFilesOnDiskForBranch(BinlogBranch) + if err != nil { + return err + } + if len(files) > 0 { + // TODO: This doesn't work! If we purge all the log files and we only have the new file, then we can't + // determine the GTID to set... unless we use gtid_executed? + openFile, err := os.Open(filepath.Join(lm.binlogDirectory, files[0])) + if err != nil { + return err + } + defer openFile.Close() + + binlogEvent, err := readFirstGtidEventFromFile(openFile) + if err == io.EOF { + // If there are no GTID events in the file, then all GTIDs have been purged, so use @@gtid_executed + _, gtidExecutedValue, ok := sql.SystemVariables.GetGlobal("gtid_executed") + if !ok { + return fmt.Errorf("unable to find system variable @@gtid_executed") + } + + logrus.Errorf("setting gtid_purged to: %s", gtidExecutedValue) + + err = sql.SystemVariables.SetGlobal("gtid_purged", gtidExecutedValue) + if err != nil { + return err + } + + return nil + } else if err != nil { + return err + } + + gtid, ok, err := binlogEvent.GTID(lm.binlogFormat) + if err != nil { + return err + } + if !ok { + panic("not ok!") + } + sequenceNumber := gtid.SequenceNumber().(int64) + + gtid.SourceServer() + + err = sql.SystemVariables.SetGlobal("gtid_purged", fmt.Sprintf("%s:%d", gtid.SourceServer(), sequenceNumber-1)) + if err != nil { + return err + } + } + + return nil +} + // addRotateEventToPreviousLogFile finds the previous binlog file and appends a Rotate event that points to the // next binlog file. This is necessary so that as streamers are reading from the binlog files, they have a pointer -// to follow to the next binlog file. +// to follow to the next binlog file. This function MUST be called after the new log file has been initialized, +// and after any expired log files have been purged. func (lm *LogManager) addRotateEventToPreviousLogFile() error { - // Find the most recent log file for the binlog branch - mostRecentLogfile, err := lm.mostRecentLogFileForBranch(BinlogBranch) + previousLogFileName, err := lm.previousLogFile() if err != nil { return err } - // If there isn't a most recent log file, then there's nothing to do - if mostRecentLogfile == "" { + // If the previous log file in the sequence has been purged, then there's nothing to do + if !fileExists(filepath.Join(lm.binlogDirectory, previousLogFileName)) { return nil } // Open the log file and append a Rotate event - previousLogFile, err := os.OpenFile(filepath.Join(lm.binlogDirectory, mostRecentLogfile), os.O_WRONLY|os.O_APPEND, 0644) + previousLogFile, err := os.OpenFile(filepath.Join(lm.binlogDirectory, previousLogFileName), os.O_WRONLY|os.O_APPEND, 0644) if err != nil { return err } defer previousLogFile.Close() - nextLogFilename, err := lm.nextLogFile() - if err != nil { - return err - } - - rotateEvent := mysql.NewRotateEvent(lm.binlogFormat, lm.binlogEventMeta, 0, nextLogFilename) + rotateEvent := mysql.NewRotateEvent(lm.binlogFormat, lm.binlogEventMeta, 0, lm.currentBinlogFileName) _, err = previousLogFile.Write(rotateEvent.Bytes()) return err } @@ -131,8 +238,8 @@ func (lm *LogManager) createNewBinlogFile() error { } // nextLogFile returns the filename of the next bin log file in the current sequence. For example, if the -// current log file is "binlog-main.000008" the nextLogFile() would return "binlog-main.000009". Note that -// this function returns the file name only, not the full file path. +// current log file is "binlog-main.000008" the nextLogFile() method would return "binlog-main.000009". +// Note that this function returns the file name only, not the full file path. func (lm *LogManager) nextLogFile() (filename string, err error) { mostRecentLogfile, err := lm.mostRecentLogFileForBranch(BinlogBranch) if err != nil { @@ -150,6 +257,18 @@ func (lm *LogManager) nextLogFile() (filename string, err error) { } } +// previousLogFile returns the filename of the previous bin log file in the current sequence. For example, if +// the current log file is "binlog-main.000008" the previousLogFile() method would return "binlog-main.000007". +// Note that this function returns the file name only, not the full path, and doesn't guarantee that the named +// file actually exists on disk or not. +func (lm *LogManager) previousLogFile() (filename string, err error) { + branch, sequence, err := parseBinlogFilename(lm.currentBinlogFileName) + if err != nil { + return "", err + } + return formatBinlogFilename(branch, sequence-1), nil +} + func (lm *LogManager) logFilesOnDisk() (files []string, err error) { err = lm.fs.Iter(binlogDirectory, false, func(path string, size int64, isDir bool) (stop bool) { base := filepath.Base(path) @@ -289,21 +408,6 @@ func (lm *LogManager) initializeCurrentLogFile(binlogFormat mysql.BinlogFormat, return lm.writeEventsHelper(mysql.NewPreviousGtidsEvent(binlogFormat, binlogEventMeta, gtidSet.(mysql.Mysql56GTIDSet))) } -// lookupMaxBinlogSize looks up the value of the @@max_binlog_size system variable and returns it, along with any -// errors encountered while looking it up. -func lookupMaxBinlogSize() (int, error) { - _, value, ok := sql.SystemVariables.GetGlobal("max_binlog_size") - if !ok { - return 0, fmt.Errorf("system variable @@max_binlog_size not found") - } - - intValue, _, err := gmstypes.Int32.Convert(value) - if err != nil { - return 0, err - } - return int(intValue.(int32)), nil -} - // WriteEvents writes |binlogEvents| to the current binlog file. Access to write to the binary log is synchronized, // so that only one thread can write to the log file at a time. func (lm *LogManager) WriteEvents(binlogEvents ...mysql.BinlogEvent) error { @@ -362,6 +466,46 @@ func (lm *LogManager) currentBinlogFilepath() string { return lm.resolveLogFile(lm.currentBinlogFileName) } +// fileExists returns true if the specified |filename| exists on disk and is not a directory, otherwise returns false. +func fileExists(filename string) bool { + info, err := os.Stat(filename) + if os.IsNotExist(err) { + return false + } + return !info.IsDir() +} + +// lookupMaxBinlogSize looks up the value of the @@max_binlog_size system variable and returns it, along with any +// errors encountered while looking it up. +func lookupMaxBinlogSize() (int, error) { + _, value, ok := sql.SystemVariables.GetGlobal("max_binlog_size") + if !ok { + return 0, fmt.Errorf("system variable @@max_binlog_size not found") + } + + intValue, _, err := gmstypes.Int32.Convert(value) + if err != nil { + return 0, err + } + return int(intValue.(int32)), nil +} + +// lookupBinlogExpireLogsSeconds looks up the value of the @@binlog_expire_logs_seconds system variable and returns +// it, along with any errors encountered while looking it up. +func lookupBinlogExpireLogsSeconds() (int, error) { + _, value, ok := sql.SystemVariables.GetGlobal("binlog_expire_logs_seconds") + if !ok { + return -1, fmt.Errorf("unable to find system variable @@binlog_expire_logs_seconds") + } + + int32Value, _, err := gmstypes.Int32.Convert(value) + if err != nil { + return -1, fmt.Errorf("unable to convert @@binlog_expire_logs_seconds value to integer: %s", err.Error()) + } + + return int(int32Value.(int32)), nil +} + // formatBinlogFilename formats a binlog filename using the specified |branch| and |sequence| number. The // returned filename will be of the form "binlog-main.000001". func formatBinlogFilename(branch string, sequence int) string { diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go index 2d3fbc0568f..8fb910c3500 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go @@ -159,6 +159,46 @@ func TestBinlogPrimary_Rotation(t *testing.T) { }) } +// TestBinlogPrimary_AutoPurging tests that the primary server correctly purges binary log files older than +// @@binlog_expire_logs_seconds on restart. +func TestBinlogPrimary_AutoPurging(t *testing.T) { + defer teardown(t) + mapCopy := copyMap(doltReplicationPrimarySystemVars) + mapCopy["binlog_expire_logs_seconds"] = "1" + startSqlServersWithDoltSystemVars(t, mapCopy) + setupForDoltToMySqlReplication() + + // Generate binary log content + primaryDatabase.MustExec("create database db01;") + primaryDatabase.MustExec("create table db01.t (n int);") + for i := range 100 { + primaryDatabase.MustExec(fmt.Sprintf("insert into db01.t values (%d);", i)) + } + requirePrimaryResults(t, "SHOW BINARY LOGS;", [][]any{ + {"binlog-main.000001", "21346", "No"}, + }) + + // Restart and confirm the binary log has been purged + stopDoltSqlServer(t) + mustRestartDoltPrimaryServer(t) + requirePrimaryResults(t, "SHOW BINARY LOGS;", [][]any{ + {"binlog-main.000002", "191", "No"}, + }) + + // Check the value of @@gtid_purged + requirePrimaryResults(t, "SELECT @@gtid_purged;", [][]any{ + {fmt.Sprintf("%s:1-102", queryPrimaryServerUuid(t))}, + }) + + // Verify the replica reports an error about the GTIDs not being available + startReplication(t, doltPort) + status := queryReplicaStatus(t) + require.Equal(t, "13114", status["Last_IO_Errno"]) + require.Contains(t, status["Last_IO_Error"], + "Got fatal error 1236 from source when reading data from binary log: "+ + "'Cannot replicate because the source purged required binary logs.") +} + // TestBinlogPrimary_ReplicaAndPrimaryRestart tests that a replica can disconnect and reconnect to the primary to // restart the replication stream, even when the primary has been restarted and log files have rotated. func TestBinlogPrimary_ReplicaAndPrimaryRestart(t *testing.T) { @@ -196,12 +236,7 @@ func TestBinlogPrimary_ReplicaAndPrimaryRestart(t *testing.T) { stopDoltSqlServer(t) // Restart the Dolt primary server - var err error - prevReplicaDatabase := replicaDatabase - doltPort, doltProcess, err = startDoltSqlServer(testDir, nil) - require.NoError(t, err) - primaryDatabase = replicaDatabase - replicaDatabase = prevReplicaDatabase + mustRestartDoltPrimaryServer(t) // Generate more data on the primary after restarting primaryDatabase.MustExec("use db01;") @@ -210,11 +245,7 @@ func TestBinlogPrimary_ReplicaAndPrimaryRestart(t *testing.T) { } // Restart the MySQL replica and reconnect to the Dolt primary - prevPrimaryDatabase := primaryDatabase - mySqlPort, mySqlProcess, err = startMySqlServer(testDir) - require.NoError(t, err) - replicaDatabase = primaryDatabase - primaryDatabase = prevPrimaryDatabase + mustRestartMySqlReplicaServer(t) startReplication(t, doltPort) waitForReplicaToCatchUp(t) @@ -326,14 +357,9 @@ func TestBinlogPrimary_PrimaryRestart(t *testing.T) { // Stop the Dolt primary server stopDoltSqlServer(t) time.Sleep(2_000 * time.Millisecond) - prevReplicaDatabase := replicaDatabase // Restart the Dolt primary server - var err error - doltPort, doltProcess, err = startDoltSqlServer(testDir, nil) - require.NoError(t, err) - primaryDatabase = replicaDatabase - replicaDatabase = prevReplicaDatabase + mustRestartDoltPrimaryServer(t) waitForReplicaToReconnect(t) // Create another table and assert that it gets replicated @@ -935,3 +961,25 @@ func waitForReplicaToReconnect(t *testing.T) { } } } + +// mustRestartDoltPrimaryServer starts up the Dolt sql-server, after it has already been stopped before this function +// is called, and configures it as the primary database. +func mustRestartDoltPrimaryServer(t *testing.T) { + var err error + prevReplicaDatabase := replicaDatabase + doltPort, doltProcess, err = startDoltSqlServer(testDir, nil) + require.NoError(t, err) + primaryDatabase = replicaDatabase + replicaDatabase = prevReplicaDatabase +} + +// mustRestartMySqlReplicaServer starts up the MySQL server, after it has already been stopped before this function +// is called, and configures it as the replica database. +func mustRestartMySqlReplicaServer(t *testing.T) { + var err error + prevPrimaryDatabase := primaryDatabase + mySqlPort, mySqlProcess, err = startMySqlServer(testDir) + require.NoError(t, err) + replicaDatabase = primaryDatabase + primaryDatabase = prevPrimaryDatabase +} diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go index 2d1b3ffc8dc..347598d0fa1 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go @@ -31,6 +31,7 @@ import ( "github.com/dolthub/dolt/go/libraries/doltcore/doltdb/durable" "github.com/dolthub/dolt/go/libraries/doltcore/schema" "github.com/dolthub/dolt/go/libraries/doltcore/sqle/sqlfmt" + "github.com/dolthub/dolt/go/libraries/utils/filesys" "github.com/dolthub/dolt/go/store/prolly" "github.com/dolthub/dolt/go/store/prolly/tree" "github.com/dolthub/dolt/go/store/val" @@ -62,19 +63,25 @@ var _ doltdb.DatabaseUpdateListener = (*binlogProducer)(nil) // NewBinlogProducer creates and returns a new instance of BinlogProducer. Note that callers must register the // returned binlogProducer as a DatabaseUpdateListener before it will start receiving database updates and start // producing binlog events. -func NewBinlogProducer(streamerManager *binlogStreamerManager) (*binlogProducer, error) { +func NewBinlogProducer(fs filesys.Filesys, streamerManager *binlogStreamerManager) (*binlogProducer, error) { binlogFormat := createBinlogFormat() binlogEventMeta, err := createBinlogEventMetadata() if err != nil { return nil, err } - return &binlogProducer{ + b := &binlogProducer{ binlogEventMeta: *binlogEventMeta, binlogFormat: binlogFormat, streamerManager: streamerManager, mu: &sync.Mutex{}, - }, nil + } + + if err = b.initializeGtidPosition(fs); err != nil { + return nil, err + } + + return b, nil } func (b *binlogProducer) BinlogFormat() *mysql.BinlogFormat { @@ -183,7 +190,7 @@ func (b *binlogProducer) DatabaseDropped(ctx *sql.Context, databaseName string) // in this binlogStreamerManager instance. If the gtidPosition has already been loaded // from disk and initialized, this method simply returns. If any problems were encountered // loading the GTID position from disk, or parsing its value, an error is returned. -func (b *binlogProducer) initializeGtidPosition(ctx *sql.Context) error { +func (b *binlogProducer) initializeGtidPosition(fs filesys.Filesys) error { if b.gtidPosition != nil { return nil } @@ -191,7 +198,7 @@ func (b *binlogProducer) initializeGtidPosition(ctx *sql.Context) error { b.mu.Lock() defer b.mu.Unlock() - position, err := positionStore.Load(ctx) + position, err := positionStore.Load(fs) if err != nil { return err } @@ -233,17 +240,14 @@ func (b *binlogProducer) initializeGtidPosition(ctx *sql.Context) error { return fmt.Errorf("unable to parse GTID position (%s): %s", gtidString, err.Error()) } b.gtidSequence = int64(i + 1) - return nil + + return sql.SystemVariables.AssignValues(map[string]any{ + "gtid_executed": b.gtidPosition.GTIDSet.String()}) } // createGtidEvent creates a new GTID event for the current transaction and updates the stream's // current log position. func (b *binlogProducer) createGtidEvent(ctx *sql.Context) (mysql.BinlogEvent, error) { - err := b.initializeGtidPosition(ctx) - if err != nil { - return nil, err - } - serverUuid, err := getServerUuid(ctx) if err != nil { return nil, err diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_replica_applier.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_replica_applier.go index c0ba2e83223..a2846af86e9 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_replica_applier.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_replica_applier.go @@ -186,7 +186,10 @@ func (a *binlogReplicaApplier) startReplicationEventStream(ctx *sql.Context, con return err } - position, err := positionStore.Load(ctx) + doltSession := dsess.DSessFromSess(ctx.Session) + filesys := doltSession.Provider().FileSystem() + + position, err := positionStore.Load(filesys) if err != nil { return err } diff --git a/go/libraries/doltcore/sqle/system_variables.go b/go/libraries/doltcore/sqle/system_variables.go index 5f069857f95..f49ee3e765f 100644 --- a/go/libraries/doltcore/sqle/system_variables.go +++ b/go/libraries/doltcore/sqle/system_variables.go @@ -31,6 +31,15 @@ func init() { func AddDoltSystemVariables() { sql.SystemVariables.AddSystemVariables([]sql.SystemVariable{ + // TODO: Remove this and move it to GMS! + &sql.MysqlSystemVariable{ + Name: "binlog_expire_logs_seconds", + Scope: sql.GetMysqlScope(sql.SystemVariableScope_Global), + Dynamic: true, + SetVarHintApplies: false, + Type: types.NewSystemIntType("binlog_expire_logs_seconds", 0, 4294967295, false), + Default: int64(2592000), + }, &sql.MysqlSystemVariable{ Name: "log_bin_branch", Scope: sql.GetMysqlScope(sql.SystemVariableScope_Persist), From 5c1244dcf2becc6effbfc4f71569f93ed6e2fc1f Mon Sep 17 00:00:00 2001 From: Jason Fulghum Date: Mon, 15 Jul 2024 13:56:46 -0700 Subject: [PATCH 06/14] Adding support for calculating the set of GTIDs a replica needs and determining if those GTIDs are still available in the server's binary logs. --- .../binlog_primary_controller.go | 19 +++- .../binlog_primary_log_manager.go | 89 ++++++++++++++++--- 2 files changed, 94 insertions(+), 14 deletions(-) diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go index c6743dcf456..b16aea0b66f 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go @@ -65,7 +65,24 @@ func (d *DoltBinlogPrimaryController) RegisterReplica(ctx *sql.Context, c *mysql // BinlogDumpGtid implements the BinlogPrimaryController interface. func (d *DoltBinlogPrimaryController) BinlogDumpGtid(ctx *sql.Context, conn *mysql.Conn, replicaExecutedGtids mysql.GTIDSet) error { if d.BinlogProducer == nil { - return fmt.Errorf("no binlog currently being recorded; make sure the server is started with @@log_bin enabled") + // TODO: Add a test for this, now that we have errors being + // reported in replica status correctly + return mysql.NewSQLError(mysql.ERMasterFatalReadingBinlog, "HY000", + "no binlog currently being recorded; make sure the server is started with @@log_bin enabled") + + } + + primaryExecutedGtids := d.BinlogProducer.gtidPosition.GTIDSet + // TODO: This is awkward... should controller have a direct link to log manager? + // Or should BinlogProducer have a direct link to the log manager? + missingGtids := d.BinlogProducer.streamerManager.logManager.calculateMissingGtids(replicaExecutedGtids, primaryExecutedGtids) + if !missingGtids.Equal(mysql.Mysql56GTIDSet{}) { + // We must send back error code 1236 (ER_MASTER_FATAL_ERROR_READING_BINLOG) to the replica to signal an error, + // otherwise the replica won't expose the error in replica status and will just keep trying to reconnect and + // only log the error to MySQL's error log. + return mysql.NewSQLError(mysql.ERMasterFatalReadingBinlog, "HY000", + "Cannot replicate because the source purged required binary logs. Replicate the missing transactions from elsewhere, or provision a new replica from backup. Consider increasing the source's binary log expiration period. The GTID set sent by the replica is '%s', and the missing transactions are '%s'.", + replicaExecutedGtids.String(), missingGtids.String()) } err := d.streamerManager.StartStream(ctx, conn, replicaExecutedGtids, d.BinlogProducer.binlogFormat, d.BinlogProducer.binlogEventMeta) diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go index 08863bae049..d4ac9783653 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go @@ -50,6 +50,7 @@ type LogManager struct { binlogFormat mysql.BinlogFormat binlogEventMeta mysql.BinlogEventMetadata binlogDirectory string + availableGtids mysql.GTIDSet } // NewLogManager creates a new LogManager instance where binlog files are stored in the .dolt/binlog directory @@ -90,9 +91,48 @@ func NewLogManager(fs filesys.Filesys, binlogFormat mysql.BinlogFormat, binlogEv return nil, err } + // Initialize the set of GTIDs that are available in the current log files + if err := lm.initializeAvailableGtids(); err != nil { + return nil, err + } + return lm, nil } +// initializeAvailableGtids sets the value of availableGtids by seeing what GTIDs have been executed (@@gtid_executed) +// and subtracting any GTIDs that have been marked as purged (@@gtid_purged). +func (lm *LogManager) initializeAvailableGtids() (err error) { + // Initialize availableGtids from @@gtid_executed – we start by assuming we have all executed GTIDs available + // in the logs, and then adjust availableGtids based on which GTIDs we detect have been purged. + _, gtidExecutedValue, ok := sql.SystemVariables.GetGlobal("gtid_executed") + if !ok { + fmt.Errorf("unable to find system variable @@gtid_executed") + } + if _, ok := gtidExecutedValue.(string); !ok { + return fmt.Errorf("unexpected type for @@gtid_executed: %T", gtidExecutedValue) + } + lm.availableGtids, err = mysql.ParseMysql56GTIDSet(gtidExecutedValue.(string)) + if err != nil { + return err + } + + _, gtidPurgedValue, ok := sql.SystemVariables.GetGlobal("gtid_purged") + if !ok { + return fmt.Errorf("unable to find system variable @@gtid_purged") + } + if _, ok := gtidExecutedValue.(string); !ok { + return fmt.Errorf("unexpected type for @@gtid_purged: %T", gtidPurgedValue) + } + purgedGtids, err := mysql.ParseMysql56GTIDSet(gtidPurgedValue.(string)) + if err != nil { + return err + } + + lm.availableGtids = lm.availableGtids.Subtract(purgedGtids) + logrus.Debugf("setting availableGtids to %s after removing purgedGtids %s", lm.availableGtids, purgedGtids) + return nil +} + // purgeExpiredLogFiles removes any binlog files that are older than @@binlog_expire_logs_seconds. This automatic // binlog purging currently happens only on server startup. Eventually this should also be hooked into the `FLUSH LOGS` // (or `FLUSH BINARY LOGS`) statement as well to match MySQL's behavior. @@ -150,9 +190,7 @@ func (lm *LogManager) purgeExpiredLogFiles() error { return err } if len(files) > 0 { - // TODO: This doesn't work! If we purge all the log files and we only have the new file, then we can't - // determine the GTID to set... unless we use gtid_executed? - openFile, err := os.Open(filepath.Join(lm.binlogDirectory, files[0])) + openFile, err := openBinlogFileForReading(filepath.Join(lm.binlogDirectory, files[0])) if err != nil { return err } @@ -178,20 +216,16 @@ func (lm *LogManager) purgeExpiredLogFiles() error { return err } - gtid, ok, err := binlogEvent.GTID(lm.binlogFormat) + gtid, _, err := binlogEvent.GTID(lm.binlogFormat) if err != nil { return err } - if !ok { - panic("not ok!") - } sequenceNumber := gtid.SequenceNumber().(int64) - - gtid.SourceServer() - - err = sql.SystemVariables.SetGlobal("gtid_purged", fmt.Sprintf("%s:%d", gtid.SourceServer(), sequenceNumber-1)) - if err != nil { - return err + if sequenceNumber > 1 { + err = sql.SystemVariables.SetGlobal("gtid_purged", fmt.Sprintf("%s:%d", gtid.SourceServer(), sequenceNumber-1)) + if err != nil { + return err + } } } @@ -446,6 +480,15 @@ func (lm *LogManager) writeEventsHelper(binlogEvents ...mysql.BinlogEvent) error if _, err := lm.currentBinlogFile.Write(event.Bytes()); err != nil { return err } + + if event.IsGTID() { + gtid, _, err := event.GTID(lm.binlogFormat) + if err != nil { + return err + } + // TODO: Consider locking around lm.availableGtids + lm.availableGtids = lm.availableGtids.AddGTID(gtid) + } } if rotateLogFile { @@ -457,6 +500,26 @@ func (lm *LogManager) writeEventsHelper(binlogEvents ...mysql.BinlogEvent) error return nil } +// calculateMissingGtids takes the set of GTIDs that a replica reports it has executed, |replicaExecutedGtids|, +// and the full set of GTIDs executed in this primary server, |primaryExecutedGtids|, and determines which GTIDs +// are needed to bring the replica in sync with this server, but not available in the current binary logs. The +// results are a returned as the set of GTIDs that this server is unable to provide to the replica, since they are +// no longer available in this server's binary logs. If the returned set of GTIDs is empty, then this server can +// accept the connection from the replica and start streaming it the GTIDs needed to get it in sync with this +// primary. +func (lm *LogManager) calculateMissingGtids(replicaExecutedGtids mysql.GTIDSet, primaryExecutedGtids mysql.GTIDSet) mysql.GTIDSet { + // First, subtract all the GTIDs that the replica has executed from the GTIDs this server has executed, + // in order to determine which GTIDs are needed to get the replica in sync with the primary. + neededGtids := primaryExecutedGtids.Subtract(replicaExecutedGtids) + + // Next subtract all the GTIDs that are available in the logs to determine which GTIDs are missing. + missingGtids := neededGtids.Subtract(lm.availableGtids) + + logrus.Debugf("calculateMissingGtids: replicaExecutedGtids: %s, primaryExecutedGtids: %s, neededGtids: %s, availableGtids: %s, missingGtids: %s", replicaExecutedGtids, primaryExecutedGtids, neededGtids, lm.availableGtids, missingGtids) + + return missingGtids +} + func (lm *LogManager) resolveLogFile(filename string) string { // TODO: Should we make sure it exists? return filepath.Join(lm.binlogDirectory, filename) From ae4b3eb777e0ce1b537a276e7cf75275fe518be7 Mon Sep 17 00:00:00 2001 From: Jason Fulghum Date: Mon, 15 Jul 2024 14:32:56 -0700 Subject: [PATCH 07/14] Outputting MySQL error log on failed tests Removing old waitForReplicaToHaveLatestGtid() function, now that we can use waitForReplicaToCatchup() for that one use case Cleaning up and testing initialization for @@gtid_purged and @@gtid_executed --- .../binlog_primary_log_manager.go | 93 ++++++++++--------- .../binlogreplication/binlog_primary_test.go | 49 +++++++++- .../binlog_replication_test.go | 46 +-------- 3 files changed, 100 insertions(+), 88 deletions(-) diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go index d4ac9783653..f65b049b84f 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go @@ -91,6 +91,14 @@ func NewLogManager(fs filesys.Filesys, binlogFormat mysql.BinlogFormat, binlogEv return nil, err } + // Initialize @@gtid_purged based on the first GTID we see available in the available binary logs + // NOTE that we assume that all GTIDs are available after the first GTID we find in the logs. This won't + // be true if someone goes directly to the file system and deletes binary log files, but that isn't + // how we expect people to manage the binary log files. + if err := lm.initializePurgedGtids(); err != nil { + return nil, err + } + // Initialize the set of GTIDs that are available in the current log files if err := lm.initializeAvailableGtids(); err != nil { return nil, err @@ -168,68 +176,63 @@ func (lm *LogManager) purgeExpiredLogFiles() error { } } } + return nil +} - // TODO: This is where things get messy... - // This is where we find the oldest binlog file, and attempt to find the first GTID in it, so that we can - // set @@gtid_purged. - // This should mostly work, but not in the case where there is only one binlog file after purging (the - // current/new binlog file) and it doesn't have a GTID in it. In that case, we could default to - // @@gtid_executed. - // - // One of the problems here is that @@gtid_executed doesn't get set at startup, like it should. We currently need - // a *sql.Context to grab the DoltSession and pull out the location of the GTID position store/file. This happens - // the first time we process a GTID binlog event, but it needs to happen at startup. - // - // Another way to solve this issue would be to have an index file that tracks which GTIDs are present in each log - // file. I'd like to avoid this if possible, just because it adds more complexity and seems like we can get away - // without it right now. - - // Set @@gtid_purged if we deleted any files - files, err = lm.logFilesOnDiskForBranch(BinlogBranch) +// initializePurgedGtids searches through the available binary logs to find the first GTID available +// in the binary logs. If a GTID is found in the available logs, then @@gtid_purged is set to the GTID immediately +// preceding the found GTID, unless the found GTID is sequence number 1. If no GTIDs are found in the available +// binary logs, then it is assumed that all GTIDs have been purged, so @@gtid_purged is set to the same value +// held in @@gtid_executed. +func (lm *LogManager) initializePurgedGtids() error { + files, err := lm.logFilesOnDiskForBranch(BinlogBranch) if err != nil { return err } - if len(files) > 0 { - openFile, err := openBinlogFileForReading(filepath.Join(lm.binlogDirectory, files[0])) - if err != nil { - return err - } - defer openFile.Close() - - binlogEvent, err := readFirstGtidEventFromFile(openFile) - if err == io.EOF { - // If there are no GTID events in the file, then all GTIDs have been purged, so use @@gtid_executed - _, gtidExecutedValue, ok := sql.SystemVariables.GetGlobal("gtid_executed") - if !ok { - return fmt.Errorf("unable to find system variable @@gtid_executed") - } - logrus.Errorf("setting gtid_purged to: %s", gtidExecutedValue) + for _, file := range files { + var gtid mysql.GTID + err := func() error { + openFile, err := openBinlogFileForReading(filepath.Join(lm.binlogDirectory, file)) + if err != nil { + return err + } + defer openFile.Close() - err = sql.SystemVariables.SetGlobal("gtid_purged", gtidExecutedValue) + binlogEvent, err := readFirstGtidEventFromFile(openFile) if err != nil { return err } + gtid, _, err = binlogEvent.GTID(lm.binlogFormat) + if err != nil { + return err + } + sequenceNumber := gtid.SequenceNumber().(int64) + if sequenceNumber > 1 { + // If the first found GTID in the available binary logs is sequence number 1, then all GTIDs + // are available, so no need to set @@gtid_purged to anything. + return sql.SystemVariables.SetGlobal("gtid_purged", fmt.Sprintf("%s:%d", gtid.SourceServer(), sequenceNumber-1)) + } return nil - } else if err != nil { + }() + if err != nil && err != io.EOF { return err } - gtid, _, err := binlogEvent.GTID(lm.binlogFormat) - if err != nil { - return err - } - sequenceNumber := gtid.SequenceNumber().(int64) - if sequenceNumber > 1 { - err = sql.SystemVariables.SetGlobal("gtid_purged", fmt.Sprintf("%s:%d", gtid.SourceServer(), sequenceNumber-1)) - if err != nil { - return err - } + // If we found a GTID, then stop searching through logs and return + if gtid != nil { + return nil } } - return nil + // If there are no GTID events in any of the files, then all GTIDs have been purged, so use @@gtid_executed + _, gtidExecutedValue, ok := sql.SystemVariables.GetGlobal("gtid_executed") + if !ok { + return fmt.Errorf("unable to find system variable @@gtid_executed") + } + logrus.Debugf("setting gtid_purged to: %s", gtidExecutedValue) + return sql.SystemVariables.SetGlobal("gtid_purged", gtidExecutedValue) } // addRotateEventToPreviousLogFile finds the previous binlog file and appends a Rotate event that points to the diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go index 8fb910c3500..f4e5e763b87 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go @@ -16,6 +16,8 @@ package binlogreplication import ( "fmt" + "os" + "path/filepath" "testing" "time" @@ -199,6 +201,51 @@ func TestBinlogPrimary_AutoPurging(t *testing.T) { "'Cannot replicate because the source purged required binary logs.") } +// TestBinlogPrimary_InitializeGTIDPurged asserts that @@gtid_purged is set correctly in a variety of +// scenarios, such as when a fresh server starts up, or when a server is restarted multiple times. +func TestBinlogPrimary_InitializeGTIDPurged(t *testing.T) { + defer teardown(t) + startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) + setupForDoltToMySqlReplication() + + // On a fresh server, @@gtid_purged and @@gtid_executed should be empty + requirePrimaryResults(t, "SELECT @@gtid_executed;", [][]any{{""}}) + requirePrimaryResults(t, "SELECT @@gtid_purged;", [][]any{{""}}) + + // Create a GTID in the first binary log file, and restart the server to rotate to a new binary log file + // After the first restart, @@gtid_purged should be empty and @@gtid_executed should be the first GTID + primaryDatabase.MustExec("CREATE DATABASE db01;") + stopDoltSqlServer(t) + mustRestartDoltPrimaryServer(t) + requirePrimaryResults(t, "SELECT @@gtid_executed;", [][]any{ + {fmt.Sprintf("%s:1", queryPrimaryServerUuid(t))}, + }) + requirePrimaryResults(t, "SELECT @@gtid_purged;", [][]any{{""}}) + + // Manually remove the first binary log file, containing GTID 1 and restart the server + // When no GTID is found in any available logs, @@gtid_purged should be set to @@gtid_executed + require.NoError(t, os.Remove(filepath.Join(testDir, "dolt", ".dolt", "binlog", "binlog-main.000001"))) + stopDoltSqlServer(t) + mustRestartDoltPrimaryServer(t) + requirePrimaryResults(t, "SELECT @@gtid_executed;", [][]any{ + {fmt.Sprintf("%s:1", queryPrimaryServerUuid(t))}, + }) + requirePrimaryResults(t, "SELECT @@gtid_purged;", [][]any{ + {fmt.Sprintf("%s:1", queryPrimaryServerUuid(t))}, + }) + + // Create a new GTID in the current binary log file, restart, and test @@gtid_executed and @@gtid_purged + primaryDatabase.MustExec("CREATE DATABASE db02;") + stopDoltSqlServer(t) + mustRestartDoltPrimaryServer(t) + requirePrimaryResults(t, "SELECT @@gtid_executed;", [][]any{ + {fmt.Sprintf("%s:1-2", queryPrimaryServerUuid(t))}, + }) + requirePrimaryResults(t, "SELECT @@gtid_purged;", [][]any{ + {fmt.Sprintf("%s:1", queryPrimaryServerUuid(t))}, + }) +} + // TestBinlogPrimary_ReplicaAndPrimaryRestart tests that a replica can disconnect and reconnect to the primary to // restart the replication stream, even when the primary has been restarted and log files have rotated. func TestBinlogPrimary_ReplicaAndPrimaryRestart(t *testing.T) { @@ -328,7 +375,7 @@ func TestBinlogPrimary_ReplicaRestart(t *testing.T) { // Create another table and assert that it gets replicated primaryDatabase.MustExec("create table db01.t2 (pk int primary key, c1 varchar(255));") - waitForReplicaToHaveLatestGtid(t) + waitForReplicaToCatchUp(t) requireReplicaResults(t, "show tables;", [][]any{{"t1"}, {"t2"}}) // Assert the executed GTID position now contains all GTIDs diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_test.go index 80befb065dc..97d07cc4be1 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_test.go @@ -33,7 +33,6 @@ import ( "testing" "time" - "github.com/dolthub/vitess/go/mysql" _ "github.com/go-sql-driver/mysql" "github.com/jmoiron/sqlx" "github.com/stretchr/testify/require" @@ -74,6 +73,9 @@ func teardown(t *testing.T) { printFile(doltLogFilePath) fmt.Printf("\nMySQL server log from %s:\n", mysqlLogFilePath) printFile(mysqlLogFilePath) + mysqlErrorLogFilePath := filepath.Join(filepath.Dir(mysqlLogFilePath), "error_log.err") + fmt.Printf("\nMySQL server error log from %s:\n", mysqlErrorLogFilePath) + printFile(mysqlErrorLogFilePath) } else { // clean up temp files on clean test runs defer os.RemoveAll(testDir) @@ -479,45 +481,6 @@ func waitForReplicaToCatchUp(t *testing.T) { t.Fatal("primary and replica did not synchronize within " + timeLimit.String()) } -// waitForReplicaToHaveLatestGtid waits (up to 10s) for the replica to contain the -// most recent GTID executed from the primary. Both the primary and replica are queried -// for the value of @@gtid_executed to determine if the replica contains the most recent -// transaction from the primary. -func waitForReplicaToHaveLatestGtid(t *testing.T) { - timeLimit := 10 * time.Second - endTime := time.Now().Add(timeLimit) - for time.Now().Before(endTime) { - replicaGtid := queryGtid(t, replicaDatabase) - primaryGtid := queryGtid(t, primaryDatabase) - - replicaGtidSet, err := mysql.ParseMysql56GTIDSet(replicaGtid) - require.NoError(t, err) - - idx := strings.Index(primaryGtid, ":") - require.True(t, idx > 0) - uuid := primaryGtid[:idx] - sequencePortion := primaryGtid[idx+1:] - if strings.Contains(sequencePortion, ":") { - sequencePortion = sequencePortion[strings.LastIndex(sequencePortion, ":")+1:] - } - if strings.Contains(sequencePortion, "-") { - sequencePortion = sequencePortion[strings.LastIndex(sequencePortion, "-")+1:] - } - - latestGtid, err := mysql.ParseGTID("MySQL56", uuid+":"+sequencePortion) - require.NoError(t, err) - - if replicaGtidSet.ContainsGTID(latestGtid) { - return - } else { - fmt.Printf("replica does not contain latest GTID from the primary yet... (primary: %s, replica: %s)\n", primaryGtid, replicaGtid) - time.Sleep(250 * time.Millisecond) - } - } - - t.Fatal("replica did not synchronize the latest GTID from the primary within " + timeLimit.String()) -} - // waitForReplicaToReachGtid waits (up to 10s) for the replica's @@gtid_executed sys var to show that // it has executed the |target| gtid transaction number. func waitForReplicaToReachGtid(t *testing.T, target int) { @@ -822,9 +785,8 @@ func startMySqlServer(dir string) (int, *os.Process, error) { "--server-id=11223344", fmt.Sprintf("--socket=mysql-%v.sock", mySqlPort), "--general_log_file="+dir+"general_log", - "--log-bin="+dir+"log_bin", "--slow_query_log_file="+dir+"slow_query_log", - "--log-error="+dir+"log_error", + "--log-error="+dir+"error_log", fmt.Sprintf("--pid-file="+dir+"pid-%v.pid", mySqlPort)) mysqlLogFilePath = filepath.Join(dir, fmt.Sprintf("mysql-%d.out.log", time.Now().Unix())) From 5ec79ee706f8eccd30525bc5012a4ad7cea1d92a Mon Sep 17 00:00:00 2001 From: Jason Fulghum Date: Mon, 15 Jul 2024 16:51:24 -0700 Subject: [PATCH 08/14] Additional tests for log file rotation on server startup --- .../sqle/binlogreplication/binlog_primary_test.go | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go index f4e5e763b87..2e02d65bd75 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go @@ -391,6 +391,11 @@ func TestBinlogPrimary_PrimaryRestart(t *testing.T) { setupForDoltToMySqlReplication() startReplication(t, doltPort) + // Only one binary log file should be present on a fresh server + requirePrimaryResults(t, "show binary logs;", [][]any{ + {"binlog-main.000001", "263", "No"}, + }) + // Create a table on the primary and assert that it gets replicated primaryDatabase.MustExec("create table db01.t1 (pk int primary key, c1 varchar(255));") waitForReplicaToCatchUp(t) @@ -409,6 +414,12 @@ func TestBinlogPrimary_PrimaryRestart(t *testing.T) { mustRestartDoltPrimaryServer(t) waitForReplicaToReconnect(t) + // A new binary log file is created on each server restart + requirePrimaryResults(t, "show binary logs;", [][]any{ + {"binlog-main.000001", "549", "No"}, + {"binlog-main.000002", "191", "No"}, + }) + // Create another table and assert that it gets replicated primaryDatabase.MustExec("create table db01.t2 (pk int primary key, c1 varchar(255));") waitForReplicaToCatchUp(t) From 17a1627d1b2ae7485694cc45037b34d70028ed36 Mon Sep 17 00:00:00 2001 From: Jason Fulghum Date: Fri, 19 Jul 2024 14:07:07 -0700 Subject: [PATCH 09/14] Adding error checking and tests for edge cases with primary server configuration --- go/cmd/dolt/commands/engine/sqlengine.go | 1 - go/cmd/dolt/commands/sqlserver/server.go | 38 +++-- .../binlog_primary_controller.go | 97 +++++++++--- .../binlog_primary_log_manager.go | 115 ++++++++------ .../binlog_primary_streamer.go | 65 ++++---- .../binlogreplication/binlog_primary_test.go | 146 ++++++++++++------ .../sqle/binlogreplication/binlog_producer.go | 27 ++-- .../binlog_replication_alltypes_test.go | 2 +- .../binlog_replication_filters_test.go | 6 +- .../binlog_replication_multidb_test.go | 4 +- .../binlog_replication_reconnect_test.go | 2 +- .../binlog_replication_restart_test.go | 2 +- .../binlog_replication_test.go | 45 +++--- .../doltcore/sqle/system_variables.go | 9 -- 14 files changed, 339 insertions(+), 220 deletions(-) diff --git a/go/cmd/dolt/commands/engine/sqlengine.go b/go/cmd/dolt/commands/engine/sqlengine.go index 59938bba290..f5ef921ac64 100644 --- a/go/cmd/dolt/commands/engine/sqlengine.go +++ b/go/cmd/dolt/commands/engine/sqlengine.go @@ -348,7 +348,6 @@ func configureBinlogReplicaController(config *SqlEngineConfig, engine *gms.Engin func configureBinlogPrimaryController(engine *gms.Engine) error { primaryController := dblr.NewDoltBinlogPrimaryController() engine.Analyzer.Catalog.BinlogPrimaryController = primaryController - return nil } diff --git a/go/cmd/dolt/commands/sqlserver/server.go b/go/cmd/dolt/commands/sqlserver/server.go index 257c7ab150e..63b33258aec 100644 --- a/go/cmd/dolt/commands/sqlserver/server.go +++ b/go/cmd/dolt/commands/sqlserver/server.go @@ -285,6 +285,22 @@ func ConfigureServices( } controller.Register(InitSqlEngine) + // Persist any system variables that have a non-deterministic default value (i.e. @@server_uuid) + // We only do this on sql-server startup initially since we want to keep the persisted server_uuid + // in the configuration files for a sql-server, and not global for the whole host. + PersistNondeterministicSystemVarDefaults := &svcs.AnonService{ + InitF: func(ctx context.Context) error { + err := dsess.PersistSystemVarDefaults(dEnv) + if err != nil { + logrus.Errorf("unable to persist system variable defaults: %v", err) + } + // Always return nil, because we don't want an invalid config value to prevent + // the server from starting up. + return nil + }, + } + controller.Register(PersistNondeterministicSystemVarDefaults) + InitBinlogging := &svcs.AnonService{ InitF: func(context.Context) error { primaryController := sqlEngine.GetUnderlyingEngine().Analyzer.Catalog.BinlogPrimaryController @@ -303,29 +319,25 @@ func ConfigureServices( } if logBin == 1 { logrus.Debug("Enabling binary logging") - binlogProducer, err := binlogreplication.NewBinlogProducer(dEnv.FS, doltBinlogPrimaryController.StreamerManager()) + binlogProducer, err := binlogreplication.NewBinlogProducer(dEnv.FS) if err != nil { return err } - // NOTE: Hooks need to be applied AFTER this! + + logManager, err := binlogreplication.NewLogManager(fs) + if err != nil { + return err + } + binlogProducer.LogManager(logManager) doltdb.RegisterDatabaseUpdateListener(binlogProducer) - doltBinlogPrimaryController.BinlogProducer = binlogProducer + doltBinlogPrimaryController.BinlogProducer(binlogProducer) - // TODO: This is a mess! How to clean this up! + // Register binlog hooks for database creation/deletion provider := sqlEngine.GetUnderlyingEngine().Analyzer.Catalog.DbProvider if doltProvider, ok := provider.(*sqle.DoltDatabaseProvider); ok { doltProvider.AddInitDatabaseHook(binlogreplication.NewBinlogInitDatabaseHook(nil, doltdb.DatabaseUpdateListeners)) doltProvider.AddDropDatabaseHook(binlogreplication.NewBinlogDropDatabaseHook(nil, doltdb.DatabaseUpdateListeners)) } - - // TODO: How do we feed the binlogStream and binlogFormat to the log manager? - // Needs format to write the initial format event to the stream - // Needs binlogStream to write the initial format event - logManager, err := binlogreplication.NewLogManager(fs, *binlogProducer.BinlogFormat(), binlogProducer.BinlogStream()) - if err != nil { - return err - } - doltBinlogPrimaryController.StreamerManager().LogManager(logManager) } _, logBinBranchValue, ok := sql.SystemVariables.GetGlobal("log_bin_branch") diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go index b16aea0b66f..7098defcf8b 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go @@ -21,6 +21,7 @@ import ( "github.com/dolthub/go-mysql-server/sql" "github.com/dolthub/go-mysql-server/sql/binlogreplication" + gmstypes "github.com/dolthub/go-mysql-server/sql/types" "github.com/dolthub/vitess/go/mysql" "github.com/sirupsen/logrus" ) @@ -30,21 +31,21 @@ import ( // interprets commands and statements related to serving binlog events. type DoltBinlogPrimaryController struct { streamerManager *binlogStreamerManager - BinlogProducer *binlogProducer + binlogProducer *binlogProducer } var _ binlogreplication.BinlogPrimaryController = (*DoltBinlogPrimaryController)(nil) // NewDoltBinlogPrimaryController creates a new DoltBinlogPrimaryController instance. func NewDoltBinlogPrimaryController() *DoltBinlogPrimaryController { - controller := DoltBinlogPrimaryController{ + return &DoltBinlogPrimaryController{ streamerManager: newBinlogStreamerManager(), } - return &controller } -func (d *DoltBinlogPrimaryController) StreamerManager() *binlogStreamerManager { - return d.streamerManager +func (d *DoltBinlogPrimaryController) BinlogProducer(binlogProducer *binlogProducer) { + d.binlogProducer = binlogProducer + d.streamerManager.logManager = binlogProducer.logManager } // RegisterReplica implements the BinlogPrimaryController interface. @@ -55,27 +56,73 @@ func (d *DoltBinlogPrimaryController) StreamerManager() *binlogStreamerManager { // replica if bin logging isn't enabled, since errors returned from the BinlogDumpGtid method seem to be dropped // by the replica, instead of being displayed as an error. func (d *DoltBinlogPrimaryController) RegisterReplica(ctx *sql.Context, c *mysql.Conn, replicaHost string, replicaPort uint16) error { - if d.BinlogProducer == nil { + if d.binlogProducer == nil { return fmt.Errorf("no binlog currently being recorded; make sure the server is started with @@log_bin enabled") } return nil } -// BinlogDumpGtid implements the BinlogPrimaryController interface. -func (d *DoltBinlogPrimaryController) BinlogDumpGtid(ctx *sql.Context, conn *mysql.Conn, replicaExecutedGtids mysql.GTIDSet) error { - if d.BinlogProducer == nil { - // TODO: Add a test for this, now that we have errors being - // reported in replica status correctly +// validateReplicationConfiguration checks that this server is properly configured to replicate databases, meaning +// that @@log_bin is enabled, @@gtid_mode is enabled, @@enforce_gtid_consistency is enabled, and the binlog producer +// has been instantiated. If any of this configuration is not valid, then an error is returned. +func (d *DoltBinlogPrimaryController) validateReplicationConfiguration() *mysql.SQLError { + if d.binlogProducer == nil { return mysql.NewSQLError(mysql.ERMasterFatalReadingBinlog, "HY000", "no binlog currently being recorded; make sure the server is started with @@log_bin enabled") + } + _, logBinValue, ok := sql.SystemVariables.GetGlobal("log_bin") + if !ok { + return mysql.NewSQLError(mysql.ERUnknownError, "HY000", "unable to find system variable @@log_bin") + } + logBin, _, err := gmstypes.Boolean.Convert(logBinValue) + if err != nil { + return mysql.NewSQLError(mysql.ERUnknownError, "HY000", err.Error()) + } + if logBin.(int8) != 1 { + return mysql.NewSQLError(mysql.ERMasterFatalReadingBinlog, "HY000", + "no binlog currently being recorded; make sure the server is started with @@log_bin enabled") } - primaryExecutedGtids := d.BinlogProducer.gtidPosition.GTIDSet - // TODO: This is awkward... should controller have a direct link to log manager? - // Or should BinlogProducer have a direct link to the log manager? - missingGtids := d.BinlogProducer.streamerManager.logManager.calculateMissingGtids(replicaExecutedGtids, primaryExecutedGtids) + _, gtidModeValue, ok := sql.SystemVariables.GetGlobal("gtid_mode") + if !ok { + return mysql.NewSQLError(mysql.ERUnknownError, "HY000", "unable to find system variable @@log_bin") + } + gtidMode, ok := gtidModeValue.(string) + if !ok { + return mysql.NewSQLError(mysql.ERUnknownError, "HY000", "unexpected type for @@gtid_mode: %T", gtidModeValue) + } + if gtidMode != "ON" { + return mysql.NewSQLError(mysql.ERMasterFatalReadingBinlog, "HY000", + "@@gtid_mode must be enabled for binlog replication") + } + + _, enforceGtidConsistencyValue, ok := sql.SystemVariables.GetGlobal("enforce_gtid_consistency") + if !ok { + return mysql.NewSQLError(mysql.ERUnknownError, "HY000", "unable to find system variable @@log_bin") + } + enforceGtidConsistency, ok := enforceGtidConsistencyValue.(string) + if !ok { + return mysql.NewSQLError(mysql.ERUnknownError, "HY000", + "unexpected type for @@enforce_gtid_consistency: %T", enforceGtidConsistencyValue) + } + if enforceGtidConsistency != "ON" { + return mysql.NewSQLError(mysql.ERMasterFatalReadingBinlog, "HY000", + "@@enforce_gtid_consistency must be enabled for binlog replication") + } + + return nil +} + +// BinlogDumpGtid implements the BinlogPrimaryController interface. +func (d *DoltBinlogPrimaryController) BinlogDumpGtid(ctx *sql.Context, conn *mysql.Conn, replicaExecutedGtids mysql.GTIDSet) error { + if err := d.validateReplicationConfiguration(); err != nil { + return err + } + + primaryExecutedGtids := d.binlogProducer.gtidPosition.GTIDSet + missingGtids := d.binlogProducer.logManager.calculateMissingGtids(replicaExecutedGtids, primaryExecutedGtids) if !missingGtids.Equal(mysql.Mysql56GTIDSet{}) { // We must send back error code 1236 (ER_MASTER_FATAL_ERROR_READING_BINLOG) to the replica to signal an error, // otherwise the replica won't expose the error in replica status and will just keep trying to reconnect and @@ -85,7 +132,7 @@ func (d *DoltBinlogPrimaryController) BinlogDumpGtid(ctx *sql.Context, conn *mys replicaExecutedGtids.String(), missingGtids.String()) } - err := d.streamerManager.StartStream(ctx, conn, replicaExecutedGtids, d.BinlogProducer.binlogFormat, d.BinlogProducer.binlogEventMeta) + err := d.streamerManager.StartStream(ctx, conn, replicaExecutedGtids, d.binlogProducer.binlogFormat, d.binlogProducer.binlogEventMeta) if err != nil { logrus.Warnf("exiting binlog streamer due to error: %s", err.Error()) } else { @@ -102,7 +149,11 @@ func (d *DoltBinlogPrimaryController) ListReplicas(ctx *sql.Context) error { // ListBinaryLogs implements the BinlogPrimaryController interface. func (d *DoltBinlogPrimaryController) ListBinaryLogs(_ *sql.Context) ([]binlogreplication.BinaryLogFileMetadata, error) { - logManager := d.streamerManager.logManager + if d.binlogProducer == nil || d.binlogProducer.logManager == nil { + return nil, nil + } + logManager := d.binlogProducer.logManager + logFiles, err := logManager.logFilesOnDiskForBranch(BinlogBranch) if err != nil { return nil, err @@ -123,10 +174,14 @@ func (d *DoltBinlogPrimaryController) ListBinaryLogs(_ *sql.Context) ([]binlogre } // GetBinaryLogStatus implements the BinlogPrimaryController interface. -func (d *DoltBinlogPrimaryController) GetBinaryLogStatus(ctx *sql.Context) ([]binlogreplication.BinaryLogStatus, error) { +func (d *DoltBinlogPrimaryController) GetBinaryLogStatus(_ *sql.Context) ([]binlogreplication.BinaryLogStatus, error) { + if d.binlogProducer == nil || d.binlogProducer.logManager == nil { + return nil, nil + } + return []binlogreplication.BinaryLogStatus{{ - File: d.streamerManager.logManager.currentBinlogFileName, - Position: uint(d.streamerManager.logManager.currentPosition), - ExecutedGtids: d.BinlogProducer.currentGtidPosition(), + File: d.binlogProducer.logManager.currentBinlogFileName, + Position: uint(d.binlogProducer.logManager.currentPosition), + ExecutedGtids: d.binlogProducer.currentGtidPosition(), }}, nil } diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go index f65b049b84f..3f2fc3126ea 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go @@ -40,32 +40,38 @@ var binlogDirectory = filepath.Join(".dolt", "binlog") var binlogFileMagicNumber = []byte{0xfe, 0x62, 0x69, 0x6e} // LogManager is responsible for the binary log files on disk, including actually writing events to the log files, -// rotating the log files, listing the available log files, and purging old log files. +// rotating the log files, listing the available log files, purging old log files, and keeping track of what GTIDs +// are available in the log files. type LogManager struct { + binlogFormat mysql.BinlogFormat + binlogEventMeta mysql.BinlogEventMetadata + mu *sync.Mutex currentBinlogFile *os.File currentBinlogFileName string currentPosition int fs filesys.Filesys - binlogFormat mysql.BinlogFormat - binlogEventMeta mysql.BinlogEventMetadata binlogDirectory string availableGtids mysql.GTIDSet } // NewLogManager creates a new LogManager instance where binlog files are stored in the .dolt/binlog directory -// underneath the specified |fs| filesystem. The |binlogFormat| and |binlogStream| are used to initialize the -// new binlog file. -func NewLogManager(fs filesys.Filesys, binlogFormat mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) (*LogManager, error) { +// underneath the specified |fs| filesystem. This method also initializes the binlog logging system, including +// rotating to a new log file and purging expired log files. +func NewLogManager(fs filesys.Filesys) (*LogManager, error) { + binlogFormat := createBinlogFormat() + binlogEventMeta, err := createBinlogEventMetadata() + if err != nil { + return nil, err + } + lm := &LogManager{ mu: &sync.Mutex{}, fs: fs, - binlogFormat: binlogFormat, - binlogEventMeta: binlogEventMeta, + binlogFormat: *binlogFormat, + binlogEventMeta: *binlogEventMeta, } - // TODO: Could resolve the base dir for the binlog file directory here; would it help us avoid returning errors in other APIs? - // Initialize binlog file storage directory if err := fs.MkDirs(binlogDirectory); err != nil { return nil, err @@ -157,20 +163,20 @@ func (lm *LogManager) purgeExpiredLogFiles() error { purgeThresholdTime := time.Now().Add(-time.Duration(expireLogsSeconds) * time.Second) - files, err := lm.logFilesOnDiskForBranch(BinlogBranch) + filenames, err := lm.logFilesOnDiskForBranch(BinlogBranch) if err != nil { return err } - for i := 0; i < len(files); i++ { - fullLogFilepath := filepath.Join(lm.binlogDirectory, files[i]) + for _, filename := range filenames { + fullLogFilepath := lm.resolveLogFile(filename) stat, err := os.Stat(fullLogFilepath) if err != nil { return err } if stat.ModTime().Before(purgeThresholdTime) { - logrus.Debugf("purging expired binlog file: %s", files[i]) + logrus.Debugf("purging expired binlog filename: %s", filename) if err := os.Remove(fullLogFilepath); err != nil { return err } @@ -185,56 +191,64 @@ func (lm *LogManager) purgeExpiredLogFiles() error { // binary logs, then it is assumed that all GTIDs have been purged, so @@gtid_purged is set to the same value // held in @@gtid_executed. func (lm *LogManager) initializePurgedGtids() error { - files, err := lm.logFilesOnDiskForBranch(BinlogBranch) + filenames, err := lm.logFilesOnDiskForBranch(BinlogBranch) if err != nil { return err } - for _, file := range files { - var gtid mysql.GTID - err := func() error { - openFile, err := openBinlogFileForReading(filepath.Join(lm.binlogDirectory, file)) - if err != nil { - return err - } - defer openFile.Close() - - binlogEvent, err := readFirstGtidEventFromFile(openFile) - if err != nil { - return err - } - - gtid, _, err = binlogEvent.GTID(lm.binlogFormat) - if err != nil { - return err - } - sequenceNumber := gtid.SequenceNumber().(int64) - if sequenceNumber > 1 { - // If the first found GTID in the available binary logs is sequence number 1, then all GTIDs - // are available, so no need to set @@gtid_purged to anything. - return sql.SystemVariables.SetGlobal("gtid_purged", fmt.Sprintf("%s:%d", gtid.SourceServer(), sequenceNumber-1)) - } - return nil - }() - if err != nil && err != io.EOF { + for _, filename := range filenames { + gtid, err := lm.findFirstGtidInFile(filename) + if err == io.EOF { + continue + } else if err != nil { return err } - // If we found a GTID, then stop searching through logs and return - if gtid != nil { + // If the first found GTID in the available binary logs is anything other than sequence number 1, + // then we need to set @@gtid_purged to indicate that not all GTIDs are available in the logs, and + // all GTIDs before the first sequence number found have been purged. + sequenceNumber := gtid.SequenceNumber().(int64) + if sequenceNumber > 1 { + gtidPurged := fmt.Sprintf("%s:%d", gtid.SourceServer(), sequenceNumber-1) + logrus.Debugf("setting gtid_purged to: %s", gtidPurged) + return sql.SystemVariables.SetGlobal("gtid_purged", gtidPurged) + } else { return nil } } - // If there are no GTID events in any of the files, then all GTIDs have been purged, so use @@gtid_executed + // If there are no GTID events in any of the files, then all GTIDs have been purged, so + // initialize @@gtid_purged with the value of @@gtid_executed. _, gtidExecutedValue, ok := sql.SystemVariables.GetGlobal("gtid_executed") if !ok { return fmt.Errorf("unable to find system variable @@gtid_executed") } - logrus.Debugf("setting gtid_purged to: %s", gtidExecutedValue) + logrus.Debugf("no available GTIDs found in logs, setting gtid_purged to: %s", gtidExecutedValue) return sql.SystemVariables.SetGlobal("gtid_purged", gtidExecutedValue) } +// findFirstGtidInFile opens the file with the base name |filename| in the binlog directory and +// reads the events until a GTID event is found, then extracts the GTID value and returns it. +func (lm *LogManager) findFirstGtidInFile(filename string) (gtid mysql.GTID, err error) { + openFile, err := openBinlogFileForReading(lm.resolveLogFile(filename)) + if err != nil { + return nil, err + } + defer openFile.Close() + + binlogEvent, err := readFirstGtidEventFromFile(openFile) + if err != nil { + return nil, err + } + + gtid, _, err = binlogEvent.GTID(lm.binlogFormat) + if err != nil { + return nil, err + } + + return gtid, nil +} + // addRotateEventToPreviousLogFile finds the previous binlog file and appends a Rotate event that points to the // next binlog file. This is necessary so that as streamers are reading from the binlog files, they have a pointer // to follow to the next binlog file. This function MUST be called after the new log file has been initialized, @@ -246,12 +260,12 @@ func (lm *LogManager) addRotateEventToPreviousLogFile() error { } // If the previous log file in the sequence has been purged, then there's nothing to do - if !fileExists(filepath.Join(lm.binlogDirectory, previousLogFileName)) { + if !fileExists(lm.resolveLogFile(previousLogFileName)) { return nil } // Open the log file and append a Rotate event - previousLogFile, err := os.OpenFile(filepath.Join(lm.binlogDirectory, previousLogFileName), os.O_WRONLY|os.O_APPEND, 0644) + previousLogFile, err := os.OpenFile(lm.resolveLogFile(previousLogFileName), os.O_WRONLY|os.O_APPEND, 0644) if err != nil { return err } @@ -393,8 +407,6 @@ func (lm *LogManager) PurgeLogFiles() error { // TODO: implement support for purging older binlog files // This also requires setting gtid_purged // https://dev.mysql.com/doc/refman/8.0/en/replication-options-gtids.html#sysvar_gtid_purged - // Need to test the case where the GTID requested is not - // available –has been executed, but has been purged return nil } @@ -523,8 +535,9 @@ func (lm *LogManager) calculateMissingGtids(replicaExecutedGtids mysql.GTIDSet, return missingGtids } +// resolveLogFile accepts a base filename of a binlog file and returns a fully qualified file +// path to the file in the binlog storage directory. func (lm *LogManager) resolveLogFile(filename string) string { - // TODO: Should we make sure it exists? return filepath.Join(lm.binlogDirectory, filename) } diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go index 8810bf2046e..517a10dbcb8 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go @@ -51,12 +51,12 @@ func newBinlogStreamer() *binlogStreamer { // and |binlogEventMeta| records the position of the stream. This method blocks until an error // is received over the stream (e.g. the connection closing) or the streamer is closed, // through it's quit channel. -func (streamer *binlogStreamer) startStream(ctxs *sql.Context, conn *mysql.Conn, executedGtids mysql.GTIDSet, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata, logfile string) error { +func (streamer *binlogStreamer) startStream(ctx *sql.Context, conn *mysql.Conn, executedGtids mysql.GTIDSet, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata, logfile string) error { logrus.WithField("connection_id", conn.ConnectionID). WithField("executed_gtids", executedGtids.String()). Trace("starting binlog stream") - if err := sendInitialEvents(ctx, conn, binlogFormat, binlogEventMeta); err != nil { + if err := sendInitialEvents(ctx, conn, logfile, binlogFormat, binlogEventMeta); err != nil { return err } @@ -68,7 +68,7 @@ func (streamer *binlogStreamer) startStream(ctxs *sql.Context, conn *mysql.Conn, binlogEventMetaCopy := binlogEventMeta binlogEventMetaCopy.NextLogPosition = 0 - rotateEvent := mysql.NewFakeRotateEvent(*binlogFormat, binlogEventMetaCopy, filepath.Base(logfile)) + rotateEvent := mysql.NewFakeRotateEvent(*binlogFormat, *binlogEventMetaCopy, filepath.Base(logfile)) if err = conn.WriteBinlogEvent(rotateEvent, false); err != nil { return err } @@ -97,9 +97,6 @@ func (streamer *binlogStreamer) startStream(ctxs *sql.Context, conn *mysql.Conn, default: logrus.Debug("checking file for new data...") skippingGtids := false - // TODO: This for loop is nested in side a select block, inside another for loop - // We should be able to get rid of this inner for loop, otherwise the select - // block won't ever let us read from ticker or quitChan for { binlogEvent, err := readBinlogEventFromFile(file) if err == io.EOF { @@ -226,31 +223,26 @@ func (m *binlogStreamerManager) copyStreamers() []*binlogStreamer { // StartStream starts a new binlogStreamer and streams events over |conn| until the connection // is closed, the streamer is sent a quit signal over its quit channel, or the streamer receives // errors while sending events over the connection. Note that this method blocks until the -// streamer exits. +// streamer exits. Note that this function does NOT validate that the primary has the correct set +// of GTIDs available to get the replica in sync with the primary – it is expected for that +// validation to have been completed before starting a binlog stream. func (m *binlogStreamerManager) StartStream(ctx *sql.Context, conn *mysql.Conn, executedGtids mysql.GTIDSet, binlogFormat *mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) error { streamer := newBinlogStreamer() m.addStreamer(streamer) defer m.removeStreamer(streamer) - // TODO: This may be a little tricky with branch switching... perhaps we should destroy the index when we - // switch branches and rebuild it as we create more logs? - // It seems like part of the contract with switching branches for replication is that the primary will - // invalidate/destroy all the previous binlog files? It doesn't seem safe to rely on them being - // correct after changing the branch back and forth. - file, err := m.findLogFileForPosition(executedGtids) if err != nil { return err } - // TODO: We also need to handle cases where we are missing GTIDs that the replica doesn't have yet. In these - // cases we need to send errors back to the replica. - return streamer.startStream(ctx, conn, executedGtids, binlogFormat, binlogEventMeta, file) + return streamer.startStream(ctx, conn, executedGtids, binlogFormat, &binlogEventMeta, file) } -// findLogFileForPosition searches through the binlog files on disk for the first log file that contains GTIDs that -// are not present in |executedGtids|. This is determined by reading the first GTID event from each log file and -// selecting the previous file when the first GTID not in |executedGtids| is found. +// findLogFileForPosition searches through the available binlog files on disk for the first log file that +// contains GTIDs that are NOT present in |executedGtids|. This is determined by reading the first GTID event +// from each log file and selecting the previous file when the first GTID not in |executedGtids| is found. If +// the first GTID event in all available logs files is in |executedGtids|, then the current log file is returned. func (m *binlogStreamerManager) findLogFileForPosition(executedGtids mysql.GTIDSet) (string, error) { files, err := m.logManager.logFilesOnDiskForBranch(BinlogBranch) if err != nil { @@ -258,13 +250,16 @@ func (m *binlogStreamerManager) findLogFileForPosition(executedGtids mysql.GTIDS } for i, f := range files { - file, err := openBinlogFileForReading(filepath.Join(m.logManager.binlogDirectory, f)) + binlogFilePath := filepath.Join(m.logManager.binlogDirectory, f) + file, err := openBinlogFileForReading(binlogFilePath) if err != nil { return "", err } binlogEvent, err := readFirstGtidEventFromFile(file) - file.Close() + if fileCloseErr := file.Close(); fileCloseErr != nil { + logrus.Errorf("unable to cleanly close binlog file %s: %s", f, err.Error()) + } if err == io.EOF { continue } else if err != nil { @@ -282,19 +277,19 @@ func (m *binlogStreamerManager) findLogFileForPosition(executedGtids mysql.GTIDS continue } + // If we found an unexecuted GTID in the first binlog file, return the first file, + // otherwise return the previous file if i == 0 { - return m.logManager.currentBinlogFilepath(), nil + return binlogFilePath, nil + } else { + return filepath.Join(m.logManager.binlogDirectory, files[i-1]), nil } - return filepath.Join(m.logManager.binlogDirectory, files[i-1]), nil } } - // TODO: If we didn't find an unexecuted GTID in any of the files, just return - // the current log file. This is a bit of a hack, but it's enough to - // keep the following tests passing: - // TestBinlogPrimary_ChangeReplicationBranch - // TestBinlogPrimary_PrimaryRestart - // It might actually be better to just return the first available log file? + // If we don't find any GTIDs that are missing from |executedGtids|, then return the current + // log file so the streamer can reply events from it, potentially finding GTIDs later in the + // file that need to be sent to the connected replica, or waiting for new events to be written. return m.logManager.currentBinlogFilepath(), nil } @@ -358,12 +353,6 @@ func (m *binlogStreamerManager) removeStreamer(streamer *binlogStreamer) { } } -// LogManager sets the LogManager this streamer manager will work with to find -// and read from binlog files. -func (m *binlogStreamerManager) LogManager(manager *LogManager) { - m.logManager = manager -} - func sendHeartbeat(conn *mysql.Conn, binlogFormat *mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) error { binlogEventMeta.Timestamp = uint32(0) // Timestamp is zero for a heartbeat event logrus.WithField("log_position", binlogEventMeta.NextLogPosition).Tracef("sending heartbeat") @@ -374,8 +363,8 @@ func sendHeartbeat(conn *mysql.Conn, binlogFormat *mysql.BinlogFormat, binlogEve // sendInitialEvents sends the initial binlog events (i.e. Rotate, FormatDescription) over a newly established binlog // streaming connection. -func sendInitialEvents(_ *sql.Context, conn *mysql.Conn, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata) error { - err := sendRotateEvent(conn, binlogFormat, binlogEventMeta) +func sendInitialEvents(_ *sql.Context, conn *mysql.Conn, binlogFilename string, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata) error { + err := sendRotateEvent(conn, binlogFilename, binlogFormat, binlogEventMeta) if err != nil { return err } @@ -388,7 +377,7 @@ func sendInitialEvents(_ *sql.Context, conn *mysql.Conn, binlogFormat *mysql.Bin return conn.FlushBuffer() } -func sendRotateEvent(conn *mysql.Conn, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata) error { +func sendRotateEvent(conn *mysql.Conn, binlogFilename string, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata) error { binlogFilePosition := uint64(0) binlogEventMeta.NextLogPosition = uint32(binlogFilePosition) diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go index 2e02d65bd75..877fb66a961 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go @@ -35,13 +35,74 @@ var doltReplicationPrimarySystemVars = map[string]string{ "gtid_mode": "ON", } +// TestBinlogPrimary_BinlogNotEnabled tests that when binary logging is NOT enabled, primary commands such as +// SHOW BINARY LOGS still work, and that attempts to start replication fail with an error. +func TestBinlogPrimary_BinlogNotEnabled(t *testing.T) { + defer teardown(t) + startSqlServersWithDoltSystemVars(t, map[string]string{ + "enforce_gtid_consistency": "ON", + "gtid_mode": "ON", + }) + setupForDoltToMySqlReplication() + + // When binary logging is NOT enabled, binary log commands such as SHOW MASTER STATUS, SHOW BINARY LOG STATUS, + // and SHOW BINARY LOGS should not error out. + requirePrimaryResults(t, "SHOW MASTER STATUS", [][]any{}) + requirePrimaryResults(t, "SHOW BINARY LOG STATUS", [][]any{}) + requirePrimaryResults(t, "SHOW BINARY LOGS", [][]any{}) + + startReplicationAndCreateTestDb(t, doltPort) + status := queryReplicaStatus(t) + require.Equal(t, "13120", status["Last_IO_Errno"]) + require.Contains(t, status["Last_IO_Error"], + "Source command COM_REGISTER_REPLICA failed: unknown error: no binlog currently being recorded") +} + +// TestBinlogPrimary_GtidModeNotEnabled asserts that when @@gtid_mode is NOT enabled, +// attempting to start replication will fail with an error visible in the replica's status. +func TestBinlogPrimary_GtidModeNotEnabled(t *testing.T) { + defer teardown(t) + startSqlServersWithDoltSystemVars(t, map[string]string{"log_bin": "1"}) + setupForDoltToMySqlReplication() + + requirePrimaryResults(t, "SHOW MASTER STATUS", [][]any{{"binlog-main.000001", "151", "", "", ""}}) + requirePrimaryResults(t, "SHOW BINARY LOG STATUS", [][]any{{"binlog-main.000001", "151", "", "", ""}}) + requirePrimaryResults(t, "SHOW BINARY LOGS", [][]any{{"binlog-main.000001", "151", "No"}}) + + startReplication(t, doltPort) + time.Sleep(500 * time.Millisecond) + status := queryReplicaStatus(t) + require.Equal(t, "13117", status["Last_IO_Errno"]) + require.Contains(t, status["Last_IO_Error"], + "The replication receiver thread cannot start because the source has GTID_MODE = OFF and this server has GTID_MODE = ON") +} + +// TestBinlogPrimary_EnforceGtidConsistencyNotEnabled asserts that when @@enforce_gtid_consistency is NOT enabled, +// attempting to start replication will fail with an error visible in the replica's status. +func TestBinlogPrimary_EnforceGtidConsistencyNotEnabled(t *testing.T) { + defer teardown(t) + startSqlServersWithDoltSystemVars(t, map[string]string{"log_bin": "1", "gtid_mode": "ON"}) + setupForDoltToMySqlReplication() + + requirePrimaryResults(t, "SHOW MASTER STATUS", [][]any{{"binlog-main.000001", "151", "", "", ""}}) + requirePrimaryResults(t, "SHOW BINARY LOG STATUS", [][]any{{"binlog-main.000001", "151", "", "", ""}}) + requirePrimaryResults(t, "SHOW BINARY LOGS", [][]any{{"binlog-main.000001", "151", "No"}}) + + startReplication(t, doltPort) + time.Sleep(500 * time.Millisecond) + status := queryReplicaStatus(t) + require.Equal(t, "13114", status["Last_IO_Errno"]) + require.Contains(t, status["Last_IO_Error"], + "@@enforce_gtid_consistency must be enabled for binlog replication") +} + // TestBinlogPrimary runs a simple sanity check that a MySQL replica can connect to a Dolt primary and receive // binlog events from a wide variety of SQL data types. func TestBinlogPrimary(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) primaryDatabase.MustExec("create table db01.t (" + "pk int primary key, " + @@ -135,7 +196,7 @@ func TestBinlogPrimary_Rotation(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) // Change the binlog rotation threshold on the primary to 10KB (instead of the default 1GB) primaryDatabase.MustExec("SET @@GLOBAL.max_binlog_size = 10240;") @@ -193,7 +254,7 @@ func TestBinlogPrimary_AutoPurging(t *testing.T) { }) // Verify the replica reports an error about the GTIDs not being available - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) status := queryReplicaStatus(t) require.Equal(t, "13114", status["Last_IO_Errno"]) require.Contains(t, status["Last_IO_Error"], @@ -252,7 +313,7 @@ func TestBinlogPrimary_ReplicaAndPrimaryRestart(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) // Change the binlog rotation threshold on the primary to 10KB (instead of the default 1GB) so // that log files will rotate more often @@ -293,7 +354,7 @@ func TestBinlogPrimary_ReplicaAndPrimaryRestart(t *testing.T) { // Restart the MySQL replica and reconnect to the Dolt primary mustRestartMySqlReplicaServer(t) - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) waitForReplicaToCatchUp(t) // Assert the executed GTID position now contains all GTIDs @@ -313,7 +374,7 @@ func TestBinlogPrimary_Heartbeats(t *testing.T) { // Start replication, with a 45s delay before any commands are sent to the primary. // This gives enough time for the first heartbeat event to be sent, before any user // initiated binlog events, so we can test that scenario. - startReplicationWithDelay(t, doltPort, 45*time.Second) + startReplicationAndCreateTestDbWithDelay(t, doltPort, 45*time.Second) // Insert a row every second, for 70s, which gives the server a chance to send two heartbeats primaryDatabase.MustExec("create table db01.heartbeatTest(pk int);") @@ -344,7 +405,7 @@ func TestBinlogPrimary_ReplicaRestart(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) // Create a table on the primary and assert that it gets replicated primaryDatabase.MustExec("create table db01.t1 (pk int primary key, c1 varchar(255));") @@ -371,7 +432,7 @@ func TestBinlogPrimary_ReplicaRestart(t *testing.T) { require.NoError(t, err) replicaDatabase = primaryDatabase primaryDatabase = prevPrimaryDatabase - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) // Create another table and assert that it gets replicated primaryDatabase.MustExec("create table db01.t2 (pk int primary key, c1 varchar(255));") @@ -389,7 +450,7 @@ func TestBinlogPrimary_PrimaryRestart(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) // Only one binary log file should be present on a fresh server requirePrimaryResults(t, "show binary logs;", [][]any{ @@ -430,33 +491,32 @@ func TestBinlogPrimary_PrimaryRestart(t *testing.T) { require.Equal(t, serverUuid+":1-3", status["Executed_Gtid_Set"]) } -// TestBinlogPrimary_OptIn asserts that binary logging does not work when the log_bin system variable is not set. -func TestBinlogPrimary_OptIn(t *testing.T) { +// TestBinlogPrimary_PrimaryRestartBeforeReplicaConnects tests that a MySQL replica can connect to a Dolt primary +// when the Dolt primary has multiple binlog files and the replica needs events from a non-current binlog file. +func TestBinlogPrimary_PrimaryRestartBeforeReplicaConnects(t *testing.T) { defer teardown(t) - startSqlServersWithDoltSystemVars(t, map[string]string{ - "enforce_gtid_consistency": "ON", - "gtid_mode": "ON", - }) + startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - // Ensure that log_bin is disabled before having the replica try to connect - requirePrimaryResults(t, "select @@log_bin;", [][]any{{"0"}}) - startReplication(t, doltPort) + // Create a test database to trigger the first GTID binlog event + primaryDatabase.MustExec("CREATE DATABASE db02;") - // MySQL doesn't return errors directly from the START REPLICA statement; instead, - // callers must check the replica status information for errors - replicaStatus := queryReplicaStatus(t) - require.Equal(t, "Source command COM_REGISTER_REPLICA failed: unknown error: no binlog currently being recorded; "+ - "make sure the server is started with @@log_bin enabled (Errno: 1105)", replicaStatus["Last_IO_Error"]) + // Restart the Dolt primary server to trigger a binlog file rotation + stopDoltSqlServer(t) + mustRestartDoltPrimaryServer(t) - // Create a database and assert that it does not get replicated - primaryDatabase.MustExec("create database newDb;") + // Start replication and verify the replica receives the CREATE DATABASE event from the first binlog file + startReplication(t, doltPort) + waitForReplicaToCatchUp(t) + requireReplicaResults(t, "SHOW DATABASES;", [][]any{ + {"db02"}, {"information_schema"}, {"mysql"}, {"performance_schema"}, {"sys"}, + }) - // Note that we use a sleep here, instead of waitForReplicaToCatchUp, since - // replication is not enabled in this test - time.Sleep(200 * time.Millisecond) - requireReplicaResults(t, "show databases;", - [][]any{{"information_schema"}, {"mysql"}, {"performance_schema"}, {"sys"}}) + // Verify that the Dolt primary server has two binary log files + requirePrimaryResults(t, "SHOW BINARY LOGS;", [][]any{ + {"binlog-main.000001", "312", "No"}, + {"binlog-main.000002", "191", "No"}, + }) } // TestBinlogPrimary_ChangeReplicationBranch asserts that the log_bin_branch system variable can @@ -467,7 +527,7 @@ func TestBinlogPrimary_ChangeReplicationBranch(t *testing.T) { mapCopy["log_bin_branch"] = "branch1" startSqlServersWithDoltSystemVars(t, mapCopy) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) // No events should be generated when we're not updating the configured replication branch primaryDatabase.MustExec("create table db01.t (pk varchar(100) primary key, c1 int, c2 year);") @@ -502,7 +562,7 @@ func TestBinlogPrimary_SimpleSchemaChangesWithAutocommit(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) // Create a table primaryDatabase.MustExec("create table db01.t1 (pk int primary key, c1 varchar(255) NOT NULL comment 'foo bar baz');") @@ -552,7 +612,7 @@ func TestBinlogPrimary_SchemaChangesWithManualCommit(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) // Create table primaryDatabase.MustExec("set @@autocommit=0;") @@ -585,7 +645,7 @@ func TestBinlogPrimary_Rollback(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) // Create a test table primaryDatabase.MustExec("set @@autocommit=0;") @@ -610,7 +670,7 @@ func TestBinlogPrimary_MultipleTablesManualCommit(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) // Insert to multiple tables in a single SQL transaction primaryDatabase.MustExec("set @@autocommit=0;") @@ -656,7 +716,7 @@ func TestBinlogPrimary_ReplicateCreateDropDatabase(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) // Test CREATE DATABASE primaryDatabase.MustExec("create database foobar1;") @@ -687,7 +747,7 @@ func TestBinlogPrimary_InsertUpdateDelete(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) primaryDatabase.MustExec("create table db01.t (pk varchar(100) primary key, c1 int, c2 year);") @@ -727,7 +787,7 @@ func TestBinlogPrimary_OnlyReplicateMainBranch(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) primaryDatabase.MustExec("create table db01.t (pk varchar(100) primary key, c1 int, c2 year);") primaryDatabase.MustExec("call dolt_commit('-Am', 'creating table t');") @@ -757,7 +817,7 @@ func TestBinlogPrimary_KeylessTables(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) primaryDatabase.MustExec("create table db01.t (c1 varchar(100), c2 int, c3 int unsigned);") primaryDatabase.MustExec("call dolt_commit('-Am', 'creating table t');") @@ -793,7 +853,7 @@ func TestBinlogPrimary_Merge(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) primaryDatabase.MustExec("create table db01.t (pk varchar(100) primary key, c1 int, c2 year);") primaryDatabase.MustExec("call dolt_commit('-Am', 'creating table t');") @@ -826,7 +886,7 @@ func TestBinlogPrimary_Cherrypick(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) primaryDatabase.MustExec("create table db01.t (pk varchar(100) primary key, c1 int);") primaryDatabase.MustExec("call dolt_commit('-Am', 'creating table t');") @@ -863,7 +923,7 @@ func TestBinlogPrimary_Revert(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) primaryDatabase.MustExec("create table db01.t (pk varchar(100) primary key, c1 int);") primaryDatabase.MustExec("call dolt_commit('-Am', 'creating table t');") @@ -900,7 +960,7 @@ func TestBinlogPrimary_Reset(t *testing.T) { defer teardown(t) startSqlServersWithDoltSystemVars(t, doltReplicationPrimarySystemVars) setupForDoltToMySqlReplication() - startReplication(t, doltPort) + startReplicationAndCreateTestDb(t, doltPort) primaryDatabase.MustExec("create table db01.t (pk varchar(100) primary key, c1 int);") primaryDatabase.MustExec("call dolt_commit('-Am', 'creating table t');") diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go index 347598d0fa1..3a6515c0fed 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go @@ -25,6 +25,7 @@ import ( "github.com/dolthub/go-mysql-server/sql" "github.com/dolthub/vitess/go/mysql" + "github.com/sirupsen/logrus" "github.com/dolthub/dolt/go/libraries/doltcore/diff" "github.com/dolthub/dolt/go/libraries/doltcore/doltdb" @@ -47,15 +48,15 @@ var BinlogBranch = "main" // Note that the initial version of binlogProducer currently delivers the generated binlog events directly to the // connected replicas, without actually writing them to a real binlog file on disk. type binlogProducer struct { - binlogEventMeta mysql.BinlogEventMetadata binlogFormat *mysql.BinlogFormat + binlogEventMeta mysql.BinlogEventMetadata mu *sync.Mutex gtidPosition *mysql.Position gtidSequence int64 - streamerManager *binlogStreamerManager + logManager *LogManager } var _ doltdb.DatabaseUpdateListener = (*binlogProducer)(nil) @@ -63,7 +64,7 @@ var _ doltdb.DatabaseUpdateListener = (*binlogProducer)(nil) // NewBinlogProducer creates and returns a new instance of BinlogProducer. Note that callers must register the // returned binlogProducer as a DatabaseUpdateListener before it will start receiving database updates and start // producing binlog events. -func NewBinlogProducer(fs filesys.Filesys, streamerManager *binlogStreamerManager) (*binlogProducer, error) { +func NewBinlogProducer(fs filesys.Filesys) (*binlogProducer, error) { binlogFormat := createBinlogFormat() binlogEventMeta, err := createBinlogEventMetadata() if err != nil { @@ -73,7 +74,6 @@ func NewBinlogProducer(fs filesys.Filesys, streamerManager *binlogStreamerManage b := &binlogProducer{ binlogEventMeta: *binlogEventMeta, binlogFormat: binlogFormat, - streamerManager: streamerManager, mu: &sync.Mutex{}, } @@ -84,15 +84,9 @@ func NewBinlogProducer(fs filesys.Filesys, streamerManager *binlogStreamerManage return b, nil } -func (b *binlogProducer) BinlogFormat() *mysql.BinlogFormat { - return b.binlogFormat -} - -// TODO: It's kinda weird for us to expose BinlogStream here... This type doesn't seem fully necessary, and -// -// stream seems like a bad name. Might be a better way to shape this in Vitess. -func (b *binlogProducer) BinlogStream() mysql.BinlogEventMetadata { - return b.binlogEventMeta +// LogManager sets the |logManager| this producer will send events to. +func (b *binlogProducer) LogManager(logManager *LogManager) { + b.logManager = logManager } // WorkingRootUpdated implements the doltdb.DatabaseUpdateListener interface. When a working root changes, @@ -149,7 +143,7 @@ func (b *binlogProducer) WorkingRootUpdated(ctx *sql.Context, databaseName strin binlogEvents = append(binlogEvents, b.newXIDEvent()) } - return b.streamerManager.logManager.WriteEvents(binlogEvents...) + return b.logManager.WriteEvents(binlogEvents...) } // DatabaseCreated implements the doltdb.DatabaseUpdateListener interface. @@ -168,7 +162,7 @@ func (b *binlogProducer) DatabaseCreated(ctx *sql.Context, databaseName string) createDatabaseStatement := fmt.Sprintf("create database `%s`;", databaseName) binlogEvents = append(binlogEvents, b.newQueryEvent(databaseName, createDatabaseStatement)) - return b.streamerManager.logManager.WriteEvents(binlogEvents...) + return b.logManager.WriteEvents(binlogEvents...) } // DatabaseDropped implements the doltdb.DatabaseUpdateListener interface. @@ -183,7 +177,7 @@ func (b *binlogProducer) DatabaseDropped(ctx *sql.Context, databaseName string) dropDatabaseStatement := fmt.Sprintf("drop database `%s`;", databaseName) binlogEvents = append(binlogEvents, b.newQueryEvent(databaseName, dropDatabaseStatement)) - return b.streamerManager.logManager.WriteEvents(binlogEvents...) + return b.logManager.WriteEvents(binlogEvents...) } // initializeGtidPosition loads the persisted GTID position from disk and initializes it @@ -241,6 +235,7 @@ func (b *binlogProducer) initializeGtidPosition(fs filesys.Filesys) error { } b.gtidSequence = int64(i + 1) + logrus.Tracef("setting @@gtid_executed to %s", b.gtidPosition.GTIDSet.String()) return sql.SystemVariables.AssignValues(map[string]any{ "gtid_executed": b.gtidPosition.GTIDSet.String()}) } diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_alltypes_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_alltypes_test.go index e486b8836eb..179aa0861db 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_alltypes_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_alltypes_test.go @@ -29,7 +29,7 @@ import ( func TestBinlogReplicationForAllTypes(t *testing.T) { defer teardown(t) startSqlServers(t) - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) // Set the session's timezone to UTC, to avoid TIMESTAMP test values changing // when they are converted to UTC for storage. diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_filters_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_filters_test.go index ce129172e5b..d4242944c78 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_filters_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_filters_test.go @@ -26,7 +26,7 @@ import ( func TestBinlogReplicationFilters_ignoreTablesOnly(t *testing.T) { defer teardown(t) startSqlServers(t) - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) // Ignore replication events for db01.t2. Also tests that the first filter setting is overwritten by // the second and that db and that db and table names are case-insensitive. @@ -77,7 +77,7 @@ func TestBinlogReplicationFilters_ignoreTablesOnly(t *testing.T) { func TestBinlogReplicationFilters_doTablesOnly(t *testing.T) { defer teardown(t) startSqlServers(t) - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) // Do replication events for db01.t1. Also tests that the first filter setting is overwritten by // the second and that db and that db and table names are case-insensitive. @@ -128,7 +128,7 @@ func TestBinlogReplicationFilters_doTablesOnly(t *testing.T) { func TestBinlogReplicationFilters_doTablesAndIgnoreTables(t *testing.T) { defer teardown(t) startSqlServers(t) - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) // Do replication events for db01.t1, and db01.t2 replicaDatabase.MustExec("CHANGE REPLICATION FILTER REPLICATE_DO_TABLE=(db01.t1, db01.t2);") diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_multidb_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_multidb_test.go index b148676dd2b..15e01efcdbf 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_multidb_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_multidb_test.go @@ -25,7 +25,7 @@ import ( func TestBinlogReplicationMultiDb(t *testing.T) { defer teardown(t) startSqlServers(t) - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) // Make changes on the primary to db01 and db02 primaryDatabase.MustExec("create database db02;") @@ -127,7 +127,7 @@ func TestBinlogReplicationMultiDb(t *testing.T) { func TestBinlogReplicationMultiDbTransactions(t *testing.T) { defer teardown(t) startSqlServers(t) - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) // Make changes on the primary to db01 and db02 primaryDatabase.MustExec("create database db02;") diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_reconnect_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_reconnect_test.go index bafd82ac48f..455477afa43 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_reconnect_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_reconnect_test.go @@ -39,7 +39,7 @@ func TestBinlogReplicationAutoReconnect(t *testing.T) { startSqlServers(t) configureToxiProxy(t) configureFastConnectionRetry(t) - startReplication(t, proxyPort) + startReplicationAndCreateTestDb(t, proxyPort) // Get the replica started up and ensure it's in sync with the primary before turning on the limit_data toxic testInitialReplicaStatus(t) diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_restart_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_restart_test.go index d56adb5241e..c52947b793e 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_restart_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_restart_test.go @@ -27,7 +27,7 @@ import ( func TestBinlogReplicationServerRestart(t *testing.T) { defer teardown(t) startSqlServers(t) - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) primaryDatabase.MustExec("create table t (pk int auto_increment primary key)") diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_test.go index 97d07cc4be1..c1c0e3f7dd6 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_replication_test.go @@ -95,7 +95,7 @@ func teardown(t *testing.T) { func TestBinlogReplicationSanityCheck(t *testing.T) { defer teardown(t) startSqlServers(t) - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) // Make changes on the primary and verify on the replica primaryDatabase.MustExec("create table t (pk int primary key)") @@ -120,7 +120,7 @@ func TestBinlogSystemUserIsLocked(t *testing.T) { require.ErrorContains(t, err, "User not found") // After starting replication, the system account is locked - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) err = db.Ping() require.Error(t, err) require.ErrorContains(t, err, "Access denied for user") @@ -132,7 +132,7 @@ func TestBinlogSystemUserIsLocked(t *testing.T) { func TestFlushLogs(t *testing.T) { defer teardown(t) startSqlServers(t) - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) // Make changes on the primary and verify on the replica primaryDatabase.MustExec("create table t (pk int primary key)") @@ -156,7 +156,7 @@ func TestFlushLogs(t *testing.T) { func TestResetReplica(t *testing.T) { defer teardown(t) startSqlServers(t) - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) // RESET REPLICA returns an error if replication is running _, err := replicaDatabase.Queryx("RESET REPLICA") @@ -195,7 +195,7 @@ func TestResetReplica(t *testing.T) { require.NoError(t, rows.Close()) // Start replication again and verify that we can still query replica status - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) replicaStatus := showReplicaStatus(t) require.Equal(t, "0", replicaStatus["Last_Errno"]) require.Equal(t, "", replicaStatus["Last_Error"]) @@ -229,7 +229,7 @@ func TestStartReplicaErrors(t *testing.T) { require.Nil(t, rows) // START REPLICA logs a warning if replication is already running - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) replicaDatabase.MustExec("START REPLICA;") assertWarning(t, replicaDatabase, 3083, "Replication thread(s) for channel '' are already running.") } @@ -272,7 +272,7 @@ func TestStopReplica(t *testing.T) { require.Equal(t, "No", status["Replica_SQL_Running"]) // START REPLICA and verify status - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) time.Sleep(100 * time.Millisecond) status = showReplicaStatus(t) require.True(t, status["Replica_IO_Running"] == "Connecting" || status["Replica_IO_Running"] == "Yes") @@ -293,7 +293,7 @@ func TestStopReplica(t *testing.T) { func TestDoltCommits(t *testing.T) { defer teardown(t) startSqlServers(t) - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) // First transaction (DDL) primaryDatabase.MustExec("create table t1 (pk int primary key);") @@ -373,7 +373,7 @@ func TestDoltCommits(t *testing.T) { func TestForeignKeyChecks(t *testing.T) { defer teardown(t) startSqlServers(t) - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) // Test that we can execute statement-based replication that requires foreign_key_checks // being turned off (referenced table doesn't exist yet). @@ -430,7 +430,7 @@ func TestForeignKeyChecks(t *testing.T) { func TestCharsetsAndCollations(t *testing.T) { defer teardown(t) startSqlServers(t) - startReplication(t, mySqlPort) + startReplicationAndCreateTestDb(t, mySqlPort) // Use non-default charset/collations to create data on the primary primaryDatabase.MustExec("CREATE TABLE t1 (pk int primary key, c1 varchar(255) COLLATE ascii_general_ci, c2 varchar(255) COLLATE utf16_general_ci);") @@ -646,16 +646,8 @@ func stopDoltSqlServer(t *testing.T) { } } -// startReplication starts up replication on the replica, connecting to |port| on the primary, -// creates the test database, db01, on the primary, and ensures it gets replicated to the replica. -func startReplication(t *testing.T, port int) { - startReplicationWithDelay(t, port, 100*time.Millisecond) -} - -// startReplication starts up replication on the replica, connecting to |port| on the primary, -// pauses for |delay| before creating the test database, db01, on the primary, and ensures it -// gets replicated to the replica. -func startReplicationWithDelay(t *testing.T, port int, delay time.Duration) { +// startReplication configures the replication source on the replica and runs the START REPLICA statement. +func startReplication(_ *testing.T, port int) { replicaDatabase.MustExec("SET @@GLOBAL.server_id=123;") replicaDatabase.MustExec( fmt.Sprintf("change replication source to SOURCE_HOST='localhost', "+ @@ -663,6 +655,19 @@ func startReplicationWithDelay(t *testing.T, port int, delay time.Duration) { "SOURCE_PORT=%v, SOURCE_AUTO_POSITION=1, SOURCE_CONNECT_RETRY=5;", port)) replicaDatabase.MustExec("start replica;") +} + +// startReplicationAndCreateTestDb starts up replication on the replica, connecting to |port| on the primary, +// creates the test database, db01, on the primary, and ensures it gets replicated to the replica. +func startReplicationAndCreateTestDb(t *testing.T, port int) { + startReplicationAndCreateTestDbWithDelay(t, port, 100*time.Millisecond) +} + +// startReplicationAndCreateTestDbWithDelay starts up replication on the replica, connecting to |port| on the primary, +// pauses for |delay| before creating the test database, db01, on the primary, and ensures it +// gets replicated to the replica. +func startReplicationAndCreateTestDbWithDelay(t *testing.T, port int, delay time.Duration) { + startReplication(t, port) time.Sleep(delay) // Look to see if the test database, db01, has been created yet. If not, create it and wait for it to diff --git a/go/libraries/doltcore/sqle/system_variables.go b/go/libraries/doltcore/sqle/system_variables.go index f49ee3e765f..5f069857f95 100644 --- a/go/libraries/doltcore/sqle/system_variables.go +++ b/go/libraries/doltcore/sqle/system_variables.go @@ -31,15 +31,6 @@ func init() { func AddDoltSystemVariables() { sql.SystemVariables.AddSystemVariables([]sql.SystemVariable{ - // TODO: Remove this and move it to GMS! - &sql.MysqlSystemVariable{ - Name: "binlog_expire_logs_seconds", - Scope: sql.GetMysqlScope(sql.SystemVariableScope_Global), - Dynamic: true, - SetVarHintApplies: false, - Type: types.NewSystemIntType("binlog_expire_logs_seconds", 0, 4294967295, false), - Default: int64(2592000), - }, &sql.MysqlSystemVariable{ Name: "log_bin_branch", Scope: sql.GetMysqlScope(sql.SystemVariableScope_Persist), From 6b9f1a7d47a665a26a79c3683bce1ce8206e77ce Mon Sep 17 00:00:00 2001 From: Jason Fulghum Date: Tue, 23 Jul 2024 13:42:29 -0700 Subject: [PATCH 10/14] Bug fixes and cleanup --- go/cmd/dolt/commands/sqlserver/server.go | 4 +- .../binlog_position_store.go | 2 +- .../binlog_primary_controller.go | 8 +- .../binlog_primary_log_manager.go | 70 +++---- .../binlog_primary_streamer.go | 179 ++++++++---------- .../binlogreplication/binlog_primary_test.go | 1 + .../sqle/binlogreplication/binlog_producer.go | 4 +- 7 files changed, 129 insertions(+), 139 deletions(-) diff --git a/go/cmd/dolt/commands/sqlserver/server.go b/go/cmd/dolt/commands/sqlserver/server.go index 63b33258aec..075f83b3fda 100644 --- a/go/cmd/dolt/commands/sqlserver/server.go +++ b/go/cmd/dolt/commands/sqlserver/server.go @@ -306,7 +306,7 @@ func ConfigureServices( primaryController := sqlEngine.GetUnderlyingEngine().Analyzer.Catalog.BinlogPrimaryController doltBinlogPrimaryController, ok := primaryController.(*binlogreplication.DoltBinlogPrimaryController) if !ok { - panic("Found unexpected type of binlog controller!") + return fmt.Errorf("unexpected type of binlog controller: %T", primaryController) } _, logBinValue, ok := sql.SystemVariables.GetGlobal("log_bin") @@ -318,7 +318,7 @@ func ConfigureServices( return fmt.Errorf("unexpected type for @@log_bin system variable: %T", logBinValue) } if logBin == 1 { - logrus.Debug("Enabling binary logging") + logrus.Info("Enabling binary logging") binlogProducer, err := binlogreplication.NewBinlogProducer(dEnv.FS) if err != nil { return err diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_position_store.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_position_store.go index f96dcc32dfa..9f856251e15 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_position_store.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_position_store.go @@ -39,7 +39,7 @@ type binlogPositionStore struct { mu sync.Mutex } -// Load loads a mysql.Position instance from the .doltcfg/binlog-position file at the root of the provider's filesystem. +// Load loads a mysql.Position instance from the .doltcfg/binlog-position file at the root of the specified |filesystem|. // This file MUST be stored at the root of the provider's filesystem, and NOT inside a nested database's .doltcfg directory, // since the binlog position contains events that cover all databases in a SQL server. The returned mysql.Position // represents the set of GTIDs that have been successfully executed and applied on this replica. Currently only the diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go index 7098defcf8b..154f88c48df 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_controller.go @@ -121,6 +121,10 @@ func (d *DoltBinlogPrimaryController) BinlogDumpGtid(ctx *sql.Context, conn *mys return err } + if replicaExecutedGtids == nil { + replicaExecutedGtids = mysql.Mysql56GTIDSet{} + } + primaryExecutedGtids := d.binlogProducer.gtidPosition.GTIDSet missingGtids := d.binlogProducer.logManager.calculateMissingGtids(replicaExecutedGtids, primaryExecutedGtids) if !missingGtids.Equal(mysql.Mysql56GTIDSet{}) { @@ -128,7 +132,9 @@ func (d *DoltBinlogPrimaryController) BinlogDumpGtid(ctx *sql.Context, conn *mys // otherwise the replica won't expose the error in replica status and will just keep trying to reconnect and // only log the error to MySQL's error log. return mysql.NewSQLError(mysql.ERMasterFatalReadingBinlog, "HY000", - "Cannot replicate because the source purged required binary logs. Replicate the missing transactions from elsewhere, or provision a new replica from backup. Consider increasing the source's binary log expiration period. The GTID set sent by the replica is '%s', and the missing transactions are '%s'.", + "Cannot replicate because the source purged required binary logs. Replicate the missing transactions "+ + "from elsewhere, or provision a new replica from backup. Consider increasing the source's binary log "+ + "expiration period. The GTID set sent by the replica is '%s', and the missing transactions are '%s'.", replicaExecutedGtids.String(), missingGtids.String()) } diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go index 3f2fc3126ea..c93d415012d 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go @@ -25,12 +25,12 @@ import ( "sync" "time" + "github.com/dolthub/go-mysql-server/sql" + gmstypes "github.com/dolthub/go-mysql-server/sql/types" "github.com/dolthub/vitess/go/mysql" "github.com/sirupsen/logrus" "github.com/dolthub/dolt/go/libraries/utils/filesys" - "github.com/dolthub/go-mysql-server/sql" - gmstypes "github.com/dolthub/go-mysql-server/sql/types" ) var binlogDirectory = filepath.Join(".dolt", "binlog") @@ -39,10 +39,10 @@ var binlogDirectory = filepath.Join(".dolt", "binlog") // MySQL binlog file and identify the file as a MySQL binlog. var binlogFileMagicNumber = []byte{0xfe, 0x62, 0x69, 0x6e} -// LogManager is responsible for the binary log files on disk, including actually writing events to the log files, +// logManager is responsible for the binary log files on disk, including actually writing events to the log files, // rotating the log files, listing the available log files, purging old log files, and keeping track of what GTIDs // are available in the log files. -type LogManager struct { +type logManager struct { binlogFormat mysql.BinlogFormat binlogEventMeta mysql.BinlogEventMetadata @@ -55,17 +55,17 @@ type LogManager struct { availableGtids mysql.GTIDSet } -// NewLogManager creates a new LogManager instance where binlog files are stored in the .dolt/binlog directory +// NewLogManager creates a new logManager instance where binlog files are stored in the .dolt/binlog directory // underneath the specified |fs| filesystem. This method also initializes the binlog logging system, including // rotating to a new log file and purging expired log files. -func NewLogManager(fs filesys.Filesys) (*LogManager, error) { +func NewLogManager(fs filesys.Filesys) (*logManager, error) { binlogFormat := createBinlogFormat() binlogEventMeta, err := createBinlogEventMetadata() if err != nil { return nil, err } - lm := &LogManager{ + lm := &logManager{ mu: &sync.Mutex{}, fs: fs, binlogFormat: *binlogFormat, @@ -115,12 +115,12 @@ func NewLogManager(fs filesys.Filesys) (*LogManager, error) { // initializeAvailableGtids sets the value of availableGtids by seeing what GTIDs have been executed (@@gtid_executed) // and subtracting any GTIDs that have been marked as purged (@@gtid_purged). -func (lm *LogManager) initializeAvailableGtids() (err error) { +func (lm *logManager) initializeAvailableGtids() (err error) { // Initialize availableGtids from @@gtid_executed – we start by assuming we have all executed GTIDs available // in the logs, and then adjust availableGtids based on which GTIDs we detect have been purged. _, gtidExecutedValue, ok := sql.SystemVariables.GetGlobal("gtid_executed") if !ok { - fmt.Errorf("unable to find system variable @@gtid_executed") + return fmt.Errorf("unable to find system variable @@gtid_executed") } if _, ok := gtidExecutedValue.(string); !ok { return fmt.Errorf("unexpected type for @@gtid_executed: %T", gtidExecutedValue) @@ -154,7 +154,7 @@ func (lm *LogManager) initializeAvailableGtids() (err error) { // When this method is called, it is expected that the new, current binlog file has already been initialized and that // adding a Rotate event to the previous log has NOT occurred yet (otherwise adding the Rotate event would update the // log file's last modified time and would not be purged). -func (lm *LogManager) purgeExpiredLogFiles() error { +func (lm *logManager) purgeExpiredLogFiles() error { expireLogsSeconds, err := lookupBinlogExpireLogsSeconds() if expireLogsSeconds == 0 { // If @@binlog_expire_logs_seconds is set to 0, then binlog files are never automatically expired @@ -168,6 +168,8 @@ func (lm *LogManager) purgeExpiredLogFiles() error { return err } + logrus.WithField("logfiles", strings.Join(filenames, ", ")).Tracef("examining available log files for expiration...") + for _, filename := range filenames { fullLogFilepath := lm.resolveLogFile(filename) stat, err := os.Stat(fullLogFilepath) @@ -175,6 +177,11 @@ func (lm *LogManager) purgeExpiredLogFiles() error { return err } + logrus.WithField("file", filename). + WithField("mod_time", stat.ModTime()). + WithField("purge_threshold", purgeThresholdTime). + Tracef("checking log file") + if stat.ModTime().Before(purgeThresholdTime) { logrus.Debugf("purging expired binlog filename: %s", filename) if err := os.Remove(fullLogFilepath); err != nil { @@ -190,7 +197,7 @@ func (lm *LogManager) purgeExpiredLogFiles() error { // preceding the found GTID, unless the found GTID is sequence number 1. If no GTIDs are found in the available // binary logs, then it is assumed that all GTIDs have been purged, so @@gtid_purged is set to the same value // held in @@gtid_executed. -func (lm *LogManager) initializePurgedGtids() error { +func (lm *logManager) initializePurgedGtids() error { filenames, err := lm.logFilesOnDiskForBranch(BinlogBranch) if err != nil { return err @@ -229,7 +236,7 @@ func (lm *LogManager) initializePurgedGtids() error { // findFirstGtidInFile opens the file with the base name |filename| in the binlog directory and // reads the events until a GTID event is found, then extracts the GTID value and returns it. -func (lm *LogManager) findFirstGtidInFile(filename string) (gtid mysql.GTID, err error) { +func (lm *logManager) findFirstGtidInFile(filename string) (gtid mysql.GTID, err error) { openFile, err := openBinlogFileForReading(lm.resolveLogFile(filename)) if err != nil { return nil, err @@ -253,7 +260,7 @@ func (lm *LogManager) findFirstGtidInFile(filename string) (gtid mysql.GTID, err // next binlog file. This is necessary so that as streamers are reading from the binlog files, they have a pointer // to follow to the next binlog file. This function MUST be called after the new log file has been initialized, // and after any expired log files have been purged. -func (lm *LogManager) addRotateEventToPreviousLogFile() error { +func (lm *logManager) addRotateEventToPreviousLogFile() error { previousLogFileName, err := lm.previousLogFile() if err != nil { return err @@ -278,7 +285,7 @@ func (lm *LogManager) addRotateEventToPreviousLogFile() error { // createNewBinlogFile creates a new binlog file and initializes it with the binlog magic number, a Format Description // event, and a Previous GTIDs event. The new binlog file is opened for append only writing. -func (lm *LogManager) createNewBinlogFile() error { +func (lm *logManager) createNewBinlogFile() error { nextLogFilename, err := lm.nextLogFile() if err != nil { return err @@ -291,7 +298,7 @@ func (lm *LogManager) createNewBinlogFile() error { // nextLogFile returns the filename of the next bin log file in the current sequence. For example, if the // current log file is "binlog-main.000008" the nextLogFile() method would return "binlog-main.000009". // Note that this function returns the file name only, not the full file path. -func (lm *LogManager) nextLogFile() (filename string, err error) { +func (lm *logManager) nextLogFile() (filename string, err error) { mostRecentLogfile, err := lm.mostRecentLogFileForBranch(BinlogBranch) if err != nil { return "", err @@ -312,7 +319,7 @@ func (lm *LogManager) nextLogFile() (filename string, err error) { // the current log file is "binlog-main.000008" the previousLogFile() method would return "binlog-main.000007". // Note that this function returns the file name only, not the full path, and doesn't guarantee that the named // file actually exists on disk or not. -func (lm *LogManager) previousLogFile() (filename string, err error) { +func (lm *logManager) previousLogFile() (filename string, err error) { branch, sequence, err := parseBinlogFilename(lm.currentBinlogFileName) if err != nil { return "", err @@ -320,7 +327,7 @@ func (lm *LogManager) previousLogFile() (filename string, err error) { return formatBinlogFilename(branch, sequence-1), nil } -func (lm *LogManager) logFilesOnDisk() (files []string, err error) { +func (lm *logManager) logFilesOnDisk() (files []string, err error) { err = lm.fs.Iter(binlogDirectory, false, func(path string, size int64, isDir bool) (stop bool) { base := filepath.Base(path) if strings.HasPrefix(base, "binlog-") { @@ -336,7 +343,7 @@ func (lm *LogManager) logFilesOnDisk() (files []string, err error) { return files, nil } -func (lm *LogManager) logFilesOnDiskForBranch(branch string) (files []string, err error) { +func (lm *logManager) logFilesOnDiskForBranch(branch string) (files []string, err error) { branch = strings.ToLower(branch) err = lm.fs.Iter(binlogDirectory, false, func(path string, size int64, isDir bool) (stop bool) { base := filepath.Base(path) @@ -353,7 +360,7 @@ func (lm *LogManager) logFilesOnDiskForBranch(branch string) (files []string, er return files, nil } -func (lm *LogManager) mostRecentLogfile() (logFile string, err error) { +func (lm *logManager) mostRecentLogfile() (logFile string, err error) { logFiles, err := lm.logFilesOnDisk() if err != nil { return "", err @@ -362,7 +369,7 @@ func (lm *LogManager) mostRecentLogfile() (logFile string, err error) { return logFiles[len(logFiles)-1], nil } -func (lm *LogManager) mostRecentLogFileForBranch(branch string) (logFile string, err error) { +func (lm *logManager) mostRecentLogFileForBranch(branch string) (logFile string, err error) { logFiles, err := lm.logFilesOnDiskForBranch(branch) if err != nil { return "", err @@ -381,7 +388,7 @@ func (lm *LogManager) mostRecentLogFileForBranch(branch string) (logFile string, // Rotation should occur when an administrator explicitly requests it with the `FLUSH LOGS` statement, during server // shutdown or restart, or when the current binary log file size exceeds the maximum size defined by the // @@max_binlog_size system variable. -func (lm *LogManager) RotateLogFile() error { +func (lm *logManager) RotateLogFile() error { nextLogFile, err := lm.nextLogFile() if err != nil { return err @@ -403,7 +410,7 @@ func (lm *LogManager) RotateLogFile() error { return lm.initializeCurrentLogFile(lm.binlogFormat, lm.binlogEventMeta) } -func (lm *LogManager) PurgeLogFiles() error { +func (lm *logManager) PurgeLogFiles() error { // TODO: implement support for purging older binlog files // This also requires setting gtid_purged // https://dev.mysql.com/doc/refman/8.0/en/replication-options-gtids.html#sysvar_gtid_purged @@ -412,7 +419,7 @@ func (lm *LogManager) PurgeLogFiles() error { // initializeCurrentLogFile creates and opens the current binlog file for append only writing, writes the first four // bytes with the binlog magic numbers, then writes a Format Description event and a Previous GTIDs event. -func (lm *LogManager) initializeCurrentLogFile(binlogFormat mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) error { +func (lm *logManager) initializeCurrentLogFile(binlogFormat mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) error { logrus.Tracef("Initializing binlog file: %s", lm.currentBinlogFilepath()) // Open the file in append mode @@ -437,10 +444,9 @@ func (lm *LogManager) initializeCurrentLogFile(binlogFormat mysql.BinlogFormat, } // Write the Previous GTIDs event - // TODO: Instead of using the @@gtid_executed system variable, LogManager could keep track of which GTIDs + // TODO: Instead of using the @@gtid_executed system variable, logManager could keep track of which GTIDs // it has seen logged and use that as the source of truth for the Previous GTIDs event. This would - // eliminate a race condition. In general, LogManager needs to track which GTIDs are represented in - // which log files better to support clients seeking to the right point in the stream. + // eliminate a race condition. _, rawValue, ok := sql.SystemVariables.GetGlobal("gtid_executed") if !ok { panic("unable to find @@gtid_executed system variable") @@ -459,7 +465,7 @@ func (lm *LogManager) initializeCurrentLogFile(binlogFormat mysql.BinlogFormat, // WriteEvents writes |binlogEvents| to the current binlog file. Access to write to the binary log is synchronized, // so that only one thread can write to the log file at a time. -func (lm *LogManager) WriteEvents(binlogEvents ...mysql.BinlogEvent) error { +func (lm *logManager) WriteEvents(binlogEvents ...mysql.BinlogEvent) error { // synchronize on WriteEvents so that only one thread is writing to the log file at a time lm.mu.Lock() defer lm.mu.Unlock() @@ -468,8 +474,8 @@ func (lm *LogManager) WriteEvents(binlogEvents ...mysql.BinlogEvent) error { } // writeEventsHelper writes |binlogEvents| to the current binlog file. This function is NOT synchronized, and is only -// intended to be used from code inside LogManager that needs to be called transitively from the WriteEvents method. -func (lm *LogManager) writeEventsHelper(binlogEvents ...mysql.BinlogEvent) error { +// intended to be used from code inside logManager that needs to be called transitively from the WriteEvents method. +func (lm *logManager) writeEventsHelper(binlogEvents ...mysql.BinlogEvent) error { maxBinlogSize, err := lookupMaxBinlogSize() if err != nil { return err @@ -522,7 +528,7 @@ func (lm *LogManager) writeEventsHelper(binlogEvents ...mysql.BinlogEvent) error // no longer available in this server's binary logs. If the returned set of GTIDs is empty, then this server can // accept the connection from the replica and start streaming it the GTIDs needed to get it in sync with this // primary. -func (lm *LogManager) calculateMissingGtids(replicaExecutedGtids mysql.GTIDSet, primaryExecutedGtids mysql.GTIDSet) mysql.GTIDSet { +func (lm *logManager) calculateMissingGtids(replicaExecutedGtids mysql.GTIDSet, primaryExecutedGtids mysql.GTIDSet) mysql.GTIDSet { // First, subtract all the GTIDs that the replica has executed from the GTIDs this server has executed, // in order to determine which GTIDs are needed to get the replica in sync with the primary. neededGtids := primaryExecutedGtids.Subtract(replicaExecutedGtids) @@ -537,11 +543,11 @@ func (lm *LogManager) calculateMissingGtids(replicaExecutedGtids mysql.GTIDSet, // resolveLogFile accepts a base filename of a binlog file and returns a fully qualified file // path to the file in the binlog storage directory. -func (lm *LogManager) resolveLogFile(filename string) string { +func (lm *logManager) resolveLogFile(filename string) string { return filepath.Join(lm.binlogDirectory, filename) } -func (lm *LogManager) currentBinlogFilepath() string { +func (lm *logManager) currentBinlogFilepath() string { return lm.resolveLogFile(lm.currentBinlogFileName) } diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go index 517a10dbcb8..93a3efb7dbe 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go @@ -33,8 +33,10 @@ import ( // It also sends heartbeat events to the replica over the same connection at // regular intervals. There is one streamer per connected replica. type binlogStreamer struct { - quitChan chan struct{} - ticker *time.Ticker + quitChan chan struct{} + ticker *time.Ticker + skippingGtids bool + currentLogFile *os.File } // NewBinlogStreamer creates a new binlogStreamer instance. @@ -51,42 +53,35 @@ func newBinlogStreamer() *binlogStreamer { // and |binlogEventMeta| records the position of the stream. This method blocks until an error // is received over the stream (e.g. the connection closing) or the streamer is closed, // through it's quit channel. -func (streamer *binlogStreamer) startStream(ctx *sql.Context, conn *mysql.Conn, executedGtids mysql.GTIDSet, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata, logfile string) error { +func (streamer *binlogStreamer) startStream(ctx *sql.Context, conn *mysql.Conn, executedGtids mysql.GTIDSet, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata, logfile string) (err error) { logrus.WithField("connection_id", conn.ConnectionID). - WithField("executed_gtids", executedGtids.String()). + WithField("executed_gtids", executedGtids). Trace("starting binlog stream") - if err := sendInitialEvents(ctx, conn, logfile, binlogFormat, binlogEventMeta); err != nil { - return err - } - - file, err := openBinlogFileForReading(logfile) + streamer.currentLogFile, err = openBinlogFileForReading(logfile) if err != nil { return err } - binlogEventMetaCopy := binlogEventMeta - binlogEventMetaCopy.NextLogPosition = 0 - - rotateEvent := mysql.NewFakeRotateEvent(*binlogFormat, *binlogEventMetaCopy, filepath.Base(logfile)) + // Send a fake rotate event to let the replica know what file and position we're at + binlogEventMeta.NextLogPosition = 4 + rotateEvent := mysql.NewFakeRotateEvent(*binlogFormat, *binlogEventMeta, filepath.Base(logfile)) if err = conn.WriteBinlogEvent(rotateEvent, false); err != nil { return err } _ = conn.FlushBuffer() - defer file.Close() + defer streamer.currentLogFile.Close() for { - logrus.Trace("binlog streamer is listening for messages") - select { case <-streamer.quitChan: - logrus.Trace("received message from streamer's quit channel") + logrus.Debug("received message from streamer's quit channel") streamer.ticker.Stop() return nil case <-streamer.ticker.C: - logrus.Trace("sending binlog heartbeat") + logrus.Debug("sending binlog heartbeat") if err := sendHeartbeat(conn, binlogFormat, *binlogEventMeta); err != nil { return err } @@ -95,65 +90,81 @@ func (streamer *binlogStreamer) startStream(ctx *sql.Context, conn *mysql.Conn, } default: - logrus.Debug("checking file for new data...") - skippingGtids := false - for { - binlogEvent, err := readBinlogEventFromFile(file) - if err == io.EOF { - logrus.Tracef("End of binlog file! Waiting for new events...") - time.Sleep(250 * time.Millisecond) - continue - } else if err != nil { - return err - } + logrus.Trace("checking binlog file for new events...") + // TODO: Being able to select on new updates from the file would be nicer + err := streamer.streamNextEvents(ctx, conn, + *binlogFormat, binlogEventMeta, filepath.Dir(logfile), executedGtids) + if err == io.EOF { + logrus.Debug("End of binlog file! Pausing for new events...") + time.Sleep(250 * time.Millisecond) + } else if err != nil { + return err + } + } + } +} - if binlogEvent.IsRotate() { - bytes := binlogEvent.Bytes() - newLogfile := string(bytes[19+8 : (len(bytes) - 4)]) - logrus.Errorf("Rotatating to new binlog file: %s", newLogfile) +// streamNextEvents streams up to 50 of the next events from the current binary logfile to a replica connected on +// |conn|. |executedGtids| indicates which GTIDs the connected replica has already executed. |logFileDir| indicates +// where the streamer can look for more binary log files with the current file rotates. If an error, including +// io.EOF, occurs while reading from the file, it is returned. +func (streamer *binlogStreamer) streamNextEvents(_ *sql.Context, conn *mysql.Conn, binlogFormat mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata, logFileDir string, executedGtids mysql.GTIDSet) error { + for range 50 { + binlogEvent, err := readBinlogEventFromFile(streamer.currentLogFile) + if err != nil { + return err + } - dir := filepath.Dir(logfile) - newLogfile = filepath.Join(dir, newLogfile) + // Update next log position in the stream so that we can send the correct position + // when a heartbeat needs to be sent by the timer signal. + binlogEventMeta.NextLogPosition += binlogEvent.Length() - if err = file.Close(); err != nil { - logrus.Errorf("unable to close previous binlog file: %s", err.Error()) - } + if binlogEvent.IsRotate() { + bytes := binlogEvent.Bytes() + newLogfile := string(bytes[19+8 : (len(bytes) - 4)]) + logrus.Debugf("Rotatating to new binlog file: %s", newLogfile) - if file, err = openBinlogFileForReading(newLogfile); err != nil { - return err - } + if err = streamer.currentLogFile.Close(); err != nil { + logrus.Errorf("unable to close previous binlog file: %s", err.Error()) + } - continue - } + newLogfile = filepath.Join(logFileDir, newLogfile) + if streamer.currentLogFile, err = openBinlogFileForReading(newLogfile); err != nil { + return err + } - if binlogEvent.IsGTID() { - gtid, _, err := binlogEvent.GTID(*binlogFormat) - if err != nil { - return err - } - - // If the replica has already executed this GTID, then skip it. - if executedGtids.ContainsGTID(gtid) { - skippingGtids = true - } else { - skippingGtids = false - } - } + // Reset log position to right after the 4 byte magic number for the file type + binlogEventMeta.NextLogPosition = 4 + continue + } - if skippingGtids { - continue - } + if binlogEvent.IsGTID() { + gtid, _, err := binlogEvent.GTID(binlogFormat) + if err != nil { + return err + } - if err := conn.WriteBinlogEvent(binlogEvent, false); err != nil { - return err - } - if err = conn.FlushBuffer(); err != nil { - return err - } + // If the replica has already executed this GTID, then skip it. + if executedGtids.ContainsGTID(gtid) { + streamer.skippingGtids = true + } else { + streamer.skippingGtids = false } - time.Sleep(500 * time.Millisecond) + } + + if streamer.skippingGtids { + continue + } + + if err := conn.WriteBinlogEvent(binlogEvent, false); err != nil { + return err + } + if err := conn.FlushBuffer(); err != nil { + return err } } + + return nil } // openBinlogFileForReading opens the specified |logfile| for reading and reads the first four bytes to make sure they @@ -183,7 +194,7 @@ type binlogStreamerManager struct { streamers []*binlogStreamer streamersMutex sync.Mutex quitChan chan struct{} - logManager *LogManager + logManager *logManager } // NewBinlogStreamerManager creates a new binlogStreamerManager instance. @@ -360,37 +371,3 @@ func sendHeartbeat(conn *mysql.Conn, binlogFormat *mysql.BinlogFormat, binlogEve binlogEvent := mysql.NewHeartbeatEvent(*binlogFormat, binlogEventMeta) return conn.WriteBinlogEvent(binlogEvent, false) } - -// sendInitialEvents sends the initial binlog events (i.e. Rotate, FormatDescription) over a newly established binlog -// streaming connection. -func sendInitialEvents(_ *sql.Context, conn *mysql.Conn, binlogFilename string, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata) error { - err := sendRotateEvent(conn, binlogFilename, binlogFormat, binlogEventMeta) - if err != nil { - return err - } - - err = sendFormatDescription(conn, binlogFormat, binlogEventMeta) - if err != nil { - return err - } - - return conn.FlushBuffer() -} - -func sendRotateEvent(conn *mysql.Conn, binlogFilename string, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata) error { - binlogFilePosition := uint64(0) - binlogEventMeta.NextLogPosition = uint32(binlogFilePosition) - - // The Rotate event sent at the start of a stream is a "virtual" event that isn't actually - // recorded to the binary log file, but sent to the replica so it knows what file is being - // read from. Because it is virtual, we do NOT update the nextLogPosition field of - // BinlogEventMetadata. - binlogEvent := mysql.NewRotateEvent(*binlogFormat, *binlogEventMeta, binlogFilePosition, binlogFilename) - return conn.WriteBinlogEvent(binlogEvent, false) -} - -func sendFormatDescription(conn *mysql.Conn, binlogFormat *mysql.BinlogFormat, binlogEventMeta *mysql.BinlogEventMetadata) error { - binlogEvent := mysql.NewFormatDescriptionEvent(*binlogFormat, *binlogEventMeta) - binlogEventMeta.NextLogPosition += binlogEvent.Length() - return conn.WriteBinlogEvent(binlogEvent, false) -} diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go index 877fb66a961..cda0a9c2e8f 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go @@ -243,6 +243,7 @@ func TestBinlogPrimary_AutoPurging(t *testing.T) { // Restart and confirm the binary log has been purged stopDoltSqlServer(t) + time.Sleep(1 * time.Second) mustRestartDoltPrimaryServer(t) requirePrimaryResults(t, "SHOW BINARY LOGS;", [][]any{ {"binlog-main.000002", "191", "No"}, diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go index 3a6515c0fed..737028c466e 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_producer.go @@ -56,7 +56,7 @@ type binlogProducer struct { gtidPosition *mysql.Position gtidSequence int64 - logManager *LogManager + logManager *logManager } var _ doltdb.DatabaseUpdateListener = (*binlogProducer)(nil) @@ -85,7 +85,7 @@ func NewBinlogProducer(fs filesys.Filesys) (*binlogProducer, error) { } // LogManager sets the |logManager| this producer will send events to. -func (b *binlogProducer) LogManager(logManager *LogManager) { +func (b *binlogProducer) LogManager(logManager *logManager) { b.logManager = logManager } From f2ee91a1c353db25d460f3187701cbcf85a241ba Mon Sep 17 00:00:00 2001 From: Jason Fulghum Date: Fri, 26 Jul 2024 13:14:21 -0700 Subject: [PATCH 11/14] First pass on reorganizing functions --- .../binlogreplication/binlog_file_utils.go | 131 ++++++++++++++++++ .../binlog_primary_log_manager.go | 112 +++++++-------- .../binlog_primary_streamer.go | 112 +-------------- 3 files changed, 190 insertions(+), 165 deletions(-) create mode 100644 go/libraries/doltcore/sqle/binlogreplication/binlog_file_utils.go diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_file_utils.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_file_utils.go new file mode 100644 index 00000000000..f9b71c14eb4 --- /dev/null +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_file_utils.go @@ -0,0 +1,131 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// 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 binlogreplication + +import ( + "encoding/binary" + "fmt" + "github.com/dolthub/vitess/go/mysql" + "os" + "strconv" + "strings" +) + +// binlogFileMagicNumber holds the four bytes that start off every +// MySQL binlog file and identify the file as a MySQL binlog. +var binlogFileMagicNumber = []byte{0xfe, 0x62, 0x69, 0x6e} + +// fileExists returns true if the specified |filepath| exists on disk and is not a directory, +// otherwise returns false. |filepath| is a fully specified path to a file on disk. +func fileExists(filepath string) bool { + info, err := os.Stat(filepath) + if os.IsNotExist(err) { + return false + } + return !info.IsDir() +} + +// openBinlogFileForReading opens the specified |logfile| for reading and reads the first four bytes to make sure they +// are the expected binlog file magic numbers. If any problems are encountered opening the file or reading the first +// four bytes, an error is returned. +func openBinlogFileForReading(logfile string) (*os.File, error) { + file, err := os.Open(logfile) + if err != nil { + return nil, err + } + buffer := make([]byte, len(binlogFileMagicNumber)) + bytesRead, err := file.Read(buffer) + if err != nil { + return nil, err + } + if bytesRead != len(binlogFileMagicNumber) || string(buffer) != string(binlogFileMagicNumber) { + return nil, fmt.Errorf("invalid magic number in binlog file!") + } + + return file, nil +} + +// readBinlogEventFromFile reads the next binlog event from the specified, open |file| and +// returns it. If no more events are available in the file, then io.EOF is returned. +func readBinlogEventFromFile(file *os.File) (mysql.BinlogEvent, error) { + headerBuffer := make([]byte, 4+1+4+4+4+2) + _, err := file.Read(headerBuffer) + if err != nil { + return nil, err + } + + // Event Header: + //timestamp := headerBuffer[0:4] + //eventType := headerBuffer[4] + //serverId := binary.LittleEndian.Uint32(headerBuffer[5:5+4]) + eventSize := binary.LittleEndian.Uint32(headerBuffer[9 : 9+4]) + + payloadBuffer := make([]byte, eventSize-uint32(len(headerBuffer))) + _, err = file.Read(payloadBuffer) + if err != nil { + return nil, err + } + + return mysql.NewMysql56BinlogEvent(append(headerBuffer, payloadBuffer...)), nil +} + +// readFirstGtidEventFromFile reads events from |file| until a GTID event is found, then +// returns that GTID event. If |file| has been read completely and no GTID events were found, +// then io.EOF is returned. +func readFirstGtidEventFromFile(file *os.File) (mysql.BinlogEvent, error) { + for { + binlogEvent, err := readBinlogEventFromFile(file) + if err != nil { + return nil, err + } + + if binlogEvent.IsGTID() { + return binlogEvent, nil + } + } +} + +// formatBinlogFilename formats a binlog filename using the specified |branch| and |sequence| number. The +// returned filename will be of the form "binlog-main.000001". +func formatBinlogFilename(branch string, sequence int) string { + return fmt.Sprintf("binlog-%s.%06d", branch, sequence) +} + +// parseBinlogFilename parses a binlog filename, of the form "binlog-main.000001", into its branch and +// sequence number. +func parseBinlogFilename(filename string) (branch string, sequence int, err error) { + if !strings.HasPrefix(filename, "binlog-") { + return "", 0, fmt.Errorf("invalid binlog filename: %s; must start with 'binlog-'", filename) + } + + filename = strings.TrimPrefix(filename, "binlog-") + + splits := strings.Split(filename, ".") + if len(splits) != 2 { + return "", 0, fmt.Errorf( + "unable to parse binlog filename: %s; expected format 'binlog-branch.sequence'", filename) + } + + branch = splits[0] + sequenceString := splits[1] + + sequence, err = strconv.Atoi(sequenceString) + if err != nil { + return "", 0, fmt.Errorf( + "unable to parse binlog sequence number: %s; %s", sequenceString, err.Error()) + } + + return branch, sequence, nil +} diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go index c93d415012d..4f592840f89 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_log_manager.go @@ -20,7 +20,6 @@ import ( "io" "os" "path/filepath" - "strconv" "strings" "sync" "time" @@ -35,10 +34,6 @@ import ( var binlogDirectory = filepath.Join(".dolt", "binlog") -// binlogFileMagicNumber holds the four bytes that start off every -// MySQL binlog file and identify the file as a MySQL binlog. -var binlogFileMagicNumber = []byte{0xfe, 0x62, 0x69, 0x6e} - // logManager is responsible for the binary log files on disk, including actually writing events to the log files, // rotating the log files, listing the available log files, purging old log files, and keeping track of what GTIDs // are available in the log files. @@ -234,6 +229,60 @@ func (lm *logManager) initializePurgedGtids() error { return sql.SystemVariables.SetGlobal("gtid_purged", gtidExecutedValue) } +// findLogFileForPosition searches through the available binlog files on disk for the first log file that +// contains GTIDs that are NOT present in |executedGtids|. This is determined by reading the first GTID event +// from each log file and selecting the previous file when the first GTID not in |executedGtids| is found. If +// the first GTID event in all available logs files is in |executedGtids|, then the current log file is returned. +func (lm *logManager) findLogFileForPosition(executedGtids mysql.GTIDSet) (string, error) { + files, err := lm.logFilesOnDiskForBranch(BinlogBranch) + if err != nil { + return "", err + } + + for i, f := range files { + binlogFilePath := filepath.Join(lm.binlogDirectory, f) + file, err := openBinlogFileForReading(binlogFilePath) + if err != nil { + return "", err + } + + binlogEvent, err := readFirstGtidEventFromFile(file) + if fileCloseErr := file.Close(); fileCloseErr != nil { + logrus.Errorf("unable to cleanly close binlog file %s: %s", f, err.Error()) + } + if err == io.EOF { + continue + } else if err != nil { + return "", err + } + + if binlogEvent.IsGTID() { + gtid, _, err := binlogEvent.GTID(lm.binlogFormat) + if err != nil { + return "", err + } + // If the first GTID in this file is contained in the set of GTIDs that the replica + // has already executed, then move on to check the next file. + if executedGtids.ContainsGTID(gtid) { + continue + } + + // If we found an unexecuted GTID in the first binlog file, return the first file, + // otherwise return the previous file + if i == 0 { + return binlogFilePath, nil + } else { + return filepath.Join(lm.binlogDirectory, files[i-1]), nil + } + } + } + + // If we don't find any GTIDs that are missing from |executedGtids|, then return the current + // log file so the streamer can reply events from it, potentially finding GTIDs later in the + // file that need to be sent to the connected replica, or waiting for new events to be written. + return lm.currentBinlogFilepath(), nil +} + // findFirstGtidInFile opens the file with the base name |filename| in the binlog directory and // reads the events until a GTID event is found, then extracts the GTID value and returns it. func (lm *logManager) findFirstGtidInFile(filename string) (gtid mysql.GTID, err error) { @@ -327,6 +376,8 @@ func (lm *logManager) previousLogFile() (filename string, err error) { return formatBinlogFilename(branch, sequence-1), nil } +// TODO: consider moving these to the helper function file + func (lm *logManager) logFilesOnDisk() (files []string, err error) { err = lm.fs.Iter(binlogDirectory, false, func(path string, size int64, isDir bool) (stop bool) { base := filepath.Base(path) @@ -410,13 +461,6 @@ func (lm *logManager) RotateLogFile() error { return lm.initializeCurrentLogFile(lm.binlogFormat, lm.binlogEventMeta) } -func (lm *logManager) PurgeLogFiles() error { - // TODO: implement support for purging older binlog files - // This also requires setting gtid_purged - // https://dev.mysql.com/doc/refman/8.0/en/replication-options-gtids.html#sysvar_gtid_purged - return nil -} - // initializeCurrentLogFile creates and opens the current binlog file for append only writing, writes the first four // bytes with the binlog magic numbers, then writes a Format Description event and a Previous GTIDs event. func (lm *logManager) initializeCurrentLogFile(binlogFormat mysql.BinlogFormat, binlogEventMeta mysql.BinlogEventMetadata) error { @@ -547,19 +591,12 @@ func (lm *logManager) resolveLogFile(filename string) string { return filepath.Join(lm.binlogDirectory, filename) } +// TODO: Consider moving lookup SystemVar helper functions into a separate file + func (lm *logManager) currentBinlogFilepath() string { return lm.resolveLogFile(lm.currentBinlogFileName) } -// fileExists returns true if the specified |filename| exists on disk and is not a directory, otherwise returns false. -func fileExists(filename string) bool { - info, err := os.Stat(filename) - if os.IsNotExist(err) { - return false - } - return !info.IsDir() -} - // lookupMaxBinlogSize looks up the value of the @@max_binlog_size system variable and returns it, along with any // errors encountered while looking it up. func lookupMaxBinlogSize() (int, error) { @@ -590,36 +627,3 @@ func lookupBinlogExpireLogsSeconds() (int, error) { return int(int32Value.(int32)), nil } - -// formatBinlogFilename formats a binlog filename using the specified |branch| and |sequence| number. The -// returned filename will be of the form "binlog-main.000001". -func formatBinlogFilename(branch string, sequence int) string { - return fmt.Sprintf("binlog-%s.%06d", branch, sequence) -} - -// parseBinlogFilename parses a binlog filename, of the form "binlog-main.000001", into its branch and -// sequence number. -func parseBinlogFilename(filename string) (branch string, sequence int, err error) { - if !strings.HasPrefix(filename, "binlog-") { - return "", 0, fmt.Errorf("invalid binlog filename: %s; must start with 'binlog-'", filename) - } - - filename = strings.TrimPrefix(filename, "binlog-") - - splits := strings.Split(filename, ".") - if len(splits) != 2 { - return "", 0, fmt.Errorf( - "unable to parse binlog filename: %s; expected format 'binlog-branch.sequence'", filename) - } - - branch = splits[0] - sequenceString := splits[1] - - sequence, err = strconv.Atoi(sequenceString) - if err != nil { - return "", 0, fmt.Errorf( - "unable to parse binlog sequence number: %s; %s", sequenceString, err.Error()) - } - - return branch, sequence, nil -} diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go index 93a3efb7dbe..ed57062fdd2 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_streamer.go @@ -15,7 +15,6 @@ package binlogreplication import ( - "encoding/binary" "fmt" "io" "os" @@ -167,26 +166,6 @@ func (streamer *binlogStreamer) streamNextEvents(_ *sql.Context, conn *mysql.Con return nil } -// openBinlogFileForReading opens the specified |logfile| for reading and reads the first four bytes to make sure they -// are the expected binlog file magic numbers. If any problems are encountered opening the file or reading the first -// four bytes, an error is returned. -func openBinlogFileForReading(logfile string) (*os.File, error) { - file, err := os.Open(logfile) - if err != nil { - return nil, err - } - buffer := make([]byte, len(binlogFileMagicNumber)) - bytesRead, err := file.Read(buffer) - if err != nil { - return nil, err - } - if bytesRead != len(binlogFileMagicNumber) || string(buffer) != string(binlogFileMagicNumber) { - return nil, fmt.Errorf("invalid magic number in binlog file!") - } - - return file, nil -} - // binlogStreamerManager manages a collection of binlogStreamers, one for reach connected replica, // and implements the doltdb.DatabaseUpdateListener interface to receive notifications of database // changes that need to be turned into binlog events and then sent to connected replicas. @@ -242,7 +221,7 @@ func (m *binlogStreamerManager) StartStream(ctx *sql.Context, conn *mysql.Conn, m.addStreamer(streamer) defer m.removeStreamer(streamer) - file, err := m.findLogFileForPosition(executedGtids) + file, err := m.logManager.findLogFileForPosition(executedGtids) if err != nil { return err } @@ -250,95 +229,6 @@ func (m *binlogStreamerManager) StartStream(ctx *sql.Context, conn *mysql.Conn, return streamer.startStream(ctx, conn, executedGtids, binlogFormat, &binlogEventMeta, file) } -// findLogFileForPosition searches through the available binlog files on disk for the first log file that -// contains GTIDs that are NOT present in |executedGtids|. This is determined by reading the first GTID event -// from each log file and selecting the previous file when the first GTID not in |executedGtids| is found. If -// the first GTID event in all available logs files is in |executedGtids|, then the current log file is returned. -func (m *binlogStreamerManager) findLogFileForPosition(executedGtids mysql.GTIDSet) (string, error) { - files, err := m.logManager.logFilesOnDiskForBranch(BinlogBranch) - if err != nil { - return "", err - } - - for i, f := range files { - binlogFilePath := filepath.Join(m.logManager.binlogDirectory, f) - file, err := openBinlogFileForReading(binlogFilePath) - if err != nil { - return "", err - } - - binlogEvent, err := readFirstGtidEventFromFile(file) - if fileCloseErr := file.Close(); fileCloseErr != nil { - logrus.Errorf("unable to cleanly close binlog file %s: %s", f, err.Error()) - } - if err == io.EOF { - continue - } else if err != nil { - return "", err - } - - if binlogEvent.IsGTID() { - gtid, _, err := binlogEvent.GTID(m.logManager.binlogFormat) - if err != nil { - return "", err - } - // If the first GTID in this file is contained in the set of GTIDs that the replica - // has already executed, then move on to check the next file. - if executedGtids.ContainsGTID(gtid) { - continue - } - - // If we found an unexecuted GTID in the first binlog file, return the first file, - // otherwise return the previous file - if i == 0 { - return binlogFilePath, nil - } else { - return filepath.Join(m.logManager.binlogDirectory, files[i-1]), nil - } - } - } - - // If we don't find any GTIDs that are missing from |executedGtids|, then return the current - // log file so the streamer can reply events from it, potentially finding GTIDs later in the - // file that need to be sent to the connected replica, or waiting for new events to be written. - return m.logManager.currentBinlogFilepath(), nil -} - -func readBinlogEventFromFile(file *os.File) (mysql.BinlogEvent, error) { - headerBuffer := make([]byte, 4+1+4+4+4+2) - _, err := file.Read(headerBuffer) - if err != nil { - return nil, err - } - - // Event Header: - //timestamp := headerBuffer[0:4] - //eventType := headerBuffer[4] - //serverId := binary.LittleEndian.Uint32(headerBuffer[5:5+4]) - eventSize := binary.LittleEndian.Uint32(headerBuffer[9 : 9+4]) - - payloadBuffer := make([]byte, eventSize-uint32(len(headerBuffer))) - _, err = file.Read(payloadBuffer) - if err != nil { - return nil, err - } - - return mysql.NewMysql56BinlogEvent(append(headerBuffer, payloadBuffer...)), nil -} - -func readFirstGtidEventFromFile(file *os.File) (mysql.BinlogEvent, error) { - for { - binlogEvent, err := readBinlogEventFromFile(file) - if err != nil { - return nil, err - } - - if binlogEvent.IsGTID() { - return binlogEvent, nil - } - } -} - // addStreamer adds |streamer| to the slice of streamers managed by this binlogStreamerManager. func (m *binlogStreamerManager) addStreamer(streamer *binlogStreamer) { m.streamersMutex.Lock() From 877942d3a008f1bdfe4aed2672e77c0045c9d285 Mon Sep 17 00:00:00 2001 From: fulghum Date: Sat, 27 Jul 2024 04:41:09 +0000 Subject: [PATCH 12/14] [ga-format-pr] Run go/utils/repofmt/format_repo.sh and go/Godeps/update.sh --- .../doltcore/sqle/binlogreplication/binlog_file_utils.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_file_utils.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_file_utils.go index f9b71c14eb4..84f904ef575 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_file_utils.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_file_utils.go @@ -17,10 +17,11 @@ package binlogreplication import ( "encoding/binary" "fmt" - "github.com/dolthub/vitess/go/mysql" "os" "strconv" "strings" + + "github.com/dolthub/vitess/go/mysql" ) // binlogFileMagicNumber holds the four bytes that start off every From c563747082003222f501de5b819a23d1ac110053 Mon Sep 17 00:00:00 2001 From: Maximilian Hoffman Date: Mon, 29 Jul 2024 12:51:45 -0700 Subject: [PATCH 13/14] [kvexec] Fix panic in non-covering strict lookup (#8166) * [kvexec] Fix panic in non-covering strict lookup * [ga-format-pr] Run go/utils/repofmt/format_repo.sh and go/Godeps/update.sh --------- Co-authored-by: max-hoffman --- go/cmd/dolt/commands/engine/sqlengine.go | 4 +-- go/go.mod | 4 +-- go/go.sum | 8 +++--- .../doltcore/sqle/enginetest/dolt_harness.go | 4 +-- .../doltcore/sqle/index/index_reader.go | 2 +- .../doltcore/sqle/index/secondary_iter.go | 25 ++++++++++--------- .../sqle/{rowexec => kvexec}/builder.go | 2 +- .../sqle/{rowexec => kvexec}/lookup_join.go | 2 +- .../{rowexec => kvexec}/lookup_join_test.go | 2 +- 9 files changed, 27 insertions(+), 26 deletions(-) rename go/libraries/doltcore/sqle/{rowexec => kvexec}/builder.go (99%) rename go/libraries/doltcore/sqle/{rowexec => kvexec}/lookup_join.go (99%) rename go/libraries/doltcore/sqle/{rowexec => kvexec}/lookup_join_test.go (99%) diff --git a/go/cmd/dolt/commands/engine/sqlengine.go b/go/cmd/dolt/commands/engine/sqlengine.go index 645b6c74a55..395db2b949d 100644 --- a/go/cmd/dolt/commands/engine/sqlengine.go +++ b/go/cmd/dolt/commands/engine/sqlengine.go @@ -42,8 +42,8 @@ import ( dblr "github.com/dolthub/dolt/go/libraries/doltcore/sqle/binlogreplication" "github.com/dolthub/dolt/go/libraries/doltcore/sqle/cluster" "github.com/dolthub/dolt/go/libraries/doltcore/sqle/dsess" + "github.com/dolthub/dolt/go/libraries/doltcore/sqle/kvexec" "github.com/dolthub/dolt/go/libraries/doltcore/sqle/mysql_file_handler" - drowexec "github.com/dolthub/dolt/go/libraries/doltcore/sqle/rowexec" "github.com/dolthub/dolt/go/libraries/doltcore/sqle/statsnoms" "github.com/dolthub/dolt/go/libraries/doltcore/sqle/statspro" "github.com/dolthub/dolt/go/libraries/doltcore/sqle/writer" @@ -195,7 +195,7 @@ func NewSqlEngine( statsPro := statspro.NewProvider(pro, statsnoms.NewNomsStatsFactory(mrEnv.RemoteDialProvider())) engine.Analyzer.Catalog.StatsProvider = statsPro - engine.Analyzer.ExecBuilder = rowexec.NewOverrideBuilder(drowexec.Builder{}) + engine.Analyzer.ExecBuilder = rowexec.NewOverrideBuilder(kvexec.Builder{}) sessFactory := doltSessionFactory(pro, statsPro, mrEnv.Config(), bcController, config.Autocommit) sqlEngine.provider = pro sqlEngine.contextFactory = sqlContextFactory() diff --git a/go/go.mod b/go/go.mod index eb42f75ddef..5cab0f75442 100644 --- a/go/go.mod +++ b/go/go.mod @@ -15,7 +15,7 @@ require ( github.com/dolthub/fslock v0.0.3 github.com/dolthub/ishell v0.0.0-20240701202509-2b217167d718 github.com/dolthub/sqllogictest/go v0.0.0-20201107003712-816f3ae12d81 - github.com/dolthub/vitess v0.0.0-20240711213744-4232e1c4edae + github.com/dolthub/vitess v0.0.0-20240723185540-9f8cd7c03829 github.com/dustin/go-humanize v1.0.1 github.com/fatih/color v1.13.0 github.com/flynn-archive/go-shlex v0.0.0-20150515145356-3f9db97f8568 @@ -57,7 +57,7 @@ require ( github.com/cespare/xxhash/v2 v2.2.0 github.com/creasty/defaults v1.6.0 github.com/dolthub/flatbuffers/v23 v23.3.3-dh.2 - github.com/dolthub/go-mysql-server v0.18.2-0.20240726192758-72470d302577 + github.com/dolthub/go-mysql-server v0.18.2-0.20240729182701-b4cec29302d1 github.com/dolthub/gozstd v0.0.0-20240423170813-23a2903bca63 github.com/dolthub/swiss v0.1.0 github.com/goccy/go-json v0.10.2 diff --git a/go/go.sum b/go/go.sum index e12599f691c..bd884dc97f0 100644 --- a/go/go.sum +++ b/go/go.sum @@ -183,8 +183,8 @@ github.com/dolthub/fslock v0.0.3 h1:iLMpUIvJKMKm92+N1fmHVdxJP5NdyDK5bK7z7Ba2s2U= github.com/dolthub/fslock v0.0.3/go.mod h1:QWql+P17oAAMLnL4HGB5tiovtDuAjdDTPbuqx7bYfa0= 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.20240726192758-72470d302577 h1:k1zmiHU6vXi9xn5EtSRrAuys4YvRl0bOUCce4afEbsU= -github.com/dolthub/go-mysql-server v0.18.2-0.20240726192758-72470d302577/go.mod h1:P6bG0p+3mH4LS4DLo3BySh10ZJTDqgWyfWBu8gGE3eU= +github.com/dolthub/go-mysql-server v0.18.2-0.20240729182701-b4cec29302d1 h1:ydrib93syT0bpnYjgeKuLkn0R0B++lPNWMN9knWGMJY= +github.com/dolthub/go-mysql-server v0.18.2-0.20240729182701-b4cec29302d1/go.mod h1:ctH+JB+7+NeXPOfcujw/dzzPPgT9Byb/nOE44RRAztg= github.com/dolthub/gozstd v0.0.0-20240423170813-23a2903bca63 h1:OAsXLAPL4du6tfbBgK0xXHZkOlos63RdKYS3Sgw/dfI= github.com/dolthub/gozstd v0.0.0-20240423170813-23a2903bca63/go.mod h1:lV7lUeuDhH5thVGDCKXbatwKy2KW80L4rMT46n+Y2/Q= github.com/dolthub/ishell v0.0.0-20240701202509-2b217167d718 h1:lT7hE5k+0nkBdj/1UOSFwjWpNxf+LCApbRHgnCA17XE= @@ -197,8 +197,8 @@ github.com/dolthub/sqllogictest/go v0.0.0-20201107003712-816f3ae12d81 h1:7/v8q9X github.com/dolthub/sqllogictest/go v0.0.0-20201107003712-816f3ae12d81/go.mod h1:siLfyv2c92W1eN/R4QqG/+RjjX5W2+gCTRjZxBjI3TY= github.com/dolthub/swiss v0.1.0 h1:EaGQct3AqeP/MjASHLiH6i4TAmgbG/c4rA6a1bzCOPc= github.com/dolthub/swiss v0.1.0/go.mod h1:BeucyB08Vb1G9tumVN3Vp/pyY4AMUnr9p7Rz7wJ7kAQ= -github.com/dolthub/vitess v0.0.0-20240711213744-4232e1c4edae h1:SAYP6+hzkoYLWVzQTwQO0QbhVOwMsgy0dpRcL/QriS8= -github.com/dolthub/vitess v0.0.0-20240711213744-4232e1c4edae/go.mod h1:uBvlRluuL+SbEWTCZ68o0xvsdYZER3CEG/35INdzfJM= +github.com/dolthub/vitess v0.0.0-20240723185540-9f8cd7c03829 h1:/SEo3jm8zy0EVKIUPRcfagB9drQYI4KwQlSU/DNVRAQ= +github.com/dolthub/vitess v0.0.0-20240723185540-9f8cd7c03829/go.mod h1:uBvlRluuL+SbEWTCZ68o0xvsdYZER3CEG/35INdzfJM= github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= diff --git a/go/libraries/doltcore/sqle/enginetest/dolt_harness.go b/go/libraries/doltcore/sqle/enginetest/dolt_harness.go index dd5660f1397..5b75ed05a43 100644 --- a/go/libraries/doltcore/sqle/enginetest/dolt_harness.go +++ b/go/libraries/doltcore/sqle/enginetest/dolt_harness.go @@ -35,7 +35,7 @@ import ( "github.com/dolthub/dolt/go/libraries/doltcore/env" "github.com/dolthub/dolt/go/libraries/doltcore/sqle" "github.com/dolthub/dolt/go/libraries/doltcore/sqle/dsess" - drowexec "github.com/dolthub/dolt/go/libraries/doltcore/sqle/rowexec" + "github.com/dolthub/dolt/go/libraries/doltcore/sqle/kvexec" "github.com/dolthub/dolt/go/libraries/doltcore/sqle/statsnoms" "github.com/dolthub/dolt/go/libraries/doltcore/sqle/statspro" "github.com/dolthub/dolt/go/libraries/doltcore/sqle/writer" @@ -252,7 +252,7 @@ func (d *DoltHarness) NewEngine(t *testing.T) (enginetest.QueryEngine, error) { if err != nil { return nil, err } - e.Analyzer.ExecBuilder = rowexec.NewOverrideBuilder(drowexec.Builder{}) + e.Analyzer.ExecBuilder = rowexec.NewOverrideBuilder(kvexec.Builder{}) d.engine = e ctx := enginetest.NewContext(d) diff --git a/go/libraries/doltcore/sqle/index/index_reader.go b/go/libraries/doltcore/sqle/index/index_reader.go index 0e55f3d6b66..93c1d8e344d 100644 --- a/go/libraries/doltcore/sqle/index/index_reader.go +++ b/go/libraries/doltcore/sqle/index/index_reader.go @@ -589,7 +589,7 @@ func (ib *nonCoveringIndexImplBuilder) NewPartitionRowIter(ctx *sql.Context, par func (ib *nonCoveringIndexImplBuilder) NewSecondaryIter(strict bool, cnt int, nullSafe []bool) SecondaryLookupIterGen { if strict { - return &nonCovStrictSecondaryLookupGen{pri: ib.pri, sec: ib.sec, pkMap: ib.pkMap, pkBld: ib.pkBld, sch: ib.idx.tableSch} + return &nonCovStrictSecondaryLookupGen{pri: ib.pri, sec: ib.sec, pkMap: ib.pkMap, pkBld: ib.pkBld, sch: ib.idx.tableSch, prefixDesc: ib.secKd.PrefixDesc(cnt)} } else { return &nonCovLaxSecondaryLookupGen{pri: ib.pri, sec: ib.sec, pkMap: ib.pkMap, pkBld: ib.pkBld, sch: ib.idx.tableSch, prefixDesc: ib.secKd.PrefixDesc(cnt), nullSafe: nullSafe} } diff --git a/go/libraries/doltcore/sqle/index/secondary_iter.go b/go/libraries/doltcore/sqle/index/secondary_iter.go index 16fd6d23ef7..0a26aa5eb9f 100644 --- a/go/libraries/doltcore/sqle/index/secondary_iter.go +++ b/go/libraries/doltcore/sqle/index/secondary_iter.go @@ -94,12 +94,12 @@ func (c *covStrictSecondaryLookupGen) New(ctx context.Context, k val.Tuple) (pro } type nonCovStrictSecondaryLookupGen struct { - pri prolly.Map - sec prolly.Map - sch schema.Schema - k, v val.Tuple - pkMap val.OrdinalMapping - pkBld *val.TupleBuilder + pri prolly.Map + sec prolly.Map + prefixDesc val.TupleDesc + sch schema.Schema + pkMap val.OrdinalMapping + pkBld *val.TupleBuilder } func (c *nonCovStrictSecondaryLookupGen) InputKeyDesc() val.TupleDesc { @@ -124,12 +124,15 @@ func (c *nonCovStrictSecondaryLookupGen) NodeStore() tree.NodeStore { func (c *nonCovStrictSecondaryLookupGen) New(ctx context.Context, k val.Tuple) (prolly.MapIter, error) { var idxKey val.Tuple - if err := c.sec.Get(ctx, k, func(key val.Tuple, value val.Tuple) error { + if err := c.sec.GetPrefix(ctx, k, c.prefixDesc, func(key val.Tuple, value val.Tuple) error { idxKey = key return nil }); err != nil { return nil, err } + if idxKey == nil { + return &strictLookupIter{}, nil + } for to := range c.pkMap { from := c.pkMap.MapOrdinal(to) c.pkBld.PutRaw(to, idxKey.GetField(from)) @@ -137,7 +140,7 @@ func (c *nonCovStrictSecondaryLookupGen) New(ctx context.Context, k val.Tuple) ( pk := c.pkBld.Build(sharePool) iter := &strictLookupIter{k: pk} - if err := c.pri.Get(ctx, c.k, func(key val.Tuple, value val.Tuple) error { + if err := c.pri.Get(ctx, pk, func(key val.Tuple, value val.Tuple) error { iter.v = value return nil }); err != nil { @@ -150,7 +153,6 @@ type covLaxSecondaryLookupGen struct { m prolly.Map index *doltIndex prefixDesc val.TupleDesc - k, v val.Tuple nullSafe []bool } @@ -182,7 +184,7 @@ func (c *covLaxSecondaryLookupGen) New(ctx context.Context, k val.Tuple) (prolly } var err error - if c.prefixDesc.Count() == c.m.KeyDesc().Count() { + if c.prefixDesc.Count() >= c.m.KeyDesc().Count()-1 { // key range optimization only works for full length key start := k stop, ok := prolly.IncrementTuple(start, c.prefixDesc.Count()-1, c.prefixDesc, c.m.Pool()) @@ -204,7 +206,6 @@ type nonCovLaxSecondaryLookupGen struct { pri prolly.Map sec prolly.Map sch schema.Schema - k, v val.Tuple prefixDesc val.TupleDesc pkMap val.OrdinalMapping pkBld *val.TupleBuilder @@ -297,7 +298,7 @@ func (c *keylessSecondaryLookupGen) New(ctx context.Context, k val.Tuple) (proll var err error if c.prefixDesc.Count() == c.sec.KeyDesc().Count() { // key range optimization only works if full key - // keyless indexs should include all rows + // keyless indexes should include all rows start := k stop, ok := prolly.IncrementTuple(start, c.prefixDesc.Count()-1, c.prefixDesc, c.sec.Pool()) if ok { diff --git a/go/libraries/doltcore/sqle/rowexec/builder.go b/go/libraries/doltcore/sqle/kvexec/builder.go similarity index 99% rename from go/libraries/doltcore/sqle/rowexec/builder.go rename to go/libraries/doltcore/sqle/kvexec/builder.go index 0a3571fe53b..1283321d308 100644 --- a/go/libraries/doltcore/sqle/rowexec/builder.go +++ b/go/libraries/doltcore/sqle/kvexec/builder.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package rowexec +package kvexec import ( "context" diff --git a/go/libraries/doltcore/sqle/rowexec/lookup_join.go b/go/libraries/doltcore/sqle/kvexec/lookup_join.go similarity index 99% rename from go/libraries/doltcore/sqle/rowexec/lookup_join.go rename to go/libraries/doltcore/sqle/kvexec/lookup_join.go index 80621a42f1a..cbccec15b91 100644 --- a/go/libraries/doltcore/sqle/rowexec/lookup_join.go +++ b/go/libraries/doltcore/sqle/kvexec/lookup_join.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package rowexec +package kvexec import ( "context" diff --git a/go/libraries/doltcore/sqle/rowexec/lookup_join_test.go b/go/libraries/doltcore/sqle/kvexec/lookup_join_test.go similarity index 99% rename from go/libraries/doltcore/sqle/rowexec/lookup_join_test.go rename to go/libraries/doltcore/sqle/kvexec/lookup_join_test.go index f5a3a1c8d8c..ac468f7d147 100644 --- a/go/libraries/doltcore/sqle/rowexec/lookup_join_test.go +++ b/go/libraries/doltcore/sqle/kvexec/lookup_join_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package rowexec +package kvexec import ( "context" From 16d89aec15a25ddfd77b300711ab5b25c6faed1d Mon Sep 17 00:00:00 2001 From: Jason Fulghum Date: Mon, 29 Jul 2024 15:02:39 -0700 Subject: [PATCH 14/14] Adding validation that @@log_bin_branch isn't set to a branch name containing a slash --- go/cmd/dolt/commands/sqlserver/server.go | 37 ++++++++++++------- .../binlogreplication/binlog_primary_test.go | 14 +++++++ 2 files changed, 38 insertions(+), 13 deletions(-) diff --git a/go/cmd/dolt/commands/sqlserver/server.go b/go/cmd/dolt/commands/sqlserver/server.go index d1758940e4d..fc0f723ae6c 100644 --- a/go/cmd/dolt/commands/sqlserver/server.go +++ b/go/cmd/dolt/commands/sqlserver/server.go @@ -321,8 +321,31 @@ func ConfigureServices( if !ok { return fmt.Errorf("unexpected type for @@log_bin system variable: %T", logBinValue) } + + _, logBinBranchValue, ok := sql.SystemVariables.GetGlobal("log_bin_branch") + if !ok { + return fmt.Errorf("unable to load @@log_bin_branch system variable") + } + logBinBranch, ok := logBinBranchValue.(string) + if !ok { + return fmt.Errorf("unexpected type for @@log_bin_branch system variable: %T", logBinBranchValue) + } + if logBinBranch != "" { + // If an invalid branch has been configured, let the server start up so that it's + // easier for customers to correct the value, but log a warning and don't enable + // binlog replication. + if strings.Contains(logBinBranch, "/") { + logrus.Warnf("branch names containing '/' are not supported "+ + "for binlog replication. Not enabling binlog replication; fix "+ + "@@log_bin_branch value and restart Dolt (current value: %s)", logBinBranch) + return nil + } + + binlogreplication.BinlogBranch = logBinBranch + } + if logBin == 1 { - logrus.Info("Enabling binary logging") + logrus.Infof("Enabling binary logging for branch %s", logBinBranch) binlogProducer, err := binlogreplication.NewBinlogProducer(dEnv.FS) if err != nil { return err @@ -344,18 +367,6 @@ func ConfigureServices( } } - _, logBinBranchValue, ok := sql.SystemVariables.GetGlobal("log_bin_branch") - if !ok { - return fmt.Errorf("unable to load @@log_bin_branch system variable") - } - logBinBranch, ok := logBinBranchValue.(string) - if !ok { - return fmt.Errorf("unexpected type for @@log_bin_branch system variable: %T", logBinBranchValue) - } - if logBinBranch != "" { - logrus.Debugf("Setting binary logging branch to %s", logBinBranch) - binlogreplication.BinlogBranch = logBinBranch - } return nil }, } diff --git a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go index cda0a9c2e8f..f8d8f9bf138 100644 --- a/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go +++ b/go/libraries/doltcore/sqle/binlogreplication/binlog_primary_test.go @@ -520,6 +520,20 @@ func TestBinlogPrimary_PrimaryRestartBeforeReplicaConnects(t *testing.T) { }) } +// TestBinlogPrimary_DisallowBranchesWithSlashes asserts that trying to set @@log_bin_branch to +// a branch name containing a slash results in an error. +func TestBinlogPrimary_DisallowBranchesWithSlashes(t *testing.T) { + defer teardown(t) + mapCopy := copyMap(doltReplicationPrimarySystemVars) + mapCopy["log_bin_branch"] = "'branch/withslash'" + startSqlServersWithDoltSystemVars(t, mapCopy) + setupForDoltToMySqlReplication() + + // Because the replication branch was invalid, the binary log status should be + // empty, indicating that no binary logs are being recorded. + requirePrimaryResults(t, "SHOW BINARY LOG STATUS;", [][]any{}) +} + // TestBinlogPrimary_ChangeReplicationBranch asserts that the log_bin_branch system variable can // be used to control what branch is replicated. func TestBinlogPrimary_ChangeReplicationBranch(t *testing.T) {