mirror of
https://github.com/kemko/nomad.git
synced 2026-01-01 16:05:42 +03:00
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:
@@ -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 {
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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
|
||||
|
||||
|
||||
@@ -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()
|
||||
}
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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
176
drivers/docker/stats.go
Normal 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
|
||||
}
|
||||
@@ -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) {
|
||||
|
||||
@@ -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))
|
||||
}
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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))
|
||||
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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())
|
||||
|
||||
@@ -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,
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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,
|
||||
}
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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)
|
||||
|
||||
Reference in New Issue
Block a user