fix: serialize updateEntitlements() (#14974)

fixes #14961

Adding the license and updating entitlements is flaky, especially at the start of our `coderdent` testing because, while the actual modifications to the `entitlements.Set` were threadsafe, we could have multiple goroutines reading from the database and writing to the set, so we could end up writing stale data.

This enforces serialization on updates, so that if you modify the database and kick off an update, you know the state of the `Set` is at least as fresh as your database update.
This commit is contained in:
Spike Curtis
2024-10-05 06:58:43 +04:00
committed by GitHub
parent ea3b13c78e
commit 288df75686
7 changed files with 313 additions and 217 deletions

View File

@ -1,21 +1,31 @@
package entitlements
import (
"context"
"encoding/json"
"net/http"
"sync"
"time"
"golang.org/x/exp/slices"
"golang.org/x/xerrors"
"github.com/coder/coder/v2/codersdk"
)
type Set struct {
entitlementsMu sync.RWMutex
entitlements codersdk.Entitlements
// right2Update works like a semaphore. Reading from the chan gives the right to update the set,
// and you send on the chan when you are done. We only allow one simultaneous update, so this
// serve to serialize them. You MUST NOT attempt to read from this channel while holding the
// entitlementsMu lock. It is permissible to acquire the entitlementsMu lock while holding the
// right2Update token.
right2Update chan struct{}
}
func New() *Set {
return &Set{
s := &Set{
// Some defaults for an unlicensed instance.
// These will be updated when coderd is initialized.
entitlements: codersdk.Entitlements{
@ -27,7 +37,44 @@ func New() *Set {
RequireTelemetry: false,
RefreshedAt: time.Time{},
},
right2Update: make(chan struct{}, 1),
}
s.right2Update <- struct{}{} // one token, serialized updates
return s
}
// ErrLicenseRequiresTelemetry is an error returned by a fetch passed to Update to indicate that the
// fetched license cannot be used because it requires telemetry.
var ErrLicenseRequiresTelemetry = xerrors.New("License requires telemetry but telemetry is disabled")
func (l *Set) Update(ctx context.Context, fetch func(context.Context) (codersdk.Entitlements, error)) error {
select {
case <-ctx.Done():
return ctx.Err()
case <-l.right2Update:
defer func() {
l.right2Update <- struct{}{}
}()
}
ents, err := fetch(ctx)
if xerrors.Is(err, ErrLicenseRequiresTelemetry) {
// We can't fail because then the user couldn't remove the offending
// license w/o a restart.
//
// We don't simply append to entitlement.Errors since we don't want any
// enterprise features enabled.
l.Modify(func(entitlements *codersdk.Entitlements) {
entitlements.Errors = []string{err.Error()}
})
return nil
}
if err != nil {
return err
}
l.entitlementsMu.Lock()
defer l.entitlementsMu.Unlock()
l.entitlements = ents
return nil
}
// AllowRefresh returns whether the entitlements are allowed to be refreshed.
@ -74,14 +121,7 @@ func (l *Set) AsJSON() json.RawMessage {
return b
}
func (l *Set) Replace(entitlements codersdk.Entitlements) {
l.entitlementsMu.Lock()
defer l.entitlementsMu.Unlock()
l.entitlements = entitlements
}
func (l *Set) Update(do func(entitlements *codersdk.Entitlements)) {
func (l *Set) Modify(do func(entitlements *codersdk.Entitlements)) {
l.entitlementsMu.Lock()
defer l.entitlementsMu.Unlock()
@ -107,3 +147,9 @@ func (l *Set) WriteEntitlementWarningHeaders(header http.Header) {
header.Add(codersdk.EntitlementsWarningHeader, warning)
}
}
func (l *Set) Errors() []string {
l.entitlementsMu.RLock()
defer l.entitlementsMu.RUnlock()
return slices.Clone(l.entitlements.Errors)
}

View File

@ -1,6 +1,7 @@
package entitlements_test
import (
"context"
"testing"
"time"
@ -8,15 +9,16 @@ import (
"github.com/coder/coder/v2/coderd/entitlements"
"github.com/coder/coder/v2/codersdk"
"github.com/coder/coder/v2/testutil"
)
func TestUpdate(t *testing.T) {
func TestModify(t *testing.T) {
t.Parallel()
set := entitlements.New()
require.False(t, set.Enabled(codersdk.FeatureMultipleOrganizations))
set.Update(func(entitlements *codersdk.Entitlements) {
set.Modify(func(entitlements *codersdk.Entitlements) {
entitlements.Features[codersdk.FeatureMultipleOrganizations] = codersdk.Feature{
Enabled: true,
Entitlement: codersdk.EntitlementEntitled,
@ -30,7 +32,7 @@ func TestAllowRefresh(t *testing.T) {
now := time.Now()
set := entitlements.New()
set.Update(func(entitlements *codersdk.Entitlements) {
set.Modify(func(entitlements *codersdk.Entitlements) {
entitlements.RefreshedAt = now
})
@ -38,7 +40,7 @@ func TestAllowRefresh(t *testing.T) {
require.False(t, ok)
require.InDelta(t, time.Minute.Seconds(), wait.Seconds(), 5)
set.Update(func(entitlements *codersdk.Entitlements) {
set.Modify(func(entitlements *codersdk.Entitlements) {
entitlements.RefreshedAt = now.Add(time.Minute * -2)
})
@ -47,17 +49,76 @@ func TestAllowRefresh(t *testing.T) {
require.Equal(t, time.Duration(0), wait)
}
func TestReplace(t *testing.T) {
func TestUpdate(t *testing.T) {
t.Parallel()
ctx := testutil.Context(t, testutil.WaitShort)
set := entitlements.New()
require.False(t, set.Enabled(codersdk.FeatureMultipleOrganizations))
set.Replace(codersdk.Entitlements{
Features: map[codersdk.FeatureName]codersdk.Feature{
codersdk.FeatureMultipleOrganizations: {
Enabled: true,
},
},
})
fetchStarted := make(chan struct{})
firstDone := make(chan struct{})
errCh := make(chan error, 2)
go func() {
err := set.Update(ctx, func(_ context.Context) (codersdk.Entitlements, error) {
close(fetchStarted)
select {
case <-firstDone:
// OK!
case <-ctx.Done():
t.Error("timeout")
return codersdk.Entitlements{}, ctx.Err()
}
return codersdk.Entitlements{
Features: map[codersdk.FeatureName]codersdk.Feature{
codersdk.FeatureMultipleOrganizations: {
Enabled: true,
},
},
}, nil
})
errCh <- err
}()
testutil.RequireRecvCtx(ctx, t, fetchStarted)
require.False(t, set.Enabled(codersdk.FeatureMultipleOrganizations))
// start a second update while the first one is in progress
go func() {
err := set.Update(ctx, func(_ context.Context) (codersdk.Entitlements, error) {
return codersdk.Entitlements{
Features: map[codersdk.FeatureName]codersdk.Feature{
codersdk.FeatureMultipleOrganizations: {
Enabled: true,
},
codersdk.FeatureAppearance: {
Enabled: true,
},
},
}, nil
})
errCh <- err
}()
close(firstDone)
err := testutil.RequireRecvCtx(ctx, t, errCh)
require.NoError(t, err)
err = testutil.RequireRecvCtx(ctx, t, errCh)
require.NoError(t, err)
require.True(t, set.Enabled(codersdk.FeatureMultipleOrganizations))
require.True(t, set.Enabled(codersdk.FeatureAppearance))
}
func TestUpdate_LicenseRequiresTelemetry(t *testing.T) {
t.Parallel()
ctx := testutil.Context(t, testutil.WaitShort)
set := entitlements.New()
set.Modify(func(entitlements *codersdk.Entitlements) {
entitlements.Errors = []string{"some error"}
entitlements.Features[codersdk.FeatureAppearance] = codersdk.Feature{
Enabled: true,
}
})
err := set.Update(ctx, func(_ context.Context) (codersdk.Entitlements, error) {
return codersdk.Entitlements{}, entitlements.ErrLicenseRequiresTelemetry
})
require.NoError(t, err)
require.True(t, set.Enabled(codersdk.FeatureAppearance))
require.Equal(t, []string{entitlements.ErrLicenseRequiresTelemetry.Error()}, set.Errors())
}

View File

@ -613,226 +613,215 @@ func (api *API) Close() error {
}
func (api *API) updateEntitlements(ctx context.Context) error {
replicas := api.replicaManager.AllPrimary()
agedReplicas := make([]database.Replica, 0, len(replicas))
for _, replica := range replicas {
// If a replica is less than the update interval old, we don't
// want to display a warning. In the open-source version of Coder,
// Kubernetes Pods will start up before shutting down the other,
// and we don't want to display a warning in that case.
//
// Only display warnings for long-lived replicas!
if dbtime.Now().Sub(replica.StartedAt) < api.ReplicaErrorGracePeriod {
continue
}
agedReplicas = append(agedReplicas, replica)
}
reloadedEntitlements, err := license.Entitlements(
ctx, api.Database,
len(agedReplicas), len(api.ExternalAuthConfigs), api.LicenseKeys, map[codersdk.FeatureName]bool{
codersdk.FeatureAuditLog: api.AuditLogging,
codersdk.FeatureBrowserOnly: api.BrowserOnly,
codersdk.FeatureSCIM: len(api.SCIMAPIKey) != 0,
codersdk.FeatureMultipleExternalAuth: len(api.ExternalAuthConfigs) > 1,
codersdk.FeatureTemplateRBAC: api.RBAC,
codersdk.FeatureExternalTokenEncryption: len(api.ExternalTokenEncryption) > 0,
codersdk.FeatureExternalProvisionerDaemons: true,
codersdk.FeatureAdvancedTemplateScheduling: true,
codersdk.FeatureWorkspaceProxy: true,
codersdk.FeatureUserRoleManagement: true,
codersdk.FeatureAccessControl: true,
codersdk.FeatureControlSharedPorts: true,
})
if err != nil {
return err
}
if reloadedEntitlements.RequireTelemetry && !api.DeploymentValues.Telemetry.Enable.Value() {
// We can't fail because then the user couldn't remove the offending
// license w/o a restart.
//
// We don't simply append to entitlement.Errors since we don't want any
// enterprise features enabled.
api.Entitlements.Update(func(entitlements *codersdk.Entitlements) {
entitlements.Errors = []string{
"License requires telemetry but telemetry is disabled",
return api.Entitlements.Update(ctx, func(ctx context.Context) (codersdk.Entitlements, error) {
replicas := api.replicaManager.AllPrimary()
agedReplicas := make([]database.Replica, 0, len(replicas))
for _, replica := range replicas {
// If a replica is less than the update interval old, we don't
// want to display a warning. In the open-source version of Coder,
// Kubernetes Pods will start up before shutting down the other,
// and we don't want to display a warning in that case.
//
// Only display warnings for long-lived replicas!
if dbtime.Now().Sub(replica.StartedAt) < api.ReplicaErrorGracePeriod {
continue
}
})
api.Logger.Error(ctx, "license requires telemetry enabled")
return nil
}
featureChanged := func(featureName codersdk.FeatureName) (initial, changed, enabled bool) {
return api.Entitlements.FeatureChanged(featureName, reloadedEntitlements.Features[featureName])
}
shouldUpdate := func(initial, changed, enabled bool) bool {
// Avoid an initial tick on startup unless the feature is enabled.
return changed || (initial && enabled)
}
if initial, changed, enabled := featureChanged(codersdk.FeatureAuditLog); shouldUpdate(initial, changed, enabled) {
auditor := agplaudit.NewNop()
if enabled {
auditor = api.AGPL.Options.Auditor
agedReplicas = append(agedReplicas, replica)
}
api.AGPL.Auditor.Store(&auditor)
}
if initial, changed, enabled := featureChanged(codersdk.FeatureBrowserOnly); shouldUpdate(initial, changed, enabled) {
var handler func(rw http.ResponseWriter) bool
if enabled {
handler = api.shouldBlockNonBrowserConnections
reloadedEntitlements, err := license.Entitlements(
ctx, api.Database,
len(agedReplicas), len(api.ExternalAuthConfigs), api.LicenseKeys, map[codersdk.FeatureName]bool{
codersdk.FeatureAuditLog: api.AuditLogging,
codersdk.FeatureBrowserOnly: api.BrowserOnly,
codersdk.FeatureSCIM: len(api.SCIMAPIKey) != 0,
codersdk.FeatureMultipleExternalAuth: len(api.ExternalAuthConfigs) > 1,
codersdk.FeatureTemplateRBAC: api.RBAC,
codersdk.FeatureExternalTokenEncryption: len(api.ExternalTokenEncryption) > 0,
codersdk.FeatureExternalProvisionerDaemons: true,
codersdk.FeatureAdvancedTemplateScheduling: true,
codersdk.FeatureWorkspaceProxy: true,
codersdk.FeatureUserRoleManagement: true,
codersdk.FeatureAccessControl: true,
codersdk.FeatureControlSharedPorts: true,
})
if err != nil {
return codersdk.Entitlements{}, err
}
api.AGPL.WorkspaceClientCoordinateOverride.Store(&handler)
}
if initial, changed, enabled := featureChanged(codersdk.FeatureTemplateRBAC); shouldUpdate(initial, changed, enabled) {
if enabled {
committer := committer{
Log: api.Logger.Named("quota_committer"),
Database: api.Database,
if reloadedEntitlements.RequireTelemetry && !api.DeploymentValues.Telemetry.Enable.Value() {
api.Logger.Error(ctx, "license requires telemetry enabled")
return codersdk.Entitlements{}, entitlements.ErrLicenseRequiresTelemetry
}
featureChanged := func(featureName codersdk.FeatureName) (initial, changed, enabled bool) {
return api.Entitlements.FeatureChanged(featureName, reloadedEntitlements.Features[featureName])
}
shouldUpdate := func(initial, changed, enabled bool) bool {
// Avoid an initial tick on startup unless the feature is enabled.
return changed || (initial && enabled)
}
if initial, changed, enabled := featureChanged(codersdk.FeatureAuditLog); shouldUpdate(initial, changed, enabled) {
auditor := agplaudit.NewNop()
if enabled {
auditor = api.AGPL.Options.Auditor
}
qcPtr := proto.QuotaCommitter(&committer)
api.AGPL.QuotaCommitter.Store(&qcPtr)
} else {
api.AGPL.QuotaCommitter.Store(nil)
api.AGPL.Auditor.Store(&auditor)
}
}
if initial, changed, enabled := featureChanged(codersdk.FeatureAdvancedTemplateScheduling); shouldUpdate(initial, changed, enabled) {
if enabled {
templateStore := schedule.NewEnterpriseTemplateScheduleStore(api.AGPL.UserQuietHoursScheduleStore, api.NotificationsEnqueuer, api.Logger.Named("template.schedule-store"))
templateStoreInterface := agplschedule.TemplateScheduleStore(templateStore)
api.AGPL.TemplateScheduleStore.Store(&templateStoreInterface)
if api.DefaultQuietHoursSchedule == "" {
api.Logger.Warn(ctx, "template autostop requirement will default to UTC midnight as the default user quiet hours schedule. Set a custom default quiet hours schedule using CODER_QUIET_HOURS_DEFAULT_SCHEDULE to avoid this warning")
api.DefaultQuietHoursSchedule = "CRON_TZ=UTC 0 0 * * *"
if initial, changed, enabled := featureChanged(codersdk.FeatureBrowserOnly); shouldUpdate(initial, changed, enabled) {
var handler func(rw http.ResponseWriter) bool
if enabled {
handler = api.shouldBlockNonBrowserConnections
}
quietHoursStore, err := schedule.NewEnterpriseUserQuietHoursScheduleStore(api.DefaultQuietHoursSchedule, api.DeploymentValues.UserQuietHoursSchedule.AllowUserCustom.Value())
if err != nil {
api.Logger.Error(ctx, "unable to set up enterprise user quiet hours schedule store, template autostop requirements will not be applied to workspace builds", slog.Error(err))
api.AGPL.WorkspaceClientCoordinateOverride.Store(&handler)
}
if initial, changed, enabled := featureChanged(codersdk.FeatureTemplateRBAC); shouldUpdate(initial, changed, enabled) {
if enabled {
committer := committer{
Log: api.Logger.Named("quota_committer"),
Database: api.Database,
}
qcPtr := proto.QuotaCommitter(&committer)
api.AGPL.QuotaCommitter.Store(&qcPtr)
} else {
api.AGPL.QuotaCommitter.Store(nil)
}
}
if initial, changed, enabled := featureChanged(codersdk.FeatureAdvancedTemplateScheduling); shouldUpdate(initial, changed, enabled) {
if enabled {
templateStore := schedule.NewEnterpriseTemplateScheduleStore(api.AGPL.UserQuietHoursScheduleStore, api.NotificationsEnqueuer, api.Logger.Named("template.schedule-store"))
templateStoreInterface := agplschedule.TemplateScheduleStore(templateStore)
api.AGPL.TemplateScheduleStore.Store(&templateStoreInterface)
if api.DefaultQuietHoursSchedule == "" {
api.Logger.Warn(ctx, "template autostop requirement will default to UTC midnight as the default user quiet hours schedule. Set a custom default quiet hours schedule using CODER_QUIET_HOURS_DEFAULT_SCHEDULE to avoid this warning")
api.DefaultQuietHoursSchedule = "CRON_TZ=UTC 0 0 * * *"
}
quietHoursStore, err := schedule.NewEnterpriseUserQuietHoursScheduleStore(api.DefaultQuietHoursSchedule, api.DeploymentValues.UserQuietHoursSchedule.AllowUserCustom.Value())
if err != nil {
api.Logger.Error(ctx, "unable to set up enterprise user quiet hours schedule store, template autostop requirements will not be applied to workspace builds", slog.Error(err))
} else {
api.AGPL.UserQuietHoursScheduleStore.Store(&quietHoursStore)
}
} else {
templateStore := agplschedule.NewAGPLTemplateScheduleStore()
api.AGPL.TemplateScheduleStore.Store(&templateStore)
quietHoursStore := agplschedule.NewAGPLUserQuietHoursScheduleStore()
api.AGPL.UserQuietHoursScheduleStore.Store(&quietHoursStore)
}
} else {
templateStore := agplschedule.NewAGPLTemplateScheduleStore()
api.AGPL.TemplateScheduleStore.Store(&templateStore)
quietHoursStore := agplschedule.NewAGPLUserQuietHoursScheduleStore()
api.AGPL.UserQuietHoursScheduleStore.Store(&quietHoursStore)
}
}
if initial, changed, enabled := featureChanged(codersdk.FeatureHighAvailability); shouldUpdate(initial, changed, enabled) {
var coordinator agpltailnet.Coordinator
// If HA is enabled, but the database is in-memory, we can't actually
// run HA and the PG coordinator. So throw a log line, and continue to use
// the in memory AGPL coordinator.
if enabled && api.DeploymentValues.InMemoryDatabase.Value() {
api.Logger.Warn(ctx, "high availability is enabled, but cannot be configured due to the database being set to in-memory")
}
if enabled && !api.DeploymentValues.InMemoryDatabase.Value() {
haCoordinator, err := tailnet.NewPGCoord(api.ctx, api.Logger, api.Pubsub, api.Database)
if err != nil {
api.Logger.Error(ctx, "unable to set up high availability coordinator", slog.Error(err))
// If we try to setup the HA coordinator and it fails, nothing
// is actually changing.
} else {
coordinator = haCoordinator
if initial, changed, enabled := featureChanged(codersdk.FeatureHighAvailability); shouldUpdate(initial, changed, enabled) {
var coordinator agpltailnet.Coordinator
// If HA is enabled, but the database is in-memory, we can't actually
// run HA and the PG coordinator. So throw a log line, and continue to use
// the in memory AGPL coordinator.
if enabled && api.DeploymentValues.InMemoryDatabase.Value() {
api.Logger.Warn(ctx, "high availability is enabled, but cannot be configured due to the database being set to in-memory")
}
api.replicaManager.SetCallback(func() {
// Only update DERP mesh if the built-in server is enabled.
if api.Options.DeploymentValues.DERP.Server.Enable {
addresses := make([]string, 0)
for _, replica := range api.replicaManager.Regional() {
// Don't add replicas with an empty relay address.
if replica.RelayAddress == "" {
continue
}
addresses = append(addresses, replica.RelayAddress)
}
api.derpMesh.SetAddresses(addresses, false)
if enabled && !api.DeploymentValues.InMemoryDatabase.Value() {
haCoordinator, err := tailnet.NewPGCoord(api.ctx, api.Logger, api.Pubsub, api.Database)
if err != nil {
api.Logger.Error(ctx, "unable to set up high availability coordinator", slog.Error(err))
// If we try to setup the HA coordinator and it fails, nothing
// is actually changing.
} else {
coordinator = haCoordinator
}
_ = api.updateEntitlements(ctx)
})
} else {
coordinator = agpltailnet.NewCoordinator(api.Logger)
if api.Options.DeploymentValues.DERP.Server.Enable {
api.derpMesh.SetAddresses([]string{}, false)
}
api.replicaManager.SetCallback(func() {
// If the amount of replicas change, so should our entitlements.
// This is to display a warning in the UI if the user is unlicensed.
_ = api.updateEntitlements(ctx)
})
}
// Recheck changed in case the HA coordinator failed to set up.
if coordinator != nil {
oldCoordinator := *api.AGPL.TailnetCoordinator.Swap(&coordinator)
err := oldCoordinator.Close()
if err != nil {
api.Logger.Error(ctx, "close old tailnet coordinator", slog.Error(err))
api.replicaManager.SetCallback(func() {
// Only update DERP mesh if the built-in server is enabled.
if api.Options.DeploymentValues.DERP.Server.Enable {
addresses := make([]string, 0)
for _, replica := range api.replicaManager.Regional() {
// Don't add replicas with an empty relay address.
if replica.RelayAddress == "" {
continue
}
addresses = append(addresses, replica.RelayAddress)
}
api.derpMesh.SetAddresses(addresses, false)
}
_ = api.updateEntitlements(ctx)
})
} else {
coordinator = agpltailnet.NewCoordinator(api.Logger)
if api.Options.DeploymentValues.DERP.Server.Enable {
api.derpMesh.SetAddresses([]string{}, false)
}
api.replicaManager.SetCallback(func() {
// If the amount of replicas change, so should our entitlements.
// This is to display a warning in the UI if the user is unlicensed.
_ = api.updateEntitlements(ctx)
})
}
// Recheck changed in case the HA coordinator failed to set up.
if coordinator != nil {
oldCoordinator := *api.AGPL.TailnetCoordinator.Swap(&coordinator)
err := oldCoordinator.Close()
if err != nil {
api.Logger.Error(ctx, "close old tailnet coordinator", slog.Error(err))
}
}
}
}
if initial, changed, enabled := featureChanged(codersdk.FeatureWorkspaceProxy); shouldUpdate(initial, changed, enabled) {
if enabled {
fn := derpMapper(api.Logger, api.ProxyHealth)
api.AGPL.DERPMapper.Store(&fn)
} else {
api.AGPL.DERPMapper.Store(nil)
if initial, changed, enabled := featureChanged(codersdk.FeatureWorkspaceProxy); shouldUpdate(initial, changed, enabled) {
if enabled {
fn := derpMapper(api.Logger, api.ProxyHealth)
api.AGPL.DERPMapper.Store(&fn)
} else {
api.AGPL.DERPMapper.Store(nil)
}
}
}
if initial, changed, enabled := featureChanged(codersdk.FeatureAccessControl); shouldUpdate(initial, changed, enabled) {
var acs agpldbauthz.AccessControlStore = agpldbauthz.AGPLTemplateAccessControlStore{}
if enabled {
acs = dbauthz.EnterpriseTemplateAccessControlStore{}
if initial, changed, enabled := featureChanged(codersdk.FeatureAccessControl); shouldUpdate(initial, changed, enabled) {
var acs agpldbauthz.AccessControlStore = agpldbauthz.AGPLTemplateAccessControlStore{}
if enabled {
acs = dbauthz.EnterpriseTemplateAccessControlStore{}
}
api.AGPL.AccessControlStore.Store(&acs)
}
api.AGPL.AccessControlStore.Store(&acs)
}
if initial, changed, enabled := featureChanged(codersdk.FeatureAppearance); shouldUpdate(initial, changed, enabled) {
if enabled {
f := newAppearanceFetcher(
api.Database,
api.DeploymentValues.Support.Links.Value,
api.DeploymentValues.DocsURL.String(),
buildinfo.Version(),
)
api.AGPL.AppearanceFetcher.Store(&f)
} else {
f := appearance.NewDefaultFetcher(api.DeploymentValues.DocsURL.String())
api.AGPL.AppearanceFetcher.Store(&f)
if initial, changed, enabled := featureChanged(codersdk.FeatureAppearance); shouldUpdate(initial, changed, enabled) {
if enabled {
f := newAppearanceFetcher(
api.Database,
api.DeploymentValues.Support.Links.Value,
api.DeploymentValues.DocsURL.String(),
buildinfo.Version(),
)
api.AGPL.AppearanceFetcher.Store(&f)
} else {
f := appearance.NewDefaultFetcher(api.DeploymentValues.DocsURL.String())
api.AGPL.AppearanceFetcher.Store(&f)
}
}
}
if initial, changed, enabled := featureChanged(codersdk.FeatureControlSharedPorts); shouldUpdate(initial, changed, enabled) {
var ps agplportsharing.PortSharer = agplportsharing.DefaultPortSharer
if enabled {
ps = portsharing.NewEnterprisePortSharer()
if initial, changed, enabled := featureChanged(codersdk.FeatureControlSharedPorts); shouldUpdate(initial, changed, enabled) {
var ps agplportsharing.PortSharer = agplportsharing.DefaultPortSharer
if enabled {
ps = portsharing.NewEnterprisePortSharer()
}
api.AGPL.PortSharer.Store(&ps)
}
api.AGPL.PortSharer.Store(&ps)
}
// External token encryption is soft-enforced
featureExternalTokenEncryption := reloadedEntitlements.Features[codersdk.FeatureExternalTokenEncryption]
featureExternalTokenEncryption.Enabled = len(api.ExternalTokenEncryption) > 0
if featureExternalTokenEncryption.Enabled && featureExternalTokenEncryption.Entitlement != codersdk.EntitlementEntitled {
msg := fmt.Sprintf("%s is enabled (due to setting external token encryption keys) but your license is not entitled to this feature.", codersdk.FeatureExternalTokenEncryption.Humanize())
api.Logger.Warn(ctx, msg)
reloadedEntitlements.Warnings = append(reloadedEntitlements.Warnings, msg)
}
reloadedEntitlements.Features[codersdk.FeatureExternalTokenEncryption] = featureExternalTokenEncryption
api.Entitlements.Replace(reloadedEntitlements)
return nil
// External token encryption is soft-enforced
featureExternalTokenEncryption := reloadedEntitlements.Features[codersdk.FeatureExternalTokenEncryption]
featureExternalTokenEncryption.Enabled = len(api.ExternalTokenEncryption) > 0
if featureExternalTokenEncryption.Enabled && featureExternalTokenEncryption.Entitlement != codersdk.EntitlementEntitled {
msg := fmt.Sprintf("%s is enabled (due to setting external token encryption keys) but your license is not entitled to this feature.", codersdk.FeatureExternalTokenEncryption.Humanize())
api.Logger.Warn(ctx, msg)
reloadedEntitlements.Warnings = append(reloadedEntitlements.Warnings, msg)
}
reloadedEntitlements.Features[codersdk.FeatureExternalTokenEncryption] = featureExternalTokenEncryption
return reloadedEntitlements, nil
})
}
// getProxyDERPStartingRegionID returns the starting region ID that should be

View File

@ -19,7 +19,7 @@ func TestEnterpriseParseGroupClaims(t *testing.T) {
t.Parallel()
entitled := entitlements.New()
entitled.Update(func(entitlements *codersdk.Entitlements) {
entitled.Modify(func(entitlements *codersdk.Entitlements) {
entitlements.Features[codersdk.FeatureTemplateRBAC] = codersdk.Feature{
Entitlement: codersdk.EntitlementEntitled,
Enabled: true,

View File

@ -70,7 +70,7 @@ func TestOrganizationSync(t *testing.T) {
}
entitled := entitlements.New()
entitled.Update(func(entitlements *codersdk.Entitlements) {
entitled.Modify(func(entitlements *codersdk.Entitlements) {
entitlements.Features[codersdk.FeatureMultipleOrganizations] = codersdk.Feature{
Entitlement: codersdk.EntitlementEntitled,
Enabled: true,

View File

@ -20,7 +20,7 @@ func TestEnterpriseParseRoleClaims(t *testing.T) {
t.Parallel()
entitled := entitlements.New()
entitled.Update(func(en *codersdk.Entitlements) {
entitled.Modify(func(en *codersdk.Entitlements) {
en.Features[codersdk.FeatureUserRoleManagement] = codersdk.Feature{
Entitlement: codersdk.EntitlementEntitled,
Enabled: true,

View File

@ -27,7 +27,7 @@ func TestCollectLicenseMetrics(t *testing.T) {
userLimit = 7
)
sut.Entitlements = entitlements.New()
sut.Entitlements.Update(func(entitlements *codersdk.Entitlements) {
sut.Entitlements.Modify(func(entitlements *codersdk.Entitlements) {
entitlements.Features[codersdk.FeatureUserLimit] = codersdk.Feature{
Enabled: true,
Actual: ptr.Int64(actualUsers),