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