frostfs-node/pkg/local_object_storage/writecache/writecachebitcask/region.go
Alejandro Lopez 42e74d6aab
Some checks failed
DCO action / DCO (pull_request) Successful in 3m8s
Vulncheck / Vulncheck (pull_request) Successful in 3m16s
Build / Build Components (1.20) (pull_request) Successful in 4m13s
Build / Build Components (1.21) (pull_request) Successful in 4m16s
Tests and linters / Staticcheck (pull_request) Successful in 5m11s
Tests and linters / Lint (pull_request) Successful in 5m58s
Tests and linters / Tests with -race (pull_request) Failing after 6m3s
Tests and linters / Tests (1.20) (pull_request) Successful in 7m29s
Tests and linters / Tests (1.21) (pull_request) Successful in 7m38s
[#610] Add bitcask-inspired writecache implementation
Signed-off-by: Alejandro Lopez <a.lopez@yadro.com>
2023-08-31 14:17:10 +03:00

387 lines
8.6 KiB
Go

package writecachebitcask
import (
"bytes"
"context"
"encoding/binary"
"fmt"
"io"
"math"
"os"
"path/filepath"
"strconv"
"strings"
"sync"
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util/logicerr"
apistatus "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client/status"
cid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id"
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"go.uber.org/zap"
)
const (
logFileOpenFlags = os.O_WRONLY | os.O_CREATE | os.O_APPEND | os.O_SYNC
keyLen = len(cid.ID{}) + len(oid.ID{})
sizeLen = 4
tombstone = uint32(math.MaxUint32)
)
// entry is a key directory entry.
//
// It stores the object address and its current log file index and offset.
type entry struct {
addr oid.Address
logIndex uint32
offset int
}
// pendingWrite is a write operation in the current write batch, not yet committed to the log file.
type pendingWrite struct {
addr oid.Address
offset int
errCh chan error
isDelete bool
}
type region struct {
sync.RWMutex
// Parameters
opts *options
index int
// Key directory
keyDir [][]*entry
// Current mem-buffer and log file
wbuf bytes.Buffer
logIndex uint32
logFile *os.File
size int
// Put batch state
writeBatch []pendingWrite
// Flush state
flushCh chan uint32
}
func (r *region) init() error {
if err := r.restore(); err != nil {
return err
}
f, err := os.OpenFile(r.logFilePath(r.logIndex), logFileOpenFlags, 0644)
if err != nil {
return fmt.Errorf("creating log file for region %d: %v", r.index, err)
}
r.logFile = f
return nil
}
// restore restores the region state from existing log files, if any.
func (r *region) restore() error {
dir := filepath.Join(r.opts.path, strconv.Itoa(r.index))
if err := os.MkdirAll(dir, 0755); err != nil {
return fmt.Errorf("creating region directory %d: %v", r.index, err)
}
entries, err := os.ReadDir(dir)
if err != nil {
return fmt.Errorf("listing region directory %d: %v", r.index, err)
}
var logIndices []uint32
for _, ei := range entries {
if !ei.Type().IsRegular() {
continue
}
name := strings.TrimSuffix(filepath.Base(ei.Name()), filepath.Ext(ei.Name()))
index, err := strconv.ParseUint(name, 16, 32)
if err != nil {
return fmt.Errorf("parsing log file index %q: %v", ei.Name(), err)
}
logIndices = append(logIndices, uint32(index))
}
logCount := len(logIndices)
if logCount == 0 {
return nil
}
r.logIndex = dispatchRecoveredLogIndices(logIndices, func(i uint32) {
r.flushCh <- i
})
return nil
}
func dispatchRecoveredLogIndices(indices []uint32, dispatchFunc func(uint32)) uint32 {
n := len(indices)
i0 := 0
// Check if the indices wrap around and correct the starting point
if indices[0] == 0 && indices[n-1] == math.MaxUint32 {
i0 = 1
for indices[i0-1] == indices[i0]-1 {
i0++
}
}
for i := 0; i < n; i++ {
dispatchFunc(indices[(i0+i)%n])
}
return indices[(i0+n-1)%n] + 1
}
func (r *region) close() error {
close(r.flushCh)
if r.logFile != nil {
return r.logFile.Close()
}
return nil
}
func (r *region) put(ctx context.Context, addr oid.Address, data []byte) error {
c, o := addr.Container(), addr.Object()
pp := pendingWrite{
addr: addr,
errCh: make(chan error, 1),
}
r.Lock()
// If the current log file is too large or missing, create a new one.
if r.logFile == nil || uint64(r.size) >= r.opts.maxLogSize {
if err := r.rotateLogFile(ctx); err != nil {
r.Unlock()
return err
}
}
// Check whether we need to schedule a batch flush in the future.
wasEmpty := len(r.writeBatch) == 0
pp.offset = r.size + r.wbuf.Len()
r.writeBatch = append(r.writeBatch, pp)
// Write the entry to the mem buffer.
r.wbuf.Write(c[:])
r.wbuf.Write(o[:])
_ = binary.Write(&r.wbuf, binary.LittleEndian, uint32(len(data)))
r.wbuf.Write(data)
r.Unlock()
if wasEmpty {
time.AfterFunc(r.opts.maxBatchDelay, r.flushWriteBatch)
}
// Wait for the batch flush.
select {
case err := <-pp.errCh:
return err
case <-ctx.Done():
return ctx.Err()
}
}
func (r *region) delete(ctx context.Context, addr oid.Address) error {
c, o := addr.Container(), addr.Object()
pp := pendingWrite{
addr: addr,
errCh: make(chan error, 1),
isDelete: true,
}
bucket := r.locateBucket(addr)
r.Lock()
// If the current log file is too large or missing, create a new one.
if r.logFile == nil || uint64(r.size) >= r.opts.maxLogSize {
if err := r.rotateLogFile(ctx); err != nil {
r.Unlock()
return err
}
}
// Locate the current entry (if any) in the key directory.
offset := -1
for _, e := range r.keyDir[bucket] {
if e.addr.Equals(addr) {
offset = e.offset
break
}
}
if offset == -1 {
r.Unlock()
return logicerr.Wrap(new(apistatus.ObjectNotFound))
}
// Check whether we need to schedule a batch flush in the future.
wasEmpty := len(r.writeBatch) == 0
pp.offset = r.size + r.wbuf.Len()
r.writeBatch = append(r.writeBatch, pp)
// Write the entry to the mem buffer.
r.wbuf.Write(c[:])
r.wbuf.Write(o[:])
_ = binary.Write(&r.wbuf, binary.LittleEndian, tombstone)
r.Unlock()
if wasEmpty {
time.AfterFunc(r.opts.maxBatchDelay, r.flushWriteBatch)
}
// Wait for the batch flush.
select {
case err := <-pp.errCh:
return err
case <-ctx.Done():
return ctx.Err()
}
}
func (r *region) get(addr oid.Address) (*objectSDK.Object, error) {
bucket := r.locateBucket(addr)
r.RLock()
// Locate the log file index and offset of the entry.
var logIndex uint32
offset := -1
for _, e := range r.keyDir[bucket] {
if e.addr.Equals(addr) {
logIndex = e.logIndex
offset = e.offset
break
}
}
defer r.RUnlock()
if offset == -1 {
return nil, logicerr.Wrap(new(apistatus.ObjectNotFound))
}
// Read the entry data from the corresponding log file.
f, err := os.Open(r.logFilePath(logIndex))
if err != nil {
return nil, fmt.Errorf("reading log file: %w", err)
}
defer f.Close()
if _, err := f.Seek(int64(offset), io.SeekStart); err != nil {
return nil, fmt.Errorf("seeking log entry: %w", err)
}
_, obj, _, err := readLogFileEntry(f)
if err != nil {
return nil, fmt.Errorf("reading log entry: %w", err)
}
if obj == nil {
return nil, logicerr.Wrap(new(apistatus.ObjectNotFound))
}
return obj, nil
}
// flushWriteBatch appends the membuffer to the current log file and returns
// any error to the callers.
func (r *region) flushWriteBatch() {
r.Lock()
defer r.Unlock()
n, err := r.logFile.Write(r.wbuf.Bytes())
for _, call := range r.writeBatch {
call.errCh <- err
}
if err == nil {
for _, call := range r.writeBatch {
r.updateKeyOffset(call.addr, call.offset, call.isDelete)
}
r.size += n
}
// Reset membuffer and clear the current write batch
r.wbuf.Reset()
r.writeBatch = r.writeBatch[:0]
}
func (r *region) locateBucket(addr oid.Address) int {
id := addr.Object()
h := binary.LittleEndian.Uint32(id[4:])
bucket := h & (uint32(len(r.keyDir)) - 1)
return int(bucket)
}
func (r *region) updateKeyOffset(addr oid.Address, offset int, isDelete bool) {
bucket := r.locateBucket(addr)
exists := false
for _, e := range r.keyDir[bucket] {
if e.addr.Equals(addr) {
exists = true
// This check is necessary because the entries should be updated in the
// same order they are appended to the log file. Otherwise, a different
// state might result from recovering.
if e.offset < offset {
if isDelete {
e.offset = -1
} else {
e.offset = offset
}
e.logIndex = r.logIndex
}
break
}
}
if !exists {
r.keyDir[bucket] = append(r.keyDir[bucket], &entry{
addr: addr,
offset: offset,
logIndex: r.logIndex,
})
}
}
// rotateLogFile closes the current log file, passes it to the flushing process and starts a new one.
func (r *region) rotateLogFile(ctx context.Context) error {
if r.logFile != nil {
if err := r.logFile.Close(); err != nil {
r.opts.log.Error(logs.WritecacheBitcaskClosingLogFile,
zap.Uint32("logIndex", r.logIndex),
zap.Error(err))
}
select {
case r.flushCh <- r.logIndex:
// Mark the log file as nil only after the flushing process is aware of it.
r.logFile = nil
case <-ctx.Done():
return ctx.Err()
}
}
f, err := os.OpenFile(r.logFilePath(r.logIndex+1), logFileOpenFlags, 0644)
if err != nil {
return fmt.Errorf("creating log file for region %d: %w", r.index, err)
}
r.logIndex++
r.logFile = f
r.size = 0
return nil
}
func (r *region) logFilePath(i uint32) string {
return filepath.Join(r.opts.path, strconv.Itoa(r.index), fmt.Sprintf("%08X.wlog", i))
}