github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/binlog/reader/file.go (about) 1 // Copyright 2019 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 // binlog events generator for MySQL used to generate some binlog events for tests. 15 // Readability takes precedence over performance. 16 17 package reader 18 19 import ( 20 "context" 21 "encoding/json" 22 "sync" 23 "time" 24 25 gmysql "github.com/go-mysql-org/go-mysql/mysql" 26 "github.com/go-mysql-org/go-mysql/replication" 27 "github.com/pingcap/errors" 28 "github.com/pingcap/tiflow/dm/pkg/binlog/common" 29 "github.com/pingcap/tiflow/dm/pkg/log" 30 "github.com/pingcap/tiflow/dm/pkg/terror" 31 "go.uber.org/atomic" 32 "go.uber.org/zap" 33 ) 34 35 // FileReader is a binlog event reader which reads binlog events from a file. 36 type FileReader struct { 37 mu sync.RWMutex 38 wg sync.WaitGroup 39 40 stage common.Stage 41 readOffset atomic.Uint32 42 sendOffset atomic.Uint32 43 44 parser *replication.BinlogParser 45 ch chan *replication.BinlogEvent 46 ech chan error 47 endCh chan struct{} 48 49 logger log.Logger 50 51 ctx context.Context 52 cancel context.CancelFunc 53 } 54 55 // FileReaderConfig is the configuration used by a FileReader. 56 type FileReaderConfig struct { 57 EnableRawMode bool 58 Timezone *time.Location 59 ChBufferSize int // event channel's buffer size 60 EchBufferSize int // error channel's buffer size 61 } 62 63 // FileReaderStatus represents the status of a FileReader. 64 type FileReaderStatus struct { 65 Stage string `json:"stage"` 66 ReadOffset uint32 `json:"read-offset"` // read event's offset in the file 67 SendOffset uint32 `json:"send-offset"` // sent event's offset in the file 68 } 69 70 // String implements Stringer.String. 71 func (s *FileReaderStatus) String() string { 72 data, err := json.Marshal(s) 73 if err != nil { 74 log.L().Error("fail to marshal status to json", zap.Reflect("file reader status", s), log.ShortError(err)) 75 } 76 return string(data) 77 } 78 79 // NewFileReader creates a FileReader instance. 80 func NewFileReader(cfg *FileReaderConfig) Reader { 81 parser := replication.NewBinlogParser() 82 parser.SetVerifyChecksum(true) 83 parser.SetUseDecimal(false) 84 parser.SetRawMode(cfg.EnableRawMode) 85 if cfg.Timezone != nil { 86 parser.SetTimestampStringLocation(cfg.Timezone) 87 } 88 return &FileReader{ 89 parser: parser, 90 ch: make(chan *replication.BinlogEvent, cfg.ChBufferSize), 91 ech: make(chan error, cfg.EchBufferSize), 92 endCh: make(chan struct{}), 93 logger: log.With(zap.String("component", "binlog file reader")), 94 } 95 } 96 97 // StartSyncByPos implements Reader.StartSyncByPos. 98 // TODO: support heartbeat event. 99 func (r *FileReader) StartSyncByPos(pos gmysql.Position) error { 100 r.mu.Lock() 101 defer r.mu.Unlock() 102 103 if r.stage != common.StageNew { 104 return terror.ErrReaderAlreadyStarted.Generate(r.stage, common.StageNew) 105 } 106 107 // keep running until canceled in `Close`. 108 r.ctx, r.cancel = context.WithCancel(context.Background()) 109 r.wg.Add(1) 110 go func() { 111 defer r.wg.Done() 112 err := r.parser.ParseFile(pos.Name, int64(pos.Pos), r.onEvent) 113 if err != nil { 114 if errors.Cause(err) != context.Canceled { 115 r.logger.Error("fail to parse binlog file", zap.Error(err)) 116 } 117 select { 118 case r.ech <- err: 119 case <-r.ctx.Done(): 120 } 121 } else { 122 r.logger.Info("parse end of binlog file", zap.Uint32("pos", r.readOffset.Load())) 123 close(r.endCh) 124 } 125 }() 126 127 r.stage = common.StagePrepared 128 return nil 129 } 130 131 // StartSyncByGTID implements Reader.StartSyncByGTID. 132 func (r *FileReader) StartSyncByGTID(gSet gmysql.GTIDSet) error { 133 // NOTE: may be supported later. 134 return terror.ErrBinlogReadFileByGTID.Generate() 135 } 136 137 // Close implements Reader.Close. 138 func (r *FileReader) Close() error { 139 r.mu.Lock() 140 defer r.mu.Unlock() 141 142 if r.stage != common.StagePrepared { 143 return terror.ErrReaderStateCannotClose.Generate(r.stage, common.StagePrepared) 144 } 145 146 r.cancel() 147 r.wg.Wait() 148 r.parser.Stop() 149 r.stage = common.StageClosed 150 return nil 151 } 152 153 // GetEvent implements Reader.GetEvent. 154 func (r *FileReader) GetEvent(ctx context.Context) (*replication.BinlogEvent, error) { 155 r.mu.RLock() 156 defer r.mu.RUnlock() 157 158 if r.stage != common.StagePrepared { 159 return nil, terror.ErrReaderShouldStartSync.Generate(r.stage, common.StagePrepared) 160 } 161 162 select { 163 case ev := <-r.ch: 164 r.sendOffset.Store(ev.Header.LogPos) 165 return ev, nil 166 case err := <-r.ech: 167 return nil, err 168 case <-r.endCh: 169 return nil, terror.ErrReaderReachEndOfFile.Generate() 170 case <-ctx.Done(): 171 return nil, ctx.Err() 172 } 173 } 174 175 // Status implements Reader.Status. 176 func (r *FileReader) Status() interface{} { 177 r.mu.RLock() 178 stage := r.stage 179 r.mu.RUnlock() 180 181 return &FileReaderStatus{ 182 Stage: stage.String(), 183 ReadOffset: r.readOffset.Load(), 184 SendOffset: r.sendOffset.Load(), 185 } 186 } 187 188 func (r *FileReader) onEvent(ev *replication.BinlogEvent) error { 189 select { 190 case r.ch <- ev: 191 r.readOffset.Store(ev.Header.LogPos) 192 return nil 193 case <-r.ctx.Done(): 194 return r.ctx.Err() 195 } 196 }