package meta

import (
	"context"
	"encoding/binary"
	"errors"
	"fmt"
	gio "io"
	"strconv"
	"time"

	objectV2 "git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/object"
	objectCore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
	storagelog "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/log"
	"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
	"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util"
	"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
	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"
	"github.com/nspcc-dev/neo-go/pkg/io"
	"go.etcd.io/bbolt"
	"go.opentelemetry.io/otel/attribute"
	"go.opentelemetry.io/otel/trace"
)

type (
	namedBucketItem struct {
		name, key, val []byte
	}
)

// PutPrm groups the parameters of Put operation.
type PutPrm struct {
	obj *objectSDK.Object

	id []byte

	indexAttributes bool
}

// PutRes groups the resulting values of Put operation.
type PutRes struct {
	Inserted bool
}

// SetObject is a Put option to set object to save.
func (p *PutPrm) SetObject(obj *objectSDK.Object) {
	p.obj = obj
}

// SetStorageID is a Put option to set storage ID to save.
func (p *PutPrm) SetStorageID(id []byte) {
	p.id = id
}

func (p *PutPrm) SetIndexAttributes(v bool) {
	p.indexAttributes = v
}

var (
	ErrUnknownObjectType        = errors.New("unknown object type")
	ErrIncorrectSplitInfoUpdate = errors.New("updating split info on object without it")
	ErrIncorrectRootObject      = errors.New("invalid root object")
)

// Put saves object header in metabase. Object payload expected to be cut.
//
// Returns an error of type apistatus.ObjectAlreadyRemoved if object has been placed in graveyard.
// Returns the object.ErrObjectIsExpired if the object is presented but already expired.
func (db *DB) Put(ctx context.Context, prm PutPrm) (res PutRes, err error) {
	var (
		startedAt = time.Now()
		success   = false
	)
	defer func() {
		db.metrics.AddMethodDuration("Put", time.Since(startedAt), success)
	}()

	_, span := tracing.StartSpanFromContext(ctx, "metabase.Put",
		trace.WithAttributes(
			attribute.String("address", objectCore.AddressOf(prm.obj).EncodeToString()),
		))
	defer span.End()

	db.modeMtx.RLock()
	defer db.modeMtx.RUnlock()

	if db.mode.NoMetabase() {
		return res, ErrDegradedMode
	} else if db.mode.ReadOnly() {
		return res, ErrReadOnlyMode
	}

	currEpoch := db.epochState.CurrentEpoch()

	err = db.boltDB.Batch(func(tx *bbolt.Tx) error {
		var e error
		res, e = db.put(tx, prm.obj, prm.id, nil, currEpoch, prm.indexAttributes)
		return e
	})
	if err == nil {
		success = true
		storagelog.Write(db.log,
			storagelog.AddressField(objectCore.AddressOf(prm.obj)),
			storagelog.OpField("metabase PUT"))
	}

	return res, metaerr.Wrap(err)
}

func (db *DB) put(tx *bbolt.Tx,
	obj *objectSDK.Object,
	id []byte,
	si *objectSDK.SplitInfo,
	currEpoch uint64,
	indexAttributes bool,
) (PutRes, error) {
	cnr, ok := obj.ContainerID()
	if !ok {
		return PutRes{}, errors.New("missing container in object")
	}

	isParent := si != nil

	exists, _, err := db.exists(tx, objectCore.AddressOf(obj), oid.Address{}, currEpoch)

	var splitInfoError *objectSDK.SplitInfoError
	if errors.As(err, &splitInfoError) {
		exists = true // object exists, however it is virtual
	} else if err != nil {
		return PutRes{}, err // return any error besides SplitInfoError
	}

	if exists {
		return PutRes{}, db.updateObj(tx, obj, id, si, isParent)
	}

	return PutRes{Inserted: true}, db.insertObject(tx, obj, id, si, isParent, cnr, currEpoch, indexAttributes)
}

