github.com/matrixorigin/matrixone@v1.2.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 "io" 20 "math" 21 "sync" 22 "sync/atomic" 23 24 "github.com/matrixorigin/matrixone/pkg/common/moerr" 25 "github.com/matrixorigin/matrixone/pkg/container/types" 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 117 func (w *StoreInfo) allocateLsn(gid uint32) uint64 { 118 w.lsnmu.Lock() 119 defer w.lsnmu.Unlock() 120 lsn, ok := w.walCurrentLsn[gid] 121 if !ok { 122 w.walCurrentLsn[gid] = 1 123 return 1 124 } 125 lsn++ 126 w.walCurrentLsn[gid] = lsn 127 return lsn 128 } 129 130 func (w *StoreInfo) logDriverLsn(driverEntry *driverEntry.Entry) { 131 info := driverEntry.Info 132 133 if w.syncing[info.Group] < info.GroupLSN { 134 w.syncing[info.Group] = info.GroupLSN 135 } 136 137 w.lsnMu.Lock() 138 lsnMap, ok := w.walDriverLsnMap[info.Group] 139 if !ok { 140 lsnMap = make(map[uint64]uint64) 141 w.walDriverLsnMap[info.Group] = lsnMap 142 } 143 lsnMap[info.GroupLSN] = driverEntry.Lsn 144 w.lsnMu.Unlock() 145 } 146 147 func (w *StoreInfo) onAppend() { 148 w.commitCond.L.Lock() 149 w.commitCond.Broadcast() 150 w.commitCond.L.Unlock() 151 w.syncedMu.Lock() 152 for gid, lsn := range w.syncing { 153 w.synced[gid] = lsn 154 } 155 w.syncedMu.Unlock() 156 } 157 158 func (w *StoreInfo) retryGetDriverLsn(gid uint32, lsn uint64) (driverLsn uint64, err error) { 159 driverLsn, err = w.getDriverLsn(gid, lsn) 160 if err == ErrGroupNotFount || err == ErrLsnNotFount { 161 currLsn := w.GetCurrSeqNum(gid) 162 if lsn <= currLsn { 163 for i := 0; i < 10; i++ { 164 logutil.Debugf("retry %d-%d", gid, lsn) 165 w.commitCond.L.Lock() 166 driverLsn, err = w.getDriverLsn(gid, lsn) 167 if err != ErrGroupNotFount && err != ErrLsnNotFount { 168 w.commitCond.L.Unlock() 169 return 170 } 171 w.commitCond.Wait() 172 w.commitCond.L.Unlock() 173 driverLsn, err = w.getDriverLsn(gid, lsn) 174 if err != ErrGroupNotFount && err != ErrLsnNotFount { 175 return 176 } 177 } 178 return 0, ErrTimeOut 179 } 180 return 181 } 182 return 183 } 184 185 func (w *StoreInfo) getDriverLsn(gid uint32, lsn uint64) (driverLsn uint64, err error) { 186 w.lsnMu.RLock() 187 defer w.lsnMu.RUnlock() 188 minLsn := w.minLsn[gid] 189 if lsn < minLsn { 190 return 0, ErrLsnTooSmall 191 } 192 lsnMap, ok := w.walDriverLsnMap[gid] 193 if !ok { 194 return 0, ErrGroupNotFount 195 } 196 driverLsn, ok = lsnMap[lsn] 197 if !ok { 198 return 0, ErrLsnNotFount 199 } 200 return 201 } 202 203 func (w *StoreInfo) gcWalDriverLsnMap(drlsn uint64) { 204 w.lsnMu.Lock() 205 defer w.lsnMu.Unlock() 206 for gid, walDriverLsnMap := range w.walDriverLsnMap { 207 minLsn := w.minLsn[gid] 208 lsnsToDelete := make([]uint64, 0) 209 for storeLSN, driverLSN := range walDriverLsnMap { 210 if driverLSN < drlsn { 211 lsnsToDelete = append(lsnsToDelete, storeLSN) 212 if storeLSN > minLsn { 213 minLsn = storeLSN 214 } 215 } 216 } 217 for _, lsn := range lsnsToDelete { 218 delete(walDriverLsnMap, lsn) 219 } 220 w.minLsn[gid] = minLsn + 1 221 } 222 } 223 224 func (w *StoreInfo) logCheckpointInfo(info *entry.Info) { 225 switch info.Group { 226 case GroupCKP: 227 for _, intervals := range info.Checkpoints { 228 w.ckpMu.Lock() 229 ckpInfo, ok := w.checkpointInfo[intervals.Group] 230 if !ok { 231 ckpInfo = newCheckpointInfo() 232 w.checkpointInfo[intervals.Group] = ckpInfo 233 } 234 if intervals.Ranges != nil && len(intervals.Ranges.Intervals) > 0 { 235 ckpInfo.UpdateWtihRanges(intervals.Ranges) 236 } 237 if intervals.Command != nil { 238 ckpInfo.MergeCommandMap(intervals.Command) 239 } 240 w.ckpMu.Unlock() 241 } 242 case GroupInternal: 243 w.checkpointedMu.Lock() 244 w.checkpointed[GroupCKP] = info.TargetLsn 245 w.checkpointed[GroupInternal] = info.GroupLSN - 1 246 w.checkpointedMu.Unlock() 247 } 248 } 249 250 func (w *StoreInfo) onCheckpoint() { 251 w.checkpointedMu.Lock() 252 for gid, ckp := range w.checkpointInfo { 253 ckped := ckp.GetCheckpointed() 254 // logutil.Infof("%d-%v", gid, ckp) 255 if ckped == 0 { 256 continue 257 } 258 w.checkpointed[gid] = ckped 259 } 260 w.checkpointedMu.Unlock() 261 w.ckpcntMu.Lock() 262 for gid, ckp := range w.checkpointInfo { 263 w.ckpcnt[gid] = ckp.GetCkpCnt() 264 } 265 w.ckpcntMu.Unlock() 266 } 267 268 func (w *StoreInfo) getDriverCheckpointed() (gid uint32, driverLsn uint64) { 269 groups := make(map[uint32]uint64, 0) 270 w.lsnmu.Lock() 271 for g, lsn := range w.walCurrentLsn { 272 groups[g] = lsn 273 } 274 w.lsnmu.Unlock() 275 276 w.checkpointedMu.RLock() 277 defer w.checkpointedMu.RUnlock() 278 if len(w.checkpointed) == 0 { 279 return 280 } 281 driverLsn = math.MaxInt64 282 for g, maxLsn := range groups { 283 lsn := w.checkpointed[g] 284 var drLsn uint64 285 var err error 286 if lsn < maxLsn { 287 drLsn, err = w.retryGetDriverLsn(g, lsn+1) 288 if err != nil { 289 if err == ErrLsnTooSmall { 290 logutil.Debugf("%d-%d too small", g, lsn) 291 return g, 0 292 } 293 logutil.Debugf("%d-%d", g, lsn) 294 panic(err) 295 } 296 drLsn-- 297 } else { 298 continue 299 } 300 if drLsn < driverLsn { 301 gid = g 302 driverLsn = drLsn 303 } 304 } 305 return 306 } 307 308 func (w *StoreInfo) makeInternalCheckpointEntry() (e entry.Entry) { 309 e = entry.GetBase() 310 lsn := w.GetSynced(GroupCKP) 311 e.SetType(entry.IOET_WALEntry_PostCommit) 312 buf, err := w.marshalPostCommitEntry() 313 if err != nil { 314 panic(err) 315 } 316 err = e.SetPayload(buf) 317 if err != nil { 318 panic(err) 319 } 320 info := &entry.Info{} 321 info.TargetLsn = lsn 322 info.Group = GroupInternal 323 e.SetInfo(info) 324 return 325 } 326 327 func (w *StoreInfo) marshalPostCommitEntry() (buf []byte, err error) { 328 var bbuf bytes.Buffer 329 if _, err = w.writePostCommitEntry(&bbuf); err != nil { 330 return 331 } 332 buf = bbuf.Bytes() 333 return 334 } 335 336 func (w *StoreInfo) unmarshalPostCommitEntry(buf []byte) (err error) { 337 bbuf := bytes.NewBuffer(buf) 338 _, err = w.readPostCommitEntry(bbuf) 339 return 340 } 341 342 func (w *StoreInfo) writePostCommitEntry(writer io.Writer) (n int64, err error) { 343 w.ckpMu.RLock() 344 defer w.ckpMu.RUnlock() 345 //checkpointing 346 length := uint32(len(w.checkpointInfo)) 347 if _, err = writer.Write(types.EncodeUint32(&length)); err != nil { 348 return 349 } 350 n += 4 351 for groupID, ckpInfo := range w.checkpointInfo { 352 if _, err = writer.Write(types.EncodeUint32(&groupID)); err != nil { 353 return 354 } 355 n += 4 356 sn, err := ckpInfo.WriteTo(writer) 357 n += sn 358 if err != nil { 359 return n, err 360 } 361 } 362 return 363 } 364 365 func (w *StoreInfo) readPostCommitEntry(reader io.Reader) (n int64, err error) { 366 w.ckpMu.Lock() 367 defer w.ckpMu.Unlock() 368 //checkpointing 369 length := uint32(0) 370 if _, err = reader.Read(types.EncodeUint32(&length)); err != nil { 371 return 372 } 373 n += 4 374 for i := 0; i < int(length); i++ { 375 groupID := uint32(0) 376 if _, err = reader.Read(types.EncodeUint32(&groupID)); err != nil { 377 return 378 } 379 n += 4 380 ckpInfo := newCheckpointInfo() 381 sn, err := ckpInfo.ReadFrom(reader) 382 n += sn 383 if err != nil { 384 return n, err 385 } 386 ckp, ok := w.checkpointInfo[groupID] 387 if ok { 388 ckp.MergeCheckpointInfo(ckpInfo) 389 } else { 390 w.checkpointInfo[groupID] = ckpInfo 391 } 392 } 393 return 394 }