[#446] Add support virtual-hosted-style

Signed-off-by: Roman Loginov <r.loginov@yadro.com>
This commit is contained in:
Roman Loginov 2024-07-31 09:45:46 +03:00 committed by Alexey Vanin
parent 77673797f9
commit 534ae7f0f1
19 changed files with 420 additions and 242 deletions

View file

@ -4,6 +4,9 @@ This document outlines major changes between releases.
## [Unreleased] ## [Unreleased]
### Added
- Add support for virtual hosted style addressing (#446)
## [0.30.0] - Kangshung -2024-07-19 ## [0.30.0] - Kangshung -2024-07-19
### Fixed ### Fixed

View file

@ -41,7 +41,6 @@ type (
RetryMaxAttempts() int RetryMaxAttempts() int
RetryMaxBackoff() time.Duration RetryMaxBackoff() time.Duration
RetryStrategy() RetryStrategy RetryStrategy() RetryStrategy
Domains() []string
} }
FrostFSID interface { FrostFSID interface {

View file

@ -73,7 +73,6 @@ type configMock struct {
defaultCopiesNumbers []uint32 defaultCopiesNumbers []uint32
bypassContentEncodingInChunks bool bypassContentEncodingInChunks bool
md5Enabled bool md5Enabled bool
domains []string
} }
func (c *configMock) DefaultPlacementPolicy(_ string) netmap.PlacementPolicy { func (c *configMock) DefaultPlacementPolicy(_ string) netmap.PlacementPolicy {
@ -137,10 +136,6 @@ func (c *configMock) RetryStrategy() RetryStrategy {
return RetryStrategyConstant return RetryStrategyConstant
} }
func (c *configMock) Domains() []string {
return c.domains
}
func prepareHandlerContext(t *testing.T) *handlerContext { func prepareHandlerContext(t *testing.T) *handlerContext {
log := zaptest.NewLogger(t) log := zaptest.NewLogger(t)
return prepareHandlerContextBase(t, layer.DefaultCachesConfigs(log), log) return prepareHandlerContextBase(t, layer.DefaultCachesConfigs(log), log)

View file

@ -7,7 +7,6 @@ import (
"net/url" "net/url"
"path" "path"
"strconv" "strconv"
"strings"
"time" "time"
"git.frostfs.info/TrueCloudLab/frostfs-s3-gw/api" "git.frostfs.info/TrueCloudLab/frostfs-s3-gw/api"
@ -429,7 +428,7 @@ func (h *handler) CompleteMultipartUploadHandler(w http.ResponseWriter, r *http.
Bucket: objInfo.Bucket, Bucket: objInfo.Bucket,
Key: objInfo.Name, Key: objInfo.Name,
ETag: data.Quote(objInfo.ETag(h.cfg.MD5Enabled())), ETag: data.Quote(objInfo.ETag(h.cfg.MD5Enabled())),
Location: getObjectLocation(r, h.cfg.Domains(), reqInfo.BucketName, reqInfo.ObjectName), Location: getObjectLocation(r, reqInfo.BucketName, reqInfo.ObjectName, reqInfo.RequestVHSEnabled),
} }
if settings.VersioningEnabled() { if settings.VersioningEnabled() {
@ -450,7 +449,7 @@ func getURLScheme(r *http.Request) string {
} }
// getObjectLocation gets the fully qualified URL of an object. // getObjectLocation gets the fully qualified URL of an object.
func getObjectLocation(r *http.Request, domains []string, bucket, object string) string { func getObjectLocation(r *http.Request, bucket, object string, vhsEnabled bool) string {
proto := middleware.GetSourceScheme(r) proto := middleware.GetSourceScheme(r)
if proto == "" { if proto == "" {
proto = getURLScheme(r) proto = getURLScheme(r)
@ -460,13 +459,12 @@ func getObjectLocation(r *http.Request, domains []string, bucket, object string)
Path: path.Join("/", bucket, object), Path: path.Join("/", bucket, object),
Scheme: proto, Scheme: proto,
} }
// If domain is set then we need to use bucket DNS style.
for _, domain := range domains { // If vhs enabled then we need to use bucket DNS style.
if strings.HasPrefix(r.Host, bucket+"."+domain) { if vhsEnabled {
u.Path = path.Join("/", object) u.Path = path.Join("/", object)
break
}
} }
return u.String() return u.String()
} }

View file

@ -446,7 +446,7 @@ func TestMultipartObjectLocation(t *testing.T) {
req *http.Request req *http.Request
bucket string bucket string
object string object string
domains []string vhsEnabled bool
expected string expected string
}{ }{
{ {
@ -492,9 +492,9 @@ func TestMultipartObjectLocation(t *testing.T) {
req: &http.Request{ req: &http.Request{
Host: "mybucket.s3dev.frostfs.devenv", Host: "mybucket.s3dev.frostfs.devenv",
}, },
domains: []string{"s3dev.frostfs.devenv"},
bucket: "mybucket", bucket: "mybucket",
object: "test/1.txt", object: "test/1.txt",
vhsEnabled: true,
expected: "http://mybucket.s3dev.frostfs.devenv/test/1.txt", expected: "http://mybucket.s3dev.frostfs.devenv/test/1.txt",
}, },
{ {
@ -502,14 +502,14 @@ func TestMultipartObjectLocation(t *testing.T) {
Host: "mybucket.s3dev.frostfs.devenv", Host: "mybucket.s3dev.frostfs.devenv",
Header: map[string][]string{"X-Forwarded-Scheme": {"https"}}, Header: map[string][]string{"X-Forwarded-Scheme": {"https"}},
}, },
domains: []string{"s3dev.frostfs.devenv"},
bucket: "mybucket", bucket: "mybucket",
object: "test/1.txt", object: "test/1.txt",
vhsEnabled: true,
expected: "https://mybucket.s3dev.frostfs.devenv/test/1.txt", expected: "https://mybucket.s3dev.frostfs.devenv/test/1.txt",
}, },
} { } {
t.Run("", func(t *testing.T) { t.Run("", func(t *testing.T) {
location := getObjectLocation(tc.req, tc.domains, tc.bucket, tc.object) location := getObjectLocation(tc.req, tc.bucket, tc.object, tc.vhsEnabled)
require.Equal(t, tc.expected, location) require.Equal(t, tc.expected, location)
}) })
} }

View file

@ -0,0 +1,133 @@
package middleware
import (
"net/http"
"net/url"
"strings"
"git.frostfs.info/TrueCloudLab/frostfs-s3-gw/internal/logs"
"go.uber.org/zap"
)
const wildcardPlaceholder = "<wildcard>"
type VHSSettings interface {
Domains() []string
GlobalVHS() bool
VHSNamespacesEnabled() map[string]bool
}
func PrepareAddressStyle(settings VHSSettings, log *zap.Logger) Func {
return func(h http.Handler) http.Handler {
return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
ctx := r.Context()
reqInfo := GetReqInfo(ctx)
reqLogger := reqLogOrDefault(ctx, log)
if isVHSAddress(settings.GlobalVHS(), settings.VHSNamespacesEnabled(), reqInfo.Namespace) {
prepareVHSAddress(reqInfo, r, settings)
} else {
preparePathStyleAddress(reqInfo, r, reqLogger)
}
h.ServeHTTP(w, r)
})
}
}
func isVHSAddress(enabledFlag bool, vhsNamespaces map[string]bool, namespace string) bool {
result := enabledFlag
if v, ok := vhsNamespaces[namespace]; ok {
result = v
}
return result
}
func prepareVHSAddress(reqInfo *ReqInfo, r *http.Request, settings VHSSettings) {
reqInfo.RequestVHSEnabled = true
bktName, match := checkDomain(r.Host, settings.Domains())
if match {
if bktName == "" {
reqInfo.RequestType = noneType
} else {
if objName := strings.TrimPrefix(r.URL.Path, "/"); objName != "" {
reqInfo.RequestType = objectType
reqInfo.ObjectName = objName
reqInfo.BucketName = bktName
} else {
reqInfo.RequestType = bucketType
reqInfo.BucketName = bktName
}
}
} else {
parts := strings.Split(r.Host, ".")
reqInfo.BucketName = parts[0]
if objName := strings.TrimPrefix(r.URL.Path, "/"); objName != "" {
reqInfo.RequestType = objectType
reqInfo.ObjectName = objName
} else {
reqInfo.RequestType = bucketType
}
}
}
func preparePathStyleAddress(reqInfo *ReqInfo, r *http.Request, reqLogger *zap.Logger) {
bktObj := strings.TrimPrefix(r.URL.Path, "/")
if bktObj == "" {
reqInfo.RequestType = noneType
} else if ind := strings.IndexByte(bktObj, '/'); ind != -1 && bktObj[ind+1:] != "" {
reqInfo.RequestType = objectType
reqInfo.BucketName = bktObj[:ind]
reqInfo.ObjectName = bktObj[ind+1:]
if r.URL.RawPath != "" {
// we have to do this because of
// https://github.com/go-chi/chi/issues/641
// https://github.com/go-chi/chi/issues/642
if obj, err := url.PathUnescape(reqInfo.ObjectName); err != nil {
reqLogger.Warn(logs.FailedToUnescapeObjectName, zap.Error(err))
} else {
reqInfo.ObjectName = obj
}
}
} else {
reqInfo.RequestType = bucketType
reqInfo.BucketName = strings.TrimSuffix(bktObj, "/")
}
}
func checkDomain(host string, domains []string) (bktName string, match bool) {
partsHost := strings.Split(host, ".")
for _, pattern := range domains {
partsPattern := strings.Split(pattern, ".")
bktName, match = compareMatch(partsHost, partsPattern)
if match {
break
}
}
return
}
func compareMatch(host, pattern []string) (bktName string, match bool) {
if len(host) < len(pattern) {
return "", false
}
i, j := len(host)-1, len(pattern)-1
for j >= 0 && (pattern[j] == wildcardPlaceholder || host[i] == pattern[j]) {
i--
j--
}
switch {
case i == -1:
return "", true
case i == 0 && (j != 0 || host[i] == pattern[j]):
return host[0], true
default:
return "", false
}
}

View file

@ -73,7 +73,6 @@ type PolicyConfig struct {
Storage engine.ChainRouter Storage engine.ChainRouter
FrostfsID FrostFSIDInformer FrostfsID FrostFSIDInformer
Settings PolicySettings Settings PolicySettings
Domains []string
Log *zap.Logger Log *zap.Logger
BucketResolver BucketResolveFunc BucketResolver BucketResolveFunc
Decoder XMLDecoder Decoder XMLDecoder
@ -99,21 +98,21 @@ func PolicyCheck(cfg PolicyConfig) Func {
} }
func policyCheck(r *http.Request, cfg PolicyConfig) error { func policyCheck(r *http.Request, cfg PolicyConfig) error {
reqType, bktName, objName := getBucketObject(r, cfg.Domains) reqInfo := GetReqInfo(r.Context())
req, userKey, userGroups, err := getPolicyRequest(r, cfg, reqType, bktName, objName)
req, userKey, userGroups, err := getPolicyRequest(r, cfg, reqInfo.RequestType, reqInfo.BucketName, reqInfo.ObjectName)
if err != nil { if err != nil {
return err return err
} }
var bktInfo *data.BucketInfo var bktInfo *data.BucketInfo
if reqType != noneType && !strings.HasSuffix(req.Operation(), CreateBucketOperation) { if reqInfo.RequestType != noneType && !strings.HasSuffix(req.Operation(), CreateBucketOperation) {
bktInfo, err = cfg.BucketResolver(r.Context(), bktName) bktInfo, err = cfg.BucketResolver(r.Context(), reqInfo.BucketName)
if err != nil { if err != nil {
return err return err
} }
} }
reqInfo := GetReqInfo(r.Context())
target := engine.NewRequestTargetWithNamespace(reqInfo.Namespace) target := engine.NewRequestTargetWithNamespace(reqInfo.Namespace)
if bktInfo != nil { if bktInfo != nil {
cnrTarget := engine.ContainerTarget(bktInfo.CID.EncodeToString()) cnrTarget := engine.ContainerTarget(bktInfo.CID.EncodeToString())
@ -208,33 +207,6 @@ const (
objectType objectType
) )
func getBucketObject(r *http.Request, domains []string) (reqType ReqType, bktName string, objName string) {
for _, domain := range domains {
ind := strings.Index(r.Host, "."+domain)
if ind == -1 {
continue
}
bkt := r.Host[:ind]
if obj := strings.TrimPrefix(r.URL.Path, "/"); obj != "" {
return objectType, bkt, obj
}
return bucketType, bkt, ""
}
bktObj := strings.TrimPrefix(r.URL.Path, "/")
if bktObj == "" {
return noneType, "", ""
}
if ind := strings.IndexByte(bktObj, '/'); ind != -1 && bktObj[ind+1:] != "" {
return objectType, bktObj[:ind], bktObj[ind+1:]
}
return bucketType, strings.TrimSuffix(bktObj, "/"), ""
}
func determineOperation(r *http.Request, reqType ReqType) (operation string) { func determineOperation(r *http.Request, reqType ReqType) (operation string) {
switch reqType { switch reqType {
case objectType: case objectType:

View file

@ -8,79 +8,6 @@ import (
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
) )
func TestReqTypeDetermination(t *testing.T) {
bkt, obj, domain := "test-bucket", "test-object", "domain"
for _, tc := range []struct {
name string
target string
host string
domains []string
expectedType ReqType
expectedBktName string
expectedObjName string
}{
{
name: "bucket request, path-style",
target: "/" + bkt,
expectedType: bucketType,
expectedBktName: bkt,
},
{
name: "bucket request with slash, path-style",
target: "/" + bkt + "/",
expectedType: bucketType,
expectedBktName: bkt,
},
{
name: "object request, path-style",
target: "/" + bkt + "/" + obj,
expectedType: objectType,
expectedBktName: bkt,
expectedObjName: obj,
},
{
name: "object request with slash, path-style",
target: "/" + bkt + "/" + obj + "/",
expectedType: objectType,
expectedBktName: bkt,
expectedObjName: obj + "/",
},
{
name: "none type request",
target: "/",
expectedType: noneType,
},
{
name: "bucket request, virtual-hosted style",
target: "/",
host: bkt + "." + domain,
domains: []string{"some-domain", domain},
expectedType: bucketType,
expectedBktName: bkt,
},
{
name: "object request, virtual-hosted style",
target: "/" + obj,
host: bkt + "." + domain,
domains: []string{"some-domain", domain},
expectedType: objectType,
expectedBktName: bkt,
expectedObjName: obj,
},
} {
t.Run(tc.name, func(t *testing.T) {
r := httptest.NewRequest(http.MethodPut, tc.target, nil)
r.Host = tc.host
reqType, bktName, objName := getBucketObject(r, tc.domains)
require.Equal(t, tc.expectedType, reqType)
require.Equal(t, tc.expectedBktName, bktName)
require.Equal(t, tc.expectedObjName, objName)
})
}
}
func TestDetermineBucketOperation(t *testing.T) { func TestDetermineBucketOperation(t *testing.T) {
const defaultValue = "value" const defaultValue = "value"

View file

@ -12,7 +12,6 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-s3-gw/api/data" "git.frostfs.info/TrueCloudLab/frostfs-s3-gw/api/data"
"git.frostfs.info/TrueCloudLab/frostfs-s3-gw/internal/logs" "git.frostfs.info/TrueCloudLab/frostfs-s3-gw/internal/logs"
treepool "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/pool/tree" treepool "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/pool/tree"
"github.com/go-chi/chi/v5"
"github.com/google/uuid" "github.com/google/uuid"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc/metadata" "google.golang.org/grpc/metadata"
@ -41,6 +40,8 @@ type (
Namespace string Namespace string
User string // User owner id User string // User owner id
Tagging *data.Tagging Tagging *data.Tagging
RequestVHSEnabled bool
RequestType ReqType
} }
// ObjectRequest represents object request data. // ObjectRequest represents object request data.
@ -61,10 +62,6 @@ const (
const HdrAmzRequestID = "x-amz-request-id" const HdrAmzRequestID = "x-amz-request-id"
const (
BucketURLPrm = "bucket"
)
var deploymentID = uuid.Must(uuid.NewRandom()) var deploymentID = uuid.Must(uuid.NewRandom())
var ( var (
@ -202,57 +199,6 @@ func Request(log *zap.Logger, settings RequestSettings) Func {
} }
} }
// AddBucketName adds bucket name to ReqInfo from context.
func AddBucketName(l *zap.Logger) Func {
return func(h http.Handler) http.Handler {
return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
ctx := r.Context()
reqInfo := GetReqInfo(ctx)
reqInfo.BucketName = chi.URLParam(r, BucketURLPrm)
if reqInfo.BucketName != "" {
reqLogger := reqLogOrDefault(ctx, l)
r = r.WithContext(SetReqLogger(ctx, reqLogger.With(zap.String("bucket", reqInfo.BucketName))))
}
h.ServeHTTP(w, r)
})
}
}
// AddObjectName adds objects name to ReqInfo from context.
func AddObjectName(l *zap.Logger) Func {
return func(h http.Handler) http.Handler {
return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
ctx := r.Context()
reqInfo := GetReqInfo(ctx)
reqLogger := reqLogOrDefault(ctx, l)
rctx := chi.RouteContext(ctx)
// trim leading slash (always present)
reqInfo.ObjectName = rctx.RoutePath[1:]
if r.URL.RawPath != "" {
// we have to do this because of
// https://github.com/go-chi/chi/issues/641
// https://github.com/go-chi/chi/issues/642
if obj, err := url.PathUnescape(reqInfo.ObjectName); err != nil {
reqLogger.Warn(logs.FailedToUnescapeObjectName, zap.Error(err))
} else {
reqInfo.ObjectName = obj
}
}
if reqInfo.ObjectName != "" {
r = r.WithContext(SetReqLogger(ctx, reqLogger.With(zap.String("object", reqInfo.ObjectName))))
}
h.ServeHTTP(w, r)
})
}
}
// getSourceIP retrieves the IP from the X-Forwarded-For, X-Real-IP and RFC7239 // getSourceIP retrieves the IP from the X-Forwarded-For, X-Real-IP and RFC7239
// Forwarded headers (in that order), falls back to r.RemoteAddr when everything // Forwarded headers (in that order), falls back to r.RemoteAddr when everything
// else fails. // else fails.

View file

@ -97,6 +97,7 @@ type Settings interface {
s3middleware.RequestSettings s3middleware.RequestSettings
s3middleware.PolicySettings s3middleware.PolicySettings
s3middleware.MetricsSettings s3middleware.MetricsSettings
s3middleware.VHSSettings
} }
type FrostFSID interface { type FrostFSID interface {
@ -113,9 +114,6 @@ type Config struct {
MiddlewareSettings Settings MiddlewareSettings Settings
// Domains optional. If empty no virtual hosted domains will be attached.
Domains []string
FrostfsID FrostFSID FrostfsID FrostFSID
FrostFSIDValidation bool FrostFSIDValidation bool
@ -142,11 +140,11 @@ func NewRouter(cfg Config) *chi.Mux {
api.Use(s3middleware.FrostfsIDValidation(cfg.FrostfsID, cfg.Log)) api.Use(s3middleware.FrostfsIDValidation(cfg.FrostfsID, cfg.Log))
} }
api.Use(s3middleware.PrepareAddressStyle(cfg.MiddlewareSettings, cfg.Log))
api.Use(s3middleware.PolicyCheck(s3middleware.PolicyConfig{ api.Use(s3middleware.PolicyCheck(s3middleware.PolicyConfig{
Storage: cfg.PolicyChecker, Storage: cfg.PolicyChecker,
FrostfsID: cfg.FrostfsID, FrostfsID: cfg.FrostfsID,
Settings: cfg.MiddlewareSettings, Settings: cfg.MiddlewareSettings,
Domains: cfg.Domains,
Log: cfg.Log, Log: cfg.Log,
BucketResolver: cfg.Handler.ResolveBucket, BucketResolver: cfg.Handler.ResolveBucket,
Decoder: cfg.XMLDecoder, Decoder: cfg.XMLDecoder,
@ -154,22 +152,41 @@ func NewRouter(cfg Config) *chi.Mux {
})) }))
defaultRouter := chi.NewRouter() defaultRouter := chi.NewRouter()
defaultRouter.Mount(fmt.Sprintf("/{%s}", s3middleware.BucketURLPrm), bucketRouter(cfg.Handler, cfg.Log)) defaultRouter.Mount("/{bucket}", bucketRouter(cfg.Handler))
defaultRouter.Get("/", named("ListBuckets", cfg.Handler.ListBucketsHandler)) defaultRouter.Get("/", named(s3middleware.ListBucketsOperation, cfg.Handler.ListBucketsHandler))
attachErrorHandler(defaultRouter) attachErrorHandler(defaultRouter)
hr := NewHostBucketRouter("bucket") vhsRouter := bucketRouter(cfg.Handler)
hr.Default(defaultRouter) router := newGlobalRouter(defaultRouter, vhsRouter)
for _, domain := range cfg.Domains {
hr.Map(domain, bucketRouter(cfg.Handler, cfg.Log)) api.Mount("/", router)
}
api.Mount("/", hr)
attachErrorHandler(api) attachErrorHandler(api)
return api return api
} }
type globalRouter struct {
pathStyleRouter chi.Router
vhsRouter chi.Router
}
func newGlobalRouter(pathStyleRouter, vhsRouter chi.Router) *globalRouter {
return &globalRouter{
pathStyleRouter: pathStyleRouter,
vhsRouter: vhsRouter,
}
}
func (g *globalRouter) ServeHTTP(w http.ResponseWriter, r *http.Request) {
router := g.pathStyleRouter
if reqInfo := s3middleware.GetReqInfo(r.Context()); reqInfo.RequestVHSEnabled {
router = g.vhsRouter
}
router.ServeHTTP(w, r)
}
func named(name string, handlerFunc http.HandlerFunc) http.HandlerFunc { func named(name string, handlerFunc http.HandlerFunc) http.HandlerFunc {
return func(w http.ResponseWriter, r *http.Request) { return func(w http.ResponseWriter, r *http.Request) {
reqInfo := s3middleware.GetReqInfo(r.Context()) reqInfo := s3middleware.GetReqInfo(r.Context())
@ -214,14 +231,13 @@ func attachErrorHandler(api *chi.Mux) {
api.MethodNotAllowed(named("MethodNotAllowed", errorHandler)) api.MethodNotAllowed(named("MethodNotAllowed", errorHandler))
} }
func bucketRouter(h Handler, log *zap.Logger) chi.Router { func bucketRouter(h Handler) chi.Router {
bktRouter := chi.NewRouter() bktRouter := chi.NewRouter()
bktRouter.Use( bktRouter.Use(
s3middleware.AddBucketName(log),
s3middleware.WrapHandler(h.AppendCORSHeaders), s3middleware.WrapHandler(h.AppendCORSHeaders),
) )
bktRouter.Mount("/", objectRouter(h, log)) bktRouter.Mount("/", objectRouter(h))
bktRouter.Options("/", named(s3middleware.OptionsBucketOperation, h.Preflight)) bktRouter.Options("/", named(s3middleware.OptionsBucketOperation, h.Preflight))
@ -293,7 +309,7 @@ func bucketRouter(h Handler, log *zap.Logger) chi.Router {
Add(NewFilter(). Add(NewFilter().
Queries(s3middleware.VersionsQuery). Queries(s3middleware.VersionsQuery).
Handler(named(s3middleware.ListBucketObjectVersionsOperation, h.ListBucketObjectVersionsHandler))). Handler(named(s3middleware.ListBucketObjectVersionsOperation, h.ListBucketObjectVersionsHandler))).
DefaultHandler(named(s3middleware.ListObjectsV1Operation, h.ListObjectsV1Handler))) DefaultHandler(listWrapper(h)))
}) })
// PUT method handlers // PUT method handlers
@ -368,9 +384,20 @@ func bucketRouter(h Handler, log *zap.Logger) chi.Router {
return bktRouter return bktRouter
} }
func objectRouter(h Handler, l *zap.Logger) chi.Router { func listWrapper(h Handler) http.HandlerFunc {
return func(w http.ResponseWriter, r *http.Request) {
if reqInfo := s3middleware.GetReqInfo(r.Context()); reqInfo.BucketName == "" {
reqInfo.API = s3middleware.ListBucketsOperation
h.ListBucketsHandler(w, r)
} else {
reqInfo.API = s3middleware.ListObjectsV1Operation
h.ListObjectsV1Handler(w, r)
}
}
}
func objectRouter(h Handler) chi.Router {
objRouter := chi.NewRouter() objRouter := chi.NewRouter()
objRouter.Use(s3middleware.AddObjectName(l))
objRouter.Options("/*", named(s3middleware.OptionsObjectOperation, h.Preflight)) objRouter.Options("/*", named(s3middleware.OptionsObjectOperation, h.Preflight))

View file

@ -73,6 +73,9 @@ func (c *centerMock) Authenticate(*http.Request) (*middleware.Box, error) {
type middlewareSettingsMock struct { type middlewareSettingsMock struct {
denyByDefault bool denyByDefault bool
sourceIPHeader string sourceIPHeader string
domains []string
vhsEnabled bool
vhsNamespacesEnabled map[string]bool
} }
func (r *middlewareSettingsMock) SourceIPHeader() string { func (r *middlewareSettingsMock) SourceIPHeader() string {
@ -91,6 +94,18 @@ func (r *middlewareSettingsMock) PolicyDenyByDefault() bool {
return r.denyByDefault return r.denyByDefault
} }
func (r *middlewareSettingsMock) Domains() []string {
return r.domains
}
func (r *middlewareSettingsMock) GlobalVHS() bool {
return r.vhsEnabled
}
func (r *middlewareSettingsMock) VHSNamespacesEnabled() map[string]bool {
return r.vhsNamespacesEnabled
}
type frostFSIDMock struct { type frostFSIDMock struct {
tags map[string]string tags map[string]string
validateError bool validateError bool

View file

@ -78,7 +78,6 @@ func prepareRouter(t *testing.T, opts ...option) *routerMock {
Metrics: metrics.NewAppMetrics(metricsConfig), Metrics: metrics.NewAppMetrics(metricsConfig),
MiddlewareSettings: middlewareSettings, MiddlewareSettings: middlewareSettings,
PolicyChecker: policyChecker, PolicyChecker: policyChecker,
Domains: []string{"domain1", "domain2"},
FrostfsID: &frostFSIDMock{}, FrostfsID: &frostFSIDMock{},
XMLDecoder: &xmlMock{}, XMLDecoder: &xmlMock{},
Tagging: &resourceTaggingMock{}, Tagging: &resourceTaggingMock{},
@ -847,6 +846,31 @@ func TestFrostFSIDValidation(t *testing.T) {
createBucketErr(chiRouter, "", "bkt-3", nil, apiErrors.ErrInternalError) createBucketErr(chiRouter, "", "bkt-3", nil, apiErrors.ErrInternalError)
} }
func TestRouterListObjectsV2Domains(t *testing.T) {
chiRouter := prepareRouter(t, enableVHSDomains("domain.com"))
chiRouter.handler.buckets["bucket"] = &data.BucketInfo{}
w := httptest.NewRecorder()
r := httptest.NewRequest(http.MethodGet, "/", nil)
r.Host = "bucket.domain.com"
query := make(url.Values)
query.Set(s3middleware.ListTypeQuery, "2")
r.URL.RawQuery = query.Encode()
chiRouter.ServeHTTP(w, r)
resp := readResponse(t, w)
require.Equal(t, s3middleware.ListObjectsV2Operation, resp.Method)
}
func enableVHSDomains(domains ...string) option {
return func(cfg *Config) {
setting := cfg.MiddlewareSettings.(*middlewareSettingsMock)
setting.vhsEnabled = true
setting.domains = domains
}
}
func readResponse(t *testing.T, w *httptest.ResponseRecorder) handlerResult { func readResponse(t *testing.T, w *httptest.ResponseRecorder) handlerResult {
var res handlerResult var res handlerResult

View file

@ -105,9 +105,11 @@ type (
policyDenyByDefault bool policyDenyByDefault bool
sourceIPHeader string sourceIPHeader string
retryMaxAttempts int retryMaxAttempts int
domains []string
vhsEnabled bool
vhsNamespacesEnabled map[string]bool
retryMaxBackoff time.Duration retryMaxBackoff time.Duration
retryStrategy handler.RetryStrategy retryStrategy handler.RetryStrategy
domains []string
} }
maxClientsConfig struct { maxClientsConfig struct {
@ -256,13 +258,39 @@ func (s *appSettings) updateNamespacesSettings(v *viper.Viper, log *zap.Logger)
s.namespaces = nsConfig.Namespaces s.namespaces = nsConfig.Namespaces
} }
func (s *appSettings) setVHSSettings(v *viper.Viper, _ *zap.Logger) { func (s *appSettings) setVHSSettings(v *viper.Viper, log *zap.Logger) {
domains := v.GetStringSlice(cfgListenDomains) domains := fetchDomains(v, log)
vhsEnabled := v.GetBool(cfgVHSEnabled)
nsMap := fetchVHSNamespaces(v, log)
vhsNamespaces := make(map[string]bool, len(nsMap))
for ns, flag := range nsMap {
vhsNamespaces[s.ResolveNamespaceAlias(ns)] = flag
}
s.mu.Lock() s.mu.Lock()
defer s.mu.Unlock() defer s.mu.Unlock()
s.domains = domains s.domains = domains
s.vhsEnabled = vhsEnabled
s.vhsNamespacesEnabled = vhsNamespaces
}
func (s *appSettings) Domains() []string {
s.mu.RLock()
defer s.mu.RUnlock()
return s.domains
}
func (s *appSettings) GlobalVHS() bool {
s.mu.RLock()
defer s.mu.RUnlock()
return s.vhsEnabled
}
func (s *appSettings) VHSNamespacesEnabled() map[string]bool {
s.mu.RLock()
defer s.mu.RUnlock()
return s.vhsNamespacesEnabled
} }
func (s *appSettings) BypassContentEncodingInChunks() bool { func (s *appSettings) BypassContentEncodingInChunks() bool {
@ -467,12 +495,6 @@ func (s *appSettings) RetryStrategy() handler.RetryStrategy {
return s.retryStrategy return s.retryStrategy
} }
func (s *appSettings) Domains() []string {
s.mu.RLock()
defer s.mu.RUnlock()
return s.domains
}
func (a *App) initAPI(ctx context.Context) { func (a *App) initAPI(ctx context.Context) {
a.initLayer(ctx) a.initLayer(ctx)
a.initHandler() a.initHandler()
@ -710,9 +732,6 @@ func (a *App) setHealthStatus() {
// Serve runs HTTP server to handle S3 API requests. // Serve runs HTTP server to handle S3 API requests.
func (a *App) Serve(ctx context.Context) { func (a *App) Serve(ctx context.Context) {
// Attach S3 API:
a.log.Info(logs.FetchDomainsPrepareToUseAPI, zap.Strings("domains", a.settings.Domains()))
cfg := api.Config{ cfg := api.Config{
Throttle: middleware.ThrottleOpts{ Throttle: middleware.ThrottleOpts{
Limit: a.settings.maxClient.count, Limit: a.settings.maxClient.count,
@ -722,7 +741,6 @@ func (a *App) Serve(ctx context.Context) {
Center: a.ctr, Center: a.ctr,
Log: a.log, Log: a.log,
Metrics: a.metrics, Metrics: a.metrics,
Domains: a.settings.Domains(),
MiddlewareSettings: a.settings, MiddlewareSettings: a.settings,
PolicyChecker: a.policyStorage, PolicyChecker: a.policyStorage,

View file

@ -30,6 +30,8 @@ import (
const ( const (
destinationStdout = "stdout" destinationStdout = "stdout"
destinationJournald = "journald" destinationJournald = "journald"
wildcardPlaceholder = "<wildcard>"
) )
const ( const (
@ -144,6 +146,9 @@ const ( // Settings.
cfgListenDomains = "listen_domains" cfgListenDomains = "listen_domains"
cfgVHSEnabled = "vhs.enabled"
cfgVHSNamespaces = "vhs.namespaces"
// Peers. // Peers.
cfgPeers = "peers" cfgPeers = "peers"
@ -668,6 +673,41 @@ func fetchServers(v *viper.Viper, log *zap.Logger) []ServerInfo {
return servers return servers
} }
func fetchDomains(v *viper.Viper, log *zap.Logger) []string {
domains := validateDomains(v.GetStringSlice(cfgListenDomains), log)
countParts := func(domain string) int {
return strings.Count(domain, ".")
}
sort.Slice(domains, func(i, j int) bool {
return countParts(domains[i]) > countParts(domains[j])
})
return domains
}
func fetchVHSNamespaces(v *viper.Viper, log *zap.Logger) map[string]bool {
vhsNamespacesEnabled := make(map[string]bool)
nsMap := v.GetStringMap(cfgVHSNamespaces)
for ns, val := range nsMap {
if _, ok := vhsNamespacesEnabled[ns]; ok {
log.Warn(logs.WarnDuplicateNamespaceVHS, zap.String("namespace", ns))
continue
}
enabledFlag, ok := val.(bool)
if !ok {
log.Warn(logs.WarnValueVHSEnabledFlagWrongType, zap.String("namespace", ns))
continue
}
vhsNamespacesEnabled[ns] = enabledFlag
}
return vhsNamespacesEnabled
}
func newSettings() *viper.Viper { func newSettings() *viper.Viper {
v := viper.New() v := viper.New()
@ -1029,3 +1069,19 @@ func getLogLevel(v *viper.Viper) (zapcore.Level, error) {
} }
return lvl, nil return lvl, nil
} }
func validateDomains(domains []string, log *zap.Logger) []string {
validDomains := make([]string, 0, len(domains))
LOOP:
for _, domain := range domains {
domainParts := strings.Split(domain, ".")
for _, part := range domainParts {
if strings.ContainsAny(part, "<>") && part != wildcardPlaceholder {
log.Warn(logs.WarnDomainContainsInvalidPlaceholder, zap.String("domain", domain))
continue LOOP
}
}
validDomains = append(validDomains, domain)
}
return validDomains
}

View file

@ -0,0 +1,34 @@
package main
import (
"testing"
"github.com/stretchr/testify/require"
"go.uber.org/zap/zaptest"
)
func TestValidateDomains(t *testing.T) {
inputDomains := []string{
"s3dev.frostfs.devenv",
"s3dev.<invalid>.frostfs.devenv",
"s3dev.<wildcard>.frostfs.devenv",
"s3dev.<wildcard.frostfs.devenv",
"s3dev.wildcard>.frostfs.devenv",
"s3dev.<wild.card>.frostfs.devenv",
"<invalid>.frostfs.devenv",
"<wildcard>.frostfs.devenv>",
"<wildcard>.frostfs.devenv",
"s3dev.fro<stfs.devenv",
"<wildcard>.dev.<wildcard>.frostfs.devenv",
"<wildcard>.dev.<wildc>ard>.frostfs.devenv",
}
expectedDomains := []string{
"s3dev.frostfs.devenv",
"s3dev.<wildcard>.frostfs.devenv",
"<wildcard>.frostfs.devenv",
"<wildcard>.dev.<wildcard>.frostfs.devenv",
}
actualDomains := validateDomains(inputDomains, zaptest.NewLogger(t))
require.Equal(t, expectedDomains, actualDomains)
}

View file

@ -36,8 +36,11 @@ S3_GW_SERVER_1_TLS_KEY_FILE=/path/to/tls/key
# How often to reconnect to the servers # How often to reconnect to the servers
S3_GW_RECONNECT_INTERVAL: 1m S3_GW_RECONNECT_INTERVAL: 1m
# Domains to be able to use virtual-hosted-style access to bucket. # Domains to be able to use virtual-hosted-style access to bucket
S3_GW_LISTEN_DOMAINS=s3dev.frostfs.devenv S3_GW_LISTEN_DOMAINS="domain.com <wildcard>.domain.com"
# VHS enabled flag
S3_GW_VHS_ENABLED=false
# Config file # Config file
S3_GW_CONFIG=/path/to/config/yaml S3_GW_CONFIG=/path/to/config/yaml

View file

@ -42,6 +42,13 @@ server:
# Domains to be able to use virtual-hosted-style access to bucket. # Domains to be able to use virtual-hosted-style access to bucket.
listen_domains: listen_domains:
- s3dev.frostfs.devenv - s3dev.frostfs.devenv
- s3dev.<wildcard>.frostfs.devenv
vhs:
enabled: false
namespaces:
"ns1": false
"ns2": true
logger: logger:
level: debug level: debug

View file

@ -193,12 +193,14 @@ There are some custom types used for brevity:
| `namespaces` | [Namespaces configuration](#namespaces-section) | | `namespaces` | [Namespaces configuration](#namespaces-section) |
| `retry` | [Retry configuration](#retry-section) | | `retry` | [Retry configuration](#retry-section) |
| `containers` | [Containers configuration](#containers-section) | | `containers` | [Containers configuration](#containers-section) |
| `vhs` | [VHS configuration](#vhs-section) |
### General section ### General section
```yaml ```yaml
listen_domains: listen_domains:
- s3dev.frostfs.devenv - s3dev.frostfs.devenv
- s3dev.<wildcard>.frostfs.devenv
- s3dev2.frostfs.devenv - s3dev2.frostfs.devenv
rpc_endpoint: http://morph-chain.frostfs.devenv:30333 rpc_endpoint: http://morph-chain.frostfs.devenv:30333
@ -226,7 +228,7 @@ source_ip_header: "Source-Ip"
| Parameter | Type | SIGHUP reload | Default value | Description | | Parameter | Type | SIGHUP reload | Default value | Description |
|----------------------------------|------------|---------------|---------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| |----------------------------------|------------|---------------|---------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
| `listen_domains` | `[]string` | no | | Domains to be able to use virtual-hosted-style access to bucket. | | `listen_domains` | `[]string` | yes | | Domains to be able to use virtual-hosted-style access to bucket. The presence of placeholders of the <wildcard> type is supported. |
| `rpc_endpoint` | `string` | no | | The address of the RPC host to which the gateway connects to resolve bucket names and interact with frostfs contracts (required to use the `nns` resolver and `frostfsid` contract). | | `rpc_endpoint` | `string` | no | | The address of the RPC host to which the gateway connects to resolve bucket names and interact with frostfs contracts (required to use the `nns` resolver and `frostfsid` contract). |
| `resolve_order` | `[]string` | yes | `[dns]` | Order of bucket name resolvers to use. Available resolvers: `dns`, `nns`. | | `resolve_order` | `[]string` | yes | `[dns]` | Order of bucket name resolvers to use. Available resolvers: `dns`, `nns`. |
| `connect_timeout` | `duration` | no | `10s` | Timeout to connect to a node. | | `connect_timeout` | `duration` | no | `10s` | Timeout to connect to a node. |
@ -723,3 +725,20 @@ containers:
|-------------|----------|---------------|---------------|-------------------------------------------------------------------------------------------| |-------------|----------|---------------|---------------|-------------------------------------------------------------------------------------------|
| `cors` | `string` | no | | Container name for CORS configurations. If not set, container of the bucket is used. | | `cors` | `string` | no | | Container name for CORS configurations. If not set, container of the bucket is used. |
| `lifecycle` | `string` | no | | Container name for lifecycle configurations. If not set, container of the bucket is used. | | `lifecycle` | `string` | no | | Container name for lifecycle configurations. If not set, container of the bucket is used. |
# `vhs` section
Configuration of virtual hosted addressing style.
```yaml
vhs:
enabled: false
namespaces:
"ns1": false
"ns2": true
```
| Parameter | Type | SIGHUP reload | Default value | Description |
| ------------ | ----------------- | ------------- | ------------- | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- |
| `enabled` | `bool` | yes | `false` | Enables the use of virtual host addressing for banquets at the application level. |
| `namespaces` | `map[string]bool` | yes | | A map in which the keys are the name of the namespace, and the values are the flag responsible for enabling VHS for the specified namespace. Overrides global 'enabled' setting even when it is disabled. |

View file

@ -20,7 +20,6 @@ const (
UsingCredentials = "using credentials" // Info in ../../cmd/s3-gw/app.go UsingCredentials = "using credentials" // Info in ../../cmd/s3-gw/app.go
ApplicationStarted = "application started" // Info in ../../cmd/s3-gw/app.go ApplicationStarted = "application started" // Info in ../../cmd/s3-gw/app.go
ApplicationFinished = "application finished" // Info in ../../cmd/s3-gw/app.go ApplicationFinished = "application finished" // Info in ../../cmd/s3-gw/app.go
FetchDomainsPrepareToUseAPI = "fetch domains, prepare to use API" // Info in ../../cmd/s3-gw/app.go
StartingServer = "starting server" // Info in ../../cmd/s3-gw/app.go StartingServer = "starting server" // Info in ../../cmd/s3-gw/app.go
StoppingServer = "stopping server" // Info in ../../cmd/s3-gw/app.go StoppingServer = "stopping server" // Info in ../../cmd/s3-gw/app.go
SIGHUPConfigReloadStarted = "SIGHUP config reload started" // Info in ../../cmd/s3-gw/app.go SIGHUPConfigReloadStarted = "SIGHUP config reload started" // Info in ../../cmd/s3-gw/app.go
@ -159,4 +158,7 @@ const (
CouldNotFetchLifecycleContainerInfo = "couldn't fetch lifecycle container info" CouldNotFetchLifecycleContainerInfo = "couldn't fetch lifecycle container info"
BucketLifecycleNodeHasMultipleIDs = "bucket lifecycle node has multiple ids" BucketLifecycleNodeHasMultipleIDs = "bucket lifecycle node has multiple ids"
GetBucketLifecycle = "get bucket lifecycle" GetBucketLifecycle = "get bucket lifecycle"
WarnDuplicateNamespaceVHS = "duplicate namespace with enabled VHS, config value skipped"
WarnValueVHSEnabledFlagWrongType = "the value of the VHS enable flag for the namespace is of the wrong type, config value skipped"
WarnDomainContainsInvalidPlaceholder = "the domain contains an invalid placeholder, domain skipped"
) )