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