2022-04-22 13:22:40 +00:00
|
|
|
package pilorama
|
|
|
|
|
|
|
|
import (
|
|
|
|
"bytes"
|
2023-04-13 12:36:20 +00:00
|
|
|
"context"
|
2022-04-22 13:22:40 +00:00
|
|
|
"encoding/binary"
|
2022-09-07 08:46:13 +00:00
|
|
|
"errors"
|
2022-06-09 08:09:18 +00:00
|
|
|
"fmt"
|
2022-04-22 13:22:40 +00:00
|
|
|
"math/rand"
|
|
|
|
"os"
|
|
|
|
"path/filepath"
|
2024-07-10 06:30:01 +00:00
|
|
|
"slices"
|
2024-03-28 12:53:26 +00:00
|
|
|
"sort"
|
2023-08-16 15:30:33 +00:00
|
|
|
"strconv"
|
2022-07-05 04:55:46 +00:00
|
|
|
"sync"
|
2022-06-09 11:10:01 +00:00
|
|
|
"time"
|
2022-04-22 13:22:40 +00:00
|
|
|
|
2023-06-15 10:19:36 +00:00
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
|
2023-03-07 13:38:26 +00:00
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
|
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util/logicerr"
|
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util"
|
2023-05-31 09:24:04 +00:00
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
|
2023-03-07 13:38:26 +00:00
|
|
|
cidSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id"
|
2022-04-22 13:22:40 +00:00
|
|
|
"github.com/nspcc-dev/neo-go/pkg/io"
|
|
|
|
"go.etcd.io/bbolt"
|
2023-04-13 12:36:20 +00:00
|
|
|
"go.opentelemetry.io/otel/attribute"
|
|
|
|
"go.opentelemetry.io/otel/trace"
|
2022-04-22 13:22:40 +00:00
|
|
|
)
|
|
|
|
|
|
|
|
type boltForest struct {
|
2022-06-09 08:09:18 +00:00
|
|
|
db *bbolt.DB
|
2022-07-05 04:55:46 +00:00
|
|
|
|
2023-01-11 10:08:12 +00:00
|
|
|
modeMtx sync.RWMutex
|
2022-07-05 04:55:46 +00:00
|
|
|
mode mode.Mode
|
2022-07-22 08:53:05 +00:00
|
|
|
|
|
|
|
// mtx protects batches field.
|
|
|
|
mtx sync.Mutex
|
|
|
|
batches []*batch
|
|
|
|
|
2022-06-09 08:09:18 +00:00
|
|
|
cfg
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2023-06-20 10:39:18 +00:00
|
|
|
const (
|
|
|
|
childrenKeySize = 17
|
|
|
|
maxKeySize = childrenKeySize
|
|
|
|
)
|
|
|
|
|
2022-04-22 13:22:40 +00:00
|
|
|
var (
|
|
|
|
dataBucket = []byte{0}
|
|
|
|
logBucket = []byte{1}
|
|
|
|
)
|
|
|
|
|
2023-01-11 10:08:12 +00:00
|
|
|
// ErrDegradedMode is returned when pilorama is in a degraded mode.
|
|
|
|
var ErrDegradedMode = logicerr.New("pilorama is in a degraded mode")
|
|
|
|
|
|
|
|
// ErrReadOnlyMode is returned when pilorama is in a read-only mode.
|
|
|
|
var ErrReadOnlyMode = logicerr.New("pilorama is in a read-only mode")
|
|
|
|
|
2022-04-22 13:22:40 +00:00
|
|
|
// NewBoltForest returns storage wrapper for storing operations on CRDT trees.
|
|
|
|
//
|
|
|
|
// Each tree is stored in a separate bucket by `CID + treeID` key.
|
|
|
|
// All integers are stored in little-endian unless explicitly specified otherwise.
|
|
|
|
//
|
|
|
|
// DB schema (for a single tree):
|
|
|
|
// timestamp is 8-byte, id is 4-byte.
|
|
|
|
//
|
|
|
|
// log storage (logBucket):
|
|
|
|
// timestamp in big-endian -> log operation
|
|
|
|
//
|
|
|
|
// tree storage (dataBucket):
|
2022-10-17 12:03:55 +00:00
|
|
|
// - 't' + node (id) -> timestamp when the node first appeared,
|
|
|
|
// - 'p' + node (id) -> parent (id),
|
|
|
|
// - 'm' + node (id) -> serialized meta,
|
|
|
|
// - 'c' + parent (id) + child (id) -> 0/1,
|
|
|
|
// - 'i' + 0 + attrKey + 0 + attrValue + 0 + parent (id) + node (id) -> 0/1 (1 for automatically created nodes).
|
2022-06-09 08:09:18 +00:00
|
|
|
func NewBoltForest(opts ...Option) ForestStorage {
|
|
|
|
b := boltForest{
|
|
|
|
cfg: cfg{
|
|
|
|
perm: os.ModePerm,
|
|
|
|
maxBatchDelay: bbolt.DefaultMaxBatchDelay,
|
|
|
|
maxBatchSize: bbolt.DefaultMaxBatchSize,
|
2023-03-21 10:38:44 +00:00
|
|
|
openFile: os.OpenFile,
|
2023-06-07 09:27:53 +00:00
|
|
|
metrics: &noopMetrics{},
|
2022-06-09 08:09:18 +00:00
|
|
|
},
|
2024-06-10 14:48:05 +00:00
|
|
|
mode: mode.Disabled,
|
2022-06-09 08:09:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
for i := range opts {
|
|
|
|
opts[i](&b.cfg)
|
|
|
|
}
|
|
|
|
|
|
|
|
return &b
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2022-07-05 04:55:46 +00:00
|
|
|
func (t *boltForest) SetMode(m mode.Mode) error {
|
|
|
|
t.modeMtx.Lock()
|
|
|
|
defer t.modeMtx.Unlock()
|
|
|
|
|
|
|
|
if t.mode == m {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
err := t.Close()
|
2023-01-11 10:08:12 +00:00
|
|
|
if err == nil && !m.NoMetabase() {
|
2024-02-09 06:17:17 +00:00
|
|
|
if err = t.openBolt(m); err == nil {
|
2022-07-05 04:55:46 +00:00
|
|
|
err = t.Init()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if err != nil {
|
|
|
|
return fmt.Errorf("can't set pilorama mode (old=%s, new=%s): %w", t.mode, m, err)
|
|
|
|
}
|
|
|
|
|
|
|
|
t.mode = m
|
2024-06-04 13:28:47 +00:00
|
|
|
t.metrics.SetMode(mode.ConvertToComponentModeDegraded(m))
|
2022-07-05 04:55:46 +00:00
|
|
|
return nil
|
|
|
|
}
|
2023-10-31 11:56:55 +00:00
|
|
|
|
2024-02-09 06:17:17 +00:00
|
|
|
func (t *boltForest) Open(_ context.Context, mode mode.Mode) error {
|
|
|
|
t.modeMtx.Lock()
|
|
|
|
defer t.modeMtx.Unlock()
|
|
|
|
t.mode = mode
|
|
|
|
if mode.NoMetabase() {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
return t.openBolt(mode)
|
|
|
|
}
|
|
|
|
|
2024-06-04 13:28:47 +00:00
|
|
|
func (t *boltForest) openBolt(m mode.Mode) error {
|
|
|
|
readOnly := m.ReadOnly()
|
2022-06-09 08:09:18 +00:00
|
|
|
err := util.MkdirAllX(filepath.Dir(t.path), t.perm)
|
|
|
|
if err != nil {
|
2023-06-15 10:19:36 +00:00
|
|
|
return metaerr.Wrap(fmt.Errorf("can't create dir %s for the pilorama: %w", t.path, err))
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2022-06-09 08:09:18 +00:00
|
|
|
opts := *bbolt.DefaultOptions
|
2022-06-28 13:42:50 +00:00
|
|
|
opts.ReadOnly = readOnly
|
2022-06-09 08:09:18 +00:00
|
|
|
opts.NoSync = t.noSync
|
2022-06-09 11:10:01 +00:00
|
|
|
opts.Timeout = 100 * time.Millisecond
|
2023-03-21 10:38:44 +00:00
|
|
|
opts.OpenFile = t.openFile
|
2022-06-09 08:09:18 +00:00
|
|
|
|
|
|
|
t.db, err = bbolt.Open(t.path, t.perm, &opts)
|
2022-04-22 13:22:40 +00:00
|
|
|
if err != nil {
|
2023-06-15 10:19:36 +00:00
|
|
|
return metaerr.Wrap(fmt.Errorf("can't open the pilorama DB: %w", err))
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2022-06-09 08:09:18 +00:00
|
|
|
t.db.MaxBatchSize = t.maxBatchSize
|
|
|
|
t.db.MaxBatchDelay = t.maxBatchDelay
|
2024-06-04 13:28:47 +00:00
|
|
|
t.metrics.SetMode(mode.ConvertToComponentModeDegraded(m))
|
2022-06-28 13:42:50 +00:00
|
|
|
return nil
|
|
|
|
}
|
2023-10-31 11:56:55 +00:00
|
|
|
|
2022-06-28 13:42:50 +00:00
|
|
|
func (t *boltForest) Init() error {
|
2023-01-11 10:08:12 +00:00
|
|
|
if t.mode.NoMetabase() || t.db.IsReadOnly() {
|
2022-06-28 13:42:50 +00:00
|
|
|
return nil
|
|
|
|
}
|
2022-06-09 08:09:18 +00:00
|
|
|
return t.db.Update(func(tx *bbolt.Tx) error {
|
2022-04-22 13:22:40 +00:00
|
|
|
_, err := tx.CreateBucketIfNotExists(dataBucket)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
_, err = tx.CreateBucketIfNotExists(logBucket)
|
2023-08-14 11:06:17 +00:00
|
|
|
return err
|
2022-04-22 13:22:40 +00:00
|
|
|
})
|
|
|
|
}
|
2023-10-31 11:56:55 +00:00
|
|
|
|
2022-06-28 13:42:50 +00:00
|
|
|
func (t *boltForest) Close() error {
|
2023-06-07 09:27:53 +00:00
|
|
|
var err error
|
2022-06-28 13:42:50 +00:00
|
|
|
if t.db != nil {
|
2023-06-07 09:27:53 +00:00
|
|
|
err = t.db.Close()
|
2022-06-28 13:42:50 +00:00
|
|
|
}
|
2023-06-07 09:27:53 +00:00
|
|
|
if err == nil {
|
|
|
|
t.metrics.Close()
|
|
|
|
}
|
|
|
|
return err
|
2022-06-28 13:42:50 +00:00
|
|
|
}
|
2022-04-22 13:22:40 +00:00
|
|
|
|
2023-06-07 11:39:03 +00:00
|
|
|
func (t *boltForest) SetParentID(id string) {
|
|
|
|
t.metrics.SetParentID(id)
|
|
|
|
}
|
|
|
|
|
2022-04-22 13:22:40 +00:00
|
|
|
// TreeMove implements the Forest interface.
|
2023-04-13 12:36:20 +00:00
|
|
|
func (t *boltForest) TreeMove(ctx context.Context, d CIDDescriptor, treeID string, m *Move) (*Move, error) {
|
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeMove",
|
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", d.CID.EncodeToString()),
|
|
|
|
attribute.Int("position", d.Position),
|
|
|
|
attribute.Int("size", d.Size),
|
|
|
|
attribute.String("tree_id", treeID),
|
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
2022-05-27 12:55:02 +00:00
|
|
|
if !d.checkValid() {
|
|
|
|
return nil, ErrInvalidCIDDescriptor
|
|
|
|
}
|
|
|
|
|
2023-01-11 10:08:12 +00:00
|
|
|
t.modeMtx.RLock()
|
|
|
|
defer t.modeMtx.RUnlock()
|
|
|
|
|
|
|
|
if t.mode.NoMetabase() {
|
|
|
|
return nil, ErrDegradedMode
|
|
|
|
} else if t.mode.ReadOnly() {
|
|
|
|
return nil, ErrReadOnlyMode
|
|
|
|
}
|
|
|
|
|
2023-01-17 13:16:50 +00:00
|
|
|
lm := *m
|
2023-01-18 06:38:42 +00:00
|
|
|
fullID := bucketName(d.CID, treeID)
|
2023-06-15 10:19:36 +00:00
|
|
|
return &lm, metaerr.Wrap(t.db.Batch(func(tx *bbolt.Tx) error {
|
2023-01-18 06:38:42 +00:00
|
|
|
bLog, bTree, err := t.getTreeBuckets(tx, fullID)
|
2022-04-22 13:22:40 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2022-12-30 13:29:11 +00:00
|
|
|
lm.Time = t.getLatestTimestamp(bLog, d.Position, d.Size)
|
|
|
|
if lm.Child == RootID {
|
|
|
|
lm.Child = t.findSpareID(bTree)
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
2023-06-20 10:39:18 +00:00
|
|
|
return t.do(bLog, bTree, make([]byte, maxKeySize), &lm)
|
2023-06-15 10:19:36 +00:00
|
|
|
}))
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2023-06-13 08:26:59 +00:00
|
|
|
func (t *boltForest) TreeHeight(ctx context.Context, cid cidSDK.ID, treeID string) (uint64, error) {
|
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeHeight",
|
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", cid.EncodeToString()),
|
|
|
|
attribute.String("tree_id", treeID),
|
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
|
|
|
t.modeMtx.RLock()
|
|
|
|
defer t.modeMtx.RUnlock()
|
|
|
|
|
|
|
|
if t.mode.NoMetabase() {
|
|
|
|
return 0, ErrDegradedMode
|
|
|
|
}
|
|
|
|
|
|
|
|
var height uint64
|
|
|
|
var retErr error
|
|
|
|
err := t.db.View(func(tx *bbolt.Tx) error {
|
|
|
|
treeRoot := tx.Bucket(bucketName(cid, treeID))
|
|
|
|
if treeRoot != nil {
|
|
|
|
k, _ := treeRoot.Bucket(logBucket).Cursor().Last()
|
|
|
|
height = binary.BigEndian.Uint64(k)
|
|
|
|
} else {
|
|
|
|
retErr = ErrTreeNotFound
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
})
|
|
|
|
if err == nil {
|
|
|
|
err = retErr
|
|
|
|
}
|
2023-06-15 10:19:36 +00:00
|
|
|
return height, metaerr.Wrap(err)
|
2023-06-13 08:26:59 +00:00
|
|
|
}
|
|
|
|
|
2022-10-06 16:06:19 +00:00
|
|
|
// TreeExists implements the Forest interface.
|
2023-04-13 12:36:20 +00:00
|
|
|
func (t *boltForest) TreeExists(ctx context.Context, cid cidSDK.ID, treeID string) (bool, error) {
|
2023-06-07 09:27:53 +00:00
|
|
|
var (
|
|
|
|
startedAt = time.Now()
|
|
|
|
success = false
|
|
|
|
)
|
|
|
|
defer func() {
|
|
|
|
t.metrics.AddMethodDuration("TreeExists", time.Since(startedAt), success)
|
|
|
|
}()
|
|
|
|
|
2023-04-13 12:36:20 +00:00
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeExists",
|
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", cid.EncodeToString()),
|
|
|
|
attribute.String("tree_id", treeID),
|
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
2023-01-11 10:08:12 +00:00
|
|
|
t.modeMtx.RLock()
|
|
|
|
defer t.modeMtx.RUnlock()
|
|
|
|
|
|
|
|
if t.mode.NoMetabase() {
|
|
|
|
return false, ErrDegradedMode
|
|
|
|
}
|
|
|
|
|
2022-10-06 16:06:19 +00:00
|
|
|
var exists bool
|
|
|
|
|
|
|
|
err := t.db.View(func(tx *bbolt.Tx) error {
|
|
|
|
treeRoot := tx.Bucket(bucketName(cid, treeID))
|
|
|
|
exists = treeRoot != nil
|
|
|
|
return nil
|
|
|
|
})
|
2023-06-07 09:27:53 +00:00
|
|
|
success = err == nil
|
2023-06-15 10:19:36 +00:00
|
|
|
return exists, metaerr.Wrap(err)
|
2022-10-06 16:06:19 +00:00
|
|
|
}
|
|
|
|
|
2023-01-25 10:25:45 +00:00
|
|
|
var syncHeightKey = []byte{'h'}
|
|
|
|
|
|
|
|
// TreeUpdateLastSyncHeight implements the pilorama.Forest interface.
|
2023-04-13 12:36:20 +00:00
|
|
|
func (t *boltForest) TreeUpdateLastSyncHeight(ctx context.Context, cid cidSDK.ID, treeID string, height uint64) error {
|
2023-06-07 09:27:53 +00:00
|
|
|
var (
|
|
|
|
startedAt = time.Now()
|
|
|
|
success = false
|
|
|
|
)
|
|
|
|
defer func() {
|
|
|
|
t.metrics.AddMethodDuration("TreeUpdateLastSyncHeight", time.Since(startedAt), success)
|
|
|
|
}()
|
|
|
|
|
2023-04-13 12:36:20 +00:00
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeUpdateLastSyncHeight",
|
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", cid.EncodeToString()),
|
|
|
|
attribute.String("tree_id", treeID),
|
2023-08-16 15:30:33 +00:00
|
|
|
attribute.String("height", strconv.FormatUint(height, 10)),
|
2023-04-13 12:36:20 +00:00
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
2023-01-25 10:25:45 +00:00
|
|
|
rawHeight := make([]byte, 8)
|
|
|
|
binary.LittleEndian.PutUint64(rawHeight, height)
|
|
|
|
|
|
|
|
buck := bucketName(cid, treeID)
|
2023-06-07 09:27:53 +00:00
|
|
|
err := metaerr.Wrap(t.db.Batch(func(tx *bbolt.Tx) error {
|
2023-01-25 10:25:45 +00:00
|
|
|
treeRoot := tx.Bucket(buck)
|
|
|
|
if treeRoot == nil {
|
|
|
|
return ErrTreeNotFound
|
|
|
|
}
|
|
|
|
|
|
|
|
b := treeRoot.Bucket(dataBucket)
|
|
|
|
return b.Put(syncHeightKey, rawHeight)
|
2023-06-15 10:19:36 +00:00
|
|
|
}))
|
2023-06-07 09:27:53 +00:00
|
|
|
success = err == nil
|
|
|
|
return err
|
2023-01-25 10:25:45 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// TreeLastSyncHeight implements the pilorama.Forest interface.
|
2023-04-13 12:36:20 +00:00
|
|
|
func (t *boltForest) TreeLastSyncHeight(ctx context.Context, cid cidSDK.ID, treeID string) (uint64, error) {
|
2023-06-07 09:27:53 +00:00
|
|
|
var (
|
|
|
|
startedAt = time.Now()
|
|
|
|
success = false
|
|
|
|
)
|
|
|
|
defer func() {
|
|
|
|
t.metrics.AddMethodDuration("TreeLastSyncHeight", time.Since(startedAt), success)
|
|
|
|
}()
|
|
|
|
|
2023-04-13 12:36:20 +00:00
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeLastSyncHeight",
|
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", cid.EncodeToString()),
|
|
|
|
attribute.String("tree_id", treeID),
|
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
2023-01-25 10:25:45 +00:00
|
|
|
var height uint64
|
|
|
|
|
|
|
|
buck := bucketName(cid, treeID)
|
|
|
|
err := t.db.View(func(tx *bbolt.Tx) error {
|
|
|
|
treeRoot := tx.Bucket(buck)
|
|
|
|
if treeRoot == nil {
|
|
|
|
return ErrTreeNotFound
|
|
|
|
}
|
|
|
|
|
|
|
|
b := treeRoot.Bucket(dataBucket)
|
|
|
|
data := b.Get(syncHeightKey)
|
|
|
|
if len(data) == 8 {
|
|
|
|
height = binary.LittleEndian.Uint64(data)
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
})
|
2023-06-07 09:27:53 +00:00
|
|
|
success = err == nil
|
2023-06-15 10:19:36 +00:00
|
|
|
return height, metaerr.Wrap(err)
|
2023-01-25 10:25:45 +00:00
|
|
|
}
|
|
|
|
|
2022-04-22 13:22:40 +00:00
|
|
|
// TreeAddByPath implements the Forest interface.
|
2023-04-13 12:36:20 +00:00
|
|
|
func (t *boltForest) TreeAddByPath(ctx context.Context, d CIDDescriptor, treeID string, attr string, path []string, meta []KeyValue) ([]Move, error) {
|
2023-06-07 09:27:53 +00:00
|
|
|
var (
|
|
|
|
startedAt = time.Now()
|
|
|
|
success = false
|
|
|
|
)
|
|
|
|
defer func() {
|
|
|
|
t.metrics.AddMethodDuration("TreeAddByPath", time.Since(startedAt), success)
|
|
|
|
}()
|
|
|
|
|
2023-04-13 12:36:20 +00:00
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeAddByPath",
|
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", d.CID.EncodeToString()),
|
|
|
|
attribute.Int("position", d.Position),
|
|
|
|
attribute.Int("size", d.Size),
|
|
|
|
attribute.String("tree_id", treeID),
|
|
|
|
attribute.String("attr", attr),
|
|
|
|
attribute.Int("path_count", len(path)),
|
|
|
|
attribute.Int("meta_count", len(meta)),
|
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
2023-06-07 09:27:53 +00:00
|
|
|
res, err := t.addByPathInternal(d, attr, treeID, path, meta)
|
|
|
|
success = err == nil
|
|
|
|
return res, err
|
|
|
|
}
|
|
|
|
|
|
|
|
func (t *boltForest) addByPathInternal(d CIDDescriptor, attr string, treeID string, path []string, meta []KeyValue) ([]Move, error) {
|
2022-05-27 12:55:02 +00:00
|
|
|
if !d.checkValid() {
|
|
|
|
return nil, ErrInvalidCIDDescriptor
|
|
|
|
}
|
2022-05-24 13:12:50 +00:00
|
|
|
if !isAttributeInternal(attr) {
|
|
|
|
return nil, ErrNotPathAttribute
|
|
|
|
}
|
|
|
|
|
2023-01-11 10:08:12 +00:00
|
|
|
t.modeMtx.RLock()
|
|
|
|
defer t.modeMtx.RUnlock()
|
|
|
|
|
|
|
|
if t.mode.NoMetabase() {
|
|
|
|
return nil, ErrDegradedMode
|
|
|
|
} else if t.mode.ReadOnly() {
|
|
|
|
return nil, ErrReadOnlyMode
|
|
|
|
}
|
|
|
|
|
2023-01-25 11:12:02 +00:00
|
|
|
var lm []Move
|
2023-06-20 10:39:18 +00:00
|
|
|
var key [maxKeySize]byte
|
2022-04-22 13:22:40 +00:00
|
|
|
|
2023-01-18 06:38:42 +00:00
|
|
|
fullID := bucketName(d.CID, treeID)
|
2022-05-25 08:33:23 +00:00
|
|
|
err := t.db.Batch(func(tx *bbolt.Tx) error {
|
2023-01-18 06:38:42 +00:00
|
|
|
bLog, bTree, err := t.getTreeBuckets(tx, fullID)
|
2022-04-22 13:22:40 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
i, node, err := t.getPathPrefix(bTree, attr, path)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2022-05-27 12:55:02 +00:00
|
|
|
ts := t.getLatestTimestamp(bLog, d.Position, d.Size)
|
2023-01-25 11:12:02 +00:00
|
|
|
lm = make([]Move, len(path)-i+1)
|
2022-04-22 13:22:40 +00:00
|
|
|
for j := i; j < len(path); j++ {
|
2023-01-17 13:16:50 +00:00
|
|
|
lm[j-i] = Move{
|
2022-04-22 13:22:40 +00:00
|
|
|
Parent: node,
|
|
|
|
Meta: Meta{
|
2022-05-27 12:55:02 +00:00
|
|
|
Time: ts,
|
2022-04-22 13:22:40 +00:00
|
|
|
Items: []KeyValue{{Key: attr, Value: []byte(path[j])}},
|
|
|
|
},
|
|
|
|
Child: t.findSpareID(bTree),
|
|
|
|
}
|
|
|
|
|
|
|
|
err := t.do(bLog, bTree, key[:], &lm[j-i])
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2022-05-27 12:55:02 +00:00
|
|
|
ts = nextTimestamp(ts, uint64(d.Position), uint64(d.Size))
|
2022-04-22 13:22:40 +00:00
|
|
|
node = lm[j-i].Child
|
|
|
|
}
|
|
|
|
|
2023-01-17 13:16:50 +00:00
|
|
|
lm[len(lm)-1] = Move{
|
2022-04-22 13:22:40 +00:00
|
|
|
Parent: node,
|
|
|
|
Meta: Meta{
|
2022-05-27 12:55:02 +00:00
|
|
|
Time: ts,
|
2022-04-22 13:22:40 +00:00
|
|
|
Items: meta,
|
|
|
|
},
|
|
|
|
Child: t.findSpareID(bTree),
|
|
|
|
}
|
|
|
|
return t.do(bLog, bTree, key[:], &lm[len(lm)-1])
|
|
|
|
})
|
2023-06-15 10:19:36 +00:00
|
|
|
return lm, metaerr.Wrap(err)
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// getLatestTimestamp returns timestamp for a new operation which is guaranteed to be bigger than
|
|
|
|
// all timestamps corresponding to already stored operations.
|
2022-05-27 12:55:02 +00:00
|
|
|
func (t *boltForest) getLatestTimestamp(bLog *bbolt.Bucket, pos, size int) uint64 {
|
|
|
|
var ts uint64
|
|
|
|
|
2022-04-22 13:22:40 +00:00
|
|
|
c := bLog.Cursor()
|
|
|
|
key, _ := c.Last()
|
2022-05-27 12:55:02 +00:00
|
|
|
if len(key) != 0 {
|
|
|
|
ts = binary.BigEndian.Uint64(key)
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
2022-05-27 12:55:02 +00:00
|
|
|
return nextTimestamp(ts, uint64(pos), uint64(size))
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// findSpareID returns random unused ID.
|
|
|
|
func (t *boltForest) findSpareID(bTree *bbolt.Bucket) uint64 {
|
|
|
|
id := uint64(rand.Int63())
|
2023-01-17 13:16:50 +00:00
|
|
|
key := make([]byte, 9)
|
2022-04-22 13:22:40 +00:00
|
|
|
|
|
|
|
for {
|
2023-01-17 13:16:50 +00:00
|
|
|
_, _, _, ok := t.getState(bTree, stateKey(key, id))
|
|
|
|
if !ok {
|
2022-04-22 13:22:40 +00:00
|
|
|
return id
|
|
|
|
}
|
|
|
|
id = uint64(rand.Int63())
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// TreeApply implements the Forest interface.
|
2023-04-13 12:36:20 +00:00
|
|
|
func (t *boltForest) TreeApply(ctx context.Context, cnr cidSDK.ID, treeID string, m *Move, backgroundSync bool) error {
|
2023-06-07 09:27:53 +00:00
|
|
|
var (
|
|
|
|
startedAt = time.Now()
|
|
|
|
success = false
|
|
|
|
)
|
|
|
|
defer func() {
|
|
|
|
t.metrics.AddMethodDuration("TreeApply", time.Since(startedAt), success)
|
|
|
|
}()
|
|
|
|
|
2023-04-13 12:36:20 +00:00
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeApply",
|
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", cnr.EncodeToString()),
|
|
|
|
attribute.String("tree_id", treeID),
|
|
|
|
attribute.Bool("background", backgroundSync),
|
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
2023-01-11 10:08:12 +00:00
|
|
|
t.modeMtx.RLock()
|
|
|
|
defer t.modeMtx.RUnlock()
|
|
|
|
|
|
|
|
if t.mode.NoMetabase() {
|
|
|
|
return ErrDegradedMode
|
|
|
|
} else if t.mode.ReadOnly() {
|
|
|
|
return ErrReadOnlyMode
|
|
|
|
}
|
|
|
|
|
2022-12-19 16:01:57 +00:00
|
|
|
if backgroundSync {
|
|
|
|
var seen bool
|
|
|
|
err := t.db.View(func(tx *bbolt.Tx) error {
|
2023-03-21 12:43:12 +00:00
|
|
|
treeRoot := tx.Bucket(bucketName(cnr, treeID))
|
2022-12-19 16:01:57 +00:00
|
|
|
if treeRoot == nil {
|
2023-06-07 09:27:53 +00:00
|
|
|
success = true
|
2022-12-19 16:01:57 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
b := treeRoot.Bucket(logBucket)
|
|
|
|
|
|
|
|
var logKey [8]byte
|
|
|
|
binary.BigEndian.PutUint64(logKey[:], m.Time)
|
|
|
|
seen = b.Get(logKey[:]) != nil
|
2023-06-07 09:27:53 +00:00
|
|
|
success = true
|
2022-12-19 16:01:57 +00:00
|
|
|
return nil
|
|
|
|
})
|
|
|
|
if err != nil || seen {
|
2023-06-07 09:27:53 +00:00
|
|
|
success = err == nil
|
2023-06-15 10:19:36 +00:00
|
|
|
return metaerr.Wrap(err)
|
2022-12-19 16:01:57 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-07-22 08:53:05 +00:00
|
|
|
if t.db.MaxBatchSize == 1 {
|
2023-03-21 12:43:12 +00:00
|
|
|
fullID := bucketName(cnr, treeID)
|
2023-06-07 09:27:53 +00:00
|
|
|
err := metaerr.Wrap(t.db.Update(func(tx *bbolt.Tx) error {
|
2023-01-18 06:38:42 +00:00
|
|
|
bLog, bTree, err := t.getTreeBuckets(tx, fullID)
|
2022-07-22 08:53:05 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2023-01-25 11:12:02 +00:00
|
|
|
var lm Move
|
2022-07-22 08:53:05 +00:00
|
|
|
return t.applyOperation(bLog, bTree, []*Move{m}, &lm)
|
2023-06-15 10:19:36 +00:00
|
|
|
}))
|
2023-06-07 09:27:53 +00:00
|
|
|
success = err == nil
|
|
|
|
return err
|
2022-07-22 08:53:05 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
ch := make(chan error, 1)
|
2023-03-21 12:43:12 +00:00
|
|
|
t.addBatch(cnr, treeID, m, ch)
|
2023-06-07 09:27:53 +00:00
|
|
|
err := <-ch
|
|
|
|
success = err == nil
|
|
|
|
return metaerr.Wrap(err)
|
2022-07-22 08:53:05 +00:00
|
|
|
}
|
|
|
|
|
2024-02-06 10:59:50 +00:00
|
|
|
// TreeApplyStream should be used with caution: this method locks other write transactions while `source` is not closed.
|
|
|
|
func (t *boltForest) TreeApplyStream(ctx context.Context, cnr cidSDK.ID, treeID string, source <-chan *Move) error {
|
|
|
|
var (
|
|
|
|
startedAt = time.Now()
|
|
|
|
success = false
|
|
|
|
)
|
|
|
|
defer func() {
|
|
|
|
t.metrics.AddMethodDuration("TreeApplyStream", time.Since(startedAt), success)
|
|
|
|
}()
|
|
|
|
|
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeApplyStream",
|
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", cnr.EncodeToString()),
|
|
|
|
attribute.String("tree_id", treeID),
|
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
|
|
|
t.modeMtx.RLock()
|
|
|
|
defer t.modeMtx.RUnlock()
|
|
|
|
|
|
|
|
if t.mode.NoMetabase() {
|
|
|
|
return ErrDegradedMode
|
|
|
|
} else if t.mode.ReadOnly() {
|
|
|
|
return ErrReadOnlyMode
|
|
|
|
}
|
|
|
|
|
|
|
|
fullID := bucketName(cnr, treeID)
|
|
|
|
err := metaerr.Wrap(t.db.Update(func(tx *bbolt.Tx) error {
|
|
|
|
bLog, bTree, err := t.getTreeBuckets(tx, fullID)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return ctx.Err()
|
|
|
|
case m, ok := <-source:
|
|
|
|
if !ok {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
var lm Move
|
|
|
|
if e := t.applyOperation(bLog, bTree, []*Move{m}, &lm); e != nil {
|
|
|
|
return e
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}))
|
|
|
|
success = err == nil
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2023-03-21 12:43:12 +00:00
|
|
|
func (t *boltForest) addBatch(cnr cidSDK.ID, treeID string, m *Move, ch chan error) {
|
2022-07-22 08:53:05 +00:00
|
|
|
t.mtx.Lock()
|
|
|
|
for i := 0; i < len(t.batches); i++ {
|
|
|
|
t.batches[i].mtx.Lock()
|
|
|
|
if t.batches[i].timer == nil {
|
|
|
|
t.batches[i].mtx.Unlock()
|
|
|
|
copy(t.batches[i:], t.batches[i+1:])
|
|
|
|
t.batches = t.batches[:len(t.batches)-1]
|
|
|
|
i--
|
|
|
|
continue
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
2022-05-28 14:11:50 +00:00
|
|
|
|
2023-03-21 12:43:12 +00:00
|
|
|
found := t.batches[i].cid.Equals(cnr) && t.batches[i].treeID == treeID
|
2022-07-22 08:53:05 +00:00
|
|
|
if found {
|
|
|
|
t.batches[i].results = append(t.batches[i].results, ch)
|
|
|
|
t.batches[i].operations = append(t.batches[i].operations, m)
|
|
|
|
if len(t.batches[i].operations) == t.db.MaxBatchSize {
|
|
|
|
t.batches[i].timer.Stop()
|
|
|
|
t.batches[i].timer = nil
|
|
|
|
t.batches[i].mtx.Unlock()
|
|
|
|
b := t.batches[i]
|
|
|
|
t.mtx.Unlock()
|
|
|
|
b.trigger()
|
|
|
|
return
|
|
|
|
}
|
|
|
|
t.batches[i].mtx.Unlock()
|
|
|
|
t.mtx.Unlock()
|
|
|
|
return
|
|
|
|
}
|
|
|
|
t.batches[i].mtx.Unlock()
|
|
|
|
}
|
|
|
|
b := &batch{
|
|
|
|
forest: t,
|
2023-03-21 12:43:12 +00:00
|
|
|
cid: cnr,
|
2022-07-22 08:53:05 +00:00
|
|
|
treeID: treeID,
|
|
|
|
results: []chan<- error{ch},
|
|
|
|
operations: []*Move{m},
|
|
|
|
}
|
2023-02-01 08:58:16 +00:00
|
|
|
b.mtx.Lock()
|
2022-07-22 08:53:05 +00:00
|
|
|
b.timer = time.AfterFunc(t.db.MaxBatchDelay, b.trigger)
|
2023-02-01 08:58:16 +00:00
|
|
|
b.mtx.Unlock()
|
2022-07-22 08:53:05 +00:00
|
|
|
t.batches = append(t.batches, b)
|
|
|
|
t.mtx.Unlock()
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2023-01-18 06:38:42 +00:00
|
|
|
func (t *boltForest) getTreeBuckets(tx *bbolt.Tx, treeRoot []byte) (*bbolt.Bucket, *bbolt.Bucket, error) {
|
2022-07-22 06:31:24 +00:00
|
|
|
child := tx.Bucket(treeRoot)
|
|
|
|
if child != nil {
|
|
|
|
return child.Bucket(logBucket), child.Bucket(dataBucket), nil
|
|
|
|
}
|
|
|
|
|
2022-04-22 13:22:40 +00:00
|
|
|
child, err := tx.CreateBucket(treeRoot)
|
2022-07-22 06:31:24 +00:00
|
|
|
if err != nil {
|
2022-04-22 13:22:40 +00:00
|
|
|
return nil, nil, err
|
|
|
|
}
|
2022-07-22 06:31:24 +00:00
|
|
|
bLog, err := child.CreateBucket(logBucket)
|
|
|
|
if err != nil {
|
|
|
|
return nil, nil, err
|
|
|
|
}
|
|
|
|
bData, err := child.CreateBucket(dataBucket)
|
|
|
|
if err != nil {
|
|
|
|
return nil, nil, err
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
return bLog, bData, nil
|
|
|
|
}
|
|
|
|
|
2022-07-22 08:53:05 +00:00
|
|
|
// applyOperations applies log operations. Assumes lm are sorted by timestamp.
|
2023-01-25 11:12:02 +00:00
|
|
|
func (t *boltForest) applyOperation(logBucket, treeBucket *bbolt.Bucket, ms []*Move, lm *Move) error {
|
|
|
|
var tmp Move
|
2023-06-20 10:39:18 +00:00
|
|
|
var cKey [maxKeySize]byte
|
2022-04-22 13:22:40 +00:00
|
|
|
|
|
|
|
c := logBucket.Cursor()
|
|
|
|
|
|
|
|
key, value := c.Last()
|
|
|
|
|
2022-05-28 09:39:37 +00:00
|
|
|
b := bytes.NewReader(nil)
|
|
|
|
r := io.NewBinReaderFromIO(b)
|
|
|
|
|
2022-04-22 13:22:40 +00:00
|
|
|
// 1. Undo up until the desired timestamp is here.
|
2022-07-22 08:53:05 +00:00
|
|
|
for len(key) == 8 && ms[0].Time < binary.BigEndian.Uint64(key) {
|
2022-05-28 09:39:37 +00:00
|
|
|
b.Reset(value)
|
2023-01-17 13:16:50 +00:00
|
|
|
|
|
|
|
tmp.Child = r.ReadU64LE()
|
|
|
|
tmp.Parent = r.ReadU64LE()
|
|
|
|
tmp.Time = r.ReadVarUint()
|
|
|
|
if r.Err != nil {
|
|
|
|
return r.Err
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
2022-07-22 08:53:05 +00:00
|
|
|
if err := t.undo(&tmp, treeBucket, cKey[:]); err != nil {
|
2022-05-28 14:11:50 +00:00
|
|
|
return err
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
key, value = c.Prev()
|
|
|
|
}
|
|
|
|
|
2022-07-22 08:53:05 +00:00
|
|
|
for i := 0; i < len(ms); i++ {
|
|
|
|
// Loop invariant: key represents the next stored timestamp after ms[i].Time.
|
2022-07-22 06:47:12 +00:00
|
|
|
|
2022-07-22 08:53:05 +00:00
|
|
|
// 2. Insert the operation.
|
2023-01-17 13:16:50 +00:00
|
|
|
*lm = *ms[i]
|
2022-05-28 14:11:50 +00:00
|
|
|
if err := t.do(logBucket, treeBucket, cKey[:], lm); err != nil {
|
|
|
|
return err
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
2022-11-04 15:07:17 +00:00
|
|
|
|
2022-07-22 08:53:05 +00:00
|
|
|
// Cursor can be invalid, seek again.
|
|
|
|
binary.BigEndian.PutUint64(cKey[:], lm.Time)
|
|
|
|
_, _ = c.Seek(cKey[:8])
|
|
|
|
key, value = c.Next()
|
2022-04-22 13:22:40 +00:00
|
|
|
|
2022-07-22 08:53:05 +00:00
|
|
|
// 3. Re-apply all other operations.
|
|
|
|
for len(key) == 8 && (i == len(ms)-1 || binary.BigEndian.Uint64(key) < ms[i+1].Time) {
|
2023-01-17 13:16:50 +00:00
|
|
|
if err := t.logFromBytes(&tmp, value); err != nil {
|
2022-07-22 08:53:05 +00:00
|
|
|
return err
|
|
|
|
}
|
2023-01-17 13:16:50 +00:00
|
|
|
if err := t.redo(treeBucket, cKey[:], &tmp, value[16:]); err != nil {
|
2022-07-22 08:53:05 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
key, value = c.Next()
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-05-28 14:11:50 +00:00
|
|
|
return nil
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2023-01-25 11:12:02 +00:00
|
|
|
func (t *boltForest) do(lb *bbolt.Bucket, b *bbolt.Bucket, key []byte, op *Move) error {
|
2022-04-22 13:22:40 +00:00
|
|
|
binary.BigEndian.PutUint64(key, op.Time)
|
2023-01-17 13:16:50 +00:00
|
|
|
rawLog := t.logToBytes(op)
|
|
|
|
if err := lb.Put(key[:8], rawLog); err != nil {
|
2022-04-22 13:22:40 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2023-01-17 13:16:50 +00:00
|
|
|
return t.redo(b, key, op, rawLog[16:])
|
|
|
|
}
|
|
|
|
|
2023-01-25 11:12:02 +00:00
|
|
|
func (t *boltForest) redo(b *bbolt.Bucket, key []byte, op *Move, rawMeta []byte) error {
|
2023-01-17 13:16:50 +00:00
|
|
|
var err error
|
|
|
|
|
|
|
|
parent, ts, currMeta, inTree := t.getState(b, stateKey(key, op.Child))
|
|
|
|
if inTree {
|
|
|
|
err = t.putState(b, oldKey(key, op.Time), parent, ts, currMeta)
|
|
|
|
} else {
|
|
|
|
ts = op.Time
|
|
|
|
err = b.Delete(oldKey(key, op.Time))
|
|
|
|
}
|
|
|
|
|
|
|
|
if err != nil || op.Child == op.Parent || t.isAncestor(b, op.Child, op.Parent) {
|
|
|
|
return err
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2023-01-17 13:16:50 +00:00
|
|
|
if inTree {
|
|
|
|
if err := b.Delete(childrenKey(key, op.Child, parent)); err != nil {
|
2022-04-22 13:22:40 +00:00
|
|
|
return err
|
|
|
|
}
|
2023-01-17 13:16:50 +00:00
|
|
|
|
|
|
|
var meta Meta
|
|
|
|
if err := meta.FromBytes(currMeta); err != nil {
|
2022-04-22 13:22:40 +00:00
|
|
|
return err
|
|
|
|
}
|
2023-01-17 13:16:50 +00:00
|
|
|
for i := range meta.Items {
|
|
|
|
if isAttributeInternal(meta.Items[i].Key) {
|
|
|
|
key = internalKey(key, meta.Items[i].Key, string(meta.Items[i].Value), parent, op.Child)
|
2022-07-22 09:49:28 +00:00
|
|
|
err := b.Delete(key)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
2022-05-24 13:12:50 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
2023-01-17 13:16:50 +00:00
|
|
|
return t.addNode(b, key, op.Child, op.Parent, ts, op.Meta, rawMeta)
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// removeNode removes node keys from the tree except the children key or its parent.
|
2022-05-24 13:12:50 +00:00
|
|
|
func (t *boltForest) removeNode(b *bbolt.Bucket, key []byte, node, parent Node) error {
|
2023-01-17 13:16:50 +00:00
|
|
|
k := stateKey(key, node)
|
|
|
|
_, _, rawMeta, _ := t.getState(b, k)
|
|
|
|
|
2022-05-24 13:12:50 +00:00
|
|
|
var meta Meta
|
2023-01-17 13:16:50 +00:00
|
|
|
if err := meta.FromBytes(rawMeta); err == nil {
|
2022-05-24 13:12:50 +00:00
|
|
|
for i := range meta.Items {
|
|
|
|
if isAttributeInternal(meta.Items[i].Key) {
|
|
|
|
err := b.Delete(internalKey(nil, meta.Items[i].Key, string(meta.Items[i].Value), parent, node))
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2023-01-17 13:16:50 +00:00
|
|
|
return b.Delete(k)
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// addNode adds node keys to the tree except the timestamp key.
|
2023-01-17 13:16:50 +00:00
|
|
|
func (t *boltForest) addNode(b *bbolt.Bucket, key []byte, child, parent Node, time Timestamp, meta Meta, rawMeta []byte) error {
|
|
|
|
if err := t.putState(b, stateKey(key, child), parent, time, rawMeta); err != nil {
|
2022-04-22 13:22:40 +00:00
|
|
|
return err
|
|
|
|
}
|
2023-01-17 13:16:50 +00:00
|
|
|
|
|
|
|
err := b.Put(childrenKey(key, child, parent), []byte{1})
|
2022-05-24 13:12:50 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
for i := range meta.Items {
|
|
|
|
if !isAttributeInternal(meta.Items[i].Key) {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
key = internalKey(key, meta.Items[i].Key, string(meta.Items[i].Value), parent, child)
|
|
|
|
if len(meta.Items) == 1 {
|
|
|
|
err = b.Put(key, []byte{1})
|
|
|
|
} else {
|
|
|
|
err = b.Put(key, []byte{0})
|
|
|
|
}
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2023-01-25 11:12:02 +00:00
|
|
|
func (t *boltForest) undo(m *Move, b *bbolt.Bucket, key []byte) error {
|
2022-04-22 13:22:40 +00:00
|
|
|
if err := b.Delete(childrenKey(key, m.Child, m.Parent)); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2023-01-17 13:16:50 +00:00
|
|
|
parent, ts, rawMeta, ok := t.getState(b, oldKey(key, m.Time))
|
|
|
|
if !ok {
|
2022-05-24 13:12:50 +00:00
|
|
|
return t.removeNode(b, key, m.Child, m.Parent)
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
2023-01-17 13:16:50 +00:00
|
|
|
|
|
|
|
var meta Meta
|
|
|
|
if err := meta.FromBytes(rawMeta); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
return t.addNode(b, key, m.Child, parent, ts, meta, rawMeta)
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2022-07-22 09:49:28 +00:00
|
|
|
func (t *boltForest) isAncestor(b *bbolt.Bucket, parent, child Node) bool {
|
|
|
|
key := make([]byte, 9)
|
2023-01-17 13:16:50 +00:00
|
|
|
key[0] = 's'
|
2022-07-22 09:49:28 +00:00
|
|
|
for node := child; node != parent; {
|
|
|
|
binary.LittleEndian.PutUint64(key[1:], node)
|
2023-01-17 13:16:50 +00:00
|
|
|
parent, _, _, ok := t.getState(b, key)
|
|
|
|
if !ok {
|
2022-04-22 13:22:40 +00:00
|
|
|
return false
|
|
|
|
}
|
2023-01-17 13:16:50 +00:00
|
|
|
node = parent
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
|
|
|
|
// TreeGetByPath implements the Forest interface.
|
2023-04-13 12:36:20 +00:00
|
|
|
func (t *boltForest) TreeGetByPath(ctx context.Context, cid cidSDK.ID, treeID string, attr string, path []string, latest bool) ([]Node, error) {
|
2023-06-07 09:27:53 +00:00
|
|
|
var (
|
|
|
|
startedAt = time.Now()
|
|
|
|
success = false
|
|
|
|
)
|
|
|
|
defer func() {
|
|
|
|
t.metrics.AddMethodDuration("TreeGetByPath", time.Since(startedAt), success)
|
|
|
|
}()
|
|
|
|
|
2023-04-13 12:36:20 +00:00
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeGetByPath",
|
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", cid.EncodeToString()),
|
|
|
|
attribute.String("tree_id", treeID),
|
|
|
|
attribute.String("attr", attr),
|
|
|
|
attribute.Int("path_count", len(path)),
|
|
|
|
attribute.Bool("latest", latest),
|
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
2022-05-24 13:12:50 +00:00
|
|
|
if !isAttributeInternal(attr) {
|
|
|
|
return nil, ErrNotPathAttribute
|
|
|
|
}
|
|
|
|
|
2022-04-22 13:22:40 +00:00
|
|
|
if len(path) == 0 {
|
2023-06-07 09:27:53 +00:00
|
|
|
success = true
|
2022-04-22 13:22:40 +00:00
|
|
|
return nil, nil
|
|
|
|
}
|
|
|
|
|
2023-01-11 10:08:12 +00:00
|
|
|
t.modeMtx.RLock()
|
|
|
|
defer t.modeMtx.RUnlock()
|
|
|
|
|
|
|
|
if t.mode.NoMetabase() {
|
|
|
|
return nil, ErrDegradedMode
|
|
|
|
}
|
|
|
|
|
2022-04-22 13:22:40 +00:00
|
|
|
var nodes []Node
|
|
|
|
|
2023-06-07 09:27:53 +00:00
|
|
|
err := metaerr.Wrap(t.db.View(func(tx *bbolt.Tx) error {
|
2022-04-22 13:22:40 +00:00
|
|
|
treeRoot := tx.Bucket(bucketName(cid, treeID))
|
|
|
|
if treeRoot == nil {
|
|
|
|
return ErrTreeNotFound
|
|
|
|
}
|
|
|
|
|
|
|
|
b := treeRoot.Bucket(dataBucket)
|
|
|
|
|
2024-07-08 10:52:59 +00:00
|
|
|
i, curNodes, err := t.getPathPrefixMultiTraversal(b, attr, path[:len(path)-1])
|
2022-04-22 13:22:40 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if i < len(path)-1 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2023-01-17 13:16:50 +00:00
|
|
|
var maxTimestamp uint64
|
2022-04-22 13:22:40 +00:00
|
|
|
|
2022-05-24 13:12:50 +00:00
|
|
|
c := b.Cursor()
|
2022-04-22 13:22:40 +00:00
|
|
|
|
2024-07-08 10:52:59 +00:00
|
|
|
for i := range curNodes {
|
|
|
|
attrKey := internalKey(nil, attr, path[len(path)-1], curNodes[i], 0)
|
|
|
|
attrKey = attrKey[:len(attrKey)-8]
|
|
|
|
childKey, _ := c.Seek(attrKey)
|
|
|
|
for len(childKey) == len(attrKey)+8 && bytes.Equal(attrKey, childKey[:len(childKey)-8]) {
|
|
|
|
child := binary.LittleEndian.Uint64(childKey[len(childKey)-8:])
|
|
|
|
if latest {
|
|
|
|
_, ts, _, _ := t.getState(b, stateKey(make([]byte, 9), child))
|
|
|
|
if ts >= maxTimestamp {
|
|
|
|
nodes = append(nodes[:0], child)
|
|
|
|
maxTimestamp = ts
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
nodes = append(nodes, child)
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
2024-07-08 10:52:59 +00:00
|
|
|
childKey, _ = c.Next()
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
2023-06-15 10:19:36 +00:00
|
|
|
}))
|
2023-06-07 09:27:53 +00:00
|
|
|
success = err == nil
|
|
|
|
return nodes, err
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// TreeGetMeta implements the forest interface.
|
2023-04-13 12:36:20 +00:00
|
|
|
func (t *boltForest) TreeGetMeta(ctx context.Context, cid cidSDK.ID, treeID string, nodeID Node) (Meta, Node, error) {
|
2023-06-07 09:27:53 +00:00
|
|
|
var (
|
|
|
|
startedAt = time.Now()
|
|
|
|
success = false
|
|
|
|
)
|
|
|
|
defer func() {
|
|
|
|
t.metrics.AddMethodDuration("TreeGetMeta", time.Since(startedAt), success)
|
|
|
|
}()
|
|
|
|
|
2023-04-13 12:36:20 +00:00
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeGetMeta",
|
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", cid.EncodeToString()),
|
|
|
|
attribute.String("tree_id", treeID),
|
2023-08-16 15:30:33 +00:00
|
|
|
attribute.String("node_id", strconv.FormatUint(nodeID, 10)),
|
2023-04-13 12:36:20 +00:00
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
2023-01-11 10:08:12 +00:00
|
|
|
t.modeMtx.RLock()
|
|
|
|
defer t.modeMtx.RUnlock()
|
|
|
|
|
|
|
|
if t.mode.NoMetabase() {
|
|
|
|
return Meta{}, 0, ErrDegradedMode
|
|
|
|
}
|
|
|
|
|
2023-01-17 13:16:50 +00:00
|
|
|
key := stateKey(make([]byte, 9), nodeID)
|
2022-04-22 13:22:40 +00:00
|
|
|
|
|
|
|
var m Meta
|
2022-05-20 08:41:37 +00:00
|
|
|
var parentID uint64
|
|
|
|
|
2022-04-22 13:22:40 +00:00
|
|
|
err := t.db.View(func(tx *bbolt.Tx) error {
|
|
|
|
treeRoot := tx.Bucket(bucketName(cid, treeID))
|
|
|
|
if treeRoot == nil {
|
|
|
|
return ErrTreeNotFound
|
|
|
|
}
|
|
|
|
|
|
|
|
b := treeRoot.Bucket(dataBucket)
|
2023-01-17 13:16:50 +00:00
|
|
|
if data := b.Get(key); len(data) != 0 {
|
2022-05-20 08:41:37 +00:00
|
|
|
parentID = binary.LittleEndian.Uint64(data)
|
|
|
|
}
|
2023-01-17 13:16:50 +00:00
|
|
|
_, _, meta, _ := t.getState(b, stateKey(key, nodeID))
|
|
|
|
return m.FromBytes(meta)
|
2022-04-22 13:22:40 +00:00
|
|
|
})
|
2023-06-07 09:27:53 +00:00
|
|
|
success = err == nil
|
2023-06-15 10:19:36 +00:00
|
|
|
return m, parentID, metaerr.Wrap(err)
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2024-07-10 06:30:01 +00:00
|
|
|
func (t *boltForest) hasFewChildren(b *bbolt.Bucket, nodeIDs MultiNode, threshold int) bool {
|
2024-03-28 12:53:26 +00:00
|
|
|
key := make([]byte, 9)
|
|
|
|
key[0] = 'c'
|
|
|
|
|
|
|
|
count := 0
|
2024-07-10 06:30:01 +00:00
|
|
|
for _, nodeID := range nodeIDs {
|
|
|
|
binary.LittleEndian.PutUint64(key[1:], nodeID)
|
|
|
|
|
|
|
|
c := b.Cursor()
|
|
|
|
for k, _ := c.Seek(key); len(k) == childrenKeySize && binary.LittleEndian.Uint64(k[1:]) == nodeID; k, _ = c.Next() {
|
|
|
|
if count++; count > threshold {
|
|
|
|
return false
|
|
|
|
}
|
2024-03-28 12:53:26 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
|
|
|
|
// TreeSortedByFilename implements the Forest interface.
|
2024-07-10 06:30:01 +00:00
|
|
|
func (t *boltForest) TreeSortedByFilename(ctx context.Context, cid cidSDK.ID, treeID string, nodeIDs MultiNode, last *string, count int) ([]MultiNodeInfo, *string, error) {
|
2023-06-07 09:27:53 +00:00
|
|
|
var (
|
|
|
|
startedAt = time.Now()
|
|
|
|
success = false
|
|
|
|
)
|
|
|
|
defer func() {
|
2024-03-28 12:53:26 +00:00
|
|
|
t.metrics.AddMethodDuration("TreeSortedByFilename", time.Since(startedAt), success)
|
2023-06-07 09:27:53 +00:00
|
|
|
}()
|
|
|
|
|
2024-03-28 12:53:26 +00:00
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeSortedByFilename",
|
2023-04-13 12:36:20 +00:00
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", cid.EncodeToString()),
|
|
|
|
attribute.String("tree_id", treeID),
|
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
2023-01-11 10:08:12 +00:00
|
|
|
t.modeMtx.RLock()
|
|
|
|
defer t.modeMtx.RUnlock()
|
|
|
|
|
|
|
|
if t.mode.NoMetabase() {
|
2024-04-04 07:40:21 +00:00
|
|
|
return nil, last, ErrDegradedMode
|
2023-01-11 10:08:12 +00:00
|
|
|
}
|
2024-07-10 06:30:01 +00:00
|
|
|
if len(nodeIDs) == 0 {
|
|
|
|
return nil, last, errors.New("empty node list")
|
|
|
|
}
|
2023-01-11 10:08:12 +00:00
|
|
|
|
2024-03-28 12:53:26 +00:00
|
|
|
h := newHeap(last, count)
|
2022-04-29 10:06:10 +00:00
|
|
|
key := make([]byte, 9)
|
|
|
|
|
2023-07-11 08:39:17 +00:00
|
|
|
var result []NodeInfo
|
2024-03-28 12:53:26 +00:00
|
|
|
var fewChildren bool
|
2022-04-29 10:06:10 +00:00
|
|
|
|
|
|
|
err := t.db.View(func(tx *bbolt.Tx) error {
|
|
|
|
treeRoot := tx.Bucket(bucketName(cid, treeID))
|
|
|
|
if treeRoot == nil {
|
|
|
|
return ErrTreeNotFound
|
|
|
|
}
|
|
|
|
|
|
|
|
b := treeRoot.Bucket(dataBucket)
|
2024-03-28 12:53:26 +00:00
|
|
|
|
|
|
|
// If the node is a leaf, we could scan all filenames in the tree.
|
|
|
|
// To prevent this we first count the number of children: if it is less than
|
|
|
|
// the number of nodes we need to return, fallback to TreeGetChildren() implementation.
|
2024-07-10 06:30:01 +00:00
|
|
|
if fewChildren = t.hasFewChildren(b, nodeIDs, count); fewChildren {
|
2024-03-28 12:53:26 +00:00
|
|
|
var err error
|
2024-07-10 06:30:01 +00:00
|
|
|
result, err = t.getChildren(b, nodeIDs)
|
2024-03-28 12:53:26 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2024-07-10 06:30:01 +00:00
|
|
|
t.fillSortedChildren(b, nodeIDs, h)
|
2024-03-28 12:53:26 +00:00
|
|
|
|
|
|
|
for info, ok := h.pop(); ok; info, ok = h.pop() {
|
2024-07-10 06:30:01 +00:00
|
|
|
for _, id := range info.id {
|
|
|
|
childInfo, err := t.getChildInfo(b, key, id)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
result = append(result, childInfo)
|
2023-07-11 08:39:17 +00:00
|
|
|
}
|
2022-04-29 10:06:10 +00:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
})
|
2024-03-28 12:53:26 +00:00
|
|
|
|
|
|
|
success = err == nil
|
|
|
|
if err != nil {
|
|
|
|
return nil, last, metaerr.Wrap(err)
|
|
|
|
}
|
|
|
|
|
|
|
|
if fewChildren {
|
2024-04-04 07:40:21 +00:00
|
|
|
result = sortAndCut(result, last)
|
2024-03-28 12:53:26 +00:00
|
|
|
}
|
2024-07-10 06:30:01 +00:00
|
|
|
res := mergeNodeInfos(result)
|
|
|
|
if len(res) > count {
|
|
|
|
res = res[:count]
|
|
|
|
}
|
|
|
|
if len(res) != 0 {
|
|
|
|
s := string(findAttr(res[len(res)-1].Meta, AttributeFilename))
|
2024-04-04 07:40:21 +00:00
|
|
|
last = &s
|
2024-03-28 12:53:26 +00:00
|
|
|
}
|
2024-07-10 06:30:01 +00:00
|
|
|
return res, last, metaerr.Wrap(err)
|
2024-03-28 12:53:26 +00:00
|
|
|
}
|
|
|
|
|
2024-04-04 07:40:21 +00:00
|
|
|
func sortAndCut(result []NodeInfo, last *string) []NodeInfo {
|
|
|
|
var lastBytes []byte
|
|
|
|
if last != nil {
|
|
|
|
lastBytes = []byte(*last)
|
|
|
|
}
|
2024-04-02 12:02:14 +00:00
|
|
|
sort.Slice(result, func(i, j int) bool {
|
|
|
|
return bytes.Compare(result[i].Meta.GetAttr(AttributeFilename), result[j].Meta.GetAttr(AttributeFilename)) == -1
|
|
|
|
})
|
|
|
|
for i := range result {
|
2024-04-04 07:40:21 +00:00
|
|
|
if lastBytes == nil || bytes.Compare(lastBytes, result[i].Meta.GetAttr(AttributeFilename)) == -1 {
|
2024-04-02 12:02:14 +00:00
|
|
|
return result[i:]
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2024-03-28 12:53:26 +00:00
|
|
|
func (t *boltForest) getChildInfo(b *bbolt.Bucket, key []byte, childID Node) (NodeInfo, error) {
|
|
|
|
childInfo := NodeInfo{ID: childID}
|
|
|
|
parentID, _, metaBytes, found := t.getState(b, stateKey(key, childID))
|
|
|
|
if found {
|
|
|
|
childInfo.ParentID = parentID
|
|
|
|
if err := childInfo.Meta.FromBytes(metaBytes); err != nil {
|
|
|
|
return NodeInfo{}, err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return childInfo, nil
|
|
|
|
}
|
|
|
|
|
2024-07-10 06:30:01 +00:00
|
|
|
func (t *boltForest) fillSortedChildren(b *bbolt.Bucket, nodeIDs MultiNode, h *fixedHeap) {
|
2024-03-28 12:53:26 +00:00
|
|
|
c := b.Cursor()
|
|
|
|
prefix := internalKeyPrefix(nil, AttributeFilename)
|
|
|
|
|
|
|
|
length := uint16(0)
|
|
|
|
count := 0
|
2024-07-10 06:30:01 +00:00
|
|
|
|
|
|
|
var nodes []uint64
|
|
|
|
var lastFilename *string
|
2024-03-28 12:53:26 +00:00
|
|
|
for k, _ := c.Seek(prefix); len(k) > 0 && k[0] == 'i'; k, _ = c.Next() {
|
|
|
|
if len(k) < len(prefix)+2+16 {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
parentID := binary.LittleEndian.Uint64(k[len(k)-16:])
|
2024-07-10 06:30:01 +00:00
|
|
|
if !slices.Contains(nodeIDs, parentID) {
|
2024-03-28 12:53:26 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
actualLength := binary.LittleEndian.Uint16(k[len(prefix):])
|
|
|
|
childID := binary.LittleEndian.Uint64(k[len(k)-8:])
|
|
|
|
filename := string(k[len(prefix)+2 : len(k)-16])
|
2024-07-10 06:30:01 +00:00
|
|
|
|
|
|
|
if lastFilename == nil {
|
|
|
|
lastFilename = &filename
|
|
|
|
nodes = append(nodes, childID)
|
|
|
|
} else if *lastFilename == filename {
|
|
|
|
nodes = append(nodes, childID)
|
|
|
|
} else {
|
|
|
|
processed := h.push(nodes, *lastFilename)
|
|
|
|
nodes = MultiNode{childID}
|
|
|
|
lastFilename = &filename
|
|
|
|
if actualLength != length {
|
|
|
|
length = actualLength
|
|
|
|
count = 1
|
|
|
|
} else if processed {
|
|
|
|
if count++; count > h.count {
|
|
|
|
lastFilename = nil
|
|
|
|
nodes = nil
|
|
|
|
length = actualLength + 1
|
|
|
|
c.Seek(append(prefix, byte(length), byte(length>>8)))
|
|
|
|
c.Prev() // c.Next() will be performed by for loop
|
|
|
|
}
|
2024-03-28 12:53:26 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2024-07-10 06:30:01 +00:00
|
|
|
|
|
|
|
if len(nodes) != 0 && lastFilename != nil {
|
|
|
|
h.push(nodes, *lastFilename)
|
|
|
|
}
|
2024-03-28 12:53:26 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// TreeGetChildren implements the Forest interface.
|
|
|
|
func (t *boltForest) TreeGetChildren(ctx context.Context, cid cidSDK.ID, treeID string, nodeID Node) ([]NodeInfo, error) {
|
|
|
|
var (
|
|
|
|
startedAt = time.Now()
|
|
|
|
success = false
|
|
|
|
)
|
|
|
|
defer func() {
|
|
|
|
t.metrics.AddMethodDuration("TreeGetChildren", time.Since(startedAt), success)
|
|
|
|
}()
|
|
|
|
|
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeGetChildren",
|
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", cid.EncodeToString()),
|
|
|
|
attribute.String("tree_id", treeID),
|
|
|
|
attribute.String("node_id", strconv.FormatUint(nodeID, 10)),
|
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
|
|
|
t.modeMtx.RLock()
|
|
|
|
defer t.modeMtx.RUnlock()
|
|
|
|
|
|
|
|
if t.mode.NoMetabase() {
|
|
|
|
return nil, ErrDegradedMode
|
|
|
|
}
|
|
|
|
|
|
|
|
var result []NodeInfo
|
|
|
|
|
|
|
|
err := t.db.View(func(tx *bbolt.Tx) error {
|
|
|
|
treeRoot := tx.Bucket(bucketName(cid, treeID))
|
|
|
|
if treeRoot == nil {
|
|
|
|
return ErrTreeNotFound
|
|
|
|
}
|
|
|
|
|
|
|
|
b := treeRoot.Bucket(dataBucket)
|
|
|
|
|
|
|
|
var err error
|
2024-07-10 06:30:01 +00:00
|
|
|
result, err = t.getChildren(b, []Node{nodeID})
|
2024-03-28 12:53:26 +00:00
|
|
|
return err
|
|
|
|
})
|
2023-06-07 09:27:53 +00:00
|
|
|
success = err == nil
|
2023-07-11 08:39:17 +00:00
|
|
|
return result, metaerr.Wrap(err)
|
2022-04-29 10:06:10 +00:00
|
|
|
}
|
|
|
|
|
2024-07-10 06:30:01 +00:00
|
|
|
func (t *boltForest) getChildren(b *bbolt.Bucket, nodeIDs MultiNode) ([]NodeInfo, error) {
|
2024-03-28 12:53:26 +00:00
|
|
|
var result []NodeInfo
|
|
|
|
|
|
|
|
key := make([]byte, 9)
|
2024-07-10 06:30:01 +00:00
|
|
|
for _, nodeID := range nodeIDs {
|
|
|
|
key[0] = 'c'
|
|
|
|
binary.LittleEndian.PutUint64(key[1:], nodeID)
|
2024-03-28 12:53:26 +00:00
|
|
|
|
2024-07-10 06:30:01 +00:00
|
|
|
c := b.Cursor()
|
|
|
|
for k, _ := c.Seek(key); len(k) == childrenKeySize && binary.LittleEndian.Uint64(k[1:]) == nodeID; k, _ = c.Next() {
|
|
|
|
childID := binary.LittleEndian.Uint64(k[9:])
|
|
|
|
childInfo, err := t.getChildInfo(b, key, childID)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
result = append(result, childInfo)
|
2024-03-28 12:53:26 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return result, nil
|
|
|
|
}
|
|
|
|
|
2022-10-18 11:59:32 +00:00
|
|
|
// TreeList implements the Forest interface.
|
2023-04-13 12:36:20 +00:00
|
|
|
func (t *boltForest) TreeList(ctx context.Context, cid cidSDK.ID) ([]string, error) {
|
2023-06-07 09:27:53 +00:00
|
|
|
var (
|
|
|
|
startedAt = time.Now()
|
|
|
|
success = false
|
|
|
|
)
|
|
|
|
defer func() {
|
|
|
|
t.metrics.AddMethodDuration("TreeList", time.Since(startedAt), success)
|
|
|
|
}()
|
|
|
|
|
2023-04-13 12:36:20 +00:00
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeList",
|
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", cid.EncodeToString()),
|
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
2023-01-11 10:08:12 +00:00
|
|
|
t.modeMtx.RLock()
|
|
|
|
defer t.modeMtx.RUnlock()
|
|
|
|
|
|
|
|
if t.mode.NoMetabase() {
|
|
|
|
return nil, ErrDegradedMode
|
|
|
|
}
|
|
|
|
|
2022-10-18 11:59:32 +00:00
|
|
|
var ids []string
|
2023-01-18 06:38:42 +00:00
|
|
|
cidRaw := make([]byte, 32)
|
|
|
|
cid.Encode(cidRaw)
|
|
|
|
|
2022-10-18 11:59:32 +00:00
|
|
|
cidLen := len(cidRaw)
|
|
|
|
|
|
|
|
err := t.db.View(func(tx *bbolt.Tx) error {
|
|
|
|
c := tx.Cursor()
|
|
|
|
for k, _ := c.Seek(cidRaw); k != nil; k, _ = c.Next() {
|
|
|
|
if !bytes.HasPrefix(k, cidRaw) {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
ids = append(ids, string(k[cidLen:]))
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
})
|
|
|
|
if err != nil {
|
2023-06-15 10:19:36 +00:00
|
|
|
return nil, metaerr.Wrap(fmt.Errorf("could not list trees: %w", err))
|
2022-10-18 11:59:32 +00:00
|
|
|
}
|
2023-06-07 09:27:53 +00:00
|
|
|
success = true
|
2022-10-18 11:59:32 +00:00
|
|
|
return ids, nil
|
|
|
|
}
|
|
|
|
|
2022-05-11 13:29:04 +00:00
|
|
|
// TreeGetOpLog implements the pilorama.Forest interface.
|
2023-04-13 12:36:20 +00:00
|
|
|
func (t *boltForest) TreeGetOpLog(ctx context.Context, cid cidSDK.ID, treeID string, height uint64) (Move, error) {
|
2023-06-07 09:27:53 +00:00
|
|
|
var (
|
|
|
|
startedAt = time.Now()
|
|
|
|
success = false
|
|
|
|
)
|
|
|
|
defer func() {
|
|
|
|
t.metrics.AddMethodDuration("TreeGetOpLog", time.Since(startedAt), success)
|
|
|
|
}()
|
|
|
|
|
2023-04-13 12:36:20 +00:00
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeGetOpLog",
|
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", cid.EncodeToString()),
|
|
|
|
attribute.String("tree_id", treeID),
|
2023-08-16 15:30:33 +00:00
|
|
|
attribute.String("height", strconv.FormatUint(height, 10)),
|
2023-04-13 12:36:20 +00:00
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
2023-01-11 10:08:12 +00:00
|
|
|
t.modeMtx.RLock()
|
|
|
|
defer t.modeMtx.RUnlock()
|
|
|
|
|
|
|
|
if t.mode.NoMetabase() {
|
|
|
|
return Move{}, ErrDegradedMode
|
|
|
|
}
|
|
|
|
|
2022-05-11 13:29:04 +00:00
|
|
|
key := make([]byte, 8)
|
|
|
|
binary.BigEndian.PutUint64(key, height)
|
|
|
|
|
|
|
|
var lm Move
|
|
|
|
|
|
|
|
err := t.db.View(func(tx *bbolt.Tx) error {
|
|
|
|
treeRoot := tx.Bucket(bucketName(cid, treeID))
|
|
|
|
if treeRoot == nil {
|
|
|
|
return ErrTreeNotFound
|
|
|
|
}
|
|
|
|
|
|
|
|
c := treeRoot.Bucket(logBucket).Cursor()
|
|
|
|
if _, data := c.Seek(key); data != nil {
|
|
|
|
return t.moveFromBytes(&lm, data)
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
})
|
2023-06-07 09:27:53 +00:00
|
|
|
success = err == nil
|
2023-06-15 10:19:36 +00:00
|
|
|
return lm, metaerr.Wrap(err)
|
2022-05-11 13:29:04 +00:00
|
|
|
}
|
|
|
|
|
2022-09-07 08:46:13 +00:00
|
|
|
// TreeDrop implements the pilorama.Forest interface.
|
2023-04-13 12:36:20 +00:00
|
|
|
func (t *boltForest) TreeDrop(ctx context.Context, cid cidSDK.ID, treeID string) error {
|
2023-06-07 09:27:53 +00:00
|
|
|
var (
|
|
|
|
startedAt = time.Now()
|
|
|
|
success = false
|
|
|
|
)
|
|
|
|
defer func() {
|
|
|
|
t.metrics.AddMethodDuration("TreeDrop", time.Since(startedAt), success)
|
|
|
|
}()
|
|
|
|
|
2023-04-13 12:36:20 +00:00
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeDrop",
|
|
|
|
trace.WithAttributes(
|
|
|
|
attribute.String("container_id", cid.EncodeToString()),
|
|
|
|
attribute.String("tree_id", treeID),
|
|
|
|
),
|
|
|
|
)
|
|
|
|
defer span.End()
|
|
|
|
|
2023-01-11 10:08:12 +00:00
|
|
|
t.modeMtx.RLock()
|
|
|
|
defer t.modeMtx.RUnlock()
|
|
|
|
|
|
|
|
if t.mode.NoMetabase() {
|
|
|
|
return ErrDegradedMode
|
|
|
|
} else if t.mode.ReadOnly() {
|
|
|
|
return ErrReadOnlyMode
|
|
|
|
}
|
|
|
|
|
2023-06-07 09:27:53 +00:00
|
|
|
err := metaerr.Wrap(t.db.Batch(func(tx *bbolt.Tx) error {
|
2022-11-08 12:32:38 +00:00
|
|
|
if treeID == "" {
|
|
|
|
c := tx.Cursor()
|
2023-01-18 06:38:42 +00:00
|
|
|
prefix := make([]byte, 32)
|
|
|
|
cid.Encode(prefix)
|
2023-11-15 09:03:31 +00:00
|
|
|
for k, _ := c.Seek(prefix); k != nil && bytes.HasPrefix(k, prefix); k, _ = c.Seek(prefix) {
|
2022-11-08 12:32:38 +00:00
|
|
|
err := tx.DeleteBucket(k)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2023-11-15 09:03:31 +00:00
|
|
|
_, _ = c.First() // rewind the cursor to the root page
|
2022-11-08 12:32:38 +00:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
2022-09-07 08:46:13 +00:00
|
|
|
err := tx.DeleteBucket(bucketName(cid, treeID))
|
|
|
|
if errors.Is(err, bbolt.ErrBucketNotFound) {
|
|
|
|
return ErrTreeNotFound
|
|
|
|
}
|
|
|
|
return err
|
2023-06-15 10:19:36 +00:00
|
|
|
}))
|
2023-06-07 09:27:53 +00:00
|
|
|
success = err == nil
|
|
|
|
return err
|
2022-09-07 08:46:13 +00:00
|
|
|
}
|
|
|
|
|
2024-02-05 11:09:58 +00:00
|
|
|
// TreeListTrees implements ForestStorage.
|
|
|
|
func (t *boltForest) TreeListTrees(ctx context.Context, prm TreeListTreesPrm) (*TreeListTreesResult, error) {
|
|
|
|
var (
|
|
|
|
startedAt = time.Now()
|
|
|
|
success = false
|
|
|
|
)
|
|
|
|
defer func() {
|
|
|
|
t.metrics.AddMethodDuration("TreeListTrees", time.Since(startedAt), success)
|
|
|
|
}()
|
|
|
|
|
|
|
|
_, span := tracing.StartSpanFromContext(ctx, "boltForest.TreeListTrees")
|
|
|
|
defer span.End()
|
|
|
|
|
|
|
|
t.modeMtx.RLock()
|
|
|
|
defer t.modeMtx.RUnlock()
|
|
|
|
|
|
|
|
if t.mode.NoMetabase() {
|
|
|
|
return nil, ErrDegradedMode
|
|
|
|
}
|
|
|
|
|
|
|
|
batchSize := prm.BatchSize
|
|
|
|
if batchSize <= 0 {
|
|
|
|
batchSize = treeListTreesBatchSizeDefault
|
|
|
|
}
|
|
|
|
var res TreeListTreesResult
|
|
|
|
err := metaerr.Wrap(t.db.View(func(tx *bbolt.Tx) error {
|
|
|
|
c := tx.Cursor()
|
|
|
|
checkNextPageToken := true
|
|
|
|
for k, _ := c.Seek(prm.NextPageToken); k != nil; k, _ = c.Next() {
|
|
|
|
if bytes.Equal(k, dataBucket) || bytes.Equal(k, logBucket) {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
if checkNextPageToken && bytes.Equal(k, prm.NextPageToken) {
|
|
|
|
checkNextPageToken = false
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
var contID cidSDK.ID
|
|
|
|
if err := contID.Decode(k[:32]); err != nil {
|
|
|
|
return fmt.Errorf("failed to decode containerID: %w", err)
|
|
|
|
}
|
|
|
|
res.Items = append(res.Items, ContainerIDTreeID{
|
|
|
|
CID: contID,
|
|
|
|
TreeID: string(k[32:]),
|
|
|
|
})
|
|
|
|
|
|
|
|
if len(res.Items) == batchSize {
|
|
|
|
res.NextPageToken = make([]byte, len(k))
|
|
|
|
copy(res.NextPageToken, k)
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}))
|
|
|
|
success = err == nil
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return &res, nil
|
|
|
|
}
|
|
|
|
|
2024-07-08 10:52:59 +00:00
|
|
|
func (t *boltForest) getPathPrefixMultiTraversal(bTree *bbolt.Bucket, attr string, path []string) (int, []Node, error) {
|
|
|
|
c := bTree.Cursor()
|
|
|
|
|
|
|
|
var curNodes []Node
|
|
|
|
nextNodes := []Node{RootID}
|
|
|
|
var attrKey []byte
|
|
|
|
|
|
|
|
for i := range path {
|
|
|
|
curNodes, nextNodes = nextNodes, curNodes[:0]
|
|
|
|
for j := range curNodes {
|
|
|
|
attrKey = internalKey(attrKey, attr, path[i], curNodes[j], 0)
|
|
|
|
attrKey = attrKey[:len(attrKey)-8]
|
|
|
|
|
|
|
|
childKey, value := c.Seek(attrKey)
|
|
|
|
for len(childKey) == len(attrKey)+8 && bytes.Equal(attrKey, childKey[:len(childKey)-8]) {
|
|
|
|
if len(value) == 1 && value[0] == 1 {
|
|
|
|
nextNodes = append(nextNodes, binary.LittleEndian.Uint64(childKey[len(childKey)-8:]))
|
|
|
|
}
|
|
|
|
childKey, value = c.Next()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(nextNodes) == 0 {
|
|
|
|
return i, curNodes, nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return len(path), nextNodes, nil
|
|
|
|
}
|
|
|
|
|
2022-04-22 13:22:40 +00:00
|
|
|
func (t *boltForest) getPathPrefix(bTree *bbolt.Bucket, attr string, path []string) (int, Node, error) {
|
|
|
|
c := bTree.Cursor()
|
|
|
|
|
|
|
|
var curNode Node
|
2022-05-24 13:12:50 +00:00
|
|
|
var attrKey []byte
|
2022-04-22 13:22:40 +00:00
|
|
|
|
|
|
|
loop:
|
|
|
|
for i := range path {
|
2022-05-24 13:12:50 +00:00
|
|
|
attrKey = internalKey(attrKey, attr, path[i], curNode, 0)
|
|
|
|
attrKey = attrKey[:len(attrKey)-8]
|
2022-04-22 13:22:40 +00:00
|
|
|
|
2022-05-24 13:12:50 +00:00
|
|
|
childKey, value := c.Seek(attrKey)
|
|
|
|
for len(childKey) == len(attrKey)+8 && bytes.Equal(attrKey, childKey[:len(childKey)-8]) {
|
|
|
|
if len(value) == 1 && value[0] == 1 {
|
|
|
|
curNode = binary.LittleEndian.Uint64(childKey[len(childKey)-8:])
|
2022-05-23 11:32:24 +00:00
|
|
|
continue loop
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
2022-05-24 13:12:50 +00:00
|
|
|
childKey, value = c.Next()
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
2022-05-24 13:12:50 +00:00
|
|
|
|
2022-04-22 13:22:40 +00:00
|
|
|
return i, curNode, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
return len(path), curNode, nil
|
|
|
|
}
|
|
|
|
|
2022-05-11 13:29:04 +00:00
|
|
|
func (t *boltForest) moveFromBytes(m *Move, data []byte) error {
|
2023-01-17 13:16:50 +00:00
|
|
|
return t.logFromBytes(m, data)
|
2022-05-11 13:29:04 +00:00
|
|
|
}
|
|
|
|
|
2023-01-25 11:12:02 +00:00
|
|
|
func (t *boltForest) logFromBytes(lm *Move, data []byte) error {
|
2023-01-17 13:16:50 +00:00
|
|
|
lm.Child = binary.LittleEndian.Uint64(data)
|
|
|
|
lm.Parent = binary.LittleEndian.Uint64(data[8:])
|
|
|
|
return lm.Meta.FromBytes(data[16:])
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2023-01-25 11:12:02 +00:00
|
|
|
func (t *boltForest) logToBytes(lm *Move) []byte {
|
2022-04-22 13:22:40 +00:00
|
|
|
w := io.NewBufBinWriter()
|
2022-05-28 09:39:37 +00:00
|
|
|
size := 8 + 8 + lm.Meta.Size() + 1
|
2023-03-07 13:38:26 +00:00
|
|
|
// if lm.HasOld {
|
2023-01-17 13:16:50 +00:00
|
|
|
// size += 8 + lm.Old.Meta.Size()
|
2023-03-07 13:38:26 +00:00
|
|
|
// }
|
2022-05-28 09:39:37 +00:00
|
|
|
|
|
|
|
w.Grow(size)
|
2022-04-22 13:22:40 +00:00
|
|
|
w.WriteU64LE(lm.Child)
|
|
|
|
w.WriteU64LE(lm.Parent)
|
2022-05-28 09:39:37 +00:00
|
|
|
lm.Meta.EncodeBinary(w.BinWriter)
|
2023-03-07 13:38:26 +00:00
|
|
|
// w.WriteBool(lm.HasOld)
|
|
|
|
// if lm.HasOld {
|
2023-01-17 13:16:50 +00:00
|
|
|
// w.WriteU64LE(lm.Old.Parent)
|
|
|
|
// lm.Old.Meta.EncodeBinary(w.BinWriter)
|
2023-03-07 13:38:26 +00:00
|
|
|
// }
|
2022-04-22 13:22:40 +00:00
|
|
|
return w.Bytes()
|
|
|
|
}
|
|
|
|
|
|
|
|
func bucketName(cid cidSDK.ID, treeID string) []byte {
|
2023-01-18 06:38:42 +00:00
|
|
|
treeRoot := make([]byte, 32+len(treeID))
|
|
|
|
cid.Encode(treeRoot)
|
|
|
|
copy(treeRoot[32:], treeID)
|
|
|
|
return treeRoot
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2023-01-17 13:16:50 +00:00
|
|
|
// 'o' + time -> old meta.
|
|
|
|
func oldKey(key []byte, ts Timestamp) []byte {
|
|
|
|
key[0] = 'o'
|
|
|
|
binary.LittleEndian.PutUint64(key[1:], ts)
|
2022-04-22 13:22:40 +00:00
|
|
|
return key[:9]
|
|
|
|
}
|
|
|
|
|
2023-01-17 13:16:50 +00:00
|
|
|
// 's' + child ID -> parent + timestamp of the first appearance + meta.
|
|
|
|
func stateKey(key []byte, child Node) []byte {
|
|
|
|
key[0] = 's'
|
2022-04-22 13:22:40 +00:00
|
|
|
binary.LittleEndian.PutUint64(key[1:], child)
|
|
|
|
return key[:9]
|
|
|
|
}
|
|
|
|
|
2023-01-17 13:16:50 +00:00
|
|
|
func (t *boltForest) putState(b *bbolt.Bucket, key []byte, parent Node, timestamp Timestamp, meta []byte) error {
|
|
|
|
data := make([]byte, len(meta)+8+8)
|
|
|
|
binary.LittleEndian.PutUint64(data, parent)
|
|
|
|
binary.LittleEndian.PutUint64(data[8:], timestamp)
|
|
|
|
copy(data[16:], meta)
|
|
|
|
return b.Put(key, data)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (t *boltForest) getState(b *bbolt.Bucket, key []byte) (Node, Timestamp, []byte, bool) {
|
|
|
|
data := b.Get(key)
|
|
|
|
if data == nil {
|
|
|
|
return 0, 0, nil, false
|
|
|
|
}
|
|
|
|
|
|
|
|
parent := binary.LittleEndian.Uint64(data)
|
|
|
|
timestamp := binary.LittleEndian.Uint64(data[8:])
|
|
|
|
return parent, timestamp, data[16:], true
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2022-10-17 12:03:55 +00:00
|
|
|
// 'c' + parent (id) + child (id) -> 0/1.
|
2022-04-22 13:22:40 +00:00
|
|
|
func childrenKey(key []byte, child, parent Node) []byte {
|
|
|
|
key[0] = 'c'
|
|
|
|
binary.LittleEndian.PutUint64(key[1:], parent)
|
|
|
|
binary.LittleEndian.PutUint64(key[9:], child)
|
2023-06-20 10:39:18 +00:00
|
|
|
return key[:childrenKeySize]
|
2022-04-22 13:22:40 +00:00
|
|
|
}
|
|
|
|
|
2024-03-28 12:53:26 +00:00
|
|
|
func internalKeyPrefix(key []byte, k string) []byte {
|
|
|
|
key = key[:0]
|
|
|
|
key = append(key, 'i')
|
|
|
|
|
|
|
|
l := len(k)
|
|
|
|
key = append(key, byte(l), byte(l>>8))
|
|
|
|
key = append(key, k...)
|
|
|
|
return key
|
|
|
|
}
|
|
|
|
|
2022-10-17 12:03:55 +00:00
|
|
|
// 'i' + attribute name (string) + attribute value (string) + parent (id) + node (id) -> 0/1.
|
2022-05-24 13:12:50 +00:00
|
|
|
func internalKey(key []byte, k, v string, parent, node Node) []byte {
|
|
|
|
size := 1 /* prefix */ + 2*2 /* len */ + 2*8 /* nodes */ + len(k) + len(v)
|
|
|
|
if cap(key) < size {
|
|
|
|
key = make([]byte, 0, size)
|
|
|
|
}
|
|
|
|
|
2024-03-28 12:53:26 +00:00
|
|
|
key = internalKeyPrefix(key, k)
|
2022-05-24 13:12:50 +00:00
|
|
|
|
2024-03-28 12:53:26 +00:00
|
|
|
l := len(v)
|
2022-05-24 13:12:50 +00:00
|
|
|
key = append(key, byte(l), byte(l>>8))
|
|
|
|
key = append(key, v...)
|
|
|
|
|
|
|
|
var raw [8]byte
|
|
|
|
binary.LittleEndian.PutUint64(raw[:], parent)
|
|
|
|
key = append(key, raw[:]...)
|
|
|
|
|
|
|
|
binary.LittleEndian.PutUint64(raw[:], node)
|
|
|
|
key = append(key, raw[:]...)
|
|
|
|
return key
|
|
|
|
}
|