feat: refactor internal iam service

This moves the internal iam service from the posix backend so
that we can start implementing new iam services right in the auth
module.

The internal iam service has same behavior as before, but now
must be enabled with the --iam-dir cli option.

New single user service is the default when no other iam service
is selected. This just runs the gateway in single user mode with
just the root account.
This commit is contained in:
Ben McClelland
2023-10-02 09:47:16 -07:00
parent 4661af11dd
commit ae4e382e61
10 changed files with 301 additions and 275 deletions

View File

@@ -17,7 +17,6 @@ package auth
import (
"encoding/json"
"fmt"
"os"
"strings"
"github.com/aws/aws-sdk-go-v2/service/s3"
@@ -236,29 +235,14 @@ func VerifyACL(acl ACL, access string, permission types.Permission, isRoot bool)
return s3err.GetAPIError(s3err.ErrAccessDenied)
}
func IsAdmin(access string, isRoot bool) error {
var data IAMConfig
func IsAdmin(acct Account, isRoot bool) error {
if isRoot {
return nil
}
file, err := os.ReadFile("users.json")
if err != nil {
return fmt.Errorf("unable to read config file: %w", err)
}
if err := json.Unmarshal(file, &data); err != nil {
return err
}
acc, ok := data.AccessAccounts[access]
if !ok {
return fmt.Errorf("user does not exist")
}
if acc.Role == "admin" {
if acct.Role == "admin" {
return nil
}
return s3err.GetAPIError(s3err.ErrAccessDenied)
}

View File

@@ -16,45 +16,59 @@ package auth
import (
"encoding/json"
"errors"
"fmt"
"hash/crc32"
"io/fs"
"os"
"path/filepath"
"sync"
"time"
)
const (
iamFile = "users.json"
iamBackupFile = "users.json.backup"
)
var (
cacheDuration = 5 * time.Minute
)
// IAMServiceInternal manages the internal IAM service
type IAMServiceInternal struct {
storer Storer
path string
mu sync.RWMutex
accts IAMConfig
serial uint32
mu sync.RWMutex
accts iAMConfig
serial uint32
iamcache []byte
iamvalid bool
iamexpire time.Time
}
// UpdateAcctFunc accepts the current data and returns the new data to be stored
type UpdateAcctFunc func([]byte) ([]byte, error)
// Storer is the interface to manage the peristent IAM data for the internal
// IAM service
type Storer interface {
InitIAM() error
GetIAM() ([]byte, error)
StoreIAM(UpdateAcctFunc) error
}
// IAMConfig stores all internal IAM accounts
type IAMConfig struct {
// iAMConfig stores all internal IAM accounts
type iAMConfig struct {
AccessAccounts map[string]Account `json:"accessAccounts"`
}
var _ IAMService = &IAMServiceInternal{}
// NewInternal creates a new instance for the Internal IAM service
func NewInternal(s Storer) (*IAMServiceInternal, error) {
func NewInternal(path string) (*IAMServiceInternal, error) {
i := &IAMServiceInternal{
storer: s,
path: path,
}
err := i.updateCache()
err := i.initIAM()
if err != nil {
return nil, fmt.Errorf("init iam: %w", err)
}
err = i.updateCache()
if err != nil {
return nil, fmt.Errorf("refresh iam cache: %w", err)
}
@@ -68,15 +82,15 @@ func (s *IAMServiceInternal) CreateAccount(account Account) error {
s.mu.Lock()
defer s.mu.Unlock()
return s.storer.StoreIAM(func(data []byte) ([]byte, error) {
var conf IAMConfig
return s.storeIAM(func(data []byte) ([]byte, error) {
var conf iAMConfig
if len(data) > 0 {
if err := json.Unmarshal(data, &conf); err != nil {
return nil, fmt.Errorf("failed to parse iam: %w", err)
}
} else {
conf = IAMConfig{AccessAccounts: map[string]Account{}}
conf = iAMConfig{AccessAccounts: map[string]Account{}}
}
_, ok := conf.AccessAccounts[account.Access]
@@ -101,7 +115,7 @@ func (s *IAMServiceInternal) GetUserAccount(access string) (Account, error) {
s.mu.RLock()
defer s.mu.RUnlock()
data, err := s.storer.GetIAM()
data, err := s.getIAM()
if err != nil {
return Account{}, fmt.Errorf("get iam data: %w", err)
}
@@ -129,7 +143,7 @@ func (s *IAMServiceInternal) updateCache() error {
s.mu.Lock()
defer s.mu.Unlock()
data, err := s.storer.GetIAM()
data, err := s.getIAM()
if err != nil {
return fmt.Errorf("get iam data: %w", err)
}
@@ -155,13 +169,13 @@ func (s *IAMServiceInternal) DeleteUserAccount(access string) error {
s.mu.Lock()
defer s.mu.Unlock()
return s.storer.StoreIAM(func(data []byte) ([]byte, error) {
return s.storeIAM(func(data []byte) ([]byte, error) {
if len(data) == 0 {
// empty config, do nothing
return data, nil
}
var conf IAMConfig
var conf iAMConfig
if err := json.Unmarshal(data, &conf); err != nil {
return nil, fmt.Errorf("failed to parse iam: %w", err)
@@ -185,7 +199,7 @@ func (s *IAMServiceInternal) ListUserAccounts() (accs []Account, err error) {
s.mu.RLock()
defer s.mu.RUnlock()
data, err := s.storer.GetIAM()
data, err := s.getIAM()
if err != nil {
return []Account{}, fmt.Errorf("get iam data: %w", err)
}
@@ -210,3 +224,186 @@ func (s *IAMServiceInternal) ListUserAccounts() (accs []Account, err error) {
return accs, nil
}
const (
iamMode = 0600
)
func (s *IAMServiceInternal) initIAM() error {
fname := filepath.Join(s.path, iamFile)
_, err := os.ReadFile(fname)
if errors.Is(err, fs.ErrNotExist) {
b, err := json.Marshal(iAMConfig{AccessAccounts: map[string]Account{}})
if err != nil {
return fmt.Errorf("marshal default iam: %w", err)
}
err = os.WriteFile(fname, b, iamMode)
if err != nil {
return fmt.Errorf("write default iam: %w", err)
}
}
return nil
}
func (s *IAMServiceInternal) getIAM() ([]byte, error) {
if !s.iamvalid || !s.iamexpire.After(time.Now()) {
err := s.refreshIAM()
if err != nil {
return nil, err
}
}
return s.iamcache, nil
}
const (
backoff = 100 * time.Millisecond
maxretry = 300
)
func (s *IAMServiceInternal) refreshIAM() error {
// We are going to be racing with other running gateways without any
// coordination. So we might find the file does not exist at times.
// For this case we need to retry for a while assuming the other gateway
// will eventually write the file. If it doesn't after the max retries,
// then we will return the error.
retries := 0
for {
b, err := os.ReadFile(filepath.Join(s.path, iamFile))
if errors.Is(err, fs.ErrNotExist) {
// racing with someone else updating
// keep retrying after backoff
retries++
if retries < maxretry {
time.Sleep(backoff)
continue
}
return fmt.Errorf("read iam file: %w", err)
}
if err != nil {
return err
}
s.iamcache = b
s.iamvalid = true
s.iamexpire = time.Now().Add(cacheDuration)
break
}
return nil
}
func (s *IAMServiceInternal) storeIAM(update UpdateAcctFunc) error {
// We are going to be racing with other running gateways without any
// coordination. So the strategy here is to read the current file data.
// If the file doesn't exist, then we assume someone else is currently
// updating the file. So we just need to keep retrying. We also need
// to make sure the data is consistent within a single update. So racing
// writes to a file would possibly leave this in some invalid state.
// We can get atomic updates with rename. If we read the data, update
// the data, write to a temp file, then rename the tempfile back to the
// data file. This should always result in a complete data image.
// There is at least one unsolved failure mode here.
// If a gateway removes the data file and then crashes, all other
// gateways will retry forever thinking that the original will eventually
// write the file.
retries := 0
fname := filepath.Join(s.path, iamFile)
for {
b, err := os.ReadFile(fname)
if errors.Is(err, fs.ErrNotExist) {
// racing with someone else updating
// keep retrying after backoff
retries++
if retries < maxretry {
time.Sleep(backoff)
continue
}
// we have been unsuccessful trying to read the iam file
// so this must be the case where something happened and
// the file did not get updated successfully, and probably
// isn't going to be. The recovery procedure would be to
// copy the backup file into place of the original.
return fmt.Errorf("no iam file, needs backup recovery")
}
if err != nil && !errors.Is(err, fs.ErrNotExist) {
return fmt.Errorf("read iam file: %w", err)
}
// reset retries on successful read
retries = 0
err = os.Remove(iamFile)
if errors.Is(err, fs.ErrNotExist) {
// racing with someone else updating
// keep retrying after backoff
time.Sleep(backoff)
continue
}
if err != nil && !errors.Is(err, fs.ErrNotExist) {
return fmt.Errorf("remove old iam file: %w", err)
}
// save copy of data
datacopy := make([]byte, len(b))
copy(datacopy, b)
// make a backup copy in case we crash before update
// this is after remove, so there is a small window something
// can go wrong, but the remove should barrier other gateways
// from trying to write backup at the same time. Only one
// gateway will successfully remove the file.
os.WriteFile(filepath.Join(s.path, iamBackupFile), b, iamMode)
b, err = update(b)
if err != nil {
// update failed, try to write old data back out
os.WriteFile(fname, datacopy, iamMode)
return fmt.Errorf("update iam data: %w", err)
}
err = s.writeTempFile(b)
if err != nil {
// update failed, try to write old data back out
os.WriteFile(fname, datacopy, iamMode)
return err
}
s.iamcache = b
s.iamvalid = true
s.iamexpire = time.Now().Add(cacheDuration)
break
}
return nil
}
func (s *IAMServiceInternal) writeTempFile(b []byte) error {
fname := filepath.Join(s.path, iamFile)
f, err := os.CreateTemp(s.path, iamFile)
if err != nil {
return fmt.Errorf("create temp file: %w", err)
}
defer os.Remove(f.Name())
_, err = f.Write(b)
if err != nil {
return fmt.Errorf("write temp file: %w", err)
}
err = os.Rename(f.Name(), fname)
if err != nil {
return fmt.Errorf("rename temp file: %w", err)
}
return nil
}

40
auth/iam_single.go Normal file
View File

@@ -0,0 +1,40 @@
// Copyright 2023 Versity Software
// This file is licensed under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package auth
import "fmt"
// IAMServiceSingle manages the single tenant (root-only) IAM service
type IAMServiceSingle struct{}
// CreateAccount not valid in single tenant mode
func (IAMServiceSingle) CreateAccount(account Account) error {
return fmt.Errorf("create user not valid in single tenant mode")
}
// GetUserAccount no accounts in single tenant mode
func (IAMServiceSingle) GetUserAccount(access string) (Account, error) {
return Account{}, ErrNoSuchUser
}
// DeleteUserAccount no accounts in single tenant mode
func (IAMServiceSingle) DeleteUserAccount(access string) error {
return ErrNoSuchUser
}
// ListUserAccounts no accounts in single tenant mode
func (IAMServiceSingle) ListUserAccounts() ([]Account, error) {
return []Account{}, nil
}

View File

@@ -29,9 +29,7 @@ import (
"sort"
"strconv"
"strings"
"sync"
"syscall"
"time"
"github.com/aws/aws-sdk-go-v2/service/s3"
"github.com/aws/aws-sdk-go-v2/service/s3/types"
@@ -48,19 +46,10 @@ type Posix struct {
rootfd *os.File
rootdir string
mu sync.RWMutex
iamcache []byte
iamvalid bool
iamexpire time.Time
}
var _ backend.Backend = &Posix{}
var (
cacheDuration = 5 * time.Minute
)
const (
metaTmpDir = ".sgwtmp"
metaTmpMultipartDir = metaTmpDir + "/multipart"
@@ -70,8 +59,6 @@ const (
contentTypeHdr = "content-type"
contentEncHdr = "content-encoding"
emptyMD5 = "d41d8cd98f00b204e9800998ecf8427e"
iamFile = "users.json"
iamBackupFile = "users.json.backup"
aclkey = "user.acl"
etagkey = "user.etag"
)
@@ -1656,198 +1643,6 @@ func (p *Posix) ListBucketsAndOwners(ctx context.Context) (buckets []s3response.
return buckets, nil
}
const (
iamMode = 0600
)
func (p *Posix) InitIAM() error {
p.mu.RLock()
defer p.mu.RUnlock()
_, err := os.ReadFile(iamFile)
if errors.Is(err, fs.ErrNotExist) {
b, err := json.Marshal(auth.IAMConfig{AccessAccounts: map[string]auth.Account{}})
if err != nil {
return fmt.Errorf("marshal default iam: %w", err)
}
err = os.WriteFile(iamFile, b, iamMode)
if err != nil {
return fmt.Errorf("write default iam: %w", err)
}
}
return nil
}
func (p *Posix) GetIAM() ([]byte, error) {
p.mu.RLock()
defer p.mu.RUnlock()
if !p.iamvalid || !p.iamexpire.After(time.Now()) {
p.mu.RUnlock()
err := p.refreshIAM()
p.mu.RLock()
if err != nil {
return nil, err
}
}
return p.iamcache, nil
}
const (
backoff = 100 * time.Millisecond
maxretry = 300
)
func (p *Posix) refreshIAM() error {
p.mu.Lock()
defer p.mu.Unlock()
// We are going to be racing with other running gateways without any
// coordination. So we might find the file does not exist at times.
// For this case we need to retry for a while assuming the other gateway
// will eventually write the file. If it doesn't after the max retries,
// then we will return the error.
retries := 0
for {
b, err := os.ReadFile(iamFile)
if errors.Is(err, fs.ErrNotExist) {
// racing with someone else updating
// keep retrying after backoff
retries++
if retries < maxretry {
time.Sleep(backoff)
continue
}
return fmt.Errorf("read iam file: %w", err)
}
if err != nil {
return err
}
p.iamcache = b
p.iamvalid = true
p.iamexpire = time.Now().Add(cacheDuration)
break
}
return nil
}
func (p *Posix) StoreIAM(update auth.UpdateAcctFunc) error {
p.mu.Lock()
defer p.mu.Unlock()
// We are going to be racing with other running gateways without any
// coordination. So the strategy here is to read the current file data.
// If the file doesn't exist, then we assume someone else is currently
// updating the file. So we just need to keep retrying. We also need
// to make sure the data is consistent within a single update. So racing
// writes to a file would possibly leave this in some invalid state.
// We can get atomic updates with rename. If we read the data, update
// the data, write to a temp file, then rename the tempfile back to the
// data file. This should always result in a complete data image.
// There is at least one unsolved failure mode here.
// If a gateway removes the data file and then crashes, all other
// gateways will retry forever thinking that the original will eventually
// write the file.
retries := 0
for {
b, err := os.ReadFile(iamFile)
if errors.Is(err, fs.ErrNotExist) {
// racing with someone else updating
// keep retrying after backoff
retries++
if retries < maxretry {
time.Sleep(backoff)
continue
}
// we have been unsuccessful trying to read the iam file
// so this must be the case where something happened and
// the file did not get updated successfully, and probably
// isn't going to be. The recovery procedure would be to
// copy the backup file into place of the original.
return fmt.Errorf("no iam file, needs backup recovery")
}
if err != nil && !errors.Is(err, fs.ErrNotExist) {
return fmt.Errorf("read iam file: %w", err)
}
// reset retries on successful read
retries = 0
err = os.Remove(iamFile)
if errors.Is(err, fs.ErrNotExist) {
// racing with someone else updating
// keep retrying after backoff
time.Sleep(backoff)
continue
}
if err != nil && !errors.Is(err, fs.ErrNotExist) {
return fmt.Errorf("remove old iam file: %w", err)
}
// save copy of data
datacopy := make([]byte, len(b))
copy(datacopy, b)
// make a backup copy in case we crash before update
// this is after remove, so there is a small window something
// can go wrong, but the remove should barrier other gateways
// from trying to write backup at the same time. Only one
// gateway will successfully remove the file.
os.WriteFile(iamBackupFile, b, iamMode)
b, err = update(b)
if err != nil {
// update failed, try to write old data back out
os.WriteFile(iamFile, datacopy, iamMode)
return fmt.Errorf("update iam data: %w", err)
}
err = writeTempFile(b)
if err != nil {
// update failed, try to write old data back out
os.WriteFile(iamFile, datacopy, iamMode)
return err
}
p.iamcache = b
p.iamvalid = true
p.iamexpire = time.Now().Add(cacheDuration)
break
}
return nil
}
func writeTempFile(b []byte) error {
f, err := os.CreateTemp(".", iamFile)
if err != nil {
return fmt.Errorf("create temp file: %w", err)
}
defer os.Remove(f.Name())
_, err = f.Write(b)
if err != nil {
return fmt.Errorf("write temp file: %w", err)
}
err = os.Rename(f.Name(), iamFile)
if err != nil {
return fmt.Errorf("rename temp file: %w", err)
}
return nil
}
func isNoAttr(err error) bool {
if err == nil {
return false

View File

@@ -44,6 +44,7 @@ var (
logWebhookURL string
accessLog string
debug bool
iamDir string
)
var (
@@ -207,10 +208,15 @@ func initFlags() []cli.Flag {
Destination: &natsTopic,
Aliases: []string{"ent"},
},
&cli.StringFlag{
Name: "iam-dir",
Usage: "if defined, run internal iam service within this directory",
Destination: &iamDir,
},
}
}
func runGateway(ctx *cli.Context, be backend.Backend, s auth.Storer) error {
func runGateway(ctx *cli.Context, be backend.Backend) error {
// int32 max for 32 bit arch
blimit := int64(2*1024*1024*1024 - 1)
if strconv.IntSize > 32 {
@@ -269,14 +275,18 @@ func runGateway(ctx *cli.Context, be backend.Backend, s auth.Storer) error {
admOpts = append(admOpts, s3api.WithAdminSrvTLS(cert))
}
err := s.InitIAM()
if err != nil {
return fmt.Errorf("init iam: %w", err)
}
iam, err := auth.NewInternal(s)
if err != nil {
return fmt.Errorf("setup internal iam service: %w", err)
var iam auth.IAMService
switch {
case iamDir != "":
var err error
iam, err = auth.NewInternal(iamDir)
if err != nil {
return fmt.Errorf("setup internal iam service: %w", err)
}
default:
// default gateway to single user mode when
// no other iam service configured
iam = auth.IAMServiceSingle{}
}
logger, err := s3log.InitLogger(&s3log.LogConfig{

View File

@@ -49,5 +49,5 @@ func runPosix(ctx *cli.Context) error {
return fmt.Errorf("init posix: %v", err)
}
return runGateway(ctx, be, be)
return runGateway(ctx, be)
}

View File

@@ -69,5 +69,5 @@ func runScoutfs(ctx *cli.Context) error {
return fmt.Errorf("init scoutfs: %v", err)
}
return runGateway(ctx, be, be)
return runGateway(ctx, be)
}

View File

@@ -6,7 +6,7 @@ rm -rf /tmp/covdata
mkdir /tmp/covdata
# run server in background
GOCOVERDIR=/tmp/covdata ./versitygw -a user -s pass posix /tmp/gw &
GOCOVERDIR=/tmp/covdata ./versitygw -a user -s pass --iam-dir /tmp/gw posix /tmp/gw &
GW_PID=$!
# wait a second for server to start up

View File

@@ -42,7 +42,7 @@ func TestAdminController_CreateUser(t *testing.T) {
app := fiber.New()
app.Use(func(ctx *fiber.Ctx) error {
ctx.Locals("role", "admin")
ctx.Locals("account", auth.Account{Access: "admin1", Secret: "secret", Role: "admin"})
return ctx.Next()
})
@@ -51,7 +51,7 @@ func TestAdminController_CreateUser(t *testing.T) {
appErr := fiber.New()
appErr.Use(func(ctx *fiber.Ctx) error {
ctx.Locals("role", "user")
ctx.Locals("account", auth.Account{Access: "user1", Secret: "secret", Role: "user"})
return ctx.Next()
})
@@ -121,7 +121,7 @@ func TestAdminController_DeleteUser(t *testing.T) {
app := fiber.New()
app.Use(func(ctx *fiber.Ctx) error {
ctx.Locals("role", "admin")
ctx.Locals("account", auth.Account{Access: "admin1", Secret: "secret", Role: "admin"})
return ctx.Next()
})
@@ -130,7 +130,7 @@ func TestAdminController_DeleteUser(t *testing.T) {
appErr := fiber.New()
appErr.Use(func(ctx *fiber.Ctx) error {
ctx.Locals("role", "user")
ctx.Locals("account", auth.Account{Access: "user1", Secret: "secret", Role: "user"})
return ctx.Next()
})
@@ -199,7 +199,7 @@ func TestAdminController_ListUsers(t *testing.T) {
appErr := fiber.New()
appErr.Use(func(ctx *fiber.Ctx) error {
ctx.Locals("role", "admin")
ctx.Locals("account", auth.Account{Access: "admin1", Secret: "secret", Role: "admin"})
return ctx.Next()
})
@@ -208,7 +208,7 @@ func TestAdminController_ListUsers(t *testing.T) {
appRoleErr := fiber.New()
appRoleErr.Use(func(ctx *fiber.Ctx) error {
ctx.Locals("role", "user")
ctx.Locals("account", auth.Account{Access: "user1", Secret: "secret", Role: "user"})
return ctx.Next()
})
@@ -217,7 +217,7 @@ func TestAdminController_ListUsers(t *testing.T) {
appSucc := fiber.New()
appSucc.Use(func(ctx *fiber.Ctx) error {
ctx.Locals("role", "admin")
ctx.Locals("account", auth.Account{Access: "admin1", Secret: "secret", Role: "admin"})
return ctx.Next()
})
@@ -307,7 +307,7 @@ func TestAdminController_ChangeBucketOwner(t *testing.T) {
app := fiber.New()
app.Use(func(ctx *fiber.Ctx) error {
ctx.Locals("role", "admin")
ctx.Locals("account", auth.Account{Access: "admin1", Secret: "secret", Role: "admin"})
return ctx.Next()
})
@@ -316,7 +316,7 @@ func TestAdminController_ChangeBucketOwner(t *testing.T) {
appRoleErr := fiber.New()
appRoleErr.Use(func(ctx *fiber.Ctx) error {
ctx.Locals("role", "user")
ctx.Locals("account", auth.Account{Access: "user1", Secret: "secret", Role: "user"})
return ctx.Next()
})
@@ -325,7 +325,7 @@ func TestAdminController_ChangeBucketOwner(t *testing.T) {
appIamErr := fiber.New()
appIamErr.Use(func(ctx *fiber.Ctx) error {
ctx.Locals("role", "admin")
ctx.Locals("account", auth.Account{Access: "admin1", Secret: "secret", Role: "admin"})
return ctx.Next()
})
@@ -334,7 +334,7 @@ func TestAdminController_ChangeBucketOwner(t *testing.T) {
appIamNoSuchUser := fiber.New()
appIamNoSuchUser.Use(func(ctx *fiber.Ctx) error {
ctx.Locals("role", "admin")
ctx.Locals("account", auth.Account{Access: "admin1", Secret: "secret", Role: "admin"})
return ctx.Next()
})
@@ -412,7 +412,7 @@ func TestAdminController_ListBuckets(t *testing.T) {
app := fiber.New()
app.Use(func(ctx *fiber.Ctx) error {
ctx.Locals("role", "admin")
ctx.Locals("account", auth.Account{Access: "admin1", Secret: "secret", Role: "admin"})
return ctx.Next()
})
@@ -421,7 +421,7 @@ func TestAdminController_ListBuckets(t *testing.T) {
appRoleErr := fiber.New()
appRoleErr.Use(func(ctx *fiber.Ctx) error {
ctx.Locals("role", "user")
ctx.Locals("account", auth.Account{Access: "user1", Secret: "secret", Role: "user"})
return ctx.Next()
})

View File

@@ -39,7 +39,7 @@ func AclParser(be backend.Backend, logger s3log.AuditLogger) fiber.Handler {
return ctx.Next()
}
if len(pathParts) == 2 && pathParts[1] != "" && ctx.Method() == http.MethodPut && !ctx.Request().URI().QueryArgs().Has("acl") {
if err := auth.IsAdmin(acct.Access, isRoot); err != nil {
if err := auth.IsAdmin(acct, isRoot); err != nil {
return controllers.SendXMLResponse(ctx, nil, err, &controllers.MetaOpts{Logger: logger, Action: "CreateBucket"})
}
return ctx.Next()