github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/logstore/driver/logservicedriver/replay.go (about) 1 // Copyright 2021 Matrix Origin 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 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package logservicedriver 16 17 import ( 18 "bytes" 19 "context" 20 "fmt" 21 "io" 22 "math" 23 "sync" 24 "time" 25 26 "github.com/matrixorigin/matrixone/pkg/container/types" 27 "github.com/matrixorigin/matrixone/pkg/logutil" 28 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver" 29 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver/entry" 30 ) 31 32 type replayer struct { 33 readMaxSize int 34 35 truncatedLogserviceLsn uint64 36 37 minDriverLsn uint64 38 maxDriverLsn uint64 39 40 driverLsnLogserviceLsnMap map[uint64]uint64 //start-lsn 41 42 replayHandle driver.ApplyHandle 43 replayedLsn uint64 44 inited bool 45 safeLsn uint64 46 nextToReadLsn uint64 47 d *LogServiceDriver 48 appended []uint64 49 50 recordChan chan *entry.Entry 51 52 applyDuration time.Duration 53 readCount int 54 internalCount int 55 applyCount int 56 57 wg sync.WaitGroup 58 } 59 60 func newReplayer(h driver.ApplyHandle, readmaxsize int, d *LogServiceDriver) *replayer { 61 truncated := d.getLogserviceTruncate() 62 logutil.Infof("truncated %d", truncated) 63 return &replayer{ 64 minDriverLsn: math.MaxUint64, 65 driverLsnLogserviceLsnMap: make(map[uint64]uint64), 66 replayHandle: h, 67 readMaxSize: readmaxsize, 68 nextToReadLsn: truncated + 1, 69 replayedLsn: math.MaxUint64, 70 d: d, 71 appended: make([]uint64, 0), 72 recordChan: make(chan *entry.Entry, 100), 73 wg: sync.WaitGroup{}, 74 truncatedLogserviceLsn: truncated, 75 } 76 } 77 78 func (r *replayer) replay() { 79 var err error 80 r.wg.Add(1) 81 go r.replayRecords() 82 for !r.readRecords() { 83 for r.replayedLsn < r.safeLsn { 84 err := r.replayLogserviceEntry(r.replayedLsn+1, true) 85 if err != nil { 86 panic(err) 87 } 88 } 89 } 90 err = r.replayLogserviceEntry(r.replayedLsn+1, false) 91 for err != ErrAllRecordsRead { 92 err = r.replayLogserviceEntry(r.replayedLsn+1, false) 93 94 } 95 r.d.lsns = make([]uint64, 0) 96 r.recordChan <- entry.NewEndEntry() 97 r.wg.Wait() 98 close(r.recordChan) 99 } 100 101 func (r *replayer) readRecords() (readEnd bool) { 102 nextLsn, safeLsn := r.d.readFromLogServiceInReplay(r.nextToReadLsn, r.readMaxSize, func(lsn uint64, record *recordEntry) { 103 r.readCount++ 104 if record.meta.metaType == TReplay { 105 r.internalCount++ 106 cmd := NewEmptyReplayCmd() 107 cmd.Unmarshal(record.payload) 108 r.removeEntries(cmd.skipLsns) 109 return 110 } 111 drlsn := record.GetMinLsn() 112 r.driverLsnLogserviceLsnMap[drlsn] = lsn 113 if drlsn < r.replayedLsn { 114 if r.inited { 115 panic("logic err") 116 } 117 r.replayedLsn = drlsn - 1 118 } 119 }) 120 if nextLsn == r.nextToReadLsn { 121 return true 122 } 123 r.nextToReadLsn = nextLsn 124 if safeLsn > r.safeLsn { 125 r.safeLsn = safeLsn 126 } 127 return false 128 } 129 func (r *replayer) removeEntries(skipMap map[uint64]uint64) { 130 for lsn := range skipMap { 131 if _, ok := r.driverLsnLogserviceLsnMap[lsn]; !ok { 132 panic(fmt.Sprintf("lsn %d not existed, map is %v", lsn, r.driverLsnLogserviceLsnMap)) 133 } 134 delete(r.driverLsnLogserviceLsnMap, lsn) 135 } 136 } 137 138 func (r *replayer) replayRecords() { 139 defer r.wg.Done() 140 for { 141 e := <-r.recordChan 142 if e.IsEnd() { 143 break 144 } 145 t0 := time.Now() 146 r.replayHandle(e) 147 e.Entry.Free() 148 r.applyDuration += time.Since(t0) 149 } 150 } 151 152 func (r *replayer) replayLogserviceEntry(lsn uint64, safe bool) error { 153 logserviceLsn, ok := r.driverLsnLogserviceLsnMap[lsn] 154 if !ok { 155 if safe { 156 logutil.Infof("drlsn %d has been truncated", lsn) 157 r.minDriverLsn = lsn + 1 158 r.replayedLsn++ 159 return nil 160 } 161 if len(r.driverLsnLogserviceLsnMap) == 0 { 162 return ErrAllRecordsRead 163 } 164 r.AppendSkipCmd(r.driverLsnLogserviceLsnMap) 165 logutil.Infof("skip lsns %v", r.driverLsnLogserviceLsnMap) 166 return ErrAllRecordsRead 167 } 168 record, err := r.d.readFromCache(logserviceLsn) 169 if err == ErrAllRecordsRead { 170 return err 171 } 172 if err != nil { 173 panic(err) 174 } 175 r.applyCount++ 176 intervals := record.replay(r) 177 r.d.onReplayRecordEntry(logserviceLsn, intervals) 178 r.onReplayDriverLsn(intervals.GetMax()) 179 r.onReplayDriverLsn(intervals.GetMin()) 180 r.d.dropRecordByLsn(logserviceLsn) 181 r.replayedLsn = record.GetMaxLsn() 182 r.inited = true 183 delete(r.driverLsnLogserviceLsnMap, lsn) 184 return nil 185 } 186 187 func (r *replayer) AppendSkipCmd(skipMap map[uint64]uint64) { 188 logutil.Infof("skip %v", skipMap) 189 cmd := NewReplayCmd(skipMap) 190 recordEntry := newRecordEntry() 191 recordEntry.meta.metaType = TReplay 192 recordEntry.cmd = cmd 193 size := recordEntry.prepareRecord() 194 c, lsn := r.d.getClient() 195 r.appended = append(r.appended, lsn) 196 c.TryResize(size) 197 record := c.record 198 copy(record.Payload(), recordEntry.payload) 199 record.ResizePayload(size) 200 ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) 201 _, err := c.c.Append(ctx, c.record) 202 cancel() 203 if err != nil { 204 err = RetryWithTimeout(r.d.config.RetryTimeout, func() (shouldReturn bool) { 205 ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) 206 _, err := c.c.Append(ctx, c.record) 207 cancel() 208 return err == nil 209 }) 210 if err != nil { 211 panic(err) 212 } 213 } 214 } 215 func (r *replayer) onReplayDriverLsn(lsn uint64) { 216 if lsn == 0 { 217 return 218 } 219 if lsn < r.minDriverLsn { 220 r.minDriverLsn = lsn 221 } 222 if lsn > r.maxDriverLsn { 223 r.maxDriverLsn = lsn 224 } 225 } 226 227 type ReplayCmd struct { 228 skipLsns map[uint64]uint64 229 } 230 231 func NewReplayCmd(skipLsns map[uint64]uint64) *ReplayCmd { 232 return &ReplayCmd{ 233 skipLsns: skipLsns, 234 } 235 } 236 func NewEmptyReplayCmd() *ReplayCmd { 237 return &ReplayCmd{ 238 skipLsns: make(map[uint64]uint64), 239 } 240 } 241 242 func (c *ReplayCmd) WriteTo(w io.Writer) (n int64, err error) { 243 length := uint16(len(c.skipLsns)) 244 if _, err = w.Write(types.EncodeUint16(&length)); err != nil { 245 return 246 } 247 n += 2 248 for drlsn, logserviceLsn := range c.skipLsns { 249 if _, err = w.Write(types.EncodeUint64(&drlsn)); err != nil { 250 return 251 } 252 n += 8 253 if _, err = w.Write(types.EncodeUint64(&logserviceLsn)); err != nil { 254 return 255 } 256 n += 8 257 } 258 return 259 } 260 261 func (c *ReplayCmd) ReadFrom(r io.Reader) (n int64, err error) { 262 length := uint16(0) 263 if _, err = r.Read(types.EncodeUint16(&length)); err != nil { 264 return 265 } 266 n += 2 267 for i := 0; i < int(length); i++ { 268 drlsn := uint64(0) 269 lsn := uint64(0) 270 if _, err = r.Read(types.EncodeUint64(&drlsn)); err != nil { 271 return 272 } 273 n += 8 274 if _, err = r.Read(types.EncodeUint64(&lsn)); err != nil { 275 return 276 } 277 n += 8 278 c.skipLsns[drlsn] = lsn 279 } 280 return 281 } 282 283 func (c *ReplayCmd) Unmarshal(buf []byte) error { 284 bbuf := bytes.NewBuffer(buf) 285 _, err := c.ReadFrom(bbuf) 286 return err 287 } 288 289 func (c *ReplayCmd) Marshal() (buf []byte, err error) { 290 var bbuf bytes.Buffer 291 if _, err = c.WriteTo(&bbuf); err != nil { 292 return 293 } 294 buf = bbuf.Bytes() 295 return 296 }