github.com/matrixorigin/matrixone@v0.7.0/pkg/vm/engine/tae/logstore/store/store.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 store 16 17 import ( 18 "sync" 19 20 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 21 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver" 22 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver/batchstoredriver" 23 driverEntry "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver/entry" 24 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver/logservicedriver" 25 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/entry" 26 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/sm" 27 ) 28 29 var DefaultMaxBatchSize = 10000 30 31 type StoreImpl struct { 32 *StoreInfo 33 common.ClosedState 34 35 driver driver.Driver 36 37 appendWg sync.WaitGroup 38 appendMu sync.RWMutex 39 driverAppendQueue sm.Queue 40 doneWithErrQueue sm.Queue 41 logInfoQueue sm.Queue 42 43 checkpointQueue sm.Queue 44 45 truncatingQueue sm.Queue 46 truncateQueue sm.Queue 47 } 48 49 func NewStoreWithLogserviceDriver(factory logservicedriver.LogServiceClientFactory) Store { 50 cfg := logservicedriver.NewDefaultConfig(factory) 51 driver := logservicedriver.NewLogServiceDriver(cfg) 52 return NewStore(driver) 53 } 54 55 func NewStoreWithBatchStoreDriver(dir, name string, cfg *batchstoredriver.StoreCfg) Store { 56 driver, err := batchstoredriver.NewBaseStore(dir, name, cfg) 57 if err != nil { 58 panic(err) 59 } 60 return NewStore(driver) 61 } 62 func NewStore(driver driver.Driver) *StoreImpl { 63 w := &StoreImpl{ 64 StoreInfo: newWalInfo(), 65 driver: driver, 66 appendWg: sync.WaitGroup{}, 67 appendMu: sync.RWMutex{}, 68 } 69 w.driverAppendQueue = sm.NewSafeQueue(DefaultMaxBatchSize*10, DefaultMaxBatchSize, w.onDriverAppendQueue) 70 w.doneWithErrQueue = sm.NewSafeQueue(DefaultMaxBatchSize*10, DefaultMaxBatchSize, w.onDoneWithErrQueue) 71 w.logInfoQueue = sm.NewSafeQueue(DefaultMaxBatchSize*10, DefaultMaxBatchSize, w.onLogInfoQueue) 72 w.checkpointQueue = sm.NewSafeQueue(DefaultMaxBatchSize*10, DefaultMaxBatchSize, w.onLogCKPInfoQueue) 73 w.truncatingQueue = sm.NewSafeQueue(DefaultMaxBatchSize*10, DefaultMaxBatchSize, w.onTruncatingQueue) 74 w.truncateQueue = sm.NewSafeQueue(DefaultMaxBatchSize*10, DefaultMaxBatchSize, w.onTruncateQueue) 75 w.Start() 76 return w 77 } 78 func (w *StoreImpl) Start() { 79 w.driverAppendQueue.Start() 80 w.doneWithErrQueue.Start() 81 w.logInfoQueue.Start() 82 w.checkpointQueue.Start() 83 w.truncatingQueue.Start() 84 w.truncateQueue.Start() 85 } 86 func (w *StoreImpl) Close() error { 87 if !w.TryClose() { 88 return nil 89 } 90 w.appendMu.RLock() 91 w.appendWg.Wait() 92 w.appendMu.RUnlock() 93 w.driverAppendQueue.Stop() 94 w.doneWithErrQueue.Stop() 95 w.logInfoQueue.Stop() 96 w.checkpointQueue.Stop() 97 w.truncatingQueue.Stop() 98 w.truncateQueue.Stop() 99 err := w.driver.Close() 100 if err != nil { 101 return err 102 } 103 return nil 104 } 105 func (w *StoreImpl) Append(gid uint32, e entry.Entry) (lsn uint64, err error) { 106 _, lsn, err = w.doAppend(gid, e) 107 return 108 } 109 110 func (w *StoreImpl) doAppend(gid uint32, e entry.Entry) (drEntry *driverEntry.Entry, lsn uint64, err error) { 111 if w.IsClosed() { 112 return nil, 0, common.ErrClose 113 } 114 w.appendMu.Lock() 115 defer w.appendMu.Unlock() 116 w.appendWg.Add(1) 117 if w.IsClosed() { 118 w.appendWg.Done() 119 return nil, 0, common.ErrClose 120 } 121 lsn = w.allocateLsn(gid) 122 v1 := e.GetInfo() 123 var info *entry.Info 124 if v1 == nil { 125 info = &entry.Info{} 126 e.SetInfo(info) 127 } else { 128 info = v1.(*entry.Info) 129 } 130 info.Group = gid 131 info.GroupLSN = lsn 132 drEntry = driverEntry.NewEntry(e) 133 // e.DoneWithErr(nil) 134 // return 135 _, err = w.driverAppendQueue.Enqueue(drEntry) 136 if err != nil { 137 panic(err) 138 } 139 return 140 } 141 142 func (w *StoreImpl) onDriverAppendQueue(items ...any) { 143 for _, item := range items { 144 driverEntry := item.(*driverEntry.Entry) 145 driverEntry.Entry.PrepareWrite() 146 err := w.driver.Append(driverEntry) 147 if err != nil { 148 panic(err) 149 } 150 // driverEntry.Entry.DoneWithErr(nil) 151 _, err = w.doneWithErrQueue.Enqueue(driverEntry) 152 if err != nil { 153 panic(err) 154 } 155 } 156 } 157 158 func (w *StoreImpl) onDoneWithErrQueue(items ...any) { 159 for _, item := range items { 160 e := item.(*driverEntry.Entry) 161 err := e.WaitDone() 162 if err != nil { 163 panic(err) 164 } 165 e.Entry.DoneWithErr(nil) 166 _, err = w.logInfoQueue.Enqueue(e) 167 if err != nil { 168 panic(err) 169 } 170 } 171 w.appendWg.Add(-len(items)) 172 } 173 174 func (w *StoreImpl) onLogInfoQueue(items ...any) { 175 for _, item := range items { 176 e := item.(*driverEntry.Entry) 177 w.logDriverLsn(e) 178 } 179 w.onAppend() 180 }