[17449] Introduces a locking mechanism over variables (#18207)

It includes the work over the state store, the PRC server, the HTTP server, the go API package and the CLI's  command. To read more on the actuall functionality, refer to the RFCs [NMD-178] Locking with Nomad Variables and [NMD-179] Leader election using locking mechanism for the Autoscaler.
This commit is contained in:
Juana De La Cuesta
2023-09-21 17:56:33 +02:00
committed by GitHub
parent 86d2cdcf80
commit 72acaf6623
40 changed files with 5960 additions and 525 deletions

3
.changelog/18520.txt Normal file
View File

@@ -0,0 +1,3 @@
```release-note:improvement
vars: Added a locking feature for Nomad Variables
```

View File

@@ -12,6 +12,7 @@ import (
"errors"
"fmt"
"io"
"math"
"net"
"net/http"
"net/url"
@@ -200,6 +201,10 @@ type Config struct {
TLSConfig *TLSConfig
Headers http.Header
// retryOptions holds the configuration necessary to perform retries
// on put calls.
retryOptions *retryOptions
}
// ClientConfig copies the configuration with a new client address, region, and
@@ -578,6 +583,40 @@ func (c *Client) SetSecretID(secretID string) {
c.config.SecretID = secretID
}
func (c *Client) configureRetries(ro *retryOptions) {
c.config.retryOptions = &retryOptions{
maxRetries: defaultNumberOfRetries,
maxBackoffDelay: defaultMaxBackoffDelay,
delayBase: defaultDelayTimeBase,
}
if ro.delayBase != 0 {
c.config.retryOptions.delayBase = ro.delayBase
}
if ro.maxRetries != defaultNumberOfRetries {
c.config.retryOptions.maxRetries = ro.maxRetries
}
if ro.maxBackoffDelay != 0 {
c.config.retryOptions.maxBackoffDelay = ro.maxBackoffDelay
}
if ro.maxToLastCall != 0 {
c.config.retryOptions.maxToLastCall = ro.maxToLastCall
}
if ro.fixedDelay != 0 {
c.config.retryOptions.fixedDelay = ro.fixedDelay
}
// Ensure that a big attempt number or a big delayBase number will not cause
// a negative delay by overflowing the delay increase.
c.config.retryOptions.maxValidAttempt = int64(math.Log2(float64(math.MaxInt64 /
c.config.retryOptions.delayBase.Nanoseconds())))
}
// request is used to help build up a request
type request struct {
config *Config

377
api/locks.go Normal file
View File

@@ -0,0 +1,377 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: MPL-2.0
package api
import (
"context"
"errors"
"fmt"
"math/rand"
"net/http"
"time"
"github.com/hashicorp/go-multierror"
)
const (
lockLeaseRenewalFactor = 0.7
lockRetryBackoffFactor = 1.1
// DefaultLockTTL is the default value used to maintain a lock before it needs to
// be renewed. The actual value comes from the experience with Consul.
DefaultLockTTL = 15 * time.Second
// DefaultLockDelay is the default a lock will be blocked after the TTL
// went by without any renews. It is intended to prevent split brain situations.
// The actual value comes from the experience with Consul.
DefaultLockDelay = 15 * time.Second
)
var (
// ErrLockConflict is returned in case a lock operation can't be performed
// because the caller is not the current holder of the lock.
ErrLockConflict = errors.New("conflicting operation over lock")
//LockNoPathErr is returned when no path is provided in the variable to be
// used for the lease mechanism
LockNoPathErr = errors.New("variable's path can't be empty")
)
// Locks returns a new handle on a lock for the given variable.
func (c *Client) Locks(wo WriteOptions, v Variable, opts ...LocksOption) (*Locks, error) {
if v.Path == "" {
return nil, LockNoPathErr
}
ttl, err := time.ParseDuration(v.Lock.TTL)
if err != nil {
return nil, err
}
l := &Locks{
c: c,
WriteOptions: wo,
variable: v,
ttl: ttl,
ro: retryOptions{
maxToLastCall: ttl,
maxRetries: defaultNumberOfRetries,
},
}
for _, opt := range opts {
opt(l)
}
l.c.configureRetries(&l.ro)
return l, nil
}
// Locks is used to maintain all the resources necessary to operate over a lock.
// It makes the calls to the http using an exponential retry mechanism that will
// try until it either reaches 5 attempts or the ttl of the lock expires.
// The variable doesn't need to exist, one will be created internally
// but a path most be provided.
//
// Important: It will be on the user to remove the variable created for the lock.
type Locks struct {
c *Client
variable Variable
ttl time.Duration
ro retryOptions
WriteOptions
}
type LocksOption = func(l *Locks)
// LocksOptionWithMaxRetries allows access to configure the number of max retries the lock
// handler will perform in case of an expected response while interacting with the
// locks endpoint.
func LocksOptionWithMaxRetries(maxRetries int64) LocksOption {
return func(l *Locks) {
l.ro.maxRetries = maxRetries
}
}
// Acquire will make the actual call to acquire the lock over the variable using
// the ttl in the Locks to create the VariableLock. It will return the
// path of the variable holding the lock.
//
// Acquire returns the path to the variable holding the lock.
func (l *Locks) Acquire(ctx context.Context) (string, error) {
var out Variable
_, err := l.c.retryPut(ctx, "/v1/var/"+l.variable.Path+"?lock-acquire", l.variable, &out, &l.WriteOptions)
if err != nil {
callErr, ok := err.(UnexpectedResponseError)
// http.StatusConflict means the lock is already held. This will happen
// under the normal execution if multiple instances are fighting for the same lock and
// doesn't disrupt the flow.
if ok && callErr.statusCode == http.StatusConflict {
return "", fmt.Errorf("acquire conflict %w", ErrLockConflict)
}
return "", err
}
l.variable.Lock = out.Lock
return l.variable.Path, nil
}
// Release makes the call to release the lock over a variable, even if the ttl
// has not yet passed.
// In case of a call to release a non held lock, Release returns ErrLockConflict.
func (l *Locks) Release(ctx context.Context) error {
var out Variable
rv := &Variable{
Lock: &VariableLock{
ID: l.variable.LockID(),
},
}
_, err := l.c.retryPut(ctx, "/v1/var/"+l.variable.Path+"?lock-release", rv,
&out, &l.WriteOptions)
if err != nil {
callErr, ok := err.(UnexpectedResponseError)
if ok && callErr.statusCode == http.StatusConflict {
return fmt.Errorf("release conflict %w", ErrLockConflict)
}
return err
}
return nil
}
// Renew is used to extend the ttl of a lock. It can be used as a heartbeat or a
// lease to maintain the hold over the lock for longer periods or as a sync
// mechanism among multiple instances looking to acquire the same lock.
// Renew will return true if the renewal was successful.
//
// In case of a call to renew a non held lock, Renew returns ErrLockConflict.
func (l *Locks) Renew(ctx context.Context) error {
var out VariableMetadata
_, err := l.c.retryPut(ctx, "/v1/var/"+l.variable.Path+"?lock-renew", l.variable, &out, &l.WriteOptions)
if err != nil {
callErr, ok := err.(UnexpectedResponseError)
if ok && callErr.statusCode == http.StatusConflict {
return fmt.Errorf("renew conflict %w", ErrLockConflict)
}
return err
}
return nil
}
func (l *Locks) LockTTL() time.Duration {
return l.ttl
}
// Locker is the interface that wraps the lock handler. It is used by the lock
// leaser to handle all lock operations.
type Locker interface {
// Acquire will make the actual call to acquire the lock over the variable using
// the ttl in the Locks to create the VariableLock.
//
// Acquire returns the path to the variable holding the lock.
Acquire(ctx context.Context) (string, error)
// Release makes the call to release the lock over a variable, even if the ttl
// has not yet passed.
Release(ctx context.Context) error
// Renew is used to extend the ttl of a lock. It can be used as a heartbeat or a
// lease to maintain the hold over the lock for longer periods or as a sync
// mechanism among multiple instances looking to acquire the same lock.
Renew(ctx context.Context) error
// LockTTL returns the expiration time of the underlying lock.
LockTTL() time.Duration
}
// LockLeaser is a helper used to run a protected function that should only be
// active if the instance that runs it is currently holding the lock.
// Can be used to provide synchrony among multiple independent instances.
//
// It includes the lease renewal mechanism and tracking in case the protected
// function returns an error. Internally it uses an exponential retry mechanism
// for the api calls.
type LockLeaser struct {
Name string
renewalPeriod time.Duration
waitPeriod time.Duration
randomDelay time.Duration
earlyReturn bool
locked bool
locker Locker
}
type LockLeaserOption = func(l *LockLeaser)
// LockLeaserOptionWithEarlyReturn informs the leaser to return after the lock
// acquire fails and to not wait to attempt again.
func LockLeaserOptionWithEarlyReturn(er bool) LockLeaserOption {
return func(l *LockLeaser) {
l.earlyReturn = er
}
}
// LockLeaserOptionWithWaitPeriod is used to set a back off period between
// calls to attempt to acquire the lock. By default it is set to 1.1 * TTLs.
func LockLeaserOptionWithWaitPeriod(wp time.Duration) LockLeaserOption {
return func(l *LockLeaser) {
l.waitPeriod = wp
}
}
// NewLockLeaser returns an instance of LockLeaser. callerID
// is optional, in case they it is not provided, internal one will be created.
func (c *Client) NewLockLeaser(l Locker, opts ...LockLeaserOption) *LockLeaser {
rn := rand.New(rand.NewSource(time.Now().Unix())).Intn(100)
ll := &LockLeaser{
renewalPeriod: time.Duration(float64(l.LockTTL()) * lockLeaseRenewalFactor),
waitPeriod: time.Duration(float64(l.LockTTL()) * lockRetryBackoffFactor),
randomDelay: time.Duration(rn) * time.Millisecond,
locker: l,
earlyReturn: false,
}
for _, opt := range opts {
opt(ll)
}
return ll
}
// Start wraps the start function in charge of executing the protected
// function and maintain the lease but is in charge of releasing the
// lock before exiting. It is a blocking function.
func (ll *LockLeaser) Start(ctx context.Context, protectedFuncs ...func(ctx context.Context) error) error {
var mErr multierror.Error
err := ll.start(ctx, protectedFuncs...)
if err != nil {
mErr.Errors = append(mErr.Errors, err)
}
if ll.locked {
err = ll.locker.Release(ctx)
if err != nil {
mErr.Errors = append(mErr.Errors, fmt.Errorf("lock release: %w", err))
}
}
return mErr.ErrorOrNil()
}
// start starts the process of maintaining the lease and executes the protected
// function on an independent go routine. It is a blocking function, it
// will return once the protected function is done or an execution error
// arises.
func (ll *LockLeaser) start(ctx context.Context, protectedFuncs ...func(ctx context.Context) error) error {
ctx, cancel := context.WithCancel(ctx)
defer cancel()
// errChannel is used track execution errors
errChannel := make(chan error, 1)
defer close(errChannel)
// To avoid collisions if all the instances start at the same time, wait
// a random time before making the first call.
waitWithContext(ctx, ll.randomDelay)
waitTicker := time.NewTicker(ll.waitPeriod)
defer waitTicker.Stop()
for {
lockID, err := ll.locker.Acquire(ctx)
if err != nil {
if errors.Is(err, ErrLockConflict) && ll.earlyReturn {
return nil
}
if !errors.Is(err, ErrLockConflict) {
errChannel <- err
}
}
if lockID != "" {
ll.locked = true
funcCtx, funcCancel := context.WithCancel(ctx)
defer funcCancel()
// Execute the lock protected function.
go func() {
defer funcCancel()
for _, f := range protectedFuncs {
err := f(funcCtx)
if err != nil {
errChannel <- fmt.Errorf("error executing protected function %w", err)
return
}
cancel()
}
}()
// Maintain lease is a blocking function, it will return if there is
// an error maintaining the lease or the protected function returned.
err = ll.maintainLease(funcCtx)
if err != nil && !errors.Is(err, ErrLockConflict) {
errChannel <- fmt.Errorf("error renewing the lease: %w", err)
}
}
waitTicker.Stop()
waitTicker = time.NewTicker(ll.waitPeriod)
select {
case <-ctx.Done():
return nil
case err := <-errChannel:
return fmt.Errorf("locks: %w", err)
case <-waitTicker.C:
}
}
}
func (ll *LockLeaser) maintainLease(ctx context.Context) error {
renewTicker := time.NewTicker(ll.renewalPeriod)
defer renewTicker.Stop()
for {
select {
case <-ctx.Done():
return nil
case <-renewTicker.C:
err := ll.locker.Renew(ctx)
if err != nil {
return err
}
}
}
}
func waitWithContext(ctx context.Context, d time.Duration) {
t := time.NewTimer(d)
defer t.Stop()
select {
case <-ctx.Done():
case <-t.C:
}
}

499
api/locks_test.go Normal file
View File

@@ -0,0 +1,499 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: MPL-2.0
package api
import (
"context"
"errors"
"sync"
"testing"
"time"
"github.com/shoenig/test/must"
)
var testLease = 10 * time.Millisecond
type mockLock struct {
locked bool
acquireCalls map[string]int
renewsCounter int
mu sync.Mutex
leaseStartTime time.Time
}
func (ml *mockLock) acquire(_ context.Context, callerID string) (string, error) {
ml.mu.Lock()
defer ml.mu.Unlock()
if callerID == "hac-early-return" {
return "", ErrLockConflict
}
ml.acquireCalls[callerID] += 1
if ml.locked {
return "", nil
}
ml.locked = true
ml.leaseStartTime = time.Now()
ml.renewsCounter = 0
return "lockPath", nil
}
type lockHandler struct {
*mockLock
callerID string
}
func (lh *lockHandler) LockTTL() time.Duration {
return testLease
}
func (lh *lockHandler) Acquire(ctx context.Context) (string, error) {
return lh.acquire(ctx, lh.callerID)
}
func (ml *mockLock) Release(_ context.Context) error {
ml.mu.Lock()
defer ml.mu.Unlock()
if !ml.locked {
return errors.New("lock not locked")
}
ml.locked = false
ml.renewsCounter = 0
return nil
}
// The behavior of renew is not an exact replication of
// the lock work, its intended to test the behavior of the
// multiple instances running.
func (ml *mockLock) Renew(_ context.Context) error {
ml.mu.Lock()
defer ml.mu.Unlock()
if !ml.locked {
return errors.New("error")
}
if time.Since(ml.leaseStartTime) > testLease {
ml.locked = false
return ErrLockConflict
}
ml.leaseStartTime = time.Now()
ml.renewsCounter += 1
return nil
}
func (ml *mockLock) getLockState() mockLock {
ml.mu.Lock()
defer ml.mu.Unlock()
return mockLock{
locked: ml.locked,
acquireCalls: copyMap(ml.acquireCalls),
renewsCounter: ml.renewsCounter,
}
}
type mockService struct {
mockLock
mu sync.Mutex
startsCounter int
starterID string
}
func (ms *mockService) Run(callerID string, _ context.Context) func(ctx context.Context) error {
return func(ctx context.Context) error {
ms.mu.Lock()
ms.startsCounter += 1
ms.starterID = callerID
ms.mu.Unlock()
<-ctx.Done()
ms.mu.Lock()
ms.starterID = ""
ms.mu.Unlock()
return nil
}
}
func (ms *mockService) getServiceState() mockService {
ms.mu.Lock()
defer ms.mu.Unlock()
return mockService{
startsCounter: ms.startsCounter,
starterID: ms.starterID,
}
}
func TestAcquireLock_MultipleInstances(t *testing.T) {
l := mockLock{
acquireCalls: map[string]int{},
}
s := mockService{}
testCtx := context.Background()
// Set up independent contexts to test the switch when one controller stops
hac1Ctx, hac1Cancel := context.WithCancel(testCtx)
defer hac1Cancel()
// Wait time on hac1 is 0, it should always get the lock.
hac1 := LockLeaser{
Name: "hac1",
locker: &lockHandler{
mockLock: &l,
callerID: "hac1",
},
renewalPeriod: time.Duration(float64(testLease) * lockLeaseRenewalFactor),
waitPeriod: time.Duration(float64(testLease) * lockRetryBackoffFactor),
randomDelay: 0,
}
hac2 := LockLeaser{
Name: "hac2",
locker: &lockHandler{
mockLock: &l,
callerID: "hac2",
},
renewalPeriod: time.Duration(float64(testLease) * lockLeaseRenewalFactor),
waitPeriod: time.Duration(float64(testLease) * lockRetryBackoffFactor),
randomDelay: 6 * time.Millisecond,
}
lock := l.getLockState()
must.False(t, lock.locked)
go func() {
err := hac1.Start(hac1Ctx, s.Run(hac1.Name, testCtx))
must.NoError(t, err)
}()
go func() {
err := hac2.Start(testCtx, s.Run(hac2.Name, testCtx))
must.NoError(t, err)
}()
time.Sleep(4 * time.Millisecond)
/*
After 4 ms more (4 ms total):
* hac2 should not have tried to acquire the lock because it has an initial delay of 6ms.
* hac1 should have the lock and the service should be running.
* The first lease is not over yet, no calls to renew should have been made.
*/
lock = l.getLockState()
service := s.getServiceState()
must.True(t, lock.locked)
must.Eq(t, 1, lock.acquireCalls[hac1.Name])
must.Eq(t, 0, lock.acquireCalls[hac2.Name])
must.Eq(t, 0, lock.renewsCounter)
must.Eq(t, 1, service.startsCounter)
must.StrContains(t, hac1.Name, service.starterID)
time.Sleep(6 * time.Millisecond)
/*
After 6 ms more (10 ms total):
* hac2 should have tried to acquire the lock at least once, after the 6ms
initial delay has passed.
* hc1 should have renewed once the lease and still hold the lock.
*/
lock = l.getLockState()
service = s.getServiceState()
must.True(t, lock.locked)
must.Eq(t, 1, lock.acquireCalls[hac1.Name])
must.Eq(t, 1, lock.acquireCalls[hac2.Name])
must.One(t, lock.renewsCounter)
must.One(t, service.startsCounter)
must.StrContains(t, hac1.Name, service.starterID)
time.Sleep(5 * time.Millisecond)
/*
After 5 ms more (15 ms total):
* hac2 should have tried to acquire the lock still just once:
initialDelay(6) + waitTime(11) = 18.
* hac1 should have renewed the lease 2 times and still hold the lock:
initialDelay(0) + renewals(2) * renewalPeriod(7) = 14.
*/
lock = l.getLockState()
service = s.getServiceState()
must.Eq(t, 1, lock.acquireCalls[hac1.Name])
must.Eq(t, 1, lock.acquireCalls[hac2.Name])
must.True(t, lock.locked)
must.Eq(t, 2, lock.renewsCounter)
must.Eq(t, 1, service.startsCounter)
must.StrContains(t, hac1.Name, service.starterID)
time.Sleep(15 * time.Millisecond)
/*
After 15 ms more (30 ms total):
* hac2 should have tried to acquire the lock 3 times:
initialDelay(6) + calls(2)* waitTime(11) = 28.
* hac1 should have renewed the lease 4 times and still hold the lock:
initialDelay(0) + renewals(4) * renewalPeriod(7) = 28.
*/
lock = l.getLockState()
service = s.getServiceState()
must.Eq(t, 1, lock.acquireCalls[hac1.Name])
must.Eq(t, 3, lock.acquireCalls[hac2.Name])
must.True(t, lock.locked)
must.Eq(t, 4, lock.renewsCounter)
must.Eq(t, 1, service.startsCounter)
must.StrContains(t, hac1.Name, service.starterID)
// Start a new instance of the service with ha running, initial delay of 1ms
hac3 := LockLeaser{
Name: "hac3",
locker: &lockHandler{
mockLock: &l,
callerID: "hac3",
},
renewalPeriod: time.Duration(float64(testLease) * lockLeaseRenewalFactor),
waitPeriod: time.Duration(float64(testLease) * lockRetryBackoffFactor),
randomDelay: 1 * time.Millisecond,
}
go func() {
err := hac3.Start(testCtx, s.Run(hac3.Name, testCtx))
must.NoError(t, err)
}()
time.Sleep(15 * time.Millisecond)
/*
After 15 ms more (45 ms total):
* hac3 should have tried to acquire the lock twice, once on start and
once after waitTime(11).
* hac2 should have tried to acquire the lock 4 times:
initialDelay(6) + calls(3) * waitTime(11) = 39.
* hac1 should have renewed the lease 4 times and still hold the lock:
initialDelay(0) + renewals(6) * renewalPeriod(7) = 42.
*/
lock = l.getLockState()
service = s.getServiceState()
must.Eq(t, 1, lock.acquireCalls[hac1.Name])
must.Eq(t, 4, lock.acquireCalls[hac2.Name])
must.Eq(t, 2, lock.acquireCalls[hac3.Name])
must.True(t, lock.locked)
must.Eq(t, 6, lock.renewsCounter)
must.Eq(t, 1, service.startsCounter)
must.StrContains(t, hac1.Name, service.starterID)
// Stop hac1 and release the lock
hac1Cancel()
time.Sleep(10 * time.Millisecond)
/*
After 10 ms more (55 ms total):
* hac3 should have tried to acquire the lock 3 times.
* hac2 should have tried to acquire the lock 5 times and succeeded on the
the fifth, is currently holding the lock and Run the service, no renewals.
* hc1 is stopped.
*/
lock = l.getLockState()
service = s.getServiceState()
must.Eq(t, 1, lock.acquireCalls[hac1.Name])
must.Eq(t, 5, lock.acquireCalls[hac2.Name])
must.Eq(t, 3, lock.acquireCalls[hac3.Name])
must.True(t, lock.locked)
must.Eq(t, 0, lock.renewsCounter)
must.Eq(t, 2, service.startsCounter)
must.StrContains(t, hac2.Name, service.starterID)
time.Sleep(5 * time.Millisecond)
/*
After 5 ms more (60 ms total):
* hac3 should have tried to acquire the lock 3 times.
* hac2 should have renewed the lock once.
* hc1 is stopped.
*/
lock = l.getLockState()
service = s.getServiceState()
must.Eq(t, 1, lock.acquireCalls[hac1.Name])
must.Eq(t, 5, lock.acquireCalls[hac2.Name])
must.Eq(t, 3, lock.acquireCalls[hac3.Name])
must.True(t, lock.locked)
must.Eq(t, 1, lock.renewsCounter)
must.Eq(t, 2, service.startsCounter)
must.StrContains(t, hac2.Name, service.starterID)
}
func TestFailedRenewal(t *testing.T) {
l := mockLock{
acquireCalls: map[string]int{},
}
s := mockService{}
testCtx, testCancel := context.WithCancel(context.Background())
defer testCancel()
// Set the renewal period to 1.5 * testLease (15 ms) to force and error.
hac := LockLeaser{
Name: "hac1",
locker: &lockHandler{
mockLock: &l,
callerID: "hac1",
},
renewalPeriod: time.Duration(float64(testLease) * 1.5),
waitPeriod: time.Duration(float64(testLease) * lockRetryBackoffFactor),
randomDelay: 0,
}
lock := l.getLockState()
must.False(t, lock.locked)
go hac.Start(testCtx, s.Run(hac.Name, testCtx))
time.Sleep(5 * time.Millisecond)
/*
After 5ms, the service should be running and the lock held,
no renewals needed or performed yet.
*/
lock = l.getLockState()
service := s.getServiceState()
must.Eq(t, 1, lock.acquireCalls[hac.Name])
must.True(t, lock.locked)
must.Eq(t, 0, lock.renewsCounter)
must.Eq(t, 1, service.startsCounter)
must.StrContains(t, hac.Name, service.starterID)
time.Sleep(15 * time.Millisecond)
/*
After 15ms (20ms total) hac should have tried and failed at renewing the
lock, causing the service to return, no new calls to acquire the lock yet
either.
*/
lock = l.getLockState()
service = s.getServiceState()
must.Eq(t, 1, lock.acquireCalls[hac.Name])
must.False(t, lock.locked)
must.Eq(t, 0, lock.renewsCounter)
must.Eq(t, 1, service.startsCounter)
must.StrContains(t, hac.Name, "")
time.Sleep(10 * time.Millisecond)
/*
After 10ms (30ms total) hac should have tried and succeeded at getting
the lock and the service should be running again.
*/
lock = l.getLockState()
service = s.getServiceState()
must.Eq(t, 2, lock.acquireCalls[hac.Name])
must.True(t, lock.locked)
must.Eq(t, 0, lock.renewsCounter)
must.Eq(t, 2, service.startsCounter)
must.StrContains(t, hac.Name, service.starterID)
}
func TestStart_ProtectedFunctionError(t *testing.T) {
l := mockLock{
acquireCalls: map[string]int{},
}
testCtx := context.Background()
hac := LockLeaser{
locker: &lockHandler{
mockLock: &l,
callerID: "hac",
},
renewalPeriod: time.Duration(float64(testLease) * lockLeaseRenewalFactor),
waitPeriod: time.Duration(float64(testLease) * lockRetryBackoffFactor),
}
lock := l.getLockState()
must.False(t, lock.locked)
err := hac.Start(testCtx, func(ctx context.Context) error {
return errors.New("error")
})
must.Error(t, err)
lock = l.getLockState()
must.False(t, lock.locked)
must.Zero(t, lock.renewsCounter)
}
func copyMap(originalMap map[string]int) map[string]int {
newMap := map[string]int{}
for k, v := range originalMap {
newMap[k] = v
}
return newMap
}
func Test_EarlyReturn(t *testing.T) {
l := mockLock{
acquireCalls: map[string]int{},
}
testCtx := context.Background()
hac := LockLeaser{
locker: &lockHandler{
mockLock: &l,
callerID: "hac-early-return",
},
renewalPeriod: time.Duration(float64(testLease) * lockLeaseRenewalFactor),
waitPeriod: time.Duration(float64(testLease) * lockRetryBackoffFactor),
earlyReturn: true,
}
lock := l.getLockState()
must.False(t, lock.locked)
err := hac.Start(testCtx, func(ctx context.Context) error {
return errors.New("error")
})
must.NoError(t, err)
lock = l.getLockState()
must.False(t, lock.locked)
must.Zero(t, lock.renewsCounter)
}

124
api/retry.go Normal file
View File

@@ -0,0 +1,124 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: MPL-2.0
package api
import (
"context"
"errors"
"net/http"
"time"
)
const (
defaultNumberOfRetries = 5
defaultDelayTimeBase = time.Second
defaultMaxBackoffDelay = 5 * time.Minute
)
type retryOptions struct {
maxRetries int64 // Optional, defaults to 5
// maxBackoffDelay sets a capping value for the delay between calls, to avoid it growing infinitely
maxBackoffDelay time.Duration // Optional, defaults to 5 min
// maxToLastCall sets a capping value for all the retry process, in case there is a deadline to make the call.
maxToLastCall time.Duration // Optional, defaults to 0, meaning no time cap
// fixedDelay is used in case an uniform distribution of the calls is preferred.
fixedDelay time.Duration // Optional, defaults to 0, meaning Delay is exponential, starting at 1sec
// delayBase is used to calculate the starting value at which the delay starts to grow,
// When left empty, a value of 1 sec will be used as base and then the delays will
// grow exponentially with every attempt: starting at 1s, then 2s, 4s, 8s...
delayBase time.Duration // Optional, defaults to 1sec
// maxValidAttempt is used to ensure that a big attempts number or a big delayBase number will not cause
// a negative delay by overflowing the delay increase. Every attempt after the
// maxValid will use the maxBackoffDelay if configured, or the defaultMaxBackoffDelay if not.
maxValidAttempt int64
}
func (c *Client) retryPut(ctx context.Context, endpoint string, in, out any, q *WriteOptions) (*WriteMeta, error) {
var err error
var wm *WriteMeta
attemptDelay := time.Duration(100 * time.Second) // Avoid a tick before starting
startTime := time.Now()
t := time.NewTimer(attemptDelay)
defer t.Stop()
for attempt := int64(0); attempt < c.config.retryOptions.maxRetries+1; attempt++ {
attemptDelay = c.calculateDelay(attempt)
t.Reset(attemptDelay)
select {
case <-ctx.Done():
return nil, ctx.Err()
case <-t.C:
}
wm, err = c.put(endpoint, in, out, q)
// Maximum retry period is up, don't retry
if c.config.retryOptions.maxToLastCall != 0 && time.Now().Sub(startTime) > c.config.retryOptions.maxToLastCall {
break
}
// The put function only returns WriteMetadata if the call was successful
// don't retry
if wm != nil {
break
}
// If WriteMetadata is nil, we need to process the error to decide if a retry is
// necessary or not
var callErr UnexpectedResponseError
ok := errors.As(err, &callErr)
// If is not UnexpectedResponseError, it is an error while performing the call
// don't retry
if !ok {
break
}
// Only 500+ or 429 status calls may be retried, otherwise
// don't retry
if !isCallRetriable(callErr.StatusCode()) {
break
}
}
return wm, err
}
// According to the HTTP protocol, it only makes sense to retry calls
// when the error is caused by a temporary situation, like a server being down
// (500s+) or the call being rate limited (429), this function checks if the
// statusCode is between the errors worth retrying.
func isCallRetriable(statusCode int) bool {
return statusCode > http.StatusInternalServerError &&
statusCode < http.StatusNetworkAuthenticationRequired ||
statusCode == http.StatusTooManyRequests
}
func (c *Client) calculateDelay(attempt int64) time.Duration {
if c.config.retryOptions.fixedDelay != 0 {
return c.config.retryOptions.fixedDelay
}
if attempt == 0 {
return 0
}
if attempt > c.config.retryOptions.maxValidAttempt {
return c.config.retryOptions.maxBackoffDelay
}
newDelay := c.config.retryOptions.delayBase << (attempt - 1)
if c.config.retryOptions.maxBackoffDelay != defaultMaxBackoffDelay &&
newDelay > c.config.retryOptions.maxBackoffDelay {
return c.config.retryOptions.maxBackoffDelay
}
return newDelay
}

129
api/retry_test.go Normal file
View File

@@ -0,0 +1,129 @@
package api
import (
"context"
"encoding/json"
"math"
"net/http"
"net/http/httptest"
"testing"
"time"
"github.com/shoenig/test/must"
)
type mockHandler struct {
callsCounter []time.Time
}
func (mh *mockHandler) Handle(rw http.ResponseWriter, req *http.Request) {
mh.callsCounter = append(mh.callsCounter, time.Now())
// return a populated meta after 7 tries to test he retries stops after a
// successful call
if len(mh.callsCounter) < 7 {
http.Error(rw, http.StatusText(http.StatusBadGateway), http.StatusBadGateway)
return
}
rw.WriteHeader(http.StatusOK)
rw.Header().Set("Content-Type", "application/json")
resp := &WriteMeta{}
jsonResp, _ := json.Marshal(resp)
rw.Write(jsonResp)
return
}
func Test_RetryPut_multiple_calls(t *testing.T) {
t.Run("successfully retries until no error, delayed capped to 100ms", func(t *testing.T) {
mh := mockHandler{
callsCounter: []time.Time{},
}
server := httptest.NewServer(http.HandlerFunc(mh.Handle))
cm := &Client{
httpClient: server.Client(),
config: Config{
Address: server.URL,
retryOptions: &retryOptions{
delayBase: 10 * time.Millisecond,
maxRetries: 10,
maxBackoffDelay: 100 * time.Millisecond,
},
},
}
md, err := cm.retryPut(context.TODO(), "/endpoint", nil, nil, &WriteOptions{})
must.NoError(t, err)
must.Len(t, 7, mh.callsCounter)
must.NotNil(t, md)
must.Greater(t, 10*time.Millisecond, mh.callsCounter[1].Sub(mh.callsCounter[0]))
must.Greater(t, 20*time.Millisecond, mh.callsCounter[2].Sub(mh.callsCounter[1]))
must.Greater(t, 40*time.Millisecond, mh.callsCounter[3].Sub(mh.callsCounter[2]))
must.Greater(t, 80*time.Millisecond, mh.callsCounter[4].Sub(mh.callsCounter[3]))
must.Greater(t, 100*time.Millisecond, mh.callsCounter[5].Sub(mh.callsCounter[4]))
must.Greater(t, 100*time.Millisecond, mh.callsCounter[6].Sub(mh.callsCounter[5]))
})
}
func Test_RetryPut_one_call(t *testing.T) {
t.Run("successfully retries until no error, delayed capped to 100ms", func(t *testing.T) {
mh := mockHandler{
callsCounter: []time.Time{},
}
server := httptest.NewServer(http.HandlerFunc(mh.Handle))
cm := &Client{
httpClient: server.Client(),
config: Config{
Address: server.URL,
retryOptions: &retryOptions{
delayBase: 10 * time.Millisecond,
maxRetries: 1,
},
},
}
md, err := cm.retryPut(context.TODO(), "/endpoint/", nil, nil, &WriteOptions{})
must.Error(t, err)
must.Nil(t, md)
must.Len(t, 2, mh.callsCounter)
})
}
func Test_RetryPut_capped_base_too_big(t *testing.T) {
t.Run("successfully retries until no error, delayed capped to 100ms", func(t *testing.T) {
mh := mockHandler{
callsCounter: []time.Time{},
}
server := httptest.NewServer(http.HandlerFunc(mh.Handle))
cm := &Client{
httpClient: server.Client(),
config: Config{
Address: server.URL,
retryOptions: &retryOptions{
delayBase: math.MaxInt64 * time.Nanosecond,
maxRetries: 3,
maxBackoffDelay: 200 * time.Millisecond,
},
},
}
md, err := cm.retryPut(context.TODO(), "/endpoint", nil, nil, &WriteOptions{})
must.Error(t, err)
must.Len(t, 4, mh.callsCounter)
must.Nil(t, md)
must.Greater(t, cm.config.retryOptions.maxBackoffDelay, mh.callsCounter[1].Sub(mh.callsCounter[0]))
must.Greater(t, cm.config.retryOptions.maxBackoffDelay, mh.callsCounter[2].Sub(mh.callsCounter[1]))
})
}

View File

@@ -183,6 +183,44 @@ func (vars *Variables) GetVariableItems(path string, qo *QueryOptions) (Variable
return v.Items, qm, nil
}
// RenewLock renews the lease for the lock on the given variable. It has to be called
// before the lock's TTL expires or the lock will be automatically released after the
// delay period.
func (vars *Variables) RenewLock(v *Variable, qo *WriteOptions) (*VariableMetadata, *WriteMeta, error) {
v.Path = cleanPathString(v.Path)
var out VariableMetadata
wm, err := vars.client.put("/v1/var/"+v.Path+"?lock-renew", v, &out, qo)
if err != nil {
return nil, wm, err
}
return &out, wm, nil
}
// ReleaseLock removes the lock on the given variable.
func (vars *Variables) ReleaseLock(v *Variable, qo *WriteOptions) (*Variable, *WriteMeta, error) {
return vars.lockOperation(v, qo, "lock-release")
}
// AcquireLock adds a lock on the given variable and starts a lease on it. In order
// to make any update on the locked variable, the lock ID has to be included in the
// request. In order to maintain ownership of the lock, the lease needs to be
// periodically renewed before the lock's TTL expires.
func (vars *Variables) AcquireLock(v *Variable, qo *WriteOptions) (*Variable, *WriteMeta, error) {
return vars.lockOperation(v, qo, "lock-acquire")
}
func (vars *Variables) lockOperation(v *Variable, qo *WriteOptions, operation string) (*Variable, *WriteMeta, error) {
v.Path = cleanPathString(v.Path)
var out Variable
wm, err := vars.client.put("/v1/var/"+v.Path+"?"+operation, v, &out, qo)
if err != nil {
return nil, wm, err
}
return &out, wm, nil
}
// readInternal exists because the API's higher-level read method requires
// the status code to be 200 (OK). For Peek(), we do not consider 403 (Permission
// Denied or 404 (Not Found) an error, this function just returns a nil in those
@@ -358,6 +396,9 @@ type Variable struct {
// Items contains the k/v variable component
Items VariableItems `hcl:"items"`
// Lock holds the information about the variable lock if its being used.
Lock *VariableLock `hcl:",lock,optional" json:",omitempty"`
}
// VariableMetadata specifies the metadata for a variable and
@@ -380,6 +421,24 @@ type VariableMetadata struct {
// ModifyTime is the unix nano of the last modified time
ModifyTime int64 `hcl:"modify_time"`
// Lock holds the information about the variable lock if its being used.
Lock *VariableLock `hcl:",lock,optional" json:",omitempty"`
}
type VariableLock struct {
// ID is generated by Nomad to provide a unique caller ID which can be used
// for renewals and unlocking.
ID string
// TTL describes the time-to-live of the current lock holder.
// This is a string version of a time.Duration like "2m".
TTL string
// LockDelay describes a grace period that exists after a lock is lost,
// before another client may acquire the lock. This helps protect against
// split-brains. This is a string version of a time.Duration like "2m".
LockDelay string
}
// VariableItems are the key/value pairs of a Variable.
@@ -446,6 +505,16 @@ func (v *Variable) AsPrettyJSON() string {
return string(b)
}
// LockID returns the ID of the lock. In the event this is not held, or the
// variable is not a lock, this string will be empty.
func (v *Variable) LockID() string {
if v.Lock == nil {
return ""
}
return v.Lock.ID
}
type ErrCASConflict struct {
CheckIndex uint64
Conflict *Variable

View File

@@ -20,7 +20,7 @@ func TestVariables_SimpleCRUD(t *testing.T) {
defer s.Stop()
nsv := c.Variables()
sv1 := NewVariable("my/first/variable")
sv1 := NewVariable("my/first/variable/SimpleCRUD")
sv1.Namespace = "default"
sv1.Items["k1"] = "v1"
sv1.Items["k2"] = "v2"
@@ -290,7 +290,6 @@ func TestVariables_GetVariableItems(t *testing.T) {
func writeTestVariable(t *testing.T, c *Client, sv *Variable) {
_, err := c.put("/v1/var/"+sv.Path, sv, sv, nil)
must.NoError(t, err, must.Sprint("failed writing test variable"))
must.NoError(t, err, must.Sprint("failed writing test variable"))
}
func TestVariable_CreateReturnsContent(t *testing.T) {
@@ -298,7 +297,7 @@ func TestVariable_CreateReturnsContent(t *testing.T) {
defer s.Stop()
nsv := c.Variables()
sv1 := NewVariable("my/first/variable")
sv1 := NewVariable("my/first/variable/create")
sv1.Namespace = "default"
sv1.Items["k1"] = "v1"
sv1.Items["k2"] = "v2"
@@ -308,3 +307,66 @@ func TestVariable_CreateReturnsContent(t *testing.T) {
must.NotNil(t, sv1n)
must.Eq(t, sv1.Items, sv1n.Items)
}
func TestVariables_LockRenewRelease(t *testing.T) {
testutil.Parallel(t)
c, s := makeClient(t, nil, nil)
defer s.Stop()
path := fmt.Sprintf("%s-%v", "/first/variable", time.Now().UnixMilli())
nsv := c.Variables()
sv1 := NewVariable(path)
sv1.Namespace = "default"
sv1.Items["k1"] = "v1"
sv1.Items["k2"] = "v2"
t.Run("1 create sv1", func(t *testing.T) {
get, _, err := nsv.Create(sv1, nil)
must.NoError(t, err)
must.NotNil(t, get)
must.Positive(t, get.CreateIndex)
must.Positive(t, get.CreateTime)
must.Positive(t, get.ModifyIndex)
must.Positive(t, get.ModifyTime)
must.Eq(t, sv1.Items, get.Items)
*sv1 = *get
})
t.Run("2 acquire lock on sv1", func(t *testing.T) {
get, _, err := nsv.AcquireLock(sv1, nil)
must.NoError(t, err)
must.NotNil(t, get)
must.NotEq(t, sv1.ModifyIndex, get.ModifyIndex)
must.Eq(t, sv1.Items, get.Items)
must.NotNil(t, get.Lock)
*sv1 = *get
})
t.Run("3 renew lock on sv1", func(t *testing.T) {
rlsv := *sv1
get, _, err := nsv.RenewLock(&rlsv, nil)
must.NoError(t, err)
must.NotNil(t, get)
must.Eq(t, sv1.ModifyIndex, get.ModifyIndex)
must.NotNil(t, get.Lock)
must.Eq(t, sv1.Lock.ID, get.Lock.ID)
})
t.Run("4 list vars", func(t *testing.T) {
l, _, err := nsv.List(nil)
must.NoError(t, err)
must.Len(t, 1, l)
must.Nil(t, l[0].Lock)
})
t.Run("5 release lock on sv1", func(t *testing.T) {
sv1.Items = nil
get, _, err := nsv.ReleaseLock(sv1, nil)
must.NoError(t, err)
must.NotNil(t, get)
must.NotEq(t, sv1.ModifyIndex, get.ModifyIndex)
must.Zero(t, len(get.Items))
must.Nil(t, get.Lock)
})
}

View File

@@ -36,6 +36,7 @@
"lib/...",
"nomad/deploymentwatcher/...",
"nomad/drainer/...",
"nomad/lock/...",
"nomad/state/...",
"nomad/stream/...",
"nomad/structs/...",

View File

@@ -4,14 +4,23 @@
package agent
import (
"errors"
"fmt"
"net/http"
"net/url"
"strconv"
"strings"
"github.com/hashicorp/nomad/nomad/structs"
)
var (
renewLockQueryParam = "lock-renew"
acquireLockQueryParam = string(structs.VarOpLockAcquire)
releaseLockQueryParam = string(structs.VarOpLockRelease)
)
func (s *HTTPServer) VariablesListRequest(resp http.ResponseWriter, req *http.Request) (interface{}, error) {
if req.Method != http.MethodGet {
return nil, CodedError(http.StatusMethodNotAllowed, ErrInvalidMethod)
@@ -40,11 +49,33 @@ func (s *HTTPServer) VariableSpecificRequest(resp http.ResponseWriter, req *http
if len(path) == 0 {
return nil, CodedError(http.StatusBadRequest, "missing variable path")
}
switch req.Method {
case http.MethodGet:
return s.variableQuery(resp, req, path)
case http.MethodPut, http.MethodPost:
return s.variableUpsert(resp, req, path)
urlParams := req.URL.Query()
lockOperation, err := getLockOperation(urlParams)
if err != nil {
return nil, CodedError(http.StatusBadRequest, err.Error())
}
cq := req.URL.Query().Get("cas")
if cq != "" && lockOperation != "" {
return nil, CodedError(http.StatusBadRequest, "CAS can't be used with lock operations")
}
if lockOperation == "" {
return s.variableUpsert(resp, req, path)
}
if lockOperation == renewLockQueryParam {
return s.variableLockRenew(resp, req, path)
}
return s.variableLockOperation(resp, req, path, lockOperation)
case http.MethodDelete:
return s.variableDelete(resp, req, path)
default:
@@ -52,6 +83,65 @@ func (s *HTTPServer) VariableSpecificRequest(resp http.ResponseWriter, req *http
}
}
func (s *HTTPServer) variableLockRenew(resp http.ResponseWriter, req *http.Request, path string) (interface{}, error) {
// Parse the Variable
var Variable structs.VariableDecrypted
if err := decodeBody(req, &Variable); err != nil {
return nil, CodedError(http.StatusBadRequest, err.Error())
}
args := structs.VariablesRenewLockRequest{
Path: path,
LockID: Variable.LockID(),
}
s.parseWriteRequest(req, &args.WriteRequest)
var out structs.VariablesRenewLockResponse
if err := s.agent.RPC(structs.VariablesRenewLockRPCMethod, &args, &out); err != nil {
return nil, err
}
return out.VarMeta, nil
}
func (s *HTTPServer) variableLockOperation(resp http.ResponseWriter, req *http.Request,
path, operation string) (interface{}, error) {
// Parse the Variable
var Variable structs.VariableDecrypted
if err := decodeBody(req, &Variable); err != nil {
return nil, CodedError(http.StatusBadRequest, err.Error())
}
// At this point, the operation can be either acquire or release, and they are
// both handled by the VariablesApplyRPCMethod.
args := structs.VariablesApplyRequest{
Op: structs.VarOp(operation),
Var: &Variable,
}
Variable.Path = path
s.parseWriteRequest(req, &args.WriteRequest)
var out structs.VariablesApplyResponse
err := s.agent.RPC(structs.VariablesApplyRPCMethod, &args, &out)
defer setIndex(resp, out.WriteMeta.Index)
if err != nil {
return nil, err
}
if out.Conflict != nil {
resp.WriteHeader(http.StatusConflict)
return out.Conflict, nil
}
// Finally, we know that this is a success response, send it to the caller
return out.Output, nil
}
func (s *HTTPServer) variableQuery(resp http.ResponseWriter, req *http.Request,
path string) (interface{}, error) {
args := structs.VariablesReadRequest{
@@ -75,11 +165,13 @@ func (s *HTTPServer) variableQuery(resp http.ResponseWriter, req *http.Request,
func (s *HTTPServer) variableUpsert(resp http.ResponseWriter, req *http.Request,
path string) (interface{}, error) {
// Parse the Variable
var Variable structs.VariableDecrypted
if err := decodeBody(req, &Variable); err != nil {
return nil, CodedError(http.StatusBadRequest, err.Error())
}
if len(Variable.Items) == 0 {
return nil, CodedError(http.StatusBadRequest, "variable missing required Items object")
}
@@ -186,3 +278,34 @@ func parseCAS(req *http.Request) (bool, uint64, error) {
}
return false, 0, nil
}
func isOneAndOnlyOneSet(a, b, c bool) bool {
return (a || b || c) && !a != !b != !c != !(a && b && c)
}
// getLockOperation returns the lock operation to be performed in case there is
// one. It returns error if more than one is set.
func getLockOperation(queryParams url.Values) (string, error) {
_, renewLock := queryParams[renewLockQueryParam]
_, acquireLock := queryParams[acquireLockQueryParam]
_, releaseLock := queryParams[releaseLockQueryParam]
if !renewLock && !acquireLock && !releaseLock {
return "", nil
}
if !isOneAndOnlyOneSet(renewLock, acquireLock, releaseLock) {
return "", errors.New("multiple lock operations")
}
switch {
case renewLock:
return renewLockQueryParam, nil
case acquireLock:
return acquireLockQueryParam, nil
case releaseLock:
return releaseLockQueryParam, nil
default:
return "", errors.New("unspecified lock operation")
}
}

View File

@@ -6,6 +6,7 @@ package agent
import (
"bytes"
"encoding/json"
"errors"
"fmt"
"io"
"net/http"
@@ -15,6 +16,7 @@ import (
"github.com/hashicorp/nomad/ci"
"github.com/hashicorp/nomad/nomad/mock"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/shoenig/test/must"
"github.com/stretchr/testify/require"
)
@@ -35,26 +37,27 @@ func TestHTTP_Variables(t *testing.T) {
// the costs of server startup and allow as much parallelization as possible.
t.Run("error_badverb_list", func(t *testing.T) {
req, err := http.NewRequest("LOLWUT", "/v1/vars", nil)
require.NoError(t, err)
must.NoError(t, err)
respW := httptest.NewRecorder()
_, err = s.Server.VariablesListRequest(respW, req)
require.EqualError(t, err, ErrInvalidMethod)
must.ErrorContains(t, err, ErrInvalidMethod)
})
t.Run("error_parse_list", func(t *testing.T) {
req, err := http.NewRequest(http.MethodGet, "/v1/vars?wait=99a", nil)
require.NoError(t, err)
respW := httptest.NewRecorder()
_, _ = s.Server.VariablesListRequest(respW, req)
require.Equal(t, http.StatusBadRequest, respW.Code)
require.Equal(t, "Invalid wait time", string(respW.Body.Bytes()))
must.Eq(t, http.StatusBadRequest, respW.Code)
must.Eq(t, "Invalid wait time", string(respW.Body.Bytes()))
})
t.Run("error_rpc_list", func(t *testing.T) {
req, err := http.NewRequest(http.MethodGet, "/v1/vars?region=bad", nil)
require.NoError(t, err)
respW := httptest.NewRecorder()
obj, err := s.Server.VariablesListRequest(respW, req)
require.EqualError(t, err, "No path to region")
require.Nil(t, obj)
must.ErrorContains(t, err, "No path to region")
must.Nil(t, obj)
})
t.Run("list", func(t *testing.T) {
// Test the empty list case
@@ -64,14 +67,14 @@ func TestHTTP_Variables(t *testing.T) {
// Make the request
obj, err := s.Server.VariablesListRequest(respW, req)
require.NoError(t, err)
must.NoError(t, err)
// add vars and test a populated backend
svMap := mock.Variables(4, 4)
svs := svMap.List()
svs[3].Path = svs[0].Path + "/child"
for _, sv := range svs {
require.NoError(t, rpcWriteSV(s, sv, nil))
must.NoError(t, rpcWriteSV(s, sv, nil))
}
// Make the HTTP request
@@ -81,15 +84,16 @@ func TestHTTP_Variables(t *testing.T) {
// Make the request
obj, err = s.Server.VariablesListRequest(respW, req)
require.NoError(t, err)
must.NoError(t, err)
// Check for the index
require.NotZero(t, respW.HeaderMap.Get("X-Nomad-Index"))
require.Equal(t, "true", respW.HeaderMap.Get("X-Nomad-KnownLeader"))
require.NotZero(t, respW.HeaderMap.Get("X-Nomad-LastContact"))
must.NonZero(t, len(respW.HeaderMap.Get("X-Nomad-Index")))
must.Eq(t, "true", respW.HeaderMap.Get("X-Nomad-KnownLeader"))
must.NonZero(t, len(respW.HeaderMap.Get("X-Nomad-LastContact")))
// Check the output (the 4 we register )
require.Len(t, obj.([]*structs.VariableMetadata), 4)
must.Len(t, 4, obj.([]*structs.VariableMetadata))
// test prefix query
req, err = http.NewRequest(http.MethodGet, "/v1/vars?prefix="+svs[0].Path, nil)
@@ -98,34 +102,34 @@ func TestHTTP_Variables(t *testing.T) {
// Make the request
obj, err = s.Server.VariablesListRequest(respW, req)
require.NoError(t, err)
require.Len(t, obj.([]*structs.VariableMetadata), 2)
must.NoError(t, err)
must.Len(t, 2, obj.([]*structs.VariableMetadata))
})
rpcResetSV(s)
t.Run("error_badverb_query", func(t *testing.T) {
req, err := http.NewRequest("LOLWUT", "/v1/var/does/not/exist", nil)
require.NoError(t, err)
must.NoError(t, err)
respW := httptest.NewRecorder()
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.EqualError(t, err, ErrInvalidMethod)
require.Nil(t, obj)
must.ErrorContains(t, err, ErrInvalidMethod)
must.Nil(t, obj)
})
t.Run("error_parse_query", func(t *testing.T) {
req, err := http.NewRequest(http.MethodGet, "/v1/var/does/not/exist?wait=99a", nil)
require.NoError(t, err)
respW := httptest.NewRecorder()
_, _ = s.Server.VariableSpecificRequest(respW, req)
require.Equal(t, http.StatusBadRequest, respW.Code)
require.Equal(t, "Invalid wait time", string(respW.Body.Bytes()))
must.Eq(t, http.StatusBadRequest, respW.Code)
must.Eq(t, "Invalid wait time", string(respW.Body.Bytes()))
})
t.Run("error_rpc_query", func(t *testing.T) {
req, err := http.NewRequest(http.MethodGet, "/v1/var/does/not/exist?region=bad", nil)
require.NoError(t, err)
respW := httptest.NewRecorder()
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.EqualError(t, err, "No path to region")
require.Nil(t, obj)
must.ErrorContains(t, err, "No path to region")
must.Nil(t, obj)
})
t.Run("query_unset_path", func(t *testing.T) {
// Make a request for a non-existing variable
@@ -133,8 +137,8 @@ func TestHTTP_Variables(t *testing.T) {
require.NoError(t, err)
respW := httptest.NewRecorder()
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.EqualError(t, err, "missing variable path")
require.Nil(t, obj)
must.ErrorContains(t, err, "missing variable path")
must.Nil(t, obj)
})
t.Run("query_unset_variable", func(t *testing.T) {
// Make a request for a non-existing variable
@@ -142,29 +146,29 @@ func TestHTTP_Variables(t *testing.T) {
require.NoError(t, err)
respW := httptest.NewRecorder()
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.EqualError(t, err, "variable not found")
require.Nil(t, obj)
must.ErrorContains(t, err, "variable not found")
must.Nil(t, obj)
})
t.Run("query", func(t *testing.T) {
// Use RPC to make a test variable
out := new(structs.VariableDecrypted)
sv1 := mock.Variable()
require.NoError(t, rpcWriteSV(s, sv1, out))
must.NoError(t, rpcWriteSV(s, sv1, out))
// Query a variable
req, err := http.NewRequest(http.MethodGet, "/v1/var/"+sv1.Path, nil)
require.NoError(t, err)
respW := httptest.NewRecorder()
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.NoError(t, err)
must.NoError(t, err)
// Check for the index
require.NotZero(t, respW.HeaderMap.Get("X-Nomad-Index"))
require.Equal(t, "true", respW.HeaderMap.Get("X-Nomad-KnownLeader"))
require.NotZero(t, respW.HeaderMap.Get("X-Nomad-LastContact"))
must.NonZero(t, len(respW.HeaderMap.Get("X-Nomad-Index")))
must.Eq(t, "true", respW.HeaderMap.Get("X-Nomad-KnownLeader"))
must.NonZero(t, len(respW.HeaderMap.Get("X-Nomad-LastContact")))
// Check the output
require.Equal(t, out, obj.(*structs.VariableDecrypted))
must.Eq(t, out, obj.(*structs.VariableDecrypted))
})
rpcResetSV(s)
@@ -175,8 +179,8 @@ func TestHTTP_Variables(t *testing.T) {
require.NoError(t, err)
respW := httptest.NewRecorder()
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.EqualError(t, err, "unexpected EOF")
require.Nil(t, obj)
must.ErrorContains(t, err, "unexpected EOF")
must.Nil(t, obj)
})
t.Run("error_rpc_create", func(t *testing.T) {
buf := encodeReq(sv1)
@@ -184,8 +188,8 @@ func TestHTTP_Variables(t *testing.T) {
require.NoError(t, err)
respW := httptest.NewRecorder()
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.EqualError(t, err, "No path to region")
require.Nil(t, obj)
must.ErrorContains(t, err, "No path to region")
must.Nil(t, obj)
})
t.Run("create_no_items", func(t *testing.T) {
sv2 := sv1.Copy()
@@ -195,8 +199,8 @@ func TestHTTP_Variables(t *testing.T) {
require.NoError(t, err)
respW := httptest.NewRecorder()
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.EqualError(t, err, "variable missing required Items object")
require.Nil(t, obj)
must.ErrorContains(t, err, "variable missing required Items object")
must.Nil(t, obj)
})
t.Run("create", func(t *testing.T) {
buf := encodeReq(sv1)
@@ -204,23 +208,23 @@ func TestHTTP_Variables(t *testing.T) {
require.NoError(t, err)
respW := httptest.NewRecorder()
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.NoError(t, err)
must.NoError(t, err)
// Test the returned object and rehydrate to a VariableDecrypted
require.NotNil(t, obj)
must.NotNil(t, obj)
sv1, ok := obj.(*structs.VariableDecrypted)
require.True(t, ok, "Unable to convert obj to VariableDecrypted")
must.True(t, ok, must.Sprint(must.Sprint("Unable to convert obj to VariableDecrypted")))
// Check for the index
require.NotZero(t, respW.HeaderMap.Get("X-Nomad-Index"))
require.Equal(t, fmt.Sprint(sv1.ModifyIndex), respW.HeaderMap.Get("X-Nomad-Index"))
must.NonZero(t, len(respW.HeaderMap.Get("X-Nomad-Index")))
must.Eq(t, fmt.Sprint(sv1.ModifyIndex), respW.HeaderMap.Get("X-Nomad-Index"))
// Check the variable was put and that the returned item matched the
// fetched value
out, err := rpcReadSV(s, sv1.Namespace, sv1.Path)
require.NoError(t, err)
require.NotNil(t, out)
require.Equal(t, sv1, out)
must.NoError(t, err)
must.NotNil(t, out)
must.Eq(t, sv1, out)
})
rpcResetSV(s)
@@ -237,11 +241,14 @@ func TestHTTP_Variables(t *testing.T) {
// Make the request
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.EqualError(t, err, "unexpected EOF")
must.ErrorContains(t, err, "unexpected EOF")
var cErr HTTPCodedError
require.ErrorAs(t, err, &cErr)
require.Equal(t, http.StatusBadRequest, cErr.Code())
require.Nil(t, obj)
if !errors.As(err, &cErr) {
t.Fatalf("unexpected error")
}
must.Eq(t, http.StatusBadRequest, cErr.Code())
must.Nil(t, obj)
})
t.Run("error_rpc_update", func(t *testing.T) {
sv1U := sv1.Copy()
@@ -255,12 +262,12 @@ func TestHTTP_Variables(t *testing.T) {
// Make the request
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.EqualError(t, err, "No path to region")
require.Nil(t, obj)
must.ErrorContains(t, err, "No path to region")
must.Nil(t, obj)
})
t.Run("update", func(t *testing.T) {
sv := mock.Variable()
require.NoError(t, rpcWriteSV(s, sv, sv))
must.NoError(t, rpcWriteSV(s, sv, sv))
svU := sv.Copy()
svU.Items["new"] = "new"
@@ -272,31 +279,32 @@ func TestHTTP_Variables(t *testing.T) {
// Make the request
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.NoError(t, err)
must.NoError(t, err)
// Test the returned object and rehydrate to a VariableDecrypted
require.NotNil(t, obj)
must.NotNil(t, obj)
out, ok := obj.(*structs.VariableDecrypted)
require.True(t, ok, "Unable to convert obj to VariableDecrypted")
must.True(t, ok, must.Sprint("Unable to convert obj to VariableDecrypted"))
// Check for the index
require.NotZero(t, respW.HeaderMap.Get("X-Nomad-Index"))
require.Equal(t, fmt.Sprint(out.ModifyIndex), respW.HeaderMap.Get("X-Nomad-Index"))
must.NonZero(t, len(respW.HeaderMap.Get("X-Nomad-Index")))
must.Eq(t, fmt.Sprint(out.ModifyIndex), respW.HeaderMap.Get("X-Nomad-Index"))
{
// Check that written varible does not equal the input to rule out input mutation
require.NotEqual(t, &svU.VariableMetadata, out.VariableMetadata)
must.NotEqual(t, svU.VariableMetadata, out.VariableMetadata)
// Update the input token with the updated metadata so that we
// can use a simple equality check
svU.ModifyIndex = out.ModifyIndex
svU.ModifyTime = out.ModifyTime
require.Equal(t, &svU, out)
must.Eq(t, &svU, out)
}
})
t.Run("update-cas", func(t *testing.T) {
t.Run("update_cas", func(t *testing.T) {
sv := mock.Variable()
require.NoError(t, rpcWriteSV(s, sv, sv))
must.NoError(t, rpcWriteSV(s, sv, sv))
svU := sv.Copy()
svU.Items["new"] = "new"
@@ -310,23 +318,23 @@ func TestHTTP_Variables(t *testing.T) {
// Make the request
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.NoError(t, err)
require.Equal(t, http.StatusConflict, respW.Result().StatusCode)
must.NoError(t, err)
must.Eq(t, http.StatusConflict, respW.Result().StatusCode)
// Evaluate the conflict variable
require.NotNil(t, obj)
must.NotNil(t, obj)
conflict, ok := obj.(*structs.VariableDecrypted)
require.True(t, ok, "Expected *structs.VariableDecrypted, got %T", obj)
require.Equal(t, conflict, sv)
must.True(t, ok, must.Sprintf("Expected *structs.VariableDecrypted, got %T", obj))
must.Eq(t, conflict, sv)
// Check for the index
require.NotZero(t, respW.HeaderMap.Get("X-Nomad-Index"))
must.NonZero(t, len(respW.HeaderMap.Get("X-Nomad-Index")))
}
// Check the variable was not updated
{
out, err := rpcReadSV(s, sv.Namespace, sv.Path)
require.NoError(t, err)
require.Equal(t, sv, out)
must.NoError(t, err)
must.Eq(t, sv, out)
}
// Make the HTTP request
{
@@ -337,49 +345,213 @@ func TestHTTP_Variables(t *testing.T) {
// Make the request
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.NoError(t, err)
must.NoError(t, err)
// Test the returned object and rehydrate to a VariableDecrypted
require.NotNil(t, obj)
must.NotNil(t, obj)
sv1, ok := obj.(*structs.VariableDecrypted)
require.True(t, ok, "Unable to convert obj to VariableDecrypted")
must.True(t, ok, must.Sprint("Unable to convert obj to VariableDecrypted"))
// Check for the index
require.NotZero(t, respW.HeaderMap.Get("X-Nomad-Index"))
require.Equal(t, fmt.Sprint(sv1.ModifyIndex), respW.HeaderMap.Get("X-Nomad-Index"))
must.NonZero(t, len(respW.HeaderMap.Get("X-Nomad-Index")))
must.Eq(t, fmt.Sprint(sv1.ModifyIndex), respW.HeaderMap.Get("X-Nomad-Index"))
// Check the variable was put and that the returned item matched the
// fetched value
out, err := rpcReadSV(s, sv.Namespace, sv.Path)
require.NoError(t, err)
require.NotNil(t, out)
require.Equal(t, sv1, out)
must.NoError(t, err)
must.NotNil(t, out)
must.Eq(t, sv1, out)
}
// Check the variable was created correctly
{
out, err := rpcReadSV(s, sv.Namespace, sv.Path)
require.NoError(t, err)
require.NotNil(t, out)
must.NoError(t, err)
must.NotNil(t, out)
require.NotEqual(t, sv, out)
require.NotEqual(t, svU.VariableMetadata, out.VariableMetadata)
must.NotEq(t, sv, out)
must.NotEqual(t, svU.VariableMetadata, out.VariableMetadata)
// Update the input token with the updated metadata so that we
// can use a simple equality check
svU.CreateIndex, svU.ModifyIndex = out.CreateIndex, out.ModifyIndex
svU.CreateTime, svU.ModifyTime = out.CreateTime, out.ModifyTime
require.Equal(t, svU.VariableMetadata, out.VariableMetadata)
must.Eq(t, svU.VariableMetadata, out.VariableMetadata)
// fmt writes sorted output of maps for testability.
require.Equal(t, fmt.Sprint(svU.Items), fmt.Sprint(out.Items))
must.Eq(t, fmt.Sprint(svU.Items), fmt.Sprint(out.Items))
}
})
rpcResetSV(s)
t.Run("error_cas_and_acquire_lock", func(t *testing.T) {
svLA := sv1.Copy()
svLA.Items["new"] = "new"
// break the request body
badBuf := encodeBrokenReq(&svLA)
req, err := http.NewRequest("PUT", "/v1/var/"+sv1.Path+"?cas=1&"+acquireLockQueryParam, badBuf)
must.NoError(t, err)
respW := httptest.NewRecorder()
// Make the request
obj, err := s.Server.VariableSpecificRequest(respW, req)
must.ErrorContains(t, err, "CAS can't be used with lock operations")
var cErr HTTPCodedError
if !errors.As(err, &cErr) {
t.Fatalf("unexpected error")
}
must.Eq(t, http.StatusBadRequest, cErr.Code())
must.Nil(t, obj)
})
t.Run("error_parse_acquire_lock", func(t *testing.T) {
svLA := sv1.Copy()
svLA.Items["new"] = "new"
// break the request body
badBuf := encodeBrokenReq(&svLA)
req, err := http.NewRequest("PUT", "/v1/var/"+sv1.Path+"?"+acquireLockQueryParam, badBuf)
must.NoError(t, err)
respW := httptest.NewRecorder()
// Make the request
obj, err := s.Server.VariableSpecificRequest(respW, req)
must.ErrorContains(t, err, "unexpected EOF")
var cErr HTTPCodedError
if !errors.As(err, &cErr) {
t.Fatalf("unexpected error")
}
must.Eq(t, http.StatusBadRequest, cErr.Code())
must.Nil(t, obj)
})
t.Run("error_rpc_acquire_lock", func(t *testing.T) {
svLA := sv1.Copy()
svLA.Items["new"] = "new"
// test broken rpc error
buf := encodeReq(&svLA)
req, err := http.NewRequest("PUT", "/v1/var/"+sv1.Path+"?region=bad&"+acquireLockQueryParam, buf)
must.NoError(t, err)
respW := httptest.NewRecorder()
// Make the request
obj, err := s.Server.VariableSpecificRequest(respW, req)
must.ErrorContains(t, err, "No path to region")
must.Nil(t, obj)
})
t.Run("acquire_lock", func(t *testing.T) {
svLA := sv1
svLA.Items["new"] = "new"
// Make the HTTP request
buf := encodeReq(&svLA)
req, err := http.NewRequest("PUT", "/v1/var/"+svLA.Path+"?"+acquireLockQueryParam, buf)
must.NoError(t, err)
respW := httptest.NewRecorder()
obj, err := s.Server.VariableSpecificRequest(respW, req)
must.NoError(t, err)
// Test the returned object and rehydrate to a VariableDecrypted
must.NotNil(t, obj)
out, ok := obj.(*structs.VariableDecrypted)
must.True(t, ok, must.Sprint("Unable to convert obj to VariableDecrypted"))
// Check for the index
must.NonZero(t, len(respW.HeaderMap.Get("X-Nomad-Index")))
must.Eq(t, fmt.Sprint(out.ModifyIndex), respW.HeaderMap.Get("X-Nomad-Index"))
// Check for the lock
must.NotNil(t, out.VariableMetadata.Lock)
must.NonZero(t, len(out.LockID()))
// Check that written varible includes the new items
must.Eq(t, svLA.Items, out.Items)
// Update the lock information for the following tests
sv1.VariableMetadata = out.VariableMetadata
})
t.Run("error_rpc_renew_lock", func(t *testing.T) {
svRL := sv1.Copy()
// test broken rpc error
buf := encodeReq(&svRL)
req, err := http.NewRequest("PUT", "/v1/var/"+sv1.Path+"?region=bad&"+renewLockQueryParam, buf)
must.NoError(t, err)
respW := httptest.NewRecorder()
// Make the request
obj, err := s.Server.VariableSpecificRequest(respW, req)
must.ErrorContains(t, err, "No path to region")
must.Nil(t, obj)
})
t.Run("renew_lock", func(t *testing.T) {
svRL := sv1.Copy()
// Make the HTTP request
buf := encodeReq(&svRL)
req, err := http.NewRequest("PUT", "/v1/var/"+svRL.Path+"?"+renewLockQueryParam, buf)
must.NoError(t, err)
respW := httptest.NewRecorder()
obj, err := s.Server.VariableSpecificRequest(respW, req)
must.NoError(t, err)
// Test the returned object and rehydrate to a VariableDecrypted
must.NotNil(t, obj)
out, ok := obj.(*structs.VariableMetadata)
must.True(t, ok, must.Sprint("Unable to convert obj to VariableDecrypted"))
// Check for the lock
must.NotNil(t, out.Lock)
must.Eq(t, sv1.LockID(), out.Lock.ID)
})
t.Run("release_lock", func(t *testing.T) {
svLR := *sv1
svLR.Items = nil
// Make the HTTP request
buf := encodeReq(&svLR)
req, err := http.NewRequest("PUT", "/v1/var/"+svLR.Path+"?"+releaseLockQueryParam, buf)
must.NoError(t, err)
respW := httptest.NewRecorder()
obj, err := s.Server.VariableSpecificRequest(respW, req)
must.NoError(t, err)
// Test the returned object and rehydrate to a VariableDecrypted
must.NotNil(t, obj)
out, ok := obj.(*structs.VariableDecrypted)
must.True(t, ok, must.Sprint("Unable to convert obj to VariableDecrypted"))
// Check for the index
must.NonZero(t, len(respW.HeaderMap.Get("X-Nomad-Index")))
must.Eq(t, fmt.Sprint(out.ModifyIndex), respW.HeaderMap.Get("X-Nomad-Index"))
// Check for the lock
must.Nil(t, out.VariableMetadata.Lock)
// Check that written variable is equal the input
must.Zero(t, len(out.Items))
// Remove the lock information from the mock variable for the following tests
sv1.VariableMetadata = out.VariableMetadata
})
t.Run("error_rpc_delete", func(t *testing.T) {
sv1 := mock.Variable()
require.NoError(t, rpcWriteSV(s, sv1, nil))
must.NoError(t, rpcWriteSV(s, sv1, nil))
// Make the HTTP request
req, err := http.NewRequest(http.MethodDelete, "/v1/var/"+sv1.Path+"?region=bad", nil)
@@ -388,14 +560,14 @@ func TestHTTP_Variables(t *testing.T) {
// Make the request
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.EqualError(t, err, "No path to region")
require.Nil(t, obj)
must.ErrorContains(t, err, "No path to region")
must.Nil(t, obj)
})
t.Run("delete-cas", func(t *testing.T) {
sv := mock.Variable()
require.NoError(t, rpcWriteSV(s, sv, nil))
must.NoError(t, rpcWriteSV(s, sv, nil))
sv, err := rpcReadSV(s, sv.Namespace, sv.Path)
require.NoError(t, err)
must.NoError(t, err)
// Make the HTTP request
{
@@ -405,25 +577,25 @@ func TestHTTP_Variables(t *testing.T) {
// Make the request
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.NoError(t, err)
require.Equal(t, http.StatusConflict, respW.Result().StatusCode)
must.NoError(t, err)
must.Eq(t, http.StatusConflict, respW.Result().StatusCode)
// Evaluate the conflict variable
require.NotNil(t, obj)
must.NotNil(t, obj)
conflict, ok := obj.(*structs.VariableDecrypted)
require.True(t, ok, "Expected *structs.VariableDecrypted, got %T", obj)
require.True(t, sv.Equal(*conflict))
must.True(t, ok, must.Sprintf("Expected *structs.VariableDecrypted, got %T", obj))
must.True(t, sv.Equal(*conflict))
// Check for the index
require.NotZero(t, respW.HeaderMap.Get("X-Nomad-Index"))
must.NonZero(t, len(respW.HeaderMap.Get("X-Nomad-Index")))
}
// Check variable was not deleted
{
svChk, err := rpcReadSV(s, sv.Namespace, sv.Path)
require.NoError(t, err)
require.NotNil(t, svChk)
require.Equal(t, sv, svChk)
must.NoError(t, err)
must.NotNil(t, svChk)
must.Eq(t, sv, svChk)
}
// Make the HTTP request
{
@@ -433,19 +605,19 @@ func TestHTTP_Variables(t *testing.T) {
// Make the request
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.NoError(t, err)
require.Nil(t, obj)
must.NoError(t, err)
must.Nil(t, obj)
}
// Check variable was deleted
{
svChk, err := rpcReadSV(s, sv.Namespace, sv.Path)
require.NoError(t, err)
require.Nil(t, svChk)
must.NoError(t, err)
must.Nil(t, svChk)
}
})
t.Run("delete", func(t *testing.T) {
sv1 := mock.Variable()
require.NoError(t, rpcWriteSV(s, sv1, nil))
must.NoError(t, rpcWriteSV(s, sv1, nil))
// Make the HTTP request
req, err := http.NewRequest(http.MethodDelete, "/v1/var/"+sv1.Path, nil)
@@ -454,17 +626,27 @@ func TestHTTP_Variables(t *testing.T) {
// Make the request
obj, err := s.Server.VariableSpecificRequest(respW, req)
require.NoError(t, err)
require.Nil(t, obj)
must.NoError(t, err)
must.Nil(t, obj)
// Check for the index
require.NotZero(t, respW.HeaderMap.Get("X-Nomad-Index"))
require.Equal(t, http.StatusNoContent, respW.Result().StatusCode)
must.NonZero(t, len(respW.HeaderMap.Get("X-Nomad-Index")))
must.Eq(t, http.StatusNoContent, respW.Result().StatusCode)
// Check variable was deleted
sv, err := rpcReadSV(s, sv1.Namespace, sv1.Path)
require.NoError(t, err)
require.Nil(t, sv)
must.NoError(t, err)
must.Nil(t, sv)
})
// WIP
t.Run("error_parse_lock_acquire", func(t *testing.T) {
req, err := http.NewRequest("GET", "/v1/var/does/not/exist?wait=99a&lock=acquire", nil)
must.NoError(t, err)
respW := httptest.NewRecorder()
_, _ = s.Server.VariableSpecificRequest(respW, req)
must.Eq(t, http.StatusBadRequest, respW.Code)
must.Eq(t, "Invalid wait time", string(respW.Body.Bytes()))
})
})
}
@@ -521,7 +703,7 @@ func rpcResetSV(s *TestAgent) {
},
}
err := s.Agent.RPC(structs.VariablesListRPCMethod, &lArgs, &lResp)
require.NoError(s.T, err)
must.NoError(s.T, err)
dArgs := structs.VariablesApplyRequest{
Op: structs.VarOpDelete,
@@ -537,10 +719,10 @@ func rpcResetSV(s *TestAgent) {
dArgs.Var.Path = v.Path
dArgs.Var.Namespace = v.Namespace
err := s.Agent.RPC(structs.VariablesApplyRPCMethod, &dArgs, &dResp)
require.NoError(s.T, err)
must.NoError(s.T, err)
}
err = s.Agent.RPC(structs.VariablesListRPCMethod, &lArgs, &lResp)
require.NoError(s.T, err)
require.Equal(s.T, 0, len(lResp.Data))
must.NoError(s.T, err)
must.Eq(s.T, 0, len(lResp.Data))
}

View File

@@ -1121,6 +1121,13 @@ func Commands(metaPtr *Meta, agentUi cli.Ui) map[string]cli.CommandFactory {
Meta: meta,
}, nil
},
"var lock": func() (cli.Command, error) {
return &VarLockCommand{
varPutCommand: &VarPutCommand{
Meta: meta,
},
}, nil
},
"var get": func() (cli.Command, error) {
return &VarGetCommand{
Meta: meta,

369
command/var_lock.go Normal file
View File

@@ -0,0 +1,369 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: BUSL-1.1
package command
import (
"context"
"fmt"
"os"
"os/exec"
"os/signal"
"strings"
"syscall"
"time"
"github.com/hashicorp/nomad/api"
"github.com/posener/complete"
)
const defaultMaxClientRetries = 5
type VarLockCommand struct {
shell bool
inFmt string
ttl string
lockDelay string
varPutCommand *VarPutCommand
}
func (c *VarLockCommand) Help() string {
helpText := `
Usage:
nomad var lock [options] <lock spec file reference> child...
nomad var lock [options] <path to store variable> [<variable spec file reference>] child...
The lock command provides a mechanism for simple distributed locking. A lock
is created in the given variable, and only when held, is a child process invoked.
The lock command can be called on an existing variable or an entire new variable
specification can be provided to the command from a file by using an
@-prefixed path to a variable specification file. Items to be stored in the
variable can be supplied using the specification file as well.
Nomad lock launches its children in a shell. By default, Nomad will use the
shell defined in the environment variable SHELL. If SHELL is not defined,
it will default to /bin/sh. It should be noted that not all shells terminate
child processes when they receive SIGTERM. Under Ubuntu, /bin/sh is linked
to dash, which does not terminate its children. In order to ensure that
child processes are killed when the lock is lost, be sure to set the SHELL
environment variable appropriately, or run without a shell by setting -shell=false.
If ACLs are enabled, this command requires the 'variables:write' capability
for the destination namespace and path.
General Options:
` + generalOptionsUsage(usageOptsDefault) + `
Var lock Options:
-verbose
Provides additional information via standard error to preserve standard
output (stdout) for redirected output.
-ttl
Optional, TTL for the lock, time the variable will be locked. Defaults to 15s.
-delay
Optional, time the variable is blocked from locking when a lease is not renewed.
Defaults to 15s.
-max-retry
Optional, max-retry up to this number of times if Nomad returns a 500 error
while monitoring the lock. This allows riding out brief periods of
unavailability without causing leader elections, but increases the amount of
time required to detect a lost lock in some cases. Defaults to 5. Set to 0 to
disable.
-early-return
Optional, early-return indicates the command to return if the lock is not
acquired instead of waiting on stand by to try again. Defaults to false.
-backoff
Optional, indicates how long to wait between attempts to obtain the lock.
By default the lease algorithm waits for 1.1 times the lock TTL.
-shell
Optional, use a shell to run the command (can set a custom shell via
the SHELL environment variable). The default value is true.
`
return strings.TrimSpace(helpText)
}
func (c *VarLockCommand) AutocompleteFlags() complete.Flags {
return mergeAutocompleteFlags(c.varPutCommand.Meta.AutocompleteFlags(FlagSetClient),
complete.Flags{},
)
}
func (c *VarLockCommand) AutocompleteArgs() complete.Predictor {
return VariablePathPredictor(c.varPutCommand.Meta.Client)
}
func (c *VarLockCommand) Synopsis() string {
return "Put a lock on a variable and run a child command if operation is successful"
}
func (c *VarLockCommand) Name() string { return "var lock" }
func (c *VarLockCommand) Run(args []string) int {
var doVerbose bool
var err error
var path string
var maxRetry int64
var earlyReturn bool
var backoff time.Duration
flags := c.varPutCommand.Meta.FlagSet(c.Name(), FlagSetClient)
flags.Usage = func() { c.varPutCommand.Ui.Output(c.Help()) }
flags.BoolVar(&doVerbose, "verbose", false, "")
flags.StringVar(&c.ttl, "ttl", "", "")
flags.StringVar(&c.lockDelay, "delay", "", "")
flags.BoolVar(&c.shell, "shell", true, "")
flags.BoolVar(&earlyReturn, "early-return", false, "")
flags.Int64Var(&maxRetry, "max-retry", 5, "")
flags.DurationVar(&backoff, "backoff", 0, "")
if fileInfo, _ := os.Stdout.Stat(); (fileInfo.Mode() & os.ModeCharDevice) != 0 {
flags.StringVar(&c.varPutCommand.outFmt, "out", "none", "")
} else {
flags.StringVar(&c.varPutCommand.outFmt, "out", "json", "")
}
if err := flags.Parse(args); err != nil {
c.varPutCommand.Ui.Error(commandErrorText(c))
return 1
}
args = flags.Args()
// Manage verbose output
verbose := func(_ string) {} //no-op
if doVerbose {
verbose = func(msg string) {
c.varPutCommand.Ui.Warn(msg)
}
}
c.varPutCommand.verbose = verbose
if c.varPutCommand.Meta.namespace == api.AllNamespacesNamespace {
c.varPutCommand.Ui.Error(errWildcardNamespaceNotAllowed)
return 1
}
if len(args) < 2 {
c.varPutCommand.Ui.Error(fmt.Sprintf("Not enough arguments (expected >2, got %d)", len(args)))
return 1
}
path, args, err = c.readPathFromArgs(args)
if err != nil {
c.varPutCommand.Ui.Error(err.Error())
return 1
}
sv, err := c.varPutCommand.makeVariable(path)
if err != nil {
c.varPutCommand.Ui.Error(fmt.Sprintf("Failed to parse variable data: %s", err))
return 1
}
if sv.Lock == nil {
if c.ttl == "" && c.lockDelay == "" {
c.varPutCommand.verbose("Using defaults for the lock")
}
sv.Lock = &api.VariableLock{
TTL: api.DefaultLockTTL.String(),
LockDelay: api.DefaultLockDelay.String(),
}
}
if c.ttl != "" {
c.varPutCommand.verbose("Using TTL for the lock of " + c.ttl)
_, err := time.ParseDuration(c.ttl)
if err != nil {
c.varPutCommand.Ui.Error(fmt.Sprintf("Invalid TTL: %s", err))
return 1
}
sv.Lock.TTL = c.ttl
}
if c.lockDelay != "" {
c.varPutCommand.verbose("Using delay for the lock of " + c.lockDelay)
_, err := time.ParseDuration(c.ttl)
if err != nil {
c.varPutCommand.Ui.Error(fmt.Sprintf("Invalid Lock Delay: %s", err))
return 1
}
sv.Lock.LockDelay = c.lockDelay
}
// Get the HTTP client
client, err := c.varPutCommand.Meta.Client()
if err != nil {
c.varPutCommand.Ui.Error(fmt.Sprintf("Error initializing client: %s", err))
return 1
}
// Set up the locks handler
lo := []api.LocksOption{}
if maxRetry != defaultMaxClientRetries {
lo = append(lo, api.LocksOptionWithMaxRetries(maxRetry))
}
l, err := client.Locks(api.WriteOptions{}, *sv, lo...)
if err != nil {
c.varPutCommand.Ui.Error(fmt.Sprintf("Error initializing lock handler: %s", err))
return 1
}
ctx := context.Background()
// Set up the locks handler
llo := []api.LockLeaserOption{}
if earlyReturn {
c.varPutCommand.verbose("Setting up early return")
llo = append(llo, api.LockLeaserOptionWithEarlyReturn(true))
}
if backoff != 0 {
c.varPutCommand.verbose("Setting up backoff period")
llo = append(llo, api.LockLeaserOptionWithWaitPeriod(backoff))
}
ll := client.NewLockLeaser(l, llo...)
c.varPutCommand.verbose("Attempting to acquire lock")
// Run the shell inside the protected function.
if err := ll.Start(ctx, func(ctx context.Context) error {
c.varPutCommand.verbose(fmt.Sprintf("Variable locked, ready to execute: %s",
strings.Join(args, " ")))
var newCommand func(ctx context.Context, args []string) (*exec.Cmd, error)
if !c.shell {
newCommand = subprocess
} else {
newCommand = script
}
cmd, err := newCommand(ctx, args)
if err != nil {
return err
}
cmd.Stdin = os.Stdin
cmd.Stdout = os.Stdout
cmd.Stderr = os.Stderr
signalCh := make(chan os.Signal, 10)
signal.Notify(signalCh, os.Interrupt, syscall.SIGTERM)
defer signal.Stop(signalCh)
go c.forwardSignals(ctx, cmd, signalCh)
if err := cmd.Start(); err != nil {
return err
}
return cmd.Wait()
}); err != nil {
c.varPutCommand.Ui.Error("Lock error:" + err.Error())
return 1
}
c.varPutCommand.verbose("Releasing the lock")
return 0
}
func (c *VarLockCommand) readPathFromArgs(args []string) (string, []string, error) {
var err error
var path string
// Handle first argument: @file or «var path»
arg := args[0]
switch {
case isArgFileRef(arg):
// ArgFileRefs start with "@" so we need to peel that off
// detect format based on file extension
specPath := arg[1:]
err = c.varPutCommand.setParserForFileArg(specPath)
if err != nil {
return "", args, err
}
c.varPutCommand.verbose(fmt.Sprintf("Reading whole variable specification from %q", specPath))
c.varPutCommand.contents, err = os.ReadFile(specPath)
if err != nil {
return "", args, fmt.Errorf("Error reading %q: %w", specPath, err)
}
default:
path = sanitizePath(arg)
c.varPutCommand.verbose(fmt.Sprintf("Writing to path %q", path))
}
// Handle second argument: can @file, or child process
args = args[1:]
switch {
case isArgFileRef(args[0]):
arg := args[0]
err = c.varPutCommand.setParserForFileArg(arg)
if err != nil {
return "", args, err
}
c.varPutCommand.verbose(fmt.Sprintf("Creating variable %q from specification file %q", path, arg))
fPath := arg[1:]
c.varPutCommand.contents, err = os.ReadFile(fPath)
if err != nil {
return "", args, fmt.Errorf("error reading %q: %s", fPath, err)
}
args = args[1:]
default:
// no-op - should be child process
}
return path, args, nil
}
// script returns a command to execute a script through a shell.
func script(ctx context.Context, args []string) (*exec.Cmd, error) {
shell := "/bin/sh"
if other := os.Getenv("SHELL"); other != "" {
shell = other
}
return exec.CommandContext(ctx, shell, "-c", strings.Join(args, " ")), nil
}
// subprocess returns a command to execute a subprocess directly.
func subprocess(ctx context.Context, args []string) (*exec.Cmd, error) {
if len(args) == 0 {
return nil, fmt.Errorf("need an executable to run")
}
return exec.CommandContext(ctx, args[0], args[1:]...), nil
}
// ForwardSignals will fire up a goroutine to forward signals to the given
// subprocess until the context is canceled.
func (c *VarLockCommand) forwardSignals(ctx context.Context, cmd *exec.Cmd, sg chan os.Signal) {
for {
select {
case sig := <-sg:
if err := cmd.Process.Signal(sig); err != nil {
c.varPutCommand.Ui.Error(fmt.Sprintf("failed to send signal %q: %v", sig, err))
}
case <-ctx.Done():
return
}
}
}

152
command/var_lock_test.go Normal file
View File

@@ -0,0 +1,152 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: BUSL-1.1
package command
import (
"fmt"
"os"
"testing"
"time"
"github.com/hashicorp/nomad/ci"
"github.com/mitchellh/cli"
"github.com/shoenig/test/must"
"github.com/stretchr/testify/require"
)
func TestVarLockCommand_Implements(t *testing.T) {
ci.Parallel(t)
var _ cli.Command = &VarLockCommand{}
}
func TestVarLockCommand_Fails(t *testing.T) {
ci.Parallel(t)
t.Run("bad_args", func(t *testing.T) {
ci.Parallel(t)
ui := cli.NewMockUi()
cmd := &VarLockCommand{
varPutCommand: &VarPutCommand{Meta: Meta{Ui: ui}},
}
code := cmd.Run([]string{"-bad-flag"})
out := ui.ErrorWriter.String()
must.One(t, code)
must.StrContains(t, out, commandErrorText(cmd))
})
t.Run("bad_address", func(t *testing.T) {
ci.Parallel(t)
ui := cli.NewMockUi()
cmd := &VarLockCommand{
varPutCommand: &VarPutCommand{Meta: Meta{Ui: ui}},
}
code := cmd.Run([]string{"-address=nope", "foo", "-"})
out := ui.ErrorWriter.String()
must.One(t, code)
must.StrContains(t, out, "unsupported protocol scheme")
})
t.Run("missing_args", func(t *testing.T) {
ci.Parallel(t)
ui := cli.NewMockUi()
cmd := &VarLockCommand{
varPutCommand: &VarPutCommand{Meta: Meta{Ui: ui}},
}
code := cmd.Run([]string{"foo"})
out := ui.ErrorWriter.String()
must.One(t, code)
must.StrContains(t, out, "Not enough arguments (expected >2, got 1)")
})
t.Run("invalid_TTL", func(t *testing.T) {
ci.Parallel(t)
ui := cli.NewMockUi()
cmd := &VarLockCommand{
varPutCommand: &VarPutCommand{Meta: Meta{Ui: ui}},
}
code := cmd.Run([]string{"-ttl=2", "bar", "foo"})
out := ui.ErrorWriter.String()
must.One(t, code)
must.StrContains(t, out, "Invalid TTL: time")
})
t.Run("invalid_lock_delay", func(t *testing.T) {
ci.Parallel(t)
ui := cli.NewMockUi()
cmd := &VarLockCommand{
varPutCommand: &VarPutCommand{Meta: Meta{Ui: ui}},
}
code := cmd.Run([]string{"-delay=2", "bar", "foo"})
out := ui.ErrorWriter.String()
must.One(t, code)
must.StrContains(t, out, "Invalid Lock Delay: time")
})
}
func TestVarLockCommand_Good(t *testing.T) {
ci.Parallel(t)
// Create a server
srv, client, url := testServer(t, true, nil)
defer srv.Shutdown()
ui := cli.NewMockUi()
cmd := &VarLockCommand{
varPutCommand: &VarPutCommand{Meta: Meta{Ui: ui}},
}
filePath := fmt.Sprintf("%v.txt", time.Now().Unix())
// Get the variable
code := cmd.Run([]string{"-address=" + url, "test/var/shell", "touch ", filePath})
require.Equal(t, 0, code, "expected exit 0, got: %d; %v", code, ui.ErrorWriter.String())
sv, _, err := srv.Client().Variables().Peek("test/var/shell", nil)
must.NoError(t, err)
must.NotNil(t, sv)
must.Eq(t, "test/var/shell", sv.Path)
// Check for the file
_, err = os.ReadFile(filePath)
must.NoError(t, err)
t.Cleanup(func() {
os.Remove(filePath)
_, _ = client.Variables().Delete("test/var/shell", nil)
})
}
func TestVarLockCommand_Good_NoShell(t *testing.T) {
ci.Parallel(t)
// Create a server
srv, client, url := testServer(t, true, nil)
defer srv.Shutdown()
ui := cli.NewMockUi()
cmd := &VarLockCommand{
varPutCommand: &VarPutCommand{Meta: Meta{Ui: ui}},
}
filePath := fmt.Sprintf("%v.txt", time.Now().Unix())
// Get the variable
code := cmd.Run([]string{"-address=" + url, "-shell=false", "test/var/noShell", "touch", filePath})
require.Zero(t, 0, code)
sv, _, err := srv.Client().Variables().Peek("test/var/noShell", nil)
must.NoError(t, err)
must.NotNil(t, sv)
must.Eq(t, "test/var/noShell", sv.Path)
// Check for the file
_, err = os.ReadFile(filePath)
must.NoError(t, err)
t.Cleanup(func() {
os.Remove(filePath)
_, _ = client.Variables().Delete("test/var/noShell", nil)
})
}

View File

@@ -73,7 +73,7 @@ General Options:
` + generalOptionsUsage(usageOptsDefault) + `
Apply Options:
Var put Options:
-check-index
If set, the variable is only acted upon if the server-side version's index
@@ -376,6 +376,7 @@ func (c *VarPutCommand) makeVariable(path string) (*api.Variable, error) {
out.Items = make(map[string]string)
return out, nil
}
switch c.inFmt {
case "json":
err = json.Unmarshal(c.contents, out)

View File

@@ -2254,7 +2254,8 @@ func (f *FSMFilter) Include(item interface{}) bool {
return true
}
func (n *nomadFSM) applyVariableOperation(msgType structs.MessageType, buf []byte, index uint64) interface{} {
func (n *nomadFSM) applyVariableOperation(msgType structs.MessageType, buf []byte,
index uint64) any {
var req structs.VarApplyStateRequest
if err := structs.Decode(buf, &req); err != nil {
panic(fmt.Errorf("failed to decode request: %v", err))
@@ -2270,6 +2271,10 @@ func (n *nomadFSM) applyVariableOperation(msgType structs.MessageType, buf []byt
return n.state.VarDeleteCAS(index, &req)
case structs.VarOpCAS:
return n.state.VarSetCAS(index, &req)
case structs.VarOpLockAcquire:
return n.state.VarLockAcquire(index, &req)
case structs.VarOpLockRelease:
return n.state.VarLockRelease(index, &req)
default:
err := fmt.Errorf("Invalid variable operation '%s'", req.Op)
n.logger.Warn("Invalid variable operation", "operation", req.Op)

View File

@@ -397,6 +397,17 @@ func (s *Server) establishLeadership(stopCh chan struct{}) error {
// Periodically publish job status metrics
go s.publishJobStatusMetrics(stopCh)
// Populate the variable lock TTL timers, so we can start tracking renewals
// and expirations.
if err := s.restoreLockTTLTimers(); err != nil {
return err
}
// Periodically publish metrics for the lock timer trackers which are only
// run on the leader.
go s.lockTTLTimer.EmitMetrics(1*time.Second, stopCh)
go s.lockDelayTimer.EmitMetrics(1*time.Second, stopCh)
// Setup the heartbeat timers. This is done both when starting up or when
// a leader fail over happens. Since the timers are maintained by the leader
// node, effectively this means all the timers are renewed at the time of failover.
@@ -1465,6 +1476,10 @@ func (s *Server) revokeLeadership() error {
return err
}
// Stop all the tracked variable lock TTL and delay timers.
s.lockTTLTimer.StopAndRemoveAll()
s.lockDelayTimer.RemoveAll()
// Clear the heartbeat timers on either shutdown or step down,
// since we are no longer responsible for TTL expirations.
if err := s.clearAllHeartbeatTimers(); err != nil {

View File

@@ -12,7 +12,7 @@ import (
"time"
"github.com/hashicorp/go-hclog"
memdb "github.com/hashicorp/go-memdb"
"github.com/hashicorp/go-memdb"
"github.com/hashicorp/go-version"
"github.com/shoenig/test"
"github.com/shoenig/test/must"

90
nomad/lock/delay.go Normal file
View File

@@ -0,0 +1,90 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: MPL-2.0
package lock
import (
"sync"
"time"
"github.com/armon/go-metrics"
"github.com/hashicorp/nomad/helper"
)
// DelayTimer is used to mark certain locks as unacquirable. When a locks TTL
// expires, it is subject to the LockDelay configured within the variable. This
// prevents another caller from acquiring the lock for some period of time as a
// protection against split-brains. This is inspired by the lock-delay in
// Chubby.
type DelayTimer struct {
// delayTimers has the set of active delay expiration times, organized by
// ID, which the caller dictates when adding entries. The lock should be
// used for all access.
delayTimers map[string]time.Time
lock sync.RWMutex
}
// NewDelayTimer returns a new delay timer manager.
func NewDelayTimer() *DelayTimer {
return &DelayTimer{
delayTimers: make(map[string]time.Time),
}
}
// Get returns the expiration time of a key lock delay. This must be checked on
// the leader server only due to the variability of clocks.
func (d *DelayTimer) Get(id string) time.Time {
d.lock.RLock()
expires := d.delayTimers[id]
d.lock.RUnlock()
return expires
}
// Set sets the expiration time for the lock delay to the given delay from the
// given now time.
func (d *DelayTimer) Set(id string, now time.Time, delay time.Duration) {
d.lock.Lock()
defer d.lock.Unlock()
d.delayTimers[id] = now.Add(delay)
// Set up the after func, but ignore the returned timer as we do not need
// this for cancellation.
_ = time.AfterFunc(delay, func() {
d.lock.Lock()
delete(d.delayTimers, id)
d.lock.Unlock()
})
}
// RemoveAll removes all registered timers.
func (d *DelayTimer) RemoveAll() {
d.lock.Lock()
defer d.lock.Unlock()
d.delayTimers = make(map[string]time.Time)
}
// EmitMetrics is a long-running routine used to emit periodic metrics about
// the Delay.
func (d *DelayTimer) EmitMetrics(period time.Duration, shutdownCh chan struct{}) {
timer, stop := helper.NewSafeTimer(period)
defer stop()
for {
timer.Reset(period)
select {
case <-timer.C:
metrics.SetGauge([]string{"variables", "locks", "delay_timer", "num"}, float32(d.timerNum()))
case <-shutdownCh:
return
}
}
}
// timerNum returns the number of registered delay timers.
func (d *DelayTimer) timerNum() int {
d.lock.RLock()
defer d.lock.RUnlock()
return len(d.delayTimers)
}

43
nomad/lock/delay_test.go Normal file
View File

@@ -0,0 +1,43 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: MPL-2.0
package lock
import (
"testing"
"time"
"github.com/hashicorp/nomad/ci"
"github.com/shoenig/test/must"
)
func TestDelay(t *testing.T) {
ci.Parallel(t)
delay := NewDelayTimer()
// An unknown key should have a time in the past.
must.True(t, delay.Get("this-does-not-exist").Before(time.Now()))
must.Eq(t, 0, delay.timerNum())
// Add a key and set a short expiration.
timeNow := time.Now()
delay.Set("test-delay-1", timeNow, 100*time.Millisecond)
must.False(t, delay.Get("test-delay-1").Before(time.Now()))
must.Eq(t, 1, delay.timerNum())
// Wait for the key to expire and check again.
time.Sleep(120 * time.Millisecond)
must.True(t, delay.Get("test-delay-1").Before(timeNow))
must.Eq(t, 0, delay.timerNum())
// Add a key and set a long expiration.
timeNow = time.Now()
delay.Set("test-delay-2", timeNow, 1000*time.Millisecond)
must.False(t, delay.Get("test-delay-2").Before(time.Now()))
must.Eq(t, 1, delay.timerNum())
// Perform the stop call which the leader will do when stepping down.
delay.RemoveAll()
must.Eq(t, 0, delay.timerNum())
}

106
nomad/lock/ttl.go Normal file
View File

@@ -0,0 +1,106 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: MPL-2.0
package lock
import (
"sync"
"time"
"github.com/armon/go-metrics"
"github.com/hashicorp/nomad/helper"
)
// TTLTimer provides a map of named timers which is safe for concurrent use.
// Each timer is created using time.AfterFunc which will be triggered once the
// timer fires.
type TTLTimer struct {
// timers is a mapping of timers which represent when a lock TTL will
// expire. The lock should be used for all access.
ttlTimers map[string]*time.Timer
lock sync.RWMutex
}
// NewTTLTimer initializes a new TTLTimer.
func NewTTLTimer() *TTLTimer {
return &TTLTimer{
ttlTimers: make(map[string]*time.Timer),
}
}
// Get returns the timer with the given ID. If the timer is not found, nil is
// returned, so callers should be expected to handle this case.
func (t *TTLTimer) Get(id string) *time.Timer {
t.lock.RLock()
defer t.lock.RUnlock()
return t.ttlTimers[id]
}
// Delete removes the timer with the given ID from the tracking. If the timer
// is not found, the call is noop.
func (t *TTLTimer) Delete(id string) {
t.lock.Lock()
defer t.lock.Unlock()
delete(t.ttlTimers, id)
}
// Create sets the TTL of the timer with the given ID or creates a new
// one if it does not exist.
func (t *TTLTimer) Create(id string, ttl time.Duration, afterFn func()) {
t.lock.Lock()
defer t.lock.Unlock()
if tm := t.ttlTimers[id]; tm != nil {
tm.Reset(ttl)
return
}
t.ttlTimers[id] = time.AfterFunc(ttl, afterFn)
}
// StopAndRemove stops the timer with the given ID and removes it from
// tracking.
func (t *TTLTimer) StopAndRemove(id string) {
t.lock.Lock()
defer t.lock.Unlock()
if tm := t.ttlTimers[id]; tm != nil {
tm.Stop()
delete(t.ttlTimers, id)
}
}
// StopAndRemoveAll stops and removes all registered timers.
func (t *TTLTimer) StopAndRemoveAll() {
t.lock.Lock()
defer t.lock.Unlock()
for _, tm := range t.ttlTimers {
tm.Stop()
}
t.ttlTimers = make(map[string]*time.Timer)
}
// EmitMetrics is a long-running routine used to emit periodic metrics about
// the Timer.
func (t *TTLTimer) EmitMetrics(period time.Duration, shutdownCh chan struct{}) {
timer, stop := helper.NewSafeTimer(period)
defer stop()
for {
timer.Reset(period)
select {
case <-timer.C:
metrics.SetGauge([]string{"variables", "locks", "ttl_timer", "num"}, float32(t.TimerNum()))
case <-shutdownCh:
return
}
}
}
// timerNum returns the number of registered timers.
func (t *TTLTimer) TimerNum() int {
t.lock.RLock()
defer t.lock.RUnlock()
return len(t.ttlTimers)
}

75
nomad/lock/ttl_test.go Normal file
View File

@@ -0,0 +1,75 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: MPL-2.0
package lock
import (
"testing"
"time"
"github.com/hashicorp/nomad/ci"
"github.com/shoenig/test/must"
)
func TestTimer(t *testing.T) {
ci.Parallel(t)
// Create a test channel and timer test function that will be used
// throughout the test.
newTimerCh := make(chan int)
waitForTimer := func() {
select {
case <-newTimerCh:
return
case <-time.After(100 * time.Millisecond):
t.Fatal("timer did not fire")
}
}
timer := NewTTLTimer()
// Perform a lookup on a timer that doesn't exist, to ensure this is
// handled properly.
must.Nil(t, timer.Get("this-does-not-exist"))
// Perform a create, read, update, and delete on a single timer.
timer.Create("test-timer-2", time.Millisecond, func() { newTimerCh <- 1 })
must.Eq(t, 1, timer.TimerNum())
waitForTimer()
// Ensure the timer is still held within the mapping.
must.Eq(t, 1, timer.TimerNum())
// Update the timer and check that it fires again.
timer.Create("test-timer-2", time.Millisecond, nil)
waitForTimer()
// Reset the timer with a long ttl and then stop it.
timer.Create("test-timer-2", 20*time.Millisecond, func() { newTimerCh <- 1 })
timer.StopAndRemove("test-timer-2")
select {
case <-newTimerCh:
t.Fatal("timer fired although it shouldn't")
case <-time.After(100 * time.Millisecond):
}
// Ensure that stopping a stopped timer does not break anything.
timer.StopAndRemove("test-timer-2")
must.Eq(t, 0, timer.TimerNum())
// Create two timers, stopping them using the StopAll function to signify
// leadership loss.
timer.Create("test-timer-3", 20*time.Millisecond, func() { newTimerCh <- 1 })
timer.Create("test-timer-4", 30*time.Millisecond, func() { newTimerCh <- 2 })
timer.StopAndRemoveAll()
select {
case msg := <-newTimerCh:
t.Fatalf("timer %d fired although it shouldn't", msg)
case <-time.After(100 * time.Millisecond):
}
must.Eq(t, 0, timer.TimerNum())
}

154
nomad/locks.go Normal file
View File

@@ -0,0 +1,154 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: BUSL-1.1
package nomad
import (
"errors"
"fmt"
"time"
"github.com/hashicorp/nomad/nomad/structs"
)
var (
errTimerNotFound = errors.New("lock doesn't have a running timer ")
)
// restoreLockTTLTimers iterates the stored variables and creates a lock TTL
// timer for each variable lock. This is used during leadership establishment
// to populate the in-memory timer.
func (s *Server) restoreLockTTLTimers() error {
varIterator, err := s.fsm.State().Variables(nil)
if err != nil {
return fmt.Errorf("failed to list variables for lock TTL restore: %v", err)
}
// Iterate the variables, identifying each one that is associated to a lock
// and adding a TTL timer for each.
for varInterface := varIterator.Next(); varInterface != nil; varInterface = varIterator.Next() {
if realVar, ok := varInterface.(*structs.VariableEncrypted); ok && realVar.IsLock() {
// The variable will be modified in order to show that it no longer
// is held. We therefore need to ensure we perform modifications on
// a copy.
s.CreateVariableLockTTLTimer(realVar.Copy())
}
}
return nil
}
// CreateVariableLockTTLTimer creates a TTL timer for the given lock.
// It is in charge of integrating the delay after the TTL expires.
func (s *Server) CreateVariableLockTTLTimer(variable structs.VariableEncrypted) {
s.logger.Debug("locks: adding lock", "namespace", variable.Namespace, "path", variable.Path)
// Adjust the given TTL by multiplier of 2. This is done to give a client a
// grace period and to compensate for network and processing delays. The
// contract is that a variable lock is not expired before the TTL expires,
// but there is no explicit promise about the upper bound so this is
// allowable.
lockTTL := variable.Lock.TTL * 2
// The lock ID is used a couple of times, so grab this now.
lockID := variable.LockID()
lock := s.lockTTLTimer.Get(lockID)
if lock != nil {
// If this was to happen, there is a sync issue somewhere else
s.logger.Error("attempting to recreate existing lock: %s", lockID)
return
}
s.lockTTLTimer.Create(lockID, lockTTL, func() {
s.logger.Debug("locks: lock TTL expired, starting delay",
"namespace", variable.Namespace, "path", variable.Path, "ttl", variable.Lock.TTL)
s.lockTTLTimer.StopAndRemove(lockID)
_ = time.AfterFunc(variable.Lock.LockDelay, func() {
s.invalidateVariableLock(variable)
})
})
}
// invalidateVariableLock exponentially tries to update Nomad's state to remove
// the lock ID from the variable. This can be used when a variable lock's TTL
// has expired.
func (s *Server) invalidateVariableLock(variable structs.VariableEncrypted) {
lockID := variable.LockID()
s.logger.Debug("locks: lock delay expired, removing lock",
"namespace", variable.Namespace, "path", variable.Path)
// Remove the lock from the variable
variable.VariableMetadata.Lock = nil
args := structs.VarApplyStateRequest{
Op: structs.VarOpLockRelease,
Var: &structs.VariableEncrypted{
VariableMetadata: variable.VariableMetadata,
},
WriteRequest: structs.WriteRequest{
Region: s.Region(),
Namespace: variable.Namespace,
},
}
// Retry with exponential backoff to remove the lock
for attempt := 0; attempt < maxAttemptsToRaftApply; attempt++ {
_, _, err := s.raftApply(structs.VarApplyStateRequestType, args)
if err == nil {
return
}
s.logger.Error("lock expiration failed",
"namespace", variable.Namespace, "path", variable.Path,
"lock_id", lockID, "error", err)
// This exponential backoff will extend the lock Delay beyond the expected
// time if there is any raft error, should we make it dependant on the LockDelay?
time.Sleep((1 << attempt) * 10 * time.Second)
}
}
func (s *Server) RenewTTLTimer(variable structs.VariableEncrypted) error {
lockID := variable.LockID()
s.logger.Debug("locks: renewing the lock",
"namespace", variable.Namespace, "path", variable.Path, "ttl", variable.Lock.TTL)
lock := s.lockTTLTimer.Get(lockID)
if lock == nil {
return errTimerNotFound
}
// Adjust the given TTL by multiplier of 2. This is done to give a client a
// grace period and to compensate for network and processing delays. The
// contract is that a variable lock is not expired before the TTL expires,
// but there is no explicit promise about the upper bound so this is
// allowable.
lockTTL := variable.Lock.TTL * 2
// The create function resets the timer when it exists already, there is no
// need to provide the release function again.
s.lockTTLTimer.Create(lockID, lockTTL, nil)
return nil
}
// RemoveVariableLockTTLTimer creates a TTL timer for the given lock. The
// passed ID is expected to be generated via the variable NamespacedID
// function.
func (s *Server) RemoveVariableLockTTLTimer(variable structs.VariableEncrypted) {
// The lock ID is used a couple of times, so grab this now.
lockID := variable.LockID()
lock := s.lockTTLTimer.Get(lockID)
if lock == nil {
// If this was to happen, there is a sync issue somewhere else.
s.logger.Error("attempting to removed missing lock: %s", lockID)
return
}
s.lockTTLTimer.StopAndRemove(lockID)
}

202
nomad/locks_test.go Normal file
View File

@@ -0,0 +1,202 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: BUSL-1.1
package nomad
import (
"errors"
"testing"
"time"
"github.com/hashicorp/nomad/ci"
"github.com/hashicorp/nomad/helper/uuid"
"github.com/hashicorp/nomad/nomad/mock"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/testutil"
"github.com/shoenig/test/must"
)
func TestServer_restoreLockTTLTimers(t *testing.T) {
ci.Parallel(t)
testServer, testServerCleanup := TestServer(t, nil)
defer testServerCleanup()
testutil.WaitForLeader(t, testServer.RPC)
// Generate two variables, one with and one without a lock and upsert these
// into state.
mockVar1 := mock.VariableEncrypted()
mockVar2 := mock.VariableEncrypted()
mockVar2.Lock = &structs.VariableLock{
ID: uuid.Generate(),
TTL: 15 * time.Second,
LockDelay: 15 * time.Second,
}
upsertResp1 := testServer.fsm.State().VarSet(10, &structs.VarApplyStateRequest{Var: mockVar1, Op: structs.VarOpSet})
must.NoError(t, upsertResp1.Error)
upsertResp2 := testServer.fsm.State().VarSet(20, &structs.VarApplyStateRequest{Var: mockVar2, Op: structs.VarOpLockAcquire})
must.NoError(t, upsertResp2.Error)
// Call the server function that restores the lock TTL timers. This would
// usually happen on leadership transition.
must.NoError(t, testServer.restoreLockTTLTimers())
// Ensure the TTL timer tracking has the expected entries.
must.Nil(t, testServer.lockTTLTimer.Get(mockVar1.LockID()))
must.NotNil(t, testServer.lockTTLTimer.Get(mockVar2.LockID()))
}
func TestServer_invalidateVariableLock(t *testing.T) {
ci.Parallel(t)
testServer, testServerCleanup := TestServer(t, nil)
defer testServerCleanup()
testutil.WaitForLeader(t, testServer.RPC)
// Generate a variable that includes a lock entry and upsert this into our
// state.
mockVar1 := mock.VariableEncrypted()
mockVar1.Lock = &structs.VariableLock{
ID: uuid.Generate(),
TTL: 15 * time.Second,
LockDelay: 15 * time.Second,
}
upsertResp1 := testServer.fsm.State().VarSet(10, &structs.VarApplyStateRequest{Var: mockVar1, Op: structs.VarOpLockAcquire})
must.NoError(t, upsertResp1.Error)
// Create the timer manually, so we can control the invalidation for
// testing.
testServer.lockTTLTimer.Create(mockVar1.LockID(), mockVar1.Lock.TTL, func() {})
// Perform the invalidation call.
testServer.invalidateVariableLock(*mockVar1)
// Pull the variable out of state and check that the lock ID has been
// removed.
varGetResp, err := testServer.fsm.State().GetVariable(nil, mockVar1.Namespace, mockVar1.Path)
must.NoError(t, err)
must.Nil(t, varGetResp.Lock)
}
func TestServer_createVariableLockTimer(t *testing.T) {
ci.Parallel(t)
testServer, testServerCleanup := TestServer(t, nil)
defer testServerCleanup()
testutil.WaitForLeader(t, testServer.RPC)
// Generate a variable that includes a lock entry and upsert this into our
// state.
mockVar1 := mock.VariableEncrypted()
mockVar1.Lock = &structs.VariableLock{
ID: uuid.Generate(),
TTL: 10 * time.Millisecond,
LockDelay: 10 * time.Millisecond,
}
upsertResp1 := testServer.fsm.State().VarSet(10, &structs.VarApplyStateRequest{Var: mockVar1, Op: structs.VarOpLockAcquire})
must.NoError(t, upsertResp1.Error)
testServer.CreateVariableLockTTLTimer(*mockVar1)
time.Sleep(10 * time.Millisecond)
// Check the timer is still present, meaning it didn't expired and the variable
// still holds a lock
must.NotNil(t, testServer.lockTTLTimer.Get(mockVar1.LockID()))
varGetResp, err := testServer.fsm.State().GetVariable(nil, mockVar1.Namespace, mockVar1.Path)
must.NoError(t, err)
must.Eq(t, mockVar1.LockID(), varGetResp.LockID())
// After 15ms more, the TTL has expired, no timer should be running but the variable
// must still hold the lock.
time.Sleep(15 * time.Millisecond)
must.Nil(t, testServer.lockTTLTimer.Get(mockVar1.LockID()))
varGetResp, err = testServer.fsm.State().GetVariable(nil, mockVar1.Namespace, mockVar1.Path)
must.NoError(t, err)
must.Eq(t, mockVar1.LockID(), varGetResp.LockID())
// After 10ms more, the delay should have expired as well and the variable
// should not hold the lock
time.Sleep(10 * time.Millisecond)
must.Nil(t, testServer.lockTTLTimer.Get(mockVar1.LockID()))
varGetResp, err = testServer.fsm.State().GetVariable(nil, mockVar1.Namespace, mockVar1.Path)
must.NoError(t, err)
must.Nil(t, varGetResp.Lock)
}
func TestServer_createAndRenewVariableLockTimer(t *testing.T) {
ci.Parallel(t)
testServer, testServerCleanup := TestServer(t, nil)
defer testServerCleanup()
testutil.WaitForLeader(t, testServer.RPC)
// Generate a variable that includes a lock entry and upsert this into our
// state.
mockVar1 := mock.VariableEncrypted()
mockVar1.Lock = &structs.VariableLock{
ID: uuid.Generate(),
TTL: 10 * time.Millisecond,
LockDelay: 10 * time.Millisecond,
}
// Attempt to renew a lock that has no timer yet
err := testServer.RenewTTLTimer(*mockVar1)
if !errors.Is(errTimerNotFound, err) {
t.Fatalf("expected error, got %s", err)
}
upsertResp1 := testServer.fsm.State().VarSet(10, &structs.VarApplyStateRequest{Var: mockVar1, Op: structs.VarOpLockAcquire})
must.NoError(t, upsertResp1.Error)
testServer.CreateVariableLockTTLTimer(*mockVar1)
time.Sleep(10 * time.Millisecond)
// Check the timer is still present, meaning it didn't expired and the variable
// still holds a lock
must.NotNil(t, testServer.lockTTLTimer.Get(mockVar1.LockID()))
varGetResp, err := testServer.fsm.State().GetVariable(nil, mockVar1.Namespace, mockVar1.Path)
must.NoError(t, err)
must.Eq(t, mockVar1.LockID(), varGetResp.LockID())
for i := 0; i < 3; i++ {
// Renew the lock
err = testServer.RenewTTLTimer(*mockVar1)
must.NoError(t, err)
// After 10 ms the timer is must still present, meaning it was correctly renewed
// and the variable still holds a lock.
time.Sleep(10 * time.Millisecond)
must.NotNil(t, testServer.lockTTLTimer.Get(mockVar1.LockID()))
varGetResp, err = testServer.fsm.State().GetVariable(nil, mockVar1.Namespace, mockVar1.Path)
must.NoError(t, err)
must.Eq(t, mockVar1.LockID(), varGetResp.LockID())
}
// After 15ms more, the TTL has expired, no timer should be running but the variable
// must still hold the lock.
time.Sleep(15 * time.Millisecond)
must.Nil(t, testServer.lockTTLTimer.Get(mockVar1.LockID()))
varGetResp, err = testServer.fsm.State().GetVariable(nil, mockVar1.Namespace, mockVar1.Path)
must.NoError(t, err)
must.Eq(t, mockVar1.LockID(), varGetResp.LockID())
// After 10ms more, the delay should have expired as well and the variable
// should not hold the lock
time.Sleep(10 * time.Millisecond)
must.Nil(t, testServer.lockTTLTimer.Get(mockVar1.LockID()))
varGetResp, err = testServer.fsm.State().GetVariable(nil, mockVar1.Namespace, mockVar1.Path)
must.NoError(t, err)
must.Nil(t, varGetResp.Lock)
}

View File

@@ -41,6 +41,7 @@ import (
"github.com/hashicorp/nomad/lib/auth/oidc"
"github.com/hashicorp/nomad/nomad/deploymentwatcher"
"github.com/hashicorp/nomad/nomad/drainer"
"github.com/hashicorp/nomad/nomad/lock"
"github.com/hashicorp/nomad/nomad/state"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/nomad/structs/config"
@@ -270,6 +271,12 @@ type Server struct {
// shutting down, the oidcProviderCache.Shutdown() function must be called.
oidcProviderCache *oidc.ProviderCache
// lockTTLTimer and lockDelayTimer are used to track variable lock timers.
// These are held in memory on the leader rather than in state to avoid
// large amount of Raft writes.
lockTTLTimer *lock.TTLTimer
lockDelayTimer *lock.DelayTimer
// leaderAcl is the management ACL token that is valid when resolved by the
// current leader.
leaderAcl string
@@ -353,6 +360,8 @@ func NewServer(config *Config, consulCatalog consul.CatalogAPI, consulConfigEntr
rpcTLS: incomingTLS,
aclCache: aclCache,
workersEventCh: make(chan interface{}, 1),
lockTTLTimer: lock.NewTTLTimer(),
lockDelayTimer: lock.NewDelayTimer(),
}
s.shutdownCtx, s.shutdownCancel = context.WithCancel(context.Background())

View File

@@ -4,6 +4,7 @@
package state
import (
"errors"
"fmt"
"math"
@@ -11,6 +12,12 @@ import (
"github.com/hashicorp/nomad/nomad/structs"
)
var (
errVarAlreadyLocked = errors.New("variable already holds a lock")
errVarNotFound = errors.New("variable doesn't exist")
errLockNotFound = errors.New("variable doesn't hold a lock")
)
// Variables queries all the variables and is used only for
// snapshot/restore and key rotation
func (s *StateStore) Variables(ws memdb.WatchSet) (memdb.ResultIterator, error) {
@@ -175,10 +182,12 @@ func (s *StateStore) varSetCASTxn(tx WriteTxn, idx uint64, req *structs.VarApply
return req.ConflictResponse(idx, zeroVal)
}
// If the existing index does not match the provided CAS index arg, then we
// shouldn't update anything and can safely return early here.
if ok && sv.ModifyIndex != svEx.ModifyIndex {
return req.ConflictResponse(idx, svEx)
if ok {
// If the existing index does not match the provided CAS index arg, then we
// shouldn't update anything and can safely return early here.
if sv.ModifyIndex != svEx.ModifyIndex {
return req.ConflictResponse(idx, svEx)
}
}
// If we made it this far, we should perform the set.
@@ -193,6 +202,7 @@ func (s *StateStore) varSetTxn(tx WriteTxn, idx uint64, req *structs.VarApplySta
if err != nil {
return req.ErrorResponse(idx, fmt.Errorf("failed sve lookup: %s", err))
}
existing, _ := existingRaw.(*structs.VariableEncrypted)
existingQuota, err := tx.First(TableVariablesQuotas, indexID, sv.Namespace)
@@ -201,9 +211,21 @@ func (s *StateStore) varSetTxn(tx WriteTxn, idx uint64, req *structs.VarApplySta
}
var quotaChange int64
// Set the CreateIndex and CreateTime
if existing != nil {
// If the variable is locked, it can only be updated by providing the correct
// lock ID.
if isLocked(existing.Lock, req) {
zeroVal := &structs.VariableEncrypted{
VariableMetadata: structs.VariableMetadata{
Namespace: sv.Namespace,
Path: sv.Path,
},
}
return req.ConflictResponse(idx, zeroVal)
}
sv.CreateIndex = existing.CreateIndex
sv.CreateTime = existing.CreateTime
@@ -379,13 +401,23 @@ func (s *StateStore) svDeleteTxn(tx WriteTxn, idx uint64, req *structs.VarApplyS
return req.SuccessResponse(idx, nil)
}
sv := existingRaw.(*structs.VariableEncrypted)
if isLocked(sv.Lock, req) {
zeroVal := &structs.VariableEncrypted{
VariableMetadata: structs.VariableMetadata{
Namespace: sv.Namespace,
Path: sv.Path,
},
}
return req.ConflictResponse(idx, zeroVal)
}
existingQuota, err := tx.First(TableVariablesQuotas, indexID, req.Var.Namespace)
if err != nil {
return req.ErrorResponse(idx, fmt.Errorf("variable quota lookup failed: %v", err))
}
sv := existingRaw.(*structs.VariableEncrypted)
// Track quota usage
if existingQuota != nil {
quotaUsed := existingQuota.(*structs.VariablesQuota)
@@ -448,3 +480,117 @@ func (s *StateStore) VariablesQuotaByNamespace(ws memdb.WatchSet, namespace stri
quotaUsed := raw.(*structs.VariablesQuota)
return quotaUsed, nil
}
// VarLockAcquire is the method used to append a lock to a variable, if the
// variable doesn't exists, it is created.
// IMPORTANT: this method overwrites the variable, data included.
func (s *StateStore) VarLockAcquire(idx uint64,
req *structs.VarApplyStateRequest) *structs.VarApplyStateResponse {
tx := s.db.WriteTxn(idx)
defer tx.Abort()
// Try to fetch the variable.
raw, err := tx.First(TableVariables, indexID, req.Var.Namespace, req.Var.Path)
if err != nil {
return req.ErrorResponse(idx, fmt.Errorf("variable lookup failed: %v", err))
}
sv, ok := raw.(*structs.VariableEncrypted)
// If the variable exist, we must make sure it doesn't hold a lock already
if ok {
if isLocked(sv.VariableMetadata.Lock, req) {
zeroVal := &structs.VariableEncrypted{
VariableMetadata: structs.VariableMetadata{
Namespace: sv.Namespace,
Path: sv.Path,
},
}
return req.ConflictResponse(idx, zeroVal)
}
}
resp := s.varSetTxn(tx, idx, req)
if resp.IsError() {
return resp
}
if err := tx.Commit(); err != nil {
return req.ErrorResponse(idx, err)
}
return resp
}
func (s *StateStore) VarLockRelease(idx uint64,
req *structs.VarApplyStateRequest) *structs.VarApplyStateResponse {
tx := s.db.WriteTxn(idx)
defer tx.Abort()
// Look up the entry in the state store.
raw, err := tx.First(TableVariables, indexID, req.Var.Namespace, req.Var.Path)
if err != nil {
return req.ErrorResponse(idx, fmt.Errorf("failed variable lookup: %s", err))
}
sv, ok := raw.(*structs.VariableEncrypted)
if !ok {
return req.ErrorResponse(idx, errVarNotFound)
}
if sv.Lock == nil || sv.Lock.ID == "" {
return req.ErrorResponse(idx, errLockNotFound)
}
if req.Var.Lock != nil && sv.Lock.ID != req.Var.Lock.ID {
// Avoid showing the variable data on a failed lock release
zeroVal := &structs.VariableEncrypted{
VariableMetadata: structs.VariableMetadata{
Namespace: sv.Namespace,
Path: sv.Path,
Lock: &structs.VariableLock{},
},
}
return req.ConflictResponse(idx, zeroVal)
}
// Avoid overwriting the variable data when releasing the lock, to prevent
// a delay release to remove customer data.
updated := sv.Copy()
updated.Lock = nil
updated.ModifyIndex = idx
err = s.updateVarsAndIndexTxn(tx, idx, &updated)
if err != nil {
req.ErrorResponse(idx, fmt.Errorf("failed lock release: %s", err))
}
if err := tx.Commit(); err != nil {
return req.ErrorResponse(idx, err)
}
return req.SuccessResponse(idx, &updated.VariableMetadata)
}
func (s *StateStore) updateVarsAndIndexTxn(tx WriteTxn, idx uint64, sv *structs.VariableEncrypted) error {
if err := tx.Insert(TableVariables, sv); err != nil {
return fmt.Errorf("failed inserting variable: %w", err)
}
if err := tx.Insert(tableIndex,
&IndexEntry{TableVariables, idx}); err != nil {
return fmt.Errorf("failed updating variable index: %w", err)
}
return nil
}
func isLocked(lock *structs.VariableLock, req *structs.VarApplyStateRequest) bool {
if lock != nil {
if req.Var.VariableMetadata.Lock == nil ||
req.Var.VariableMetadata.Lock.ID != lock.ID {
return true
}
}
return false
}

View File

@@ -5,6 +5,7 @@ package state
import (
"encoding/json"
"errors"
"sort"
"strings"
"testing"
@@ -65,16 +66,8 @@ func TestStateStore_UpsertVariables(t *testing.T) {
must.Eq(t, insertIndex, initialIndex)
// List all the variables in the table
iter, err := testState.Variables(ws)
got, err := getAllVariables(testState, ws)
must.NoError(t, err)
got := []*structs.VariableEncrypted{}
for raw := iter.Next(); raw != nil; raw = iter.Next() {
sv := raw.(*structs.VariableEncrypted)
var svCopy structs.VariableEncrypted
svCopy = sv.Copy()
got = append(got, &svCopy)
}
must.Len(t, 2, got, must.Sprintf("incorrect number of variables found"))
// Ensure the create and modify indexes are populated correctly.
@@ -148,19 +141,9 @@ func TestStateStore_UpsertVariables(t *testing.T) {
must.NoError(t, err)
must.Eq(t, update1Index, updateActualIndex, must.Sprintf("index should have changed"))
// Get the variables from the table.
iter, err := testState.Variables(ws)
must.NoError(t, err)
got := []*structs.VariableEncrypted{}
// Iterate all the stored variables and assert indexes have been updated as expected
for raw := iter.Next(); raw != nil; raw = iter.Next() {
sv := raw.(*structs.VariableEncrypted)
var svCopy structs.VariableEncrypted
svCopy = sv.Copy()
got = append(got, &svCopy)
}
got, err := getAllVariables(testState, ws)
must.NoError(t, err)
must.Len(t, 2, got)
must.Eq(t, update1Index, got[0].ModifyIndex)
must.Eq(t, insertIndex, got[1].ModifyIndex)
@@ -216,6 +199,68 @@ func TestStateStore_UpsertVariables(t *testing.T) {
must.Eq(t, expectedQuotaSize+1, quotaUsed.Size)
})
// Acquire lock on first variable to test upserting on a locked variable.
t.Run("5 lock and upsert", func(t *testing.T) {
acquireIndex := uint64(60)
sv3 := svs[0].Copy()
sv3.VariableMetadata.Lock = &structs.VariableLock{
ID: "theLockID",
}
resp := testState.VarLockAcquire(acquireIndex,
&structs.VarApplyStateRequest{
Op: structs.VarOpLockAcquire,
Var: &sv3,
})
must.NoError(t, resp.Error)
// Check that the index for the table was modified as expected.
afterAcquireIndex, err := testState.Index(TableVariables)
must.NoError(t, err)
must.Eq(t, acquireIndex, afterAcquireIndex)
// Attempt to upsert variable without the lock ID
update4Index := uint64(65)
sv4 := svs[0].Copy()
sv4.KeyID = "sv4-update"
sv4.ModifyIndex = update4Index
resp = testState.VarSet(update4Index, &structs.VarApplyStateRequest{
Op: structs.VarOpSet,
Var: &sv4,
})
must.NoError(t, resp.Error)
afterFailedUpsertIndex, err := testState.Index(TableVariables)
must.NoError(t, err)
must.Eq(t, afterAcquireIndex, afterFailedUpsertIndex, must.Sprintf("index should not have changed"))
must.True(t, resp.IsConflict())
// Attempt to upsert variable but this time include the lock ID
sv4.VariableMetadata.Lock = &structs.VariableLock{
ID: "theLockID",
}
resp = testState.VarSet(update4Index, &structs.VarApplyStateRequest{
Op: structs.VarOpSet,
Var: &sv4,
})
must.NoError(t, resp.Error)
// Check that the index for the table was modified as expected.
updateActualIndex, err := testState.Index(TableVariables)
must.NoError(t, err)
must.Eq(t, update4Index, updateActualIndex, must.Sprintf("index should have changed"))
// Iterate all the stored variables and assert indexes have been updated as expected
got, err := getAllVariables(testState, ws)
must.NoError(t, err)
must.Len(t, 2, got)
must.Eq(t, update4Index, got[0].ModifyIndex)
})
}
func TestStateStore_DeleteVariable(t *testing.T) {
@@ -303,8 +348,66 @@ func TestStateStore_DeleteVariable(t *testing.T) {
must.Eq(t, expectedQuotaSize, quotaUsed.Size)
})
t.Run("3 delete remaining variable", func(t *testing.T) {
delete2Index := uint64(30)
t.Run("3 lock the variable and attempt to delete it", func(t *testing.T) {
ws := memdb.NewWatchSet()
acquireIndex := uint64(25)
lsv := svs[1].Copy()
lsv.VariableMetadata.Lock = &structs.VariableLock{
ID: "theLockID",
}
resp := testState.VarLockAcquire(acquireIndex,
&structs.VarApplyStateRequest{
Op: structs.VarOpLockAcquire,
Var: &lsv,
})
must.NoError(t, resp.Error)
must.True(t, resp.IsOk())
deleteLockedIndex := uint64(27)
// Attempt to delete without the lock ID
resp2 := testState.VarDelete(deleteLockedIndex, &structs.VarApplyStateRequest{
Op: structs.VarOpDelete,
Var: svs[1],
})
must.NoError(t, resp2.Error)
must.True(t, resp2.IsConflict())
// Check that the index for the table was not modified.
failedDeleteIndex, err := testState.Index(TableVariables)
must.NoError(t, err)
must.Eq(t, acquireIndex, failedDeleteIndex)
svs, err := getAllVariables(testState, ws)
must.NoError(t, err)
must.One(t, len(svs))
// Release lock
releaseIndex := uint64(30)
lsv.VariableMetadata.Lock = &structs.VariableLock{
ID: "theLockID",
}
resp3 := testState.VarLockRelease(releaseIndex,
&structs.VarApplyStateRequest{
Op: structs.VarOpLockRelease,
Var: &lsv,
})
must.NoError(t, err)
must.True(t, resp3.IsOk())
svs, err = getAllVariables(testState, ws)
must.NoError(t, err)
must.One(t, len(svs))
})
t.Run("4 delete remaining variable", func(t *testing.T) {
delete2Index := uint64(40)
resp := testState.VarDelete(delete2Index, &structs.VarApplyStateRequest{
Op: structs.VarOpDelete,
@@ -581,6 +684,7 @@ func TestStateStore_ListVariablesByNamespaceAndPrefix(t *testing.T) {
}
})
}
func TestStateStore_ListVariablesByKeyID(t *testing.T) {
ci.Parallel(t)
testState := testStateStore(t)
@@ -714,6 +818,48 @@ func TestStateStore_Variables_DeleteCAS(t *testing.T) {
must.NotNil(t, resp.Conflict)
must.Eq(t, sv.VariableMetadata, resp.Conflict.VariableMetadata)
})
t.Run("real_locked_var-cas_0", func(t *testing.T) {
ci.Parallel(t)
sv := mock.VariableEncrypted()
sv.Path = "real_var/cas_0"
resp := ts.VarSet(10, &structs.VarApplyStateRequest{
Op: structs.VarOpSet,
Var: sv,
})
must.True(t, resp.IsOk())
svCopy := sv.Copy()
svCopy.VariableMetadata.Lock = &structs.VariableLock{
ID: "theLockID",
}
resp = ts.VarLockAcquire(15,
&structs.VarApplyStateRequest{
Op: structs.VarOpLockAcquire,
Var: &svCopy,
})
must.True(t, resp.IsOk())
// A CAS delete with a correct index should succeed.
req := &structs.VarApplyStateRequest{
Op: structs.VarOpDelete,
Var: sv,
}
resp = ts.VarDeleteCAS(15, req)
must.True(t, resp.IsConflict())
resp = ts.VarLockRelease(20,
&structs.VarApplyStateRequest{
Op: structs.VarOpLockRelease,
Var: &svCopy,
})
must.True(t, resp.IsOk())
})
t.Run("real_var-cas_ok", func(t *testing.T) {
ci.Parallel(t)
sv := mock.VariableEncrypted()
@@ -729,7 +875,346 @@ func TestStateStore_Variables_DeleteCAS(t *testing.T) {
Op: structs.VarOpDelete,
Var: sv,
}
resp = ts.VarDeleteCAS(0, req)
resp = ts.VarDeleteCAS(10, req)
must.True(t, resp.IsOk())
})
}
func TestStateStore_AcquireAndReleaseLock(t *testing.T) {
ci.Parallel(t)
testState := testStateStore(t)
ws := memdb.NewWatchSet()
mv := mock.VariableEncrypted()
mv.Path = "thePath"
mv.Lock = &structs.VariableLock{
ID: "theLockID",
}
insertIndex := uint64(20)
allVars, err := getAllVariables(testState, ws)
must.NoError(t, err)
t.Run("1 lock on missing variable", func(t *testing.T) {
/* Attempt to acquire the lock on a variable that doesn't exist. */
resp := testState.VarLockAcquire(insertIndex,
&structs.VarApplyStateRequest{
Op: structs.VarOpLockAcquire,
Var: mv,
})
must.NoError(t, resp.Error)
// Check that the index for the table was modified as expected.
initialIndex, err := testState.Index(TableVariables)
must.NoError(t, err)
must.Eq(t, insertIndex, initialIndex)
got, err := getAllVariables(testState, ws)
must.Eq(t, len(allVars)+1, len(got), must.Sprintf("incorrect number of variables found"))
// Ensure the create and modify indexes are populated correctly.
must.Eq(t, 20, got[0].CreateIndex, must.Sprintf("%s: incorrect create index", got[0].Path))
must.Eq(t, 20, got[0].ModifyIndex, must.Sprintf("%s: incorrect modify index", got[0].Path))
// Ensure the lock was persisted.
must.Eq(t, "theLockID", got[0].Lock.ID)
allVars = got
})
t.Run("2 lock on same variable", func(t *testing.T) {
/* Attempt to acquire the lock on the same variable again. */
sv := *allVars[0]
sv.Lock = &structs.VariableLock{
ID: "aDifferentLockID",
}
resp := testState.VarLockAcquire(insertIndex+1,
&structs.VarApplyStateRequest{
Op: structs.VarOpLockAcquire,
Var: &sv,
})
must.NoError(t, resp.Error)
must.Eq(t, structs.VarOpResultConflict, resp.Result)
// Ensure the create and modify were NOT modified
must.Eq(t, 20, sv.CreateIndex, must.Sprintf("%s: incorrect create index", sv.Path))
must.Eq(t, 20, sv.ModifyIndex, must.Sprintf("%s: incorrect modify index", sv.Path))
})
t.Run("3 release lock", func(t *testing.T) {
/* Test to release the lock */
allVars, err := getAllVariables(testState, ws)
releaseIndex := uint64(40)
resp := testState.VarLockRelease(releaseIndex,
&structs.VarApplyStateRequest{
Op: structs.VarOpLockRelease,
Var: allVars[0],
})
must.NoError(t, resp.Error)
// Check that the index for the table was modified as expected.
afterReleaseIndex, err := testState.Index(TableVariables)
must.NoError(t, err)
must.Eq(t, releaseIndex, afterReleaseIndex)
// Ensure the lock was removed, but the variable was not.
sve, err := testState.GetVariable(ws, mv.Namespace, mv.Path)
must.NoError(t, err)
must.NotNil(t, sve)
must.Nil(t, sve.VariableMetadata.Lock)
// Ensure the create and modify indexes are populated correctly.
must.Eq(t, 20, sve.CreateIndex, must.Sprintf("%s: incorrect create index", sve.Path))
must.Eq(t, 40, sve.ModifyIndex, must.Sprintf("%s: incorrect modify index", sve.Path))
// Ensure the variable data didn't change
must.Eq(t, mv.Data, sve.Data)
})
t.Run("3 reacquire lock", func(t *testing.T) {
/* Reacquire the lock, testing the mechanism to lock a previously existing variable */
acquireIndex := uint64(60)
resp := testState.VarLockAcquire(acquireIndex,
&structs.VarApplyStateRequest{
Op: structs.VarOpLockAcquire,
Var: mv,
})
must.NoError(t, resp.Error)
// Check that the index for the table was modified as expected.
afterAcquireIndex, err := testState.Index(TableVariables)
must.NoError(t, err)
must.Eq(t, acquireIndex, afterAcquireIndex)
sve, err := testState.GetVariable(ws, mv.Namespace, mv.Path)
must.NoError(t, err)
// Ensure the create and modify indexes are populated correctly.
must.Eq(t, 20, sve.CreateIndex, must.Sprintf("%s: incorrect create index", sve.Path))
must.Eq(t, 60, sve.ModifyIndex, must.Sprintf("%s: incorrect modify index", sve.Path))
// Ensure the lock was persisted again.
must.Eq(t, "theLockID", sve.Lock.ID)
// Ensure the variable data didn't change
must.Eq(t, mv.Data, sve.Data)
})
}
func TestStateStore_ReleaseLock(t *testing.T) {
ci.Parallel(t)
testState := testStateStore(t)
insertIndex := uint64(20)
resp := testState.VarSet(insertIndex, &structs.VarApplyStateRequest{
Op: structs.VarOpSet,
Var: &structs.VariableEncrypted{
VariableMetadata: structs.VariableMetadata{
Path: "/non/lock/variable/path",
Namespace: "default",
},
VariableData: mock.VariableEncrypted().VariableData,
},
})
insertIndex++
must.NoError(t, resp.Error)
resp = testState.VarSet(insertIndex, &structs.VarApplyStateRequest{
Op: structs.VarOpSet,
Var: &structs.VariableEncrypted{
VariableMetadata: structs.VariableMetadata{
Path: "lock/variable/path",
Namespace: "default",
Lock: &structs.VariableLock{
ID: "theLockID",
},
},
VariableData: mock.VariableEncrypted().VariableData,
},
})
must.NoError(t, resp.Error)
testCases := []struct {
name string
lookUpPath string
lockID string
expErr error
expResult structs.VarOpResult
}{
{
name: "variable_not_found",
lookUpPath: "fake/path/",
expErr: errVarNotFound,
expResult: structs.VarOpResultError,
},
{
name: "variable_has_no_lock",
lookUpPath: "/non/lock/variable/path",
expErr: errLockNotFound,
expResult: structs.VarOpResultError,
},
{
name: "lock_id_doesn't_match",
lookUpPath: "lock/variable/path",
lockID: "wrongLockID",
expErr: nil,
expResult: structs.VarOpResultConflict,
},
{
name: "lock_released",
lookUpPath: "lock/variable/path",
lockID: "theLockID",
expErr: nil,
expResult: structs.VarOpResultOk,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
req := &structs.VarApplyStateRequest{
Op: structs.VarOpLockRelease,
Var: &structs.VariableEncrypted{
VariableMetadata: structs.VariableMetadata{
Path: tc.lookUpPath,
Namespace: "default",
},
},
}
if tc.lockID != "" {
req.Var.VariableMetadata.Lock = &structs.VariableLock{
ID: tc.lockID,
}
}
resp = testState.VarLockRelease(insertIndex, req)
if !errors.Is(tc.expErr, resp.Error) {
t.Fatalf("expected error, got %s", resp.Error)
}
must.Eq(t, tc.expResult, resp.Result)
})
}
}
func TestStateStore_Release(t *testing.T) {
ci.Parallel(t)
testState := testStateStore(t)
insertIndex := uint64(20)
resp := testState.VarSet(insertIndex, &structs.VarApplyStateRequest{
Op: structs.VarOpSet,
Var: &structs.VariableEncrypted{
VariableMetadata: structs.VariableMetadata{
Path: "/non/lock/variable/path",
Namespace: "default",
},
VariableData: mock.VariableEncrypted().VariableData,
},
})
insertIndex++
must.NoError(t, resp.Error)
resp = testState.VarSet(insertIndex, &structs.VarApplyStateRequest{
Op: structs.VarOpSet,
Var: &structs.VariableEncrypted{
VariableMetadata: structs.VariableMetadata{
Path: "lock/variable/path",
Namespace: "default",
Lock: &structs.VariableLock{
ID: "theLockID",
},
},
VariableData: mock.VariableEncrypted().VariableData,
},
})
must.NoError(t, resp.Error)
testCases := []struct {
name string
lookUpPath string
lockID string
expErr error
expResult structs.VarOpResult
}{
{
name: "variable_not_found",
lookUpPath: "fake/path/",
expErr: errVarNotFound,
expResult: structs.VarOpResultError,
},
{
name: "variable_has_no_lock",
lookUpPath: "/non/lock/variable/path",
expErr: errLockNotFound,
expResult: structs.VarOpResultError,
},
{
name: "lock_id_doesn't_match",
lookUpPath: "lock/variable/path",
lockID: "wrongLockID",
expErr: nil,
expResult: structs.VarOpResultConflict,
},
{
name: "lock_released",
lookUpPath: "lock/variable/path",
lockID: "theLockID",
expErr: nil,
expResult: structs.VarOpResultOk,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
req := &structs.VarApplyStateRequest{
Op: structs.VarOpLockRelease,
Var: &structs.VariableEncrypted{
VariableMetadata: structs.VariableMetadata{
Path: tc.lookUpPath,
Namespace: "default",
},
},
}
if tc.lockID != "" {
req.Var.VariableMetadata.Lock = &structs.VariableLock{
ID: tc.lockID,
}
}
resp = testState.VarLockRelease(insertIndex, req)
if !errors.Is(tc.expErr, resp.Error) {
t.Fatalf("expected error, got %s", resp.Error)
}
must.Eq(t, tc.expResult, resp.Result)
})
}
}
func getAllVariables(ss *StateStore, ws memdb.WatchSet) ([]*structs.VariableEncrypted, error) {
// List all the variables in the table
iter, err := ss.Variables(ws)
if err != nil {
return []*structs.VariableEncrypted{}, err
}
got := []*structs.VariableEncrypted{}
for raw := iter.Next(); raw != nil; raw = iter.Next() {
sv := raw.(*structs.VariableEncrypted)
var svCopy structs.VariableEncrypted
svCopy = sv.Copy()
got = append(got, &svCopy)
}
return got, nil
}

View File

@@ -5,11 +5,16 @@ package structs
import (
"bytes"
"encoding/json"
"errors"
"fmt"
"reflect"
"regexp"
"strings"
"time"
"github.com/hashicorp/go-multierror"
"github.com/hashicorp/nomad/helper/uuid"
)
const (
@@ -28,24 +33,59 @@ const (
VariablesListRPCMethod = "Variables.List"
// VariablesReadRPCMethod is the RPC method for fetching a variable
// according to its namepace and path.
// according to its namespace and path.
//
// Args: VariablesByNameRequest
// Reply: VariablesByNameResponse
VariablesReadRPCMethod = "Variables.Read"
// VariablesRenewLockRPCMethod is the RPC method for renewing the lease on
// a lock according to its namespace, path and lock ID.
//
// Args: VariablesRenewLockRequest
// Reply: VariablesRenewLockResponse
VariablesRenewLockRPCMethod = "Variables.RenewLock"
// maxVariableSize is the maximum size of the unencrypted contents of a
// variable. This size is deliberately set low and is not configurable, to
// discourage DoS'ing the cluster
maxVariableSize = 65536
// minVariableLockTTL and maxVariableLockTTL determine the range of valid durations for the
// TTL on a lock.They come from the experience on Consul.
minVariableLockTTL = 10 * time.Second
maxVariableLockTTL = 24 * time.Hour
// defaultLockTTL is the default value used to maintain a lock before it needs to
// be renewed. The actual value comes from the experience with Consul.
defaultLockTTL = 15 * time.Second
// defaultLockDelay is the default a lock will be blocked after the TTL
// went by without any renews. It is intended to prevent split brain situations.
// The actual value comes from the experience with Consul.
defaultLockDelay = 15 * time.Second
)
var (
errNoPath = errors.New("missing path")
errNoNamespace = errors.New("missing namespace")
errNoLock = errors.New("missing lock ID")
errWildCardNamespace = errors.New("can not target wildcard (\"*\")namespace")
errQuotaExhausted = errors.New("variables are limited to 64KiB in total size")
errNegativeDelayOrTTL = errors.New("Lock delay and TTL must be positive")
errInvalidTTL = errors.New("TTL must be between 10 seconds and 24 hours")
)
// VariableMetadata is the metadata envelope for a Variable, it is the list
// object and is shared data between an VariableEncrypted and a
// VariableDecrypted object.
type VariableMetadata struct {
Namespace string
Path string
Namespace string
Path string
// Lock represents a variable which is used for locking functionality.
Lock *VariableLock `json:",omitempty"`
CreateIndex uint64
CreateTime int64
ModifyIndex uint64
@@ -66,17 +106,167 @@ type VariableData struct {
}
// VariableDecrypted structs are returned from the Encrypter's decrypt
// method. Since they contains sensitive material, they should never be
// method. Since they contain sensitive material, they should never be
// persisted to disk.
type VariableDecrypted struct {
VariableMetadata
Items VariableItems
Items VariableItems `json:",omitempty"`
}
// VariableItems are the actual secrets stored in a variable. They are always
// encrypted and decrypted as a single unit.
type VariableItems map[string]string
// VariableLock represent a Nomad variable which is used to performing locking
// functionality such as leadership election.
type VariableLock struct {
// ID is generated by Nomad to provide a unique caller ID which can be used
// for renewals and unlocking.
ID string
// TTL describes the time-to-live of the current lock holder. The client
// must renew the lock before this TTL expires, otherwise the lock is
// considered lost.
TTL time.Duration
// LockDelay describes a grace period that exists after a lock is lost,
// before another client may acquire the lock. This helps protect against
// split-brains.
LockDelay time.Duration
}
// Equal performs an equality check on the two variable lock objects. It
// handles nil objects.
func (vl *VariableLock) Equal(vl2 *VariableLock) bool {
if vl == nil || vl2 == nil {
return vl == vl2
}
if vl.ID != vl2.ID {
return false
}
if vl.TTL != vl2.TTL {
return false
}
if vl.LockDelay != vl2.LockDelay {
return false
}
return true
}
// MarshalJSON implements the json.Marshaler interface and allows
// VariableLock.TTL and VariableLock.Delay to be marshaled correctly.
func (vl *VariableLock) MarshalJSON() ([]byte, error) {
type Alias VariableLock
exported := &struct {
TTL string
LockDelay string
*Alias
}{
TTL: vl.TTL.String(),
LockDelay: vl.LockDelay.String(),
Alias: (*Alias)(vl),
}
if vl.TTL == 0 {
exported.TTL = ""
}
if vl.LockDelay == 0 {
exported.LockDelay = ""
}
return json.Marshal(exported)
}
// UnmarshalJSON implements the json.Unmarshaler interface and allows
// VariableLock.TTL and VariableLock.Delay to be unmarshalled correctly.
func (vl *VariableLock) UnmarshalJSON(data []byte) (err error) {
type Alias VariableLock
aux := &struct {
TTL interface{}
LockDelay interface{}
*Alias
}{
Alias: (*Alias)(vl),
}
if err = json.Unmarshal(data, &aux); err != nil {
return err
}
if aux.TTL != nil {
switch v := aux.TTL.(type) {
case string:
if v != "" {
if vl.TTL, err = time.ParseDuration(v); err != nil {
return err
}
}
case float64:
vl.TTL = time.Duration(v)
}
}
if aux.LockDelay != nil {
switch v := aux.LockDelay.(type) {
case string:
if v != "" {
if vl.LockDelay, err = time.ParseDuration(v); err != nil {
return err
}
}
case float64:
vl.LockDelay = time.Duration(v)
}
}
return nil
}
// Copy creates a deep copy of the variable lock. This copy can then be safely
// modified. It handles nil objects.
func (vl *VariableLock) Copy() *VariableLock {
if vl == nil {
return nil
}
nvl := new(VariableLock)
*nvl = *vl
return nvl
}
func (vl *VariableLock) Canonicalize() {
// If the lock ID is empty, it means this is a creation of a lock on this variable.
if vl.ID == "" {
vl.ID = uuid.Generate()
}
if vl.LockDelay == 0 {
vl.LockDelay = defaultLockDelay
}
if vl.TTL == 0 {
vl.TTL = defaultLockTTL
}
}
func (vl *VariableLock) Validate() error {
var mErr *multierror.Error
if vl.LockDelay < 0 || vl.TTL < 0 {
mErr = multierror.Append(mErr, errNegativeDelayOrTTL)
}
if vl.TTL > maxVariableLockTTL || vl.TTL < minVariableLockTTL {
mErr = multierror.Append(mErr, errInvalidTTL)
}
return mErr.ErrorOrNil()
}
func (vi VariableItems) Size() uint64 {
var out uint64
for k, v := range vi {
@@ -94,8 +284,26 @@ func (vd VariableDecrypted) Equal(v2 VariableDecrypted) bool {
// Equal is a convenience method to provide similar equality checking syntax
// for metadata and the VariablesData or VariableItems struct
func (sv VariableMetadata) Equal(sv2 VariableMetadata) bool {
return sv == sv2
func (sv VariableMetadata) Equal(vm2 VariableMetadata) bool {
if sv.Namespace != vm2.Namespace {
return false
}
if sv.Path != vm2.Path {
return false
}
if sv.CreateIndex != vm2.CreateIndex {
return false
}
if sv.CreateTime != vm2.CreateTime {
return false
}
if sv.ModifyIndex != vm2.ModifyIndex {
return false
}
if sv.ModifyTime != vm2.ModifyTime {
return false
}
return sv.Lock.Equal(vm2.Lock)
}
// Equal performs deep equality checking on the cleartext items of a
@@ -167,18 +375,43 @@ func (vd VariableDecrypted) Validate() error {
if len(vd.Items) == 0 {
return errors.New("empty variables are invalid")
}
if vd.Items.Size() > maxVariableSize {
return errors.New("variables are limited to 64KiB in total size")
}
if err := validatePath(vd.Path); err != nil {
if err := ValidatePath(vd.Path); err != nil {
return err
}
if vd.Lock != nil {
return vd.Lock.Validate()
}
return nil
}
func validatePath(path string) error {
// A new variable can be crated just to support a lock, it doesn't require to hold
// any items and it will validate the lock.
func (vd VariableDecrypted) ValidateForLock() error {
var mErr multierror.Error
if vd.Namespace == AllNamespacesSentinel {
mErr.Errors = append(mErr.Errors, errWildCardNamespace)
return &mErr
}
if vd.Items.Size() > maxVariableSize {
return errQuotaExhausted
}
if err := ValidatePath(vd.Path); err != nil {
return err
}
return vd.Lock.Validate()
}
func ValidatePath(path string) error {
if len(path) == 0 {
return fmt.Errorf("variable requires path")
}
@@ -217,13 +450,27 @@ func (vd *VariableDecrypted) Canonicalize() {
if vd.Namespace == "" {
vd.Namespace = DefaultNamespace
}
if vd.Lock != nil {
vd.Lock.Canonicalize()
}
}
// Copy returns a fully hydrated copy of VariableMetadata that can be
// manipulated while ensuring the original is not touched.
func (sv *VariableMetadata) Copy() *VariableMetadata {
var out = *sv
return &out
if sv == nil {
return nil
}
nsl := new(VariableMetadata)
*nsl = *sv
if sv.Lock != nil {
nsl.Lock = sv.Lock.Copy()
}
return nsl
}
// GetNamespace returns the variable's namespace. Used for pagination.
@@ -241,6 +488,19 @@ func (sv VariableMetadata) GetCreateIndex() uint64 {
return sv.CreateIndex
}
// LockID returns the ID of the lock. In the event this is not held, or the
// variable is not a lock, this string will be empty.
func (sv *VariableMetadata) LockID() string {
if sv.Lock == nil {
return ""
}
return sv.Lock.ID
}
// IsLock is a helper to indicate whether the variable is being used for
// locking.
func (sv *VariableMetadata) IsLock() bool { return sv.Lock != nil }
// VariablesQuota is used to track the total size of variables entries per
// namespace. The total length of Variable.EncryptedData in bytes will be added
// to the VariablesQuota table in the same transaction as a write, update, or
@@ -273,6 +533,14 @@ const (
VarOpDelete VarOp = "delete"
VarOpDeleteCAS VarOp = "delete-cas"
VarOpCAS VarOp = "cas"
// VarOpLockAcquire is the variable operation used when attempting to
// acquire a variable lock.
VarOpLockAcquire VarOp = "lock-acquire"
// VarOpLockRelease is the variable operation used when attempting to
// release a held variable lock.
VarOpLockRelease VarOp = "lock-release"
)
// VarOpResult constants give possible operations results from a transaction.
@@ -401,3 +669,34 @@ type VariablesReadResponse struct {
Data *VariableDecrypted
QueryMeta
}
// VariablesRenewLockRequest is used to renew the lease on a lock. This request
// behaves like a write because the renewal needs to be forwarded to the leader
// where the timers and lock work is kept.
type VariablesRenewLockRequest struct {
//Namespace string
Path string
LockID string
WriteRequest
}
func (v *VariablesRenewLockRequest) Validate() error {
var mErr multierror.Error
if v.Path == "" {
mErr.Errors = append(mErr.Errors, errNoPath)
}
if v.LockID == "" {
mErr.Errors = append(mErr.Errors, errNoLock)
}
return mErr.ErrorOrNil()
}
// VariablesRenewLockResponse is sent back to the user to inform them of success or failure
// of the renewal process.
type VariablesRenewLockResponse struct {
VarMeta *VariableMetadata
WriteMeta
}

View File

@@ -4,13 +4,261 @@
package structs
import (
"errors"
"fmt"
"testing"
"time"
"github.com/hashicorp/nomad/ci"
"github.com/stretchr/testify/require"
"github.com/shoenig/test/must"
)
func TestVariableMetadata_Equal(t *testing.T) {
ci.Parallel(t)
testCases := []struct {
name string
inputMetadata VariableMetadata
inputMetadataFn VariableMetadata
expectedOutput bool
}{
{
name: "no lock equal",
inputMetadata: VariableMetadata{
Namespace: "default",
Path: "custom/test/path",
Lock: nil,
CreateIndex: 10,
CreateTime: 1687251815,
ModifyIndex: 100,
ModifyTime: 1687251868,
},
inputMetadataFn: VariableMetadata{
Namespace: "default",
Path: "custom/test/path",
Lock: nil,
CreateIndex: 10,
CreateTime: 1687251815,
ModifyIndex: 100,
ModifyTime: 1687251868,
},
expectedOutput: true,
},
{
name: "no lock unequal",
inputMetadata: VariableMetadata{
Namespace: "default",
Path: "custom/test/path",
Lock: nil,
CreateIndex: 10,
CreateTime: 1687251815,
ModifyIndex: 100,
ModifyTime: 1687251876,
},
inputMetadataFn: VariableMetadata{
Namespace: "default",
Path: "custom/test/path",
Lock: nil,
CreateIndex: 10,
CreateTime: 1687251815,
ModifyIndex: 200,
ModifyTime: 1687251885,
},
expectedOutput: false,
},
{
name: "lock equal",
inputMetadata: VariableMetadata{
Namespace: "default",
Path: "custom/test/path",
Lock: &VariableLock{
ID: "896bdbef-8ce7-4b1d-9b4c-4e6c5639196d",
TTL: 20 * time.Second,
LockDelay: 5 * time.Second,
},
CreateIndex: 10,
CreateTime: 1687251815,
ModifyIndex: 100,
ModifyTime: 1687251868,
},
inputMetadataFn: VariableMetadata{
Namespace: "default",
Path: "custom/test/path",
Lock: &VariableLock{
ID: "896bdbef-8ce7-4b1d-9b4c-4e6c5639196d",
TTL: 20 * time.Second,
LockDelay: 5 * time.Second,
},
CreateIndex: 10,
CreateTime: 1687251815,
ModifyIndex: 100,
ModifyTime: 1687251868,
},
expectedOutput: true,
},
{
name: "lock unequal",
inputMetadata: VariableMetadata{
Namespace: "default",
Path: "custom/test/path",
Lock: &VariableLock{
ID: "896bdbef-8ce7-4b1d-9b4c-4e6c5639196d",
TTL: 20 * time.Second,
LockDelay: 5 * time.Second,
},
CreateIndex: 10,
CreateTime: 1687251815,
ModifyIndex: 100,
ModifyTime: 1687251876,
},
inputMetadataFn: VariableMetadata{
Namespace: "default",
Path: "custom/test/path",
Lock: &VariableLock{
ID: "896bdbef-8ce7-4b1d-9b4c-4e6c5639196d",
TTL: 20 * time.Second,
LockDelay: 15 * time.Second,
},
CreateIndex: 10,
CreateTime: 1687251815,
ModifyIndex: 100,
ModifyTime: 1687251876,
},
expectedOutput: false,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
must.Eq(t, tc.expectedOutput, tc.inputMetadata.Equal(tc.inputMetadataFn))
})
}
}
func TestVariableMetadata_Copy(t *testing.T) {
ci.Parallel(t)
testCases := []struct {
name string
inputVariableMetadata *VariableMetadata
}{
{
name: "no lock",
inputVariableMetadata: &VariableMetadata{
Namespace: "default",
Path: "custom/test/path",
Lock: nil,
CreateIndex: 10,
CreateTime: 1687251815,
ModifyIndex: 100,
ModifyTime: 1687251876,
},
},
{
name: "lock",
inputVariableMetadata: &VariableMetadata{
Namespace: "default",
Path: "custom/test/path",
Lock: &VariableLock{
ID: "896bdbef-8ce7-4b1d-9b4c-4e6c5639196d",
TTL: 20 * time.Second,
LockDelay: 15 * time.Second,
},
CreateIndex: 10,
CreateTime: 1687251815,
ModifyIndex: 100,
ModifyTime: 1687251876,
},
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
actualOutput := tc.inputVariableMetadata.Copy()
must.Eq(t, tc.inputVariableMetadata, actualOutput)
must.NotEqOp(t,
fmt.Sprintf("%p", tc.inputVariableMetadata),
fmt.Sprintf("%p", actualOutput))
if tc.inputVariableMetadata.Lock != nil {
must.NotEqOp(t,
fmt.Sprintf("%p", tc.inputVariableMetadata.Lock),
fmt.Sprintf("%p", actualOutput.Lock))
}
})
}
}
func TestVariableMetadata_LockID(t *testing.T) {
ci.Parallel(t)
testCases := []struct {
name string
inputVariableMetadata *VariableMetadata
expectedOutput string
}{
{
name: "nil lock",
inputVariableMetadata: &VariableMetadata{
Lock: nil,
},
expectedOutput: "",
},
{
name: "empty ID",
inputVariableMetadata: &VariableMetadata{
Lock: &VariableLock{ID: ""},
},
expectedOutput: "",
},
{
name: "populated ID",
inputVariableMetadata: &VariableMetadata{
Lock: &VariableLock{ID: "mylovelylovelyid"},
},
expectedOutput: "mylovelylovelyid",
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
must.Eq(t, tc.expectedOutput, tc.inputVariableMetadata.LockID())
})
}
}
func TestVariableMetadata_IsLock(t *testing.T) {
ci.Parallel(t)
testCases := []struct {
name string
inputVariableMetadata *VariableMetadata
expectedOutput bool
}{
{
name: "nil",
inputVariableMetadata: &VariableMetadata{
Lock: nil,
},
expectedOutput: false,
},
{
name: "not nil",
inputVariableMetadata: &VariableMetadata{
Lock: &VariableLock{},
},
expectedOutput: true,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
must.Eq(t, tc.expectedOutput, tc.inputVariableMetadata.IsLock())
})
}
}
func TestStructs_VariableDecrypted_Copy(t *testing.T) {
ci.Parallel(t)
n := time.Now()
@@ -30,9 +278,9 @@ func TestStructs_VariableDecrypted_Copy(t *testing.T) {
},
}
sv2 := sv.Copy()
require.True(t, sv.Equal(sv2), "sv and sv2 should be equal")
must.True(t, sv.Equal(sv2), must.Sprintf("sv and sv2 should be equal"))
sv2.Items["new"] = "new"
require.False(t, sv.Equal(sv2), "sv and sv2 should not be equal")
must.False(t, sv.Equal(sv2), must.Sprintf("sv and sv2 should not be equal"))
}
func TestStructs_VariableDecrypted_Validate(t *testing.T) {
@@ -65,9 +313,102 @@ func TestStructs_VariableDecrypted_Validate(t *testing.T) {
sv.Path = tc.path
err := sv.Validate()
if tc.ok {
require.NoError(t, err, "should not get error for: %s", tc.path)
must.NoError(t, err, must.Sprintf("should not get error for: %s", tc.path))
} else {
require.Error(t, err, "should get error for: %s", tc.path)
must.Error(t, err, must.Sprintf("should get error for: %s", tc.path))
}
}
}
func TestStructs_VariablesRenewLockRequest_Validate(t *testing.T) {
ci.Parallel(t)
testCases := []struct {
name string
request *VariablesRenewLockRequest
expErr error
}{
{
name: "missing_lockID",
request: &VariablesRenewLockRequest{
Path: "path",
},
expErr: errNoLock,
},
{
name: "missing_path",
request: &VariablesRenewLockRequest{
LockID: "lockID",
},
expErr: errNoPath,
},
{
name: "valid_request",
request: &VariablesRenewLockRequest{
Path: "path",
LockID: "lockID",
},
expErr: nil,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
err := tc.request.Validate()
if !errors.Is(err, tc.expErr) {
t.Errorf("Expected error %v, but got error %v", tc.expErr, err)
}
})
}
}
func TestStructs_Lock_Validate(t *testing.T) {
ci.Parallel(t)
testCases := []struct {
name string
lock *VariableLock
expErr error
}{
{
name: "lock_delay_is_negative",
lock: &VariableLock{
TTL: 5 * time.Second,
LockDelay: -5 * time.Second,
},
expErr: errNegativeDelayOrTTL,
},
{
name: "lock_ttl_is_negative",
lock: &VariableLock{
TTL: -5 * time.Second,
LockDelay: 5 * time.Second,
},
expErr: errNegativeDelayOrTTL,
},
{
name: "lock_ttl_is_bigger_than_max",
lock: &VariableLock{
TTL: maxVariableLockTTL + 5*time.Second,
LockDelay: 5 * time.Second,
},
expErr: errInvalidTTL,
},
{
name: "lock_ttl_is_smaller_than_min",
lock: &VariableLock{
TTL: 5 * time.Second,
LockDelay: minVariableLockTTL - 5*time.Second,
},
expErr: errInvalidTTL,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
err := tc.lock.Validate()
if !errors.Is(err, tc.expErr) {
t.Errorf("Expected error %v, but got error %v", tc.expErr, err)
}
})
}
}

View File

@@ -20,6 +20,27 @@ import (
"github.com/hashicorp/nomad/nomad/structs"
)
const (
maxAttemptsToRaftApply = 6
)
var (
errVarAlreadyLocked = structs.NewErrRPCCoded(http.StatusBadRequest, "variable already holds a lock")
errVarNotFound = structs.NewErrRPCCoded(http.StatusNotFound, "variable doesn't exist")
errLockNotFound = structs.NewErrRPCCoded(http.StatusConflict, "variable doesn't hold a lock")
errVarIsLocked = structs.NewErrRPCCoded(http.StatusConflict, "attempting to modify locked variable")
errMissingLockInfo = structs.NewErrRPCCoded(http.StatusBadRequest, "missing lock information")
errLockOnVarCreation = structs.NewErrRPCCoded(http.StatusBadRequest, "variable should not contain lock definition")
errItemsOnRelease = structs.NewErrRPCCoded(http.StatusBadRequest, "lock release operation doesn't take variable items")
errNoPath = structs.NewErrRPCCoded(http.StatusBadRequest, "delete requires a Path")
)
type variableTimers interface {
CreateVariableLockTTLTimer(structs.VariableEncrypted)
RemoveVariableLockTTLTimer(structs.VariableEncrypted)
RenewTTLTimer(structs.VariableEncrypted) error
}
// Variables encapsulates the variables RPC endpoint which is
// callable via the Variables RPCs and externally via the "/v1/var{s}"
// HTTP API.
@@ -27,12 +48,13 @@ type Variables struct {
srv *Server
ctx *RPCContext
logger hclog.Logger
timers variableTimers
encrypter *Encrypter
}
func NewVariablesEndpoint(srv *Server, ctx *RPCContext, enc *Encrypter) *Variables {
return &Variables{srv: srv, ctx: ctx, logger: srv.logger.Named("variables"), encrypter: enc}
return &Variables{srv: srv, ctx: ctx, logger: srv.logger.Named("variables"), encrypter: enc, timers: srv}
}
// Apply is used to apply a SV update request to the data store.
@@ -49,12 +71,14 @@ func (sv *Variables) Apply(args *structs.VariablesApplyRequest, reply *structs.V
defer metrics.MeasureSince([]string{
"nomad", "variables", "apply", string(args.Op)}, time.Now())
// TODO: Add metrics for acquire and release if the operation is lock related
// Check if the Namespace is explicitly set on the variable. If
// not, use the RequestNamespace
if args.Var == nil {
return fmt.Errorf("variable must not be nil")
}
// Check if the Namespace is explicitly set on the variable. If
// not, use the RequestNamespace
targetNS := args.Var.Namespace
if targetNS == "" {
targetNS = args.RequestNamespace()
@@ -66,15 +90,30 @@ func (sv *Variables) Apply(args *structs.VariablesApplyRequest, reply *structs.V
return fmt.Errorf("all servers must be running version %v or later to apply variables", minVersionKeyring)
}
canRead, err := svePreApply(sv, args, args.Var)
// Perform the ACL resolution.
aclObj, err := sv.srv.ResolveACL(args)
if err != nil {
return err
}
// IF ACL is being used,
if aclObj != nil {
err := hasOperationPermissions(aclObj, args.Var.Namespace, args.Var.Path, args.Op)
if err != nil {
return err
}
}
err = canonicalizeAndValidate(args)
if err != nil {
return structs.NewErrRPCCoded(http.StatusBadRequest, err.Error())
}
var ev *structs.VariableEncrypted
switch args.Op {
case structs.VarOpSet, structs.VarOpCAS:
case structs.VarOpSet, structs.VarOpCAS, structs.VarOpLockAcquire,
structs.VarOpLockRelease:
ev, err = sv.encrypt(args.Var)
if err != nil {
return fmt.Errorf("variable error: encrypt: %w", err)
@@ -82,6 +121,7 @@ func (sv *Variables) Apply(args *structs.VariablesApplyRequest, reply *structs.V
now := time.Now().UnixNano()
ev.CreateTime = now // existing will override if it exists
ev.ModifyTime = now
case structs.VarOpDelete, structs.VarOpDeleteCAS:
ev = &structs.VariableEncrypted{
VariableMetadata: structs.VariableMetadata{
@@ -100,76 +140,124 @@ func (sv *Variables) Apply(args *structs.VariablesApplyRequest, reply *structs.V
}
// Apply the update.
out, index, err := sv.srv.raftApply(structs.VarApplyStateRequestType, sveArgs)
o, index, err := sv.srv.raftApply(structs.VarApplyStateRequestType, sveArgs)
if err != nil {
return fmt.Errorf("raft apply failed: %w", err)
}
r, err := sv.makeVariablesApplyResponse(args, out.(*structs.VarApplyStateResponse), canRead)
out, _ := o.(*structs.VarApplyStateResponse)
// The return value depends on the operation results and the callers permissions
r, err := sv.makeVariablesApplyResponse(args, out, aclObj)
if err != nil {
return err
}
*reply = *r
reply.Index = index
if out.IsOk() {
switch args.Op {
case structs.VarOpLockAcquire:
sv.timers.CreateVariableLockTTLTimer(ev.Copy())
case structs.VarOpLockRelease:
sv.timers.RemoveVariableLockTTLTimer(ev.Copy())
}
}
return nil
}
func svePreApply(sv *Variables, args *structs.VariablesApplyRequest, vd *structs.VariableDecrypted) (canRead bool, err error) {
func hasReadPermission(aclObj *acl.ACL, namespace, path string) bool {
return aclObj.AllowVariableOperation(namespace,
path, acl.VariablesCapabilityRead, nil)
}
canRead = false
var aclObj *acl.ACL
func hasOperationPermissions(aclObj *acl.ACL, namespace, path string, op structs.VarOp) error {
// Perform the ACL resolution.
if aclObj, err = sv.srv.ResolveACL(args); err != nil {
return
} else if aclObj != nil {
hasPerm := func(perm string) bool {
return aclObj.AllowVariableOperation(args.Var.Namespace,
args.Var.Path, perm, nil)
}
canRead = hasPerm(acl.VariablesCapabilityRead)
switch args.Op {
case structs.VarOpSet, structs.VarOpCAS:
if !hasPerm(acl.VariablesCapabilityWrite) {
err = structs.ErrPermissionDenied
return
}
case structs.VarOpDelete, structs.VarOpDeleteCAS:
if !hasPerm(acl.VariablesCapabilityDestroy) {
err = structs.ErrPermissionDenied
return
}
default:
err = fmt.Errorf("svPreApply: unexpected VarOp received: %q", args.Op)
return
}
} else {
// ACLs are not enabled.
canRead = true
hasPerm := func(perm string) bool {
return aclObj.AllowVariableOperation(namespace,
path, perm, nil)
}
switch args.Op {
case structs.VarOpSet, structs.VarOpCAS:
args.Var.Canonicalize()
if err = args.Var.Validate(); err != nil {
return
switch op {
case structs.VarOpSet, structs.VarOpCAS, structs.VarOpLockAcquire,
structs.VarOpLockRelease:
if !hasPerm(acl.VariablesCapabilityWrite) {
return structs.ErrPermissionDenied
}
case structs.VarOpDelete, structs.VarOpDeleteCAS:
if args.Var == nil || args.Var.Path == "" {
err = fmt.Errorf("delete requires a Path")
return
if !hasPerm(acl.VariablesCapabilityDestroy) {
return structs.ErrPermissionDenied
}
default:
return fmt.Errorf("svPreApply: unexpected VarOp received: %q", op)
}
return
return nil
}
func canonicalizeAndValidate(args *structs.VariablesApplyRequest) error {
switch args.Op {
case structs.VarOpLockAcquire:
// In case the user wants to use the default values so no lock data was provided.
if args.Var.VariableMetadata.Lock == nil {
args.Var.VariableMetadata.Lock = &structs.VariableLock{}
}
args.Var.Canonicalize()
err := args.Var.ValidateForLock()
if err != nil {
return structs.NewErrRPCCoded(http.StatusBadRequest, err.Error())
}
return nil
case structs.VarOpSet, structs.VarOpCAS:
// Avoid creating a variable with a lock that is not actionable
if args.Var.VariableMetadata.Lock != nil &&
(args.Var.VariableMetadata.Lock.LockDelay != 0 || args.Var.VariableMetadata.Lock.TTL != 0) {
return structs.NewErrRPCCoded(http.StatusBadRequest, errLockOnVarCreation.Error())
}
args.Var.Canonicalize()
err := args.Var.Validate()
if err != nil {
return structs.NewErrRPCCoded(http.StatusBadRequest, err.Error())
}
return nil
case structs.VarOpDelete, structs.VarOpDeleteCAS:
if args.Var == nil || args.Var.Path == "" {
return errNoPath
}
case structs.VarOpLockRelease:
if args.Var == nil || args.Var.Lock == nil ||
args.Var.Lock.ID == "" {
return errMissingLockInfo
}
// If the operation is a lock release and there are items on the variable
// reject the request, release doesn't update the variable.
if args.Var.Items != nil || len(args.Var.Items) != 0 {
return errItemsOnRelease
}
return structs.ValidatePath(args.Var.Path)
}
return nil
}
// MakeVariablesApplyResponse merges the output of this VarApplyStateResponse with the
// VariableDataItems
func (sv *Variables) makeVariablesApplyResponse(
req *structs.VariablesApplyRequest, eResp *structs.VarApplyStateResponse,
canRead bool) (*structs.VariablesApplyResponse, error) {
aclObj *acl.ACL) (*structs.VariablesApplyResponse, error) {
out := structs.VariablesApplyResponse{
Op: eResp.Op,
@@ -179,6 +267,15 @@ func (sv *Variables) makeVariablesApplyResponse(
WriteMeta: eResp.WriteMeta,
}
// The read permission modify the way the response is populated. If ACL is not
// used, read permission is granted by default and every call is treated as management.
var canRead bool = true
var isManagement = true
if aclObj != nil {
canRead = hasReadPermission(aclObj, req.Var.Namespace, req.Var.Path)
isManagement = aclObj.IsManagement()
}
if eResp.IsOk() {
if eResp.WrittenSVMeta != nil {
// The writer is allowed to read their own write
@@ -186,9 +283,18 @@ func (sv *Variables) makeVariablesApplyResponse(
VariableMetadata: *eResp.WrittenSVMeta,
Items: req.Var.Items.Copy(),
}
// Verify the caller is providing the correct lockID, meaning it is the
// lock holder and has access to the lock information or is a management call.
// If locked, remove the lock information from response.
if !(isCallerOwner(req, eResp.WrittenSVMeta) || isManagement) {
out.Output.VariableMetadata.Lock = nil
}
}
return &out, nil
}
if eResp.IsError() {
return &out, eResp.Error
}
@@ -240,7 +346,7 @@ func (sv *Variables) Read(args *structs.VariablesReadRequest, reply *structs.Var
defer metrics.MeasureSince([]string{"nomad", "variables", "read"}, time.Now())
_, _, err := sv.handleMixedAuthEndpoint(args.QueryOptions,
aclObj, err := sv.handleMixedAuthEndpoint(args.QueryOptions,
acl.PolicyRead, args.Path)
if err != nil {
return err
@@ -259,11 +365,17 @@ func (sv *Variables) Read(args *structs.VariablesReadRequest, reply *structs.Var
// Setup the output
reply.Data = nil
if out != nil {
dv, err := sv.decrypt(out)
if err != nil {
return err
}
ov := dv.Copy()
if !(aclObj != nil && aclObj.IsManagement()) {
ov.Lock = nil
}
reply.Data = &ov
reply.Index = out.ModifyIndex
} else {
@@ -336,6 +448,7 @@ func (sv *Variables) List(
if !strings.HasPrefix(v.Path, args.Prefix) {
return false, nil
}
// Note: the authorize method modifies the aclObj parameter.
err := sv.authorize(aclObj, claims, v.Namespace, acl.PolicyList, v.Path)
return err == nil, nil
},
@@ -352,6 +465,11 @@ func (sv *Variables) List(
func(raw interface{}) error {
sv := raw.(*structs.VariableEncrypted)
svStub := sv.VariableMetadata
if !(aclObj != nil && aclObj.IsManagement()) {
svStub.Lock = nil
}
svs = append(svs, &svStub)
return nil
})
@@ -427,6 +545,8 @@ func (sv *Variables) listAllVariables(
if !strings.HasPrefix(v.Path, args.Prefix) {
return false, nil
}
// Note: the authorize method modifies the aclObj parameter.
err := sv.authorize(aclObj, claims, v.Namespace, acl.PolicyList, v.Path)
return err == nil, nil
},
@@ -439,6 +559,11 @@ func (sv *Variables) listAllVariables(
func(raw interface{}) error {
v := raw.(*structs.VariableEncrypted)
svStub := v.VariableMetadata
if !(aclObj != nil && aclObj.IsManagement()) {
svStub.Lock = nil
}
svs = append(svs, &svStub)
return nil
})
@@ -474,6 +599,7 @@ func (sv *Variables) encrypt(v *structs.VariableDecrypted) (*structs.VariableEnc
ev := structs.VariableEncrypted{
VariableMetadata: v.VariableMetadata,
}
ev.Data, ev.KeyID, err = sv.encrypter.Encrypt(b)
if err != nil {
return nil, err
@@ -499,7 +625,7 @@ func (sv *Variables) decrypt(v *structs.VariableEncrypted) (*structs.VariableDec
// handleMixedAuthEndpoint is a helper to handle auth on RPC endpoints that can
// either be called by external clients or by workload identity
func (sv *Variables) handleMixedAuthEndpoint(args structs.QueryOptions, policy, pathOrPrefix string) (*acl.ACL, *structs.IdentityClaims, error) {
func (sv *Variables) handleMixedAuthEndpoint(args structs.QueryOptions, policy, pathOrPrefix string) (*acl.ACL, error) {
var aclObj *acl.ACL
var err error
@@ -507,19 +633,23 @@ func (sv *Variables) handleMixedAuthEndpoint(args structs.QueryOptions, policy,
if aclToken != nil {
aclObj, err = sv.srv.ResolveACLForToken(aclToken)
if err != nil {
return nil, nil, err
return nil, err
}
}
claims := args.GetIdentity().GetClaims()
// Note: the authorize method modifies the aclObj parameter.
err = sv.authorize(aclObj, claims, args.RequestNamespace(), policy, pathOrPrefix)
if err != nil {
return aclObj, claims, err
return aclObj, err
}
return aclObj, claims, nil
return aclObj, nil
}
// The authorize method modifies the aclObj parameter. In case the incoming request
// uses identity workload claims, the aclObj is populated. This logic will be
// updated when the work to eliminate nil ACLs is merged.
func (sv *Variables) authorize(aclObj *acl.ACL, claims *structs.IdentityClaims, ns, policy, pathOrPrefix string) error {
if aclObj == nil && claims == nil {
@@ -538,8 +668,9 @@ func (sv *Variables) authorize(aclObj *acl.ACL, claims *structs.IdentityClaims,
// Check the workload-associated policies and automatic task access to
// variables.
var err error
if claims != nil {
aclObj, err := sv.srv.ResolveClaims(claims)
aclObj, err = sv.srv.ResolveClaims(claims)
if err != nil {
return err // returns internal errors only
}
@@ -579,3 +710,83 @@ func (sv *Variables) groupForAlloc(claims *structs.IdentityClaims) (string, erro
}
return alloc.TaskGroup, nil
}
// RenewLock is used to apply a SV renew lock operation on a variable to maintain the lease.
func (sv *Variables) RenewLock(args *structs.VariablesRenewLockRequest, reply *structs.VariablesRenewLockResponse) error {
authErr := sv.srv.Authenticate(sv.ctx, args)
if done, err := sv.srv.forward(structs.VariablesRenewLockRPCMethod, args, args, reply); done {
return err
}
sv.srv.MeasureRPCRate("variables", structs.RateMetricWrite, args)
if authErr != nil {
return structs.ErrPermissionDenied
}
defer metrics.MeasureSince([]string{
"nomad", "variables", "lock", "renew"}, time.Now())
aclObj, err := sv.srv.ResolveACL(args)
if err != nil {
return err
}
// ACLs are enabled, check for the correct permissions
if aclObj != nil {
if !aclObj.AllowVariableOperation(args.WriteRequest.Namespace, args.Path,
acl.VariablesCapabilityWrite, nil) {
return structs.ErrPermissionDenied
}
}
if err := args.Validate(); err != nil {
return err
}
// Get the variable from the SS to verify it exists and is currently lock
stateSnapshot, err := sv.srv.State().Snapshot()
if err != nil {
return err
}
encryptedVar, err := stateSnapshot.GetVariable(nil, args.WriteRequest.Namespace, args.Path)
if err != nil {
return err
}
if encryptedVar == nil {
return errVarNotFound
}
if encryptedVar.Lock == nil {
return errLockNotFound
}
// Verify the caller is providing the correct lockID, meaning it is the lock holder and
// can renew the lock.
if encryptedVar.Lock.ID != args.LockID {
return errVarIsLocked
}
// if the lock exists in the variable, but not in the timer, it means
// it expired and cant be renewed anymore. The delay will take care of
// removing the lock from the variable when it expires.
err = sv.timers.RenewTTLTimer(encryptedVar.Copy())
if err != nil {
return errVarIsLocked
}
updatedVar := encryptedVar.Copy()
reply.VarMeta = &updatedVar.VariableMetadata
reply.Index = encryptedVar.ModifyIndex
return nil
}
func isCallerOwner(req *structs.VariablesApplyRequest, respVarMeta *structs.VariableMetadata) bool {
reqLock := req.Var.VariableMetadata.Lock
savedLock := respVarMeta.Lock
return reqLock != nil &&
savedLock != nil &&
reqLock.ID == savedLock.ID
}

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,21 @@
---
layout: api
page_title: Variables - HTTP API
description: |-
The /var endpoints are used to query for and interact with variables and variable
locking.
---
# Vars HTTP API
The `/var` and `/vars` endpoints are used to query for and interact with
variables, and set up locks and leases over them.
See the [Variables][] documentation for information how these capabilities are
used. For a CLI to perform these operations manually, please see the
documentation for the [`nomad var`][] commands.
Please choose a sub-section in the navigation for more information
[`nomad var`]: /nomad/docs/commands/var
[Variables]: /nomad/docs/concepts/variables

View File

@@ -0,0 +1,211 @@
---
layout: api
page_title: Variable Locks - HTTP API
description: The /var endpoints are used to query for and interact with variables and locks.
---
# Locks HTTP API
The `/var` endpoint is used to hold, renew and release a lock over a variable.
## Lock Variable
The endpoint to create a variable can also be used to hold a lock and interact with
it through the use of a parameter defining the operation to be performed.
| Method | Path | Produces |
|--------|--------------------------------------|--------------------|
| `PUT` | `/v1/var/:var_path?<lock-operation>` | `application/json` |
### Parameters
The lock operation parameter can be:
- `lock-acquire`: When used, the call will introduce a lock over the variable if
it exists, or create a new one if it doesn't. The lock ID will be returned in the
response and it must be provided to perform any other operation over the lock.
The variable items can be updated at any time using the lock ID, but the lock
parameters are unmmutable, attempting to modify them while a lock is present will
generate an error.
In the case of attempting to acquire a variable that is already locked, a conflict
response will be returned.
The lock-acquire operation will override the variable items if new values are
present.
#### Sample Request
```shell-session
$ curl \
-XPUT -d@spec.nsv.json \
https://localhost:4646/v1/var/example/first?lock-acquire
```
#### Sample Payload
```json
{
"Namespace": "prod",
"Path": "example/first",
"Items": {
"user": "me",
"password": "passw0rd1"
},
"Lock": {
"TTL": "15s",
"LockDelay": "1m"
}
}
```
#### Sample Response
The response body returns the created or updated variable including the lock
parameters and ID, along with metadata created by the server:
```json
{
"CreateIndex": 15,
"CreateTime": 1694552155379696000,
"Items": {
"user": "me",
"password": "passw0rd1"
},
"Lock": {
"TTL": "15s",
"LockDelay": "15s",
"ID": "670c7248-e2ef-f982-e4c5-f4437f75f1e4"
},
"ModifyIndex": 16,
"ModifyTime": 1694552206138804000,
"Namespace": "prod",
"Path": "example/first"
}
```
- `lock-renew`: A valid call to lock renew needs to be placed before the lock's
TTL is up in order to mantain the variable locked. A valid call must include the
lock ID as part of the request body. If the lock TTL is up without a renewal or
release calls, the variable will remain unlockable for at least the lock delay.
#### Sample Request
```shell-session
$ curl \
-XPUT -d@spec.nsv.json \
https://localhost:4646/v1/var/example/first?lock-renew
```
#### Sample Payload
```json
{
"Path": "example/first",
"Namespace": "prod",
"Lock": {
"ID": "670c7248-e2ef-f982-e4c5-f4437f75f1e4"
}
}
```
#### Sample Response
The response body only returns metadata created by the server and the lock
parameters:
```json
{
"CreateIndex": 11,
"CreateTime": 1694555280887153000,
"Lock": {
"TTL": "15s",
"LockDelay": "15s",
"ID": "670c7248-e2ef-f982-e4c5-f4437f75f1e4"
},
"ModifyIndex": 43,
"ModifyTime": 1694556175092779000,
"Namespace": "prod",
"Path": "example/first"
}
```
- `lock-release`: A call to the endpoint with the `lock-release` operation will
immediately remove the lock over the variable, making it modifiable without
restrictions again.
The lock-release operation will not override the variable items, if the request
body contains any item, it will generate a bad request response.
#### Sample Request
```shell-session
$ curl \
-XPUT -d@spec.nsv.json \
https://localhost:4646/v1/var/example/first?lock-release
```
#### Sample Payload
```json
{
"Path": "example/first",
"Namespace": "prod",
"Lock": {
"ID": "670c7248-e2ef-f982-e4c5-f4437f75f1e4"
}
}
```
#### Sample Response
The response body returns the released variable along with metadata
created by the server:
```json
{
"CreateIndex": 11,
"CreateTime": 1694555280887153000,
"ModifyIndex": 66,
"ModifyTime": 1694556922600469000,
"Namespace": "prod",
"Path": "example/first"
}
```
### Sample Response for Conflict
In the case of an attempt to lock, renew or modify a locked variable
without the correct ID, the API will return HTTP error code
409 and a response body showing the conflicting variable. If the provided ACL
token does not also have `read` permissions to the variable path, the response
will include only metadata and not the `Items` field:
```json
{
"CreateIndex": 0,
"CreateTime": 0,
"Items": null,
"Lock": null,
"ModifyIndex": 0,
"ModifyTime": 0,
"Namespace": "default",
"Path": "example/first"
}
```
## Restrictions
When creating a new variable using the lock-acquire operation, all the known
[restrictions][] regarding the path and size of the content apply, but unlike
regular variables, locked variables can be created with or without any items.
The lock TTL and Delay must be values between 10 seconds and 24 hours.
[Variables]: /nomad/docs/concepts/variables
[restrictions]: /nomad/api-docs/variables/variables#restrictions
[`nomad var`]: /nomad/docs/commands/var
[blocking queries]: /nomad/api-docs#blocking-queries
[required ACLs]: /nomad/api-docs#acls
[RFC3986]: https://www.rfc-editor.org/rfc/rfc3986#section-2

View File

@@ -9,9 +9,8 @@ description: The /var endpoints are used to query for and interact with variable
The `/var` and `/vars` endpoints are used to query for and interact with
variables.
See the [Variables] documentation for information how these capabilities are
used. For a CLI to perform these operations manually, please see the
documentation for the [`nomad var`] commands.
See the [Variables][] documentation for information how these capabilities are
used.
## List Variables
@@ -150,6 +149,10 @@ The table below shows this endpoint's support for [blocking queries] and
0, the variable is only created if it does not already exist. This paradigm
allows check-and-set style updates.
- `lock-operation` `(string: <unset>)` - This endpoint can also be used to create
and hold a lock over a variable, refer to the [locks section][] for more
information.
## Restrictions
Variable paths are restricted to [RFC3986][] URL-safe characters that don't
@@ -290,9 +293,8 @@ the response will include only metadata and not the `Items` field:
}
```
[Variables]: /nomad/docs/concepts/variables
[`nomad var`]: /nomad/docs/commands/var
[locks section]:/nomad/api-docs/variables/locks
[blocking queries]: /nomad/api-docs#blocking-queries
[required ACLs]: /nomad/api-docs#acls
[RFC3986]: https://www.rfc-editor.org/rfc/rfc3986#section-2

View File

@@ -21,6 +21,7 @@ subcommands are available:
- [`var get`][get] - Retrieve a variable
- [`var put`][put] - Insert or update a variable
- [`var purge`][purge] - Permanently delete a variable
- [`var lock`][lock] - Acquire a lock over a variable
## Examples
@@ -62,3 +63,4 @@ user = dba
[list]: /nomad/docs/commands/var/list
[put]: /nomad/docs/commands/var/put
[purge]: /nomad/docs/commands/var/purge
[lock]: /nomad/docs/commands/var/lock

View File

@@ -0,0 +1,100 @@
---
layout: docs
page_title: "Command: var lock"
description: |-
The "var lock" command locks a variable, only allowing the lock owner to modify it.
---
# Command: var lock
The `var lock` command holds a lock on a [variable][].
## Usage
```plaintext
nomad var lock [options] <lock spec file reference> child...
nomad var lock [options] <path to store variable> [<variable spec file reference>] child
```
The lock command provides a mechanism for simple distributed locking. A lock
is created in the given variable, and only when held, is a child process invoked.
The lock command can be called on an existing variable or an entire new variable
specification can be provided to the command from a file by using an
@-prefixed path to a variable specification file. Items to be stored in the
variable can be supplied using the specification file as well.
Nomad lock launches its children in a shell. By default, Nomad will use the
shell defined in the environment variable SHELL. If SHELL is not defined,
it will default to /bin/sh. It should be noted that not all shells terminate
child processes when they receive SIGTERM. Under Ubuntu, /bin/sh is linked
to dash, which does not terminate its children. In order to ensure that
child processes are killed when the lock is lost, be sure to set the SHELL
environment variable appropriately, or run without a shell by setting -shell=false.
If [ACLs][] are enabled, this command requires the 'variables:write' capability
for the destination namespace and path.
## Restrictions
Variable paths are restricted to [RFC3986][] URL-safe characters that don't
conflict with the use of the characters `@` and `.` in template blocks. This
includes alphanumeric characters and the special characters `-`, `_`, `~`, and
`/`. Paths may be up to 128 bytes long. The following regex matches the allowed
paths: `^[a-zA-Z0-9-_~/]{1,128}$`
The keys for the items in a variable may contain any character, but keys
containing characters outside the set of Unicode letters, Unicode digits, and
the underscore (`_`) can not be read directly using dotted references in Nomad's
template engine. Instead, they require the use of the `index` template function
to directly access their values. This does not impact cases where the keys and
values are read using the `range` function.
Variable items are restricted to 64KiB in size. This limit is calculated by
taking the sum of the length in bytes of all of the unencrypted keys and values.
## General Options
@include 'general_options.mdx'
## Lock Options
- `-verbose`: Provides additional information via standard error to preserve
standard output (stdout) for redirected output.
- `ttl`: Optional, TTL for the lock, time the variable will be locked. Defaults to 15s.
- `delay`: Optional, time the variable is blocked from locking when a lease is not renewed.
Defaults to 15s.
- `max-retry`:Optional, max-retry up to this number of times if Nomad returns a 500 error
while monitoring the lock. This allows riding out brief periods of
unavailability without causing leader elections, but increases the amount of
time required to detect a lost lock in some cases. Defaults to 5. Set to 0 to
disable.
- `shell`: Optional, use a shell to run the command (can set a custom shell via
the SHELL environment variable). The default value is true.
## Examples
Attempts to acquire a lock over the variable at path "secret/creds" for a time of
15s and executes `nomad job run webapp.nomad.hcl` if it succeeds:
```shell-session
$ nomad var lock -ttl=15s secret/creds "nomad job run webapp.nomad.hcl"
```
The data can also be consumed from a file on disk by prefixing with the "@"
symbol. For example, you can store a variable using a specification created with
the `nomad var init` command.
```shell-session
$ nomad var lock secret/foo @spec.nv.json `nomad job run webapp.nomad.hcl`
```
[variable]: /nomad/docs/concepts/variables
[varspec]: /nomad/docs/other-specifications/variables
[ACL Policy]: /nomad/docs/other-specifications/acl-policy#variables
[RFC3986]: https://www.rfc-editor.org/rfc/rfc3986#section-2
[ACL]: /nomad/docs/other-specifications/acl-policy#variables

View File

@@ -186,6 +186,55 @@ apply.
See [Workload Associated ACL Policies] for more details.
## Locks
Nomad provides the ability to block a variable from being updated for a period
of time by setting a lock on it. Once a variable is locked, it can be read by
everyone, but it can only be updated by the lock holder.
The locks are designed to provide granular locking and are heavily inspired by
[The Chubby Lock Service for Loosely-Coupled Distributed Systems][].
A lock is composed of an ID, a TTL, and a lock delay. The ID is generated by
the Nomad Server and must be provided on every request to modify the variable's
items or the lock itself. The TTL defines the time the lock will be held; if the
lock needs to be in place for longer, it can be renewed for as many new periods
as wanted.
Once it is no longer needed, it must be released. If by the time the TTL expires,
and no renew or release calls were made, the variable will remain locked for at
least the lock delay duration, to avoid a possible split-brain situation, where
there are two holders at the same time.
### Leader election backed by Nomad Variable Locks
For some applications, like HDFS or the Nomad Autoscaler, it is necessary to
have multiple instances running to ensure redundancy in case of a failure, but
only one of them may be active at a time as a leader.
As part of the [Go Package][], Nomad offers a helper that takes one variable and
uses a lock over it as a sync mechanism to run multiple instances but always
keeping just one running at any given time, using the following algorithm:
[![Leader Election Helper](/img/nomad-vars-leader-election-sdk.png)](/img/nomad-vars-leader-election-sdk.png)
As soon as any instance starts, it tries to lock the sync variable. If it succeeds,
it continues to execute while a secondary thread is in charge of keeping track of
the lock and renewing it when necessary. If by any chance the renewal fails,
the main process is forced to return, and the instance goes into standby until it
attempts to acquire the lock over the sync variable.
Only threads 1 and 3 or thread 2 are running at any given time, because every
instance is either executing as normal while renewing the lock or waiting for a
chance to acquire it and run.
When the main process, or protected function, returns, the helper releases the
lock, allowing a second instance to start running.
To see it implemented live, look for the [`nomad var lock`][] command
implementation or the [Nomad Autoscaler][] High Availability implementation.
[HashiCorp Consul]: https://www.consul.io/
[HashiCorp Vault]: https://www.vaultproject.io/
[Key Management]: /nomad/docs/operations/key-management
@@ -194,3 +243,8 @@ See [Workload Associated ACL Policies] for more details.
[workload identity]: /nomad/docs/concepts/workload-identity
[Workload Associated ACL Policies]: /nomad/docs/concepts/workload-identity#workload-associated-acl-policies
[ACL policy namespace rules]: /nomad/docs/other-specifications/acl-policy#namespace-rules
[The Chubby Lock Service for Loosely-Coupled Distributed Systems]: https://research.google/pubs/pub27897/
[`nomad var lock`]: /nomad/docs/commands/var
[Go Package]: https://pkg.go.dev/github.com/hashicorp/nomad/api
[implementation]: https://github.com/hashicorp/nomad/blob/release/1.7.0/command/var_lock.go#L240
[Nomad Autoscaler]: https://github.com/hashicorp/nomad-autoscaler/release/0.3.7command/agent.go#L368

View File

@@ -185,7 +185,20 @@
},
{
"title": "Variables",
"path": "variables"
"routes": [
{
"title": "Overview",
"path": "variables"
},
{
"title": "Variables",
"path": "variables/variables"
},
{
"title": "Locks",
"path": "variables/locks"
}
]
},
{
"title": "Volumes",

View File

@@ -1088,6 +1088,10 @@
"title": "put",
"path": "commands/var/put"
},
{
"title": "lock",
"path": "commands/var/lock"
},
{
"title": "purge",
"path": "commands/var/purge"

Binary file not shown.

After

Width:  |  Height:  |  Size: 544 KiB