drivermanager: use allocID and task name to route task events

This commit is contained in:
Nick Ethier 2018-12-17 22:36:06 -05:00
parent bda32f9c79
commit ce1a5cba0e
No known key found for this signature in database
GPG Key ID: 07C1A3ECED90D24A
13 changed files with 358 additions and 365 deletions

View File

@ -24,6 +24,7 @@ import (
"github.com/hashicorp/nomad/helper"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/device"
"github.com/hashicorp/nomad/plugins/drivers"
)
// allocRunner is used to run all the tasks in a given allocation
@ -891,3 +892,17 @@ func (ar *allocRunner) LatestAllocStats(taskFilter string) (*cstructs.AllocResou
return astat, nil
}
func (ar *allocRunner) GetTaskEventHandler(taskName string) drivermanager.EventHandler {
if tr, ok := ar.tasks[taskName]; ok {
return func(ev *drivers.TaskEvent) {
tr.EmitEvent(&structs.TaskEvent{
Type: structs.TaskDriverMessage,
Time: ev.Timestamp.UnixNano(),
Details: ev.Annotations,
DriverMessage: ev.Message,
})
}
}
return nil
}

View File

@ -567,9 +567,6 @@ func (tr *TaskRunner) runDriver() error {
//XXX Evaluate and encode driver config
// Register an event handler with the driver manager to emit task events
tr.driverManager.RegisterEventHandler(tr.Task().Driver, taskConfig.ID, tr.handleTaskEvent)
// If there's already a task handle (eg from a Restore) there's nothing
// to do except update state.
if tr.getDriverHandle() != nil {
@ -608,46 +605,6 @@ func (tr *TaskRunner) runDriver() error {
return nil
}
func (tr *TaskRunner) handleTaskEvent(ev *drivers.TaskEvent) {
tr.EmitEvent(&structs.TaskEvent{
Type: structs.TaskDriverMessage,
Time: ev.Timestamp.UnixNano(),
Details: ev.Annotations,
DriverMessage: ev.Message,
})
}
// 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 {
driver, err := tr.driverManager.Dispense(tr.Task().Driver)

View File

@ -81,7 +81,6 @@ func (tr *TaskRunner) clearDriverHandle() {
tr.handleLock.Lock()
defer tr.handleLock.Unlock()
if tr.handle != nil {
tr.driverManager.DeregisterEventHandler(tr.Task().Driver, tr.handle.ID())
tr.driver.DestroyTask(tr.handle.ID(), true)
}
tr.handle = nil

View File

@ -123,6 +123,7 @@ type AllocRunner interface {
WaitCh() <-chan struct{}
DestroyCh() <-chan struct{}
ShutdownCh() <-chan struct{}
GetTaskEventHandler(taskName string) drivermanager.EventHandler
}
// Client is used to implement the client interaction with Nomad. Clients
@ -331,13 +332,14 @@ func NewClient(cfg *config.Config, consulCatalog consul.CatalogAPI, consulServic
// Setup the driver manager
driverConfig := &drivermanager.Config{
Logger: c.logger,
Loader: c.configCopy.PluginSingletonLoader,
PluginConfig: c.configCopy.NomadPluginConfig(),
Updater: c.batchNodeUpdates.updateNodeFromDriver,
State: c.stateDB,
AllowedDrivers: allowlistDrivers,
BlockedDrivers: blocklistDrivers,
Logger: c.logger,
Loader: c.configCopy.PluginSingletonLoader,
PluginConfig: c.configCopy.NomadPluginConfig(),
Updater: c.batchNodeUpdates.updateNodeFromDriver,
EventHandlerFactory: c.GetTaskEventHandler,
State: c.stateDB,
AllowedDrivers: allowlistDrivers,
BlockedDrivers: blocklistDrivers,
}
drvManager := drivermanager.New(driverConfig)
c.drivermanager = drvManager
@ -2616,6 +2618,16 @@ func (c *Client) allAllocs() map[string]*structs.Allocation {
return allocs
}
// GetTaskEventHandler returns an event handler for the given allocID and task name
func (c *Client) GetTaskEventHandler(allocID, taskName string) drivermanager.EventHandler {
c.allocLock.RLock()
defer c.allocLock.RUnlock()
if ar, ok := c.allocs[allocID]; ok {
return ar.GetTaskEventHandler(taskName)
}
return nil
}
// group wraps a func() in a goroutine and provides a way to block until it
// exits. Inspired by https://godoc.org/golang.org/x/sync/errgroup
type group struct {

View File

@ -49,6 +49,9 @@ type instanceManagerConfig struct {
// updateNodeFromDriver is the callback used to update the node from fingerprinting
UpdateNodeFromDriver UpdateNodeDriverInfoFn
// EventHandlerFactory is used to fetch a task event handler
EventHandlerFactory TaskEventHandlerFactory
}
// instanceManager is used to manage a single driver plugin
@ -92,9 +95,8 @@ type instanceManager struct {
// updateNodeFromDriver is the callback used to update the node from fingerprinting
updateNodeFromDriver UpdateNodeDriverInfoFn
// handlers is the map of taskID to handler func
handlers map[string]EventHandler
handlersLock sync.RWMutex
// eventHandlerFactory is used to fetch a handler for a task event
eventHandlerFactory TaskEventHandlerFactory
// firstFingerprintCh is used to trigger that we have successfully
// fingerprinted once. It is used to gate launching the stats collection.
@ -122,7 +124,7 @@ func newInstanceManager(c *instanceManagerConfig) *instanceManager {
pluginConfig: c.PluginConfig,
id: c.ID,
updateNodeFromDriver: c.UpdateNodeFromDriver,
handlers: map[string]EventHandler{},
eventHandlerFactory: c.EventHandlerFactory,
firstFingerprintCh: make(chan struct{}),
}
@ -140,23 +142,6 @@ func (i *instanceManager) WaitForFirstFingerprint(ctx context.Context) {
}
}
// registerEventHandler registers the given handler to run for events with the
// given taskID
func (i *instanceManager) registerEventHandler(taskID string, handler EventHandler) {
i.handlersLock.Lock()
defer i.handlersLock.Unlock()
i.handlers[taskID] = handler
}
// deregisterEventHandler removed the handlers registered for the given taskID
// and is serlialized so as to not be called concurrently with the registered
// handler
func (i *instanceManager) deregisterEventHandler(taskID string) {
i.handlersLock.Lock()
defer i.handlersLock.Unlock()
delete(i.handlers, taskID)
}
// run is a long lived goroutine that starts the fingerprinting and stats
// collection goroutine and then shutsdown the plugin on exit.
func (i *instanceManager) run() {
@ -462,9 +447,7 @@ func (i *instanceManager) handleEvents() {
// handleEvent looks up the event handler(s) for the event and runs them
func (i *instanceManager) handleEvent(ev *drivers.TaskEvent) {
i.handlersLock.RLock()
defer i.handlersLock.RUnlock()
if handler, ok := i.handlers[ev.TaskID]; ok {
if handler := i.eventHandlerFactory(ev.AllocID, ev.TaskName); handler != nil {
i.logger.Trace("task event received", "event", ev)
handler(ev)
return

View File

@ -24,23 +24,18 @@ var ErrDriverNotFound = fmt.Errorf("driver not found")
type Manager interface {
pluginmanager.PluginManager
// RegisterEventHandler will cause the given EventHandler to be called when
// an event is received that matches the given driver and taskID
RegisterEventHandler(driver, taskID string, handler EventHandler)
// DeregisterEventHandler stops the EventHandler registered for the given
// driver and taskID to be called if exists
DeregisterEventHandler(driver, taskID string)
// Dispense returns a drivers.DriverPlugin for the given driver plugin name
// handling reattaching to an existing driver if available
Dispense(driver string) (drivers.DriverPlugin, error)
}
// EventHandler can be registered with a Manager to be called for a matching task.
// EventHandler can is a callback to be called for a task.
// The handler should not block execution.
type EventHandler func(*drivers.TaskEvent)
// TaskEventHandlerFactory returns an event handler for a given allocID/task name
type TaskEventHandlerFactory func(allocID, taskName string) EventHandler
// StateStorage is used to persist the driver managers state across
// agent restarts.
type StateStorage interface {
@ -78,6 +73,9 @@ type Config struct {
// Updater is used to update the node when driver information changes
Updater UpdateNodeDriverInfoFn
// EventHandlerFactory is used to retrieve a task event handler
EventHandlerFactory TaskEventHandlerFactory
// State is used to manage the device managers state
State StateStorage
@ -110,6 +108,10 @@ type manager struct {
// updater is used to update the node when device information changes
updater UpdateNodeDriverInfoFn
// eventHandlerFactory is passed to the instance managers and used to forward
// task events
eventHandlerFactory TaskEventHandlerFactory
// instances is the list of managed devices, access is serialized by instanceMu
instances map[string]*instanceManager
instancesMu sync.RWMutex
@ -130,18 +132,19 @@ type manager struct {
func New(c *Config) *manager {
ctx, cancel := context.WithCancel(context.Background())
return &manager{
logger: c.Logger.Named("driver_mgr"),
state: c.State,
ctx: ctx,
cancel: cancel,
loader: c.Loader,
pluginConfig: c.PluginConfig,
updater: c.Updater,
instances: make(map[string]*instanceManager),
reattachConfigs: make(map[loader.PluginID]*shared.ReattachConfig),
allowedDrivers: c.AllowedDrivers,
blockedDrivers: c.BlockedDrivers,
readyCh: make(chan struct{}),
logger: c.Logger.Named("driver_mgr"),
state: c.State,
ctx: ctx,
cancel: cancel,
loader: c.Loader,
pluginConfig: c.PluginConfig,
updater: c.Updater,
eventHandlerFactory: c.EventHandlerFactory,
instances: make(map[string]*instanceManager),
reattachConfigs: make(map[loader.PluginID]*shared.ReattachConfig),
allowedDrivers: c.AllowedDrivers,
blockedDrivers: c.BlockedDrivers,
readyCh: make(chan struct{}),
}
}
@ -189,6 +192,7 @@ func (m *manager) Run() {
PluginConfig: m.pluginConfig,
ID: &id,
UpdateNodeFromDriver: m.updater,
EventHandlerFactory: m.eventHandlerFactory,
})
m.instancesMu.Lock()
@ -324,22 +328,6 @@ func (m *manager) fetchPluginReattachConfig(id loader.PluginID) (*plugin.Reattac
return nil, false
}
func (m *manager) RegisterEventHandler(driver, taskID string, handler EventHandler) {
m.instancesMu.RLock()
if d, ok := m.instances[driver]; ok {
d.registerEventHandler(taskID, handler)
}
m.instancesMu.Unlock()
}
func (m *manager) DeregisterEventHandler(driver, taskID string) {
m.instancesMu.RLock()
if d, ok := m.instances[driver]; ok {
d.deregisterEventHandler(taskID)
}
m.instancesMu.Unlock()
}
func (m *manager) Dispense(d string) (drivers.DriverPlugin, error) {
m.instancesMu.RLock()
defer m.instancesMu.RUnlock()

View File

@ -31,13 +31,14 @@ func testSetup(t *testing.T) (chan *drivers.Fingerprint, chan *drivers.TaskEvent
drv := mockDriver(fpChan, evChan)
cat := mockCatalog(map[string]drivers.DriverPlugin{"mock": drv})
cfg := &Config{
Logger: testlog.HCLogger(t),
Loader: cat,
PluginConfig: &base.ClientAgentConfig{},
Updater: noopUpdater,
State: state.NoopDB{},
AllowedDrivers: make(map[string]struct{}),
BlockedDrivers: make(map[string]struct{}),
Logger: testlog.HCLogger(t),
Loader: cat,
PluginConfig: &base.ClientAgentConfig{},
Updater: noopUpdater,
EventHandlerFactory: noopEventHandlerFactory,
State: state.NoopDB{},
AllowedDrivers: make(map[string]struct{}),
BlockedDrivers: make(map[string]struct{}),
}
mgr := New(cfg)
@ -96,7 +97,8 @@ func mockTaskEvent(taskID string) *drivers.TaskEvent {
}
}
func noopUpdater(string, *structs.DriverInfo) {}
func noopUpdater(string, *structs.DriverInfo) {}
func noopEventHandlerFactory(string, string) EventHandler { return nil }
func TestMananger_Fingerprint(t *testing.T) {
t.Parallel()
@ -183,10 +185,12 @@ func TestMananger_TaskEvents(t *testing.T) {
event1 := mockTaskEvent("abc1")
var wg sync.WaitGroup
wg.Add(1)
mgr.RegisterEventHandler("mock", "abc1", func(ev *drivers.TaskEvent) {
defer wg.Done()
assert.Exactly(t, event1, ev)
})
mgr.eventHandlerFactory = func(string, string) EventHandler {
return func(ev *drivers.TaskEvent) {
defer wg.Done()
assert.Exactly(t, event1, ev)
}
}
evChan <- event1
wg.Wait()

View File

@ -475,6 +475,8 @@ func (d *Driver) pullImage(task *drivers.TaskConfig, driverConfig *TaskConfig, c
d.eventer.EmitEvent(&drivers.TaskEvent{
TaskID: task.ID,
AllocID: task.AllocID,
TaskName: task.Name,
Timestamp: time.Now(),
Message: "Downloading image",
Annotations: map[string]string{
@ -489,6 +491,8 @@ func (d *Driver) emitEventFunc(task *drivers.TaskConfig) LogEventFn {
return func(msg string, annotations map[string]string) {
d.eventer.EmitEvent(&drivers.TaskEvent{
TaskID: task.ID,
AllocID: task.AllocID,
TaskName: task.Name,
Timestamp: time.Now(),
Message: msg,
Annotations: annotations,

View File

@ -310,6 +310,8 @@ func (d *driverPluginClient) handleTaskEvents(ctx context.Context, ch chan *Task
timestamp, _ := ptypes.Timestamp(ev.Timestamp)
event := &TaskEvent{
TaskID: ev.TaskId,
AllocID: ev.AllocId,
TaskName: ev.TaskName,
Annotations: ev.Annotations,
Message: ev.Message,
Timestamp: timestamp,

View File

@ -268,6 +268,8 @@ type TaskStatus struct {
type TaskEvent struct {
TaskID string
TaskName string
AllocID string
Timestamp time.Time
Message string
Annotations map[string]string

View File

@ -50,7 +50,7 @@ func (x TaskState) String() string {
return proto.EnumName(TaskState_name, int32(x))
}
func (TaskState) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{0}
return fileDescriptor_driver_f8179d6379690e6b, []int{0}
}
type FingerprintResponse_HealthState int32
@ -76,7 +76,7 @@ func (x FingerprintResponse_HealthState) String() string {
return proto.EnumName(FingerprintResponse_HealthState_name, int32(x))
}
func (FingerprintResponse_HealthState) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{5, 0}
return fileDescriptor_driver_f8179d6379690e6b, []int{5, 0}
}
type StartTaskResponse_Result int32
@ -102,7 +102,7 @@ func (x StartTaskResponse_Result) String() string {
return proto.EnumName(StartTaskResponse_Result_name, int32(x))
}
func (StartTaskResponse_Result) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{9, 0}
return fileDescriptor_driver_f8179d6379690e6b, []int{9, 0}
}
type DriverCapabilities_FSIsolation int32
@ -128,7 +128,7 @@ func (x DriverCapabilities_FSIsolation) String() string {
return proto.EnumName(DriverCapabilities_FSIsolation_name, int32(x))
}
func (DriverCapabilities_FSIsolation) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{25, 0}
return fileDescriptor_driver_f8179d6379690e6b, []int{25, 0}
}
type CPUUsage_Fields int32
@ -163,7 +163,7 @@ func (x CPUUsage_Fields) String() string {
return proto.EnumName(CPUUsage_Fields_name, int32(x))
}
func (CPUUsage_Fields) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{43, 0}
return fileDescriptor_driver_f8179d6379690e6b, []int{43, 0}
}
type MemoryUsage_Fields int32
@ -195,7 +195,7 @@ func (x MemoryUsage_Fields) String() string {
return proto.EnumName(MemoryUsage_Fields_name, int32(x))
}
func (MemoryUsage_Fields) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{44, 0}
return fileDescriptor_driver_f8179d6379690e6b, []int{44, 0}
}
type TaskConfigSchemaRequest struct {
@ -208,7 +208,7 @@ func (m *TaskConfigSchemaRequest) Reset() { *m = TaskConfigSchemaRequest
func (m *TaskConfigSchemaRequest) String() string { return proto.CompactTextString(m) }
func (*TaskConfigSchemaRequest) ProtoMessage() {}
func (*TaskConfigSchemaRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{0}
return fileDescriptor_driver_f8179d6379690e6b, []int{0}
}
func (m *TaskConfigSchemaRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskConfigSchemaRequest.Unmarshal(m, b)
@ -240,7 +240,7 @@ func (m *TaskConfigSchemaResponse) Reset() { *m = TaskConfigSchemaRespon
func (m *TaskConfigSchemaResponse) String() string { return proto.CompactTextString(m) }
func (*TaskConfigSchemaResponse) ProtoMessage() {}
func (*TaskConfigSchemaResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{1}
return fileDescriptor_driver_f8179d6379690e6b, []int{1}
}
func (m *TaskConfigSchemaResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskConfigSchemaResponse.Unmarshal(m, b)
@ -277,7 +277,7 @@ func (m *CapabilitiesRequest) Reset() { *m = CapabilitiesRequest{} }
func (m *CapabilitiesRequest) String() string { return proto.CompactTextString(m) }
func (*CapabilitiesRequest) ProtoMessage() {}
func (*CapabilitiesRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{2}
return fileDescriptor_driver_f8179d6379690e6b, []int{2}
}
func (m *CapabilitiesRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CapabilitiesRequest.Unmarshal(m, b)
@ -312,7 +312,7 @@ func (m *CapabilitiesResponse) Reset() { *m = CapabilitiesResponse{} }
func (m *CapabilitiesResponse) String() string { return proto.CompactTextString(m) }
func (*CapabilitiesResponse) ProtoMessage() {}
func (*CapabilitiesResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{3}
return fileDescriptor_driver_f8179d6379690e6b, []int{3}
}
func (m *CapabilitiesResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CapabilitiesResponse.Unmarshal(m, b)
@ -349,7 +349,7 @@ func (m *FingerprintRequest) Reset() { *m = FingerprintRequest{} }
func (m *FingerprintRequest) String() string { return proto.CompactTextString(m) }
func (*FingerprintRequest) ProtoMessage() {}
func (*FingerprintRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{4}
return fileDescriptor_driver_f8179d6379690e6b, []int{4}
}
func (m *FingerprintRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_FingerprintRequest.Unmarshal(m, b)
@ -392,7 +392,7 @@ func (m *FingerprintResponse) Reset() { *m = FingerprintResponse{} }
func (m *FingerprintResponse) String() string { return proto.CompactTextString(m) }
func (*FingerprintResponse) ProtoMessage() {}
func (*FingerprintResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{5}
return fileDescriptor_driver_f8179d6379690e6b, []int{5}
}
func (m *FingerprintResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_FingerprintResponse.Unmarshal(m, b)
@ -447,7 +447,7 @@ func (m *RecoverTaskRequest) Reset() { *m = RecoverTaskRequest{} }
func (m *RecoverTaskRequest) String() string { return proto.CompactTextString(m) }
func (*RecoverTaskRequest) ProtoMessage() {}
func (*RecoverTaskRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{6}
return fileDescriptor_driver_f8179d6379690e6b, []int{6}
}
func (m *RecoverTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_RecoverTaskRequest.Unmarshal(m, b)
@ -491,7 +491,7 @@ func (m *RecoverTaskResponse) Reset() { *m = RecoverTaskResponse{} }
func (m *RecoverTaskResponse) String() string { return proto.CompactTextString(m) }
func (*RecoverTaskResponse) ProtoMessage() {}
func (*RecoverTaskResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{7}
return fileDescriptor_driver_f8179d6379690e6b, []int{7}
}
func (m *RecoverTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_RecoverTaskResponse.Unmarshal(m, b)
@ -523,7 +523,7 @@ func (m *StartTaskRequest) Reset() { *m = StartTaskRequest{} }
func (m *StartTaskRequest) String() string { return proto.CompactTextString(m) }
func (*StartTaskRequest) ProtoMessage() {}
func (*StartTaskRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{8}
return fileDescriptor_driver_f8179d6379690e6b, []int{8}
}
func (m *StartTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StartTaskRequest.Unmarshal(m, b)
@ -577,7 +577,7 @@ func (m *StartTaskResponse) Reset() { *m = StartTaskResponse{} }
func (m *StartTaskResponse) String() string { return proto.CompactTextString(m) }
func (*StartTaskResponse) ProtoMessage() {}
func (*StartTaskResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{9}
return fileDescriptor_driver_f8179d6379690e6b, []int{9}
}
func (m *StartTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StartTaskResponse.Unmarshal(m, b)
@ -637,7 +637,7 @@ func (m *WaitTaskRequest) Reset() { *m = WaitTaskRequest{} }
func (m *WaitTaskRequest) String() string { return proto.CompactTextString(m) }
func (*WaitTaskRequest) ProtoMessage() {}
func (*WaitTaskRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{10}
return fileDescriptor_driver_f8179d6379690e6b, []int{10}
}
func (m *WaitTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_WaitTaskRequest.Unmarshal(m, b)
@ -678,7 +678,7 @@ func (m *WaitTaskResponse) Reset() { *m = WaitTaskResponse{} }
func (m *WaitTaskResponse) String() string { return proto.CompactTextString(m) }
func (*WaitTaskResponse) ProtoMessage() {}
func (*WaitTaskResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{11}
return fileDescriptor_driver_f8179d6379690e6b, []int{11}
}
func (m *WaitTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_WaitTaskResponse.Unmarshal(m, b)
@ -730,7 +730,7 @@ func (m *StopTaskRequest) Reset() { *m = StopTaskRequest{} }
func (m *StopTaskRequest) String() string { return proto.CompactTextString(m) }
func (*StopTaskRequest) ProtoMessage() {}
func (*StopTaskRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{12}
return fileDescriptor_driver_f8179d6379690e6b, []int{12}
}
func (m *StopTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StopTaskRequest.Unmarshal(m, b)
@ -781,7 +781,7 @@ func (m *StopTaskResponse) Reset() { *m = StopTaskResponse{} }
func (m *StopTaskResponse) String() string { return proto.CompactTextString(m) }
func (*StopTaskResponse) ProtoMessage() {}
func (*StopTaskResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{13}
return fileDescriptor_driver_f8179d6379690e6b, []int{13}
}
func (m *StopTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StopTaskResponse.Unmarshal(m, b)
@ -815,7 +815,7 @@ func (m *DestroyTaskRequest) Reset() { *m = DestroyTaskRequest{} }
func (m *DestroyTaskRequest) String() string { return proto.CompactTextString(m) }
func (*DestroyTaskRequest) ProtoMessage() {}
func (*DestroyTaskRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{14}
return fileDescriptor_driver_f8179d6379690e6b, []int{14}
}
func (m *DestroyTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DestroyTaskRequest.Unmarshal(m, b)
@ -859,7 +859,7 @@ func (m *DestroyTaskResponse) Reset() { *m = DestroyTaskResponse{} }
func (m *DestroyTaskResponse) String() string { return proto.CompactTextString(m) }
func (*DestroyTaskResponse) ProtoMessage() {}
func (*DestroyTaskResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{15}
return fileDescriptor_driver_f8179d6379690e6b, []int{15}
}
func (m *DestroyTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DestroyTaskResponse.Unmarshal(m, b)
@ -891,7 +891,7 @@ func (m *InspectTaskRequest) Reset() { *m = InspectTaskRequest{} }
func (m *InspectTaskRequest) String() string { return proto.CompactTextString(m) }
func (*InspectTaskRequest) ProtoMessage() {}
func (*InspectTaskRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{16}
return fileDescriptor_driver_f8179d6379690e6b, []int{16}
}
func (m *InspectTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_InspectTaskRequest.Unmarshal(m, b)
@ -934,7 +934,7 @@ func (m *InspectTaskResponse) Reset() { *m = InspectTaskResponse{} }
func (m *InspectTaskResponse) String() string { return proto.CompactTextString(m) }
func (*InspectTaskResponse) ProtoMessage() {}
func (*InspectTaskResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{17}
return fileDescriptor_driver_f8179d6379690e6b, []int{17}
}
func (m *InspectTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_InspectTaskResponse.Unmarshal(m, b)
@ -987,7 +987,7 @@ func (m *TaskStatsRequest) Reset() { *m = TaskStatsRequest{} }
func (m *TaskStatsRequest) String() string { return proto.CompactTextString(m) }
func (*TaskStatsRequest) ProtoMessage() {}
func (*TaskStatsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{18}
return fileDescriptor_driver_f8179d6379690e6b, []int{18}
}
func (m *TaskStatsRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskStatsRequest.Unmarshal(m, b)
@ -1026,7 +1026,7 @@ func (m *TaskStatsResponse) Reset() { *m = TaskStatsResponse{} }
func (m *TaskStatsResponse) String() string { return proto.CompactTextString(m) }
func (*TaskStatsResponse) ProtoMessage() {}
func (*TaskStatsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{19}
return fileDescriptor_driver_f8179d6379690e6b, []int{19}
}
func (m *TaskStatsResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskStatsResponse.Unmarshal(m, b)
@ -1063,7 +1063,7 @@ func (m *TaskEventsRequest) Reset() { *m = TaskEventsRequest{} }
func (m *TaskEventsRequest) String() string { return proto.CompactTextString(m) }
func (*TaskEventsRequest) ProtoMessage() {}
func (*TaskEventsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{20}
return fileDescriptor_driver_f8179d6379690e6b, []int{20}
}
func (m *TaskEventsRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskEventsRequest.Unmarshal(m, b)
@ -1097,7 +1097,7 @@ func (m *SignalTaskRequest) Reset() { *m = SignalTaskRequest{} }
func (m *SignalTaskRequest) String() string { return proto.CompactTextString(m) }
func (*SignalTaskRequest) ProtoMessage() {}
func (*SignalTaskRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{21}
return fileDescriptor_driver_f8179d6379690e6b, []int{21}
}
func (m *SignalTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SignalTaskRequest.Unmarshal(m, b)
@ -1141,7 +1141,7 @@ func (m *SignalTaskResponse) Reset() { *m = SignalTaskResponse{} }
func (m *SignalTaskResponse) String() string { return proto.CompactTextString(m) }
func (*SignalTaskResponse) ProtoMessage() {}
func (*SignalTaskResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{22}
return fileDescriptor_driver_f8179d6379690e6b, []int{22}
}
func (m *SignalTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SignalTaskResponse.Unmarshal(m, b)
@ -1178,7 +1178,7 @@ func (m *ExecTaskRequest) Reset() { *m = ExecTaskRequest{} }
func (m *ExecTaskRequest) String() string { return proto.CompactTextString(m) }
func (*ExecTaskRequest) ProtoMessage() {}
func (*ExecTaskRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{23}
return fileDescriptor_driver_f8179d6379690e6b, []int{23}
}
func (m *ExecTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExecTaskRequest.Unmarshal(m, b)
@ -1235,7 +1235,7 @@ func (m *ExecTaskResponse) Reset() { *m = ExecTaskResponse{} }
func (m *ExecTaskResponse) String() string { return proto.CompactTextString(m) }
func (*ExecTaskResponse) ProtoMessage() {}
func (*ExecTaskResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{24}
return fileDescriptor_driver_f8179d6379690e6b, []int{24}
}
func (m *ExecTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExecTaskResponse.Unmarshal(m, b)
@ -1294,7 +1294,7 @@ func (m *DriverCapabilities) Reset() { *m = DriverCapabilities{} }
func (m *DriverCapabilities) String() string { return proto.CompactTextString(m) }
func (*DriverCapabilities) ProtoMessage() {}
func (*DriverCapabilities) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{25}
return fileDescriptor_driver_f8179d6379690e6b, []int{25}
}
func (m *DriverCapabilities) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DriverCapabilities.Unmarshal(m, b)
@ -1375,7 +1375,7 @@ func (m *TaskConfig) Reset() { *m = TaskConfig{} }
func (m *TaskConfig) String() string { return proto.CompactTextString(m) }
func (*TaskConfig) ProtoMessage() {}
func (*TaskConfig) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{26}
return fileDescriptor_driver_f8179d6379690e6b, []int{26}
}
func (m *TaskConfig) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskConfig.Unmarshal(m, b)
@ -1507,7 +1507,7 @@ func (m *Resources) Reset() { *m = Resources{} }
func (m *Resources) String() string { return proto.CompactTextString(m) }
func (*Resources) ProtoMessage() {}
func (*Resources) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{27}
return fileDescriptor_driver_f8179d6379690e6b, []int{27}
}
func (m *Resources) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Resources.Unmarshal(m, b)
@ -1554,7 +1554,7 @@ func (m *AllocatedTaskResources) Reset() { *m = AllocatedTaskResources{}
func (m *AllocatedTaskResources) String() string { return proto.CompactTextString(m) }
func (*AllocatedTaskResources) ProtoMessage() {}
func (*AllocatedTaskResources) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{28}
return fileDescriptor_driver_f8179d6379690e6b, []int{28}
}
func (m *AllocatedTaskResources) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_AllocatedTaskResources.Unmarshal(m, b)
@ -1606,7 +1606,7 @@ func (m *AllocatedCpuResources) Reset() { *m = AllocatedCpuResources{} }
func (m *AllocatedCpuResources) String() string { return proto.CompactTextString(m) }
func (*AllocatedCpuResources) ProtoMessage() {}
func (*AllocatedCpuResources) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{29}
return fileDescriptor_driver_f8179d6379690e6b, []int{29}
}
func (m *AllocatedCpuResources) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_AllocatedCpuResources.Unmarshal(m, b)
@ -1644,7 +1644,7 @@ func (m *AllocatedMemoryResources) Reset() { *m = AllocatedMemoryResourc
func (m *AllocatedMemoryResources) String() string { return proto.CompactTextString(m) }
func (*AllocatedMemoryResources) ProtoMessage() {}
func (*AllocatedMemoryResources) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{30}
return fileDescriptor_driver_f8179d6379690e6b, []int{30}
}
func (m *AllocatedMemoryResources) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_AllocatedMemoryResources.Unmarshal(m, b)
@ -1687,7 +1687,7 @@ func (m *NetworkResource) Reset() { *m = NetworkResource{} }
func (m *NetworkResource) String() string { return proto.CompactTextString(m) }
func (*NetworkResource) ProtoMessage() {}
func (*NetworkResource) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{31}
return fileDescriptor_driver_f8179d6379690e6b, []int{31}
}
func (m *NetworkResource) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_NetworkResource.Unmarshal(m, b)
@ -1761,7 +1761,7 @@ func (m *NetworkPort) Reset() { *m = NetworkPort{} }
func (m *NetworkPort) String() string { return proto.CompactTextString(m) }
func (*NetworkPort) ProtoMessage() {}
func (*NetworkPort) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{32}
return fileDescriptor_driver_f8179d6379690e6b, []int{32}
}
func (m *NetworkPort) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_NetworkPort.Unmarshal(m, b)
@ -1821,7 +1821,7 @@ func (m *LinuxResources) Reset() { *m = LinuxResources{} }
func (m *LinuxResources) String() string { return proto.CompactTextString(m) }
func (*LinuxResources) ProtoMessage() {}
func (*LinuxResources) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{33}
return fileDescriptor_driver_f8179d6379690e6b, []int{33}
}
func (m *LinuxResources) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_LinuxResources.Unmarshal(m, b)
@ -1913,7 +1913,7 @@ func (m *Mount) Reset() { *m = Mount{} }
func (m *Mount) String() string { return proto.CompactTextString(m) }
func (*Mount) ProtoMessage() {}
func (*Mount) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{34}
return fileDescriptor_driver_f8179d6379690e6b, []int{34}
}
func (m *Mount) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Mount.Unmarshal(m, b)
@ -1976,7 +1976,7 @@ func (m *Device) Reset() { *m = Device{} }
func (m *Device) String() string { return proto.CompactTextString(m) }
func (*Device) ProtoMessage() {}
func (*Device) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{35}
return fileDescriptor_driver_f8179d6379690e6b, []int{35}
}
func (m *Device) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Device.Unmarshal(m, b)
@ -2034,7 +2034,7 @@ func (m *TaskHandle) Reset() { *m = TaskHandle{} }
func (m *TaskHandle) String() string { return proto.CompactTextString(m) }
func (*TaskHandle) ProtoMessage() {}
func (*TaskHandle) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{36}
return fileDescriptor_driver_f8179d6379690e6b, []int{36}
}
func (m *TaskHandle) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskHandle.Unmarshal(m, b)
@ -2094,7 +2094,7 @@ func (m *NetworkOverride) Reset() { *m = NetworkOverride{} }
func (m *NetworkOverride) String() string { return proto.CompactTextString(m) }
func (*NetworkOverride) ProtoMessage() {}
func (*NetworkOverride) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{37}
return fileDescriptor_driver_f8179d6379690e6b, []int{37}
}
func (m *NetworkOverride) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_NetworkOverride.Unmarshal(m, b)
@ -2152,7 +2152,7 @@ func (m *ExitResult) Reset() { *m = ExitResult{} }
func (m *ExitResult) String() string { return proto.CompactTextString(m) }
func (*ExitResult) ProtoMessage() {}
func (*ExitResult) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{38}
return fileDescriptor_driver_f8179d6379690e6b, []int{38}
}
func (m *ExitResult) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExitResult.Unmarshal(m, b)
@ -2215,7 +2215,7 @@ func (m *TaskStatus) Reset() { *m = TaskStatus{} }
func (m *TaskStatus) String() string { return proto.CompactTextString(m) }
func (*TaskStatus) ProtoMessage() {}
func (*TaskStatus) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{39}
return fileDescriptor_driver_f8179d6379690e6b, []int{39}
}
func (m *TaskStatus) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskStatus.Unmarshal(m, b)
@ -2290,7 +2290,7 @@ func (m *TaskDriverStatus) Reset() { *m = TaskDriverStatus{} }
func (m *TaskDriverStatus) String() string { return proto.CompactTextString(m) }
func (*TaskDriverStatus) ProtoMessage() {}
func (*TaskDriverStatus) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{40}
return fileDescriptor_driver_f8179d6379690e6b, []int{40}
}
func (m *TaskDriverStatus) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskDriverStatus.Unmarshal(m, b)
@ -2335,7 +2335,7 @@ func (m *TaskStats) Reset() { *m = TaskStats{} }
func (m *TaskStats) String() string { return proto.CompactTextString(m) }
func (*TaskStats) ProtoMessage() {}
func (*TaskStats) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{41}
return fileDescriptor_driver_f8179d6379690e6b, []int{41}
}
func (m *TaskStats) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskStats.Unmarshal(m, b)
@ -2397,7 +2397,7 @@ func (m *TaskResourceUsage) Reset() { *m = TaskResourceUsage{} }
func (m *TaskResourceUsage) String() string { return proto.CompactTextString(m) }
func (*TaskResourceUsage) ProtoMessage() {}
func (*TaskResourceUsage) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{42}
return fileDescriptor_driver_f8179d6379690e6b, []int{42}
}
func (m *TaskResourceUsage) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskResourceUsage.Unmarshal(m, b)
@ -2449,7 +2449,7 @@ func (m *CPUUsage) Reset() { *m = CPUUsage{} }
func (m *CPUUsage) String() string { return proto.CompactTextString(m) }
func (*CPUUsage) ProtoMessage() {}
func (*CPUUsage) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{43}
return fileDescriptor_driver_f8179d6379690e6b, []int{43}
}
func (m *CPUUsage) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CPUUsage.Unmarshal(m, b)
@ -2535,7 +2535,7 @@ func (m *MemoryUsage) Reset() { *m = MemoryUsage{} }
func (m *MemoryUsage) String() string { return proto.CompactTextString(m) }
func (*MemoryUsage) ProtoMessage() {}
func (*MemoryUsage) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{44}
return fileDescriptor_driver_f8179d6379690e6b, []int{44}
}
func (m *MemoryUsage) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_MemoryUsage.Unmarshal(m, b)
@ -2600,12 +2600,16 @@ func (m *MemoryUsage) GetMeasuredFields() []MemoryUsage_Fields {
type DriverTaskEvent struct {
// TaskId is the id of the task for the event
TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
// AllocId of the task for the event
AllocId string `protobuf:"bytes,2,opt,name=alloc_id,json=allocId,proto3" json:"alloc_id,omitempty"`
// TaskName is the name of the task for the event
TaskName string `protobuf:"bytes,3,opt,name=task_name,json=taskName,proto3" json:"task_name,omitempty"`
// Timestamp when the event occurred
Timestamp *timestamp.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
Timestamp *timestamp.Timestamp `protobuf:"bytes,4,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
// Message is the body of the event
Message string `protobuf:"bytes,3,opt,name=message,proto3" json:"message,omitempty"`
Message string `protobuf:"bytes,5,opt,name=message,proto3" json:"message,omitempty"`
// Annotations allows for additional key/value data to be sent along with the event
Annotations map[string]string `protobuf:"bytes,4,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
Annotations map[string]string `protobuf:"bytes,6,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -2615,7 +2619,7 @@ func (m *DriverTaskEvent) Reset() { *m = DriverTaskEvent{} }
func (m *DriverTaskEvent) String() string { return proto.CompactTextString(m) }
func (*DriverTaskEvent) ProtoMessage() {}
func (*DriverTaskEvent) Descriptor() ([]byte, []int) {
return fileDescriptor_driver_9320f8b051f51d4e, []int{45}
return fileDescriptor_driver_f8179d6379690e6b, []int{45}
}
func (m *DriverTaskEvent) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DriverTaskEvent.Unmarshal(m, b)
@ -2642,6 +2646,20 @@ func (m *DriverTaskEvent) GetTaskId() string {
return ""
}
func (m *DriverTaskEvent) GetAllocId() string {
if m != nil {
return m.AllocId
}
return ""
}
func (m *DriverTaskEvent) GetTaskName() string {
if m != nil {
return m.TaskName
}
return ""
}
func (m *DriverTaskEvent) GetTimestamp() *timestamp.Timestamp {
if m != nil {
return m.Timestamp
@ -3308,190 +3326,191 @@ var _Driver_serviceDesc = grpc.ServiceDesc{
}
func init() {
proto.RegisterFile("plugins/drivers/proto/driver.proto", fileDescriptor_driver_9320f8b051f51d4e)
proto.RegisterFile("plugins/drivers/proto/driver.proto", fileDescriptor_driver_f8179d6379690e6b)
}
var fileDescriptor_driver_9320f8b051f51d4e = []byte{
// 2888 bytes of a gzipped FileDescriptorProto
var fileDescriptor_driver_f8179d6379690e6b = []byte{
// 2908 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x59, 0x4b, 0x6f, 0x23, 0xc7,
0xf1, 0x17, 0x49, 0x91, 0x22, 0x8b, 0x12, 0x35, 0xdb, 0xbb, 0x6b, 0xd3, 0x34, 0xfe, 0x7f, 0xaf,
0x07, 0x70, 0x20, 0xd8, 0x5e, 0xca, 0x96, 0x91, 0x7d, 0x25, 0x7e, 0xd0, 0xd4, 0xac, 0x24, 0xaf,
0x44, 0x29, 0x4d, 0x0a, 0xeb, 0x4d, 0x62, 0x4f, 0x46, 0x33, 0x2d, 0x72, 0x56, 0x9c, 0x87, 0xa7,
0x7b, 0x64, 0x09, 0x41, 0x90, 0x20, 0x01, 0x82, 0xe4, 0x10, 0x20, 0x17, 0x23, 0x40, 0x8e, 0xc9,
0x29, 0xc8, 0x37, 0x48, 0xe0, 0x4b, 0x3e, 0x45, 0x80, 0x5c, 0x72, 0x09, 0x90, 0x6b, 0xbe, 0x41,
0xd0, 0x8f, 0x19, 0x0e, 0x25, 0xad, 0x77, 0xc8, 0xcd, 0x89, 0xec, 0xea, 0xae, 0x5f, 0xd7, 0x54,
0x55, 0x57, 0x55, 0x77, 0x81, 0x1e, 0x8e, 0xe3, 0xa1, 0xeb, 0xd3, 0x75, 0x27, 0x72, 0x4f, 0x49,
0x44, 0xd7, 0xc3, 0x28, 0x60, 0x81, 0x1a, 0xb5, 0xc5, 0x00, 0xbd, 0x31, 0xb2, 0xe8, 0xc8, 0xb5,
0x83, 0x28, 0x6c, 0xfb, 0x81, 0x67, 0x39, 0x6d, 0xc5, 0xd3, 0x56, 0x3c, 0x72, 0x59, 0xeb, 0xff,
0x87, 0x41, 0x30, 0x1c, 0x13, 0x89, 0x70, 0x14, 0x1f, 0xaf, 0x3b, 0x71, 0x64, 0x31, 0x37, 0xf0,
0xd5, 0xfc, 0x6b, 0x17, 0xe7, 0x99, 0xeb, 0x11, 0xca, 0x2c, 0x2f, 0x54, 0x0b, 0x3e, 0x1a, 0xba,
0x6c, 0x14, 0x1f, 0xb5, 0xed, 0xc0, 0x5b, 0x4f, 0xb7, 0x5c, 0x17, 0x5b, 0xae, 0x27, 0x62, 0xd2,
0x91, 0x15, 0x11, 0x67, 0x7d, 0x64, 0x8f, 0x69, 0x48, 0x6c, 0xfe, 0x6b, 0xf2, 0x3f, 0x0a, 0x61,
0x2b, 0x3f, 0x02, 0x65, 0x51, 0x6c, 0xb3, 0xe4, 0x7b, 0x2d, 0xc6, 0x22, 0xf7, 0x28, 0x66, 0x44,
0x02, 0xe9, 0xaf, 0xc0, 0xcb, 0x03, 0x8b, 0x9e, 0x74, 0x03, 0xff, 0xd8, 0x1d, 0xf6, 0xed, 0x11,
0xf1, 0x2c, 0x4c, 0xbe, 0x88, 0x09, 0x65, 0xfa, 0x0f, 0xa1, 0x79, 0x79, 0x8a, 0x86, 0x81, 0x4f,
0x09, 0xfa, 0x08, 0x16, 0xb9, 0x34, 0xcd, 0xc2, 0xad, 0xc2, 0x5a, 0x7d, 0xe3, 0xed, 0xf6, 0xb3,
0x14, 0x27, 0x65, 0x68, 0xab, 0xaf, 0x68, 0xf7, 0x43, 0x62, 0x63, 0xc1, 0xa9, 0xdf, 0x84, 0xeb,
0x5d, 0x2b, 0xb4, 0x8e, 0xdc, 0xb1, 0xcb, 0x5c, 0x42, 0x93, 0x4d, 0x63, 0xb8, 0x31, 0x4d, 0x56,
0x1b, 0x7e, 0x06, 0xcb, 0x76, 0x86, 0xae, 0x36, 0xbe, 0xdf, 0xce, 0x65, 0xb1, 0xf6, 0xa6, 0x18,
0x4d, 0x01, 0x4f, 0xc1, 0xe9, 0x37, 0x00, 0x3d, 0x74, 0xfd, 0x21, 0x89, 0xc2, 0xc8, 0xf5, 0x59,
0x22, 0xcc, 0xd7, 0x25, 0xb8, 0x3e, 0x45, 0x56, 0xc2, 0x3c, 0x05, 0x48, 0xf5, 0xc8, 0x45, 0x29,
0xad, 0xd5, 0x37, 0x3e, 0xc9, 0x29, 0xca, 0x15, 0x78, 0xed, 0x4e, 0x0a, 0x66, 0xf8, 0x2c, 0x3a,
0xc7, 0x19, 0x74, 0xf4, 0x39, 0x54, 0x46, 0xc4, 0x1a, 0xb3, 0x51, 0xb3, 0x78, 0xab, 0xb0, 0xd6,
0xd8, 0x78, 0xf8, 0x02, 0xfb, 0x6c, 0x0b, 0xa0, 0x3e, 0xb3, 0x18, 0xc1, 0x0a, 0x15, 0xdd, 0x06,
0x24, 0xff, 0x99, 0x0e, 0xa1, 0x76, 0xe4, 0x86, 0xdc, 0x91, 0x9b, 0xa5, 0x5b, 0x85, 0xb5, 0x1a,
0xbe, 0x26, 0x67, 0x36, 0x27, 0x13, 0xad, 0x10, 0x56, 0x2f, 0x48, 0x8b, 0x34, 0x28, 0x9d, 0x90,
0x73, 0x61, 0x91, 0x1a, 0xe6, 0x7f, 0xd1, 0x16, 0x94, 0x4f, 0xad, 0x71, 0x4c, 0x84, 0xc8, 0xf5,
0x8d, 0x77, 0x9f, 0xe7, 0x1e, 0xca, 0x45, 0x27, 0x7a, 0xc0, 0x92, 0xff, 0x41, 0xf1, 0x5e, 0x41,
0xbf, 0x0f, 0xf5, 0x8c, 0xdc, 0xa8, 0x01, 0x70, 0xd8, 0xdb, 0x34, 0x06, 0x46, 0x77, 0x60, 0x6c,
0x6a, 0x0b, 0x68, 0x05, 0x6a, 0x87, 0xbd, 0x6d, 0xa3, 0xb3, 0x3b, 0xd8, 0x7e, 0xa2, 0x15, 0x50,
0x1d, 0x96, 0x92, 0x41, 0x51, 0x3f, 0x03, 0x84, 0x89, 0x1d, 0x9c, 0x92, 0x88, 0x3b, 0xb2, 0xb2,
0x2a, 0x7a, 0x19, 0x96, 0x98, 0x45, 0x4f, 0x4c, 0xd7, 0x51, 0x32, 0x57, 0xf8, 0x70, 0xc7, 0x41,
0x3b, 0x50, 0x19, 0x59, 0xbe, 0x33, 0x7e, 0xbe, 0xdc, 0xd3, 0xaa, 0xe6, 0xe0, 0xdb, 0x82, 0x11,
0x2b, 0x00, 0xee, 0xdd, 0x53, 0x3b, 0x4b, 0x03, 0xe8, 0x4f, 0x40, 0xeb, 0x33, 0x2b, 0x62, 0x59,
0x71, 0x0c, 0x58, 0xe4, 0xfb, 0x2b, 0x8f, 0x9e, 0x65, 0x4f, 0x79, 0x32, 0xb1, 0x60, 0xd7, 0xff,
0x53, 0x84, 0x6b, 0x19, 0x6c, 0xe5, 0xa9, 0x8f, 0xa1, 0x12, 0x11, 0x1a, 0x8f, 0x99, 0x80, 0x6f,
0x6c, 0x7c, 0x98, 0x13, 0xfe, 0x12, 0x52, 0x1b, 0x0b, 0x18, 0xac, 0xe0, 0xd0, 0x1a, 0x68, 0x92,
0xc3, 0x24, 0x51, 0x14, 0x44, 0xa6, 0x47, 0x87, 0x42, 0x6b, 0x35, 0xdc, 0x90, 0x74, 0x83, 0x93,
0xf7, 0xe8, 0x30, 0xa3, 0xd5, 0xd2, 0x0b, 0x6a, 0x15, 0x59, 0xa0, 0xf9, 0x84, 0x7d, 0x19, 0x44,
0x27, 0x26, 0x57, 0x6d, 0xe4, 0x3a, 0xa4, 0xb9, 0x28, 0x40, 0xef, 0xe4, 0x04, 0xed, 0x49, 0xf6,
0x7d, 0xc5, 0x8d, 0x57, 0xfd, 0x69, 0x82, 0xfe, 0x16, 0x54, 0xe4, 0x97, 0x72, 0x4f, 0xea, 0x1f,
0x76, 0xbb, 0x46, 0xbf, 0xaf, 0x2d, 0xa0, 0x1a, 0x94, 0xb1, 0x31, 0xc0, 0xdc, 0xc3, 0x6a, 0x50,
0x7e, 0xd8, 0x19, 0x74, 0x76, 0xb5, 0xa2, 0xfe, 0x26, 0xac, 0x3e, 0xb6, 0x5c, 0x96, 0xc7, 0xb9,
0xf4, 0x00, 0xb4, 0xc9, 0x5a, 0x65, 0x9d, 0x9d, 0x29, 0xeb, 0xe4, 0x57, 0x8d, 0x71, 0xe6, 0xb2,
0x0b, 0xf6, 0xd0, 0xa0, 0x44, 0xa2, 0x48, 0x99, 0x80, 0xff, 0xd5, 0xbf, 0x84, 0xd5, 0x3e, 0x0b,
0xc2, 0x5c, 0x9e, 0xff, 0x1e, 0x2c, 0xf1, 0x1c, 0x15, 0xc4, 0x4c, 0xb9, 0xfe, 0x2b, 0x6d, 0x99,
0xc3, 0xda, 0x49, 0x0e, 0x6b, 0x6f, 0xaa, 0x1c, 0x87, 0x93, 0x95, 0xe8, 0x25, 0xa8, 0x50, 0x77,
0xe8, 0x5b, 0x63, 0x15, 0x2d, 0xd4, 0x48, 0x47, 0xdc, 0xc9, 0x93, 0x8d, 0x95, 0xe3, 0x77, 0x01,
0x6d, 0x12, 0xca, 0xa2, 0xe0, 0x3c, 0x97, 0x3c, 0x37, 0xa0, 0x7c, 0x1c, 0x44, 0xb6, 0x3c, 0x88,
0x55, 0x2c, 0x07, 0xfc, 0x50, 0x4d, 0x81, 0x28, 0xec, 0xdb, 0x80, 0x76, 0x7c, 0x9e, 0x53, 0xf2,
0x19, 0xe2, 0xb7, 0x45, 0xb8, 0x3e, 0xb5, 0x5e, 0x19, 0x63, 0xfe, 0x73, 0xc8, 0x03, 0x53, 0x4c,
0xe5, 0x39, 0x44, 0xfb, 0x50, 0x91, 0x2b, 0x94, 0x26, 0xef, 0xce, 0x00, 0x24, 0xd3, 0x94, 0x82,
0x53, 0x30, 0x57, 0x3a, 0x7d, 0xe9, 0x7f, 0xed, 0xf4, 0x5a, 0xf2, 0x1d, 0xf4, 0xb9, 0xfa, 0xfb,
0x01, 0x5c, 0xcb, 0x2c, 0x56, 0xca, 0x7b, 0x08, 0x65, 0xca, 0x09, 0x4a, 0x7b, 0xef, 0xcc, 0xa8,
0x3d, 0x8a, 0x25, 0xbb, 0x7e, 0x5d, 0x82, 0x1b, 0xa7, 0xc4, 0x4f, 0x45, 0xd1, 0x37, 0xe1, 0x5a,
0x5f, 0xb8, 0x56, 0x2e, 0xdf, 0x99, 0xb8, 0x65, 0x71, 0xca, 0x2d, 0x6f, 0x00, 0xca, 0xa2, 0x28,
0xe7, 0x39, 0x87, 0x55, 0xe3, 0x8c, 0xd8, 0xb9, 0x90, 0x9b, 0xb0, 0x64, 0x07, 0x9e, 0x67, 0xf9,
0x4e, 0xb3, 0x78, 0xab, 0xb4, 0x56, 0xc3, 0xc9, 0x30, 0x7b, 0x7e, 0x4a, 0x79, 0xcf, 0x8f, 0xfe,
0x9b, 0x02, 0x68, 0x93, 0xbd, 0x95, 0x22, 0xb9, 0xf4, 0xcc, 0xe1, 0x40, 0x7c, 0xef, 0x65, 0xac,
0x46, 0x8a, 0x9e, 0x1c, 0x71, 0x49, 0x27, 0x51, 0x94, 0x09, 0x21, 0xa5, 0x17, 0x0c, 0x21, 0xfa,
0xbf, 0x0a, 0x80, 0x2e, 0x17, 0x4a, 0xe8, 0x75, 0x58, 0xa6, 0xc4, 0x77, 0x4c, 0xa9, 0x46, 0x69,
0xe1, 0x2a, 0xae, 0x73, 0x9a, 0xd4, 0x27, 0x45, 0x08, 0x16, 0xc9, 0x19, 0xb1, 0xd5, 0x69, 0x15,
0xff, 0xd1, 0x08, 0x96, 0x8f, 0xa9, 0xe9, 0xd2, 0x60, 0x6c, 0xa5, 0x15, 0x45, 0x63, 0xc3, 0x98,
0xbb, 0x60, 0x6b, 0x3f, 0xec, 0xef, 0x24, 0x60, 0xb8, 0x7e, 0x4c, 0xd3, 0x81, 0xde, 0x86, 0x7a,
0x66, 0x0e, 0x55, 0x61, 0xb1, 0xb7, 0xdf, 0x33, 0xb4, 0x05, 0x04, 0x50, 0xe9, 0x6e, 0xe3, 0xfd,
0xfd, 0x81, 0x8c, 0xda, 0x3b, 0x7b, 0x9d, 0x2d, 0x43, 0x2b, 0xea, 0xbf, 0x2f, 0x03, 0x4c, 0xd2,
0x27, 0x6a, 0x40, 0x31, 0xb5, 0x74, 0xd1, 0x75, 0xf8, 0xc7, 0xf8, 0x96, 0x47, 0x94, 0xf7, 0x88,
0xff, 0x68, 0x03, 0x6e, 0x7a, 0x74, 0x18, 0x5a, 0xf6, 0x89, 0xa9, 0xb2, 0x9e, 0x2d, 0x98, 0xc5,
0x57, 0x2d, 0xe3, 0xeb, 0x6a, 0x52, 0x49, 0x2d, 0x71, 0x77, 0xa1, 0x44, 0xfc, 0xd3, 0xe6, 0xa2,
0xa8, 0x0e, 0x1f, 0xcc, 0x9c, 0xd6, 0xdb, 0x86, 0x7f, 0x2a, 0xab, 0x41, 0x0e, 0x83, 0x7a, 0x50,
0x8b, 0x08, 0x0d, 0xe2, 0xc8, 0x26, 0xb4, 0x59, 0x9e, 0xe9, 0x90, 0xe1, 0x84, 0x0f, 0x4f, 0x20,
0xd0, 0x26, 0x54, 0xbc, 0x20, 0xf6, 0x19, 0x6d, 0x56, 0x84, 0x80, 0x6f, 0xe7, 0x04, 0xdb, 0xe3,
0x4c, 0x58, 0xf1, 0xa2, 0x2d, 0x58, 0x72, 0xc8, 0xa9, 0xcb, 0x65, 0x5a, 0x12, 0x30, 0xb7, 0xf3,
0xda, 0x57, 0x70, 0xe1, 0x84, 0x9b, 0x2b, 0x3d, 0xa6, 0x24, 0x6a, 0x56, 0xa5, 0xd2, 0xf9, 0x7f,
0xf4, 0x2a, 0xd4, 0xac, 0xf1, 0x38, 0xb0, 0x4d, 0xc7, 0x8d, 0x9a, 0x35, 0x31, 0x51, 0x15, 0x84,
0x4d, 0x37, 0x42, 0xaf, 0x41, 0x5d, 0x9e, 0x0c, 0x33, 0xb4, 0xd8, 0xa8, 0x09, 0x62, 0x1a, 0x24,
0xe9, 0xc0, 0x62, 0x23, 0xb5, 0x80, 0x44, 0x91, 0x5c, 0x50, 0x4f, 0x17, 0x90, 0x28, 0x12, 0x0b,
0xbe, 0x05, 0xab, 0xe2, 0x98, 0x0f, 0xa3, 0x20, 0x0e, 0x4d, 0x61, 0xf2, 0x65, 0xb1, 0x68, 0x85,
0x93, 0xb7, 0x38, 0xb5, 0xc7, 0x6d, 0xff, 0x0a, 0x54, 0x9f, 0x06, 0x47, 0x72, 0xc1, 0x8a, 0x58,
0xb0, 0xf4, 0x34, 0x38, 0x4a, 0xa6, 0xa4, 0x84, 0xae, 0xd3, 0x6c, 0xc8, 0x29, 0x31, 0xde, 0x71,
0x5a, 0x77, 0xa0, 0x9a, 0x18, 0xf0, 0x8a, 0x02, 0xf9, 0x46, 0xb6, 0x40, 0xae, 0x65, 0xab, 0xdd,
0x7f, 0x14, 0xa0, 0x96, 0x1a, 0x0c, 0xf9, 0x70, 0x5d, 0x00, 0x5a, 0x8c, 0x38, 0xe6, 0xc4, 0xfe,
0x32, 0xc8, 0xbe, 0x9f, 0x53, 0xd7, 0x9d, 0x04, 0x41, 0x05, 0x1a, 0xe5, 0x0c, 0x28, 0x45, 0x9e,
0xec, 0xf7, 0x39, 0xac, 0x8e, 0x5d, 0x3f, 0x3e, 0xcb, 0xec, 0x25, 0xb3, 0xd8, 0xb7, 0x73, 0xee,
0xb5, 0xcb, 0xb9, 0x27, 0x7b, 0x34, 0xc6, 0x53, 0x63, 0xfd, 0xab, 0x22, 0xbc, 0x74, 0xb5, 0x38,
0xa8, 0x07, 0x25, 0x3b, 0x8c, 0xd5, 0xa7, 0x7d, 0x77, 0xd6, 0x4f, 0xeb, 0x86, 0xf1, 0x64, 0x57,
0x0e, 0xc4, 0x2b, 0x5f, 0x8f, 0x78, 0x41, 0x74, 0xae, 0xbe, 0xe0, 0xc3, 0x59, 0x21, 0xf7, 0x04,
0xf7, 0x04, 0x55, 0xc1, 0x21, 0x0c, 0x55, 0x95, 0x3f, 0xf9, 0x41, 0x2c, 0xcd, 0x9e, 0x87, 0x13,
0x48, 0x9c, 0xe2, 0xe8, 0x77, 0xe0, 0xe6, 0x95, 0x9f, 0x82, 0xfe, 0x0f, 0xc0, 0x0e, 0x63, 0x53,
0xdc, 0x93, 0xa4, 0xdd, 0x4b, 0xb8, 0x66, 0x87, 0x71, 0x5f, 0x10, 0xf4, 0xbb, 0xd0, 0x7c, 0x96,
0xbc, 0xfc, 0xf8, 0x48, 0x89, 0x4d, 0xef, 0x48, 0xe8, 0xa0, 0x84, 0xab, 0x92, 0xb0, 0x77, 0xa4,
0xff, 0xae, 0x08, 0xab, 0x17, 0xc4, 0xe1, 0x29, 0x46, 0x1e, 0xc7, 0x24, 0xed, 0xc9, 0x11, 0x3f,
0x9b, 0xb6, 0xeb, 0x24, 0xb5, 0xa5, 0xf8, 0x2f, 0x82, 0x66, 0xa8, 0xea, 0xbe, 0xa2, 0x1b, 0x72,
0x87, 0xf6, 0x8e, 0x5c, 0x46, 0x45, 0x39, 0x5e, 0xc6, 0x72, 0x80, 0x9e, 0x40, 0x23, 0x22, 0x94,
0x44, 0xa7, 0xc4, 0x31, 0xc3, 0x20, 0x62, 0x89, 0xc2, 0x36, 0x66, 0x53, 0xd8, 0x41, 0x10, 0x31,
0xbc, 0x92, 0x20, 0xf1, 0x11, 0x45, 0x8f, 0x61, 0xc5, 0x39, 0xf7, 0x2d, 0xcf, 0xb5, 0x15, 0x72,
0x65, 0x6e, 0xe4, 0x65, 0x05, 0x24, 0x80, 0xf9, 0x75, 0x33, 0x33, 0xc9, 0x3f, 0x6c, 0x6c, 0x1d,
0x91, 0xb1, 0xd2, 0x89, 0x1c, 0x4c, 0x9f, 0xdf, 0xb2, 0x3a, 0xbf, 0xfa, 0x1f, 0x8b, 0xd0, 0x98,
0x3e, 0x00, 0x89, 0xfd, 0x42, 0x12, 0xb9, 0x81, 0x93, 0xb1, 0xdf, 0x81, 0x20, 0x70, 0x1b, 0xf1,
0xe9, 0x2f, 0xe2, 0x80, 0x59, 0x89, 0x8d, 0xec, 0x30, 0xfe, 0x1e, 0x1f, 0x5f, 0xb0, 0x7d, 0xe9,
0x82, 0xed, 0xd1, 0xdb, 0x80, 0x94, 0x7d, 0xc7, 0xae, 0xe7, 0x32, 0xf3, 0xe8, 0x9c, 0x11, 0xa9,
0xff, 0x12, 0xd6, 0xe4, 0xcc, 0x2e, 0x9f, 0xf8, 0x98, 0xd3, 0x91, 0x0e, 0x2b, 0x41, 0xe0, 0x99,
0xd4, 0x0e, 0x22, 0x62, 0x5a, 0xce, 0x53, 0x91, 0x43, 0x4a, 0xb8, 0x1e, 0x04, 0x5e, 0x9f, 0xd3,
0x3a, 0xce, 0x53, 0x1e, 0x32, 0xed, 0x30, 0xa6, 0x84, 0x99, 0xfc, 0xa7, 0x59, 0x91, 0x21, 0x53,
0x92, 0xba, 0x61, 0x4c, 0x33, 0x0b, 0x3c, 0xe2, 0xf1, 0x90, 0x9f, 0x59, 0xb0, 0x47, 0x3c, 0xbe,
0xcb, 0xf2, 0x01, 0x89, 0x6c, 0xe2, 0xb3, 0x81, 0x6b, 0x9f, 0x50, 0x11, 0xce, 0x0b, 0x78, 0x8a,
0xa6, 0x7f, 0x06, 0x65, 0x91, 0x44, 0xf8, 0xc7, 0x8b, 0x00, 0x2c, 0xe2, 0xb3, 0x54, 0x6f, 0x95,
0x13, 0x44, 0x74, 0x7e, 0x15, 0x6a, 0xa3, 0x80, 0xaa, 0xe8, 0x2e, 0x3d, 0xaf, 0xca, 0x09, 0x62,
0xb2, 0x05, 0xd5, 0x88, 0x58, 0x4e, 0xe0, 0x8f, 0xcf, 0x85, 0x5e, 0xaa, 0x38, 0x1d, 0xeb, 0x5f,
0x40, 0x45, 0x26, 0x97, 0x17, 0xc0, 0xbf, 0x0d, 0xc8, 0x96, 0x69, 0x21, 0x24, 0x91, 0xe7, 0x52,
0xea, 0x06, 0x3e, 0x4d, 0xde, 0x44, 0xe4, 0xcc, 0xc1, 0x64, 0x42, 0xff, 0x5b, 0x41, 0x16, 0x14,
0xf2, 0xb6, 0xca, 0x4b, 0x32, 0x55, 0x1d, 0xcc, 0x7d, 0xa5, 0x57, 0x00, 0x49, 0x59, 0x4d, 0xd4,
0xdb, 0xcf, 0xac, 0x65, 0x35, 0x91, 0x65, 0x35, 0xe1, 0x35, 0x9c, 0xaa, 0x5b, 0x24, 0x9c, 0x2c,
0x5b, 0xea, 0x4e, 0x7a, 0xdf, 0x20, 0xfa, 0xbf, 0x0b, 0x69, 0x44, 0x48, 0xee, 0x05, 0xe8, 0x73,
0xa8, 0xf2, 0xc3, 0x65, 0x7a, 0x56, 0xa8, 0x5e, 0xb9, 0xba, 0xf3, 0x5d, 0x39, 0xda, 0xfc, 0x2c,
0xed, 0x59, 0xa1, 0x2c, 0x68, 0x96, 0x42, 0x39, 0xe2, 0x91, 0xc5, 0x72, 0x26, 0x91, 0x85, 0xff,
0x47, 0x6f, 0x40, 0xc3, 0x8a, 0x59, 0x60, 0x5a, 0xce, 0x29, 0x89, 0x98, 0x4b, 0x89, 0xb2, 0xf0,
0x0a, 0xa7, 0x76, 0x12, 0x62, 0xeb, 0x01, 0x2c, 0x67, 0x31, 0x9f, 0x97, 0x63, 0xcb, 0xd9, 0x1c,
0xfb, 0x23, 0x80, 0x49, 0xf9, 0xcb, 0x3d, 0x81, 0x9c, 0xb9, 0xcc, 0xb4, 0x03, 0x47, 0x46, 0xbe,
0x32, 0xae, 0x72, 0x42, 0x37, 0x70, 0xc8, 0x85, 0xcb, 0x44, 0x39, 0xb9, 0x4c, 0xf0, 0xb3, 0xc9,
0x8f, 0xd3, 0x89, 0x3b, 0x1e, 0x13, 0x47, 0x49, 0x58, 0x0b, 0x02, 0xef, 0x91, 0x20, 0xe8, 0x5f,
0x17, 0xa5, 0x47, 0xc8, 0xab, 0x5c, 0xae, 0x12, 0x33, 0x35, 0x75, 0xe9, 0xc5, 0x4c, 0x7d, 0x1f,
0x80, 0x32, 0x2b, 0xe2, 0x05, 0x83, 0xc5, 0xd4, 0xeb, 0x48, 0xeb, 0xd2, 0x6d, 0x64, 0x90, 0xbc,
0x48, 0xe3, 0x9a, 0x5a, 0xdd, 0x61, 0xe8, 0x7d, 0x58, 0xb6, 0x03, 0x2f, 0x1c, 0x13, 0xc5, 0x5c,
0x7e, 0x2e, 0x73, 0x3d, 0x5d, 0xdf, 0x61, 0x99, 0xab, 0x48, 0xe5, 0x45, 0xaf, 0x22, 0x7f, 0x29,
0xc8, 0x1b, 0x69, 0xf6, 0x42, 0x8c, 0x86, 0x57, 0xbc, 0xba, 0x6e, 0xcd, 0x79, 0xbb, 0xfe, 0xa6,
0x27, 0xd7, 0xd6, 0xfb, 0x79, 0xde, 0x38, 0x9f, 0x5d, 0xc2, 0xfd, 0xb5, 0x04, 0xb5, 0xf4, 0x62,
0x7b, 0xc9, 0xf6, 0xf7, 0xa0, 0x96, 0xb6, 0x03, 0x54, 0x69, 0xf2, 0x8d, 0xe6, 0x49, 0x17, 0xa3,
0x63, 0x40, 0xd6, 0x70, 0x98, 0x96, 0x66, 0x66, 0x4c, 0xad, 0x61, 0xf2, 0x14, 0x70, 0x6f, 0x06,
0x3d, 0x24, 0xd9, 0xe9, 0x90, 0xf3, 0x63, 0xcd, 0x1a, 0x0e, 0xa7, 0x28, 0xe8, 0xc7, 0x70, 0x73,
0x7a, 0x0f, 0xf3, 0xe8, 0xdc, 0x0c, 0x5d, 0x47, 0x5d, 0x65, 0xb6, 0x67, 0xbd, 0xdb, 0xb7, 0xa7,
0xe0, 0x3f, 0x3e, 0x3f, 0x70, 0x1d, 0xa9, 0x73, 0x14, 0x5d, 0x9a, 0x68, 0xfd, 0x14, 0x5e, 0x7e,
0xc6, 0xf2, 0x2b, 0x6c, 0xd0, 0x9b, 0x7e, 0x67, 0x9e, 0x5f, 0x09, 0x19, 0xeb, 0xfd, 0xa1, 0x20,
0x9f, 0x20, 0xa6, 0x75, 0xd2, 0xc9, 0x56, 0xa7, 0xeb, 0x39, 0xf7, 0xe9, 0x1e, 0x1c, 0x4a, 0x78,
0x51, 0x90, 0x7e, 0x72, 0xa1, 0x20, 0xcd, 0x5b, 0xaa, 0xc8, 0xba, 0x4e, 0x02, 0x29, 0x04, 0xfd,
0xcf, 0x25, 0xa8, 0x26, 0xe8, 0xe2, 0xa6, 0x73, 0x4e, 0x19, 0xf1, 0x4c, 0x2f, 0x09, 0x61, 0x05,
0x0c, 0x92, 0xb4, 0xc7, 0x83, 0xd8, 0xab, 0x50, 0xe3, 0x17, 0x2a, 0x39, 0x5d, 0x14, 0xd3, 0x55,
0x4e, 0x10, 0x93, 0xaf, 0x41, 0x9d, 0x05, 0xcc, 0x1a, 0x9b, 0x4c, 0x64, 0xec, 0x92, 0xe4, 0x16,
0x24, 0x91, 0xaf, 0xd1, 0x5b, 0x70, 0x8d, 0x8d, 0xa2, 0x80, 0xb1, 0x31, 0xaf, 0xe2, 0x44, 0xdd,
0x22, 0xcb, 0x8c, 0x45, 0xac, 0xa5, 0x13, 0xb2, 0x9e, 0xa1, 0x3c, 0x7a, 0x4f, 0x16, 0x73, 0xd7,
0x15, 0x41, 0x64, 0x11, 0xaf, 0xa4, 0x54, 0xee, 0xda, 0xa8, 0x09, 0x4b, 0xa1, 0xac, 0x09, 0x44,
0xac, 0x28, 0xe0, 0x64, 0x88, 0x4c, 0x58, 0xf5, 0x88, 0x45, 0xe3, 0x88, 0x38, 0xe6, 0xb1, 0x4b,
0xc6, 0x8e, 0xbc, 0x59, 0x36, 0x72, 0x17, 0xd9, 0x89, 0x5a, 0xda, 0x0f, 0x05, 0x37, 0x6e, 0x24,
0x70, 0x72, 0xcc, 0xeb, 0x03, 0xf9, 0x0f, 0xad, 0x42, 0xbd, 0xff, 0xa4, 0x3f, 0x30, 0xf6, 0xcc,
0xbd, 0xfd, 0x4d, 0x43, 0xb5, 0x12, 0xfa, 0x06, 0x96, 0xc3, 0x02, 0x9f, 0x1f, 0xec, 0x0f, 0x3a,
0xbb, 0xe6, 0x60, 0xa7, 0xfb, 0xa8, 0xaf, 0x15, 0xd1, 0x4d, 0xb8, 0x36, 0xd8, 0xc6, 0xfb, 0x83,
0xc1, 0xae, 0xb1, 0x69, 0x1e, 0x18, 0x78, 0x67, 0x7f, 0xb3, 0xaf, 0x95, 0x10, 0x82, 0xc6, 0x84,
0x3c, 0xd8, 0xd9, 0x33, 0xb4, 0x45, 0x54, 0x87, 0xa5, 0x03, 0x03, 0x77, 0x8d, 0xde, 0x40, 0x2b,
0xeb, 0x7f, 0x2f, 0x42, 0x3d, 0x63, 0x45, 0xee, 0xc8, 0x11, 0x95, 0xd5, 0xfc, 0x22, 0xe6, 0x7f,
0x79, 0x30, 0xb1, 0x2d, 0x7b, 0x24, 0xad, 0xb3, 0x88, 0xe5, 0x40, 0x54, 0xf0, 0xd6, 0x59, 0xe6,
0x9c, 0x2f, 0xe2, 0xaa, 0x67, 0x9d, 0x49, 0x90, 0xd7, 0x61, 0xf9, 0x84, 0x44, 0x3e, 0x19, 0xab,
0x79, 0x69, 0x91, 0xba, 0xa4, 0xc9, 0x25, 0x6b, 0xa0, 0xa9, 0x25, 0x13, 0x18, 0x69, 0x8e, 0x86,
0xa4, 0xef, 0x25, 0x60, 0x47, 0x97, 0xb5, 0x5e, 0x11, 0x5a, 0xbf, 0x3f, 0xbb, 0x93, 0x3e, 0x4b,
0xf1, 0xfd, 0x54, 0xf1, 0x4b, 0x50, 0xc2, 0xc9, 0xab, 0x7a, 0xb7, 0xd3, 0xdd, 0xe6, 0xca, 0x5e,
0x81, 0xda, 0x5e, 0xe7, 0x53, 0xf3, 0xb0, 0x2f, 0xde, 0x68, 0x90, 0x06, 0xcb, 0x8f, 0x0c, 0xdc,
0x33, 0x76, 0x15, 0xa5, 0x84, 0x6e, 0x80, 0xa6, 0x28, 0x93, 0x75, 0x8b, 0xfa, 0x9f, 0x8a, 0xb0,
0x2a, 0xe3, 0x7a, 0xfa, 0x6c, 0xf8, 0xec, 0xf7, 0xbb, 0xf9, 0x43, 0x6f, 0x13, 0x96, 0x3c, 0x42,
0x53, 0x3b, 0xd4, 0x70, 0x32, 0x44, 0x2e, 0xd4, 0x2d, 0xdf, 0x0f, 0x98, 0x78, 0x7b, 0xa2, 0x2a,
0x44, 0x6e, 0xcd, 0xf4, 0xca, 0x95, 0x4a, 0xde, 0xee, 0x4c, 0x90, 0x64, 0x84, 0xcc, 0x62, 0xb7,
0x3e, 0x00, 0xed, 0xe2, 0x82, 0x59, 0xf2, 0xd2, 0x9b, 0xef, 0x4e, 0xd2, 0x12, 0xe1, 0x0e, 0x7a,
0xd8, 0x7b, 0xd4, 0xdb, 0x7f, 0xdc, 0xd3, 0x16, 0xf8, 0x00, 0x1f, 0xf6, 0x7a, 0x3b, 0xbd, 0x2d,
0xad, 0x80, 0x00, 0x2a, 0xc6, 0xa7, 0x3b, 0x03, 0x63, 0x53, 0x2b, 0x6e, 0xfc, 0x73, 0x05, 0x2a,
0x52, 0x48, 0xf4, 0x95, 0x4a, 0xc9, 0xd9, 0x76, 0x30, 0xfa, 0x60, 0xe6, 0xd2, 0x76, 0xaa, 0xc5,
0xdc, 0xfa, 0x70, 0x6e, 0x7e, 0xf5, 0x7c, 0xbb, 0x80, 0x7e, 0x5d, 0x80, 0xe5, 0xa9, 0xf7, 0xca,
0xbc, 0x4f, 0x6d, 0x57, 0x74, 0x9f, 0x5b, 0xdf, 0x99, 0x8b, 0x37, 0x95, 0xe5, 0x57, 0x05, 0xa8,
0x67, 0xfa, 0xae, 0xe8, 0xfe, 0x3c, 0xbd, 0x5a, 0x29, 0xc9, 0x83, 0xf9, 0xdb, 0xbc, 0xfa, 0xc2,
0x3b, 0x05, 0xf4, 0xcb, 0x02, 0xd4, 0x33, 0x1d, 0xc8, 0xdc, 0xa2, 0x5c, 0xee, 0x97, 0xe6, 0x16,
0xe5, 0xaa, 0x86, 0xe7, 0x02, 0xfa, 0x59, 0x01, 0x6a, 0x69, 0x37, 0x11, 0xdd, 0x9d, 0xbd, 0xff,
0x28, 0x85, 0xb8, 0x37, 0x6f, 0xe3, 0x52, 0x5f, 0x40, 0x3f, 0x81, 0x6a, 0xd2, 0x7a, 0x43, 0x79,
0xd3, 0xc8, 0x85, 0xbe, 0x5e, 0xeb, 0xee, 0xcc, 0x7c, 0xd9, 0xed, 0x93, 0x7e, 0x58, 0xee, 0xed,
0x2f, 0x74, 0xee, 0x5a, 0x77, 0x67, 0xe6, 0x4b, 0xb7, 0xe7, 0x9e, 0x90, 0x69, 0x9b, 0xe5, 0xf6,
0x84, 0xcb, 0xfd, 0xba, 0xdc, 0x9e, 0x70, 0x55, 0x97, 0x4e, 0x0a, 0x92, 0x69, 0xbc, 0xe5, 0x16,
0xe4, 0x72, 0x73, 0x2f, 0xb7, 0x20, 0x57, 0xf4, 0xf9, 0x94, 0x4b, 0x4e, 0x0a, 0xf4, 0xbb, 0x33,
0xf7, 0xaa, 0x66, 0x74, 0xc9, 0x4b, 0xdd, 0x32, 0x7d, 0x01, 0xfd, 0x5c, 0x3d, 0x19, 0xc8, 0x46,
0x17, 0x9a, 0x05, 0x6a, 0xaa, 0x37, 0xd6, 0xba, 0x33, 0x5f, 0xaa, 0x11, 0x31, 0xe2, 0x17, 0x05,
0x80, 0x49, 0x4b, 0x2c, 0xb7, 0x10, 0x97, 0x7a, 0x71, 0xad, 0xfb, 0x73, 0x70, 0x66, 0x8f, 0x47,
0xd2, 0x05, 0xcb, 0x7d, 0x3c, 0x2e, 0xb4, 0xec, 0x72, 0x1f, 0x8f, 0x8b, 0xed, 0x36, 0x7d, 0xe1,
0xe3, 0xa5, 0xef, 0x97, 0x65, 0xee, 0xaf, 0x88, 0x9f, 0xf7, 0xfe, 0x1b, 0x00, 0x00, 0xff, 0xff,
0xdc, 0x12, 0xd9, 0x00, 0x29, 0x26, 0x00, 0x00,
0xf1, 0x17, 0x9f, 0x22, 0x8b, 0x12, 0x35, 0xdb, 0xbb, 0x6b, 0xd3, 0x34, 0xfe, 0x7f, 0xaf, 0x07,
0x70, 0xb0, 0xb0, 0xbd, 0x94, 0x2d, 0x23, 0xfb, 0x4a, 0xfc, 0xa0, 0xa9, 0x59, 0x49, 0x5e, 0x89,
0x52, 0x9a, 0x14, 0xd6, 0x9b, 0xc4, 0x9e, 0x8c, 0x66, 0x7a, 0xc9, 0x59, 0x71, 0x1e, 0xee, 0xe9,
0x91, 0x25, 0x04, 0x41, 0x82, 0x04, 0x08, 0x92, 0x43, 0x80, 0x5c, 0x8c, 0x00, 0x39, 0x26, 0xc7,
0x7c, 0x83, 0x04, 0xbe, 0xe4, 0x53, 0x04, 0xc8, 0x25, 0x39, 0x04, 0xc8, 0x35, 0xdf, 0x20, 0xe8,
0xc7, 0x0c, 0x87, 0x92, 0xd6, 0x3b, 0xe4, 0xe6, 0x44, 0x76, 0x75, 0xd7, 0xaf, 0x6b, 0xaa, 0xaa,
0xab, 0xaa, 0xbb, 0x40, 0x0f, 0x27, 0xf1, 0xc8, 0xf5, 0xa3, 0x75, 0x87, 0xba, 0x27, 0x84, 0x46,
0xeb, 0x21, 0x0d, 0x58, 0xa0, 0x46, 0x1d, 0x31, 0x40, 0x6f, 0x8c, 0xad, 0x68, 0xec, 0xda, 0x01,
0x0d, 0x3b, 0x7e, 0xe0, 0x59, 0x4e, 0x47, 0xf1, 0x74, 0x14, 0x8f, 0x5c, 0xd6, 0xfe, 0xff, 0x51,
0x10, 0x8c, 0x26, 0x44, 0x22, 0x1c, 0xc5, 0x4f, 0xd6, 0x9d, 0x98, 0x5a, 0xcc, 0x0d, 0x7c, 0x35,
0xff, 0xda, 0xf9, 0x79, 0xe6, 0x7a, 0x24, 0x62, 0x96, 0x17, 0xaa, 0x05, 0x1f, 0x8d, 0x5c, 0x36,
0x8e, 0x8f, 0x3a, 0x76, 0xe0, 0xad, 0xa7, 0x5b, 0xae, 0x8b, 0x2d, 0xd7, 0x13, 0x31, 0xa3, 0xb1,
0x45, 0x89, 0xb3, 0x3e, 0xb6, 0x27, 0x51, 0x48, 0x6c, 0xfe, 0x6b, 0xf2, 0x3f, 0x0a, 0x61, 0x2b,
0x3f, 0x42, 0xc4, 0x68, 0x6c, 0xb3, 0xe4, 0x7b, 0x2d, 0xc6, 0xa8, 0x7b, 0x14, 0x33, 0x22, 0x81,
0xf4, 0x57, 0xe0, 0xe5, 0xa1, 0x15, 0x1d, 0xf7, 0x02, 0xff, 0x89, 0x3b, 0x1a, 0xd8, 0x63, 0xe2,
0x59, 0x98, 0x7c, 0x11, 0x93, 0x88, 0xe9, 0x3f, 0x84, 0xd6, 0xc5, 0xa9, 0x28, 0x0c, 0xfc, 0x88,
0xa0, 0x8f, 0xa0, 0xcc, 0xa5, 0x69, 0x15, 0x6e, 0x14, 0x6e, 0x36, 0x36, 0xde, 0xee, 0x3c, 0x4b,
0x71, 0x52, 0x86, 0x8e, 0xfa, 0x8a, 0xce, 0x20, 0x24, 0x36, 0x16, 0x9c, 0xfa, 0x75, 0xb8, 0xda,
0xb3, 0x42, 0xeb, 0xc8, 0x9d, 0xb8, 0xcc, 0x25, 0x51, 0xb2, 0x69, 0x0c, 0xd7, 0x66, 0xc9, 0x6a,
0xc3, 0xcf, 0x60, 0xc5, 0xce, 0xd0, 0xd5, 0xc6, 0xf7, 0x3a, 0xb9, 0x2c, 0xd6, 0xd9, 0x14, 0xa3,
0x19, 0xe0, 0x19, 0x38, 0xfd, 0x1a, 0xa0, 0x07, 0xae, 0x3f, 0x22, 0x34, 0xa4, 0xae, 0xcf, 0x12,
0x61, 0xbe, 0x2e, 0xc1, 0xd5, 0x19, 0xb2, 0x12, 0xe6, 0x29, 0x40, 0xaa, 0x47, 0x2e, 0x4a, 0xe9,
0x66, 0x63, 0xe3, 0x93, 0x9c, 0xa2, 0x5c, 0x82, 0xd7, 0xe9, 0xa6, 0x60, 0x86, 0xcf, 0xe8, 0x19,
0xce, 0xa0, 0xa3, 0xcf, 0xa1, 0x3a, 0x26, 0xd6, 0x84, 0x8d, 0x5b, 0xc5, 0x1b, 0x85, 0x9b, 0xcd,
0x8d, 0x07, 0x2f, 0xb0, 0xcf, 0xb6, 0x00, 0x1a, 0x30, 0x8b, 0x11, 0xac, 0x50, 0xd1, 0x2d, 0x40,
0xf2, 0x9f, 0xe9, 0x90, 0xc8, 0xa6, 0x6e, 0xc8, 0x1d, 0xb9, 0x55, 0xba, 0x51, 0xb8, 0x59, 0xc7,
0x57, 0xe4, 0xcc, 0xe6, 0x74, 0xa2, 0x1d, 0xc2, 0xda, 0x39, 0x69, 0x91, 0x06, 0xa5, 0x63, 0x72,
0x26, 0x2c, 0x52, 0xc7, 0xfc, 0x2f, 0xda, 0x82, 0xca, 0x89, 0x35, 0x89, 0x89, 0x10, 0xb9, 0xb1,
0xf1, 0xee, 0xf3, 0xdc, 0x43, 0xb9, 0xe8, 0x54, 0x0f, 0x58, 0xf2, 0xdf, 0x2f, 0xde, 0x2d, 0xe8,
0xf7, 0xa0, 0x91, 0x91, 0x1b, 0x35, 0x01, 0x0e, 0xfb, 0x9b, 0xc6, 0xd0, 0xe8, 0x0d, 0x8d, 0x4d,
0x6d, 0x09, 0xad, 0x42, 0xfd, 0xb0, 0xbf, 0x6d, 0x74, 0x77, 0x87, 0xdb, 0x8f, 0xb5, 0x02, 0x6a,
0xc0, 0x72, 0x32, 0x28, 0xea, 0xa7, 0x80, 0x30, 0xb1, 0x83, 0x13, 0x42, 0xb9, 0x23, 0x2b, 0xab,
0xa2, 0x97, 0x61, 0x99, 0x59, 0xd1, 0xb1, 0xe9, 0x3a, 0x4a, 0xe6, 0x2a, 0x1f, 0xee, 0x38, 0x68,
0x07, 0xaa, 0x63, 0xcb, 0x77, 0x26, 0xcf, 0x97, 0x7b, 0x56, 0xd5, 0x1c, 0x7c, 0x5b, 0x30, 0x62,
0x05, 0xc0, 0xbd, 0x7b, 0x66, 0x67, 0x69, 0x00, 0xfd, 0x31, 0x68, 0x03, 0x66, 0x51, 0x96, 0x15,
0xc7, 0x80, 0x32, 0xdf, 0x5f, 0x79, 0xf4, 0x3c, 0x7b, 0xca, 0x93, 0x89, 0x05, 0xbb, 0xfe, 0x9f,
0x22, 0x5c, 0xc9, 0x60, 0x2b, 0x4f, 0x7d, 0x04, 0x55, 0x4a, 0xa2, 0x78, 0xc2, 0x04, 0x7c, 0x73,
0xe3, 0xc3, 0x9c, 0xf0, 0x17, 0x90, 0x3a, 0x58, 0xc0, 0x60, 0x05, 0x87, 0x6e, 0x82, 0x26, 0x39,
0x4c, 0x42, 0x69, 0x40, 0x4d, 0x2f, 0x1a, 0x09, 0xad, 0xd5, 0x71, 0x53, 0xd2, 0x0d, 0x4e, 0xde,
0x8b, 0x46, 0x19, 0xad, 0x96, 0x5e, 0x50, 0xab, 0xc8, 0x02, 0xcd, 0x27, 0xec, 0xcb, 0x80, 0x1e,
0x9b, 0x5c, 0xb5, 0xd4, 0x75, 0x48, 0xab, 0x2c, 0x40, 0x6f, 0xe7, 0x04, 0xed, 0x4b, 0xf6, 0x7d,
0xc5, 0x8d, 0xd7, 0xfc, 0x59, 0x82, 0xfe, 0x16, 0x54, 0xe5, 0x97, 0x72, 0x4f, 0x1a, 0x1c, 0xf6,
0x7a, 0xc6, 0x60, 0xa0, 0x2d, 0xa1, 0x3a, 0x54, 0xb0, 0x31, 0xc4, 0xdc, 0xc3, 0xea, 0x50, 0x79,
0xd0, 0x1d, 0x76, 0x77, 0xb5, 0xa2, 0xfe, 0x26, 0xac, 0x3d, 0xb2, 0x5c, 0x96, 0xc7, 0xb9, 0xf4,
0x00, 0xb4, 0xe9, 0x5a, 0x65, 0x9d, 0x9d, 0x19, 0xeb, 0xe4, 0x57, 0x8d, 0x71, 0xea, 0xb2, 0x73,
0xf6, 0xd0, 0xa0, 0x44, 0x28, 0x55, 0x26, 0xe0, 0x7f, 0xf5, 0x2f, 0x61, 0x6d, 0xc0, 0x82, 0x30,
0x97, 0xe7, 0xbf, 0x07, 0xcb, 0x3c, 0x47, 0x05, 0x31, 0x53, 0xae, 0xff, 0x4a, 0x47, 0xe6, 0xb0,
0x4e, 0x92, 0xc3, 0x3a, 0x9b, 0x2a, 0xc7, 0xe1, 0x64, 0x25, 0x7a, 0x09, 0xaa, 0x91, 0x3b, 0xf2,
0xad, 0x89, 0x8a, 0x16, 0x6a, 0xa4, 0x23, 0xee, 0xe4, 0xc9, 0xc6, 0xca, 0xf1, 0x7b, 0x80, 0x36,
0x49, 0xc4, 0x68, 0x70, 0x96, 0x4b, 0x9e, 0x6b, 0x50, 0x79, 0x12, 0x50, 0x5b, 0x1e, 0xc4, 0x1a,
0x96, 0x03, 0x7e, 0xa8, 0x66, 0x40, 0x14, 0xf6, 0x2d, 0x40, 0x3b, 0x3e, 0xcf, 0x29, 0xf9, 0x0c,
0xf1, 0xdb, 0x22, 0x5c, 0x9d, 0x59, 0xaf, 0x8c, 0xb1, 0xf8, 0x39, 0xe4, 0x81, 0x29, 0x8e, 0xe4,
0x39, 0x44, 0xfb, 0x50, 0x95, 0x2b, 0x94, 0x26, 0xef, 0xcc, 0x01, 0x24, 0xd3, 0x94, 0x82, 0x53,
0x30, 0x97, 0x3a, 0x7d, 0xe9, 0x7f, 0xed, 0xf4, 0x5a, 0xf2, 0x1d, 0xd1, 0x73, 0xf5, 0xf7, 0x03,
0xb8, 0x92, 0x59, 0xac, 0x94, 0xf7, 0x00, 0x2a, 0x11, 0x27, 0x28, 0xed, 0xbd, 0x33, 0xa7, 0xf6,
0x22, 0x2c, 0xd9, 0xf5, 0xab, 0x12, 0xdc, 0x38, 0x21, 0x7e, 0x2a, 0x8a, 0xbe, 0x09, 0x57, 0x06,
0xc2, 0xb5, 0x72, 0xf9, 0xce, 0xd4, 0x2d, 0x8b, 0x33, 0x6e, 0x79, 0x0d, 0x50, 0x16, 0x45, 0x39,
0xcf, 0x19, 0xac, 0x19, 0xa7, 0xc4, 0xce, 0x85, 0xdc, 0x82, 0x65, 0x3b, 0xf0, 0x3c, 0xcb, 0x77,
0x5a, 0xc5, 0x1b, 0xa5, 0x9b, 0x75, 0x9c, 0x0c, 0xb3, 0xe7, 0xa7, 0x94, 0xf7, 0xfc, 0xe8, 0xbf,
0x29, 0x80, 0x36, 0xdd, 0x5b, 0x29, 0x92, 0x4b, 0xcf, 0x1c, 0x0e, 0xc4, 0xf7, 0x5e, 0xc1, 0x6a,
0xa4, 0xe8, 0xc9, 0x11, 0x97, 0x74, 0x42, 0x69, 0x26, 0x84, 0x94, 0x5e, 0x30, 0x84, 0xe8, 0xff,
0x2a, 0x00, 0xba, 0x58, 0x28, 0xa1, 0xd7, 0x61, 0x25, 0x22, 0xbe, 0x63, 0x4a, 0x35, 0x4a, 0x0b,
0xd7, 0x70, 0x83, 0xd3, 0xa4, 0x3e, 0x23, 0x84, 0xa0, 0x4c, 0x4e, 0x89, 0xad, 0x4e, 0xab, 0xf8,
0x8f, 0xc6, 0xb0, 0xf2, 0x24, 0x32, 0xdd, 0x28, 0x98, 0x58, 0x69, 0x45, 0xd1, 0xdc, 0x30, 0x16,
0x2e, 0xd8, 0x3a, 0x0f, 0x06, 0x3b, 0x09, 0x18, 0x6e, 0x3c, 0x89, 0xd2, 0x81, 0xde, 0x81, 0x46,
0x66, 0x0e, 0xd5, 0xa0, 0xdc, 0xdf, 0xef, 0x1b, 0xda, 0x12, 0x02, 0xa8, 0xf6, 0xb6, 0xf1, 0xfe,
0xfe, 0x50, 0x46, 0xed, 0x9d, 0xbd, 0xee, 0x96, 0xa1, 0x15, 0xf5, 0xdf, 0x57, 0x00, 0xa6, 0xe9,
0x13, 0x35, 0xa1, 0x98, 0x5a, 0xba, 0xe8, 0x3a, 0xfc, 0x63, 0x7c, 0xcb, 0x23, 0xca, 0x7b, 0xc4,
0x7f, 0xb4, 0x01, 0xd7, 0xbd, 0x68, 0x14, 0x5a, 0xf6, 0xb1, 0xa9, 0xb2, 0x9e, 0x2d, 0x98, 0xc5,
0x57, 0xad, 0xe0, 0xab, 0x6a, 0x52, 0x49, 0x2d, 0x71, 0x77, 0xa1, 0x44, 0xfc, 0x93, 0x56, 0x59,
0x54, 0x87, 0xf7, 0xe7, 0x4e, 0xeb, 0x1d, 0xc3, 0x3f, 0x91, 0xd5, 0x20, 0x87, 0x41, 0x7d, 0xa8,
0x53, 0x12, 0x05, 0x31, 0xb5, 0x49, 0xd4, 0xaa, 0xcc, 0x75, 0xc8, 0x70, 0xc2, 0x87, 0xa7, 0x10,
0x68, 0x13, 0xaa, 0x5e, 0x10, 0xfb, 0x2c, 0x6a, 0x55, 0x85, 0x80, 0x6f, 0xe7, 0x04, 0xdb, 0xe3,
0x4c, 0x58, 0xf1, 0xa2, 0x2d, 0x58, 0x76, 0xc8, 0x89, 0xcb, 0x65, 0x5a, 0x16, 0x30, 0xb7, 0xf2,
0xda, 0x57, 0x70, 0xe1, 0x84, 0x9b, 0x2b, 0x3d, 0x8e, 0x08, 0x6d, 0xd5, 0xa4, 0xd2, 0xf9, 0x7f,
0xf4, 0x2a, 0xd4, 0xad, 0xc9, 0x24, 0xb0, 0x4d, 0xc7, 0xa5, 0xad, 0xba, 0x98, 0xa8, 0x09, 0xc2,
0xa6, 0x4b, 0xd1, 0x6b, 0xd0, 0x90, 0x27, 0xc3, 0x0c, 0x2d, 0x36, 0x6e, 0x81, 0x98, 0x06, 0x49,
0x3a, 0xb0, 0xd8, 0x58, 0x2d, 0x20, 0x94, 0xca, 0x05, 0x8d, 0x74, 0x01, 0xa1, 0x54, 0x2c, 0xf8,
0x16, 0xac, 0x89, 0x63, 0x3e, 0xa2, 0x41, 0x1c, 0x9a, 0xc2, 0xe4, 0x2b, 0x62, 0xd1, 0x2a, 0x27,
0x6f, 0x71, 0x6a, 0x9f, 0xdb, 0xfe, 0x15, 0xa8, 0x3d, 0x0d, 0x8e, 0xe4, 0x82, 0x55, 0xb1, 0x60,
0xf9, 0x69, 0x70, 0x94, 0x4c, 0x49, 0x09, 0x5d, 0xa7, 0xd5, 0x94, 0x53, 0x62, 0xbc, 0xe3, 0xb4,
0x6f, 0x43, 0x2d, 0x31, 0xe0, 0x25, 0x05, 0xf2, 0xb5, 0x6c, 0x81, 0x5c, 0xcf, 0x56, 0xbb, 0x7f,
0x2f, 0x40, 0x3d, 0x35, 0x18, 0xf2, 0xe1, 0xaa, 0x00, 0xb4, 0x18, 0x71, 0xcc, 0xa9, 0xfd, 0x65,
0x90, 0x7d, 0x3f, 0xa7, 0xae, 0xbb, 0x09, 0x82, 0x0a, 0x34, 0xca, 0x19, 0x50, 0x8a, 0x3c, 0xdd,
0xef, 0x73, 0x58, 0x9b, 0xb8, 0x7e, 0x7c, 0x9a, 0xd9, 0x4b, 0x66, 0xb1, 0x6f, 0xe7, 0xdc, 0x6b,
0x97, 0x73, 0x4f, 0xf7, 0x68, 0x4e, 0x66, 0xc6, 0xfa, 0x57, 0x45, 0x78, 0xe9, 0x72, 0x71, 0x50,
0x1f, 0x4a, 0x76, 0x18, 0xab, 0x4f, 0xfb, 0xee, 0xbc, 0x9f, 0xd6, 0x0b, 0xe3, 0xe9, 0xae, 0x1c,
0x88, 0x57, 0xbe, 0x1e, 0xf1, 0x02, 0x7a, 0xa6, 0xbe, 0xe0, 0xc3, 0x79, 0x21, 0xf7, 0x04, 0xf7,
0x14, 0x55, 0xc1, 0x21, 0x0c, 0x35, 0x95, 0x3f, 0xf9, 0x41, 0x2c, 0xcd, 0x9f, 0x87, 0x13, 0x48,
0x9c, 0xe2, 0xe8, 0xb7, 0xe1, 0xfa, 0xa5, 0x9f, 0x82, 0xfe, 0x0f, 0xc0, 0x0e, 0x63, 0x53, 0xdc,
0x93, 0xa4, 0xdd, 0x4b, 0xb8, 0x6e, 0x87, 0xf1, 0x40, 0x10, 0xf4, 0x3b, 0xd0, 0x7a, 0x96, 0xbc,
0xfc, 0xf8, 0x48, 0x89, 0x4d, 0xef, 0x48, 0xe8, 0xa0, 0x84, 0x6b, 0x92, 0xb0, 0x77, 0xa4, 0xff,
0xae, 0x08, 0x6b, 0xe7, 0xc4, 0xe1, 0x29, 0x46, 0x1e, 0xc7, 0x24, 0xed, 0xc9, 0x11, 0x3f, 0x9b,
0xb6, 0xeb, 0x24, 0xb5, 0xa5, 0xf8, 0x2f, 0x82, 0x66, 0xa8, 0xea, 0xbe, 0xa2, 0x1b, 0x72, 0x87,
0xf6, 0x8e, 0x5c, 0x16, 0x89, 0x72, 0xbc, 0x82, 0xe5, 0x00, 0x3d, 0x86, 0x26, 0x25, 0x11, 0xa1,
0x27, 0xc4, 0x31, 0xc3, 0x80, 0xb2, 0x44, 0x61, 0x1b, 0xf3, 0x29, 0xec, 0x20, 0xa0, 0x0c, 0xaf,
0x26, 0x48, 0x7c, 0x14, 0xa1, 0x47, 0xb0, 0xea, 0x9c, 0xf9, 0x96, 0xe7, 0xda, 0x0a, 0xb9, 0xba,
0x30, 0xf2, 0x8a, 0x02, 0x12, 0xc0, 0xfc, 0xba, 0x99, 0x99, 0xe4, 0x1f, 0x36, 0xb1, 0x8e, 0xc8,
0x44, 0xe9, 0x44, 0x0e, 0x66, 0xcf, 0x6f, 0x45, 0x9d, 0x5f, 0xfd, 0x8f, 0x45, 0x68, 0xce, 0x1e,
0x80, 0xc4, 0x7e, 0x21, 0xa1, 0x6e, 0xe0, 0x64, 0xec, 0x77, 0x20, 0x08, 0xdc, 0x46, 0x7c, 0xfa,
0x8b, 0x38, 0x60, 0x56, 0x62, 0x23, 0x3b, 0x8c, 0xbf, 0xc7, 0xc7, 0xe7, 0x6c, 0x5f, 0x3a, 0x67,
0x7b, 0xf4, 0x36, 0x20, 0x65, 0xdf, 0x89, 0xeb, 0xb9, 0xcc, 0x3c, 0x3a, 0x63, 0x44, 0xea, 0xbf,
0x84, 0x35, 0x39, 0xb3, 0xcb, 0x27, 0x3e, 0xe6, 0x74, 0xa4, 0xc3, 0x6a, 0x10, 0x78, 0x66, 0x64,
0x07, 0x94, 0x98, 0x96, 0xf3, 0x54, 0xe4, 0x90, 0x12, 0x6e, 0x04, 0x81, 0x37, 0xe0, 0xb4, 0xae,
0xf3, 0x94, 0x87, 0x4c, 0x3b, 0x8c, 0x23, 0xc2, 0x4c, 0xfe, 0xd3, 0xaa, 0xca, 0x90, 0x29, 0x49,
0xbd, 0x30, 0x8e, 0x32, 0x0b, 0x3c, 0xe2, 0xf1, 0x90, 0x9f, 0x59, 0xb0, 0x47, 0x3c, 0xbe, 0xcb,
0xca, 0x01, 0xa1, 0x36, 0xf1, 0xd9, 0xd0, 0xb5, 0x8f, 0x23, 0x11, 0xce, 0x0b, 0x78, 0x86, 0xa6,
0x7f, 0x06, 0x15, 0x91, 0x44, 0xf8, 0xc7, 0x8b, 0x00, 0x2c, 0xe2, 0xb3, 0x54, 0x6f, 0x8d, 0x13,
0x44, 0x74, 0x7e, 0x15, 0xea, 0xe3, 0x20, 0x52, 0xd1, 0x5d, 0x7a, 0x5e, 0x8d, 0x13, 0xc4, 0x64,
0x1b, 0x6a, 0x94, 0x58, 0x4e, 0xe0, 0x4f, 0xce, 0x84, 0x5e, 0x6a, 0x38, 0x1d, 0xeb, 0x5f, 0x40,
0x55, 0x26, 0x97, 0x17, 0xc0, 0xbf, 0x05, 0xc8, 0x96, 0x69, 0x21, 0x24, 0xd4, 0x73, 0xa3, 0xc8,
0x0d, 0xfc, 0x28, 0x79, 0x13, 0x91, 0x33, 0x07, 0xd3, 0x09, 0xfd, 0xaf, 0x05, 0x59, 0x50, 0xc8,
0xdb, 0x2a, 0x2f, 0xc9, 0x54, 0x75, 0xb0, 0xf0, 0x95, 0x5e, 0x01, 0x24, 0x65, 0x35, 0x51, 0x6f,
0x3f, 0xf3, 0x96, 0xd5, 0x44, 0x96, 0xd5, 0x84, 0xd7, 0x70, 0xaa, 0x6e, 0x91, 0x70, 0xb2, 0x6c,
0x69, 0x38, 0xe9, 0x7d, 0x83, 0xe8, 0xff, 0x2e, 0xa4, 0x11, 0x21, 0xb9, 0x17, 0xa0, 0xcf, 0xa1,
0xc6, 0x0f, 0x97, 0xe9, 0x59, 0xa1, 0x7a, 0xe5, 0xea, 0x2d, 0x76, 0xe5, 0xe8, 0xf0, 0xb3, 0xb4,
0x67, 0x85, 0xb2, 0xa0, 0x59, 0x0e, 0xe5, 0x88, 0x47, 0x16, 0xcb, 0x99, 0x46, 0x16, 0xfe, 0x1f,
0xbd, 0x01, 0x4d, 0x2b, 0x66, 0x81, 0x69, 0x39, 0x27, 0x84, 0x32, 0x37, 0x22, 0xca, 0xc2, 0xab,
0x9c, 0xda, 0x4d, 0x88, 0xed, 0xfb, 0xb0, 0x92, 0xc5, 0x7c, 0x5e, 0x8e, 0xad, 0x64, 0x73, 0xec,
0x8f, 0x00, 0xa6, 0xe5, 0x2f, 0xf7, 0x04, 0x72, 0xea, 0x32, 0xd3, 0x0e, 0x1c, 0x19, 0xf9, 0x2a,
0xb8, 0xc6, 0x09, 0xbd, 0xc0, 0x21, 0xe7, 0x2e, 0x13, 0x95, 0xe4, 0x32, 0xc1, 0xcf, 0x26, 0x3f,
0x4e, 0xc7, 0xee, 0x64, 0x42, 0x1c, 0x25, 0x61, 0x3d, 0x08, 0xbc, 0x87, 0x82, 0xa0, 0x7f, 0x5d,
0x94, 0x1e, 0x21, 0xaf, 0x72, 0xb9, 0x4a, 0xcc, 0xd4, 0xd4, 0xa5, 0x17, 0x33, 0xf5, 0x3d, 0x80,
0x88, 0x59, 0x94, 0x17, 0x0c, 0x16, 0x53, 0xaf, 0x23, 0xed, 0x0b, 0xb7, 0x91, 0x61, 0xf2, 0x22,
0x8d, 0xeb, 0x6a, 0x75, 0x97, 0xa1, 0xf7, 0x61, 0xc5, 0x0e, 0xbc, 0x70, 0x42, 0x14, 0x73, 0xe5,
0xb9, 0xcc, 0x8d, 0x74, 0x7d, 0x97, 0x65, 0xae, 0x22, 0xd5, 0x17, 0xbd, 0x8a, 0xfc, 0xb9, 0x20,
0x6f, 0xa4, 0xd9, 0x0b, 0x31, 0x1a, 0x5d, 0xf2, 0xea, 0xba, 0xb5, 0xe0, 0xed, 0xfa, 0x9b, 0x9e,
0x5c, 0xdb, 0xef, 0xe7, 0x79, 0xe3, 0x7c, 0x76, 0x09, 0xf7, 0x97, 0x12, 0xd4, 0xd3, 0x8b, 0xed,
0x05, 0xdb, 0xdf, 0x85, 0x7a, 0xda, 0x0e, 0x50, 0xa5, 0xc9, 0x37, 0x9a, 0x27, 0x5d, 0x8c, 0x9e,
0x00, 0xb2, 0x46, 0xa3, 0xb4, 0x34, 0x33, 0xe3, 0xc8, 0x1a, 0x25, 0x4f, 0x01, 0x77, 0xe7, 0xd0,
0x43, 0x92, 0x9d, 0x0e, 0x39, 0x3f, 0xd6, 0xac, 0xd1, 0x68, 0x86, 0x82, 0x7e, 0x0c, 0xd7, 0x67,
0xf7, 0x30, 0x8f, 0xce, 0xcc, 0xd0, 0x75, 0xd4, 0x55, 0x66, 0x7b, 0xde, 0xbb, 0x7d, 0x67, 0x06,
0xfe, 0xe3, 0xb3, 0x03, 0xd7, 0x91, 0x3a, 0x47, 0xf4, 0xc2, 0x44, 0xfb, 0xa7, 0xf0, 0xf2, 0x33,
0x96, 0x5f, 0x62, 0x83, 0xfe, 0xec, 0x3b, 0xf3, 0xe2, 0x4a, 0xc8, 0x58, 0xef, 0x0f, 0x05, 0xf9,
0x04, 0x31, 0xab, 0x93, 0x6e, 0xb6, 0x3a, 0x5d, 0xcf, 0xb9, 0x4f, 0xef, 0xe0, 0x50, 0xc2, 0x8b,
0x82, 0xf4, 0x93, 0x73, 0x05, 0x69, 0xde, 0x52, 0x45, 0xd6, 0x75, 0x12, 0x48, 0x21, 0xe8, 0x7f,
0x2a, 0x41, 0x2d, 0x41, 0x17, 0x37, 0x9d, 0xb3, 0x88, 0x11, 0xcf, 0xf4, 0x92, 0x10, 0x56, 0xc0,
0x20, 0x49, 0x7b, 0x3c, 0x88, 0xbd, 0x0a, 0x75, 0x7e, 0xa1, 0x92, 0xd3, 0x45, 0x31, 0x5d, 0xe3,
0x04, 0x31, 0xf9, 0x1a, 0x34, 0x58, 0xc0, 0xac, 0x89, 0xc9, 0x44, 0xc6, 0x2e, 0x49, 0x6e, 0x41,
0x12, 0xf9, 0x1a, 0xbd, 0x05, 0x57, 0xd8, 0x98, 0x06, 0x8c, 0x4d, 0x78, 0x15, 0x27, 0xea, 0x16,
0x59, 0x66, 0x94, 0xb1, 0x96, 0x4e, 0xc8, 0x7a, 0x26, 0xe2, 0xd1, 0x7b, 0xba, 0x98, 0xbb, 0xae,
0x08, 0x22, 0x65, 0xbc, 0x9a, 0x52, 0xb9, 0x6b, 0xa3, 0x16, 0x2c, 0x87, 0xb2, 0x26, 0x10, 0xb1,
0xa2, 0x80, 0x93, 0x21, 0x32, 0x61, 0xcd, 0x23, 0x56, 0x14, 0x53, 0xe2, 0x98, 0x4f, 0x5c, 0x32,
0x71, 0xe4, 0xcd, 0xb2, 0x99, 0xbb, 0xc8, 0x4e, 0xd4, 0xd2, 0x79, 0x20, 0xb8, 0x71, 0x33, 0x81,
0x93, 0x63, 0x5e, 0x1f, 0xc8, 0x7f, 0x68, 0x0d, 0x1a, 0x83, 0xc7, 0x83, 0xa1, 0xb1, 0x67, 0xee,
0xed, 0x6f, 0x1a, 0xaa, 0x95, 0x30, 0x30, 0xb0, 0x1c, 0x16, 0xf8, 0xfc, 0x70, 0x7f, 0xd8, 0xdd,
0x35, 0x87, 0x3b, 0xbd, 0x87, 0x03, 0xad, 0x88, 0xae, 0xc3, 0x95, 0xe1, 0x36, 0xde, 0x1f, 0x0e,
0x77, 0x8d, 0x4d, 0xf3, 0xc0, 0xc0, 0x3b, 0xfb, 0x9b, 0x03, 0xad, 0x84, 0x10, 0x34, 0xa7, 0xe4,
0xe1, 0xce, 0x9e, 0xa1, 0x95, 0x51, 0x03, 0x96, 0x0f, 0x0c, 0xdc, 0x33, 0xfa, 0x43, 0xad, 0xa2,
0xff, 0xad, 0x08, 0x8d, 0x8c, 0x15, 0xb9, 0x23, 0xd3, 0x48, 0x56, 0xf3, 0x65, 0xcc, 0xff, 0xf2,
0x60, 0x62, 0x5b, 0xf6, 0x58, 0x5a, 0xa7, 0x8c, 0xe5, 0x40, 0x54, 0xf0, 0xd6, 0x69, 0xe6, 0x9c,
0x97, 0x71, 0xcd, 0xb3, 0x4e, 0x25, 0xc8, 0xeb, 0xb0, 0x72, 0x4c, 0xa8, 0x4f, 0x26, 0x6a, 0x5e,
0x5a, 0xa4, 0x21, 0x69, 0x72, 0xc9, 0x4d, 0xd0, 0xd4, 0x92, 0x29, 0x8c, 0x34, 0x47, 0x53, 0xd2,
0xf7, 0x12, 0xb0, 0xa3, 0x8b, 0x5a, 0xaf, 0x0a, 0xad, 0xdf, 0x9b, 0xdf, 0x49, 0x9f, 0xa5, 0xf8,
0x41, 0xaa, 0xf8, 0x65, 0x28, 0xe1, 0xe4, 0x55, 0xbd, 0xd7, 0xed, 0x6d, 0x73, 0x65, 0xaf, 0x42,
0x7d, 0xaf, 0xfb, 0xa9, 0x79, 0x38, 0x10, 0x6f, 0x34, 0x48, 0x83, 0x95, 0x87, 0x06, 0xee, 0x1b,
0xbb, 0x8a, 0x52, 0x42, 0xd7, 0x40, 0x53, 0x94, 0xe9, 0xba, 0xb2, 0xfe, 0xcf, 0x22, 0xac, 0xc9,
0xb8, 0x9e, 0x3e, 0x1b, 0x3e, 0xfb, 0xfd, 0x2e, 0x7b, 0x5d, 0x2f, 0xce, 0x5c, 0xd7, 0xd3, 0x5a,
0x51, 0xa4, 0xe5, 0xd2, 0xb4, 0x56, 0x14, 0xd7, 0xfc, 0x99, 0x90, 0x5d, 0x9e, 0x27, 0x64, 0xb7,
0x60, 0xd9, 0x23, 0x51, 0xaa, 0xf8, 0x3a, 0x4e, 0x86, 0xc8, 0x85, 0x86, 0xe5, 0xfb, 0x01, 0x13,
0x6f, 0x56, 0xc9, 0xed, 0x65, 0x6b, 0xae, 0xd7, 0xb1, 0xf4, 0x8b, 0x3b, 0xdd, 0x29, 0x92, 0x8c,
0xac, 0x59, 0xec, 0xf6, 0x07, 0xa0, 0x9d, 0x5f, 0x30, 0x4f, 0x3e, 0x7b, 0xf3, 0xdd, 0x69, 0x3a,
0x23, 0xdc, 0xb1, 0x0f, 0xfb, 0x0f, 0xfb, 0xfb, 0x8f, 0xfa, 0xda, 0x12, 0x1f, 0xe0, 0xc3, 0x7e,
0x7f, 0xa7, 0xbf, 0xa5, 0x15, 0x10, 0x40, 0xd5, 0xf8, 0x74, 0x67, 0x68, 0x6c, 0x6a, 0xc5, 0x8d,
0x7f, 0xac, 0x42, 0x55, 0x0a, 0x89, 0xbe, 0x52, 0xa9, 0x3c, 0xdb, 0x46, 0x46, 0x1f, 0xcc, 0x5d,
0x12, 0xcf, 0xb4, 0xa6, 0xdb, 0x1f, 0x2e, 0xcc, 0xaf, 0x9e, 0x7d, 0x97, 0xd0, 0xaf, 0x0b, 0xb0,
0x32, 0xf3, 0xce, 0x99, 0xf7, 0x89, 0xee, 0x92, 0xae, 0x75, 0xfb, 0x3b, 0x0b, 0xf1, 0xa6, 0xb2,
0xfc, 0xaa, 0x00, 0x8d, 0x4c, 0xbf, 0x16, 0xdd, 0x5b, 0xa4, 0xc7, 0x2b, 0x25, 0xb9, 0xbf, 0x78,
0x7b, 0x58, 0x5f, 0x7a, 0xa7, 0x80, 0x7e, 0x59, 0x80, 0x46, 0xa6, 0x73, 0x99, 0x5b, 0x94, 0x8b,
0x7d, 0xd6, 0xdc, 0xa2, 0x5c, 0xd6, 0x28, 0x5d, 0x42, 0x3f, 0x2b, 0x40, 0x3d, 0xed, 0x42, 0xa2,
0x3b, 0xf3, 0xf7, 0x2d, 0xa5, 0x10, 0x77, 0x17, 0x6d, 0x78, 0xea, 0x4b, 0xe8, 0x27, 0x50, 0x4b,
0x5a, 0x76, 0x28, 0x6f, 0xfa, 0x39, 0xd7, 0x0f, 0x6c, 0xdf, 0x99, 0x9b, 0x2f, 0xbb, 0x7d, 0xd2,
0x47, 0xcb, 0xbd, 0xfd, 0xb9, 0x8e, 0x5f, 0xfb, 0xce, 0xdc, 0x7c, 0xe9, 0xf6, 0xdc, 0x13, 0x32,
0xed, 0xb6, 0xdc, 0x9e, 0x70, 0xb1, 0xcf, 0x97, 0xdb, 0x13, 0x2e, 0xeb, 0xee, 0x49, 0x41, 0x32,
0x0d, 0xbb, 0xdc, 0x82, 0x5c, 0x6c, 0x0a, 0xe6, 0x16, 0xe4, 0x92, 0xfe, 0xa0, 0x72, 0xc9, 0x69,
0x61, 0x7f, 0x67, 0xee, 0x1e, 0xd7, 0x9c, 0x2e, 0x79, 0xa1, 0xcb, 0xa6, 0x2f, 0xa1, 0x9f, 0xab,
0xa7, 0x06, 0xd9, 0x20, 0x43, 0xf3, 0x40, 0xcd, 0xf4, 0xd4, 0xda, 0xb7, 0x17, 0x4b, 0x35, 0x22,
0x46, 0xfc, 0xa2, 0x00, 0x30, 0x6d, 0xa5, 0xe5, 0x16, 0xe2, 0x42, 0x0f, 0xaf, 0x7d, 0x6f, 0x01,
0xce, 0xec, 0xf1, 0x48, 0xba, 0x67, 0xb9, 0x8f, 0xc7, 0xb9, 0x56, 0x5f, 0xee, 0xe3, 0x71, 0xbe,
0x4d, 0xa7, 0x2f, 0x7d, 0xbc, 0xfc, 0xfd, 0x8a, 0xcc, 0xfd, 0x55, 0xf1, 0xf3, 0xde, 0x7f, 0x03,
0x00, 0x00, 0xff, 0xff, 0xa1, 0x73, 0x76, 0xe7, 0x61, 0x26, 0x00, 0x00,
}

View File

@ -572,12 +572,18 @@ message DriverTaskEvent {
// TaskId is the id of the task for the event
string task_id = 1;
// AllocId of the task for the event
string alloc_id = 2;
// TaskName is the name of the task for the event
string task_name = 3;
// Timestamp when the event occurred
google.protobuf.Timestamp timestamp = 2;
google.protobuf.Timestamp timestamp = 4;
// Message is the body of the event
string message = 3;
string message = 5;
// Annotations allows for additional key/value data to be sent along with the event
map<string,string> annotations = 4;
map<string,string> annotations = 6;
}

View File

@ -289,6 +289,8 @@ func (b *driverPluginServer) TaskEvents(req *proto.TaskEventsRequest, srv proto.
pbEvent := &proto.DriverTaskEvent{
TaskId: event.TaskID,
AllocId: event.AllocID,
TaskName: event.TaskName,
Timestamp: pbTimestamp,
Message: event.Message,
Annotations: event.Annotations,