github.com/matrixorigin/matrixone@v0.7.0/pkg/vm/engine/tae/logstore/store/walinfo.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 store 16 17 import ( 18 "bytes" 19 "encoding/binary" 20 "io" 21 "math" 22 "sync" 23 "sync/atomic" 24 25 "github.com/matrixorigin/matrixone/pkg/common/moerr" 26 "github.com/matrixorigin/matrixone/pkg/logutil" 27 driverEntry "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver/entry" 28 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/entry" 29 ) 30 31 var ( 32 ErrGroupNotFount = moerr.NewInternalErrorNoCtx("group not found") 33 ErrLsnNotFount = moerr.NewInternalErrorNoCtx("lsn not found") 34 ErrTimeOut = moerr.NewInternalErrorNoCtx("retry timeout") 35 ErrLsnTooSmall = moerr.NewInternalErrorNoCtx("lsn is too small") 36 ) 37 38 type StoreInfo struct { 39 checkpointInfo map[uint32]*checkpointInfo 40 ckpMu sync.RWMutex 41 walDriverLsnMap map[uint32]map[uint64]uint64 42 lsnMu sync.RWMutex 43 driverCheckpointing atomic.Uint64 44 driverCheckpointed uint64 45 walCurrentLsn map[uint32]uint64 //todo 46 lsnmu sync.RWMutex 47 syncing map[uint32]uint64 //todo 48 49 synced map[uint32]uint64 //todo 50 syncedMu sync.RWMutex 51 commitCond sync.Cond 52 53 checkpointed map[uint32]uint64 54 checkpointedMu sync.RWMutex 55 ckpcnt map[uint32]uint64 56 ckpcntMu sync.RWMutex 57 58 minLsn map[uint32]uint64 59 } 60 61 func newWalInfo() *StoreInfo { 62 return &StoreInfo{ 63 checkpointInfo: make(map[uint32]*checkpointInfo), 64 ckpMu: sync.RWMutex{}, 65 walDriverLsnMap: make(map[uint32]map[uint64]uint64), 66 lsnMu: sync.RWMutex{}, 67 walCurrentLsn: make(map[uint32]uint64), 68 lsnmu: sync.RWMutex{}, 69 syncing: make(map[uint32]uint64), 70 commitCond: *sync.NewCond(&sync.Mutex{}), 71 72 checkpointed: make(map[uint32]uint64), 73 checkpointedMu: sync.RWMutex{}, 74 synced: make(map[uint32]uint64), 75 syncedMu: sync.RWMutex{}, 76 ckpcnt: make(map[uint32]uint64), 77 ckpcntMu: sync.RWMutex{}, 78 79 minLsn: make(map[uint32]uint64), 80 } 81 } 82 83 func (w *StoreInfo) GetCurrSeqNum(gid uint32) (lsn uint64) { 84 w.lsnmu.RLock() 85 defer w.lsnmu.RUnlock() 86 lsn = w.walCurrentLsn[gid] 87 return 88 } 89 func (w *StoreInfo) GetSynced(gid uint32) (lsn uint64) { 90 w.syncedMu.RLock() 91 defer w.syncedMu.RUnlock() 92 lsn = w.synced[gid] 93 return 94 } 95 96 func (w *StoreInfo) GetPendding(gid uint32) (cnt uint64) { 97 lsn := w.GetCurrSeqNum(gid) 98 w.ckpcntMu.RLock() 99 ckpcnt := w.ckpcnt[gid] 100 w.ckpcntMu.RUnlock() 101 cnt = lsn - ckpcnt 102 return 103 } 104 func (w *StoreInfo) GetCheckpointed(gid uint32) (lsn uint64) { 105 w.checkpointedMu.RLock() 106 lsn = w.checkpointed[gid] 107 w.checkpointedMu.RUnlock() 108 return 109 } 110 111 func (w *StoreInfo) SetCheckpointed(gid uint32, lsn uint64) { 112 w.checkpointedMu.Lock() 113 w.checkpointed[gid] = lsn 114 w.checkpointedMu.Unlock() 115 } 116 func (w *StoreInfo) allocateLsn(gid uint32) uint64 { 117 w.lsnmu.Lock() 118 defer w.lsnmu.Unlock() 119 lsn, ok := w.walCurrentLsn[gid] 120 if !ok { 121 w.walCurrentLsn[gid] = 1 122 return 1 123 } 124 lsn++ 125 w.walCurrentLsn[gid] = lsn 126 return lsn 127 } 128 129 func (w *StoreInfo) logDriverLsn(driverEntry *driverEntry.Entry) { 130 info := driverEntry.Info 131 132 if w.syncing[info.Group] < info.GroupLSN { 133 w.syncing[info.Group] = info.GroupLSN 134 } 135 136 w.lsnMu.Lock() 137 lsnMap, ok := w.walDriverLsnMap[info.Group] 138 if !ok { 139 lsnMap = make(map[uint64]uint64) 140 w.walDriverLsnMap[info.Group] = lsnMap 141 } 142 lsnMap[info.GroupLSN] = driverEntry.Lsn 143 w.lsnMu.Unlock() 144 } 145 146 func (w *StoreInfo) onAppend() { 147 w.commitCond.L.Lock() 148 w.commitCond.Broadcast() 149 w.commitCond.L.Unlock() 150 w.syncedMu.Lock() 151 for gid, lsn := range w.syncing { 152 w.synced[gid] = lsn 153 } 154 w.syncedMu.Unlock() 155 } 156 157 func (w *StoreInfo) retryGetDriverLsn(gid uint32, lsn uint64) (driverLsn uint64, err error) { 158 driverLsn, err = w.getDriverLsn(gid, lsn) 159 if err == ErrGroupNotFount || err == ErrLsnNotFount { 160 currLsn := w.GetCurrSeqNum(gid) 161 if lsn <= currLsn { 162 for i := 0; i < 10; i++ { 163 logutil.Infof("retry %d-%d", gid, lsn) 164 w.commitCond.L.Lock() 165 driverLsn, err = w.getDriverLsn(gid, lsn) 166 if err != ErrGroupNotFount && err != ErrLsnNotFount { 167 w.commitCond.L.Unlock() 168 return 169 } 170 w.commitCond.Wait() 171 w.commitCond.L.Unlock() 172 driverLsn, err = w.getDriverLsn(gid, lsn) 173 if err != ErrGroupNotFount && err != ErrLsnNotFount { 174 return 175 } 176 } 177 return 0, ErrTimeOut 178 } 179 return 180 } 181 return 182 } 183 184 func (w *StoreInfo) getDriverLsn(gid uint32, lsn uint64) (driverLsn uint64, err error) { 185 w.lsnMu.RLock() 186 defer w.lsnMu.RUnlock() 187 minLsn := w.minLsn[gid] 188 if lsn < minLsn { 189 return 0, ErrLsnTooSmall 190 } 191 lsnMap, ok := w.walDriverLsnMap[gid] 192 if !ok { 193 return 0, ErrGroupNotFount 194 } 195 driverLsn, ok = lsnMap[lsn] 196 if !ok { 197 return 0, ErrLsnNotFount 198 } 199 return 200 } 201 202 func (w *StoreInfo) logCheckpointInfo(info *entry.Info) { 203 switch info.Group { 204 case GroupCKP: 205 for _, intervals := range info.Checkpoints { 206 w.ckpMu.Lock() 207 ckpInfo, ok := w.checkpointInfo[intervals.Group] 208 if !ok { 209 ckpInfo = newCheckpointInfo() 210 w.checkpointInfo[intervals.Group] = ckpInfo 211 } 212 if intervals.Ranges != nil && len(intervals.Ranges.Intervals) > 0 { 213 ckpInfo.UpdateWtihRanges(intervals.Ranges) 214 } 215 if intervals.Command != nil { 216 ckpInfo.MergeCommandMap(intervals.Command) 217 } 218 w.ckpMu.Unlock() 219 } 220 case GroupInternal: 221 w.checkpointedMu.Lock() 222 w.checkpointed[GroupCKP] = info.TargetLsn 223 w.checkpointed[GroupInternal] = info.GroupLSN - 1 224 w.checkpointedMu.Unlock() 225 } 226 } 227 228 func (w *StoreInfo) onCheckpoint() { 229 w.checkpointedMu.Lock() 230 for gid, ckp := range w.checkpointInfo { 231 ckped := ckp.GetCheckpointed() 232 // logutil.Infof("%d-%v", gid, ckp) 233 if ckped == 0 { 234 continue 235 } 236 w.checkpointed[gid] = ckped 237 } 238 w.checkpointedMu.Unlock() 239 w.ckpcntMu.Lock() 240 for gid, ckp := range w.checkpointInfo { 241 w.ckpcnt[gid] = ckp.GetCkpCnt() 242 } 243 w.ckpcntMu.Unlock() 244 } 245 246 func (w *StoreInfo) getDriverCheckpointed() (gid uint32, driverLsn uint64) { 247 groups := make(map[uint32]uint64, 0) 248 w.lsnmu.Lock() 249 for g, lsn := range w.walCurrentLsn { 250 groups[g] = lsn 251 } 252 w.lsnmu.Unlock() 253 254 w.checkpointedMu.RLock() 255 defer w.checkpointedMu.RUnlock() 256 if len(w.checkpointed) == 0 { 257 return 258 } 259 driverLsn = math.MaxInt64 260 for g, maxLsn := range groups { 261 lsn := w.checkpointed[g] 262 var drLsn uint64 263 var err error 264 if lsn < maxLsn { 265 drLsn, err = w.retryGetDriverLsn(g, lsn+1) 266 if err != nil { 267 if err == ErrLsnTooSmall { 268 logutil.Infof("%d-%d too small", g, lsn) 269 return g, 0 270 } 271 logutil.Infof("%d-%d", g, lsn) 272 panic(err) 273 } 274 drLsn-- 275 } else { 276 continue 277 } 278 if drLsn < driverLsn { 279 gid = g 280 driverLsn = drLsn 281 } 282 } 283 return 284 } 285 286 func (w *StoreInfo) makeInternalCheckpointEntry() (e entry.Entry) { 287 e = entry.GetBase() 288 lsn := w.GetSynced(GroupCKP) 289 e.SetType(entry.ETPostCommit) 290 buf, err := w.marshalPostCommitEntry() 291 if err != nil { 292 panic(err) 293 } 294 err = e.SetPayload(buf) 295 if err != nil { 296 panic(err) 297 } 298 info := &entry.Info{} 299 info.TargetLsn = lsn 300 info.Group = GroupInternal 301 e.SetInfo(info) 302 return 303 } 304 305 func (w *StoreInfo) marshalPostCommitEntry() (buf []byte, err error) { 306 var bbuf bytes.Buffer 307 if _, err = w.writePostCommitEntry(&bbuf); err != nil { 308 return 309 } 310 buf = bbuf.Bytes() 311 return 312 } 313 314 func (w *StoreInfo) unmarshalPostCommitEntry(buf []byte) (err error) { 315 bbuf := bytes.NewBuffer(buf) 316 _, err = w.readPostCommitEntry(bbuf) 317 return 318 } 319 320 func (w *StoreInfo) writePostCommitEntry(writer io.Writer) (n int64, err error) { 321 w.ckpMu.RLock() 322 defer w.ckpMu.RUnlock() 323 //checkpointing 324 length := uint32(len(w.checkpointInfo)) 325 if err = binary.Write(writer, binary.BigEndian, length); err != nil { 326 return 327 } 328 n += 4 329 for groupID, ckpInfo := range w.checkpointInfo { 330 if err = binary.Write(writer, binary.BigEndian, groupID); err != nil { 331 return 332 } 333 n += 4 334 sn, err := ckpInfo.WriteTo(writer) 335 n += sn 336 if err != nil { 337 return n, err 338 } 339 } 340 return 341 } 342 343 func (w *StoreInfo) readPostCommitEntry(reader io.Reader) (n int64, err error) { 344 w.ckpMu.Lock() 345 defer w.ckpMu.Unlock() 346 //checkpointing 347 length := uint32(0) 348 if err = binary.Read(reader, binary.BigEndian, &length); err != nil { 349 return 350 } 351 n += 4 352 for i := 0; i < int(length); i++ { 353 groupID := uint32(0) 354 if err = binary.Read(reader, binary.BigEndian, &groupID); err != nil { 355 return 356 } 357 n += 4 358 ckpInfo := newCheckpointInfo() 359 sn, err := ckpInfo.ReadFrom(reader) 360 n += sn 361 if err != nil { 362 return n, err 363 } 364 ckp, ok := w.checkpointInfo[groupID] 365 if ok { 366 ckp.MergeCheckpointInfo(ckpInfo) 367 } else { 368 w.checkpointInfo[groupID] = ckpInfo 369 } 370 } 371 return 372 }