check organization membership when authorizing orgsecret access

This commit is contained in:
Brad Rydzewski 2019-09-29 13:51:56 -07:00
parent dce18f9afd
commit 4572cc8ddd
14 changed files with 576 additions and 115 deletions

View file

@ -15,6 +15,8 @@
package main
import (
"time"
"github.com/drone/drone/cmd/drone-server/config"
"github.com/drone/drone/core"
"github.com/drone/drone/livelog"
@ -47,7 +49,6 @@ var serviceSet = wire.NewSet(
commit.New,
cron.New,
livelog.New,
orgs.New,
linker.New,
parser.New,
pubsub.New,
@ -60,6 +61,7 @@ var serviceSet = wire.NewSet(
provideDatadog,
provideHookService,
provideNetrcService,
provideOrgService,
provideSession,
provideStatusService,
provideSyncer,
@ -92,6 +94,12 @@ func provideNetrcService(client *scm.Client, renewer core.Renewer, config config
)
}
// provideOrgService is a Wire provider function that
// returns an organization service wrapped with a simple cache.
func provideOrgService(client *scm.Client, renewer core.Renewer) core.OrganizationService {
return orgs.NewCache(orgs.New(client, renewer), 10, time.Minute*5)
}
// provideRepo is a Wire provider function that returns
// a repo based on the environment configuration
func provideRepositoryService(client *scm.Client, renewer core.Renewer, config config.Config) core.RepositoryService {

View file

@ -16,7 +16,6 @@ import (
"github.com/drone/drone/service/hook/parser"
"github.com/drone/drone/service/license"
"github.com/drone/drone/service/linker"
"github.com/drone/drone/service/org"
"github.com/drone/drone/service/token"
"github.com/drone/drone/service/user"
"github.com/drone/drone/store/cron"
@ -79,6 +78,7 @@ func InitializeApplication(config2 config.Config) (application, error) {
runner := provideRunner(buildManager, secretService, registryService, config2)
hookService := provideHookService(client, renewer, config2)
licenseService := license.NewService(userStore, repositoryStore, buildStore, coreLicense)
organizationService := provideOrgService(client, renewer)
permStore := perm.New(db)
repositoryService := provideRepositoryService(client, renewer, config2)
session, err := provideSession(userStore, config2)
@ -88,8 +88,7 @@ func InitializeApplication(config2 config.Config) (application, error) {
batcher := provideBatchStore(db, config2)
syncer := provideSyncer(repositoryService, repositoryStore, userStore, batcher, config2)
userService := user.New(client, renewer)
server := api.New(buildStore, commitService, cronStore, corePubsub, globalSecretStore, hookService, logStore, coreLicense, licenseService, permStore, repositoryStore, repositoryService, scheduler, secretStore, stageStore, stepStore, statusService, session, logStream, syncer, system, triggerer, userStore, userService, webhookSender)
organizationService := orgs.New(client, renewer)
server := api.New(buildStore, commitService, cronStore, corePubsub, globalSecretStore, hookService, logStore, coreLicense, licenseService, organizationService, permStore, repositoryStore, repositoryService, scheduler, secretStore, stageStore, stepStore, statusService, session, logStream, syncer, system, triggerer, userStore, userService, webhookSender)
admissionService := provideAdmissionPlugin(client, organizationService, userService, config2)
hookParser := parser.New(client)
coreLinker := linker.New(client)

View file

@ -27,5 +27,12 @@ type Organization struct {
// team access in the external source code management system
// (e.g. GitHub).
type OrganizationService interface {
// List returns a list of organization to which the
// user is a member.
List(context.Context, *User) ([]*Organization, error)
// Membership returns true if the user is a member
// of the organization, and true if the user is an
// of the organization.
Membership(context.Context, *User, string) (bool, bool, error)
}

View file

@ -28,6 +28,13 @@ var (
Active: true,
}
mockUserAdmin = &core.User{
ID: 1,
Login: "octocat",
Admin: true,
Active: true,
}
mockUserInactive = &core.User{
ID: 1,
Login: "octocat",

81
handler/api/acl/org.go Normal file
View file

@ -0,0 +1,81 @@
// Copyright 2019 Drone IO, Inc.
//
// 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 acl
import (
"net/http"
"github.com/drone/drone/core"
"github.com/drone/drone/handler/api/errors"
"github.com/drone/drone/handler/api/render"
"github.com/drone/drone/handler/api/request"
"github.com/drone/drone/logger"
"github.com/go-chi/chi"
)
// CheckMembership returns an http.Handler middleware that authorizes only
// authenticated users with the required membership to an organization
// to the requested repository resource.
func CheckMembership(service core.OrganizationService, admin bool) func(http.Handler) http.Handler {
return func(next http.Handler) http.Handler {
return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
namespace := chi.URLParam(r, "namespace")
log := logger.FromRequest(r)
ctx := r.Context()
user, ok := request.UserFrom(ctx)
if !ok {
render.Unauthorized(w, errors.ErrUnauthorized)
log.Debugln("api: authentication required for access")
return
}
log = log.WithField("user.admin", user.Admin)
// if the user is an administrator they are always
// granted access to the organization data.
if user.Admin {
next.ServeHTTP(w, r)
return
}
isMember, isAdmin, err := service.Membership(ctx, user, namespace)
if err != nil {
render.Unauthorized(w, errors.ErrNotFound)
log.Debugln("api: organization membership not found")
return
}
log = log.
WithField("organization.member", isMember).
WithField("organization.admin", isAdmin)
if isMember == false {
render.Unauthorized(w, errors.ErrNotFound)
log.Debugln("api: organization membership is required")
return
}
if isAdmin == false && admin == true {
render.Unauthorized(w, errors.ErrNotFound)
log.Debugln("api: organization administrator is required")
return
}
log.Debugln("api: organization membership verified")
next.ServeHTTP(w, r)
})
}
}

205
handler/api/acl/org_test.go Normal file
View file

@ -0,0 +1,205 @@
// Copyright 2019 Drone.IO Inc. All rights reserved.
// Use of this source code is governed by the Drone Non-Commercial License
// that can be found in the LICENSE file.
package acl
import (
"errors"
"net/http"
"net/http/httptest"
"testing"
"github.com/drone/drone/handler/api/request"
"github.com/drone/drone/mock"
"github.com/go-chi/chi"
"github.com/golang/mock/gomock"
)
func TestCheckMembership_Admin(t *testing.T) {
controller := gomock.NewController(t)
defer controller.Finish()
w := httptest.NewRecorder()
r := httptest.NewRequest("GET", "/api/secrets/github", nil)
r = r.WithContext(
request.WithUser(noContext, mockUserAdmin),
)
router := chi.NewRouter()
router.Route("/api/secrets/{namespace}", func(router chi.Router) {
router.Use(CheckMembership(nil, true))
router.Get("/", func(w http.ResponseWriter, r *http.Request) {
w.WriteHeader(http.StatusTeapot)
})
})
router.ServeHTTP(w, r)
if got, want := w.Code, http.StatusTeapot; got != want {
t.Errorf("Want status code %d, got %d", want, got)
}
}
func TestCheckMembership_NilUser_Unauthorized(t *testing.T) {
controller := gomock.NewController(t)
defer controller.Finish()
w := httptest.NewRecorder()
r := httptest.NewRequest("GET", "/api/secrets/github", nil)
router := chi.NewRouter()
router.Route("/api/secrets/{namespace}", func(router chi.Router) {
router.Use(CheckMembership(nil, true))
router.Get("/", func(w http.ResponseWriter, r *http.Request) {
t.Errorf("Must not invoke next handler in middleware chain")
})
})
router.ServeHTTP(w, r)
if got, want := w.Code, http.StatusUnauthorized; got != want {
t.Errorf("Want status code %d, got %d", want, got)
}
}
func TestCheckMembership_AuthorizeRead(t *testing.T) {
controller := gomock.NewController(t)
defer controller.Finish()
w := httptest.NewRecorder()
r := httptest.NewRequest("GET", "/api/secrets/github", nil)
r = r.WithContext(
request.WithUser(noContext, mockUser),
)
mockOrgService := mock.NewMockOrganizationService(controller)
mockOrgService.EXPECT().Membership(gomock.Any(), gomock.Any(), "github").Return(true, false, nil).Times(1)
router := chi.NewRouter()
router.Route("/api/secrets/{namespace}", func(router chi.Router) {
router.Use(CheckMembership(mockOrgService, false))
router.Get("/", func(w http.ResponseWriter, r *http.Request) {
w.WriteHeader(http.StatusTeapot)
})
})
router.ServeHTTP(w, r)
if got, want := w.Code, http.StatusTeapot; got != want {
t.Errorf("Want status code %d, got %d", want, got)
}
}
func TestCheckMembership_AuthorizeAdmin(t *testing.T) {
controller := gomock.NewController(t)
defer controller.Finish()
w := httptest.NewRecorder()
r := httptest.NewRequest("GET", "/api/secrets/github", nil)
r = r.WithContext(
request.WithUser(noContext, mockUser),
)
mockOrgService := mock.NewMockOrganizationService(controller)
mockOrgService.EXPECT().Membership(gomock.Any(), gomock.Any(), "github").Return(true, true, nil).Times(1)
router := chi.NewRouter()
router.Route("/api/secrets/{namespace}", func(router chi.Router) {
router.Use(CheckMembership(mockOrgService, true))
router.Get("/", func(w http.ResponseWriter, r *http.Request) {
w.WriteHeader(http.StatusTeapot)
})
})
router.ServeHTTP(w, r)
if got, want := w.Code, http.StatusTeapot; got != want {
t.Errorf("Want status code %d, got %d", want, got)
}
}
func TestCheckMembership_Unauthorized_Admin(t *testing.T) {
controller := gomock.NewController(t)
defer controller.Finish()
w := httptest.NewRecorder()
r := httptest.NewRequest("GET", "/api/secrets/github", nil)
r = r.WithContext(
request.WithUser(noContext, mockUser),
)
mockOrgService := mock.NewMockOrganizationService(controller)
mockOrgService.EXPECT().Membership(gomock.Any(), gomock.Any(), "github").Return(true, false, nil).Times(1)
router := chi.NewRouter()
router.Route("/api/secrets/{namespace}", func(router chi.Router) {
router.Use(CheckMembership(mockOrgService, true))
router.Get("/", func(w http.ResponseWriter, r *http.Request) {
t.Errorf("Must not invoke next handler in middleware chain")
})
})
router.ServeHTTP(w, r)
if got, want := w.Code, http.StatusUnauthorized; got != want {
t.Errorf("Want status code %d, got %d", want, got)
}
}
func TestCheckMembership_Unauthorized_Read(t *testing.T) {
controller := gomock.NewController(t)
defer controller.Finish()
w := httptest.NewRecorder()
r := httptest.NewRequest("GET", "/api/secrets/github", nil)
r = r.WithContext(
request.WithUser(noContext, mockUser),
)
mockOrgService := mock.NewMockOrganizationService(controller)
mockOrgService.EXPECT().Membership(gomock.Any(), gomock.Any(), "github").Return(false, false, nil).Times(1)
router := chi.NewRouter()
router.Route("/api/secrets/{namespace}", func(router chi.Router) {
router.Use(CheckMembership(mockOrgService, false))
router.Get("/", func(w http.ResponseWriter, r *http.Request) {
t.Errorf("Must not invoke next handler in middleware chain")
})
})
router.ServeHTTP(w, r)
if got, want := w.Code, http.StatusUnauthorized; got != want {
t.Errorf("Want status code %d, got %d", want, got)
}
}
func TestCheckMembership_Unauthorized_Error(t *testing.T) {
controller := gomock.NewController(t)
defer controller.Finish()
w := httptest.NewRecorder()
r := httptest.NewRequest("GET", "/api/secrets/github", nil)
r = r.WithContext(
request.WithUser(noContext, mockUser),
)
mockOrgService := mock.NewMockOrganizationService(controller)
mockOrgService.EXPECT().Membership(gomock.Any(), gomock.Any(), "github").Return(true, true, errors.New("")).Times(1)
router := chi.NewRouter()
router.Route("/api/secrets/{namespace}", func(router chi.Router) {
router.Use(CheckMembership(mockOrgService, false))
router.Get("/", func(w http.ResponseWriter, r *http.Request) {
t.Errorf("Must not invoke next handler in middleware chain")
})
})
router.ServeHTTP(w, r)
if got, want := w.Code, http.StatusUnauthorized; got != want {
t.Errorf("Want status code %d, got %d", want, got)
}
}

View file

@ -65,6 +65,7 @@ func New(
logs core.LogStore,
license *core.License,
licenses core.LicenseService,
orgs core.OrganizationService,
perms core.PermStore,
repos core.RepositoryStore,
repoz core.RepositoryService,
@ -92,6 +93,7 @@ func New(
Logs: logs,
License: license,
Licenses: licenses,
Orgs: orgs,
Perms: perms,
Repos: repos,
Repoz: repoz,
@ -122,6 +124,7 @@ type Server struct {
Logs core.LogStore
License *core.License
Licenses core.LicenseService
Orgs core.OrganizationService
Perms core.PermStore
Repos core.RepositoryStore
Repoz core.RepositoryService
@ -312,14 +315,13 @@ func (s Server) Handler() http.Handler {
})
r.Route("/secrets", func(r chi.Router) {
r.Use(acl.AuthorizeAdmin)
r.Get("/", globalsecrets.HandleAll(s.Globals))
r.Get("/{namespace}", globalsecrets.HandleList(s.Globals))
r.Post("/{namespace}", globalsecrets.HandleCreate(s.Globals))
r.Get("/{namespace}/{name}", globalsecrets.HandleFind(s.Globals))
r.Post("/{namespace}/{name}", globalsecrets.HandleUpdate(s.Globals))
r.Patch("/{namespace}/{name}", globalsecrets.HandleUpdate(s.Globals))
r.Delete("/{namespace}/{name}", globalsecrets.HandleDelete(s.Globals))
r.With(acl.AuthorizeAdmin).Get("/", globalsecrets.HandleAll(s.Globals))
r.With(acl.CheckMembership(s.Orgs, false)).Get("/{namespace}", globalsecrets.HandleList(s.Globals))
r.With(acl.CheckMembership(s.Orgs, true)).Post("/{namespace}", globalsecrets.HandleCreate(s.Globals))
r.With(acl.CheckMembership(s.Orgs, false)).Get("/{namespace}/{name}", globalsecrets.HandleFind(s.Globals))
r.With(acl.CheckMembership(s.Orgs, true)).Post("/{namespace}/{name}", globalsecrets.HandleUpdate(s.Globals))
r.With(acl.CheckMembership(s.Orgs, true)).Patch("/{namespace}/{name}", globalsecrets.HandleUpdate(s.Globals))
r.With(acl.CheckMembership(s.Orgs, true)).Delete("/{namespace}/{name}", globalsecrets.HandleDelete(s.Globals))
})
r.Route("/system", func(r chi.Router) {

View file

@ -56,14 +56,6 @@ func HandleCreate(users core.UserStore, service core.UserService, sender core.We
user.Hash = uniuri.NewLen(32)
}
err = user.Validate()
if err != nil {
render.ErrorCode(w, err, 400)
logger.FromRequest(r).WithError(err).
Errorln("api: invlid username")
return
}
// if the user is not a machine account, we lookup
// the user in the remote system. We can then augment
// the user input with the remote system data.
@ -79,6 +71,14 @@ func HandleCreate(users core.UserStore, service core.UserService, sender core.We
}
}
err = user.Validate()
if err != nil {
render.ErrorCode(w, err, 400)
logger.FromRequest(r).WithError(err).
Errorln("api: invlid username")
return
}
err = users.Create(r.Context(), user)
if err == core.ErrUserLimit {
render.ErrorCode(w, err, 402)

View file

@ -2078,6 +2078,22 @@ func (mr *MockOrganizationServiceMockRecorder) List(arg0, arg1 interface{}) *gom
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "List", reflect.TypeOf((*MockOrganizationService)(nil).List), arg0, arg1)
}
// Membership mocks base method
func (m *MockOrganizationService) Membership(arg0 context.Context, arg1 *core.User, arg2 string) (bool, bool, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Membership", arg0, arg1, arg2)
ret0, _ := ret[0].(bool)
ret1, _ := ret[1].(bool)
ret2, _ := ret[2].(error)
return ret0, ret1, ret2
}
// Membership indicates an expected call of Membership
func (mr *MockOrganizationServiceMockRecorder) Membership(arg0, arg1, arg2 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Membership", reflect.TypeOf((*MockOrganizationService)(nil).Membership), arg0, arg1, arg2)
}
// MockSecretService is a mock of SecretService interface
type MockSecretService struct {
ctrl *gomock.Controller

View file

@ -1,8 +1,16 @@
// Copyright 2019 Drone.IO Inc. All rights reserved.
// Use of this source code is governed by the Drone Non-Commercial License
// that can be found in the LICENSE file.
// +build !oss
// Copyright 2019 Drone IO, Inc.
//
// 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 server

View file

@ -1,90 +0,0 @@
// Copyright 2019 Drone IO, Inc.
//
// 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.
// +build oss
package server
import (
"context"
"net/http"
"golang.org/x/sync/errgroup"
)
// A Server defines parameters for running an HTTP server.
type Server struct {
Acme bool
Addr string
Cert string
Key string
Host string
Handler http.Handler
}
// ListenAndServe initializes a server to respond to HTTP network requests.
func (s Server) ListenAndServe(ctx context.Context) error {
if s.Key != "" {
return s.listenAndServeTLS(ctx)
}
return s.listenAndServe(ctx)
}
func (s Server) listenAndServe(ctx context.Context) error {
var g errgroup.Group
s1 := &http.Server{
Addr: s.Addr,
Handler: s.Handler,
}
g.Go(func() error {
select {
case <-ctx.Done():
return s1.Shutdown(ctx)
}
})
g.Go(func() error {
return s1.ListenAndServe()
})
return g.Wait()
}
func (s Server) listenAndServeTLS(ctx context.Context) error {
var g errgroup.Group
s1 := &http.Server{
Addr: ":http",
Handler: s.Handler,
}
s2 := &http.Server{
Addr: ":https",
Handler: s.Handler,
}
g.Go(func() error {
return s1.ListenAndServe()
})
g.Go(func() error {
return s2.ListenAndServeTLS(
s.Cert,
s.Key,
)
})
g.Go(func() error {
select {
case <-ctx.Done():
s1.Shutdown(ctx)
s2.Shutdown(ctx)
return nil
}
})
return g.Wait()
}

99
service/org/cache.go Normal file
View file

@ -0,0 +1,99 @@
// Copyright 2019 Drone IO, Inc.
//
// 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 orgs
import (
"context"
"fmt"
"sync"
"time"
"github.com/drone/drone/core"
lru "github.com/hashicorp/golang-lru"
)
// content key pattern used in the cache, comprised of the
// organization name and username.
const contentKey = "%s/%s"
// NewCache wraps the service with a simple cache to store
// organization membership.
func NewCache(base core.OrganizationService, size int, ttl time.Duration) core.OrganizationService {
// simple cache prevents the same yaml file from being
// requested multiple times in a short period.
cache, _ := lru.New(25)
return &cacher{
cache: cache,
base: base,
size: size,
ttl: ttl,
}
}
type cacher struct {
mu sync.Mutex
base core.OrganizationService
size int
ttl time.Duration
cache *lru.Cache
}
type item struct {
expiry time.Time
member bool
admin bool
}
func (c *cacher) List(ctx context.Context, user *core.User) ([]*core.Organization, error) {
return c.base.List(ctx, user)
}
func (c *cacher) Membership(ctx context.Context, user *core.User, name string) (bool, bool, error) {
key := fmt.Sprintf(contentKey, user.Login, name)
now := time.Now()
// get the membership details from the cache.
cached, ok := c.cache.Get(key)
if ok {
item := cached.(*item)
// if the item is expired it can be ejected
// from the cache, else if not expired we return
// the cached results.
if now.After(item.expiry) {
c.cache.Remove(cached)
} else {
return item.member, item.admin, nil
}
}
// get up-to-date membership details due to a cache
// miss or expired cache item.
member, admin, err := c.base.Membership(ctx, user, name)
if err != nil {
return false, false, err
}
c.cache.Add(key, &item{
expiry: now.Add(c.ttl),
member: member,
admin: admin,
})
return member, admin, nil
}

90
service/org/cache_test.go Normal file
View file

@ -0,0 +1,90 @@
// Copyright 2019 Drone.IO Inc. All rights reserved.
// Use of this source code is governed by the Drone Non-Commercial License
// that can be found in the LICENSE file.
package orgs
import (
"testing"
"time"
"github.com/drone/drone/core"
"github.com/drone/drone/mock"
"github.com/golang/mock/gomock"
)
func TestCache(t *testing.T) {
controller := gomock.NewController(t)
defer controller.Finish()
mockUser := &core.User{
Login: "octocat",
}
mockOrgService := mock.NewMockOrganizationService(controller)
mockOrgService.EXPECT().Membership(gomock.Any(), gomock.Any(), "github").Return(true, true, nil).Times(1)
service := NewCache(mockOrgService, 10, time.Minute).(*cacher)
admin, member, err := service.Membership(noContext, mockUser, "github")
if err != nil {
t.Error(err)
}
if got, want := service.cache.Len(), 1; got != want {
t.Errorf("Expect cache size %d, got %d", want, got)
}
if admin == false {
t.Errorf("Expect admin true, got false")
}
if member == false {
t.Errorf("Expect member true, got false")
}
admin, member, err = service.Membership(noContext, mockUser, "github")
if err != nil {
t.Error(err)
}
if got, want := service.cache.Len(), 1; got != want {
t.Errorf("Expect cache size still %d, got %d", want, got)
}
if admin == false {
t.Errorf("Expect cached admin true, got false")
}
if member == false {
t.Errorf("Expect cached member true, got false")
}
}
func TestCache_Expired(t *testing.T) {
controller := gomock.NewController(t)
defer controller.Finish()
mockUser := &core.User{
Login: "octocat",
}
mockOrgService := mock.NewMockOrganizationService(controller)
mockOrgService.EXPECT().Membership(gomock.Any(), gomock.Any(), "github").Return(true, true, nil).Times(1)
service := NewCache(mockOrgService, 10, time.Minute).(*cacher)
service.cache.Add("octocat/github", &item{
expiry: time.Now().Add(time.Hour * -1),
member: true,
admin: true,
})
admin, member, err := service.Membership(noContext, mockUser, "github")
if err != nil {
t.Error(err)
}
if got, want := service.cache.Len(), 1; got != want {
t.Errorf("Expect cache size still %d, got %d", want, got)
}
if admin == false {
t.Errorf("Expect cached admin true, got false")
}
if member == false {
t.Errorf("Expect cached member true, got false")
}
}

View file

@ -61,3 +61,32 @@ func (s *service) List(ctx context.Context, user *core.User) ([]*core.Organizati
}
return orgs, nil
}
func (s *service) Membership(ctx context.Context, user *core.User, name string) (bool, bool, error) {
err := s.renewer.Renew(ctx, user, false)
if err != nil {
return false, false, err
}
token := &scm.Token{
Token: user.Token,
Refresh: user.Refresh,
}
if user.Expiry != 0 {
token.Expires = time.Unix(user.Expiry, 0)
}
ctx = context.WithValue(ctx, scm.TokenKey{}, token)
out, _, err := s.client.Organizations.FindMembership(ctx, name, user.Login)
if err != nil {
return false, false, err
}
switch {
case out.Active == false:
return false, false, nil
case out.Role == scm.RoleUndefined:
return false, false, nil
case out.Role == scm.RoleAdmin:
return true, true, nil
default:
return true, false, nil
}
}