github.com/matrixorigin/matrixone@v1.2.0/pkg/logservice/truncation.go (about) 1 // Copyright 2021 - 2022 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 logservice 16 17 import ( 18 "context" 19 "errors" 20 "time" 21 22 "github.com/lni/dragonboat/v4" 23 "github.com/matrixorigin/matrixone/pkg/hakeeper" 24 "go.uber.org/zap" 25 ) 26 27 type snapshotInfo struct { 28 // truncatedLsn is LSN that has been take in snapshot manager. 29 // Log entries before it have been removed. 30 truncatedLsn uint64 31 // snapshotIndex is the latest snapshot index. 32 snapshotIndex uint64 33 } 34 35 type shardSnapshotInfo map[uint64]*snapshotInfo // shardID => snapshotInfo 36 37 func newShardSnapshotInfo() shardSnapshotInfo { 38 return make(shardSnapshotInfo) 39 } 40 41 func (s shardSnapshotInfo) init(shardID uint64) { 42 s[shardID] = &snapshotInfo{ 43 truncatedLsn: 0, 44 snapshotIndex: 0, 45 } 46 } 47 48 func (s shardSnapshotInfo) forwardTruncate(shardID uint64, lsn uint64) { 49 if _, ok := s[shardID]; !ok { 50 s.init(shardID) 51 } 52 if lsn > s[shardID].truncatedLsn { 53 s[shardID].truncatedLsn = lsn 54 } 55 } 56 57 func (s shardSnapshotInfo) forwardSnapshot(shardID uint64, index uint64) { 58 if _, ok := s[shardID]; !ok { 59 s.init(shardID) 60 } 61 if index > s[shardID].snapshotIndex { 62 s[shardID].snapshotIndex = index 63 } 64 } 65 66 func (s shardSnapshotInfo) getTruncatedLsn(shardID uint64) uint64 { 67 if _, ok := s[shardID]; !ok { 68 s.init(shardID) 69 return 0 70 } 71 return s[shardID].truncatedLsn 72 } 73 74 func (s shardSnapshotInfo) getSnapshotIndex(shardID uint64) uint64 { 75 if _, ok := s[shardID]; !ok { 76 s.init(shardID) 77 return 0 78 } 79 return s[shardID].snapshotIndex 80 } 81 82 func (l *store) truncationWorker(ctx context.Context) { 83 defer func() { 84 l.runtime.Logger().Info("truncation worker stopped") 85 }() 86 87 if l.cfg.TruncateInterval.Duration == 0 || l.cfg.HAKeeperTruncateInterval.Duration == 0 { 88 panic("TruncateInterval is 0") 89 } 90 ticker := time.NewTicker(l.cfg.TruncateInterval.Duration) 91 defer ticker.Stop() 92 93 haTicker := time.NewTicker(l.cfg.HAKeeperTruncateInterval.Duration) 94 defer haTicker.Stop() 95 96 for { 97 select { 98 case <-ctx.Done(): 99 return 100 case <-ticker.C: 101 if err := l.processTruncateLog(ctx); err != nil { 102 l.runtime.Logger().Error("truncate failed", zap.Error(err)) 103 } 104 case <-haTicker.C: 105 if err := l.processHAKeeperTruncation(ctx); err != nil { 106 l.runtime.Logger().Error("HAKeeper truncate failed", zap.Error(err)) 107 } 108 } 109 } 110 } 111 112 // processTruncateLog process log truncation for all shards excpet 113 // hakeeper shard. 114 func (l *store) processTruncateLog(ctx context.Context) error { 115 for _, shard := range l.getShards() { 116 if shard.ShardID == hakeeper.DefaultHAKeeperShardID { 117 continue 118 } 119 if err := l.processShardTruncateLog(ctx, shard.ShardID); err != nil { 120 l.runtime.Logger().Error("process truncate log failed", zap.Error(err)) 121 } 122 } 123 return nil 124 } 125 126 // exportSnapshot just export the snapshot but do NOT truncate logs. 127 func (l *store) exportSnapshot(ctx context.Context, shardID uint64, replicaID uint64) error { 128 opts := dragonboat.SnapshotOption{ 129 // set to true means just export snapshot, do NOT truncate logs. 130 Exported: true, 131 ExportPath: l.snapshotMgr.exportPath(shardID, replicaID), 132 } 133 // Just take a snapshot and export it. This snapshot is invisible to system. 134 idx, err := l.nh.SyncRequestSnapshot(ctx, shardID, opts) 135 if err != nil { 136 l.runtime.Logger().Error("request export snapshot failed", zap.Error(err)) 137 return err 138 } 139 // Add the exported snapshot to snapshot manager. 140 if err := l.snapshotMgr.Add(shardID, replicaID, idx); err != nil { 141 l.runtime.Logger().Error("add exported snapshot failed", zap.Error(err)) 142 return err 143 } 144 // forward the snapshot index. 145 l.shardSnapshotInfo.forwardSnapshot(shardID, idx) 146 return nil 147 } 148 149 func (l *store) importSnapshot( 150 ctx context.Context, shardID uint64, replicaID uint64, lsn uint64, dir string, 151 ) error { 152 // Import a snapshot to override the snapshot in system. 153 if err := l.nh.SyncRequestImportSnapshot(ctx, shardID, replicaID, dir); err != nil { 154 l.runtime.Logger().Error("import snapshot failed", zap.Error(err)) 155 return err 156 } 157 // Then remove the exported snapshot in manager. 158 if err := l.snapshotMgr.Remove(shardID, replicaID, lsn); err != nil { 159 l.runtime.Logger().Error("remove exported snapshots failed") 160 return err 161 } 162 // forward the truncate lsn. 163 l.shardSnapshotInfo.forwardTruncate(shardID, lsn) 164 return nil 165 } 166 167 // shouldProcess checks whether we should do the truncation for the shard. 168 // The param lsn comes from statemachine, that client sets. 169 // TODO(liubo): add more policies for truncation. 170 func (l *store) shouldProcess(shardID uint64, lsn uint64) bool { 171 // the first 4 entries for a 3-replica raft group are tiny anyway 172 return lsn > 1 173 } 174 175 // shouldDoImport checks whether compact logs and import snapshot. 176 // The param lsn is the biggest one from exported snapshots that are 177 // managed by snapshotManager. 178 // This check avoid doing the truncation at the same lsn. 179 func (l *store) shouldDoImport(shardID uint64, lsn uint64, dir string) bool { 180 if len(dir) == 0 { 181 return false 182 } 183 return lsn > l.shardSnapshotInfo.getTruncatedLsn(shardID) 184 } 185 186 // shouldDoExport checks whether current applied index is greater than 187 // snapshot index, which is last applied index. Means that, if applied 188 // index does not advance, do not do export. 189 func (l *store) shouldDoExport(ctx context.Context, shardID uint64, replicaID uint64) bool { 190 if l.snapshotMgr.Count(shardID, replicaID) >= l.cfg.MaxExportedSnapshot { 191 return false 192 } 193 v, err := l.read(ctx, shardID, indexQuery{}) 194 if err != nil { 195 return false 196 } 197 return v.(uint64) > l.shardSnapshotInfo.getSnapshotIndex(shardID) 198 } 199 200 func (l *store) processShardTruncateLog(ctx context.Context, shardID uint64) error { 201 // Do NOT process before leader is OK. 202 leaderID, err := l.leaderID(shardID) 203 if err != nil { 204 l.runtime.Logger().Debug("cannot get leader ID, skip truncate", 205 zap.Uint64("shard ID", shardID)) 206 return nil 207 } 208 if leaderID == 0 { 209 l.runtime.Logger().Debug("no leader yet, skip truncate", 210 zap.Uint64("shard ID", shardID)) 211 return nil 212 } 213 214 ctx, cancel := context.WithTimeout(ctx, 3*time.Second) 215 defer cancel() 216 217 lsnInSM, err := l.getTruncatedLsn(ctx, shardID) 218 if err != nil { 219 if !errors.Is(err, dragonboat.ErrTimeout) && !errors.Is(err, dragonboat.ErrInvalidDeadline) { 220 l.runtime.Logger().Error("get truncated lsn in state machine failed", 221 zap.Uint64("shard ID", shardID), zap.Error(err)) 222 return err 223 } 224 return nil 225 } 226 227 if !l.shouldProcess(shardID, lsnInSM) { 228 return nil 229 } 230 231 replicaID := uint64(l.getReplicaID(shardID)) 232 dir, lsn := l.snapshotMgr.EvalImportSnapshot(shardID, replicaID, lsnInSM) 233 if l.shouldDoImport(shardID, lsn, dir) { 234 if err := l.importSnapshot(ctx, shardID, replicaID, lsn, dir); err != nil { 235 l.runtime.Logger().Error("do truncate log failed", 236 zap.Uint64("shard ID", shardID), 237 zap.Uint64("replica ID", replicaID), 238 zap.Uint64("lsn", lsn), 239 zap.String("dir", dir), 240 zap.Error(err)) 241 return err 242 } 243 return nil 244 } 245 246 if l.shouldDoExport(ctx, shardID, replicaID) { 247 if err := l.exportSnapshot(ctx, shardID, replicaID); err != nil { 248 l.runtime.Logger().Error("export snapshot failed", 249 zap.Uint64("shard ID", shardID), 250 zap.Uint64("replica ID", replicaID), 251 zap.Uint64("lsn", lsn), 252 zap.Error(err)) 253 return err 254 } 255 } 256 return nil 257 } 258 259 // processHAKeeperTruncation processes the truncation for HAKeeper shard. 260 // It is different from the other shards. For HAKeeper shard, we just send 261 // snapshot request to dragonboat directly and nothing else needs to do. 262 func (l *store) processHAKeeperTruncation(ctx context.Context) error { 263 ctx, cancel := context.WithTimeout(ctx, time.Second*5) 264 defer cancel() 265 v, err := l.read(ctx, hakeeper.DefaultHAKeeperShardID, &hakeeper.IndexQuery{}) 266 if err != nil { 267 return err 268 } 269 lsn := v.(uint64) 270 if lsn > 1 { 271 opts := dragonboat.SnapshotOption{ 272 OverrideCompactionOverhead: true, 273 CompactionIndex: lsn - 1, 274 } 275 if _, err := l.nh.SyncRequestSnapshot(ctx, hakeeper.DefaultHAKeeperShardID, opts); err != nil { 276 l.runtime.Logger().Error("SyncRequestSnapshot failed", zap.Error(err)) 277 return err 278 } 279 l.runtime.Logger().Info("HAKeeper shard truncated", zap.Uint64("LSN", lsn)) 280 } 281 return nil 282 }