func (db *DB) updateObj(tx *bbolt.Tx, obj *objectSDK.Object, id []byte, si *objectSDK.SplitInfo, isParent bool) error {
	// most right child and split header overlap parent so we have to
	// check if object exists to not overwrite it twice

	// When storage engine moves objects between different sub-storages,
	// it calls metabase.Put method with new storage ID, thus triggering this code.
	if !isParent && id != nil {
		return setStorageID(tx, objectCore.AddressOf(obj), id, true)
	}

	// when storage already has last object in split hierarchy and there is
	// a linking object to put (or vice versa), we should update split info
	// with object ids of these objects
	if isParent {
		return updateSplitInfo(tx, objectCore.AddressOf(obj), si)
	}

	return nil
}

func (db *DB) insertObject(tx *bbolt.Tx, obj *objectSDK.Object, id []byte, si *objectSDK.SplitInfo, isParent bool, cnr cid.ID, currEpoch uint64, indexAttributes bool) error {
	if par := obj.Parent(); par != nil && !isParent { // limit depth by two
		parentSI, err := splitInfoFromObject(obj)
		if err != nil {
			return err
		}

		_, err = db.put(tx, par, id, parentSI, currEpoch, indexAttributes)
		if err != nil {
			return err
		}
	}

	err := putUniqueIndexes(tx, obj, si, id)
	if err != nil {
		return fmt.Errorf("can't put unique indexes: %w", err)
	}

	err = updateListIndexes(tx, obj, putListIndexItem)
	if err != nil {
		return fmt.Errorf("can't put list indexes: %w", err)
	}

	if indexAttributes {
		err = updateFKBTIndexes(tx, obj, putFKBTIndexItem)
		if err != nil {
			return fmt.Errorf("can't put fake bucket tree indexes: %w", err)
		}
	}

	// update container volume size estimation
	if obj.Type() == objectSDK.TypeRegular && !isParent {
		err = changeContainerSize(tx, cnr, obj.PayloadSize(), true)
		if err != nil {
			return err
		}
	}

	if !isParent {
		if err = db.incCounters(tx, cnr, IsUserObject(obj)); err != nil {
			return err
		}
	}

	return nil
}

func putUniqueIndexes(tx *bbolt.Tx, obj *objectSDK.Object, si *objectSDK.SplitInfo, id []byte) error {
	isParent := si != nil
	addr := objectCore.AddressOf(obj)
	objKey := objectKey(addr.Object(), make([]byte, objectKeySize))

	bucketName := make([]byte, bucketKeySize)
	if !isParent {
		err := putRawObjectData(tx, obj, bucketName, addr, objKey)
		if err != nil {
			return err
		}
		if id != nil {
			if err = setStorageID(tx, objectCore.AddressOf(obj), id, false); err != nil {
				return err
			}
		}
	}

	if err := putExpirationEpoch(tx, obj, addr, objKey); err != nil {
		return err
	}

	return putSplitInfo(tx, obj, bucketName, addr, si, objKey)
}

func putRawObjectData(tx *bbolt.Tx, obj *objectSDK.Object, bucketName []byte, addr oid.Address, objKey []byte) error {
	switch obj.Type() {
	case objectSDK.TypeRegular:
		bucketName = primaryBucketName(addr.Container(), bucketName)
	case objectSDK.TypeTombstone:
		bucketName = tombstoneBucketName(addr.Container(), bucketName)
	case objectSDK.TypeLock:
		bucketName = bucketNameLockers(addr.Container(), bucketName)
	default:
		return ErrUnknownObjectType
	}
	rawObject, err := obj.CutPayload().Marshal()
	if err != nil {
		return fmt.Errorf("can't marshal object header: %w", err)
	}
	return putUniqueIndexItem(tx, namedBucketItem{
		name: bucketName,
		key:  objKey,
		val:  rawObject,
	})
}

func putExpirationEpoch(tx *bbolt.Tx, obj *objectSDK.Object, addr oid.Address, objKey []byte) error {
	if expEpoch, ok := hasExpirationEpoch(obj); ok {
		err := putUniqueIndexItem(tx, namedBucketItem{
			name: expEpochToObjectBucketName,
			key:  expirationEpochKey(expEpoch, addr.Container(), addr.Object()),
			val:  zeroValue,
		})
		if err != nil {
			return err
		}
		val := make([]byte, epochSize)
		binary.LittleEndian.PutUint64(val, expEpoch)
		err = putUniqueIndexItem(tx, namedBucketItem{
			name: objectToExpirationEpochBucketName(addr.Container(), make([]byte, bucketKeySize)),
			key:  objKey,
			val:  val,
		})
		if err != nil {
			return err
		}
	}
	return nil
}

