mirror of
https://github.com/tailscale/tailscale.git
synced 2025-12-31 08:16:23 +00:00
This updates the syspolicy package to support multiple policy sources in the three policy scopes: user, profile, and device, and provides a merged resultant policy. A policy source is a syspolicy/source.Store that has a name and provides access to policy settings for a given scope. It can be registered with syspolicy/rsop.RegisterStore. Policy sources and policy stores can be either platform-specific or platform-agnostic. On Windows, we have the Registry-based, platform-specific policy store implemented as syspolicy/source.PlatformPolicyStore. This store provides access to the Group Policy and MDM policy settings stored in the Registry. On other platforms, we currently provide a wrapper that converts a syspolicy.Handler into a syspolicy/source.Store. However, we should update them in follow-up PRs. An example of a platform-agnostic policy store would be a policy deployed from the control, a local policy config file, or even environment variables. We maintain the current, most recent version of the resultant policy for each scope in an rsop.Policy. This is done by reading and merging the policy settings from the registered stores the first time the resultant policy is requested, then re-reading and re-merging them if a store implements the source.Changeable interface and reports a policy change. Policy change notifications are debounced to avoid re-reading policy settings multiple times if there are several changes within a short period. The rsop.Policy can notify clients if the resultant policy has changed. However, we do not currently expose this via the syspolicy package and plan to do so differently along with a struct-based policy hierarchy in the next PR. To facilitate this, all policy settings should be registered with the setting.Register function. The syspolicy package does this automatically for all policy settings defined in policy_keys.go. The new functionality is available through the existing syspolicy.Read* set of functions. However, we plan to expose it via a struct-based policy hierarchy, along with policy change notifications that other subsystems can use, in the next PR. We also plan to send the resultant policy back from tailscaled to the clients via the LocalAPI. This is primarily a foundational PR to facilitate future changes, but the immediate observable changes on Windows include: - The service will use the current policy setting values instead of those read at OS boot time. - The GUI has access to policy settings configured on a per-user basis. On Android: - We now report policy setting usage via clientmetrics. Updates #12687 Signed-off-by: Nick Khyl <nickk@tailscale.com>
394 lines
11 KiB
Go
394 lines
11 KiB
Go
// Copyright (c) Tailscale Inc & AUTHORS
|
|
// SPDX-License-Identifier: BSD-3-Clause
|
|
|
|
package source
|
|
|
|
import (
|
|
"errors"
|
|
"fmt"
|
|
"io"
|
|
"slices"
|
|
"sort"
|
|
"sync"
|
|
"time"
|
|
|
|
"tailscale.com/util/mak"
|
|
"tailscale.com/util/set"
|
|
"tailscale.com/util/syspolicy/internal/loggerx"
|
|
"tailscale.com/util/syspolicy/internal/metrics"
|
|
"tailscale.com/util/syspolicy/setting"
|
|
)
|
|
|
|
// Reader reads all configured policy settings from a given [Store].
|
|
// It registers a change callback with the [Store] and maintains the current version
|
|
// of the [setting.Snapshot] by lazily re-reading policy settings from the [Store]
|
|
// whenever a new snapshot is requested
|
|
// It is safe for concurrent use.
|
|
type Reader struct {
|
|
store Store
|
|
origin *setting.Origin
|
|
settings []*setting.Definition
|
|
unregisterChangeNotifier func()
|
|
doneCh chan struct{} // closed when policyCache is closed.
|
|
|
|
mu sync.RWMutex
|
|
closing bool
|
|
upToDate bool
|
|
lastPolicy *setting.Snapshot
|
|
sessions set.HandleSet[*ReadingSession]
|
|
}
|
|
|
|
// newReader returns a new [Reader] that reads policy settings from a given [Store].
|
|
// The returned reader takes ownership of the store. If the store implements [io.Closer],
|
|
// the returned reader will close the store when it is closed.
|
|
func newReader(store Store, origin *setting.Origin) (*Reader, error) {
|
|
settings, err := setting.Definitions()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if expirable, ok := store.(Expirable); ok {
|
|
select {
|
|
case <-expirable.Done():
|
|
return nil, ErrStoreClosed
|
|
default:
|
|
}
|
|
}
|
|
|
|
reader := &Reader{store: store, origin: origin, settings: settings, doneCh: make(chan struct{})}
|
|
if changeable, ok := store.(Changeable); ok {
|
|
// We should subscribe to policy change notifications first before reading
|
|
// the policy settings from the store. This way we won't miss any notifications.
|
|
if reader.unregisterChangeNotifier, err = changeable.RegisterChangeCallback(reader.onPolicyChange); err != nil {
|
|
// Errors registering policy change callbacks are non-fatal.
|
|
// TODO(nickkhyl): implement a background policy refresh every X minutes?
|
|
loggerx.Errorf("failed to register %v policy change callback: %v\n", origin, err)
|
|
}
|
|
}
|
|
|
|
if _, err := reader.reload(true); err != nil {
|
|
if reader.unregisterChangeNotifier != nil {
|
|
reader.unregisterChangeNotifier()
|
|
}
|
|
return nil, err
|
|
}
|
|
|
|
if expirable, ok := store.(Expirable); ok {
|
|
if waitCh := expirable.Done(); waitCh != nil {
|
|
go func() {
|
|
select {
|
|
case <-waitCh:
|
|
reader.Close()
|
|
case <-reader.doneCh:
|
|
}
|
|
}()
|
|
}
|
|
}
|
|
|
|
return reader, nil
|
|
}
|
|
|
|
// GetSettings returns the current [*setting.Snapshot],
|
|
// re-reading it from from the underlying [Store] only if the policy
|
|
// has changed since it was read last. It never fails and returns
|
|
// the previous version of the policy settings if a read attempt fails.
|
|
func (r *Reader) GetSettings() *setting.Snapshot {
|
|
r.mu.RLock()
|
|
if r.upToDate {
|
|
r.mu.RUnlock()
|
|
return r.lastPolicy
|
|
}
|
|
r.mu.RUnlock()
|
|
|
|
policy, err := r.reload(false)
|
|
if err != nil {
|
|
// If the policy could not be reloaded at all, we'll return the last cached version of it.
|
|
// On the contrary, errors specific to individual policy items are always propagated to the callers.
|
|
loggerx.Errorf("failed to reload %v policy: %v\n", r.origin, err)
|
|
}
|
|
return policy
|
|
}
|
|
|
|
// ReadSettings reads policy settings from the underlying [Store] even if no
|
|
// changes were detected. It returns the new [*setting.Snapshot], nil on
|
|
// success, or nil, error in case of failure.
|
|
func (r *Reader) ReadSettings() (*setting.Snapshot, error) {
|
|
b, err := r.reload(true)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
return b, nil
|
|
}
|
|
|
|
// reload is like [Reader.ReadSettings], but allows specifying whether to re-read
|
|
// an unchanged policy, and returns the last [*setting.Snapshot] if the read fails.
|
|
func (r *Reader) reload(force bool) (*setting.Snapshot, error) {
|
|
r.mu.Lock()
|
|
defer r.mu.Unlock()
|
|
if r.upToDate && !force {
|
|
return r.lastPolicy, nil
|
|
}
|
|
|
|
if lockable, ok := r.store.(Lockable); ok {
|
|
if err := lockable.Lock(); err != nil {
|
|
return r.lastPolicy, err
|
|
}
|
|
defer lockable.Unlock()
|
|
}
|
|
|
|
r.upToDate = true
|
|
|
|
metrics.Reset(r.origin)
|
|
|
|
var m map[setting.Key]setting.RawItem
|
|
if lastPolicyCount := r.lastPolicy.Len(); lastPolicyCount > 0 {
|
|
m = make(map[setting.Key]setting.RawItem, lastPolicyCount)
|
|
}
|
|
for _, s := range r.settings {
|
|
if !r.origin.Scope().IsConfigurableSetting(s) {
|
|
// Skip settings that cannot be configured in the current scope.
|
|
continue
|
|
}
|
|
|
|
val, err := readPolicySettingValue(r.store, s)
|
|
if err != nil && (errors.Is(err, setting.ErrNoSuchKey) || errors.Is(err, setting.ErrNotConfigured)) {
|
|
metrics.ReportNotConfigured(r.origin, s)
|
|
continue
|
|
}
|
|
|
|
if err == nil {
|
|
metrics.ReportConfigured(r.origin, s, val)
|
|
} else {
|
|
metrics.ReportError(r.origin, s, err)
|
|
}
|
|
|
|
// If there's an error reading a single policy, such as a value type mismatch,
|
|
// we'll wrap the error to preserve its text and return it
|
|
// whenever someone attempts to fetch the value.
|
|
mak.Set(&m, s.Key(), setting.RawItemWith(val, setting.WrapError(err), r.origin))
|
|
}
|
|
|
|
newPolicy := setting.NewSnapshot(m, setting.SummaryWith(r.origin))
|
|
if r.lastPolicy == nil || !newPolicy.EqualItems(r.lastPolicy) {
|
|
r.lastPolicy = newPolicy
|
|
}
|
|
return r.lastPolicy, nil
|
|
}
|
|
|
|
// ReadingSession is like [Reader], but with a channel that's written
|
|
// to when there's a policy change, and closed when the session is terminated.
|
|
type ReadingSession struct {
|
|
reader *Reader
|
|
policyChangedCh chan struct{} // 1-buffered channel
|
|
handle set.Handle // in the reader.sessions
|
|
closeInternal func()
|
|
}
|
|
|
|
// OpenSession opens and returns a new session to r, allowing the caller
|
|
// to get notified whenever a policy change is reported by the [source.Store],
|
|
// or an [ErrStoreClosed] if the reader has already been closed.
|
|
func (r *Reader) OpenSession() (*ReadingSession, error) {
|
|
session := &ReadingSession{
|
|
reader: r,
|
|
policyChangedCh: make(chan struct{}, 1),
|
|
}
|
|
session.closeInternal = sync.OnceFunc(func() { close(session.policyChangedCh) })
|
|
r.mu.Lock()
|
|
if !r.closing {
|
|
session.handle = r.sessions.Add(session)
|
|
r.mu.Unlock()
|
|
return session, nil
|
|
}
|
|
r.mu.Unlock()
|
|
return nil, ErrStoreClosed
|
|
}
|
|
|
|
// GetSettings is like [Reader.GetSettings].
|
|
func (s *ReadingSession) GetSettings() *setting.Snapshot {
|
|
return s.reader.GetSettings()
|
|
}
|
|
|
|
// ReadSettings is like [Reader.ReadSettings].
|
|
func (s *ReadingSession) ReadSettings() (*setting.Snapshot, error) {
|
|
return s.reader.ReadSettings()
|
|
}
|
|
|
|
// PolicyChanged returns a channel that's written to when
|
|
// there's a policy change, closed when the session is terminated.
|
|
func (s *ReadingSession) PolicyChanged() <-chan struct{} {
|
|
return s.policyChangedCh
|
|
}
|
|
|
|
// Close unregisters this session with the [Reader].
|
|
func (s *ReadingSession) Close() {
|
|
s.reader.mu.Lock()
|
|
delete(s.reader.sessions, s.handle)
|
|
s.closeInternal()
|
|
s.reader.mu.Unlock()
|
|
}
|
|
|
|
// onPolicyChange handles a policy change notification from the [Store],
|
|
// invalidating the current [setting.Snapshot] in r,
|
|
// and notifying the active [ReadingSession]s.
|
|
func (r *Reader) onPolicyChange() {
|
|
r.mu.Lock()
|
|
defer r.mu.Unlock()
|
|
r.upToDate = false
|
|
for _, s := range r.sessions {
|
|
select {
|
|
case s.policyChangedCh <- struct{}{}:
|
|
// Notified.
|
|
default:
|
|
// 1-buffered channel is full, meaning that another policy change
|
|
// notification is already en route.
|
|
}
|
|
}
|
|
}
|
|
|
|
// Close closes the store reader and the underlying store.
|
|
func (r *Reader) Close() error {
|
|
r.mu.Lock()
|
|
if r.closing {
|
|
r.mu.Unlock()
|
|
return nil
|
|
}
|
|
r.closing = true
|
|
r.mu.Unlock()
|
|
|
|
if r.unregisterChangeNotifier != nil {
|
|
r.unregisterChangeNotifier()
|
|
r.unregisterChangeNotifier = nil
|
|
}
|
|
|
|
if closer, ok := r.store.(io.Closer); ok {
|
|
if err := closer.Close(); err != nil {
|
|
return err
|
|
}
|
|
}
|
|
r.store = nil
|
|
|
|
close(r.doneCh)
|
|
|
|
r.mu.Lock()
|
|
defer r.mu.Unlock()
|
|
for _, c := range r.sessions {
|
|
c.closeInternal()
|
|
}
|
|
r.sessions = nil
|
|
return nil
|
|
}
|
|
|
|
// Done returns a channel that is closed when the reader is closed.
|
|
func (r *Reader) Done() <-chan struct{} {
|
|
return r.doneCh
|
|
}
|
|
|
|
// ReadableSource is a [Source] open for reading.
|
|
type ReadableSource struct {
|
|
*Source
|
|
*ReadingSession
|
|
}
|
|
|
|
// Close closes the underlying [ReadingSession].
|
|
func (s ReadableSource) Close() {
|
|
s.ReadingSession.Close()
|
|
}
|
|
|
|
// ReadableSources is a slice of [ReadableSource].
|
|
type ReadableSources []ReadableSource
|
|
|
|
// Contains reports whether s contains the specified source.
|
|
func (s ReadableSources) Contains(source *Source) bool {
|
|
return s.IndexOf(source) != -1
|
|
}
|
|
|
|
// IndexOf returns position of the specified source in s, or -1
|
|
// if the source does not exist.
|
|
func (s ReadableSources) IndexOf(source *Source) int {
|
|
return slices.IndexFunc(s, func(rs ReadableSource) bool {
|
|
return rs.Source == source
|
|
})
|
|
}
|
|
|
|
// InsertionIndexOf returns the position at which source can be inserted
|
|
// to maintain the sorted order of the readableSources.
|
|
// The return value is unspecified if s is not sorted on entry to InsertionIndexOf.
|
|
func (s ReadableSources) InsertionIndexOf(source *Source) int {
|
|
low, high := 0, len(s)
|
|
for low < high {
|
|
mid := (low + high) / 2
|
|
if s[mid].Compare(source) <= 0 {
|
|
low = mid + 1
|
|
} else {
|
|
high = mid
|
|
}
|
|
}
|
|
return low
|
|
}
|
|
|
|
// StableSort sorts the readableSources by the precedence, so that policy settings
|
|
// from sources with higher precedence (e.g., [DeviceScope]) will be merged last,
|
|
// overriding any policy settings with the same keys configured in sources with
|
|
// lower precedence (e.g., [CurrentUserScope]).
|
|
func (s *ReadableSources) StableSort() {
|
|
sort.SliceStable(*s, func(i, j int) bool {
|
|
return (*s)[i].Source.Compare((*s)[j].Source) < 0
|
|
})
|
|
}
|
|
|
|
// DeleteAt closes and deletes the i-th source from s.
|
|
func (s *ReadableSources) DeleteAt(i int) {
|
|
(*s)[i].Close()
|
|
*s = slices.Delete(*s, i, i+1)
|
|
}
|
|
|
|
// Close closes and deletes all sources in s.
|
|
func (s *ReadableSources) Close() {
|
|
for _, s := range *s {
|
|
s.Close()
|
|
}
|
|
*s = nil
|
|
}
|
|
|
|
func readPolicySettingValue(store Store, s *setting.Definition) (value any, err error) {
|
|
switch key := s.Key(); s.Type() {
|
|
case setting.BooleanValue:
|
|
return store.ReadBoolean(key)
|
|
case setting.IntegerValue:
|
|
return store.ReadUInt64(key)
|
|
case setting.StringValue:
|
|
return store.ReadString(key)
|
|
case setting.StringListValue:
|
|
return store.ReadStringArray(key)
|
|
case setting.PreferenceOptionValue:
|
|
s, err := store.ReadString(key)
|
|
if err == nil {
|
|
var value setting.PreferenceOption
|
|
if err = value.UnmarshalText([]byte(s)); err == nil {
|
|
return value, nil
|
|
}
|
|
}
|
|
return setting.ShowChoiceByPolicy, err
|
|
case setting.VisibilityValue:
|
|
s, err := store.ReadString(key)
|
|
if err == nil {
|
|
var value setting.Visibility
|
|
if err = value.UnmarshalText([]byte(s)); err == nil {
|
|
return value, nil
|
|
}
|
|
}
|
|
return setting.VisibleByPolicy, err
|
|
case setting.DurationValue:
|
|
s, err := store.ReadString(key)
|
|
if err == nil {
|
|
var value time.Duration
|
|
if value, err = time.ParseDuration(s); err == nil {
|
|
return value, nil
|
|
}
|
|
}
|
|
return nil, err
|
|
default:
|
|
return nil, fmt.Errorf("%w: unsupported setting type: %v", setting.ErrTypeMismatch, s.Type())
|
|
}
|
|
}
|