WIP: Improve debug logging for APE check failures #1655

Draft
aarifullin wants to merge 6 commits from aarifullin/frostfs-node:feat/improve_ape_log into master
10 changed files with 244 additions and 43 deletions

View file

@ -56,7 +56,7 @@ func initContainerService(_ context.Context, c *cfg) {
)
service := containerService.NewSignService(
&c.key.PrivateKey,
containerService.NewAPEServer(defaultChainRouter, cnrRdr,
containerService.NewAPEServer(c.log, defaultChainRouter, cnrRdr,
newCachedIRFetcher(createInnerRingFetcher(c)), c.netMapSource, c.shared.frostfsidClient,
containerService.NewSplitterService(
c.cfgContainer.containerBatchSize, c.respSvc,

View file

@ -442,6 +442,7 @@ func createACLServiceV2(c *cfg, apeSvc *objectAPE.Service, irFetcher *cachedIRFe
func createAPEService(c *cfg, splitSvc *objectService.TransportSplitter) *objectAPE.Service {
return objectAPE.NewService(
c.log,
objectAPE.NewChecker(
c.cfgObject.cfgAccessPolicyEngine.accessPolicyEngine.LocalStorage(),
c.cfgObject.cfgAccessPolicyEngine.accessPolicyEngine.MorphRuleChainStorage(),

View file

@ -513,4 +513,5 @@ const (
FailedToParseIncomingIOTag = "failed to parse incoming IO tag"
NotSupportedIncomingIOTagReplacedWithClient = "incoming IO tag is not supported, replaced with `client`"
FailedToGetNetmapToAdjustIOTag = "failed to get netmap to adjust IO tag, replaced with `client`"
APECheckDeniedRequest = "ape check denied request"
)

View file

@ -32,6 +32,10 @@ func (r Request) Resource() aperesource.Resource {
return r.resource
}
func (r Request) Properties() map[string]string {
return r.properties
}
type Resource struct {
name string
properties map[string]string
@ -53,3 +57,7 @@ func (r Resource) Name() string {
func (r Resource) Property(key string) string {
return r.properties[key]
}
func (r Resource) Properties() map[string]string {
return r.properties
}

View file

@ -104,7 +104,7 @@ func (c *checkerCoreImpl) CheckAPE(ctx context.Context, prm CheckPrm) error {
if found && status == apechain.Allow {
return nil
}
return newChainRouterError(prm.Request.Operation(), status)
return NewChainRouterError(rt, prm.Request, status)
}
// isValidBearer checks whether bearer token was correctly signed by authorized

View file

@ -3,31 +3,138 @@ package ape
import (
"fmt"
aperequest "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/ape/request"
apechain "git.frostfs.info/TrueCloudLab/policy-engine/pkg/chain"
policyengine "git.frostfs.info/TrueCloudLab/policy-engine/pkg/engine"
nativeschema "git.frostfs.info/TrueCloudLab/policy-engine/schema/native"
"go.uber.org/zap/zapcore"
)
// ChainRouterError is returned when chain router validation prevents
// the APE request from being processed (no rule found, access denied, etc.).
type ChainRouterError struct {
operation string
status apechain.Status
target policyengine.RequestTarget
request aperequest.Request
status apechain.Status
}
func (e *ChainRouterError) Error() string {
return fmt.Sprintf("access to operation %s is denied by access policy engine: %s", e.Operation(), e.Status())
return fmt.Sprintf("access to operation %s is denied by access policy engine: %s", e.request.Operation(), e.status)
}
func (e *ChainRouterError) Operation() string {
return e.operation
}
func (e *ChainRouterError) Status() apechain.Status {
return e.status
}
func newChainRouterError(operation string, status apechain.Status) *ChainRouterError {
func NewChainRouterError(target policyengine.RequestTarget, request aperequest.Request, status apechain.Status) *ChainRouterError {
return &ChainRouterError{
operation: operation,
status: status,
target: target,
request: request,
status: status,
}
}
func (e *ChainRouterError) MarshalLogObject(enc zapcore.ObjectEncoder) error {
enc.AddString("status", e.status.String())
if err := enc.AddObject("request", zapcore.ObjectMarshalerFunc(e.marshalRequest)); err != nil {
return err
}
return nil
}
func (e *ChainRouterError) marshalTarget(enc zapcore.ObjectEncoder) error {
target := e.target
hasNamespace := target.Namespace != nil
hasContainer := target.Container != nil
hasUser := target.User != nil
hasGroups := len(target.Groups) > 0
if !hasNamespace && !hasContainer && !hasUser && !hasGroups {
enc.AddString("type", "empty")
return nil
}
if hasNamespace {
enc.AddString("namespace", target.Namespace.Name)
}
if hasContainer {
enc.AddString("container", target.Container.Name)
}
if hasUser {
enc.AddString("user", target.User.Name)
}
if hasGroups {
if err := enc.AddArray("groups", zapcore.ArrayMarshalerFunc(
func(arrayEnc zapcore.ArrayEncoder) error {
for i := range target.Groups {
arrayEnc.AppendString(target.Groups[i].Name)
}
return nil
})); err != nil {
return err
}
}
return nil
}
func (e *ChainRouterError) marshalRequest(enc zapcore.ObjectEncoder) error {
enc.AddString("operation", e.request.Operation())
if err := enc.AddObject("target", zapcore.ObjectMarshalerFunc(e.marshalTarget)); err != nil {
return err
}
fyrchik marked this conversation as resolved Outdated

enc.AddObject("target", zapcore.ObjectMarshalerFunc(e.marshalTarget)) should work too and takes less space.

`enc.AddObject("target", zapcore.ObjectMarshalerFunc(e.marshalTarget))` should work too and takes less space.

Also, in some other places this could be simplified too.

Also, in some other places this could be simplified too.

fixed

fixed
if err := enc.AddObject("properties", zapcore.ObjectMarshalerFunc(
func(innerEnc zapcore.ObjectEncoder) error {
marshalProperties(innerEnc, e.request.Properties())
return nil
})); err != nil {
return err
}
if err := enc.AddObject("resource", zapcore.ObjectMarshalerFunc(e.marshalResource)); err != nil {
return err
}
return nil
}
func (e *ChainRouterError) marshalResource(enc zapcore.ObjectEncoder) error {
resource, ok := e.request.Resource().(aperequest.Resource)
if !ok {
return nil
}
enc.AddString("name", resource.Name())
if err := enc.AddObject("properties", zapcore.ObjectMarshalerFunc(
func(innerEnc zapcore.ObjectEncoder) error {
marshalProperties(innerEnc, resource.Properties())
return nil
})); err != nil {
return err
}
return nil
}
func marshalProperties(enc zapcore.ObjectEncoder, props map[string]string) {
// For some properties, we can display the value in a shorter format.
const shortFormatLength = 15
for key, value := range props {
switch key {
case nativeschema.PropertyKeyObjectPayloadHash,
nativeschema.PropertyKeyObjectHomomorphicHash,
nativeschema.PropertyKeyActorPublicKey:
if len(value) >= shortFormatLength {
value = value[:shortFormatLength] + "..."
}
default:
}
enc.AddString(key, value)
}
}

View file

@ -12,10 +12,13 @@ import (
"net"
"strings"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
aperequest "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/ape/request"
containercore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/container"
frostfsidcore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/frostfsid"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/netmap"
apecommon "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/services/common/ape"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/api/container"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/api/refs"
@ -31,6 +34,7 @@ import (
commonschema "git.frostfs.info/TrueCloudLab/policy-engine/schema/common"
nativeschema "git.frostfs.info/TrueCloudLab/policy-engine/schema/native"
"github.com/nspcc-dev/neo-go/pkg/crypto/keys"
"go.uber.org/zap"
"google.golang.org/grpc/peer"
)
@ -57,6 +61,7 @@ type containers interface {
}
type apeChecker struct {
logger *logger.Logger
router policyengine.ChainRouter
reader containers
ir ir
@ -67,8 +72,9 @@ type apeChecker struct {
next Server
}
func NewAPEServer(router policyengine.ChainRouter, reader containers, ir ir, nm netmap.Source, frostFSIDClient frostfsidcore.SubjectProvider, srv Server) Server {
func NewAPEServer(logger *logger.Logger, router policyengine.ChainRouter, reader containers, ir ir, nm netmap.Source, frostFSIDClient frostfsidcore.SubjectProvider, srv Server) Server {
return &apeChecker{
logger: logger,
router: router,
reader: reader,
ir: ir,
@ -172,7 +178,10 @@ func (ac *apeChecker) List(ctx context.Context, req *container.ListRequest) (*co
return ac.next.List(ctx, req)
}
return nil, apeErr(nativeschema.MethodListContainers, s)
chRouterErr := apecommon.NewChainRouterError(rt, request, s)
ac.logger.Debug(ctx, logs.APECheckDeniedRequest, zap.Object("details", chRouterErr))
return nil, apeErr(chRouterErr)
}
func (ac *apeChecker) ListStream(req *container.ListStreamRequest, stream ListStream) error {
@ -245,7 +254,10 @@ func (ac *apeChecker) ListStream(req *container.ListStreamRequest, stream ListSt
return ac.next.ListStream(req, stream)
}
return apeErr(nativeschema.MethodListContainers, s)
chRouterErr := apecommon.NewChainRouterError(rt, request, s)
ac.logger.Debug(ctx, logs.APECheckDeniedRequest, zap.Object("details", chRouterErr))
return apeErr(chRouterErr)
}
func (ac *apeChecker) Put(ctx context.Context, req *container.PutRequest) (*container.PutResponse, error) {
@ -318,7 +330,10 @@ func (ac *apeChecker) Put(ctx context.Context, req *container.PutRequest) (*cont
return ac.next.Put(ctx, req)
}
return nil, apeErr(nativeschema.MethodPutContainer, s)
chRouterErr := apecommon.NewChainRouterError(rt, request, s)
ac.logger.Debug(ctx, logs.APECheckDeniedRequest, zap.Object("details", chRouterErr))
return nil, apeErr(chRouterErr)
}
func (ac *apeChecker) getRoleWithoutContainerID(ctx context.Context, oID *refs.OwnerID, mh *session.RequestMetaHeader, vh *session.RequestVerificationHeader) (string, *keys.PublicKey, error) {
@ -400,8 +415,9 @@ func (ac *apeChecker) validateContainerBoundedOperation(ctx context.Context, con
reqProps,
)
rt := policyengine.NewRequestTargetExtended(namespace, id.EncodeToString(), fmt.Sprintf("%s:%s", namespace, pk.Address()), groups)
s, found, err := ac.router.IsAllowed(apechain.Ingress,
policyengine.NewRequestTargetExtended(namespace, id.EncodeToString(), fmt.Sprintf("%s:%s", namespace, pk.Address()), groups),
rt,
request)
if err != nil {
return err
@ -411,12 +427,15 @@ func (ac *apeChecker) validateContainerBoundedOperation(ctx context.Context, con
return nil
}
return apeErr(op, s)
chRouterErr := apecommon.NewChainRouterError(rt, request, s)
ac.logger.Debug(ctx, logs.APECheckDeniedRequest, zap.Object("details", chRouterErr))
return apeErr(chRouterErr)
}
func apeErr(operation string, status apechain.Status) error {
func apeErr(err error) error {
errAccessDenied := &apistatus.ObjectAccessDenied{}
errAccessDenied.WriteReason(fmt.Sprintf("access to container operation %s is denied by access policy engine: %s", operation, status.String()))
errAccessDenied.WriteReason(err.Error())
return errAccessDenied
}

View file

@ -12,6 +12,7 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-contract/frostfsid/client"
containercore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/container"
frostfsidcore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/frostfsid"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/api/container"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/api/refs"
session "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/api/session"
@ -32,6 +33,7 @@ import (
"github.com/nspcc-dev/neo-go/pkg/crypto/keys"
"github.com/nspcc-dev/neo-go/pkg/util"
"github.com/stretchr/testify/require"
"go.uber.org/zap"
"google.golang.org/grpc/peer"
)
@ -85,7 +87,7 @@ func testAllowThenDenyGetContainerRuleDefined(t *testing.T) {
frostfsIDSubjectReader := &frostfsidStub{
subjects: map[util.Uint160]*client.Subject{},
}
apeSrv := NewAPEServer(router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
apeSrv := NewAPEServer(logger.NewLoggerWrapper(zap.NewNop()), router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
contID := cidtest.ID()
testContainer := containertest.Container()
@ -184,7 +186,7 @@ func TestAllowByGroupIDs(t *testing.T) {
},
},
}
apeSrv := NewAPEServer(router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
apeSrv := NewAPEServer(logger.NewLoggerWrapper(zap.NewNop()), router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
contID := cidtest.ID()
testContainer := containertest.Container()
@ -257,7 +259,7 @@ func testDenyGetContainerNoRuleFound(t *testing.T) {
frostfsIDSubjectReader := &frostfsidStub{
subjects: map[util.Uint160]*client.Subject{},
}
apeSrv := NewAPEServer(router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
apeSrv := NewAPEServer(logger.NewLoggerWrapper(zap.NewNop()), router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
contID := cidtest.ID()
testContainer := containertest.Container()
@ -307,7 +309,7 @@ func testDenyGetContainerForOthers(t *testing.T) {
frostfsIDSubjectReader := &frostfsidStub{
subjects: map[util.Uint160]*client.Subject{},
}
apeSrv := NewAPEServer(router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
apeSrv := NewAPEServer(logger.NewLoggerWrapper(zap.NewNop()), router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
contID := cidtest.ID()
testContainer := containertest.Container()
@ -407,7 +409,7 @@ func testDenyGetContainerByUserClaimTag(t *testing.T) {
},
}
apeSrv := NewAPEServer(router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
apeSrv := NewAPEServer(logger.NewLoggerWrapper(zap.NewNop()), router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
contID := cidtest.ID()
testContainer := containertest.Container()
@ -505,7 +507,7 @@ func testDenyGetContainerByIP(t *testing.T) {
},
}
apeSrv := NewAPEServer(router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
apeSrv := NewAPEServer(logger.NewLoggerWrapper(zap.NewNop()), router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
contID := cidtest.ID()
testContainer := containertest.Container()
@ -604,7 +606,7 @@ func testDenyGetContainerByGroupID(t *testing.T) {
},
}
apeSrv := NewAPEServer(router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
apeSrv := NewAPEServer(logger.NewLoggerWrapper(zap.NewNop()), router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
contID := cidtest.ID()
testContainer := containertest.Container()
@ -684,7 +686,7 @@ func testDenyPutContainerForOthersSessionToken(t *testing.T) {
ownerAddr: {},
},
}
apeSrv := NewAPEServer(router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
apeSrv := NewAPEServer(logger.NewLoggerWrapper(zap.NewNop()), router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
nm.currentEpoch = 100
nm.netmaps = map[uint64]*netmap.NetMap{}
@ -797,7 +799,7 @@ func testDenyPutContainerReadNamespaceFromFrostfsID(t *testing.T) {
},
},
}
apeSrv := NewAPEServer(router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
apeSrv := NewAPEServer(logger.NewLoggerWrapper(zap.NewNop()), router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
resp, err := apeSrv.Put(context.Background(), req)
require.Nil(t, resp)
var errAccessDenied *apistatus.ObjectAccessDenied
@ -881,7 +883,7 @@ func testDenyPutContainerInvalidNamespace(t *testing.T) {
},
},
}
apeSrv := NewAPEServer(router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
apeSrv := NewAPEServer(logger.NewLoggerWrapper(zap.NewNop()), router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
resp, err := apeSrv.Put(context.Background(), req)
require.Nil(t, resp)
require.ErrorContains(t, err, "invalid domain zone")
@ -903,7 +905,7 @@ func testDenyListContainersForPK(t *testing.T) {
frostfsIDSubjectReader := &frostfsidStub{
subjects: map[util.Uint160]*client.Subject{},
}
apeSrv := NewAPEServer(router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
apeSrv := NewAPEServer(logger.NewLoggerWrapper(zap.NewNop()), router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
nm.currentEpoch = 100
nm.netmaps = map[uint64]*netmap.NetMap{}
@ -1020,7 +1022,7 @@ func testDenyListContainersValidationNamespaceError(t *testing.T) {
},
}
apeSrv := NewAPEServer(router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
apeSrv := NewAPEServer(logger.NewLoggerWrapper(zap.NewNop()), router, contRdr, ir, nm, frostfsIDSubjectReader, srv)
nm.currentEpoch = 100
nm.netmaps = map[uint64]*netmap.NetMap{}
@ -1188,6 +1190,7 @@ func newTestAPEServer() testAPEServer {
}
apeChecker := &apeChecker{
logger: logger.NewLoggerWrapper(zap.NewNop()),
router: engine,
reader: containerReader,
ir: ir,
@ -1247,8 +1250,8 @@ func TestValidateContainerBoundedOperation(t *testing.T) {
req := initTestGetContainerRequest(t, contID)
err = components.apeChecker.validateContainerBoundedOperation(ctxWithPeerInfo(), req.GetBody().GetContainerID(), req.GetMetaHeader(), req.GetVerificationHeader(), nativeschema.MethodGetContainer)
aErr := apeErr(nativeschema.MethodGetContainer, chain.AccessDenied)
require.ErrorContains(t, err, aErr.Error())
var accessDeniedErr *apistatus.ObjectAccessDenied
require.ErrorAs(t, err, &accessDeniedErr)
})
t.Run("check root-defined container in testdomain-defined container target rule", func(t *testing.T) {
@ -1420,8 +1423,8 @@ func TestValidateContainerBoundedOperation(t *testing.T) {
req := initTestGetContainerRequest(t, contID)
err = components.apeChecker.validateContainerBoundedOperation(ctxWithPeerInfo(), req.GetBody().GetContainerID(), req.GetMetaHeader(), req.GetVerificationHeader(), nativeschema.MethodGetContainer)
aErr := apeErr(nativeschema.MethodGetContainer, chain.AccessDenied)
require.ErrorContains(t, err, aErr.Error())
var accessDeniedErr *apistatus.ObjectAccessDenied
require.ErrorAs(t, err, &accessDeniedErr)
})
t.Run("check testdomain-defined container in testdomain namespace target rule", func(t *testing.T) {
@ -1462,8 +1465,8 @@ func TestValidateContainerBoundedOperation(t *testing.T) {
req := initTestGetContainerRequest(t, contID)
err = components.apeChecker.validateContainerBoundedOperation(ctxWithPeerInfo(), req.GetBody().GetContainerID(), req.GetMetaHeader(), req.GetVerificationHeader(), nativeschema.MethodGetContainer)
aErr := apeErr(nativeschema.MethodGetContainer, chain.AccessDenied)
require.ErrorContains(t, err, aErr.Error())
var accessDeniedErr *apistatus.ObjectAccessDenied
require.ErrorAs(t, err, &accessDeniedErr)
})
}

View file

@ -6,10 +6,13 @@ import (
"errors"
"fmt"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/engine"
apecommon "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/services/common/ape"
objectSvc "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/services/object"
getsvc "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/services/object/get"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/services/object/util"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger"
objectV2 "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/api/object"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/api/refs"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/bearer"
@ -18,16 +21,23 @@ import (
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/user"
nativeschema "git.frostfs.info/TrueCloudLab/policy-engine/schema/native"
"go.uber.org/zap"
)
var errFailedToCastToRequestContext = errors.New("failed cast to RequestContext")
type Service struct {
logger *logger.Logger
apeChecker Checker
next objectSvc.ServiceServer
}
func logChainRouterError(ctx context.Context, logger *logger.Logger, chainRouterErr *apecommon.ChainRouterError) {
logger.Debug(ctx, logs.APECheckDeniedRequest, zap.Object("details", chainRouterErr))
}
var _ objectSvc.ServiceServer = (*Service)(nil)
fyrchik marked this conversation as resolved Outdated

It might be easier to implement custom encoder for *apecommon.ChainRouterError type (or another custom type defined in this package).
Sth like "request": {"properties": {...}, "target": ...}, "resource": {"name": ..., "properties": ...}.
Have you considered this? 6d482535bd/zapcore/marshaler.go (L30)

Why?

  1. Because .Any or .Stringer allocates (likely) even when debug is enabled.
  2. Because passing variadic parameter passing may allocate too.
  3. Because .Request().Properties() may allocate too.
It _might_ be easier to implement custom encoder for `*apecommon.ChainRouterError` type (or another custom type defined in this package). Sth like `"request": {"properties": {...}, "target": ...}, "resource": {"name": ..., "properties": ...}`. Have you considered this? https://github.com/uber-go/zap/blob/6d482535bdd97f4d97b2f9573ac308f1cf9b574e/zapcore/marshaler.go#L30 Why? 1. Because `.Any` or `.Stringer` allocates (likely) even when debug is enabled. 2. Because passing variadic parameter passing may allocate too. 3. Because `.Request().Properties()` may allocate too.

I have implemented an encoder for *apecommon.ChainRouterError. Please, check.

I agreed with your suggestion

I have implemented an encoder for `*apecommon.ChainRouterError`. Please, check. I agreed with your suggestion

Looks neat!

Looks neat!
type HeaderProvider interface {
@ -64,8 +74,9 @@ func NewStorageEngineHeaderProvider(e *engine.StorageEngine, s *getsvc.Service)
}
}
func NewService(apeChecker Checker, next objectSvc.ServiceServer) *Service {
func NewService(logger *logger.Logger, apeChecker Checker, next objectSvc.ServiceServer) *Service {
return &Service{
logger: logger,
apeChecker: apeChecker,
next: next,
}
@ -74,6 +85,8 @@ func NewService(apeChecker Checker, next objectSvc.ServiceServer) *Service {
type getStreamBasicChecker struct {
objectSvc.GetObjectStream
fyrchik marked this conversation as resolved Outdated

The default should be zap.NewNop(), because we don't want to log sensitive data somewhere, we want to precisely control all the sinks.

The default should be `zap.NewNop()`, because we don't want to log sensitive data somewhere, we want to precisely control all the sinks.

Or, even better, make this argument explicit and remove variadic opts array.

Or, even better, make this argument explicit and remove variadic `opts` array.

Fixed

Fixed
logger *logger.Logger
apeChecker Checker
namespace string
@ -108,6 +121,10 @@ func (g *getStreamBasicChecker) Send(resp *objectV2.GetResponse) error {
}
if err := g.apeChecker.CheckAPE(g.Context(), prm); err != nil {
var chainRouterErr *apecommon.ChainRouterError
if errors.As(err, &chainRouterErr) {
logChainRouterError(g.Context(), g.logger, chainRouterErr)

Why don't we log error just once in g.apeChecker.CheckAPE

func (c *checkerImpl) CheckAPE(ctx context.Context, prm Prm) error {

?

Why don't we log error just once in `g.apeChecker.CheckAPE` https://git.frostfs.info/aarifullin/frostfs-node/src/commit/ab3922488a671e0e5d342036eff9d73b2ef19e01/pkg/services/object/ape/checker.go#L77 ?
}
return toStatusErr(err)
}
}
@ -134,6 +151,7 @@ func (c *Service) Get(request *objectV2.GetRequest, stream objectSvc.GetObjectSt
return c.next.Get(request, &getStreamBasicChecker{
GetObjectStream: stream,
logger: c.logger,
apeChecker: c.apeChecker,
namespace: reqCtx.Namespace,
senderKey: reqCtx.SenderKey,
@ -144,6 +162,8 @@ func (c *Service) Get(request *objectV2.GetRequest, stream objectSvc.GetObjectSt
}
type putStreamBasicChecker struct {
logger *logger.Logger
apeChecker Checker
next objectSvc.PutObjectStream
@ -180,6 +200,10 @@ func (p *putStreamBasicChecker) Send(ctx context.Context, request *objectV2.PutR
}
if err := p.apeChecker.CheckAPE(ctx, prm); err != nil {
var chainRouterErr *apecommon.ChainRouterError
if errors.As(err, &chainRouterErr) {
logChainRouterError(ctx, p.logger, chainRouterErr)
}
return toStatusErr(err)
}
}
@ -195,12 +219,15 @@ func (c *Service) Put(ctx context.Context) (objectSvc.PutObjectStream, error) {
streamer, err := c.next.Put(ctx)
return &putStreamBasicChecker{
logger: c.logger,
apeChecker: c.apeChecker,

I think this is a bad practice:
Logging an error is one way to handle an error.
At the same time we return this error too, so it may be handled twice.
Do you have any other solutions in mind?

I think this is a bad practice: Logging an error is one way to _handle_ an error. At the same time we _return_ this error too, so it may be handled twice. Do you have any other solutions in mind?

Logging an error is one way to handle an error.

Are you talking about auto-tests? Because I have no idea who else might be processing error from debug logs.

Do you have any other solutions in mind?

A client should receive a status going along with concise reason. We can't make an error message that contains thorough details about failure - it may cause a degradation as the server will send big messages to the client on unsuccessful checks.

The idea "client gets the chain router check error. We go to server logs to find out the true reason" is absolutely OK with me

> Logging an error is one way to handle an error. Are you talking about auto-tests? Because I have no idea who else might be processing error from debug logs. > Do you have any other solutions in mind? A client should receive a status going along with concise reason. We can't make an error message that contains thorough details about failure - it may cause a degradation as the server will send big messages to the client on unsuccessful checks. The idea "client gets the chain router check error. We go to server logs to find out the true reason" is absolutely OK with me

We go to server logs to find out the true reason

Logging an error is perfectly fine.
Logging it twice is not.
In your example it is logged, then returned (on the caller it can also be logged (again), then returned etc.).
The easiest rule of thumb to solve this problem is to handle errors once.
Thus, my comment.

>We go to server logs to find out the true reason Logging an error is perfectly fine. Logging it twice is not. In your example it is logged, then returned (on the caller it can also be logged (again), then returned etc.). The easiest rule of thumb to solve this problem is to handle errors once. Thus, my comment.

As an example of how it might be solved: provide details for all requests and hide it behind logging tag.
This way it is not an error and no garbage is seen.

As an example of how it might be solved: provide details for all requests and hide it behind logging tag. This way it is not an error and no garbage is seen.

In your example it is logged, then returned (on the caller it can also be logged (again), then returned etc.).

I caught your idea for now. Yes, I agreed. This point is absolutely fair

> In your example it is logged, then returned (on the caller it can also be logged (again), then returned etc.). I caught your idea for now. Yes, I agreed. This point is absolutely fair

Let me explain why this is barely possible and justify the current approach

At the same time we return this error too, so it may be handled twice.

  1. ChainRouterError won't be processed from calling side ever. For instance, object service request executor (analyzeStatus) gets ObjectAccessDenied error with a reason. It still gets the same message ("access to operation <method> is denied by access policy engine: <status>"). So,ChainRouterError is kind of an intermediate error and its encoder used to log messages only in APE middleware. Please, check out how ChainRouterError implements Error() and how it goes to apeErr.
  2. No way to carry the error as logging tag. We should keep some information within the context, but this won't work for methods like Put and Patch (grpc-streaming and all)
  3. Operations which are logging "operation finished with error" are Get, Delete and Search.
    Put and Patch are logged with audit middleware on Info level
  4. I suggest to log message only when error occurs. Logging every operation will blow journalctl
  5. So, if take in account 1), 2), 3), 4) then I can conclude that the current logging is very helpful and fine

I'd like to leave this as it is and we'll be fine

Let me explain why this is barely possible and justify the current approach > At the same time we return this error too, so it may be handled twice. 1. `ChainRouterError` won't be processed from calling side ever. For instance, object service request executor (`analyzeStatus`) gets `ObjectAccessDenied` error with a reason. It still gets the same message (`"access to operation <method> is denied by access policy engine: <status>"`). So,`ChainRouterError` is kind of an intermediate error and its encoder used to log messages only in APE middleware. Please, check out how `ChainRouterError` implements [Error()](https://git.frostfs.info/TrueCloudLab/frostfs-node/src/commit/ab3922488a671e0e5d342036eff9d73b2ef19e01/pkg/services/common/ape/error.go#L21) and how it goes to [apeErr](https://git.frostfs.info/TrueCloudLab/frostfs-node/src/commit/ab3922488a671e0e5d342036eff9d73b2ef19e01/pkg/services/common/ape/error.go#L21). 2. No way to carry the error as [logging tag](https://git.frostfs.info/TrueCloudLab/frostfs-node/pulls/1655#issuecomment-68578). We should keep some information within the context, but this won't work for methods like `Put` and `Patch` (grpc-streaming and all) 3. Operations which are logging `"operation finished with error"` are `Get`, `Delete` and `Search`. `Put` and `Patch` are logged with `audit` middleware on `Info` level 4. I suggest to log message only when error occurs. Logging every operation will blow `journalctl` 5. So, if take in account 1), 2), 3), 4) then I can conclude that the current logging is very helpful and fine I'd like to leave this as it is and we'll be fine

That we don't see how to do better, doesn't imply the current approach is either helpful or fine.

ChainRouterError won't be processed from calling side ever.

Agree. But should it be an error then? What we log is operation context.

No way to carry the error as logging tag. We should keep some information within the context

We shouldn't, it specifies a place in code and is not context.Context-dependent.
log.WithTag(TagAPEOperationContext).Debug() will do the job.
See #1619.

I suggest to log message only when error occurs.

Why, though? #1574 talks about understanding "what goes wrong". But unexpected ALLOW could also happen, and they also need to be debugged. Having someone access my private container is definitely a bug, even a more dangerous one.

Logging every operation will blow journalctl

As enabling debug level already do. Having a separate tag looks like a perfectly valid usecase for tags to me.
That PR is in review, but if we had a single function that logs verbose info reused between container and object services, that would help tag adoption. And, of course, it depends on the load, consider node living in the old epoch spamming HEAD requests to all nodes in the container.

Currently, container and object services do the same thing here, but in different ways.
The common denominator seems to be router (used directly in container and indirectly via CheckAPE in object).
Maybe have a custom router that logs everything?

That we don't see how to do better, doesn't imply the current approach is either helpful or fine. >ChainRouterError won't be processed from calling side ever. Agree. But should it be an error then? What we log is operation context. >No way to carry the error as logging tag. We should keep some information within the context We shouldn't, it specifies a place in code and is not `context.Context`-dependent. `log.WithTag(TagAPEOperationContext).Debug()` will do the job. See #1619. >I suggest to log message only when error occurs. Why, though? #1574 talks about understanding "what goes wrong". But unexpected ALLOW could also happen, and they also need to be debugged. Having someone access my private container is definitely a bug, even a more dangerous one. >Logging every operation will blow journalctl As enabling debug level already do. Having a separate tag looks like a perfectly valid usecase for tags to me. That PR is in review, but if we had a _single_ function that logs verbose info reused between container and object services, that would help tag adoption. And, of course, it depends on the load, consider node living in the old epoch spamming HEAD requests to all nodes in the container. Currently, container and object services do the same thing here, but in different ways. The common denominator seems to be `router` (used directly in `container` and indirectly via `CheckAPE` in `object`). Maybe have a custom router that logs everything?
next: streamer,
}, err
}
type patchStreamBasicChecker struct {
logger *logger.Logger
apeChecker Checker
next objectSvc.PatchObjectStream
@ -240,6 +267,10 @@ func (p *patchStreamBasicChecker) Send(ctx context.Context, request *objectV2.Pa
}
if err := p.apeChecker.CheckAPE(ctx, prm); err != nil {
var chainRouterErr *apecommon.ChainRouterError
if errors.As(err, &chainRouterErr) {
logChainRouterError(ctx, p.logger, chainRouterErr)
}
return toStatusErr(err)
}
}
@ -255,6 +286,7 @@ func (c *Service) Patch(ctx context.Context) (objectSvc.PatchObjectStream, error
streamer, err := c.next.Patch(ctx)
return &patchStreamBasicChecker{
logger: c.logger,
apeChecker: c.apeChecker,
next: streamer,
}, err
@ -313,6 +345,10 @@ func (c *Service) Head(ctx context.Context, request *objectV2.HeadRequest) (*obj
XHeaders: meta.GetXHeaders(),
})
if err != nil {
var chainRouterErr *apecommon.ChainRouterError
if errors.As(err, &chainRouterErr) {
logChainRouterError(ctx, c.logger, chainRouterErr)
}
return nil, toStatusErr(err)
}
return resp, nil
@ -347,6 +383,10 @@ func (c *Service) Search(request *objectV2.SearchRequest, stream objectSvc.Searc
XHeaders: meta.GetXHeaders(),
})
if err != nil {
var chainRouterErr *apecommon.ChainRouterError
if errors.As(err, &chainRouterErr) {
logChainRouterError(stream.Context(), c.logger, chainRouterErr)
}
return toStatusErr(err)
}
@ -381,6 +421,10 @@ func (c *Service) Delete(ctx context.Context, request *objectV2.DeleteRequest) (
XHeaders: meta.GetXHeaders(),
})
if err != nil {
var chainRouterErr *apecommon.ChainRouterError
if errors.As(err, &chainRouterErr) {
logChainRouterError(ctx, c.logger, chainRouterErr)
}
return nil, toStatusErr(err)
}
@ -420,6 +464,10 @@ func (c *Service) GetRange(request *objectV2.GetRangeRequest, stream objectSvc.G
XHeaders: meta.GetXHeaders(),
})
if err != nil {
var chainRouterErr *apecommon.ChainRouterError
if errors.As(err, &chainRouterErr) {
logChainRouterError(stream.Context(), c.logger, chainRouterErr)
}
return toStatusErr(err)
}
@ -460,6 +508,10 @@ func (c *Service) GetRangeHash(ctx context.Context, request *objectV2.GetRangeHa
}
if err = c.apeChecker.CheckAPE(ctx, prm); err != nil {
var chainRouterErr *apecommon.ChainRouterError
if errors.As(err, &chainRouterErr) {
logChainRouterError(ctx, c.logger, chainRouterErr)
}
return nil, toStatusErr(err)
}
return resp, nil
@ -495,6 +547,10 @@ func (c *Service) PutSingle(ctx context.Context, request *objectV2.PutSingleRequ
}
if err = c.apeChecker.CheckAPE(ctx, prm); err != nil {
var chainRouterErr *apecommon.ChainRouterError
if errors.As(err, &chainRouterErr) {
logChainRouterError(ctx, c.logger, chainRouterErr)
}
return nil, toStatusErr(err)
}

View file

@ -8,6 +8,7 @@ import (
"errors"
"fmt"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
core "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/container"
checkercore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/services/common/ape"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/api/refs"
@ -19,6 +20,7 @@ import (
frostfsecdsa "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/crypto/ecdsa"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/user"
"github.com/nspcc-dev/neo-go/pkg/crypto/keys"
"go.uber.org/zap"
)
type message interface {
@ -65,6 +67,10 @@ func (s *Service) verifyClient(ctx context.Context, req message, cid cidSDK.ID,
}
if err = s.checkAPE(ctx, bt, cnr, cid, op, role, pubKey); err != nil {
var chainRouterErr *checkercore.ChainRouterError
if errors.As(err, &chainRouterErr) {
s.log.Debug(ctx, logs.APECheckDeniedRequest, zap.Object("details", chainRouterErr))
}
return apeErr(err)
}
return nil