diff --git a/client/allocrunner/alloc_runner.go b/client/allocrunner/alloc_runner.go index 80fc1d6a1..01accbde3 100644 --- a/client/allocrunner/alloc_runner.go +++ b/client/allocrunner/alloc_runner.go @@ -17,13 +17,14 @@ import ( "github.com/hashicorp/nomad/client/consul" "github.com/hashicorp/nomad/client/devicemanager" cinterfaces "github.com/hashicorp/nomad/client/interfaces" + "github.com/hashicorp/nomad/client/pluginmanager/drivermanager" cstate "github.com/hashicorp/nomad/client/state" cstructs "github.com/hashicorp/nomad/client/structs" "github.com/hashicorp/nomad/client/vaultclient" "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/plugins/device" - "github.com/hashicorp/nomad/plugins/shared/loader" + "github.com/hashicorp/nomad/plugins/drivers" ) // allocRunner is used to run all the tasks in a given allocation @@ -131,13 +132,13 @@ type allocRunner struct { // prevAllocMigrator allows the migration of a previous allocations alloc dir. prevAllocMigrator allocwatcher.PrevAllocMigrator - // pluginSingletonLoader is a plugin loader that will returns singleton - // instances of the plugins. - pluginSingletonLoader loader.PluginCatalog - // devicemanager is used to mount devices as well as lookup device // statistics devicemanager devicemanager.Manager + + // driverManager is responsible for dispensing driver plugins and registering + // event handlers + driverManager drivermanager.Manager } // NewAllocRunner returns a new allocation runner. @@ -167,8 +168,8 @@ func NewAllocRunner(config *Config) (*allocRunner, error) { deviceStatsReporter: config.DeviceStatsReporter, prevAllocWatcher: config.PrevAllocWatcher, prevAllocMigrator: config.PrevAllocMigrator, - pluginSingletonLoader: config.PluginSingletonLoader, devicemanager: config.DeviceManager, + driverManager: config.DriverManager, } // Create the logger based on the allocation ID @@ -195,18 +196,18 @@ func NewAllocRunner(config *Config) (*allocRunner, error) { func (ar *allocRunner) initTaskRunners(tasks []*structs.Task) error { for _, task := range tasks { config := &taskrunner.Config{ - Alloc: ar.alloc, - ClientConfig: ar.clientConfig, - Task: task, - TaskDir: ar.allocDir.NewTaskDir(task.Name), - Logger: ar.logger, - StateDB: ar.stateDB, - StateUpdater: ar, - Consul: ar.consulClient, - Vault: ar.vaultClient, - PluginSingletonLoader: ar.pluginSingletonLoader, - DeviceStatsReporter: ar.deviceStatsReporter, - DeviceManager: ar.devicemanager, + Alloc: ar.alloc, + ClientConfig: ar.clientConfig, + Task: task, + TaskDir: ar.allocDir.NewTaskDir(task.Name), + Logger: ar.logger, + StateDB: ar.stateDB, + StateUpdater: ar, + Consul: ar.consulClient, + Vault: ar.vaultClient, + DeviceStatsReporter: ar.deviceStatsReporter, + DeviceManager: ar.devicemanager, + DriverManager: ar.driverManager, } // Create, but do not Run, the task runner @@ -891,3 +892,17 @@ func (ar *allocRunner) LatestAllocStats(taskFilter string) (*cstructs.AllocResou return astat, nil } + +func (ar *allocRunner) GetTaskEventHandler(taskName string) drivermanager.EventHandler { + if tr, ok := ar.tasks[taskName]; ok { + return func(ev *drivers.TaskEvent) { + tr.EmitEvent(&structs.TaskEvent{ + Type: structs.TaskDriverMessage, + Time: ev.Timestamp.UnixNano(), + Details: ev.Annotations, + DriverMessage: ev.Message, + }) + } + } + return nil +} diff --git a/client/allocrunner/config.go b/client/allocrunner/config.go index 6406d2a5a..16f3e27db 100644 --- a/client/allocrunner/config.go +++ b/client/allocrunner/config.go @@ -7,10 +7,10 @@ import ( "github.com/hashicorp/nomad/client/consul" "github.com/hashicorp/nomad/client/devicemanager" "github.com/hashicorp/nomad/client/interfaces" + "github.com/hashicorp/nomad/client/pluginmanager/drivermanager" cstate "github.com/hashicorp/nomad/client/state" "github.com/hashicorp/nomad/client/vaultclient" "github.com/hashicorp/nomad/nomad/structs" - "github.com/hashicorp/nomad/plugins/shared/loader" ) // Config holds the configuration for creating an allocation runner. @@ -45,14 +45,10 @@ type Config struct { // PrevAllocMigrator allows the migration of a previous allocations alloc dir PrevAllocMigrator allocwatcher.PrevAllocMigrator - // PluginLoader is used to load plugins. - PluginLoader loader.PluginCatalog - - // PluginSingletonLoader is a plugin loader that will returns singleton - // instances of the plugins. - PluginSingletonLoader loader.PluginCatalog - // DeviceManager is used to mount devices as well as lookup device // statistics DeviceManager devicemanager.Manager + + // DriverManager handles dispensing of driver plugins + DriverManager drivermanager.Manager } diff --git a/client/allocrunner/taskrunner/task_runner.go b/client/allocrunner/taskrunner/task_runner.go index 92ef7b067..d6420e47e 100644 --- a/client/allocrunner/taskrunner/task_runner.go +++ b/client/allocrunner/taskrunner/task_runner.go @@ -21,17 +21,16 @@ import ( "github.com/hashicorp/nomad/client/consul" "github.com/hashicorp/nomad/client/devicemanager" cinterfaces "github.com/hashicorp/nomad/client/interfaces" + "github.com/hashicorp/nomad/client/pluginmanager/drivermanager" cstate "github.com/hashicorp/nomad/client/state" cstructs "github.com/hashicorp/nomad/client/structs" "github.com/hashicorp/nomad/client/taskenv" "github.com/hashicorp/nomad/client/vaultclient" "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/structs" - "github.com/hashicorp/nomad/plugins/base" "github.com/hashicorp/nomad/plugins/drivers" "github.com/hashicorp/nomad/plugins/shared" "github.com/hashicorp/nomad/plugins/shared/hclspec" - "github.com/hashicorp/nomad/plugins/shared/loader" ) const ( @@ -174,13 +173,13 @@ type TaskRunner struct { // deviceStatsReporter is used to lookup resource usage for alloc devices deviceStatsReporter cinterfaces.DeviceStatsReporter - // PluginSingletonLoader is a plugin loader that will returns singleton - // instances of the plugins. - pluginSingletonLoader loader.PluginCatalog - // devicemanager is used to mount devices as well as lookup device // statistics devicemanager devicemanager.Manager + + // driverManager is used to dispense driver plugins and register event + // handlers + driverManager drivermanager.Manager } type Config struct { @@ -203,13 +202,13 @@ type Config struct { // deviceStatsReporter is used to lookup resource usage for alloc devices DeviceStatsReporter cinterfaces.DeviceStatsReporter - // PluginSingletonLoader is a plugin loader that will returns singleton - // instances of the plugins. - PluginSingletonLoader loader.PluginCatalog - // DeviceManager is used to mount devices as well as lookup device // statistics DeviceManager devicemanager.Manager + + // DriverManager is used to dispense driver plugins and register event + // handlers + DriverManager drivermanager.Manager } func NewTaskRunner(config *Config) (*TaskRunner, error) { @@ -234,29 +233,29 @@ func NewTaskRunner(config *Config) (*TaskRunner, error) { } tr := &TaskRunner{ - alloc: config.Alloc, - allocID: config.Alloc.ID, - clientConfig: config.ClientConfig, - task: config.Task, - taskDir: config.TaskDir, - taskName: config.Task.Name, - taskLeader: config.Task.Leader, - envBuilder: envBuilder, - consulClient: config.Consul, - vaultClient: config.Vault, - state: tstate, - localState: state.NewLocalState(), - stateDB: config.StateDB, - stateUpdater: config.StateUpdater, - deviceStatsReporter: config.DeviceStatsReporter, - killCtx: killCtx, - killCtxCancel: killCancel, - ctx: trCtx, - ctxCancel: trCancel, - triggerUpdateCh: make(chan struct{}, triggerUpdateChCap), - waitCh: make(chan struct{}), - pluginSingletonLoader: config.PluginSingletonLoader, - devicemanager: config.DeviceManager, + alloc: config.Alloc, + allocID: config.Alloc.ID, + clientConfig: config.ClientConfig, + task: config.Task, + taskDir: config.TaskDir, + taskName: config.Task.Name, + taskLeader: config.Task.Leader, + envBuilder: envBuilder, + consulClient: config.Consul, + vaultClient: config.Vault, + state: tstate, + localState: state.NewLocalState(), + stateDB: config.StateDB, + stateUpdater: config.StateUpdater, + deviceStatsReporter: config.DeviceStatsReporter, + killCtx: killCtx, + killCtxCancel: killCancel, + ctx: trCtx, + ctxCancel: trCancel, + triggerUpdateCh: make(chan struct{}, triggerUpdateChCap), + waitCh: make(chan struct{}), + devicemanager: config.DeviceManager, + driverManager: config.DriverManager, } // Create the logger based on the allocation ID @@ -606,50 +605,12 @@ func (tr *TaskRunner) runDriver() error { return nil } -// initDriver creates the driver for the task -/*func (tr *TaskRunner) initDriver() error { - // Create a task-specific event emitter callback to expose minimal - // state to drivers - //XXX Replace with EmitEvent -- no need for a shim - eventEmitter := func(m string, args ...interface{}) { - msg := fmt.Sprintf(m, args...) - tr.logger.Debug("driver event", "event", msg) - tr.EmitEvent(structs.NewTaskEvent(structs.TaskDriverMessage).SetDriverMessage(msg)) - } - - alloc := tr.Alloc() - driverCtx := driver.NewDriverContext( - alloc.Job.Name, - alloc.TaskGroup, - tr.taskName, - tr.allocID, - tr.clientConfig, // XXX Why does it need this - tr.clientConfig.Node, // XXX THIS I NEED TO FIX - tr.logger.StandardLogger(nil), // XXX Should pass this through - eventEmitter) - - driver, err := driver.NewDriver(tr.task.Driver, driverCtx) - if err != nil { - return err - } - - tr.driver = driver - return nil -}*/ - // initDriver retrives the DriverPlugin from the plugin loader for this task func (tr *TaskRunner) initDriver() error { - plugin, err := tr.pluginSingletonLoader.Dispense(tr.Task().Driver, base.PluginTypeDriver, tr.clientConfig.NomadPluginConfig(), tr.logger) + driver, err := tr.driverManager.Dispense(tr.Task().Driver) if err != nil { return err } - - // XXX need to be able to reattach to running drivers - driver, ok := plugin.Plugin().(drivers.DriverPlugin) - if !ok { - return fmt.Errorf("plugin loaded for driver %s does not implement DriverPlugin interface", tr.task.Driver) - } - tr.driver = driver schema, err := tr.driver.TaskConfigSchema() diff --git a/client/allocrunner/taskrunner/task_runner_test.go b/client/allocrunner/taskrunner/task_runner_test.go index 19a42aa71..0ab5be2ff 100644 --- a/client/allocrunner/taskrunner/task_runner_test.go +++ b/client/allocrunner/taskrunner/task_runner_test.go @@ -12,6 +12,7 @@ import ( "github.com/hashicorp/nomad/client/config" consulapi "github.com/hashicorp/nomad/client/consul" "github.com/hashicorp/nomad/client/devicemanager" + "github.com/hashicorp/nomad/client/pluginmanager/drivermanager" cstate "github.com/hashicorp/nomad/client/state" "github.com/hashicorp/nomad/client/vaultclient" mockdriver "github.com/hashicorp/nomad/drivers/mock" @@ -19,8 +20,6 @@ import ( "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/plugins/device" - "github.com/hashicorp/nomad/plugins/shared/catalog" - "github.com/hashicorp/nomad/plugins/shared/singleton" "github.com/hashicorp/nomad/testutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -47,7 +46,6 @@ func (m *MockTaskStateUpdater) TaskStateUpdated() { // plus a cleanup func. func testTaskRunnerConfig(t *testing.T, alloc *structs.Allocation, taskName string) (*Config, func()) { logger := testlog.HCLogger(t) - pluginLoader := catalog.TestPluginLoader(t) clientConf, cleanup := config.TestClientConfig(t) // Find the task @@ -85,17 +83,17 @@ func testTaskRunnerConfig(t *testing.T, alloc *structs.Allocation, taskName stri } conf := &Config{ - Alloc: alloc, - ClientConfig: clientConf, - Consul: consulapi.NewMockConsulServiceClient(t, logger), - Task: thisTask, - TaskDir: taskDir, - Logger: clientConf.Logger, - Vault: vaultclient.NewMockVaultClient(), - StateDB: cstate.NoopDB{}, - StateUpdater: NewMockTaskStateUpdater(), - PluginSingletonLoader: singleton.NewSingletonLoader(logger, pluginLoader), - DeviceManager: devicemanager.NoopMockManager(), + Alloc: alloc, + ClientConfig: clientConf, + Consul: consulapi.NewMockConsulServiceClient(t, logger), + Task: thisTask, + TaskDir: taskDir, + Logger: clientConf.Logger, + Vault: vaultclient.NewMockVaultClient(), + StateDB: cstate.NoopDB{}, + StateUpdater: NewMockTaskStateUpdater(), + DeviceManager: devicemanager.NoopMockManager(), + DriverManager: drivermanager.TestDriverManager(t), } return conf, trCleanup } @@ -196,14 +194,9 @@ func TestTaskRunner_TaskEnv(t *testing.T) { } // Get the mock driver plugin - driverPlugin, err := conf.PluginSingletonLoader.Dispense( - mockdriver.PluginID.Name, - mockdriver.PluginID.PluginType, - nil, - conf.Logger, - ) + driverPlugin, err := conf.DriverManager.Dispense(mockdriver.PluginID.Name) require.NoError(err) - mockDriver := driverPlugin.Plugin().(*mockdriver.Driver) + mockDriver := driverPlugin.(*mockdriver.Driver) // Assert its config has been properly interpolated driverCfg, mockCfg := mockDriver.GetTaskConfig() @@ -272,14 +265,9 @@ func TestTaskRunner_DevicePropogation(t *testing.T) { } // Get the mock driver plugin - driverPlugin, err := conf.PluginSingletonLoader.Dispense( - mockdriver.PluginID.Name, - mockdriver.PluginID.PluginType, - nil, - conf.Logger, - ) + driverPlugin, err := conf.DriverManager.Dispense(mockdriver.PluginID.Name) require.NoError(err) - mockDriver := driverPlugin.Plugin().(*mockdriver.Driver) + mockDriver := driverPlugin.(*mockdriver.Driver) // Assert its config has been properly interpolated driverCfg, _ := mockDriver.GetTaskConfig() diff --git a/client/allocrunner/testing.go b/client/allocrunner/testing.go index 0a3851b1d..7c8e4c296 100644 --- a/client/allocrunner/testing.go +++ b/client/allocrunner/testing.go @@ -8,11 +8,10 @@ import ( clientconfig "github.com/hashicorp/nomad/client/config" "github.com/hashicorp/nomad/client/consul" "github.com/hashicorp/nomad/client/devicemanager" + "github.com/hashicorp/nomad/client/pluginmanager/drivermanager" "github.com/hashicorp/nomad/client/state" "github.com/hashicorp/nomad/client/vaultclient" "github.com/hashicorp/nomad/nomad/structs" - "github.com/hashicorp/nomad/plugins/shared/catalog" - "github.com/hashicorp/nomad/plugins/shared/singleton" "github.com/stretchr/testify/require" ) @@ -51,21 +50,20 @@ func (m *MockStateUpdater) Reset() { } func testAllocRunnerConfig(t *testing.T, alloc *structs.Allocation) (*Config, func()) { - pluginLoader := catalog.TestPluginLoader(t) clientConf, cleanup := clientconfig.TestClientConfig(t) conf := &Config{ // Copy the alloc in case the caller edits and reuses it - Alloc: alloc.Copy(), - Logger: clientConf.Logger, - ClientConfig: clientConf, - StateDB: state.NoopDB{}, - Consul: consul.NewMockConsulServiceClient(t, clientConf.Logger), - Vault: vaultclient.NewMockVaultClient(), - StateUpdater: &MockStateUpdater{}, - PrevAllocWatcher: allocwatcher.NoopPrevAlloc{}, - PrevAllocMigrator: allocwatcher.NoopPrevAlloc{}, - PluginSingletonLoader: singleton.NewSingletonLoader(clientConf.Logger, pluginLoader), - DeviceManager: devicemanager.NoopMockManager(), + Alloc: alloc.Copy(), + Logger: clientConf.Logger, + ClientConfig: clientConf, + StateDB: state.NoopDB{}, + Consul: consul.NewMockConsulServiceClient(t, clientConf.Logger), + Vault: vaultclient.NewMockVaultClient(), + StateUpdater: &MockStateUpdater{}, + PrevAllocWatcher: allocwatcher.NoopPrevAlloc{}, + PrevAllocMigrator: allocwatcher.NoopPrevAlloc{}, + DeviceManager: devicemanager.NoopMockManager(), + DriverManager: drivermanager.TestDriverManager(t), } return conf, cleanup } diff --git a/client/client.go b/client/client.go index 882b7d42e..1e5552c75 100644 --- a/client/client.go +++ b/client/client.go @@ -28,6 +28,8 @@ import ( consulApi "github.com/hashicorp/nomad/client/consul" "github.com/hashicorp/nomad/client/devicemanager" "github.com/hashicorp/nomad/client/fingerprint" + "github.com/hashicorp/nomad/client/pluginmanager" + "github.com/hashicorp/nomad/client/pluginmanager/drivermanager" "github.com/hashicorp/nomad/client/servers" "github.com/hashicorp/nomad/client/state" "github.com/hashicorp/nomad/client/stats" @@ -121,6 +123,7 @@ type AllocRunner interface { WaitCh() <-chan struct{} DestroyCh() <-chan struct{} ShutdownCh() <-chan struct{} + GetTaskEventHandler(taskName string) drivermanager.EventHandler } // Client is used to implement the client interaction with Nomad. Clients @@ -219,12 +222,21 @@ type Client struct { endpoints rpcEndpoints streamingRpcs *structs.StreamingRpcRegistry + // pluginManagers is the set of PluginManagers registered by the client + pluginManagers *pluginmanager.PluginGroup + // devicemanger is responsible for managing device plugins. devicemanager devicemanager.Manager + // drivermanager is responsible for managing driver plugins + drivermanager drivermanager.Manager + // baseLabels are used when emitting tagged metrics. All client metrics will // have these tags, and optionally more. baseLabels []metrics.Label + + // batchNodeUpdates is used to batch initial updates to the node + batchNodeUpdates *batchNodeUpdates } var ( @@ -271,6 +283,11 @@ func NewClient(cfg *config.Config, consulCatalog consul.CatalogAPI, consulServic triggerEmitNodeEvent: make(chan *structs.NodeEvent, 8), } + c.batchNodeUpdates = newBatchNodeUpdates( + c.updateNodeFromDriver, + c.updateNodeFromDevices, + ) + // Initialize the server manager c.servers = servers.New(c.logger, c.shutdownCh, c) @@ -298,26 +315,52 @@ func NewClient(cfg *config.Config, consulCatalog consul.CatalogAPI, consulServic c.configCopy = c.config.Copy() c.configLock.Unlock() - fingerprintManager := NewFingerprintManager(c.configCopy.PluginSingletonLoader, c.GetConfig, c.configCopy.Node, - c.shutdownCh, c.updateNodeFromFingerprint, c.updateNodeFromDriver, - c.logger) + fingerprintManager := NewFingerprintManager( + c.configCopy.PluginSingletonLoader, c.GetConfig, c.configCopy.Node, + c.shutdownCh, c.updateNodeFromFingerprint, c.logger) + + c.pluginManagers = pluginmanager.New(c.logger) // Fingerprint the node and scan for drivers if err := fingerprintManager.Run(); err != nil { return nil, fmt.Errorf("fingerprinting failed: %v", err) } + // Build the white/blacklists of drivers. + allowlistDrivers := cfg.ReadStringListToMap("driver.whitelist") + blocklistDrivers := cfg.ReadStringListToMap("driver.blacklist") + + // Setup the driver manager + driverConfig := &drivermanager.Config{ + Logger: c.logger, + Loader: c.configCopy.PluginSingletonLoader, + PluginConfig: c.configCopy.NomadPluginConfig(), + Updater: c.batchNodeUpdates.updateNodeFromDriver, + EventHandlerFactory: c.GetTaskEventHandler, + State: c.stateDB, + AllowedDrivers: allowlistDrivers, + BlockedDrivers: blocklistDrivers, + } + drvManager := drivermanager.New(driverConfig) + c.drivermanager = drvManager + c.pluginManagers.RegisterAndRun(drvManager) + // Setup the device manager devConfig := &devicemanager.Config{ Logger: c.logger, Loader: c.configCopy.PluginSingletonLoader, PluginConfig: c.configCopy.NomadPluginConfig(), - Updater: c.updateNodeFromDevices, + Updater: c.batchNodeUpdates.updateNodeFromDevices, StatsInterval: c.configCopy.StatsCollectionInterval, State: c.stateDB, } - c.devicemanager = devicemanager.New(devConfig) - go c.devicemanager.Run() + devManager := devicemanager.New(devConfig) + c.devicemanager = devManager + c.pluginManagers.RegisterAndRun(devManager) + + // Batching of initial fingerprints is done to reduce the number of node + // updates sent to the server on startup. + go c.batchFirstFingerprints() // Add the stats collector statsCollector := stats.NewHostStatsCollector(c.logger, c.config.AllocDir, c.devicemanager.AllStats) @@ -558,8 +601,8 @@ func (c *Client) Shutdown() error { } c.logger.Info("shutting down") - // Shutdown the device manager - c.devicemanager.Shutdown() + // Shutdown the plugin managers + c.pluginManagers.Shutdown() // Stop renewing tokens and secrets if c.vaultClient != nil { @@ -866,19 +909,18 @@ func (c *Client) restoreState() error { c.configLock.RLock() arConf := &allocrunner.Config{ - Alloc: alloc, - Logger: c.logger, - ClientConfig: c.configCopy, - StateDB: c.stateDB, - StateUpdater: c, - DeviceStatsReporter: c, - Consul: c.consulService, - Vault: c.vaultClient, - PrevAllocWatcher: prevAllocWatcher, - PrevAllocMigrator: prevAllocMigrator, - PluginLoader: c.config.PluginLoader, - PluginSingletonLoader: c.config.PluginSingletonLoader, - DeviceManager: c.devicemanager, + Alloc: alloc, + Logger: c.logger, + ClientConfig: c.configCopy, + StateDB: c.stateDB, + StateUpdater: c, + DeviceStatsReporter: c, + Consul: c.consulService, + Vault: c.vaultClient, + PrevAllocWatcher: prevAllocWatcher, + PrevAllocMigrator: prevAllocMigrator, + DeviceManager: c.devicemanager, + DriverManager: c.drivermanager, } c.configLock.RUnlock() @@ -1142,97 +1184,6 @@ func (c *Client) updateNodeFromFingerprint(response *fingerprint.FingerprintResp return c.configCopy.Node } -// updateNodeFromDriver receives a DriverInfo struct for the driver and updates -// the node accordingly -func (c *Client) updateNodeFromDriver(name string, info *structs.DriverInfo) *structs.Node { - c.configLock.Lock() - defer c.configLock.Unlock() - if info == nil { - return c.configCopy.Node - } - - var hasChanged bool - - hadDriver := c.config.Node.Drivers[name] != nil - if !hadDriver { - // If the driver info has not yet been set, do that here - hasChanged = true - for attrName, newVal := range info.Attributes { - c.config.Node.Attributes[attrName] = newVal - } - } else { - oldVal := c.config.Node.Drivers[name] - // The driver info has already been set, fix it up - if oldVal.Detected != info.Detected { - hasChanged = true - } - - if oldVal.Healthy != info.Healthy || oldVal.HealthDescription != info.HealthDescription { - hasChanged = true - - if info.HealthDescription != "" { - event := &structs.NodeEvent{ - Subsystem: "Driver", - Message: info.HealthDescription, - Timestamp: time.Now(), - Details: map[string]string{"driver": name}, - } - c.triggerNodeEvent(event) - } - } - - for attrName, newVal := range info.Attributes { - oldVal := c.config.Node.Drivers[name].Attributes[attrName] - if oldVal == newVal { - continue - } - - hasChanged = true - - if newVal == "" { - delete(c.config.Node.Attributes, attrName) - } else { - c.config.Node.Attributes[attrName] = newVal - } - } - } - - // COMPAT Remove in Nomad 0.10 - // We maintain the driver enabled attribute until all drivers expose - // their attributes as DriverInfo - driverName := fmt.Sprintf("driver.%s", name) - if info.Detected { - c.config.Node.Attributes[driverName] = "1" - } else { - delete(c.config.Node.Attributes, driverName) - } - - if hasChanged { - c.config.Node.Drivers[name] = info - c.config.Node.Drivers[name].UpdateTime = time.Now() - c.updateNodeLocked() - } - - return c.configCopy.Node -} - -// updateNodeFromFingerprint updates the node with the result of -// fingerprinting the node from the diff that was created -func (c *Client) updateNodeFromDevices(devices []*structs.NodeDeviceResource) { - c.configLock.Lock() - defer c.configLock.Unlock() - - // Not updating node.Resources: the field is deprecated and includes - // dispatched task resources and not appropriate for expressing - // node available device resources - if !structs.DevicesEquals(c.config.Node.NodeResources.Devices, devices) { - c.logger.Debug("new devices detected", "devices", len(devices)) - - c.config.Node.NodeResources.Devices = devices - c.updateNodeLocked() - } -} - // resourcesAreEqual is a temporary function to compare whether resources are // equal. We can use this until we change fingerprinters to set pointers on a // return type. @@ -2056,19 +2007,18 @@ func (c *Client) addAlloc(alloc *structs.Allocation, migrateToken string) error // we don't have to do a copy. c.configLock.RLock() arConf := &allocrunner.Config{ - Alloc: alloc, - Logger: c.logger, - ClientConfig: c.configCopy, - StateDB: c.stateDB, - Consul: c.consulService, - Vault: c.vaultClient, - StateUpdater: c, - DeviceStatsReporter: c, - PrevAllocWatcher: prevAllocWatcher, - PrevAllocMigrator: prevAllocMigrator, - PluginLoader: c.config.PluginLoader, - PluginSingletonLoader: c.config.PluginSingletonLoader, - DeviceManager: c.devicemanager, + Alloc: alloc, + Logger: c.logger, + ClientConfig: c.configCopy, + StateDB: c.stateDB, + Consul: c.consulService, + Vault: c.vaultClient, + StateUpdater: c, + DeviceStatsReporter: c, + PrevAllocWatcher: prevAllocWatcher, + PrevAllocMigrator: prevAllocMigrator, + DeviceManager: c.devicemanager, + DriverManager: c.drivermanager, } c.configLock.RUnlock() @@ -2668,6 +2618,16 @@ func (c *Client) allAllocs() map[string]*structs.Allocation { return allocs } +// GetTaskEventHandler returns an event handler for the given allocID and task name +func (c *Client) GetTaskEventHandler(allocID, taskName string) drivermanager.EventHandler { + c.allocLock.RLock() + defer c.allocLock.RUnlock() + if ar, ok := c.allocs[allocID]; ok { + return ar.GetTaskEventHandler(taskName) + } + return nil +} + // group wraps a func() in a goroutine and provides a way to block until it // exits. Inspired by https://godoc.org/golang.org/x/sync/errgroup type group struct { diff --git a/client/client_test.go b/client/client_test.go index f81cfe93f..723f5fbfb 100644 --- a/client/client_test.go +++ b/client/client_test.go @@ -1199,7 +1199,8 @@ func TestClient_updateNodeFromDriverUpdatesAll(t *testing.T) { "node.mock.testattr1": "val1", }, } - n := client.updateNodeFromDriver("mock", info) + client.updateNodeFromDriver("mock", info) + n := client.config.Node updatedInfo := *n.Drivers["mock"] // compare without update time @@ -1220,7 +1221,8 @@ func TestClient_updateNodeFromDriverUpdatesAll(t *testing.T) { "node.mock.testattr1": "val2", }, } - n := client.updateNodeFromDriver("mock", info) + client.updateNodeFromDriver("mock", info) + n := client.Node() updatedInfo := *n.Drivers["mock"] // compare without update time @@ -1231,7 +1233,8 @@ func TestClient_updateNodeFromDriverUpdatesAll(t *testing.T) { assert.Equal(t, "val2", n.Attributes["node.mock.testattr1"]) // update once more with the same info, updateTime shouldn't change - un := client.updateNodeFromDriver("mock", info) + client.updateNodeFromDriver("mock", info) + un := client.Node() assert.EqualValues(t, n, un) } @@ -1245,7 +1248,8 @@ func TestClient_updateNodeFromDriverUpdatesAll(t *testing.T) { "node.mock.testattr1": "", }, } - n := client.updateNodeFromDriver("mock", info) + client.updateNodeFromDriver("mock", info) + n := client.Node() updatedInfo := *n.Drivers["mock"] // compare without update time @@ -1256,7 +1260,8 @@ func TestClient_updateNodeFromDriverUpdatesAll(t *testing.T) { assert.Equal(t, "", n.Attributes["node.mock.testattr1"]) // update once more with the same info, updateTime shouldn't change - un := client.updateNodeFromDriver("mock", info) + client.updateNodeFromDriver("mock", info) + un := client.Node() assert.EqualValues(t, n, un) } } diff --git a/client/devicemanager/manager.go b/client/devicemanager/manager.go index 2709b2230..86bc0eca3 100644 --- a/client/devicemanager/manager.go +++ b/client/devicemanager/manager.go @@ -11,6 +11,7 @@ import ( multierror "github.com/hashicorp/go-multierror" plugin "github.com/hashicorp/go-plugin" "github.com/hashicorp/nomad/client/devicemanager/state" + "github.com/hashicorp/nomad/client/pluginmanager" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/plugins/base" "github.com/hashicorp/nomad/plugins/device" @@ -18,13 +19,9 @@ import ( "github.com/hashicorp/nomad/plugins/shared/loader" ) -// Manaager is the interface used to manage device plugins +// Manager is the interface used to manage device plugins type Manager interface { - // Run starts the device manager - Run() - - // Shutdown shutsdown the manager and all launched plugins - Shutdown() + pluginmanager.PluginManager // Reserve is used to reserve a set of devices Reserve(d *structs.AllocatedDeviceResource) (*device.ContainerReservation, error) @@ -127,6 +124,9 @@ func New(c *Config) *manager { } } +// PluginType identifies this manager to the plugin manager and satisfies the PluginManager interface. +func (*manager) PluginType() string { return base.PluginTypeDevice } + // Run starts thed device manager. The manager will shutdown any previously // launched plugin and then begin fingerprinting and stats collection on all new // device plugins. @@ -161,17 +161,12 @@ func (m *manager) Run() { }) } - // XXX we should eventually remove this and have it be done in the client - // Give all the fingerprinters a chance to run at least once before we - // update the node. This prevents initial fingerprinting from causing too - // many server side updates. - ctx, cancel := context.WithTimeout(m.ctx, 5*time.Second) - for _, i := range m.instances { - i.WaitForFirstFingerprint(ctx) - } - cancel() - // Now start the fingerprint handler + go m.fingerprint() +} + +// fingerprint is the main fingerprint loop +func (m *manager) fingerprint() { for { select { case <-m.ctx.Done(): @@ -207,6 +202,23 @@ func (m *manager) Shutdown() { } } +func (m *manager) WaitForFirstFingerprint(ctx context.Context) <-chan struct{} { + ctx, cancel := context.WithCancel(ctx) + go func() { + var wg sync.WaitGroup + for i := range m.instances { + wg.Add(1) + go func(instance *instanceManager) { + instance.WaitForFirstFingerprint(ctx) + wg.Done() + }(m.instances[i]) + } + wg.Wait() + cancel() + }() + return ctx.Done() +} + // Reserve reserves the given allocated device. If the device is unknown, an // UnknownDeviceErr is returned. func (m *manager) Reserve(d *structs.AllocatedDeviceResource) (*device.ContainerReservation, error) { diff --git a/client/devicemanager/manager_test.go b/client/devicemanager/manager_test.go index c59e1fb04..ac82be354 100644 --- a/client/devicemanager/manager_test.go +++ b/client/devicemanager/manager_test.go @@ -236,20 +236,19 @@ func TestManager_AllStats(t *testing.T) { t.Parallel() require := require.New(t) - config, updateCh, catalog := baseTestConfig(t) + config, _, catalog := baseTestConfig(t) nvidiaAndIntelDefaultPlugins(catalog) m := New(config) - go m.Run() + m.Run() defer m.Shutdown() + require.Len(m.instances, 2) // Wait till we get a fingerprint result - select { - case <-time.After(5 * time.Second): - t.Fatal("timeout") - case devices := <-updateCh: - require.Len(devices, 2) - } + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + <-m.WaitForFirstFingerprint(ctx) + require.NoError(ctx.Err()) // Now collect all the stats var stats []*device.DeviceGroupStats @@ -286,20 +285,18 @@ func TestManager_DeviceStats(t *testing.T) { t.Parallel() require := require.New(t) - config, updateCh, catalog := baseTestConfig(t) + config, _, catalog := baseTestConfig(t) nvidiaAndIntelDefaultPlugins(catalog) m := New(config) - go m.Run() + m.Run() defer m.Shutdown() // Wait till we get a fingerprint result - select { - case <-time.After(5 * time.Second): - t.Fatal("timeout") - case devices := <-updateCh: - require.Len(devices, 2) - } + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + <-m.WaitForFirstFingerprint(ctx) + require.NoError(ctx.Err()) testutil.WaitForResult(func() (bool, error) { stats := m.AllStats() @@ -335,20 +332,18 @@ func TestManager_Reserve(t *testing.T) { t.Parallel() r := require.New(t) - config, updateCh, catalog := baseTestConfig(t) + config, _, catalog := baseTestConfig(t) nvidiaAndIntelDefaultPlugins(catalog) m := New(config) - go m.Run() + m.Run() defer m.Shutdown() // Wait till we get a fingerprint result - select { - case <-time.After(5 * time.Second): - t.Fatal("timeout") - case devices := <-updateCh: - r.Len(devices, 2) - } + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + <-m.WaitForFirstFingerprint(ctx) + r.NoError(ctx.Err()) cases := []struct { in *structs.AllocatedDeviceResource @@ -435,20 +430,18 @@ func TestManager_Shutdown(t *testing.T) { t.Parallel() require := require.New(t) - config, updateCh, catalog := baseTestConfig(t) + config, _, catalog := baseTestConfig(t) nvidiaAndIntelDefaultPlugins(catalog) m := New(config) - go m.Run() + m.Run() defer m.Shutdown() // Wait till we get a fingerprint result - select { - case <-time.After(5 * time.Second): - t.Fatal("timeout") - case devices := <-updateCh: - require.Len(devices, 2) - } + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + <-m.WaitForFirstFingerprint(ctx) + require.NoError(ctx.Err()) // Call shutdown and assert that we killed the plugins m.Shutdown() @@ -464,20 +457,18 @@ func TestManager_Run_ShutdownOld(t *testing.T) { t.Parallel() require := require.New(t) - config, updateCh, catalog := baseTestConfig(t) + config, _, catalog := baseTestConfig(t) nvidiaAndIntelDefaultPlugins(catalog) m := New(config) - go m.Run() + m.Run() defer m.Shutdown() // Wait till we get a fingerprint result - select { - case <-time.After(5 * time.Second): - t.Fatal("timeout") - case devices := <-updateCh: - require.Len(devices, 2) - } + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + <-m.WaitForFirstFingerprint(ctx) + require.NoError(ctx.Err()) // Create a new manager with the same config so that it reads the old state m2 := New(config) diff --git a/client/devicemanager/testing.go b/client/devicemanager/testing.go index 3b9f9833b..602d9007d 100644 --- a/client/devicemanager/testing.go +++ b/client/devicemanager/testing.go @@ -2,6 +2,7 @@ package devicemanager import ( "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/plugins/base" "github.com/hashicorp/nomad/plugins/device" ) @@ -37,6 +38,7 @@ type MockManager struct { func (m *MockManager) Run() {} func (m *MockManager) Shutdown() {} +func (m *MockManager) PluginType() string { return base.PluginTypeDevice } func (m *MockManager) AllStats() []*device.DeviceGroupStats { return m.AllStatsF() } func (m *MockManager) Reserve(d *structs.AllocatedDeviceResource) (*device.ContainerReservation, error) { diff --git a/client/fingerprint_manager.go b/client/fingerprint_manager.go index 4b856d9da..03a2b3ca9 100644 --- a/client/fingerprint_manager.go +++ b/client/fingerprint_manager.go @@ -1,8 +1,6 @@ package client import ( - "context" - "fmt" "sync" "time" @@ -10,20 +8,7 @@ import ( "github.com/hashicorp/nomad/client/config" "github.com/hashicorp/nomad/client/fingerprint" "github.com/hashicorp/nomad/nomad/structs" - "github.com/hashicorp/nomad/plugins/base" - "github.com/hashicorp/nomad/plugins/drivers" "github.com/hashicorp/nomad/plugins/shared/loader" - pstructs "github.com/hashicorp/nomad/plugins/shared/structs" -) - -const ( - // driverFPBackoffBaseline is the baseline time for exponential backoff while - // fingerprinting a driver. - driverFPBackoffBaseline = 5 * time.Second - - // driverFPBackoffLimit is the limit of the exponential backoff for fingerprinting - // a driver. - driverFPBackoffLimit = 2 * time.Minute ) // FingerprintManager runs a client fingerprinters on a continuous basis, and @@ -39,10 +24,7 @@ type FingerprintManager struct { // associated node updateNodeAttributes func(*fingerprint.FingerprintResponse) *structs.Node - // updateNodeFromDriver is a callback to the client to update the state of a - // specific driver for the node - updateNodeFromDriver func(string, *structs.DriverInfo) *structs.Node - logger log.Logger + logger log.Logger } // NewFingerprintManager is a constructor that creates and returns an instance @@ -53,14 +35,12 @@ func NewFingerprintManager( node *structs.Node, shutdownCh chan struct{}, updateNodeAttributes func(*fingerprint.FingerprintResponse) *structs.Node, - updateNodeFromDriver func(string, *structs.DriverInfo) *structs.Node, logger log.Logger) *FingerprintManager { return &FingerprintManager{ singletonLoader: singletonLoader, getConfig: getConfig, updateNodeAttributes: updateNodeAttributes, - updateNodeFromDriver: updateNodeFromDriver, node: node, shutdownCh: shutdownCh, logger: logger.Named("fingerprint_mgr"), @@ -120,39 +100,6 @@ func (fp *FingerprintManager) Run() error { "skipped_fingerprinters", skippedFingerprints) } - // Next, set up drivers - // Build the white/blacklists of drivers. - whitelistDrivers := cfg.ReadStringListToMap("driver.whitelist") - whitelistDriversEnabled := len(whitelistDrivers) > 0 - blacklistDrivers := cfg.ReadStringListToMap("driver.blacklist") - - var availDrivers []string - var skippedDrivers []string - - for _, pl := range fp.singletonLoader.Catalog()[base.PluginTypeDriver] { - name := pl.Name - // Skip fingerprinting drivers that are not in the whitelist if it is - // enabled. - if _, ok := whitelistDrivers[name]; whitelistDriversEnabled && !ok { - skippedDrivers = append(skippedDrivers, name) - continue - } - // Skip fingerprinting drivers that are in the blacklist - if _, ok := blacklistDrivers[name]; ok { - skippedDrivers = append(skippedDrivers, name) - continue - } - - availDrivers = append(availDrivers, name) - } - - if err := fp.setupDrivers(availDrivers); err != nil { - return err - } - - if len(skippedDrivers) > 0 { - fp.logger.Debug("drivers skipped due to white/blacklist", "skipped_drivers", skippedDrivers) - } return nil } @@ -189,38 +136,6 @@ func (fm *FingerprintManager) setupFingerprinters(fingerprints []string) error { return nil } -// setupDrivers is used to fingerprint the node to see if these drivers are -// supported -func (fm *FingerprintManager) setupDrivers(driverNames []string) error { - //TODO(alex,hclog) Update fingerprinters to hclog - var availDrivers []string - for _, name := range driverNames { - // TODO: driver reattach - fingerCh, cancel, err := fm.dispenseDriverFingerprint(name) - if err != nil { - return err - } - - finger := <-fingerCh - - // Start a periodic watcher to detect changes to a drivers health and - // attributes. - go fm.watchDriverFingerprint(fingerCh, name, cancel) - - if fm.logger.IsTrace() { - fm.logger.Trace("initial driver fingerprint", "driver", name, "fingerprint", finger) - } - // Log the fingerprinters which have been applied - if finger.Health != drivers.HealthStateUndetected { - availDrivers = append(availDrivers, name) - } - fm.processDriverFingerprint(finger, name) - } - - fm.logger.Debug("detected drivers", "drivers", availDrivers) - return nil -} - // runFingerprint runs each fingerprinter individually on an ongoing basis func (fm *FingerprintManager) runFingerprint(f fingerprint.Fingerprint, period time.Duration, name string) { fm.logger.Debug("fingerprinting periodically", "fingerprinter", name, "period", period) @@ -266,99 +181,3 @@ func (fm *FingerprintManager) fingerprint(name string, f fingerprint.Fingerprint return response.Detected, nil } - -// watchDrivers facilitates the different periods between fingerprint and -// health checking a driver -func (fm *FingerprintManager) watchDriverFingerprint(fpChan <-chan *drivers.Fingerprint, name string, cancel context.CancelFunc) { - var backoff time.Duration - var retry int - for { - if backoff > 0 { - time.Sleep(backoff) - } - select { - case <-fm.shutdownCh: - cancel() - return - case fp, ok := <-fpChan: - if ok && fp.Err == nil { - fm.processDriverFingerprint(fp, name) - continue - } - // if the channel is closed attempt to open a new one - newFpChan, newCancel, err := fm.dispenseDriverFingerprint(name) - if err != nil { - fm.logger.Warn("failed to fingerprint driver, retrying in 30s", "error", err, "retry", retry) - di := &structs.DriverInfo{ - Healthy: false, - HealthDescription: "failed to fingerprint driver", - UpdateTime: time.Now(), - } - if n := fm.updateNodeFromDriver(name, di); n != nil { - fm.setNode(n) - } - - // Calculate the new backoff - backoff = (1 << (2 * uint64(retry))) * driverFPBackoffBaseline - if backoff > driverFPBackoffLimit { - backoff = driverFPBackoffLimit - } - retry++ - continue - } - cancel() - fpChan = newFpChan - cancel = newCancel - - // Reset backoff - backoff = 0 - retry = 0 - } - } -} - -// processDriverFringerprint converts a Fingerprint from a driver into a DriverInfo -// struct and updates the Node with it -func (fm *FingerprintManager) processDriverFingerprint(fp *drivers.Fingerprint, driverName string) { - di := &structs.DriverInfo{ - Attributes: stringify(fp.Attributes), - Detected: fp.Health != drivers.HealthStateUndetected, - Healthy: fp.Health == drivers.HealthStateHealthy, - HealthDescription: fp.HealthDescription, - UpdateTime: time.Now(), - } - if n := fm.updateNodeFromDriver(driverName, di); n != nil { - fm.setNode(n) - } -} -func stringify(attributes map[string]*pstructs.Attribute) map[string]string { - ret := make(map[string]string, len(attributes)) - for key, attribute := range attributes { - ret[key] = attribute.GoString() - } - return ret -} - -// dispenseDriverFingerprint dispenses a driver plugin for the given driver name -// and requests a fingerprint channel. The channel and a context cancel function -// is returned to the caller -func (fm *FingerprintManager) dispenseDriverFingerprint(driverName string) (<-chan *drivers.Fingerprint, context.CancelFunc, error) { - plug, err := fm.singletonLoader.Dispense(driverName, base.PluginTypeDriver, fm.getConfig().NomadPluginConfig(), fm.logger) - if err != nil { - return nil, nil, err - } - - driver, ok := plug.Plugin().(drivers.DriverPlugin) - if !ok { - return nil, nil, fmt.Errorf("registered driver plugin %q does not implement DriverPlugin interface", driverName) - } - - ctx, cancel := context.WithCancel(context.Background()) - fingerCh, err := driver.Fingerprint(ctx) - if err != nil { - cancel() - return nil, nil, err - } - - return fingerCh, cancel, nil -} diff --git a/client/fingerprint_manager_test.go b/client/fingerprint_manager_test.go index f9095a0b2..a70000954 100644 --- a/client/fingerprint_manager_test.go +++ b/client/fingerprint_manager_test.go @@ -1,50 +1,12 @@ package client import ( - "fmt" "testing" - log "github.com/hashicorp/go-hclog" "github.com/hashicorp/nomad/client/config" - "github.com/hashicorp/nomad/helper/testlog" - "github.com/hashicorp/nomad/plugins/base" - "github.com/hashicorp/nomad/plugins/shared/loader" - "github.com/hashicorp/nomad/testutil" "github.com/stretchr/testify/require" - - // registering raw_exec driver plugin used in testing - _ "github.com/hashicorp/nomad/drivers/rawexec" ) -func TestFingerprintManager_Run_MockDriver(t *testing.T) { - t.Skip("missing mock driver plugin implementation") - t.Parallel() - require := require.New(t) - testClient, cleanup := TestClient(t, nil) - - testClient.logger = testlog.HCLogger(t) - defer cleanup() - - fm := NewFingerprintManager( - testClient.config.PluginSingletonLoader, - testClient.GetConfig, - testClient.config.Node, - testClient.shutdownCh, - testClient.updateNodeFromFingerprint, - testClient.updateNodeFromDriver, - testlog.HCLogger(t), - ) - - err := fm.Run() - require.Nil(err) - - node := testClient.config.Node - - require.NotNil(node.Drivers["mock_driver"]) - require.True(node.Drivers["mock_driver"].Detected) - require.True(node.Drivers["mock_driver"].Healthy) -} - func TestFingerprintManager_Run_ResourcesFingerprint(t *testing.T) { t.Parallel() require := require.New(t) @@ -57,7 +19,6 @@ func TestFingerprintManager_Run_ResourcesFingerprint(t *testing.T) { testClient.config.Node, testClient.shutdownCh, testClient.updateNodeFromFingerprint, - testClient.updateNodeFromDriver, testClient.logger, ) @@ -71,295 +32,7 @@ func TestFingerprintManager_Run_ResourcesFingerprint(t *testing.T) { require.NotZero(node.Resources.DiskMB) } -func TestFingerprintManager_Fingerprint_Run(t *testing.T) { - t.Parallel() - require := require.New(t) - testClient, cleanup := TestClient(t, func(c *config.Config) { - c.Options = map[string]string{ - "driver.raw_exec.enable": "true", - } - }) - defer cleanup() - - fm := NewFingerprintManager( - testClient.config.PluginSingletonLoader, - testClient.GetConfig, - testClient.config.Node, - testClient.shutdownCh, - testClient.updateNodeFromFingerprint, - testClient.updateNodeFromDriver, - testClient.logger, - ) - - err := fm.Run() - require.Nil(err) - - node := testClient.config.Node - - require.NotNil(node.Drivers["raw_exec"]) - require.True(node.Drivers["raw_exec"].Detected) - require.True(node.Drivers["raw_exec"].Healthy) -} - -func TestFingerprintManager_Fingerprint_Periodic(t *testing.T) { - t.Skip("missing mock driver plugin implementation") - t.Parallel() - require := require.New(t) - testClient, cleanup := TestClient(t, func(c *config.Config) { - c.Options = map[string]string{ - "test.shutdown_periodic_after": "true", - "test.shutdown_periodic_duration": "2", - } - }) - defer cleanup() - - fm := NewFingerprintManager( - testClient.config.PluginSingletonLoader, - testClient.GetConfig, - testClient.config.Node, - testClient.shutdownCh, - testClient.updateNodeFromFingerprint, - testClient.updateNodeFromDriver, - testClient.logger, - ) - - err := fm.Run() - require.Nil(err) - - { - // Ensure the mock driver is registered and healthy on the client - testutil.WaitForResult(func() (bool, error) { - fm.nodeLock.Lock() - defer fm.nodeLock.Unlock() - node := fm.node - dinfo, ok := node.Drivers["mock_driver"] - if !ok || !dinfo.Detected || !dinfo.Healthy { - return false, fmt.Errorf("mock driver should be detected and healthy: %+v", dinfo) - } - - return true, nil - }, func(err error) { - t.Fatalf("err: %v", err) - }) - } - // Ensure that the client fingerprinter eventually removes this attribute and - // marks the driver as unhealthy - { - testutil.WaitForResult(func() (bool, error) { - fm.nodeLock.Lock() - defer fm.nodeLock.Unlock() - node := fm.node - dinfo, ok := node.Drivers["mock_driver"] - if !ok || dinfo.Detected || dinfo.Healthy { - return false, fmt.Errorf("mock driver should not be detected and healthy") - } - return true, nil - }, func(err error) { - t.Fatalf("err: %v", err) - }) - } -} - -// 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.Skip("missing mock driver plugin implementation") - t.Parallel() - require := require.New(t) - testClient, cleanup := TestClient(t, func(c *config.Config) { - c.Options = map[string]string{ - "driver.raw_exec.enable": "1", - "test.shutdown_periodic_after": "true", - "test.shutdown_periodic_duration": "2", - } - }) - defer cleanup() - - fm := NewFingerprintManager( - testClient.config.PluginSingletonLoader, - testClient.GetConfig, - testClient.config.Node, - testClient.shutdownCh, - testClient.updateNodeFromFingerprint, - testClient.updateNodeFromDriver, - testClient.logger, - ) - - err := fm.Run() - require.Nil(err) - - // Ensure the mock driver is registered and healthy on the client - testutil.WaitForResult(func() (bool, error) { - fm.nodeLock.Lock() - node := fm.node - defer fm.nodeLock.Unlock() - - 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) { - fm.nodeLock.Lock() - node := fm.node - defer fm.nodeLock.Unlock() - - 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) - }) - - // Ensure the mock driver is registered - testutil.WaitForResult(func() (bool, error) { - fm.nodeLock.Lock() - node := fm.node - defer fm.nodeLock.Unlock() - - mockDriverAttribute := node.Attributes["driver.mock_driver"] - if mockDriverAttribute == "" { - return false, fmt.Errorf("mock driver info should 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 not be healthy") - } - return true, nil - }, func(err error) { - t.Fatalf("err: %v", err) - }) - - // Ensure that we don't duplicate health check information on the driver - // health information - fm.nodeLock.Lock() - node := fm.node - fm.nodeLock.Unlock() - mockDriverAttributes := node.Drivers["mock_driver"].Attributes - require.NotContains(mockDriverAttributes, "driver.mock_driver") -} - -func TestFingerprintManager_HealthCheck_Periodic(t *testing.T) { - t.Skip("missing mock driver plugin implementation") - t.Parallel() - require := require.New(t) - testClient, cleanup := TestClient(t, func(c *config.Config) { - c.Options = map[string]string{ - "test.shutdown_periodic_after": "true", - "test.shutdown_periodic_duration": "2", - } - }) - defer cleanup() - - fm := NewFingerprintManager( - testClient.config.PluginSingletonLoader, - testClient.GetConfig, - testClient.config.Node, - testClient.shutdownCh, - testClient.updateNodeFromFingerprint, - testClient.updateNodeFromDriver, - testClient.logger, - ) - - err := fm.Run() - require.Nil(err) - - { - // Ensure the mock driver is registered and healthy on the client - testutil.WaitForResult(func() (bool, error) { - fm.nodeLock.Lock() - node := fm.node - defer fm.nodeLock.Unlock() - - 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) { - fm.nodeLock.Lock() - node := fm.node - defer fm.nodeLock.Unlock() - - 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) { - fm.nodeLock.Lock() - node := fm.node - defer fm.nodeLock.Unlock() - - 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.Skip("missing mock driver plugin implementation") t.Parallel() require := require.New(t) @@ -377,7 +50,6 @@ func TestFimgerprintManager_Run_InWhitelist(t *testing.T) { testClient.config.Node, testClient.shutdownCh, testClient.updateNodeFromFingerprint, - testClient.updateNodeFromDriver, testClient.logger, ) @@ -406,7 +78,6 @@ func TestFingerprintManager_Run_InBlacklist(t *testing.T) { testClient.config.Node, testClient.shutdownCh, testClient.updateNodeFromFingerprint, - testClient.updateNodeFromDriver, testClient.logger, ) @@ -437,7 +108,6 @@ func TestFingerprintManager_Run_Combination(t *testing.T) { testClient.config.Node, testClient.shutdownCh, testClient.updateNodeFromFingerprint, - testClient.updateNodeFromDriver, testClient.logger, ) @@ -451,187 +121,3 @@ func TestFingerprintManager_Run_Combination(t *testing.T) { require.NotContains(node.Attributes, "memory.totalbytes") require.NotContains(node.Attributes, "os.name") } - -func TestFingerprintManager_Run_WhitelistDrivers(t *testing.T) { - t.Parallel() - require := require.New(t) - testClient, cleanup := TestClient(t, func(c *config.Config) { - c.Options = map[string]string{ - "driver.raw_exec.enable": "1", - "driver.whitelist": " raw_exec , foo ", - } - }) - defer cleanup() - - fm := NewFingerprintManager( - testClient.config.PluginSingletonLoader, - testClient.GetConfig, - testClient.config.Node, - testClient.shutdownCh, - testClient.updateNodeFromFingerprint, - testClient.updateNodeFromDriver, - testClient.logger, - ) - - err := fm.Run() - require.Nil(err) - - node := testClient.config.Node - require.NotNil(node.Drivers["raw_exec"]) - require.NotContains(node.Drivers, "java") -} - -func TestFingerprintManager_Run_AllDriversBlacklisted(t *testing.T) { - t.Parallel() - require := require.New(t) - - testClient, cleanup := TestClient(t, func(c *config.Config) { - c.Options = map[string]string{ - "driver.raw_exec.enable": "1", - "driver.whitelist": " foo,bar,baz ", - } - }) - defer cleanup() - - fm := NewFingerprintManager( - testClient.config.PluginSingletonLoader, - testClient.GetConfig, - testClient.config.Node, - testClient.shutdownCh, - testClient.updateNodeFromFingerprint, - testClient.updateNodeFromDriver, - testClient.logger, - ) - - err := fm.Run() - require.Nil(err) - - node := testClient.config.Node - - require.NotContains(node.Attributes, "driver.raw_exec") - // TODO(nickethier): uncomment after missing driver implementations added - //require.NotContains(node.Attributes, "driver.exec") - //require.NotContains(node.Attributes, "driver.docker") -} - -func TestFingerprintManager_Run_DriversWhiteListBlacklistCombination(t *testing.T) { - t.Parallel() - require := require.New(t) - - testClient, cleanup := TestClient(t, func(c *config.Config) { - c.Options = map[string]string{ - "driver.raw_exec.enable": "1", - "driver.whitelist": " raw_exec,exec,foo,bar,baz ", - "driver.blacklist": " exec,foo,bar,baz ", - } - }) - - testClient.logger = testlog.HCLogger(t) - defer cleanup() - - fm := NewFingerprintManager( - testClient.config.PluginSingletonLoader, - testClient.GetConfig, - testClient.config.Node, - testClient.shutdownCh, - testClient.updateNodeFromFingerprint, - testClient.updateNodeFromDriver, - testClient.logger, - ) - - err := fm.Run() - require.Nil(err) - - node := testClient.config.Node - - require.NotNil(node.Drivers["raw_exec"]) - require.NotContains(node.Drivers, "exec") -} - -func TestFingerprintManager_Run_DriverFailure(t *testing.T) { - t.Parallel() - require := require.New(t) - - testClient, cleanup := TestClient(t, func(c *config.Config) { - c.Options = map[string]string{ - "driver.raw_exec.enable": "1", - } - }) - - testClient.logger = testlog.HCLogger(t) - defer cleanup() - - singLoader := testClient.config.PluginSingletonLoader - - dispenseCalls := 0 - loader := &loader.MockCatalog{ - DispenseF: func(name, pluginType string, cfg *base.AgentConfig, logger log.Logger) (loader.PluginInstance, error) { - if pluginType == base.PluginTypeDriver && name == "raw_exec" { - dispenseCalls++ - } - return singLoader.Dispense(name, pluginType, cfg, logger) - }, - ReattachF: singLoader.Reattach, - CatalogF: singLoader.Catalog, - } - - fm := NewFingerprintManager( - loader, - testClient.GetConfig, - testClient.config.Node, - testClient.shutdownCh, - testClient.updateNodeFromFingerprint, - testClient.updateNodeFromDriver, - testClient.logger, - ) - - fpChan, cancel, err := fm.dispenseDriverFingerprint("raw_exec") - require.NoError(err) - require.Equal(1, dispenseCalls) - - cancel() - go fm.watchDriverFingerprint(fpChan, "raw_exec", cancel) - - testutil.WaitForResult(func() (bool, error) { - if 2 != dispenseCalls { - return false, fmt.Errorf("expected dispenseCalls to be 2 but was %d", dispenseCalls) - } - return true, nil - }, func(err error) { - require.NoError(err) - }) -} - -func TestFingerprintManager_Run_DriversInBlacklist(t *testing.T) { - t.Parallel() - require := require.New(t) - - testClient, cleanup := TestClient(t, func(c *config.Config) { - c.Options = map[string]string{ - "driver.raw_exec.enable": "1", - "driver.whitelist": " raw_exec,foo,bar,baz ", - "driver.blacklist": " exec,foo,bar,baz ", - } - }) - - testClient.logger = testlog.HCLogger(t) - defer cleanup() - - fm := NewFingerprintManager( - testClient.config.PluginSingletonLoader, - testClient.GetConfig, - testClient.config.Node, - testClient.shutdownCh, - testClient.updateNodeFromFingerprint, - testClient.updateNodeFromDriver, - testClient.logger, - ) - - err := fm.Run() - require.Nil(err) - - node := testClient.config.Node - - require.NotNil(node.Drivers["raw_exec"]) - require.NotContains(node.Drivers, "exec") -} diff --git a/client/node_updater.go b/client/node_updater.go new file mode 100644 index 000000000..24e5f76bc --- /dev/null +++ b/client/node_updater.go @@ -0,0 +1,255 @@ +package client + +import ( + "context" + "fmt" + "sync" + "time" + + "github.com/hashicorp/nomad/client/devicemanager" + "github.com/hashicorp/nomad/client/pluginmanager/drivermanager" + "github.com/hashicorp/nomad/nomad/structs" +) + +var ( + // batchFirstFingerprintsTimeout is the maximum amount of time to wait for + // initial fingerprinting to complete before sending a batched Node update + batchFirstFingerprintsTimeout = 5 * time.Second +) + +// batchFirstFingerprints waits for the first fingerprint response from all +// plugin managers and sends a single Node update for all fingerprints +func (c *Client) batchFirstFingerprints() { + ctx, cancel := context.WithTimeout(context.Background(), batchFirstFingerprintsTimeout) + defer cancel() + + ch, err := c.pluginManagers.WaitForFirstFingerprint(ctx) + if err != nil { + c.logger.Warn("failed to batch initial fingerprint updates, switching to incemental updates") + goto SEND_BATCH + } + + // Wait for fingerprinting to complete or timeout before processing batches + select { + case <-ch: + case <-ctx.Done(): + } + +SEND_BATCH: + c.configLock.Lock() + defer c.configLock.Unlock() + + // driver node updates + var driverChanged bool + c.batchNodeUpdates.batchDriverUpdates(func(driver string, info *structs.DriverInfo) { + if c.updateNodeFromDriverLocked(driver, info) { + c.config.Node.Drivers[driver] = info + if c.config.Node.Drivers[driver].UpdateTime.IsZero() { + c.config.Node.Drivers[driver].UpdateTime = time.Now() + } + driverChanged = true + } + }) + + // device node updates + var devicesChanged bool + c.batchNodeUpdates.batchDevicesUpdates(func(devices []*structs.NodeDeviceResource) { + if c.updateNodeFromDevicesLocked(devices) { + devicesChanged = true + } + }) + + // only update the node if changes occurred + if driverChanged || devicesChanged { + c.updateNodeLocked() + } +} + +// updateNodeFromDriver receives a DriverInfo struct for the driver and updates +// the node accordingly +func (c *Client) updateNodeFromDriver(name string, info *structs.DriverInfo) { + c.configLock.Lock() + defer c.configLock.Unlock() + + if c.updateNodeFromDriverLocked(name, info) { + c.config.Node.Drivers[name] = info + if c.config.Node.Drivers[name].UpdateTime.IsZero() { + c.config.Node.Drivers[name].UpdateTime = time.Now() + } + c.updateNodeLocked() + } +} + +// updateNodeFromDriverLocked makes the changes to the node from a driver update +// but does not send the update to the server. c.configLock must be held before +// calling this func +func (c *Client) updateNodeFromDriverLocked(name string, info *structs.DriverInfo) bool { + var hasChanged bool + + hadDriver := c.config.Node.Drivers[name] != nil + if !hadDriver { + // If the driver info has not yet been set, do that here + hasChanged = true + for attrName, newVal := range info.Attributes { + c.config.Node.Attributes[attrName] = newVal + } + } else { + oldVal := c.config.Node.Drivers[name] + // The driver info has already been set, fix it up + if oldVal.Detected != info.Detected { + hasChanged = true + } + + // If health state has change, trigger node event + if oldVal.Healthy != info.Healthy || oldVal.HealthDescription != info.HealthDescription { + hasChanged = true + if info.HealthDescription != "" { + event := &structs.NodeEvent{ + Subsystem: "Driver", + Message: info.HealthDescription, + Timestamp: time.Now(), + Details: map[string]string{"driver": name}, + } + c.triggerNodeEvent(event) + } + } + + for attrName, newVal := range info.Attributes { + oldVal := c.config.Node.Drivers[name].Attributes[attrName] + if oldVal == newVal { + continue + } + + hasChanged = true + if newVal == "" { + delete(c.config.Node.Attributes, attrName) + } else { + c.config.Node.Attributes[attrName] = newVal + } + } + } + + // COMPAT Remove in Nomad 0.10 + // We maintain the driver enabled attribute until all drivers expose + // their attributes as DriverInfo + driverName := fmt.Sprintf("driver.%s", name) + if info.Detected { + c.config.Node.Attributes[driverName] = "1" + } else { + delete(c.config.Node.Attributes, driverName) + } + + return hasChanged +} + +// updateNodeFromFingerprint updates the node with the result of +// fingerprinting the node from the diff that was created +func (c *Client) updateNodeFromDevices(devices []*structs.NodeDeviceResource) { + c.configLock.Lock() + defer c.configLock.Unlock() + + // Not updating node.Resources: the field is deprecated and includes + // dispatched task resources and not appropriate for expressing + // node available device resources + if c.updateNodeFromDevicesLocked(devices) { + c.updateNodeLocked() + } +} + +// updateNodeFromDevicesLocked updates the node with the results of devices, +// but does send the update to the server. c.configLock must be held before +// calling this func +func (c *Client) updateNodeFromDevicesLocked(devices []*structs.NodeDeviceResource) bool { + if !structs.DevicesEquals(c.config.Node.NodeResources.Devices, devices) { + c.logger.Debug("new devices detected", "devices", len(devices)) + c.config.Node.NodeResources.Devices = devices + return true + } + + return false +} + +// batchNodeUpdates allows for batching multiple Node updates from fingerprinting. +// Once ready, the batches can be flushed and toggled to stop batching and forward +// all updates to a configured callback to be performed incrementally +type batchNodeUpdates struct { + // access to driver fields must hold driversMu lock + drivers map[string]*structs.DriverInfo + driversBatched bool + driverCB drivermanager.UpdateNodeDriverInfoFn + driversMu sync.Mutex + + // access to devices fields must hold devicesMu lock + devices []*structs.NodeDeviceResource + devicesBatched bool + devicesCB devicemanager.UpdateNodeDevicesFn + devicesMu sync.Mutex +} + +func newBatchNodeUpdates( + driverCB drivermanager.UpdateNodeDriverInfoFn, + devicesCB devicemanager.UpdateNodeDevicesFn) *batchNodeUpdates { + + return &batchNodeUpdates{ + drivers: make(map[string]*structs.DriverInfo), + driverCB: driverCB, + devices: []*structs.NodeDeviceResource{}, + devicesCB: devicesCB, + } +} + +// updateNodeFromDriver implements drivermanager.UpdateNodeDriverInfoFn and is +// used in the driver manager to send driver fingerprints to +func (b *batchNodeUpdates) updateNodeFromDriver(driver string, info *structs.DriverInfo) { + b.driversMu.Lock() + defer b.driversMu.Unlock() + if b.driversBatched { + b.driverCB(driver, info) + return + } + + b.drivers[driver] = info +} + +// batchDriverUpdates sends all of the batched driver node updates by calling f +// for each driver batched +func (b *batchNodeUpdates) batchDriverUpdates(f drivermanager.UpdateNodeDriverInfoFn) error { + b.driversMu.Lock() + defer b.driversMu.Unlock() + if b.driversBatched { + return fmt.Errorf("driver updates already batched") + } + + b.driversBatched = true + for driver, info := range b.drivers { + f(driver, info) + } + return nil +} + +// updateNodeFromDevices implements devicemanager.UpdateNodeDevicesFn and is +// used in the device manager to send device fingerprints to +func (b *batchNodeUpdates) updateNodeFromDevices(devices []*structs.NodeDeviceResource) { + b.devicesMu.Lock() + defer b.devicesMu.Unlock() + if b.devicesBatched { + b.devicesCB(devices) + return + } + + b.devices = devices +} + +// batchDevicesUpdates sends the batched device node updates by calling f with +// the devices +func (b *batchNodeUpdates) batchDevicesUpdates(f devicemanager.UpdateNodeDevicesFn) error { + b.devicesMu.Lock() + defer b.devicesMu.Unlock() + if b.devicesBatched { + return fmt.Errorf("devices updates already batched") + } + + b.devicesBatched = true + f(b.devices) + return nil +} diff --git a/client/pluginmanager/drivermanager/instance.go b/client/pluginmanager/drivermanager/instance.go new file mode 100644 index 000000000..a9f21eac6 --- /dev/null +++ b/client/pluginmanager/drivermanager/instance.go @@ -0,0 +1,457 @@ +package drivermanager + +import ( + "context" + "fmt" + "sync" + "time" + + log "github.com/hashicorp/go-hclog" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/plugins/base" + "github.com/hashicorp/nomad/plugins/drivers" + "github.com/hashicorp/nomad/plugins/shared/loader" + "github.com/hashicorp/nomad/plugins/shared/singleton" +) + +const ( + // driverFPBackoffBaseline is the baseline time for exponential backoff while + // fingerprinting a driver. + driverFPBackoffBaseline = 5 * time.Second + + // driverFPBackoffLimit is the limit of the exponential backoff for fingerprinting + // a driver. + driverFPBackoffLimit = 2 * time.Minute +) + +// instanceManagerConfig configures a driver instance manager +type instanceManagerConfig struct { + // Logger is the logger used by the driver instance manager + Logger log.Logger + + // Ctx is used to shutdown the driver instance manager + Ctx context.Context + + // Loader is the plugin loader + Loader loader.PluginCatalog + + // StoreReattach is used to store a plugins reattach config + StoreReattach StorePluginReattachFn + + // FetchReattach is used to retrieve a plugin's reattach config + FetchReattach FetchPluginReattachFn + + // PluginConfig is the config passed to the launched plugins + PluginConfig *base.AgentConfig + + // ID is the ID of the plugin being managed + ID *loader.PluginID + + // updateNodeFromDriver is the callback used to update the node from fingerprinting + UpdateNodeFromDriver UpdateNodeDriverInfoFn + + // EventHandlerFactory is used to fetch a task event handler + EventHandlerFactory TaskEventHandlerFactory +} + +// instanceManager is used to manage a single driver plugin +type instanceManager struct { + // logger is the logger used by the driver instance manager + logger log.Logger + + // ctx is used to shutdown the driver manager + ctx context.Context + + // cancel is used to shutdown management of this driver plugin + cancel context.CancelFunc + + // loader is the plugin loader + loader loader.PluginCatalog + + // storeReattach is used to store a plugins reattach config + storeReattach StorePluginReattachFn + + // fetchReattach is used to retrieve a plugin's reattach config + fetchReattach FetchPluginReattachFn + + // pluginConfig is the config passed to the launched plugins + pluginConfig *base.AgentConfig + + // id is the ID of the plugin being managed + id *loader.PluginID + + // plugin is the plugin instance being managed + plugin loader.PluginInstance + + // driver is the driver plugin being managed + driver drivers.DriverPlugin + + // pluginLock locks access to the driver and plugin + pluginLock sync.Mutex + + // shutdownLock is used to serialize attempts to shutdown + shutdownLock sync.Mutex + + // updateNodeFromDriver is the callback used to update the node from fingerprinting + updateNodeFromDriver UpdateNodeDriverInfoFn + + // eventHandlerFactory is used to fetch a handler for a task event + eventHandlerFactory TaskEventHandlerFactory + + // firstFingerprintCh is used to trigger that we have successfully + // fingerprinted once. It is used to gate launching the stats collection. + firstFingerprintCh chan struct{} + hasFingerprinted bool + + // lastHealthState is the last known health fingerprinted by the manager + lastHealthState drivers.HealthState + lastHealthStateMu sync.Mutex +} + +// newInstanceManager returns a new driver instance manager. It is expected that +// the context passed in the configuration is cancelled in order to shutdown +// launched goroutines. +func newInstanceManager(c *instanceManagerConfig) *instanceManager { + + ctx, cancel := context.WithCancel(c.Ctx) + i := &instanceManager{ + logger: c.Logger.With("driver", c.ID.Name), + ctx: ctx, + cancel: cancel, + loader: c.Loader, + storeReattach: c.StoreReattach, + fetchReattach: c.FetchReattach, + pluginConfig: c.PluginConfig, + id: c.ID, + updateNodeFromDriver: c.UpdateNodeFromDriver, + eventHandlerFactory: c.EventHandlerFactory, + firstFingerprintCh: make(chan struct{}), + } + + go i.run() + return i +} + +// WaitForFirstFingerprint waits until either the plugin fingerprints, the +// passed context is done, or the plugin instance manager is shutdown. +func (i *instanceManager) WaitForFirstFingerprint(ctx context.Context) { + select { + case <-i.ctx.Done(): + case <-ctx.Done(): + case <-i.firstFingerprintCh: + } +} + +// run is a long lived goroutine that starts the fingerprinting and stats +// collection goroutine and then shutsdown the plugin on exit. +func (i *instanceManager) run() { + // Dispense once to ensure we are given a valid plugin + if _, err := i.dispense(); err != nil { + i.logger.Error("dispensing initial plugin failed", "error", err) + return + } + + // Create a waitgroup to block on shutdown for all created goroutines to + // exit + var wg sync.WaitGroup + + // Start the fingerprinter + wg.Add(1) + go func() { + i.fingerprint() + wg.Done() + }() + + // Start event handler + wg.Add(1) + go func() { + i.handleEvents() + wg.Done() + }() + + // Do a final cleanup + wg.Wait() + i.cleanup() +} + +// dispense is used to dispense a plugin. +func (i *instanceManager) dispense() (plugin drivers.DriverPlugin, err error) { + i.pluginLock.Lock() + defer i.pluginLock.Unlock() + + // See if we already have a running instance + if i.plugin != nil && !i.plugin.Exited() { + return i.driver, nil + } + + var pluginInstance loader.PluginInstance + + if reattach, ok := i.fetchReattach(); ok { + // Reattach to existing plugin + pluginInstance, err = i.loader.Reattach(i.id.Name, i.id.PluginType, reattach) + } else { + // Get an instance of the plugin + pluginInstance, err = i.loader.Dispense(i.id.Name, i.id.PluginType, i.pluginConfig, i.logger) + } + if err != nil { + // Retry as the error just indicates the singleton has exited + if err == singleton.SingletonPluginExited { + pluginInstance, err = i.loader.Dispense(i.id.Name, i.id.PluginType, i.pluginConfig, i.logger) + } + + // If we still have an error there is a real problem + if err != nil { + return nil, fmt.Errorf("failed to start plugin: %v", err) + } + } + + // Convert to a driver plugin + driver, ok := pluginInstance.Plugin().(drivers.DriverPlugin) + if !ok { + pluginInstance.Kill() + return nil, fmt.Errorf("plugin loaded does not implement the driver interface") + } + + // Store the plugin and driver + i.plugin = pluginInstance + i.driver = driver + + // Store the reattach config + if c, ok := pluginInstance.ReattachConfig(); ok { + if err := i.storeReattach(c); err != nil { + i.logger.Error("error storing driver plugin reattach config", "error", err) + } + } + + return driver, nil +} + +// cleanup shutsdown the plugin +func (i *instanceManager) cleanup() { + i.shutdownLock.Lock() + i.pluginLock.Lock() + defer i.pluginLock.Unlock() + defer i.shutdownLock.Unlock() + + if i.plugin != nil && !i.plugin.Exited() { + i.plugin.Kill() + if err := i.storeReattach(nil); err != nil { + i.logger.Warn("error clearing plugin reattach config from state store", "error", err) + } + } +} + +// dispenseFingerprintCh dispenses a driver and makes a Fingerprint RPC call +// to the driver. The fingerprint chan is returned along with the cancel func +// for the context used in the RPC. This cancel func should always be called +// when the caller is finished with the channel. +func (i *instanceManager) dispenseFingerprintCh() (<-chan *drivers.Fingerprint, context.CancelFunc, error) { + driver, err := i.dispense() + if err != nil { + return nil, nil, err + } + + ctx, cancel := context.WithCancel(i.ctx) + fingerCh, err := driver.Fingerprint(ctx) + if err != nil { + cancel() + return nil, nil, err + } + + return fingerCh, cancel, nil +} + +// fingerprint is the main loop for fingerprinting. +func (i *instanceManager) fingerprint() { + fpChan, cancel, err := i.dispenseFingerprintCh() + if err != nil { + i.logger.Error("failed to dispense driver plugin", "error", err) + } + + // backoff and retry used if the RPC is closed by the other end + var backoff time.Duration + var retry int + for { + if backoff > 0 { + select { + case <-time.After(backoff): + case <-i.ctx.Done(): + cancel() + return + } + } + + select { + case <-i.ctx.Done(): + cancel() + return + case fp, ok := <-fpChan: + if ok { + if fp.Err == nil { + i.handleFingerprint(fp) + } else { + i.logger.Warn("received fingerprint error from driver", "error", fp.Err) + i.handleFingerprintError() + } + continue + } + + // if the channel is closed attempt to open a new one + newFpChan, newCancel, err := i.dispenseFingerprintCh() + if err != nil { + i.logger.Warn("error fingerprinting driver", "error", err, "retry", retry) + i.handleFingerprintError() + + // Calculate the new backoff + backoff = (1 << (2 * uint64(retry))) * driverFPBackoffBaseline + if backoff > driverFPBackoffLimit { + backoff = driverFPBackoffLimit + } + // Increment retry counter + retry++ + continue + } + cancel() + fpChan = newFpChan + cancel = newCancel + + // Reset backoff + backoff = 0 + retry = 0 + } + } +} + +// handleFingerprintError is called when an error occurred while fingerprinting +// and will set the driver to unhealthy +func (i *instanceManager) handleFingerprintError() { + di := &structs.DriverInfo{ + Healthy: false, + HealthDescription: "failed to fingerprint driver", + UpdateTime: time.Now(), + } + i.updateNodeFromDriver(i.id.Name, di) +} + +// handleFingerprint updates the node with the current fingerprint status +func (i *instanceManager) handleFingerprint(fp *drivers.Fingerprint) { + attrs := make(map[string]string, len(fp.Attributes)) + for key, attr := range fp.Attributes { + attrs[key] = attr.GoString() + } + di := &structs.DriverInfo{ + Attributes: attrs, + Detected: fp.Health != drivers.HealthStateUndetected, + Healthy: fp.Health == drivers.HealthStateHealthy, + HealthDescription: fp.HealthDescription, + UpdateTime: time.Now(), + } + i.updateNodeFromDriver(i.id.Name, di) + + // log detected/undetected state changes after the initial fingerprint + i.lastHealthStateMu.Lock() + if i.hasFingerprinted { + if i.lastHealthState != fp.Health { + i.logger.Info("driver health state has changed", "previous", i.lastHealthState, "current", fp.Health, "description", fp.HealthDescription) + } + } + i.lastHealthState = fp.Health + i.lastHealthStateMu.Unlock() + + // if this is the first fingerprint, mark that we have received it + if !i.hasFingerprinted { + i.logger.Trace("initial driver fingerprint", "fingerprint", fp) + close(i.firstFingerprintCh) + i.hasFingerprinted = true + } +} + +// getLastHealth returns the most recent HealthState from fingerprinting +func (i *instanceManager) getLastHealth() drivers.HealthState { + i.lastHealthStateMu.Lock() + defer i.lastHealthStateMu.Unlock() + return i.lastHealthState +} + +// dispenseTaskEventsCh dispenses a driver plugin and makes a TaskEvents RPC. +// The TaskEvent chan and cancel func for the RPC is return. The cancel func must +// be called by the caller to properly cleanup the context +func (i *instanceManager) dispenseTaskEventsCh() (<-chan *drivers.TaskEvent, context.CancelFunc, error) { + driver, err := i.dispense() + if err != nil { + return nil, nil, err + } + + ctx, cancel := context.WithCancel(i.ctx) + eventsCh, err := driver.TaskEvents(ctx) + if err != nil { + cancel() + return nil, nil, err + } + + return eventsCh, cancel, nil +} + +// handleEvents is the main loop that receives task events from the driver +func (i *instanceManager) handleEvents() { + eventsCh, cancel, err := i.dispenseTaskEventsCh() + if err != nil { + i.logger.Error("failed to dispense driver", "error", err) + } + + var backoff time.Duration + var retry int + for { + if backoff > 0 { + select { + case <-time.After(backoff): + case <-i.ctx.Done(): + cancel() + return + } + } + + select { + case <-i.ctx.Done(): + cancel() + return + case ev, ok := <-eventsCh: + if ok { + i.handleEvent(ev) + continue + } + + // if the channel is closed attempt to open a new one + newEventsChan, newCancel, err := i.dispenseTaskEventsCh() + if err != nil { + i.logger.Warn("failed to receive task events, retrying", "error", err, "retry", retry) + + // Calculate the new backoff + backoff = (1 << (2 * uint64(retry))) * driverFPBackoffBaseline + if backoff > driverFPBackoffLimit { + backoff = driverFPBackoffLimit + } + retry++ + continue + } + cancel() + eventsCh = newEventsChan + cancel = newCancel + + // Reset backoff + backoff = 0 + retry = 0 + } + } +} + +// handleEvent looks up the event handler(s) for the event and runs them +func (i *instanceManager) handleEvent(ev *drivers.TaskEvent) { + if handler := i.eventHandlerFactory(ev.AllocID, ev.TaskName); handler != nil { + i.logger.Trace("task event received", "event", ev) + handler(ev) + return + } + + i.logger.Warn("no handler registered for event", "event", ev) +} diff --git a/client/pluginmanager/drivermanager/manager.go b/client/pluginmanager/drivermanager/manager.go new file mode 100644 index 000000000..76efd50da --- /dev/null +++ b/client/pluginmanager/drivermanager/manager.go @@ -0,0 +1,377 @@ +package drivermanager + +import ( + "context" + "fmt" + "sync" + + log "github.com/hashicorp/go-hclog" + plugin "github.com/hashicorp/go-plugin" + "github.com/hashicorp/nomad/client/pluginmanager" + "github.com/hashicorp/nomad/client/pluginmanager/drivermanager/state" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/plugins/base" + "github.com/hashicorp/nomad/plugins/drivers" + "github.com/hashicorp/nomad/plugins/shared" + "github.com/hashicorp/nomad/plugins/shared/loader" +) + +// ErrDriverNotFound is returned during Dispense when the requested driver +// plugin is not found in the plugin catalog +var ErrDriverNotFound = fmt.Errorf("driver not found") + +// Manager is the interface used to manage driver plugins +type Manager interface { + pluginmanager.PluginManager + + // Dispense returns a drivers.DriverPlugin for the given driver plugin name + // handling reattaching to an existing driver if available + Dispense(driver string) (drivers.DriverPlugin, error) +} + +// EventHandler is a callback to be called for a task. +// The handler should not block execution. +type EventHandler func(*drivers.TaskEvent) + +// TaskEventHandlerFactory returns an event handler for a given allocID/task name +type TaskEventHandlerFactory func(allocID, taskName string) EventHandler + +// StateStorage is used to persist the driver managers state across +// agent restarts. +type StateStorage interface { + // GetDevicePluginState is used to retrieve the device manager's plugin + // state. + GetDriverPluginState() (*state.PluginState, error) + + // PutDevicePluginState is used to store the device manager's plugin + // state. + PutDriverPluginState(state *state.PluginState) error +} + +// UpdateNodeDriverInfoFn is the callback used to update the node from +// fingerprinting +type UpdateNodeDriverInfoFn func(string, *structs.DriverInfo) + +// StorePluginReattachFn is used to store plugin reattachment configurations. +type StorePluginReattachFn func(*plugin.ReattachConfig) error + +// FetchPluginReattachFn is used to retrieve the stored plugin reattachment +// configuration. +type FetchPluginReattachFn func() (*plugin.ReattachConfig, bool) + +// Config is used to configure a driver manager +type Config struct { + // Logger is the logger used by the device manager + Logger log.Logger + + // Loader is the plugin loader + Loader loader.PluginCatalog + + // PluginConfig is the config passed to the launched plugins + PluginConfig *base.AgentConfig + + // Updater is used to update the node when driver information changes + Updater UpdateNodeDriverInfoFn + + // EventHandlerFactory is used to retrieve a task event handler + EventHandlerFactory TaskEventHandlerFactory + + // State is used to manage the device managers state + State StateStorage + + // AllowedDrivers if set will only start driver plugins for the given + // drivers + AllowedDrivers map[string]struct{} + + // BlockedDrivers if set will not allow the given driver plugins to start + BlockedDrivers map[string]struct{} +} + +// manager is used to manage a set of driver plugins +type manager struct { + // logger is the logger used by the device manager + logger log.Logger + + // state is used to manage the device managers state + state StateStorage + + // ctx is used to shutdown the device manager + ctx context.Context + cancel context.CancelFunc + + // loader is the plugin loader + loader loader.PluginCatalog + + // pluginConfig is the config passed to the launched plugins + pluginConfig *base.AgentConfig + + // updater is used to update the node when device information changes + updater UpdateNodeDriverInfoFn + + // eventHandlerFactory is passed to the instance managers and used to forward + // task events + eventHandlerFactory TaskEventHandlerFactory + + // instances is the list of managed devices, access is serialized by instanceMu + instances map[string]*instanceManager + instancesMu sync.RWMutex + + // reattachConfigs stores the plugin reattach configs + reattachConfigs map[loader.PluginID]*shared.ReattachConfig + reattachConfigLock sync.Mutex + + // allows/block lists + allowedDrivers map[string]struct{} + blockedDrivers map[string]struct{} + + // readyCh is ticked once at the end of Run() + readyCh chan struct{} +} + +// New returns a new driver manager +func New(c *Config) *manager { + ctx, cancel := context.WithCancel(context.Background()) + return &manager{ + logger: c.Logger.Named("driver_mgr"), + state: c.State, + ctx: ctx, + cancel: cancel, + loader: c.Loader, + pluginConfig: c.PluginConfig, + updater: c.Updater, + eventHandlerFactory: c.EventHandlerFactory, + instances: make(map[string]*instanceManager), + reattachConfigs: make(map[loader.PluginID]*shared.ReattachConfig), + allowedDrivers: c.AllowedDrivers, + blockedDrivers: c.BlockedDrivers, + readyCh: make(chan struct{}), + } +} + +// PluginType returns the type of plugin this mananger mananges +func (*manager) PluginType() string { return base.PluginTypeDriver } + +// Run starts the mananger, initializes driver plugins and blocks until Shutdown +// is called. +func (m *manager) Run() { + // Load any previous plugin reattach configuration + if err := m.loadReattachConfigs(); err != nil { + m.logger.Warn("unable to load driver plugin reattach configs, a driver process may have been leaked", + "error", err) + } + + // Get driver plugins + driversPlugins := m.loader.Catalog()[base.PluginTypeDriver] + if len(driversPlugins) == 0 { + m.logger.Debug("exiting since there are no driver plugins") + m.cancel() + return + } + + var skippedDrivers []string + for _, d := range driversPlugins { + id := loader.PluginInfoID(d) + if m.isDriverBlocked(id.Name) { + skippedDrivers = append(skippedDrivers, id.Name) + continue + } + + storeFn := func(c *plugin.ReattachConfig) error { + return m.storePluginReattachConfig(id, c) + } + fetchFn := func() (*plugin.ReattachConfig, bool) { + return m.fetchPluginReattachConfig(id) + } + + instance := newInstanceManager(&instanceManagerConfig{ + Logger: m.logger, + Ctx: m.ctx, + Loader: m.loader, + StoreReattach: storeFn, + FetchReattach: fetchFn, + PluginConfig: m.pluginConfig, + ID: &id, + UpdateNodeFromDriver: m.updater, + EventHandlerFactory: m.eventHandlerFactory, + }) + + m.instancesMu.Lock() + m.instances[id.Name] = instance + m.instancesMu.Unlock() + } + + if len(skippedDrivers) > 0 { + m.logger.Debug("drivers skipped due to allow/block list", "skipped_drivers", skippedDrivers) + } + + // signal ready + close(m.readyCh) +} + +// Shutdown cleans up all the plugins +func (m *manager) Shutdown() { + // Cancel the context to stop any requests + m.cancel() + + m.instancesMu.RLock() + defer m.instancesMu.RUnlock() + + // Go through and shut everything down + for _, i := range m.instances { + i.cleanup() + } +} + +func (m *manager) WaitForFirstFingerprint(ctx context.Context) <-chan struct{} { + ctx, cancel := context.WithCancel(ctx) + go m.waitForFirstFingerprint(ctx, cancel) + return ctx.Done() +} + +func (m *manager) waitForFirstFingerprint(ctx context.Context, cancel context.CancelFunc) { + defer cancel() + // We don't want to start initial fingerprint wait until Run loop has + // finished + select { + case <-m.readyCh: + case <-ctx.Done(): + // parent context canceled or timedout + return + case <-m.ctx.Done(): + // shutdown called + return + } + + var availDrivers []string + var wg sync.WaitGroup + + // loop through instances and wait for each to finish initial fingerprint + m.instancesMu.RLock() + for n, i := range m.instances { + wg.Add(1) + go func(name string, instance *instanceManager) { + defer wg.Done() + instance.WaitForFirstFingerprint(ctx) + if instance.getLastHealth() != drivers.HealthStateUndetected { + availDrivers = append(availDrivers, name) + } + }(n, i) + } + m.instancesMu.RUnlock() + wg.Wait() + m.logger.Debug("detected drivers", "drivers", availDrivers) +} + +func (m *manager) loadReattachConfigs() error { + m.reattachConfigLock.Lock() + defer m.reattachConfigLock.Unlock() + + s, err := m.state.GetDriverPluginState() + if err != nil { + return err + } + + if s != nil { + for name, c := range s.ReattachConfigs { + if m.isDriverBlocked(name) { + m.logger.Warn("reattach config for driver plugin found but driver is blocked due to allow/block list, killing plugin", + "driver", name) + m.shutdownBlockedDriver(name, c) + continue + } + + id := loader.PluginID{ + PluginType: base.PluginTypeDriver, + Name: name, + } + + m.reattachConfigs[id] = c + } + } + return nil +} + +// shutdownBlockedDriver is used to forcefully shutdown a running driver plugin +// when it has been blocked due to allow/block lists +func (m *manager) shutdownBlockedDriver(name string, reattach *shared.ReattachConfig) { + c, err := shared.ReattachConfigToGoPlugin(reattach) + if err != nil { + m.logger.Warn("failed to reattach and kill blocked driver plugin", + "driver", name, "error", err) + return + + } + pluginInstance, err := m.loader.Reattach(name, base.PluginTypeDriver, c) + if err != nil { + m.logger.Warn("failed to reattach and kill blocked driver plugin", + "driver", name, "error", err) + return + } + + if !pluginInstance.Exited() { + pluginInstance.Kill() + } +} + +// storePluginReattachConfig is used as a callback to the instance managers and +// persists thhe plugin reattach configurations. +func (m *manager) storePluginReattachConfig(id loader.PluginID, c *plugin.ReattachConfig) error { + m.reattachConfigLock.Lock() + defer m.reattachConfigLock.Unlock() + + // Store the new reattach config + m.reattachConfigs[id] = shared.ReattachConfigFromGoPlugin(c) + + // Persist the state + s := &state.PluginState{ + ReattachConfigs: make(map[string]*shared.ReattachConfig, len(m.reattachConfigs)), + } + + for id, c := range m.reattachConfigs { + s.ReattachConfigs[id.Name] = c + } + + return m.state.PutDriverPluginState(s) +} + +// fetchPluginReattachConfig is used as a callback to the instance managers and +// retrieves the plugin reattach config. If it has not been stored it will +// return nil +func (m *manager) fetchPluginReattachConfig(id loader.PluginID) (*plugin.ReattachConfig, bool) { + m.reattachConfigLock.Lock() + defer m.reattachConfigLock.Unlock() + + if cfg, ok := m.reattachConfigs[id]; ok { + c, err := shared.ReattachConfigToGoPlugin(cfg) + if err != nil { + m.logger.Warn("failed to read plugin reattach config", "config", cfg, "error", err) + delete(m.reattachConfigs, id) + return nil, false + } + return c, true + } + return nil, false +} + +func (m *manager) Dispense(d string) (drivers.DriverPlugin, error) { + m.instancesMu.RLock() + defer m.instancesMu.RUnlock() + if instance, ok := m.instances[d]; ok { + return instance.dispense() + } + + return nil, ErrDriverNotFound +} + +func (m *manager) isDriverBlocked(name string) bool { + // Block drivers that are not in the allowed list if it is set. + if _, ok := m.allowedDrivers[name]; len(m.allowedDrivers) > 0 && !ok { + return true + } + + // Block drivers that are in the blocked list + if _, ok := m.blockedDrivers[name]; ok { + return true + } + return false +} diff --git a/client/pluginmanager/drivermanager/manager_test.go b/client/pluginmanager/drivermanager/manager_test.go new file mode 100644 index 000000000..3e8de7014 --- /dev/null +++ b/client/pluginmanager/drivermanager/manager_test.go @@ -0,0 +1,297 @@ +package drivermanager + +import ( + "context" + "fmt" + "sync" + "testing" + "time" + + log "github.com/hashicorp/go-hclog" + plugin "github.com/hashicorp/go-plugin" + "github.com/hashicorp/nomad/client/pluginmanager" + "github.com/hashicorp/nomad/client/state" + "github.com/hashicorp/nomad/helper/testlog" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/plugins/base" + "github.com/hashicorp/nomad/plugins/drivers" + dtu "github.com/hashicorp/nomad/plugins/drivers/testutils" + "github.com/hashicorp/nomad/plugins/shared/loader" + "github.com/hashicorp/nomad/testutil" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +var _ Manager = (*manager)(nil) +var _ pluginmanager.PluginManager = (*manager)(nil) + +func testSetup(t *testing.T) (chan *drivers.Fingerprint, chan *drivers.TaskEvent, *manager) { + fpChan := make(chan *drivers.Fingerprint) + evChan := make(chan *drivers.TaskEvent) + drv := mockDriver(fpChan, evChan) + cat := mockCatalog(map[string]drivers.DriverPlugin{"mock": drv}) + cfg := &Config{ + Logger: testlog.HCLogger(t), + Loader: cat, + PluginConfig: &base.AgentConfig{}, + Updater: noopUpdater, + EventHandlerFactory: noopEventHandlerFactory, + State: state.NoopDB{}, + AllowedDrivers: make(map[string]struct{}), + BlockedDrivers: make(map[string]struct{}), + } + + mgr := New(cfg) + return fpChan, evChan, mgr +} + +func mockDriver(fpChan chan *drivers.Fingerprint, evChan chan *drivers.TaskEvent) drivers.DriverPlugin { + return &dtu.MockDriver{ + FingerprintF: func(ctx context.Context) (<-chan *drivers.Fingerprint, error) { + return fpChan, nil + }, + TaskEventsF: func(ctx context.Context) (<-chan *drivers.TaskEvent, error) { + return evChan, nil + }, + } +} + +func mockCatalog(drivers map[string]drivers.DriverPlugin) *loader.MockCatalog { + cat := map[string][]*base.PluginInfoResponse{ + base.PluginTypeDriver: {}, + } + for d := range drivers { + cat[base.PluginTypeDriver] = append(cat[base.PluginTypeDriver], &base.PluginInfoResponse{ + Name: d, + Type: base.PluginTypeDriver, + }) + } + + return &loader.MockCatalog{ + DispenseF: func(name, pluginType string, cfg *base.AgentConfig, logger log.Logger) (loader.PluginInstance, error) { + d, ok := drivers[name] + if !ok { + return nil, fmt.Errorf("driver not found") + } + return loader.MockBasicExternalPlugin(d, "0.1.0"), nil + }, + ReattachF: func(name, pluginType string, config *plugin.ReattachConfig) (loader.PluginInstance, error) { + d, ok := drivers[name] + if !ok { + return nil, fmt.Errorf("driver not found") + } + return loader.MockBasicExternalPlugin(d, "0.1.0"), nil + }, + CatalogF: func() map[string][]*base.PluginInfoResponse { + return cat + }, + } +} + +func mockTaskEvent(taskID string) *drivers.TaskEvent { + return &drivers.TaskEvent{ + TaskID: taskID, + Timestamp: time.Now(), + Annotations: map[string]string{}, + Message: "event from " + taskID, + } +} + +func noopUpdater(string, *structs.DriverInfo) {} +func noopEventHandlerFactory(string, string) EventHandler { return nil } + +func TestMananger_Fingerprint(t *testing.T) { + t.Parallel() + require := require.New(t) + fpChan, _, mgr := testSetup(t) + var infos []*structs.DriverInfo + mgr.updater = func(d string, i *structs.DriverInfo) { + infos = append(infos, i) + } + go mgr.Run() + defer mgr.Shutdown() + fpChan <- &drivers.Fingerprint{Health: drivers.HealthStateHealthy} + testutil.WaitForResult(func() (bool, error) { + mgr.instancesMu.Lock() + defer mgr.instancesMu.Unlock() + return len(mgr.instances) == 1, fmt.Errorf("mananger should have registered 1 instance") + }, func(err error) { + require.NoError(err) + }) + + testutil.WaitForResult(func() (bool, error) { + mgr.instancesMu.Lock() + defer mgr.instancesMu.Unlock() + if mgr.instances["mock"].getLastHealth() != drivers.HealthStateHealthy { + return false, fmt.Errorf("mock instance should be healthy") + } + return true, nil + }, func(err error) { + require.NoError(err) + }) + + fpChan <- &drivers.Fingerprint{ + Health: drivers.HealthStateUnhealthy, + } + testutil.WaitForResult(func() (bool, error) { + mgr.instancesMu.Lock() + defer mgr.instancesMu.Unlock() + if mgr.instances["mock"].getLastHealth() == drivers.HealthStateHealthy { + return false, fmt.Errorf("mock instance should be unhealthy") + } + return true, nil + }, func(err error) { + require.NoError(err) + }) + + fpChan <- &drivers.Fingerprint{ + Health: drivers.HealthStateUndetected, + } + testutil.WaitForResult(func() (bool, error) { + mgr.instancesMu.Lock() + defer mgr.instancesMu.Unlock() + if mgr.instances["mock"].getLastHealth() != drivers.HealthStateUndetected { + return false, fmt.Errorf("mock instance should be undetected") + } + return true, nil + }, func(err error) { + require.NoError(err) + }) + + require.Len(infos, 3) + require.True(infos[0].Healthy) + require.True(infos[0].Detected) + require.False(infos[1].Healthy) + require.True(infos[1].Detected) + require.False(infos[2].Healthy) + require.False(infos[2].Detected) +} + +func TestMananger_TaskEvents(t *testing.T) { + t.Parallel() + require := require.New(t) + fpChan, evChan, mgr := testSetup(t) + go mgr.Run() + defer mgr.Shutdown() + fpChan <- &drivers.Fingerprint{Health: drivers.HealthStateHealthy} + testutil.WaitForResult(func() (bool, error) { + mgr.instancesMu.Lock() + defer mgr.instancesMu.Unlock() + return len(mgr.instances) == 1, fmt.Errorf("mananger should have registered 1 instance") + }, func(err error) { + require.NoError(err) + }) + + event1 := mockTaskEvent("abc1") + var wg sync.WaitGroup + wg.Add(1) + mgr.instancesMu.Lock() + mgr.instances["mock"].eventHandlerFactory = func(string, string) EventHandler { + return func(ev *drivers.TaskEvent) { + defer wg.Done() + assert.Exactly(t, event1, ev) + } + } + mgr.instancesMu.Unlock() + + evChan <- event1 + wg.Wait() +} + +func TestManager_Run_AllowedDrivers(t *testing.T) { + t.Parallel() + require := require.New(t) + fpChan, _, mgr := testSetup(t) + mgr.allowedDrivers = map[string]struct{}{"foo": {}} + go mgr.Run() + defer mgr.Shutdown() + select { + case fpChan <- &drivers.Fingerprint{Health: drivers.HealthStateHealthy}: + default: + } + testutil.AssertUntil(200*time.Millisecond, func() (bool, error) { + mgr.instancesMu.Lock() + defer mgr.instancesMu.Unlock() + return len(mgr.instances) == 0, fmt.Errorf("mananger should have no registered instances") + }, func(err error) { + require.NoError(err) + }) +} + +func TestManager_Run_BlockedDrivers(t *testing.T) { + t.Parallel() + require := require.New(t) + fpChan, _, mgr := testSetup(t) + mgr.blockedDrivers = map[string]struct{}{"mock": {}} + go mgr.Run() + defer mgr.Shutdown() + select { + case fpChan <- &drivers.Fingerprint{Health: drivers.HealthStateHealthy}: + default: + } + testutil.AssertUntil(200*time.Millisecond, func() (bool, error) { + mgr.instancesMu.Lock() + defer mgr.instancesMu.Unlock() + return len(mgr.instances) == 0, fmt.Errorf("mananger should have no registered instances") + }, func(err error) { + require.NoError(err) + }) +} + +func TestManager_Run_AllowedBlockedDrivers_Combined(t *testing.T) { + t.Parallel() + require := require.New(t) + drvs := map[string]drivers.DriverPlugin{} + fpChs := map[string]chan *drivers.Fingerprint{} + names := []string{"mock1", "mock2", "mock3", "mock4", "mock5"} + for _, d := range names { + ch := make(chan *drivers.Fingerprint) + drv := mockDriver(ch, nil) + drvs[d] = drv + fpChs[d] = ch + } + cat := mockCatalog(drvs) + cfg := &Config{ + Logger: testlog.HCLogger(t), + Loader: cat, + PluginConfig: nil, + Updater: noopUpdater, + State: state.NoopDB{}, + AllowedDrivers: map[string]struct{}{ + "mock2": {}, + "mock3": {}, + "mock4": {}, + "foo": {}, + }, + BlockedDrivers: map[string]struct{}{ + "mock2": {}, + "mock4": {}, + "bar": {}, + }, + } + mgr := New(cfg) + + go mgr.Run() + defer mgr.Shutdown() + for _, d := range names { + go func(drv string) { + select { + case fpChs[drv] <- &drivers.Fingerprint{Health: drivers.HealthStateHealthy}: + case <-time.After(200 * time.Millisecond): + } + }(d) + } + + testutil.AssertUntil(250*time.Millisecond, func() (bool, error) { + mgr.instancesMu.Lock() + defer mgr.instancesMu.Unlock() + return len(mgr.instances) < 2, fmt.Errorf("mananger should have 1 registered instance, %v", len(mgr.instances)) + }, func(err error) { + require.NoError(err) + }) + mgr.instancesMu.Lock() + require.Len(mgr.instances, 1) + _, ok := mgr.instances["mock3"] + mgr.instancesMu.Unlock() + require.True(ok) +} diff --git a/client/pluginmanager/drivermanager/state/state.go b/client/pluginmanager/drivermanager/state/state.go new file mode 100644 index 000000000..53be7616e --- /dev/null +++ b/client/pluginmanager/drivermanager/state/state.go @@ -0,0 +1,11 @@ +package state + +import "github.com/hashicorp/nomad/plugins/shared" + +// PluginState is used to store the driver managers state across restarts of the +// agent +type PluginState struct { + // ReattachConfigs are the set of reattach configs for plugin's launched by + // the driver manager + ReattachConfigs map[string]*shared.ReattachConfig +} diff --git a/client/pluginmanager/drivermanager/testing.go b/client/pluginmanager/drivermanager/testing.go new file mode 100644 index 000000000..e26af558f --- /dev/null +++ b/client/pluginmanager/drivermanager/testing.go @@ -0,0 +1,51 @@ +// +build !release + +package drivermanager + +import ( + "fmt" + "testing" + + log "github.com/hashicorp/go-hclog" + "github.com/hashicorp/nomad/helper/testlog" + "github.com/hashicorp/nomad/plugins/base" + "github.com/hashicorp/nomad/plugins/drivers" + "github.com/hashicorp/nomad/plugins/shared/catalog" + "github.com/hashicorp/nomad/plugins/shared/loader" + "github.com/hashicorp/nomad/plugins/shared/singleton" +) + +type testManager struct { + logger log.Logger + loader loader.PluginCatalog +} + +func TestDriverManager(t *testing.T) Manager { + logger := testlog.HCLogger(t).Named("driver_mgr") + pluginLoader := catalog.TestPluginLoader(t) + return &testManager{ + logger: logger, + loader: singleton.NewSingletonLoader(logger, pluginLoader), + } +} + +func (m *testManager) Run() {} +func (m *testManager) Shutdown() {} +func (m *testManager) PluginType() string { return base.PluginTypeDriver } + +func (m *testManager) Dispense(driver string) (drivers.DriverPlugin, error) { + instance, err := m.loader.Dispense(driver, base.PluginTypeDriver, nil, m.logger) + if err != nil { + return nil, err + } + + d, ok := instance.Plugin().(drivers.DriverPlugin) + if !ok { + return nil, fmt.Errorf("plugin does not implement DriverPlugin interface") + } + + return d, nil +} + +func (m *testManager) RegisterEventHandler(driver, taskID string, handler EventHandler) {} +func (m *testManager) DeregisterEventHandler(driver, taskID string) {} diff --git a/client/pluginmanager/group.go b/client/pluginmanager/group.go index db9c75674..a246334b1 100644 --- a/client/pluginmanager/group.go +++ b/client/pluginmanager/group.go @@ -1,6 +1,7 @@ package pluginmanager import ( + "context" "fmt" "sync" @@ -33,20 +34,57 @@ func New(logger log.Logger) *PluginGroup { // RegisterAndRun registers the manager and starts it in a separate goroutine func (m *PluginGroup) RegisterAndRun(manager PluginManager) error { m.mLock.Lock() + defer m.mLock.Unlock() if m.shutdown { return fmt.Errorf("plugin group already shutdown") } m.managers = append(m.managers, manager) - m.mLock.Unlock() - go func() { - m.logger.Info("starting plugin manager", "plugin-type", manager.PluginType()) - manager.Run() - m.logger.Info("plugin manager finished", "plugin-type", manager.PluginType()) - }() + m.logger.Info("starting plugin manager", "plugin-type", manager.PluginType()) + manager.Run() return nil } +// Ready returns a channel which will be closed once all plugin manangers are ready. +// A timeout for waiting on each manager is given +func (m *PluginGroup) WaitForFirstFingerprint(ctx context.Context) (<-chan struct{}, error) { + m.mLock.Lock() + defer m.mLock.Unlock() + if m.shutdown { + return nil, fmt.Errorf("plugin group already shutdown") + } + + var wg sync.WaitGroup + for i := range m.managers { + manager, ok := m.managers[i].(FingerprintingPluginManager) + if !ok { + continue + } + logger := m.logger.With("plugin-type", manager.PluginType()) + wg.Add(1) + go func() { + defer wg.Done() + logger.Debug("waiting on plugin mananger initial fingerprint") + select { + case <-manager.WaitForFirstFingerprint(ctx): + select { + case <-ctx.Done(): + logger.Warn("timeout waiting for plugin manager to be ready") + default: + logger.Debug("finished plugin mananger initial fingerprint") + } + } + }() + } + + ret := make(chan struct{}) + go func() { + wg.Wait() + close(ret) + }() + return ret, nil +} + // Shutdown shutsdown all registered PluginManagers in reverse order of how // they were started. func (m *PluginGroup) Shutdown() { @@ -55,6 +93,7 @@ func (m *PluginGroup) Shutdown() { for i := len(m.managers) - 1; i >= 0; i-- { m.logger.Info("shutting down plugin manager", "plugin-type", m.managers[i].PluginType()) m.managers[i].Shutdown() + m.logger.Info("plugin manager finished", "plugin-type", m.managers[i].PluginType()) } m.shutdown = true } diff --git a/client/pluginmanager/manager.go b/client/pluginmanager/manager.go index 0acce2ac9..f9d83a9ee 100644 --- a/client/pluginmanager/manager.go +++ b/client/pluginmanager/manager.go @@ -1,8 +1,10 @@ package pluginmanager +import "context" + // PluginManager orchestrates the lifecycle of a set of plugins type PluginManager interface { - // Run starts a plugin manager and must block until shutdown + // Run starts a plugin manager and should return early Run() // Shutdown should gracefully shutdown all plugins managed by the manager. @@ -12,3 +14,14 @@ type PluginManager interface { // PluginType is the type of plugin which the manager manages PluginType() string } + +// FingerprintingPluginManager is an interface that exposes fingerprinting +// coordination for plugin managers +type FingerprintingPluginManager interface { + PluginManager + + // WaitForFirstFingerprint returns a channel that is closed once all plugin + // instances managed by the plugin manager have fingerprinted once. A + // context can be passed which when done will also close the channel + WaitForFirstFingerprint(context.Context) <-chan struct{} +} diff --git a/client/state/db_test.go b/client/state/db_test.go index 9e6be8a14..c28078657 100644 --- a/client/state/db_test.go +++ b/client/state/db_test.go @@ -8,6 +8,7 @@ import ( trstate "github.com/hashicorp/nomad/client/allocrunner/taskrunner/state" dmstate "github.com/hashicorp/nomad/client/devicemanager/state" + driverstate "github.com/hashicorp/nomad/client/pluginmanager/drivermanager/state" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" "github.com/kr/pretty" @@ -216,3 +217,28 @@ func TestStateDB_DeviceManager(t *testing.T) { require.Equal(state, ps) }) } + +// TestStateDB_DriverManager asserts the behavior of device manager state related StateDB +// methods. +func TestStateDB_DriverManager(t *testing.T) { + t.Parallel() + + testDB(t, func(t *testing.T, db StateDB) { + require := require.New(t) + + // Getting nonexistent state should return nils + ps, err := db.GetDriverPluginState() + require.NoError(err) + require.Nil(ps) + + // Putting PluginState should work + state := &driverstate.PluginState{} + require.NoError(db.PutDriverPluginState(state)) + + // Getting should return the available state + ps, err = db.GetDriverPluginState() + require.NoError(err) + require.NotNil(ps) + require.Equal(state, ps) + }) +} diff --git a/client/state/interface.go b/client/state/interface.go index 5ccc3f0c3..7f2cac7a9 100644 --- a/client/state/interface.go +++ b/client/state/interface.go @@ -3,6 +3,7 @@ package state import ( "github.com/hashicorp/nomad/client/allocrunner/taskrunner/state" dmstate "github.com/hashicorp/nomad/client/devicemanager/state" + driverstate "github.com/hashicorp/nomad/client/pluginmanager/drivermanager/state" "github.com/hashicorp/nomad/nomad/structs" ) @@ -50,6 +51,14 @@ type StateDB interface { // state. PutDevicePluginState(state *dmstate.PluginState) error + // GetDriverPluginState is used to retrieve the driver manager's plugin + // state. + GetDriverPluginState() (*driverstate.PluginState, error) + + // PutDriverPluginState is used to store the driver manager's plugin + // state. + PutDriverPluginState(state *driverstate.PluginState) error + // Close the database. Unsafe for further use after calling regardless // of return value. Close() error diff --git a/client/state/memdb.go b/client/state/memdb.go index d16e6f232..0e01c01dc 100644 --- a/client/state/memdb.go +++ b/client/state/memdb.go @@ -5,6 +5,7 @@ import ( "github.com/hashicorp/nomad/client/allocrunner/taskrunner/state" dmstate "github.com/hashicorp/nomad/client/devicemanager/state" + driverstate "github.com/hashicorp/nomad/client/pluginmanager/drivermanager/state" "github.com/hashicorp/nomad/nomad/structs" ) @@ -21,6 +22,9 @@ type MemDB struct { // devicemanager -> plugin-state devManagerPs *dmstate.PluginState + // drivermanager -> plugin-state + driverManagerPs *driverstate.PluginState + mu sync.RWMutex } @@ -150,6 +154,19 @@ func (m *MemDB) GetDevicePluginState() (*dmstate.PluginState, error) { return m.devManagerPs, nil } +func (m *MemDB) GetDriverPluginState() (*driverstate.PluginState, error) { + m.mu.Lock() + defer m.mu.Unlock() + return m.driverManagerPs, nil +} + +func (m *MemDB) PutDriverPluginState(ps *driverstate.PluginState) error { + m.mu.Lock() + defer m.mu.Unlock() + m.driverManagerPs = ps + return nil +} + func (m *MemDB) Close() error { m.mu.Lock() defer m.mu.Unlock() diff --git a/client/state/noopdb.go b/client/state/noopdb.go index 1d4975de2..cb03a55f5 100644 --- a/client/state/noopdb.go +++ b/client/state/noopdb.go @@ -3,6 +3,7 @@ package state import ( "github.com/hashicorp/nomad/client/allocrunner/taskrunner/state" dmstate "github.com/hashicorp/nomad/client/devicemanager/state" + driverstate "github.com/hashicorp/nomad/client/pluginmanager/drivermanager/state" "github.com/hashicorp/nomad/nomad/structs" ) @@ -49,6 +50,14 @@ func (n NoopDB) GetDevicePluginState() (*dmstate.PluginState, error) { return nil, nil } +func (n NoopDB) PutDriverPluginState(ps *driverstate.PluginState) error { + return nil +} + +func (n NoopDB) GetDriverPluginState() (*driverstate.PluginState, error) { + return nil, nil +} + func (n NoopDB) Close() error { return nil } diff --git a/client/state/state_database.go b/client/state/state_database.go index ef499b502..2fa6d88b7 100644 --- a/client/state/state_database.go +++ b/client/state/state_database.go @@ -6,6 +6,7 @@ import ( trstate "github.com/hashicorp/nomad/client/allocrunner/taskrunner/state" dmstate "github.com/hashicorp/nomad/client/devicemanager/state" + driverstate "github.com/hashicorp/nomad/client/pluginmanager/drivermanager/state" "github.com/hashicorp/nomad/helper/boltdd" "github.com/hashicorp/nomad/nomad/structs" ) @@ -22,6 +23,9 @@ allocations/ (bucket) devicemanager/ |--> plugin-state -> *dmstate.PluginState + +drivermanager/ +|--> plugin-state -> *driverstate.PluginState */ var ( @@ -45,9 +49,13 @@ var ( // data devManagerBucket = []byte("devicemanager") - // devManagerPluginStateKey is the key serialized device manager - // plugin state is stored at - devManagerPluginStateKey = []byte("plugin_state") + // driverManagerBucket is the bucket name container all driver manager + // related data + driverManagerBucket = []byte("drivermanager") + + // managerPluginStateKey is the key by which plugin manager plugin state is + // stored at + managerPluginStateKey = []byte("plugin_state") ) // NewStateDBFunc creates a StateDB given a state directory. @@ -377,7 +385,7 @@ func (s *BoltStateDB) PutDevicePluginState(ps *dmstate.PluginState) error { return err } - return devBkt.Put(devManagerPluginStateKey, ps) + return devBkt.Put(managerPluginStateKey, ps) }) } @@ -395,7 +403,7 @@ func (s *BoltStateDB) GetDevicePluginState() (*dmstate.PluginState, error) { // Restore Plugin State if it exists ps = &dmstate.PluginState{} - if err := devBkt.Get(devManagerPluginStateKey, ps); err != nil { + if err := devBkt.Get(managerPluginStateKey, ps); err != nil { if !boltdd.IsErrNotFound(err) { return fmt.Errorf("failed to read device manager plugin state: %v", err) } @@ -413,3 +421,50 @@ func (s *BoltStateDB) GetDevicePluginState() (*dmstate.PluginState, error) { return ps, nil } + +// PutDriverPluginState stores the driver manager's plugin state or returns an +// error. +func (s *BoltStateDB) PutDriverPluginState(ps *driverstate.PluginState) error { + return s.db.Update(func(tx *boltdd.Tx) error { + // Retrieve the root driver manager bucket + driverBkt, err := tx.CreateBucketIfNotExists(driverManagerBucket) + if err != nil { + return err + } + + return driverBkt.Put(managerPluginStateKey, ps) + }) +} + +// GetDriverPluginState stores the driver manager's plugin state or returns an +// error. +func (s *BoltStateDB) GetDriverPluginState() (*driverstate.PluginState, error) { + var ps *driverstate.PluginState + + err := s.db.View(func(tx *boltdd.Tx) error { + driverBkt := tx.Bucket(driverManagerBucket) + if driverBkt == nil { + // No state, return + return nil + } + + // Restore Plugin State if it exists + ps = &driverstate.PluginState{} + if err := driverBkt.Get(managerPluginStateKey, ps); err != nil { + if !boltdd.IsErrNotFound(err) { + return fmt.Errorf("failed to read driver manager plugin state: %v", err) + } + + // Key not found, reset ps to nil + ps = nil + } + + return nil + }) + + if err != nil { + return nil, err + } + + return ps, nil +} diff --git a/command/agent/consul/int_test.go b/command/agent/consul/int_test.go index da6431ffc..816607e1e 100644 --- a/command/agent/consul/int_test.go +++ b/command/agent/consul/int_test.go @@ -15,14 +15,13 @@ import ( "github.com/hashicorp/nomad/client/allocrunner/taskrunner" "github.com/hashicorp/nomad/client/config" "github.com/hashicorp/nomad/client/devicemanager" + "github.com/hashicorp/nomad/client/pluginmanager/drivermanager" "github.com/hashicorp/nomad/client/state" "github.com/hashicorp/nomad/client/vaultclient" "github.com/hashicorp/nomad/command/agent/consul" "github.com/hashicorp/nomad/helper/testlog" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" - "github.com/hashicorp/nomad/plugins/shared/catalog" - "github.com/hashicorp/nomad/plugins/shared/singleton" "github.com/stretchr/testify/require" ) @@ -146,19 +145,18 @@ func TestConsul_Integration(t *testing.T) { }() // Build the config - pluginLoader := catalog.TestPluginLoader(t) config := &taskrunner.Config{ - Alloc: alloc, - ClientConfig: conf, - Consul: serviceClient, - Task: task, - TaskDir: taskDir, - Logger: logger, - Vault: vclient, - StateDB: state.NoopDB{}, - StateUpdater: logUpdate, - PluginSingletonLoader: singleton.NewSingletonLoader(logger, pluginLoader), - DeviceManager: devicemanager.NoopMockManager(), + Alloc: alloc, + ClientConfig: conf, + Consul: serviceClient, + Task: task, + TaskDir: taskDir, + Logger: logger, + Vault: vclient, + StateDB: state.NoopDB{}, + StateUpdater: logUpdate, + DeviceManager: devicemanager.NoopMockManager(), + DriverManager: drivermanager.TestDriverManager(t), } tr, err := taskrunner.NewTaskRunner(config) diff --git a/drivers/docker/driver.go b/drivers/docker/driver.go index d11acc422..d3c26d312 100644 --- a/drivers/docker/driver.go +++ b/drivers/docker/driver.go @@ -475,6 +475,8 @@ func (d *Driver) pullImage(task *drivers.TaskConfig, driverConfig *TaskConfig, c d.eventer.EmitEvent(&drivers.TaskEvent{ TaskID: task.ID, + AllocID: task.AllocID, + TaskName: task.Name, Timestamp: time.Now(), Message: "Downloading image", Annotations: map[string]string{ @@ -489,6 +491,8 @@ func (d *Driver) emitEventFunc(task *drivers.TaskConfig) LogEventFn { return func(msg string, annotations map[string]string) { d.eventer.EmitEvent(&drivers.TaskEvent{ TaskID: task.ID, + AllocID: task.AllocID, + TaskName: task.Name, Timestamp: time.Now(), Message: msg, Annotations: annotations, diff --git a/plugins/drivers/client.go b/plugins/drivers/client.go index d8a1415fb..edefec092 100644 --- a/plugins/drivers/client.go +++ b/plugins/drivers/client.go @@ -310,6 +310,8 @@ func (d *driverPluginClient) handleTaskEvents(ctx context.Context, ch chan *Task timestamp, _ := ptypes.Timestamp(ev.Timestamp) event := &TaskEvent{ TaskID: ev.TaskId, + AllocID: ev.AllocId, + TaskName: ev.TaskName, Annotations: ev.Annotations, Message: ev.Message, Timestamp: timestamp, diff --git a/plugins/drivers/driver.go b/plugins/drivers/driver.go index 25d60787e..e912c85b4 100644 --- a/plugins/drivers/driver.go +++ b/plugins/drivers/driver.go @@ -268,6 +268,8 @@ type TaskStatus struct { type TaskEvent struct { TaskID string + TaskName string + AllocID string Timestamp time.Time Message string Annotations map[string]string diff --git a/plugins/drivers/proto/driver.pb.go b/plugins/drivers/proto/driver.pb.go index cdba16ecc..37ce75247 100644 --- a/plugins/drivers/proto/driver.pb.go +++ b/plugins/drivers/proto/driver.pb.go @@ -50,7 +50,7 @@ func (x TaskState) String() string { return proto.EnumName(TaskState_name, int32(x)) } func (TaskState) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{0} + return fileDescriptor_driver_f8179d6379690e6b, []int{0} } type FingerprintResponse_HealthState int32 @@ -76,7 +76,7 @@ func (x FingerprintResponse_HealthState) String() string { return proto.EnumName(FingerprintResponse_HealthState_name, int32(x)) } func (FingerprintResponse_HealthState) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{5, 0} + return fileDescriptor_driver_f8179d6379690e6b, []int{5, 0} } type StartTaskResponse_Result int32 @@ -102,7 +102,7 @@ func (x StartTaskResponse_Result) String() string { return proto.EnumName(StartTaskResponse_Result_name, int32(x)) } func (StartTaskResponse_Result) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{9, 0} + return fileDescriptor_driver_f8179d6379690e6b, []int{9, 0} } type DriverCapabilities_FSIsolation int32 @@ -128,7 +128,7 @@ func (x DriverCapabilities_FSIsolation) String() string { return proto.EnumName(DriverCapabilities_FSIsolation_name, int32(x)) } func (DriverCapabilities_FSIsolation) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{25, 0} + return fileDescriptor_driver_f8179d6379690e6b, []int{25, 0} } type CPUUsage_Fields int32 @@ -163,7 +163,7 @@ func (x CPUUsage_Fields) String() string { return proto.EnumName(CPUUsage_Fields_name, int32(x)) } func (CPUUsage_Fields) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{43, 0} + return fileDescriptor_driver_f8179d6379690e6b, []int{43, 0} } type MemoryUsage_Fields int32 @@ -195,7 +195,7 @@ func (x MemoryUsage_Fields) String() string { return proto.EnumName(MemoryUsage_Fields_name, int32(x)) } func (MemoryUsage_Fields) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{44, 0} + return fileDescriptor_driver_f8179d6379690e6b, []int{44, 0} } type TaskConfigSchemaRequest struct { @@ -208,7 +208,7 @@ func (m *TaskConfigSchemaRequest) Reset() { *m = TaskConfigSchemaRequest func (m *TaskConfigSchemaRequest) String() string { return proto.CompactTextString(m) } func (*TaskConfigSchemaRequest) ProtoMessage() {} func (*TaskConfigSchemaRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{0} + return fileDescriptor_driver_f8179d6379690e6b, []int{0} } func (m *TaskConfigSchemaRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_TaskConfigSchemaRequest.Unmarshal(m, b) @@ -240,7 +240,7 @@ func (m *TaskConfigSchemaResponse) Reset() { *m = TaskConfigSchemaRespon func (m *TaskConfigSchemaResponse) String() string { return proto.CompactTextString(m) } func (*TaskConfigSchemaResponse) ProtoMessage() {} func (*TaskConfigSchemaResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{1} + return fileDescriptor_driver_f8179d6379690e6b, []int{1} } func (m *TaskConfigSchemaResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_TaskConfigSchemaResponse.Unmarshal(m, b) @@ -277,7 +277,7 @@ func (m *CapabilitiesRequest) Reset() { *m = CapabilitiesRequest{} } func (m *CapabilitiesRequest) String() string { return proto.CompactTextString(m) } func (*CapabilitiesRequest) ProtoMessage() {} func (*CapabilitiesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{2} + return fileDescriptor_driver_f8179d6379690e6b, []int{2} } func (m *CapabilitiesRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_CapabilitiesRequest.Unmarshal(m, b) @@ -312,7 +312,7 @@ func (m *CapabilitiesResponse) Reset() { *m = CapabilitiesResponse{} } func (m *CapabilitiesResponse) String() string { return proto.CompactTextString(m) } func (*CapabilitiesResponse) ProtoMessage() {} func (*CapabilitiesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{3} + return fileDescriptor_driver_f8179d6379690e6b, []int{3} } func (m *CapabilitiesResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_CapabilitiesResponse.Unmarshal(m, b) @@ -349,7 +349,7 @@ func (m *FingerprintRequest) Reset() { *m = FingerprintRequest{} } func (m *FingerprintRequest) String() string { return proto.CompactTextString(m) } func (*FingerprintRequest) ProtoMessage() {} func (*FingerprintRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{4} + return fileDescriptor_driver_f8179d6379690e6b, []int{4} } func (m *FingerprintRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_FingerprintRequest.Unmarshal(m, b) @@ -392,7 +392,7 @@ func (m *FingerprintResponse) Reset() { *m = FingerprintResponse{} } func (m *FingerprintResponse) String() string { return proto.CompactTextString(m) } func (*FingerprintResponse) ProtoMessage() {} func (*FingerprintResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{5} + return fileDescriptor_driver_f8179d6379690e6b, []int{5} } func (m *FingerprintResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_FingerprintResponse.Unmarshal(m, b) @@ -447,7 +447,7 @@ func (m *RecoverTaskRequest) Reset() { *m = RecoverTaskRequest{} } func (m *RecoverTaskRequest) String() string { return proto.CompactTextString(m) } func (*RecoverTaskRequest) ProtoMessage() {} func (*RecoverTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{6} + return fileDescriptor_driver_f8179d6379690e6b, []int{6} } func (m *RecoverTaskRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_RecoverTaskRequest.Unmarshal(m, b) @@ -491,7 +491,7 @@ func (m *RecoverTaskResponse) Reset() { *m = RecoverTaskResponse{} } func (m *RecoverTaskResponse) String() string { return proto.CompactTextString(m) } func (*RecoverTaskResponse) ProtoMessage() {} func (*RecoverTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{7} + return fileDescriptor_driver_f8179d6379690e6b, []int{7} } func (m *RecoverTaskResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_RecoverTaskResponse.Unmarshal(m, b) @@ -523,7 +523,7 @@ func (m *StartTaskRequest) Reset() { *m = StartTaskRequest{} } func (m *StartTaskRequest) String() string { return proto.CompactTextString(m) } func (*StartTaskRequest) ProtoMessage() {} func (*StartTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{8} + return fileDescriptor_driver_f8179d6379690e6b, []int{8} } func (m *StartTaskRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StartTaskRequest.Unmarshal(m, b) @@ -577,7 +577,7 @@ func (m *StartTaskResponse) Reset() { *m = StartTaskResponse{} } func (m *StartTaskResponse) String() string { return proto.CompactTextString(m) } func (*StartTaskResponse) ProtoMessage() {} func (*StartTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{9} + return fileDescriptor_driver_f8179d6379690e6b, []int{9} } func (m *StartTaskResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StartTaskResponse.Unmarshal(m, b) @@ -637,7 +637,7 @@ func (m *WaitTaskRequest) Reset() { *m = WaitTaskRequest{} } func (m *WaitTaskRequest) String() string { return proto.CompactTextString(m) } func (*WaitTaskRequest) ProtoMessage() {} func (*WaitTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{10} + return fileDescriptor_driver_f8179d6379690e6b, []int{10} } func (m *WaitTaskRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_WaitTaskRequest.Unmarshal(m, b) @@ -678,7 +678,7 @@ func (m *WaitTaskResponse) Reset() { *m = WaitTaskResponse{} } func (m *WaitTaskResponse) String() string { return proto.CompactTextString(m) } func (*WaitTaskResponse) ProtoMessage() {} func (*WaitTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{11} + return fileDescriptor_driver_f8179d6379690e6b, []int{11} } func (m *WaitTaskResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_WaitTaskResponse.Unmarshal(m, b) @@ -730,7 +730,7 @@ func (m *StopTaskRequest) Reset() { *m = StopTaskRequest{} } func (m *StopTaskRequest) String() string { return proto.CompactTextString(m) } func (*StopTaskRequest) ProtoMessage() {} func (*StopTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{12} + return fileDescriptor_driver_f8179d6379690e6b, []int{12} } func (m *StopTaskRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StopTaskRequest.Unmarshal(m, b) @@ -781,7 +781,7 @@ func (m *StopTaskResponse) Reset() { *m = StopTaskResponse{} } func (m *StopTaskResponse) String() string { return proto.CompactTextString(m) } func (*StopTaskResponse) ProtoMessage() {} func (*StopTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{13} + return fileDescriptor_driver_f8179d6379690e6b, []int{13} } func (m *StopTaskResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StopTaskResponse.Unmarshal(m, b) @@ -815,7 +815,7 @@ func (m *DestroyTaskRequest) Reset() { *m = DestroyTaskRequest{} } func (m *DestroyTaskRequest) String() string { return proto.CompactTextString(m) } func (*DestroyTaskRequest) ProtoMessage() {} func (*DestroyTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{14} + return fileDescriptor_driver_f8179d6379690e6b, []int{14} } func (m *DestroyTaskRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DestroyTaskRequest.Unmarshal(m, b) @@ -859,7 +859,7 @@ func (m *DestroyTaskResponse) Reset() { *m = DestroyTaskResponse{} } func (m *DestroyTaskResponse) String() string { return proto.CompactTextString(m) } func (*DestroyTaskResponse) ProtoMessage() {} func (*DestroyTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{15} + return fileDescriptor_driver_f8179d6379690e6b, []int{15} } func (m *DestroyTaskResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DestroyTaskResponse.Unmarshal(m, b) @@ -891,7 +891,7 @@ func (m *InspectTaskRequest) Reset() { *m = InspectTaskRequest{} } func (m *InspectTaskRequest) String() string { return proto.CompactTextString(m) } func (*InspectTaskRequest) ProtoMessage() {} func (*InspectTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{16} + return fileDescriptor_driver_f8179d6379690e6b, []int{16} } func (m *InspectTaskRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_InspectTaskRequest.Unmarshal(m, b) @@ -934,7 +934,7 @@ func (m *InspectTaskResponse) Reset() { *m = InspectTaskResponse{} } func (m *InspectTaskResponse) String() string { return proto.CompactTextString(m) } func (*InspectTaskResponse) ProtoMessage() {} func (*InspectTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{17} + return fileDescriptor_driver_f8179d6379690e6b, []int{17} } func (m *InspectTaskResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_InspectTaskResponse.Unmarshal(m, b) @@ -987,7 +987,7 @@ func (m *TaskStatsRequest) Reset() { *m = TaskStatsRequest{} } func (m *TaskStatsRequest) String() string { return proto.CompactTextString(m) } func (*TaskStatsRequest) ProtoMessage() {} func (*TaskStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{18} + return fileDescriptor_driver_f8179d6379690e6b, []int{18} } func (m *TaskStatsRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_TaskStatsRequest.Unmarshal(m, b) @@ -1026,7 +1026,7 @@ func (m *TaskStatsResponse) Reset() { *m = TaskStatsResponse{} } func (m *TaskStatsResponse) String() string { return proto.CompactTextString(m) } func (*TaskStatsResponse) ProtoMessage() {} func (*TaskStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{19} + return fileDescriptor_driver_f8179d6379690e6b, []int{19} } func (m *TaskStatsResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_TaskStatsResponse.Unmarshal(m, b) @@ -1063,7 +1063,7 @@ func (m *TaskEventsRequest) Reset() { *m = TaskEventsRequest{} } func (m *TaskEventsRequest) String() string { return proto.CompactTextString(m) } func (*TaskEventsRequest) ProtoMessage() {} func (*TaskEventsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{20} + return fileDescriptor_driver_f8179d6379690e6b, []int{20} } func (m *TaskEventsRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_TaskEventsRequest.Unmarshal(m, b) @@ -1097,7 +1097,7 @@ func (m *SignalTaskRequest) Reset() { *m = SignalTaskRequest{} } func (m *SignalTaskRequest) String() string { return proto.CompactTextString(m) } func (*SignalTaskRequest) ProtoMessage() {} func (*SignalTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{21} + return fileDescriptor_driver_f8179d6379690e6b, []int{21} } func (m *SignalTaskRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SignalTaskRequest.Unmarshal(m, b) @@ -1141,7 +1141,7 @@ func (m *SignalTaskResponse) Reset() { *m = SignalTaskResponse{} } func (m *SignalTaskResponse) String() string { return proto.CompactTextString(m) } func (*SignalTaskResponse) ProtoMessage() {} func (*SignalTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{22} + return fileDescriptor_driver_f8179d6379690e6b, []int{22} } func (m *SignalTaskResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SignalTaskResponse.Unmarshal(m, b) @@ -1178,7 +1178,7 @@ func (m *ExecTaskRequest) Reset() { *m = ExecTaskRequest{} } func (m *ExecTaskRequest) String() string { return proto.CompactTextString(m) } func (*ExecTaskRequest) ProtoMessage() {} func (*ExecTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{23} + return fileDescriptor_driver_f8179d6379690e6b, []int{23} } func (m *ExecTaskRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ExecTaskRequest.Unmarshal(m, b) @@ -1235,7 +1235,7 @@ func (m *ExecTaskResponse) Reset() { *m = ExecTaskResponse{} } func (m *ExecTaskResponse) String() string { return proto.CompactTextString(m) } func (*ExecTaskResponse) ProtoMessage() {} func (*ExecTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{24} + return fileDescriptor_driver_f8179d6379690e6b, []int{24} } func (m *ExecTaskResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ExecTaskResponse.Unmarshal(m, b) @@ -1294,7 +1294,7 @@ func (m *DriverCapabilities) Reset() { *m = DriverCapabilities{} } func (m *DriverCapabilities) String() string { return proto.CompactTextString(m) } func (*DriverCapabilities) ProtoMessage() {} func (*DriverCapabilities) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{25} + return fileDescriptor_driver_f8179d6379690e6b, []int{25} } func (m *DriverCapabilities) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DriverCapabilities.Unmarshal(m, b) @@ -1375,7 +1375,7 @@ func (m *TaskConfig) Reset() { *m = TaskConfig{} } func (m *TaskConfig) String() string { return proto.CompactTextString(m) } func (*TaskConfig) ProtoMessage() {} func (*TaskConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{26} + return fileDescriptor_driver_f8179d6379690e6b, []int{26} } func (m *TaskConfig) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_TaskConfig.Unmarshal(m, b) @@ -1507,7 +1507,7 @@ func (m *Resources) Reset() { *m = Resources{} } func (m *Resources) String() string { return proto.CompactTextString(m) } func (*Resources) ProtoMessage() {} func (*Resources) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{27} + return fileDescriptor_driver_f8179d6379690e6b, []int{27} } func (m *Resources) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Resources.Unmarshal(m, b) @@ -1554,7 +1554,7 @@ func (m *AllocatedTaskResources) Reset() { *m = AllocatedTaskResources{} func (m *AllocatedTaskResources) String() string { return proto.CompactTextString(m) } func (*AllocatedTaskResources) ProtoMessage() {} func (*AllocatedTaskResources) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{28} + return fileDescriptor_driver_f8179d6379690e6b, []int{28} } func (m *AllocatedTaskResources) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_AllocatedTaskResources.Unmarshal(m, b) @@ -1606,7 +1606,7 @@ func (m *AllocatedCpuResources) Reset() { *m = AllocatedCpuResources{} } func (m *AllocatedCpuResources) String() string { return proto.CompactTextString(m) } func (*AllocatedCpuResources) ProtoMessage() {} func (*AllocatedCpuResources) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{29} + return fileDescriptor_driver_f8179d6379690e6b, []int{29} } func (m *AllocatedCpuResources) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_AllocatedCpuResources.Unmarshal(m, b) @@ -1644,7 +1644,7 @@ func (m *AllocatedMemoryResources) Reset() { *m = AllocatedMemoryResourc func (m *AllocatedMemoryResources) String() string { return proto.CompactTextString(m) } func (*AllocatedMemoryResources) ProtoMessage() {} func (*AllocatedMemoryResources) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{30} + return fileDescriptor_driver_f8179d6379690e6b, []int{30} } func (m *AllocatedMemoryResources) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_AllocatedMemoryResources.Unmarshal(m, b) @@ -1687,7 +1687,7 @@ func (m *NetworkResource) Reset() { *m = NetworkResource{} } func (m *NetworkResource) String() string { return proto.CompactTextString(m) } func (*NetworkResource) ProtoMessage() {} func (*NetworkResource) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{31} + return fileDescriptor_driver_f8179d6379690e6b, []int{31} } func (m *NetworkResource) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_NetworkResource.Unmarshal(m, b) @@ -1761,7 +1761,7 @@ func (m *NetworkPort) Reset() { *m = NetworkPort{} } func (m *NetworkPort) String() string { return proto.CompactTextString(m) } func (*NetworkPort) ProtoMessage() {} func (*NetworkPort) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{32} + return fileDescriptor_driver_f8179d6379690e6b, []int{32} } func (m *NetworkPort) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_NetworkPort.Unmarshal(m, b) @@ -1821,7 +1821,7 @@ func (m *LinuxResources) Reset() { *m = LinuxResources{} } func (m *LinuxResources) String() string { return proto.CompactTextString(m) } func (*LinuxResources) ProtoMessage() {} func (*LinuxResources) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{33} + return fileDescriptor_driver_f8179d6379690e6b, []int{33} } func (m *LinuxResources) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_LinuxResources.Unmarshal(m, b) @@ -1913,7 +1913,7 @@ func (m *Mount) Reset() { *m = Mount{} } func (m *Mount) String() string { return proto.CompactTextString(m) } func (*Mount) ProtoMessage() {} func (*Mount) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{34} + return fileDescriptor_driver_f8179d6379690e6b, []int{34} } func (m *Mount) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Mount.Unmarshal(m, b) @@ -1976,7 +1976,7 @@ func (m *Device) Reset() { *m = Device{} } func (m *Device) String() string { return proto.CompactTextString(m) } func (*Device) ProtoMessage() {} func (*Device) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{35} + return fileDescriptor_driver_f8179d6379690e6b, []int{35} } func (m *Device) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Device.Unmarshal(m, b) @@ -2034,7 +2034,7 @@ func (m *TaskHandle) Reset() { *m = TaskHandle{} } func (m *TaskHandle) String() string { return proto.CompactTextString(m) } func (*TaskHandle) ProtoMessage() {} func (*TaskHandle) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{36} + return fileDescriptor_driver_f8179d6379690e6b, []int{36} } func (m *TaskHandle) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_TaskHandle.Unmarshal(m, b) @@ -2094,7 +2094,7 @@ func (m *NetworkOverride) Reset() { *m = NetworkOverride{} } func (m *NetworkOverride) String() string { return proto.CompactTextString(m) } func (*NetworkOverride) ProtoMessage() {} func (*NetworkOverride) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{37} + return fileDescriptor_driver_f8179d6379690e6b, []int{37} } func (m *NetworkOverride) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_NetworkOverride.Unmarshal(m, b) @@ -2152,7 +2152,7 @@ func (m *ExitResult) Reset() { *m = ExitResult{} } func (m *ExitResult) String() string { return proto.CompactTextString(m) } func (*ExitResult) ProtoMessage() {} func (*ExitResult) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{38} + return fileDescriptor_driver_f8179d6379690e6b, []int{38} } func (m *ExitResult) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ExitResult.Unmarshal(m, b) @@ -2215,7 +2215,7 @@ func (m *TaskStatus) Reset() { *m = TaskStatus{} } func (m *TaskStatus) String() string { return proto.CompactTextString(m) } func (*TaskStatus) ProtoMessage() {} func (*TaskStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{39} + return fileDescriptor_driver_f8179d6379690e6b, []int{39} } func (m *TaskStatus) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_TaskStatus.Unmarshal(m, b) @@ -2290,7 +2290,7 @@ func (m *TaskDriverStatus) Reset() { *m = TaskDriverStatus{} } func (m *TaskDriverStatus) String() string { return proto.CompactTextString(m) } func (*TaskDriverStatus) ProtoMessage() {} func (*TaskDriverStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{40} + return fileDescriptor_driver_f8179d6379690e6b, []int{40} } func (m *TaskDriverStatus) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_TaskDriverStatus.Unmarshal(m, b) @@ -2335,7 +2335,7 @@ func (m *TaskStats) Reset() { *m = TaskStats{} } func (m *TaskStats) String() string { return proto.CompactTextString(m) } func (*TaskStats) ProtoMessage() {} func (*TaskStats) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{41} + return fileDescriptor_driver_f8179d6379690e6b, []int{41} } func (m *TaskStats) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_TaskStats.Unmarshal(m, b) @@ -2397,7 +2397,7 @@ func (m *TaskResourceUsage) Reset() { *m = TaskResourceUsage{} } func (m *TaskResourceUsage) String() string { return proto.CompactTextString(m) } func (*TaskResourceUsage) ProtoMessage() {} func (*TaskResourceUsage) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{42} + return fileDescriptor_driver_f8179d6379690e6b, []int{42} } func (m *TaskResourceUsage) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_TaskResourceUsage.Unmarshal(m, b) @@ -2449,7 +2449,7 @@ func (m *CPUUsage) Reset() { *m = CPUUsage{} } func (m *CPUUsage) String() string { return proto.CompactTextString(m) } func (*CPUUsage) ProtoMessage() {} func (*CPUUsage) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{43} + return fileDescriptor_driver_f8179d6379690e6b, []int{43} } func (m *CPUUsage) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_CPUUsage.Unmarshal(m, b) @@ -2535,7 +2535,7 @@ func (m *MemoryUsage) Reset() { *m = MemoryUsage{} } func (m *MemoryUsage) String() string { return proto.CompactTextString(m) } func (*MemoryUsage) ProtoMessage() {} func (*MemoryUsage) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{44} + return fileDescriptor_driver_f8179d6379690e6b, []int{44} } func (m *MemoryUsage) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_MemoryUsage.Unmarshal(m, b) @@ -2600,12 +2600,16 @@ func (m *MemoryUsage) GetMeasuredFields() []MemoryUsage_Fields { type DriverTaskEvent struct { // TaskId is the id of the task for the event TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"` + // AllocId of the task for the event + AllocId string `protobuf:"bytes,2,opt,name=alloc_id,json=allocId,proto3" json:"alloc_id,omitempty"` + // TaskName is the name of the task for the event + TaskName string `protobuf:"bytes,3,opt,name=task_name,json=taskName,proto3" json:"task_name,omitempty"` // Timestamp when the event occurred - Timestamp *timestamp.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + Timestamp *timestamp.Timestamp `protobuf:"bytes,4,opt,name=timestamp,proto3" json:"timestamp,omitempty"` // Message is the body of the event - Message string `protobuf:"bytes,3,opt,name=message,proto3" json:"message,omitempty"` + Message string `protobuf:"bytes,5,opt,name=message,proto3" json:"message,omitempty"` // Annotations allows for additional key/value data to be sent along with the event - Annotations map[string]string `protobuf:"bytes,4,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Annotations map[string]string `protobuf:"bytes,6,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2615,7 +2619,7 @@ func (m *DriverTaskEvent) Reset() { *m = DriverTaskEvent{} } func (m *DriverTaskEvent) String() string { return proto.CompactTextString(m) } func (*DriverTaskEvent) ProtoMessage() {} func (*DriverTaskEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_driver_9320f8b051f51d4e, []int{45} + return fileDescriptor_driver_f8179d6379690e6b, []int{45} } func (m *DriverTaskEvent) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_DriverTaskEvent.Unmarshal(m, b) @@ -2642,6 +2646,20 @@ func (m *DriverTaskEvent) GetTaskId() string { return "" } +func (m *DriverTaskEvent) GetAllocId() string { + if m != nil { + return m.AllocId + } + return "" +} + +func (m *DriverTaskEvent) GetTaskName() string { + if m != nil { + return m.TaskName + } + return "" +} + func (m *DriverTaskEvent) GetTimestamp() *timestamp.Timestamp { if m != nil { return m.Timestamp @@ -3308,190 +3326,191 @@ var _Driver_serviceDesc = grpc.ServiceDesc{ } func init() { - proto.RegisterFile("plugins/drivers/proto/driver.proto", fileDescriptor_driver_9320f8b051f51d4e) + proto.RegisterFile("plugins/drivers/proto/driver.proto", fileDescriptor_driver_f8179d6379690e6b) } -var fileDescriptor_driver_9320f8b051f51d4e = []byte{ - // 2888 bytes of a gzipped FileDescriptorProto +var fileDescriptor_driver_f8179d6379690e6b = []byte{ + // 2908 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x59, 0x4b, 0x6f, 0x23, 0xc7, - 0xf1, 0x17, 0x49, 0x91, 0x22, 0x8b, 0x12, 0x35, 0xdb, 0xbb, 0x6b, 0xd3, 0x34, 0xfe, 0x7f, 0xaf, - 0x07, 0x70, 0x20, 0xd8, 0x5e, 0xca, 0x96, 0x91, 0x7d, 0x25, 0x7e, 0xd0, 0xd4, 0xac, 0x24, 0xaf, - 0x44, 0x29, 0x4d, 0x0a, 0xeb, 0x4d, 0x62, 0x4f, 0x46, 0x33, 0x2d, 0x72, 0x56, 0x9c, 0x87, 0xa7, - 0x7b, 0x64, 0x09, 0x41, 0x90, 0x20, 0x01, 0x82, 0xe4, 0x10, 0x20, 0x17, 0x23, 0x40, 0x8e, 0xc9, - 0x29, 0xc8, 0x37, 0x48, 0xe0, 0x4b, 0x3e, 0x45, 0x80, 0x5c, 0x72, 0x09, 0x90, 0x6b, 0xbe, 0x41, - 0xd0, 0x8f, 0x19, 0x0e, 0x25, 0xad, 0x77, 0xc8, 0xcd, 0x89, 0xec, 0xea, 0xae, 0x5f, 0xd7, 0x54, - 0x55, 0x57, 0x55, 0x77, 0x81, 0x1e, 0x8e, 0xe3, 0xa1, 0xeb, 0xd3, 0x75, 0x27, 0x72, 0x4f, 0x49, - 0x44, 0xd7, 0xc3, 0x28, 0x60, 0x81, 0x1a, 0xb5, 0xc5, 0x00, 0xbd, 0x31, 0xb2, 0xe8, 0xc8, 0xb5, - 0x83, 0x28, 0x6c, 0xfb, 0x81, 0x67, 0x39, 0x6d, 0xc5, 0xd3, 0x56, 0x3c, 0x72, 0x59, 0xeb, 0xff, - 0x87, 0x41, 0x30, 0x1c, 0x13, 0x89, 0x70, 0x14, 0x1f, 0xaf, 0x3b, 0x71, 0x64, 0x31, 0x37, 0xf0, - 0xd5, 0xfc, 0x6b, 0x17, 0xe7, 0x99, 0xeb, 0x11, 0xca, 0x2c, 0x2f, 0x54, 0x0b, 0x3e, 0x1a, 0xba, - 0x6c, 0x14, 0x1f, 0xb5, 0xed, 0xc0, 0x5b, 0x4f, 0xb7, 0x5c, 0x17, 0x5b, 0xae, 0x27, 0x62, 0xd2, - 0x91, 0x15, 0x11, 0x67, 0x7d, 0x64, 0x8f, 0x69, 0x48, 0x6c, 0xfe, 0x6b, 0xf2, 0x3f, 0x0a, 0x61, - 0x2b, 0x3f, 0x02, 0x65, 0x51, 0x6c, 0xb3, 0xe4, 0x7b, 0x2d, 0xc6, 0x22, 0xf7, 0x28, 0x66, 0x44, - 0x02, 0xe9, 0xaf, 0xc0, 0xcb, 0x03, 0x8b, 0x9e, 0x74, 0x03, 0xff, 0xd8, 0x1d, 0xf6, 0xed, 0x11, - 0xf1, 0x2c, 0x4c, 0xbe, 0x88, 0x09, 0x65, 0xfa, 0x0f, 0xa1, 0x79, 0x79, 0x8a, 0x86, 0x81, 0x4f, - 0x09, 0xfa, 0x08, 0x16, 0xb9, 0x34, 0xcd, 0xc2, 0xad, 0xc2, 0x5a, 0x7d, 0xe3, 0xed, 0xf6, 0xb3, - 0x14, 0x27, 0x65, 0x68, 0xab, 0xaf, 0x68, 0xf7, 0x43, 0x62, 0x63, 0xc1, 0xa9, 0xdf, 0x84, 0xeb, - 0x5d, 0x2b, 0xb4, 0x8e, 0xdc, 0xb1, 0xcb, 0x5c, 0x42, 0x93, 0x4d, 0x63, 0xb8, 0x31, 0x4d, 0x56, - 0x1b, 0x7e, 0x06, 0xcb, 0x76, 0x86, 0xae, 0x36, 0xbe, 0xdf, 0xce, 0x65, 0xb1, 0xf6, 0xa6, 0x18, - 0x4d, 0x01, 0x4f, 0xc1, 0xe9, 0x37, 0x00, 0x3d, 0x74, 0xfd, 0x21, 0x89, 0xc2, 0xc8, 0xf5, 0x59, - 0x22, 0xcc, 0xd7, 0x25, 0xb8, 0x3e, 0x45, 0x56, 0xc2, 0x3c, 0x05, 0x48, 0xf5, 0xc8, 0x45, 0x29, - 0xad, 0xd5, 0x37, 0x3e, 0xc9, 0x29, 0xca, 0x15, 0x78, 0xed, 0x4e, 0x0a, 0x66, 0xf8, 0x2c, 0x3a, - 0xc7, 0x19, 0x74, 0xf4, 0x39, 0x54, 0x46, 0xc4, 0x1a, 0xb3, 0x51, 0xb3, 0x78, 0xab, 0xb0, 0xd6, - 0xd8, 0x78, 0xf8, 0x02, 0xfb, 0x6c, 0x0b, 0xa0, 0x3e, 0xb3, 0x18, 0xc1, 0x0a, 0x15, 0xdd, 0x06, - 0x24, 0xff, 0x99, 0x0e, 0xa1, 0x76, 0xe4, 0x86, 0xdc, 0x91, 0x9b, 0xa5, 0x5b, 0x85, 0xb5, 0x1a, - 0xbe, 0x26, 0x67, 0x36, 0x27, 0x13, 0xad, 0x10, 0x56, 0x2f, 0x48, 0x8b, 0x34, 0x28, 0x9d, 0x90, - 0x73, 0x61, 0x91, 0x1a, 0xe6, 0x7f, 0xd1, 0x16, 0x94, 0x4f, 0xad, 0x71, 0x4c, 0x84, 0xc8, 0xf5, - 0x8d, 0x77, 0x9f, 0xe7, 0x1e, 0xca, 0x45, 0x27, 0x7a, 0xc0, 0x92, 0xff, 0x41, 0xf1, 0x5e, 0x41, - 0xbf, 0x0f, 0xf5, 0x8c, 0xdc, 0xa8, 0x01, 0x70, 0xd8, 0xdb, 0x34, 0x06, 0x46, 0x77, 0x60, 0x6c, - 0x6a, 0x0b, 0x68, 0x05, 0x6a, 0x87, 0xbd, 0x6d, 0xa3, 0xb3, 0x3b, 0xd8, 0x7e, 0xa2, 0x15, 0x50, - 0x1d, 0x96, 0x92, 0x41, 0x51, 0x3f, 0x03, 0x84, 0x89, 0x1d, 0x9c, 0x92, 0x88, 0x3b, 0xb2, 0xb2, - 0x2a, 0x7a, 0x19, 0x96, 0x98, 0x45, 0x4f, 0x4c, 0xd7, 0x51, 0x32, 0x57, 0xf8, 0x70, 0xc7, 0x41, - 0x3b, 0x50, 0x19, 0x59, 0xbe, 0x33, 0x7e, 0xbe, 0xdc, 0xd3, 0xaa, 0xe6, 0xe0, 0xdb, 0x82, 0x11, - 0x2b, 0x00, 0xee, 0xdd, 0x53, 0x3b, 0x4b, 0x03, 0xe8, 0x4f, 0x40, 0xeb, 0x33, 0x2b, 0x62, 0x59, - 0x71, 0x0c, 0x58, 0xe4, 0xfb, 0x2b, 0x8f, 0x9e, 0x65, 0x4f, 0x79, 0x32, 0xb1, 0x60, 0xd7, 0xff, - 0x53, 0x84, 0x6b, 0x19, 0x6c, 0xe5, 0xa9, 0x8f, 0xa1, 0x12, 0x11, 0x1a, 0x8f, 0x99, 0x80, 0x6f, - 0x6c, 0x7c, 0x98, 0x13, 0xfe, 0x12, 0x52, 0x1b, 0x0b, 0x18, 0xac, 0xe0, 0xd0, 0x1a, 0x68, 0x92, - 0xc3, 0x24, 0x51, 0x14, 0x44, 0xa6, 0x47, 0x87, 0x42, 0x6b, 0x35, 0xdc, 0x90, 0x74, 0x83, 0x93, - 0xf7, 0xe8, 0x30, 0xa3, 0xd5, 0xd2, 0x0b, 0x6a, 0x15, 0x59, 0xa0, 0xf9, 0x84, 0x7d, 0x19, 0x44, - 0x27, 0x26, 0x57, 0x6d, 0xe4, 0x3a, 0xa4, 0xb9, 0x28, 0x40, 0xef, 0xe4, 0x04, 0xed, 0x49, 0xf6, - 0x7d, 0xc5, 0x8d, 0x57, 0xfd, 0x69, 0x82, 0xfe, 0x16, 0x54, 0xe4, 0x97, 0x72, 0x4f, 0xea, 0x1f, - 0x76, 0xbb, 0x46, 0xbf, 0xaf, 0x2d, 0xa0, 0x1a, 0x94, 0xb1, 0x31, 0xc0, 0xdc, 0xc3, 0x6a, 0x50, - 0x7e, 0xd8, 0x19, 0x74, 0x76, 0xb5, 0xa2, 0xfe, 0x26, 0xac, 0x3e, 0xb6, 0x5c, 0x96, 0xc7, 0xb9, - 0xf4, 0x00, 0xb4, 0xc9, 0x5a, 0x65, 0x9d, 0x9d, 0x29, 0xeb, 0xe4, 0x57, 0x8d, 0x71, 0xe6, 0xb2, - 0x0b, 0xf6, 0xd0, 0xa0, 0x44, 0xa2, 0x48, 0x99, 0x80, 0xff, 0xd5, 0xbf, 0x84, 0xd5, 0x3e, 0x0b, - 0xc2, 0x5c, 0x9e, 0xff, 0x1e, 0x2c, 0xf1, 0x1c, 0x15, 0xc4, 0x4c, 0xb9, 0xfe, 0x2b, 0x6d, 0x99, - 0xc3, 0xda, 0x49, 0x0e, 0x6b, 0x6f, 0xaa, 0x1c, 0x87, 0x93, 0x95, 0xe8, 0x25, 0xa8, 0x50, 0x77, - 0xe8, 0x5b, 0x63, 0x15, 0x2d, 0xd4, 0x48, 0x47, 0xdc, 0xc9, 0x93, 0x8d, 0x95, 0xe3, 0x77, 0x01, - 0x6d, 0x12, 0xca, 0xa2, 0xe0, 0x3c, 0x97, 0x3c, 0x37, 0xa0, 0x7c, 0x1c, 0x44, 0xb6, 0x3c, 0x88, - 0x55, 0x2c, 0x07, 0xfc, 0x50, 0x4d, 0x81, 0x28, 0xec, 0xdb, 0x80, 0x76, 0x7c, 0x9e, 0x53, 0xf2, - 0x19, 0xe2, 0xb7, 0x45, 0xb8, 0x3e, 0xb5, 0x5e, 0x19, 0x63, 0xfe, 0x73, 0xc8, 0x03, 0x53, 0x4c, - 0xe5, 0x39, 0x44, 0xfb, 0x50, 0x91, 0x2b, 0x94, 0x26, 0xef, 0xce, 0x00, 0x24, 0xd3, 0x94, 0x82, - 0x53, 0x30, 0x57, 0x3a, 0x7d, 0xe9, 0x7f, 0xed, 0xf4, 0x5a, 0xf2, 0x1d, 0xf4, 0xb9, 0xfa, 0xfb, - 0x01, 0x5c, 0xcb, 0x2c, 0x56, 0xca, 0x7b, 0x08, 0x65, 0xca, 0x09, 0x4a, 0x7b, 0xef, 0xcc, 0xa8, - 0x3d, 0x8a, 0x25, 0xbb, 0x7e, 0x5d, 0x82, 0x1b, 0xa7, 0xc4, 0x4f, 0x45, 0xd1, 0x37, 0xe1, 0x5a, - 0x5f, 0xb8, 0x56, 0x2e, 0xdf, 0x99, 0xb8, 0x65, 0x71, 0xca, 0x2d, 0x6f, 0x00, 0xca, 0xa2, 0x28, - 0xe7, 0x39, 0x87, 0x55, 0xe3, 0x8c, 0xd8, 0xb9, 0x90, 0x9b, 0xb0, 0x64, 0x07, 0x9e, 0x67, 0xf9, - 0x4e, 0xb3, 0x78, 0xab, 0xb4, 0x56, 0xc3, 0xc9, 0x30, 0x7b, 0x7e, 0x4a, 0x79, 0xcf, 0x8f, 0xfe, - 0x9b, 0x02, 0x68, 0x93, 0xbd, 0x95, 0x22, 0xb9, 0xf4, 0xcc, 0xe1, 0x40, 0x7c, 0xef, 0x65, 0xac, - 0x46, 0x8a, 0x9e, 0x1c, 0x71, 0x49, 0x27, 0x51, 0x94, 0x09, 0x21, 0xa5, 0x17, 0x0c, 0x21, 0xfa, - 0xbf, 0x0a, 0x80, 0x2e, 0x17, 0x4a, 0xe8, 0x75, 0x58, 0xa6, 0xc4, 0x77, 0x4c, 0xa9, 0x46, 0x69, - 0xe1, 0x2a, 0xae, 0x73, 0x9a, 0xd4, 0x27, 0x45, 0x08, 0x16, 0xc9, 0x19, 0xb1, 0xd5, 0x69, 0x15, - 0xff, 0xd1, 0x08, 0x96, 0x8f, 0xa9, 0xe9, 0xd2, 0x60, 0x6c, 0xa5, 0x15, 0x45, 0x63, 0xc3, 0x98, - 0xbb, 0x60, 0x6b, 0x3f, 0xec, 0xef, 0x24, 0x60, 0xb8, 0x7e, 0x4c, 0xd3, 0x81, 0xde, 0x86, 0x7a, - 0x66, 0x0e, 0x55, 0x61, 0xb1, 0xb7, 0xdf, 0x33, 0xb4, 0x05, 0x04, 0x50, 0xe9, 0x6e, 0xe3, 0xfd, - 0xfd, 0x81, 0x8c, 0xda, 0x3b, 0x7b, 0x9d, 0x2d, 0x43, 0x2b, 0xea, 0xbf, 0x2f, 0x03, 0x4c, 0xd2, - 0x27, 0x6a, 0x40, 0x31, 0xb5, 0x74, 0xd1, 0x75, 0xf8, 0xc7, 0xf8, 0x96, 0x47, 0x94, 0xf7, 0x88, - 0xff, 0x68, 0x03, 0x6e, 0x7a, 0x74, 0x18, 0x5a, 0xf6, 0x89, 0xa9, 0xb2, 0x9e, 0x2d, 0x98, 0xc5, - 0x57, 0x2d, 0xe3, 0xeb, 0x6a, 0x52, 0x49, 0x2d, 0x71, 0x77, 0xa1, 0x44, 0xfc, 0xd3, 0xe6, 0xa2, - 0xa8, 0x0e, 0x1f, 0xcc, 0x9c, 0xd6, 0xdb, 0x86, 0x7f, 0x2a, 0xab, 0x41, 0x0e, 0x83, 0x7a, 0x50, - 0x8b, 0x08, 0x0d, 0xe2, 0xc8, 0x26, 0xb4, 0x59, 0x9e, 0xe9, 0x90, 0xe1, 0x84, 0x0f, 0x4f, 0x20, - 0xd0, 0x26, 0x54, 0xbc, 0x20, 0xf6, 0x19, 0x6d, 0x56, 0x84, 0x80, 0x6f, 0xe7, 0x04, 0xdb, 0xe3, - 0x4c, 0x58, 0xf1, 0xa2, 0x2d, 0x58, 0x72, 0xc8, 0xa9, 0xcb, 0x65, 0x5a, 0x12, 0x30, 0xb7, 0xf3, - 0xda, 0x57, 0x70, 0xe1, 0x84, 0x9b, 0x2b, 0x3d, 0xa6, 0x24, 0x6a, 0x56, 0xa5, 0xd2, 0xf9, 0x7f, - 0xf4, 0x2a, 0xd4, 0xac, 0xf1, 0x38, 0xb0, 0x4d, 0xc7, 0x8d, 0x9a, 0x35, 0x31, 0x51, 0x15, 0x84, - 0x4d, 0x37, 0x42, 0xaf, 0x41, 0x5d, 0x9e, 0x0c, 0x33, 0xb4, 0xd8, 0xa8, 0x09, 0x62, 0x1a, 0x24, - 0xe9, 0xc0, 0x62, 0x23, 0xb5, 0x80, 0x44, 0x91, 0x5c, 0x50, 0x4f, 0x17, 0x90, 0x28, 0x12, 0x0b, - 0xbe, 0x05, 0xab, 0xe2, 0x98, 0x0f, 0xa3, 0x20, 0x0e, 0x4d, 0x61, 0xf2, 0x65, 0xb1, 0x68, 0x85, - 0x93, 0xb7, 0x38, 0xb5, 0xc7, 0x6d, 0xff, 0x0a, 0x54, 0x9f, 0x06, 0x47, 0x72, 0xc1, 0x8a, 0x58, - 0xb0, 0xf4, 0x34, 0x38, 0x4a, 0xa6, 0xa4, 0x84, 0xae, 0xd3, 0x6c, 0xc8, 0x29, 0x31, 0xde, 0x71, - 0x5a, 0x77, 0xa0, 0x9a, 0x18, 0xf0, 0x8a, 0x02, 0xf9, 0x46, 0xb6, 0x40, 0xae, 0x65, 0xab, 0xdd, - 0x7f, 0x14, 0xa0, 0x96, 0x1a, 0x0c, 0xf9, 0x70, 0x5d, 0x00, 0x5a, 0x8c, 0x38, 0xe6, 0xc4, 0xfe, - 0x32, 0xc8, 0xbe, 0x9f, 0x53, 0xd7, 0x9d, 0x04, 0x41, 0x05, 0x1a, 0xe5, 0x0c, 0x28, 0x45, 0x9e, - 0xec, 0xf7, 0x39, 0xac, 0x8e, 0x5d, 0x3f, 0x3e, 0xcb, 0xec, 0x25, 0xb3, 0xd8, 0xb7, 0x73, 0xee, - 0xb5, 0xcb, 0xb9, 0x27, 0x7b, 0x34, 0xc6, 0x53, 0x63, 0xfd, 0xab, 0x22, 0xbc, 0x74, 0xb5, 0x38, - 0xa8, 0x07, 0x25, 0x3b, 0x8c, 0xd5, 0xa7, 0x7d, 0x77, 0xd6, 0x4f, 0xeb, 0x86, 0xf1, 0x64, 0x57, - 0x0e, 0xc4, 0x2b, 0x5f, 0x8f, 0x78, 0x41, 0x74, 0xae, 0xbe, 0xe0, 0xc3, 0x59, 0x21, 0xf7, 0x04, - 0xf7, 0x04, 0x55, 0xc1, 0x21, 0x0c, 0x55, 0x95, 0x3f, 0xf9, 0x41, 0x2c, 0xcd, 0x9e, 0x87, 0x13, - 0x48, 0x9c, 0xe2, 0xe8, 0x77, 0xe0, 0xe6, 0x95, 0x9f, 0x82, 0xfe, 0x0f, 0xc0, 0x0e, 0x63, 0x53, - 0xdc, 0x93, 0xa4, 0xdd, 0x4b, 0xb8, 0x66, 0x87, 0x71, 0x5f, 0x10, 0xf4, 0xbb, 0xd0, 0x7c, 0x96, - 0xbc, 0xfc, 0xf8, 0x48, 0x89, 0x4d, 0xef, 0x48, 0xe8, 0xa0, 0x84, 0xab, 0x92, 0xb0, 0x77, 0xa4, - 0xff, 0xae, 0x08, 0xab, 0x17, 0xc4, 0xe1, 0x29, 0x46, 0x1e, 0xc7, 0x24, 0xed, 0xc9, 0x11, 0x3f, - 0x9b, 0xb6, 0xeb, 0x24, 0xb5, 0xa5, 0xf8, 0x2f, 0x82, 0x66, 0xa8, 0xea, 0xbe, 0xa2, 0x1b, 0x72, - 0x87, 0xf6, 0x8e, 0x5c, 0x46, 0x45, 0x39, 0x5e, 0xc6, 0x72, 0x80, 0x9e, 0x40, 0x23, 0x22, 0x94, - 0x44, 0xa7, 0xc4, 0x31, 0xc3, 0x20, 0x62, 0x89, 0xc2, 0x36, 0x66, 0x53, 0xd8, 0x41, 0x10, 0x31, - 0xbc, 0x92, 0x20, 0xf1, 0x11, 0x45, 0x8f, 0x61, 0xc5, 0x39, 0xf7, 0x2d, 0xcf, 0xb5, 0x15, 0x72, - 0x65, 0x6e, 0xe4, 0x65, 0x05, 0x24, 0x80, 0xf9, 0x75, 0x33, 0x33, 0xc9, 0x3f, 0x6c, 0x6c, 0x1d, - 0x91, 0xb1, 0xd2, 0x89, 0x1c, 0x4c, 0x9f, 0xdf, 0xb2, 0x3a, 0xbf, 0xfa, 0x1f, 0x8b, 0xd0, 0x98, - 0x3e, 0x00, 0x89, 0xfd, 0x42, 0x12, 0xb9, 0x81, 0x93, 0xb1, 0xdf, 0x81, 0x20, 0x70, 0x1b, 0xf1, - 0xe9, 0x2f, 0xe2, 0x80, 0x59, 0x89, 0x8d, 0xec, 0x30, 0xfe, 0x1e, 0x1f, 0x5f, 0xb0, 0x7d, 0xe9, - 0x82, 0xed, 0xd1, 0xdb, 0x80, 0x94, 0x7d, 0xc7, 0xae, 0xe7, 0x32, 0xf3, 0xe8, 0x9c, 0x11, 0xa9, - 0xff, 0x12, 0xd6, 0xe4, 0xcc, 0x2e, 0x9f, 0xf8, 0x98, 0xd3, 0x91, 0x0e, 0x2b, 0x41, 0xe0, 0x99, - 0xd4, 0x0e, 0x22, 0x62, 0x5a, 0xce, 0x53, 0x91, 0x43, 0x4a, 0xb8, 0x1e, 0x04, 0x5e, 0x9f, 0xd3, - 0x3a, 0xce, 0x53, 0x1e, 0x32, 0xed, 0x30, 0xa6, 0x84, 0x99, 0xfc, 0xa7, 0x59, 0x91, 0x21, 0x53, - 0x92, 0xba, 0x61, 0x4c, 0x33, 0x0b, 0x3c, 0xe2, 0xf1, 0x90, 0x9f, 0x59, 0xb0, 0x47, 0x3c, 0xbe, - 0xcb, 0xf2, 0x01, 0x89, 0x6c, 0xe2, 0xb3, 0x81, 0x6b, 0x9f, 0x50, 0x11, 0xce, 0x0b, 0x78, 0x8a, - 0xa6, 0x7f, 0x06, 0x65, 0x91, 0x44, 0xf8, 0xc7, 0x8b, 0x00, 0x2c, 0xe2, 0xb3, 0x54, 0x6f, 0x95, - 0x13, 0x44, 0x74, 0x7e, 0x15, 0x6a, 0xa3, 0x80, 0xaa, 0xe8, 0x2e, 0x3d, 0xaf, 0xca, 0x09, 0x62, - 0xb2, 0x05, 0xd5, 0x88, 0x58, 0x4e, 0xe0, 0x8f, 0xcf, 0x85, 0x5e, 0xaa, 0x38, 0x1d, 0xeb, 0x5f, - 0x40, 0x45, 0x26, 0x97, 0x17, 0xc0, 0xbf, 0x0d, 0xc8, 0x96, 0x69, 0x21, 0x24, 0x91, 0xe7, 0x52, - 0xea, 0x06, 0x3e, 0x4d, 0xde, 0x44, 0xe4, 0xcc, 0xc1, 0x64, 0x42, 0xff, 0x5b, 0x41, 0x16, 0x14, - 0xf2, 0xb6, 0xca, 0x4b, 0x32, 0x55, 0x1d, 0xcc, 0x7d, 0xa5, 0x57, 0x00, 0x49, 0x59, 0x4d, 0xd4, - 0xdb, 0xcf, 0xac, 0x65, 0x35, 0x91, 0x65, 0x35, 0xe1, 0x35, 0x9c, 0xaa, 0x5b, 0x24, 0x9c, 0x2c, - 0x5b, 0xea, 0x4e, 0x7a, 0xdf, 0x20, 0xfa, 0xbf, 0x0b, 0x69, 0x44, 0x48, 0xee, 0x05, 0xe8, 0x73, - 0xa8, 0xf2, 0xc3, 0x65, 0x7a, 0x56, 0xa8, 0x5e, 0xb9, 0xba, 0xf3, 0x5d, 0x39, 0xda, 0xfc, 0x2c, - 0xed, 0x59, 0xa1, 0x2c, 0x68, 0x96, 0x42, 0x39, 0xe2, 0x91, 0xc5, 0x72, 0x26, 0x91, 0x85, 0xff, - 0x47, 0x6f, 0x40, 0xc3, 0x8a, 0x59, 0x60, 0x5a, 0xce, 0x29, 0x89, 0x98, 0x4b, 0x89, 0xb2, 0xf0, - 0x0a, 0xa7, 0x76, 0x12, 0x62, 0xeb, 0x01, 0x2c, 0x67, 0x31, 0x9f, 0x97, 0x63, 0xcb, 0xd9, 0x1c, - 0xfb, 0x23, 0x80, 0x49, 0xf9, 0xcb, 0x3d, 0x81, 0x9c, 0xb9, 0xcc, 0xb4, 0x03, 0x47, 0x46, 0xbe, - 0x32, 0xae, 0x72, 0x42, 0x37, 0x70, 0xc8, 0x85, 0xcb, 0x44, 0x39, 0xb9, 0x4c, 0xf0, 0xb3, 0xc9, - 0x8f, 0xd3, 0x89, 0x3b, 0x1e, 0x13, 0x47, 0x49, 0x58, 0x0b, 0x02, 0xef, 0x91, 0x20, 0xe8, 0x5f, - 0x17, 0xa5, 0x47, 0xc8, 0xab, 0x5c, 0xae, 0x12, 0x33, 0x35, 0x75, 0xe9, 0xc5, 0x4c, 0x7d, 0x1f, - 0x80, 0x32, 0x2b, 0xe2, 0x05, 0x83, 0xc5, 0xd4, 0xeb, 0x48, 0xeb, 0xd2, 0x6d, 0x64, 0x90, 0xbc, - 0x48, 0xe3, 0x9a, 0x5a, 0xdd, 0x61, 0xe8, 0x7d, 0x58, 0xb6, 0x03, 0x2f, 0x1c, 0x13, 0xc5, 0x5c, - 0x7e, 0x2e, 0x73, 0x3d, 0x5d, 0xdf, 0x61, 0x99, 0xab, 0x48, 0xe5, 0x45, 0xaf, 0x22, 0x7f, 0x29, - 0xc8, 0x1b, 0x69, 0xf6, 0x42, 0x8c, 0x86, 0x57, 0xbc, 0xba, 0x6e, 0xcd, 0x79, 0xbb, 0xfe, 0xa6, - 0x27, 0xd7, 0xd6, 0xfb, 0x79, 0xde, 0x38, 0x9f, 0x5d, 0xc2, 0xfd, 0xb5, 0x04, 0xb5, 0xf4, 0x62, - 0x7b, 0xc9, 0xf6, 0xf7, 0xa0, 0x96, 0xb6, 0x03, 0x54, 0x69, 0xf2, 0x8d, 0xe6, 0x49, 0x17, 0xa3, - 0x63, 0x40, 0xd6, 0x70, 0x98, 0x96, 0x66, 0x66, 0x4c, 0xad, 0x61, 0xf2, 0x14, 0x70, 0x6f, 0x06, - 0x3d, 0x24, 0xd9, 0xe9, 0x90, 0xf3, 0x63, 0xcd, 0x1a, 0x0e, 0xa7, 0x28, 0xe8, 0xc7, 0x70, 0x73, - 0x7a, 0x0f, 0xf3, 0xe8, 0xdc, 0x0c, 0x5d, 0x47, 0x5d, 0x65, 0xb6, 0x67, 0xbd, 0xdb, 0xb7, 0xa7, - 0xe0, 0x3f, 0x3e, 0x3f, 0x70, 0x1d, 0xa9, 0x73, 0x14, 0x5d, 0x9a, 0x68, 0xfd, 0x14, 0x5e, 0x7e, - 0xc6, 0xf2, 0x2b, 0x6c, 0xd0, 0x9b, 0x7e, 0x67, 0x9e, 0x5f, 0x09, 0x19, 0xeb, 0xfd, 0xa1, 0x20, - 0x9f, 0x20, 0xa6, 0x75, 0xd2, 0xc9, 0x56, 0xa7, 0xeb, 0x39, 0xf7, 0xe9, 0x1e, 0x1c, 0x4a, 0x78, - 0x51, 0x90, 0x7e, 0x72, 0xa1, 0x20, 0xcd, 0x5b, 0xaa, 0xc8, 0xba, 0x4e, 0x02, 0x29, 0x04, 0xfd, - 0xcf, 0x25, 0xa8, 0x26, 0xe8, 0xe2, 0xa6, 0x73, 0x4e, 0x19, 0xf1, 0x4c, 0x2f, 0x09, 0x61, 0x05, - 0x0c, 0x92, 0xb4, 0xc7, 0x83, 0xd8, 0xab, 0x50, 0xe3, 0x17, 0x2a, 0x39, 0x5d, 0x14, 0xd3, 0x55, - 0x4e, 0x10, 0x93, 0xaf, 0x41, 0x9d, 0x05, 0xcc, 0x1a, 0x9b, 0x4c, 0x64, 0xec, 0x92, 0xe4, 0x16, - 0x24, 0x91, 0xaf, 0xd1, 0x5b, 0x70, 0x8d, 0x8d, 0xa2, 0x80, 0xb1, 0x31, 0xaf, 0xe2, 0x44, 0xdd, - 0x22, 0xcb, 0x8c, 0x45, 0xac, 0xa5, 0x13, 0xb2, 0x9e, 0xa1, 0x3c, 0x7a, 0x4f, 0x16, 0x73, 0xd7, - 0x15, 0x41, 0x64, 0x11, 0xaf, 0xa4, 0x54, 0xee, 0xda, 0xa8, 0x09, 0x4b, 0xa1, 0xac, 0x09, 0x44, - 0xac, 0x28, 0xe0, 0x64, 0x88, 0x4c, 0x58, 0xf5, 0x88, 0x45, 0xe3, 0x88, 0x38, 0xe6, 0xb1, 0x4b, - 0xc6, 0x8e, 0xbc, 0x59, 0x36, 0x72, 0x17, 0xd9, 0x89, 0x5a, 0xda, 0x0f, 0x05, 0x37, 0x6e, 0x24, - 0x70, 0x72, 0xcc, 0xeb, 0x03, 0xf9, 0x0f, 0xad, 0x42, 0xbd, 0xff, 0xa4, 0x3f, 0x30, 0xf6, 0xcc, - 0xbd, 0xfd, 0x4d, 0x43, 0xb5, 0x12, 0xfa, 0x06, 0x96, 0xc3, 0x02, 0x9f, 0x1f, 0xec, 0x0f, 0x3a, - 0xbb, 0xe6, 0x60, 0xa7, 0xfb, 0xa8, 0xaf, 0x15, 0xd1, 0x4d, 0xb8, 0x36, 0xd8, 0xc6, 0xfb, 0x83, - 0xc1, 0xae, 0xb1, 0x69, 0x1e, 0x18, 0x78, 0x67, 0x7f, 0xb3, 0xaf, 0x95, 0x10, 0x82, 0xc6, 0x84, - 0x3c, 0xd8, 0xd9, 0x33, 0xb4, 0x45, 0x54, 0x87, 0xa5, 0x03, 0x03, 0x77, 0x8d, 0xde, 0x40, 0x2b, - 0xeb, 0x7f, 0x2f, 0x42, 0x3d, 0x63, 0x45, 0xee, 0xc8, 0x11, 0x95, 0xd5, 0xfc, 0x22, 0xe6, 0x7f, - 0x79, 0x30, 0xb1, 0x2d, 0x7b, 0x24, 0xad, 0xb3, 0x88, 0xe5, 0x40, 0x54, 0xf0, 0xd6, 0x59, 0xe6, - 0x9c, 0x2f, 0xe2, 0xaa, 0x67, 0x9d, 0x49, 0x90, 0xd7, 0x61, 0xf9, 0x84, 0x44, 0x3e, 0x19, 0xab, - 0x79, 0x69, 0x91, 0xba, 0xa4, 0xc9, 0x25, 0x6b, 0xa0, 0xa9, 0x25, 0x13, 0x18, 0x69, 0x8e, 0x86, - 0xa4, 0xef, 0x25, 0x60, 0x47, 0x97, 0xb5, 0x5e, 0x11, 0x5a, 0xbf, 0x3f, 0xbb, 0x93, 0x3e, 0x4b, - 0xf1, 0xfd, 0x54, 0xf1, 0x4b, 0x50, 0xc2, 0xc9, 0xab, 0x7a, 0xb7, 0xd3, 0xdd, 0xe6, 0xca, 0x5e, - 0x81, 0xda, 0x5e, 0xe7, 0x53, 0xf3, 0xb0, 0x2f, 0xde, 0x68, 0x90, 0x06, 0xcb, 0x8f, 0x0c, 0xdc, - 0x33, 0x76, 0x15, 0xa5, 0x84, 0x6e, 0x80, 0xa6, 0x28, 0x93, 0x75, 0x8b, 0xfa, 0x9f, 0x8a, 0xb0, - 0x2a, 0xe3, 0x7a, 0xfa, 0x6c, 0xf8, 0xec, 0xf7, 0xbb, 0xf9, 0x43, 0x6f, 0x13, 0x96, 0x3c, 0x42, - 0x53, 0x3b, 0xd4, 0x70, 0x32, 0x44, 0x2e, 0xd4, 0x2d, 0xdf, 0x0f, 0x98, 0x78, 0x7b, 0xa2, 0x2a, - 0x44, 0x6e, 0xcd, 0xf4, 0xca, 0x95, 0x4a, 0xde, 0xee, 0x4c, 0x90, 0x64, 0x84, 0xcc, 0x62, 0xb7, - 0x3e, 0x00, 0xed, 0xe2, 0x82, 0x59, 0xf2, 0xd2, 0x9b, 0xef, 0x4e, 0xd2, 0x12, 0xe1, 0x0e, 0x7a, - 0xd8, 0x7b, 0xd4, 0xdb, 0x7f, 0xdc, 0xd3, 0x16, 0xf8, 0x00, 0x1f, 0xf6, 0x7a, 0x3b, 0xbd, 0x2d, - 0xad, 0x80, 0x00, 0x2a, 0xc6, 0xa7, 0x3b, 0x03, 0x63, 0x53, 0x2b, 0x6e, 0xfc, 0x73, 0x05, 0x2a, - 0x52, 0x48, 0xf4, 0x95, 0x4a, 0xc9, 0xd9, 0x76, 0x30, 0xfa, 0x60, 0xe6, 0xd2, 0x76, 0xaa, 0xc5, - 0xdc, 0xfa, 0x70, 0x6e, 0x7e, 0xf5, 0x7c, 0xbb, 0x80, 0x7e, 0x5d, 0x80, 0xe5, 0xa9, 0xf7, 0xca, - 0xbc, 0x4f, 0x6d, 0x57, 0x74, 0x9f, 0x5b, 0xdf, 0x99, 0x8b, 0x37, 0x95, 0xe5, 0x57, 0x05, 0xa8, - 0x67, 0xfa, 0xae, 0xe8, 0xfe, 0x3c, 0xbd, 0x5a, 0x29, 0xc9, 0x83, 0xf9, 0xdb, 0xbc, 0xfa, 0xc2, - 0x3b, 0x05, 0xf4, 0xcb, 0x02, 0xd4, 0x33, 0x1d, 0xc8, 0xdc, 0xa2, 0x5c, 0xee, 0x97, 0xe6, 0x16, - 0xe5, 0xaa, 0x86, 0xe7, 0x02, 0xfa, 0x59, 0x01, 0x6a, 0x69, 0x37, 0x11, 0xdd, 0x9d, 0xbd, 0xff, - 0x28, 0x85, 0xb8, 0x37, 0x6f, 0xe3, 0x52, 0x5f, 0x40, 0x3f, 0x81, 0x6a, 0xd2, 0x7a, 0x43, 0x79, - 0xd3, 0xc8, 0x85, 0xbe, 0x5e, 0xeb, 0xee, 0xcc, 0x7c, 0xd9, 0xed, 0x93, 0x7e, 0x58, 0xee, 0xed, - 0x2f, 0x74, 0xee, 0x5a, 0x77, 0x67, 0xe6, 0x4b, 0xb7, 0xe7, 0x9e, 0x90, 0x69, 0x9b, 0xe5, 0xf6, - 0x84, 0xcb, 0xfd, 0xba, 0xdc, 0x9e, 0x70, 0x55, 0x97, 0x4e, 0x0a, 0x92, 0x69, 0xbc, 0xe5, 0x16, - 0xe4, 0x72, 0x73, 0x2f, 0xb7, 0x20, 0x57, 0xf4, 0xf9, 0x94, 0x4b, 0x4e, 0x0a, 0xf4, 0xbb, 0x33, - 0xf7, 0xaa, 0x66, 0x74, 0xc9, 0x4b, 0xdd, 0x32, 0x7d, 0x01, 0xfd, 0x5c, 0x3d, 0x19, 0xc8, 0x46, - 0x17, 0x9a, 0x05, 0x6a, 0xaa, 0x37, 0xd6, 0xba, 0x33, 0x5f, 0xaa, 0x11, 0x31, 0xe2, 0x17, 0x05, - 0x80, 0x49, 0x4b, 0x2c, 0xb7, 0x10, 0x97, 0x7a, 0x71, 0xad, 0xfb, 0x73, 0x70, 0x66, 0x8f, 0x47, - 0xd2, 0x05, 0xcb, 0x7d, 0x3c, 0x2e, 0xb4, 0xec, 0x72, 0x1f, 0x8f, 0x8b, 0xed, 0x36, 0x7d, 0xe1, - 0xe3, 0xa5, 0xef, 0x97, 0x65, 0xee, 0xaf, 0x88, 0x9f, 0xf7, 0xfe, 0x1b, 0x00, 0x00, 0xff, 0xff, - 0xdc, 0x12, 0xd9, 0x00, 0x29, 0x26, 0x00, 0x00, + 0xf1, 0x17, 0x9f, 0x22, 0x8b, 0x12, 0x35, 0xdb, 0xbb, 0x6b, 0xd3, 0x34, 0xfe, 0x7f, 0xaf, 0x07, + 0x70, 0xb0, 0xb0, 0xbd, 0x94, 0x2d, 0x23, 0xfb, 0x4a, 0xfc, 0xa0, 0xa9, 0x59, 0x49, 0x5e, 0x89, + 0x52, 0x9a, 0x14, 0xd6, 0x9b, 0xc4, 0x9e, 0x8c, 0x66, 0x7a, 0xc9, 0x59, 0x71, 0x1e, 0xee, 0xe9, + 0x91, 0x25, 0x04, 0x41, 0x82, 0x04, 0x08, 0x92, 0x43, 0x80, 0x5c, 0x8c, 0x00, 0x39, 0x26, 0xc7, + 0x7c, 0x83, 0x04, 0xbe, 0xe4, 0x53, 0x04, 0xc8, 0x25, 0x39, 0x04, 0xc8, 0x35, 0xdf, 0x20, 0xe8, + 0xc7, 0x0c, 0x87, 0x92, 0xd6, 0x3b, 0xe4, 0xe6, 0x44, 0x76, 0x75, 0xd7, 0xaf, 0x6b, 0xaa, 0xaa, + 0xab, 0xaa, 0xbb, 0x40, 0x0f, 0x27, 0xf1, 0xc8, 0xf5, 0xa3, 0x75, 0x87, 0xba, 0x27, 0x84, 0x46, + 0xeb, 0x21, 0x0d, 0x58, 0xa0, 0x46, 0x1d, 0x31, 0x40, 0x6f, 0x8c, 0xad, 0x68, 0xec, 0xda, 0x01, + 0x0d, 0x3b, 0x7e, 0xe0, 0x59, 0x4e, 0x47, 0xf1, 0x74, 0x14, 0x8f, 0x5c, 0xd6, 0xfe, 0xff, 0x51, + 0x10, 0x8c, 0x26, 0x44, 0x22, 0x1c, 0xc5, 0x4f, 0xd6, 0x9d, 0x98, 0x5a, 0xcc, 0x0d, 0x7c, 0x35, + 0xff, 0xda, 0xf9, 0x79, 0xe6, 0x7a, 0x24, 0x62, 0x96, 0x17, 0xaa, 0x05, 0x1f, 0x8d, 0x5c, 0x36, + 0x8e, 0x8f, 0x3a, 0x76, 0xe0, 0xad, 0xa7, 0x5b, 0xae, 0x8b, 0x2d, 0xd7, 0x13, 0x31, 0xa3, 0xb1, + 0x45, 0x89, 0xb3, 0x3e, 0xb6, 0x27, 0x51, 0x48, 0x6c, 0xfe, 0x6b, 0xf2, 0x3f, 0x0a, 0x61, 0x2b, + 0x3f, 0x42, 0xc4, 0x68, 0x6c, 0xb3, 0xe4, 0x7b, 0x2d, 0xc6, 0xa8, 0x7b, 0x14, 0x33, 0x22, 0x81, + 0xf4, 0x57, 0xe0, 0xe5, 0xa1, 0x15, 0x1d, 0xf7, 0x02, 0xff, 0x89, 0x3b, 0x1a, 0xd8, 0x63, 0xe2, + 0x59, 0x98, 0x7c, 0x11, 0x93, 0x88, 0xe9, 0x3f, 0x84, 0xd6, 0xc5, 0xa9, 0x28, 0x0c, 0xfc, 0x88, + 0xa0, 0x8f, 0xa0, 0xcc, 0xa5, 0x69, 0x15, 0x6e, 0x14, 0x6e, 0x36, 0x36, 0xde, 0xee, 0x3c, 0x4b, + 0x71, 0x52, 0x86, 0x8e, 0xfa, 0x8a, 0xce, 0x20, 0x24, 0x36, 0x16, 0x9c, 0xfa, 0x75, 0xb8, 0xda, + 0xb3, 0x42, 0xeb, 0xc8, 0x9d, 0xb8, 0xcc, 0x25, 0x51, 0xb2, 0x69, 0x0c, 0xd7, 0x66, 0xc9, 0x6a, + 0xc3, 0xcf, 0x60, 0xc5, 0xce, 0xd0, 0xd5, 0xc6, 0xf7, 0x3a, 0xb9, 0x2c, 0xd6, 0xd9, 0x14, 0xa3, + 0x19, 0xe0, 0x19, 0x38, 0xfd, 0x1a, 0xa0, 0x07, 0xae, 0x3f, 0x22, 0x34, 0xa4, 0xae, 0xcf, 0x12, + 0x61, 0xbe, 0x2e, 0xc1, 0xd5, 0x19, 0xb2, 0x12, 0xe6, 0x29, 0x40, 0xaa, 0x47, 0x2e, 0x4a, 0xe9, + 0x66, 0x63, 0xe3, 0x93, 0x9c, 0xa2, 0x5c, 0x82, 0xd7, 0xe9, 0xa6, 0x60, 0x86, 0xcf, 0xe8, 0x19, + 0xce, 0xa0, 0xa3, 0xcf, 0xa1, 0x3a, 0x26, 0xd6, 0x84, 0x8d, 0x5b, 0xc5, 0x1b, 0x85, 0x9b, 0xcd, + 0x8d, 0x07, 0x2f, 0xb0, 0xcf, 0xb6, 0x00, 0x1a, 0x30, 0x8b, 0x11, 0xac, 0x50, 0xd1, 0x2d, 0x40, + 0xf2, 0x9f, 0xe9, 0x90, 0xc8, 0xa6, 0x6e, 0xc8, 0x1d, 0xb9, 0x55, 0xba, 0x51, 0xb8, 0x59, 0xc7, + 0x57, 0xe4, 0xcc, 0xe6, 0x74, 0xa2, 0x1d, 0xc2, 0xda, 0x39, 0x69, 0x91, 0x06, 0xa5, 0x63, 0x72, + 0x26, 0x2c, 0x52, 0xc7, 0xfc, 0x2f, 0xda, 0x82, 0xca, 0x89, 0x35, 0x89, 0x89, 0x10, 0xb9, 0xb1, + 0xf1, 0xee, 0xf3, 0xdc, 0x43, 0xb9, 0xe8, 0x54, 0x0f, 0x58, 0xf2, 0xdf, 0x2f, 0xde, 0x2d, 0xe8, + 0xf7, 0xa0, 0x91, 0x91, 0x1b, 0x35, 0x01, 0x0e, 0xfb, 0x9b, 0xc6, 0xd0, 0xe8, 0x0d, 0x8d, 0x4d, + 0x6d, 0x09, 0xad, 0x42, 0xfd, 0xb0, 0xbf, 0x6d, 0x74, 0x77, 0x87, 0xdb, 0x8f, 0xb5, 0x02, 0x6a, + 0xc0, 0x72, 0x32, 0x28, 0xea, 0xa7, 0x80, 0x30, 0xb1, 0x83, 0x13, 0x42, 0xb9, 0x23, 0x2b, 0xab, + 0xa2, 0x97, 0x61, 0x99, 0x59, 0xd1, 0xb1, 0xe9, 0x3a, 0x4a, 0xe6, 0x2a, 0x1f, 0xee, 0x38, 0x68, + 0x07, 0xaa, 0x63, 0xcb, 0x77, 0x26, 0xcf, 0x97, 0x7b, 0x56, 0xd5, 0x1c, 0x7c, 0x5b, 0x30, 0x62, + 0x05, 0xc0, 0xbd, 0x7b, 0x66, 0x67, 0x69, 0x00, 0xfd, 0x31, 0x68, 0x03, 0x66, 0x51, 0x96, 0x15, + 0xc7, 0x80, 0x32, 0xdf, 0x5f, 0x79, 0xf4, 0x3c, 0x7b, 0xca, 0x93, 0x89, 0x05, 0xbb, 0xfe, 0x9f, + 0x22, 0x5c, 0xc9, 0x60, 0x2b, 0x4f, 0x7d, 0x04, 0x55, 0x4a, 0xa2, 0x78, 0xc2, 0x04, 0x7c, 0x73, + 0xe3, 0xc3, 0x9c, 0xf0, 0x17, 0x90, 0x3a, 0x58, 0xc0, 0x60, 0x05, 0x87, 0x6e, 0x82, 0x26, 0x39, + 0x4c, 0x42, 0x69, 0x40, 0x4d, 0x2f, 0x1a, 0x09, 0xad, 0xd5, 0x71, 0x53, 0xd2, 0x0d, 0x4e, 0xde, + 0x8b, 0x46, 0x19, 0xad, 0x96, 0x5e, 0x50, 0xab, 0xc8, 0x02, 0xcd, 0x27, 0xec, 0xcb, 0x80, 0x1e, + 0x9b, 0x5c, 0xb5, 0xd4, 0x75, 0x48, 0xab, 0x2c, 0x40, 0x6f, 0xe7, 0x04, 0xed, 0x4b, 0xf6, 0x7d, + 0xc5, 0x8d, 0xd7, 0xfc, 0x59, 0x82, 0xfe, 0x16, 0x54, 0xe5, 0x97, 0x72, 0x4f, 0x1a, 0x1c, 0xf6, + 0x7a, 0xc6, 0x60, 0xa0, 0x2d, 0xa1, 0x3a, 0x54, 0xb0, 0x31, 0xc4, 0xdc, 0xc3, 0xea, 0x50, 0x79, + 0xd0, 0x1d, 0x76, 0x77, 0xb5, 0xa2, 0xfe, 0x26, 0xac, 0x3d, 0xb2, 0x5c, 0x96, 0xc7, 0xb9, 0xf4, + 0x00, 0xb4, 0xe9, 0x5a, 0x65, 0x9d, 0x9d, 0x19, 0xeb, 0xe4, 0x57, 0x8d, 0x71, 0xea, 0xb2, 0x73, + 0xf6, 0xd0, 0xa0, 0x44, 0x28, 0x55, 0x26, 0xe0, 0x7f, 0xf5, 0x2f, 0x61, 0x6d, 0xc0, 0x82, 0x30, + 0x97, 0xe7, 0xbf, 0x07, 0xcb, 0x3c, 0x47, 0x05, 0x31, 0x53, 0xae, 0xff, 0x4a, 0x47, 0xe6, 0xb0, + 0x4e, 0x92, 0xc3, 0x3a, 0x9b, 0x2a, 0xc7, 0xe1, 0x64, 0x25, 0x7a, 0x09, 0xaa, 0x91, 0x3b, 0xf2, + 0xad, 0x89, 0x8a, 0x16, 0x6a, 0xa4, 0x23, 0xee, 0xe4, 0xc9, 0xc6, 0xca, 0xf1, 0x7b, 0x80, 0x36, + 0x49, 0xc4, 0x68, 0x70, 0x96, 0x4b, 0x9e, 0x6b, 0x50, 0x79, 0x12, 0x50, 0x5b, 0x1e, 0xc4, 0x1a, + 0x96, 0x03, 0x7e, 0xa8, 0x66, 0x40, 0x14, 0xf6, 0x2d, 0x40, 0x3b, 0x3e, 0xcf, 0x29, 0xf9, 0x0c, + 0xf1, 0xdb, 0x22, 0x5c, 0x9d, 0x59, 0xaf, 0x8c, 0xb1, 0xf8, 0x39, 0xe4, 0x81, 0x29, 0x8e, 0xe4, + 0x39, 0x44, 0xfb, 0x50, 0x95, 0x2b, 0x94, 0x26, 0xef, 0xcc, 0x01, 0x24, 0xd3, 0x94, 0x82, 0x53, + 0x30, 0x97, 0x3a, 0x7d, 0xe9, 0x7f, 0xed, 0xf4, 0x5a, 0xf2, 0x1d, 0xd1, 0x73, 0xf5, 0xf7, 0x03, + 0xb8, 0x92, 0x59, 0xac, 0x94, 0xf7, 0x00, 0x2a, 0x11, 0x27, 0x28, 0xed, 0xbd, 0x33, 0xa7, 0xf6, + 0x22, 0x2c, 0xd9, 0xf5, 0xab, 0x12, 0xdc, 0x38, 0x21, 0x7e, 0x2a, 0x8a, 0xbe, 0x09, 0x57, 0x06, + 0xc2, 0xb5, 0x72, 0xf9, 0xce, 0xd4, 0x2d, 0x8b, 0x33, 0x6e, 0x79, 0x0d, 0x50, 0x16, 0x45, 0x39, + 0xcf, 0x19, 0xac, 0x19, 0xa7, 0xc4, 0xce, 0x85, 0xdc, 0x82, 0x65, 0x3b, 0xf0, 0x3c, 0xcb, 0x77, + 0x5a, 0xc5, 0x1b, 0xa5, 0x9b, 0x75, 0x9c, 0x0c, 0xb3, 0xe7, 0xa7, 0x94, 0xf7, 0xfc, 0xe8, 0xbf, + 0x29, 0x80, 0x36, 0xdd, 0x5b, 0x29, 0x92, 0x4b, 0xcf, 0x1c, 0x0e, 0xc4, 0xf7, 0x5e, 0xc1, 0x6a, + 0xa4, 0xe8, 0xc9, 0x11, 0x97, 0x74, 0x42, 0x69, 0x26, 0x84, 0x94, 0x5e, 0x30, 0x84, 0xe8, 0xff, + 0x2a, 0x00, 0xba, 0x58, 0x28, 0xa1, 0xd7, 0x61, 0x25, 0x22, 0xbe, 0x63, 0x4a, 0x35, 0x4a, 0x0b, + 0xd7, 0x70, 0x83, 0xd3, 0xa4, 0x3e, 0x23, 0x84, 0xa0, 0x4c, 0x4e, 0x89, 0xad, 0x4e, 0xab, 0xf8, + 0x8f, 0xc6, 0xb0, 0xf2, 0x24, 0x32, 0xdd, 0x28, 0x98, 0x58, 0x69, 0x45, 0xd1, 0xdc, 0x30, 0x16, + 0x2e, 0xd8, 0x3a, 0x0f, 0x06, 0x3b, 0x09, 0x18, 0x6e, 0x3c, 0x89, 0xd2, 0x81, 0xde, 0x81, 0x46, + 0x66, 0x0e, 0xd5, 0xa0, 0xdc, 0xdf, 0xef, 0x1b, 0xda, 0x12, 0x02, 0xa8, 0xf6, 0xb6, 0xf1, 0xfe, + 0xfe, 0x50, 0x46, 0xed, 0x9d, 0xbd, 0xee, 0x96, 0xa1, 0x15, 0xf5, 0xdf, 0x57, 0x00, 0xa6, 0xe9, + 0x13, 0x35, 0xa1, 0x98, 0x5a, 0xba, 0xe8, 0x3a, 0xfc, 0x63, 0x7c, 0xcb, 0x23, 0xca, 0x7b, 0xc4, + 0x7f, 0xb4, 0x01, 0xd7, 0xbd, 0x68, 0x14, 0x5a, 0xf6, 0xb1, 0xa9, 0xb2, 0x9e, 0x2d, 0x98, 0xc5, + 0x57, 0xad, 0xe0, 0xab, 0x6a, 0x52, 0x49, 0x2d, 0x71, 0x77, 0xa1, 0x44, 0xfc, 0x93, 0x56, 0x59, + 0x54, 0x87, 0xf7, 0xe7, 0x4e, 0xeb, 0x1d, 0xc3, 0x3f, 0x91, 0xd5, 0x20, 0x87, 0x41, 0x7d, 0xa8, + 0x53, 0x12, 0x05, 0x31, 0xb5, 0x49, 0xd4, 0xaa, 0xcc, 0x75, 0xc8, 0x70, 0xc2, 0x87, 0xa7, 0x10, + 0x68, 0x13, 0xaa, 0x5e, 0x10, 0xfb, 0x2c, 0x6a, 0x55, 0x85, 0x80, 0x6f, 0xe7, 0x04, 0xdb, 0xe3, + 0x4c, 0x58, 0xf1, 0xa2, 0x2d, 0x58, 0x76, 0xc8, 0x89, 0xcb, 0x65, 0x5a, 0x16, 0x30, 0xb7, 0xf2, + 0xda, 0x57, 0x70, 0xe1, 0x84, 0x9b, 0x2b, 0x3d, 0x8e, 0x08, 0x6d, 0xd5, 0xa4, 0xd2, 0xf9, 0x7f, + 0xf4, 0x2a, 0xd4, 0xad, 0xc9, 0x24, 0xb0, 0x4d, 0xc7, 0xa5, 0xad, 0xba, 0x98, 0xa8, 0x09, 0xc2, + 0xa6, 0x4b, 0xd1, 0x6b, 0xd0, 0x90, 0x27, 0xc3, 0x0c, 0x2d, 0x36, 0x6e, 0x81, 0x98, 0x06, 0x49, + 0x3a, 0xb0, 0xd8, 0x58, 0x2d, 0x20, 0x94, 0xca, 0x05, 0x8d, 0x74, 0x01, 0xa1, 0x54, 0x2c, 0xf8, + 0x16, 0xac, 0x89, 0x63, 0x3e, 0xa2, 0x41, 0x1c, 0x9a, 0xc2, 0xe4, 0x2b, 0x62, 0xd1, 0x2a, 0x27, + 0x6f, 0x71, 0x6a, 0x9f, 0xdb, 0xfe, 0x15, 0xa8, 0x3d, 0x0d, 0x8e, 0xe4, 0x82, 0x55, 0xb1, 0x60, + 0xf9, 0x69, 0x70, 0x94, 0x4c, 0x49, 0x09, 0x5d, 0xa7, 0xd5, 0x94, 0x53, 0x62, 0xbc, 0xe3, 0xb4, + 0x6f, 0x43, 0x2d, 0x31, 0xe0, 0x25, 0x05, 0xf2, 0xb5, 0x6c, 0x81, 0x5c, 0xcf, 0x56, 0xbb, 0x7f, + 0x2f, 0x40, 0x3d, 0x35, 0x18, 0xf2, 0xe1, 0xaa, 0x00, 0xb4, 0x18, 0x71, 0xcc, 0xa9, 0xfd, 0x65, + 0x90, 0x7d, 0x3f, 0xa7, 0xae, 0xbb, 0x09, 0x82, 0x0a, 0x34, 0xca, 0x19, 0x50, 0x8a, 0x3c, 0xdd, + 0xef, 0x73, 0x58, 0x9b, 0xb8, 0x7e, 0x7c, 0x9a, 0xd9, 0x4b, 0x66, 0xb1, 0x6f, 0xe7, 0xdc, 0x6b, + 0x97, 0x73, 0x4f, 0xf7, 0x68, 0x4e, 0x66, 0xc6, 0xfa, 0x57, 0x45, 0x78, 0xe9, 0x72, 0x71, 0x50, + 0x1f, 0x4a, 0x76, 0x18, 0xab, 0x4f, 0xfb, 0xee, 0xbc, 0x9f, 0xd6, 0x0b, 0xe3, 0xe9, 0xae, 0x1c, + 0x88, 0x57, 0xbe, 0x1e, 0xf1, 0x02, 0x7a, 0xa6, 0xbe, 0xe0, 0xc3, 0x79, 0x21, 0xf7, 0x04, 0xf7, + 0x14, 0x55, 0xc1, 0x21, 0x0c, 0x35, 0x95, 0x3f, 0xf9, 0x41, 0x2c, 0xcd, 0x9f, 0x87, 0x13, 0x48, + 0x9c, 0xe2, 0xe8, 0xb7, 0xe1, 0xfa, 0xa5, 0x9f, 0x82, 0xfe, 0x0f, 0xc0, 0x0e, 0x63, 0x53, 0xdc, + 0x93, 0xa4, 0xdd, 0x4b, 0xb8, 0x6e, 0x87, 0xf1, 0x40, 0x10, 0xf4, 0x3b, 0xd0, 0x7a, 0x96, 0xbc, + 0xfc, 0xf8, 0x48, 0x89, 0x4d, 0xef, 0x48, 0xe8, 0xa0, 0x84, 0x6b, 0x92, 0xb0, 0x77, 0xa4, 0xff, + 0xae, 0x08, 0x6b, 0xe7, 0xc4, 0xe1, 0x29, 0x46, 0x1e, 0xc7, 0x24, 0xed, 0xc9, 0x11, 0x3f, 0x9b, + 0xb6, 0xeb, 0x24, 0xb5, 0xa5, 0xf8, 0x2f, 0x82, 0x66, 0xa8, 0xea, 0xbe, 0xa2, 0x1b, 0x72, 0x87, + 0xf6, 0x8e, 0x5c, 0x16, 0x89, 0x72, 0xbc, 0x82, 0xe5, 0x00, 0x3d, 0x86, 0x26, 0x25, 0x11, 0xa1, + 0x27, 0xc4, 0x31, 0xc3, 0x80, 0xb2, 0x44, 0x61, 0x1b, 0xf3, 0x29, 0xec, 0x20, 0xa0, 0x0c, 0xaf, + 0x26, 0x48, 0x7c, 0x14, 0xa1, 0x47, 0xb0, 0xea, 0x9c, 0xf9, 0x96, 0xe7, 0xda, 0x0a, 0xb9, 0xba, + 0x30, 0xf2, 0x8a, 0x02, 0x12, 0xc0, 0xfc, 0xba, 0x99, 0x99, 0xe4, 0x1f, 0x36, 0xb1, 0x8e, 0xc8, + 0x44, 0xe9, 0x44, 0x0e, 0x66, 0xcf, 0x6f, 0x45, 0x9d, 0x5f, 0xfd, 0x8f, 0x45, 0x68, 0xce, 0x1e, + 0x80, 0xc4, 0x7e, 0x21, 0xa1, 0x6e, 0xe0, 0x64, 0xec, 0x77, 0x20, 0x08, 0xdc, 0x46, 0x7c, 0xfa, + 0x8b, 0x38, 0x60, 0x56, 0x62, 0x23, 0x3b, 0x8c, 0xbf, 0xc7, 0xc7, 0xe7, 0x6c, 0x5f, 0x3a, 0x67, + 0x7b, 0xf4, 0x36, 0x20, 0x65, 0xdf, 0x89, 0xeb, 0xb9, 0xcc, 0x3c, 0x3a, 0x63, 0x44, 0xea, 0xbf, + 0x84, 0x35, 0x39, 0xb3, 0xcb, 0x27, 0x3e, 0xe6, 0x74, 0xa4, 0xc3, 0x6a, 0x10, 0x78, 0x66, 0x64, + 0x07, 0x94, 0x98, 0x96, 0xf3, 0x54, 0xe4, 0x90, 0x12, 0x6e, 0x04, 0x81, 0x37, 0xe0, 0xb4, 0xae, + 0xf3, 0x94, 0x87, 0x4c, 0x3b, 0x8c, 0x23, 0xc2, 0x4c, 0xfe, 0xd3, 0xaa, 0xca, 0x90, 0x29, 0x49, + 0xbd, 0x30, 0x8e, 0x32, 0x0b, 0x3c, 0xe2, 0xf1, 0x90, 0x9f, 0x59, 0xb0, 0x47, 0x3c, 0xbe, 0xcb, + 0xca, 0x01, 0xa1, 0x36, 0xf1, 0xd9, 0xd0, 0xb5, 0x8f, 0x23, 0x11, 0xce, 0x0b, 0x78, 0x86, 0xa6, + 0x7f, 0x06, 0x15, 0x91, 0x44, 0xf8, 0xc7, 0x8b, 0x00, 0x2c, 0xe2, 0xb3, 0x54, 0x6f, 0x8d, 0x13, + 0x44, 0x74, 0x7e, 0x15, 0xea, 0xe3, 0x20, 0x52, 0xd1, 0x5d, 0x7a, 0x5e, 0x8d, 0x13, 0xc4, 0x64, + 0x1b, 0x6a, 0x94, 0x58, 0x4e, 0xe0, 0x4f, 0xce, 0x84, 0x5e, 0x6a, 0x38, 0x1d, 0xeb, 0x5f, 0x40, + 0x55, 0x26, 0x97, 0x17, 0xc0, 0xbf, 0x05, 0xc8, 0x96, 0x69, 0x21, 0x24, 0xd4, 0x73, 0xa3, 0xc8, + 0x0d, 0xfc, 0x28, 0x79, 0x13, 0x91, 0x33, 0x07, 0xd3, 0x09, 0xfd, 0xaf, 0x05, 0x59, 0x50, 0xc8, + 0xdb, 0x2a, 0x2f, 0xc9, 0x54, 0x75, 0xb0, 0xf0, 0x95, 0x5e, 0x01, 0x24, 0x65, 0x35, 0x51, 0x6f, + 0x3f, 0xf3, 0x96, 0xd5, 0x44, 0x96, 0xd5, 0x84, 0xd7, 0x70, 0xaa, 0x6e, 0x91, 0x70, 0xb2, 0x6c, + 0x69, 0x38, 0xe9, 0x7d, 0x83, 0xe8, 0xff, 0x2e, 0xa4, 0x11, 0x21, 0xb9, 0x17, 0xa0, 0xcf, 0xa1, + 0xc6, 0x0f, 0x97, 0xe9, 0x59, 0xa1, 0x7a, 0xe5, 0xea, 0x2d, 0x76, 0xe5, 0xe8, 0xf0, 0xb3, 0xb4, + 0x67, 0x85, 0xb2, 0xa0, 0x59, 0x0e, 0xe5, 0x88, 0x47, 0x16, 0xcb, 0x99, 0x46, 0x16, 0xfe, 0x1f, + 0xbd, 0x01, 0x4d, 0x2b, 0x66, 0x81, 0x69, 0x39, 0x27, 0x84, 0x32, 0x37, 0x22, 0xca, 0xc2, 0xab, + 0x9c, 0xda, 0x4d, 0x88, 0xed, 0xfb, 0xb0, 0x92, 0xc5, 0x7c, 0x5e, 0x8e, 0xad, 0x64, 0x73, 0xec, + 0x8f, 0x00, 0xa6, 0xe5, 0x2f, 0xf7, 0x04, 0x72, 0xea, 0x32, 0xd3, 0x0e, 0x1c, 0x19, 0xf9, 0x2a, + 0xb8, 0xc6, 0x09, 0xbd, 0xc0, 0x21, 0xe7, 0x2e, 0x13, 0x95, 0xe4, 0x32, 0xc1, 0xcf, 0x26, 0x3f, + 0x4e, 0xc7, 0xee, 0x64, 0x42, 0x1c, 0x25, 0x61, 0x3d, 0x08, 0xbc, 0x87, 0x82, 0xa0, 0x7f, 0x5d, + 0x94, 0x1e, 0x21, 0xaf, 0x72, 0xb9, 0x4a, 0xcc, 0xd4, 0xd4, 0xa5, 0x17, 0x33, 0xf5, 0x3d, 0x80, + 0x88, 0x59, 0x94, 0x17, 0x0c, 0x16, 0x53, 0xaf, 0x23, 0xed, 0x0b, 0xb7, 0x91, 0x61, 0xf2, 0x22, + 0x8d, 0xeb, 0x6a, 0x75, 0x97, 0xa1, 0xf7, 0x61, 0xc5, 0x0e, 0xbc, 0x70, 0x42, 0x14, 0x73, 0xe5, + 0xb9, 0xcc, 0x8d, 0x74, 0x7d, 0x97, 0x65, 0xae, 0x22, 0xd5, 0x17, 0xbd, 0x8a, 0xfc, 0xb9, 0x20, + 0x6f, 0xa4, 0xd9, 0x0b, 0x31, 0x1a, 0x5d, 0xf2, 0xea, 0xba, 0xb5, 0xe0, 0xed, 0xfa, 0x9b, 0x9e, + 0x5c, 0xdb, 0xef, 0xe7, 0x79, 0xe3, 0x7c, 0x76, 0x09, 0xf7, 0x97, 0x12, 0xd4, 0xd3, 0x8b, 0xed, + 0x05, 0xdb, 0xdf, 0x85, 0x7a, 0xda, 0x0e, 0x50, 0xa5, 0xc9, 0x37, 0x9a, 0x27, 0x5d, 0x8c, 0x9e, + 0x00, 0xb2, 0x46, 0xa3, 0xb4, 0x34, 0x33, 0xe3, 0xc8, 0x1a, 0x25, 0x4f, 0x01, 0x77, 0xe7, 0xd0, + 0x43, 0x92, 0x9d, 0x0e, 0x39, 0x3f, 0xd6, 0xac, 0xd1, 0x68, 0x86, 0x82, 0x7e, 0x0c, 0xd7, 0x67, + 0xf7, 0x30, 0x8f, 0xce, 0xcc, 0xd0, 0x75, 0xd4, 0x55, 0x66, 0x7b, 0xde, 0xbb, 0x7d, 0x67, 0x06, + 0xfe, 0xe3, 0xb3, 0x03, 0xd7, 0x91, 0x3a, 0x47, 0xf4, 0xc2, 0x44, 0xfb, 0xa7, 0xf0, 0xf2, 0x33, + 0x96, 0x5f, 0x62, 0x83, 0xfe, 0xec, 0x3b, 0xf3, 0xe2, 0x4a, 0xc8, 0x58, 0xef, 0x0f, 0x05, 0xf9, + 0x04, 0x31, 0xab, 0x93, 0x6e, 0xb6, 0x3a, 0x5d, 0xcf, 0xb9, 0x4f, 0xef, 0xe0, 0x50, 0xc2, 0x8b, + 0x82, 0xf4, 0x93, 0x73, 0x05, 0x69, 0xde, 0x52, 0x45, 0xd6, 0x75, 0x12, 0x48, 0x21, 0xe8, 0x7f, + 0x2a, 0x41, 0x2d, 0x41, 0x17, 0x37, 0x9d, 0xb3, 0x88, 0x11, 0xcf, 0xf4, 0x92, 0x10, 0x56, 0xc0, + 0x20, 0x49, 0x7b, 0x3c, 0x88, 0xbd, 0x0a, 0x75, 0x7e, 0xa1, 0x92, 0xd3, 0x45, 0x31, 0x5d, 0xe3, + 0x04, 0x31, 0xf9, 0x1a, 0x34, 0x58, 0xc0, 0xac, 0x89, 0xc9, 0x44, 0xc6, 0x2e, 0x49, 0x6e, 0x41, + 0x12, 0xf9, 0x1a, 0xbd, 0x05, 0x57, 0xd8, 0x98, 0x06, 0x8c, 0x4d, 0x78, 0x15, 0x27, 0xea, 0x16, + 0x59, 0x66, 0x94, 0xb1, 0x96, 0x4e, 0xc8, 0x7a, 0x26, 0xe2, 0xd1, 0x7b, 0xba, 0x98, 0xbb, 0xae, + 0x08, 0x22, 0x65, 0xbc, 0x9a, 0x52, 0xb9, 0x6b, 0xa3, 0x16, 0x2c, 0x87, 0xb2, 0x26, 0x10, 0xb1, + 0xa2, 0x80, 0x93, 0x21, 0x32, 0x61, 0xcd, 0x23, 0x56, 0x14, 0x53, 0xe2, 0x98, 0x4f, 0x5c, 0x32, + 0x71, 0xe4, 0xcd, 0xb2, 0x99, 0xbb, 0xc8, 0x4e, 0xd4, 0xd2, 0x79, 0x20, 0xb8, 0x71, 0x33, 0x81, + 0x93, 0x63, 0x5e, 0x1f, 0xc8, 0x7f, 0x68, 0x0d, 0x1a, 0x83, 0xc7, 0x83, 0xa1, 0xb1, 0x67, 0xee, + 0xed, 0x6f, 0x1a, 0xaa, 0x95, 0x30, 0x30, 0xb0, 0x1c, 0x16, 0xf8, 0xfc, 0x70, 0x7f, 0xd8, 0xdd, + 0x35, 0x87, 0x3b, 0xbd, 0x87, 0x03, 0xad, 0x88, 0xae, 0xc3, 0x95, 0xe1, 0x36, 0xde, 0x1f, 0x0e, + 0x77, 0x8d, 0x4d, 0xf3, 0xc0, 0xc0, 0x3b, 0xfb, 0x9b, 0x03, 0xad, 0x84, 0x10, 0x34, 0xa7, 0xe4, + 0xe1, 0xce, 0x9e, 0xa1, 0x95, 0x51, 0x03, 0x96, 0x0f, 0x0c, 0xdc, 0x33, 0xfa, 0x43, 0xad, 0xa2, + 0xff, 0xad, 0x08, 0x8d, 0x8c, 0x15, 0xb9, 0x23, 0xd3, 0x48, 0x56, 0xf3, 0x65, 0xcc, 0xff, 0xf2, + 0x60, 0x62, 0x5b, 0xf6, 0x58, 0x5a, 0xa7, 0x8c, 0xe5, 0x40, 0x54, 0xf0, 0xd6, 0x69, 0xe6, 0x9c, + 0x97, 0x71, 0xcd, 0xb3, 0x4e, 0x25, 0xc8, 0xeb, 0xb0, 0x72, 0x4c, 0xa8, 0x4f, 0x26, 0x6a, 0x5e, + 0x5a, 0xa4, 0x21, 0x69, 0x72, 0xc9, 0x4d, 0xd0, 0xd4, 0x92, 0x29, 0x8c, 0x34, 0x47, 0x53, 0xd2, + 0xf7, 0x12, 0xb0, 0xa3, 0x8b, 0x5a, 0xaf, 0x0a, 0xad, 0xdf, 0x9b, 0xdf, 0x49, 0x9f, 0xa5, 0xf8, + 0x41, 0xaa, 0xf8, 0x65, 0x28, 0xe1, 0xe4, 0x55, 0xbd, 0xd7, 0xed, 0x6d, 0x73, 0x65, 0xaf, 0x42, + 0x7d, 0xaf, 0xfb, 0xa9, 0x79, 0x38, 0x10, 0x6f, 0x34, 0x48, 0x83, 0x95, 0x87, 0x06, 0xee, 0x1b, + 0xbb, 0x8a, 0x52, 0x42, 0xd7, 0x40, 0x53, 0x94, 0xe9, 0xba, 0xb2, 0xfe, 0xcf, 0x22, 0xac, 0xc9, + 0xb8, 0x9e, 0x3e, 0x1b, 0x3e, 0xfb, 0xfd, 0x2e, 0x7b, 0x5d, 0x2f, 0xce, 0x5c, 0xd7, 0xd3, 0x5a, + 0x51, 0xa4, 0xe5, 0xd2, 0xb4, 0x56, 0x14, 0xd7, 0xfc, 0x99, 0x90, 0x5d, 0x9e, 0x27, 0x64, 0xb7, + 0x60, 0xd9, 0x23, 0x51, 0xaa, 0xf8, 0x3a, 0x4e, 0x86, 0xc8, 0x85, 0x86, 0xe5, 0xfb, 0x01, 0x13, + 0x6f, 0x56, 0xc9, 0xed, 0x65, 0x6b, 0xae, 0xd7, 0xb1, 0xf4, 0x8b, 0x3b, 0xdd, 0x29, 0x92, 0x8c, + 0xac, 0x59, 0xec, 0xf6, 0x07, 0xa0, 0x9d, 0x5f, 0x30, 0x4f, 0x3e, 0x7b, 0xf3, 0xdd, 0x69, 0x3a, + 0x23, 0xdc, 0xb1, 0x0f, 0xfb, 0x0f, 0xfb, 0xfb, 0x8f, 0xfa, 0xda, 0x12, 0x1f, 0xe0, 0xc3, 0x7e, + 0x7f, 0xa7, 0xbf, 0xa5, 0x15, 0x10, 0x40, 0xd5, 0xf8, 0x74, 0x67, 0x68, 0x6c, 0x6a, 0xc5, 0x8d, + 0x7f, 0xac, 0x42, 0x55, 0x0a, 0x89, 0xbe, 0x52, 0xa9, 0x3c, 0xdb, 0x46, 0x46, 0x1f, 0xcc, 0x5d, + 0x12, 0xcf, 0xb4, 0xa6, 0xdb, 0x1f, 0x2e, 0xcc, 0xaf, 0x9e, 0x7d, 0x97, 0xd0, 0xaf, 0x0b, 0xb0, + 0x32, 0xf3, 0xce, 0x99, 0xf7, 0x89, 0xee, 0x92, 0xae, 0x75, 0xfb, 0x3b, 0x0b, 0xf1, 0xa6, 0xb2, + 0xfc, 0xaa, 0x00, 0x8d, 0x4c, 0xbf, 0x16, 0xdd, 0x5b, 0xa4, 0xc7, 0x2b, 0x25, 0xb9, 0xbf, 0x78, + 0x7b, 0x58, 0x5f, 0x7a, 0xa7, 0x80, 0x7e, 0x59, 0x80, 0x46, 0xa6, 0x73, 0x99, 0x5b, 0x94, 0x8b, + 0x7d, 0xd6, 0xdc, 0xa2, 0x5c, 0xd6, 0x28, 0x5d, 0x42, 0x3f, 0x2b, 0x40, 0x3d, 0xed, 0x42, 0xa2, + 0x3b, 0xf3, 0xf7, 0x2d, 0xa5, 0x10, 0x77, 0x17, 0x6d, 0x78, 0xea, 0x4b, 0xe8, 0x27, 0x50, 0x4b, + 0x5a, 0x76, 0x28, 0x6f, 0xfa, 0x39, 0xd7, 0x0f, 0x6c, 0xdf, 0x99, 0x9b, 0x2f, 0xbb, 0x7d, 0xd2, + 0x47, 0xcb, 0xbd, 0xfd, 0xb9, 0x8e, 0x5f, 0xfb, 0xce, 0xdc, 0x7c, 0xe9, 0xf6, 0xdc, 0x13, 0x32, + 0xed, 0xb6, 0xdc, 0x9e, 0x70, 0xb1, 0xcf, 0x97, 0xdb, 0x13, 0x2e, 0xeb, 0xee, 0x49, 0x41, 0x32, + 0x0d, 0xbb, 0xdc, 0x82, 0x5c, 0x6c, 0x0a, 0xe6, 0x16, 0xe4, 0x92, 0xfe, 0xa0, 0x72, 0xc9, 0x69, + 0x61, 0x7f, 0x67, 0xee, 0x1e, 0xd7, 0x9c, 0x2e, 0x79, 0xa1, 0xcb, 0xa6, 0x2f, 0xa1, 0x9f, 0xab, + 0xa7, 0x06, 0xd9, 0x20, 0x43, 0xf3, 0x40, 0xcd, 0xf4, 0xd4, 0xda, 0xb7, 0x17, 0x4b, 0x35, 0x22, + 0x46, 0xfc, 0xa2, 0x00, 0x30, 0x6d, 0xa5, 0xe5, 0x16, 0xe2, 0x42, 0x0f, 0xaf, 0x7d, 0x6f, 0x01, + 0xce, 0xec, 0xf1, 0x48, 0xba, 0x67, 0xb9, 0x8f, 0xc7, 0xb9, 0x56, 0x5f, 0xee, 0xe3, 0x71, 0xbe, + 0x4d, 0xa7, 0x2f, 0x7d, 0xbc, 0xfc, 0xfd, 0x8a, 0xcc, 0xfd, 0x55, 0xf1, 0xf3, 0xde, 0x7f, 0x03, + 0x00, 0x00, 0xff, 0xff, 0xa1, 0x73, 0x76, 0xe7, 0x61, 0x26, 0x00, 0x00, } diff --git a/plugins/drivers/proto/driver.proto b/plugins/drivers/proto/driver.proto index bbb93e104..b6d8e867f 100644 --- a/plugins/drivers/proto/driver.proto +++ b/plugins/drivers/proto/driver.proto @@ -572,12 +572,18 @@ message DriverTaskEvent { // TaskId is the id of the task for the event string task_id = 1; + // AllocId of the task for the event + string alloc_id = 2; + + // TaskName is the name of the task for the event + string task_name = 3; + // Timestamp when the event occurred - google.protobuf.Timestamp timestamp = 2; + google.protobuf.Timestamp timestamp = 4; // Message is the body of the event - string message = 3; + string message = 5; // Annotations allows for additional key/value data to be sent along with the event - map annotations = 4; + map annotations = 6; } diff --git a/plugins/drivers/server.go b/plugins/drivers/server.go index f24072a3c..337d78fcc 100644 --- a/plugins/drivers/server.go +++ b/plugins/drivers/server.go @@ -289,6 +289,8 @@ func (b *driverPluginServer) TaskEvents(req *proto.TaskEventsRequest, srv proto. pbEvent := &proto.DriverTaskEvent{ TaskId: event.TaskID, + AllocId: event.AllocID, + TaskName: event.TaskName, Timestamp: pbTimestamp, Message: event.Message, Annotations: event.Annotations,