func putSplitInfo(tx *bbolt.Tx, obj *objectSDK.Object, bucketName []byte, addr oid.Address, si *objectSDK.SplitInfo, objKey []byte) error {
	if obj.Type() == objectSDK.TypeRegular && !obj.HasParent() {
		if ecHead := obj.ECHeader(); ecHead != nil {
			parentID := ecHead.Parent()
			if ecHead.ParentSplitID() != nil {
				parentSplitParentID := ecHead.ParentSplitParentID()
				if parentSplitParentID == nil {
					return nil
				}

				si = objectSDK.NewSplitInfo()
				si.SetSplitID(ecHead.ParentSplitID())
				si.SetLastPart(ecHead.Parent())

				parentID = *parentSplitParentID
			}
			objKey = objectKey(parentID, objKey)
		}
		return updateSplitInfoIndex(tx, objKey, addr.Container(), bucketName, si)
	}
	return nil
}

func updateSplitInfoIndex(tx *bbolt.Tx, objKey []byte, cnr cid.ID, bucketName []byte, si *objectSDK.SplitInfo) error {
	return updateUniqueIndexItem(tx, namedBucketItem{
		name: rootBucketName(cnr, bucketName),
		key:  objKey,
	}, func(old, _ []byte) ([]byte, error) {
		switch {
		case si == nil && old == nil:
			return []byte{}, nil
		case si == nil:
			return old, nil
		case old == nil:
			return si.Marshal()
		default:
			oldSI := objectSDK.NewSplitInfo()
			if err := oldSI.Unmarshal(old); err != nil {
				return nil, err
			}
			si = util.MergeSplitInfo(si, oldSI)
			return si.Marshal()
		}
	})
}

type updateIndexItemFunc = func(tx *bbolt.Tx, item namedBucketItem) error

func updateListIndexes(tx *bbolt.Tx, obj *objectSDK.Object, f updateIndexItemFunc) error {
	idObj, _ := obj.ID()
	cnr, _ := obj.ContainerID()
	objKey := objectKey(idObj, make([]byte, objectKeySize))
	bucketName := make([]byte, bucketKeySize)

	idParent, ok := obj.ParentID()

	// index parent ids
	if ok {
		err := f(tx, namedBucketItem{
			name: parentBucketName(cnr, bucketName),
			key:  objectKey(idParent, make([]byte, objectKeySize)),
			val:  objKey,
		})
		if err != nil {
			return err
		}
	}

	// index split ids
	if obj.SplitID() != nil {
		err := f(tx, namedBucketItem{
			name: splitBucketName(cnr, bucketName),
			key:  obj.SplitID().ToV2(),
			val:  objKey,
		})
		if err != nil {
			return err
		}
	}

	if ech := obj.ECHeader(); ech != nil {
		err := f(tx, namedBucketItem{
			name: ecInfoBucketName(cnr, bucketName),
			key:  objectKey(ech.Parent(), make([]byte, objectKeySize)),
			val:  objKey,
		})
		if err != nil {
			return err
		}

		if ech.ParentSplitID() != nil {
			objKey := objectKey(ech.Parent(), make([]byte, objectKeySize))
			err := f(tx, namedBucketItem{
				name: splitBucketName(cnr, bucketName),
				key:  ech.ParentSplitID().ToV2(),
				val:  objKey,
			})
			if err != nil {
				return err
			}
		}

		if parentSplitParentID := ech.ParentSplitParentID(); parentSplitParentID != nil {
			objKey := objectKey(ech.Parent(), make([]byte, objectKeySize))
			err := f(tx, namedBucketItem{
				name: parentBucketName(cnr, bucketName),
				key:  objectKey(*parentSplitParentID, make([]byte, objectKeySize)),
				val:  objKey,
			})
			if err != nil {
				return err
			}
		}
	}

	return nil
}

