client: begin driver plugin integration

client: fingerprint driver plugins
This commit is contained in:
Nick Ethier
2018-10-04 15:08:20 -04:00
committed by Michael Schurter
parent 627e20801d
commit d335a82859
31 changed files with 2240 additions and 579 deletions

View File

@@ -13,6 +13,9 @@ import (
// TaskDir contains all of the paths relevant to a task. All paths are on the
// host system so drivers should mount/link into task containers as necessary.
type TaskDir struct {
// AllocDir is the path to the alloc directory on the host
AllocDir string
// Dir is the path to Task directory on the host
Dir string
@@ -50,6 +53,7 @@ func newTaskDir(logger hclog.Logger, allocDir, taskName string) *TaskDir {
logger = logger.Named("task_dir").With("task_name", taskName)
return &TaskDir{
AllocDir: allocDir,
Dir: taskDir,
SharedAllocDir: filepath.Join(allocDir, SharedAllocName),
LogDir: filepath.Join(allocDir, SharedAllocName, LogDirName),

View File

@@ -9,6 +9,7 @@ import (
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.
@@ -37,4 +38,11 @@ type Config struct {
// PrevAllocWatcher handles waiting on previous allocations and
// migrating their ephemeral disk when necessary.
PrevAllocWatcher allocwatcher.PrevAllocWatcher
// 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
}

View File

@@ -3,7 +3,7 @@ package interfaces
import (
"context"
"github.com/hashicorp/nomad/client/driver"
"github.com/hashicorp/nomad/client/allocrunnerv2/taskrunner/interfaces"
"github.com/hashicorp/nomad/client/driver/env"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
@@ -82,7 +82,7 @@ type DriverStats interface {
type TaskPoststartRequest struct {
// Exec hook (may be nil)
DriverExec driver.ScriptExecutor
DriverExec interfaces.ScriptExecutor
// Network info (may be nil)
DriverNetwork *cstructs.DriverNetwork

View File

@@ -0,0 +1,64 @@
package taskrunner
import (
"context"
"time"
"github.com/hashicorp/nomad/client/allocrunnerv2/taskrunner/interfaces"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/drivers"
)
func NewDriverHandle(driver drivers.DriverPlugin, taskID string, task *structs.Task, net *cstructs.DriverNetwork) interfaces.DriverHandle {
return &driverHandleImpl{
driver: driver,
net: net,
taskID: taskID,
task: task,
}
}
type driverHandleImpl struct {
driver drivers.DriverPlugin
net *cstructs.DriverNetwork
task *structs.Task
taskID string
}
func (h *driverHandleImpl) ID() string {
return h.taskID
}
func (h *driverHandleImpl) WaitCh(ctx context.Context) (<-chan *drivers.ExitResult, error) {
return h.driver.WaitTask(ctx, h.taskID)
}
func (h *driverHandleImpl) Update(task *structs.Task) error {
return nil
}
func (h *driverHandleImpl) Kill() error {
return h.driver.StopTask(h.taskID, h.task.KillTimeout, h.task.KillSignal)
}
func (h *driverHandleImpl) Stats() (*cstructs.TaskResourceUsage, error) {
return h.driver.TaskStats(h.taskID)
}
func (h *driverHandleImpl) Signal(s string) error {
return h.driver.SignalTask(h.taskID, s)
}
func (h *driverHandleImpl) Exec(timeout time.Duration, cmd string, args []string) ([]byte, int, error) {
command := append([]string{cmd}, args...)
res, err := h.driver.ExecTask(h.taskID, command, timeout)
if err != nil {
return nil, 0, err
}
return res.Stdout, res.ExitResult.ExitCode, res.ExitResult.Err
}
func (h *driverHandleImpl) Network() *cstructs.DriverNetwork {
return h.net
}

View File

@@ -0,0 +1,47 @@
package interfaces
import (
"context"
"time"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/drivers"
)
// DriverHandle wraps operations to a driver such that they are operated on a specific
// task
type DriverHandle interface {
// ID returns the task ID
ID() string
// WaitCh is used to return a channel used to wait for task completion
WaitCh(context.Context) (<-chan *drivers.ExitResult, error)
// Update is used to update the task if possible and update task related
// configurations.
Update(task *structs.Task) error
// Kill is used to stop the task
Kill() error
// Stats returns aggregated stats of the driver
Stats() (*cstructs.TaskResourceUsage, error)
// Signal is used to send a signal to the task
Signal(s string) error
// ScriptExecutor is an interface used to execute commands such as
// health check scripts in the a DriverHandle's context.
ScriptExecutor
// Network returns the driver's network or nil if the driver did not
// create a network.
Network() *cstructs.DriverNetwork
}
// ScriptExecutor is an interface that supports Exec()ing commands in the
// driver's context. Split out of DriverHandle to ease testing.
type ScriptExecutor interface {
Exec(timeout time.Duration, cmd string, args []string) ([]byte, int, error)
}

View File

@@ -2,13 +2,12 @@ package interfaces
import (
"context"
"os"
"github.com/hashicorp/nomad/nomad/structs"
)
type TaskLifecycle interface {
Restart(ctx context.Context, event *structs.TaskEvent, failure bool) error
Signal(event *structs.TaskEvent, s os.Signal) error
Signal(event *structs.TaskEvent, signal string) error
Kill(ctx context.Context, event *structs.TaskEvent) error
}

View File

@@ -2,17 +2,16 @@ package taskrunner
import (
"context"
"os"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/drivers"
)
// Restart a task. Returns immediately if no task is running. Blocks until
// existing task exits or passed-in context is canceled.
func (tr *TaskRunner) Restart(ctx context.Context, event *structs.TaskEvent, failure bool) error {
// Grab the handle
handle, result := tr.getDriverHandle()
handle := tr.getDriverHandle()
// Check it is running
if handle == nil {
return ErrTaskNotRunning
@@ -32,13 +31,17 @@ func (tr *TaskRunner) Restart(ctx context.Context, event *structs.TaskEvent, fai
}
// Drain the wait channel or wait for the request context to be canceled
result.Wait(ctx)
waitCh, err := handle.WaitCh(ctx)
if err != nil {
tr.logger.Error("failed to kill task. Resources may have been leaked", "error", err)
}
<-waitCh
return nil
}
func (tr *TaskRunner) Signal(event *structs.TaskEvent, s os.Signal) error {
func (tr *TaskRunner) Signal(event *structs.TaskEvent, s string) error {
// Grab the handle
handle, _ := tr.getDriverHandle()
handle := tr.getDriverHandle()
// Check it is running
if handle == nil {
@@ -49,6 +52,7 @@ func (tr *TaskRunner) Signal(event *structs.TaskEvent, s os.Signal) error {
tr.EmitEvent(event)
// Send the signal
return handle.Signal(s)
}
@@ -60,9 +64,9 @@ func (tr *TaskRunner) Kill(ctx context.Context, event *structs.TaskEvent) error
tr.ctxCancel()
// Grab the handle
handle, result := tr.getDriverHandle()
handle := tr.getDriverHandle()
// Check if the handle is running
// Check it is running
if handle == nil {
return ErrTaskNotRunning
}
@@ -84,7 +88,17 @@ func (tr *TaskRunner) Kill(ctx context.Context, event *structs.TaskEvent) error
}
// Block until task has exited.
result.Wait(ctx)
waitCh, err := handle.WaitCh(ctx)
// The task may have already been cleaned up
if err != nil && err != drivers.ErrTaskNotFound {
tr.logger.Error("failed to wait on task. Resources may have been leaked", "error", err)
return err
}
if waitCh != nil {
<-waitCh
}
// Store that the task has been destroyed and any associated error.
tr.UpdateState(structs.TaskStateDead, structs.NewTaskEvent(structs.TaskKilled).SetKillError(destroyErr))

View File

@@ -6,8 +6,8 @@ import (
"sync"
"time"
dstructs "github.com/hashicorp/nomad/client/driver/structs"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/drivers"
)
const (
@@ -34,7 +34,7 @@ func NewRestartTracker(policy *structs.RestartPolicy, jobType string) *RestartTr
}
type RestartTracker struct {
waitRes *dstructs.WaitResult
exitRes *drivers.ExitResult
startErr error
killed bool // Whether the task has been killed
restartTriggered bool // Whether the task has been signalled to be restarted
@@ -72,11 +72,11 @@ func (r *RestartTracker) SetStartError(err error) *RestartTracker {
return r
}
// SetWaitResult is used to mark the most recent wait result.
func (r *RestartTracker) SetWaitResult(res *dstructs.WaitResult) *RestartTracker {
// SetExitResult is used to mark the most recent wait result.
func (r *RestartTracker) SetExitResult(res *drivers.ExitResult) *RestartTracker {
r.lock.Lock()
defer r.lock.Unlock()
r.waitRes = res
r.exitRes = res
r.failure = true
return r
}
@@ -112,6 +112,13 @@ func (r *RestartTracker) GetReason() string {
return r.reason
}
// GetCount returns the current restart count
func (r *RestartTracker) GetCount() int {
r.lock.Lock()
defer r.lock.Unlock()
return r.count
}
// GetState returns the tasks next state given the set exit code and start
// error. One of the following states are returned:
// * TaskRestarting - Task should be restarted
@@ -129,7 +136,7 @@ func (r *RestartTracker) GetState() (string, time.Duration) {
// Clear out the existing state
defer func() {
r.startErr = nil
r.waitRes = nil
r.exitRes = nil
r.restartTriggered = false
r.failure = false
r.killed = false
@@ -153,7 +160,7 @@ func (r *RestartTracker) GetState() (string, time.Duration) {
// If the task does not restart on a successful exit code and
// the exit code was successful: terminate.
if !r.onSuccess && r.waitRes != nil && r.waitRes.Successful() {
if !r.onSuccess && r.exitRes != nil && r.exitRes.Successful() {
return structs.TaskTerminated, 0
}
@@ -183,10 +190,10 @@ func (r *RestartTracker) GetState() (string, time.Duration) {
r.reason = ReasonUnrecoverableErrror
return structs.TaskNotRestarting, 0
}
} else if r.waitRes != nil {
} else if r.exitRes != nil {
// If the task started successfully and restart on success isn't specified,
// don't restart but don't mark as failed.
if r.waitRes.Successful() && !r.onSuccess {
if r.exitRes.Successful() && !r.onSuccess {
r.reason = "Restart unnecessary as task terminated successfully"
return structs.TaskTerminated, 0
}

View File

@@ -7,8 +7,8 @@ import (
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
tinterfaces "github.com/hashicorp/nomad/client/allocrunner/taskrunner/interfaces"
"github.com/hashicorp/nomad/client/consul"
"github.com/hashicorp/nomad/client/driver"
"github.com/hashicorp/nomad/client/driver/env"
cstructs "github.com/hashicorp/nomad/client/structs"
agentconsul "github.com/hashicorp/nomad/command/agent/consul"
@@ -34,7 +34,7 @@ type serviceHook struct {
logger log.Logger
// The following fields may be updated
driverExec driver.ScriptExecutor
driverExec tinterfaces.ScriptExecutor
driverNet *cstructs.DriverNetwork
canary bool
services []*structs.Service

View File

@@ -3,6 +3,7 @@ package state
import (
"github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/helper"
"github.com/hashicorp/nomad/plugins/drivers"
)
// LocalState is Task state which is persisted for use when restarting Nomad
@@ -13,6 +14,9 @@ type LocalState struct {
// DriverNetwork is the network information returned by the task
// driver's Start method
DriverNetwork *structs.DriverNetwork
// TaskHandle is the handle used to reattach to the task during recovery
TaskHandle *drivers.TaskHandle
}
func NewLocalState() *LocalState {
@@ -27,6 +31,7 @@ func (s *LocalState) Copy() *LocalState {
c := &LocalState{
Hooks: make(map[string]*HookState, len(s.Hooks)),
DriverNetwork: s.DriverNetwork,
TaskHandle: s.TaskHandle,
}
// Copy the hooks

View File

@@ -38,7 +38,7 @@ func (h *taskDirHook) Prestart(ctx context.Context, req *interfaces.TaskPrestart
h.runner.EmitEvent(structs.NewTaskEvent(structs.TaskSetup).SetMessage(structs.TaskBuildingTaskDir))
// Build the task directory structure
fsi := h.runner.driver.FSIsolation()
fsi := h.runner.driverCapabilities.FSIsolation
err := h.runner.taskDir.Build(false, chroot, fsi)
if err != nil {
return err

View File

@@ -1,7 +1,7 @@
package taskrunner
import (
"github.com/hashicorp/nomad/client/driver"
"github.com/hashicorp/nomad/client/allocrunnerv2/taskrunner/interfaces"
"github.com/hashicorp/nomad/nomad/structs"
)
@@ -51,23 +51,24 @@ func (tr *TaskRunner) setVaultToken(token string) {
// getDriverHandle returns a driver handle and its result proxy. Use the
// result proxy instead of the handle's WaitCh.
func (tr *TaskRunner) getDriverHandle() (driver.DriverHandle, *handleResult) {
func (tr *TaskRunner) getDriverHandle() interfaces.DriverHandle {
tr.handleLock.Lock()
defer tr.handleLock.Unlock()
return tr.handle, tr.handleResult
return tr.handle
}
// setDriverHanlde sets the driver handle and creates a new result proxy.
func (tr *TaskRunner) setDriverHandle(handle driver.DriverHandle) {
func (tr *TaskRunner) setDriverHandle(handle interfaces.DriverHandle) {
tr.handleLock.Lock()
defer tr.handleLock.Unlock()
tr.handle = handle
tr.handleResult = newHandleResult(handle.WaitCh())
}
func (tr *TaskRunner) clearDriverHandle() {
tr.handleLock.Lock()
defer tr.handleLock.Unlock()
if tr.handle != nil {
tr.driver.DestroyTask(tr.handle.ID(), true)
}
tr.handle = nil
tr.handleResult = nil
}

View File

@@ -168,7 +168,7 @@ func (tr *TaskRunner) poststart() error {
}()
}
handle, _ := tr.getDriverHandle()
handle := tr.getDriverHandle()
net := handle.Network()
var merr multierror.Error

View File

@@ -448,7 +448,7 @@ func (tm *TaskTemplateManager) handleTemplateRerenders(allRenderedTime time.Time
for signal := range signals {
s := tm.signals[signal]
event := structs.NewTaskEvent(structs.TaskSignaling).SetTaskSignal(s).SetDisplayMessage("Template re-rendered")
if err := tm.config.Lifecycle.Signal(event, s); err != nil {
if err := tm.config.Lifecycle.Signal(event, signal); err != nil {
multierror.Append(&mErr, err)
}
}

View File

@@ -240,7 +240,7 @@ OUTER:
}
event := structs.NewTaskEvent(structs.TaskSignaling).SetTaskSignal(s).SetDisplayMessage("Vault: new Vault token acquired")
if err := h.lifecycle.Signal(event, s); err != nil {
if err := h.lifecycle.Signal(event, h.vaultStanza.ChangeSignal); err != nil {
h.logger.Error("failed to send signal", "error", err)
h.lifecycle.Kill(h.ctx,
structs.NewTaskEvent(structs.TaskKilling).

View File

@@ -0,0 +1,550 @@
package allocrunnerv2
import (
"context"
"fmt"
"path/filepath"
"sync"
"time"
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/allocrunner"
"github.com/hashicorp/nomad/client/allocrunnerv2/interfaces"
"github.com/hashicorp/nomad/client/allocrunnerv2/state"
"github.com/hashicorp/nomad/client/allocrunnerv2/taskrunner"
"github.com/hashicorp/nomad/client/allocwatcher"
"github.com/hashicorp/nomad/client/config"
"github.com/hashicorp/nomad/client/consul"
cinterfaces "github.com/hashicorp/nomad/client/interfaces"
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/shared/loader"
)
// allocRunner is used to run all the tasks in a given allocation
type allocRunner struct {
// id is the ID of the allocation. Can be accessed without a lock
id string
// Logger is the logger for the alloc runner.
logger log.Logger
clientConfig *config.Config
// stateUpdater is used to emit updated task state
stateUpdater cinterfaces.AllocStateHandler
// consulClient is the client used by the consul service hook for
// registering services and checks
consulClient consul.ConsulServiceAPI
// vaultClient is the used to manage Vault tokens
vaultClient vaultclient.VaultClient
// waitCh is closed when the Run() loop has exited
waitCh chan struct{}
// destroyed is true when the Run() loop has exited, postrun hooks have
// run, and alloc runner has been destroyed
destroyed bool
destroyedLock sync.Mutex
// Alloc captures the allocation being run.
alloc *structs.Allocation
allocLock sync.RWMutex
// state is the alloc runner's state
state *state.State
stateLock sync.RWMutex
stateDB cstate.StateDB
// allocDir is used to build the allocations directory structure.
allocDir *allocdir.AllocDir
// runnerHooks are alloc runner lifecycle hooks that should be run on state
// transistions.
runnerHooks []interfaces.RunnerHook
// tasks are the set of task runners
tasks map[string]*taskrunner.TaskRunner
tasksLock sync.RWMutex
// allocBroadcaster sends client allocation updates to all listeners
allocBroadcaster *cstructs.AllocBroadcaster
// prevAllocWatcher allows waiting for a previous allocation to exit
// and if necessary migrate its alloc dir.
prevAllocWatcher allocwatcher.PrevAllocWatcher
// 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
}
// NewAllocRunner returns a new allocation runner.
func NewAllocRunner(config *Config) (*allocRunner, error) {
alloc := config.Alloc
tg := alloc.Job.LookupTaskGroup(alloc.TaskGroup)
if tg == nil {
return nil, fmt.Errorf("failed to lookup task group %q", alloc.TaskGroup)
}
ar := &allocRunner{
id: alloc.ID,
alloc: alloc,
clientConfig: config.ClientConfig,
consulClient: config.Consul,
vaultClient: config.Vault,
tasks: make(map[string]*taskrunner.TaskRunner, len(tg.Tasks)),
waitCh: make(chan struct{}),
state: &state.State{},
stateDB: config.StateDB,
stateUpdater: config.StateUpdater,
allocBroadcaster: cstructs.NewAllocBroadcaster(),
prevAllocWatcher: config.PrevAllocWatcher,
pluginLoader: config.PluginLoader,
pluginSingletonLoader: config.PluginSingletonLoader,
}
// Create the logger based on the allocation ID
ar.logger = config.Logger.Named("alloc_runner").With("alloc_id", alloc.ID)
// Create alloc dir
ar.allocDir = allocdir.NewAllocDir(ar.logger, filepath.Join(config.ClientConfig.AllocDir, alloc.ID))
// Initialize the runners hooks.
ar.initRunnerHooks()
// Create the TaskRunners
if err := ar.initTaskRunners(tg.Tasks); err != nil {
return nil, err
}
return ar, nil
}
// initTaskRunners creates task runners but does *not* run them.
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,
VaultClient: ar.vaultClient,
PluginLoader: ar.pluginLoader,
PluginSingletonLoader: ar.pluginSingletonLoader,
}
// Create, but do not Run, the task runner
tr, err := taskrunner.NewTaskRunner(config)
if err != nil {
return fmt.Errorf("failed creating runner for task %q: %v", task.Name, err)
}
ar.tasks[task.Name] = tr
}
return nil
}
func (ar *allocRunner) WaitCh() <-chan struct{} {
return ar.waitCh
}
// XXX How does alloc Restart work
// Run is the main goroutine that executes all the tasks.
func (ar *allocRunner) Run() {
// Close the wait channel
defer close(ar.waitCh)
var taskWaitCh <-chan struct{}
// Run the prestart hooks
// XXX Equivalent to TR.Prestart hook
if err := ar.prerun(); err != nil {
ar.logger.Error("prerun failed", "error", err)
goto POST
}
// Run the runners
taskWaitCh = ar.runImpl()
MAIN:
for {
select {
case <-taskWaitCh:
// TaskRunners have all exited
break MAIN
}
}
POST:
// Run the postrun hooks
// XXX Equivalent to TR.Poststop hook
if err := ar.postrun(); err != nil {
ar.logger.Error("postrun failed", "error", err)
}
}
// runImpl is used to run the runners.
func (ar *allocRunner) runImpl() <-chan struct{} {
for _, task := range ar.tasks {
go task.Run()
}
// Return a combined WaitCh that is closed when all task runners have
// exited.
waitCh := make(chan struct{})
go func() {
defer close(waitCh)
for _, task := range ar.tasks {
<-task.WaitCh()
}
}()
return waitCh
}
// Alloc returns the current allocation being run by this runner as sent by the
// server. This view of the allocation does not have updated task states.
func (ar *allocRunner) Alloc() *structs.Allocation {
ar.allocLock.RLock()
defer ar.allocLock.RUnlock()
return ar.alloc
}
func (ar *allocRunner) setAlloc(updated *structs.Allocation) {
ar.allocLock.Lock()
ar.alloc = updated
ar.allocLock.Unlock()
}
// GetAllocDir returns the alloc dir which is safe for concurrent use.
func (ar *allocRunner) GetAllocDir() *allocdir.AllocDir {
return ar.allocDir
}
// Restore state from database. Must be called after NewAllocRunner but before
// Run.
func (ar *allocRunner) Restore() error {
// Restore task runners
for _, tr := range ar.tasks {
if err := tr.Restore(); err != nil {
return err
}
}
return nil
}
// TaskStateUpdated is called by TaskRunner when a task's state has been
// updated. This hook is used to compute changes to the alloc's ClientStatus
// and to update the server with the new state.
func (ar *allocRunner) TaskStateUpdated(taskName string, state *structs.TaskState) {
// If a task is dead, we potentially want to kill other tasks in the group
if state.State == structs.TaskStateDead {
// Find all tasks that are not the one that is dead and check if the one
// that is dead is a leader
var otherTaskRunners []*taskrunner.TaskRunner
var otherTaskNames []string
leader := false
for name, tr := range ar.tasks {
if name != taskName {
otherTaskRunners = append(otherTaskRunners, tr)
otherTaskNames = append(otherTaskNames, name)
} else if tr.Task().Leader {
leader = true
}
}
// If the task failed, we should kill all the other tasks in the task group.
if state.Failed {
if len(otherTaskRunners) > 0 {
ar.logger.Debug("task failure, destroying all tasks", "failed_task", taskName, "destroying", otherTaskNames)
}
for _, tr := range otherTaskRunners {
tr.Kill(context.Background(), structs.NewTaskEvent(structs.TaskSiblingFailed).SetFailedSibling(taskName))
}
} else if leader {
if len(otherTaskRunners) > 0 {
ar.logger.Debug("leader task dead, destroying all tasks", "leader_task", taskName, "destroying", otherTaskNames)
}
// If the task was a leader task we should kill all the other tasks.
for _, tr := range otherTaskRunners {
tr.Kill(context.Background(), structs.NewTaskEvent(structs.TaskLeaderDead))
}
}
}
// Gather the state of the other tasks
ar.tasksLock.RLock()
states := make(map[string]*structs.TaskState, len(ar.tasks))
for name, tr := range ar.tasks {
if name == taskName {
states[name] = state
} else {
states[name] = tr.TaskState()
}
}
ar.tasksLock.RUnlock()
// Get the client allocation
calloc := ar.clientAlloc(states)
// Update the server
ar.stateUpdater.AllocStateUpdated(calloc)
// Broadcast client alloc to listeners
ar.allocBroadcaster.Send(calloc)
}
// clientAlloc takes in the task states and returns an Allocation populated
// with Client specific fields
func (ar *allocRunner) clientAlloc(taskStates map[string]*structs.TaskState) *structs.Allocation {
ar.stateLock.RLock()
defer ar.stateLock.RUnlock()
// store task states for AllocState to expose
ar.state.TaskStates = taskStates
a := &structs.Allocation{
ID: ar.id,
TaskStates: taskStates,
}
if d := ar.state.DeploymentStatus; d != nil {
a.DeploymentStatus = d.Copy()
}
// Compute the ClientStatus
if ar.state.ClientStatus != "" {
// The client status is being forced
a.ClientStatus, a.ClientDescription = ar.state.ClientStatus, ar.state.ClientDescription
} else {
a.ClientStatus, a.ClientDescription = getClientStatus(taskStates)
}
// If the allocation is terminal, make sure all required fields are properly
// set.
if a.ClientTerminalStatus() {
alloc := ar.Alloc()
// If we are part of a deployment and the task has failed, mark the
// alloc as unhealthy. This guards against the watcher not be started.
if a.ClientStatus == structs.AllocClientStatusFailed &&
alloc.DeploymentID != "" && !a.DeploymentStatus.IsUnhealthy() {
a.DeploymentStatus = &structs.AllocDeploymentStatus{
Healthy: helper.BoolToPtr(false),
}
}
// Make sure we have marked the finished at for every task. This is used
// to calculate the reschedule time for failed allocations.
now := time.Now()
for _, task := range alloc.Job.LookupTaskGroup(alloc.TaskGroup).Tasks {
ts, ok := a.TaskStates[task.Name]
if !ok {
ts = &structs.TaskState{}
a.TaskStates[task.Name] = ts
}
if ts.FinishedAt.IsZero() {
ts.FinishedAt = now
}
}
}
return a
}
// getClientStatus takes in the task states for a given allocation and computes
// the client status and description
func getClientStatus(taskStates map[string]*structs.TaskState) (status, description string) {
var pending, running, dead, failed bool
for _, state := range taskStates {
switch state.State {
case structs.TaskStateRunning:
running = true
case structs.TaskStatePending:
pending = true
case structs.TaskStateDead:
if state.Failed {
failed = true
} else {
dead = true
}
}
}
// Determine the alloc status
if failed {
return structs.AllocClientStatusFailed, "Failed tasks"
} else if running {
return structs.AllocClientStatusRunning, "Tasks are running"
} else if pending {
return structs.AllocClientStatusPending, "No tasks have started"
} else if dead {
return structs.AllocClientStatusComplete, "All tasks have completed"
}
return "", ""
}
// AllocState returns a copy of allocation state including a snapshot of task
// states.
func (ar *allocRunner) AllocState() *state.State {
ar.stateLock.RLock()
state := ar.state.Copy()
ar.stateLock.RUnlock()
// If TaskStateUpdated has not been called yet, ar.state.TaskStates
// won't be set as it is not the canonical source of TaskStates.
if len(state.TaskStates) == 0 {
ar.tasksLock.RLock()
ar.state.TaskStates = make(map[string]*structs.TaskState, len(ar.tasks))
for k, tr := range ar.tasks {
state.TaskStates[k] = tr.TaskState()
}
ar.tasksLock.RUnlock()
}
return state
}
// Update the running allocation with a new version received from the server.
//
// This method sends the updated alloc to Run for serially processing updates.
// If there is already a pending update it will be discarded and replaced by
// the latest update.
func (ar *allocRunner) Update(update *structs.Allocation) {
// Update ar.alloc
ar.setAlloc(update)
// Run hooks
if err := ar.update(update); err != nil {
ar.logger.Error("error running update hooks", "error", err)
}
// Update task runners
for _, tr := range ar.tasks {
tr.Update(update)
}
}
func (ar *allocRunner) Listener() *cstructs.AllocListener {
return ar.allocBroadcaster.Listen()
}
// Destroy the alloc runner by synchronously stopping it if it is still running
// and cleaning up all of its resources.
//
// This method is safe for calling concurrently with Run() and will cause it to
// exit (thus closing WaitCh).
func (ar *allocRunner) Destroy() {
// Stop tasks
ar.tasksLock.RLock()
for name, tr := range ar.tasks {
err := tr.Kill(context.TODO(), structs.NewTaskEvent(structs.TaskKilled))
if err != nil {
if err == taskrunner.ErrTaskNotRunning {
ar.logger.Trace("task not running", "task_name", name)
} else {
ar.logger.Warn("failed to kill task", "error", err, "task_name", name)
}
}
}
ar.tasksLock.RUnlock()
// Wait for tasks to exit and postrun hooks to finish
<-ar.waitCh
// Run destroy hooks
if err := ar.destroy(); err != nil {
ar.logger.Warn("error running destroy hooks", "error", err)
}
// Cleanup state db
if err := ar.stateDB.DeleteAllocationBucket(ar.id); err != nil {
ar.logger.Warn("failed to delete allocation state", "error", err)
}
// Mark alloc as destroyed
ar.destroyedLock.Lock()
ar.destroyed = true
ar.destroyedLock.Unlock()
}
// IsDestroyed returns true if the alloc runner has been destroyed (stopped and
// garbage collected).
//
// This method is safe for calling concurrently with Run(). Callers must
// receive on WaitCh() to block until alloc runner has stopped and been
// destroyed.
func (ar *allocRunner) IsDestroyed() bool {
ar.destroyedLock.Lock()
defer ar.destroyedLock.Unlock()
return ar.destroyed
}
// IsWaiting returns true if the alloc runner is waiting for its previous
// allocation to terminate.
//
// This method is safe for calling concurrently with Run().
func (ar *allocRunner) IsWaiting() bool {
return ar.prevAllocWatcher.IsWaiting()
}
// IsMigrating returns true if the alloc runner is migrating data from its
// previous allocation.
//
// This method is safe for calling concurrently with Run().
func (ar *allocRunner) IsMigrating() bool {
return ar.prevAllocWatcher.IsMigrating()
}
func (ar *allocRunner) StatsReporter() allocrunner.AllocStatsReporter {
return ar
}
// LatestAllocStats returns the latest stats for an allocation. If taskFilter
// is set, only stats for that task -- if it exists -- are returned.
func (ar *allocRunner) LatestAllocStats(taskFilter string) (*cstructs.AllocResourceUsage, error) {
ar.tasksLock.RLock()
defer ar.tasksLock.RUnlock()
astat := &cstructs.AllocResourceUsage{
Tasks: make(map[string]*cstructs.TaskResourceUsage, len(ar.tasks)),
ResourceUsage: &cstructs.ResourceUsage{
MemoryStats: &cstructs.MemoryStats{},
CpuStats: &cstructs.CpuStats{},
},
}
for name, tr := range ar.tasks {
if taskFilter != "" && taskFilter != name {
// Getting stats for a particular task and its not this one!
continue
}
if usage := tr.LatestResourceUsage(); usage != nil {
astat.Tasks[name] = usage
astat.ResourceUsage.Add(usage.ResourceUsage)
if usage.Timestamp > astat.Timestamp {
astat.Timestamp = usage.Timestamp
}
}
}
return astat, nil
}

View File

@@ -0,0 +1,903 @@
package taskrunner
import (
"context"
"errors"
"fmt"
"sync"
"time"
metrics "github.com/armon/go-metrics"
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/hcl2/hcl"
"github.com/hashicorp/hcl2/hcldec"
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/allocrunner/taskrunner/restarts"
"github.com/hashicorp/nomad/client/allocrunnerv2/interfaces"
tinterfaces "github.com/hashicorp/nomad/client/allocrunnerv2/taskrunner/interfaces"
"github.com/hashicorp/nomad/client/allocrunnerv2/taskrunner/state"
"github.com/hashicorp/nomad/client/config"
"github.com/hashicorp/nomad/client/consul"
"github.com/hashicorp/nomad/client/driver/env"
cstate "github.com/hashicorp/nomad/client/state"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/client/vaultclient"
"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"
"github.com/zclconf/go-cty/cty"
)
const (
// killBackoffBaseline is the baseline time for exponential backoff while
// killing a task.
killBackoffBaseline = 5 * time.Second
// killBackoffLimit is the limit of the exponential backoff for killing
// the task.
killBackoffLimit = 2 * time.Minute
// killFailureLimit is how many times we will attempt to kill a task before
// giving up and potentially leaking resources.
killFailureLimit = 5
// triggerUpdatechCap is the capacity for the triggerUpdateCh used for
// triggering updates. It should be exactly 1 as even if multiple
// updates have come in since the last one was handled, we only need to
// handle the last one.
triggerUpdateChCap = 1
)
type TaskRunner struct {
// allocID and taskName are immutable so these fields may be accessed
// without locks
allocID string
taskName string
alloc *structs.Allocation
allocLock sync.Mutex
clientConfig *config.Config
// stateUpdater is used to emit updated task state
stateUpdater interfaces.TaskStateHandler
// state captures the state of the task for updating the allocation
state *structs.TaskState
stateLock sync.Mutex
// localState captures the node-local state of the task for when the
// Nomad agent restarts
localState *state.LocalState
localStateLock sync.RWMutex
// stateDB is for persisting localState and taskState
stateDB cstate.StateDB
// persistedHash is the hash of the last persisted state for skipping
// unnecessary writes
persistedHash []byte
// ctx is the task runner's context representing the tasks's lifecycle.
// Canceling the context will cause the task to be destroyed.
ctx context.Context
// ctxCancel is used to exit the task runner's Run loop without
// stopping the task. Shutdown hooks are run.
ctxCancel context.CancelFunc
// Logger is the logger for the task runner.
logger log.Logger
// triggerUpdateCh is ticked whenever update hooks need to be run and
// must be created with cap=1 to signal a pending update and prevent
// callers from deadlocking if the receiver has exited.
triggerUpdateCh chan struct{}
// waitCh is closed when the task runner has transitioned to a terminal
// state
waitCh chan struct{}
// driver is the driver for the task.
driver drivers.DriverPlugin
// driverCapabilities is the set capabilities the driver supports
driverCapabilities *drivers.Capabilities
// taskSchema is the hcl spec for the task driver configuration
taskSchema hcldec.Spec
// handleLock guards access to handle and handleResult
handleLock sync.Mutex
// handle to the running driver
handle tinterfaces.DriverHandle
// network is the configuration for the driver network if one was created
network *cstructs.DriverNetwork
// task is the task being run
task *structs.Task
taskLock sync.RWMutex
// taskDir is the directory structure for this task.
taskDir *allocdir.TaskDir
// envBuilder is used to build the task's environment
envBuilder *env.Builder
// restartTracker is used to decide if the task should be restarted.
restartTracker *restarts.RestartTracker
// runnerHooks are task runner lifecycle hooks that should be run on state
// transistions.
runnerHooks []interfaces.TaskHook
// consulClient is the client used by the consul service hook for
// registering services and checks
consulClient consul.ConsulServiceAPI
// vaultClient is the client to use to derive and renew Vault tokens
vaultClient vaultclient.VaultClient
// vaultToken is the current Vault token. It should be accessed with the
// getter.
vaultToken string
vaultTokenLock sync.Mutex
// baseLabels are used when emitting tagged metrics. All task runner metrics
// will have these tags, and optionally more.
baseLabels []metrics.Label
// logmonHookConfig is used to get the paths to the stdout and stderr fifos
// to be passed to the driver for task logging
logmonHookConfig *logmonHookConfig
// resourceUsage is written via UpdateStats and read via
// LatestResourceUsage. May be nil at all times.
resourceUsage *cstructs.TaskResourceUsage
resourceUsageLock sync.Mutex
// 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
}
type Config struct {
Alloc *structs.Allocation
ClientConfig *config.Config
Consul consul.ConsulServiceAPI
Task *structs.Task
TaskDir *allocdir.TaskDir
Logger log.Logger
// VaultClient is the client to use to derive and renew Vault tokens
VaultClient vaultclient.VaultClient
// LocalState is optionally restored task state
LocalState *state.LocalState
// StateDB is used to store and restore state.
StateDB cstate.StateDB
// StateUpdater is used to emit updated task state
StateUpdater interfaces.TaskStateHandler
// 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
}
func NewTaskRunner(config *Config) (*TaskRunner, error) {
// Create a context for the runner
trCtx, trCancel := context.WithCancel(context.Background())
// Initialize the environment builder
envBuilder := env.NewBuilder(
config.ClientConfig.Node,
config.Alloc,
config.Task,
config.ClientConfig.Region,
)
tr := &TaskRunner{
alloc: config.Alloc,
allocID: config.Alloc.ID,
clientConfig: config.ClientConfig,
task: config.Task,
taskDir: config.TaskDir,
taskName: config.Task.Name,
envBuilder: envBuilder,
consulClient: config.Consul,
vaultClient: config.VaultClient,
state: config.Alloc.TaskStates[config.Task.Name].Copy(),
localState: config.LocalState,
stateDB: config.StateDB,
stateUpdater: config.StateUpdater,
ctx: trCtx,
ctxCancel: trCancel,
triggerUpdateCh: make(chan struct{}, triggerUpdateChCap),
waitCh: make(chan struct{}),
pluginLoader: config.PluginLoader,
pluginSingletonLoader: config.PluginSingletonLoader,
}
// Create the logger based on the allocation ID
tr.logger = config.Logger.Named("task_runner").With("task", config.Task.Name)
// Build the restart tracker.
tg := tr.alloc.Job.LookupTaskGroup(tr.alloc.TaskGroup)
if tg == nil {
tr.logger.Error("alloc missing task group")
return nil, fmt.Errorf("alloc missing task group")
}
tr.restartTracker = restarts.NewRestartTracker(tg.RestartPolicy, tr.alloc.Job.Type)
// Initialize the task state
tr.initState()
// Get the driver
if err := tr.initDriver(); err != nil {
tr.logger.Error("failed to create driver", "error", err)
return nil, err
}
// Initialize the runners hooks.
tr.initHooks()
// Initialize base labels
tr.initLabels()
return tr, nil
}
func (tr *TaskRunner) initState() {
if tr.state == nil {
tr.state = &structs.TaskState{
State: structs.TaskStatePending,
}
}
if tr.localState == nil {
tr.localState = state.NewLocalState()
}
}
func (tr *TaskRunner) initLabels() {
alloc := tr.Alloc()
tr.baseLabels = []metrics.Label{
{
Name: "job",
Value: alloc.Job.Name,
},
{
Name: "task_group",
Value: alloc.TaskGroup,
},
{
Name: "alloc_id",
Value: tr.allocID,
},
{
Name: "task",
Value: tr.taskName,
},
}
}
func (tr *TaskRunner) Run() {
defer close(tr.waitCh)
var result *drivers.ExitResult
// Updates are handled asynchronously with the other hooks but each
// triggered update - whether due to alloc updates or a new vault token
// - should be handled serially.
go tr.handleUpdates()
MAIN:
for tr.ctx.Err() == nil {
// Run the prestart hooks
if err := tr.prestart(); err != nil {
tr.logger.Error("prestart failed", "error", err)
tr.restartTracker.SetStartError(err)
goto RESTART
}
if tr.ctx.Err() != nil {
break MAIN
}
// Run the task
if err := tr.runDriver(); err != nil {
tr.logger.Error("running driver failed", "error", err)
tr.restartTracker.SetStartError(err)
goto RESTART
}
// Run the poststart hooks
if err := tr.poststart(); err != nil {
tr.logger.Error("poststart failed", "error", err)
}
// Grab the result proxy and wait for task to exit
{
handle := tr.getDriverHandle()
// Do *not* use tr.ctx here as it would cause Wait() to
// unblock before the task exits when Kill() is called.
if resultCh, err := handle.WaitCh(context.Background()); err != nil {
tr.logger.Error("wait task failed", "error", err)
} else {
result = <-resultCh
}
}
// Clear the handle
tr.clearDriverHandle()
// Store the wait result on the restart tracker
tr.restartTracker.SetExitResult(result)
if err := tr.exited(); err != nil {
tr.logger.Error("exited hooks failed", "error", err)
}
RESTART:
restart, restartDelay := tr.shouldRestart()
if !restart {
break MAIN
}
// Actually restart by sleeping and also watching for destroy events
select {
case <-time.After(restartDelay):
case <-tr.ctx.Done():
tr.logger.Trace("task killed between restarts", "delay", restartDelay)
break MAIN
}
}
// If task terminated, update server. All other exit conditions (eg
// killed or out of restarts) will perform their own server updates.
if result != nil {
event := structs.NewTaskEvent(structs.TaskTerminated).
SetExitCode(result.ExitCode).
SetSignal(result.Signal).
SetExitMessage(result.Err)
tr.UpdateState(structs.TaskStateDead, event)
}
// Run the stop hooks
if err := tr.stop(); err != nil {
tr.logger.Error("stop failed", "error", err)
}
tr.logger.Debug("task run loop exiting")
}
// handleUpdates runs update hooks when triggerUpdateCh is ticked and exits
// when Run has returned. Should only be run in a goroutine from Run.
func (tr *TaskRunner) handleUpdates() {
for {
select {
case <-tr.triggerUpdateCh:
case <-tr.waitCh:
return
}
if tr.Alloc().TerminalStatus() {
// Terminal update: kill TaskRunner and let Run execute postrun hooks
err := tr.Kill(context.TODO(), structs.NewTaskEvent(structs.TaskKilled))
if err != nil {
tr.logger.Warn("error stopping task", "error", err)
}
continue
}
// Non-terminal update; run hooks
tr.updateHooks()
}
}
// shouldRestart determines whether the task should be restarted and updates
// the task state unless the task is killed or terminated.
func (tr *TaskRunner) shouldRestart() (bool, time.Duration) {
// Determine if we should restart
state, when := tr.restartTracker.GetState()
reason := tr.restartTracker.GetReason()
switch state {
case structs.TaskKilled:
// Never restart an explicitly killed task. Kill method handles
// updating the server.
return false, 0
case structs.TaskNotRestarting, structs.TaskTerminated:
tr.logger.Info("not restarting task", "reason", reason)
if state == structs.TaskNotRestarting {
tr.UpdateState(structs.TaskStateDead, structs.NewTaskEvent(structs.TaskNotRestarting).SetRestartReason(reason).SetFailsTask())
}
return false, 0
case structs.TaskRestarting:
tr.logger.Info("restarting task", "reason", reason, "delay", when)
tr.UpdateState(structs.TaskStatePending, structs.NewTaskEvent(structs.TaskRestarting).SetRestartDelay(when).SetRestartReason(reason))
return true, 0
default:
tr.logger.Error("restart tracker returned unknown state", "state", state)
return true, when
}
}
// runDriver runs the driver and waits for it to exit
func (tr *TaskRunner) runDriver() error {
taskConfig := drivers.NewTaskConfig(tr.task, tr.taskDir, tr.envBuilder.Build())
taskConfig.ID = tr.buildID()
taskConfig.StdoutPath = tr.logmonHookConfig.stdoutFifo
taskConfig.StderrPath = tr.logmonHookConfig.stderrFifo
evalCtx := &hcl.EvalContext{
Functions: shared.GetStdlibFuncs(),
Variables: map[string]cty.Value{
"NOMAD_ENV_bin": cty.StringVal("/bin/consul"),
},
}
val, diag := shared.ParseHclInterface(tr.task.Config, tr.taskSchema, evalCtx)
if diag.HasErrors() {
errStr := "failed to parse config"
for _, err := range diag.Errs() {
errStr = fmt.Sprintf("%s\n* %s", errStr, err.Error())
}
return errors.New(errStr)
}
if err := taskConfig.EncodeDriverConfig(val); err != nil {
tr.logger.Warn("failed to encode driver config", "error", err)
return err
}
//TODO mounts and devices
//XXX Evaluate and encode driver config
// Start the job
handle, net, err := tr.driver.StartTask(taskConfig)
if err != nil {
tr.logger.Warn("driver start failed", "error", err)
return err
}
tr.network = net
tr.localStateLock.Lock()
tr.localState.TaskHandle = handle
tr.localStateLock.Unlock()
tr.updateDriverHandle(taskConfig.ID)
// Emit an event that we started
tr.UpdateState(structs.TaskStateRunning, structs.NewTaskEvent(structs.TaskStarted))
return nil
}
func (tr *TaskRunner) updateDriverHandle(taskID string) {
tr.handleLock.Lock()
defer tr.handleLock.Unlock()
tr.handle = &driverHandleImpl{
driver: tr.driver,
net: tr.network,
taskID: taskID,
task: tr.Task(),
}
}
// 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.logger)
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()
if err != nil {
return err
}
spec, _ := hclspec.Convert(schema)
tr.taskSchema = spec
caps, err := tr.driver.Capabilities()
if err != nil {
return err
}
tr.driverCapabilities = caps
return nil
}
// handleDestroy kills the task handle. In the case that killing fails,
// handleDestroy will retry with an exponential backoff and will give up at a
// given limit. It returns whether the task was destroyed and the error
// associated with the last kill attempt.
func (tr *TaskRunner) handleDestroy(handle tinterfaces.DriverHandle) (destroyed bool, err error) {
// Cap the number of times we attempt to kill the task.
for i := 0; i < killFailureLimit; i++ {
if err = handle.Kill(); err != nil {
if err == drivers.ErrTaskNotFound {
tr.logger.Warn("couldn't find task to kill", "task_id", handle.ID())
return true, nil
}
// Calculate the new backoff
backoff := (1 << (2 * uint64(i))) * killBackoffBaseline
if backoff > killBackoffLimit {
backoff = killBackoffLimit
}
tr.logger.Error("failed to kill task", "backoff", backoff, "error", err)
time.Sleep(backoff)
} else {
// Kill was successful
return true, nil
}
}
return
}
// persistLocalState persists local state to disk synchronously.
func (tr *TaskRunner) persistLocalState() error {
tr.localStateLock.Lock()
defer tr.localStateLock.Unlock()
return tr.stateDB.PutTaskRunnerLocalState(tr.allocID, tr.taskName, tr.localState)
}
// buildID builds a consistent unique ID for the task from the alloc ID, task name and restart attempt
func (tr *TaskRunner) buildID() string {
return fmt.Sprintf("%s/%s/%d", tr.allocID, tr.taskName, tr.restartTracker.GetCount())
}
// XXX If the objects don't exists since the client shutdown before the task
// runner ever saved state, then we should treat it as a new task runner and not
// return an error
//
// Restore task runner state. Called by AllocRunner.Restore after NewTaskRunner
// but before Run so no locks need to be acquired.
func (tr *TaskRunner) Restore() error {
ls, ts, err := tr.stateDB.GetTaskRunnerState(tr.allocID, tr.taskName)
if err != nil {
return err
}
tr.localState = ls
tr.state = ts
return nil
}
// UpdateState sets the task runners allocation state and triggers a server
// update.
func (tr *TaskRunner) UpdateState(state string, event *structs.TaskEvent) {
tr.logger.Debug("setting task state", "state", state, "event", event.Type)
// Update the local state
stateCopy := tr.setStateLocal(state, event)
// Notify the alloc runner of the transition
tr.stateUpdater.TaskStateUpdated(tr.taskName, stateCopy)
}
// setStateLocal updates the local in-memory state, persists a copy to disk and returns a
// copy of the task's state.
func (tr *TaskRunner) setStateLocal(state string, event *structs.TaskEvent) *structs.TaskState {
tr.stateLock.Lock()
defer tr.stateLock.Unlock()
//XXX REMOVE ME AFTER TESTING
if state == "" {
panic("UpdateState must not be called with an empty state")
}
// Update the task state
oldState := tr.state.State
taskState := tr.state
taskState.State = state
// Append the event
tr.appendEvent(event)
// Handle the state transition.
switch state {
case structs.TaskStateRunning:
// Capture the start time if it is just starting
if oldState != structs.TaskStateRunning {
taskState.StartedAt = time.Now().UTC()
if !tr.clientConfig.DisableTaggedMetrics {
metrics.IncrCounterWithLabels([]string{"client", "allocs", "running"}, 1, tr.baseLabels)
}
//if r.config.BackwardsCompatibleMetrics {
//metrics.IncrCounter([]string{"client", "allocs", r.alloc.Job.Name, r.alloc.TaskGroup, taskName, "running"}, 1)
//}
}
case structs.TaskStateDead:
// Capture the finished time if not already set
if taskState.FinishedAt.IsZero() {
taskState.FinishedAt = time.Now().UTC()
}
// Emitting metrics to indicate task complete and failures
if taskState.Failed {
if !tr.clientConfig.DisableTaggedMetrics {
metrics.IncrCounterWithLabels([]string{"client", "allocs", "failed"}, 1, tr.baseLabels)
}
//if r.config.BackwardsCompatibleMetrics {
//metrics.IncrCounter([]string{"client", "allocs", r.alloc.Job.Name, r.alloc.TaskGroup, taskName, "failed"}, 1)
//}
} else {
if !tr.clientConfig.DisableTaggedMetrics {
metrics.IncrCounterWithLabels([]string{"client", "allocs", "complete"}, 1, tr.baseLabels)
}
//if r.config.BackwardsCompatibleMetrics {
//metrics.IncrCounter([]string{"client", "allocs", r.alloc.Job.Name, r.alloc.TaskGroup, taskName, "complete"}, 1)
//}
}
}
// Persist the state and event
if err := tr.stateDB.PutTaskState(tr.allocID, tr.taskName, taskState); err != nil {
// Only a warning because the next event/state-transition will
// try to persist it again.
tr.logger.Error("error persisting task state", "error", err, "event", event, "state", state)
}
return tr.state.Copy()
}
// EmitEvent appends a new TaskEvent to this task's TaskState. The actual
// TaskState.State (pending, running, dead) is not changed. Use UpdateState to
// transition states.
// Events are persisted locally and sent to the server, but errors are simply
// logged. Use AppendEvent to simply add a new event.
func (tr *TaskRunner) EmitEvent(event *structs.TaskEvent) {
tr.stateLock.Lock()
defer tr.stateLock.Unlock()
tr.appendEvent(event)
if err := tr.stateDB.PutTaskState(tr.allocID, tr.taskName, tr.state); err != nil {
// Only a warning because the next event/state-transition will
// try to persist it again.
tr.logger.Warn("error persisting event", "error", err, "event", event)
}
// Notify the alloc runner of the event
tr.stateUpdater.TaskStateUpdated(tr.taskName, tr.state.Copy())
}
// AppendEvent appends a new TaskEvent to this task's TaskState. The actual
// TaskState.State (pending, running, dead) is not changed. Use UpdateState to
// transition states.
// Events are persisted locally and errors are simply logged. Use EmitEvent
// also update AllocRunner.
func (tr *TaskRunner) AppendEvent(event *structs.TaskEvent) {
tr.stateLock.Lock()
defer tr.stateLock.Unlock()
tr.appendEvent(event)
if err := tr.stateDB.PutTaskState(tr.allocID, tr.taskName, tr.state); err != nil {
// Only a warning because the next event/state-transition will
// try to persist it again.
tr.logger.Warn("error persisting event", "error", err, "event", event)
}
}
// appendEvent to task's event slice. Caller must acquire stateLock.
func (tr *TaskRunner) appendEvent(event *structs.TaskEvent) error {
// Ensure the event is populated with human readable strings
event.PopulateEventDisplayMessage()
// Propogate failure from event to task state
if event.FailsTask {
tr.state.Failed = true
}
// XXX This seems like a super awkward spot for this? Why not shouldRestart?
// Update restart metrics
if event.Type == structs.TaskRestarting {
if !tr.clientConfig.DisableTaggedMetrics {
metrics.IncrCounterWithLabels([]string{"client", "allocs", "restart"}, 1, tr.baseLabels)
}
//if r.config.BackwardsCompatibleMetrics {
//metrics.IncrCounter([]string{"client", "allocs", r.alloc.Job.Name, r.alloc.TaskGroup, taskName, "restart"}, 1)
//}
tr.state.Restarts++
tr.state.LastRestart = time.Unix(0, event.Time)
}
// Append event to slice
appendTaskEvent(tr.state, event)
return nil
}
// WaitCh is closed when TaskRunner.Run exits.
func (tr *TaskRunner) WaitCh() <-chan struct{} {
return tr.waitCh
}
// Update the running allocation with a new version received from the server.
// Calls Update hooks asynchronously with Run().
//
// This method is safe for calling concurrently with Run() and does not modify
// the passed in allocation.
func (tr *TaskRunner) Update(update *structs.Allocation) {
// Update tr.alloc
tr.setAlloc(update)
// Trigger update hooks
tr.triggerUpdateHooks()
}
// triggerUpdate if there isn't already an update pending. Should be called
// instead of calling updateHooks directly to serialize runs of update hooks.
// TaskRunner state should be updated prior to triggering update hooks.
//
// Does not block.
func (tr *TaskRunner) triggerUpdateHooks() {
select {
case tr.triggerUpdateCh <- struct{}{}:
default:
// already an update hook pending
}
}
// LatestResourceUsage returns the last resource utilization datapoint
// collected. May return nil if the task is not running or no resource
// utilization has been collected yet.
func (tr *TaskRunner) LatestResourceUsage() *cstructs.TaskResourceUsage {
tr.resourceUsageLock.Lock()
ru := tr.resourceUsage
tr.resourceUsageLock.Unlock()
return ru
}
// UpdateStats updates and emits the latest stats from the driver.
func (tr *TaskRunner) UpdateStats(ru *cstructs.TaskResourceUsage) {
tr.resourceUsageLock.Lock()
tr.resourceUsage = ru
tr.resourceUsageLock.Unlock()
if ru != nil {
tr.emitStats(ru)
}
}
//TODO Remove Backwardscompat or use tr.Alloc()?
func (tr *TaskRunner) setGaugeForMemory(ru *cstructs.TaskResourceUsage) {
if !tr.clientConfig.DisableTaggedMetrics {
metrics.SetGaugeWithLabels([]string{"client", "allocs", "memory", "rss"},
float32(ru.ResourceUsage.MemoryStats.RSS), tr.baseLabels)
metrics.SetGaugeWithLabels([]string{"client", "allocs", "memory", "rss"},
float32(ru.ResourceUsage.MemoryStats.RSS), tr.baseLabels)
metrics.SetGaugeWithLabels([]string{"client", "allocs", "memory", "cache"},
float32(ru.ResourceUsage.MemoryStats.Cache), tr.baseLabels)
metrics.SetGaugeWithLabels([]string{"client", "allocs", "memory", "swap"},
float32(ru.ResourceUsage.MemoryStats.Swap), tr.baseLabels)
metrics.SetGaugeWithLabels([]string{"client", "allocs", "memory", "max_usage"},
float32(ru.ResourceUsage.MemoryStats.MaxUsage), tr.baseLabels)
metrics.SetGaugeWithLabels([]string{"client", "allocs", "memory", "kernel_usage"},
float32(ru.ResourceUsage.MemoryStats.KernelUsage), tr.baseLabels)
metrics.SetGaugeWithLabels([]string{"client", "allocs", "memory", "kernel_max_usage"},
float32(ru.ResourceUsage.MemoryStats.KernelMaxUsage), tr.baseLabels)
}
if tr.clientConfig.BackwardsCompatibleMetrics {
metrics.SetGauge([]string{"client", "allocs", tr.alloc.Job.Name, tr.alloc.TaskGroup, tr.allocID, tr.taskName, "memory", "rss"}, float32(ru.ResourceUsage.MemoryStats.RSS))
metrics.SetGauge([]string{"client", "allocs", tr.alloc.Job.Name, tr.alloc.TaskGroup, tr.allocID, tr.taskName, "memory", "cache"}, float32(ru.ResourceUsage.MemoryStats.Cache))
metrics.SetGauge([]string{"client", "allocs", tr.alloc.Job.Name, tr.alloc.TaskGroup, tr.allocID, tr.taskName, "memory", "swap"}, float32(ru.ResourceUsage.MemoryStats.Swap))
metrics.SetGauge([]string{"client", "allocs", tr.alloc.Job.Name, tr.alloc.TaskGroup, tr.allocID, tr.taskName, "memory", "max_usage"}, float32(ru.ResourceUsage.MemoryStats.MaxUsage))
metrics.SetGauge([]string{"client", "allocs", tr.alloc.Job.Name, tr.alloc.TaskGroup, tr.allocID, tr.taskName, "memory", "kernel_usage"}, float32(ru.ResourceUsage.MemoryStats.KernelUsage))
metrics.SetGauge([]string{"client", "allocs", tr.alloc.Job.Name, tr.alloc.TaskGroup, tr.allocID, tr.taskName, "memory", "kernel_max_usage"}, float32(ru.ResourceUsage.MemoryStats.KernelMaxUsage))
}
}
//TODO Remove Backwardscompat or use tr.Alloc()?
func (tr *TaskRunner) setGaugeForCPU(ru *cstructs.TaskResourceUsage) {
if !tr.clientConfig.DisableTaggedMetrics {
metrics.SetGaugeWithLabels([]string{"client", "allocs", "cpu", "total_percent"},
float32(ru.ResourceUsage.CpuStats.Percent), tr.baseLabels)
metrics.SetGaugeWithLabels([]string{"client", "allocs", "cpu", "system"},
float32(ru.ResourceUsage.CpuStats.SystemMode), tr.baseLabels)
metrics.SetGaugeWithLabels([]string{"client", "allocs", "cpu", "user"},
float32(ru.ResourceUsage.CpuStats.UserMode), tr.baseLabels)
metrics.SetGaugeWithLabels([]string{"client", "allocs", "cpu", "throttled_time"},
float32(ru.ResourceUsage.CpuStats.ThrottledTime), tr.baseLabels)
metrics.SetGaugeWithLabels([]string{"client", "allocs", "cpu", "throttled_periods"},
float32(ru.ResourceUsage.CpuStats.ThrottledPeriods), tr.baseLabels)
metrics.SetGaugeWithLabels([]string{"client", "allocs", "cpu", "total_ticks"},
float32(ru.ResourceUsage.CpuStats.TotalTicks), tr.baseLabels)
}
if tr.clientConfig.BackwardsCompatibleMetrics {
metrics.SetGauge([]string{"client", "allocs", tr.alloc.Job.Name, tr.alloc.TaskGroup, tr.allocID, tr.taskName, "cpu", "total_percent"}, float32(ru.ResourceUsage.CpuStats.Percent))
metrics.SetGauge([]string{"client", "allocs", tr.alloc.Job.Name, tr.alloc.TaskGroup, tr.allocID, tr.taskName, "cpu", "system"}, float32(ru.ResourceUsage.CpuStats.SystemMode))
metrics.SetGauge([]string{"client", "allocs", tr.alloc.Job.Name, tr.alloc.TaskGroup, tr.allocID, tr.taskName, "cpu", "user"}, float32(ru.ResourceUsage.CpuStats.UserMode))
metrics.SetGauge([]string{"client", "allocs", tr.alloc.Job.Name, tr.alloc.TaskGroup, tr.allocID, tr.taskName, "cpu", "throttled_time"}, float32(ru.ResourceUsage.CpuStats.ThrottledTime))
metrics.SetGauge([]string{"client", "allocs", tr.alloc.Job.Name, tr.alloc.TaskGroup, tr.allocID, tr.taskName, "cpu", "throttled_periods"}, float32(ru.ResourceUsage.CpuStats.ThrottledPeriods))
metrics.SetGauge([]string{"client", "allocs", tr.alloc.Job.Name, tr.alloc.TaskGroup, tr.allocID, tr.taskName, "cpu", "total_ticks"}, float32(ru.ResourceUsage.CpuStats.TotalTicks))
}
}
// emitStats emits resource usage stats of tasks to remote metrics collector
// sinks
func (tr *TaskRunner) emitStats(ru *cstructs.TaskResourceUsage) {
if !tr.clientConfig.PublishAllocationMetrics {
return
}
if ru.ResourceUsage.MemoryStats != nil {
tr.setGaugeForMemory(ru)
}
if ru.ResourceUsage.CpuStats != nil {
tr.setGaugeForCPU(ru)
}
}
// appendTaskEvent updates the task status by appending the new event.
func appendTaskEvent(state *structs.TaskState, event *structs.TaskEvent) {
const capacity = 10
if state.Events == nil {
state.Events = make([]*structs.TaskEvent, 1, capacity)
state.Events[0] = event
return
}
// If we hit capacity, then shift it.
if len(state.Events) == capacity {
old := state.Events
state.Events = make([]*structs.TaskEvent, 0, capacity)
state.Events = append(state.Events, old[1:]...)
}
state.Events = append(state.Events, event)
}

View File

@@ -297,7 +297,7 @@ func NewClient(cfg *config.Config, consulCatalog consul.CatalogAPI, consulServic
c.configCopy = c.config.Copy()
c.configLock.Unlock()
fingerprintManager := NewFingerprintManager(c.GetConfig, c.configCopy.Node,
fingerprintManager := NewFingerprintManager(c.configCopy.PluginSingletonLoader, c.GetConfig, c.configCopy.Node,
c.shutdownCh, c.updateNodeFromFingerprint, c.updateNodeFromDriver,
c.logger)
@@ -761,14 +761,16 @@ func (c *Client) restoreState() error {
for _, alloc := range allocs {
c.configLock.RLock()
arConf := &allocrunner.Config{
Alloc: alloc,
Logger: c.logger,
ClientConfig: c.config,
StateDB: c.stateDB,
StateUpdater: c,
Consul: c.consulService,
Vault: c.vaultClient,
arConf := &allocrunnerv2.Config{
Alloc: alloc,
Logger: c.logger,
ClientConfig: c.config,
StateDB: c.stateDB,
StateUpdater: c,
Consul: c.consulService,
Vault: c.vaultClient,
PluginLoader: c.config.PluginLoader,
PluginSingletonLoader: c.config.PluginSingletonLoader,
}
c.configLock.RUnlock()
@@ -1032,31 +1034,45 @@ func (c *Client) updateNodeFromFingerprint(response *cstructs.FingerprintRespons
return c.configCopy.Node
}
// updateNodeFromDriver receives either a fingerprint of the driver or its
// health and merges this into a single DriverInfo object
func (c *Client) updateNodeFromDriver(name string, fingerprint, health *structs.DriverInfo) *structs.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()
var hasChanged bool
hadDriver := c.config.Node.Drivers[name] != nil
if fingerprint != nil {
if info != nil {
if !hadDriver {
// If the driver info has not yet been set, do that here
hasChanged = true
c.config.Node.Drivers[name] = fingerprint
for attrName, newVal := range fingerprint.Attributes {
c.config.Node.Drivers[name] = info
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 c.config.Node.Drivers[name].Detected != fingerprint.Detected {
if oldVal.Detected != info.Detected {
hasChanged = true
c.config.Node.Drivers[name].Detected = fingerprint.Detected
c.config.Node.Drivers[name].Detected = info.Detected
}
for attrName, newVal := range fingerprint.Attributes {
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
@@ -1075,48 +1091,12 @@ func (c *Client) updateNodeFromDriver(name string, fingerprint, health *structs.
// We maintain the driver enabled attribute until all drivers expose
// their attributes as DriverInfo
driverName := fmt.Sprintf("driver.%s", name)
if fingerprint.Detected {
if info.Detected {
c.config.Node.Attributes[driverName] = "1"
} else {
delete(c.config.Node.Attributes, driverName)
}
}
if health != nil {
if !hadDriver {
hasChanged = true
if info, ok := c.config.Node.Drivers[name]; !ok {
c.config.Node.Drivers[name] = health
} else {
info.MergeHealthCheck(health)
}
} else {
oldVal := c.config.Node.Drivers[name]
if health.HealthCheckEquals(oldVal) {
// Make sure we accurately reflect the last time a health check has been
// performed for the driver.
oldVal.UpdateTime = health.UpdateTime
} else {
hasChanged = true
// Only emit an event if the health status has changed after node
// initial startup (the health description will not get populated until
// a health check has run; the initial status is equal to whether the
// node is detected or not).
if health.Healthy != oldVal.Healthy && health.HealthDescription != "" {
event := &structs.NodeEvent{
Subsystem: "Driver",
Message: health.HealthDescription,
Timestamp: time.Now(),
Details: map[string]string{"driver": name},
}
c.triggerNodeEvent(event)
}
// Update the node with the latest information
c.config.Node.Drivers[name].MergeHealthCheck(health)
}
}
}
if hasChanged {
@@ -1931,15 +1911,17 @@ func (c *Client) addAlloc(alloc *structs.Allocation, migrateToken string) error
// The long term fix is to pass in the config and node separately and then
// we don't have to do a copy.
c.configLock.RLock()
arConf := &allocrunner.Config{
Alloc: alloc,
Logger: c.logger,
ClientConfig: c.config,
StateDB: c.stateDB,
Consul: c.consulService,
Vault: c.vaultClient,
StateUpdater: c,
PrevAllocWatcher: prevAllocWatcher,
arConf := &allocrunnerv2.Config{
Alloc: alloc,
Logger: c.logger,
ClientConfig: c.config,
StateDB: c.stateDB,
Consul: c.consulService,
Vault: c.vaultClient,
StateUpdater: c,
PrevAllocWatcher: prevAllocWatcher,
PluginLoader: c.config.PluginLoader,
PluginSingletonLoader: c.config.PluginSingletonLoader,
}
c.configLock.RUnlock()

View File

@@ -1,25 +1,29 @@
package client
import (
"context"
"fmt"
"sync"
"time"
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/config"
"github.com/hashicorp/nomad/client/driver"
"github.com/hashicorp/nomad/client/fingerprint"
cstructs "github.com/hashicorp/nomad/client/structs"
"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"
)
// FingerprintManager runs a client fingerprinters on a continuous basis, and
// updates the client when the node has changed
type FingerprintManager struct {
getConfig func() *config.Config
node *structs.Node
nodeLock sync.Mutex
shutdownCh chan struct{}
singletonLoader loader.PluginCatalog
getConfig func() *config.Config
node *structs.Node
nodeLock sync.Mutex
shutdownCh chan struct{}
// updateNodeAttributes is a callback to the client to update the state of its
// associated node
@@ -27,20 +31,23 @@ type FingerprintManager struct {
// updateNodeFromDriver is a callback to the client to update the state of a
// specific driver for the node
updateNodeFromDriver func(string, *structs.DriverInfo, *structs.DriverInfo) *structs.Node
updateNodeFromDriver func(string, *structs.DriverInfo) *structs.Node
logger log.Logger
}
// NewFingerprintManager is a constructor that creates and returns an instance
// of FingerprintManager
func NewFingerprintManager(getConfig func() *config.Config,
func NewFingerprintManager(
singletonLoader loader.PluginCatalog,
getConfig func() *config.Config,
node *structs.Node,
shutdownCh chan struct{},
updateNodeAttributes func(*cstructs.FingerprintResponse) *structs.Node,
updateNodeFromDriver func(string, *structs.DriverInfo, *structs.DriverInfo) *structs.Node,
updateNodeFromDriver func(string, *structs.DriverInfo) *structs.Node,
logger log.Logger) *FingerprintManager {
return &FingerprintManager{
singletonLoader: singletonLoader,
getConfig: getConfig,
updateNodeAttributes: updateNodeAttributes,
updateNodeFromDriver: updateNodeFromDriver,
@@ -111,8 +118,13 @@ func (fp *FingerprintManager) Run() error {
var availDrivers []string
var skippedDrivers []string
var registeredDrivers []string
for name := range driver.BuiltinDrivers {
for _, pl := range fp.singletonLoader.Catalog()[base.PluginTypeDriver] {
registeredDrivers = append(registeredDrivers, pl.Name)
}
for _, name := range registeredDrivers {
// Skip fingerprinting drivers that are not in the whitelist if it is
// enabled.
if _, ok := whitelistDrivers[name]; whitelistDriversEnabled && !ok {
@@ -173,35 +185,41 @@ func (fm *FingerprintManager) setupFingerprinters(fingerprints []string) error {
// setupDrivers is used to fingerprint the node to see if these drivers are
// supported
func (fm *FingerprintManager) setupDrivers(drivers []string) error {
func (fm *FingerprintManager) setupDrivers(driverNames []string) error {
//TODO(alex,hclog) Update fingerprinters to hclog
var availDrivers []string
driverCtx := driver.NewDriverContext("", "", "", "", fm.getConfig(), fm.getNode(), fm.logger.StandardLogger(&log.StandardLoggerOptions{InferLevels: true}), nil)
for _, name := range drivers {
d, err := driver.NewDriver(name, driverCtx)
for _, name := range driverNames {
// TODO: driver reattach
plug, err := fm.singletonLoader.Dispense(name, base.PluginTypeDriver, fm.logger)
if err != nil {
return err
}
driver, ok := plug.Plugin().(drivers.DriverPlugin)
if !ok {
return fmt.Errorf("registered driver plugin %q does not implement DriverPlugin interface")
}
// Pass true for whether the health check is periodic here, so that the
// fingerprinter will not set the initial health check status (this is set
// below, with an empty health status so that a node event is not
// triggered)
// Later, the periodic health checker will update this value for drivers
// where health checks are enabled.
detected, err := fm.fingerprintDriver(name, d, true)
ctx, cancel := context.WithCancel(context.Background())
fingerCh, err := driver.Fingerprint(ctx)
if err != nil {
fm.logger.Debug("error fingerprinting driver", "error", err, "fingerprinter", name)
return err
}
finger := <-fingerCh
// Start a periodic watcher to detect changes to a drivers health and
// attributes.
go fm.watchDriver(d, name)
go fm.watchDriverFingerprint(fingerCh, name, cancel)
// Log the fingerprinters which have been applied
if detected {
if finger.Health != drivers.HealthStateUndetected {
availDrivers = append(availDrivers, name)
}
}
@@ -258,154 +276,26 @@ func (fm *FingerprintManager) fingerprint(name string, f fingerprint.Fingerprint
// watchDrivers facilitates the different periods between fingerprint and
// health checking a driver
func (fm *FingerprintManager) watchDriver(d driver.Driver, name string) {
var fingerprintTicker, healthTicker <-chan time.Time
// Determine whether the fingerprinter is periodic and health checking
isPeriodic, fingerprintPeriod := d.Periodic()
hc, isHealthCheck := d.(fingerprint.HealthCheck)
// Nothing to do since the state of this driver will never change
if !isPeriodic && !isHealthCheck {
return
}
// Setup the required tickers
if isPeriodic {
ticker := time.NewTicker(fingerprintPeriod)
fingerprintTicker = ticker.C
defer ticker.Stop()
fm.logger.Debug("periodically fingerprinting driver", "name", name, "period", fingerprintPeriod)
}
var isHealthCheckPeriodic bool
if isHealthCheck {
// Determine the interval at which to health check
req := &cstructs.HealthCheckIntervalRequest{}
var healthCheckResp cstructs.HealthCheckIntervalResponse
if err := hc.GetHealthCheckInterval(req, &healthCheckResp); err != nil {
fm.logger.Error("error getting health check interval for driver", "error", err, "driver", name)
} else if healthCheckResp.Eligible {
isHealthCheckPeriodic = true
ticker := time.NewTicker(healthCheckResp.Period)
healthTicker = ticker.C
defer ticker.Stop()
fm.logger.Debug("periodically health checking driver", "driver", name, "period", healthCheckResp.Period)
}
}
driverEverDetected := false
func (fm *FingerprintManager) watchDriverFingerprint(fpChan <-chan *drivers.Fingerprint, name string, cancel context.CancelFunc) {
for {
select {
case <-fm.shutdownCh:
cancel()
return
case <-fingerprintTicker:
if _, err := fm.fingerprintDriver(name, d, isHealthCheckPeriodic); err != nil {
fm.logger.Debug("error periodic fingerprinting driver", "error", err, "driver", name)
case fp := <-fpChan:
di := &structs.DriverInfo{
Attributes: fp.Attributes,
Detected: fp.Health != drivers.HealthStateUndetected,
Healthy: fp.Health == drivers.HealthStateHealthy,
HealthDescription: fp.HealthDescription,
UpdateTime: time.Now(),
}
fm.nodeLock.Lock()
driver, detected := fm.node.Drivers[name]
// Memoize the driver detected status, so that we know whether to run the
// health check or not.
if detected && driver != nil && driver.Detected {
if !driverEverDetected {
driverEverDetected = true
}
n := fm.updateNodeFromDriver(name, di)
if n != nil {
fm.node = n
}
fm.nodeLock.Unlock()
case <-healthTicker:
if driverEverDetected {
if err := fm.runDriverHealthCheck(name, hc); err != nil {
fm.logger.Debug("error health checking", "error", err, "driver", name)
}
}
}
}
}
// fingerprintDriver is a temporary solution to move towards DriverInfo and
// away from annotating a node's attributes to demonstrate support for a
// particular driver. Takes the FingerprintResponse and converts it to the
// proper DriverInfo update and then sets the prefix attributes as well
func (fm *FingerprintManager) fingerprintDriver(name string, f fingerprint.Fingerprint, hasPeriodicHealthCheck bool) (bool, error) {
var response cstructs.FingerprintResponse
fm.nodeLock.Lock()
// Determine if the driver has been detected before.
originalNode, haveDriver := fm.node.Drivers[name]
firstDetection := !haveDriver
// Determine if the driver is healthy
var driverIsHealthy bool
if haveDriver && originalNode.Healthy {
driverIsHealthy = true
}
// Fingerprint the driver.
request := &cstructs.FingerprintRequest{Config: fm.getConfig(), Node: fm.node}
err := f.Fingerprint(request, &response)
fm.nodeLock.Unlock()
if err != nil {
return false, err
}
// Remove the health check attribute indicating the status of the driver,
// as the overall driver info object should indicate this.
delete(response.Attributes, fmt.Sprintf("driver.%s", name))
fingerprintInfo := &structs.DriverInfo{
Attributes: response.Attributes,
Detected: response.Detected,
}
// We set the health status based on the detection state of the driver if:
// * It is the first time we are fingerprinting the driver. This gives all
// drivers an initial health.
// * If the driver becomes undetected. This gives us an immediate unhealthy
// state and description when it transistions from detected and healthy to
// undetected.
// * If the driver does not have its own health checks. Then we always
// couple the states.
var healthInfo *structs.DriverInfo
if firstDetection || !hasPeriodicHealthCheck || !response.Detected && driverIsHealthy {
state := " "
if !response.Detected {
state = " not "
}
healthInfo = &structs.DriverInfo{
Healthy: response.Detected,
HealthDescription: fmt.Sprintf("Driver %s is%sdetected", name, state),
UpdateTime: time.Now(),
}
}
if node := fm.updateNodeFromDriver(name, fingerprintInfo, healthInfo); node != nil {
fm.setNode(node)
}
return response.Detected, nil
}
// runDriverHealthCheck checks the health of the specified resource.
func (fm *FingerprintManager) runDriverHealthCheck(name string, hc fingerprint.HealthCheck) error {
request := &cstructs.HealthCheckRequest{}
var response cstructs.HealthCheckResponse
if err := hc.HealthCheck(request, &response); err != nil {
return err
}
// Update the status of the node irregardless if there was an error- in the
// case of periodic health checks, an error will occur if a health check
// fails
if node := fm.updateNodeFromDriver(name, nil, response.Drivers[name]); node != nil {
fm.setNode(node)
}
return nil
}

View File

@@ -572,9 +572,7 @@ func DevConfig() *Config {
}
conf.Client.Options = map[string]string{
"driver.raw_exec.enable": "true",
}
conf.Client.Options = map[string]string{
"driver.docker.volumes": "true",
"driver.docker.volumes": "true",
}
conf.Client.GCInterval = 10 * time.Minute
conf.Client.GCDiskUsageThreshold = 99

View File

@@ -8,7 +8,7 @@ import (
log "github.com/hashicorp/go-hclog"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/nomad/client/driver"
"github.com/hashicorp/nomad/client/allocrunnerv2/taskrunner/interfaces"
"github.com/hashicorp/nomad/nomad/structs"
)
@@ -39,7 +39,7 @@ type scriptCheck struct {
id string
check *structs.ServiceCheck
exec driver.ScriptExecutor
exec interfaces.ScriptExecutor
agent heartbeater
// lastCheckOk is true if the last check was ok; otherwise false
@@ -52,7 +52,7 @@ type scriptCheck struct {
// newScriptCheck creates a new scriptCheck. run() should be called once the
// initial check is registered with Consul.
func newScriptCheck(allocID, taskName, checkID string, check *structs.ServiceCheck,
exec driver.ScriptExecutor, agent heartbeater, logger log.Logger,
exec interfaces.ScriptExecutor, agent heartbeater, logger log.Logger,
shutdownCh <-chan struct{}) *scriptCheck {
logger = logger.ResetNamed("consul.checks").With("task", taskName, "alloc_id", allocID, "check", check.Name)
@@ -93,9 +93,8 @@ func (s *scriptCheck) run() *scriptHandle {
metrics.IncrCounter([]string{"client", "consul", "script_runs"}, 1)
// Execute check script with timeout
execctx, cancel := context.WithTimeout(ctx, s.check.Timeout)
output, code, err := s.exec.Exec(execctx, s.check.Command, s.check.Args)
switch execctx.Err() {
output, code, err := s.exec.Exec(s.check.Timeout, s.check.Command, s.check.Args)
switch err {
case context.Canceled:
// check removed during execution; exit
cancel()

View File

@@ -1,7 +1,7 @@
package consul
import (
"github.com/hashicorp/nomad/client/driver"
"github.com/hashicorp/nomad/client/allocrunnerv2/taskrunner/interfaces"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
)
@@ -26,13 +26,13 @@ type TaskServices struct {
Networks structs.Networks
// DriverExec is the script executor for the task's driver.
DriverExec driver.ScriptExecutor
DriverExec interfaces.ScriptExecutor
// DriverNetwork is the network specified by the driver and may be nil.
DriverNetwork *cstructs.DriverNetwork
}
func NewTaskServices(alloc *structs.Allocation, task *structs.Task, restarter TaskRestarter, exec driver.ScriptExecutor, net *cstructs.DriverNetwork) *TaskServices {
func NewTaskServices(alloc *structs.Allocation, task *structs.Task, restarter TaskRestarter, exec interfaces.ScriptExecutor, net *cstructs.DriverNetwork) *TaskServices {
ts := TaskServices{
AllocID: alloc.ID,
Name: task.Name,

View File

@@ -6,6 +6,8 @@ import (
"github.com/hashicorp/nomad/plugins/shared/catalog"
"github.com/hashicorp/nomad/plugins/shared/loader"
"github.com/hashicorp/nomad/plugins/shared/singleton"
_ "github.com/hashicorp/nomad/drivers/rawexec"
)
// setupPlugins is used to setup the plugin loaders.

View File

@@ -12,14 +12,40 @@ import (
plugin "github.com/hashicorp/go-plugin"
"github.com/hashicorp/nomad/client/driver/executor"
dstructs "github.com/hashicorp/nomad/client/driver/structs"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/drivers/shared/eventer"
"github.com/hashicorp/nomad/plugins/base"
"github.com/hashicorp/nomad/plugins/drivers"
"github.com/hashicorp/nomad/plugins/drivers/utils"
"github.com/hashicorp/nomad/plugins/shared/catalog"
"github.com/hashicorp/nomad/plugins/shared/hclspec"
"github.com/hashicorp/nomad/plugins/shared/loader"
"golang.org/x/net/context"
)
func init() {
catalog.RegisterDeferredConfig(loader.PluginID{
Name: pluginName,
PluginType: base.PluginTypeDriver,
}, &loader.InternalPluginConfig{
Config: map[string]interface{}{},
Factory: func(l hclog.Logger) interface{} { return NewRawExecDriver(l) },
},
func(opts map[string]string) (map[string]interface{}, error) {
fmt.Println(opts)
conf := map[string]interface{}{}
if v, ok := opts["driver.raw_exec.enable"]; ok && v == "true" {
conf["enabled"] = true
}
if v, ok := opts["driver.raw_exec.no_cgroups"]; ok && v == "true" {
conf["no_cgroups"] = true
}
return conf, nil
},
)
}
const (
// pluginName is the name of the plugin
pluginName = "raw_exec"
@@ -53,7 +79,7 @@ var (
// a task within a job. It is returned in the TaskConfigSchema RPC
taskConfigSpec = hclspec.NewObject(map[string]*hclspec.Spec{
"command": hclspec.NewAttr("command", "string", true),
"args": hclspec.NewAttr("command", "list(string)", false),
"args": hclspec.NewAttr("args", "list(string)", false),
})
// capabilities is returned by the Capabilities RPC and indicates what
@@ -61,7 +87,7 @@ var (
capabilities = &drivers.Capabilities{
SendSignals: true,
Exec: true,
FSIsolation: drivers.FSIsolationNone,
FSIsolation: cstructs.FSIsolationNone,
}
)
@@ -96,16 +122,16 @@ type RawExecDriver struct {
type Config struct {
// NoCgroups tracks whether we should use a cgroup to manage the process
// tree
NoCgroups bool `codec:"no_cgroups"`
NoCgroups bool `codec:"no_cgroups" cty:"no_cgroups"`
// Enabled is set to true to enable the raw_exec driver
Enabled bool `codec:"enabled"`
Enabled bool `codec:"enabled" cty:"enabled"`
}
// TaskConfig is the driver configuration of a task within a job
type TaskConfig struct {
Command string `codec:"command"`
Args []string `codec:"args"`
Command string `codec:"command" cty:"command"`
Args []string `codec:"args" cty:"args"`
}
// RawExecTaskState is the state which is encoded in the handle returned in
@@ -247,16 +273,17 @@ func (r *RawExecDriver) RecoverTask(handle *drivers.TaskHandle) error {
return nil
}
func (r *RawExecDriver) StartTask(cfg *drivers.TaskConfig) (*drivers.TaskHandle, error) {
func (r *RawExecDriver) StartTask(cfg *drivers.TaskConfig) (*drivers.TaskHandle, *cstructs.DriverNetwork, error) {
if _, ok := r.tasks.Get(cfg.ID); ok {
return nil, fmt.Errorf("task with ID %q already started", cfg.ID)
return nil, nil, fmt.Errorf("task with ID %q already started", cfg.ID)
}
var driverConfig TaskConfig
if err := cfg.DecodeDriverConfig(&driverConfig); err != nil {
return nil, fmt.Errorf("failed to decode driver config: %v", err)
return nil, nil, fmt.Errorf("failed to decode driver config: %v", err)
}
r.logger.Info("starting task", "driver_cfg", hclog.Fmt("%+v", driverConfig))
handle := drivers.NewTaskHandle(pluginName)
handle.Config = cfg
@@ -269,7 +296,7 @@ func (r *RawExecDriver) StartTask(cfg *drivers.TaskConfig) (*drivers.TaskHandle,
// TODO: best way to pass port ranges in from client config
exec, pluginClient, err := utils.CreateExecutor(os.Stderr, hclog.Debug, 14000, 14512, executorConfig)
if err != nil {
return nil, fmt.Errorf("failed to create executor: %v", err)
return nil, nil, fmt.Errorf("failed to create executor: %v", err)
}
execCmd := &executor.ExecCommand{
@@ -286,7 +313,7 @@ func (r *RawExecDriver) StartTask(cfg *drivers.TaskConfig) (*drivers.TaskHandle,
ps, err := exec.Launch(execCmd)
if err != nil {
pluginClient.Kill()
return nil, fmt.Errorf("failed to launch command with executor: %v", err)
return nil, nil, fmt.Errorf("failed to launch command with executor: %v", err)
}
h := &rawExecTaskHandle{
@@ -310,12 +337,12 @@ func (r *RawExecDriver) StartTask(cfg *drivers.TaskConfig) (*drivers.TaskHandle,
r.logger.Error("failed to start task, error setting driver state", "error", err)
exec.Shutdown("", 0)
pluginClient.Kill()
return nil, fmt.Errorf("failed to set driver state: %v", err)
return nil, nil, fmt.Errorf("failed to set driver state: %v", err)
}
r.tasks.Set(cfg.ID, h)
go h.run()
return handle, nil
return handle, nil, nil
}
func (r *RawExecDriver) WaitTask(ctx context.Context, taskID string) (<-chan *drivers.ExitResult, error) {
@@ -418,23 +445,13 @@ func (r *RawExecDriver) InspectTask(taskID string) (*drivers.TaskStatus, error)
return status, nil
}
func (r *RawExecDriver) TaskStats(taskID string) (*drivers.TaskStats, error) {
func (r *RawExecDriver) TaskStats(taskID string) (*cstructs.TaskResourceUsage, error) {
handle, ok := r.tasks.Get(taskID)
if !ok {
return nil, drivers.ErrTaskNotFound
}
stats, err := handle.exec.Stats()
if err != nil {
return nil, err
}
return &drivers.TaskStats{
ID: handle.task.ID,
Timestamp: stats.Timestamp,
AggResourceUsage: stats.ResourceUsage,
ResourceUsageByPid: stats.Pids,
}, nil
return handle.exec.Stats()
}
func (r *RawExecDriver) TaskEvents(ctx context.Context) (<-chan *drivers.TaskEvent, error) {

View File

@@ -8,6 +8,7 @@ import (
"github.com/golang/protobuf/ptypes"
hclog "github.com/hashicorp/go-hclog"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/plugins/base"
"github.com/hashicorp/nomad/plugins/drivers/proto"
"github.com/hashicorp/nomad/plugins/shared/hclspec"
@@ -49,13 +50,13 @@ func (d *driverPluginClient) Capabilities() (*Capabilities, error) {
switch resp.Capabilities.FsIsolation {
case proto.DriverCapabilities_NONE:
caps.FSIsolation = FSIsolationNone
caps.FSIsolation = cstructs.FSIsolationNone
case proto.DriverCapabilities_CHROOT:
caps.FSIsolation = FSIsolationChroot
caps.FSIsolation = cstructs.FSIsolationChroot
case proto.DriverCapabilities_IMAGE:
caps.FSIsolation = FSIsolationImage
caps.FSIsolation = cstructs.FSIsolationImage
default:
caps.FSIsolation = FSIsolationNone
caps.FSIsolation = cstructs.FSIsolationNone
}
}
@@ -111,17 +112,29 @@ func (d *driverPluginClient) RecoverTask(h *TaskHandle) error {
// StartTask starts execution of a task with the given TaskConfig. A TaskHandle
// is returned to the caller that can be used to recover state of the task,
// should the driver crash or exit prematurely.
func (d *driverPluginClient) StartTask(c *TaskConfig) (*TaskHandle, error) {
func (d *driverPluginClient) StartTask(c *TaskConfig) (*TaskHandle, *cstructs.DriverNetwork, error) {
req := &proto.StartTaskRequest{
Task: taskConfigToProto(c),
}
resp, err := d.client.StartTask(context.Background(), req)
if err != nil {
return nil, err
return nil, nil, err
}
return taskHandleFromProto(resp.Handle), nil
var net *cstructs.DriverNetwork
if resp.NetworkOverride != nil {
net = &cstructs.DriverNetwork{
PortMap: map[string]int{},
IP: resp.NetworkOverride.Addr,
AutoAdvertise: resp.NetworkOverride.AutoAdvertise,
}
for k, v := range resp.NetworkOverride.PortMap {
net.PortMap[k] = int(v)
}
}
return taskHandleFromProto(resp.Handle), net, nil
}
// WaitTask returns a channel that will have an ExitResult pushed to it once when the task
@@ -201,18 +214,21 @@ func (d *driverPluginClient) InspectTask(taskID string) (*TaskStatus, error) {
status.DriverAttributes = resp.Driver.Attributes
}
if resp.NetworkOverride != nil {
status.NetworkOverride = &NetworkOverride{
PortMap: resp.NetworkOverride.PortMap,
Addr: resp.NetworkOverride.Addr,
status.NetworkOverride = &cstructs.DriverNetwork{
PortMap: map[string]int{},
IP: resp.NetworkOverride.Addr,
AutoAdvertise: resp.NetworkOverride.AutoAdvertise,
}
for k, v := range resp.NetworkOverride.PortMap {
status.NetworkOverride.PortMap[k] = int(v)
}
}
return status, nil
}
// TaskStats returns resource usage statistics for the task
func (d *driverPluginClient) TaskStats(taskID string) (*TaskStats, error) {
func (d *driverPluginClient) TaskStats(taskID string) (*cstructs.TaskResourceUsage, error) {
req := &proto.TaskStatsRequest{TaskId: taskID}
resp, err := d.client.TaskStats(context.Background(), req)

View File

@@ -8,8 +8,12 @@ import (
"github.com/hashicorp/nomad/client/allocdir"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/base"
"github.com/hashicorp/nomad/plugins/shared/hclspec"
"github.com/zclconf/go-cty/cty"
"github.com/zclconf/go-cty/cty/gocty"
"github.com/zclconf/go-cty/cty/msgpack"
"golang.org/x/net/context"
)
@@ -25,12 +29,12 @@ type DriverPlugin interface {
Fingerprint(context.Context) (<-chan *Fingerprint, error)
RecoverTask(*TaskHandle) error
StartTask(*TaskConfig) (*TaskHandle, error)
StartTask(*TaskConfig) (*TaskHandle, *cstructs.DriverNetwork, error)
WaitTask(ctx context.Context, taskID string) (<-chan *ExitResult, error)
StopTask(taskID string, timeout time.Duration, signal string) error
DestroyTask(taskID string, force bool) error
InspectTask(taskID string) (*TaskStatus, error)
TaskStats(taskID string) (*TaskStats, error)
TaskStats(taskID string) (*cstructs.TaskResourceUsage, error)
TaskEvents(context.Context) (<-chan *TaskEvent, error)
SignalTask(taskID string, signal string) error
@@ -89,14 +93,14 @@ type Capabilities struct {
Exec bool
//FSIsolation indicates what kind of filesystem isolation the driver supports.
FSIsolation FSIsolation
FSIsolation cstructs.FSIsolation
}
type TaskConfig struct {
ID string
Name string
Env map[string]string
Resources Resources
Resources *Resources
Devices []DeviceConfig
Mounts []MountConfig
User string
@@ -129,14 +133,35 @@ func (tc *TaskConfig) TaskDir() *allocdir.TaskDir {
}
func (tc *TaskConfig) DecodeDriverConfig(t interface{}) error {
return base.MsgPackDecode(tc.rawDriverConfig, t)
ty, err := gocty.ImpliedType(t)
if err != nil {
return err
}
val, err := msgpack.Unmarshal(tc.rawDriverConfig, ty)
if err != nil {
return err
}
return gocty.FromCtyValue(val, t)
}
func (tc *TaskConfig) EncodeDriverConfig(t interface{}) error {
return base.MsgPackEncode(&tc.rawDriverConfig, t)
func (tc *TaskConfig) EncodeDriverConfig(val cty.Value) error {
data, err := msgpack.Marshal(val, val.Type())
if err != nil {
return err
}
tc.rawDriverConfig = data
return nil
}
type Resources struct {
NomadResources *structs.Resources
LinuxResources *LinuxResources
}
type LinuxResources struct {
CPUPeriod int64
CPUQuota int64
CPUShares int64
@@ -166,12 +191,6 @@ const (
type TaskState string
type NetworkOverride struct {
PortMap map[string]int32
Addr string
AutoAdvertise bool
}
type ExitResult struct {
ExitCode int
Signal int
@@ -191,14 +210,7 @@ type TaskStatus struct {
CompletedAt time.Time
ExitResult *ExitResult
DriverAttributes map[string]string
NetworkOverride *NetworkOverride
}
type TaskStats struct {
ID string
Timestamp int64
AggResourceUsage *cstructs.ResourceUsage
ResourceUsageByPid map[string]*cstructs.ResourceUsage
NetworkOverride *cstructs.DriverNetwork
}
type TaskEvent struct {

View File

@@ -49,7 +49,7 @@ func (x TaskState) String() string {
return proto.EnumName(TaskState_name, int32(x))
}
func (TaskState) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_driver_16b159caa26c45f9, []int{0}
return fileDescriptor_driver_60d7e28aa66468bf, []int{0}
}
type FingerprintResponse_HealthState int32
@@ -75,7 +75,7 @@ func (x FingerprintResponse_HealthState) String() string {
return proto.EnumName(FingerprintResponse_HealthState_name, int32(x))
}
func (FingerprintResponse_HealthState) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_driver_16b159caa26c45f9, []int{5, 0}
return fileDescriptor_driver_60d7e28aa66468bf, []int{5, 0}
}
type StartTaskResponse_Result int32
@@ -101,7 +101,7 @@ func (x StartTaskResponse_Result) String() string {
return proto.EnumName(StartTaskResponse_Result_name, int32(x))
}
func (StartTaskResponse_Result) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_driver_16b159caa26c45f9, []int{9, 0}
return fileDescriptor_driver_60d7e28aa66468bf, []int{9, 0}
}
type DriverCapabilities_FSIsolation int32
@@ -127,7 +127,7 @@ func (x DriverCapabilities_FSIsolation) String() string {
return proto.EnumName(DriverCapabilities_FSIsolation_name, int32(x))
}
func (DriverCapabilities_FSIsolation) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_driver_16b159caa26c45f9, []int{25, 0}
return fileDescriptor_driver_60d7e28aa66468bf, []int{25, 0}
}
type CPUUsage_Fields int32
@@ -162,7 +162,7 @@ func (x CPUUsage_Fields) String() string {
return proto.EnumName(CPUUsage_Fields_name, int32(x))
}
func (CPUUsage_Fields) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_driver_16b159caa26c45f9, []int{41, 0}
return fileDescriptor_driver_60d7e28aa66468bf, []int{41, 0}
}
type MemoryUsage_Fields int32
@@ -194,7 +194,7 @@ func (x MemoryUsage_Fields) String() string {
return proto.EnumName(MemoryUsage_Fields_name, int32(x))
}
func (MemoryUsage_Fields) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_driver_16b159caa26c45f9, []int{42, 0}
return fileDescriptor_driver_60d7e28aa66468bf, []int{42, 0}
}
type TaskConfigSchemaRequest struct {
@@ -207,7 +207,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_16b159caa26c45f9, []int{0}
return fileDescriptor_driver_60d7e28aa66468bf, []int{0}
}
func (m *TaskConfigSchemaRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskConfigSchemaRequest.Unmarshal(m, b)
@@ -239,7 +239,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_16b159caa26c45f9, []int{1}
return fileDescriptor_driver_60d7e28aa66468bf, []int{1}
}
func (m *TaskConfigSchemaResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskConfigSchemaResponse.Unmarshal(m, b)
@@ -276,7 +276,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_16b159caa26c45f9, []int{2}
return fileDescriptor_driver_60d7e28aa66468bf, []int{2}
}
func (m *CapabilitiesRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CapabilitiesRequest.Unmarshal(m, b)
@@ -311,7 +311,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_16b159caa26c45f9, []int{3}
return fileDescriptor_driver_60d7e28aa66468bf, []int{3}
}
func (m *CapabilitiesResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CapabilitiesResponse.Unmarshal(m, b)
@@ -348,7 +348,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_16b159caa26c45f9, []int{4}
return fileDescriptor_driver_60d7e28aa66468bf, []int{4}
}
func (m *FingerprintRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_FingerprintRequest.Unmarshal(m, b)
@@ -391,7 +391,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_16b159caa26c45f9, []int{5}
return fileDescriptor_driver_60d7e28aa66468bf, []int{5}
}
func (m *FingerprintResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_FingerprintResponse.Unmarshal(m, b)
@@ -446,7 +446,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_16b159caa26c45f9, []int{6}
return fileDescriptor_driver_60d7e28aa66468bf, []int{6}
}
func (m *RecoverTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_RecoverTaskRequest.Unmarshal(m, b)
@@ -490,7 +490,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_16b159caa26c45f9, []int{7}
return fileDescriptor_driver_60d7e28aa66468bf, []int{7}
}
func (m *RecoverTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_RecoverTaskResponse.Unmarshal(m, b)
@@ -522,7 +522,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_16b159caa26c45f9, []int{8}
return fileDescriptor_driver_60d7e28aa66468bf, []int{8}
}
func (m *StartTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StartTaskRequest.Unmarshal(m, b)
@@ -576,7 +576,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_16b159caa26c45f9, []int{9}
return fileDescriptor_driver_60d7e28aa66468bf, []int{9}
}
func (m *StartTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StartTaskResponse.Unmarshal(m, b)
@@ -636,7 +636,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_16b159caa26c45f9, []int{10}
return fileDescriptor_driver_60d7e28aa66468bf, []int{10}
}
func (m *WaitTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_WaitTaskRequest.Unmarshal(m, b)
@@ -677,7 +677,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_16b159caa26c45f9, []int{11}
return fileDescriptor_driver_60d7e28aa66468bf, []int{11}
}
func (m *WaitTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_WaitTaskResponse.Unmarshal(m, b)
@@ -729,7 +729,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_16b159caa26c45f9, []int{12}
return fileDescriptor_driver_60d7e28aa66468bf, []int{12}
}
func (m *StopTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StopTaskRequest.Unmarshal(m, b)
@@ -780,7 +780,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_16b159caa26c45f9, []int{13}
return fileDescriptor_driver_60d7e28aa66468bf, []int{13}
}
func (m *StopTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StopTaskResponse.Unmarshal(m, b)
@@ -814,7 +814,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_16b159caa26c45f9, []int{14}
return fileDescriptor_driver_60d7e28aa66468bf, []int{14}
}
func (m *DestroyTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DestroyTaskRequest.Unmarshal(m, b)
@@ -858,7 +858,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_16b159caa26c45f9, []int{15}
return fileDescriptor_driver_60d7e28aa66468bf, []int{15}
}
func (m *DestroyTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DestroyTaskResponse.Unmarshal(m, b)
@@ -890,7 +890,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_16b159caa26c45f9, []int{16}
return fileDescriptor_driver_60d7e28aa66468bf, []int{16}
}
func (m *InspectTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_InspectTaskRequest.Unmarshal(m, b)
@@ -933,7 +933,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_16b159caa26c45f9, []int{17}
return fileDescriptor_driver_60d7e28aa66468bf, []int{17}
}
func (m *InspectTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_InspectTaskResponse.Unmarshal(m, b)
@@ -986,7 +986,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_16b159caa26c45f9, []int{18}
return fileDescriptor_driver_60d7e28aa66468bf, []int{18}
}
func (m *TaskStatsRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskStatsRequest.Unmarshal(m, b)
@@ -1025,7 +1025,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_16b159caa26c45f9, []int{19}
return fileDescriptor_driver_60d7e28aa66468bf, []int{19}
}
func (m *TaskStatsResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskStatsResponse.Unmarshal(m, b)
@@ -1062,7 +1062,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_16b159caa26c45f9, []int{20}
return fileDescriptor_driver_60d7e28aa66468bf, []int{20}
}
func (m *TaskEventsRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskEventsRequest.Unmarshal(m, b)
@@ -1096,7 +1096,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_16b159caa26c45f9, []int{21}
return fileDescriptor_driver_60d7e28aa66468bf, []int{21}
}
func (m *SignalTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SignalTaskRequest.Unmarshal(m, b)
@@ -1140,7 +1140,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_16b159caa26c45f9, []int{22}
return fileDescriptor_driver_60d7e28aa66468bf, []int{22}
}
func (m *SignalTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SignalTaskResponse.Unmarshal(m, b)
@@ -1177,7 +1177,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_16b159caa26c45f9, []int{23}
return fileDescriptor_driver_60d7e28aa66468bf, []int{23}
}
func (m *ExecTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExecTaskRequest.Unmarshal(m, b)
@@ -1234,7 +1234,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_16b159caa26c45f9, []int{24}
return fileDescriptor_driver_60d7e28aa66468bf, []int{24}
}
func (m *ExecTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExecTaskResponse.Unmarshal(m, b)
@@ -1293,7 +1293,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_16b159caa26c45f9, []int{25}
return fileDescriptor_driver_60d7e28aa66468bf, []int{25}
}
func (m *DriverCapabilities) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DriverCapabilities.Unmarshal(m, b)
@@ -1368,7 +1368,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_16b159caa26c45f9, []int{26}
return fileDescriptor_driver_60d7e28aa66468bf, []int{26}
}
func (m *TaskConfig) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskConfig.Unmarshal(m, b)
@@ -1479,7 +1479,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_16b159caa26c45f9, []int{27}
return fileDescriptor_driver_60d7e28aa66468bf, []int{27}
}
func (m *Resources) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Resources.Unmarshal(m, b)
@@ -1528,7 +1528,7 @@ func (m *RawResources) Reset() { *m = RawResources{} }
func (m *RawResources) String() string { return proto.CompactTextString(m) }
func (*RawResources) ProtoMessage() {}
func (*RawResources) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_16b159caa26c45f9, []int{28}
return fileDescriptor_driver_60d7e28aa66468bf, []int{28}
}
func (m *RawResources) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_RawResources.Unmarshal(m, b)
@@ -1599,7 +1599,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_16b159caa26c45f9, []int{29}
return fileDescriptor_driver_60d7e28aa66468bf, []int{29}
}
func (m *NetworkResource) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_NetworkResource.Unmarshal(m, b)
@@ -1673,7 +1673,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_16b159caa26c45f9, []int{30}
return fileDescriptor_driver_60d7e28aa66468bf, []int{30}
}
func (m *NetworkPort) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_NetworkPort.Unmarshal(m, b)
@@ -1715,7 +1715,7 @@ type LinuxResources struct {
// CPU shares (relative weight vs. other containers). Default: 0 (not specified)
CpuShares int64 `protobuf:"varint,3,opt,name=cpu_shares,json=cpuShares,proto3" json:"cpu_shares,omitempty"`
// Memory limit in bytes. Default: 0 (not specified)
MemoryLimitInBytes int64 `protobuf:"varint,4,opt,name=memory_limit_in_bytes,json=memoryLimitInBytes,proto3" json:"memory_limit_in_bytes,omitempty"`
MemoryLimitBytes int64 `protobuf:"varint,4,opt,name=memory_limit_bytes,json=memoryLimitBytes,proto3" json:"memory_limit_bytes,omitempty"`
// OOMScoreAdj adjusts the oom-killer score. Default: 0 (not specified)
OomScoreAdj int64 `protobuf:"varint,5,opt,name=oom_score_adj,json=oomScoreAdj,proto3" json:"oom_score_adj,omitempty"`
// CpusetCpus constrains the allowed set of logical CPUs. Default: "" (not specified)
@@ -1731,7 +1731,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_16b159caa26c45f9, []int{31}
return fileDescriptor_driver_60d7e28aa66468bf, []int{31}
}
func (m *LinuxResources) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_LinuxResources.Unmarshal(m, b)
@@ -1772,9 +1772,9 @@ func (m *LinuxResources) GetCpuShares() int64 {
return 0
}
func (m *LinuxResources) GetMemoryLimitInBytes() int64 {
func (m *LinuxResources) GetMemoryLimitBytes() int64 {
if m != nil {
return m.MemoryLimitInBytes
return m.MemoryLimitBytes
}
return 0
}
@@ -1816,7 +1816,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_16b159caa26c45f9, []int{32}
return fileDescriptor_driver_60d7e28aa66468bf, []int{32}
}
func (m *Mount) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Mount.Unmarshal(m, b)
@@ -1879,7 +1879,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_16b159caa26c45f9, []int{33}
return fileDescriptor_driver_60d7e28aa66468bf, []int{33}
}
func (m *Device) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Device.Unmarshal(m, b)
@@ -1937,7 +1937,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_16b159caa26c45f9, []int{34}
return fileDescriptor_driver_60d7e28aa66468bf, []int{34}
}
func (m *TaskHandle) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskHandle.Unmarshal(m, b)
@@ -1997,7 +1997,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_16b159caa26c45f9, []int{35}
return fileDescriptor_driver_60d7e28aa66468bf, []int{35}
}
func (m *NetworkOverride) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_NetworkOverride.Unmarshal(m, b)
@@ -2055,7 +2055,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_16b159caa26c45f9, []int{36}
return fileDescriptor_driver_60d7e28aa66468bf, []int{36}
}
func (m *ExitResult) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExitResult.Unmarshal(m, b)
@@ -2118,7 +2118,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_16b159caa26c45f9, []int{37}
return fileDescriptor_driver_60d7e28aa66468bf, []int{37}
}
func (m *TaskStatus) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskStatus.Unmarshal(m, b)
@@ -2193,7 +2193,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_16b159caa26c45f9, []int{38}
return fileDescriptor_driver_60d7e28aa66468bf, []int{38}
}
func (m *TaskDriverStatus) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskDriverStatus.Unmarshal(m, b)
@@ -2238,7 +2238,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_16b159caa26c45f9, []int{39}
return fileDescriptor_driver_60d7e28aa66468bf, []int{39}
}
func (m *TaskStats) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskStats.Unmarshal(m, b)
@@ -2300,7 +2300,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_16b159caa26c45f9, []int{40}
return fileDescriptor_driver_60d7e28aa66468bf, []int{40}
}
func (m *TaskResourceUsage) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskResourceUsage.Unmarshal(m, b)
@@ -2352,7 +2352,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_16b159caa26c45f9, []int{41}
return fileDescriptor_driver_60d7e28aa66468bf, []int{41}
}
func (m *CPUUsage) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CPUUsage.Unmarshal(m, b)
@@ -2438,7 +2438,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_16b159caa26c45f9, []int{42}
return fileDescriptor_driver_60d7e28aa66468bf, []int{42}
}
func (m *MemoryUsage) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_MemoryUsage.Unmarshal(m, b)
@@ -2518,7 +2518,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_16b159caa26c45f9, []int{43}
return fileDescriptor_driver_60d7e28aa66468bf, []int{43}
}
func (m *DriverTaskEvent) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DriverTaskEvent.Unmarshal(m, b)
@@ -3208,181 +3208,181 @@ var _Driver_serviceDesc = grpc.ServiceDesc{
Metadata: "driver.proto",
}
func init() { proto.RegisterFile("driver.proto", fileDescriptor_driver_16b159caa26c45f9) }
func init() { proto.RegisterFile("driver.proto", fileDescriptor_driver_60d7e28aa66468bf) }
var fileDescriptor_driver_16b159caa26c45f9 = []byte{
// 2757 bytes of a gzipped FileDescriptorProto
var fileDescriptor_driver_60d7e28aa66468bf = []byte{
// 2753 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x59, 0xcb, 0x6f, 0x23, 0xc7,
0xd1, 0x17, 0x9f, 0x22, 0x8b, 0x14, 0x35, 0xdb, 0xbb, 0xfb, 0x99, 0xa6, 0xf1, 0x7d, 0x5e, 0x0f,
0x60, 0x40, 0xb0, 0xbd, 0x94, 0x2d, 0xe3, 0xf3, 0x3e, 0x02, 0x3f, 0x68, 0x72, 0x76, 0x25, 0xaf,
0x44, 0x29, 0x4d, 0x0a, 0xeb, 0x4d, 0x62, 0x4f, 0x46, 0x33, 0x2d, 0x72, 0x2c, 0xce, 0xc3, 0xdd,
0x43, 0xad, 0x84, 0x20, 0x48, 0x90, 0x04, 0x41, 0x12, 0x20, 0x40, 0x2e, 0x41, 0xee, 0xb9, 0x05,
0xb9, 0xe6, 0x94, 0xc7, 0x25, 0x40, 0xfe, 0x87, 0x1c, 0x73, 0x09, 0x90, 0x6b, 0xfe, 0x83, 0xa0,
0x1f, 0x33, 0x1c, 0x4a, 0x5a, 0x7b, 0x48, 0xe5, 0xc4, 0xa9, 0xea, 0xee, 0x5f, 0x17, 0xab, 0xaa,
0xab, 0xaa, 0xbb, 0xa0, 0xee, 0x50, 0xf7, 0x94, 0xd0, 0x76, 0x48, 0x83, 0x28, 0x40, 0xaf, 0x8f,
0x2d, 0x36, 0x76, 0xed, 0x80, 0x86, 0x6d, 0x3f, 0xf0, 0x2c, 0xa7, 0x1d, 0x4e, 0xa6, 0x23, 0xd7,
0x67, 0x6d, 0x39, 0x8b, 0xc9, 0x69, 0xad, 0xff, 0x1b, 0x05, 0xc1, 0x68, 0x42, 0x36, 0x05, 0x75,
0x34, 0x3d, 0xde, 0x74, 0xa6, 0xd4, 0x8a, 0xdc, 0xc0, 0x57, 0xe3, 0xaf, 0x5e, 0x1c, 0x8f, 0x5c,
0x8f, 0xb0, 0xc8, 0xf2, 0x42, 0x35, 0xe1, 0xa3, 0x91, 0x1b, 0x8d, 0xa7, 0x47, 0x6d, 0x3b, 0xf0,
0x36, 0x93, 0x2d, 0x37, 0xc5, 0x96, 0x9b, 0x6a, 0xcb, 0x4d, 0x36, 0xb6, 0x28, 0x71, 0x36, 0xc7,
0xf6, 0x84, 0x85, 0xc4, 0xe6, 0xbf, 0x26, 0xff, 0x90, 0x08, 0xfa, 0xcb, 0xf0, 0xd2, 0xd0, 0x62,
0x27, 0xdd, 0xc0, 0x3f, 0x76, 0x47, 0x03, 0x7b, 0x4c, 0x3c, 0x0b, 0x93, 0x2f, 0xa7, 0x84, 0x45,
0xfa, 0x77, 0xa0, 0x79, 0x79, 0x88, 0x85, 0x81, 0xcf, 0x08, 0xfa, 0x08, 0x8a, 0x1c, 0xa4, 0x99,
0xbb, 0x93, 0xdb, 0xa8, 0x6d, 0xbd, 0xd5, 0x7e, 0xd1, 0xff, 0x95, 0x9b, 0xb7, 0xd5, 0xe6, 0xed,
0x41, 0x48, 0x6c, 0x2c, 0x56, 0xea, 0xb7, 0xe1, 0x66, 0xd7, 0x0a, 0xad, 0x23, 0x77, 0xe2, 0x46,
0x2e, 0x61, 0xf1, 0xa6, 0x53, 0xb8, 0x35, 0xcf, 0x56, 0x1b, 0x7e, 0x06, 0x75, 0x3b, 0xc5, 0x57,
0x1b, 0x3f, 0x68, 0x67, 0x52, 0x74, 0xbb, 0x27, 0xa8, 0x39, 0xe0, 0x39, 0x38, 0xfd, 0x16, 0xa0,
0x47, 0xae, 0x3f, 0x22, 0x34, 0xa4, 0xae, 0x1f, 0xc5, 0xc2, 0xfc, 0xa2, 0x00, 0x37, 0xe7, 0xd8,
0x4a, 0x98, 0x2f, 0x00, 0xac, 0x28, 0xa2, 0xee, 0xd1, 0x34, 0x12, 0xa2, 0x14, 0x36, 0x6a, 0x5b,
0x9f, 0x64, 0x14, 0xe5, 0x0a, 0xbc, 0x76, 0x27, 0x01, 0x33, 0xfc, 0x88, 0x9e, 0xe3, 0x14, 0x3a,
0xfa, 0x1c, 0xca, 0x63, 0x62, 0x4d, 0xa2, 0x71, 0x33, 0x7f, 0x27, 0xb7, 0xd1, 0xd8, 0x7a, 0x74,
0x8d, 0x7d, 0xb6, 0x05, 0xd0, 0x20, 0xb2, 0x22, 0x82, 0x15, 0x2a, 0xba, 0x0b, 0x48, 0x7e, 0x99,
0x0e, 0x61, 0x36, 0x75, 0x43, 0xee, 0x7f, 0xcd, 0xc2, 0x9d, 0xdc, 0x46, 0x15, 0xdf, 0x90, 0x23,
0xbd, 0xd9, 0x40, 0xeb, 0x7d, 0x58, 0xbf, 0x20, 0x2d, 0xd2, 0xa0, 0x70, 0x42, 0xce, 0x85, 0x45,
0xaa, 0x98, 0x7f, 0xa2, 0x5b, 0x50, 0x3a, 0xb5, 0x26, 0x53, 0x22, 0x44, 0xae, 0x62, 0x49, 0x3c,
0xcc, 0xdf, 0xcf, 0xe9, 0x0f, 0xa0, 0x96, 0x12, 0x02, 0x35, 0x00, 0x0e, 0xfb, 0x3d, 0x63, 0x68,
0x74, 0x87, 0x46, 0x4f, 0x5b, 0x41, 0x6b, 0x50, 0x3d, 0xec, 0x6f, 0x1b, 0x9d, 0xdd, 0xe1, 0xf6,
0x33, 0x2d, 0x87, 0x6a, 0xb0, 0x1a, 0x13, 0x79, 0xfd, 0x0c, 0x10, 0x26, 0x76, 0x70, 0x4a, 0x28,
0xf7, 0x4a, 0x65, 0x22, 0xf4, 0x12, 0xac, 0x46, 0x16, 0x3b, 0x31, 0x5d, 0x47, 0x09, 0x50, 0xe6,
0xe4, 0x8e, 0x83, 0x76, 0xa0, 0x3c, 0xb6, 0x7c, 0x67, 0x22, 0x85, 0xa8, 0x6d, 0xbd, 0x93, 0x51,
0x6f, 0x1c, 0x7c, 0x5b, 0x2c, 0xc4, 0x0a, 0x80, 0xbb, 0xea, 0xdc, 0xce, 0x52, 0x9b, 0xfa, 0x33,
0xd0, 0x06, 0x91, 0x45, 0xa3, 0xb4, 0x38, 0x06, 0x14, 0xf9, 0xfe, 0xca, 0x3d, 0x17, 0xd9, 0x53,
0x1e, 0x33, 0x2c, 0x96, 0xeb, 0xff, 0xce, 0xc3, 0x8d, 0x14, 0xb6, 0x72, 0xbb, 0xa7, 0x50, 0xa6,
0x84, 0x4d, 0x27, 0x91, 0x80, 0x6f, 0x6c, 0x7d, 0x98, 0x11, 0xfe, 0x12, 0x52, 0x1b, 0x0b, 0x18,
0xac, 0xe0, 0xd0, 0x06, 0x68, 0x72, 0x85, 0x49, 0x28, 0x0d, 0xa8, 0xe9, 0xb1, 0x91, 0x32, 0x5d,
0x43, 0xf2, 0x0d, 0xce, 0xde, 0x63, 0xa3, 0x94, 0x56, 0x0b, 0xd7, 0xd4, 0x2a, 0xb2, 0x40, 0xf3,
0x49, 0xf4, 0x3c, 0xa0, 0x27, 0x26, 0x57, 0x2d, 0x75, 0x1d, 0xd2, 0x2c, 0x0a, 0xd0, 0xf7, 0x32,
0x82, 0xf6, 0xe5, 0xf2, 0x7d, 0xb5, 0x1a, 0xaf, 0xfb, 0xf3, 0x0c, 0xfd, 0x4d, 0x28, 0xcb, 0x7f,
0xca, 0x3d, 0x69, 0x70, 0xd8, 0xed, 0x1a, 0x83, 0x81, 0xb6, 0x82, 0xaa, 0x50, 0xc2, 0xc6, 0x10,
0x73, 0x0f, 0xab, 0x42, 0xe9, 0x51, 0x67, 0xd8, 0xd9, 0xd5, 0xf2, 0xfa, 0x1b, 0xb0, 0xfe, 0xd4,
0x72, 0xa3, 0x2c, 0xce, 0xa5, 0x07, 0xa0, 0xcd, 0xe6, 0x2a, 0xeb, 0xec, 0xcc, 0x59, 0x27, 0xbb,
0x6a, 0x8c, 0x33, 0x37, 0xba, 0x60, 0x0f, 0x0d, 0x0a, 0x84, 0x52, 0x65, 0x02, 0xfe, 0xa9, 0x3f,
0x87, 0xf5, 0x41, 0x14, 0x84, 0x99, 0x3c, 0xff, 0x5d, 0x58, 0xe5, 0x79, 0x22, 0x98, 0x46, 0xca,
0xf5, 0x5f, 0x6e, 0xcb, 0x3c, 0xd2, 0x8e, 0xf3, 0x48, 0xbb, 0xa7, 0xf2, 0x0c, 0x8e, 0x67, 0xa2,
0xff, 0x81, 0x32, 0x73, 0x47, 0xbe, 0x35, 0x51, 0x47, 0x5f, 0x51, 0x3a, 0xe2, 0x4e, 0x1e, 0x6f,
0xac, 0x1c, 0xbf, 0x0b, 0xa8, 0x47, 0x58, 0x44, 0x83, 0xf3, 0x4c, 0xf2, 0xdc, 0x82, 0xd2, 0x71,
0x40, 0x6d, 0x79, 0x10, 0x2b, 0x58, 0x12, 0xfc, 0x50, 0xcd, 0x81, 0x28, 0xec, 0xbb, 0x80, 0x76,
0x7c, 0x9e, 0x20, 0xb2, 0x19, 0xe2, 0x57, 0x79, 0xb8, 0x39, 0x37, 0x5f, 0x19, 0x63, 0xf9, 0x73,
0xc8, 0x03, 0xd3, 0x94, 0xc9, 0x73, 0x88, 0xf6, 0xa1, 0x2c, 0x67, 0x28, 0x4d, 0xde, 0x5b, 0x00,
0x48, 0xe6, 0x1c, 0x05, 0xa7, 0x60, 0xae, 0x74, 0xfa, 0xc2, 0x7f, 0xdb, 0xe9, 0xb5, 0xf8, 0x7f,
0xb0, 0xaf, 0xd5, 0xdf, 0xb7, 0xe1, 0x46, 0x6a, 0xb2, 0x52, 0xde, 0x23, 0x28, 0x31, 0xce, 0x50,
0xda, 0x7b, 0x7b, 0x41, 0xed, 0x31, 0x2c, 0x97, 0xeb, 0x37, 0x25, 0xb8, 0x71, 0x4a, 0xfc, 0x44,
0x14, 0xbd, 0x07, 0x37, 0x06, 0xc2, 0xb5, 0x32, 0xf9, 0xce, 0xcc, 0x2d, 0xf3, 0x73, 0x6e, 0x79,
0x0b, 0x50, 0x1a, 0x45, 0x39, 0xcf, 0x39, 0xac, 0x1b, 0x67, 0xc4, 0xce, 0x84, 0xdc, 0x84, 0x55,
0x3b, 0xf0, 0x3c, 0xcb, 0x77, 0x9a, 0xf9, 0x3b, 0x85, 0x8d, 0x2a, 0x8e, 0xc9, 0xf4, 0xf9, 0x29,
0x64, 0x3d, 0x3f, 0xfa, 0x2f, 0x73, 0xa0, 0xcd, 0xf6, 0x56, 0x8a, 0xe4, 0xd2, 0x47, 0x0e, 0x07,
0xe2, 0x7b, 0xd7, 0xb1, 0xa2, 0x14, 0x3f, 0x3e, 0xe2, 0x92, 0x4f, 0x28, 0x4d, 0x85, 0x90, 0xc2,
0x35, 0x43, 0x88, 0xfe, 0xcf, 0x1c, 0xa0, 0xcb, 0x55, 0x0f, 0x7a, 0x0d, 0xea, 0x8c, 0xf8, 0x8e,
0x29, 0xd5, 0x28, 0x2d, 0x5c, 0xc1, 0x35, 0xce, 0x93, 0xfa, 0x64, 0x08, 0x41, 0x91, 0x9c, 0x11,
0x5b, 0x9d, 0x56, 0xf1, 0x8d, 0xc6, 0x50, 0x3f, 0x66, 0xa6, 0xcb, 0x82, 0x89, 0x95, 0x94, 0x07,
0x8d, 0x2d, 0x63, 0xe9, 0xea, 0xab, 0xfd, 0x68, 0xb0, 0x13, 0x83, 0xe1, 0xda, 0x31, 0x4b, 0x08,
0xbd, 0x0d, 0xb5, 0xd4, 0x18, 0xaa, 0x40, 0xb1, 0xbf, 0xdf, 0x37, 0xb4, 0x15, 0x04, 0x50, 0xee,
0x6e, 0xe3, 0xfd, 0xfd, 0xa1, 0x8c, 0xda, 0x3b, 0x7b, 0x9d, 0xc7, 0x86, 0x96, 0xd7, 0xff, 0x5c,
0x04, 0x98, 0xa5, 0x4f, 0xd4, 0x80, 0x7c, 0x62, 0xe9, 0xbc, 0xeb, 0xf0, 0x3f, 0xe3, 0x5b, 0x5e,
0x5c, 0x88, 0x88, 0x6f, 0xb4, 0x05, 0xb7, 0x3d, 0x36, 0x0a, 0x2d, 0xfb, 0xc4, 0x54, 0x59, 0xcf,
0x16, 0x8b, 0xc5, 0xbf, 0xaa, 0xe3, 0x9b, 0x6a, 0x50, 0x49, 0x2d, 0x71, 0x77, 0xa1, 0x40, 0xfc,
0xd3, 0x66, 0x51, 0x94, 0x7a, 0x0f, 0x17, 0x4e, 0xeb, 0x6d, 0xc3, 0x3f, 0x95, 0xa5, 0x1d, 0x87,
0x41, 0x7d, 0xa8, 0x52, 0xc2, 0x82, 0x29, 0xb5, 0x09, 0x6b, 0x96, 0x16, 0x3a, 0x64, 0x38, 0x5e,
0x87, 0x67, 0x10, 0xa8, 0x07, 0x65, 0x2f, 0x98, 0xfa, 0x11, 0x6b, 0x96, 0x85, 0x80, 0x6f, 0x65,
0x04, 0xdb, 0xe3, 0x8b, 0xb0, 0x5a, 0x8b, 0x1e, 0xc3, 0xaa, 0x43, 0x4e, 0x5d, 0x2e, 0xd3, 0xaa,
0x80, 0xb9, 0x9b, 0xd5, 0xbe, 0x62, 0x15, 0x8e, 0x57, 0x73, 0xa5, 0x4f, 0x19, 0xa1, 0xcd, 0x8a,
0x54, 0x3a, 0xff, 0x46, 0xaf, 0x40, 0xd5, 0x9a, 0x4c, 0x02, 0xdb, 0x74, 0x5c, 0xda, 0xac, 0x8a,
0x81, 0x8a, 0x60, 0xf4, 0x5c, 0x8a, 0x5e, 0x85, 0x9a, 0x3c, 0x19, 0x66, 0x68, 0x45, 0xe3, 0x26,
0x88, 0x61, 0x90, 0xac, 0x03, 0x2b, 0x1a, 0xab, 0x09, 0x84, 0x52, 0x39, 0xa1, 0x96, 0x4c, 0x20,
0x94, 0xf2, 0x09, 0xad, 0xf7, 0xa0, 0x12, 0xab, 0x78, 0xa1, 0x7a, 0xf4, 0x6f, 0x39, 0xa8, 0x26,
0x2a, 0x45, 0x9f, 0xc2, 0x1a, 0xb5, 0x9e, 0x9b, 0x33, 0xdb, 0xc8, 0x00, 0xf8, 0x6e, 0x56, 0xdb,
0x58, 0xcf, 0x67, 0xe6, 0xa9, 0xd3, 0x14, 0x85, 0x3e, 0x87, 0xf5, 0x89, 0xeb, 0x4f, 0xcf, 0x52,
0xd8, 0x32, 0xa3, 0xfc, 0x7f, 0x46, 0xec, 0x5d, 0xbe, 0x7a, 0x86, 0xde, 0x98, 0xcc, 0xd1, 0xfa,
0x1f, 0x72, 0x50, 0x4f, 0x6f, 0xcf, 0x95, 0x60, 0x87, 0x53, 0xf1, 0x07, 0x0a, 0x98, 0x7f, 0xf2,
0xa0, 0xe3, 0x11, 0x2f, 0xa0, 0xe7, 0x62, 0xe7, 0x02, 0x56, 0x14, 0xb7, 0x96, 0xe3, 0xb2, 0x13,
0xe1, 0xfd, 0x05, 0x2c, 0xbe, 0x39, 0xcf, 0x0d, 0x42, 0x26, 0xea, 0xb1, 0x02, 0x16, 0xdf, 0x08,
0x43, 0x45, 0xa5, 0x1a, 0xee, 0xb3, 0x85, 0xc5, 0x53, 0x56, 0x2c, 0x1c, 0x4e, 0x70, 0xf4, 0xdf,
0xe4, 0x61, 0xfd, 0xc2, 0x28, 0x97, 0x53, 0x3a, 0x52, 0x1c, 0xb0, 0x25, 0xc5, 0x65, 0xb2, 0x5d,
0x27, 0xae, 0x8a, 0xc4, 0xb7, 0x38, 0xee, 0xa1, 0xaa, 0x58, 0xf2, 0x6e, 0xc8, 0x0d, 0xed, 0x1d,
0xb9, 0x91, 0x14, 0xbc, 0x84, 0x25, 0x81, 0x9e, 0x41, 0x83, 0x12, 0x46, 0xe8, 0x29, 0x71, 0xcc,
0x30, 0xa0, 0x51, 0x2c, 0xff, 0xd6, 0x62, 0xf2, 0x1f, 0x04, 0x34, 0xc2, 0x6b, 0x31, 0x12, 0xa7,
0x18, 0x7a, 0x0a, 0x6b, 0xce, 0xb9, 0x6f, 0x79, 0xae, 0xad, 0x90, 0xcb, 0x4b, 0x23, 0xd7, 0x15,
0x90, 0x00, 0xe6, 0x17, 0xa5, 0xd4, 0x20, 0xff, 0x63, 0x13, 0xeb, 0x88, 0x4c, 0x94, 0x4e, 0x24,
0x31, 0xef, 0xd7, 0x25, 0xe5, 0xd7, 0xfa, 0x4f, 0xf2, 0xd0, 0x98, 0x77, 0x17, 0xf4, 0xbf, 0x00,
0x76, 0x38, 0x35, 0x43, 0x42, 0xdd, 0xc0, 0x51, 0x4e, 0x51, 0xb5, 0xc3, 0xe9, 0x81, 0x60, 0xf0,
0xc3, 0xc9, 0x87, 0xbf, 0x9c, 0x06, 0x91, 0xa5, 0xbc, 0xa3, 0x62, 0x87, 0xd3, 0x6f, 0x72, 0x3a,
0x5e, 0x2b, 0x6e, 0xf2, 0x4c, 0x79, 0x09, 0x9f, 0x3e, 0x10, 0x0c, 0xf4, 0x0e, 0xdc, 0x96, 0x8e,
0x64, 0x4e, 0x5c, 0xcf, 0x8d, 0x4c, 0xd7, 0x37, 0x8f, 0xce, 0xf9, 0xb5, 0x58, 0xfa, 0x0e, 0x92,
0x83, 0xbb, 0x7c, 0x6c, 0xc7, 0xff, 0x98, 0x8f, 0x20, 0x1d, 0xd6, 0x82, 0xc0, 0x33, 0x99, 0x1d,
0x50, 0x62, 0x5a, 0xce, 0x17, 0x22, 0x04, 0x16, 0x70, 0x2d, 0x08, 0xbc, 0x01, 0xe7, 0x75, 0x9c,
0x2f, 0xf8, 0x89, 0xb7, 0xc3, 0x29, 0x23, 0x91, 0xc9, 0x7f, 0x9a, 0x65, 0x79, 0xe2, 0x25, 0xab,
0x1b, 0x4e, 0x59, 0x6a, 0x82, 0x47, 0x3c, 0x1e, 0xb1, 0x52, 0x13, 0xf6, 0x88, 0xc7, 0xf4, 0xcf,
0xa0, 0x24, 0xe2, 0x1b, 0xff, 0x77, 0xa2, 0x04, 0x10, 0xa1, 0x43, 0xea, 0xaf, 0xc2, 0x19, 0x22,
0xb2, 0xbc, 0x02, 0xd5, 0x71, 0xc0, 0x54, 0xe0, 0x91, 0xae, 0x55, 0xe1, 0x0c, 0x31, 0xd8, 0x82,
0x0a, 0x25, 0x96, 0x13, 0xf8, 0x93, 0x73, 0xf1, 0xc7, 0x2b, 0x38, 0xa1, 0xf5, 0x2f, 0xa1, 0x2c,
0xe3, 0xde, 0x35, 0xf0, 0xef, 0x02, 0xb2, 0x47, 0x34, 0x98, 0x86, 0xdc, 0x32, 0x9e, 0xcb, 0x98,
0x1b, 0xf8, 0x2c, 0xbe, 0x7b, 0xcb, 0x91, 0x83, 0xd9, 0x80, 0xfe, 0xd7, 0x9c, 0xcc, 0x75, 0xf2,
0x22, 0xc5, 0xab, 0x05, 0x95, 0xb8, 0x96, 0xbe, 0x6d, 0x2a, 0x80, 0xb8, 0xe2, 0x23, 0xea, 0x8d,
0x61, 0xd1, 0x8a, 0x8f, 0xc8, 0x8a, 0x8f, 0xf0, 0xf2, 0x42, 0xa5, 0x54, 0x09, 0x27, 0x33, 0x6a,
0xcd, 0x49, 0x4a, 0x61, 0xa2, 0xff, 0x2b, 0x97, 0x1c, 0xf9, 0xb8, 0x64, 0x45, 0x9f, 0x43, 0x85,
0x9f, 0x1e, 0xd3, 0xb3, 0x42, 0xf5, 0x9a, 0xd2, 0x5d, 0xae, 0x1a, 0x6e, 0xf3, 0xc3, 0xb2, 0x67,
0x85, 0x32, 0xd7, 0xae, 0x86, 0x92, 0xe2, 0xa1, 0xc3, 0x72, 0x66, 0xa1, 0x83, 0x7f, 0xa3, 0xd7,
0xa1, 0x61, 0x4d, 0xa3, 0xc0, 0xb4, 0x9c, 0x53, 0x42, 0x23, 0x97, 0x11, 0x65, 0xe1, 0x35, 0xce,
0xed, 0xc4, 0xcc, 0xd6, 0x43, 0xa8, 0xa7, 0x31, 0xbf, 0x2e, 0xb9, 0x94, 0xd2, 0xc9, 0xe5, 0xbb,
0x00, 0xb3, 0xca, 0x8c, 0x7b, 0x02, 0x39, 0x73, 0x23, 0xd3, 0x0e, 0x1c, 0x19, 0xda, 0x4a, 0xb8,
0xc2, 0x19, 0xdd, 0xc0, 0x21, 0x17, 0xea, 0xdc, 0x52, 0x5c, 0xe7, 0xf2, 0xc3, 0xc7, 0x8f, 0xca,
0x89, 0x3b, 0x99, 0x10, 0x47, 0x49, 0x58, 0x0d, 0x02, 0xef, 0x89, 0x60, 0xe8, 0x7f, 0xc9, 0x4b,
0x8f, 0x90, 0xb7, 0x8c, 0x4c, 0xd5, 0x4f, 0x62, 0xea, 0xc2, 0xf5, 0x4c, 0xfd, 0x00, 0x80, 0x45,
0x16, 0x8d, 0x88, 0x63, 0x5a, 0x91, 0xba, 0xb8, 0xb7, 0x2e, 0x15, 0xca, 0xc3, 0xf8, 0xc1, 0x12,
0x57, 0xd5, 0xec, 0x4e, 0x84, 0xde, 0x87, 0xba, 0x1d, 0x78, 0xe1, 0x84, 0xa8, 0xc5, 0xa5, 0xaf,
0x5d, 0x5c, 0x4b, 0xe6, 0x77, 0xa2, 0x54, 0x95, 0x5c, 0xbe, 0x6e, 0x95, 0xfc, 0xc7, 0x9c, 0xbc,
0x2c, 0xa5, 0xef, 0x6a, 0x68, 0x74, 0xc5, 0xeb, 0xde, 0xe3, 0x25, 0x2f, 0x7e, 0x5f, 0xf5, 0xb4,
0x77, 0xdd, 0xb7, 0xb4, 0x3f, 0x15, 0xa0, 0x9a, 0xdc, 0xb9, 0x2e, 0xd9, 0xfe, 0x3e, 0x54, 0x93,
0xd7, 0x62, 0x55, 0x6b, 0x7c, 0xa5, 0x79, 0x92, 0xc9, 0xe8, 0x18, 0x90, 0x35, 0x1a, 0x25, 0x95,
0x8a, 0x39, 0x65, 0xd6, 0x28, 0xbe, 0xa5, 0xde, 0x5f, 0x40, 0x0f, 0x71, 0xfa, 0x39, 0xe4, 0xeb,
0xb1, 0x66, 0x8d, 0x46, 0x73, 0x1c, 0xf4, 0x3d, 0xb8, 0x3d, 0xbf, 0x87, 0x79, 0x74, 0x6e, 0x86,
0xae, 0xa3, 0xaa, 0xec, 0xed, 0x45, 0xaf, 0x9d, 0xed, 0x39, 0xf8, 0x8f, 0xcf, 0x0f, 0x5c, 0x47,
0xea, 0x1c, 0xd1, 0x4b, 0x03, 0xad, 0x1f, 0xc0, 0x4b, 0x2f, 0x98, 0x7e, 0x85, 0x0d, 0xfa, 0x69,
0x1b, 0x5c, 0x47, 0x09, 0x29, 0xeb, 0xfd, 0x36, 0x27, 0x6f, 0xc7, 0xf3, 0x3a, 0xe9, 0xcc, 0xca,
0xb6, 0xda, 0xd6, 0x66, 0xc6, 0x7d, 0xba, 0x07, 0x87, 0x12, 0x5e, 0xd4, 0x79, 0x9f, 0xcc, 0xd5,
0x79, 0xd9, 0x6b, 0x91, 0x3d, 0xb1, 0x48, 0x02, 0x29, 0x04, 0xfd, 0xf7, 0x05, 0xa8, 0xc4, 0xe8,
0xa2, 0x08, 0x3f, 0x67, 0x11, 0xf1, 0x4c, 0x2f, 0x0e, 0x61, 0x39, 0x0c, 0x92, 0xb5, 0xc7, 0x83,
0xd8, 0x2b, 0x50, 0xe5, 0xb5, 0xbe, 0x1c, 0xce, 0x8b, 0xe1, 0x0a, 0x67, 0x88, 0xc1, 0x57, 0xa1,
0x16, 0x05, 0x91, 0x35, 0x31, 0x23, 0xd7, 0x3e, 0x91, 0x49, 0x2e, 0x87, 0x41, 0xb0, 0x86, 0x9c,
0x83, 0xde, 0x84, 0x1b, 0xd1, 0x98, 0x06, 0x51, 0x34, 0xe1, 0x65, 0x9a, 0x28, 0x4c, 0x64, 0x11,
0x51, 0xc4, 0x5a, 0x32, 0x20, 0x0b, 0x16, 0xc6, 0xa3, 0xf7, 0x6c, 0x32, 0x77, 0x5d, 0x11, 0x44,
0x8a, 0x78, 0x2d, 0xe1, 0x72, 0xd7, 0xe6, 0x97, 0xfc, 0x90, 0x50, 0x9b, 0xf8, 0x32, 0x56, 0xe4,
0x70, 0x4c, 0x22, 0x13, 0xd6, 0x3d, 0x62, 0xb1, 0x29, 0x25, 0x8e, 0x79, 0xec, 0x92, 0x89, 0x23,
0x2f, 0x3d, 0x8d, 0xcc, 0x45, 0x6d, 0xac, 0x96, 0xf6, 0x23, 0xb1, 0x1a, 0x37, 0x62, 0x38, 0x49,
0xf3, 0xfa, 0x40, 0x7e, 0xa1, 0x75, 0xa8, 0x0d, 0x9e, 0x0d, 0x86, 0xc6, 0x9e, 0xb9, 0xb7, 0xdf,
0x33, 0xd4, 0x2b, 0xf7, 0xc0, 0xc0, 0x92, 0xcc, 0xf1, 0xf1, 0xe1, 0xfe, 0xb0, 0xb3, 0x6b, 0x0e,
0x77, 0xba, 0x4f, 0x06, 0x5a, 0x1e, 0xdd, 0x86, 0x1b, 0xc3, 0x6d, 0xbc, 0x3f, 0x1c, 0xee, 0x1a,
0x3d, 0xf3, 0xc0, 0xc0, 0x3b, 0xfb, 0xbd, 0x81, 0x56, 0x40, 0x08, 0x1a, 0x33, 0xf6, 0x70, 0x67,
0xcf, 0xd0, 0x8a, 0xa8, 0x06, 0xab, 0x07, 0x06, 0xee, 0x1a, 0xfd, 0xa1, 0x56, 0xd2, 0xff, 0x9e,
0x87, 0x5a, 0xca, 0x8a, 0xdc, 0x91, 0x29, 0x93, 0x97, 0x98, 0x22, 0xe6, 0x9f, 0x3c, 0x98, 0xd8,
0x96, 0x3d, 0x96, 0xd6, 0x29, 0x62, 0x49, 0x70, 0xbb, 0x79, 0xd6, 0x59, 0xea, 0x9c, 0x17, 0x71,
0xc5, 0xb3, 0xce, 0x24, 0xc8, 0x6b, 0x50, 0x3f, 0x21, 0xd4, 0x27, 0x13, 0x35, 0x2e, 0x2d, 0x52,
0x93, 0x3c, 0x39, 0x65, 0x03, 0x34, 0x35, 0x65, 0x06, 0x23, 0xcd, 0xd1, 0x90, 0xfc, 0xbd, 0x18,
0xec, 0xe8, 0xb2, 0xd6, 0xcb, 0x42, 0xeb, 0x0f, 0x16, 0x77, 0xd2, 0x17, 0x29, 0x7e, 0x90, 0x28,
0x7e, 0x15, 0x0a, 0x38, 0x7e, 0xf0, 0xed, 0x76, 0xba, 0xdb, 0x5c, 0xd9, 0x6b, 0x50, 0xdd, 0xeb,
0x7c, 0x6a, 0x1e, 0x0e, 0xc4, 0xf3, 0x01, 0xd2, 0xa0, 0xfe, 0xc4, 0xc0, 0x7d, 0x63, 0x57, 0x71,
0x0a, 0xe8, 0x16, 0x68, 0x8a, 0x33, 0x9b, 0x57, 0xd4, 0x7f, 0x97, 0x87, 0x75, 0x19, 0xd7, 0x93,
0x17, 0xad, 0x17, 0x3f, 0x2d, 0x2d, 0x1f, 0x7a, 0x9b, 0xb0, 0xea, 0x11, 0x96, 0xd8, 0xa1, 0x8a,
0x63, 0x12, 0xb9, 0x50, 0xb3, 0x7c, 0x3f, 0x88, 0xc4, 0xb3, 0x08, 0x53, 0x21, 0xf2, 0xf1, 0x42,
0x0f, 0x30, 0x89, 0xe4, 0xed, 0xce, 0x0c, 0x49, 0x46, 0xc8, 0x34, 0x76, 0xeb, 0x03, 0xd0, 0x2e,
0x4e, 0x58, 0x24, 0x2f, 0xbd, 0xf1, 0xce, 0x2c, 0x2d, 0x11, 0xee, 0xa0, 0x87, 0xfd, 0x27, 0xfd,
0xfd, 0xa7, 0x7d, 0x6d, 0x85, 0x13, 0xf8, 0xb0, 0xdf, 0xdf, 0xe9, 0x3f, 0xd6, 0x72, 0x08, 0xa0,
0x6c, 0x7c, 0xba, 0x33, 0x34, 0x7a, 0x5a, 0x7e, 0xeb, 0x1f, 0x6b, 0x50, 0x96, 0x42, 0xa2, 0x5f,
0xab, 0x94, 0x9c, 0x6e, 0x3b, 0xa2, 0x0f, 0x16, 0x2e, 0x6d, 0xe7, 0x5a, 0x99, 0xad, 0x0f, 0x97,
0x5e, 0xaf, 0x5e, 0x16, 0x57, 0xd0, 0xcf, 0x73, 0x50, 0x9f, 0x7b, 0x4a, 0xcb, 0xfa, 0x0a, 0x74,
0x45, 0x97, 0xb3, 0xf5, 0x8d, 0xa5, 0xd6, 0x26, 0xb2, 0xfc, 0x2c, 0x07, 0xb5, 0x54, 0x7f, 0x0f,
0x3d, 0x58, 0xa6, 0x27, 0x28, 0x25, 0x79, 0xb8, 0x7c, 0x3b, 0x51, 0x5f, 0x79, 0x3b, 0x87, 0x7e,
0x9a, 0x83, 0x5a, 0xaa, 0x39, 0x96, 0x59, 0x94, 0xcb, 0xad, 0xbc, 0xcc, 0xa2, 0x5c, 0xd5, 0x8b,
0x5b, 0x41, 0x3f, 0xcc, 0x41, 0x35, 0x69, 0x74, 0xa1, 0x7b, 0x8b, 0xb7, 0xc6, 0xa4, 0x10, 0xf7,
0x97, 0xed, 0xa9, 0xe9, 0x2b, 0xe8, 0xfb, 0x50, 0x89, 0xbb, 0x42, 0x28, 0x6b, 0x1a, 0xb9, 0xd0,
0x72, 0x6a, 0xdd, 0x5b, 0x78, 0x5d, 0x7a, 0xfb, 0xb8, 0x55, 0x93, 0x79, 0xfb, 0x0b, 0x4d, 0xa5,
0xd6, 0xbd, 0x85, 0xd7, 0x25, 0xdb, 0x73, 0x4f, 0x48, 0x75, 0x74, 0x32, 0x7b, 0xc2, 0xe5, 0x56,
0x52, 0x66, 0x4f, 0xb8, 0xaa, 0x81, 0x24, 0x05, 0x49, 0xf5, 0x84, 0x32, 0x0b, 0x72, 0xb9, 0xef,
0x94, 0x59, 0x90, 0x2b, 0x5a, 0x50, 0xca, 0x25, 0x67, 0x05, 0xfa, 0xbd, 0x85, 0xdb, 0x28, 0x0b,
0xba, 0xe4, 0xa5, 0x46, 0x8e, 0xbe, 0x82, 0x7e, 0xa4, 0x9e, 0x0c, 0x64, 0x0f, 0x06, 0x2d, 0x02,
0x35, 0xd7, 0xb6, 0x69, 0xbd, 0xb7, 0x5c, 0xaa, 0x11, 0x31, 0xe2, 0xc7, 0x39, 0x80, 0x59, 0xb7,
0x26, 0xb3, 0x10, 0x97, 0xda, 0x44, 0xad, 0x07, 0x4b, 0xac, 0x4c, 0x1f, 0x8f, 0xb8, 0x41, 0x93,
0xf9, 0x78, 0x5c, 0xe8, 0x26, 0x65, 0x3e, 0x1e, 0x17, 0x3b, 0x41, 0xfa, 0xca, 0xc7, 0xab, 0xdf,
0x2a, 0xc9, 0xdc, 0x5f, 0x16, 0x3f, 0xef, 0xfe, 0x27, 0x00, 0x00, 0xff, 0xff, 0x44, 0x47, 0xcf,
0xbe, 0x32, 0x24, 0x00, 0x00,
0x43, 0xad, 0x84, 0x20, 0x48, 0x90, 0x00, 0x41, 0x12, 0x20, 0x40, 0x2e, 0x41, 0xee, 0xb9, 0x05,
0xb9, 0xe6, 0x94, 0xc7, 0x25, 0x40, 0xfe, 0x87, 0x1c, 0x73, 0x09, 0x90, 0x6b, 0xfe, 0x82, 0x04,
0xfd, 0x98, 0xe1, 0x50, 0xd2, 0xda, 0x43, 0x2a, 0x27, 0x4e, 0x55, 0x77, 0xff, 0xba, 0x58, 0x55,
0x5d, 0x55, 0xdd, 0x05, 0x75, 0x87, 0xba, 0xa7, 0x84, 0xb6, 0x43, 0x1a, 0x44, 0x01, 0x7a, 0x7d,
0x6c, 0xb1, 0xb1, 0x6b, 0x07, 0x34, 0x6c, 0xfb, 0x81, 0x67, 0x39, 0xed, 0x70, 0x32, 0x1d, 0xb9,
0x3e, 0x6b, 0xcb, 0x59, 0x4c, 0x4e, 0x6b, 0xfd, 0xdf, 0x28, 0x08, 0x46, 0x13, 0xb2, 0x29, 0xa8,
0xa3, 0xe9, 0xf1, 0xa6, 0x33, 0xa5, 0x56, 0xe4, 0x06, 0xbe, 0x1a, 0x7f, 0xf5, 0xe2, 0x78, 0xe4,
0x7a, 0x84, 0x45, 0x96, 0x17, 0xaa, 0x09, 0x1f, 0x8d, 0xdc, 0x68, 0x3c, 0x3d, 0x6a, 0xdb, 0x81,
0xb7, 0x99, 0x6c, 0xb9, 0x29, 0xb6, 0xdc, 0x54, 0x5b, 0x6e, 0xb2, 0xb1, 0x45, 0x89, 0xb3, 0x39,
0xb6, 0x27, 0x2c, 0x24, 0x36, 0xff, 0x35, 0xf9, 0x87, 0x44, 0xd0, 0x5f, 0x86, 0x97, 0x86, 0x16,
0x3b, 0xe9, 0x06, 0xfe, 0xb1, 0x3b, 0x1a, 0xd8, 0x63, 0xe2, 0x59, 0x98, 0x7c, 0x39, 0x25, 0x2c,
0xd2, 0xbf, 0x03, 0xcd, 0xcb, 0x43, 0x2c, 0x0c, 0x7c, 0x46, 0xd0, 0x47, 0x50, 0xe4, 0x20, 0xcd,
0xdc, 0x9d, 0xdc, 0x46, 0x6d, 0xeb, 0xad, 0xf6, 0x8b, 0xfe, 0xaf, 0xdc, 0xbc, 0xad, 0x36, 0x6f,
0x0f, 0x42, 0x62, 0x63, 0xb1, 0x52, 0xbf, 0x0d, 0x37, 0xbb, 0x56, 0x68, 0x1d, 0xb9, 0x13, 0x37,
0x72, 0x09, 0x8b, 0x37, 0x9d, 0xc2, 0xad, 0x79, 0xb6, 0xda, 0xf0, 0x33, 0xa8, 0xdb, 0x29, 0xbe,
0xda, 0xf8, 0x41, 0x3b, 0x93, 0xa2, 0xdb, 0x3d, 0x41, 0xcd, 0x01, 0xcf, 0xc1, 0xe9, 0xb7, 0x00,
0x3d, 0x72, 0xfd, 0x11, 0xa1, 0x21, 0x75, 0xfd, 0x28, 0x16, 0xe6, 0xe7, 0x05, 0xb8, 0x39, 0xc7,
0x56, 0xc2, 0x7c, 0x01, 0x60, 0x45, 0x11, 0x75, 0x8f, 0xa6, 0x91, 0x10, 0xa5, 0xb0, 0x51, 0xdb,
0xfa, 0x24, 0xa3, 0x28, 0x57, 0xe0, 0xb5, 0x3b, 0x09, 0x98, 0xe1, 0x47, 0xf4, 0x1c, 0xa7, 0xd0,
0xd1, 0xe7, 0x50, 0x1e, 0x13, 0x6b, 0x12, 0x8d, 0x9b, 0xf9, 0x3b, 0xb9, 0x8d, 0xc6, 0xd6, 0xa3,
0x6b, 0xec, 0xb3, 0x2d, 0x80, 0x06, 0x91, 0x15, 0x11, 0xac, 0x50, 0xd1, 0x5d, 0x40, 0xf2, 0xcb,
0x74, 0x08, 0xb3, 0xa9, 0x1b, 0x72, 0xff, 0x6b, 0x16, 0xee, 0xe4, 0x36, 0xaa, 0xf8, 0x86, 0x1c,
0xe9, 0xcd, 0x06, 0x5a, 0xef, 0xc3, 0xfa, 0x05, 0x69, 0x91, 0x06, 0x85, 0x13, 0x72, 0x2e, 0x2c,
0x52, 0xc5, 0xfc, 0x13, 0xdd, 0x82, 0xd2, 0xa9, 0x35, 0x99, 0x12, 0x21, 0x72, 0x15, 0x4b, 0xe2,
0x61, 0xfe, 0x7e, 0x4e, 0x7f, 0x00, 0xb5, 0x94, 0x10, 0xa8, 0x01, 0x70, 0xd8, 0xef, 0x19, 0x43,
0xa3, 0x3b, 0x34, 0x7a, 0xda, 0x0a, 0x5a, 0x83, 0xea, 0x61, 0x7f, 0xdb, 0xe8, 0xec, 0x0e, 0xb7,
0x9f, 0x69, 0x39, 0x54, 0x83, 0xd5, 0x98, 0xc8, 0xeb, 0x67, 0x80, 0x30, 0xb1, 0x83, 0x53, 0x42,
0xb9, 0x57, 0x2a, 0x13, 0xa1, 0x97, 0x60, 0x35, 0xb2, 0xd8, 0x89, 0xe9, 0x3a, 0x4a, 0x80, 0x32,
0x27, 0x77, 0x1c, 0xb4, 0x03, 0xe5, 0xb1, 0xe5, 0x3b, 0x13, 0x29, 0x44, 0x6d, 0xeb, 0x9d, 0x8c,
0x7a, 0xe3, 0xe0, 0xdb, 0x62, 0x21, 0x56, 0x00, 0xdc, 0x55, 0xe7, 0x76, 0x96, 0xda, 0xd4, 0x9f,
0x81, 0x36, 0x88, 0x2c, 0x1a, 0xa5, 0xc5, 0x31, 0xa0, 0xc8, 0xf7, 0x57, 0xee, 0xb9, 0xc8, 0x9e,
0xf2, 0x98, 0x61, 0xb1, 0x5c, 0xff, 0x57, 0x1e, 0x6e, 0xa4, 0xb0, 0x95, 0xdb, 0x3d, 0x85, 0x32,
0x25, 0x6c, 0x3a, 0x89, 0x04, 0x7c, 0x63, 0xeb, 0xc3, 0x8c, 0xf0, 0x97, 0x90, 0xda, 0x58, 0xc0,
0x60, 0x05, 0x87, 0x36, 0x40, 0x93, 0x2b, 0x4c, 0x42, 0x69, 0x40, 0x4d, 0x8f, 0x8d, 0x94, 0xe9,
0x1a, 0x92, 0x6f, 0x70, 0xf6, 0x1e, 0x1b, 0xa5, 0xb4, 0x5a, 0xb8, 0xa6, 0x56, 0x91, 0x05, 0x9a,
0x4f, 0xa2, 0xe7, 0x01, 0x3d, 0x31, 0xb9, 0x6a, 0xa9, 0xeb, 0x90, 0x66, 0x51, 0x80, 0xbe, 0x97,
0x11, 0xb4, 0x2f, 0x97, 0xef, 0xab, 0xd5, 0x78, 0xdd, 0x9f, 0x67, 0xe8, 0x6f, 0x42, 0x59, 0xfe,
0x53, 0xee, 0x49, 0x83, 0xc3, 0x6e, 0xd7, 0x18, 0x0c, 0xb4, 0x15, 0x54, 0x85, 0x12, 0x36, 0x86,
0x98, 0x7b, 0x58, 0x15, 0x4a, 0x8f, 0x3a, 0xc3, 0xce, 0xae, 0x96, 0xd7, 0xdf, 0x80, 0xf5, 0xa7,
0x96, 0x1b, 0x65, 0x71, 0x2e, 0x3d, 0x00, 0x6d, 0x36, 0x57, 0x59, 0x67, 0x67, 0xce, 0x3a, 0xd9,
0x55, 0x63, 0x9c, 0xb9, 0xd1, 0x05, 0x7b, 0x68, 0x50, 0x20, 0x94, 0x2a, 0x13, 0xf0, 0x4f, 0xfd,
0x39, 0xac, 0x0f, 0xa2, 0x20, 0xcc, 0xe4, 0xf9, 0xef, 0xc2, 0x2a, 0xcf, 0x13, 0xc1, 0x34, 0x52,
0xae, 0xff, 0x72, 0x5b, 0xe6, 0x91, 0x76, 0x9c, 0x47, 0xda, 0x3d, 0x95, 0x67, 0x70, 0x3c, 0x13,
0xfd, 0x0f, 0x94, 0x99, 0x3b, 0xf2, 0xad, 0x89, 0x3a, 0xfa, 0x8a, 0xd2, 0x11, 0x77, 0xf2, 0x78,
0x63, 0xe5, 0xf8, 0x5d, 0x40, 0x3d, 0xc2, 0x22, 0x1a, 0x9c, 0x67, 0x92, 0xe7, 0x16, 0x94, 0x8e,
0x03, 0x6a, 0xcb, 0x83, 0x58, 0xc1, 0x92, 0xe0, 0x87, 0x6a, 0x0e, 0x44, 0x61, 0xdf, 0x05, 0xb4,
0xe3, 0xf3, 0x04, 0x91, 0xcd, 0x10, 0xbf, 0xcc, 0xc3, 0xcd, 0xb9, 0xf9, 0xca, 0x18, 0xcb, 0x9f,
0x43, 0x1e, 0x98, 0xa6, 0x4c, 0x9e, 0x43, 0xb4, 0x0f, 0x65, 0x39, 0x43, 0x69, 0xf2, 0xde, 0x02,
0x40, 0x32, 0xe7, 0x28, 0x38, 0x05, 0x73, 0xa5, 0xd3, 0x17, 0xfe, 0xdb, 0x4e, 0xaf, 0xc5, 0xff,
0x83, 0x7d, 0xad, 0xfe, 0xbe, 0x0d, 0x37, 0x52, 0x93, 0x95, 0xf2, 0x1e, 0x41, 0x89, 0x71, 0x86,
0xd2, 0xde, 0xdb, 0x0b, 0x6a, 0x8f, 0x61, 0xb9, 0x5c, 0xbf, 0x29, 0xc1, 0x8d, 0x53, 0xe2, 0x27,
0xa2, 0xe8, 0x3d, 0xb8, 0x31, 0x10, 0xae, 0x95, 0xc9, 0x77, 0x66, 0x6e, 0x99, 0x9f, 0x73, 0xcb,
0x5b, 0x80, 0xd2, 0x28, 0xca, 0x79, 0xce, 0x61, 0xdd, 0x38, 0x23, 0x76, 0x26, 0xe4, 0x26, 0xac,
0xda, 0x81, 0xe7, 0x59, 0xbe, 0xd3, 0xcc, 0xdf, 0x29, 0x6c, 0x54, 0x71, 0x4c, 0xa6, 0xcf, 0x4f,
0x21, 0xeb, 0xf9, 0xd1, 0x7f, 0x91, 0x03, 0x6d, 0xb6, 0xb7, 0x52, 0x24, 0x97, 0x3e, 0x72, 0x38,
0x10, 0xdf, 0xbb, 0x8e, 0x15, 0xa5, 0xf8, 0xf1, 0x11, 0x97, 0x7c, 0x42, 0x69, 0x2a, 0x84, 0x14,
0xae, 0x19, 0x42, 0xf4, 0x7f, 0xe4, 0x00, 0x5d, 0xae, 0x7a, 0xd0, 0x6b, 0x50, 0x67, 0xc4, 0x77,
0x4c, 0xa9, 0x46, 0x69, 0xe1, 0x0a, 0xae, 0x71, 0x9e, 0xd4, 0x27, 0x43, 0x08, 0x8a, 0xe4, 0x8c,
0xd8, 0xea, 0xb4, 0x8a, 0x6f, 0x34, 0x86, 0xfa, 0x31, 0x33, 0x5d, 0x16, 0x4c, 0xac, 0xa4, 0x3c,
0x68, 0x6c, 0x19, 0x4b, 0x57, 0x5f, 0xed, 0x47, 0x83, 0x9d, 0x18, 0x0c, 0xd7, 0x8e, 0x59, 0x42,
0xe8, 0x6d, 0xa8, 0xa5, 0xc6, 0x50, 0x05, 0x8a, 0xfd, 0xfd, 0xbe, 0xa1, 0xad, 0x20, 0x80, 0x72,
0x77, 0x1b, 0xef, 0xef, 0x0f, 0x65, 0xd4, 0xde, 0xd9, 0xeb, 0x3c, 0x36, 0xb4, 0xbc, 0xfe, 0xa7,
0x22, 0xc0, 0x2c, 0x7d, 0xa2, 0x06, 0xe4, 0x13, 0x4b, 0xe7, 0x5d, 0x87, 0xff, 0x19, 0xdf, 0xf2,
0xe2, 0x42, 0x44, 0x7c, 0xa3, 0x2d, 0xb8, 0xed, 0xb1, 0x51, 0x68, 0xd9, 0x27, 0xa6, 0xca, 0x7a,
0xb6, 0x58, 0x2c, 0xfe, 0x55, 0x1d, 0xdf, 0x54, 0x83, 0x4a, 0x6a, 0x89, 0xbb, 0x0b, 0x05, 0xe2,
0x9f, 0x36, 0x8b, 0xa2, 0xd4, 0x7b, 0xb8, 0x70, 0x5a, 0x6f, 0x1b, 0xfe, 0xa9, 0x2c, 0xed, 0x38,
0x0c, 0xea, 0x43, 0x95, 0x12, 0x16, 0x4c, 0xa9, 0x4d, 0x58, 0xb3, 0xb4, 0xd0, 0x21, 0xc3, 0xf1,
0x3a, 0x3c, 0x83, 0x40, 0x3d, 0x28, 0x7b, 0xc1, 0xd4, 0x8f, 0x58, 0xb3, 0x2c, 0x04, 0x7c, 0x2b,
0x23, 0xd8, 0x1e, 0x5f, 0x84, 0xd5, 0x5a, 0xf4, 0x18, 0x56, 0x1d, 0x72, 0xea, 0x72, 0x99, 0x56,
0x05, 0xcc, 0xdd, 0xac, 0xf6, 0x15, 0xab, 0x70, 0xbc, 0x9a, 0x2b, 0x7d, 0xca, 0x08, 0x6d, 0x56,
0xa4, 0xd2, 0xf9, 0x37, 0x7a, 0x05, 0xaa, 0xd6, 0x64, 0x12, 0xd8, 0xa6, 0xe3, 0xd2, 0x66, 0x55,
0x0c, 0x54, 0x04, 0xa3, 0xe7, 0x52, 0xf4, 0x2a, 0xd4, 0xe4, 0xc9, 0x30, 0x43, 0x2b, 0x1a, 0x37,
0x41, 0x0c, 0x83, 0x64, 0x1d, 0x58, 0xd1, 0x58, 0x4d, 0x20, 0x94, 0xca, 0x09, 0xb5, 0x64, 0x02,
0xa1, 0x94, 0x4f, 0x68, 0xbd, 0x07, 0x95, 0x58, 0xc5, 0x0b, 0xd5, 0xa3, 0x7f, 0xcd, 0x41, 0x35,
0x51, 0x29, 0xfa, 0x14, 0xd6, 0xa8, 0xf5, 0xdc, 0x9c, 0xd9, 0x46, 0x06, 0xc0, 0x77, 0xb3, 0xda,
0xc6, 0x7a, 0x3e, 0x33, 0x4f, 0x9d, 0xa6, 0x28, 0xf4, 0x39, 0xac, 0x4f, 0x5c, 0x7f, 0x7a, 0x96,
0xc2, 0x96, 0x19, 0xe5, 0xff, 0x33, 0x62, 0xef, 0xf2, 0xd5, 0x33, 0xf4, 0xc6, 0x64, 0x8e, 0xd6,
0x7f, 0x9f, 0x83, 0x7a, 0x7a, 0x7b, 0xae, 0x04, 0x3b, 0x9c, 0x8a, 0x3f, 0x50, 0xc0, 0xfc, 0x93,
0x07, 0x1d, 0x8f, 0x78, 0x01, 0x3d, 0x17, 0x3b, 0x17, 0xb0, 0xa2, 0xb8, 0xb5, 0x1c, 0x97, 0x9d,
0x08, 0xef, 0x2f, 0x60, 0xf1, 0xcd, 0x79, 0x6e, 0x10, 0x32, 0x51, 0x8f, 0x15, 0xb0, 0xf8, 0x46,
0x18, 0x2a, 0x2a, 0xd5, 0x70, 0x9f, 0x2d, 0x2c, 0x9e, 0xb2, 0x62, 0xe1, 0x70, 0x82, 0xa3, 0xff,
0x3a, 0x0f, 0xeb, 0x17, 0x46, 0xb9, 0x9c, 0xd2, 0x91, 0xe2, 0x80, 0x2d, 0x29, 0x2e, 0x93, 0xed,
0x3a, 0x71, 0x55, 0x24, 0xbe, 0xc5, 0x71, 0x0f, 0x55, 0xc5, 0x92, 0x77, 0x43, 0x6e, 0x68, 0xef,
0xc8, 0x8d, 0xa4, 0xe0, 0x25, 0x2c, 0x09, 0xf4, 0x0c, 0x1a, 0x94, 0x30, 0x42, 0x4f, 0x89, 0x63,
0x86, 0x01, 0x8d, 0x62, 0xf9, 0xb7, 0x16, 0x93, 0xff, 0x20, 0xa0, 0x11, 0x5e, 0x8b, 0x91, 0x38,
0xc5, 0xd0, 0x53, 0x58, 0x73, 0xce, 0x7d, 0xcb, 0x73, 0x6d, 0x85, 0x5c, 0x5e, 0x1a, 0xb9, 0xae,
0x80, 0x04, 0x30, 0xbf, 0x28, 0xa5, 0x06, 0xf9, 0x1f, 0x9b, 0x58, 0x47, 0x64, 0xa2, 0x74, 0x22,
0x89, 0x79, 0xbf, 0x2e, 0x29, 0xbf, 0xd6, 0xff, 0x9d, 0x83, 0xc6, 0xbc, 0xbb, 0xa0, 0xff, 0x05,
0xb0, 0xc3, 0xa9, 0x19, 0x12, 0xea, 0x06, 0x8e, 0x72, 0x8a, 0xaa, 0x1d, 0x4e, 0x0f, 0x04, 0x83,
0x1f, 0x4e, 0x3e, 0xfc, 0xe5, 0x34, 0x88, 0x2c, 0xe5, 0x1d, 0x15, 0x3b, 0x9c, 0x7e, 0x93, 0xd3,
0xf1, 0x5a, 0x71, 0x93, 0x67, 0xca, 0x4b, 0xf8, 0xf4, 0x81, 0x60, 0xa0, 0xb7, 0x00, 0x49, 0x47,
0x32, 0x27, 0xae, 0xe7, 0x46, 0xe6, 0xd1, 0x39, 0xbf, 0x13, 0x4b, 0xc7, 0xd1, 0xe4, 0xc8, 0x2e,
0x1f, 0xf8, 0x98, 0xf3, 0x91, 0x0e, 0x6b, 0x41, 0xe0, 0x99, 0xcc, 0x0e, 0x28, 0x31, 0x2d, 0xe7,
0x0b, 0x11, 0xfd, 0x0a, 0xb8, 0x16, 0x04, 0xde, 0x80, 0xf3, 0x3a, 0xce, 0x17, 0xfc, 0xb0, 0xdb,
0xe1, 0x94, 0x91, 0xc8, 0xe4, 0x3f, 0xcd, 0xb2, 0x3c, 0xec, 0x92, 0xd5, 0x0d, 0xa7, 0x2c, 0x35,
0xc1, 0x23, 0x1e, 0x0f, 0x56, 0xa9, 0x09, 0x7b, 0xc4, 0x63, 0xfa, 0x67, 0x50, 0x12, 0xa1, 0x8d,
0xff, 0x31, 0x91, 0xfd, 0x45, 0xd4, 0x90, 0xaa, 0xab, 0x70, 0x86, 0x08, 0x2a, 0xaf, 0x40, 0x75,
0x1c, 0x30, 0x15, 0x73, 0xa4, 0x57, 0x55, 0x38, 0x43, 0x0c, 0xb6, 0xa0, 0x42, 0x89, 0xe5, 0x04,
0xfe, 0xe4, 0x5c, 0xfc, 0xe7, 0x0a, 0x4e, 0x68, 0xfd, 0x4b, 0x28, 0xcb, 0x90, 0x77, 0x0d, 0xfc,
0xbb, 0x80, 0xec, 0x11, 0x0d, 0xa6, 0x21, 0x37, 0x8a, 0xe7, 0x32, 0xe6, 0x06, 0x3e, 0x8b, 0xaf,
0xdd, 0x72, 0xe4, 0x60, 0x36, 0xa0, 0xff, 0x25, 0x27, 0xd3, 0x9c, 0xbc, 0x43, 0xf1, 0x42, 0x41,
0xe5, 0xac, 0xa5, 0x2f, 0x9a, 0x0a, 0x20, 0x2e, 0xf6, 0x88, 0x7a, 0x5e, 0x58, 0xb4, 0xd8, 0x23,
0xb2, 0xd8, 0x23, 0xbc, 0xb2, 0x50, 0xd9, 0x54, 0xc2, 0xc9, 0x64, 0x5a, 0x73, 0x92, 0x2a, 0x98,
0xe8, 0xff, 0xcc, 0x25, 0xa7, 0x3d, 0xae, 0x56, 0xd1, 0xe7, 0x50, 0xe1, 0x07, 0xc7, 0xf4, 0xac,
0x50, 0x3d, 0xa4, 0x74, 0x97, 0x2b, 0x84, 0xdb, 0xfc, 0x9c, 0xec, 0x59, 0xa1, 0x4c, 0xb3, 0xab,
0xa1, 0xa4, 0x78, 0xd4, 0xb0, 0x9c, 0x59, 0xd4, 0xe0, 0xdf, 0xe8, 0x75, 0x68, 0x58, 0xd3, 0x28,
0x30, 0x2d, 0xe7, 0x94, 0xd0, 0xc8, 0x65, 0x44, 0x59, 0x78, 0x8d, 0x73, 0x3b, 0x31, 0xb3, 0xf5,
0x10, 0xea, 0x69, 0xcc, 0xaf, 0xcb, 0x2b, 0xa5, 0x74, 0x5e, 0xf9, 0x2e, 0xc0, 0xac, 0x28, 0xe3,
0x9e, 0x40, 0xce, 0xdc, 0xc8, 0xb4, 0x03, 0x47, 0x46, 0xb5, 0x12, 0xae, 0x70, 0x46, 0x37, 0x70,
0xc8, 0x85, 0x12, 0xb7, 0x14, 0x97, 0xb8, 0xfc, 0xdc, 0xf1, 0xa3, 0x72, 0xe2, 0x4e, 0x26, 0xc4,
0x51, 0x12, 0x56, 0x83, 0xc0, 0x7b, 0x22, 0x18, 0xfa, 0x9f, 0xf3, 0xd2, 0x23, 0xe4, 0x05, 0x23,
0x53, 0xe1, 0x93, 0x98, 0xba, 0x70, 0x3d, 0x53, 0x3f, 0x00, 0x60, 0x91, 0x45, 0x23, 0xe2, 0x98,
0x56, 0xa4, 0xee, 0xec, 0xad, 0x4b, 0x35, 0xf2, 0x30, 0x7e, 0xab, 0xc4, 0x55, 0x35, 0xbb, 0x13,
0xa1, 0xf7, 0xa1, 0x6e, 0x07, 0x5e, 0x38, 0x21, 0x6a, 0x71, 0xe9, 0x6b, 0x17, 0xd7, 0x92, 0xf9,
0x9d, 0x28, 0x55, 0x20, 0x97, 0xaf, 0x5b, 0x20, 0xff, 0x21, 0x27, 0xef, 0x49, 0xe9, 0x6b, 0x1a,
0x1a, 0x5d, 0xf1, 0xb0, 0xf7, 0x78, 0xc9, 0x3b, 0xdf, 0x57, 0xbd, 0xea, 0x5d, 0xf7, 0x19, 0xed,
0x8f, 0x05, 0xa8, 0x26, 0xd7, 0xad, 0x4b, 0xb6, 0xbf, 0x0f, 0xd5, 0xe4, 0xa1, 0x58, 0x95, 0x19,
0x5f, 0x69, 0x9e, 0x64, 0x32, 0x3a, 0x06, 0x64, 0x8d, 0x46, 0x49, 0x91, 0x62, 0x4e, 0x99, 0x35,
0x8a, 0x2f, 0xa8, 0xf7, 0x17, 0xd0, 0x43, 0x9c, 0x79, 0x0e, 0xf9, 0x7a, 0xac, 0x59, 0xa3, 0xd1,
0x1c, 0x07, 0x7d, 0x0f, 0x6e, 0xcf, 0xef, 0x61, 0x1e, 0x9d, 0x9b, 0xa1, 0xeb, 0xa8, 0x02, 0x7b,
0x7b, 0xd1, 0x1b, 0x67, 0x7b, 0x0e, 0xfe, 0xe3, 0xf3, 0x03, 0xd7, 0x91, 0x3a, 0x47, 0xf4, 0xd2,
0x40, 0xeb, 0x07, 0xf0, 0xd2, 0x0b, 0xa6, 0x5f, 0x61, 0x83, 0x7e, 0xda, 0x06, 0xd7, 0x51, 0x42,
0xca, 0x7a, 0xbf, 0xc9, 0xc9, 0x8b, 0xf1, 0xbc, 0x4e, 0x3a, 0xb3, 0x8a, 0xad, 0xb6, 0xb5, 0x99,
0x71, 0x9f, 0xee, 0xc1, 0xa1, 0x84, 0x17, 0x25, 0xde, 0x27, 0x73, 0x25, 0x5e, 0xf6, 0x32, 0x64,
0x4f, 0x2c, 0x92, 0x40, 0x0a, 0x41, 0xff, 0x5d, 0x01, 0x2a, 0x31, 0xba, 0xa8, 0xbf, 0xcf, 0x59,
0x44, 0x3c, 0xd3, 0x8b, 0x43, 0x58, 0x0e, 0x83, 0x64, 0xed, 0xf1, 0x20, 0xf6, 0x0a, 0x54, 0x79,
0x99, 0x2f, 0x87, 0xf3, 0x62, 0xb8, 0xc2, 0x19, 0x62, 0xf0, 0x55, 0xa8, 0x45, 0x41, 0x64, 0x4d,
0xcc, 0xc8, 0xb5, 0x4f, 0x64, 0x92, 0xcb, 0x61, 0x10, 0xac, 0x21, 0xe7, 0xa0, 0x37, 0xe1, 0x46,
0x34, 0xa6, 0x41, 0x14, 0x4d, 0x78, 0x85, 0x26, 0x6a, 0x12, 0x59, 0x42, 0x14, 0xb1, 0x96, 0x0c,
0xc8, 0x5a, 0x85, 0xf1, 0xe8, 0x3d, 0x9b, 0xcc, 0x5d, 0x57, 0x04, 0x91, 0x22, 0x5e, 0x4b, 0xb8,
0xdc, 0xb5, 0xf9, 0xfd, 0x3e, 0x24, 0xd4, 0x26, 0xbe, 0x8c, 0x15, 0x39, 0x1c, 0x93, 0xc8, 0x84,
0x75, 0x8f, 0x58, 0x6c, 0x4a, 0x89, 0x63, 0x1e, 0xbb, 0x64, 0xe2, 0xc8, 0xfb, 0x4e, 0x23, 0x73,
0x3d, 0x1b, 0xab, 0xa5, 0xfd, 0x48, 0xac, 0xc6, 0x8d, 0x18, 0x4e, 0xd2, 0xbc, 0x3e, 0x90, 0x5f,
0x68, 0x1d, 0x6a, 0x83, 0x67, 0x83, 0xa1, 0xb1, 0x67, 0xee, 0xed, 0xf7, 0x0c, 0xf5, 0xc0, 0x3d,
0x30, 0xb0, 0x24, 0x73, 0x7c, 0x7c, 0xb8, 0x3f, 0xec, 0xec, 0x9a, 0xc3, 0x9d, 0xee, 0x93, 0x81,
0x96, 0x47, 0xb7, 0xe1, 0xc6, 0x70, 0x1b, 0xef, 0x0f, 0x87, 0xbb, 0x46, 0xcf, 0x3c, 0x30, 0xf0,
0xce, 0x7e, 0x6f, 0xa0, 0x15, 0x10, 0x82, 0xc6, 0x8c, 0x3d, 0xdc, 0xd9, 0x33, 0xb4, 0x22, 0xaa,
0xc1, 0xea, 0x81, 0x81, 0xbb, 0x46, 0x7f, 0xa8, 0x95, 0xf4, 0xbf, 0xe5, 0xa1, 0x96, 0xb2, 0x22,
0x77, 0x64, 0xca, 0xe4, 0xfd, 0xa5, 0x88, 0xf9, 0x27, 0x0f, 0x26, 0xb6, 0x65, 0x8f, 0xa5, 0x75,
0x8a, 0x58, 0x12, 0xdc, 0x6e, 0x9e, 0x75, 0x96, 0x3a, 0xe7, 0x45, 0x5c, 0xf1, 0xac, 0x33, 0x09,
0xf2, 0x1a, 0xd4, 0x4f, 0x08, 0xf5, 0xc9, 0x44, 0x8d, 0x4b, 0x8b, 0xd4, 0x24, 0x4f, 0x4e, 0xd9,
0x00, 0x4d, 0x4d, 0x99, 0xc1, 0x48, 0x73, 0x34, 0x24, 0x7f, 0x2f, 0x06, 0x3b, 0xba, 0xac, 0xf5,
0xb2, 0xd0, 0xfa, 0x83, 0xc5, 0x9d, 0xf4, 0x45, 0x8a, 0x1f, 0x24, 0x8a, 0x5f, 0x85, 0x02, 0x8e,
0xdf, 0x7a, 0xbb, 0x9d, 0xee, 0x36, 0x57, 0xf6, 0x1a, 0x54, 0xf7, 0x3a, 0x9f, 0x9a, 0x87, 0x03,
0xf1, 0x72, 0x80, 0x34, 0xa8, 0x3f, 0x31, 0x70, 0xdf, 0xd8, 0x55, 0x9c, 0x02, 0xba, 0x05, 0x9a,
0xe2, 0xcc, 0xe6, 0x15, 0xf5, 0xdf, 0xe6, 0x61, 0x5d, 0xc6, 0xf5, 0xe4, 0x31, 0xeb, 0xc5, 0xaf,
0x4a, 0xcb, 0x87, 0xde, 0x26, 0xac, 0x7a, 0x84, 0x25, 0x76, 0xa8, 0xe2, 0x98, 0x44, 0x2e, 0xd4,
0x2c, 0xdf, 0x0f, 0x22, 0xf1, 0x22, 0xc2, 0x54, 0x88, 0x7c, 0xbc, 0xd0, 0xdb, 0x4b, 0x22, 0x79,
0xbb, 0x33, 0x43, 0x92, 0x11, 0x32, 0x8d, 0xdd, 0xfa, 0x00, 0xb4, 0x8b, 0x13, 0x16, 0xc9, 0x4b,
0x6f, 0xbc, 0x33, 0x4b, 0x4b, 0x84, 0x3b, 0xe8, 0x61, 0xff, 0x49, 0x7f, 0xff, 0x69, 0x5f, 0x5b,
0xe1, 0x04, 0x3e, 0xec, 0xf7, 0x77, 0xfa, 0x8f, 0xb5, 0x1c, 0x02, 0x28, 0x1b, 0x9f, 0xee, 0x0c,
0x8d, 0x9e, 0x96, 0xdf, 0xfa, 0xfb, 0x1a, 0x94, 0xa5, 0x90, 0xe8, 0x57, 0x2a, 0x25, 0xa7, 0x3b,
0x8e, 0xe8, 0x83, 0x85, 0x4b, 0xdb, 0xb9, 0x2e, 0x66, 0xeb, 0xc3, 0xa5, 0xd7, 0xab, 0x47, 0xc5,
0x15, 0xf4, 0xb3, 0x1c, 0xd4, 0xe7, 0x5e, 0xd1, 0xb2, 0x3e, 0x00, 0x5d, 0xd1, 0xe0, 0x6c, 0x7d,
0x63, 0xa9, 0xb5, 0x89, 0x2c, 0x3f, 0xcd, 0x41, 0x2d, 0xd5, 0xda, 0x43, 0x0f, 0x96, 0x69, 0x07,
0x4a, 0x49, 0x1e, 0x2e, 0xdf, 0x49, 0xd4, 0x57, 0xde, 0xce, 0xa1, 0x9f, 0xe4, 0xa0, 0x96, 0xea,
0x8b, 0x65, 0x16, 0xe5, 0x72, 0x17, 0x2f, 0xb3, 0x28, 0x57, 0xb5, 0xe1, 0x56, 0xd0, 0x0f, 0x73,
0x50, 0x4d, 0x7a, 0x5c, 0xe8, 0xde, 0xe2, 0x5d, 0x31, 0x29, 0xc4, 0xfd, 0x65, 0xdb, 0x69, 0xfa,
0x0a, 0xfa, 0x3e, 0x54, 0xe2, 0x86, 0x10, 0xca, 0x9a, 0x46, 0x2e, 0x74, 0x9b, 0x5a, 0xf7, 0x16,
0x5e, 0x97, 0xde, 0x3e, 0xee, 0xd2, 0x64, 0xde, 0xfe, 0x42, 0x3f, 0xa9, 0x75, 0x6f, 0xe1, 0x75,
0xc9, 0xf6, 0xdc, 0x13, 0x52, 0xcd, 0x9c, 0xcc, 0x9e, 0x70, 0xb9, 0x8b, 0x94, 0xd9, 0x13, 0xae,
0xea, 0x1d, 0x49, 0x41, 0x52, 0xed, 0xa0, 0xcc, 0x82, 0x5c, 0x6e, 0x39, 0x65, 0x16, 0xe4, 0x8a,
0xee, 0x93, 0x72, 0xc9, 0x59, 0x81, 0x7e, 0x6f, 0xe1, 0x0e, 0xca, 0x82, 0x2e, 0x79, 0xa9, 0x87,
0xa3, 0xaf, 0xa0, 0x1f, 0xa9, 0x27, 0x03, 0xd9, 0x7e, 0x41, 0x8b, 0x40, 0xcd, 0x75, 0x6c, 0x5a,
0xef, 0x2d, 0x97, 0x6a, 0x44, 0x8c, 0xf8, 0x71, 0x0e, 0x60, 0xd6, 0xa8, 0xc9, 0x2c, 0xc4, 0xa5,
0x0e, 0x51, 0xeb, 0xc1, 0x12, 0x2b, 0xd3, 0xc7, 0x23, 0xee, 0xcd, 0x64, 0x3e, 0x1e, 0x17, 0x1a,
0x49, 0x99, 0x8f, 0xc7, 0xc5, 0x26, 0x90, 0xbe, 0xf2, 0xf1, 0xea, 0xb7, 0x4a, 0x32, 0xf7, 0x97,
0xc5, 0xcf, 0xbb, 0xff, 0x09, 0x00, 0x00, 0xff, 0xff, 0x34, 0x96, 0x6b, 0x66, 0x2d, 0x24, 0x00,
0x00,
}

View File

@@ -373,7 +373,7 @@ message LinuxResources {
// CPU shares (relative weight vs. other containers). Default: 0 (not specified)
int64 cpu_shares = 3;
// Memory limit in bytes. Default: 0 (not specified)
int64 memory_limit_in_bytes = 4;
int64 memory_limit_bytes = 4;
// OOMScoreAdj adjusts the oom-killer score. Default: 0 (not specified)
int64 oom_score_adj = 5;
// CpusetCpus constrains the allowed set of logical CPUs. Default: "" (not specified)

View File

@@ -8,6 +8,7 @@ import (
"github.com/golang/protobuf/ptypes"
hclog "github.com/hashicorp/go-hclog"
plugin "github.com/hashicorp/go-plugin"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/plugins/drivers/proto"
)
@@ -42,12 +43,14 @@ func (b *driverPluginServer) Capabilities(ctx context.Context, req *proto.Capabi
}
switch caps.FSIsolation {
case FSIsolationNone:
case cstructs.FSIsolationNone:
resp.Capabilities.FsIsolation = proto.DriverCapabilities_NONE
case FSIsolationChroot:
case cstructs.FSIsolationChroot:
resp.Capabilities.FsIsolation = proto.DriverCapabilities_CHROOT
case FSIsolationImage:
case cstructs.FSIsolationImage:
resp.Capabilities.FsIsolation = proto.DriverCapabilities_IMAGE
default:
resp.Capabilities.FsIsolation = proto.DriverCapabilities_NONE
}
return resp, nil
}
@@ -91,13 +94,26 @@ func (b *driverPluginServer) RecoverTask(ctx context.Context, req *proto.Recover
}
func (b *driverPluginServer) StartTask(ctx context.Context, req *proto.StartTaskRequest) (*proto.StartTaskResponse, error) {
handle, err := b.impl.StartTask(taskConfigFromProto(req.Task))
handle, net, err := b.impl.StartTask(taskConfigFromProto(req.Task))
if err != nil {
return nil, err
}
var pbNet *proto.NetworkOverride
if net != nil {
pbNet = &proto.NetworkOverride{
PortMap: map[string]int32{},
Addr: net.IP,
AutoAdvertise: net.AutoAdvertise,
}
for k, v := range net.PortMap {
pbNet.PortMap[k] = int32(v)
}
}
resp := &proto.StartTaskResponse{
Handle: taskHandleToProto(handle),
Handle: taskHandleToProto(handle),
NetworkOverride: pbNet,
}
return resp, nil
@@ -159,16 +175,24 @@ func (b *driverPluginServer) InspectTask(ctx context.Context, req *proto.Inspect
return nil, err
}
var pbNet *proto.NetworkOverride
if status.NetworkOverride != nil {
pbNet = &proto.NetworkOverride{
PortMap: map[string]int32{},
Addr: status.NetworkOverride.IP,
AutoAdvertise: status.NetworkOverride.AutoAdvertise,
}
for k, v := range status.NetworkOverride.PortMap {
pbNet.PortMap[k] = int32(v)
}
}
resp := &proto.InspectTaskResponse{
Task: protoStatus,
Driver: &proto.TaskDriverStatus{
Attributes: status.DriverAttributes,
},
NetworkOverride: &proto.NetworkOverride{
PortMap: status.NetworkOverride.PortMap,
Addr: status.NetworkOverride.Addr,
AutoAdvertise: status.NetworkOverride.AutoAdvertise,
},
NetworkOverride: pbNet,
}
return resp, nil

View File

@@ -15,6 +15,7 @@ import (
plugin "github.com/hashicorp/go-plugin"
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/logmon"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/helper/testlog"
"github.com/hashicorp/nomad/helper/uuid"
"github.com/hashicorp/nomad/plugins/base"
@@ -150,12 +151,12 @@ type MockDriver struct {
FingerprintF func(context.Context) (<-chan *Fingerprint, error)
CapabilitiesF func() (*Capabilities, error)
RecoverTaskF func(*TaskHandle) error
StartTaskF func(*TaskConfig) (*TaskHandle, error)
StartTaskF func(*TaskConfig) (*TaskHandle, *cstructs.DriverNetwork, error)
WaitTaskF func(context.Context, string) (<-chan *ExitResult, error)
StopTaskF func(string, time.Duration, string) error
DestroyTaskF func(string, bool) error
InspectTaskF func(string) (*TaskStatus, error)
TaskStatsF func(string) (*TaskStats, error)
TaskStatsF func(string) (*cstructs.TaskResourceUsage, error)
TaskEventsF func(context.Context) (<-chan *TaskEvent, error)
SignalTaskF func(string, string) error
ExecTaskF func(string, []string, time.Duration) (*ExecTaskResult, error)
@@ -165,9 +166,11 @@ func (d *MockDriver) TaskConfigSchema() (*hclspec.Spec, error) { return d.TaskCo
func (d *MockDriver) Fingerprint(ctx context.Context) (<-chan *Fingerprint, error) {
return d.FingerprintF(ctx)
}
func (d *MockDriver) Capabilities() (*Capabilities, error) { return d.CapabilitiesF() }
func (d *MockDriver) RecoverTask(h *TaskHandle) error { return d.RecoverTaskF(h) }
func (d *MockDriver) StartTask(c *TaskConfig) (*TaskHandle, error) { return d.StartTaskF(c) }
func (d *MockDriver) Capabilities() (*Capabilities, error) { return d.CapabilitiesF() }
func (d *MockDriver) RecoverTask(h *TaskHandle) error { return d.RecoverTaskF(h) }
func (d *MockDriver) StartTask(c *TaskConfig) (*TaskHandle, *cstructs.DriverNetwork, error) {
return d.StartTaskF(c)
}
func (d *MockDriver) WaitTask(ctx context.Context, id string) (<-chan *ExitResult, error) {
return d.WaitTaskF(ctx, id)
}
@@ -178,7 +181,9 @@ func (d *MockDriver) DestroyTask(taskID string, force bool) error {
return d.DestroyTaskF(taskID, force)
}
func (d *MockDriver) InspectTask(taskID string) (*TaskStatus, error) { return d.InspectTaskF(taskID) }
func (d *MockDriver) TaskStats(taskID string) (*TaskStats, error) { return d.TaskStats(taskID) }
func (d *MockDriver) TaskStats(taskID string) (*cstructs.TaskResourceUsage, error) {
return d.TaskStats(taskID)
}
func (d *MockDriver) TaskEvents(ctx context.Context) (<-chan *TaskEvent, error) {
return d.TaskEventsF(ctx)
}

View File

@@ -4,7 +4,10 @@ import (
"time"
"github.com/golang/protobuf/ptypes"
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/driver/env"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/drivers/proto"
)
@@ -44,6 +47,19 @@ func healthStateFromProto(pb proto.FingerprintResponse_HealthState) HealthState
return HealthStateUndetected
}
// NewTaskConfig builds a TaskConfig from a Task struct
func NewTaskConfig(task *structs.Task, taskDir *allocdir.TaskDir, env *env.TaskEnv) *TaskConfig {
return &TaskConfig{
Name: task.Name,
Resources: &Resources{
NomadResources: task.Resources,
},
Env: env.Map(),
User: task.User,
AllocDir: taskDir.AllocDir,
}
}
func taskConfigFromProto(pb *proto.TaskConfig) *TaskConfig {
if pb == nil {
return &TaskConfig{}
@@ -53,7 +69,7 @@ func taskConfigFromProto(pb *proto.TaskConfig) *TaskConfig {
Name: pb.Name,
Env: pb.Env,
rawDriverConfig: pb.MsgpackDriverConfig,
Resources: Resources{}, //TODO
Resources: resourcesFromProto(pb.Resources),
Devices: []DeviceConfig{}, //TODO
Mounts: []MountConfig{}, //TODO
User: pb.User,
@@ -71,7 +87,7 @@ func taskConfigToProto(cfg *TaskConfig) *proto.TaskConfig {
Id: cfg.ID,
Name: cfg.Name,
Env: cfg.Env,
Resources: &proto.Resources{},
Resources: resourcesToProto(cfg.Resources),
Mounts: []*proto.Mount{},
Devices: []*proto.Device{},
User: cfg.User,
@@ -83,6 +99,106 @@ func taskConfigToProto(cfg *TaskConfig) *proto.TaskConfig {
return pb
}
func resourcesFromProto(pb *proto.Resources) *Resources {
var r Resources
if pb == nil {
return &r
}
if pb.RawResources != nil {
r.NomadResources = &structs.Resources{
CPU: int(pb.RawResources.Cpu),
MemoryMB: int(pb.RawResources.Memory),
IOPS: int(pb.RawResources.Iops),
DiskMB: int(pb.RawResources.Disk),
}
for _, network := range pb.RawResources.Networks {
var n structs.NetworkResource
n.Device = network.Device
n.IP = network.Ip
n.CIDR = network.Cidr
n.MBits = int(network.Mbits)
for _, port := range network.ReservedPorts {
n.ReservedPorts = append(n.ReservedPorts, structs.Port{
Label: port.Label,
Value: int(port.Value),
})
}
for _, port := range network.DynamicPorts {
n.DynamicPorts = append(n.DynamicPorts, structs.Port{
Label: port.Label,
Value: int(port.Value),
})
}
r.NomadResources.Networks = append(r.NomadResources.Networks, &n)
}
}
if pb.LinuxResources != nil {
r.LinuxResources = &LinuxResources{
CPUPeriod: pb.LinuxResources.CpuPeriod,
CPUQuota: pb.LinuxResources.CpuQuota,
CPUShares: pb.LinuxResources.CpuShares,
MemoryLimitBytes: pb.LinuxResources.MemoryLimitBytes,
OOMScoreAdj: pb.LinuxResources.OomScoreAdj,
CpusetCPUs: pb.LinuxResources.CpusetCpus,
CpusetMems: pb.LinuxResources.CpusetMems,
}
}
return &r
}
func resourcesToProto(r *Resources) *proto.Resources {
var pb proto.Resources
if r.NomadResources != nil {
pb.RawResources = &proto.RawResources{
Cpu: int64(r.NomadResources.CPU),
Memory: int64(r.NomadResources.MemoryMB),
Iops: int64(r.NomadResources.IOPS),
Disk: int64(r.NomadResources.DiskMB),
Networks: []*proto.NetworkResource{},
}
for _, network := range r.NomadResources.Networks {
var n proto.NetworkResource
n.Device = network.Device
n.Ip = network.IP
n.Cidr = network.CIDR
n.Mbits = int32(network.MBits)
n.ReservedPorts = []*proto.NetworkPort{}
for _, port := range network.ReservedPorts {
n.ReservedPorts = append(n.ReservedPorts, &proto.NetworkPort{
Label: port.Label,
Value: int32(port.Value),
})
}
for _, port := range network.DynamicPorts {
n.DynamicPorts = append(n.DynamicPorts, &proto.NetworkPort{
Label: port.Label,
Value: int32(port.Value),
})
}
}
}
if r.LinuxResources != nil {
pb.LinuxResources = &proto.LinuxResources{
CpuPeriod: r.LinuxResources.CPUPeriod,
CpuQuota: r.LinuxResources.CPUQuota,
CpuShares: r.LinuxResources.CPUShares,
MemoryLimitBytes: r.LinuxResources.MemoryLimitBytes,
OomScoreAdj: r.LinuxResources.OOMScoreAdj,
CpusetCpus: r.LinuxResources.CpusetCPUs,
CpusetMems: r.LinuxResources.CpusetMems,
}
}
return &pb
}
func taskHandleFromProto(pb *proto.TaskHandle) *TaskHandle {
if pb == nil {
return &TaskHandle{}
@@ -158,26 +274,25 @@ func taskStatusFromProto(pb *proto.TaskStatus) (*TaskStatus, error) {
}, nil
}
func taskStatsToProto(stats *TaskStats) (*proto.TaskStats, error) {
func taskStatsToProto(stats *cstructs.TaskResourceUsage) (*proto.TaskStats, error) {
timestamp, err := ptypes.TimestampProto(time.Unix(stats.Timestamp, 0))
if err != nil {
return nil, err
}
pids := map[string]*proto.TaskResourceUsage{}
for pid, ru := range stats.ResourceUsageByPid {
for pid, ru := range stats.Pids {
pids[pid] = resourceUsageToProto(ru)
}
return &proto.TaskStats{
Id: stats.ID,
Timestamp: timestamp,
AggResourceUsage: resourceUsageToProto(stats.AggResourceUsage),
AggResourceUsage: resourceUsageToProto(stats.ResourceUsage),
ResourceUsageByPid: pids,
}, nil
}
func taskStatsFromProto(pb *proto.TaskStats) (*TaskStats, error) {
func taskStatsFromProto(pb *proto.TaskStats) (*cstructs.TaskResourceUsage, error) {
timestamp, err := ptypes.Timestamp(pb.Timestamp)
if err != nil {
return nil, err
@@ -188,11 +303,10 @@ func taskStatsFromProto(pb *proto.TaskStats) (*TaskStats, error) {
pids[pid] = resourceUsageFromProto(ru)
}
stats := &TaskStats{
ID: pb.Id,
Timestamp: timestamp.Unix(),
AggResourceUsage: resourceUsageFromProto(pb.AggResourceUsage),
ResourceUsageByPid: pids,
stats := &cstructs.TaskResourceUsage{
Timestamp: timestamp.Unix(),
ResourceUsage: resourceUsageFromProto(pb.AggResourceUsage),
Pids: pids,
}
return stats, nil