add concept of health checks to fingerprinters and nodes

fix up feedback from code review

add driver info for all drivers to node
This commit is contained in:
Chelsea Holland Komlo
2018-01-25 11:30:15 -05:00
parent 92504f7d01
commit 1570972cb3
11 changed files with 621 additions and 22 deletions

View File

@@ -4,6 +4,7 @@ import (
"fmt"
"sort"
"strconv"
"time"
)
// Nodes is used to query node-related API endpoints
@@ -96,6 +97,15 @@ func (n *Nodes) GcAlloc(allocID string, q *QueryOptions) error {
return err
}
// DriverInfo is used to deserialize a DriverInfo entry
type DriverInfo struct {
Attributes map[string]string
Detected bool
Healthy bool
HealthDescription string
UpdateTime time.Time
}
// Node is used to deserialize a node entry.
type Node struct {
ID string
@@ -114,6 +124,7 @@ type Node struct {
StatusDescription string
StatusUpdatedAt int64
Events []*NodeEvent
Drivers map[string]*DriverInfo
CreateIndex uint64
ModifyIndex uint64
}

View File

@@ -259,7 +259,7 @@ func NewClient(cfg *config.Config, consulCatalog consul.CatalogAPI, consulServic
}
fingerprintManager := NewFingerprintManager(c.GetConfig, c.config.Node,
c.shutdownCh, c.updateNodeFromFingerprint, c.logger)
c.shutdownCh, c.updateNodeFromFingerprint, c.updateNodeFromHealthCheck, c.logger)
// Fingerprint the node and scan for drivers
if err := fingerprintManager.Run(); err != nil {
@@ -894,6 +894,9 @@ func (c *Client) setupNode() error {
if node.Links == nil {
node.Links = make(map[string]string)
}
if node.Drivers == nil {
node.Drivers = make(map[string]*structs.DriverInfo)
}
if node.Meta == nil {
node.Meta = make(map[string]string)
}
@@ -1003,9 +1006,48 @@ func (c *Client) updateNodeFromFingerprint(response *cstructs.FingerprintRespons
c.config.Node.Resources.Merge(response.Resources)
}
for name, new_val := range response.Drivers {
old_val := c.config.Node.Drivers[name]
if new_val.Equals(old_val) {
continue
}
if old_val == nil {
c.config.Node.Drivers[name] = new_val
} else {
c.config.Node.Drivers[name].MergeFingerprintInfo(new_val)
}
}
if nodeHasChanged {
c.updateNode()
}
return c.config.Node
}
func (c *Client) updateNodeFromHealthCheck(response *cstructs.HealthCheckResponse) *structs.Node {
c.configLock.Lock()
defer c.configLock.Unlock()
nodeHasChanged := false
// update the node with the latest driver health information
for name, new_val := range response.Drivers {
old_val := c.config.Node.Drivers[name]
if new_val.Equals(old_val) {
continue
}
nodeHasChanged = true
if old_val == nil {
c.config.Node.Drivers[name] = new_val
} else {
c.config.Node.Drivers[name].MergeHealthCheck(new_val)
}
}
if nodeHasChanged {
c.updateNode()
}
return c.config.Node
}

View File

@@ -148,6 +148,21 @@ func TestClient_Fingerprint_Periodic(t *testing.T) {
if mockDriverStatus == "" {
return false, fmt.Errorf("mock driver attribute should be set on the client")
}
// assert that the Driver information for the node is also set correctly
mockDriverInfo := node.Drivers["mock_driver"]
if mockDriverInfo == nil {
return false, fmt.Errorf("mock driver is nil when it should be set on node Drivers")
}
if !mockDriverInfo.Detected {
return false, fmt.Errorf("mock driver should be set as healthy")
}
if !mockDriverInfo.Healthy {
return false, fmt.Errorf("mock driver should be set as healthy")
}
if mockDriverInfo.HealthDescription == "" {
return false, fmt.Errorf("mock driver description should not be empty")
}
return true, nil
}, func(err error) {
t.Fatalf("err: %v", err)

View File

@@ -492,7 +492,6 @@ func (d *DockerDriver) Fingerprint(req *cstructs.FingerprintRequest, resp *cstru
d.logger.Printf("[INFO] driver.docker: failed to initialize client: %s", err)
}
d.fingerprintSuccess = helper.BoolToPtr(false)
resp.RemoveAttribute(dockerDriverAttr)
return nil
}
@@ -552,6 +551,41 @@ func (d *DockerDriver) Fingerprint(req *cstructs.FingerprintRequest, resp *cstru
return nil
}
// HealthChck implements the interface for the HealthCheck interface. This
// performs a health check on the docker driver, asserting whether the docker
// driver is responsive to a `docker ps` command.
func (d *DockerDriver) HealthCheck(req *cstructs.HealthCheckRequest, resp *cstructs.HealthCheckResponse) error {
unhealthy := &structs.DriverInfo{
HealthDescription: "Docker driver is available but unresponsive",
UpdateTime: time.Now(),
}
_, _, err := d.dockerClients()
if err != nil {
d.logger.Printf("[WARN] driver.docker: docker driver is available but is unresponsive to `docker ps`")
resp.AddDriverInfo("docker", unhealthy)
return err
}
d.logger.Printf("[TRACE] driver.docker: docker driver is available and is responsive to `docker ps`")
healthy := &structs.DriverInfo{
Healthy: true,
HealthDescription: "Docker driver is available and responsive",
UpdateTime: time.Now(),
}
resp.AddDriverInfo("docker", healthy)
return nil
}
// GetHealthChecks implements the interface for the HealthCheck interface. This
// sets whether the driver is eligible for periodic health checks and the
// interval at which to do them.
func (d *DockerDriver) GetHealthCheckInterval(req *cstructs.HealthCheckIntervalRequest, resp *cstructs.HealthCheckIntervalResponse) error {
resp.Eligible = true
resp.Period = 1 * time.Minute
return nil
}
// Validate is used to validate the driver configuration
func (d *DockerDriver) Validate(config map[string]interface{}) error {
fd := &fields.FieldData{

View File

@@ -21,6 +21,7 @@ import (
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/config"
"github.com/hashicorp/nomad/client/driver/env"
"github.com/hashicorp/nomad/client/fingerprint"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/client/testutil"
"github.com/hashicorp/nomad/helper/uuid"
@@ -164,6 +165,7 @@ func TestDockerDriver_Fingerprint(t *testing.T) {
if !tu.IsTravis() {
t.Parallel()
}
ctx := testDockerDriverContexts(t, &structs.Task{Name: "foo", Driver: "docker", Resources: basicResources})
//ctx.DriverCtx.config.Options = map[string]string{"docker.cleanup.image": "false"}
defer ctx.AllocDir.Destroy()
@@ -227,6 +229,7 @@ func TestDockerDriver_Fingerprint_Bridge(t *testing.T) {
request := &cstructs.FingerprintRequest{Config: conf, Node: conf.Node}
var response cstructs.FingerprintResponse
err = dd.Fingerprint(request, &response)
if err != nil {
t.Fatalf("error fingerprinting docker: %v", err)
@@ -251,6 +254,46 @@ func TestDockerDriver_Fingerprint_Bridge(t *testing.T) {
t.Logf("docker bridge ip: %q", attributes["driver.docker.bridge_ip"])
}
func TestDockerDriver_Check_DockerHealthStatus(t *testing.T) {
if !tu.IsTravis() {
t.Parallel()
}
if !testutil.DockerIsConnected(t) {
t.Skip("requires Docker")
}
if runtime.GOOS != "linux" {
t.Skip("expect only on linux")
}
require := require.New(t)
// This seems fragile, so we might need to reconsider this test if it
// proves flaky
expectedAddr, err := sockaddr.GetInterfaceIP("docker0")
if err != nil {
t.Fatalf("unable to get ip for docker0: %v", err)
}
if expectedAddr == "" {
t.Fatalf("unable to get ip for docker bridge")
}
conf := testConfig(t)
conf.Node = mock.Node()
dd := NewDockerDriver(NewDriverContext("", "", conf, conf.Node, testLogger(), nil))
request := &cstructs.HealthCheckRequest{}
var response cstructs.HealthCheckResponse
dc, ok := dd.(fingerprint.HealthCheck)
require.True(ok)
err = dc.HealthCheck(request, &response)
require.Nil(err)
driverInfo := response.Drivers["docker"]
require.NotNil(driverInfo)
require.True(driverInfo.Healthy)
}
func TestDockerDriver_StartOpen_Wait(t *testing.T) {
if !tu.IsTravis() {
t.Parallel()

View File

@@ -30,6 +30,8 @@ const (
// to "stop" a previously functioning driver after the specified duration
// (specified in seconds) for testing of periodic drivers and fingerprinters.
ShutdownPeriodicDuration = "test.shutdown_periodic_duration"
mockDriverName = "driver.mock_driver"
)
// MockDriverConfig is the driver configuration for the MockDriver
@@ -234,9 +236,9 @@ func (m *MockDriver) Fingerprint(req *cstructs.FingerprintRequest, resp *cstruct
// current time is after the time which the node should shut down, simulate
// driver failure
case !m.shutdownFingerprintTime.IsZero() && time.Now().After(m.shutdownFingerprintTime):
resp.RemoveAttribute("driver.mock_driver")
resp.RemoveAttribute(mockDriverName)
default:
resp.AddAttribute("driver.mock_driver", "1")
resp.AddAttribute(mockDriverName, "1")
resp.Detected = true
}
return nil
@@ -247,6 +249,39 @@ func (m *MockDriver) Periodic() (bool, time.Duration) {
return true, 500 * time.Millisecond
}
// HealthCheck implements the interface for HealthCheck, and indicates the current
// health status of the mock driver.
func (m *MockDriver) HealthCheck(req *cstructs.HealthCheckRequest, resp *cstructs.HealthCheckResponse) error {
switch {
case !m.shutdownFingerprintTime.IsZero() && time.Now().After(m.shutdownFingerprintTime):
notHealthy := &structs.DriverInfo{
Healthy: false,
HealthDescription: "not running",
UpdateTime: time.Now(),
}
resp.AddDriverInfo("mock_driver", notHealthy)
return nil
default:
healthy := &structs.DriverInfo{
Healthy: true,
HealthDescription: "running",
UpdateTime: time.Now(),
}
resp.AddDriverInfo("mock_driver", healthy)
return nil
}
}
// GetHealthCheckInterval implements the interface for HealthCheck and indicates
// that mock driver should be checked periodically. Returns a boolean
// indicating if ti should be checked, and the duration at which to do this
// check.
func (m *MockDriver) GetHealthCheckInterval(req *cstructs.HealthCheckIntervalRequest, resp *cstructs.HealthCheckIntervalResponse) error {
resp.Eligible = true
resp.Period = 1 * time.Second
return nil
}
// MockDriverHandle is a driver handler which supervises a mock task
type mockDriverHandle struct {
ctx *ExecContext

View File

@@ -85,6 +85,21 @@ func NewFingerprint(name string, logger *log.Logger) (Fingerprint, error) {
// Factory is used to instantiate a new Fingerprint
type Factory func(*log.Logger) Fingerprint
// HealthCheck is used for doing periodic health checks. On a given time
// interfal, a health check will be called by the fingerprint manager of the
// node.
type HealthCheck interface {
// Check is used to update properties of the node on the status of the health
// check
HealthCheck(*cstructs.HealthCheckRequest, *cstructs.HealthCheckResponse) error
// GetHealthCheckInterval is a mechanism for the health checker to indicate that
// it should be run periodically. The return value is a boolean indicating
// whether it should be done periodically, and the time interval at which
// this check should happen.
GetHealthCheckInterval(*cstructs.HealthCheckIntervalRequest, *cstructs.HealthCheckIntervalResponse) error
}
// Fingerprint is used for doing "fingerprinting" of the
// host to automatically determine attributes, resources,
// and metadata about it. Each of these is a heuristic, and

View File

@@ -20,10 +20,14 @@ type FingerprintManager struct {
nodeLock sync.Mutex
shutdownCh chan struct{}
// updateNode is a callback to the client to update the state of its
// updateNodeAttributes is a callback to the client to update the state of its
// associated node
updateNode func(*cstructs.FingerprintResponse) *structs.Node
logger *log.Logger
updateNodeAttributes func(*cstructs.FingerprintResponse) *structs.Node
// UpdateHealthCheck is a callback to the client to update the state of the
// node for resources that require a health check
updateHealthCheck func(*cstructs.HealthCheckResponse) *structs.Node
logger *log.Logger
}
// NewFingerprintManager is a constructor that creates and returns an instance
@@ -31,19 +35,21 @@ type FingerprintManager struct {
func NewFingerprintManager(getConfig func() *config.Config,
node *structs.Node,
shutdownCh chan struct{},
updateNode func(*cstructs.FingerprintResponse) *structs.Node,
updateNodeAttributes func(*cstructs.FingerprintResponse) *structs.Node,
updateHealthCheck func(*cstructs.HealthCheckResponse) *structs.Node,
logger *log.Logger) *FingerprintManager {
return &FingerprintManager{
getConfig: getConfig,
updateNode: updateNode,
node: node,
shutdownCh: shutdownCh,
logger: logger,
getConfig: getConfig,
updateNodeAttributes: updateNodeAttributes,
updateHealthCheck: updateHealthCheck,
node: node,
shutdownCh: shutdownCh,
logger: logger,
}
}
// run runs each fingerprinter individually on an ongoing basis
func (fm *FingerprintManager) run(f fingerprint.Fingerprint, period time.Duration, name string) {
// runFingerprint runs each fingerprinter individually on an ongoing basis
func (fm *FingerprintManager) runFingerprint(f fingerprint.Fingerprint, period time.Duration, name string) {
fm.logger.Printf("[DEBUG] client.fingerprint_manager: fingerprinting %s every %v", name, period)
for {
@@ -61,6 +67,25 @@ func (fm *FingerprintManager) run(f fingerprint.Fingerprint, period time.Duratio
}
}
// runHealthCheck runs each health check individually on an ongoing basis
func (fm *FingerprintManager) runHealthCheck(hc fingerprint.HealthCheck, period time.Duration, name string) {
fm.logger.Printf("[DEBUG] client.fingerprint_manager: healthchecking %s every %v", name, period)
for {
select {
case <-time.After(period):
err := fm.healthCheck(name, hc)
if err != nil {
fm.logger.Printf("[DEBUG] client.fingerprint_manager: health checking for %v failed: %+v", name, err)
continue
}
case <-fm.shutdownCh:
return
}
}
}
// setupDrivers is used to fingerprint the node to see if these drivers are
// supported
func (fm *FingerprintManager) setupDrivers(drivers []string) error {
@@ -73,7 +98,7 @@ func (fm *FingerprintManager) setupDrivers(drivers []string) error {
return err
}
detected, err := fm.fingerprint(name, d)
detected, err := fm.fingerprintDriver(name, d)
if err != nil {
fm.logger.Printf("[DEBUG] client.fingerprint_manager: fingerprinting for %v failed: %+v", name, err)
return err
@@ -86,7 +111,19 @@ func (fm *FingerprintManager) setupDrivers(drivers []string) error {
p, period := d.Periodic()
if p {
go fm.run(d, period, name)
go fm.runFingerprint(d, period, name)
}
// We should only run the health check task if the driver is detected
// Note that if the driver is detected later in a periodic health check,
// this won't automateically trigger the periodic health check.
if hc, ok := d.(fingerprint.HealthCheck); ok && detected {
req := &cstructs.HealthCheckIntervalRequest{}
resp := &cstructs.HealthCheckIntervalResponse{}
hc.GetHealthCheckInterval(req, resp)
if resp.Eligible {
go fm.runHealthCheck(hc, resp.Period, name)
}
}
}
@@ -94,6 +131,53 @@ func (fm *FingerprintManager) setupDrivers(drivers []string) error {
return nil
}
// fingerprintDriver is a temporary solution to move towards DriverInfo and
// away from annotating a node's attributes to demonstrate support for a
// particular driver. Takes the FingerprintResponse and converts it to the
// proper DriverInfo update and then sets the prefix attributes as well
func (fm *FingerprintManager) fingerprintDriver(name string, f fingerprint.Fingerprint) (bool, error) {
request := &cstructs.FingerprintRequest{Config: fm.getConfig(), Node: fm.node}
var response cstructs.FingerprintResponse
if err := f.Fingerprint(request, &response); err != nil {
return false, err
}
// TODO This is a temporary measure, as eventually all drivers will need to
// support this. Doing this so that we can enable this iteratively and also
// in a backwards compatible way, where node attributes for drivers will
// eventually be phased out.
di := &structs.DriverInfo{
Attributes: response.Attributes,
Detected: response.Detected,
}
response.AddDriver(name, di)
if node := fm.updateNodeAttributes(&response); node != nil {
fm.nodeLock.Lock()
fm.node = node
fm.nodeLock.Unlock()
}
if hc, ok := f.(fingerprint.HealthCheck); ok {
fm.healthCheck(name, hc)
} else {
resp := &cstructs.HealthCheckResponse{
Drivers: map[string]*structs.DriverInfo{
name: di,
},
}
if node := fm.updateHealthCheck(resp); node != nil {
fm.nodeLock.Lock()
fm.node = node
fm.nodeLock.Unlock()
}
}
return response.Detected, nil
}
// fingerprint does an initial fingerprint of the client. If the fingerprinter
// is meant to be run continuously, a process is launched to perform this
// fingerprint on an ongoing basis in the background.
@@ -109,7 +193,7 @@ func (fm *FingerprintManager) fingerprint(name string, f fingerprint.Fingerprint
return false, err
}
if node := fm.updateNode(&response); node != nil {
if node := fm.updateNodeAttributes(&response); node != nil {
fm.nodeLock.Lock()
fm.node = node
fm.nodeLock.Unlock()
@@ -118,6 +202,23 @@ func (fm *FingerprintManager) fingerprint(name string, f fingerprint.Fingerprint
return response.Detected, nil
}
// healthcheck checks the health of the specified resource.
func (fm *FingerprintManager) healthCheck(name string, hc fingerprint.HealthCheck) error {
request := &cstructs.HealthCheckRequest{}
var response cstructs.HealthCheckResponse
if err := hc.HealthCheck(request, &response); err != nil {
return err
}
if node := fm.updateHealthCheck(&response); node != nil {
fm.nodeLock.Lock()
fm.node = node
fm.nodeLock.Unlock()
}
return nil
}
// setupFingerprints is used to fingerprint the node to see if these attributes are
// supported
func (fm *FingerprintManager) setupFingerprinters(fingerprints []string) error {
@@ -143,7 +244,17 @@ func (fm *FingerprintManager) setupFingerprinters(fingerprints []string) error {
p, period := f.Periodic()
if p {
go fm.run(f, period, name)
go fm.runFingerprint(f, period, name)
}
if hc, ok := f.(fingerprint.HealthCheck); ok {
req := &cstructs.HealthCheckIntervalRequest{}
var resp cstructs.HealthCheckIntervalResponse
if err := hc.GetHealthCheckInterval(req, &resp); err != nil {
if resp.Eligible {
go fm.runHealthCheck(hc, resp.Period, name)
}
}
}
}

View File

@@ -6,6 +6,7 @@ import (
"github.com/hashicorp/nomad/client/config"
"github.com/hashicorp/nomad/client/driver"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/testutil"
"github.com/stretchr/testify/require"
@@ -20,6 +21,7 @@ func TestFingerprintManager_Run_MockDriver(t *testing.T) {
Attributes: make(map[string]string, 0),
Links: make(map[string]string, 0),
Resources: &structs.Resources{},
Drivers: make(map[string]*structs.DriverInfo, 0),
}
testConfig := config.Config{Node: node}
testClient := &Client{config: &testConfig}
@@ -34,6 +36,7 @@ func TestFingerprintManager_Run_MockDriver(t *testing.T) {
node,
make(chan struct{}),
testClient.updateNodeFromFingerprint,
testClient.updateNodeFromHealthCheck,
testLogger(),
)
@@ -51,6 +54,7 @@ func TestFingerprintManager_Run_ResourcesFingerprint(t *testing.T) {
Attributes: make(map[string]string, 0),
Links: make(map[string]string, 0),
Resources: &structs.Resources{},
Drivers: make(map[string]*structs.DriverInfo, 0),
}
testConfig := config.Config{Node: node}
testClient := &Client{config: &testConfig}
@@ -65,6 +69,7 @@ func TestFingerprintManager_Run_ResourcesFingerprint(t *testing.T) {
node,
make(chan struct{}),
testClient.updateNodeFromFingerprint,
testClient.updateNodeFromHealthCheck,
testLogger(),
)
@@ -83,6 +88,7 @@ func TestFingerprintManager_Fingerprint_Run(t *testing.T) {
Attributes: make(map[string]string, 0),
Links: make(map[string]string, 0),
Resources: &structs.Resources{},
Drivers: make(map[string]*structs.DriverInfo, 0),
}
testConfig := config.Config{Node: node}
testClient := &Client{config: &testConfig}
@@ -98,6 +104,7 @@ func TestFingerprintManager_Fingerprint_Run(t *testing.T) {
node,
make(chan struct{}),
testClient.updateNodeFromFingerprint,
testClient.updateNodeFromHealthCheck,
testLogger(),
)
@@ -105,6 +112,7 @@ func TestFingerprintManager_Fingerprint_Run(t *testing.T) {
require.Nil(err)
require.NotEqual("", node.Attributes["driver.raw_exec"])
require.True(node.Drivers["raw_exec"].Detected)
}
func TestFingerprintManager_Fingerprint_Periodic(t *testing.T) {
@@ -115,6 +123,7 @@ func TestFingerprintManager_Fingerprint_Periodic(t *testing.T) {
Attributes: make(map[string]string, 0),
Links: make(map[string]string, 0),
Resources: &structs.Resources{},
Drivers: make(map[string]*structs.DriverInfo, 0),
}
testConfig := config.Config{Node: node}
testClient := &Client{config: &testConfig}
@@ -138,13 +147,14 @@ func TestFingerprintManager_Fingerprint_Periodic(t *testing.T) {
node,
shutdownCh,
testClient.updateNodeFromFingerprint,
testClient.updateNodeFromHealthCheck,
testLogger(),
)
err := fm.Run()
require.Nil(err)
// Ensure the mock driver is registered on the client
// Ensure the mock driver is registered and healthy on the client
testutil.WaitForResult(func() (bool, error) {
mockDriverStatus := node.Attributes["driver.mock_driver"]
if mockDriverStatus == "" {
@@ -155,7 +165,8 @@ func TestFingerprintManager_Fingerprint_Periodic(t *testing.T) {
t.Fatalf("err: %v", err)
})
// Ensure that the client fingerprinter eventually removes this attribute
// Ensure that the client fingerprinter eventually removes this attribute and
// marks the driver as unhealthy
testutil.WaitForResult(func() (bool, error) {
mockDriverStatus := node.Attributes["driver.mock_driver"]
if mockDriverStatus != "" {
@@ -167,7 +178,187 @@ func TestFingerprintManager_Fingerprint_Periodic(t *testing.T) {
})
}
func TestFingerprintManager_Run_InWhitelist(t *testing.T) {
// This is a temporary measure to check that a driver has both attributes on a
// node set as well as DriverInfo.
func TestFingerprintManager_HealthCheck_Driver(t *testing.T) {
t.Parallel()
require := require.New(t)
node := &structs.Node{
Attributes: make(map[string]string, 0),
Drivers: make(map[string]*structs.DriverInfo, 0),
}
updateNode := func(r *cstructs.FingerprintResponse) *structs.Node {
if r.Attributes != nil {
for k, v := range r.Attributes {
node.Attributes[k] = v
}
}
return node
}
updateHealthCheck := func(resp *cstructs.HealthCheckResponse) *structs.Node {
if resp.Drivers != nil {
for k, v := range resp.Drivers {
node.Drivers[k] = v
}
}
return node
}
conf := config.DefaultConfig()
conf.Options = map[string]string{
"driver.raw_exec.enable": "1",
"test.shutdown_periodic_after": "true",
"test.shutdown_periodic_duration": "2",
}
getConfig := func() *config.Config {
return conf
}
shutdownCh := make(chan struct{})
defer (func() {
close(shutdownCh)
})()
fm := NewFingerprintManager(
getConfig,
node,
shutdownCh,
updateNode,
updateHealthCheck,
testLogger(),
)
err := fm.Run()
require.Nil(err)
// Ensure the mock driver is registered and healthy on the client
testutil.WaitForResult(func() (bool, error) {
mockDriverAttribute := node.Attributes["driver.mock_driver"]
if mockDriverAttribute == "" {
return false, fmt.Errorf("mock driver info should be set on the client attributes")
}
mockDriverInfo := node.Drivers["mock_driver"]
if mockDriverInfo == nil {
return false, fmt.Errorf("mock driver info should be set on the client")
}
if !mockDriverInfo.Healthy {
return false, fmt.Errorf("mock driver info should be healthy")
}
return true, nil
}, func(err error) {
t.Fatalf("err: %v", err)
})
// Ensure that a default driver without health checks enabled is registered and healthy on the client
testutil.WaitForResult(func() (bool, error) {
rawExecAttribute := node.Attributes["driver.raw_exec"]
if rawExecAttribute == "" {
return false, fmt.Errorf("raw exec info should be set on the client attributes")
}
rawExecInfo := node.Drivers["raw_exec"]
if rawExecInfo == nil {
return false, fmt.Errorf("raw exec driver info should be set on the client")
}
if !rawExecInfo.Detected {
return false, fmt.Errorf("raw exec driver should be detected")
}
return true, nil
}, func(err error) {
t.Fatalf("err: %v", err)
})
}
func TestFingerprintManager_HealthCheck_Periodic(t *testing.T) {
t.Parallel()
require := require.New(t)
node := &structs.Node{
Drivers: make(map[string]*structs.DriverInfo, 0),
}
updateNode := func(resp *cstructs.FingerprintResponse) *structs.Node {
for k, v := range resp.Drivers {
if node.Drivers[k] == nil {
node.Drivers[k] = v
} else {
node.Drivers[k].MergeFingerprintInfo(v)
}
}
return node
}
updateHealthCheck := func(resp *cstructs.HealthCheckResponse) *structs.Node {
for k, v := range resp.Drivers {
if node.Drivers[k] == nil {
node.Drivers[k] = v
} else {
node.Drivers[k].MergeHealthCheck(v)
}
}
return node
}
conf := config.DefaultConfig()
conf.Options = map[string]string{
"test.shutdown_periodic_after": "true",
"test.shutdown_periodic_duration": "2",
}
getConfig := func() *config.Config {
return conf
}
shutdownCh := make(chan struct{})
defer (func() {
close(shutdownCh)
})()
fm := NewFingerprintManager(
getConfig,
node,
shutdownCh,
updateNode,
updateHealthCheck,
testLogger(),
)
err := fm.Run()
require.Nil(err)
// Ensure the mock driver is registered and healthy on the client
testutil.WaitForResult(func() (bool, error) {
mockDriverInfo := node.Drivers["mock_driver"]
if mockDriverInfo == nil {
return false, fmt.Errorf("mock driver info should be set on the client")
}
if !mockDriverInfo.Detected {
return false, fmt.Errorf("mock driver info should be detected")
}
if !mockDriverInfo.Healthy {
return false, fmt.Errorf("mock driver info should be healthy")
}
return true, nil
}, func(err error) {
t.Fatalf("err: %v", err)
})
// Ensure that the client health check eventually removes this attribute and
// marks the driver as unhealthy
testutil.WaitForResult(func() (bool, error) {
mockDriverInfo := node.Drivers["mock_driver"]
if mockDriverInfo == nil {
return false, fmt.Errorf("mock driver info should be set on the client")
}
if !mockDriverInfo.Detected {
return false, fmt.Errorf("mock driver should be detected")
}
if mockDriverInfo.Healthy {
return false, fmt.Errorf("mock driver should not be healthy")
}
return true, nil
}, func(err error) {
t.Fatalf("err: %v", err)
})
}
func TestFimgerprintManager_Run_InWhitelist(t *testing.T) {
t.Parallel()
require := require.New(t)
@@ -175,6 +366,7 @@ func TestFingerprintManager_Run_InWhitelist(t *testing.T) {
Attributes: make(map[string]string, 0),
Links: make(map[string]string, 0),
Resources: &structs.Resources{},
Drivers: make(map[string]*structs.DriverInfo, 0),
}
testConfig := config.Config{Node: node}
testClient := &Client{config: &testConfig}
@@ -195,6 +387,7 @@ func TestFingerprintManager_Run_InWhitelist(t *testing.T) {
node,
shutdownCh,
testClient.updateNodeFromFingerprint,
testClient.updateNodeFromHealthCheck,
testLogger(),
)
@@ -211,6 +404,7 @@ func TestFingerprintManager_Run_InBlacklist(t *testing.T) {
Attributes: make(map[string]string, 0),
Links: make(map[string]string, 0),
Resources: &structs.Resources{},
Drivers: make(map[string]*structs.DriverInfo, 0),
}
testConfig := config.Config{Node: node}
testClient := &Client{config: &testConfig}
@@ -232,6 +426,7 @@ func TestFingerprintManager_Run_InBlacklist(t *testing.T) {
node,
shutdownCh,
testClient.updateNodeFromFingerprint,
testClient.updateNodeFromHealthCheck,
testLogger(),
)
@@ -249,6 +444,7 @@ func TestFingerprintManager_Run_Combination(t *testing.T) {
Attributes: make(map[string]string, 0),
Links: make(map[string]string, 0),
Resources: &structs.Resources{},
Drivers: make(map[string]*structs.DriverInfo, 0),
}
testConfig := config.Config{Node: node}
testClient := &Client{config: &testConfig}
@@ -270,6 +466,7 @@ func TestFingerprintManager_Run_Combination(t *testing.T) {
node,
shutdownCh,
testClient.updateNodeFromFingerprint,
testClient.updateNodeFromHealthCheck,
testLogger(),
)
@@ -289,6 +486,7 @@ func TestFingerprintManager_Run_WhitelistDrivers(t *testing.T) {
Attributes: make(map[string]string, 0),
Links: make(map[string]string, 0),
Resources: &structs.Resources{},
Drivers: make(map[string]*structs.DriverInfo, 0),
}
testConfig := config.Config{Node: node}
testClient := &Client{config: &testConfig}
@@ -312,6 +510,7 @@ func TestFingerprintManager_Run_WhitelistDrivers(t *testing.T) {
node,
shutdownCh,
testClient.updateNodeFromFingerprint,
testClient.updateNodeFromHealthCheck,
testLogger(),
)
@@ -328,6 +527,7 @@ func TestFingerprintManager_Run_AllDriversBlacklisted(t *testing.T) {
Attributes: make(map[string]string, 0),
Links: make(map[string]string, 0),
Resources: &structs.Resources{},
Drivers: make(map[string]*structs.DriverInfo, 0),
}
testConfig := config.Config{Node: node}
testClient := &Client{config: &testConfig}
@@ -350,6 +550,7 @@ func TestFingerprintManager_Run_AllDriversBlacklisted(t *testing.T) {
node,
shutdownCh,
testClient.updateNodeFromFingerprint,
testClient.updateNodeFromHealthCheck,
testLogger(),
)
@@ -368,6 +569,7 @@ func TestFingerprintManager_Run_DriversWhiteListBlacklistCombination(t *testing.
Attributes: make(map[string]string, 0),
Links: make(map[string]string, 0),
Resources: &structs.Resources{},
Drivers: make(map[string]*structs.DriverInfo, 0),
}
testConfig := config.Config{Node: node}
testClient := &Client{config: &testConfig}
@@ -392,6 +594,7 @@ func TestFingerprintManager_Run_DriversWhiteListBlacklistCombination(t *testing.
node,
shutdownCh,
testClient.updateNodeFromFingerprint,
testClient.updateNodeFromHealthCheck,
testLogger(),
)
@@ -412,6 +615,7 @@ func TestFingerprintManager_Run_DriversInBlacklist(t *testing.T) {
Attributes: make(map[string]string, 0),
Links: make(map[string]string, 0),
Resources: &structs.Resources{},
Drivers: make(map[string]*structs.DriverInfo, 0),
}
conf := config.DefaultConfig()
conf.Options = map[string]string{
@@ -433,6 +637,7 @@ func TestFingerprintManager_Run_DriversInBlacklist(t *testing.T) {
node,
shutdownCh,
testClient.updateNodeFromFingerprint,
testClient.updateNodeFromHealthCheck,
testLogger(),
)

View File

@@ -361,6 +361,19 @@ type FingerprintResponse struct {
// Detected is a boolean indicating whether the fingerprinter detected
// if the resource was available
Detected bool
// Drivers is a map of driver names to driver info. This allows the
// fingerprint method of each driver to set whether the driver is enabled or
// not, as well as its attributes
Drivers map[string]*structs.DriverInfo
}
func (f *FingerprintResponse) AddDriver(name string, value *structs.DriverInfo) {
if f.Drivers == nil {
f.Drivers = make(map[string]*structs.DriverInfo, 0)
}
f.Drivers[name] = value
}
// AddAttribute adds the name and value for a node attribute to the fingerprint
@@ -405,3 +418,31 @@ func (f *FingerprintResponse) RemoveLink(name string) {
f.Links[name] = ""
}
// HealthCheckRequest is the request type for a type that fulfils the Health
// Check interface
type HealthCheckRequest struct{}
// HealthCheckResponse is the response type for a type that fulfills the Health
// Check interface
type HealthCheckResponse struct {
// Drivers is a map of driver names to current driver information
Drivers map[string]*structs.DriverInfo
}
type HealthCheckIntervalRequest struct{}
type HealthCheckIntervalResponse struct {
Eligible bool
Period time.Duration
}
// AddDriverInfo adds information about a driver to the fingerprint response.
// If the Drivers field has not yet been initialized, it does so here.
func (h *HealthCheckResponse) AddDriverInfo(name string, driverInfo *structs.DriverInfo) {
// initialize Drivers if it has not been already
if h.Drivers == nil {
h.Drivers = make(map[string]*structs.DriverInfo, 0)
}
h.Drivers[name] = driverInfo
}

View File

@@ -1163,6 +1163,50 @@ func ValidNodeStatus(status string) bool {
}
}
// DriverInfo is the current state of a single driver. This is updated
// regularly as driver health changes on the node.
type DriverInfo struct {
Attributes map[string]string
Detected bool
Healthy bool
HealthDescription string
UpdateTime time.Time
}
func (di *DriverInfo) MergeHealthCheck(other *DriverInfo) {
di.Healthy = other.Healthy
di.HealthDescription = other.HealthDescription
di.UpdateTime = other.UpdateTime
}
func (di *DriverInfo) MergeFingerprintInfo(other *DriverInfo) {
di.Detected = other.Detected
di.Attributes = other.Attributes
}
func (di *DriverInfo) Equals(other *DriverInfo) bool {
if di == nil && other == nil {
return true
}
if di == nil && other != nil || di != nil && other == nil {
return false
}
if !di.Detected == other.Detected {
return false
}
if !di.Healthy == other.Healthy {
return false
}
if strings.Compare(di.HealthDescription, other.HealthDescription) != 0 {
return false
}
return true
}
// Node is a representation of a schedulable client node
type Node struct {
// ID is a unique identifier for the node. It can be constructed
@@ -1241,6 +1285,9 @@ type Node struct {
// retaining only MaxRetainedNodeEvents number at a time
Events []*NodeEvent
// Drivers is a map of driver names to current driver information
Drivers map[string]*DriverInfo
// Raft Indexes
CreateIndex uint64
ModifyIndex uint64