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
Signed-off-by: Alejandro Lopez <a.lopez@yadro.com>
387 lines
8.6 KiB
Go
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))
|
|
}
|