var indexedAttributes = map[string]struct{}{
	"S3-Access-Box-CRDT-Name":   {},
	objectSDK.AttributeFilePath: {},
}

// IsAtrributeIndexed returns True if attribute is indexed by metabase.
func IsAtrributeIndexed(attr string) bool {
	_, found := indexedAttributes[attr]
	return found
}

func updateFKBTIndexes(tx *bbolt.Tx, obj *objectSDK.Object, f updateIndexItemFunc) error {
	id, _ := obj.ID()
	cnr, _ := obj.ContainerID()
	objKey := objectKey(id, make([]byte, objectKeySize))

	key := make([]byte, bucketKeySize)
	var attrs []objectSDK.Attribute
	if obj.ECHeader() != nil {
		attrs = obj.ECHeader().ParentAttributes()
		objKey = objectKey(obj.ECHeader().Parent(), make([]byte, objectKeySize))
	} else {
		attrs = obj.Attributes()
	}

	// user specified attributes
	for i := range attrs {
		if !IsAtrributeIndexed(attrs[i].Key()) {
			continue
		}
		key = attributeBucketName(cnr, attrs[i].Key(), key)
		err := f(tx, namedBucketItem{
			name: key,
			key:  []byte(attrs[i].Value()),
			val:  objKey,
		})
		if err != nil {
			return err
		}
	}

	return nil
}

func hasExpirationEpoch(obj *objectSDK.Object) (uint64, bool) {
	attributes := obj.Attributes()
	if ech := obj.ECHeader(); ech != nil {
		attributes = ech.ParentAttributes()
	}
	for _, attr := range attributes {
		if attr.Key() == objectV2.SysAttributeExpEpoch {
			expEpoch, err := strconv.ParseUint(attr.Value(), 10, 64)
			return expEpoch, err == nil
		}
	}
	return 0, false
}

type bucketContainer interface {
	Bucket([]byte) *bbolt.Bucket
	CreateBucket([]byte) (*bbolt.Bucket, error)
	CreateBucketIfNotExists([]byte) (*bbolt.Bucket, error)
}

func createBucketLikelyExists[T bucketContainer](tx T, name []byte) (*bbolt.Bucket, error) {
	if bkt := tx.Bucket(name); bkt != nil {
		return bkt, nil
	}
	return tx.CreateBucket(name)
}

func updateUniqueIndexItem(tx *bbolt.Tx, item namedBucketItem, update func(oldData, newData []byte) ([]byte, error)) error {
	bkt, err := createBucketLikelyExists(tx, item.name)
	if err != nil {
		return fmt.Errorf("can't create index %v: %w", item.name, err)
	}

	data, err := update(bkt.Get(item.key), item.val)
	if err != nil {
		return err
	}
	return bkt.Put(item.key, data)
}

func putUniqueIndexItem(tx *bbolt.Tx, item namedBucketItem) error {
	return updateUniqueIndexItem(tx, item, func(_, val []byte) ([]byte, error) { return val, nil })
}

func putFKBTIndexItem(tx *bbolt.Tx, item namedBucketItem) error {
	bkt, err := createBucketLikelyExists(tx, item.name)
	if err != nil {
		return fmt.Errorf("can't create index %v: %w", item.name, err)
	}

	fkbtRoot, err := createBucketLikelyExists(bkt, item.key)
	if err != nil {
		return fmt.Errorf("can't create fake bucket tree index %v: %w", item.key, err)
	}

	return fkbtRoot.Put(item.val, zeroValue)
}

func putListIndexItem(tx *bbolt.Tx, item namedBucketItem) error {
	bkt, err := createBucketLikelyExists(tx, item.name)
	if err != nil {
		return fmt.Errorf("can't create index %v: %w", item.name, err)
	}

	lst, err := decodeList(bkt.Get(item.key))
	if err != nil {
		return fmt.Errorf("can't decode leaf list %v: %w", item.key, err)
	}

	lst = append(lst, item.val)

	encodedLst, err := encodeList(lst)
	if err != nil {
		return fmt.Errorf("can't encode leaf list %v: %w", item.key, err)
	}

	return bkt.Put(item.key, encodedLst)
}

