github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/logstore/driver/batchstoredriver/vfile.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 batchstoredriver 16 17 import ( 18 "bytes" 19 "errors" 20 "fmt" 21 "io" 22 "os" 23 "sync" 24 "sync/atomic" 25 26 "github.com/matrixorigin/matrixone/pkg/logutil" 27 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver/entry" 28 ) 29 30 var Metasize = 2 31 32 type vFileState struct { 33 bufPos int 34 pos int 35 file *vFile 36 } 37 38 type vFile struct { 39 *sync.RWMutex 40 *os.File 41 *vInfo 42 version int 43 committed atomic.Int32 44 size int //update when write 45 wg sync.WaitGroup 46 commitCond sync.Cond 47 history History 48 buf *bytes.Buffer 49 bufpos int //update when write 50 syncpos int //update when sync 51 52 } 53 54 func newVFile(mu *sync.RWMutex, name string, version int, history History) (*vFile, error) { 55 if mu == nil { 56 mu = new(sync.RWMutex) 57 } 58 file, err := os.Create(name) 59 if err != nil { 60 return nil, err 61 } 62 vf := &vFile{ 63 RWMutex: mu, 64 File: file, 65 version: version, 66 commitCond: *sync.NewCond(new(sync.Mutex)), 67 history: history, 68 buf: &bytes.Buffer{}, 69 } 70 vf.vInfo = newVInfo(vf) 71 return vf, nil 72 } 73 74 func (vf *vFile) String() string { 75 var w bytes.Buffer 76 w.WriteString(fmt.Sprintf("[%s]\n%s", vf.Name(), vf.vInfo.String())) 77 return w.String() 78 } 79 80 func (vf *vFile) Archive() error { 81 // if vf.history == nil { 82 // if err := vf.Destroy(); err != nil { 83 // return err 84 // } 85 // } 86 vf.history.Append(vf) 87 return nil 88 } 89 90 func (vf *vFile) Id() int { 91 return vf.version 92 } 93 94 func (vf *vFile) GetState() *vFileState { 95 vf.RLock() 96 defer vf.RUnlock() 97 return &vFileState{ 98 bufPos: vf.bufpos, 99 pos: vf.size, 100 file: vf, 101 } 102 } 103 104 func (vf *vFile) HasCommitted() bool { 105 return vf.committed.Load() == int32(1) 106 } 107 108 func (vf *vFile) PrepareWrite(size int) { 109 vf.wg.Add(1) 110 } 111 112 func (vf *vFile) FinishWrite() { 113 vf.wg.Done() 114 } 115 116 func (vf *vFile) Close() error { 117 // logutil.Infof("v%d addr is %v",vf.Id(),vf.Addrs) 118 err := vf.File.Close() 119 if err != nil { 120 return err 121 } 122 if vf.vInfo != nil { 123 vf.vInfo.close() 124 } 125 vf.buf = nil 126 return nil 127 } 128 129 func (vf *vFile) Commit() { 130 logutil.Debugf("Committing %s\n", vf.Name()) 131 vf.wg.Wait() 132 vf.flushWg.Wait() 133 err := vf.Sync() 134 if err != nil { 135 panic(err) 136 } 137 vf.Lock() 138 vf.buf = nil 139 vf.Unlock() 140 vf.commitCond.L.Lock() 141 vf.committed.Store(1) 142 vf.commitCond.Broadcast() 143 vf.commitCond.L.Unlock() 144 vf.vInfo.close() 145 // logutil.Infof("sync-%s\n", vf.String()) 146 // vf.FreeMeta() 147 } 148 149 func (vf *vFile) Sync() error { 150 vf.Lock() 151 defer vf.Unlock() 152 if vf.buf == nil { 153 err := vf.File.Sync() 154 return err 155 } 156 targetSize := vf.size 157 targetpos := vf.bufpos 158 buf := vf.buf.Bytes() 159 n, err := vf.File.WriteAt(buf[:targetpos], int64(vf.syncpos)) 160 if n != targetpos { 161 panic(fmt.Sprintf("logic err, expect %d, write %d err is %v", targetpos, n, err)) 162 } 163 if err != nil { 164 return err 165 } 166 // logutil.Infof("%p|sync [%v,%v](total%v|n=%d)\n", vf, vf.syncpos, vf.syncpos+vf.bufpos, vf.bufpos, n) 167 // buf := make([]byte, 10) 168 // _, err = vf.ReadAt(buf, int64(vf.syncpos)) 169 // logutil.Infof("%p|read at %v, buf is %v, n=%d, err is %v\n", vf, vf.syncpos, buf, n, err) 170 vf.syncpos += targetpos 171 // logutil.Infof("syncpos is %v\n", vf.syncpos) 172 if vf.syncpos != targetSize { 173 panic(fmt.Sprintf("%p|logic error, sync %v, size %v", vf, vf.syncpos, targetSize)) 174 } 175 vf.bufpos = 0 176 vf.buf.Reset() 177 // logutil.Infof("199bufpos is %v\n",vf.bufpos) 178 err = vf.File.Sync() 179 if err != nil { 180 return err 181 } 182 183 return nil 184 } 185 186 func (vf *vFile) WaitCommitted() { 187 if vf.committed.Load() == int32(1) { 188 return 189 } 190 vf.commitCond.L.Lock() 191 if vf.committed.Load() != int32(1) { 192 vf.commitCond.Wait() 193 } 194 vf.commitCond.L.Unlock() 195 } 196 197 func (vf *vFile) WriteAt(b []byte, off int64) (n int, err error) { 198 dataLength := len(b) 199 vf.Lock() 200 if vf.buf == nil { 201 vf.bufpos = 0 202 vf.buf = &bytes.Buffer{} 203 } 204 if dataLength+int(off)-vf.syncpos > vf.buf.Cap() { 205 vf.buf.Grow(dataLength + int(off) - vf.syncpos) 206 } 207 n, err = vf.buf.Write(b) 208 if err != nil { 209 panic(err) 210 } 211 // logutil.Infof("%p|write in buf[%v,%v]", vf, int(off)-vf.syncpos, int(off)-vf.syncpos+n) 212 // logutil.Infof("%p|write vf in buf [%v,%v]", vf, int(off), int(off)+n) 213 vf.bufpos = int(off) + n - vf.syncpos 214 vf.size += n 215 // logutil.Infof("%p|size is %v",vf,vf.size) 216 vf.Unlock() 217 // logutil.Infof("%p|bufpos is %v",vf,vf.bufpos) 218 if err != nil { 219 return 220 } 221 return 222 } 223 224 func (vf *vFile) Write(b []byte) (n int, err error) { 225 n, err = vf.File.Write(b) 226 if err != nil { 227 return 228 } 229 return 230 } 231 232 func (vf *vFile) SizeLocked() int { 233 vf.RLock() 234 defer vf.RUnlock() 235 return vf.size 236 } 237 238 func (vf *vFile) Destroy() error { 239 if err := vf.Close(); err != nil { 240 return err 241 } 242 name := vf.Name() 243 logutil.Debugf("Removing version file: %s", name) 244 err := os.Remove(name) 245 return err 246 } 247 248 func (vf *vFile) Replay(r *replayer) error { 249 for { 250 if err := r.replayHandler(vf); err != nil { 251 if errors.Is(err, io.EOF) { 252 break 253 } 254 return err 255 } 256 } 257 // logutil.Infof("v%d addr is %v",vf.Id(),vf.Addrs) 258 return nil 259 } 260 261 func (vf *vFile) OnLogInfo(info any) { 262 err := vf.Log(info) 263 if err != nil { 264 panic(err) 265 } 266 } 267 func (vf *vFile) Load(lsn uint64) (*entry.Entry, error) { 268 offset, err := vf.GetOffsetByLSN(lsn) 269 if err == ErrVFileGroupNotExist || err == ErrVFileLsnNotExist { 270 for i := 0; i < 10; i++ { 271 logutil.Debugf("load retry %d", lsn) 272 vf.addrCond.L.Lock() 273 offset, err = vf.GetOffsetByLSN(lsn) 274 if err == nil { 275 vf.addrCond.L.Unlock() 276 break 277 } 278 vf.addrCond.Wait() 279 vf.addrCond.L.Unlock() 280 offset, err = vf.GetOffsetByLSN(lsn) 281 if err == nil { 282 break 283 } 284 } 285 if err != nil { 286 return nil, ErrVFileVersionTimeOut 287 } 288 } 289 if err != nil { 290 return nil, err 291 } 292 return vf.readEntryAt(offset) 293 } 294 func (vf *vFile) LoadByOffset(offset int) (*entry.Entry, error) { 295 return vf.readEntryAt(offset) 296 } 297 func (vf *vFile) readEntryAt(offset int) (*entry.Entry, error) { 298 e := entry.NewEmptyEntry() 299 _, err := e.ReadAt(vf.File, offset) 300 return e, err 301 } 302 func (vf *vFile) OnReplayCommitted() { 303 vf.committed.Store(1) 304 }