github.com/matrixorigin/matrixone@v0.7.0/pkg/txn/storage/mem/kv_txn_storage.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 mem 16 17 import ( 18 "bytes" 19 "context" 20 "encoding/json" 21 "fmt" 22 "math" 23 "sync" 24 "time" 25 26 "github.com/matrixorigin/matrixone/pkg/common/moerr" 27 "github.com/matrixorigin/matrixone/pkg/logservice" 28 logpb "github.com/matrixorigin/matrixone/pkg/pb/logservice" 29 "github.com/matrixorigin/matrixone/pkg/pb/timestamp" 30 "github.com/matrixorigin/matrixone/pkg/pb/txn" 31 "github.com/matrixorigin/matrixone/pkg/txn/clock" 32 "github.com/matrixorigin/matrixone/pkg/txn/storage" 33 ) 34 35 // MustParseGetPayload must parse get payload 36 func MustParseGetPayload(payload []byte) [][]byte { 37 r := &message{} 38 r.mustUnmarshal(payload) 39 return r.Values 40 } 41 42 // NewSetTxnRequest returns a kv set txn request 43 func NewSetTxnRequest(ks, vs [][]byte) txn.TxnRequest { 44 r := &message{Keys: ks, Values: vs} 45 return txn.TxnRequest{ 46 Method: txn.TxnMethod_Write, 47 CNRequest: &txn.CNOpRequest{ 48 OpCode: setOpCode, 49 Payload: r.mustMarshal(), 50 }, 51 } 52 } 53 54 // NewGetTxnRequest returns a kv get txn request 55 func NewGetTxnRequest(ks [][]byte) txn.TxnRequest { 56 r := &message{Keys: ks} 57 return txn.TxnRequest{ 58 Method: txn.TxnMethod_Read, 59 CNRequest: &txn.CNOpRequest{ 60 OpCode: getOpCode, 61 Payload: r.mustMarshal(), 62 }, 63 } 64 } 65 66 // EventType event type 67 type EventType int 68 69 var ( 70 // PrepareType prepare event 71 PrepareType = EventType(0) 72 // CommitType commit event 73 CommitType = EventType(1) 74 // CommittingType committing type 75 CommittingType = EventType(2) 76 // RollbackType rollback type 77 RollbackType = EventType(3) 78 ) 79 80 // Event event 81 type Event struct { 82 // Txn event txn 83 Txn txn.TxnMeta 84 // Type event type 85 Type EventType 86 } 87 88 // KVTxnStorage KV-based implementation of TxnStorage. Just used to test. 89 type KVTxnStorage struct { 90 sync.RWMutex 91 logClient logservice.Client 92 clock clock.Clock 93 recoverFrom logservice.Lsn 94 uncommittedTxn map[string]*txn.TxnMeta 95 uncommittedKeyTxnMap map[string]*txn.TxnMeta 96 uncommitted *KV 97 committed *MVCCKV 98 eventC chan Event 99 } 100 101 // NewKVTxnStorage create KV-based implementation of TxnStorage 102 func NewKVTxnStorage(recoverFrom logservice.Lsn, logClient logservice.Client, clock clock.Clock) *KVTxnStorage { 103 return &KVTxnStorage{ 104 logClient: logClient, 105 clock: clock, 106 recoverFrom: recoverFrom, 107 uncommittedKeyTxnMap: make(map[string]*txn.TxnMeta), 108 uncommittedTxn: make(map[string]*txn.TxnMeta), 109 uncommitted: NewKV(), 110 committed: NewMVCCKV(), 111 eventC: make(chan Event, 1024*10), 112 } 113 } 114 115 func (kv *KVTxnStorage) GetEventC() chan Event { 116 return kv.eventC 117 } 118 119 func (kv *KVTxnStorage) GetUncommittedTxn(txnID []byte) *txn.TxnMeta { 120 kv.RLock() 121 defer kv.RUnlock() 122 123 return kv.uncommittedTxn[string(txnID)] 124 } 125 126 func (kv *KVTxnStorage) GetCommittedKV() *MVCCKV { 127 return kv.committed 128 } 129 130 func (kv *KVTxnStorage) GetUncommittedKV() *KV { 131 return kv.uncommitted 132 } 133 134 func (kv *KVTxnStorage) StartRecovery(ctx context.Context, c chan txn.TxnMeta) { 135 defer close(c) 136 137 if kv.recoverFrom < 1 { 138 return 139 } 140 141 for { 142 logs, lsn, err := kv.logClient.Read(ctx, kv.recoverFrom, math.MaxUint64) 143 if err != nil { 144 panic(err) 145 } 146 147 for _, log := range logs { 148 if log.Type == logpb.UserRecord { 149 klog := &KVLog{} 150 klog.MustUnmarshal(log.Data) 151 152 switch klog.Txn.Status { 153 case txn.TxnStatus_Prepared: 154 req := &message{} 155 req.Keys = klog.Keys 156 req.Values = klog.Values 157 _, err := kv.Write(ctx, klog.Txn, setOpCode, req.mustMarshal()) 158 if err != nil { 159 panic(err) 160 } 161 case txn.TxnStatus_Committed: 162 kv.Lock() 163 if len(klog.Keys) == 0 { 164 kv.commitKeysLocked(klog.Txn, kv.getWriteKeysLocked(klog.Txn)) 165 } else { 166 kv.commitWithKVLogLocked(klog) 167 } 168 kv.Unlock() 169 case txn.TxnStatus_Committing: 170 kv.Lock() 171 newTxn := kv.changeUncommittedTxnStatusLocked(klog.Txn.ID, txn.TxnStatus_Committing) 172 newTxn.CommitTS = klog.Txn.CommitTS 173 kv.Unlock() 174 default: 175 panic(fmt.Sprintf("invalid txn status %s", klog.Txn.Status.String())) 176 } 177 178 c <- klog.Txn 179 } 180 } 181 182 if lsn == kv.recoverFrom { 183 return 184 } 185 } 186 } 187 188 func (kv *KVTxnStorage) Start() error { 189 return nil 190 } 191 192 func (kv *KVTxnStorage) Close(ctx context.Context) error { 193 return nil 194 } 195 196 func (kv *KVTxnStorage) Destroy(ctx context.Context) error { 197 return nil 198 } 199 200 func (kv *KVTxnStorage) Read(ctx context.Context, txnMeta txn.TxnMeta, op uint32, payload []byte) (storage.ReadResult, error) { 201 kv.RLock() 202 defer kv.RUnlock() 203 204 req := &message{} 205 req.mustUnmarshal(payload) 206 207 result := newReadResult(req.Keys, txnMeta, kv.continueRead) 208 for idx, key := range req.Keys { 209 if t, ok := kv.uncommittedKeyTxnMap[string(key)]; ok && needWait(*t, txnMeta) { 210 result.waitTxns = append(result.waitTxns, t.ID) 211 result.unreaded = append(result.unreaded, idx) 212 continue 213 } 214 215 result.values[idx] = kv.readValue(key, txnMeta) 216 } 217 return result, nil 218 } 219 220 func (kv *KVTxnStorage) continueRead(rs *readResult) bool { 221 kv.RLock() 222 defer kv.RUnlock() 223 224 if len(rs.unreaded) == 0 { 225 return true 226 } 227 228 for _, idx := range rs.unreaded { 229 key := rs.keys[idx] 230 txnMeta := rs.txnMeta 231 if t, ok := kv.uncommittedKeyTxnMap[string(key)]; ok && needWait(*t, txnMeta) { 232 return false 233 } 234 235 rs.values[idx] = kv.readValue(key, txnMeta) 236 } 237 return true 238 } 239 240 func (kv *KVTxnStorage) readValue(key []byte, txnMeta txn.TxnMeta) []byte { 241 if t, ok := kv.uncommittedKeyTxnMap[string(key)]; ok && bytes.Equal(t.ID, txnMeta.ID) { 242 if v, ok := kv.uncommitted.Get(key); ok { 243 return v 244 } 245 } 246 247 var value []byte 248 kv.committed.AscendRange(key, timestamp.Timestamp{}, txnMeta.SnapshotTS, func(v []byte, _ timestamp.Timestamp) { 249 value = v 250 }) 251 return value 252 } 253 254 func (kv *KVTxnStorage) Write(ctx context.Context, txnMeta txn.TxnMeta, op uint32, payload []byte) ([]byte, error) { 255 kv.Lock() 256 defer kv.Unlock() 257 258 req := &message{} 259 req.mustUnmarshal(payload) 260 261 newTxn := txnMeta 262 for idx, key := range req.Keys { 263 if t, ok := kv.uncommittedKeyTxnMap[string(key)]; ok { 264 if !bytes.Equal(t.ID, txnMeta.ID) { 265 return nil, moerr.NewTxnWriteConflictNoCtx("%s %s", t.ID, txnMeta.ID) 266 } 267 } else { 268 kv.uncommittedKeyTxnMap[string(key)] = &newTxn 269 } 270 271 if _, ok := kv.uncommittedTxn[string(txnMeta.ID)]; !ok { 272 kv.uncommittedTxn[string(txnMeta.ID)] = &newTxn 273 } 274 275 kv.uncommitted.Set(key, req.Values[idx]) 276 } 277 return nil, nil 278 } 279 280 func (kv *KVTxnStorage) Prepare(ctx context.Context, txnMeta txn.TxnMeta) (timestamp.Timestamp, error) { 281 kv.Lock() 282 defer kv.Unlock() 283 284 if _, ok := kv.uncommittedTxn[string(txnMeta.ID)]; !ok { 285 return timestamp.Timestamp{}, moerr.NewMissingTxnNoCtx() 286 } 287 288 txnMeta.PreparedTS, _ = kv.clock.Now() 289 writeKeys := kv.getWriteKeysLocked(txnMeta) 290 if kv.hasConflict(txnMeta.SnapshotTS, 291 timestamp.Timestamp{PhysicalTime: math.MaxInt64, LogicalTime: math.MaxUint32}, 292 writeKeys) { 293 return timestamp.Timestamp{}, moerr.NewTxnWriteConflictNoCtx("") 294 } 295 296 log := kv.getLogWithDataLocked(txnMeta) 297 log.Txn.Status = txn.TxnStatus_Prepared 298 lsn, err := kv.saveLog(log) 299 if err != nil { 300 return timestamp.Timestamp{}, err 301 } 302 303 newTxn := kv.changeUncommittedTxnStatusLocked(txnMeta.ID, txn.TxnStatus_Prepared) 304 newTxn.PreparedTS = txnMeta.PreparedTS 305 newTxn.DNShards = txnMeta.DNShards 306 kv.recoverFrom = lsn 307 kv.eventC <- Event{Txn: *newTxn, Type: PrepareType} 308 return txnMeta.PreparedTS, nil 309 } 310 311 func (kv *KVTxnStorage) Committing(ctx context.Context, txnMeta txn.TxnMeta) error { 312 kv.Lock() 313 defer kv.Unlock() 314 315 if _, ok := kv.uncommittedTxn[string(txnMeta.ID)]; !ok { 316 return moerr.NewMissingTxnNoCtx() 317 } 318 319 log := &KVLog{Txn: txnMeta} 320 log.Txn.Status = txn.TxnStatus_Committing 321 lsn, err := kv.saveLog(log) 322 if err != nil { 323 return err 324 } 325 326 newTxn := kv.changeUncommittedTxnStatusLocked(txnMeta.ID, txn.TxnStatus_Committing) 327 newTxn.CommitTS = txnMeta.CommitTS 328 kv.recoverFrom = lsn 329 kv.eventC <- Event{Txn: *newTxn, Type: CommittingType} 330 return nil 331 } 332 333 func (kv *KVTxnStorage) Commit(ctx context.Context, txnMeta txn.TxnMeta) error { 334 kv.Lock() 335 defer kv.Unlock() 336 337 if _, ok := kv.uncommittedTxn[string(txnMeta.ID)]; !ok { 338 return nil 339 } 340 341 writeKeys := kv.getWriteKeysLocked(txnMeta) 342 if kv.hasConflict(txnMeta.SnapshotTS, txnMeta.CommitTS.Next(), writeKeys) { 343 return moerr.NewTxnWriteConflictNoCtx("") 344 } 345 346 var log *KVLog 347 if txnMeta.Status == txn.TxnStatus_Active { 348 log = kv.getLogWithDataLocked(txnMeta) 349 } else if txnMeta.Status == txn.TxnStatus_Prepared || 350 txnMeta.Status == txn.TxnStatus_Committing { 351 log = &KVLog{Txn: txnMeta} 352 } else { 353 panic(fmt.Sprintf("commit with invalid status: %s", txnMeta.Status)) 354 } 355 log.Txn.Status = txn.TxnStatus_Committed 356 lsn, err := kv.saveLog(log) 357 if err != nil { 358 return err 359 } 360 361 kv.commitKeysLocked(txnMeta, writeKeys) 362 kv.recoverFrom = lsn 363 kv.eventC <- Event{Txn: log.Txn, Type: CommitType} 364 return nil 365 } 366 367 func (kv *KVTxnStorage) Rollback(ctx context.Context, txnMeta txn.TxnMeta) error { 368 kv.Lock() 369 defer kv.Unlock() 370 371 if _, ok := kv.uncommittedTxn[string(txnMeta.ID)]; !ok { 372 return nil 373 } 374 375 var writeKeys [][]byte 376 for k, v := range kv.uncommittedKeyTxnMap { 377 if bytes.Equal(v.ID, txnMeta.ID) { 378 writeKeys = append(writeKeys, []byte(k)) 379 } 380 } 381 382 for _, key := range writeKeys { 383 kv.uncommitted.Delete(key) 384 delete(kv.uncommittedKeyTxnMap, string(key)) 385 } 386 387 delete(kv.uncommittedTxn, string(txnMeta.ID)) 388 kv.eventC <- Event{Txn: txnMeta, Type: RollbackType} 389 return nil 390 } 391 392 func (kv *KVTxnStorage) Debug(ctx context.Context, meta txn.TxnMeta, op uint32, data []byte) ([]byte, error) { 393 return data, nil 394 } 395 396 func (kv *KVTxnStorage) getLogWithDataLocked(txnMeta txn.TxnMeta) *KVLog { 397 log := &KVLog{Txn: txnMeta} 398 for k, v := range kv.uncommittedKeyTxnMap { 399 if bytes.Equal(v.ID, txnMeta.ID) { 400 log.Keys = append(log.Keys, []byte(k)) 401 } 402 } 403 if len(log.Keys) == 0 { 404 panic("commit empty write set") 405 } 406 407 for _, key := range log.Keys { 408 v, ok := kv.uncommitted.Get(key) 409 if !ok { 410 panic("missing write set") 411 } 412 413 log.Values = append(log.Values, v) 414 } 415 return log 416 } 417 418 func (kv *KVTxnStorage) hasConflict(from, to timestamp.Timestamp, writeKeys [][]byte) bool { 419 for _, key := range writeKeys { 420 n := 0 421 kv.committed.AscendRange(key, from, to, func(_ []byte, _ timestamp.Timestamp) { 422 n++ 423 }) 424 if n > 0 { 425 return true 426 } 427 } 428 return false 429 } 430 431 func (kv *KVTxnStorage) getWriteKeysLocked(txnMeta txn.TxnMeta) [][]byte { 432 var writeKeys [][]byte 433 for k, v := range kv.uncommittedKeyTxnMap { 434 if bytes.Equal(v.ID, txnMeta.ID) { 435 writeKeys = append(writeKeys, []byte(k)) 436 } 437 } 438 return writeKeys 439 } 440 441 func (kv *KVTxnStorage) saveLog(log *KVLog) (logservice.Lsn, error) { 442 ctx, cancel := context.WithTimeout(context.Background(), time.Minute) 443 defer cancel() 444 data := log.MustMarshal() 445 record := kv.logClient.GetLogRecord(len(data)) 446 if len(record.Data) == 0 { 447 record.Data = data 448 } else { 449 copy(record.Data[len(record.Data)-len(data):], data) 450 } 451 return kv.logClient.Append(ctx, record) 452 } 453 454 func (kv *KVTxnStorage) commitWithKVLogLocked(klog *KVLog) { 455 for idx := range klog.Keys { 456 key := klog.Keys[idx] 457 value := klog.Values[idx] 458 kv.committed.Set(key, klog.Txn.CommitTS, value) 459 } 460 } 461 462 func (kv *KVTxnStorage) commitKeysLocked(txnMeta txn.TxnMeta, keys [][]byte) { 463 for _, key := range keys { 464 v, ok := kv.uncommitted.Get(key) 465 if !ok { 466 panic("missing write set") 467 } 468 469 kv.uncommitted.Delete(key) 470 delete(kv.uncommittedKeyTxnMap, string(key)) 471 kv.committed.Set(key, txnMeta.CommitTS, v) 472 } 473 delete(kv.uncommittedTxn, string(txnMeta.ID)) 474 } 475 476 func (kv *KVTxnStorage) changeUncommittedTxnStatusLocked(id []byte, status txn.TxnStatus) *txn.TxnMeta { 477 newTxn := kv.uncommittedTxn[string(id)] 478 newTxn.Status = status 479 return newTxn 480 } 481 482 func needWait(writeTxn, readTxn txn.TxnMeta) bool { 483 if bytes.Equal(writeTxn.ID, readTxn.ID) { 484 return false 485 } 486 487 switch writeTxn.Status { 488 case txn.TxnStatus_Prepared: 489 return readTxn.SnapshotTS.Greater(writeTxn.PreparedTS) 490 case txn.TxnStatus_Committing: 491 return readTxn.SnapshotTS.Greater(writeTxn.CommitTS) 492 } 493 return false 494 } 495 496 var ( 497 setOpCode uint32 = 1 498 getOpCode uint32 = 2 499 ) 500 501 type message struct { 502 Keys [][]byte `json:"key,omitempty"` 503 Values [][]byte `json:"value,omitempty"` 504 } 505 506 func (r *message) mustUnmarshal(payload []byte) { 507 if err := json.Unmarshal(payload, r); err != nil { 508 panic(err) 509 } 510 } 511 512 func (r *message) mustMarshal() []byte { 513 v, err := json.Marshal(r) 514 if err != nil { 515 panic(err) 516 } 517 return v 518 } 519 520 type readResult struct { 521 txnMeta txn.TxnMeta 522 keys [][]byte 523 waitTxns [][]byte 524 values [][]byte 525 unreaded []int 526 continueReadFunc func(rs *readResult) bool 527 } 528 529 func newReadResult(keys [][]byte, txnMeta txn.TxnMeta, continueReadFunc func(rs *readResult) bool) *readResult { 530 return &readResult{ 531 keys: keys, 532 values: make([][]byte, len(keys)), 533 continueReadFunc: continueReadFunc, 534 txnMeta: txnMeta, 535 } 536 } 537 538 func (rs *readResult) WaitTxns() [][]byte { 539 return rs.waitTxns 540 } 541 542 func (rs *readResult) Read() ([]byte, error) { 543 if !rs.continueReadFunc(rs) { 544 return nil, moerr.NewMissingTxnNoCtx() 545 } 546 547 resp := &message{Values: rs.values} 548 return resp.mustMarshal(), nil 549 } 550 551 func (rs *readResult) Release() { 552 553 }