// encodeList decodes list of bytes into a single blog for list bucket indexes.
func encodeList(lst [][]byte) ([]byte, error) {
	w := io.NewBufBinWriter()
	w.WriteVarUint(uint64(len(lst)))
	for i := range lst {
		w.WriteVarBytes(lst[i])
	}
	if w.Err != nil {
		return nil, w.Err
	}
	return w.Bytes(), nil
}

// decodeList decodes blob into the list of bytes from list bucket index.
func decodeList(data []byte) (lst [][]byte, err error) {
	if len(data) == 0 {
		return nil, nil
	}

	var offset uint64
	size, n, err := getVarUint(data)
	if err != nil {
		return nil, err
	}

	offset += uint64(n)
	lst = make([][]byte, size, size+1)
	for i := range lst {
		sz, n, err := getVarUint(data[offset:])
		if err != nil {
			return nil, err
		}
		offset += uint64(n)

		next := offset + sz
		if uint64(len(data)) < next {
			return nil, gio.ErrUnexpectedEOF
		}
		lst[i] = data[offset:next]
		offset = next
	}
	return lst, nil
}

func getVarUint(data []byte) (uint64, int, error) {
	if len(data) == 0 {
		return 0, 0, gio.ErrUnexpectedEOF
	}

	switch b := data[0]; b {
	case 0xfd:
		if len(data) < 3 {
			return 0, 1, gio.ErrUnexpectedEOF
		}
		return uint64(binary.LittleEndian.Uint16(data[1:])), 3, nil
	case 0xfe:
		if len(data) < 5 {
			return 0, 1, gio.ErrUnexpectedEOF
		}
		return uint64(binary.LittleEndian.Uint32(data[1:])), 5, nil
	case 0xff:
		if len(data) < 9 {
			return 0, 1, gio.ErrUnexpectedEOF
		}
		return binary.LittleEndian.Uint64(data[1:]), 9, nil
	default:
		return uint64(b), 1, nil
	}
}

// setStorageID for existing objects if they were moved from one
// storage location to another.
func setStorageID(tx *bbolt.Tx, addr oid.Address, id []byte, override bool) error {
	key := make([]byte, bucketKeySize)
	bkt, err := createBucketLikelyExists(tx, smallBucketName(addr.Container(), key))
	if err != nil {
		return err
	}
	key = objectKey(addr.Object(), key)
	if override || bkt.Get(key) == nil {
		return bkt.Put(key, id)
	}
	return nil
}

// updateSpliInfo for existing objects if storage filled with extra information
// about last object in split hierarchy or linking object.
func updateSplitInfo(tx *bbolt.Tx, addr oid.Address, from *objectSDK.SplitInfo) error {
	objKey := objectKey(addr.Object(), make([]byte, bucketKeySize))
	return updateSplitInfoIndex(tx, objKey, addr.Container(), make([]byte, bucketKeySize), from)
}

// splitInfoFromObject returns split info based on last or linkin object.
// Otherwise returns nil, nil.
func splitInfoFromObject(obj *objectSDK.Object) (*objectSDK.SplitInfo, error) {
	if obj.Parent() == nil {
		return nil, nil
	}

	info := objectSDK.NewSplitInfo()
	info.SetSplitID(obj.SplitID())

	switch {
	case isLinkObject(obj):
		id, ok := obj.ID()
		if !ok {
			return nil, errors.New("missing object ID")
		}

		info.SetLink(id)
	case isLastObject(obj):
		id, ok := obj.ID()
		if !ok {
			return nil, errors.New("missing object ID")
		}

		info.SetLastPart(id)
	default:
		return nil, ErrIncorrectRootObject // should never happen
	}

	return info, nil
}

// isLinkObject returns true if object contains parent header and list
// of children.
func isLinkObject(obj *objectSDK.Object) bool {
	return len(obj.Children()) > 0 && obj.Parent() != nil
}

// isLastObject returns true if object contains only parent header without list
// of children.
func isLastObject(obj *objectSDK.Object) bool {
	return len(obj.Children()) == 0 && obj.Parent() != nil
}