WIP: Process EC container for Get/GetRange/Head concurrently #1237

Closed
dstepanov-yadro wants to merge 6 commits from dstepanov-yadro/frostfs-node:fix/ec_get_failover into master
3 changed files with 178 additions and 1 deletions
Showing only changes of commit 4fff95a386 - Show all commits

View file

@ -123,6 +123,7 @@ const (
GetRequestedObjectIsVirtual = "requested object is virtual" GetRequestedObjectIsVirtual = "requested object is virtual"
GetRequestedObjectIsEC = "requested object is erasure-coded" GetRequestedObjectIsEC = "requested object is erasure-coded"
GetRequestedRangeIsOutOfObjectBounds = "requested range is out of object bounds" GetRequestedRangeIsOutOfObjectBounds = "requested range is out of object bounds"
GetCouldNotGetContainer = "could not get container"
PutAdditionalContainerBroadcastFailure = "additional container broadcast failure" PutAdditionalContainerBroadcastFailure = "additional container broadcast failure"
SearchReturnResultDirectly = "return result directly" SearchReturnResultDirectly = "return result directly"
SearchCouldNotConstructRemoteNodeClient = "could not construct remote node client" SearchCouldNotConstructRemoteNodeClient = "could not construct remote node client"

View file

@ -2,10 +2,19 @@ package getsvc
import ( import (
"context" "context"
"encoding/hex"
"errors"
"sync"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs" "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/client"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/policy"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/services/object_manager/placement"
apistatus "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client/status"
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
"go.uber.org/zap" "go.uber.org/zap"
"golang.org/x/sync/errgroup"
) )
func (r *request) executeOnContainer(ctx context.Context) { func (r *request) executeOnContainer(ctx context.Context) {
@ -53,11 +62,28 @@ func (r *request) processCurrentEpoch(ctx context.Context) bool {
return true return true
} }
// In most cases, the value from the cache will be returned.
// Container appears in the cache when traverser is generated.
cnr, err := r.containerSource.Get(r.address().Container())
elebedeva marked this conversation as resolved Outdated

Looks like a grammar mistake: traverser generates --> traverser is generated.

Looks like a grammar mistake: `traverser generates` --> `traverser is generated`.

Fixed

Fixed
if err != nil {

Can we get it from the traverser directly?

Can we get it from the traverser directly?

Container is not traverser part, but container is requered to generate traverser.
Fixed: now traverser generator returns container.

Container is not traverser part, but container is requered to generate traverser. Fixed: now traverser generator returns container.
r.status = statusUndefined
r.err = err
r.log.Debug(logs.GetCouldNotGetContainer, zap.Error(err))
return true
}
ctx, cancel := context.WithCancel(ctx) ctx, cancel := context.WithCancel(ctx)
defer cancel() defer cancel()
r.status = statusUndefined r.status = statusUndefined
if policy.IsECPlacement(cnr.Value.PlacementPolicy()) {
return r.processECNodes(ctx, traverser, policy.ECDataCount(cnr.Value.PlacementPolicy()))
}
return r.processRepNodes(ctx, traverser)
}
func (r *request) processRepNodes(ctx context.Context, traverser *placement.Traverser) bool {
for { for {
addrs := traverser.Next() addrs := traverser.Next()
if len(addrs) == 0 { if len(addrs) == 0 {
@ -91,3 +117,139 @@ func (r *request) processCurrentEpoch(ctx context.Context) bool {
} }
} }
} }
func (r *request) processECNodes(ctx context.Context, traverser *placement.Traverser, dataCount int) bool {
err := r.traverseECNodes(ctx, traverser, dataCount)
var errSplitInfo *objectSDK.SplitInfoError
var errECInfo *objectSDK.ECInfoError
var errRemoved *apistatus.ObjectAlreadyRemoved
var errOutOfRange *apistatus.ObjectOutOfRange
var errSuccess *ecGetSuccessErr
switch {
case err == nil: // nil is returned if all nodes failed or incomplete EC info received
if len(r.infoEC.Chunks) > 0 {

What is the benefit of using a separate error type for successfully performed requests?
Go has multiple value returns can we use them?

What is the benefit of using a separate error type for successfully performed requests? Go has multiple value returns can we use them?

To cancel errgroup's context and do not run other requests: if eg.Go's argument returns error, then errgroup cancels context. Of course it is possible to make the same thing without errgroup, but it will be not so simple.
Also added ctx.Done check.

To cancel `errgroup`'s context and do not run other requests: if `eg.Go`'s argument returns error, then errgroup cancels context. Of course it is possible to make the same thing without errgroup, but it will be not so simple. Also added `ctx.Done` check.

It can be done with a special error, e.g. errStopIteration is what we use somewhere.

It can be done with a special error, e.g. `errStopIteration` is what we use somewhere.

But ecGetSuccessErr is special error used only to stop EC handling and pass result object.

But `ecGetSuccessErr` is special error used only to stop EC handling and pass result object.
r.status = statusEC
r.err = objectSDK.NewECInfoError(r.infoEC)
} else {
fyrchik marked this conversation as resolved Outdated

s/returns/is returned/

`s/returns/is returned/`

done

done
r.status = statusUndefined
r.err = new(apistatus.ObjectNotFound)
}
case errors.As(err, &errRemoved):
r.status = statusINHUMED
r.err = errRemoved
case errors.As(err, &errOutOfRange):
r.status = statusOutOfRange
r.err = errOutOfRange
case errors.As(err, &errSplitInfo):
r.status = statusVIRTUAL
mergeSplitInfo(r.splitInfo(), errSplitInfo.SplitInfo())
r.err = objectSDK.NewSplitInfoError(r.infoSplit)
case errors.As(err, &errECInfo):
r.status = statusEC
r.err = err
case errors.As(err, &errSuccess):
acid-ant marked this conversation as resolved Outdated

You are canceling context here and using it below for r.writeCollectedObject(ctx), looks wrong.

You are canceling context here and using it below for `r.writeCollectedObject(ctx)`, looks wrong.

Right, fixed.

Right, fixed.
r.status = statusOK
r.err = nil
if errSuccess.Object != nil {
r.collectedObject = errSuccess.Object
r.writeCollectedObject(ctx)
}
}
return r.status != statusUndefined
}
func (r *request) traverseECNodes(ctx context.Context, traverser *placement.Traverser, dataCount int) error {
nodes := make(chan placement.Node, dataCount)
ctx, cancel := context.WithCancel(ctx)
defer cancel()
var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
for {
batch := traverser.Next()
aarifullin marked this conversation as resolved Outdated

[Optional]

What do you think: won't this approach cause goroutine spam if too many get-like requests are processed at the same time? Using errgroup would be safe I guess

[Optional] What do you think: won't this approach cause goroutine spam if too many get-like requests are processed at the same time? Using `errgroup` would be safe I guess

I didn't catch the thought.
errgroup creates the same goroutines.

I didn't catch the thought. `errgroup` creates the same goroutines.

errgroup creates the same goroutines.

Yes, it does but with the fixed number of workers.

Nevermind, sorry. I incorrecly read this part:

go func() {
		defer wg.Done()

		for {
			batch := traverser.Next()
			if len(batch) == 0 {
				r.log.Debug(logs.NoMoreNodesAbortPlacementIteration)
				close(nodes)
				return
			}
			for _, node := range batch {
				select {
				case <-ctx.Done():
					return
				case nodes <- node:
				}
			}
		}
	}()

I've mistaken this goroutine generation within for-loop

> errgroup creates the same goroutines. Yes, it does but with the fixed number of workers. Nevermind, sorry. I incorrecly read this part: ```go go func() { defer wg.Done() for { batch := traverser.Next() if len(batch) == 0 { r.log.Debug(logs.NoMoreNodesAbortPlacementIteration) close(nodes) return } for _, node := range batch { select { case <-ctx.Done(): return case nodes <- node: } } } }() ``` I've mistaken this goroutine generation within for-loop
if len(batch) == 0 {
r.log.Debug(logs.NoMoreNodesAbortPlacementIteration)
close(nodes)
return
}
for _, node := range batch {
select {
case <-ctx.Done():
return
case nodes <- node:
}
}
}
}()
err := r.processECNodesRequests(ctx, nodes, dataCount)
cancel()
wg.Wait()
return err
}
func (r *request) processECNodesRequests(ctx context.Context, nodes <-chan placement.Node, dataCount int) error {
var ecInfoGuard sync.Mutex
eg, ctx := errgroup.WithContext(ctx)
eg.SetLimit(dataCount)
for node := range nodes {
var info client.NodeInfo
client.NodeInfoFromNetmapElement(&info, node)
eg.Go(func() error {
select {
case <-ctx.Done():
return ctx.Err()
default:
}
rs, err := r.remoteStorageConstructor.Get(info)
if err != nil {
r.log.Debug(logs.GetCouldNotConstructRemoteNodeClient, zap.String("node_key", hex.EncodeToString(info.PublicKey())))
return err
}

So for each node we call getRemote with the same parameters.
Where is the place in code where we do combine chunks in the final object?

So for each node we call `getRemote` with the same parameters. Where is the place in code where we do combine chunks in the final object?

func (a *assemblerec) retrieveParts(mainCtx context.Context, headOnly bool) []*objectSDK.Object {

https://git.frostfs.info/TrueCloudLab/frostfs-node/src/commit/3bf6e6dde60f15f4f90caf44eda429e8f6269cc1/pkg/services/object/get/assemblerec.go#L122

I have realised where my confusion comes from: we have 2 steps:

  1. Get ECInfo from enough nodes
  2. Pull EC chunks

The "enough nodes" is defined on step 1, even though nodes may go down at step 2. In this case we will fail to fetch object?
The situation will happen during failover tests.

I have realised where my confusion comes from: we have 2 steps: 1. Get ECInfo from enough nodes 2. Pull EC chunks The "enough nodes" is defined on step 1, even though nodes may go down at step 2. In this case we will fail to fetch object? The situation _will_ happen during failover tests.

Right, if node fails between step 1 and step 2, then such request will fail.

Right, if node fails between step 1 and step 2, then such request will fail.
obj, err := r.getRemote(ctx, rs, info)
var errSplitInfo *objectSDK.SplitInfoError
var errECInfo *objectSDK.ECInfoError
var errRemoved *apistatus.ObjectAlreadyRemoved
var errOutOfRange *apistatus.ObjectOutOfRange
switch {
default:
// something failed, continue
r.log.Debug(logs.GetRemoteCallFailed, zap.Error(err))
return nil
case err == nil:
// non EC object found (tombstone, linking, lock), stop
return &ecGetSuccessErr{Object: obj}
case errors.As(err, &errRemoved) || errors.As(err, &errOutOfRange) || errors.As(err, &errSplitInfo):
// non EC error found, stop
return err
case errors.As(err, &errECInfo):
ecInfoGuard.Lock()
defer ecInfoGuard.Unlock()
r.infoEC = util.MergeECInfo(errECInfo.ECInfo(), r.infoEC)

Could you explain this line a bit? Why do we NOT return error if the number of chunks is not equal to the expected?

Could you explain this line a bit? Why do we NOT return error if the number of chunks is not equal to the expected?

In case of HEAD request with raw flag it is expected to get raw EC info error with all chunks.

In case of `HEAD` request with `raw` flag it is expected to get raw EC info error with all chunks.
if r.isRaw() {
if len(r.infoEC.Chunks) == int(r.infoEC.Chunks[0].Total) {
return objectSDK.NewECInfoError(r.infoEC)
}
return nil
}
if len(r.infoEC.Chunks) >= dataCount {
return objectSDK.NewECInfoError(r.infoEC)
}
return nil
}
})
}
return eg.Wait()
}
type ecGetSuccessErr struct {
Object *objectSDK.Object
}
func (s *ecGetSuccessErr) Error() string { return "" }

View file

@ -11,6 +11,7 @@ import (
"testing" "testing"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/client" "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/client"
containerCore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/container"
netmapcore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/netmap" netmapcore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/netmap"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object" "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/network" "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/network"
@ -273,6 +274,16 @@ func (ks *testKeyStorage) GetKey(_ *util.SessionInfo) (*ecdsa.PrivateKey, error)
return &ecdsa.PrivateKey{}, nil return &ecdsa.PrivateKey{}, nil
} }
type testContainerSource struct{}
func (s *testContainerSource) Get(idCnr cid.ID) (*containerCore.Container, error) {
return &containerCore.Container{
Value: container.Container{},
}, nil
}
func (s *testContainerSource) DeletionInfo(cid.ID) (*containerCore.DelInfo, error) { return nil, nil }
func TestGetLocalOnly(t *testing.T) { func TestGetLocalOnly(t *testing.T) {
ctx := context.Background() ctx := context.Background()
@ -551,6 +562,7 @@ func TestGetRemoteSmall(t *testing.T) {
epochSource: testEpochReceiver(curEpoch), epochSource: testEpochReceiver(curEpoch),
remoteStorageConstructor: c, remoteStorageConstructor: c,
keyStore: &testKeyStorage{}, keyStore: &testKeyStorage{},
containerSource: &testContainerSource{},
} }
} }
@ -1722,6 +1734,7 @@ func TestGetRange(t *testing.T) {
epochSource: testEpochReceiver(curEpoch), epochSource: testEpochReceiver(curEpoch),
remoteStorageConstructor: c, remoteStorageConstructor: c,
keyStore: &testKeyStorage{}, keyStore: &testKeyStorage{},
containerSource: &testContainerSource{},
} }
} }
@ -1880,6 +1893,7 @@ func TestGetFromPastEpoch(t *testing.T) {
}, },
}, },
keyStore: &testKeyStorage{}, keyStore: &testKeyStorage{},
containerSource: &testContainerSource{},
} }
w := NewSimpleObjectWriter() w := NewSimpleObjectWriter()