github.com/pingcap/badger@v1.5.1-0.20230103063557-828f39b09b6d/value.go (about) 1 /* 2 * Copyright 2017 Dgraph Labs, Inc. and Contributors 3 * 4 * Licensed under the Apache License, Version 2.0 (the "License"); 5 * you may not use this file except in compliance with the License. 6 * You may obtain a copy of the License at 7 * 8 * http://www.apache.org/licenses/LICENSE-2.0 9 * 10 * Unless required by applicable law or agreed to in writing, software 11 * distributed under the License is distributed on an "AS IS" BASIS, 12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 * See the License for the specific language governing permissions and 14 * limitations under the License. 15 */ 16 17 package badger 18 19 import ( 20 "bufio" 21 "bytes" 22 "encoding/binary" 23 "fmt" 24 "hash/crc32" 25 "io" 26 "io/ioutil" 27 "os" 28 "sort" 29 "strconv" 30 "strings" 31 "sync" 32 "sync/atomic" 33 34 "github.com/pingcap/badger/fileutil" 35 "github.com/pingcap/badger/y" 36 "github.com/pingcap/errors" 37 ) 38 39 // Values have their first byte being byteData or byteDelete. This helps us distinguish between 40 // a key that has never been seen and a key that has been explicitly deleted. 41 const ( 42 bitDelete byte = 1 << 0 // Set if the key has been deleted. 43 bitValuePointer byte = 1 << 1 // Set if the value is NOT stored directly next to key. 44 45 // The MSB 2 bits are for transactions. 46 bitTxn byte = 1 << 6 // Set if the entry is part of a txn. 47 bitFinTxn byte = 1 << 7 // Set if the entry is to indicate end of txn in value log. 48 49 mi int64 = 1 << 20 50 ) 51 52 type logFile struct { 53 path string 54 fd *os.File 55 fid uint32 56 size uint32 57 } 58 59 // openReadOnly assumes that we have a write lock on logFile. 60 func (lf *logFile) openReadOnly() error { 61 var err error 62 lf.fd, err = os.OpenFile(lf.path, os.O_RDONLY, 0666) 63 if err != nil { 64 return errors.Wrapf(err, "Unable to open %q as RDONLY.", lf.path) 65 } 66 67 fi, err := lf.fd.Stat() 68 if err != nil { 69 return errors.Wrapf(err, "Unable to check stat for %q", lf.path) 70 } 71 lf.size = uint32(fi.Size()) 72 return nil 73 } 74 75 func (lf *logFile) doneWriting(offset uint32) error { 76 if err := lf.fd.Truncate(int64(offset)); err != nil { 77 return errors.Wrapf(err, "Unable to truncate file: %q", lf.path) 78 } 79 if err := fileutil.Fsync(lf.fd); err != nil { 80 return errors.Wrapf(err, "Unable to sync value log: %q", lf.path) 81 } 82 if err := lf.fd.Close(); err != nil { 83 return errors.Wrapf(err, "Unable to close value log: %q", lf.path) 84 } 85 86 return lf.openReadOnly() 87 } 88 89 // You must hold lf.lock to sync() 90 func (lf *logFile) sync() error { 91 return fileutil.Fsync(lf.fd) 92 } 93 94 var errStop = errors.New("Stop iteration") 95 var errTruncate = errors.New("Do truncate") 96 97 type logEntry func(e Entry) error 98 99 type safeRead struct { 100 k []byte 101 v []byte 102 um []byte 103 104 recordOffset uint32 105 } 106 107 func (r *safeRead) Entry(reader *bufio.Reader) (*Entry, error) { 108 var hbuf [headerBufSize]byte 109 var err error 110 111 hash := crc32.New(y.CastagnoliCrcTable) 112 tee := io.TeeReader(reader, hash) 113 if _, err = io.ReadFull(tee, hbuf[:]); err != nil { 114 return nil, err 115 } 116 117 // Encounter preallocated region, just act as EOF. 118 if !isEncodedHeader(hbuf[:]) { 119 return nil, io.EOF 120 } 121 122 var h header 123 h.Decode(hbuf[:]) 124 if h.klen > maxKeySize { 125 return nil, errTruncate 126 } 127 kl := int(h.klen) 128 if cap(r.k) < kl { 129 r.k = make([]byte, 2*kl) 130 } 131 vl := int(h.vlen) 132 if cap(r.v) < vl { 133 r.v = make([]byte, 2*vl) 134 } 135 136 e := &Entry{} 137 e.offset = r.recordOffset 138 e.Key.UserKey = r.k[:kl] 139 e.Key.Version = h.ver 140 e.Value = r.v[:vl] 141 if h.umlen > 0 { 142 if cap(r.um) < int(h.umlen) { 143 r.um = make([]byte, 2*h.umlen) 144 } 145 e.UserMeta = r.um[:h.umlen] 146 if _, err = io.ReadFull(tee, e.UserMeta); err != nil { 147 if err == io.EOF { 148 err = errTruncate 149 } 150 return nil, err 151 } 152 } 153 154 if _, err = io.ReadFull(tee, r.k[:kl]); err != nil { 155 if err == io.EOF { 156 err = errTruncate 157 } 158 return nil, err 159 } 160 if _, err = io.ReadFull(tee, e.Value); err != nil { 161 if err == io.EOF { 162 err = errTruncate 163 } 164 return nil, err 165 } 166 var crcBuf [4]byte 167 if _, err = io.ReadFull(reader, crcBuf[:]); err != nil { 168 if err == io.EOF { 169 err = errTruncate 170 } 171 return nil, err 172 } 173 crc := binary.BigEndian.Uint32(crcBuf[:]) 174 if crc != hash.Sum32() { 175 return nil, errTruncate 176 } 177 e.meta = h.meta 178 return e, nil 179 } 180 181 // iterate iterates over log file. It doesn't not allocate new memory for every kv pair. 182 // Therefore, the kv pair is only valid for the duration of fn call. 183 func (vlog *valueLog) iterate(lf *logFile, offset uint32, fn logEntry) (uint32, error) { 184 _, err := lf.fd.Seek(int64(offset), io.SeekStart) 185 if err != nil { 186 return 0, y.Wrap(err) 187 } 188 189 reader := bufio.NewReader(lf.fd) 190 read := &safeRead{ 191 k: make([]byte, 10), 192 v: make([]byte, 10), 193 recordOffset: offset, 194 } 195 196 var lastCommit uint64 197 validEndOffset := read.recordOffset 198 for { 199 e, err := read.Entry(reader) 200 if err == io.EOF { 201 break 202 } else if err == io.ErrUnexpectedEOF || err == errTruncate { 203 break 204 } else if err != nil { 205 return validEndOffset, err 206 } else if e == nil { 207 continue 208 } 209 210 read.recordOffset += uint32(headerBufSize + len(e.Key.UserKey) + len(e.Value) + len(e.UserMeta) + 4) // len(crcBuf) 211 212 if e.meta&bitTxn > 0 { 213 if !vlog.kv.IsManaged() { 214 txnTs := e.Key.Version 215 if lastCommit == 0 { 216 lastCommit = txnTs 217 } 218 if lastCommit != txnTs { 219 break 220 } 221 } 222 } else if e.meta&bitFinTxn > 0 { 223 if !vlog.kv.IsManaged() { 224 txnTs, err := strconv.ParseUint(string(e.Value), 10, 64) 225 if err != nil || lastCommit != txnTs { 226 break 227 } 228 // Got the end of txn. Now we can store them. 229 lastCommit = 0 230 } 231 validEndOffset = read.recordOffset 232 } else { 233 if !vlog.kv.IsManaged() { 234 if lastCommit != 0 { 235 // This is most likely an entry which was moved as part of GC. 236 // We shouldn't get this entry in the middle of a transaction. 237 break 238 } 239 } 240 validEndOffset = read.recordOffset 241 } 242 243 if vlog.opt.ReadOnly { 244 return validEndOffset, ErrReplayNeeded 245 } 246 if err := fn(*e); err != nil { 247 if err == errStop { 248 break 249 } 250 return validEndOffset, y.Wrap(err) 251 } 252 } 253 254 return validEndOffset, nil 255 } 256 257 func (vlog *valueLog) deleteLogFile(lf *logFile) error { 258 path := vlog.fpath(lf.fid) 259 if err := lf.fd.Close(); err != nil { 260 return err 261 } 262 return os.Remove(path) 263 } 264 265 // lfDiscardStats keeps track of the amount of data that could be discarded for 266 // a given logfile. 267 type lfDiscardStats struct { 268 sync.Mutex 269 m map[uint32]int64 270 } 271 272 type valueLog struct { 273 buf bytes.Buffer 274 pendingLen int 275 dirPath string 276 curWriter *fileutil.BufferedWriter 277 files []*logFile 278 279 kv *DB 280 maxPtr uint64 281 282 numEntriesWritten uint32 283 opt Options 284 metrics *y.MetricsSet 285 } 286 287 func vlogFilePath(dirPath string, fid uint32) string { 288 return fmt.Sprintf("%s%s%06d.vlog", dirPath, string(os.PathSeparator), fid) 289 } 290 291 func (vlog *valueLog) fpath(fid uint32) string { 292 return vlogFilePath(vlog.dirPath, fid) 293 } 294 295 func (vlog *valueLog) currentLogFile() *logFile { 296 if len(vlog.files) > 0 { 297 return vlog.files[len(vlog.files)-1] 298 } 299 return nil 300 } 301 302 func (vlog *valueLog) openOrCreateFiles(readOnly bool) error { 303 files, err := ioutil.ReadDir(vlog.dirPath) 304 if err != nil { 305 return errors.Wrapf(err, "Error while opening value log") 306 } 307 308 found := make(map[uint64]struct{}) 309 var maxFid uint32 // Beware len(files) == 0 case, this starts at 0. 310 for _, file := range files { 311 if !strings.HasSuffix(file.Name(), ".vlog") { 312 continue 313 } 314 fsz := len(file.Name()) 315 fid, err := strconv.ParseUint(file.Name()[:fsz-5], 10, 32) 316 if err != nil { 317 return errors.Wrapf(err, "Error while parsing value log id for file: %q", file.Name()) 318 } 319 if _, ok := found[fid]; ok { 320 return errors.Errorf("Found the same value log file twice: %d", fid) 321 } 322 found[fid] = struct{}{} 323 324 lf := &logFile{ 325 fid: uint32(fid), 326 path: vlog.fpath(uint32(fid)), 327 } 328 vlog.files = append(vlog.files, lf) 329 if uint32(fid) > maxFid { 330 maxFid = uint32(fid) 331 } 332 } 333 vlog.maxPtr = uint64(maxFid) << 32 334 sort.Slice(vlog.files, func(i, j int) bool { 335 return vlog.files[i].fid < vlog.files[j].fid 336 }) 337 338 // Open all previous log files as read only. Open the last log file 339 // as read write (unless the DB is read only). 340 for _, lf := range vlog.files { 341 if lf.fid == maxFid { 342 var flags uint32 343 if readOnly { 344 flags |= y.ReadOnly 345 } 346 if lf.fd, err = y.OpenExistingFile(lf.path, flags); err != nil { 347 return errors.Wrapf(err, "Unable to open value log file") 348 } 349 opt := &vlog.opt.ValueLogWriteOptions 350 vlog.curWriter = fileutil.NewBufferedWriter(lf.fd, opt.WriteBufferSize, nil) 351 } else { 352 if err := lf.openReadOnly(); err != nil { 353 return err 354 } 355 } 356 } 357 358 // If no files are found, then create a new file. 359 if len(vlog.files) == 0 { 360 // We already set vlog.maxFid above 361 err = vlog.createVlogFile(0) 362 if err != nil { 363 return err 364 } 365 } 366 return nil 367 } 368 369 func (vlog *valueLog) createVlogFile(fid uint32) error { 370 atomic.StoreUint64(&vlog.maxPtr, uint64(fid)<<32) 371 372 path := vlog.fpath(fid) 373 lf := &logFile{fid: fid, path: path} 374 vlog.numEntriesWritten = 0 375 376 var err error 377 if lf.fd, err = y.CreateSyncedFile(path, false); err != nil { 378 return errors.Wrapf(err, "Unable to create value log file") 379 } 380 if err = fileutil.Preallocate(lf.fd, vlog.opt.ValueLogFileSize); err != nil { 381 return errors.Wrap(err, "Unable to preallocate value log file") 382 } 383 opt := &vlog.opt.ValueLogWriteOptions 384 if vlog.curWriter == nil { 385 vlog.curWriter = fileutil.NewBufferedWriter(lf.fd, opt.WriteBufferSize, nil) 386 } else { 387 vlog.curWriter.Reset(lf.fd) 388 } 389 390 if err = syncDir(vlog.dirPath); err != nil { 391 return errors.Wrapf(err, "Unable to sync value log file dir") 392 } 393 vlog.files = append(vlog.files, lf) 394 syncedFid := atomic.LoadUint32(&vlog.kv.syncedFid) 395 for len(vlog.files) > vlog.opt.ValueLogMaxNumFiles { 396 deleteCandidate := vlog.files[0] 397 if deleteCandidate.fid < syncedFid { 398 os.Remove(deleteCandidate.path) 399 deleteCandidate.fd.Close() 400 vlog.files = vlog.files[1:] 401 continue 402 } 403 break 404 } 405 return nil 406 } 407 408 func (vlog *valueLog) Open(kv *DB, opt Options) error { 409 vlog.dirPath = opt.ValueDir 410 vlog.opt = opt 411 vlog.kv = kv 412 if err := vlog.openOrCreateFiles(kv.opt.ReadOnly); err != nil { 413 return errors.Wrapf(err, "Unable to open value log") 414 } 415 return nil 416 } 417 418 func (vlog *valueLog) Close() error { 419 var err error 420 for _, f := range vlog.files { 421 // A successful close does not guarantee that the data has been successfully saved to disk, as the kernel defers writes. 422 // It is not common for a file system to flush the buffers when the stream is closed. 423 if syncErr := fileutil.Fdatasync(f.fd); syncErr != nil { 424 err = syncErr 425 } 426 if closeErr := f.fd.Close(); closeErr != nil && err == nil { 427 err = closeErr 428 } 429 } 430 return err 431 } 432 433 // Replay replays the value log. The kv provided is only valid for the lifetime of function call. 434 func (vlog *valueLog) Replay(off logOffset, fn logEntry) error { 435 fid := off.fid 436 offset := off.offset 437 var lastOffset uint32 438 for _, lf := range vlog.files { 439 if lf.fid < fid { 440 continue 441 } 442 of := offset 443 if lf.fid > fid { 444 of = 0 445 } 446 endAt, err := vlog.iterate(lf, of, fn) 447 if err != nil { 448 return errors.Wrapf(err, "Unable to replay value log: %q", lf.path) 449 } 450 if lf.fid == vlog.maxFid() { 451 lastOffset = endAt 452 } 453 } 454 455 // Seek to the end to start writing. 456 var err error 457 last := vlog.files[len(vlog.files)-1] 458 _, err = last.fd.Seek(int64(lastOffset), io.SeekStart) 459 atomic.AddUint64(&vlog.maxPtr, uint64(lastOffset)) 460 return errors.Wrapf(err, "Unable to seek to end of value log: %q", last.path) 461 } 462 463 type logOffset struct { 464 fid uint32 465 offset uint32 466 } 467 468 func (lo logOffset) Less(logOff logOffset) bool { 469 if lo.fid == logOff.fid { 470 return lo.offset < logOff.offset 471 } 472 return lo.fid < logOff.fid 473 } 474 475 func (lo logOffset) Encode() []byte { 476 buf := make([]byte, 8) 477 binary.LittleEndian.PutUint32(buf, lo.fid) 478 binary.LittleEndian.PutUint32(buf[4:], lo.offset) 479 return buf 480 } 481 482 func (lo *logOffset) Decode(buf []byte) { 483 lo.fid = binary.LittleEndian.Uint32(buf) 484 lo.offset = binary.LittleEndian.Uint32(buf[4:]) 485 } 486 487 type request struct { 488 // Input values 489 Entries []*Entry 490 Wg sync.WaitGroup 491 Err error 492 } 493 494 func (req *request) Wait() error { 495 req.Wg.Wait() 496 req.Entries = nil 497 err := req.Err 498 requestPool.Put(req) 499 return err 500 } 501 502 func (vlog *valueLog) getMaxPtr() uint64 { 503 return atomic.LoadUint64(&vlog.maxPtr) 504 } 505 506 func (vlog *valueLog) maxFid() uint32 { 507 return uint32(atomic.LoadUint64(&vlog.maxPtr) >> 32) 508 } 509 510 func (vlog *valueLog) writableOffset() uint32 { 511 return uint32(atomic.LoadUint64(&vlog.maxPtr)) 512 } 513 514 func (vlog *valueLog) flush() error { 515 curlf := vlog.currentLogFile() 516 if vlog.pendingLen == 0 { 517 return nil 518 } 519 err := vlog.curWriter.Flush() 520 if err != nil { 521 return errors.Wrapf(err, "Unable to write to value log file: %q", curlf.path) 522 } 523 vlog.metrics.NumWrites.Inc() 524 vlog.metrics.NumVLogBytesWritten.Add(float64(vlog.pendingLen)) 525 atomic.AddUint64(&vlog.maxPtr, uint64(vlog.pendingLen)) 526 vlog.pendingLen = 0 527 528 if vlog.writableOffset() > uint32(vlog.opt.ValueLogFileSize) || 529 vlog.numEntriesWritten > vlog.opt.ValueLogMaxEntries { 530 var err error 531 if err = curlf.doneWriting(vlog.writableOffset()); err != nil { 532 return err 533 } 534 err = vlog.createVlogFile(vlog.maxFid() + 1) 535 if err != nil { 536 return err 537 } 538 } 539 return nil 540 } 541 542 // write is thread-unsafe by design and should not be called concurrently. 543 func (vlog *valueLog) write(reqs []*request) error { 544 for i := range reqs { 545 b := reqs[i] 546 for j := range b.Entries { 547 e := b.Entries[j] 548 plen, err := encodeEntry(e, &vlog.buf) // Now encode the entry into buffer. 549 if err != nil { 550 return err 551 } 552 vlog.curWriter.Append(vlog.buf.Bytes()) 553 vlog.buf.Reset() 554 vlog.pendingLen += plen 555 e.logOffset.fid = vlog.currentLogFile().fid 556 // Use the offset including buffer length so far. 557 e.logOffset.offset = vlog.writableOffset() + uint32(vlog.pendingLen) 558 } 559 vlog.numEntriesWritten += uint32(len(b.Entries)) 560 // We write to disk here so that all entries that are part of the same transaction are 561 // written to the same vlog file. 562 writeNow := 563 vlog.writableOffset()+uint32(vlog.pendingLen) > uint32(vlog.opt.ValueLogFileSize) || 564 vlog.numEntriesWritten > uint32(vlog.opt.ValueLogMaxEntries) 565 if writeNow { 566 if err := vlog.flush(); err != nil { 567 return err 568 } 569 } 570 } 571 return vlog.flush() 572 573 // Acquire mutex locks around this manipulation, so that the reads don't try to use 574 // an invalid file descriptor. 575 } 576 577 // Gets the logFile. 578 func (vlog *valueLog) getFile(fid uint32) (*logFile, error) { 579 for i := len(vlog.files) - 1; i >= 0; i-- { 580 file := vlog.files[i] 581 if file.fid == fid { 582 return file, nil 583 } 584 } 585 // log file has gone away, will need to retry the operation. 586 return nil, ErrRetry 587 }