Commit Graph

4654 Commits

Author SHA1 Message Date
Tim Gross 989d7d9fcf
csi: avoid a nil pointer when handling plugin events (#15518)
If a plugin crashes quickly enough, we can get into a situation where the
deregister function is called before it's ever registered. Safely handle the
resulting nil pointer in the dynamic registry by not emitting a plugin event,
but also update the plugin event handler to tolerate nil pointers in case we
wire it up elsewhere in the future.
2022-12-12 08:42:57 -05:00
Seth Hoenig be3f89b5f9
artifact: enable inheriting environment variables from client (#15514)
* artifact: enable inheriting environment variables from client

This PR adds client configuration for specifying environment variables that
should be inherited by the artifact sandbox process from the Nomad Client agent.

Most users should not need to set these values but the configuration is provided
to ensure backwards compatability. Configuration of go-getter should ideally be
done through the artifact block in a jobspec task.

e.g.

```hcl
client {
  artifact {
    set_environment_variables = "TMPDIR,GIT_SSH_OPTS"
  }
}
```

Closes #15498

* website: update set_environment_variables text to mention PATH
2022-12-09 15:46:07 -06:00
Seth Hoenig 825c5cc65e
artifact: add client toggle to disable filesystem isolation (#15503)
This PR adds the client config option for turning off filesystem isolation,
applicable on Linux systems where filesystem isolation is possible and
enabled by default.

```hcl
client{
  artifact {
    disable_filesystem_isolation = <bool:false>
  }
}
```

Closes #15496
2022-12-08 12:29:23 -06:00
Seth Hoenig 51a2212d3d
client: sandbox go-getter subprocess with landlock (#15328)
* client: sandbox go-getter subprocess with landlock

This PR re-implements the getter package for artifact downloads as a subprocess.

Key changes include

On all platforms, run getter as a child process of the Nomad agent.
On Linux platforms running as root, run the child process as the nobody user.
On supporting Linux kernels, uses landlock for filesystem isolation (via go-landlock).
On all platforms, restrict environment variables of the child process to a static set.
notably TMP/TEMP now points within the allocation's task directory
kernel.landlock attribute is fingerprinted (version number or unavailable)
These changes make Nomad client more resilient against a faulty go-getter implementation that may panic, and more secure against bad actors attempting to use artifact downloads as a privilege escalation vector.

Adds new e2e/artifact suite for ensuring artifact downloading works.

TODO: Windows git test (need to modify the image, etc... followup PR)

* landlock: fixup items from cr

* cr: fixup tests and go.mod file
2022-12-07 16:02:25 -06:00
Seth Hoenig 3ed37b0b1d
fingerprint: add fingerprinting for CNI plugins presense and version (#15452)
This PR adds a fingerprinter to set the attribute
"plugins.cni.version.<name>" => "<version>"

for each CNI plugin in <client>.cni_path (/opt/cni/bin by default).
2022-12-05 14:22:47 -06:00
Seth Hoenig a65fbeb3b3
client: manually cleanup leaked iptables rules (#15407)
This PR adds a secondary path for cleaning up iptables created for an allocation
when the normal CNI library fails to do so. This typically happens when the state
of the pause container is unexpected - e.g. deleted out of band from Nomad. Before,
the iptables rules would be leaked which could lead to unexpected nat routing
behavior later on (in addition to leaked resources). With this change, we scan
for the rules created on behalf of the allocation being GC'd and delete them.

Fixes #6385
2022-11-28 11:32:16 -06:00
Sam 4689822628
Fix missing host header in http check (#15337) 2022-11-23 08:58:13 -05:00
James Rasell e2a2ea68fc
client: accommodate Consul 1.14.0 gRPC and agent self changes. (#15309)
* client: accommodate Consul 1.14.0 gRPC and agent self changes.

Consul 1.14.0 changed the way in which gRPC listeners are
configured, particularly when using TLS. Prior to the change, a
single listener was responsible for handling plain-text and
encrypted gRPC requests. In 1.14.0 and beyond, separate listeners
will be used for each, defaulting to 8502 and 8503 for plain-text
and TLS respectively.

The change means that Nomad’s Consul Connect integration would not
work when integrated with Consul clusters using TLS and running
1.14.0 or greater.

The Nomad Consul fingerprinter identifies the gRPC port Consul has
exposed using the "DebugConfig.GRPCPort" value from Consul’s
“/v1/agent/self” endpoint. In Consul 1.14.0 and greater, this only
represents the plain-text gRPC port which is likely to be disbaled
in clusters running TLS. In order to fix this issue, Nomad now
takes into account the Consul version and configured scheme to
optionally use “DebugConfig.GRPCTLSPort” value from Consul’s agent
self return.

The “consul_grcp_socket” allocrunner hook has also been updated so
that the fingerprinted gRPC port attribute is passed in. This
provides a better fallback method, when the operator does not
configure the “consul.grpc_address” option.

* docs: modify Consul Connect entries to detail 1.14.0 changes.

* changelog: add entry for #15309

* fixup: tidy tests and clean version match from review feedback.

* fixup: use strings tolower func.
2022-11-21 09:19:09 -06:00
Charlie Voiselle c73fb51d3a
[bug] Return a spec on reconnect (#15214)
client: fixed a bug where non-`docker` tasks with network isolation would leak network namespaces and iptables rules if the client was restarted while they were running
2022-11-11 13:27:36 -05:00
Seth Hoenig 21237d8337
client: avoid unconsumed channel in timer construction (#15215)
* client: avoid unconsumed channel in timer construction

This PR fixes a bug introduced in #11983 where a Timer initialized with 0
duration causes an immediate tick, even if Reset is called before reading the
channel. The fix is to avoid doing that, instead creating a Timer with a non-zero
initial wait time, and then immediately calling Stop.

* pr: remove redundant stop
2022-11-11 09:31:34 -06:00
Seth Hoenig 6e3309ebc6
template: protect use of template manager with a lock (#15192)
This PR protects access to `templateHook.templateManager` with its lock. So
far we have not been able to reproduce the panic - but it seems either Poststart
is running without a Prestart being run first (should be impossible), or the
Update hook is running concurrently with Poststart, nil-ing out the templateManager
in a race with Poststart.

Fixes #15189
2022-11-10 08:30:27 -06:00
Derek Strickland 80b6f27efd
api: remove `mapstructure` tags from`Port` struct (#12916)
This PR solves a defect in the deserialization of api.Port structs when returning structs from theEventStream.

Previously, the api.Port struct's fields were decorated with both mapstructure and hcl tags to support the network.port stanza's use of the keyword static when posting a static port value. This works fine when posting a job and when retrieving any struct that has an embedded api.Port instance as long as the value is deserialized using JSON decoding. The EventStream, however, uses mapstructure to decode event payloads in the api package. mapstructure expects an underlying field named static which does not exist. The result was that the Port.Value field would always be set to 0.

Upon further inspection, a few things became apparent.

The struct already has hcl tags that support the indirection during job submission.
Serialization/deserialization with both the json and hcl packages produce the desired result.
The use of of the mapstructure tags provided no value as the Port struct contains only fields with primitive types.
This PR:

Removes the mapstructure tags from the api.Port structs
Updates the job parsing logic to use hcl instead of mapstructure when decoding Port instances.
Closes #11044

Co-authored-by: DerekStrickland <dstrickland@hashicorp.com>
Co-authored-by: Piotr Kazmierczak <470696+pkazmierczak@users.noreply.github.com>
2022-11-08 11:26:28 +01:00
Luiz Aoqui e4c8b59919
Update alloc after reconnect and enforece client heartbeat order (#15068)
* scheduler: allow updates after alloc reconnects

When an allocation reconnects to a cluster the scheduler needs to run
special logic to handle the reconnection, check if a replacement was
create and stop one of them.

If the allocation kept running while the node was disconnected, it will
be reconnected with `ClientStatus: running` and the node will have
`Status: ready`. This combination is the same as the normal steady state
of allocation, where everything is running as expected.

In order to differentiate between the two states (an allocation that is
reconnecting and one that is just running) the scheduler needs an extra
piece of state.

The current implementation uses the presence of a
`TaskClientReconnected` task event to detect when the allocation has
reconnected and thus must go through the reconnection process. But this
event remains even after the allocation is reconnected, causing all
future evals to consider the allocation as still reconnecting.

This commit changes the reconnect logic to use an `AllocState` to
register when the allocation was reconnected. This provides the
following benefits:

  - Only a limited number of task states are kept, and they are used for
    many other events. It's possible that, upon reconnecting, several
    actions are triggered that could cause the `TaskClientReconnected`
    event to be dropped.
  - Task events are set by clients and so their timestamps are subject
    to time skew from servers. This prevents using time to determine if
    an allocation reconnected after a disconnect event.
  - Disconnect events are already stored as `AllocState` and so storing
    reconnects there as well makes it the only source of information
    required.

With the new logic, the reconnection logic is only triggered if the
last `AllocState` is a disconnect event, meaning that the allocation has
not been reconnected yet. After the reconnection is handled, the new
`ClientStatus` is store in `AllocState` allowing future evals to skip
the reconnection logic.

* scheduler: prevent spurious placement on reconnect

When a client reconnects it makes two independent RPC calls:

  - `Node.UpdateStatus` to heartbeat and set its status as `ready`.
  - `Node.UpdateAlloc` to update the status of its allocations.

These two calls can happen in any order, and in case the allocations are
updated before a heartbeat it causes the state to be the same as a node
being disconnected: the node status will still be `disconnected` while
the allocation `ClientStatus` is set to `running`.

The current implementation did not handle this order of events properly,
and the scheduler would create an unnecessary placement since it
considered the allocation was being disconnected. This extra allocation
would then be quickly stopped by the heartbeat eval.

This commit adds a new code path to handle this order of events. If the
node is `disconnected` and the allocation `ClientStatus` is `running`
the scheduler will check if the allocation is actually reconnecting
using its `AllocState` events.

* rpc: only allow alloc updates from `ready` nodes

Clients interact with servers using three main RPC methods:

  - `Node.GetAllocs` reads allocation data from the server and writes it
    to the client.
  - `Node.UpdateAlloc` reads allocation from from the client and writes
    them to the server.
  - `Node.UpdateStatus` writes the client status to the server and is
    used as the heartbeat mechanism.

These three methods are called periodically by the clients and are done
so independently from each other, meaning that there can't be any
assumptions in their ordering.

This can generate scenarios that are hard to reason about and to code
for. For example, when a client misses too many heartbeats it will be
considered `down` or `disconnected` and the allocations it was running
are set to `lost` or `unknown`.

When connectivity is restored the to rest of the cluster, the natural
mental model is to think that the client will heartbeat first and then
update its allocations status into the servers.

But since there's no inherit order in these calls the reverse is just as
possible: the client updates the alloc status and then heartbeats. This
results in a state where allocs are, for example, `running` while the
client is still `disconnected`.

This commit adds a new verification to the `Node.UpdateAlloc` method to
reject updates from nodes that are not `ready`, forcing clients to
heartbeat first. Since this check is done server-side there is no need
to coordinate operations client-side: they can continue sending these
requests independently and alloc update will succeed after the heartbeat
is done.

* chagelog: add entry for #15068

* code review

* client: skip terminal allocations on reconnect

When the client reconnects with the server it synchronizes the state of
its allocations by sending data using the `Node.UpdateAlloc` RPC and
fetching data using the `Node.GetClientAllocs` RPC.

If the data fetch happens before the data write, `unknown` allocations
will still be in this state and would trigger the
`allocRunner.Reconnect` flow.

But when the server `DesiredStatus` for the allocation is `stop` the
client should not reconnect the allocation.

* apply more code review changes

* scheduler: persist changes to reconnected allocs

Reconnected allocs have a new AllocState entry that must be persisted by
the plan applier.

* rpc: read node ID from allocs in UpdateAlloc

The AllocUpdateRequest struct is used in three disjoint use cases:

1. Stripped allocs from clients Node.UpdateAlloc RPC using the Allocs,
   and WriteRequest fields
2. Raft log message using the Allocs, Evals, and WriteRequest fields
3. Plan updates using the AllocsStopped, AllocsUpdated, and Job fields

Adding a new field that would only be used in one these cases (1) made
things more confusing and error prone. While in theory an
AllocUpdateRequest could send allocations from different nodes, in
practice this never actually happens since only clients call this method
with their own allocations.

* scheduler: remove logic to handle exceptional case

This condition could only be hit if, somehow, the allocation status was
set to "running" while the client was "unknown". This was addressed by
enforcing an order in "Node.UpdateStatus" and "Node.UpdateAlloc" RPC
calls, so this scenario is not expected to happen.

Adding unnecessary code to the scheduler makes it harder to read and
reason about it.

* more code review

* remove another unused test
2022-11-04 16:25:11 -04:00
Luiz Aoqui 1b87d292a3
client: retry RPC call when no server is available (#15140)
When a Nomad service starts it tries to establish a connection with
servers, but it also runs alloc runners to manage whatever allocations
it needs to run.

The alloc runner will invoke several hooks to perform actions, with some
of them requiring access to the Nomad servers, such as Native Service
Discovery Registration.

If the alloc runner starts before a connection is established the alloc
runner will fail, causing the allocation to be shutdown. This is
particularly problematic for disconnected allocations that are
reconnecting, as they may fail as soon as the client reconnects.

This commit changes the RPC request logic to retry it, using the
existing retry mechanism, if there are no servers available.
2022-11-04 14:09:39 -04:00
Charlie Voiselle 79c4478f5b
template: error on missing key (#15141)
* Support error_on_missing_value for templates
* Update docs for template stanza
2022-11-04 13:23:01 -04:00
Ethan 654ae1d591
fix: batchFirstFingerprints does not update device on node after v1.3.5 (#15125)
* fix: update device in batch first footprint

* cl: add cl note

Co-authored-by: Seth Hoenig <shoenig@duck.com>
2022-11-03 16:31:39 -05:00
Tim Gross 672fb46d12
WI: set identity to client secret if missing (#15121)
Allocations created before 1.4.0 will not have a workload identity token. When
the client running these allocs is upgraded to 1.4.x, the identity hook will run
and replace the node secret ID token used previously with an empty string. This
causes service discovery queries to fail.

Fallback to the node's secret ID when the allocation doesn't have a signed
identity. Note that pre-1.4.0 allocations won't have templates that read
Variables, so there's no threat that this new node ID secret will be able to
read data that the allocation shouldn't have access to.
2022-11-03 11:10:11 -04:00
Seth Hoenig ec915b2436
build: update linters (#15063)
Remove dead linters and add some interesting new ones.
2022-10-27 15:02:30 -05:00
Seth Hoenig d69556fb35
client: ensure minimal cgroup controllers enabled (#15027)
* client: ensure minimal cgroup controllers enabled

This PR fixes a bug where Nomad could not operate properly on operating
systems that set the root cgroup.subtree_control to a set of controllers that
do not include the minimal set of controllers needed by Nomad.

Nomad needs these controllers enabled to operate:
- cpuset
- cpu
- io
- memory
- pids

Now, Nomad will ensure these controllers are enabled during Client initialization,
adding them to cgroup.subtree_control as necessary. This should be particularly
helpful on the RHEL/CentOS/Fedora family of system. Ubuntu systems should be
unaffected as they enable all controllers by default.

Fixes: https://github.com/hashicorp/nomad/issues/14494

* docs: cleanup doc string

* client: cleanup controller writes, enhance log messages
2022-10-24 16:08:54 -05:00
James Rasell d7b311ce55
acl: correctly resolve ACL roles within client cache. (#14922)
The client ACL cache was not accounting for tokens which included
ACL role links. This change modifies the behaviour to resolve role
links to policies. It will also now store ACL roles within the
cache for quick lookup. The cache TTL is configurable in the same
manner as policies or tokens.

Another small fix is included that takes into account the ACL
token expiry time. This was not included, which meant tokens with
expiry could be used past the expiry time, until they were GC'd.
2022-10-20 09:37:32 +02:00
Seth Hoenig 57375566d4
consul: register checks along with service on initial registration (#14944)
* consul: register checks along with service on initial registration

This PR updates Nomad's Consul service client to include checks in
an initial service registration, so that the checks associated with
the service are registered "atomically" with the service. Before, we
would only register the checks after the service registration, which
causes problems where the service is deemed healthy, even if one or
more checks are unhealthy - especially problematic in the case where
SuccessBeforePassing is configured.

Fixes #3935

* cr: followup to fix cause of extra consul logging

* cr: fix another bug

* cr: fixup changelog
2022-10-19 12:40:56 -05:00
Seth Hoenig e66c9ede24
e2e: convert flaky exec download in chroot unit test into e2e test (#14949)
Similar to https://github.com/hashicorp/nomad/pull/14710, convert flaky
test into e2e test.
2022-10-19 08:22:32 -05:00
Michael Schurter 45ce8c13cf
client: remove unused LogOutput and LogLevel (#14867)
* client: remove unused LogOutput

* client: remove unused config.LogLevel
2022-10-11 09:24:40 -07:00
Seth Hoenig ba1e337f8b
helpers: lockfree lookup of nobody user on unix systems (#14866)
* helpers: lockfree lookup of nobody user on linux and darwin

This PR continues the nobody user lookup saga, by making the nobody
user lookup lock-free on linux and darwin.

By doing the lookup in an init block this originally broke on Windows,
where we must avoid doing the lookup at all. We can get around that
breakage by only doing the lookup on linux/darwin where the nobody
user is going to exist.

Also return the nobody user by value so that a copy is created that
cannot be modified by callers of Nobody().

* helper: move nobody code into unix file
2022-10-11 08:38:05 -05:00
Seth Hoenig 5e38a0e82c
cleanup: rename Equals to Equal for consistency (#14759) 2022-10-10 09:28:46 -05:00
Hemanth Krishna e516fc266f
enhancement: UpdateTask when Task is waiting for ShutdownDelay (#14775)
Signed-off-by: Hemanth Krishna <hkpdev008@gmail.com>
2022-10-06 16:33:28 -04:00
Pablo Ruiz García 40416be7b1
Invoke FingerprintManager's Reload() func during agent's SIGHUP (#14615)
Fixes #14614
2022-10-06 16:22:59 -04:00
Tim Gross 322ecc858f
client: defer `nobody` user lookup so Windows doesn't panic (#14790)
In #14742 we introduced a cached lookup of the `nobody` user, which is only ever
called on Unixish machines. But the initial caching was being done in an `init`
block, which meant it was being run on Windows as well. This prevents the Nomad
agent from starting on Windows.

An alternative fix here would be to have a separate `init` block for Windows and
Unix, but this potentially masks incorrect behavior if we accidentally added a
call to the `Nobody()` method on Windows later. This way we're forced to handle
the error in the caller.
2022-10-04 11:52:12 -04:00
Luiz Aoqui b924802958
template: apply splay value on change_mode script (#14749)
Previously, the splay timeout was only applied if a template re-render
caused a restart or a signal action. The `change_mode = "script"` was
running after the `if restart || len(signals) != 0` check, so it was
invoked at all times.

This change refactors the logic so it's easier to notice that new
`change_mode` options should start only after `splay` is applied.
2022-09-30 12:04:22 -04:00
Seth Hoenig c68ed3b4c8
client: protect user lookups with global lock (#14742)
* client: protect user lookups with global lock

This PR updates Nomad client to always do user lookups while holding
a global process lock. This is to prevent concurrency unsafe implementations
of NSS, but still enabling NSS lookups of users (i.e. cannot not use osusergo).

* cl: add cl
2022-09-29 09:30:13 -05:00
Michael Schurter 0e95fb03c0
test: skip chown test if nonroot (#14738)
CI always runs this as root, so it worked there and always scared me
when I ran it locally.
2022-09-28 14:45:38 -07:00
Seth Hoenig 7235d9988b
e2e: convert chroot env unit tests into e2e tests (#14710)
This PR translates two of our most flakey unit tests into
e2e tests where they are fit much more naturally.
2022-09-26 15:40:29 -05:00
Michael Schurter e6af1c0a14
fingerprint: add node attr for reserverable cores (#14694)
* fingerprint: add node attr for reserverable cores

Add an attribute for the number of reservable CPU cores as they may
differ from the existing `cpu.numcores` due to client configuration or
OS support.

Hopefully clarifies some confusion in #14676

* add changelog

* num_reservable_cores -> reservablecores
2022-09-26 13:03:03 -07:00
Luiz Aoqui 5c100c0d3d
client: recover from getter panics (#14696)
The artifact getter uses the go-getter library to fetch files from
different sources. Any bug in this library that results in a panic can
cause the entire Nomad client to crash due to a single file download
attempt.

This change aims to guard against this types of crashes by recovering
from panics when the getter attempts to download an artifact. The
resulting panic is converted to an error that is stored as a task event
for operator visibility and the panic stack trace is logged to the
client's log.
2022-09-26 15:16:26 -04:00
Michael Schurter b554f9344a
fingerprint: lengthen Vault check after seen (#14693)
Extension of #14673

Once Vault is initially fingerprinted, extend the period since changes
should be infrequent and the fingerprint is relatively expensive since
it is contacting a central Vault server.

Also move the period timer reset *after* the fingerprint. This is
similar to #9435 where the idea is to ensure the retry period starts
*after* the operation is attempted. 15s will be the *minimum* time
between fingerprints now instead of the *maximum* time between
fingerprints.

In the case of Vault fingerprinting, the original behavior might cause
the following:

1. Timer is reset to 15s
2. Fingerprint takes 16s
3. Timer has already elapsed so we immediately Fingerprint again

Even if fingerprinting Vault only takes a few seconds, that may very
well be due to excessive load and backing off our fingerprints is
desirable. The new bevahior ensures we always wait at least 15s between
fingerprint attempts and should allow some natural jittering based on
server load and network latency.
2022-09-26 12:14:19 -07:00
Seth Hoenig 87ec5fdee5
deps: update set and test (#14680)
This PR updates go-set and shoenig/test, which introduced some breaking
API changes.
2022-09-26 08:28:03 -05:00
Tim Gross 17aee4d69c
fingerprint: don't clear Consul/Vault attributes on failure (#14673)
Clients periodically fingerprint Vault and Consul to ensure the server has
updated attributes in the client's fingerprint. If the client can't reach
Vault/Consul, the fingerprinter clears the attributes and requires a node
update. Although this seems like correct behavior so that we can detect
intentional removal of Vault/Consul access, it has two serious failure modes:

(1) If a local Consul agent is restarted to pick up configuration changes and the
client happens to fingerprint at that moment, the client will update its
fingerprint and result in evaluations for all its jobs and all the system jobs
in the cluster.

(2) If a client loses Vault connectivity, the same thing happens. But the
consequences are much worse in the Vault case because Vault is not run as a
local agent, so Vault connectivity failures are highly correlated across the
entire cluster. A 15 second Vault outage will cause a new `node-update`
evalution for every system job on the cluster times the number of nodes, plus
one `node-update` evaluation for every non-system job on each node. On large
clusters of 1000s of nodes, we've seen this create a large backlog of evaluations.

This changeset updates the fingerprinting behavior to keep the last fingerprint
if Consul or Vault queries fail. This prevents a storm of evaluations at the
cost of requiring a client restart if Consul or Vault is intentionally removed
from the client.
2022-09-23 14:45:12 -04:00
Jorge Marey 92158a1c62
connect: add nomad env to envoy bootstrap (#12959)
* Add nomad env to envoy bootstrap

* Add changelog file
2022-09-22 13:18:18 -05:00
Jorge Marey 584ddfe859
Add Namespace, Job and Group to envoy stats (#14311) 2022-09-22 10:38:21 -04:00
Seth Hoenig 2088ca3345
cleanup more helper updates (#14638)
* cleanup: refactor MapStringStringSliceValueSet to be cleaner

* cleanup: replace SliceStringToSet with actual set

* cleanup: replace SliceStringSubset with real set

* cleanup: replace SliceStringContains with slices.Contains

* cleanup: remove unused function SliceStringHasPrefix

* cleanup: fixup StringHasPrefixInSlice doc string

* cleanup: refactor SliceSetDisjoint to use real set

* cleanup: replace CompareSliceSetString with SliceSetEq

* cleanup: replace CompareMapStringString with maps.Equal

* cleanup: replace CopyMapStringString with CopyMap

* cleanup: replace CopyMapStringInterface with CopyMap

* cleanup: fixup more CopyMapStringString and CopyMapStringInt

* cleanup: replace CopySliceString with slices.Clone

* cleanup: remove unused CopySliceInt

* cleanup: refactor CopyMapStringSliceString to be generic as CopyMapOfSlice

* cleanup: replace CopyMap with maps.Clone

* cleanup: run go mod tidy
2022-09-21 14:53:25 -05:00
Luiz Aoqui 25a63195da
test: remove flaky Gate test (#14575)
The concurrent gate access test is flaky since it depends on the order
of operations of two concurrent goroutines. Despite the heavy bias
towards one of the results, it's still possible to end the execution
with a closed gate.

I believe this case was created to test an earlier implementation where
the gate state was stored and mutated internally, so the access had to
be protected by a lock. However, the final implementation changed this
approach to be only channel-based, so there is no need for this flaky
test anymore.
2022-09-19 11:31:03 -04:00
Seth Hoenig 5187f92c5e
cleanup: create interface for check watcher and mock it in nsd tests (#14577)
* cleanup: create interface for check watcher and mock it in nsd tests

* cleanup: add comments for check watcher interface
2022-09-14 08:25:20 -05:00
Michael Schurter b2d22aef65
2 small data race fixes in logmon and check tests (#14538)
* logmon: fix data race around oldestLogFileIdx

* checks: fix 2 data races in tests

* logmon: move & rename lock to logically group
2022-09-13 12:54:06 -07:00
Seth Hoenig 9a943107c7 servicedisco: implement check_restart for nomad service checks
This PR implements support for check_restart for checks registered
in the Nomad service provider.

Unlike Consul, Nomad service checks never report a "warning" status,
and so the check_restart.ignore_warnings configuration is not valid
for Nomad service checks.
2022-09-13 08:59:23 -05:00
Seth Hoenig b960925939
Merge pull request #14546 from hashicorp/f-refactor-check-watcher
client: refactor check watcher to be reusable
2022-09-13 07:32:32 -05:00
Seth Hoenig feff36f3f7 client: refactor check watcher to be reusable
This PR refactors agent/consul/check_watcher into client/serviceregistration,
and abstracts away the Consul-specific check lookups.

In doing so we should be able to reuse the existing check watcher logic for
also watching NSD checks in a followup PR.

A chunk of consul/unit_test.go is removed - we'll cover that in e2e tests
in a follow PR if needed. In the long run I'd like to remove this whole file.
2022-09-12 10:13:31 -05:00
Tim Gross e5454362dc
CI: make `make check` clean on macOS (#14528)
Running `make check` on macOS identifies some dead code because the code is used
only with the Linux build tag. Move this code into appropriately-tagged code
files.
2022-09-09 12:26:34 -04:00
Seth Hoenig 31234d6a62 cleanup: consolidate interfaces for workload restarting
This PR combines two of the same interface definitions around workload restarting
2022-09-09 08:59:04 -05:00
Charlie Voiselle e58998e218
Add client scheduling eligibility to heartbeat (#14483) 2022-09-08 14:31:36 -04:00
Tiernan 98022376be
Fix error handling in Client consulDiscoveryImpl (#14431)
Added a missing `continue` on non-nil error to avoid accidentally using a bad peer.
2022-09-02 15:13:03 -04:00