github.com/matrixorigin/matrixone@v0.7.0/pkg/vm/engine/tae/wal/wal_test.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 wal 16 17 import ( 18 "math/rand" 19 "strconv" 20 "sync" 21 "testing" 22 "time" 23 24 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 25 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver/batchstoredriver" 26 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/entry" 27 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/store" 28 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/testutils" 29 "github.com/panjf2000/ants/v2" 30 "github.com/stretchr/testify/assert" 31 ) 32 33 const ( 34 ModuleName = "TAEWAL" 35 ) 36 37 func initWal(t *testing.T) (Driver, string) { 38 dir := testutils.InitTestEnv(ModuleName, t) 39 cfg := &batchstoredriver.StoreCfg{ 40 RotateChecker: batchstoredriver.NewMaxSizeRotateChecker(int(common.K) * 2), 41 } 42 dcfg := &DriverConfig{ 43 BatchStoreConfig: cfg, 44 CheckpointDuration: time.Millisecond * 10, 45 } 46 driver := NewDriverWithBatchStore(dir, "store", dcfg) 47 return driver, dir 48 } 49 50 func restart(t *testing.T, driver Driver, dir string) Driver { 51 assert.NoError(t, driver.Close()) 52 cfg := &batchstoredriver.StoreCfg{ 53 RotateChecker: batchstoredriver.NewMaxSizeRotateChecker(int(common.K) * 2), 54 } 55 dcfg := &DriverConfig{ 56 BatchStoreConfig: cfg, 57 CheckpointDuration: time.Millisecond * 10, 58 } 59 driver = NewDriverWithBatchStore(dir, "store", dcfg) 60 return driver 61 } 62 63 func appendGroupC(t *testing.T, driver Driver, tid string) entry.Entry { 64 e := entry.GetBase() 65 info := &entry.Info{ 66 TxnId: tid, 67 } 68 err := e.SetPayload([]byte(strconv.Itoa(rand.Intn(10)))) 69 assert.NoError(t, err) 70 e.SetInfo(info) 71 _, err = driver.AppendEntry(GroupPrepare, e) 72 assert.NoError(t, err) 73 return e 74 } 75 76 func appendGroupUC(t *testing.T, driver Driver, tid string) entry.Entry { 77 e := entry.GetBase() 78 info := &entry.Info{ 79 Uncommits: tid, 80 } 81 err := e.SetPayload([]byte(strconv.Itoa(rand.Intn(10)))) 82 assert.NoError(t, err) 83 e.SetInfo(info) 84 _, err = driver.AppendEntry(GroupUC, e) 85 assert.NoError(t, err) 86 return e 87 } 88 89 func fuzzyCheckpointGroupC(t *testing.T, driver Driver, lsn uint64, offset, length, size uint32) entry.Entry { 90 if length == 0 { 91 panic("invalid length") 92 } 93 if offset+length > size { 94 panic("invalid size") 95 } 96 index := make([]*store.Index, 0) 97 for i := uint32(0); i < length; i++ { 98 idx := &store.Index{LSN: lsn, CSN: offset + i, Size: size} 99 index = append(index, idx) 100 } 101 e, err := driver.Checkpoint(index) 102 assert.NoError(t, err) 103 return e 104 } 105 106 func getCheckpointed(driver Driver, group uint32) (lsn uint64) { 107 dr := driver.(*walDriver) 108 lsn = dr.impl.GetCheckpointed(group) 109 return 110 } 111 112 func getCurrSeqNum(driver Driver, group uint32) (lsn uint64) { 113 dr := driver.(*walDriver) 114 lsn = dr.impl.GetCurrSeqNum(group) 115 return 116 } 117 118 func getLsn(e entry.Entry) (group uint32, lsn uint64) { 119 v := e.GetInfo() 120 if v == nil { 121 return 122 } 123 info := v.(*entry.Info) 124 return info.Group, info.GroupLSN 125 } 126 127 // append C, append UC 128 // ckp C 129 // check whether UC is checkpointed 130 func TestCheckpointUC(t *testing.T) { 131 defer testutils.AfterTest(t)() 132 driver, dir := initWal(t) 133 134 wg := &sync.WaitGroup{} 135 appendworker, _ := ants.NewPool(100) 136 ckpworker, _ := ants.NewPool(100) 137 defer appendworker.Release() 138 defer ckpworker.Release() 139 140 ckpfn := func(lsn uint64) func() { 141 return func() { 142 ckpEntry := fuzzyCheckpointGroupC(t, driver, lsn, 0, 2, 2) 143 assert.NoError(t, ckpEntry.WaitDone()) 144 ckpEntry.Free() 145 wg.Done() 146 } 147 } 148 appendfn := func(tid string) func() { 149 return func() { 150 uncommitEntry := appendGroupUC(t, driver, tid) 151 commitEntry := appendGroupC(t, driver, tid) 152 _, commitLsn := getLsn(commitEntry) 153 assert.NoError(t, uncommitEntry.WaitDone()) 154 assert.NoError(t, commitEntry.WaitDone()) 155 wg.Add(1) 156 _ = ckpworker.Submit(ckpfn(commitLsn)) 157 commitEntry.Free() 158 uncommitEntry.Free() 159 wg.Done() 160 } 161 } 162 163 for i := 0; i < 100; i++ { 164 wg.Add(1) 165 tid := common.NewTxnIDAllocator().Alloc() 166 _ = appendworker.Submit(appendfn(string(tid))) 167 } 168 wg.Wait() 169 170 driver = restart(t, driver, dir) 171 172 testutils.WaitExpect(4000, func() bool { 173 return getCurrSeqNum(driver, GroupUC) == getCheckpointed(driver, GroupUC) 174 }) 175 assert.Equal(t, getCurrSeqNum(driver, GroupUC), getCheckpointed(driver, GroupUC)) 176 177 assert.NoError(t, driver.Close()) 178 }