github.com/vescale/zgraph@v0.0.0-20230410094002-959c02d50f95/storage/snapshot.go (about) 1 // Copyright 2022 zGraph Authors. All rights reserved. 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 storage 16 17 import ( 18 "context" 19 "sync" 20 21 "github.com/cenkalti/backoff" 22 "github.com/cockroachdb/pebble" 23 "github.com/vescale/zgraph/storage/kv" 24 "github.com/vescale/zgraph/storage/mvcc" 25 "github.com/vescale/zgraph/storage/resolver" 26 ) 27 28 // KVSnapshot represent the MVCC snapshot of the low-level key/value store. 29 // All values read from the KVSnapshot will be checked via mvcc.Version. 30 // And only the committed key/values can be retrieved or iterated. 31 type KVSnapshot struct { 32 db *pebble.DB 33 vp kv.VersionProvider 34 ver kv.Version 35 resolver *resolver.Scheduler 36 37 // The KVSnapshot instance may be accessed concurrently. 38 mu struct { 39 sync.RWMutex 40 resolved []kv.Version 41 } 42 } 43 44 // Get implements the Snapshot interface. 45 func (s *KVSnapshot) Get(_ context.Context, key kv.Key) ([]byte, error) { 46 var val []byte 47 err := backoff.RetryNotify(func() error { 48 v, err := s.get(key) 49 if err != nil { 50 lockedErr, ok := err.(*mvcc.LockedError) 51 if !ok { 52 return &backoff.PermanentError{Err: err} 53 } 54 55 // Try to resolve lock 56 status, err := resolver.CheckTxnStatus(s.db, s.vp, lockedErr.Primary, lockedErr.StartVer) 57 if err != nil { 58 return &backoff.PermanentError{Err: err} 59 } 60 switch status.Action { 61 case resolver.TxnActionNone: 62 // Transaction is still alive and try it letter. 63 return resolver.ErrRetryable("txn still alive") 64 65 case resolver.TxnActionTTLExpireRollback, 66 resolver.TxnActionLockNotExistRollback: 67 // Resolve the current key. 68 s.resolver.Resolve([]kv.Key{key}, lockedErr.StartVer, 0, nil) 69 // Put the resolve transaction into the resolved list to make the 70 // subsequent request bypass them. 71 s.mu.Lock() 72 s.mu.resolved = append(s.mu.resolved, lockedErr.StartVer) 73 s.mu.Unlock() 74 return resolver.ErrRetryable("bypass rollback transaction") 75 76 default: 77 // TxnActionLockNotExistDoNothing 78 // Transaction committed: we try to resolve the current key and backoff. 79 s.resolver.Resolve([]kv.Key{key}, lockedErr.StartVer, status.CommitVer, nil) 80 return resolver.ErrRetryable("resolving committed transaction") 81 } 82 } 83 val = v 84 return nil 85 }, expoBackoff(), BackoffErrReporter("KVSnapshot.Get")) 86 87 return val, err 88 } 89 90 // Iter implements the Snapshot interface. 91 func (s *KVSnapshot) Iter(lowerBound kv.Key, upperBound kv.Key) (kv.Iterator, error) { 92 // The lower-level database stored key-value with versions. We need 93 // to append the startVer to the raw keys. 94 var start, end mvcc.Key 95 if len(lowerBound) > 0 { 96 start = mvcc.Encode(lowerBound, mvcc.LockVer) 97 } 98 if len(upperBound) > 0 { 99 end = mvcc.Encode(upperBound, mvcc.LockVer) 100 } 101 102 inner := s.db.NewIter(&pebble.IterOptions{ 103 LowerBound: start, 104 UpperBound: end, 105 }) 106 107 // Ignore the return boolean value of positioning the cursor of the iterator 108 // to the first key/value. The inner iterator status of the field `valid` will 109 // be same as the returned value of `inner.First()`. So it will be checked 110 // while the `Next` method calling. 111 _ = inner.First() 112 113 iter := &SnapshotIter{ 114 db: s.db, 115 inner: inner, 116 resolver: s.resolver, 117 ver: s.ver, 118 } 119 120 // Handle startKey is nil, in this case, the real startKey 121 // should be changed the first key of the lower-level database. 122 if inner.Valid() { 123 key, _, err := mvcc.Decode(inner.Key()) 124 if err != nil { 125 // Close the inner SnapshotIter if error encountered. 126 _ = inner.Close() 127 return nil, err 128 } 129 iter.nextKey = key 130 } 131 132 return iter, iter.Next() 133 } 134 135 // IterReverse implements the Snapshot interface. 136 func (s *KVSnapshot) IterReverse(lowerBound kv.Key, upperBound kv.Key) (kv.Iterator, error) { 137 var start, end mvcc.Key 138 if len(lowerBound) > 0 { 139 start = mvcc.Encode(lowerBound, mvcc.LockVer) 140 } 141 if len(upperBound) > 0 { 142 end = mvcc.Encode(upperBound, mvcc.LockVer) 143 } 144 145 inner := s.db.NewIter(&pebble.IterOptions{ 146 LowerBound: start, 147 UpperBound: end, 148 }) 149 150 // Ignore the return boolean value of positioning the cursor of the iterator 151 // to the last key/value. The inner iterator status of the field `valid` will 152 // be same as the returned value of `inner.Last()`. So it will be checked 153 // while the `Next` method calling. 154 _ = inner.Last() 155 156 iter := &SnapshotIter{ 157 db: s.db, 158 reverse: true, 159 inner: inner, 160 resolver: s.resolver, 161 ver: s.ver, 162 } 163 164 // Set the next key to the last valid key between lowerBound and upperBound. 165 if inner.Valid() { 166 key, _, err := mvcc.Decode(inner.Key()) 167 if err != nil { 168 _ = inner.Close() 169 return nil, err 170 } 171 iter.nextKey = key 172 } 173 174 return iter, iter.Next() 175 } 176 177 func (s *KVSnapshot) BatchGet(_ context.Context, keys []kv.Key) (map[string][]byte, error) { 178 results := map[string][]byte{} 179 err := backoff.RetryNotify(func() error { 180 rollbacks := map[kv.Version][]kv.Key{} 181 committed := map[kv.VersionPair][]kv.Key{} 182 for _, key := range keys { 183 _, found := results[string(key)] 184 if found { 185 continue 186 } 187 188 value, err := s.get(key) 189 if err != nil { 190 lockedErr, ok := err.(*mvcc.LockedError) 191 if !ok { 192 return &backoff.PermanentError{Err: err} 193 } 194 195 // Try to resolve lock 196 status, err := resolver.CheckTxnStatus(s.db, s.vp, lockedErr.Primary, lockedErr.StartVer) 197 if err != nil { 198 return &backoff.PermanentError{Err: err} 199 } 200 switch status.Action { 201 case resolver.TxnActionNone: 202 // Transaction is still alive and try it letter. 203 continue 204 205 case resolver.TxnActionTTLExpireRollback, 206 resolver.TxnActionLockNotExistRollback: 207 // Resolve the current key. 208 rollbacks[lockedErr.StartVer] = append(rollbacks[lockedErr.StartVer], key) 209 continue 210 211 default: 212 // TxnActionLockNotExistDoNothing 213 // Transaction committed: we try to resolve the current key and backoff. 214 pair := kv.VersionPair{StartVer: lockedErr.StartVer, CommitVer: status.CommitVer} 215 committed[pair] = append(committed[pair], key) 216 continue 217 } 218 } 219 results[string(key)] = value 220 } 221 222 if len(rollbacks) > 0 { 223 for startVer, keys := range rollbacks { 224 s.resolver.Resolve(keys, startVer, 0, nil) 225 // Put the resolve transaction into the resolved list to make the 226 // subsequent request bypass them. 227 s.mu.Lock() 228 s.mu.resolved = append(s.mu.resolved, startVer) 229 s.mu.Unlock() 230 } 231 } 232 if len(committed) > 0 { 233 for pair, keys := range committed { 234 s.resolver.Resolve(keys, pair.StartVer, pair.CommitVer, nil) 235 } 236 } 237 238 if len(results) != len(keys) { 239 return resolver.ErrRetryable("some keys still resolving") 240 } 241 return nil 242 }, expoBackoff(), BackoffErrReporter("KVSnapshot.BatchGet")) 243 244 return results, err 245 } 246 247 // StartVer implements the Snapshot interface. 248 func (s *KVSnapshot) StartVer() kv.Version { 249 return s.ver 250 } 251 252 func (s *KVSnapshot) get(key kv.Key) ([]byte, error) { 253 opt := pebble.IterOptions{LowerBound: mvcc.Encode(key, mvcc.LockVer)} 254 iter := s.db.NewIter(&opt) 255 iter.First() 256 defer iter.Close() 257 258 s.mu.RLock() 259 resolved := s.mu.resolved 260 s.mu.RUnlock() 261 262 return getValue(iter, key, s.ver, resolved) 263 } 264 265 func getValue(iter *pebble.Iterator, key kv.Key, startVer kv.Version, resolvedLocks []kv.Version) ([]byte, error) { 266 dec1 := mvcc.LockDecoder{ExpectKey: key} 267 ok, err := dec1.Decode(iter) 268 if ok { 269 startVer, err = dec1.Lock.Check(startVer, key, resolvedLocks) 270 } 271 if err != nil { 272 return nil, err 273 } 274 dec2 := mvcc.ValueDecoder{ExpectKey: key} 275 for iter.Valid() { 276 ok, err := dec2.Decode(iter) 277 if err != nil { 278 return nil, err 279 } 280 if !ok { 281 break 282 } 283 284 value := &dec2.Value 285 if value.Type == mvcc.ValueTypeRollback || 286 value.Type == mvcc.ValueTypeLock { 287 continue 288 } 289 // Read the first committed value that can be seen at startVer. 290 if value.CommitVer <= startVer { 291 if value.Type == mvcc.ValueTypeDelete { 292 return nil, nil 293 } 294 return value.Value, nil 295 } 296 } 297 return nil, nil 298 }