2024-08-30 09:09:14 +00:00
|
|
|
package writer
|
2024-03-28 10:46:19 +00:00
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
|
|
|
"crypto/ecdsa"
|
2024-07-08 08:33:40 +00:00
|
|
|
"encoding/hex"
|
2024-03-28 10:46:19 +00:00
|
|
|
"errors"
|
|
|
|
"fmt"
|
2024-07-08 08:33:40 +00:00
|
|
|
"sync/atomic"
|
2024-03-28 10:46:19 +00:00
|
|
|
|
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
|
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/client"
|
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
|
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/policy"
|
|
|
|
svcutil "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/services/object/util"
|
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/services/object_manager/placement"
|
|
|
|
containerSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container"
|
|
|
|
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
|
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/erasurecode"
|
2024-08-01 13:08:50 +00:00
|
|
|
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
|
2024-03-28 10:46:19 +00:00
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/transformer"
|
|
|
|
"go.uber.org/zap"
|
|
|
|
"golang.org/x/sync/errgroup"
|
|
|
|
)
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
var _ transformer.ObjectWriter = (*ECWriter)(nil)
|
2024-03-28 10:46:19 +00:00
|
|
|
|
|
|
|
var errUnsupportedECObject = errors.New("object is not supported for erasure coding")
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
type ECWriter struct {
|
|
|
|
Config *Config
|
|
|
|
PlacementOpts []placement.Option
|
|
|
|
Container containerSDK.Container
|
|
|
|
Key *ecdsa.PrivateKey
|
|
|
|
CommonPrm *svcutil.CommonPrm
|
|
|
|
Relay func(context.Context, client.NodeInfo, client.MultiAddressClient) error
|
2024-03-28 10:46:19 +00:00
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
ObjectMeta object.ContentMeta
|
|
|
|
ObjectMetaValid bool
|
2024-03-28 10:46:19 +00:00
|
|
|
}
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
func (e *ECWriter) WriteObject(ctx context.Context, obj *objectSDK.Object) error {
|
2024-08-01 13:08:50 +00:00
|
|
|
relayed, err := e.relayIfNotContainerNode(ctx, obj)
|
2024-03-28 10:46:19 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if relayed {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if !object.IsECSupported(obj) {
|
|
|
|
// must be resolved by caller
|
|
|
|
return errUnsupportedECObject
|
|
|
|
}
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
if !e.ObjectMetaValid {
|
|
|
|
if e.ObjectMeta, err = e.Config.FormatValidator.ValidateContent(obj); err != nil {
|
2024-03-28 10:46:19 +00:00
|
|
|
return fmt.Errorf("(%T) could not validate payload content: %w", e, err)
|
|
|
|
}
|
2024-08-30 09:09:14 +00:00
|
|
|
e.ObjectMetaValid = true
|
2024-03-28 10:46:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if obj.ECHeader() != nil {
|
|
|
|
return e.writeECPart(ctx, obj)
|
|
|
|
}
|
|
|
|
return e.writeRawObject(ctx, obj)
|
|
|
|
}
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
func (e *ECWriter) relayIfNotContainerNode(ctx context.Context, obj *objectSDK.Object) (bool, error) {
|
|
|
|
if e.Relay == nil {
|
2024-03-28 10:46:19 +00:00
|
|
|
return false, nil
|
|
|
|
}
|
|
|
|
currentNodeIsContainerNode, err := e.currentNodeIsContainerNode()
|
|
|
|
if err != nil {
|
|
|
|
return false, err
|
|
|
|
}
|
|
|
|
if currentNodeIsContainerNode {
|
|
|
|
// object can be splitted or saved local
|
|
|
|
return false, nil
|
|
|
|
}
|
2024-08-01 13:08:50 +00:00
|
|
|
objID := object.AddressOf(obj).Object()
|
|
|
|
var index uint32
|
|
|
|
if obj.ECHeader() != nil {
|
|
|
|
objID = obj.ECHeader().Parent()
|
|
|
|
index = obj.ECHeader().Index()
|
|
|
|
}
|
|
|
|
if err := e.relayToContainerNode(ctx, objID, index); err != nil {
|
2024-03-28 10:46:19 +00:00
|
|
|
return false, err
|
|
|
|
}
|
|
|
|
return true, nil
|
|
|
|
}
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
func (e *ECWriter) currentNodeIsContainerNode() (bool, error) {
|
|
|
|
t, err := placement.NewTraverser(e.PlacementOpts...)
|
2024-03-28 10:46:19 +00:00
|
|
|
if err != nil {
|
|
|
|
return false, err
|
|
|
|
}
|
|
|
|
for {
|
|
|
|
nodes := t.Next()
|
|
|
|
if len(nodes) == 0 {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
for _, node := range nodes {
|
2024-08-30 09:09:14 +00:00
|
|
|
if e.Config.NetmapKeys.IsLocalKey(node.PublicKey()) {
|
2024-03-28 10:46:19 +00:00
|
|
|
return true, nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return false, nil
|
|
|
|
}
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
func (e *ECWriter) relayToContainerNode(ctx context.Context, objID oid.ID, index uint32) error {
|
|
|
|
t, err := placement.NewTraverser(append(e.PlacementOpts, placement.ForObject(objID))...)
|
2024-03-28 10:46:19 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
var lastErr error
|
2024-08-01 13:08:50 +00:00
|
|
|
offset := int(index)
|
2024-03-28 10:46:19 +00:00
|
|
|
for {
|
|
|
|
nodes := t.Next()
|
|
|
|
if len(nodes) == 0 {
|
|
|
|
break
|
|
|
|
}
|
2024-08-01 13:08:50 +00:00
|
|
|
for idx := range nodes {
|
|
|
|
node := nodes[(idx+offset)%len(nodes)]
|
2024-03-28 10:46:19 +00:00
|
|
|
var info client.NodeInfo
|
|
|
|
client.NodeInfoFromNetmapElement(&info, node)
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
c, err := e.Config.ClientConstructor.Get(info)
|
2024-03-28 10:46:19 +00:00
|
|
|
if err != nil {
|
|
|
|
return fmt.Errorf("could not create SDK client %s: %w", info.AddressGroup(), err)
|
|
|
|
}
|
|
|
|
|
|
|
|
completed := make(chan interface{})
|
2024-08-30 09:09:14 +00:00
|
|
|
if poolErr := e.Config.RemotePool.Submit(func() {
|
2024-03-28 10:46:19 +00:00
|
|
|
defer close(completed)
|
2024-08-30 09:09:14 +00:00
|
|
|
err = e.Relay(ctx, info, c)
|
2024-03-28 10:46:19 +00:00
|
|
|
}); poolErr != nil {
|
|
|
|
close(completed)
|
2024-08-30 09:09:14 +00:00
|
|
|
svcutil.LogWorkerPoolError(e.Config.Logger, "PUT", poolErr)
|
2024-03-28 10:46:19 +00:00
|
|
|
return poolErr
|
|
|
|
}
|
|
|
|
<-completed
|
|
|
|
|
|
|
|
if err == nil {
|
|
|
|
return nil
|
|
|
|
}
|
2024-08-30 09:09:14 +00:00
|
|
|
e.Config.Logger.Logger.Warn(logs.ECFailedToSendToContainerNode, zap.Stringers("address_group", info.AddressGroup()))
|
2024-03-28 10:46:19 +00:00
|
|
|
lastErr = err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if lastErr == nil {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
return errIncompletePut{
|
|
|
|
singleErr: lastErr,
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
func (e *ECWriter) writeECPart(ctx context.Context, obj *objectSDK.Object) error {
|
|
|
|
if e.CommonPrm.LocalOnly() {
|
2024-08-01 13:08:50 +00:00
|
|
|
return e.writePartLocal(ctx, obj)
|
|
|
|
}
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
t, err := placement.NewTraverser(append(e.PlacementOpts, placement.ForObject(obj.ECHeader().Parent()))...)
|
2024-03-28 10:46:19 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
eg, egCtx := errgroup.WithContext(ctx)
|
|
|
|
for {
|
|
|
|
nodes := t.Next()
|
|
|
|
if len(nodes) == 0 {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
|
|
|
eg.Go(func() error {
|
2024-07-08 08:33:40 +00:00
|
|
|
return e.writePart(egCtx, obj, int(obj.ECHeader().Index()), nodes, make([]atomic.Bool, len(nodes)))
|
2024-03-28 10:46:19 +00:00
|
|
|
})
|
2024-05-03 14:20:41 +00:00
|
|
|
t.SubmitSuccess()
|
2024-03-28 10:46:19 +00:00
|
|
|
}
|
|
|
|
if err := eg.Wait(); err != nil {
|
|
|
|
return errIncompletePut{
|
|
|
|
singleErr: err,
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
func (e *ECWriter) writeRawObject(ctx context.Context, obj *objectSDK.Object) error {
|
2024-03-28 10:46:19 +00:00
|
|
|
// now only single EC policy is supported
|
2024-08-30 09:09:14 +00:00
|
|
|
c, err := erasurecode.NewConstructor(policy.ECDataCount(e.Container.PlacementPolicy()), policy.ECParityCount(e.Container.PlacementPolicy()))
|
2024-03-28 10:46:19 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2024-08-30 09:09:14 +00:00
|
|
|
parts, err := c.Split(obj, e.Key)
|
2024-03-28 10:46:19 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2024-10-11 12:40:01 +00:00
|
|
|
partsProcessed := make([]atomic.Bool, len(parts))
|
2024-05-03 14:20:41 +00:00
|
|
|
objID, _ := obj.ID()
|
2024-08-30 09:09:14 +00:00
|
|
|
t, err := placement.NewTraverser(append(e.PlacementOpts, placement.ForObject(objID))...)
|
2024-03-28 10:46:19 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
for {
|
2024-10-11 12:40:01 +00:00
|
|
|
eg, egCtx := errgroup.WithContext(ctx)
|
2024-03-28 10:46:19 +00:00
|
|
|
nodes := t.Next()
|
|
|
|
if len(nodes) == 0 {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
2024-07-08 08:33:40 +00:00
|
|
|
visited := make([]atomic.Bool, len(nodes))
|
|
|
|
for idx := range parts {
|
|
|
|
visited[idx%len(nodes)].Store(true)
|
|
|
|
}
|
|
|
|
|
2024-03-28 10:46:19 +00:00
|
|
|
for idx := range parts {
|
2024-10-11 12:40:01 +00:00
|
|
|
if !partsProcessed[idx].Load() {
|
|
|
|
eg.Go(func() error {
|
|
|
|
err := e.writePart(egCtx, parts[idx], idx, nodes, visited)
|
|
|
|
if err == nil {
|
|
|
|
partsProcessed[idx].Store(true)
|
|
|
|
t.SubmitSuccess()
|
|
|
|
}
|
|
|
|
return err
|
|
|
|
})
|
|
|
|
}
|
2024-03-28 10:46:19 +00:00
|
|
|
}
|
2024-10-11 12:40:01 +00:00
|
|
|
err = eg.Wait()
|
2024-03-28 10:46:19 +00:00
|
|
|
}
|
2024-10-11 12:40:01 +00:00
|
|
|
if err != nil {
|
2024-03-28 10:46:19 +00:00
|
|
|
return errIncompletePut{
|
|
|
|
singleErr: err,
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
func (e *ECWriter) writePart(ctx context.Context, obj *objectSDK.Object, partIdx int, nodes []placement.Node, visited []atomic.Bool) error {
|
2024-07-08 08:33:40 +00:00
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return ctx.Err()
|
|
|
|
default:
|
2024-03-28 10:46:19 +00:00
|
|
|
}
|
2024-07-08 08:33:40 +00:00
|
|
|
|
|
|
|
// try to save to node for current part index
|
|
|
|
node := nodes[partIdx%len(nodes)]
|
|
|
|
err := e.putECPartToNode(ctx, obj, node)
|
2024-03-28 10:46:19 +00:00
|
|
|
if err == nil {
|
|
|
|
return nil
|
|
|
|
}
|
2024-08-30 09:09:14 +00:00
|
|
|
e.Config.Logger.Warn(logs.ECFailedToSaveECPart, zap.Stringer("part_address", object.AddressOf(obj)),
|
2024-07-08 08:33:40 +00:00
|
|
|
zap.Stringer("parent_address", obj.ECHeader().Parent()), zap.Int("part_index", partIdx),
|
|
|
|
zap.String("node", hex.EncodeToString(node.PublicKey())), zap.Error(err))
|
|
|
|
|
|
|
|
partVisited := make([]bool, len(nodes))
|
|
|
|
partVisited[partIdx%len(nodes)] = true
|
|
|
|
|
|
|
|
// try to save to any node not visited by any of other parts
|
|
|
|
for i := 1; i < len(nodes); i++ {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return ctx.Err()
|
|
|
|
default:
|
|
|
|
}
|
|
|
|
|
|
|
|
idx := (partIdx + i) % len(nodes)
|
|
|
|
if !visited[idx].CompareAndSwap(false, true) {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
node = nodes[idx]
|
|
|
|
err := e.putECPartToNode(ctx, obj, node)
|
|
|
|
if err == nil {
|
|
|
|
return nil
|
|
|
|
}
|
2024-08-30 09:09:14 +00:00
|
|
|
e.Config.Logger.Warn(logs.ECFailedToSaveECPart, zap.Stringer("part_address", object.AddressOf(obj)),
|
2024-07-08 08:33:40 +00:00
|
|
|
zap.Stringer("parent_address", obj.ECHeader().Parent()), zap.Int("part_index", partIdx),
|
|
|
|
zap.String("node", hex.EncodeToString(node.PublicKey())),
|
|
|
|
zap.Error(err))
|
|
|
|
|
|
|
|
partVisited[idx] = true
|
|
|
|
}
|
|
|
|
|
|
|
|
// try to save to any node not visited by current part
|
2024-08-30 16:20:55 +00:00
|
|
|
for i := range len(nodes) {
|
2024-07-08 08:33:40 +00:00
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return ctx.Err()
|
|
|
|
default:
|
|
|
|
}
|
|
|
|
|
|
|
|
if partVisited[i] {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
node = nodes[i]
|
|
|
|
err := e.putECPartToNode(ctx, obj, node)
|
|
|
|
if err == nil {
|
|
|
|
return nil
|
|
|
|
}
|
2024-08-30 09:09:14 +00:00
|
|
|
e.Config.Logger.Warn(logs.ECFailedToSaveECPart, zap.Stringer("part_address", object.AddressOf(obj)),
|
2024-07-08 08:33:40 +00:00
|
|
|
zap.Stringer("parent_address", obj.ECHeader().Parent()), zap.Int("part_index", partIdx),
|
|
|
|
zap.String("node", hex.EncodeToString(node.PublicKey())),
|
|
|
|
zap.Error(err))
|
|
|
|
}
|
|
|
|
|
|
|
|
return fmt.Errorf("failed to save EC chunk %s to any node", object.AddressOf(obj))
|
|
|
|
}
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
func (e *ECWriter) putECPartToNode(ctx context.Context, obj *objectSDK.Object, node placement.Node) error {
|
|
|
|
if e.Config.NetmapKeys.IsLocalKey(node.PublicKey()) {
|
2024-07-08 08:33:40 +00:00
|
|
|
return e.writePartLocal(ctx, obj)
|
|
|
|
}
|
|
|
|
return e.writePartRemote(ctx, obj, node)
|
2024-03-28 10:46:19 +00:00
|
|
|
}
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
func (e *ECWriter) writePartLocal(ctx context.Context, obj *objectSDK.Object) error {
|
2024-03-28 10:46:19 +00:00
|
|
|
var err error
|
2024-08-30 09:09:14 +00:00
|
|
|
localTarget := LocalTarget{
|
2024-10-01 12:27:06 +00:00
|
|
|
Storage: e.Config.LocalStore,
|
|
|
|
Container: e.Container,
|
2024-03-28 10:46:19 +00:00
|
|
|
}
|
|
|
|
completed := make(chan interface{})
|
2024-08-30 09:09:14 +00:00
|
|
|
if poolErr := e.Config.LocalPool.Submit(func() {
|
2024-03-28 10:46:19 +00:00
|
|
|
defer close(completed)
|
2024-08-30 09:09:14 +00:00
|
|
|
err = localTarget.WriteObject(ctx, obj, e.ObjectMeta)
|
2024-03-28 10:46:19 +00:00
|
|
|
}); poolErr != nil {
|
|
|
|
close(completed)
|
|
|
|
return poolErr
|
|
|
|
}
|
|
|
|
<-completed
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
func (e *ECWriter) writePartRemote(ctx context.Context, obj *objectSDK.Object, node placement.Node) error {
|
2024-03-28 10:46:19 +00:00
|
|
|
var clientNodeInfo client.NodeInfo
|
|
|
|
client.NodeInfoFromNetmapElement(&clientNodeInfo, node)
|
|
|
|
|
2024-08-30 09:09:14 +00:00
|
|
|
remoteTaget := remoteWriter{
|
|
|
|
privateKey: e.Key,
|
|
|
|
clientConstructor: e.Config.ClientConstructor,
|
|
|
|
commonPrm: e.CommonPrm,
|
2024-03-28 10:46:19 +00:00
|
|
|
nodeInfo: clientNodeInfo,
|
|
|
|
}
|
|
|
|
|
|
|
|
var err error
|
|
|
|
completed := make(chan interface{})
|
2024-08-30 09:09:14 +00:00
|
|
|
if poolErr := e.Config.RemotePool.Submit(func() {
|
2024-03-28 10:46:19 +00:00
|
|
|
defer close(completed)
|
2024-08-30 09:09:14 +00:00
|
|
|
err = remoteTaget.WriteObject(ctx, obj, e.ObjectMeta)
|
2024-03-28 10:46:19 +00:00
|
|
|
}); poolErr != nil {
|
|
|
|
close(completed)
|
|
|
|
return poolErr
|
|
|
|
}
|
|
|
|
<-completed
|
|
|
|
return err
|
|
|
|
}
|