executor: implement streaming stats API

plugins/driver: update driver interface to support streaming stats

client/tr: use streaming stats api

TODO:
 * how to handle errors and closed channel during stats streaming
 * prevent tight loop if Stats(ctx) returns an error

drivers: update drivers TaskStats RPC to handle streaming results

executor: better error handling in stats rpc

docker: better control and error handling of stats rpc

driver: allow stats to return a recoverable error
This commit is contained in:
Nick Ethier
2018-12-11 15:27:50 -05:00
parent 215a137e4a
commit fbf9a4c772
33 changed files with 1082 additions and 652 deletions

View File

@@ -2,6 +2,7 @@ package interfaces
import (
"context"
"time"
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/allocrunner/taskrunner/interfaces"
@@ -88,7 +89,7 @@ type TaskPrestartHook interface {
// DriverStats is the interface implemented by DriverHandles to return task stats.
type DriverStats interface {
Stats() (*cstructs.TaskResourceUsage, error)
Stats(context.Context, time.Duration) (<-chan *cstructs.TaskResourceUsage, error)
}
type TaskPoststartRequest struct {

View File

@@ -44,8 +44,8 @@ func (h *DriverHandle) Kill() error {
return h.driver.StopTask(h.taskID, h.task.KillTimeout, h.task.KillSignal)
}
func (h *DriverHandle) Stats() (*cstructs.TaskResourceUsage, error) {
return h.driver.TaskStats(h.taskID)
func (h *DriverHandle) Stats(ctx context.Context, interval time.Duration) (<-chan *cstructs.TaskResourceUsage, error) {
return h.driver.TaskStats(ctx, h.taskID, interval)
}
func (h *DriverHandle) Signal(s string) error {

View File

@@ -128,7 +128,7 @@ TRY:
return out, c, err
}
func (l *LazyHandle) Stats() (*cstructs.TaskResourceUsage, error) {
func (l *LazyHandle) Stats(ctx context.Context, interval time.Duration) (<-chan *cstructs.TaskResourceUsage, error) {
h, err := l.getHandle()
if err != nil {
return nil, err
@@ -138,7 +138,7 @@ func (l *LazyHandle) Stats() (*cstructs.TaskResourceUsage, error) {
first := true
TRY:
out, err := h.Stats()
out, err := h.Stats(ctx, interval)
if err == bstructs.ErrPluginShutdown && first {
first = false

View File

@@ -8,6 +8,7 @@ import (
hclog "github.com/hashicorp/go-hclog"
"github.com/hashicorp/nomad/client/allocrunner/interfaces"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
bstructs "github.com/hashicorp/nomad/plugins/base/structs"
)
@@ -22,8 +23,8 @@ type statsHook struct {
updater StatsUpdater
interval time.Duration
// stopCh is closed by Exited or Canceled
stopCh chan struct{}
// cancel is called by Exited or Canceled
cancel context.CancelFunc
mu sync.Mutex
@@ -48,13 +49,14 @@ func (h *statsHook) Poststart(ctx context.Context, req *interfaces.TaskPoststart
defer h.mu.Unlock()
// This shouldn't happen, but better safe than risk leaking a goroutine
if h.stopCh != nil {
if h.cancel != nil {
h.logger.Debug("poststart called twice without exiting between")
close(h.stopCh)
h.cancel()
}
h.stopCh = make(chan struct{})
go h.collectResourceUsageStats(req.DriverStats, h.stopCh)
ctx, cancel := context.WithCancel(ctx)
h.cancel = cancel
go h.collectResourceUsageStats(ctx, req.DriverStats)
return nil
}
@@ -63,40 +65,42 @@ func (h *statsHook) Exited(context.Context, *interfaces.TaskExitedRequest, *inte
h.mu.Lock()
defer h.mu.Unlock()
if h.stopCh == nil {
if h.cancel == nil {
// No stats running
return nil
}
// Close chan to stop stats collection
close(h.stopCh)
// Call cancel to stop stats collection
h.cancel()
// Clear chan so we don't double close for any reason
h.stopCh = nil
// Clear cancel func so we don't double call for any reason
h.cancel = nil
return nil
}
// collectResourceUsageStats starts collecting resource usage stats of a Task.
// Collection ends when the passed channel is closed
func (h *statsHook) collectResourceUsageStats(handle interfaces.DriverStats, stopCh <-chan struct{}) {
// start collecting the stats right away and then start collecting every
// collection interval
next := time.NewTimer(0)
defer next.Stop()
func (h *statsHook) collectResourceUsageStats(ctx context.Context, handle interfaces.DriverStats) {
ch, err := handle.Stats(ctx, h.interval)
if err != nil {
// Check if the driver doesn't implement stats
if err.Error() == cstructs.DriverStatsNotImplemented.Error() {
h.logger.Debug("driver does not support stats")
return
}
h.logger.Error("failed to start stats collection for task", "error", err)
}
for {
select {
case <-next.C:
// Reset the timer
next.Reset(h.interval)
// Collect stats from driver
ru, err := handle.Stats()
if err != nil {
// Check if the driver doesn't implement stats
if err.Error() == cstructs.DriverStatsNotImplemented.Error() {
h.logger.Debug("driver does not support stats")
return
case ru, ok := <-ch:
// Channel is closed
if !ok {
ch, err = handle.Stats(ctx, h.interval)
if err == nil {
continue
}
// We do not log when the plugin is shutdown since this is
@@ -105,15 +109,31 @@ func (h *statsHook) collectResourceUsageStats(handle interfaces.DriverStats, sto
// on the stop channel is the correct behavior
if err != bstructs.ErrPluginShutdown {
h.logger.Debug("error fetching stats of task", "error", err)
continue
}
// check if the error is terminal otherwise it's likely a
// transport error and we should retry
re, ok := err.(*structs.RecoverableError)
if ok && !re.IsRecoverable() {
return
}
h.logger.Warn("stats collection for task failed", "error", err)
continue
}
if ru.Err != nil {
h.logger.Warn("stats collection for task failed", "error", err)
}
// Update stats on TaskRunner and emit them
h.updater.UpdateStats(ru)
case <-stopCh:
return
default:
select {
case <-ctx.Done():
return
default:
}
}
}
}
@@ -122,14 +142,9 @@ func (h *statsHook) Shutdown() {
h.mu.Lock()
defer h.mu.Unlock()
if h.stopCh == nil {
if h.cancel == nil {
return
}
select {
case <-h.stopCh:
// Already closed
default:
close(h.stopCh)
}
h.cancel()
}

View File

@@ -40,7 +40,7 @@ type mockDriverStats struct {
err error
}
func (m *mockDriverStats) Stats() (*cstructs.TaskResourceUsage, error) {
func (m *mockDriverStats) Stats(ctx context.Context, interval time.Duration) (<-chan *cstructs.TaskResourceUsage, error) {
if m.err != nil {
return nil, m.err
}
@@ -59,7 +59,15 @@ func (m *mockDriverStats) Stats() (*cstructs.TaskResourceUsage, error) {
Pids: map[string]*cstructs.ResourceUsage{},
}
ru.Pids["task"] = ru.ResourceUsage
return ru, nil
ch := make(chan *cstructs.TaskResourceUsage)
go func() {
defer close(ch)
select {
case <-ctx.Done():
case ch <- ru:
}
}()
return ch, nil
}
// TestTaskRunner_StatsHook_PoststartExited asserts the stats hook starts and

View File

@@ -236,6 +236,9 @@ type TaskResourceUsage struct {
ResourceUsage *ResourceUsage
Timestamp int64 // UnixNano
Pids map[string]*ResourceUsage
// Err is set by the driver if an error occurred during an RPC stream
Err error
}
// AllocResourceUsage holds the aggregated task resource usage of the

View File

@@ -146,7 +146,6 @@ func (d *Driver) RecoverTask(handle *drivers.TaskHandle) error {
}
d.tasks.Set(handle.Config.ID, h)
go h.collectStats()
go h.run()
return nil
@@ -290,7 +289,6 @@ CREATE:
}
d.tasks.Set(cfg.ID, h)
go h.collectStats()
go h.run()
return handle, net, nil
@@ -1087,13 +1085,13 @@ func (d *Driver) InspectTask(taskID string) (*drivers.TaskStatus, error) {
return status, nil
}
func (d *Driver) TaskStats(taskID string) (*drivers.TaskResourceUsage, error) {
func (d *Driver) TaskStats(ctx context.Context, taskID string, interval time.Duration) (<-chan *drivers.TaskResourceUsage, error) {
h, ok := d.tasks.Get(taskID)
if !ok {
return nil, drivers.ErrTaskNotFound
}
return h.Stats()
return h.Stats(ctx, interval)
}
func (d *Driver) TaskEvents(ctx context.Context) (<-chan *drivers.TaskEvent, error) {

View File

@@ -1478,16 +1478,17 @@ func TestDockerDriver_Stats(t *testing.T) {
require.NoError(t, d.WaitUntilStarted(task.ID, 5*time.Second))
go func() {
time.Sleep(3 * time.Second)
ru, err := handle.Stats()
if err != nil {
t.Fatalf("err: %v", err)
defer d.DestroyTask(task.ID, true)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ch, err := handle.Stats(ctx, 1*time.Second)
assert.NoError(t, err)
select {
case ru := <-ch:
assert.NotNil(t, ru.ResourceUsage)
case <-time.After(3 * time.Second):
assert.Fail(t, "stats timeout")
}
if ru.ResourceUsage == nil {
d.DestroyTask(task.ID, true)
t.Fatalf("expected resource usage")
}
d.DestroyTask(task.ID, true)
}()
waitCh, err := d.WaitTask(context.Background(), task.ID)

View File

@@ -3,7 +3,6 @@ package docker
import (
"fmt"
"os"
"runtime"
"strings"
"sync"
"syscall"
@@ -14,7 +13,6 @@ import (
hclog "github.com/hashicorp/go-hclog"
plugin "github.com/hashicorp/go-plugin"
"github.com/hashicorp/nomad/drivers/docker/docklog"
"github.com/hashicorp/nomad/helper/stats"
"github.com/hashicorp/nomad/plugins/drivers"
"github.com/hashicorp/nomad/plugins/shared"
"golang.org/x/net/context"
@@ -160,16 +158,6 @@ func (h *taskHandle) Kill(killTimeout time.Duration, signal os.Signal) error {
return nil
}
func (h *taskHandle) Stats() (*drivers.TaskResourceUsage, error) {
h.resourceUsageLock.RLock()
defer h.resourceUsageLock.RUnlock()
var err error
if h.resourceUsage == nil {
err = fmt.Errorf("stats collection hasn't started yet")
}
return h.resourceUsage, err
}
func (h *taskHandle) run() {
exitCode, werr := h.waitClient.WaitContainer(h.containerID)
if werr != nil {
@@ -189,10 +177,9 @@ func (h *taskHandle) run() {
werr = fmt.Errorf("OOM Killed")
}
// Shutdown stats collection
close(h.doneCh)
// Shutdown the syslog collector
// Stop the container just incase the docker daemon's wait returned
// incorrectly
if err := h.client.StopContainer(h.containerID, 0); err != nil {
@@ -223,71 +210,3 @@ func (h *taskHandle) run() {
h.exitResultLock.Unlock()
close(h.waitCh)
}
// collectStats starts collecting resource usage stats of a docker container
func (h *taskHandle) collectStats() {
statsCh := make(chan *docker.Stats)
statsOpts := docker.StatsOptions{ID: h.containerID, Done: h.doneCh, Stats: statsCh, Stream: true}
go func() {
//TODO handle Stats error
if err := h.waitClient.Stats(statsOpts); err != nil {
h.logger.Debug("error collecting stats from container", "error", err)
}
}()
numCores := runtime.NumCPU()
for {
select {
case s := <-statsCh:
if s != nil {
ms := &drivers.MemoryStats{
RSS: s.MemoryStats.Stats.Rss,
Cache: s.MemoryStats.Stats.Cache,
Swap: s.MemoryStats.Stats.Swap,
MaxUsage: s.MemoryStats.MaxUsage,
Measured: DockerMeasuredMemStats,
}
cs := &drivers.CpuStats{
ThrottledPeriods: s.CPUStats.ThrottlingData.ThrottledPeriods,
ThrottledTime: s.CPUStats.ThrottlingData.ThrottledTime,
Measured: DockerMeasuredCpuStats,
}
// Calculate percentage
cs.Percent = calculatePercent(
s.CPUStats.CPUUsage.TotalUsage, s.PreCPUStats.CPUUsage.TotalUsage,
s.CPUStats.SystemCPUUsage, s.PreCPUStats.SystemCPUUsage, numCores)
cs.SystemMode = calculatePercent(
s.CPUStats.CPUUsage.UsageInKernelmode, s.PreCPUStats.CPUUsage.UsageInKernelmode,
s.CPUStats.CPUUsage.TotalUsage, s.PreCPUStats.CPUUsage.TotalUsage, numCores)
cs.UserMode = calculatePercent(
s.CPUStats.CPUUsage.UsageInUsermode, s.PreCPUStats.CPUUsage.UsageInUsermode,
s.CPUStats.CPUUsage.TotalUsage, s.PreCPUStats.CPUUsage.TotalUsage, numCores)
cs.TotalTicks = (cs.Percent / 100) * stats.TotalTicksAvailable() / float64(numCores)
h.resourceUsageLock.Lock()
h.resourceUsage = &drivers.TaskResourceUsage{
ResourceUsage: &drivers.ResourceUsage{
MemoryStats: ms,
CpuStats: cs,
},
Timestamp: s.Read.UTC().UnixNano(),
}
h.resourceUsageLock.Unlock()
}
case <-h.doneCh:
return
}
}
}
func calculatePercent(newSample, oldSample, newTotal, oldTotal uint64, cores int) float64 {
numerator := newSample - oldSample
denom := newTotal - oldTotal
if numerator <= 0 || denom <= 0 {
return 0.0
}
return (float64(numerator) / float64(denom)) * float64(cores) * 100.0
}

176
drivers/docker/stats.go Normal file
View File

@@ -0,0 +1,176 @@
package docker
import (
"context"
"fmt"
"io"
"runtime"
"time"
docker "github.com/fsouza/go-dockerclient"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/helper/stats"
nstructs "github.com/hashicorp/nomad/nomad/structs"
)
const (
// statsCollectorBackoffBaseline is the baseline time for exponential
// backoff while calling the docker stats api.
statsCollectorBackoffBaseline = 5 * time.Second
// statsCollectorBackoffLimit is the limit of the exponential backoff for
// calling the docker stats api.
statsCollectorBackoffLimit = 2 * time.Minute
)
// Stats starts collecting stats from the docker daemon and sends them on the
// returned channel.
func (h *taskHandle) Stats(ctx context.Context, interval time.Duration) (<-chan *cstructs.TaskResourceUsage, error) {
select {
case <-h.doneCh:
return nil, nstructs.NewRecoverableError(fmt.Errorf("container stopped"), false)
default:
}
ch := make(chan *cstructs.TaskResourceUsage, 1)
go h.collectStats(ctx, ch, interval)
return ch, nil
}
// collectStats starts collecting resource usage stats of a docker container
func (h *taskHandle) collectStats(ctx context.Context, ch chan *cstructs.TaskResourceUsage, interval time.Duration) {
defer close(ch)
// backoff and retry used if the docker stats API returns an error
var backoff time.Duration
var retry int
// loops until doneCh is closed
for {
if backoff > 0 {
select {
case <-time.After(backoff):
case <-ctx.Done():
return
case <-h.doneCh:
return
}
}
// make a channel for docker stats structs and start a collector to
// receive stats from docker and emit nomad stats
// statsCh will always be closed by docker client.
statsCh := make(chan *docker.Stats)
go h.collector(ch, statsCh, interval)
statsOpts := docker.StatsOptions{
ID: h.containerID,
Context: ctx,
Done: h.doneCh,
Stats: statsCh,
Stream: true,
}
// Stats blocks until an error has occurred, or doneCh has been closed
if err := h.client.Stats(statsOpts); err != nil && err != io.ErrClosedPipe {
// An error occurred during stats collection, retry with backoff
h.logger.Debug("error collecting stats from container", "error", err)
// Calculate the new backoff
backoff = (1 << (2 * uint64(retry))) * statsCollectorBackoffBaseline
if backoff > statsCollectorBackoffLimit {
backoff = statsCollectorBackoffLimit
}
// Increment retry counter
retry++
continue
}
// Stats finished either because context was canceled, doneCh was closed
// or the container stopped. Stop stats collections.
return
}
}
func (h *taskHandle) collector(destCh chan *cstructs.TaskResourceUsage, statsCh <-chan *docker.Stats, interval time.Duration) {
var resourceUsage *cstructs.TaskResourceUsage
// hasSentInitialStats is used so as to emit the first stats received from
// the docker daemon
var hasSentInitialStats bool
// timer is used to send nomad status at the specified interval
timer := time.NewTimer(interval)
for {
select {
case <-timer.C:
// it is possible for the timer to go off before the first stats
// has been emitted from docker
if resourceUsage == nil {
continue
}
// sending to destCh could block, drop this interval if it does
select {
case destCh <- resourceUsage:
default:
// Backpressure caused missed interval
}
timer.Reset(interval)
case s, ok := <-statsCh:
// if statsCh is closed stop collection
if !ok {
return
}
// s should always be set, but check and skip just in case
if s != nil {
resourceUsage = dockerStatsToTaskResourceUsage(s)
// send stats next interation if this is the first time received
// from docker
if !hasSentInitialStats {
timer.Reset(0)
hasSentInitialStats = true
}
}
}
}
}
func dockerStatsToTaskResourceUsage(s *docker.Stats) *cstructs.TaskResourceUsage {
ms := &cstructs.MemoryStats{
RSS: s.MemoryStats.Stats.Rss,
Cache: s.MemoryStats.Stats.Cache,
Swap: s.MemoryStats.Stats.Swap,
MaxUsage: s.MemoryStats.MaxUsage,
Measured: DockerMeasuredMemStats,
}
cs := &cstructs.CpuStats{
ThrottledPeriods: s.CPUStats.ThrottlingData.ThrottledPeriods,
ThrottledTime: s.CPUStats.ThrottlingData.ThrottledTime,
Measured: DockerMeasuredCpuStats,
}
// Calculate percentage
cs.Percent = calculatePercent(
s.CPUStats.CPUUsage.TotalUsage, s.PreCPUStats.CPUUsage.TotalUsage,
s.CPUStats.SystemCPUUsage, s.PreCPUStats.SystemCPUUsage, runtime.NumCPU())
cs.SystemMode = calculatePercent(
s.CPUStats.CPUUsage.UsageInKernelmode, s.PreCPUStats.CPUUsage.UsageInKernelmode,
s.CPUStats.CPUUsage.TotalUsage, s.PreCPUStats.CPUUsage.TotalUsage, runtime.NumCPU())
cs.UserMode = calculatePercent(
s.CPUStats.CPUUsage.UsageInUsermode, s.PreCPUStats.CPUUsage.UsageInUsermode,
s.CPUStats.CPUUsage.TotalUsage, s.PreCPUStats.CPUUsage.TotalUsage, runtime.NumCPU())
cs.TotalTicks = (cs.Percent / 100) * stats.TotalTicksAvailable() / float64(runtime.NumCPU())
return &cstructs.TaskResourceUsage{
ResourceUsage: &cstructs.ResourceUsage{
MemoryStats: ms,
CpuStats: cs,
},
Timestamp: s.Read.UTC().UnixNano(),
}
}
func calculatePercent(newSample, oldSample, newTotal, oldTotal uint64, cores int) float64 {
numerator := newSample - oldSample
denom := newTotal - oldTotal
if numerator <= 0 || denom <= 0 {
return 0.0
}
return (float64(numerator) / float64(denom)) * float64(cores) * 100.0
}

View File

@@ -434,13 +434,13 @@ func (d *Driver) InspectTask(taskID string) (*drivers.TaskStatus, error) {
return handle.TaskStatus(), nil
}
func (d *Driver) TaskStats(taskID string) (*drivers.TaskResourceUsage, error) {
func (d *Driver) TaskStats(ctx context.Context, taskID string, interval time.Duration) (<-chan *drivers.TaskResourceUsage, error) {
handle, ok := d.tasks.Get(taskID)
if !ok {
return nil, drivers.ErrTaskNotFound
}
return handle.exec.Stats()
return handle.exec.Stats(ctx, interval)
}
func (d *Driver) TaskEvents(ctx context.Context) (<-chan *drivers.TaskEvent, error) {

View File

@@ -341,9 +341,16 @@ func TestExecDriver_Stats(t *testing.T) {
require.NotNil(handle)
require.NoError(harness.WaitUntilStarted(task.ID, 1*time.Second))
stats, err := harness.TaskStats(task.ID)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
statsCh, err := harness.TaskStats(ctx, task.ID, time.Second*10)
require.NoError(err)
require.NotZero(stats.ResourceUsage.MemoryStats.RSS)
select {
case stats := <-statsCh:
require.NotZero(stats.ResourceUsage.MemoryStats.RSS)
case <-time.After(time.Second):
require.Fail("timeout receiving from channel")
}
require.NoError(harness.DestroyTask(task.ID, true))
}

View File

@@ -496,13 +496,13 @@ func (d *Driver) InspectTask(taskID string) (*drivers.TaskStatus, error) {
return handle.TaskStatus(), nil
}
func (d *Driver) TaskStats(taskID string) (*drivers.TaskResourceUsage, error) {
func (d *Driver) TaskStats(ctx context.Context, taskID string, interval time.Duration) (<-chan *drivers.TaskResourceUsage, error) {
handle, ok := d.tasks.Get(taskID)
if !ok {
return nil, drivers.ErrTaskNotFound
}
return handle.exec.Stats()
return handle.exec.Stats(ctx, interval)
}
func (d *Driver) TaskEvents(ctx context.Context) (<-chan *drivers.TaskEvent, error) {

View File

@@ -494,18 +494,37 @@ func (d *Driver) InspectTask(taskID string) (*drivers.TaskStatus, error) {
panic("not implemented")
}
func (d *Driver) TaskStats(taskID string) (*drivers.TaskResourceUsage, error) {
// Generate random value for the memory usage
s := &drivers.TaskResourceUsage{
ResourceUsage: &drivers.ResourceUsage{
MemoryStats: &drivers.MemoryStats{
RSS: rand.Uint64(),
Measured: []string{"RSS"},
},
},
Timestamp: time.Now().UTC().UnixNano(),
func (d *Driver) TaskStats(ctx context.Context, taskID string, interval time.Duration) (<-chan *drivers.TaskResourceUsage, error) {
ch := make(chan *drivers.TaskResourceUsage)
go d.handleStats(ctx, ch)
return ch, nil
}
func (d *Driver) handleStats(ctx context.Context, ch chan<- *drivers.TaskResourceUsage) {
timer := time.NewTimer(0)
for {
select {
case <-timer.C:
// Generate random value for the memory usage
s := &drivers.TaskResourceUsage{
ResourceUsage: &drivers.ResourceUsage{
MemoryStats: &drivers.MemoryStats{
RSS: rand.Uint64(),
Measured: []string{"RSS"},
},
},
Timestamp: time.Now().UTC().UnixNano(),
}
select {
case <-ctx.Done():
return
case ch <- s:
default:
}
case <-ctx.Done():
return
}
}
return s, nil
}
func (d *Driver) TaskEvents(ctx context.Context) (<-chan *drivers.TaskEvent, error) {

View File

@@ -546,13 +546,13 @@ func (d *Driver) InspectTask(taskID string) (*drivers.TaskStatus, error) {
return handle.TaskStatus(), nil
}
func (d *Driver) TaskStats(taskID string) (*drivers.TaskResourceUsage, error) {
func (d *Driver) TaskStats(ctx context.Context, taskID string, interval time.Duration) (<-chan *drivers.TaskResourceUsage, error) {
handle, ok := d.tasks.Get(taskID)
if !ok {
return nil, drivers.ErrTaskNotFound
}
return handle.exec.Stats()
return handle.exec.Stats(ctx, interval)
}
func (d *Driver) TaskEvents(ctx context.Context) (<-chan *drivers.TaskEvent, error) {

View File

@@ -355,16 +355,19 @@ func TestQemuDriver_Stats(t *testing.T) {
// Wait until task started
require.NoError(harness.WaitUntilStarted(task.ID, 1*time.Second))
time.Sleep(30 * time.Second)
stats, err := harness.TaskStats(task.ID)
statsCtx, cancel := context.WithCancel(context.Background())
defer cancel()
statsCh, err := harness.TaskStats(statsCtx, task.ID, time.Second*10)
require.NoError(err)
// Ask for stats again
stats, err = harness.TaskStats(task.ID)
require.NoError(err)
fmt.Printf("CPU:%+v Memory:%+v\n", stats.ResourceUsage.CpuStats, stats.ResourceUsage.MemoryStats)
require.NotZero(stats.ResourceUsage.MemoryStats.RSS)
require.NoError(harness.DestroyTask(task.ID, true))
select {
case stats := <-statsCh:
fmt.Printf("CPU:%+v Memory:%+v\n", stats.ResourceUsage.CpuStats, stats.ResourceUsage.MemoryStats)
require.NotZero(stats.ResourceUsage.MemoryStats.RSS)
require.NoError(harness.DestroyTask(task.ID, true))
case <-time.After(time.Second * 1):
require.Fail("timeout receiving from stats")
}
}

View File

@@ -456,13 +456,13 @@ func (d *Driver) InspectTask(taskID string) (*drivers.TaskStatus, error) {
return handle.TaskStatus(), nil
}
func (d *Driver) TaskStats(taskID string) (*drivers.TaskResourceUsage, error) {
func (d *Driver) TaskStats(ctx context.Context, taskID string, interval time.Duration) (<-chan *drivers.TaskResourceUsage, error) {
handle, ok := d.tasks.Get(taskID)
if !ok {
return nil, drivers.ErrTaskNotFound
}
return handle.exec.Stats()
return handle.exec.Stats(ctx, interval)
}
func (d *Driver) TaskEvents(ctx context.Context) (<-chan *drivers.TaskEvent, error) {

View File

@@ -818,13 +818,13 @@ func (d *Driver) InspectTask(taskID string) (*drivers.TaskStatus, error) {
return handle.TaskStatus(), nil
}
func (d *Driver) TaskStats(taskID string) (*drivers.TaskResourceUsage, error) {
func (d *Driver) TaskStats(ctx context.Context, taskID string, interval time.Duration) (<-chan *drivers.TaskResourceUsage, error) {
handle, ok := d.tasks.Get(taskID)
if !ok {
return nil, drivers.ErrTaskNotFound
}
return handle.exec.Stats()
return handle.exec.Stats(ctx, interval)
}
func (d *Driver) TaskEvents(ctx context.Context) (<-chan *drivers.TaskEvent, error) {

View File

@@ -861,12 +861,19 @@ func TestRktDriver_Stats(t *testing.T) {
// Wait until task started
require.NoError(harness.WaitUntilStarted(task.ID, 1*time.Second))
resourceUsage, err := d.TaskStats(task.ID)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
statsCh, err := d.TaskStats(ctx, task.ID, time.Second*10)
require.Nil(err)
//TODO(preetha) why are these zero
fmt.Printf("pid map %v\n", resourceUsage.Pids)
fmt.Printf("CPU:%+v Memory:%+v", resourceUsage.ResourceUsage.CpuStats, resourceUsage.ResourceUsage.MemoryStats)
select {
case ru := <-statsCh:
//TODO(preetha) why are these zero
fmt.Printf("pid map %v\n", ru.Pids)
fmt.Printf("CPU:%+v Memory:%+v", ru.ResourceUsage.CpuStats, ru.ResourceUsage.MemoryStats)
case <-time.After(time.Second):
require.Fail("timeout receiving stats from channel")
}
require.NoError(harness.DestroyTask(task.ID, true))

View File

@@ -3,20 +3,25 @@ package executor
import (
"context"
"fmt"
"io"
"os"
"syscall"
"time"
"github.com/LK4D4/joincontext"
"github.com/golang/protobuf/ptypes"
hclog "github.com/hashicorp/go-hclog"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/drivers/shared/executor/proto"
"github.com/hashicorp/nomad/plugins/drivers"
"github.com/hashicorp/nomad/plugins/shared/grpcutils"
)
var _ Executor = (*grpcExecutorClient)(nil)
type grpcExecutorClient struct {
client proto.ExecutorClient
logger hclog.Logger
// doneCtx is close when the plugin exits
doneCtx context.Context
@@ -99,19 +104,54 @@ func (c *grpcExecutorClient) Version() (*ExecutorVersion, error) {
return &ExecutorVersion{Version: resp.Version}, nil
}
func (c *grpcExecutorClient) Stats() (*drivers.TaskResourceUsage, error) {
ctx := context.Background()
resp, err := c.client.Stats(ctx, &proto.StatsRequest{})
func (c *grpcExecutorClient) Stats(ctx context.Context, interval time.Duration) (<-chan *cstructs.TaskResourceUsage, error) {
stream, err := c.client.Stats(ctx, &proto.StatsRequest{})
if err != nil {
return nil, err
}
stats, err := drivers.TaskStatsFromProto(resp.Stats)
if err != nil {
return nil, err
}
return stats, nil
ch := make(chan *cstructs.TaskResourceUsage)
go c.handleStats(ctx, stream, ch)
return ch, nil
}
func (c *grpcExecutorClient) handleStats(ctx context.Context, stream proto.Executor_StatsClient, ch chan<- *cstructs.TaskResourceUsage) {
defer close(ch)
for {
resp, err := stream.Recv()
if err != nil {
if err != io.EOF {
c.logger.Error("error receiving stream from Stats executor RPC", "error", err)
truErr := &cstructs.TaskResourceUsage{
Err: grpcutils.HandleReqCtxGrpcErr(err, ctx, c.doneCtx),
}
select {
case ch <- truErr:
case <-ctx.Done():
}
}
// End stream
return
}
stats, err := drivers.TaskStatsFromProto(resp.Stats)
if err != nil {
truErr := &cstructs.TaskResourceUsage{
Err: fmt.Errorf("failed to decode stats from RPC: %v", err),
}
select {
case ch <- truErr:
case <-ctx.Done():
}
}
select {
case ch <- stats:
case <-ctx.Done():
return
}
}
}
func (c *grpcExecutorClient) Signal(s os.Signal) error {

View File

@@ -20,6 +20,7 @@ import (
"github.com/hashicorp/nomad/client/allocdir"
"github.com/hashicorp/nomad/client/lib/fifo"
"github.com/hashicorp/nomad/client/stats"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/plugins/drivers"
shelpers "github.com/hashicorp/nomad/helper/stats"
@@ -66,8 +67,9 @@ type Executor interface {
// Version returns the executor API version
Version() (*ExecutorVersion, error)
// Stats fetchs process usage stats for the executor and each pid if available
Stats() (*drivers.TaskResourceUsage, error)
// Returns a channel of stats. Stats are collected and
// pushed to the channel on the given interval
Stats(context.Context, time.Duration) (<-chan *cstructs.TaskResourceUsage, error)
// Signal sends the given signal to the user process
Signal(os.Signal) error
@@ -506,12 +508,36 @@ func (e *UniversalExecutor) Signal(s os.Signal) error {
return nil
}
func (e *UniversalExecutor) Stats() (*drivers.TaskResourceUsage, error) {
pidStats, err := e.pidCollector.pidStats()
if err != nil {
return nil, err
func (e *UniversalExecutor) Stats(ctx context.Context, interval time.Duration) (<-chan *cstructs.TaskResourceUsage, error) {
ch := make(chan *cstructs.TaskResourceUsage)
go e.handleStats(ch, ctx, interval)
return ch, nil
}
func (e *UniversalExecutor) handleStats(ch chan *cstructs.TaskResourceUsage, ctx context.Context, interval time.Duration) {
defer close(ch)
timer := time.NewTimer(0)
for {
select {
case <-ctx.Done():
return
case <-timer.C:
timer.Reset(interval)
}
pidStats, err := e.pidCollector.pidStats()
if err != nil {
e.logger.Warn("error collecting stats", "error", err)
return
}
select {
case <-ctx.Done():
return
case ch <- aggregatedResourceUsage(e.systemCpuStats, pidStats):
}
}
return aggregatedResourceUsage(e.systemCpuStats, pidStats), nil
}
// lookupBin looks for path to the binary to run by looking for the binary in

View File

@@ -18,6 +18,7 @@ import (
hclog "github.com/hashicorp/go-hclog"
multierror "github.com/hashicorp/go-multierror"
"github.com/hashicorp/nomad/client/stats"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/helper/discover"
shelpers "github.com/hashicorp/nomad/helper/stats"
"github.com/hashicorp/nomad/helper/uuid"
@@ -37,8 +38,10 @@ const (
)
var (
// The statistics the executor exposes when using cgroups
// ExecutorCgroupMeasuredMemStats is the list of memory stats captured by the executor
ExecutorCgroupMeasuredMemStats = []string{"RSS", "Cache", "Swap", "Max Usage", "Kernel Usage", "Kernel Max Usage"}
// ExecutorCgroupMeasuredCpuStats is the list of CPU stats captures by the executor
ExecutorCgroupMeasuredCpuStats = []string{"System Mode", "User Mode", "Throttled Periods", "Throttled Time", "Percent"}
// allCaps is all linux capabilities which is used to configure libcontainer
@@ -354,60 +357,86 @@ func (l *LibcontainerExecutor) Version() (*ExecutorVersion, error) {
}
// Stats returns the resource statistics for processes managed by the executor
func (l *LibcontainerExecutor) Stats() (*drivers.TaskResourceUsage, error) {
lstats, err := l.container.Stats()
if err != nil {
return nil, err
func (l *LibcontainerExecutor) Stats(ctx context.Context, interval time.Duration) (<-chan *cstructs.TaskResourceUsage, error) {
ch := make(chan *cstructs.TaskResourceUsage)
go l.handleStats(ch, ctx, interval)
return ch, nil
}
func (l *LibcontainerExecutor) handleStats(ch chan *cstructs.TaskResourceUsage, ctx context.Context, interval time.Duration) {
defer close(ch)
timer := time.NewTimer(0)
for {
select {
case <-ctx.Done():
return
case <-timer.C:
timer.Reset(interval)
}
lstats, err := l.container.Stats()
if err != nil {
l.logger.Warn("error collecting stats", "error", err)
return
}
pidStats, err := l.pidCollector.pidStats()
if err != nil {
l.logger.Warn("error collecting stats", "error", err)
return
}
ts := time.Now()
stats := lstats.CgroupStats
// Memory Related Stats
swap := stats.MemoryStats.SwapUsage
maxUsage := stats.MemoryStats.Usage.MaxUsage
rss := stats.MemoryStats.Stats["rss"]
cache := stats.MemoryStats.Stats["cache"]
ms := &cstructs.MemoryStats{
RSS: rss,
Cache: cache,
Swap: swap.Usage,
MaxUsage: maxUsage,
KernelUsage: stats.MemoryStats.KernelUsage.Usage,
KernelMaxUsage: stats.MemoryStats.KernelUsage.MaxUsage,
Measured: ExecutorCgroupMeasuredMemStats,
}
// CPU Related Stats
totalProcessCPUUsage := float64(stats.CpuStats.CpuUsage.TotalUsage)
userModeTime := float64(stats.CpuStats.CpuUsage.UsageInUsermode)
kernelModeTime := float64(stats.CpuStats.CpuUsage.UsageInKernelmode)
totalPercent := l.totalCpuStats.Percent(totalProcessCPUUsage)
cs := &cstructs.CpuStats{
SystemMode: l.systemCpuStats.Percent(kernelModeTime),
UserMode: l.userCpuStats.Percent(userModeTime),
Percent: totalPercent,
ThrottledPeriods: stats.CpuStats.ThrottlingData.ThrottledPeriods,
ThrottledTime: stats.CpuStats.ThrottlingData.ThrottledTime,
TotalTicks: l.systemCpuStats.TicksConsumed(totalPercent),
Measured: ExecutorCgroupMeasuredCpuStats,
}
taskResUsage := cstructs.TaskResourceUsage{
ResourceUsage: &cstructs.ResourceUsage{
MemoryStats: ms,
CpuStats: cs,
},
Timestamp: ts.UTC().UnixNano(),
Pids: pidStats,
}
select {
case <-ctx.Done():
return
case ch <- &taskResUsage:
}
}
pidStats, err := l.pidCollector.pidStats()
if err != nil {
return nil, err
}
ts := time.Now()
stats := lstats.CgroupStats
// Memory Related Stats
swap := stats.MemoryStats.SwapUsage
maxUsage := stats.MemoryStats.Usage.MaxUsage
rss := stats.MemoryStats.Stats["rss"]
cache := stats.MemoryStats.Stats["cache"]
ms := &drivers.MemoryStats{
RSS: rss,
Cache: cache,
Swap: swap.Usage,
MaxUsage: maxUsage,
KernelUsage: stats.MemoryStats.KernelUsage.Usage,
KernelMaxUsage: stats.MemoryStats.KernelUsage.MaxUsage,
Measured: ExecutorCgroupMeasuredMemStats,
}
// CPU Related Stats
totalProcessCPUUsage := float64(stats.CpuStats.CpuUsage.TotalUsage)
userModeTime := float64(stats.CpuStats.CpuUsage.UsageInUsermode)
kernelModeTime := float64(stats.CpuStats.CpuUsage.UsageInKernelmode)
totalPercent := l.totalCpuStats.Percent(totalProcessCPUUsage)
cs := &drivers.CpuStats{
SystemMode: l.systemCpuStats.Percent(kernelModeTime),
UserMode: l.userCpuStats.Percent(userModeTime),
Percent: totalPercent,
ThrottledPeriods: stats.CpuStats.ThrottlingData.ThrottledPeriods,
ThrottledTime: stats.CpuStats.ThrottlingData.ThrottledTime,
TotalTicks: l.systemCpuStats.TicksConsumed(totalPercent),
Measured: ExecutorCgroupMeasuredCpuStats,
}
taskResUsage := drivers.TaskResourceUsage{
ResourceUsage: &drivers.ResourceUsage{
MemoryStats: ms,
CpuStats: cs,
},
Timestamp: ts.UTC().UnixNano(),
Pids: pidStats,
}
return &taskResUsage, nil
}
// Signal sends a signal to the process managed by the executor

View File

@@ -29,5 +29,6 @@ func (p *ExecutorPlugin) GRPCClient(ctx context.Context, broker *plugin.GRPCBrok
return &grpcExecutorClient{
client: proto.NewExecutorClient(c),
doneCtx: ctx,
logger: p.logger,
}, nil
}

View File

@@ -20,6 +20,7 @@ import (
"github.com/hashicorp/nomad/nomad/mock"
"github.com/hashicorp/nomad/plugins/drivers"
tu "github.com/hashicorp/nomad/testutil"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
@@ -167,14 +168,20 @@ func TestExecutor_WaitExitSignal(pt *testing.T) {
require.NoError(err)
go func() {
time.Sleep(2 * time.Second)
_, err := executor.Stats()
require.NoError(err)
//require.NotEmpty(ru.Pids)
// Give process time to start
time.Sleep(time.Second)
ch, err := executor.Stats(context.Background(), time.Second)
assert.NoError(t, err)
select {
case <-time.After(time.Second):
assert.Fail(t, "stats failed to send on interval")
case ru := <-ch:
assert.NotEmpty(t, ru.Pids)
}
proc, err := os.FindProcess(ps.Pid)
require.NoError(err)
assert.NoError(t, err)
err = proc.Signal(syscall.SIGKILL)
require.NoError(err)
assert.NoError(t, err)
}()
ps, err = executor.Wait(context.Background())

View File

@@ -47,7 +47,7 @@ func (m *LaunchRequest) Reset() { *m = LaunchRequest{} }
func (m *LaunchRequest) String() string { return proto.CompactTextString(m) }
func (*LaunchRequest) ProtoMessage() {}
func (*LaunchRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{0}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{0}
}
func (m *LaunchRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_LaunchRequest.Unmarshal(m, b)
@@ -162,7 +162,7 @@ func (m *LaunchResponse) Reset() { *m = LaunchResponse{} }
func (m *LaunchResponse) String() string { return proto.CompactTextString(m) }
func (*LaunchResponse) ProtoMessage() {}
func (*LaunchResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{1}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{1}
}
func (m *LaunchResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_LaunchResponse.Unmarshal(m, b)
@@ -199,7 +199,7 @@ func (m *WaitRequest) Reset() { *m = WaitRequest{} }
func (m *WaitRequest) String() string { return proto.CompactTextString(m) }
func (*WaitRequest) ProtoMessage() {}
func (*WaitRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{2}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{2}
}
func (m *WaitRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_WaitRequest.Unmarshal(m, b)
@@ -230,7 +230,7 @@ func (m *WaitResponse) Reset() { *m = WaitResponse{} }
func (m *WaitResponse) String() string { return proto.CompactTextString(m) }
func (*WaitResponse) ProtoMessage() {}
func (*WaitResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{3}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{3}
}
func (m *WaitResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_WaitResponse.Unmarshal(m, b)
@@ -269,7 +269,7 @@ func (m *ShutdownRequest) Reset() { *m = ShutdownRequest{} }
func (m *ShutdownRequest) String() string { return proto.CompactTextString(m) }
func (*ShutdownRequest) ProtoMessage() {}
func (*ShutdownRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{4}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{4}
}
func (m *ShutdownRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ShutdownRequest.Unmarshal(m, b)
@@ -313,7 +313,7 @@ func (m *ShutdownResponse) Reset() { *m = ShutdownResponse{} }
func (m *ShutdownResponse) String() string { return proto.CompactTextString(m) }
func (*ShutdownResponse) ProtoMessage() {}
func (*ShutdownResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{5}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{5}
}
func (m *ShutdownResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ShutdownResponse.Unmarshal(m, b)
@@ -344,7 +344,7 @@ func (m *UpdateResourcesRequest) Reset() { *m = UpdateResourcesRequest{}
func (m *UpdateResourcesRequest) String() string { return proto.CompactTextString(m) }
func (*UpdateResourcesRequest) ProtoMessage() {}
func (*UpdateResourcesRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{6}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{6}
}
func (m *UpdateResourcesRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_UpdateResourcesRequest.Unmarshal(m, b)
@@ -381,7 +381,7 @@ func (m *UpdateResourcesResponse) Reset() { *m = UpdateResourcesResponse
func (m *UpdateResourcesResponse) String() string { return proto.CompactTextString(m) }
func (*UpdateResourcesResponse) ProtoMessage() {}
func (*UpdateResourcesResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{7}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{7}
}
func (m *UpdateResourcesResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_UpdateResourcesResponse.Unmarshal(m, b)
@@ -411,7 +411,7 @@ func (m *VersionRequest) Reset() { *m = VersionRequest{} }
func (m *VersionRequest) String() string { return proto.CompactTextString(m) }
func (*VersionRequest) ProtoMessage() {}
func (*VersionRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{8}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{8}
}
func (m *VersionRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_VersionRequest.Unmarshal(m, b)
@@ -442,7 +442,7 @@ func (m *VersionResponse) Reset() { *m = VersionResponse{} }
func (m *VersionResponse) String() string { return proto.CompactTextString(m) }
func (*VersionResponse) ProtoMessage() {}
func (*VersionResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{9}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{9}
}
func (m *VersionResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_VersionResponse.Unmarshal(m, b)
@@ -470,6 +470,7 @@ func (m *VersionResponse) GetVersion() string {
}
type StatsRequest struct {
Interval int64 `protobuf:"varint,1,opt,name=interval,proto3" json:"interval,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@@ -479,7 +480,7 @@ func (m *StatsRequest) Reset() { *m = StatsRequest{} }
func (m *StatsRequest) String() string { return proto.CompactTextString(m) }
func (*StatsRequest) ProtoMessage() {}
func (*StatsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{10}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{10}
}
func (m *StatsRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StatsRequest.Unmarshal(m, b)
@@ -499,6 +500,13 @@ func (m *StatsRequest) XXX_DiscardUnknown() {
var xxx_messageInfo_StatsRequest proto.InternalMessageInfo
func (m *StatsRequest) GetInterval() int64 {
if m != nil {
return m.Interval
}
return 0
}
type StatsResponse struct {
Stats *proto1.TaskStats `protobuf:"bytes,1,opt,name=stats,proto3" json:"stats,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
@@ -510,7 +518,7 @@ func (m *StatsResponse) Reset() { *m = StatsResponse{} }
func (m *StatsResponse) String() string { return proto.CompactTextString(m) }
func (*StatsResponse) ProtoMessage() {}
func (*StatsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{11}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{11}
}
func (m *StatsResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StatsResponse.Unmarshal(m, b)
@@ -548,7 +556,7 @@ func (m *SignalRequest) Reset() { *m = SignalRequest{} }
func (m *SignalRequest) String() string { return proto.CompactTextString(m) }
func (*SignalRequest) ProtoMessage() {}
func (*SignalRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{12}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{12}
}
func (m *SignalRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SignalRequest.Unmarshal(m, b)
@@ -585,7 +593,7 @@ func (m *SignalResponse) Reset() { *m = SignalResponse{} }
func (m *SignalResponse) String() string { return proto.CompactTextString(m) }
func (*SignalResponse) ProtoMessage() {}
func (*SignalResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{13}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{13}
}
func (m *SignalResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SignalResponse.Unmarshal(m, b)
@@ -618,7 +626,7 @@ func (m *ExecRequest) Reset() { *m = ExecRequest{} }
func (m *ExecRequest) String() string { return proto.CompactTextString(m) }
func (*ExecRequest) ProtoMessage() {}
func (*ExecRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{14}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{14}
}
func (m *ExecRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExecRequest.Unmarshal(m, b)
@@ -671,7 +679,7 @@ func (m *ExecResponse) Reset() { *m = ExecResponse{} }
func (m *ExecResponse) String() string { return proto.CompactTextString(m) }
func (*ExecResponse) ProtoMessage() {}
func (*ExecResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{15}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{15}
}
func (m *ExecResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExecResponse.Unmarshal(m, b)
@@ -719,7 +727,7 @@ func (m *ProcessState) Reset() { *m = ProcessState{} }
func (m *ProcessState) String() string { return proto.CompactTextString(m) }
func (*ProcessState) ProtoMessage() {}
func (*ProcessState) Descriptor() ([]byte, []int) {
return fileDescriptor_executor_6afb14e0d1b270fe, []int{16}
return fileDescriptor_executor_1eb9aa6040002cd3, []int{16}
}
func (m *ProcessState) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProcessState.Unmarshal(m, b)
@@ -804,7 +812,7 @@ type ExecutorClient interface {
Shutdown(ctx context.Context, in *ShutdownRequest, opts ...grpc.CallOption) (*ShutdownResponse, error)
UpdateResources(ctx context.Context, in *UpdateResourcesRequest, opts ...grpc.CallOption) (*UpdateResourcesResponse, error)
Version(ctx context.Context, in *VersionRequest, opts ...grpc.CallOption) (*VersionResponse, error)
Stats(ctx context.Context, in *StatsRequest, opts ...grpc.CallOption) (*StatsResponse, error)
Stats(ctx context.Context, in *StatsRequest, opts ...grpc.CallOption) (Executor_StatsClient, error)
Signal(ctx context.Context, in *SignalRequest, opts ...grpc.CallOption) (*SignalResponse, error)
Exec(ctx context.Context, in *ExecRequest, opts ...grpc.CallOption) (*ExecResponse, error)
}
@@ -862,13 +870,36 @@ func (c *executorClient) Version(ctx context.Context, in *VersionRequest, opts .
return out, nil
}
func (c *executorClient) Stats(ctx context.Context, in *StatsRequest, opts ...grpc.CallOption) (*StatsResponse, error) {
out := new(StatsResponse)
err := c.cc.Invoke(ctx, "/hashicorp.nomad.plugins.executor.proto.Executor/Stats", in, out, opts...)
func (c *executorClient) Stats(ctx context.Context, in *StatsRequest, opts ...grpc.CallOption) (Executor_StatsClient, error) {
stream, err := c.cc.NewStream(ctx, &_Executor_serviceDesc.Streams[0], "/hashicorp.nomad.plugins.executor.proto.Executor/Stats", opts...)
if err != nil {
return nil, err
}
return out, nil
x := &executorStatsClient{stream}
if err := x.ClientStream.SendMsg(in); err != nil {
return nil, err
}
if err := x.ClientStream.CloseSend(); err != nil {
return nil, err
}
return x, nil
}
type Executor_StatsClient interface {
Recv() (*StatsResponse, error)
grpc.ClientStream
}
type executorStatsClient struct {
grpc.ClientStream
}
func (x *executorStatsClient) Recv() (*StatsResponse, error) {
m := new(StatsResponse)
if err := x.ClientStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
func (c *executorClient) Signal(ctx context.Context, in *SignalRequest, opts ...grpc.CallOption) (*SignalResponse, error) {
@@ -896,7 +927,7 @@ type ExecutorServer interface {
Shutdown(context.Context, *ShutdownRequest) (*ShutdownResponse, error)
UpdateResources(context.Context, *UpdateResourcesRequest) (*UpdateResourcesResponse, error)
Version(context.Context, *VersionRequest) (*VersionResponse, error)
Stats(context.Context, *StatsRequest) (*StatsResponse, error)
Stats(*StatsRequest, Executor_StatsServer) error
Signal(context.Context, *SignalRequest) (*SignalResponse, error)
Exec(context.Context, *ExecRequest) (*ExecResponse, error)
}
@@ -995,22 +1026,25 @@ func _Executor_Version_Handler(srv interface{}, ctx context.Context, dec func(in
return interceptor(ctx, in, info, handler)
}
func _Executor_Stats_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(StatsRequest)
if err := dec(in); err != nil {
return nil, err
func _Executor_Stats_Handler(srv interface{}, stream grpc.ServerStream) error {
m := new(StatsRequest)
if err := stream.RecvMsg(m); err != nil {
return err
}
if interceptor == nil {
return srv.(ExecutorServer).Stats(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/hashicorp.nomad.plugins.executor.proto.Executor/Stats",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(ExecutorServer).Stats(ctx, req.(*StatsRequest))
}
return interceptor(ctx, in, info, handler)
return srv.(ExecutorServer).Stats(m, &executorStatsServer{stream})
}
type Executor_StatsServer interface {
Send(*StatsResponse) error
grpc.ServerStream
}
type executorStatsServer struct {
grpc.ServerStream
}
func (x *executorStatsServer) Send(m *StatsResponse) error {
return x.ServerStream.SendMsg(m)
}
func _Executor_Signal_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
@@ -1073,10 +1107,6 @@ var _Executor_serviceDesc = grpc.ServiceDesc{
MethodName: "Version",
Handler: _Executor_Version_Handler,
},
{
MethodName: "Stats",
Handler: _Executor_Stats_Handler,
},
{
MethodName: "Signal",
Handler: _Executor_Signal_Handler,
@@ -1086,69 +1116,76 @@ var _Executor_serviceDesc = grpc.ServiceDesc{
Handler: _Executor_Exec_Handler,
},
},
Streams: []grpc.StreamDesc{},
Streams: []grpc.StreamDesc{
{
StreamName: "Stats",
Handler: _Executor_Stats_Handler,
ServerStreams: true,
},
},
Metadata: "drivers/shared/executor/proto/executor.proto",
}
func init() {
proto.RegisterFile("drivers/shared/executor/proto/executor.proto", fileDescriptor_executor_6afb14e0d1b270fe)
proto.RegisterFile("drivers/shared/executor/proto/executor.proto", fileDescriptor_executor_1eb9aa6040002cd3)
}
var fileDescriptor_executor_6afb14e0d1b270fe = []byte{
// 871 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x55, 0x4b, 0x8f, 0xdc, 0x44,
0x10, 0xce, 0xac, 0x77, 0x5e, 0xe5, 0xd9, 0x87, 0x5a, 0x68, 0x71, 0xcc, 0x21, 0x83, 0x0f, 0x64,
0x24, 0x82, 0x1d, 0x6d, 0x5e, 0x5c, 0x00, 0x89, 0xdd, 0xc0, 0x65, 0x89, 0x56, 0xde, 0x40, 0x24,
0x0e, 0x0c, 0xbd, 0x76, 0x63, 0xb7, 0x76, 0xc6, 0x6d, 0xba, 0xdb, 0xc3, 0x46, 0x42, 0xe2, 0xc4,
0x3f, 0xe0, 0x97, 0x72, 0xe2, 0x88, 0xfa, 0xe5, 0x9d, 0xd9, 0x84, 0xc8, 0x03, 0xe2, 0x34, 0x5d,
0xe5, 0xfa, 0xbe, 0xaa, 0xea, 0xae, 0xfa, 0x06, 0x1e, 0xe4, 0x9c, 0xae, 0x08, 0x17, 0x89, 0x28,
0x31, 0x27, 0x79, 0x42, 0xae, 0x49, 0xd6, 0x48, 0xc6, 0x93, 0x9a, 0x33, 0xc9, 0x5a, 0x33, 0xd6,
0x26, 0xfa, 0xa8, 0xc4, 0xa2, 0xa4, 0x19, 0xe3, 0x75, 0x5c, 0xb1, 0x25, 0xce, 0xe3, 0x7a, 0xd1,
0x14, 0xb4, 0x12, 0xf1, 0x66, 0x5c, 0x78, 0xaf, 0x60, 0xac, 0x58, 0x10, 0x43, 0x72, 0xd9, 0xfc,
0x94, 0x48, 0xba, 0x24, 0x42, 0xe2, 0x65, 0x6d, 0x03, 0x3e, 0x2b, 0xa8, 0x2c, 0x9b, 0xcb, 0x38,
0x63, 0xcb, 0xa4, 0xe5, 0x4c, 0x34, 0x67, 0x62, 0x39, 0x13, 0x57, 0x99, 0xa9, 0xc4, 0x58, 0x06,
0x1e, 0xfd, 0xe9, 0xc1, 0xde, 0x19, 0x6e, 0xaa, 0xac, 0x4c, 0xc9, 0xcf, 0x0d, 0x11, 0x12, 0x1d,
0x82, 0x97, 0x2d, 0xf3, 0xa0, 0x37, 0xed, 0xcd, 0xc6, 0xa9, 0x3a, 0x22, 0x04, 0xbb, 0x98, 0x17,
0x22, 0xd8, 0x99, 0x7a, 0xb3, 0x71, 0xaa, 0xcf, 0xe8, 0x05, 0x8c, 0x39, 0x11, 0xac, 0xe1, 0x19,
0x11, 0x81, 0x37, 0xed, 0xcd, 0xfc, 0xe3, 0x87, 0xf1, 0x3f, 0xf5, 0x64, 0xf3, 0x9b, 0x94, 0x71,
0xea, 0x70, 0xe9, 0x0d, 0x05, 0xba, 0x07, 0xbe, 0x90, 0x39, 0x6b, 0xe4, 0xbc, 0xc6, 0xb2, 0x0c,
0x76, 0x75, 0x76, 0x30, 0xae, 0x73, 0x2c, 0x4b, 0x1b, 0x40, 0x38, 0x37, 0x01, 0xfd, 0x36, 0x80,
0x70, 0xae, 0x03, 0x0e, 0xc1, 0x23, 0xd5, 0x2a, 0x18, 0xe8, 0x22, 0xd5, 0x51, 0xd5, 0xdd, 0x08,
0xc2, 0x83, 0xa1, 0x8e, 0xd5, 0x67, 0x74, 0x17, 0x46, 0x12, 0x8b, 0xab, 0x79, 0x4e, 0x79, 0x30,
0xd2, 0xfe, 0xa1, 0xb2, 0x4f, 0x29, 0x47, 0xf7, 0xe1, 0xc0, 0xd5, 0x33, 0x5f, 0xd0, 0x25, 0x95,
0x22, 0x18, 0x4f, 0x7b, 0xb3, 0x51, 0xba, 0xef, 0xdc, 0x67, 0xda, 0x8b, 0x1e, 0xc2, 0x7b, 0x97,
0x58, 0xd0, 0x6c, 0x5e, 0x73, 0x96, 0x11, 0x21, 0xe6, 0x59, 0xc1, 0x59, 0x53, 0x07, 0xa0, 0xa3,
0x91, 0xfe, 0x76, 0x6e, 0x3e, 0x9d, 0xe8, 0x2f, 0xe8, 0x14, 0x06, 0x4b, 0xd6, 0x54, 0x52, 0x04,
0xfe, 0xd4, 0x9b, 0xf9, 0xc7, 0x0f, 0x3a, 0x5e, 0xd5, 0x37, 0x0a, 0x94, 0x5a, 0x2c, 0xfa, 0x1a,
0x86, 0x39, 0x59, 0x51, 0x75, 0xe3, 0x13, 0x4d, 0xf3, 0x49, 0x47, 0x9a, 0x53, 0x8d, 0x4a, 0x1d,
0x3a, 0xfa, 0x11, 0xf6, 0xdd, 0x9b, 0x8b, 0x9a, 0x55, 0x82, 0xa0, 0x17, 0x30, 0xb4, 0xcd, 0xe8,
0x87, 0xf7, 0x8f, 0x1f, 0xc7, 0xdd, 0x06, 0x34, 0xb6, 0x8d, 0x5e, 0x48, 0x2c, 0x49, 0xea, 0x48,
0xa2, 0x3d, 0xf0, 0x5f, 0x61, 0x2a, 0xed, 0x4c, 0x45, 0x3f, 0xc0, 0xc4, 0x98, 0xff, 0x53, 0xba,
0x33, 0x38, 0xb8, 0x28, 0x1b, 0x99, 0xb3, 0x5f, 0x2a, 0x37, 0xc6, 0x47, 0x30, 0x10, 0xb4, 0xa8,
0xf0, 0xc2, 0x4e, 0xb2, 0xb5, 0xd0, 0x87, 0x30, 0x29, 0x38, 0xce, 0xc8, 0xbc, 0x26, 0x9c, 0xb2,
0x3c, 0xd8, 0x99, 0xf6, 0x66, 0x5e, 0xea, 0x6b, 0xdf, 0xb9, 0x76, 0x45, 0x08, 0x0e, 0x6f, 0xd8,
0x4c, 0xc5, 0x51, 0x09, 0x47, 0xdf, 0xd6, 0xb9, 0x4a, 0xda, 0x4e, 0xaf, 0x4d, 0xb4, 0xb1, 0x09,
0xbd, 0xff, 0xbc, 0x09, 0xd1, 0x5d, 0x78, 0xff, 0x8d, 0x4c, 0xb6, 0x88, 0x43, 0xd8, 0xff, 0x8e,
0x70, 0x41, 0x99, 0xeb, 0x32, 0xfa, 0x18, 0x0e, 0x5a, 0x8f, 0xbd, 0xdb, 0x00, 0x86, 0x2b, 0xe3,
0xb2, 0x9d, 0x3b, 0x33, 0xda, 0x87, 0x89, 0xba, 0x37, 0x57, 0x79, 0xf4, 0x0a, 0xf6, 0xac, 0x6d,
0xa1, 0x5f, 0x41, 0x5f, 0x28, 0xc7, 0x96, 0x6d, 0xbc, 0xc4, 0xe2, 0xca, 0x10, 0x19, 0x78, 0x74,
0x1f, 0xf6, 0x2e, 0xf4, 0x6d, 0xbf, 0xfd, 0x31, 0xfa, 0xee, 0x31, 0x54, 0x43, 0x2e, 0xd0, 0xb6,
0x78, 0x05, 0xfe, 0xf3, 0x6b, 0x92, 0x39, 0xe0, 0x53, 0x18, 0xe5, 0x04, 0xe7, 0x0b, 0x5a, 0x11,
0x5b, 0x54, 0x18, 0x1b, 0x45, 0x8c, 0x9d, 0x22, 0xc6, 0x2f, 0x9d, 0x22, 0xa6, 0x6d, 0xac, 0x13,
0xb1, 0x9d, 0x37, 0x45, 0xcc, 0xbb, 0x11, 0xb1, 0xe8, 0x04, 0x26, 0x26, 0x99, 0xed, 0xff, 0x08,
0x06, 0xac, 0x91, 0x75, 0x23, 0x75, 0xae, 0x49, 0x6a, 0x2d, 0xf4, 0x01, 0x8c, 0xc9, 0x35, 0x95,
0xf3, 0x8c, 0xe5, 0x44, 0x73, 0xf6, 0xd3, 0x91, 0x72, 0x9c, 0xb0, 0x9c, 0x44, 0xbf, 0xf7, 0x60,
0xb2, 0x3e, 0x95, 0x2a, 0x77, 0x4d, 0x73, 0xdb, 0xa9, 0x3a, 0xbe, 0x13, 0xbf, 0x76, 0x37, 0xde,
0xfa, 0xdd, 0xa0, 0x18, 0x76, 0x95, 0xd6, 0x6b, 0x29, 0x7c, 0x77, 0xdb, 0x3a, 0xee, 0xf8, 0xaf,
0x21, 0x8c, 0x9e, 0xdb, 0x65, 0x41, 0xaf, 0x61, 0x60, 0x36, 0x1c, 0x3d, 0xe9, 0xba, 0x59, 0x1b,
0xff, 0x02, 0xe1, 0xd3, 0x6d, 0x61, 0xf6, 0xfd, 0xee, 0x20, 0x01, 0xbb, 0x6a, 0xd7, 0xd1, 0xa3,
0xae, 0x0c, 0x6b, 0x42, 0x11, 0x3e, 0xde, 0x0e, 0xd4, 0x26, 0xfd, 0x0d, 0x46, 0x6e, 0x65, 0xd1,
0xb3, 0xae, 0x1c, 0xb7, 0x24, 0x23, 0xfc, 0x74, 0x7b, 0x60, 0x5b, 0xc0, 0x1f, 0x3d, 0x38, 0xb8,
0xb5, 0xb6, 0xe8, 0xf3, 0xae, 0x7c, 0x6f, 0x57, 0x96, 0xf0, 0x8b, 0x7f, 0x8d, 0x6f, 0xcb, 0xfa,
0x15, 0x86, 0x56, 0x1f, 0x50, 0xe7, 0x17, 0xdd, 0x94, 0x98, 0xf0, 0xd9, 0xd6, 0xb8, 0x36, 0xfb,
0x0a, 0xfa, 0x5a, 0x17, 0x50, 0xe7, 0x67, 0x5d, 0xd7, 0xa7, 0xf0, 0xc9, 0x96, 0xa8, 0x36, 0xef,
0x6b, 0x18, 0x18, 0x59, 0xe9, 0x3e, 0xfd, 0x1b, 0x7a, 0xd5, 0x7d, 0xfa, 0x6f, 0xa9, 0x97, 0x9e,
0x7e, 0xb5, 0x84, 0xdd, 0xa7, 0x7f, 0x4d, 0xed, 0xba, 0x4f, 0xff, 0xba, 0x6a, 0x45, 0x77, 0xbe,
0x1c, 0x7e, 0xdf, 0x37, 0xb2, 0x30, 0xd0, 0x3f, 0x8f, 0xfe, 0x0e, 0x00, 0x00, 0xff, 0xff, 0x9f,
0x81, 0x07, 0x03, 0x8c, 0x0a, 0x00, 0x00,
var fileDescriptor_executor_1eb9aa6040002cd3 = []byte{
// 885 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x55, 0x4b, 0x6f, 0xe4, 0x44,
0x10, 0x5e, 0xc7, 0x99, 0x57, 0xcd, 0xe4, 0xa1, 0x16, 0x0a, 0x5e, 0x73, 0xd8, 0xc1, 0x07, 0x76,
0x04, 0x8b, 0x27, 0xca, 0xbe, 0xb8, 0x00, 0x12, 0xc9, 0xc2, 0x25, 0xac, 0x22, 0x67, 0x61, 0x25,
0x0e, 0x0c, 0x1d, 0xbb, 0xb1, 0x5b, 0x99, 0x71, 0x9b, 0xee, 0xf6, 0x30, 0x48, 0x48, 0x9c, 0xf8,
0x07, 0xfc, 0x52, 0x8e, 0x9c, 0x50, 0xbf, 0x9c, 0x99, 0xec, 0x12, 0x79, 0x40, 0x9c, 0xa6, 0xab,
0x5c, 0xdf, 0x57, 0x55, 0xdd, 0x55, 0xdf, 0xc0, 0xa3, 0x8c, 0xd3, 0x25, 0xe1, 0x62, 0x2a, 0x0a,
0xcc, 0x49, 0x36, 0x25, 0x2b, 0x92, 0xd6, 0x92, 0xf1, 0x69, 0xc5, 0x99, 0x64, 0x8d, 0x19, 0x6b,
0x13, 0x7d, 0x50, 0x60, 0x51, 0xd0, 0x94, 0xf1, 0x2a, 0x2e, 0xd9, 0x02, 0x67, 0x71, 0x35, 0xaf,
0x73, 0x5a, 0x8a, 0x78, 0x33, 0x2e, 0x7c, 0x90, 0x33, 0x96, 0xcf, 0x89, 0x21, 0xb9, 0xaa, 0x7f,
0x9c, 0x4a, 0xba, 0x20, 0x42, 0xe2, 0x45, 0x65, 0x03, 0x3e, 0xcd, 0xa9, 0x2c, 0xea, 0xab, 0x38,
0x65, 0x8b, 0x69, 0xc3, 0x39, 0xd5, 0x9c, 0x53, 0xcb, 0x39, 0x75, 0x95, 0x99, 0x4a, 0x8c, 0x65,
0xe0, 0xd1, 0x9f, 0x3e, 0xec, 0x9d, 0xe3, 0xba, 0x4c, 0x8b, 0x84, 0xfc, 0x54, 0x13, 0x21, 0xd1,
0x21, 0xf8, 0xe9, 0x22, 0x0b, 0xbc, 0xb1, 0x37, 0x19, 0x24, 0xea, 0x88, 0x10, 0xec, 0x62, 0x9e,
0x8b, 0x60, 0x67, 0xec, 0x4f, 0x06, 0x89, 0x3e, 0xa3, 0x97, 0x30, 0xe0, 0x44, 0xb0, 0x9a, 0xa7,
0x44, 0x04, 0xfe, 0xd8, 0x9b, 0x0c, 0x4f, 0x8e, 0xe3, 0x7f, 0xea, 0xc9, 0xe6, 0x37, 0x29, 0xe3,
0xc4, 0xe1, 0x92, 0x1b, 0x0a, 0xf4, 0x00, 0x86, 0x42, 0x66, 0xac, 0x96, 0xb3, 0x0a, 0xcb, 0x22,
0xd8, 0xd5, 0xd9, 0xc1, 0xb8, 0x2e, 0xb0, 0x2c, 0x6c, 0x00, 0xe1, 0xdc, 0x04, 0x74, 0x9a, 0x00,
0xc2, 0xb9, 0x0e, 0x38, 0x04, 0x9f, 0x94, 0xcb, 0xa0, 0xab, 0x8b, 0x54, 0x47, 0x55, 0x77, 0x2d,
0x08, 0x0f, 0x7a, 0x3a, 0x56, 0x9f, 0xd1, 0x7d, 0xe8, 0x4b, 0x2c, 0xae, 0x67, 0x19, 0xe5, 0x41,
0x5f, 0xfb, 0x7b, 0xca, 0x3e, 0xa3, 0x1c, 0x3d, 0x84, 0x03, 0x57, 0xcf, 0x6c, 0x4e, 0x17, 0x54,
0x8a, 0x60, 0x30, 0xf6, 0x26, 0xfd, 0x64, 0xdf, 0xb9, 0xcf, 0xb5, 0x17, 0x1d, 0xc3, 0x3b, 0x57,
0x58, 0xd0, 0x74, 0x56, 0x71, 0x96, 0x12, 0x21, 0x66, 0x69, 0xce, 0x59, 0x5d, 0x05, 0xa0, 0xa3,
0x91, 0xfe, 0x76, 0x61, 0x3e, 0x9d, 0xea, 0x2f, 0xe8, 0x0c, 0xba, 0x0b, 0x56, 0x97, 0x52, 0x04,
0xc3, 0xb1, 0x3f, 0x19, 0x9e, 0x3c, 0x6a, 0x79, 0x55, 0x5f, 0x2b, 0x50, 0x62, 0xb1, 0xe8, 0x2b,
0xe8, 0x65, 0x64, 0x49, 0xd5, 0x8d, 0x8f, 0x34, 0xcd, 0xc7, 0x2d, 0x69, 0xce, 0x34, 0x2a, 0x71,
0xe8, 0xe8, 0x07, 0xd8, 0x77, 0x6f, 0x2e, 0x2a, 0x56, 0x0a, 0x82, 0x5e, 0x42, 0xcf, 0x36, 0xa3,
0x1f, 0x7e, 0x78, 0xf2, 0x24, 0x6e, 0x37, 0xa0, 0xb1, 0x6d, 0xf4, 0x52, 0x62, 0x49, 0x12, 0x47,
0x12, 0xed, 0xc1, 0xf0, 0x35, 0xa6, 0xd2, 0xce, 0x54, 0xf4, 0x3d, 0x8c, 0x8c, 0xf9, 0x3f, 0xa5,
0x3b, 0x87, 0x83, 0xcb, 0xa2, 0x96, 0x19, 0xfb, 0xb9, 0x74, 0x63, 0x7c, 0x04, 0x5d, 0x41, 0xf3,
0x12, 0xcf, 0xed, 0x24, 0x5b, 0x0b, 0xbd, 0x0f, 0xa3, 0x9c, 0xe3, 0x94, 0xcc, 0x2a, 0xc2, 0x29,
0xcb, 0x82, 0x9d, 0xb1, 0x37, 0xf1, 0x93, 0xa1, 0xf6, 0x5d, 0x68, 0x57, 0x84, 0xe0, 0xf0, 0x86,
0xcd, 0x54, 0x1c, 0x15, 0x70, 0xf4, 0x4d, 0x95, 0xa9, 0xa4, 0xcd, 0xf4, 0xda, 0x44, 0x1b, 0x9b,
0xe0, 0xfd, 0xe7, 0x4d, 0x88, 0xee, 0xc3, 0xbb, 0x6f, 0x64, 0xb2, 0x45, 0x1c, 0xc2, 0xfe, 0xb7,
0x84, 0x0b, 0xca, 0x5c, 0x97, 0xd1, 0x47, 0x70, 0xd0, 0x78, 0xec, 0xdd, 0x06, 0xd0, 0x5b, 0x1a,
0x97, 0xed, 0xdc, 0x99, 0xd1, 0x87, 0x30, 0x52, 0xf7, 0xd6, 0x54, 0x1e, 0x42, 0x9f, 0x96, 0x92,
0xf0, 0xa5, 0xbd, 0x24, 0x3f, 0x69, 0xec, 0xe8, 0x35, 0xec, 0xd9, 0x58, 0x4b, 0xfb, 0x25, 0x74,
0x84, 0x72, 0x6c, 0xd9, 0xe2, 0x2b, 0x2c, 0xae, 0x0d, 0x91, 0x81, 0x47, 0x0f, 0x61, 0xef, 0x52,
0xbf, 0xc4, 0xdb, 0x1f, 0xaa, 0xe3, 0x1e, 0x4a, 0x35, 0xeb, 0x02, 0x6d, 0xfb, 0xd7, 0x30, 0x7c,
0xb1, 0x22, 0xa9, 0x03, 0x3e, 0x83, 0x7e, 0x46, 0x70, 0x36, 0xa7, 0x25, 0xb1, 0x45, 0x85, 0xb1,
0x51, 0xcb, 0xd8, 0xa9, 0x65, 0xfc, 0xca, 0xa9, 0x65, 0xd2, 0xc4, 0x3a, 0x81, 0xdb, 0x79, 0x53,
0xe0, 0xfc, 0x1b, 0x81, 0x8b, 0x4e, 0x61, 0x64, 0x92, 0xd9, 0xfe, 0x8f, 0xa0, 0xcb, 0x6a, 0x59,
0xd5, 0x52, 0xe7, 0x1a, 0x25, 0xd6, 0x42, 0xef, 0xc1, 0x80, 0xac, 0xa8, 0x9c, 0xa5, 0x2c, 0x23,
0x9a, 0xb3, 0x93, 0xf4, 0x95, 0xe3, 0x94, 0x65, 0x24, 0xfa, 0xdd, 0x83, 0xd1, 0xfa, 0xc4, 0xaa,
0xdc, 0x15, 0xcd, 0x6c, 0xa7, 0xea, 0x78, 0x27, 0x7e, 0xed, 0x6e, 0xfc, 0xf5, 0xbb, 0x41, 0x31,
0xec, 0xaa, 0xff, 0x01, 0x2d, 0x93, 0x77, 0xb7, 0xad, 0xe3, 0x4e, 0xfe, 0xea, 0x41, 0xff, 0x85,
0x5d, 0x24, 0xf4, 0x0b, 0x74, 0xcd, 0xf6, 0xa3, 0xa7, 0x6d, 0xb7, 0x6e, 0xe3, 0x1f, 0x22, 0x7c,
0xb6, 0x2d, 0xcc, 0xbe, 0xdf, 0x3d, 0x24, 0x60, 0x57, 0xe9, 0x00, 0x7a, 0xdc, 0x96, 0x61, 0x4d,
0x44, 0xc2, 0x27, 0xdb, 0x81, 0x9a, 0xa4, 0xbf, 0x41, 0xdf, 0xad, 0x33, 0x7a, 0xde, 0x96, 0xe3,
0x96, 0x9c, 0x84, 0x9f, 0x6c, 0x0f, 0x6c, 0x0a, 0xf8, 0xc3, 0x83, 0x83, 0x5b, 0x2b, 0x8d, 0x3e,
0x6b, 0xcb, 0xf7, 0x76, 0xd5, 0x09, 0x3f, 0xff, 0xd7, 0xf8, 0xa6, 0xac, 0x5f, 0xa1, 0x67, 0xb5,
0x03, 0xb5, 0x7e, 0xd1, 0x4d, 0xf9, 0x09, 0x9f, 0x6f, 0x8d, 0x6b, 0xb2, 0xaf, 0xa0, 0xa3, 0x75,
0x01, 0xb5, 0x7e, 0xd6, 0x75, 0xed, 0x0a, 0x9f, 0x6e, 0x89, 0x72, 0x79, 0x8f, 0x3d, 0x35, 0xff,
0x46, 0x58, 0xda, 0xcf, 0xff, 0x86, 0x62, 0xb5, 0x9f, 0xff, 0x5b, 0xfa, 0xa5, 0xe7, 0x5f, 0xad,
0x61, 0xfb, 0xf9, 0x5f, 0xd3, 0xbb, 0xf6, 0xf3, 0xbf, 0xae, 0x5b, 0xd1, 0xbd, 0x2f, 0x7a, 0xdf,
0x75, 0x8c, 0x30, 0x74, 0xf5, 0xcf, 0xe3, 0xbf, 0x03, 0x00, 0x00, 0xff, 0xff, 0x2f, 0x08, 0xfe,
0x1f, 0xaa, 0x0a, 0x00, 0x00,
}

View File

@@ -11,7 +11,7 @@ service Executor {
rpc Shutdown(ShutdownRequest) returns (ShutdownResponse) {}
rpc UpdateResources(UpdateResourcesRequest) returns (UpdateResourcesResponse) {}
rpc Version(VersionRequest) returns (VersionResponse) {}
rpc Stats(StatsRequest) returns (StatsResponse) {}
rpc Stats(StatsRequest) returns (stream StatsResponse) {}
rpc Signal(SignalRequest) returns (SignalResponse) {}
rpc Exec(ExecRequest) returns (ExecResponse) {}
}
@@ -60,7 +60,9 @@ message VersionResponse{
string version = 1;
}
message StatsRequest {}
message StatsRequest {
int64 interval = 1;
}
message StatsResponse {
hashicorp.nomad.plugins.drivers.proto.TaskStats stats = 1;

View File

@@ -87,20 +87,46 @@ func (s *grpcExecutorServer) Version(context.Context, *proto.VersionRequest) (*p
}, nil
}
func (s *grpcExecutorServer) Stats(context.Context, *proto.StatsRequest) (*proto.StatsResponse, error) {
stats, err := s.impl.Stats()
if err != nil {
return nil, err
func (s *grpcExecutorServer) Stats(req *proto.StatsRequest, stream proto.Executor_StatsServer) error {
ctx := stream.Context()
interval := time.Duration(req.Interval)
if interval.Nanoseconds() == 0 {
interval = time.Second
}
pbStats, err := drivers.TaskStatsToProto(stats)
outCh, err := s.impl.Stats(stream.Context(), time.Duration(req.Interval))
if err != nil {
return nil, err
return err
}
return &proto.StatsResponse{
Stats: pbStats,
}, nil
for {
select {
case <-ctx.Done():
return nil
case resp, ok := <-outCh:
// chan closed, end stream
if !ok {
return nil
}
pbStats, err := drivers.TaskStatsToProto(resp)
if err != nil {
return err
}
presp := &proto.StatsResponse{
Stats: pbStats,
}
// Send the stats
if err := stream.Send(presp); err != nil {
return err
}
}
}
}
func (s *grpcExecutorServer) Signal(ctx context.Context, req *proto.SignalRequest) (*proto.SignalResponse, error) {

View File

@@ -3,11 +3,14 @@ package drivers
import (
"context"
"errors"
"fmt"
"io"
"time"
"github.com/LK4D4/joincontext"
"github.com/golang/protobuf/ptypes"
hclog "github.com/hashicorp/go-hclog"
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/proto"
@@ -24,6 +27,7 @@ type driverPluginClient struct {
*base.BasePluginClient
client proto.DriverClient
logger hclog.Logger
// doneCtx is closed when the plugin exits
doneCtx context.Context
@@ -252,20 +256,65 @@ func (d *driverPluginClient) InspectTask(taskID string) (*TaskStatus, error) {
}
// TaskStats returns resource usage statistics for the task
func (d *driverPluginClient) TaskStats(taskID string) (*TaskResourceUsage, error) {
req := &proto.TaskStatsRequest{TaskId: taskID}
resp, err := d.client.TaskStats(d.doneCtx, req)
if err != nil {
return nil, grpcutils.HandleGrpcErr(err, d.doneCtx)
func (d *driverPluginClient) TaskStats(ctx context.Context, taskID string, interval time.Duration) (<-chan *cstructs.TaskResourceUsage, error) {
req := &proto.TaskStatsRequest{
TaskId: taskID,
Interval: int64(interval),
}
stats, err := TaskStatsFromProto(resp.Stats)
ctx, _ = joincontext.Join(ctx, d.doneCtx)
stream, err := d.client.TaskStats(ctx, req)
if err != nil {
st := status.Convert(err)
if len(st.Details()) > 0 {
if rec, ok := st.Details()[0].(*sproto.RecoverableError); ok {
return nil, structs.NewRecoverableError(err, rec.Recoverable)
}
}
return nil, err
}
return stats, nil
ch := make(chan *cstructs.TaskResourceUsage, 1)
d.handleStats(ctx, ch, stream)
return ch, nil
}
func (d *driverPluginClient) handleStats(ctx context.Context, ch chan<- *cstructs.TaskResourceUsage, stream proto.Driver_TaskStatsClient) {
defer close(ch)
for {
resp, err := stream.Recv()
if err != nil {
if err != io.EOF {
d.logger.Error("error receiving stream from TaskStats driver RPC", "error", err)
truErr := &cstructs.TaskResourceUsage{
Err: grpcutils.HandleReqCtxGrpcErr(err, ctx, d.doneCtx),
}
select {
case ch <- truErr:
case <-ctx.Done():
}
}
// End of stream
return
}
stats, err := TaskStatsFromProto(resp.Stats)
if err != nil {
truErr := &cstructs.TaskResourceUsage{
Err: fmt.Errorf("failed to decode stats from RPC: %v", err),
}
select {
case ch <- truErr:
case <-ctx.Done():
}
}
select {
case ch <- stats:
case <-ctx.Done():
}
}
}
// TaskEvents returns a channel that will receive events from the driver about all

View File

@@ -11,6 +11,7 @@ import (
"time"
"github.com/hashicorp/nomad/client/allocdir"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/helper"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/nomad/plugins/base"
@@ -39,7 +40,7 @@ type DriverPlugin interface {
StopTask(taskID string, timeout time.Duration, signal string) error
DestroyTask(taskID string, force bool) error
InspectTask(taskID string) (*TaskStatus, error)
TaskStats(taskID string) (*TaskResourceUsage, error)
TaskStats(ctx context.Context, taskID string, interval time.Duration) (<-chan *cstructs.TaskResourceUsage, error)
TaskEvents(context.Context) (<-chan *TaskEvent, error)
SignalTask(taskID string, signal string) error

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_66cfa35dd20ec741, []int{0}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{5, 0}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{9, 0}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{25, 0}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{43, 0}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{44, 0}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{0}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{1}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{2}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{3}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{4}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{5}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{6}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{7}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{8}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{9}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{10}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{11}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{12}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{13}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{14}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{15}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{16}
return fileDescriptor_driver_5a41b73816b77fc1, []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_66cfa35dd20ec741, []int{17}
return fileDescriptor_driver_5a41b73816b77fc1, []int{17}
}
func (m *InspectTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_InspectTaskResponse.Unmarshal(m, b)
@@ -977,7 +977,9 @@ func (m *InspectTaskResponse) GetNetworkOverride() *NetworkOverride {
type TaskStatsRequest struct {
// TaskId is the ID of the target task
TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
// Interval is the interval in nanosecond at which to stream stats to the caller
Interval int64 `protobuf:"varint,2,opt,name=interval,proto3" json:"interval,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@@ -987,7 +989,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_66cfa35dd20ec741, []int{18}
return fileDescriptor_driver_5a41b73816b77fc1, []int{18}
}
func (m *TaskStatsRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskStatsRequest.Unmarshal(m, b)
@@ -1014,6 +1016,13 @@ func (m *TaskStatsRequest) GetTaskId() string {
return ""
}
func (m *TaskStatsRequest) GetInterval() int64 {
if m != nil {
return m.Interval
}
return 0
}
type TaskStatsResponse struct {
// Stats for the task
Stats *TaskStats `protobuf:"bytes,1,opt,name=stats,proto3" json:"stats,omitempty"`
@@ -1026,7 +1035,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_66cfa35dd20ec741, []int{19}
return fileDescriptor_driver_5a41b73816b77fc1, []int{19}
}
func (m *TaskStatsResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskStatsResponse.Unmarshal(m, b)
@@ -1063,7 +1072,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_66cfa35dd20ec741, []int{20}
return fileDescriptor_driver_5a41b73816b77fc1, []int{20}
}
func (m *TaskEventsRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskEventsRequest.Unmarshal(m, b)
@@ -1097,7 +1106,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_66cfa35dd20ec741, []int{21}
return fileDescriptor_driver_5a41b73816b77fc1, []int{21}
}
func (m *SignalTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SignalTaskRequest.Unmarshal(m, b)
@@ -1141,7 +1150,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_66cfa35dd20ec741, []int{22}
return fileDescriptor_driver_5a41b73816b77fc1, []int{22}
}
func (m *SignalTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SignalTaskResponse.Unmarshal(m, b)
@@ -1178,7 +1187,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_66cfa35dd20ec741, []int{23}
return fileDescriptor_driver_5a41b73816b77fc1, []int{23}
}
func (m *ExecTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExecTaskRequest.Unmarshal(m, b)
@@ -1235,7 +1244,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_66cfa35dd20ec741, []int{24}
return fileDescriptor_driver_5a41b73816b77fc1, []int{24}
}
func (m *ExecTaskResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExecTaskResponse.Unmarshal(m, b)
@@ -1294,7 +1303,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_66cfa35dd20ec741, []int{25}
return fileDescriptor_driver_5a41b73816b77fc1, []int{25}
}
func (m *DriverCapabilities) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DriverCapabilities.Unmarshal(m, b)
@@ -1380,7 +1389,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_66cfa35dd20ec741, []int{26}
return fileDescriptor_driver_5a41b73816b77fc1, []int{26}
}
func (m *TaskConfig) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskConfig.Unmarshal(m, b)
@@ -1519,7 +1528,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_66cfa35dd20ec741, []int{27}
return fileDescriptor_driver_5a41b73816b77fc1, []int{27}
}
func (m *Resources) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Resources.Unmarshal(m, b)
@@ -1566,7 +1575,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_66cfa35dd20ec741, []int{28}
return fileDescriptor_driver_5a41b73816b77fc1, []int{28}
}
func (m *AllocatedTaskResources) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_AllocatedTaskResources.Unmarshal(m, b)
@@ -1618,7 +1627,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_66cfa35dd20ec741, []int{29}
return fileDescriptor_driver_5a41b73816b77fc1, []int{29}
}
func (m *AllocatedCpuResources) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_AllocatedCpuResources.Unmarshal(m, b)
@@ -1656,7 +1665,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_66cfa35dd20ec741, []int{30}
return fileDescriptor_driver_5a41b73816b77fc1, []int{30}
}
func (m *AllocatedMemoryResources) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_AllocatedMemoryResources.Unmarshal(m, b)
@@ -1699,7 +1708,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_66cfa35dd20ec741, []int{31}
return fileDescriptor_driver_5a41b73816b77fc1, []int{31}
}
func (m *NetworkResource) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_NetworkResource.Unmarshal(m, b)
@@ -1773,7 +1782,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_66cfa35dd20ec741, []int{32}
return fileDescriptor_driver_5a41b73816b77fc1, []int{32}
}
func (m *NetworkPort) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_NetworkPort.Unmarshal(m, b)
@@ -1833,7 +1842,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_66cfa35dd20ec741, []int{33}
return fileDescriptor_driver_5a41b73816b77fc1, []int{33}
}
func (m *LinuxResources) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_LinuxResources.Unmarshal(m, b)
@@ -1925,7 +1934,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_66cfa35dd20ec741, []int{34}
return fileDescriptor_driver_5a41b73816b77fc1, []int{34}
}
func (m *Mount) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Mount.Unmarshal(m, b)
@@ -1988,7 +1997,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_66cfa35dd20ec741, []int{35}
return fileDescriptor_driver_5a41b73816b77fc1, []int{35}
}
func (m *Device) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Device.Unmarshal(m, b)
@@ -2046,7 +2055,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_66cfa35dd20ec741, []int{36}
return fileDescriptor_driver_5a41b73816b77fc1, []int{36}
}
func (m *TaskHandle) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskHandle.Unmarshal(m, b)
@@ -2106,7 +2115,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_66cfa35dd20ec741, []int{37}
return fileDescriptor_driver_5a41b73816b77fc1, []int{37}
}
func (m *NetworkOverride) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_NetworkOverride.Unmarshal(m, b)
@@ -2164,7 +2173,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_66cfa35dd20ec741, []int{38}
return fileDescriptor_driver_5a41b73816b77fc1, []int{38}
}
func (m *ExitResult) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExitResult.Unmarshal(m, b)
@@ -2227,7 +2236,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_66cfa35dd20ec741, []int{39}
return fileDescriptor_driver_5a41b73816b77fc1, []int{39}
}
func (m *TaskStatus) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskStatus.Unmarshal(m, b)
@@ -2302,7 +2311,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_66cfa35dd20ec741, []int{40}
return fileDescriptor_driver_5a41b73816b77fc1, []int{40}
}
func (m *TaskDriverStatus) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskDriverStatus.Unmarshal(m, b)
@@ -2347,7 +2356,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_66cfa35dd20ec741, []int{41}
return fileDescriptor_driver_5a41b73816b77fc1, []int{41}
}
func (m *TaskStats) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskStats.Unmarshal(m, b)
@@ -2409,7 +2418,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_66cfa35dd20ec741, []int{42}
return fileDescriptor_driver_5a41b73816b77fc1, []int{42}
}
func (m *TaskResourceUsage) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TaskResourceUsage.Unmarshal(m, b)
@@ -2461,7 +2470,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_66cfa35dd20ec741, []int{43}
return fileDescriptor_driver_5a41b73816b77fc1, []int{43}
}
func (m *CPUUsage) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CPUUsage.Unmarshal(m, b)
@@ -2547,7 +2556,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_66cfa35dd20ec741, []int{44}
return fileDescriptor_driver_5a41b73816b77fc1, []int{44}
}
func (m *MemoryUsage) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_MemoryUsage.Unmarshal(m, b)
@@ -2631,7 +2640,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_66cfa35dd20ec741, []int{45}
return fileDescriptor_driver_5a41b73816b77fc1, []int{45}
}
func (m *DriverTaskEvent) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DriverTaskEvent.Unmarshal(m, b)
@@ -2802,7 +2811,7 @@ type DriverClient interface {
// InspectTask returns detailed information for the given task
InspectTask(ctx context.Context, in *InspectTaskRequest, opts ...grpc.CallOption) (*InspectTaskResponse, error)
// TaskStats collects and returns runtime metrics for the given task
TaskStats(ctx context.Context, in *TaskStatsRequest, opts ...grpc.CallOption) (*TaskStatsResponse, error)
TaskStats(ctx context.Context, in *TaskStatsRequest, opts ...grpc.CallOption) (Driver_TaskStatsClient, error)
// TaskEvents starts a streaming RPC where all task events emitted by the
// driver are streamed to the caller.
TaskEvents(ctx context.Context, in *TaskEventsRequest, opts ...grpc.CallOption) (Driver_TaskEventsClient, error)
@@ -2924,17 +2933,40 @@ func (c *driverClient) InspectTask(ctx context.Context, in *InspectTaskRequest,
return out, nil
}
func (c *driverClient) TaskStats(ctx context.Context, in *TaskStatsRequest, opts ...grpc.CallOption) (*TaskStatsResponse, error) {
out := new(TaskStatsResponse)
err := c.cc.Invoke(ctx, "/hashicorp.nomad.plugins.drivers.proto.Driver/TaskStats", in, out, opts...)
func (c *driverClient) TaskStats(ctx context.Context, in *TaskStatsRequest, opts ...grpc.CallOption) (Driver_TaskStatsClient, error) {
stream, err := c.cc.NewStream(ctx, &_Driver_serviceDesc.Streams[1], "/hashicorp.nomad.plugins.drivers.proto.Driver/TaskStats", opts...)
if err != nil {
return nil, err
}
return out, nil
x := &driverTaskStatsClient{stream}
if err := x.ClientStream.SendMsg(in); err != nil {
return nil, err
}
if err := x.ClientStream.CloseSend(); err != nil {
return nil, err
}
return x, nil
}
type Driver_TaskStatsClient interface {
Recv() (*TaskStatsResponse, error)
grpc.ClientStream
}
type driverTaskStatsClient struct {
grpc.ClientStream
}
func (x *driverTaskStatsClient) Recv() (*TaskStatsResponse, error) {
m := new(TaskStatsResponse)
if err := x.ClientStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
func (c *driverClient) TaskEvents(ctx context.Context, in *TaskEventsRequest, opts ...grpc.CallOption) (Driver_TaskEventsClient, error) {
stream, err := c.cc.NewStream(ctx, &_Driver_serviceDesc.Streams[1], "/hashicorp.nomad.plugins.drivers.proto.Driver/TaskEvents", opts...)
stream, err := c.cc.NewStream(ctx, &_Driver_serviceDesc.Streams[2], "/hashicorp.nomad.plugins.drivers.proto.Driver/TaskEvents", opts...)
if err != nil {
return nil, err
}
@@ -3020,7 +3052,7 @@ type DriverServer interface {
// InspectTask returns detailed information for the given task
InspectTask(context.Context, *InspectTaskRequest) (*InspectTaskResponse, error)
// TaskStats collects and returns runtime metrics for the given task
TaskStats(context.Context, *TaskStatsRequest) (*TaskStatsResponse, error)
TaskStats(*TaskStatsRequest, Driver_TaskStatsServer) error
// TaskEvents starts a streaming RPC where all task events emitted by the
// driver are streamed to the caller.
TaskEvents(*TaskEventsRequest, Driver_TaskEventsServer) error
@@ -3199,22 +3231,25 @@ func _Driver_InspectTask_Handler(srv interface{}, ctx context.Context, dec func(
return interceptor(ctx, in, info, handler)
}
func _Driver_TaskStats_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(TaskStatsRequest)
if err := dec(in); err != nil {
return nil, err
func _Driver_TaskStats_Handler(srv interface{}, stream grpc.ServerStream) error {
m := new(TaskStatsRequest)
if err := stream.RecvMsg(m); err != nil {
return err
}
if interceptor == nil {
return srv.(DriverServer).TaskStats(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/hashicorp.nomad.plugins.drivers.proto.Driver/TaskStats",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(DriverServer).TaskStats(ctx, req.(*TaskStatsRequest))
}
return interceptor(ctx, in, info, handler)
return srv.(DriverServer).TaskStats(m, &driverTaskStatsServer{stream})
}
type Driver_TaskStatsServer interface {
Send(*TaskStatsResponse) error
grpc.ServerStream
}
type driverTaskStatsServer struct {
grpc.ServerStream
}
func (x *driverTaskStatsServer) Send(m *TaskStatsResponse) error {
return x.ServerStream.SendMsg(m)
}
func _Driver_TaskEvents_Handler(srv interface{}, stream grpc.ServerStream) error {
@@ -3310,10 +3345,6 @@ var _Driver_serviceDesc = grpc.ServiceDesc{
MethodName: "InspectTask",
Handler: _Driver_InspectTask_Handler,
},
{
MethodName: "TaskStats",
Handler: _Driver_TaskStats_Handler,
},
{
MethodName: "SignalTask",
Handler: _Driver_SignalTask_Handler,
@@ -3329,6 +3360,11 @@ var _Driver_serviceDesc = grpc.ServiceDesc{
Handler: _Driver_Fingerprint_Handler,
ServerStreams: true,
},
{
StreamName: "TaskStats",
Handler: _Driver_TaskStats_Handler,
ServerStreams: true,
},
{
StreamName: "TaskEvents",
Handler: _Driver_TaskEvents_Handler,
@@ -3339,193 +3375,194 @@ var _Driver_serviceDesc = grpc.ServiceDesc{
}
func init() {
proto.RegisterFile("plugins/drivers/proto/driver.proto", fileDescriptor_driver_66cfa35dd20ec741)
proto.RegisterFile("plugins/drivers/proto/driver.proto", fileDescriptor_driver_5a41b73816b77fc1)
}
var fileDescriptor_driver_66cfa35dd20ec741 = []byte{
// 2940 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, 0x12, 0x35, 0xdb, 0xbb, 0x6b, 0xd3, 0x34, 0xbe, 0xcf, 0xeb, 0x01,
0xfc, 0x41, 0xb0, 0xbd, 0x94, 0x2d, 0xe3, 0xdb, 0x57, 0xfc, 0xa2, 0xc9, 0x59, 0x49, 0x5e, 0x89,
0x52, 0x9a, 0x14, 0xd6, 0x9b, 0xc4, 0x9e, 0x8c, 0x66, 0x5a, 0xe4, 0xac, 0x38, 0x0f, 0xcf, 0xf4,
0xc8, 0x12, 0x82, 0x20, 0x41, 0x02, 0x04, 0xc9, 0x21, 0x40, 0x2e, 0x46, 0xee, 0xc9, 0x31, 0x7f,
0x41, 0x12, 0xf8, 0x92, 0xbf, 0x22, 0x40, 0x2e, 0xc9, 0x21, 0x40, 0xae, 0xf9, 0x0f, 0x82, 0x7e,
0xcc, 0x70, 0x46, 0xd2, 0x7a, 0x87, 0xdc, 0x9c, 0xc8, 0xae, 0xee, 0xfa, 0x75, 0x4d, 0x55, 0x75,
0x55, 0x75, 0x17, 0xa8, 0xfe, 0x34, 0x1a, 0xdb, 0x6e, 0xb8, 0x61, 0x05, 0xf6, 0x29, 0x09, 0xc2,
0x0d, 0x3f, 0xf0, 0xa8, 0x27, 0x47, 0x1d, 0x3e, 0x40, 0x6f, 0x4c, 0x8c, 0x70, 0x62, 0x9b, 0x5e,
0xe0, 0x77, 0x5c, 0xcf, 0x31, 0xac, 0x8e, 0xe4, 0xe9, 0x48, 0x1e, 0xb1, 0xac, 0xfd, 0xbf, 0x63,
0xcf, 0x1b, 0x4f, 0x89, 0x40, 0x38, 0x8a, 0x8e, 0x37, 0xac, 0x28, 0x30, 0xa8, 0xed, 0xb9, 0x72,
0xfe, 0xb5, 0x8b, 0xf3, 0xd4, 0x76, 0x48, 0x48, 0x0d, 0xc7, 0x97, 0x0b, 0x3e, 0x1e, 0xdb, 0x74,
0x12, 0x1d, 0x75, 0x4c, 0xcf, 0xd9, 0x48, 0xb6, 0xdc, 0xe0, 0x5b, 0x6e, 0xc4, 0x62, 0x86, 0x13,
0x23, 0x20, 0xd6, 0xc6, 0xc4, 0x9c, 0x86, 0x3e, 0x31, 0xd9, 0xaf, 0xce, 0xfe, 0x48, 0x84, 0xad,
0xfc, 0x08, 0x21, 0x0d, 0x22, 0x93, 0xc6, 0xdf, 0x6b, 0x50, 0x1a, 0xd8, 0x47, 0x11, 0x25, 0x02,
0x48, 0x7d, 0x05, 0x5e, 0x1e, 0x19, 0xe1, 0x49, 0xcf, 0x73, 0x8f, 0xed, 0xf1, 0xd0, 0x9c, 0x10,
0xc7, 0xc0, 0xe4, 0xcb, 0x88, 0x84, 0x54, 0xfd, 0x01, 0xb4, 0x2e, 0x4f, 0x85, 0xbe, 0xe7, 0x86,
0x04, 0x7d, 0x0c, 0x65, 0x26, 0x4d, 0xab, 0x70, 0xab, 0xb0, 0xde, 0xd8, 0x7c, 0xbb, 0xf3, 0x2c,
0xc5, 0x09, 0x19, 0x3a, 0xf2, 0x2b, 0x3a, 0x43, 0x9f, 0x98, 0x98, 0x73, 0xaa, 0x37, 0xe1, 0x7a,
0xcf, 0xf0, 0x8d, 0x23, 0x7b, 0x6a, 0x53, 0x9b, 0x84, 0xf1, 0xa6, 0x11, 0xdc, 0xc8, 0x92, 0xe5,
0x86, 0x9f, 0xc3, 0x8a, 0x99, 0xa2, 0xcb, 0x8d, 0xef, 0x77, 0x72, 0x59, 0xac, 0xd3, 0xe7, 0xa3,
0x0c, 0x70, 0x06, 0x4e, 0xbd, 0x01, 0xe8, 0xa1, 0xed, 0x8e, 0x49, 0xe0, 0x07, 0xb6, 0x4b, 0x63,
0x61, 0xbe, 0x29, 0xc1, 0xf5, 0x0c, 0x59, 0x0a, 0xf3, 0x14, 0x20, 0xd1, 0x23, 0x13, 0xa5, 0xb4,
0xde, 0xd8, 0xfc, 0x34, 0xa7, 0x28, 0x57, 0xe0, 0x75, 0xba, 0x09, 0x98, 0xe6, 0xd2, 0xe0, 0x1c,
0xa7, 0xd0, 0xd1, 0x17, 0x50, 0x9d, 0x10, 0x63, 0x4a, 0x27, 0xad, 0xe2, 0xad, 0xc2, 0x7a, 0x73,
0xf3, 0xe1, 0x0b, 0xec, 0xb3, 0xcd, 0x81, 0x86, 0xd4, 0xa0, 0x04, 0x4b, 0x54, 0x74, 0x1b, 0x90,
0xf8, 0xa7, 0x5b, 0x24, 0x34, 0x03, 0xdb, 0x67, 0x8e, 0xdc, 0x2a, 0xdd, 0x2a, 0xac, 0xd7, 0xf1,
0x35, 0x31, 0xd3, 0x9f, 0x4d, 0xb4, 0x7d, 0x58, 0xbb, 0x20, 0x2d, 0x52, 0xa0, 0x74, 0x42, 0xce,
0xb9, 0x45, 0xea, 0x98, 0xfd, 0x45, 0x5b, 0x50, 0x39, 0x35, 0xa6, 0x11, 0xe1, 0x22, 0x37, 0x36,
0xdf, 0x7d, 0x9e, 0x7b, 0x48, 0x17, 0x9d, 0xe9, 0x01, 0x0b, 0xfe, 0x07, 0xc5, 0x7b, 0x05, 0xf5,
0x3e, 0x34, 0x52, 0x72, 0xa3, 0x26, 0xc0, 0xe1, 0xa0, 0xaf, 0x8d, 0xb4, 0xde, 0x48, 0xeb, 0x2b,
0x4b, 0x68, 0x15, 0xea, 0x87, 0x83, 0x6d, 0xad, 0xbb, 0x3b, 0xda, 0x7e, 0xa2, 0x14, 0x50, 0x03,
0x96, 0xe3, 0x41, 0x51, 0x3d, 0x03, 0x84, 0x89, 0xe9, 0x9d, 0x92, 0x80, 0x39, 0xb2, 0xb4, 0x2a,
0x7a, 0x19, 0x96, 0xa9, 0x11, 0x9e, 0xe8, 0xb6, 0x25, 0x65, 0xae, 0xb2, 0xe1, 0x8e, 0x85, 0x76,
0xa0, 0x3a, 0x31, 0x5c, 0x6b, 0xfa, 0x7c, 0xb9, 0xb3, 0xaa, 0x66, 0xe0, 0xdb, 0x9c, 0x11, 0x4b,
0x00, 0xe6, 0xdd, 0x99, 0x9d, 0x85, 0x01, 0xd4, 0x27, 0xa0, 0x0c, 0xa9, 0x11, 0xd0, 0xb4, 0x38,
0x1a, 0x94, 0xd9, 0xfe, 0xd2, 0xa3, 0xe7, 0xd9, 0x53, 0x9c, 0x4c, 0xcc, 0xd9, 0xd5, 0x7f, 0x17,
0xe1, 0x5a, 0x0a, 0x5b, 0x7a, 0xea, 0x63, 0xa8, 0x06, 0x24, 0x8c, 0xa6, 0x94, 0xc3, 0x37, 0x37,
0x3f, 0xca, 0x09, 0x7f, 0x09, 0xa9, 0x83, 0x39, 0x0c, 0x96, 0x70, 0x68, 0x1d, 0x14, 0xc1, 0xa1,
0x93, 0x20, 0xf0, 0x02, 0xdd, 0x09, 0xc7, 0x5c, 0x6b, 0x75, 0xdc, 0x14, 0x74, 0x8d, 0x91, 0xf7,
0xc2, 0x71, 0x4a, 0xab, 0xa5, 0x17, 0xd4, 0x2a, 0x32, 0x40, 0x71, 0x09, 0xfd, 0xca, 0x0b, 0x4e,
0x74, 0xa6, 0xda, 0xc0, 0xb6, 0x48, 0xab, 0xcc, 0x41, 0xef, 0xe4, 0x04, 0x1d, 0x08, 0xf6, 0x7d,
0xc9, 0x8d, 0xd7, 0xdc, 0x2c, 0x41, 0x7d, 0x0b, 0xaa, 0xe2, 0x4b, 0x99, 0x27, 0x0d, 0x0f, 0x7b,
0x3d, 0x6d, 0x38, 0x54, 0x96, 0x50, 0x1d, 0x2a, 0x58, 0x1b, 0x61, 0xe6, 0x61, 0x75, 0xa8, 0x3c,
0xec, 0x8e, 0xba, 0xbb, 0x4a, 0x51, 0x7d, 0x13, 0xd6, 0x1e, 0x1b, 0x36, 0xcd, 0xe3, 0x5c, 0xaa,
0x07, 0xca, 0x6c, 0xad, 0xb4, 0xce, 0x4e, 0xc6, 0x3a, 0xf9, 0x55, 0xa3, 0x9d, 0xd9, 0xf4, 0x82,
0x3d, 0x14, 0x28, 0x91, 0x20, 0x90, 0x26, 0x60, 0x7f, 0xd5, 0xaf, 0x60, 0x6d, 0x48, 0x3d, 0x3f,
0x97, 0xe7, 0xbf, 0x07, 0xcb, 0x2c, 0x47, 0x79, 0x11, 0x95, 0xae, 0xff, 0x4a, 0x47, 0xe4, 0xb0,
0x4e, 0x9c, 0xc3, 0x3a, 0x7d, 0x99, 0xe3, 0x70, 0xbc, 0x12, 0xbd, 0x04, 0xd5, 0xd0, 0x1e, 0xbb,
0xc6, 0x54, 0x46, 0x0b, 0x39, 0x52, 0x11, 0x73, 0xf2, 0x78, 0x63, 0xe9, 0xf8, 0x3d, 0x40, 0x7d,
0x12, 0xd2, 0xc0, 0x3b, 0xcf, 0x25, 0xcf, 0x0d, 0xa8, 0x1c, 0x7b, 0x81, 0x29, 0x0e, 0x62, 0x0d,
0x8b, 0x01, 0x3b, 0x54, 0x19, 0x10, 0x89, 0x7d, 0x1b, 0xd0, 0x8e, 0xcb, 0x72, 0x4a, 0x3e, 0x43,
0xfc, 0xa6, 0x08, 0xd7, 0x33, 0xeb, 0xa5, 0x31, 0x16, 0x3f, 0x87, 0x2c, 0x30, 0x45, 0xa1, 0x38,
0x87, 0x68, 0x1f, 0xaa, 0x62, 0x85, 0xd4, 0xe4, 0xdd, 0x39, 0x80, 0x44, 0x9a, 0x92, 0x70, 0x12,
0xe6, 0x4a, 0xa7, 0x2f, 0xfd, 0xb7, 0x9d, 0x5e, 0x89, 0xbf, 0x23, 0x7c, 0xae, 0xfe, 0xbe, 0x0f,
0xd7, 0x52, 0x8b, 0xa5, 0xf2, 0x1e, 0x42, 0x25, 0x64, 0x04, 0xa9, 0xbd, 0x77, 0xe6, 0xd4, 0x5e,
0x88, 0x05, 0xbb, 0x7a, 0x5d, 0x80, 0x6b, 0xa7, 0xc4, 0x4d, 0x44, 0x51, 0xfb, 0x70, 0x6d, 0xc8,
0x5d, 0x2b, 0x97, 0xef, 0xcc, 0xdc, 0xb2, 0x98, 0x71, 0xcb, 0x1b, 0x80, 0xd2, 0x28, 0xd2, 0x79,
0xce, 0x61, 0x4d, 0x3b, 0x23, 0x66, 0x2e, 0xe4, 0x16, 0x2c, 0x9b, 0x9e, 0xe3, 0x18, 0xae, 0xd5,
0x2a, 0xde, 0x2a, 0xad, 0xd7, 0x71, 0x3c, 0x4c, 0x9f, 0x9f, 0x52, 0xde, 0xf3, 0xa3, 0xfe, 0xba,
0x00, 0xca, 0x6c, 0x6f, 0xa9, 0x48, 0x26, 0x3d, 0xb5, 0x18, 0x10, 0xdb, 0x7b, 0x05, 0xcb, 0x91,
0xa4, 0xc7, 0x47, 0x5c, 0xd0, 0x49, 0x10, 0xa4, 0x42, 0x48, 0xe9, 0x05, 0x43, 0x88, 0xfa, 0xcf,
0x02, 0xa0, 0xcb, 0x85, 0x12, 0x7a, 0x1d, 0x56, 0x42, 0xe2, 0x5a, 0xba, 0x50, 0xa3, 0xb0, 0x70,
0x0d, 0x37, 0x18, 0x4d, 0xe8, 0x33, 0x44, 0x08, 0xca, 0xe4, 0x8c, 0x98, 0xf2, 0xb4, 0xf2, 0xff,
0x68, 0x02, 0x2b, 0xc7, 0xa1, 0x6e, 0x87, 0xde, 0xd4, 0x48, 0x2a, 0x8a, 0xe6, 0xa6, 0xb6, 0x70,
0xc1, 0xd6, 0x79, 0x38, 0xdc, 0x89, 0xc1, 0x70, 0xe3, 0x38, 0x4c, 0x06, 0x6a, 0x07, 0x1a, 0xa9,
0x39, 0x54, 0x83, 0xf2, 0x60, 0x7f, 0xa0, 0x29, 0x4b, 0x08, 0xa0, 0xda, 0xdb, 0xc6, 0xfb, 0xfb,
0x23, 0x11, 0xb5, 0x77, 0xf6, 0xba, 0x5b, 0x9a, 0x52, 0x54, 0xff, 0x58, 0x05, 0x98, 0xa5, 0x4f,
0xd4, 0x84, 0x62, 0x62, 0xe9, 0xa2, 0x6d, 0xb1, 0x8f, 0x71, 0x0d, 0x87, 0x48, 0xef, 0xe1, 0xff,
0xd1, 0x26, 0xdc, 0x74, 0xc2, 0xb1, 0x6f, 0x98, 0x27, 0xba, 0xcc, 0x7a, 0x26, 0x67, 0xe6, 0x5f,
0xb5, 0x82, 0xaf, 0xcb, 0x49, 0x29, 0xb5, 0xc0, 0xdd, 0x85, 0x12, 0x71, 0x4f, 0x5b, 0x65, 0x5e,
0x1d, 0x3e, 0x98, 0x3b, 0xad, 0x77, 0x34, 0xf7, 0x54, 0x54, 0x83, 0x0c, 0x06, 0xe9, 0x00, 0x16,
0x39, 0xb5, 0x4d, 0xa2, 0x33, 0xd0, 0x0a, 0x07, 0xfd, 0x78, 0x7e, 0xd0, 0x3e, 0xc7, 0x48, 0xa0,
0xeb, 0x56, 0x3c, 0x46, 0x03, 0xa8, 0x07, 0x24, 0xf4, 0xa2, 0xc0, 0x24, 0x61, 0xab, 0x3a, 0xd7,
0x29, 0xc6, 0x31, 0x1f, 0x9e, 0x41, 0xa0, 0x3e, 0x54, 0x1d, 0x2f, 0x72, 0x69, 0xd8, 0x5a, 0xe6,
0xc2, 0xbe, 0x9d, 0x13, 0x6c, 0x8f, 0x31, 0x61, 0xc9, 0x8b, 0xb6, 0x60, 0x59, 0x88, 0x18, 0xb6,
0x6a, 0x1c, 0xe6, 0x76, 0x5e, 0x07, 0xe2, 0x5c, 0x38, 0xe6, 0x66, 0x56, 0x8d, 0x42, 0x12, 0xb4,
0xea, 0xc2, 0xaa, 0xec, 0x3f, 0x7a, 0x15, 0xea, 0xc6, 0x74, 0xea, 0x99, 0xba, 0x65, 0x07, 0x2d,
0xe0, 0x13, 0x35, 0x4e, 0xe8, 0xdb, 0x01, 0x7a, 0x0d, 0x1a, 0xe2, 0xe8, 0xe9, 0xbe, 0x41, 0x27,
0xad, 0x06, 0x9f, 0x06, 0x41, 0x3a, 0x30, 0xe8, 0x44, 0x2e, 0x20, 0x41, 0x20, 0x16, 0xac, 0x24,
0x0b, 0x48, 0x10, 0xf0, 0x05, 0xff, 0x07, 0x6b, 0x3c, 0x8e, 0x8c, 0x03, 0x2f, 0xf2, 0x75, 0xee,
0x53, 0xab, 0x7c, 0xd1, 0x2a, 0x23, 0x6f, 0x31, 0xea, 0x80, 0x39, 0xd7, 0x2b, 0x50, 0x7b, 0xea,
0x1d, 0x89, 0x05, 0x4d, 0xbe, 0x60, 0xf9, 0xa9, 0x77, 0x14, 0x4f, 0x09, 0x09, 0x6d, 0xab, 0xb5,
0x26, 0xa6, 0xf8, 0x78, 0xc7, 0x6a, 0xdf, 0x81, 0x5a, 0x6c, 0xc6, 0x2b, 0x2a, 0xf0, 0x1b, 0xe9,
0x0a, 0xbc, 0x9e, 0x2a, 0xa7, 0xdb, 0xef, 0x43, 0x33, 0xeb, 0x04, 0xf3, 0x70, 0xab, 0x7f, 0x2b,
0x40, 0x3d, 0x31, 0x37, 0x72, 0xe1, 0x3a, 0x17, 0xc7, 0xa0, 0xc4, 0xd2, 0x67, 0xde, 0x23, 0x72,
0xc0, 0x07, 0x39, 0x2d, 0xd5, 0x8d, 0x11, 0x64, 0x1c, 0x94, 0xae, 0x84, 0x12, 0xe4, 0xd9, 0x7e,
0x5f, 0xc0, 0xda, 0xd4, 0x76, 0xa3, 0xb3, 0xd4, 0x5e, 0x22, 0xc9, 0xfe, 0x7f, 0xce, 0xbd, 0x76,
0x19, 0xf7, 0x6c, 0x8f, 0xe6, 0x34, 0x33, 0x56, 0xbf, 0x2e, 0xc2, 0x4b, 0x57, 0x8b, 0x83, 0x06,
0x50, 0x32, 0xfd, 0x48, 0x7e, 0xda, 0xfb, 0xf3, 0x7e, 0x5a, 0xcf, 0x8f, 0x66, 0xbb, 0x32, 0x20,
0x56, 0x98, 0x3b, 0xc4, 0xf1, 0x82, 0x73, 0xf9, 0x05, 0x1f, 0xcd, 0x0b, 0xb9, 0xc7, 0xb9, 0x67,
0xa8, 0x12, 0x0e, 0x61, 0xa8, 0xc9, 0xf4, 0x1e, 0xca, 0x30, 0x31, 0x67, 0x99, 0x10, 0x43, 0xe2,
0x04, 0x47, 0xbd, 0x03, 0x37, 0xaf, 0xfc, 0x14, 0xf4, 0x3f, 0x00, 0xa6, 0x1f, 0xe9, 0xfc, 0x1a,
0x27, 0xec, 0x5e, 0xc2, 0x75, 0xd3, 0x8f, 0x86, 0x9c, 0xa0, 0xde, 0x85, 0xd6, 0xb3, 0xe4, 0x65,
0x87, 0x4f, 0x48, 0xac, 0x3b, 0x47, 0x5c, 0x07, 0x25, 0x5c, 0x13, 0x84, 0xbd, 0x23, 0xf5, 0xb7,
0x45, 0x58, 0xbb, 0x20, 0x0e, 0xcb, 0x80, 0xe2, 0x30, 0xc7, 0x59, 0x59, 0x8c, 0xd8, 0xc9, 0x36,
0x6d, 0x2b, 0x2e, 0x7d, 0xf9, 0x7f, 0x1e, 0xd3, 0x7d, 0x59, 0x96, 0x16, 0x6d, 0x9f, 0x39, 0xb4,
0x73, 0x64, 0xd3, 0x90, 0xdf, 0x16, 0x2a, 0x58, 0x0c, 0xd0, 0x13, 0x68, 0x06, 0x24, 0x24, 0xc1,
0x29, 0xb1, 0x74, 0xdf, 0x0b, 0x68, 0xac, 0xb0, 0xcd, 0xf9, 0x14, 0x76, 0xe0, 0x05, 0x14, 0xaf,
0xc6, 0x48, 0x6c, 0x14, 0xa2, 0xc7, 0xb0, 0x6a, 0x9d, 0xbb, 0x86, 0x63, 0x9b, 0x12, 0xb9, 0xba,
0x30, 0xf2, 0x8a, 0x04, 0xe2, 0xc0, 0xec, 0x36, 0x9c, 0x9a, 0x64, 0x1f, 0x36, 0x35, 0x8e, 0xc8,
0x54, 0xea, 0x44, 0x0c, 0xb2, 0xe7, 0xb7, 0x22, 0xcf, 0xaf, 0xfa, 0xfb, 0x22, 0x34, 0xb3, 0x07,
0x20, 0xb6, 0x9f, 0x4f, 0x02, 0xdb, 0xb3, 0x52, 0xf6, 0x3b, 0xe0, 0x04, 0x66, 0x23, 0x36, 0xfd,
0x65, 0xe4, 0x51, 0x23, 0xb6, 0x91, 0xe9, 0x47, 0xdf, 0x65, 0xe3, 0x0b, 0xb6, 0x2f, 0x5d, 0xb0,
0x3d, 0x7a, 0x1b, 0x90, 0xb4, 0xef, 0xd4, 0x76, 0x6c, 0xaa, 0x1f, 0x9d, 0x53, 0x22, 0xf4, 0x5f,
0xc2, 0x8a, 0x98, 0xd9, 0x65, 0x13, 0x9f, 0x30, 0x3a, 0x52, 0x61, 0xd5, 0xf3, 0x1c, 0x3d, 0x34,
0xbd, 0x80, 0xe8, 0x86, 0xf5, 0xb4, 0x55, 0xe1, 0x0b, 0x1b, 0x9e, 0xe7, 0x0c, 0x19, 0xad, 0x6b,
0x3d, 0x65, 0x01, 0xd7, 0xf4, 0xa3, 0x90, 0x50, 0x9d, 0xfd, 0xf0, 0x1c, 0x55, 0xc7, 0x20, 0x48,
0x3d, 0x3f, 0x0a, 0x53, 0x0b, 0x1c, 0xe2, 0xb0, 0xbc, 0x93, 0x5a, 0xb0, 0x47, 0x1c, 0xb6, 0xcb,
0xca, 0x01, 0x09, 0x4c, 0xe2, 0xd2, 0x91, 0x6d, 0x9e, 0xb0, 0x94, 0x52, 0x58, 0x2f, 0xe0, 0x0c,
0x4d, 0xfd, 0x1c, 0x2a, 0x3c, 0x05, 0xb1, 0x8f, 0xe7, 0xe1, 0x9b, 0x47, 0x77, 0xa1, 0xde, 0x1a,
0x23, 0xf0, 0xd8, 0xfe, 0x2a, 0xd4, 0x27, 0x5e, 0x28, 0x73, 0x83, 0xf0, 0xbc, 0x1a, 0x23, 0xf0,
0xc9, 0x36, 0xd4, 0x02, 0x62, 0x58, 0x9e, 0x3b, 0x3d, 0xe7, 0x7a, 0xa9, 0xe1, 0x64, 0xac, 0x7e,
0x09, 0x55, 0x11, 0x7e, 0x5f, 0x00, 0xff, 0x36, 0x20, 0x53, 0x24, 0x15, 0x9f, 0x04, 0x8e, 0x1d,
0x86, 0xb6, 0xe7, 0x86, 0xf1, 0x93, 0x8d, 0x98, 0x39, 0x98, 0x4d, 0xa8, 0x7f, 0x29, 0x88, 0x7a,
0x47, 0x5c, 0xa6, 0x59, 0xc5, 0x28, 0x8b, 0x97, 0x85, 0x5f, 0x1c, 0x24, 0x40, 0x5c, 0xf5, 0x13,
0xf9, 0x34, 0x35, 0x6f, 0xd5, 0x4f, 0x44, 0xd5, 0x4f, 0x58, 0x89, 0x29, 0xcb, 0x2a, 0x01, 0x27,
0xaa, 0xaa, 0x86, 0x95, 0x5c, 0x87, 0x88, 0xfa, 0xaf, 0x42, 0x12, 0x11, 0xe2, 0x6b, 0x0b, 0xfa,
0x02, 0x6a, 0xec, 0x70, 0xe9, 0x8e, 0xe1, 0xcb, 0x47, 0xb8, 0xde, 0x62, 0x37, 0xa2, 0x0e, 0x3b,
0x4b, 0x7b, 0x86, 0x2f, 0x8a, 0xa2, 0x65, 0x5f, 0x8c, 0x58, 0x64, 0x31, 0xac, 0x59, 0x64, 0x61,
0xff, 0xd1, 0x1b, 0xd0, 0x34, 0x22, 0xea, 0xe9, 0x86, 0x75, 0x4a, 0x02, 0x6a, 0x87, 0x44, 0x5a,
0x78, 0x95, 0x51, 0xbb, 0x31, 0xb1, 0xfd, 0x00, 0x56, 0xd2, 0x98, 0xcf, 0xcb, 0xb1, 0x95, 0x74,
0x8e, 0xfd, 0x21, 0xc0, 0xac, 0x3a, 0x67, 0x9e, 0x40, 0xce, 0x6c, 0xaa, 0x9b, 0x9e, 0x25, 0x22,
0x5f, 0x05, 0xd7, 0x18, 0xa1, 0xe7, 0x59, 0xe4, 0xc2, 0x5d, 0xa7, 0x12, 0xdf, 0x75, 0xd8, 0xd9,
0x64, 0xc7, 0xe9, 0xc4, 0x9e, 0x4e, 0x89, 0x25, 0x25, 0xac, 0x7b, 0x9e, 0xf3, 0x88, 0x13, 0xd4,
0x6f, 0x8a, 0xc2, 0x23, 0xc4, 0x4d, 0x33, 0x57, 0x05, 0x9c, 0x98, 0xba, 0xf4, 0x62, 0xa6, 0xbe,
0x0f, 0x10, 0x52, 0x23, 0x60, 0x05, 0x83, 0x41, 0xe5, 0xe3, 0x4d, 0xfb, 0xd2, 0x65, 0x69, 0x14,
0x3f, 0x98, 0xe3, 0xba, 0x5c, 0xdd, 0xa5, 0xe8, 0x03, 0x58, 0x31, 0x3d, 0xc7, 0x9f, 0x12, 0xc9,
0x5c, 0x79, 0x2e, 0x73, 0x23, 0x59, 0xdf, 0xa5, 0xa9, 0x9b, 0x52, 0xf5, 0x45, 0x6f, 0x4a, 0x7f,
0x2a, 0x88, 0x0b, 0x73, 0xfa, 0xbe, 0x8e, 0xc6, 0x57, 0x3c, 0x0a, 0x6f, 0x2d, 0x78, 0xf9, 0xff,
0xb6, 0x17, 0xe1, 0xf6, 0x07, 0x79, 0x9e, 0x60, 0x9f, 0x5d, 0xc2, 0xfd, 0xb9, 0x04, 0xf5, 0xe4,
0xde, 0x7d, 0xc9, 0xf6, 0xf7, 0xa0, 0x9e, 0x74, 0x2b, 0x64, 0x69, 0xf2, 0xad, 0xe6, 0x49, 0x16,
0xa3, 0x63, 0x40, 0xc6, 0x78, 0x9c, 0x94, 0x66, 0x7a, 0x14, 0x1a, 0xe3, 0xf8, 0xa5, 0xe2, 0xde,
0x1c, 0x7a, 0x88, 0xb3, 0xd3, 0x21, 0xe3, 0xc7, 0x8a, 0x31, 0x1e, 0x67, 0x28, 0xe8, 0x47, 0x70,
0x33, 0xbb, 0x87, 0x7e, 0x74, 0xae, 0xfb, 0xb6, 0x25, 0x6f, 0x5a, 0xdb, 0xf3, 0x3e, 0x3d, 0x74,
0x32, 0xf0, 0x9f, 0x9c, 0x1f, 0xd8, 0x96, 0xd0, 0x39, 0x0a, 0x2e, 0x4d, 0xb4, 0x7f, 0x02, 0x2f,
0x3f, 0x63, 0xf9, 0x15, 0x36, 0x18, 0x64, 0x9f, 0xc1, 0x17, 0x57, 0x42, 0xca, 0x7a, 0xbf, 0x2b,
0x88, 0x17, 0x92, 0xac, 0x4e, 0xba, 0xe9, 0xea, 0x74, 0x23, 0xe7, 0x3e, 0xbd, 0x83, 0x43, 0x01,
0xcf, 0x0b, 0xd2, 0x4f, 0x2f, 0x14, 0xa4, 0x79, 0x4b, 0x15, 0x51, 0xd7, 0x09, 0x20, 0x89, 0xa0,
0xfe, 0xa1, 0x04, 0xb5, 0x18, 0x9d, 0xdf, 0x93, 0xce, 0x43, 0x4a, 0x1c, 0xdd, 0x89, 0x43, 0x58,
0x01, 0x83, 0x20, 0xed, 0xb1, 0x20, 0xf6, 0x2a, 0xd4, 0xd9, 0x75, 0x4c, 0x4c, 0x17, 0xf9, 0x74,
0x8d, 0x11, 0xf8, 0xe4, 0x6b, 0xd0, 0xa0, 0x1e, 0x35, 0xa6, 0x3a, 0xe5, 0x19, 0xbb, 0x24, 0xb8,
0x39, 0x89, 0xe7, 0x6b, 0xf4, 0x16, 0x5c, 0xa3, 0x93, 0xc0, 0xa3, 0x74, 0xca, 0xaa, 0x38, 0x5e,
0xb7, 0x88, 0x32, 0xa3, 0x8c, 0x95, 0x64, 0x42, 0xd4, 0x33, 0x21, 0x8b, 0xde, 0xb3, 0xc5, 0xcc,
0x75, 0x79, 0x10, 0x29, 0xe3, 0xd5, 0x84, 0xca, 0x5c, 0x1b, 0xb5, 0x60, 0xd9, 0x17, 0x35, 0x01,
0x8f, 0x15, 0x05, 0x1c, 0x0f, 0x91, 0x0e, 0x6b, 0x0e, 0x31, 0xc2, 0x28, 0x20, 0x96, 0x7e, 0x6c,
0x93, 0xa9, 0x25, 0xae, 0xb7, 0xcd, 0xdc, 0x45, 0x76, 0xac, 0x96, 0xce, 0x43, 0xce, 0x8d, 0x9b,
0x31, 0x9c, 0x18, 0xb3, 0xfa, 0x40, 0xfc, 0x43, 0x6b, 0xd0, 0x18, 0x3e, 0x19, 0x8e, 0xb4, 0x3d,
0x7d, 0x6f, 0xbf, 0xaf, 0xc9, 0x4e, 0xc7, 0x50, 0xc3, 0x62, 0x58, 0x60, 0xf3, 0xa3, 0xfd, 0x51,
0x77, 0x57, 0x1f, 0xed, 0xf4, 0x1e, 0x0d, 0x95, 0x22, 0xba, 0x09, 0xd7, 0x46, 0xdb, 0x78, 0x7f,
0x34, 0xda, 0xd5, 0xfa, 0xfa, 0x81, 0x86, 0x77, 0xf6, 0xfb, 0x43, 0xa5, 0x84, 0x10, 0x34, 0x67,
0xe4, 0xd1, 0xce, 0x9e, 0xa6, 0x94, 0x51, 0x03, 0x96, 0x0f, 0x34, 0xdc, 0xd3, 0x06, 0x23, 0xa5,
0xa2, 0xfe, 0xb5, 0x08, 0x8d, 0x94, 0x15, 0x99, 0x23, 0x07, 0xa1, 0xa8, 0xe6, 0xcb, 0x98, 0xfd,
0x65, 0xc1, 0xc4, 0x34, 0xcc, 0x89, 0xb0, 0x4e, 0x19, 0x8b, 0x01, 0xaf, 0xe0, 0x8d, 0xb3, 0xd4,
0x39, 0x2f, 0xe3, 0x9a, 0x63, 0x9c, 0x09, 0x90, 0xd7, 0x61, 0xe5, 0x84, 0x04, 0x2e, 0x99, 0xca,
0x79, 0x61, 0x91, 0x86, 0xa0, 0x89, 0x25, 0xeb, 0xa0, 0xc8, 0x25, 0x33, 0x18, 0x61, 0x8e, 0xa6,
0xa0, 0xef, 0xc5, 0x60, 0x47, 0x97, 0xb5, 0x5e, 0xe5, 0x5a, 0xbf, 0x3f, 0xbf, 0x93, 0x3e, 0x4b,
0xf1, 0xc3, 0x44, 0xf1, 0xcb, 0x50, 0xc2, 0xf1, 0xa3, 0x7f, 0xaf, 0xdb, 0xdb, 0x66, 0xca, 0x5e,
0x85, 0xfa, 0x5e, 0xf7, 0x33, 0xfd, 0x70, 0xc8, 0x9f, 0x90, 0x90, 0x02, 0x2b, 0x8f, 0x34, 0x3c,
0xd0, 0x76, 0x25, 0xa5, 0x84, 0x6e, 0x80, 0x22, 0x29, 0xb3, 0x75, 0x65, 0xf5, 0x1f, 0x45, 0x58,
0x13, 0x71, 0x3d, 0x79, 0xd5, 0x7c, 0xf6, 0xf3, 0x62, 0xfa, 0xb2, 0x5f, 0xcc, 0x5c, 0xf6, 0x93,
0x5a, 0x91, 0xa7, 0xe5, 0xd2, 0xac, 0x56, 0xe4, 0x8f, 0x04, 0x99, 0x90, 0x5d, 0x9e, 0x27, 0x64,
0xb7, 0x60, 0xd9, 0x21, 0x61, 0xa2, 0xf8, 0x3a, 0x8e, 0x87, 0xc8, 0x86, 0x86, 0xe1, 0xba, 0x1e,
0xe5, 0x4f, 0x6a, 0xf1, 0xed, 0x65, 0x6b, 0xae, 0xc7, 0xbb, 0xe4, 0x8b, 0x3b, 0xdd, 0x19, 0x92,
0x88, 0xac, 0x69, 0xec, 0xf6, 0x87, 0xa0, 0x5c, 0x5c, 0x30, 0x4f, 0x3e, 0x7b, 0xf3, 0xdd, 0x59,
0x3a, 0x23, 0xcc, 0xb1, 0x0f, 0x07, 0x8f, 0x06, 0xfb, 0x8f, 0x07, 0xca, 0x12, 0x1b, 0xe0, 0xc3,
0xc1, 0x60, 0x67, 0xb0, 0xa5, 0x14, 0x10, 0x40, 0x55, 0xfb, 0x6c, 0x67, 0xa4, 0xf5, 0x95, 0xe2,
0xe6, 0xdf, 0x57, 0xa1, 0x2a, 0x84, 0x44, 0x5f, 0xcb, 0x54, 0x9e, 0xee, 0x72, 0xa3, 0x0f, 0xe7,
0x2e, 0x89, 0x33, 0x9d, 0xf3, 0xf6, 0x47, 0x0b, 0xf3, 0xcb, 0x57, 0xe9, 0x25, 0xf4, 0xab, 0x02,
0xac, 0x64, 0x9e, 0x61, 0xf3, 0xbe, 0x20, 0x5e, 0xd1, 0x54, 0x6f, 0x7f, 0x67, 0x21, 0xde, 0x44,
0x96, 0x5f, 0x16, 0xa0, 0x91, 0x6a, 0x27, 0xa3, 0xfb, 0x8b, 0xb4, 0xa0, 0x85, 0x24, 0x0f, 0x16,
0xef, 0x5e, 0xab, 0x4b, 0xef, 0x14, 0xd0, 0x2f, 0x0a, 0xd0, 0x48, 0x35, 0x56, 0x73, 0x8b, 0x72,
0xb9, 0x0d, 0x9c, 0x5b, 0x94, 0xab, 0xfa, 0xb8, 0x4b, 0xe8, 0xa7, 0x05, 0xa8, 0x27, 0x4d, 0x52,
0x74, 0x77, 0xfe, 0xb6, 0xaa, 0x10, 0xe2, 0xde, 0xa2, 0xfd, 0x58, 0x75, 0x09, 0xfd, 0x18, 0x6a,
0x71, 0x47, 0x11, 0xe5, 0x4d, 0x3f, 0x17, 0xda, 0x95, 0xed, 0xbb, 0x73, 0xf3, 0xa5, 0xb7, 0x8f,
0xdb, 0x7c, 0xb9, 0xb7, 0xbf, 0xd0, 0x90, 0x6c, 0xdf, 0x9d, 0x9b, 0x2f, 0xd9, 0x9e, 0x79, 0x42,
0xaa, 0x1b, 0x98, 0xdb, 0x13, 0x2e, 0xb7, 0x21, 0x73, 0x7b, 0xc2, 0x55, 0xcd, 0x47, 0x21, 0x48,
0xaa, 0x9f, 0x98, 0x5b, 0x90, 0xcb, 0x3d, 0xcb, 0xdc, 0x82, 0x5c, 0xd1, 0xbe, 0x94, 0x2e, 0x39,
0x2b, 0xec, 0xef, 0xce, 0xdd, 0x82, 0x9b, 0xd3, 0x25, 0x2f, 0x35, 0x01, 0xd5, 0x25, 0xf4, 0x33,
0xf9, 0xd4, 0x20, 0xfa, 0x77, 0x68, 0x1e, 0xa8, 0x4c, 0xcb, 0xaf, 0x7d, 0x67, 0xb1, 0x54, 0xc3,
0x63, 0xc4, 0xcf, 0x0b, 0x00, 0xb3, 0x4e, 0x5f, 0x6e, 0x21, 0x2e, 0xb5, 0x18, 0xdb, 0xf7, 0x17,
0xe0, 0x4c, 0x1f, 0x8f, 0xb8, 0xb9, 0x97, 0xfb, 0x78, 0x5c, 0xe8, 0x44, 0xe6, 0x3e, 0x1e, 0x17,
0xbb, 0x88, 0xea, 0xd2, 0x27, 0xcb, 0xdf, 0xab, 0x88, 0xdc, 0x5f, 0xe5, 0x3f, 0xef, 0xfd, 0x27,
0x00, 0x00, 0xff, 0xff, 0x33, 0x61, 0xdc, 0x03, 0x00, 0x27, 0x00, 0x00,
var fileDescriptor_driver_5a41b73816b77fc1 = []byte{
// 2958 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x59, 0xcd, 0x6f, 0x23, 0xc7,
0xb1, 0x17, 0x49, 0x91, 0x22, 0x8b, 0x12, 0x35, 0xdb, 0xbb, 0x6b, 0xd3, 0x34, 0xde, 0xf3, 0x7a,
0x00, 0x3f, 0x08, 0xb6, 0x97, 0xb2, 0x65, 0xbc, 0xfd, 0x7a, 0xfe, 0xa2, 0xa9, 0x59, 0x49, 0x5e,
0x89, 0xd2, 0x6b, 0x52, 0x58, 0x6f, 0x12, 0x7b, 0x32, 0x9a, 0x69, 0x91, 0xb3, 0xe2, 0x7c, 0xb8,
0xa7, 0x47, 0x96, 0x10, 0x04, 0x09, 0x1c, 0x20, 0x48, 0x0e, 0x01, 0x72, 0x31, 0x72, 0x4f, 0x8e,
0xf9, 0x0b, 0x92, 0xc0, 0x97, 0xfc, 0x15, 0x01, 0x72, 0x4a, 0x80, 0x00, 0xb9, 0xe6, 0x3f, 0x08,
0xfa, 0x63, 0x86, 0x43, 0x49, 0xeb, 0x1d, 0x72, 0x73, 0x22, 0xbb, 0xba, 0xeb, 0xd7, 0x35, 0x55,
0xd5, 0x55, 0xd5, 0x5d, 0xa0, 0x87, 0xe3, 0x78, 0xe8, 0xfa, 0xd1, 0xba, 0x43, 0xdd, 0x53, 0x42,
0xa3, 0xf5, 0x90, 0x06, 0x2c, 0x50, 0xa3, 0xb6, 0x18, 0xa0, 0x37, 0x46, 0x56, 0x34, 0x72, 0xed,
0x80, 0x86, 0x6d, 0x3f, 0xf0, 0x2c, 0xa7, 0xad, 0x78, 0xda, 0x8a, 0x47, 0x2e, 0x6b, 0xfd, 0xf7,
0x30, 0x08, 0x86, 0x63, 0x22, 0x11, 0x8e, 0xe2, 0xe3, 0x75, 0x27, 0xa6, 0x16, 0x73, 0x03, 0x5f,
0xcd, 0xbf, 0x76, 0x71, 0x9e, 0xb9, 0x1e, 0x89, 0x98, 0xe5, 0x85, 0x6a, 0xc1, 0xc7, 0x43, 0x97,
0x8d, 0xe2, 0xa3, 0xb6, 0x1d, 0x78, 0xeb, 0xe9, 0x96, 0xeb, 0x62, 0xcb, 0xf5, 0x44, 0xcc, 0x68,
0x64, 0x51, 0xe2, 0xac, 0x8f, 0xec, 0x71, 0x14, 0x12, 0x9b, 0xff, 0x9a, 0xfc, 0x8f, 0x42, 0xd8,
0xca, 0x8f, 0x10, 0x31, 0x1a, 0xdb, 0x2c, 0xf9, 0x5e, 0x8b, 0x31, 0xea, 0x1e, 0xc5, 0x8c, 0x48,
0x20, 0xfd, 0x15, 0x78, 0x79, 0x60, 0x45, 0x27, 0xdd, 0xc0, 0x3f, 0x76, 0x87, 0x7d, 0x7b, 0x44,
0x3c, 0x0b, 0x93, 0x2f, 0x63, 0x12, 0x31, 0xfd, 0x07, 0xd0, 0xbc, 0x3c, 0x15, 0x85, 0x81, 0x1f,
0x11, 0xf4, 0x31, 0x2c, 0x72, 0x69, 0x9a, 0x85, 0x5b, 0x85, 0xb5, 0xfa, 0xc6, 0xdb, 0xed, 0x67,
0x29, 0x4e, 0xca, 0xd0, 0x56, 0x5f, 0xd1, 0xee, 0x87, 0xc4, 0xc6, 0x82, 0x53, 0xbf, 0x09, 0xd7,
0xbb, 0x56, 0x68, 0x1d, 0xb9, 0x63, 0x97, 0xb9, 0x24, 0x4a, 0x36, 0x8d, 0xe1, 0xc6, 0x34, 0x59,
0x6d, 0xf8, 0x39, 0x2c, 0xdb, 0x19, 0xba, 0xda, 0xf8, 0x7e, 0x3b, 0x97, 0xc5, 0xda, 0x9b, 0x62,
0x34, 0x05, 0x3c, 0x05, 0xa7, 0xdf, 0x00, 0xf4, 0xd0, 0xf5, 0x87, 0x84, 0x86, 0xd4, 0xf5, 0x59,
0x22, 0xcc, 0xb7, 0x25, 0xb8, 0x3e, 0x45, 0x56, 0xc2, 0x3c, 0x05, 0x48, 0xf5, 0xc8, 0x45, 0x29,
0xad, 0xd5, 0x37, 0x3e, 0xcd, 0x29, 0xca, 0x15, 0x78, 0xed, 0x4e, 0x0a, 0x66, 0xf8, 0x8c, 0x9e,
0xe3, 0x0c, 0x3a, 0xfa, 0x02, 0x2a, 0x23, 0x62, 0x8d, 0xd9, 0xa8, 0x59, 0xbc, 0x55, 0x58, 0x6b,
0x6c, 0x3c, 0x7c, 0x81, 0x7d, 0xb6, 0x05, 0x50, 0x9f, 0x59, 0x8c, 0x60, 0x85, 0x8a, 0x6e, 0x03,
0x92, 0xff, 0x4c, 0x87, 0x44, 0x36, 0x75, 0x43, 0xee, 0xc8, 0xcd, 0xd2, 0xad, 0xc2, 0x5a, 0x0d,
0x5f, 0x93, 0x33, 0x9b, 0x93, 0x89, 0x56, 0x08, 0xab, 0x17, 0xa4, 0x45, 0x1a, 0x94, 0x4e, 0xc8,
0xb9, 0xb0, 0x48, 0x0d, 0xf3, 0xbf, 0x68, 0x0b, 0xca, 0xa7, 0xd6, 0x38, 0x26, 0x42, 0xe4, 0xfa,
0xc6, 0xbb, 0xcf, 0x73, 0x0f, 0xe5, 0xa2, 0x13, 0x3d, 0x60, 0xc9, 0xff, 0xa0, 0x78, 0xaf, 0xa0,
0xdf, 0x87, 0x7a, 0x46, 0x6e, 0xd4, 0x00, 0x38, 0xec, 0x6d, 0x1a, 0x03, 0xa3, 0x3b, 0x30, 0x36,
0xb5, 0x05, 0xb4, 0x02, 0xb5, 0xc3, 0xde, 0xb6, 0xd1, 0xd9, 0x1d, 0x6c, 0x3f, 0xd1, 0x0a, 0xa8,
0x0e, 0x4b, 0xc9, 0xa0, 0xa8, 0x9f, 0x01, 0xc2, 0xc4, 0x0e, 0x4e, 0x09, 0xe5, 0x8e, 0xac, 0xac,
0x8a, 0x5e, 0x86, 0x25, 0x66, 0x45, 0x27, 0xa6, 0xeb, 0x28, 0x99, 0x2b, 0x7c, 0xb8, 0xe3, 0xa0,
0x1d, 0xa8, 0x8c, 0x2c, 0xdf, 0x19, 0x3f, 0x5f, 0xee, 0x69, 0x55, 0x73, 0xf0, 0x6d, 0xc1, 0x88,
0x15, 0x00, 0xf7, 0xee, 0xa9, 0x9d, 0xa5, 0x01, 0xf4, 0x27, 0xa0, 0xf5, 0x99, 0x45, 0x59, 0x56,
0x1c, 0x03, 0x16, 0xf9, 0xfe, 0xca, 0xa3, 0x67, 0xd9, 0x53, 0x9e, 0x4c, 0x2c, 0xd8, 0xf5, 0x7f,
0x15, 0xe1, 0x5a, 0x06, 0x5b, 0x79, 0xea, 0x63, 0xa8, 0x50, 0x12, 0xc5, 0x63, 0x26, 0xe0, 0x1b,
0x1b, 0x1f, 0xe5, 0x84, 0xbf, 0x84, 0xd4, 0xc6, 0x02, 0x06, 0x2b, 0x38, 0xb4, 0x06, 0x9a, 0xe4,
0x30, 0x09, 0xa5, 0x01, 0x35, 0xbd, 0x68, 0x28, 0xb4, 0x56, 0xc3, 0x0d, 0x49, 0x37, 0x38, 0x79,
0x2f, 0x1a, 0x66, 0xb4, 0x5a, 0x7a, 0x41, 0xad, 0x22, 0x0b, 0x34, 0x9f, 0xb0, 0xaf, 0x02, 0x7a,
0x62, 0x72, 0xd5, 0x52, 0xd7, 0x21, 0xcd, 0x45, 0x01, 0x7a, 0x27, 0x27, 0x68, 0x4f, 0xb2, 0xef,
0x2b, 0x6e, 0xbc, 0xea, 0x4f, 0x13, 0xf4, 0xb7, 0xa0, 0x22, 0xbf, 0x94, 0x7b, 0x52, 0xff, 0xb0,
0xdb, 0x35, 0xfa, 0x7d, 0x6d, 0x01, 0xd5, 0xa0, 0x8c, 0x8d, 0x01, 0xe6, 0x1e, 0x56, 0x83, 0xf2,
0xc3, 0xce, 0xa0, 0xb3, 0xab, 0x15, 0xf5, 0x37, 0x61, 0xf5, 0xb1, 0xe5, 0xb2, 0x3c, 0xce, 0xa5,
0x07, 0xa0, 0x4d, 0xd6, 0x2a, 0xeb, 0xec, 0x4c, 0x59, 0x27, 0xbf, 0x6a, 0x8c, 0x33, 0x97, 0x5d,
0xb0, 0x87, 0x06, 0x25, 0x42, 0xa9, 0x32, 0x01, 0xff, 0xab, 0x7f, 0x05, 0xab, 0x7d, 0x16, 0x84,
0xb9, 0x3c, 0xff, 0x3d, 0x58, 0xe2, 0x39, 0x2a, 0x88, 0x99, 0x72, 0xfd, 0x57, 0xda, 0x32, 0x87,
0xb5, 0x93, 0x1c, 0xd6, 0xde, 0x54, 0x39, 0x0e, 0x27, 0x2b, 0xd1, 0x4b, 0x50, 0x89, 0xdc, 0xa1,
0x6f, 0x8d, 0x55, 0xb4, 0x50, 0x23, 0x1d, 0x71, 0x27, 0x4f, 0x36, 0x56, 0x8e, 0xdf, 0x05, 0xb4,
0x49, 0x22, 0x46, 0x83, 0xf3, 0x5c, 0xf2, 0xdc, 0x80, 0xf2, 0x71, 0x40, 0x6d, 0x79, 0x10, 0xab,
0x58, 0x0e, 0xf8, 0xa1, 0x9a, 0x02, 0x51, 0xd8, 0xb7, 0x01, 0xed, 0xf8, 0x3c, 0xa7, 0xe4, 0x33,
0xc4, 0xaf, 0x8b, 0x70, 0x7d, 0x6a, 0xbd, 0x32, 0xc6, 0xfc, 0xe7, 0x90, 0x07, 0xa6, 0x38, 0x92,
0xe7, 0x10, 0xed, 0x43, 0x45, 0xae, 0x50, 0x9a, 0xbc, 0x3b, 0x03, 0x90, 0x4c, 0x53, 0x0a, 0x4e,
0xc1, 0x5c, 0xe9, 0xf4, 0xa5, 0xff, 0xac, 0xd3, 0x6f, 0x81, 0x96, 0x7c, 0x47, 0xf4, 0x5c, 0xdb,
0xb4, 0xa0, 0xea, 0xfa, 0x8c, 0xd0, 0x53, 0x6b, 0x2c, 0x3e, 0xb1, 0x84, 0xd3, 0xb1, 0xfe, 0x7d,
0xb8, 0x96, 0x01, 0x52, 0x8a, 0x7d, 0x08, 0xe5, 0x88, 0x13, 0x94, 0x66, 0xdf, 0x99, 0x51, 0xb3,
0x11, 0x96, 0xec, 0xfa, 0x75, 0x09, 0x6e, 0x9c, 0x12, 0x3f, 0x15, 0x53, 0xdf, 0x84, 0x6b, 0x7d,
0xe1, 0x76, 0xb9, 0xfc, 0x6a, 0xe2, 0xb2, 0xc5, 0x29, 0x97, 0xbd, 0x01, 0x28, 0x8b, 0xa2, 0x1c,
0xeb, 0x1c, 0x56, 0x8d, 0x33, 0x62, 0xe7, 0x42, 0x6e, 0xc2, 0x92, 0x1d, 0x78, 0x9e, 0xe5, 0x3b,
0xcd, 0xe2, 0xad, 0xd2, 0x5a, 0x0d, 0x27, 0xc3, 0xec, 0xd9, 0x2a, 0xe5, 0x3d, 0x5b, 0xfa, 0xaf,
0x0a, 0xa0, 0x4d, 0xf6, 0x56, 0x8a, 0xe4, 0xd2, 0x33, 0x87, 0x03, 0xf1, 0xbd, 0x97, 0xb1, 0x1a,
0x29, 0x7a, 0x72, 0xfc, 0x25, 0x9d, 0x50, 0x9a, 0x09, 0x2f, 0xa5, 0x17, 0x0c, 0x2f, 0xfa, 0x3f,
0x0a, 0x80, 0x2e, 0x17, 0x51, 0xe8, 0x75, 0x58, 0x8e, 0x88, 0xef, 0x98, 0x52, 0x8d, 0xd2, 0xc2,
0x55, 0x5c, 0xe7, 0x34, 0xa9, 0xcf, 0x08, 0x21, 0x58, 0x24, 0x67, 0xc4, 0x56, 0x27, 0x59, 0xfc,
0x47, 0x23, 0x58, 0x3e, 0x8e, 0x4c, 0x37, 0x0a, 0xc6, 0x56, 0x5a, 0x6d, 0x34, 0x36, 0x8c, 0xb9,
0x8b, 0xb9, 0xf6, 0xc3, 0xfe, 0x4e, 0x02, 0x86, 0xeb, 0xc7, 0x51, 0x3a, 0xd0, 0xdb, 0x50, 0xcf,
0xcc, 0xa1, 0x2a, 0x2c, 0xf6, 0xf6, 0x7b, 0x86, 0xb6, 0x80, 0x00, 0x2a, 0xdd, 0x6d, 0xbc, 0xbf,
0x3f, 0x90, 0x11, 0x7d, 0x67, 0xaf, 0xb3, 0x65, 0x68, 0x45, 0xfd, 0x0f, 0x15, 0x80, 0x49, 0x6a,
0x45, 0x0d, 0x28, 0xa6, 0x96, 0x2e, 0xba, 0x0e, 0xff, 0x18, 0xdf, 0xf2, 0x88, 0xf2, 0x1e, 0xf1,
0x1f, 0x6d, 0xc0, 0x4d, 0x2f, 0x1a, 0x86, 0x96, 0x7d, 0x62, 0xaa, 0x8c, 0x68, 0x0b, 0x66, 0xf1,
0x55, 0xcb, 0xf8, 0xba, 0x9a, 0x54, 0x52, 0x4b, 0xdc, 0x5d, 0x28, 0x11, 0xff, 0xb4, 0xb9, 0x28,
0x2a, 0xc7, 0x07, 0x33, 0xa7, 0xfc, 0xb6, 0xe1, 0x9f, 0xca, 0x4a, 0x91, 0xc3, 0x20, 0x13, 0xc0,
0x21, 0xa7, 0xae, 0x4d, 0x4c, 0x0e, 0x5a, 0x16, 0xa0, 0x1f, 0xcf, 0x0e, 0xba, 0x29, 0x30, 0x52,
0xe8, 0x9a, 0x93, 0x8c, 0x51, 0x0f, 0x6a, 0x94, 0x44, 0x41, 0x4c, 0x6d, 0x12, 0x35, 0x2b, 0x33,
0x9d, 0x62, 0x9c, 0xf0, 0xe1, 0x09, 0x04, 0xda, 0x84, 0x8a, 0x17, 0xc4, 0x3e, 0x8b, 0x9a, 0x4b,
0x42, 0xd8, 0xb7, 0x73, 0x82, 0xed, 0x71, 0x26, 0xac, 0x78, 0xd1, 0x16, 0x2c, 0x49, 0x11, 0xa3,
0x66, 0x55, 0xc0, 0xdc, 0xce, 0xeb, 0x40, 0x82, 0x0b, 0x27, 0xdc, 0xdc, 0xaa, 0x71, 0x44, 0x68,
0xb3, 0x26, 0xad, 0xca, 0xff, 0xa3, 0x57, 0xa1, 0x66, 0x8d, 0xc7, 0x81, 0x6d, 0x3a, 0x2e, 0x6d,
0x82, 0x98, 0xa8, 0x0a, 0xc2, 0xa6, 0x4b, 0xd1, 0x6b, 0x50, 0x97, 0x47, 0xcf, 0x0c, 0x2d, 0x36,
0x6a, 0xd6, 0xc5, 0x34, 0x48, 0xd2, 0x81, 0xc5, 0x46, 0x6a, 0x01, 0xa1, 0x54, 0x2e, 0x58, 0x4e,
0x17, 0x10, 0x4a, 0xc5, 0x82, 0xff, 0x81, 0x55, 0x11, 0x47, 0x86, 0x34, 0x88, 0x43, 0x53, 0xf8,
0xd4, 0x8a, 0x58, 0xb4, 0xc2, 0xc9, 0x5b, 0x9c, 0xda, 0xe3, 0xce, 0xf5, 0x0a, 0x54, 0x9f, 0x06,
0x47, 0x72, 0x41, 0x43, 0x2c, 0x58, 0x7a, 0x1a, 0x1c, 0x25, 0x53, 0x52, 0x42, 0xd7, 0x69, 0xae,
0xca, 0x29, 0x31, 0xde, 0x71, 0x5a, 0x77, 0xa0, 0x9a, 0x98, 0xf1, 0x8a, 0xea, 0xfc, 0x46, 0xb6,
0x3a, 0xaf, 0x65, 0x4a, 0xed, 0xd6, 0xfb, 0xd0, 0x98, 0x76, 0x82, 0x59, 0xb8, 0xf5, 0xbf, 0x16,
0xa0, 0x96, 0x9a, 0x1b, 0xf9, 0x70, 0x5d, 0x88, 0x63, 0x31, 0xe2, 0x98, 0x13, 0xef, 0x91, 0x39,
0xe0, 0x83, 0x9c, 0x96, 0xea, 0x24, 0x08, 0x2a, 0x0e, 0x2a, 0x57, 0x42, 0x29, 0xf2, 0x64, 0xbf,
0x2f, 0x60, 0x75, 0xec, 0xfa, 0xf1, 0x59, 0x66, 0x2f, 0x99, 0x80, 0xff, 0x37, 0xe7, 0x5e, 0xbb,
0x9c, 0x7b, 0xb2, 0x47, 0x63, 0x3c, 0x35, 0xd6, 0xbf, 0x29, 0xc2, 0x4b, 0x57, 0x8b, 0x83, 0x7a,
0x50, 0xb2, 0xc3, 0x58, 0x7d, 0xda, 0xfb, 0xb3, 0x7e, 0x5a, 0x37, 0x8c, 0x27, 0xbb, 0x72, 0x20,
0x5e, 0xb4, 0x7b, 0xc4, 0x0b, 0xe8, 0xb9, 0xfa, 0x82, 0x8f, 0x66, 0x85, 0xdc, 0x13, 0xdc, 0x13,
0x54, 0x05, 0x87, 0x30, 0x54, 0x55, 0xea, 0x8f, 0x54, 0x98, 0x98, 0xb1, 0x84, 0x48, 0x20, 0x71,
0x8a, 0xa3, 0xdf, 0x81, 0x9b, 0x57, 0x7e, 0x0a, 0xfa, 0x2f, 0x00, 0x3b, 0x8c, 0x4d, 0x71, 0xc5,
0x93, 0x76, 0x2f, 0xe1, 0x9a, 0x1d, 0xc6, 0x7d, 0x41, 0xd0, 0xef, 0x42, 0xf3, 0x59, 0xf2, 0xf2,
0xc3, 0x27, 0x25, 0x36, 0xbd, 0xa3, 0xa4, 0xc6, 0x90, 0x84, 0xbd, 0x23, 0xfd, 0x37, 0x45, 0x58,
0xbd, 0x20, 0x0e, 0xcf, 0x80, 0xf2, 0x30, 0x27, 0x59, 0x59, 0x8e, 0xf8, 0xc9, 0xb6, 0x5d, 0x27,
0x29, 0x8b, 0xc5, 0x7f, 0x11, 0xd3, 0x43, 0x55, 0xb2, 0x16, 0xdd, 0x90, 0x3b, 0xb4, 0x77, 0xe4,
0xb2, 0x48, 0xdc, 0x24, 0xca, 0x58, 0x0e, 0xd0, 0x13, 0x68, 0x50, 0x12, 0x11, 0x7a, 0x4a, 0x1c,
0x33, 0x0c, 0x28, 0x4b, 0x14, 0xb6, 0x31, 0x9b, 0xc2, 0x0e, 0x02, 0xca, 0xf0, 0x4a, 0x82, 0xc4,
0x47, 0x11, 0x7a, 0x0c, 0x2b, 0xce, 0xb9, 0x6f, 0x79, 0xae, 0xad, 0x90, 0x2b, 0x73, 0x23, 0x2f,
0x2b, 0x20, 0x01, 0xcc, 0x6f, 0xca, 0x99, 0x49, 0xfe, 0x61, 0x63, 0xeb, 0x88, 0x8c, 0x95, 0x4e,
0xe4, 0x60, 0xfa, 0xfc, 0x96, 0xd5, 0xf9, 0xd5, 0x7f, 0x57, 0x84, 0xc6, 0xf4, 0x01, 0x48, 0xec,
0x17, 0x12, 0xea, 0x06, 0x4e, 0xc6, 0x7e, 0x07, 0x82, 0xc0, 0x6d, 0xc4, 0xa7, 0xbf, 0x8c, 0x03,
0x66, 0x25, 0x36, 0xb2, 0xc3, 0xf8, 0xff, 0xf9, 0xf8, 0x82, 0xed, 0x4b, 0x17, 0x6c, 0x8f, 0xde,
0x06, 0xa4, 0xec, 0x3b, 0x76, 0x3d, 0x97, 0x99, 0x47, 0xe7, 0x8c, 0x48, 0xfd, 0x97, 0xb0, 0x26,
0x67, 0x76, 0xf9, 0xc4, 0x27, 0x9c, 0x8e, 0x74, 0x58, 0x09, 0x02, 0xcf, 0x8c, 0xec, 0x80, 0x12,
0xd3, 0x72, 0x9e, 0x36, 0xcb, 0x62, 0x61, 0x3d, 0x08, 0xbc, 0x3e, 0xa7, 0x75, 0x9c, 0xa7, 0x3c,
0xe0, 0xda, 0x61, 0x1c, 0x11, 0x66, 0xf2, 0x1f, 0x91, 0xa3, 0x6a, 0x18, 0x24, 0xa9, 0x1b, 0xc6,
0x51, 0x66, 0x81, 0x47, 0x3c, 0x9e, 0x77, 0x32, 0x0b, 0xf6, 0x88, 0xc7, 0x77, 0x59, 0x3e, 0x20,
0xd4, 0x26, 0x3e, 0x1b, 0xb8, 0xf6, 0x09, 0x4f, 0x29, 0x85, 0xb5, 0x02, 0x9e, 0xa2, 0xe9, 0x9f,
0x43, 0x59, 0xa4, 0x20, 0xfe, 0xf1, 0x22, 0x7c, 0x8b, 0xe8, 0x2e, 0xd5, 0x5b, 0xe5, 0x04, 0x11,
0xdb, 0x5f, 0x85, 0xda, 0x28, 0x88, 0x54, 0x6e, 0x90, 0x9e, 0x57, 0xe5, 0x04, 0x31, 0xd9, 0x82,
0x2a, 0x25, 0x96, 0x13, 0xf8, 0xe3, 0x73, 0xa1, 0x97, 0x2a, 0x4e, 0xc7, 0xfa, 0x97, 0x50, 0x91,
0xe1, 0xf7, 0x05, 0xf0, 0x6f, 0x03, 0xb2, 0x65, 0x52, 0x09, 0x09, 0xf5, 0xdc, 0x28, 0x72, 0x03,
0x3f, 0x4a, 0x9e, 0x73, 0xe4, 0xcc, 0xc1, 0x64, 0x42, 0xff, 0x73, 0x41, 0xd6, 0x3b, 0xf2, 0xa2,
0xcd, 0x2b, 0x46, 0x55, 0xbc, 0xcc, 0xfd, 0x1a, 0xa1, 0x00, 0x92, 0xaa, 0x9f, 0xa8, 0x67, 0xab,
0x59, 0xab, 0x7e, 0x22, 0xab, 0x7e, 0xc2, 0x4b, 0x4c, 0x55, 0x56, 0x49, 0x38, 0x59, 0x55, 0xd5,
0x9d, 0xf4, 0xaa, 0x44, 0xf4, 0x7f, 0x16, 0xd2, 0x88, 0x90, 0x5c, 0x69, 0xd0, 0x17, 0x50, 0xe5,
0x87, 0xcb, 0xf4, 0xac, 0x50, 0x3d, 0xd0, 0x75, 0xe7, 0xbb, 0x2d, 0xb5, 0xf9, 0x59, 0xda, 0xb3,
0x42, 0x59, 0x14, 0x2d, 0x85, 0x72, 0xc4, 0x23, 0x8b, 0xe5, 0x4c, 0x22, 0x0b, 0xff, 0x8f, 0xde,
0x80, 0x86, 0x15, 0xb3, 0xc0, 0xb4, 0x9c, 0x53, 0x42, 0x99, 0x1b, 0x11, 0x65, 0xe1, 0x15, 0x4e,
0xed, 0x24, 0xc4, 0xd6, 0x03, 0x58, 0xce, 0x62, 0x3e, 0x2f, 0xc7, 0x96, 0xb3, 0x39, 0xf6, 0x87,
0x00, 0x93, 0xea, 0x9c, 0x7b, 0x02, 0x39, 0x73, 0x99, 0x69, 0x07, 0x8e, 0x8c, 0x7c, 0x65, 0x5c,
0xe5, 0x84, 0x6e, 0xe0, 0x90, 0x0b, 0x77, 0x9d, 0x72, 0x72, 0xd7, 0xe1, 0x67, 0x93, 0x1f, 0xa7,
0x13, 0x77, 0x3c, 0x26, 0x8e, 0x92, 0xb0, 0x16, 0x04, 0xde, 0x23, 0x41, 0xd0, 0xbf, 0x2d, 0x4a,
0x8f, 0x90, 0xb7, 0xd0, 0x5c, 0x15, 0x70, 0x6a, 0xea, 0xd2, 0x8b, 0x99, 0xfa, 0x3e, 0x40, 0xc4,
0x2c, 0xca, 0x0b, 0x06, 0x8b, 0xa9, 0x87, 0x9d, 0xd6, 0xa5, 0xcb, 0xd2, 0x20, 0x79, 0x4c, 0xc7,
0x35, 0xb5, 0xba, 0xc3, 0xd0, 0x07, 0xb0, 0x6c, 0x07, 0x5e, 0x38, 0x26, 0x8a, 0xb9, 0xfc, 0x5c,
0xe6, 0x7a, 0xba, 0xbe, 0xc3, 0x32, 0x37, 0xa5, 0xca, 0x8b, 0xde, 0x94, 0xfe, 0x58, 0x90, 0x97,
0xe9, 0xec, 0x5d, 0x1e, 0x0d, 0xaf, 0x78, 0x30, 0xde, 0x9a, 0xf3, 0x61, 0xe0, 0xbb, 0x5e, 0x8b,
0x5b, 0x1f, 0xe4, 0x79, 0x9e, 0x7d, 0x76, 0x09, 0xf7, 0xa7, 0x12, 0xd4, 0xd2, 0x7b, 0xf7, 0x25,
0xdb, 0xdf, 0x83, 0x5a, 0xda, 0xc9, 0x50, 0xa5, 0xc9, 0x77, 0x9a, 0x27, 0x5d, 0x8c, 0x8e, 0x01,
0x59, 0xc3, 0x61, 0x5a, 0x9a, 0x99, 0x71, 0x64, 0x0d, 0x93, 0x57, 0x8c, 0x7b, 0x33, 0xe8, 0x21,
0xc9, 0x4e, 0x87, 0x9c, 0x1f, 0x6b, 0xd6, 0x70, 0x38, 0x45, 0x41, 0x3f, 0x82, 0x9b, 0xd3, 0x7b,
0x98, 0x47, 0xe7, 0x66, 0xe8, 0x3a, 0xea, 0xa6, 0xb5, 0x3d, 0xeb, 0xd3, 0x43, 0x7b, 0x0a, 0xfe,
0x93, 0xf3, 0x03, 0xd7, 0x91, 0x3a, 0x47, 0xf4, 0xd2, 0x44, 0xeb, 0x27, 0xf0, 0xf2, 0x33, 0x96,
0x5f, 0x61, 0x83, 0xde, 0xf4, 0x13, 0xf9, 0xfc, 0x4a, 0xc8, 0x58, 0xef, 0xb7, 0x05, 0xf9, 0x42,
0x32, 0xad, 0x93, 0x4e, 0xb6, 0x3a, 0x5d, 0xcf, 0xb9, 0x4f, 0xf7, 0xe0, 0x50, 0xc2, 0x8b, 0x82,
0xf4, 0xd3, 0x0b, 0x05, 0x69, 0xde, 0x52, 0x45, 0xd6, 0x75, 0x12, 0x48, 0x21, 0xe8, 0xbf, 0x2f,
0x41, 0x35, 0x41, 0x17, 0xf7, 0xa4, 0xf3, 0x88, 0x11, 0xcf, 0xf4, 0x92, 0x10, 0x56, 0xc0, 0x20,
0x49, 0x7b, 0x3c, 0x88, 0xbd, 0x0a, 0x35, 0x7e, 0x1d, 0x93, 0xd3, 0x45, 0x31, 0x5d, 0xe5, 0x04,
0x31, 0xf9, 0x1a, 0xd4, 0x59, 0xc0, 0xac, 0xb1, 0xc9, 0x44, 0xc6, 0x2e, 0x49, 0x6e, 0x41, 0x12,
0xf9, 0x1a, 0xbd, 0x05, 0xd7, 0xd8, 0x88, 0x06, 0x8c, 0x8d, 0x79, 0x15, 0x27, 0xea, 0x16, 0x59,
0x66, 0x2c, 0x62, 0x2d, 0x9d, 0x90, 0xf5, 0x4c, 0xc4, 0xa3, 0xf7, 0x64, 0x31, 0x77, 0x5d, 0x11,
0x44, 0x16, 0xf1, 0x4a, 0x4a, 0xe5, 0xae, 0x8d, 0x9a, 0xb0, 0x14, 0xca, 0x9a, 0x40, 0xc4, 0x8a,
0x02, 0x4e, 0x86, 0xc8, 0x84, 0x55, 0x8f, 0x58, 0x51, 0x4c, 0x89, 0x63, 0x1e, 0xbb, 0x64, 0xec,
0xc8, 0xeb, 0x6d, 0x23, 0x77, 0x91, 0x9d, 0xa8, 0xa5, 0xfd, 0x50, 0x70, 0xe3, 0x46, 0x02, 0x27,
0xc7, 0xbc, 0x3e, 0x90, 0xff, 0xd0, 0x2a, 0xd4, 0xfb, 0x4f, 0xfa, 0x03, 0x63, 0xcf, 0xdc, 0xdb,
0xdf, 0x34, 0x54, 0x17, 0xa4, 0x6f, 0x60, 0x39, 0x2c, 0xf0, 0xf9, 0xc1, 0xfe, 0xa0, 0xb3, 0x6b,
0x0e, 0x76, 0xba, 0x8f, 0xfa, 0x5a, 0x11, 0xdd, 0x84, 0x6b, 0x83, 0x6d, 0xbc, 0x3f, 0x18, 0xec,
0x1a, 0x9b, 0xe6, 0x81, 0x81, 0x77, 0xf6, 0x37, 0xfb, 0x5a, 0x09, 0x21, 0x68, 0x4c, 0xc8, 0x83,
0x9d, 0x3d, 0x43, 0x5b, 0x44, 0x75, 0x58, 0x3a, 0x30, 0x70, 0xd7, 0xe8, 0x0d, 0xb4, 0xb2, 0xfe,
0x97, 0x22, 0xd4, 0x33, 0x56, 0xe4, 0x8e, 0x4c, 0x23, 0x59, 0xcd, 0x2f, 0x62, 0xfe, 0x97, 0x07,
0x13, 0xdb, 0xb2, 0x47, 0xd2, 0x3a, 0x8b, 0x58, 0x0e, 0x44, 0x05, 0x6f, 0x9d, 0x65, 0xce, 0xf9,
0x22, 0xae, 0x7a, 0xd6, 0x99, 0x04, 0x79, 0x1d, 0x96, 0x4f, 0x08, 0xf5, 0xc9, 0x58, 0xcd, 0x4b,
0x8b, 0xd4, 0x25, 0x4d, 0x2e, 0x59, 0x03, 0x4d, 0x2d, 0x99, 0xc0, 0x48, 0x73, 0x34, 0x24, 0x7d,
0x2f, 0x01, 0x3b, 0xba, 0xac, 0xf5, 0x8a, 0xd0, 0xfa, 0xfd, 0xd9, 0x9d, 0xf4, 0x59, 0x8a, 0xef,
0xa7, 0x8a, 0x5f, 0x82, 0x12, 0x4e, 0x1a, 0x02, 0xdd, 0x4e, 0x77, 0x9b, 0x2b, 0x7b, 0x05, 0x6a,
0x7b, 0x9d, 0xcf, 0xcc, 0xc3, 0xbe, 0x78, 0x42, 0x42, 0x1a, 0x2c, 0x3f, 0x32, 0x70, 0xcf, 0xd8,
0x55, 0x94, 0x12, 0xba, 0x01, 0x9a, 0xa2, 0x4c, 0xd6, 0x2d, 0xea, 0x7f, 0x2b, 0xc2, 0xaa, 0x8c,
0xeb, 0xe9, 0xab, 0xe6, 0xb3, 0x9f, 0x17, 0xb3, 0x97, 0xfd, 0xe2, 0xd4, 0x65, 0x3f, 0xad, 0x15,
0x45, 0x5a, 0x2e, 0x4d, 0x6a, 0x45, 0xf1, 0x48, 0x30, 0x15, 0xb2, 0x17, 0x67, 0x09, 0xd9, 0x4d,
0x58, 0xf2, 0x48, 0x94, 0x2a, 0xbe, 0x86, 0x93, 0x21, 0x72, 0xa1, 0x6e, 0xf9, 0x7e, 0xc0, 0xc4,
0x93, 0x5a, 0x72, 0x7b, 0xd9, 0x9a, 0xe9, 0xf1, 0x2e, 0xfd, 0xe2, 0x76, 0x67, 0x82, 0x24, 0x23,
0x6b, 0x16, 0xbb, 0xf5, 0x21, 0x68, 0x17, 0x17, 0xcc, 0x92, 0xcf, 0xde, 0x7c, 0x77, 0x92, 0xce,
0x08, 0x77, 0xec, 0xc3, 0xde, 0xa3, 0xde, 0xfe, 0xe3, 0x9e, 0xb6, 0xc0, 0x07, 0xf8, 0xb0, 0xd7,
0xdb, 0xe9, 0x6d, 0x69, 0x05, 0x04, 0x50, 0x31, 0x3e, 0xdb, 0x19, 0x18, 0x9b, 0x5a, 0x71, 0xe3,
0xef, 0x2b, 0x50, 0x91, 0x42, 0xa2, 0x6f, 0x54, 0x2a, 0xcf, 0x76, 0xc0, 0xd1, 0x87, 0x33, 0x97,
0xc4, 0x53, 0x5d, 0xf5, 0xd6, 0x47, 0x73, 0xf3, 0xab, 0x57, 0xe9, 0x05, 0xf4, 0xcb, 0x02, 0x2c,
0x4f, 0x3d, 0xc3, 0xe6, 0x7d, 0x41, 0xbc, 0xa2, 0xe1, 0xde, 0xfa, 0xbf, 0xb9, 0x78, 0x53, 0x59,
0x7e, 0x51, 0x80, 0x7a, 0xa6, 0xd5, 0x8c, 0xee, 0xcf, 0xd3, 0x9e, 0x96, 0x92, 0x3c, 0x98, 0xbf,
0xb3, 0xad, 0x2f, 0xbc, 0x53, 0x40, 0x3f, 0x2f, 0x40, 0x3d, 0xd3, 0x74, 0xcd, 0x2d, 0xca, 0xe5,
0x16, 0x71, 0x6e, 0x51, 0xae, 0xea, 0xf1, 0x2e, 0xa0, 0x9f, 0x16, 0xa0, 0x96, 0x36, 0x50, 0xd1,
0xdd, 0xd9, 0x5b, 0xae, 0x52, 0x88, 0x7b, 0xf3, 0xf6, 0x6a, 0xf5, 0x05, 0xf4, 0x63, 0xa8, 0x26,
0xdd, 0x46, 0x94, 0x37, 0xfd, 0x5c, 0x68, 0x65, 0xb6, 0xee, 0xce, 0xcc, 0x97, 0xdd, 0x3e, 0x69,
0x01, 0xe6, 0xde, 0xfe, 0x42, 0xb3, 0xb2, 0x75, 0x77, 0x66, 0xbe, 0x74, 0x7b, 0xee, 0x09, 0x99,
0x4e, 0x61, 0x6e, 0x4f, 0xb8, 0xdc, 0xa2, 0xcc, 0xed, 0x09, 0x57, 0x35, 0x26, 0xa5, 0x20, 0x99,
0x5e, 0x63, 0x6e, 0x41, 0x2e, 0xf7, 0x33, 0x73, 0x0b, 0x72, 0x45, 0x6b, 0x53, 0x5f, 0x40, 0x5f,
0x17, 0xb2, 0x85, 0xfd, 0xdd, 0x99, 0x5b, 0x70, 0x33, 0xba, 0xe4, 0xa5, 0x26, 0xa0, 0x38, 0xa0,
0x5f, 0xab, 0xc7, 0x06, 0xd9, 0xc1, 0x43, 0xb3, 0x80, 0x4d, 0x35, 0xfd, 0x5a, 0x77, 0xe6, 0x4b,
0x36, 0x42, 0x88, 0x9f, 0x15, 0x00, 0x26, 0xbd, 0xbe, 0xdc, 0x42, 0x5c, 0x6a, 0x32, 0xb6, 0xee,
0xcf, 0xc1, 0x99, 0x3d, 0x20, 0x49, 0x7b, 0x2f, 0xf7, 0x01, 0xb9, 0xd0, 0x8b, 0xcc, 0x7d, 0x40,
0x2e, 0xf6, 0x11, 0xf5, 0x85, 0x4f, 0x96, 0xbe, 0x57, 0x96, 0xd9, 0xbf, 0x22, 0x7e, 0xde, 0xfb,
0x77, 0x00, 0x00, 0x00, 0xff, 0xff, 0x5d, 0xb1, 0x64, 0x16, 0x1e, 0x27, 0x00, 0x00,
}

View File

@@ -56,7 +56,7 @@ service Driver {
rpc InspectTask(InspectTaskRequest) returns (InspectTaskResponse) {}
// TaskStats collects and returns runtime metrics for the given task
rpc TaskStats(TaskStatsRequest) returns (TaskStatsResponse) {}
rpc TaskStats(TaskStatsRequest) returns (stream TaskStatsResponse) {}
// TaskEvents starts a streaming RPC where all task events emitted by the
// driver are streamed to the caller.
@@ -231,6 +231,9 @@ message TaskStatsRequest {
// TaskId is the ID of the target task
string task_id = 1;
// Interval is the interval in nanosecond at which to stream stats to the caller
int64 interval = 2;
}
message TaskStatsResponse {

View File

@@ -3,6 +3,7 @@ package drivers
import (
"fmt"
"io"
"time"
"github.com/golang/protobuf/ptypes"
plugin "github.com/hashicorp/go-plugin"
@@ -221,22 +222,36 @@ func (b *driverPluginServer) InspectTask(ctx context.Context, req *proto.Inspect
return resp, nil
}
func (b *driverPluginServer) TaskStats(ctx context.Context, req *proto.TaskStatsRequest) (*proto.TaskStatsResponse, error) {
stats, err := b.impl.TaskStats(req.TaskId)
func (b *driverPluginServer) TaskStats(req *proto.TaskStatsRequest, srv proto.Driver_TaskStatsServer) error {
ch, err := b.impl.TaskStats(srv.Context(), req.TaskId, time.Duration(req.Interval))
if err != nil {
return nil, err
if rec, ok := err.(structs.Recoverable); ok {
st := status.New(codes.FailedPrecondition, rec.Error())
st, err := st.WithDetails(&sproto.RecoverableError{Recoverable: rec.IsRecoverable()})
if err != nil {
// If this error, it will always error
panic(err)
}
return st.Err()
}
return err
}
pb, err := TaskStatsToProto(stats)
if err != nil {
return nil, fmt.Errorf("failed to encode task stats: %v", err)
for stats := range ch {
pb, err := TaskStatsToProto(stats)
if err != nil {
return fmt.Errorf("failed to encode task stats: %v", err)
}
if err = srv.Send(&proto.TaskStatsResponse{Stats: pb}); err == io.EOF {
break
} else if err != nil {
return err
}
}
resp := &proto.TaskStatsResponse{
Stats: pb,
}
return resp, nil
return nil
}
func (b *driverPluginServer) ExecTask(ctx context.Context, req *proto.ExecTaskRequest) (*proto.ExecTaskResponse, error) {

View File

@@ -198,7 +198,7 @@ type MockDriver struct {
StopTaskF func(string, time.Duration, string) error
DestroyTaskF func(string, bool) error
InspectTaskF func(string) (*drivers.TaskStatus, error)
TaskStatsF func(string) (*drivers.TaskResourceUsage, error)
TaskStatsF func(context.Context, string, time.Duration) (<-chan *drivers.TaskResourceUsage, error)
TaskEventsF func(context.Context) (<-chan *drivers.TaskEvent, error)
SignalTaskF func(string, string) error
ExecTaskF func(string, []string, time.Duration) (*drivers.ExecTaskResult, error)
@@ -225,8 +225,8 @@ func (d *MockDriver) DestroyTask(taskID string, force bool) error {
func (d *MockDriver) InspectTask(taskID string) (*drivers.TaskStatus, error) {
return d.InspectTaskF(taskID)
}
func (d *MockDriver) TaskStats(taskID string) (*drivers.TaskResourceUsage, error) {
return d.TaskStats(taskID)
func (d *MockDriver) TaskStats(ctx context.Context, taskID string, i time.Duration) (<-chan *drivers.TaskResourceUsage, error) {
return d.TaskStats(ctx, taskID, i)
}
func (d *MockDriver) TaskEvents(ctx context.Context) (<-chan *drivers.TaskEvent, error) {
return d.TaskEventsF(ctx)