2022-02-07 20:27:56 +00:00
|
|
|
package client
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
2022-04-20 16:06:47 +00:00
|
|
|
"errors"
|
2022-02-07 20:27:56 +00:00
|
|
|
"fmt"
|
2022-04-20 16:06:47 +00:00
|
|
|
"io"
|
2022-02-07 20:27:56 +00:00
|
|
|
|
2022-03-24 08:05:41 +00:00
|
|
|
"github.com/nspcc-dev/neofs-api-go/v2/acl"
|
2022-02-07 20:27:56 +00:00
|
|
|
v2object "github.com/nspcc-dev/neofs-api-go/v2/object"
|
|
|
|
rpcapi "github.com/nspcc-dev/neofs-api-go/v2/rpc"
|
|
|
|
"github.com/nspcc-dev/neofs-api-go/v2/rpc/client"
|
|
|
|
v2session "github.com/nspcc-dev/neofs-api-go/v2/session"
|
2022-03-24 08:05:41 +00:00
|
|
|
"github.com/nspcc-dev/neofs-sdk-go/bearer"
|
2022-08-25 12:14:18 +00:00
|
|
|
apistatus "github.com/nspcc-dev/neofs-sdk-go/client/status"
|
2023-03-07 10:16:56 +00:00
|
|
|
cid "github.com/nspcc-dev/neofs-sdk-go/container/id"
|
2023-04-25 08:31:27 +00:00
|
|
|
neofscrypto "github.com/nspcc-dev/neofs-sdk-go/crypto"
|
2022-02-07 20:27:56 +00:00
|
|
|
"github.com/nspcc-dev/neofs-sdk-go/object"
|
|
|
|
oid "github.com/nspcc-dev/neofs-sdk-go/object/id"
|
2023-03-07 10:16:56 +00:00
|
|
|
"github.com/nspcc-dev/neofs-sdk-go/object/slicer"
|
2022-02-07 20:27:56 +00:00
|
|
|
"github.com/nspcc-dev/neofs-sdk-go/session"
|
2023-03-07 10:16:56 +00:00
|
|
|
"github.com/nspcc-dev/neofs-sdk-go/user"
|
2022-02-07 20:27:56 +00:00
|
|
|
)
|
|
|
|
|
|
|
|
// PrmObjectPutInit groups parameters of ObjectPutInit operation.
|
2022-07-29 15:42:24 +00:00
|
|
|
type PrmObjectPutInit struct {
|
|
|
|
copyNum uint32
|
2023-04-25 08:31:27 +00:00
|
|
|
signer neofscrypto.Signer
|
2022-08-25 12:14:18 +00:00
|
|
|
meta v2session.RequestMetaHeader
|
2022-07-29 15:42:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// SetCopiesNumber sets number of object copies that is enough to consider put successful.
|
|
|
|
func (x *PrmObjectPutInit) SetCopiesNumber(copiesNumber uint32) {
|
|
|
|
x.copyNum = copiesNumber
|
|
|
|
}
|
2022-02-07 20:27:56 +00:00
|
|
|
|
|
|
|
// ResObjectPut groups the final result values of ObjectPutInit operation.
|
|
|
|
type ResObjectPut struct {
|
|
|
|
statusRes
|
|
|
|
|
2022-08-05 07:56:49 +00:00
|
|
|
obj oid.ID
|
2022-02-07 20:27:56 +00:00
|
|
|
}
|
|
|
|
|
2022-08-05 07:56:49 +00:00
|
|
|
// StoredObjectID returns identifier of the saved object.
|
|
|
|
func (x ResObjectPut) StoredObjectID() oid.ID {
|
|
|
|
return x.obj
|
2022-02-07 20:27:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// ObjectWriter is designed to write one object to NeoFS system.
|
|
|
|
//
|
|
|
|
// Must be initialized using Client.ObjectPutInit, any other
|
|
|
|
// usage is unsafe.
|
|
|
|
type ObjectWriter struct {
|
|
|
|
cancelCtxStream context.CancelFunc
|
|
|
|
|
2022-08-25 12:14:18 +00:00
|
|
|
client *Client
|
|
|
|
stream interface {
|
|
|
|
Write(*v2object.PutRequest) error
|
|
|
|
Close() error
|
|
|
|
}
|
2022-02-07 20:27:56 +00:00
|
|
|
|
2023-04-25 08:31:27 +00:00
|
|
|
signer neofscrypto.Signer
|
|
|
|
res ResObjectPut
|
|
|
|
err error
|
2022-02-07 20:27:56 +00:00
|
|
|
|
|
|
|
chunkCalled bool
|
|
|
|
|
2022-08-25 12:14:18 +00:00
|
|
|
respV2 v2object.PutResponse
|
|
|
|
req v2object.PutRequest
|
|
|
|
partInit v2object.PutObjectPartInit
|
2022-02-07 20:27:56 +00:00
|
|
|
partChunk v2object.PutObjectPartChunk
|
|
|
|
}
|
|
|
|
|
2023-04-25 08:31:27 +00:00
|
|
|
// UseSigner specifies private signer to sign the requests.
|
|
|
|
// If signer is not provided, then Client default signer is used.
|
|
|
|
func (x *PrmObjectPutInit) UseSigner(signer neofscrypto.Signer) {
|
|
|
|
x.signer = signer
|
2022-02-07 20:27:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// WithBearerToken attaches bearer token to be used for the operation.
|
|
|
|
// Should be called once before any writing steps.
|
2022-08-25 12:14:18 +00:00
|
|
|
func (x *PrmObjectPutInit) WithBearerToken(t bearer.Token) {
|
2022-03-24 08:05:41 +00:00
|
|
|
var v2token acl.BearerToken
|
|
|
|
t.WriteToV2(&v2token)
|
2022-08-25 12:14:18 +00:00
|
|
|
x.meta.SetBearerToken(&v2token)
|
2022-02-07 20:27:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// WithinSession specifies session within which object should be stored.
|
|
|
|
// Should be called once before any writing steps.
|
2022-08-25 12:14:18 +00:00
|
|
|
func (x *PrmObjectPutInit) WithinSession(t session.Object) {
|
2022-04-07 16:09:15 +00:00
|
|
|
var tv2 v2session.Token
|
|
|
|
t.WriteToV2(&tv2)
|
|
|
|
|
2022-08-25 12:14:18 +00:00
|
|
|
x.meta.SetSessionToken(&tv2)
|
2022-02-07 20:27:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// MarkLocal tells the server to execute the operation locally.
|
2022-08-25 12:14:18 +00:00
|
|
|
func (x *PrmObjectPutInit) MarkLocal() {
|
|
|
|
x.meta.SetTTL(1)
|
2022-02-07 20:27:56 +00:00
|
|
|
}
|
|
|
|
|
2022-03-03 11:04:53 +00:00
|
|
|
// WithXHeaders specifies list of extended headers (string key-value pairs)
|
|
|
|
// to be attached to the request. Must have an even length.
|
|
|
|
//
|
|
|
|
// Slice must not be mutated until the operation completes.
|
2022-08-25 12:14:18 +00:00
|
|
|
func (x *PrmObjectPutInit) WithXHeaders(hs ...string) {
|
|
|
|
writeXHeadersToMeta(hs, &x.meta)
|
2022-03-03 11:04:53 +00:00
|
|
|
}
|
|
|
|
|
2022-02-07 20:27:56 +00:00
|
|
|
// WriteHeader writes header of the object. Result means success.
|
|
|
|
// Failure reason can be received via Close.
|
|
|
|
func (x *ObjectWriter) WriteHeader(hdr object.Object) bool {
|
|
|
|
v2Hdr := hdr.ToV2()
|
|
|
|
|
|
|
|
x.partInit.SetObjectID(v2Hdr.GetObjectID())
|
|
|
|
x.partInit.SetHeader(v2Hdr.GetHeader())
|
|
|
|
x.partInit.SetSignature(v2Hdr.GetSignature())
|
|
|
|
|
2022-08-25 12:14:18 +00:00
|
|
|
x.req.GetBody().SetObjectPart(&x.partInit)
|
|
|
|
x.req.SetVerificationHeader(nil)
|
|
|
|
|
2023-04-25 08:31:27 +00:00
|
|
|
x.err = signServiceMessage(x.signer, &x.req)
|
2022-08-25 12:14:18 +00:00
|
|
|
if x.err != nil {
|
|
|
|
x.err = fmt.Errorf("sign message: %w", x.err)
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
|
|
|
x.err = x.stream.Write(&x.req)
|
|
|
|
return x.err == nil
|
2022-02-07 20:27:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// WritePayloadChunk writes chunk of the object payload. Result means success.
|
|
|
|
// Failure reason can be received via Close.
|
|
|
|
func (x *ObjectWriter) WritePayloadChunk(chunk []byte) bool {
|
|
|
|
if !x.chunkCalled {
|
|
|
|
x.chunkCalled = true
|
2022-08-25 12:14:18 +00:00
|
|
|
x.req.GetBody().SetObjectPart(&x.partChunk)
|
2022-02-07 20:27:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
for ln := len(chunk); ln > 0; ln = len(chunk) {
|
2022-02-21 18:36:02 +00:00
|
|
|
// maxChunkLen restricts maximum byte length of the chunk
|
|
|
|
// transmitted in a single stream message. It depends on
|
|
|
|
// server settings and other message fields, but for now
|
|
|
|
// we simply assume that 3MB is large enough to reduce the
|
|
|
|
// number of messages, and not to exceed the limit
|
|
|
|
// (4MB by default for gRPC servers).
|
|
|
|
const maxChunkLen = 3 << 20
|
|
|
|
if ln > maxChunkLen {
|
|
|
|
ln = maxChunkLen
|
2022-02-07 20:27:56 +00:00
|
|
|
}
|
|
|
|
|
2022-02-21 18:36:02 +00:00
|
|
|
// we deal with size limit overflow above, but there is another case:
|
|
|
|
// what if method is called with "small" chunk many times? We write
|
|
|
|
// a message to the stream on each call. Alternatively, we could use buffering.
|
|
|
|
// In most cases, the chunk length does not vary between calls. Given this
|
|
|
|
// assumption, as well as the length of the payload from the header, it is
|
|
|
|
// possible to buffer the data of intermediate chunks, and send a message when
|
|
|
|
// the allocated buffer is filled, or when the last chunk is received.
|
|
|
|
// It is mentally assumed that allocating and filling the buffer is better than
|
|
|
|
// synchronous sending, but this needs to be tested.
|
2022-02-07 20:27:56 +00:00
|
|
|
x.partChunk.SetChunk(chunk[:ln])
|
2022-08-25 12:14:18 +00:00
|
|
|
x.req.SetVerificationHeader(nil)
|
2022-02-07 20:27:56 +00:00
|
|
|
|
2023-04-25 08:31:27 +00:00
|
|
|
x.err = signServiceMessage(x.signer, &x.req)
|
2022-08-25 12:14:18 +00:00
|
|
|
if x.err != nil {
|
|
|
|
x.err = fmt.Errorf("sign message: %w", x.err)
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
|
|
|
x.err = x.stream.Write(&x.req)
|
|
|
|
if x.err != nil {
|
2022-02-07 20:27:56 +00:00
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
|
|
|
chunk = chunk[ln:]
|
|
|
|
}
|
|
|
|
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
|
|
|
|
// Close ends writing the object and returns the result of the operation
|
|
|
|
// along with the final results. Must be called after using the ObjectWriter.
|
|
|
|
//
|
|
|
|
// Exactly one return value is non-nil. By default, server status is returned in res structure.
|
|
|
|
// Any client's internal or transport errors are returned as Go built-in error.
|
|
|
|
// If Client is tuned to resolve NeoFS API statuses, then NeoFS failures
|
|
|
|
// codes are returned as error.
|
|
|
|
//
|
|
|
|
// Return statuses:
|
2022-02-28 10:56:46 +00:00
|
|
|
// - global (see Client docs);
|
|
|
|
// - *apistatus.ContainerNotFound;
|
|
|
|
// - *apistatus.ObjectAccessDenied;
|
|
|
|
// - *apistatus.ObjectLocked;
|
|
|
|
// - *apistatus.LockNonRegularObject;
|
|
|
|
// - *apistatus.SessionTokenNotFound;
|
|
|
|
// - *apistatus.SessionTokenExpired.
|
2022-02-07 20:27:56 +00:00
|
|
|
func (x *ObjectWriter) Close() (*ResObjectPut, error) {
|
|
|
|
defer x.cancelCtxStream()
|
|
|
|
|
2022-04-20 16:06:47 +00:00
|
|
|
// Ignore io.EOF error, because it is expected error for client-side
|
|
|
|
// stream termination by the server. E.g. when stream contains invalid
|
|
|
|
// message. Server returns an error in response message (in status).
|
2022-08-25 12:14:18 +00:00
|
|
|
if x.err != nil && !errors.Is(x.err, io.EOF) {
|
|
|
|
return nil, x.err
|
2022-02-07 20:27:56 +00:00
|
|
|
}
|
|
|
|
|
2022-08-25 12:14:18 +00:00
|
|
|
if x.err = x.stream.Close(); x.err != nil {
|
|
|
|
return nil, x.err
|
2022-02-07 20:27:56 +00:00
|
|
|
}
|
|
|
|
|
2022-08-25 12:14:18 +00:00
|
|
|
x.res.st, x.err = x.client.processResponse(&x.respV2)
|
|
|
|
if x.err != nil {
|
|
|
|
return nil, x.err
|
|
|
|
}
|
2022-06-15 10:44:38 +00:00
|
|
|
|
2022-08-25 12:14:18 +00:00
|
|
|
if !apistatus.IsSuccessful(x.res.st) {
|
|
|
|
return &x.res, nil
|
2022-02-07 20:27:56 +00:00
|
|
|
}
|
|
|
|
|
2022-08-25 12:14:18 +00:00
|
|
|
const fieldID = "ID"
|
|
|
|
|
|
|
|
idV2 := x.respV2.GetBody().GetObjectID()
|
|
|
|
if idV2 == nil {
|
|
|
|
return nil, newErrMissingResponseField(fieldID)
|
|
|
|
}
|
|
|
|
|
|
|
|
x.err = x.res.obj.ReadFromV2(*idV2)
|
|
|
|
if x.err != nil {
|
|
|
|
x.err = newErrInvalidResponseField(fieldID, x.err)
|
2022-08-24 13:25:54 +00:00
|
|
|
}
|
|
|
|
|
2022-08-25 12:14:18 +00:00
|
|
|
return &x.res, nil
|
2022-02-07 20:27:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// ObjectPutInit initiates writing an object through a remote server using NeoFS API protocol.
|
|
|
|
//
|
|
|
|
// The call only opens the transmission channel, explicit recording is done using the ObjectWriter.
|
|
|
|
// Exactly one return value is non-nil. Resulting writer must be finally closed.
|
|
|
|
//
|
|
|
|
// Context is required and must not be nil. It is used for network communication.
|
2022-07-29 15:42:24 +00:00
|
|
|
func (c *Client) ObjectPutInit(ctx context.Context, prm PrmObjectPutInit) (*ObjectWriter, error) {
|
2022-02-07 20:27:56 +00:00
|
|
|
// check parameters
|
|
|
|
if ctx == nil {
|
|
|
|
panic(panicMsgMissingContext)
|
|
|
|
}
|
|
|
|
|
2022-08-25 12:14:18 +00:00
|
|
|
var w ObjectWriter
|
2022-08-05 07:56:49 +00:00
|
|
|
|
2022-08-25 12:14:18 +00:00
|
|
|
ctx, cancel := context.WithCancel(ctx)
|
|
|
|
stream, err := rpcapi.PutObject(&c.c, &w.respV2, client.WithContext(ctx))
|
2022-02-07 20:27:56 +00:00
|
|
|
if err != nil {
|
2022-08-25 12:14:18 +00:00
|
|
|
cancel()
|
2022-02-07 20:27:56 +00:00
|
|
|
return nil, fmt.Errorf("open stream: %w", err)
|
|
|
|
}
|
|
|
|
|
2023-04-25 08:31:27 +00:00
|
|
|
w.signer = prm.signer
|
|
|
|
if w.signer == nil {
|
|
|
|
w.signer = c.prm.signer
|
2022-08-05 07:56:49 +00:00
|
|
|
}
|
2022-08-25 12:14:18 +00:00
|
|
|
w.cancelCtxStream = cancel
|
|
|
|
w.client = c
|
|
|
|
w.stream = stream
|
|
|
|
w.partInit.SetCopiesNumber(prm.copyNum)
|
|
|
|
w.req.SetBody(new(v2object.PutRequestBody))
|
|
|
|
c.prepareRequest(&w.req, &prm.meta)
|
2022-02-07 20:27:56 +00:00
|
|
|
|
|
|
|
return &w, nil
|
|
|
|
}
|
2023-03-07 10:16:56 +00:00
|
|
|
|
|
|
|
type objectWriter struct {
|
|
|
|
context context.Context
|
|
|
|
client *Client
|
|
|
|
}
|
|
|
|
|
|
|
|
func (x *objectWriter) InitDataStream(header object.Object) (io.Writer, error) {
|
|
|
|
var prm PrmObjectPutInit
|
|
|
|
|
|
|
|
stream, err := x.client.ObjectPutInit(x.context, prm)
|
|
|
|
if err != nil {
|
|
|
|
return nil, fmt.Errorf("init object stream: %w", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
if stream.WriteHeader(header) {
|
|
|
|
return &payloadWriter{
|
|
|
|
stream: stream,
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
res, err := stream.Close()
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil, apistatus.ErrFromStatus(res.Status())
|
|
|
|
}
|
|
|
|
|
|
|
|
type payloadWriter struct {
|
|
|
|
stream *ObjectWriter
|
|
|
|
}
|
|
|
|
|
|
|
|
func (x *payloadWriter) Write(p []byte) (int, error) {
|
|
|
|
if !x.stream.WritePayloadChunk(p) {
|
|
|
|
return 0, x.Close()
|
|
|
|
}
|
|
|
|
|
|
|
|
return len(p), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (x *payloadWriter) Close() error {
|
|
|
|
res, err := x.stream.Close()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
return apistatus.ErrFromStatus(res.Status())
|
|
|
|
}
|
|
|
|
|
|
|
|
// CreateObject creates new NeoFS object with given payload data and stores it
|
|
|
|
// in specified container of the NeoFS network using provided Client connection.
|
|
|
|
// The object is created on behalf of provided neofscrypto.Signer, and owned by
|
|
|
|
// the specified user.ID.
|
|
|
|
//
|
|
|
|
// In terms of NeoFS, parameterized neofscrypto.Signer represents object owner,
|
|
|
|
// object signer and request sender. Container SHOULD be public-write or sender
|
|
|
|
// SHOULD have corresponding rights.
|
|
|
|
//
|
|
|
|
// Client connection MUST be opened in advance, see Dial method for details.
|
|
|
|
// Network communication is carried out within a given context, so it MUST NOT
|
|
|
|
// be nil.
|
|
|
|
//
|
|
|
|
// Notice: This API is EXPERIMENTAL and is planned to be replaced/changed in the
|
|
|
|
// future. Be ready to refactor your code regarding imports and call mechanics,
|
|
|
|
// in essence the operation will not change.
|
|
|
|
func CreateObject(ctx context.Context, cli *Client, signer neofscrypto.Signer, cnr cid.ID, owner user.ID, data io.Reader, attributes ...string) (oid.ID, error) {
|
2023-04-26 13:00:00 +00:00
|
|
|
s, err := NewDataSlicer(ctx, cli, signer, cnr, owner)
|
|
|
|
if err != nil {
|
|
|
|
return oid.ID{}, err
|
|
|
|
}
|
|
|
|
|
|
|
|
return s.Slice(data, attributes...)
|
|
|
|
}
|
|
|
|
|
|
|
|
// NewDataSlicer creates slicer.Slicer that saves data in the NeoFS network
|
|
|
|
// through provided Client. The data is packaged into NeoFS objects stored in
|
|
|
|
// the specified container. Provided signer is being used to sign the resulting
|
|
|
|
// objects as a system requirement. Produced objects are owned by the
|
|
|
|
// parameterized NeoFS user.
|
|
|
|
//
|
|
|
|
// Notice: This API is EXPERIMENTAL and is planned to be replaced/changed in the
|
|
|
|
// future. Be ready to refactor your code regarding imports and call mechanics,
|
|
|
|
// in essence the operation will not change.
|
|
|
|
func NewDataSlicer(ctx context.Context, cli *Client, signer neofscrypto.Signer, cnr cid.ID, owner user.ID) (*slicer.Slicer, error) {
|
2023-03-07 10:16:56 +00:00
|
|
|
resNetInfo, err := cli.NetworkInfo(ctx, PrmNetworkInfo{})
|
|
|
|
if err != nil {
|
2023-04-26 13:00:00 +00:00
|
|
|
return nil, fmt.Errorf("read current network info: %w", err)
|
2023-03-07 10:16:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
netInfo := resNetInfo.Info()
|
|
|
|
|
|
|
|
var opts slicer.Options
|
|
|
|
opts.SetObjectPayloadLimit(netInfo.MaxObjectSize())
|
|
|
|
opts.SetCurrentNeoFSEpoch(netInfo.CurrentEpoch())
|
2023-04-26 10:17:06 +00:00
|
|
|
if !netInfo.HomomorphicHashingDisabled() {
|
|
|
|
opts.CalculateHomomorphicChecksum()
|
|
|
|
}
|
2023-03-07 10:16:56 +00:00
|
|
|
|
2023-04-26 13:00:00 +00:00
|
|
|
return slicer.New(signer, cnr, owner, &objectWriter{
|
2023-03-07 10:16:56 +00:00
|
|
|
context: ctx,
|
|
|
|
client: cli,
|
2023-04-26 13:00:00 +00:00
|
|
|
}, opts), nil
|
2023-03-07 10:16:56 +00:00
|
|
|
}
|